View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.ipc;
22  
23  import java.io.ByteArrayInputStream;
24  import java.io.ByteArrayOutputStream;
25  import java.io.DataInputStream;
26  import java.io.DataOutputStream;
27  import java.io.IOException;
28  import java.io.InterruptedIOException;
29  import java.net.BindException;
30  import java.net.InetAddress;
31  import java.net.InetSocketAddress;
32  import java.net.ServerSocket;
33  import java.net.Socket;
34  import java.net.SocketException;
35  import java.net.UnknownHostException;
36  import java.nio.ByteBuffer;
37  import java.nio.channels.CancelledKeyException;
38  import java.nio.channels.ClosedChannelException;
39  import java.nio.channels.ReadableByteChannel;
40  import java.nio.channels.SelectionKey;
41  import java.nio.channels.Selector;
42  import java.nio.channels.ServerSocketChannel;
43  import java.nio.channels.SocketChannel;
44  import java.nio.channels.WritableByteChannel;
45  import java.util.ArrayList;
46  import java.util.Collections;
47  import java.util.Iterator;
48  import java.util.LinkedList;
49  import java.util.List;
50  import java.util.Map;
51  import java.util.Random;
52  import java.util.concurrent.BlockingQueue;
53  import java.util.concurrent.ConcurrentHashMap;
54  import java.util.concurrent.ExecutorService;
55  import java.util.concurrent.Executors;
56  import java.util.concurrent.LinkedBlockingQueue;
57  import java.util.concurrent.atomic.AtomicInteger;
58  
59  import org.apache.commons.logging.Log;
60  import org.apache.commons.logging.LogFactory;
61  import org.apache.hadoop.conf.Configuration;
62  import org.apache.hadoop.hbase.HConstants;
63  import org.apache.hadoop.hbase.io.HbaseObjectWritable;
64  import org.apache.hadoop.hbase.io.WritableWithSize;
65  import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
66  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
67  import org.apache.hadoop.hbase.security.User;
68  import org.apache.hadoop.hbase.util.ByteBufferOutputStream;
69  import org.apache.hadoop.hbase.util.Bytes;
70  import org.apache.hadoop.hbase.util.SizeBasedThrottler;
71  import org.apache.hadoop.io.Writable;
72  import org.apache.hadoop.io.WritableUtils;
73  import org.apache.hadoop.ipc.RPC.VersionMismatch;
74  import org.apache.hadoop.util.ReflectionUtils;
75  import org.apache.hadoop.util.StringUtils;
76  import org.cliffc.high_scale_lib.Counter;
77  
78  import com.google.common.base.Function;
79  import com.google.common.util.concurrent.ThreadFactoryBuilder;
80  
81  /** An abstract IPC service.  IPC calls take a single {@link Writable} as a
82   * parameter, and return a {@link Writable} as their value.  A service runs on
83   * a port and is defined by a parameter class and a value class.
84   *
85   *
86   * <p>Copied local so can fix HBASE-900.
87   *
88   * @see HBaseClient
89   */
90  public abstract class HBaseServer implements RpcServer {
91  
92    /**
93     * The first four bytes of Hadoop RPC connections
94     */
95    public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
96    public static final byte CURRENT_VERSION = 3;
97  
98    /**
99     * How many calls/handler are allowed in the queue.
100    */
101   private static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
102 
103   /**
104    * The maximum size that we can hold in the IPC queue
105    */
106   private static final int DEFAULT_MAX_CALLQUEUE_SIZE =
107     1024 * 1024 * 1024;
108 
109   static final int BUFFER_INITIAL_SIZE = 1024;
110 
111   private static final String WARN_DELAYED_CALLS =
112       "hbase.ipc.warn.delayedrpc.number";
113 
114   private static final int DEFAULT_WARN_DELAYED_CALLS = 1000;
115 
116   private final int warnDelayedCalls;
117 
118   private AtomicInteger delayedCalls;
119 
120   public static final Log LOG =
121     LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer");
122   protected static final Log TRACELOG =
123       LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer.trace");
124 
125   protected static final ThreadLocal<RpcServer> SERVER =
126     new ThreadLocal<RpcServer>();
127   private volatile boolean started = false;
128 
129   private static final Map<String, Class<? extends VersionedProtocol>>
130       PROTOCOL_CACHE =
131       new ConcurrentHashMap<String, Class<? extends VersionedProtocol>>();
132 
133   static Class<? extends VersionedProtocol> getProtocolClass(
134       String protocolName, Configuration conf)
135   throws ClassNotFoundException {
136     Class<? extends VersionedProtocol> protocol =
137         PROTOCOL_CACHE.get(protocolName);
138 
139     if (protocol == null) {
140       protocol = (Class<? extends VersionedProtocol>)
141           conf.getClassByName(protocolName);
142       PROTOCOL_CACHE.put(protocolName, protocol);
143     }
144     return protocol;
145   }
146 
147   /** Returns the server instance called under or null.  May be called under
148    * {@link #call(Class, Writable, long, MonitoredRPCHandler)} implementations,
149    * and under {@link Writable} methods of paramters and return values.
150    * Permits applications to access the server context.
151    * @return HBaseServer
152    */
153   public static RpcServer get() {
154     return SERVER.get();
155   }
156 
157   /** This is set to Call object before Handler invokes an RPC and reset
158    * after the call returns.
159    */
160   protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
161 
162   /** Returns the remote side ip address when invoked inside an RPC
163    *  Returns null incase of an error.
164    *  @return InetAddress
165    */
166   public static InetAddress getRemoteIp() {
167     Call call = CurCall.get();
168     if (call != null) {
169       return call.connection.socket.getInetAddress();
170     }
171     return null;
172   }
173   /** Returns remote address as a string when invoked inside an RPC.
174    *  Returns null in case of an error.
175    *  @return String
176    */
177   public static String getRemoteAddress() {
178     Call call = CurCall.get();
179     if (call != null) {
180       return call.connection.getHostAddress();
181     }
182     return null;
183   }
184 
185   protected String bindAddress;
186   protected int port;                             // port we listen on
187   private int handlerCount;                       // number of handler threads
188   private int priorityHandlerCount;
189   private int readThreads;                        // number of read threads
190   protected Class<? extends Writable> paramClass; // class of call parameters
191   protected int maxIdleTime;                      // the maximum idle time after
192                                                   // which a client may be
193                                                   // disconnected
194   protected int thresholdIdleConnections;         // the number of idle
195                                                   // connections after which we
196                                                   // will start cleaning up idle
197                                                   // connections
198   int maxConnectionsToNuke;                       // the max number of
199                                                   // connections to nuke
200                                                   // during a cleanup
201 
202   protected HBaseRpcMetrics  rpcMetrics;
203 
204   protected Configuration conf;
205 
206   private int maxQueueLength;
207   private int maxQueueSize;
208   protected int socketSendBufferSize;
209   protected final boolean tcpNoDelay;   // if T then disable Nagle's Algorithm
210   protected final boolean tcpKeepAlive; // if T then use keepalives
211   protected final long purgeTimeout;    // in milliseconds
212 
213   // responseQueuesSizeThrottler is shared among all responseQueues,
214   // it bounds memory occupied by responses in all responseQueues
215   final SizeBasedThrottler responseQueuesSizeThrottler;
216 
217   // RESPONSE_QUEUE_MAX_SIZE limits total size of responses in every response queue
218   private static final long DEFAULT_RESPONSE_QUEUES_MAX_SIZE = 1024 * 1024 * 1024; // 1G
219   private static final String RESPONSE_QUEUES_MAX_SIZE = "ipc.server.response.queue.maxsize";
220 
221   volatile protected boolean running = true;         // true while server runs
222   protected BlockingQueue<Call> callQueue; // queued calls
223   protected final Counter callQueueSize = new Counter();
224   protected BlockingQueue<Call> priorityCallQueue;
225 
226   protected int highPriorityLevel;  // what level a high priority call is at
227 
228   protected final List<Connection> connectionList =
229     Collections.synchronizedList(new LinkedList<Connection>());
230   //maintain a list
231   //of client connections
232   private Listener listener = null;
233   protected Responder responder = null;
234   protected int numConnections = 0;
235   private Handler[] handlers = null;
236   private Handler[] priorityHandlers = null;
237   /** replication related queue; */
238   protected BlockingQueue<Call> replicationQueue;
239   private int numOfReplicationHandlers = 0;
240   private Handler[] replicationHandlers = null;
241   protected HBaseRPCErrorHandler errorHandler = null;
242 
243   /**
244    * A convenience method to bind to a given address and report
245    * better exceptions if the address is not a valid host.
246    * @param socket the socket to bind
247    * @param address the address to bind to
248    * @param backlog the number of connections allowed in the queue
249    * @throws BindException if the address can't be bound
250    * @throws UnknownHostException if the address isn't a valid host name
251    * @throws IOException other random errors from bind
252    */
253   public static void bind(ServerSocket socket, InetSocketAddress address,
254                           int backlog) throws IOException {
255     try {
256       socket.bind(address, backlog);
257     } catch (BindException e) {
258       BindException bindException =
259         new BindException("Problem binding to " + address + " : " +
260             e.getMessage());
261       bindException.initCause(e);
262       throw bindException;
263     } catch (SocketException e) {
264       // If they try to bind to a different host's address, give a better
265       // error message.
266       if ("Unresolved address".equals(e.getMessage())) {
267         throw new UnknownHostException("Invalid hostname for server: " +
268                                        address.getHostName());
269       }
270       throw e;
271     }
272   }
273 
274   /** A call queued for handling. */
275   protected class Call implements RpcCallContext {
276     protected int id;                             // the client's call id
277     protected Writable param;                     // the parameter passed
278     protected Connection connection;              // connection to client
279     protected long timestamp;      // the time received when response is null
280                                    // the time served when response is not null
281     protected ByteBuffer response;                // the response for this call
282     protected boolean delayResponse;
283     protected Responder responder;
284     protected boolean delayReturnValue;           // if the return value should be
285                                                   // set at call completion
286     protected long size;                          // size of current call
287     protected boolean isError;
288 
289     public Call(int id, Writable param, Connection connection,
290         Responder responder, long size) {
291       this.id = id;
292       this.param = param;
293       this.connection = connection;
294       this.timestamp = System.currentTimeMillis();
295       this.response = null;
296       this.delayResponse = false;
297       this.responder = responder;
298       this.isError = false;
299       this.size = size;
300     }
301 
302     @Override
303     public String toString() {
304       return param.toString() + " from " + connection.toString();
305     }
306 
307     protected synchronized void setResponse(Object value, Status status,
308         String errorClass, String error) {
309       // Avoid overwriting an error value in the response.  This can happen if
310       // endDelayThrowing is called by another thread before the actual call
311       // returning.
312       if (this.isError)
313         return;
314       if (errorClass != null) {
315         this.isError = true;
316       }
317       Writable result = null;
318       if (value instanceof Writable) {
319         result = (Writable) value;
320       } else {
321         /* We might have a null value and errors. Avoid creating a
322          * HbaseObjectWritable, because the constructor fails on null. */
323         if (value != null) {
324           result = new HbaseObjectWritable(value);
325         }
326       }
327 
328       int size = BUFFER_INITIAL_SIZE;
329       if (result instanceof WritableWithSize) {
330         // get the size hint.
331         WritableWithSize ohint = (WritableWithSize) result;
332         long hint = ohint.getWritableSize() + Bytes.SIZEOF_BYTE +
333           (2 * Bytes.SIZEOF_INT);
334         if (hint > Integer.MAX_VALUE) {
335           // oops, new problem.
336           IOException ioe =
337             new IOException("Result buffer size too large: " + hint);
338           errorClass = ioe.getClass().getName();
339           error = StringUtils.stringifyException(ioe);
340         } else {
341           size = (int)hint;
342         }
343       }
344 
345       ByteBufferOutputStream buf = new ByteBufferOutputStream(size);
346       DataOutputStream out = new DataOutputStream(buf);
347       try {
348         // Call id.
349         out.writeInt(this.id);
350         // Write flag.
351         byte flag = (error != null)?
352           ResponseFlag.getErrorAndLengthSet(): ResponseFlag.getLengthSetOnly();
353         out.writeByte(flag);
354         // Place holder for length set later below after we
355         // fill the buffer with data.
356         out.writeInt(0xdeadbeef);
357         out.writeInt(status.state);
358       } catch (IOException e) {
359         errorClass = e.getClass().getName();
360         error = StringUtils.stringifyException(e);
361       }
362 
363       try {
364         if (error == null) {
365           result.write(out);
366         } else {
367           WritableUtils.writeString(out, errorClass);
368           WritableUtils.writeString(out, error);
369         }
370       } catch (IOException e) {
371         LOG.warn("Error sending response to call: ", e);
372       }
373 
374       // Set the length into the ByteBuffer after call id and after
375       // byte flag.
376       ByteBuffer bb = buf.getByteBuffer();
377       int bufSiz = bb.remaining();
378       // Move to the size location in our ByteBuffer past call.id
379       // and past the byte flag.
380       bb.position(Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE); 
381       bb.putInt(bufSiz);
382       bb.position(0);
383       this.response = bb;
384     }
385 
386     @Override
387     public synchronized void endDelay(Object result) throws IOException {
388       assert this.delayResponse;
389       assert this.delayReturnValue || result == null;
390       this.delayResponse = false;
391       delayedCalls.decrementAndGet();
392       if (this.delayReturnValue)
393         this.setResponse(result, Status.SUCCESS, null, null);
394       this.responder.doRespond(this);
395     }
396 
397     @Override
398     public synchronized void endDelay() throws IOException {
399       this.endDelay(null);
400     }
401 
402     @Override
403     public synchronized void startDelay(boolean delayReturnValue) {
404       assert !this.delayResponse;
405       this.delayResponse = true;
406       this.delayReturnValue = delayReturnValue;
407       int numDelayed = delayedCalls.incrementAndGet();
408       if (numDelayed > warnDelayedCalls) {
409         LOG.warn("Too many delayed calls: limit " + warnDelayedCalls +
410             " current " + numDelayed);
411       }
412     }
413 
414     @Override
415     public synchronized void endDelayThrowing(Throwable t) throws IOException {
416       this.setResponse(null, Status.ERROR, t.getClass().toString(),
417           StringUtils.stringifyException(t));
418       this.delayResponse = false;
419       this.sendResponseIfReady();
420     }
421 
422     @Override
423     public synchronized boolean isDelayed() {
424       return this.delayResponse;
425     }
426 
427     @Override
428     public synchronized boolean isReturnValueDelayed() {
429       return this.delayReturnValue;
430     }
431     
432     @Override
433     public void throwExceptionIfCallerDisconnected() throws CallerDisconnectedException {
434       if (!connection.channel.isOpen()) {
435         long afterTime = System.currentTimeMillis() - timestamp;
436         throw new CallerDisconnectedException(
437             "Aborting call " + this + " after " + afterTime + " ms, since " +
438             "caller disconnected");
439       }
440     }
441 
442     public long getSize() {
443       return this.size;
444     }
445 
446     /**
447      * If we have a response, and delay is not set, then respond
448      * immediately.  Otherwise, do not respond to client.  This is
449      * called the by the RPC code in the context of the Handler thread.
450      */
451     public synchronized void sendResponseIfReady() throws IOException {
452       if (!this.delayResponse) {
453         this.responder.doRespond(this);
454       }
455     }
456   }
457 
458   /** Listens on the socket. Creates jobs for the handler threads*/
459   private class Listener extends Thread {
460 
461     private ServerSocketChannel acceptChannel = null; //the accept channel
462     private Selector selector = null; //the selector that we use for the server
463     private Reader[] readers = null;
464     private int currentReader = 0;
465     private InetSocketAddress address; //the address we bind at
466     private Random rand = new Random();
467     private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
468                                          //-tion (for idle connections) ran
469     private long cleanupInterval = 10000; //the minimum interval between
470                                           //two cleanup runs
471     private int backlogLength = conf.getInt("ipc.server.listen.queue.size", 128);
472 
473     private ExecutorService readPool;
474 
475     public Listener() throws IOException {
476       address = new InetSocketAddress(bindAddress, port);
477       // Create a new server socket and set to non blocking mode
478       acceptChannel = ServerSocketChannel.open();
479       acceptChannel.configureBlocking(false);
480 
481       // Bind the server socket to the local host and port
482       bind(acceptChannel.socket(), address, backlogLength);
483       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
484       // create a selector;
485       selector= Selector.open();
486 
487       readers = new Reader[readThreads];
488       readPool = Executors.newFixedThreadPool(readThreads,
489         new ThreadFactoryBuilder().setNameFormat(
490           "IPC Reader %d on port " + port).setDaemon(true).build());
491       for (int i = 0; i < readThreads; ++i) {
492         Reader reader = new Reader();
493         readers[i] = reader;
494         readPool.execute(reader);
495       }
496 
497       // Register accepts on the server socket with the selector.
498       acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
499       this.setName("IPC Server listener on " + port);
500       this.setDaemon(true);
501     }
502 
503 
504     private class Reader implements Runnable {
505       private volatile boolean adding = false;
506       private final Selector readSelector;
507 
508       Reader() throws IOException {
509         this.readSelector = Selector.open();
510       }
511       public void run() {
512         LOG.info("Starting " + getName());
513         try {
514           doRunLoop();
515         } finally {
516           try {
517             readSelector.close();
518           } catch (IOException ioe) {
519             LOG.error("Error closing read selector in " + getName(), ioe);
520           }
521         }
522       }
523 
524       private synchronized void doRunLoop() {
525         while (running) {
526           SelectionKey key = null;
527           try {
528             readSelector.select();
529             while (adding) {
530               this.wait(1000);
531             }
532 
533             Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
534             while (iter.hasNext()) {
535               key = iter.next();
536               iter.remove();
537               if (key.isValid()) {
538                 if (key.isReadable()) {
539                   doRead(key);
540                 }
541               }
542               key = null;
543             }
544           } catch (InterruptedException e) {
545             if (running) {                      // unexpected -- log it
546               LOG.info(getName() + " unexpectedly interrupted: " +
547                   StringUtils.stringifyException(e));
548             }
549           } catch (IOException ex) {
550             LOG.error("Error in Reader", ex);
551           }
552         }
553       }
554 
555       /**
556        * This gets reader into the state that waits for the new channel
557        * to be registered with readSelector. If it was waiting in select()
558        * the thread will be woken up, otherwise whenever select() is called
559        * it will return even if there is nothing to read and wait
560        * in while(adding) for finishAdd call
561        */
562       public void startAdd() {
563         adding = true;
564         readSelector.wakeup();
565       }
566 
567       public synchronized SelectionKey registerChannel(SocketChannel channel)
568         throws IOException {
569         return channel.register(readSelector, SelectionKey.OP_READ);
570       }
571 
572       public synchronized void finishAdd() {
573         adding = false;
574         this.notify();
575       }
576     }
577 
578     /** cleanup connections from connectionList. Choose a random range
579      * to scan and also have a limit on the number of the connections
580      * that will be cleanedup per run. The criteria for cleanup is the time
581      * for which the connection was idle. If 'force' is true then all
582      * connections will be looked at for the cleanup.
583      * @param force all connections will be looked at for cleanup
584      */
585     private void cleanupConnections(boolean force) {
586       if (force || numConnections > thresholdIdleConnections) {
587         long currentTime = System.currentTimeMillis();
588         if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
589           return;
590         }
591         int start = 0;
592         int end = numConnections - 1;
593         if (!force) {
594           start = rand.nextInt() % numConnections;
595           end = rand.nextInt() % numConnections;
596           int temp;
597           if (end < start) {
598             temp = start;
599             start = end;
600             end = temp;
601           }
602         }
603         int i = start;
604         int numNuked = 0;
605         while (i <= end) {
606           Connection c;
607           synchronized (connectionList) {
608             try {
609               c = connectionList.get(i);
610             } catch (Exception e) {return;}
611           }
612           if (c.timedOut(currentTime)) {
613             if (LOG.isDebugEnabled())
614               LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
615             closeConnection(c);
616             numNuked++;
617             end--;
618             //noinspection UnusedAssignment
619             c = null;
620             if (!force && numNuked == maxConnectionsToNuke) break;
621           }
622           else i++;
623         }
624         lastCleanupRunTime = System.currentTimeMillis();
625       }
626     }
627 
628     @Override
629     public void run() {
630       LOG.info(getName() + ": starting");
631       SERVER.set(HBaseServer.this);
632 
633       while (running) {
634         SelectionKey key = null;
635         try {
636           selector.select(); // FindBugs IS2_INCONSISTENT_SYNC
637           Iterator<SelectionKey> iter = selector.selectedKeys().iterator();
638           while (iter.hasNext()) {
639             key = iter.next();
640             iter.remove();
641             try {
642               if (key.isValid()) {
643                 if (key.isAcceptable())
644                   doAccept(key);
645               }
646             } catch (IOException ignored) {
647             }
648             key = null;
649           }
650         } catch (OutOfMemoryError e) {
651           if (errorHandler != null) {
652             if (errorHandler.checkOOME(e)) {
653               LOG.info(getName() + ": exiting on OOME");
654               closeCurrentConnection(key, e);
655               cleanupConnections(true);
656               return;
657             }
658           } else {
659             // we can run out of memory if we have too many threads
660             // log the event and sleep for a minute and give
661             // some thread(s) a chance to finish
662             LOG.warn("Out of Memory in server select", e);
663             closeCurrentConnection(key, e);
664             cleanupConnections(true);
665             try { Thread.sleep(60000); } catch (Exception ignored) {}
666       }
667         } catch (Exception e) {
668           closeCurrentConnection(key, e);
669         }
670         cleanupConnections(false);
671       }
672       LOG.info("Stopping " + this.getName());
673 
674       synchronized (this) {
675         try {
676           acceptChannel.close();
677           selector.close();
678         } catch (IOException ignored) { }
679 
680         selector= null;
681         acceptChannel= null;
682 
683         // clean up all connections
684         while (!connectionList.isEmpty()) {
685           closeConnection(connectionList.remove(0));
686         }
687       }
688     }
689 
690     private void closeCurrentConnection(SelectionKey key, Throwable e) {
691       if (key != null) {
692         Connection c = (Connection)key.attachment();
693         if (c != null) {
694           if (LOG.isDebugEnabled()) {
695             LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
696                 (e != null ? " on error " + e.getMessage() : ""));
697           }
698           closeConnection(c);
699           key.attach(null);
700         }
701       }
702     }
703 
704     InetSocketAddress getAddress() {
705       return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
706     }
707 
708     void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
709       Connection c;
710       ServerSocketChannel server = (ServerSocketChannel) key.channel();
711 
712       SocketChannel channel;
713       while ((channel = server.accept()) != null) {
714         channel.configureBlocking(false);
715         channel.socket().setTcpNoDelay(tcpNoDelay);
716         channel.socket().setKeepAlive(tcpKeepAlive);
717 
718         Reader reader = getReader();
719         try {
720           reader.startAdd();
721           SelectionKey readKey = reader.registerChannel(channel);
722           c = getConnection(channel, System.currentTimeMillis());
723           readKey.attach(c);
724           synchronized (connectionList) {
725             connectionList.add(numConnections, c);
726             numConnections++;
727           }
728           if (LOG.isDebugEnabled())
729             LOG.debug("Server connection from " + c.toString() +
730                 "; # active connections: " + numConnections +
731                 "; # queued calls: " + callQueue.size());
732         } finally {
733           reader.finishAdd();
734         }
735       }
736       rpcMetrics.numOpenConnections.set(numConnections);
737     }
738 
739     void doRead(SelectionKey key) throws InterruptedException {
740       int count = 0;
741       Connection c = (Connection)key.attachment();
742       if (c == null) {
743         return;
744       }
745       c.setLastContact(System.currentTimeMillis());
746 
747       try {
748         count = c.readAndProcess();
749       } catch (InterruptedException ieo) {
750         throw ieo;
751       } catch (Exception e) {
752         LOG.warn(getName() + ": readAndProcess threw exception " + e + ". Count of bytes read: " + count, e);
753         count = -1; //so that the (count < 0) block is executed
754       }
755       if (count < 0) {
756         if (LOG.isDebugEnabled())
757           LOG.debug(getName() + ": disconnecting client " +
758                     c.getHostAddress() + ". Number of active connections: "+
759                     numConnections);
760         closeConnection(c);
761         // c = null;
762       }
763       else {
764         c.setLastContact(System.currentTimeMillis());
765       }
766     }
767 
768     synchronized void doStop() {
769       if (selector != null) {
770         selector.wakeup();
771         Thread.yield();
772       }
773       if (acceptChannel != null) {
774         try {
775           acceptChannel.socket().close();
776         } catch (IOException e) {
777           LOG.info(getName() + ":Exception in closing listener socket. " + e);
778         }
779       }
780       readPool.shutdownNow();
781     }
782 
783     // The method that will return the next reader to work with
784     // Simplistic implementation of round robin for now
785     Reader getReader() {
786       currentReader = (currentReader + 1) % readers.length;
787       return readers[currentReader];
788     }
789   }
790 
791   // Sends responses of RPC back to clients.
792   protected class Responder extends Thread {
793     private final Selector writeSelector;
794     private int pending;         // connections waiting to register
795 
796     Responder() throws IOException {
797       this.setName("IPC Server Responder");
798       this.setDaemon(true);
799       writeSelector = Selector.open(); // create a selector
800       pending = 0;
801     }
802 
803     @Override
804     public void run() {
805       LOG.info(getName() + ": starting");
806       SERVER.set(HBaseServer.this);
807       try {
808         doRunLoop();
809       } finally {
810         LOG.info("Stopping " + this.getName());
811         try {
812           writeSelector.close();
813         } catch (IOException ioe) {
814           LOG.error("Couldn't close write selector in " + this.getName(), ioe);
815         }
816       }
817     }
818 
819     private void doRunLoop() {
820       long lastPurgeTime = 0;   // last check for old calls.
821 
822       while (running) {
823         try {
824           waitPending();     // If a channel is being registered, wait.
825           writeSelector.select(purgeTimeout);
826           Iterator<SelectionKey> iter = writeSelector.selectedKeys().iterator();
827           while (iter.hasNext()) {
828             SelectionKey key = iter.next();
829             iter.remove();
830             try {
831               if (key.isValid() && key.isWritable()) {
832                   doAsyncWrite(key);
833               }
834             } catch (IOException e) {
835               LOG.info(getName() + ": doAsyncWrite threw exception " + e);
836             }
837           }
838           long now = System.currentTimeMillis();
839           if (now < lastPurgeTime + purgeTimeout) {
840             continue;
841           }
842           lastPurgeTime = now;
843           //
844           // If there were some calls that have not been sent out for a
845           // long time, discard them.
846           //
847           LOG.debug("Checking for old call responses.");
848           ArrayList<Call> calls;
849 
850           // get the list of channels from list of keys.
851           synchronized (writeSelector.keys()) {
852             calls = new ArrayList<Call>(writeSelector.keys().size());
853             iter = writeSelector.keys().iterator();
854             while (iter.hasNext()) {
855               SelectionKey key = iter.next();
856               Call call = (Call)key.attachment();
857               if (call != null && key.channel() == call.connection.channel) {
858                 calls.add(call);
859               }
860             }
861           }
862 
863           for(Call call : calls) {
864             try {
865               doPurge(call, now);
866             } catch (IOException e) {
867               LOG.warn("Error in purging old calls " + e);
868             }
869           }
870         } catch (OutOfMemoryError e) {
871           if (errorHandler != null) {
872             if (errorHandler.checkOOME(e)) {
873               LOG.info(getName() + ": exiting on OOME");
874               return;
875             }
876           } else {
877             //
878             // we can run out of memory if we have too many threads
879             // log the event and sleep for a minute and give
880             // some thread(s) a chance to finish
881             //
882             LOG.warn("Out of Memory in server select", e);
883             try { Thread.sleep(60000); } catch (Exception ignored) {}
884           }
885         } catch (Exception e) {
886           LOG.warn("Exception in Responder " +
887                    StringUtils.stringifyException(e));
888         }
889       }
890       LOG.info("Stopping " + this.getName());
891     }
892 
893     private void doAsyncWrite(SelectionKey key) throws IOException {
894       Call call = (Call)key.attachment();
895       if (call == null) {
896         return;
897       }
898       if (key.channel() != call.connection.channel) {
899         throw new IOException("doAsyncWrite: bad channel");
900       }
901 
902       synchronized(call.connection.responseQueue) {
903         if (processResponse(call.connection.responseQueue, false)) {
904           try {
905             key.interestOps(0);
906           } catch (CancelledKeyException e) {
907             /* The Listener/reader might have closed the socket.
908              * We don't explicitly cancel the key, so not sure if this will
909              * ever fire.
910              * This warning could be removed.
911              */
912             LOG.warn("Exception while changing ops : " + e);
913           }
914         }
915       }
916     }
917 
918     //
919     // Remove calls that have been pending in the responseQueue
920     // for a long time.
921     //
922     private void doPurge(Call call, long now) throws IOException {
923       synchronized (call.connection.responseQueue) {
924         Iterator<Call> iter = call.connection.responseQueue.listIterator(0);
925         while (iter.hasNext()) {
926           Call nextCall = iter.next();
927           if (now > nextCall.timestamp + purgeTimeout) {
928             closeConnection(nextCall.connection);
929             break;
930           }
931         }
932       }
933     }
934 
935     // Processes one response. Returns true if there are no more pending
936     // data for this channel.
937     //
938     private boolean processResponse(final LinkedList<Call> responseQueue,
939                                     boolean inHandler) throws IOException {
940       boolean error = true;
941       boolean done = false;       // there is more data for this channel.
942       int numElements;
943       Call call = null;
944       try {
945         //noinspection SynchronizationOnLocalVariableOrMethodParameter
946         synchronized (responseQueue) {
947           //
948           // If there are no items for this channel, then we are done
949           //
950           numElements = responseQueue.size();
951           if (numElements == 0) {
952             error = false;
953             return true;              // no more data for this channel.
954           }
955           //
956           // Extract the first call
957           //
958           call = responseQueue.peek();
959           SocketChannel channel = call.connection.channel;
960           if (LOG.isDebugEnabled()) {
961             LOG.debug(getName() + ": responding to #" + call.id + " from " +
962                       call.connection);
963           }
964           //
965           // Send as much data as we can in the non-blocking fashion
966           //
967           int numBytes = channelWrite(channel, call.response);
968           if (numBytes < 0) {
969             // Error flag is set, so returning here closes connection and
970             // clears responseQueue.                   
971             return true;
972           }
973           if (!call.response.hasRemaining()) {
974             responseQueue.poll();
975             responseQueuesSizeThrottler.decrease(call.response.limit());    
976             call.connection.decRpcCount();
977             //noinspection RedundantIfStatement
978             if (numElements == 1) {    // last call fully processes.
979               done = true;             // no more data for this channel.
980             } else {
981               done = false;            // more calls pending to be sent.
982             }
983             if (LOG.isDebugEnabled()) {
984               LOG.debug(getName() + ": responding to #" + call.id + " from " +
985                         call.connection + " Wrote " + numBytes + " bytes.");
986             }
987           } else {
988             if (inHandler) {
989               // set the serve time when the response has to be sent later
990               call.timestamp = System.currentTimeMillis();
991               if (enqueueInSelector(call))
992                 done = true;
993             }
994             if (LOG.isDebugEnabled()) {
995               LOG.debug(getName() + ": responding to #" + call.id + " from " +
996                         call.connection + " Wrote partial " + numBytes +
997                         " bytes.");
998             }
999           }
1000           error = false;              // everything went off well
1001         }
1002       } finally {
1003         if (error && call != null) {
1004           LOG.warn(getName()+", call " + call + ": output error");
1005           done = true;               // error. no more data for this channel.
1006           closeConnection(call.connection);
1007         }
1008       }
1009       return done;
1010     }
1011 
1012     //
1013     // Enqueue for background thread to send responses out later.
1014     //
1015     private boolean enqueueInSelector(Call call) throws IOException {
1016       boolean done = false;
1017       incPending();
1018       try {
1019         // Wake up the thread blocked on select, only then can the call
1020         // to channel.register() complete.
1021         SocketChannel channel = call.connection.channel;
1022         writeSelector.wakeup();
1023         channel.register(writeSelector, SelectionKey.OP_WRITE, call);
1024       } catch (ClosedChannelException e) {
1025         //It's OK.  Channel might be closed else where.
1026         done = true;
1027       } finally {
1028         decPending();
1029       }
1030       return done;
1031     }
1032 
1033     //
1034     // Enqueue a response from the application.
1035     //
1036     void doRespond(Call call) throws IOException {
1037       // set the serve time when the response has to be sent later
1038       call.timestamp = System.currentTimeMillis();
1039 
1040       boolean doRegister = false;
1041       boolean closed;
1042       try {
1043         responseQueuesSizeThrottler.increase(call.response.remaining());
1044       } catch (InterruptedException ie) {
1045         throw new InterruptedIOException(ie.getMessage());
1046       }
1047       synchronized (call.connection.responseQueue) {
1048         closed = call.connection.closed;
1049         if (!closed) {
1050           call.connection.responseQueue.addLast(call);
1051 
1052           if (call.connection.responseQueue.size() == 1) {
1053             doRegister = !processResponse(call.connection.responseQueue, false);
1054           }
1055         }
1056       }
1057       if (doRegister) {
1058         enqueueInSelector(call);
1059       }
1060       if (closed) {
1061         // Connection was closed when we tried to submit response, but we
1062         // increased responseQueues size already. It shoud be
1063         // decreased here.
1064         responseQueuesSizeThrottler.decrease(call.response.remaining());
1065       }      
1066     }
1067 
1068     private synchronized void incPending() {   // call waiting to be enqueued.
1069       pending++;
1070     }
1071 
1072     private synchronized void decPending() { // call done enqueueing.
1073       pending--;
1074       notify();
1075     }
1076 
1077     private synchronized void waitPending() throws InterruptedException {
1078       while (pending > 0) {
1079         wait();
1080       }
1081     }
1082   }
1083 
1084   /** Reads calls from a connection and queues them for handling. */
1085   protected class Connection {
1086     private boolean versionRead = false; //if initial signature and
1087                                          //version are read
1088     private boolean headerRead = false;  //if the connection header that
1089                                          //follows version is read.
1090 
1091     protected volatile boolean closed = false;    // indicates if connection was closed
1092     protected SocketChannel channel;
1093     private ByteBuffer data;
1094     private ByteBuffer dataLengthBuffer;
1095     protected final LinkedList<Call> responseQueue;
1096     private volatile int rpcCount = 0; // number of outstanding rpcs
1097     private long lastContact;
1098     private int dataLength;
1099     protected Socket socket;
1100     // Cache the remote host & port info so that even if the socket is
1101     // disconnected, we can say where it used to connect to.
1102     protected String hostAddress;
1103     protected int remotePort;
1104     ConnectionHeader header = new ConnectionHeader();
1105     Class<? extends VersionedProtocol> protocol;
1106     protected User ticket = null;
1107 
1108     public Connection(SocketChannel channel, long lastContact) {
1109       this.channel = channel;
1110       this.lastContact = lastContact;
1111       this.data = null;
1112       this.dataLengthBuffer = ByteBuffer.allocate(4);
1113       this.socket = channel.socket();
1114       InetAddress addr = socket.getInetAddress();
1115       if (addr == null) {
1116         this.hostAddress = "*Unknown*";
1117       } else {
1118         this.hostAddress = addr.getHostAddress();
1119       }
1120       this.remotePort = socket.getPort();
1121       this.responseQueue = new LinkedList<Call>();
1122       if (socketSendBufferSize != 0) {
1123         try {
1124           socket.setSendBufferSize(socketSendBufferSize);
1125         } catch (IOException e) {
1126           LOG.warn("Connection: unable to set socket send buffer size to " +
1127                    socketSendBufferSize);
1128         }
1129       }
1130     }
1131 
1132     @Override
1133     public String toString() {
1134       return getHostAddress() + ":" + remotePort;
1135     }
1136 
1137     public String getHostAddress() {
1138       return hostAddress;
1139     }
1140 
1141     public int getRemotePort() {
1142       return remotePort;
1143     }
1144 
1145     public void setLastContact(long lastContact) {
1146       this.lastContact = lastContact;
1147     }
1148 
1149     public long getLastContact() {
1150       return lastContact;
1151     }
1152 
1153     /* Return true if the connection has no outstanding rpc */
1154     private boolean isIdle() {
1155       return rpcCount == 0;
1156     }
1157 
1158     /* Decrement the outstanding RPC count */
1159     protected void decRpcCount() {
1160       rpcCount--;
1161     }
1162 
1163     /* Increment the outstanding RPC count */
1164     protected void incRpcCount() {
1165       rpcCount++;
1166     }
1167 
1168     protected boolean timedOut(long currentTime) {
1169       return isIdle() && currentTime - lastContact > maxIdleTime;
1170     }
1171 
1172     public int readAndProcess() throws IOException, InterruptedException {
1173       while (true) {
1174         /* Read at most one RPC. If the header is not read completely yet
1175          * then iterate until we read first RPC or until there is no data left.
1176          */
1177         int count;
1178         if (dataLengthBuffer.remaining() > 0) {
1179           count = channelRead(channel, dataLengthBuffer);
1180           if (count < 0 || dataLengthBuffer.remaining() > 0)
1181             return count;
1182         }
1183 
1184         if (!versionRead) {
1185           //Every connection is expected to send the header.
1186           ByteBuffer versionBuffer = ByteBuffer.allocate(1);
1187           count = channelRead(channel, versionBuffer);
1188           if (count <= 0) {
1189             return count;
1190           }
1191           int version = versionBuffer.get(0);
1192 
1193           dataLengthBuffer.flip();
1194           if (!HEADER.equals(dataLengthBuffer) || version != CURRENT_VERSION) {
1195             //Warning is ok since this is not supposed to happen.
1196             LOG.warn("Incorrect header or version mismatch from " +
1197                      hostAddress + ":" + remotePort +
1198                      " got version " + version +
1199                      " expected version " + CURRENT_VERSION);
1200             setupBadVersionResponse(version);
1201             return -1;
1202           }
1203           dataLengthBuffer.clear();
1204           versionRead = true;
1205           continue;
1206         }
1207 
1208         if (data == null) {
1209           dataLengthBuffer.flip();
1210           dataLength = dataLengthBuffer.getInt();
1211 
1212           if (dataLength == HBaseClient.PING_CALL_ID) {
1213             dataLengthBuffer.clear();
1214             return 0;  //ping message
1215           }
1216           data = ByteBuffer.allocate(dataLength);
1217           incRpcCount();  // Increment the rpc count
1218         }
1219 
1220         count = channelRead(channel, data);
1221 
1222         if (data.remaining() == 0) {
1223           dataLengthBuffer.clear();
1224           data.flip();
1225           if (headerRead) {
1226             processData(data.array());
1227             data = null;
1228             return count;
1229           }
1230           processHeader();
1231           headerRead = true;
1232           data = null;
1233           continue;
1234         }
1235         return count;
1236       }
1237     }
1238 
1239     /**
1240      * Try to set up the response to indicate that the client version
1241      * is incompatible with the server. This can contain special-case
1242      * code to speak enough of past IPC protocols to pass back
1243      * an exception to the caller.
1244      * @param clientVersion the version the caller is using
1245      * @throws IOException
1246      */
1247     private void setupBadVersionResponse(int clientVersion) throws IOException {
1248       String errMsg = "Server IPC version " + CURRENT_VERSION +
1249       " cannot communicate with client version " + clientVersion;
1250       ByteArrayOutputStream buffer = new ByteArrayOutputStream();
1251 
1252       if (clientVersion >= 3) {
1253         // We used to return an id of -1 which caused server to close the
1254         // connection without telling the client what the problem was.  Now
1255         // we return 0 which will keep the socket up -- bad clients, unless
1256         // they switch to suit the running server -- will fail later doing
1257         // getProtocolVersion.
1258         Call fakeCall =  new Call(0, null, this, responder, 0);
1259         // Versions 3 and greater can interpret this exception
1260         // response in the same manner
1261         setupResponse(buffer, fakeCall, Status.FATAL,
1262             null, VersionMismatch.class.getName(), errMsg);
1263 
1264         responder.doRespond(fakeCall);
1265       }
1266     }
1267 
1268     /// Reads the connection header following version
1269     private void processHeader() throws IOException {
1270       DataInputStream in =
1271         new DataInputStream(new ByteArrayInputStream(data.array()));
1272       header.readFields(in);
1273       try {
1274         String protocolClassName = header.getProtocol();
1275         if (protocolClassName == null) {
1276           protocolClassName = "org.apache.hadoop.hbase.ipc.HRegionInterface";
1277         }
1278         protocol = getProtocolClass(protocolClassName, conf);
1279       } catch (ClassNotFoundException cnfe) {
1280         throw new IOException("Unknown protocol: " + header.getProtocol());
1281       }
1282 
1283       ticket = header.getUser();
1284     }
1285 
1286     protected void processData(byte[] buf) throws  IOException, InterruptedException {
1287       DataInputStream dis =
1288         new DataInputStream(new ByteArrayInputStream(buf));
1289       int id = dis.readInt();                    // try to read an id
1290       long callSize = buf.length;
1291 
1292       if (LOG.isDebugEnabled()) {
1293         LOG.debug(" got call #" + id + ", " + callSize + " bytes");
1294       }
1295 
1296       // Enforcing the call queue size, this triggers a retry in the client
1297       if ((callSize + callQueueSize.get()) > maxQueueSize) {
1298         final Call callTooBig =
1299           new Call(id, null, this, responder, callSize);
1300         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1301         setupResponse(responseBuffer, callTooBig, Status.FATAL, null,
1302             IOException.class.getName(),
1303             "Call queue is full, is ipc.server.max.callqueue.size too small?");
1304         responder.doRespond(callTooBig);
1305         return;
1306       }
1307 
1308       Writable param;
1309       try {
1310         param = ReflectionUtils.newInstance(paramClass, conf);//read param
1311         param.readFields(dis);
1312       } catch (Throwable t) {
1313         LOG.warn("Unable to read call parameters for client " +
1314                  getHostAddress(), t);
1315         final Call readParamsFailedCall =
1316           new Call(id, null, this, responder, callSize);
1317         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1318 
1319         setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
1320             t.getClass().getName(),
1321             "IPC server unable to read call parameters: " + t.getMessage());
1322         responder.doRespond(readParamsFailedCall);
1323         return;
1324       }
1325       Call call = new Call(id, param, this, responder, callSize);
1326       callQueueSize.add(callSize);
1327 
1328       if (priorityCallQueue != null && getQosLevel(param) > highPriorityLevel) {
1329         priorityCallQueue.put(call);
1330         updateCallQueueLenMetrics(priorityCallQueue);
1331       } else if (replicationQueue != null && getQosLevel(param) == HConstants.REPLICATION_QOS) {
1332         replicationQueue.put(call);
1333         updateCallQueueLenMetrics(replicationQueue);
1334       } else {
1335         callQueue.put(call); // queue the call; maybe blocked here
1336         updateCallQueueLenMetrics(callQueue);
1337       }
1338     }
1339 
1340     protected synchronized void close() {
1341       closed = true;
1342       data = null;
1343       dataLengthBuffer = null;
1344       if (!channel.isOpen())
1345         return;
1346       try {socket.shutdownOutput();} catch(Exception ignored) {} // FindBugs DE_MIGHT_IGNORE
1347       if (channel.isOpen()) {
1348         try {channel.close();} catch(Exception ignored) {}
1349       }
1350       try {socket.close();} catch(Exception ignored) {}
1351     }
1352   }
1353 
1354   /**
1355    * Reports length of the call queue to HBaseRpcMetrics.
1356    * @param queue Which queue to report
1357    */
1358   protected void updateCallQueueLenMetrics(BlockingQueue<Call> queue) {
1359     if (queue == callQueue) {
1360       rpcMetrics.callQueueLen.set(callQueue.size());
1361     } else if (queue == priorityCallQueue) {
1362       rpcMetrics.priorityCallQueueLen.set(priorityCallQueue.size());
1363     } else if (queue == replicationQueue) {
1364       rpcMetrics.replicationCallQueueLen.set(replicationQueue.size());
1365     } else {
1366       LOG.warn("Unknown call queue");
1367     }
1368   }
1369 
1370   /** Handles queued calls . */
1371   private class Handler extends Thread {
1372     private final BlockingQueue<Call> myCallQueue;
1373     private MonitoredRPCHandler status;
1374 
1375     public Handler(final BlockingQueue<Call> cq, int instanceNumber) {
1376       this.myCallQueue = cq;
1377       this.setDaemon(true);
1378 
1379       String threadName = "IPC Server handler " + instanceNumber + " on " + port;
1380       if (cq == priorityCallQueue) {
1381         // this is just an amazing hack, but it works.
1382         threadName = "PRI " + threadName;
1383       } else if (cq == replicationQueue) {
1384         threadName = "REPL " + threadName;
1385       }
1386       this.setName(threadName);
1387       this.status = TaskMonitor.get().createRPCStatus(threadName);
1388     }
1389 
1390     @Override
1391     public void run() {
1392       LOG.info(getName() + ": starting");
1393       status.setStatus("starting");
1394       SERVER.set(HBaseServer.this);
1395       while (running) {
1396         try {
1397           status.pause("Waiting for a call");
1398           Call call = myCallQueue.take(); // pop the queue; maybe blocked here
1399           updateCallQueueLenMetrics(myCallQueue);
1400           status.setStatus("Setting up call");
1401           status.setConnection(call.connection.getHostAddress(), 
1402               call.connection.getRemotePort());
1403 
1404           if (LOG.isDebugEnabled())
1405             LOG.debug(getName() + ": has #" + call.id + " from " +
1406                       call.connection);
1407 
1408           String errorClass = null;
1409           String error = null;
1410           Writable value = null;
1411 
1412           CurCall.set(call);
1413           try {
1414             if (!started)
1415               throw new ServerNotRunningYetException("Server is not running yet");
1416 
1417             if (LOG.isDebugEnabled()) {
1418               User remoteUser = call.connection.ticket;
1419               LOG.debug(getName() + ": call #" + call.id + " executing as "
1420                   + (remoteUser == null ? "NULL principal" : remoteUser.getName()));
1421             }
1422 
1423             RequestContext.set(call.connection.ticket, getRemoteIp(),
1424                 call.connection.protocol);
1425             // make the call
1426             value = call(call.connection.protocol, call.param, call.timestamp, 
1427                 status);
1428           } catch (Throwable e) {
1429             LOG.debug(getName()+", call "+call+": error: " + e, e);
1430             errorClass = e.getClass().getName();
1431             error = StringUtils.stringifyException(e);
1432           } finally {
1433             // Must always clear the request context to avoid leaking
1434             // credentials between requests.
1435             RequestContext.clear();
1436           }
1437           CurCall.set(null);
1438           callQueueSize.add(call.getSize() * -1);
1439           // Set the response for undelayed calls and delayed calls with
1440           // undelayed responses.
1441           if (!call.isDelayed() || !call.isReturnValueDelayed()) {
1442             call.setResponse(value,
1443               errorClass == null? Status.SUCCESS: Status.ERROR,
1444                 errorClass, error);
1445           }
1446           call.sendResponseIfReady();
1447           status.markComplete("Sent response");
1448         } catch (InterruptedException e) {
1449           if (running) {                          // unexpected -- log it
1450             LOG.info(getName() + " caught: " +
1451                      StringUtils.stringifyException(e));
1452           }
1453         } catch (OutOfMemoryError e) {
1454           if (errorHandler != null) {
1455             if (errorHandler.checkOOME(e)) {
1456               LOG.info(getName() + ": exiting on OOME");
1457               return;
1458             }
1459           } else {
1460             // rethrow if no handler
1461             throw e;
1462           }
1463        } catch (ClosedChannelException cce) {
1464           LOG.warn(getName() + " caught a ClosedChannelException, " +
1465             "this means that the server was processing a " +
1466             "request but the client went away. The error message was: " +
1467             cce.getMessage());
1468         } catch (Exception e) {
1469           LOG.warn(getName() + " caught: " +
1470                    StringUtils.stringifyException(e));
1471         }
1472       }
1473       LOG.info(getName() + ": exiting");
1474     }
1475 
1476   }
1477 
1478 
1479   private Function<Writable,Integer> qosFunction = null;
1480 
1481   /**
1482    * Gets the QOS level for this call.  If it is higher than the highPriorityLevel and there
1483    * are priorityHandlers available it will be processed in it's own thread set.
1484    *
1485    * @param newFunc
1486    */
1487   @Override
1488   public void setQosFunction(Function<Writable, Integer> newFunc) {
1489     qosFunction = newFunc;
1490   }
1491 
1492   protected int getQosLevel(Writable param) {
1493     if (qosFunction == null) {
1494       return 0;
1495     }
1496 
1497     Integer res = qosFunction.apply(param);
1498     if (res == null) {
1499       return 0;
1500     }
1501     return res;
1502   }
1503 
1504   /* Constructs a server listening on the named port and address.  Parameters passed must
1505    * be of the named class.  The <code>handlerCount</handlerCount> determines
1506    * the number of handler threads that will be used to process calls.
1507    *
1508    */
1509   protected HBaseServer(String bindAddress, int port,
1510                         Class<? extends Writable> paramClass, int handlerCount,
1511                         int priorityHandlerCount, Configuration conf, String serverName,
1512                         int highPriorityLevel)
1513     throws IOException {
1514     this.bindAddress = bindAddress;
1515     this.conf = conf;
1516     this.port = port;
1517     this.paramClass = paramClass;
1518     this.handlerCount = handlerCount;
1519     this.priorityHandlerCount = priorityHandlerCount;
1520     this.socketSendBufferSize = 0;
1521 
1522     // temporary backward compatibility
1523     String oldMaxQueueSize = this.conf.get("ipc.server.max.queue.size");
1524     if (oldMaxQueueSize == null) {
1525       this.maxQueueLength =
1526         this.conf.getInt("ipc.server.max.callqueue.length",
1527           handlerCount * DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
1528     } else {
1529       LOG.warn("ipc.server.max.queue.size was renamed " +
1530                "ipc.server.max.callqueue.length, " +
1531                "please update your configuration");
1532       this.maxQueueLength = Integer.getInteger(oldMaxQueueSize);
1533     }
1534 
1535     this.maxQueueSize =
1536       this.conf.getInt("ipc.server.max.callqueue.size",
1537         DEFAULT_MAX_CALLQUEUE_SIZE);
1538      this.readThreads = conf.getInt(
1539         "ipc.server.read.threadpool.size",
1540         10);
1541     this.callQueue  = new LinkedBlockingQueue<Call>(maxQueueLength);
1542     if (priorityHandlerCount > 0) {
1543       this.priorityCallQueue = new LinkedBlockingQueue<Call>(maxQueueLength); // TODO hack on size
1544     } else {
1545       this.priorityCallQueue = null;
1546     }
1547     this.highPriorityLevel = highPriorityLevel;
1548     this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
1549     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
1550     this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
1551     this.purgeTimeout = conf.getLong("ipc.client.call.purge.timeout",
1552                                      2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
1553     this.numOfReplicationHandlers = 
1554       conf.getInt("hbase.regionserver.replication.handler.count", 3);
1555     if (numOfReplicationHandlers > 0) {
1556       this.replicationQueue = new LinkedBlockingQueue<Call>(maxQueueSize);
1557     }
1558     // Start the listener here and let it bind to the port
1559     listener = new Listener();
1560     this.port = listener.getAddress().getPort();
1561     this.rpcMetrics = new HBaseRpcMetrics(
1562         serverName, Integer.toString(this.port));
1563     this.tcpNoDelay = conf.getBoolean("ipc.server.tcpnodelay", false);
1564     this.tcpKeepAlive = conf.getBoolean("ipc.server.tcpkeepalive", true);
1565 
1566     this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS,
1567                                         DEFAULT_WARN_DELAYED_CALLS);
1568     this.delayedCalls = new AtomicInteger(0);
1569 
1570 
1571     this.responseQueuesSizeThrottler = new SizeBasedThrottler(
1572         conf.getLong(RESPONSE_QUEUES_MAX_SIZE, DEFAULT_RESPONSE_QUEUES_MAX_SIZE));
1573 
1574     // Create the responder here
1575     responder = new Responder();
1576   }
1577 
1578   /**
1579    * Subclasses of HBaseServer can override this to provide their own
1580    * Connection implementations.
1581    */
1582   protected Connection getConnection(SocketChannel channel, long time) {
1583     return new Connection(channel, time);
1584   }
1585 
1586   /**
1587    * Setup response for the IPC Call.
1588    *
1589    * @param response buffer to serialize the response into
1590    * @param call {@link Call} to which we are setting up the response
1591    * @param status {@link Status} of the IPC call
1592    * @param rv return value for the IPC Call, if the call was successful
1593    * @param errorClass error class, if the the call failed
1594    * @param error error message, if the call failed
1595    * @throws IOException
1596    */
1597   private void setupResponse(ByteArrayOutputStream response,
1598                              Call call, Status status,
1599                              Writable rv, String errorClass, String error)
1600   throws IOException {
1601     response.reset();
1602     DataOutputStream out = new DataOutputStream(response);
1603 
1604     if (status == Status.SUCCESS) {
1605       try {
1606         rv.write(out);
1607         call.setResponse(rv, status, null, null);
1608       } catch (Throwable t) {
1609         LOG.warn("Error serializing call response for call " + call, t);
1610         // Call back to same function - this is OK since the
1611         // buffer is reset at the top, and since status is changed
1612         // to ERROR it won't infinite loop.
1613         call.setResponse(null, status.ERROR, t.getClass().getName(),
1614             StringUtils.stringifyException(t));
1615       }
1616     } else {
1617       call.setResponse(rv, status, errorClass, error);
1618     }
1619   }
1620 
1621   protected void closeConnection(Connection connection) {
1622     synchronized (connectionList) {
1623       if (connectionList.remove(connection)) {
1624         numConnections--;
1625       }
1626     }
1627     connection.close();
1628     long bytes = 0;
1629     synchronized (connection.responseQueue) {
1630       for (Call c : connection.responseQueue) {
1631         bytes += c.response.limit();
1632       }
1633       connection.responseQueue.clear();
1634     }
1635     responseQueuesSizeThrottler.decrease(bytes);    
1636     rpcMetrics.numOpenConnections.set(numConnections);
1637   }
1638 
1639   /** Sets the socket buffer size used for responding to RPCs.
1640    * @param size send size
1641    */
1642   @Override
1643   public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
1644 
1645   /** Starts the service.  Must be called before any calls will be handled. */
1646   @Override
1647   public void start() {
1648     startThreads();
1649     openServer();
1650   }
1651 
1652   /**
1653    * Open a previously started server.
1654    */
1655   @Override
1656   public void openServer() {
1657     started = true;
1658   }
1659 
1660   /**
1661    * Starts the service threads but does not allow requests to be responded yet.
1662    * Client will get {@link ServerNotRunningYetException} instead.
1663    */
1664   @Override
1665   public synchronized void startThreads() {
1666     responder.start();
1667     listener.start();
1668     handlers = startHandlers(callQueue, handlerCount);
1669     priorityHandlers = startHandlers(priorityCallQueue, priorityHandlerCount);
1670     replicationHandlers = startHandlers(replicationQueue, numOfReplicationHandlers);
1671     }
1672 
1673   private Handler[] startHandlers(BlockingQueue<Call> queue, int numOfHandlers) {
1674     if (numOfHandlers <= 0) {
1675       return null;
1676     }
1677     Handler[] handlers = new Handler[numOfHandlers];
1678     for (int i = 0; i < numOfHandlers; i++) {
1679       handlers[i] = new Handler(queue, i);
1680       handlers[i].start();
1681     }
1682     return handlers;
1683   }
1684 
1685   /** Stops the service.  No new calls will be handled after this is called. */
1686   @Override
1687   public synchronized void stop() {
1688     LOG.info("Stopping server on " + port);
1689     running = false;
1690     stopHandlers(handlers);
1691     stopHandlers(priorityHandlers);
1692     stopHandlers(replicationHandlers);
1693     listener.interrupt();
1694     listener.doStop();
1695     responder.interrupt();
1696     notifyAll();
1697     if (this.rpcMetrics != null) {
1698       this.rpcMetrics.shutdown();
1699     }
1700   }
1701 
1702   private void stopHandlers(Handler[] handlers) {
1703     if (handlers != null) {
1704       for (Handler handler : handlers) {
1705         if (handler != null) {
1706           handler.interrupt();
1707         }
1708       }
1709     }
1710   }
1711 
1712   /** Wait for the server to be stopped.
1713    * Does not wait for all subthreads to finish.
1714    *  See {@link #stop()}.
1715    * @throws InterruptedException e
1716    */
1717   @Override
1718   public synchronized void join() throws InterruptedException {
1719     while (running) {
1720       wait();
1721     }
1722   }
1723 
1724   /**
1725    * Return the socket (ip+port) on which the RPC server is listening to.
1726    * @return the socket (ip+port) on which the RPC server is listening to.
1727    */
1728   @Override
1729   public synchronized InetSocketAddress getListenerAddress() {
1730     return listener.getAddress();
1731   }
1732 
1733   /**
1734    * Set the handler for calling out of RPC for error conditions.
1735    * @param handler the handler implementation
1736    */
1737   @Override
1738   public void setErrorHandler(HBaseRPCErrorHandler handler) {
1739     this.errorHandler = handler;
1740   }
1741 
1742   /**
1743    * Returns the metrics instance for reporting RPC call statistics
1744    */
1745   public HBaseRpcMetrics getRpcMetrics() {
1746     return rpcMetrics;
1747   }
1748 
1749   /**
1750    * When the read or write buffer size is larger than this limit, i/o will be
1751    * done in chunks of this size. Most RPC requests and responses would be
1752    * be smaller.
1753    */
1754   private static int NIO_BUFFER_LIMIT = 64 * 1024; //should not be more than 64KB.
1755 
1756   /**
1757    * This is a wrapper around {@link java.nio.channels.WritableByteChannel#write(java.nio.ByteBuffer)}.
1758    * If the amount of data is large, it writes to channel in smaller chunks.
1759    * This is to avoid jdk from creating many direct buffers as the size of
1760    * buffer increases. This also minimizes extra copies in NIO layer
1761    * as a result of multiple write operations required to write a large
1762    * buffer.
1763    *
1764    * @param channel writable byte channel to write to
1765    * @param buffer buffer to write
1766    * @return number of bytes written
1767    * @throws java.io.IOException e
1768    * @see java.nio.channels.WritableByteChannel#write(java.nio.ByteBuffer)
1769    */
1770   protected int channelWrite(WritableByteChannel channel,
1771                                     ByteBuffer buffer) throws IOException {
1772 
1773     int count =  (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1774            channel.write(buffer) : channelIO(null, channel, buffer);
1775     if (count > 0) {
1776       rpcMetrics.sentBytes.inc(count);
1777     }
1778     return count;
1779   }
1780 
1781   /**
1782    * This is a wrapper around {@link java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)}.
1783    * If the amount of data is large, it writes to channel in smaller chunks.
1784    * This is to avoid jdk from creating many direct buffers as the size of
1785    * ByteBuffer increases. There should not be any performance degredation.
1786    *
1787    * @param channel writable byte channel to write on
1788    * @param buffer buffer to write
1789    * @return number of bytes written
1790    * @throws java.io.IOException e
1791    * @see java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)
1792    */
1793   protected int channelRead(ReadableByteChannel channel,
1794                                    ByteBuffer buffer) throws IOException {
1795 
1796     int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1797            channel.read(buffer) : channelIO(channel, null, buffer);
1798     if (count > 0) {
1799       rpcMetrics.receivedBytes.inc(count);
1800   }
1801     return count;
1802   }
1803 
1804   /**
1805    * Helper for {@link #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)}
1806    * and {@link #channelWrite(java.nio.channels.WritableByteChannel, java.nio.ByteBuffer)}. Only
1807    * one of readCh or writeCh should be non-null.
1808    *
1809    * @param readCh read channel
1810    * @param writeCh write channel
1811    * @param buf buffer to read or write into/out of
1812    * @return bytes written
1813    * @throws java.io.IOException e
1814    * @see #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)
1815    * @see #channelWrite(java.nio.channels.WritableByteChannel, java.nio.ByteBuffer)
1816    */
1817   private static int channelIO(ReadableByteChannel readCh,
1818                                WritableByteChannel writeCh,
1819                                ByteBuffer buf) throws IOException {
1820 
1821     int originalLimit = buf.limit();
1822     int initialRemaining = buf.remaining();
1823     int ret = 0;
1824 
1825     while (buf.remaining() > 0) {
1826       try {
1827         int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
1828         buf.limit(buf.position() + ioSize);
1829 
1830         ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
1831 
1832         if (ret < ioSize) {
1833           break;
1834         }
1835 
1836       } finally {
1837         buf.limit(originalLimit);
1838       }
1839     }
1840 
1841     int nBytes = initialRemaining - buf.remaining();
1842     return (nBytes > 0) ? nBytes : ret;
1843   }
1844 
1845   /**
1846    * Needed for delayed calls.  We need to be able to store the current call
1847    * so that we can complete it later.
1848    * @return Call the server is currently handling.
1849    */
1850   public static RpcCallContext getCurrentCall() {
1851     return CurCall.get();
1852   }
1853 
1854   public long getResponseQueueSize(){
1855     return responseQueuesSizeThrottler.getCurrentValue();
1856   }
1857 }