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.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  /** An abstract IPC service.  IPC calls take a single {@link Writable} as a
78   * parameter, and return a {@link Writable} as their value.  A service runs on
79   * a port and is defined by a parameter class and a value class.
80   *
81   *
82   * <p>Copied local so can fix HBASE-900.
83   *
84   * @see HBaseClient
85   */
86  public abstract class HBaseServer implements RpcServer {
87  
88    /**
89     * The first four bytes of Hadoop RPC connections
90     */
91    public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
92    public static final byte CURRENT_VERSION = 3;
93  
94    /**
95     * How many calls/handler are allowed in the queue.
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   /** Returns the server instance called under or null.  May be called under
138    * {@link #call(Class, Writable, long, MonitoredRPCHandler)} implementations,
139    * and under {@link Writable} methods of paramters and return values.
140    * Permits applications to access the server context.
141    * @return HBaseServer
142    */
143   public static RpcServer get() {
144     return SERVER.get();
145   }
146 
147   /** This is set to Call object before Handler invokes an RPC and reset
148    * after the call returns.
149    */
150   protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
151 
152   /** Returns the remote side ip address when invoked inside an RPC
153    *  Returns null incase of an error.
154    *  @return InetAddress
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   /** Returns remote address as a string when invoked inside an RPC.
164    *  Returns null in case of an error.
165    *  @return String
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;                             // port we listen on
174   private int handlerCount;                       // number of handler threads
175   private int priorityHandlerCount;
176   private int readThreads;                        // number of read threads
177   protected Class<? extends Writable> paramClass; // class of call parameters
178   protected int maxIdleTime;                      // the maximum idle time after
179                                                   // which a client may be
180                                                   // disconnected
181   protected int thresholdIdleConnections;         // the number of idle
182                                                   // connections after which we
183                                                   // will start cleaning up idle
184                                                   // connections
185   int maxConnectionsToNuke;                       // the max number of
186                                                   // connections to nuke
187                                                   // during a cleanup
188 
189   protected HBaseRpcMetrics  rpcMetrics;
190 
191   protected Configuration conf;
192 
193   private int maxQueueSize;
194   protected int socketSendBufferSize;
195   protected final boolean tcpNoDelay;   // if T then disable Nagle's Algorithm
196   protected final boolean tcpKeepAlive; // if T then use keepalives
197 
198   volatile protected boolean running = true;         // true while server runs
199   protected BlockingQueue<Call> callQueue; // queued calls
200   protected BlockingQueue<Call> priorityCallQueue;
201 
202   protected int highPriorityLevel;  // what level a high priority call is at
203 
204   protected final List<Connection> connectionList =
205     Collections.synchronizedList(new LinkedList<Connection>());
206   //maintain a list
207   //of client connections
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    * A convenience method to bind to a given address and report
217    * better exceptions if the address is not a valid host.
218    * @param socket the socket to bind
219    * @param address the address to bind to
220    * @param backlog the number of connections allowed in the queue
221    * @throws BindException if the address can't be bound
222    * @throws UnknownHostException if the address isn't a valid host name
223    * @throws IOException other random errors from bind
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       // If they try to bind to a different host's address, give a better
237       // error message.
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   /** A call queued for handling. */
247   protected class Call implements Delayable {
248     protected int id;                             // the client's call id
249     protected Writable param;                     // the parameter passed
250     protected Connection connection;              // connection to client
251     protected long timestamp;      // the time received when response is null
252                                    // the time served when response is not null
253     protected ByteBuffer response;                // the response for this call
254     protected boolean delayResponse;
255     protected Responder responder;
256     protected boolean delayReturnValue;           // if the return value should be
257                                                   // set at call completion
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       // Avoid overwriting an error value in the response.  This can happen if
280       // endDelayThrowing is called by another thread before the actual call
281       // returning.
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         /* We might have a null value and errors. Avoid creating a
292          * HbaseObjectWritable, because the constructor fails on null. */
293         if (value != null) {
294           result = new HbaseObjectWritable(value);
295         }
296       }
297 
298       int size = BUFFER_INITIAL_SIZE;
299       if (result instanceof WritableWithSize) {
300         // get the size hint.
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           // oops, new problem.
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         // Call id.
319         out.writeInt(this.id);
320         // Write flag.
321         byte flag = (error != null)?
322           ResponseFlag.getErrorAndLengthSet(): ResponseFlag.getLengthSetOnly();
323         out.writeByte(flag);
324         // Place holder for length set later below after we
325         // fill the buffer with data.
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       // Set the length into the ByteBuffer after call id and after
345       // byte flag.
346       ByteBuffer bb = buf.getByteBuffer();
347       int bufSiz = bb.remaining();
348       // Move to the size location in our ByteBuffer past call.id
349       // and past the byte flag.
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      * If we have a response, and delay is not set, then respond
404      * immediately.  Otherwise, do not respond to client.  This is
405      * called the by the RPC code in the context of the Handler thread.
406      */
407     public synchronized void sendResponseIfReady() throws IOException {
408       if (!this.delayResponse) {
409         this.responder.doRespond(this);
410       }
411     }
412   }
413 
414   /** Listens on the socket. Creates jobs for the handler threads*/
415   private class Listener extends Thread {
416 
417     private ServerSocketChannel acceptChannel = null; //the accept channel
418     private Selector selector = null; //the selector that we use for the server
419     private Reader[] readers = null;
420     private int currentReader = 0;
421     private InetSocketAddress address; //the address we bind at
422     private Random rand = new Random();
423     private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
424                                          //-tion (for idle connections) ran
425     private long cleanupInterval = 10000; //the minimum interval between
426                                           //two cleanup runs
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       // Create a new server socket and set to non blocking mode
434       acceptChannel = ServerSocketChannel.open();
435       acceptChannel.configureBlocking(false);
436 
437       // Bind the server socket to the local host and port
438       bind(acceptChannel.socket(), address, backlogLength);
439       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
440       // create a selector;
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       // Register accepts on the server socket with the selector.
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) {                      // unexpected -- log it
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        * This gets reader into the state that waits for the new channel
513        * to be registered with readSelector. If it was waiting in select()
514        * the thread will be woken up, otherwise whenever select() is called
515        * it will return even if there is nothing to read and wait
516        * in while(adding) for finishAdd call
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     /** cleanup connections from connectionList. Choose a random range
535      * to scan and also have a limit on the number of the connections
536      * that will be cleanedup per run. The criteria for cleanup is the time
537      * for which the connection was idle. If 'force' is true then all
538      * connections will be looked at for the cleanup.
539      * @param force all connections will be looked at for cleanup
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             //noinspection UnusedAssignment
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(); // FindBugs IS2_INCONSISTENT_SYNC
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             // we can run out of memory if we have too many threads
616             // log the event and sleep for a minute and give
617             // some thread(s) a chance to finish
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         // clean up all connections
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; //so that the (count < 0) block is executed
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         // c = null;
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     // The method that will return the next reader to work with
739     // Simplistic implementation of round robin for now
740     Reader getReader() {
741       currentReader = (currentReader + 1) % readers.length;
742       return readers[currentReader];
743     }
744   }
745 
746   // Sends responses of RPC back to clients.
747   protected class Responder extends Thread {
748     private final Selector writeSelector;
749     private int pending;         // connections waiting to register
750 
751     final static int PURGE_INTERVAL = 900000; // 15mins
752 
753     Responder() throws IOException {
754       this.setName("IPC Server Responder");
755       this.setDaemon(true);
756       writeSelector = Selector.open(); // create a selector
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;   // last check for old calls.
778 
779       while (running) {
780         try {
781           waitPending();     // If a channel is being registered, wait.
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           // If there were some calls that have not been sent out for a
802           // long time, discard them.
803           //
804           LOG.debug("Checking for old call responses.");
805           ArrayList<Call> calls;
806 
807           // get the list of channels from list of keys.
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             // we can run out of memory if we have too many threads
836             // log the event and sleep for a minute and give
837             // some thread(s) a chance to finish
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             /* The Listener/reader might have closed the socket.
865              * We don't explicitly cancel the key, so not sure if this will
866              * ever fire.
867              * This warning could be removed.
868              */
869             LOG.warn("Exception while changing ops : " + e);
870           }
871         }
872       }
873     }
874 
875     //
876     // Remove calls that have been pending in the responseQueue
877     // for a long time.
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     // Processes one response. Returns true if there are no more pending
893     // data for this channel.
894     //
895     private boolean processResponse(final LinkedList<Call> responseQueue,
896                                     boolean inHandler) throws IOException {
897       boolean error = true;
898       boolean done = false;       // there is more data for this channel.
899       int numElements;
900       Call call = null;
901       try {
902         //noinspection SynchronizationOnLocalVariableOrMethodParameter
903         synchronized (responseQueue) {
904           //
905           // If there are no items for this channel, then we are done
906           //
907           numElements = responseQueue.size();
908           if (numElements == 0) {
909             error = false;
910             return true;              // no more data for this channel.
911           }
912           //
913           // Extract the first call
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           // Send as much data as we can in the non-blocking fashion
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             //noinspection RedundantIfStatement
931             if (numElements == 1) {    // last call fully processes.
932               done = true;             // no more data for this channel.
933             } else {
934               done = false;            // more calls pending to be sent.
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             // If we were unable to write the entire response out, then
943             // insert in Selector queue.
944             //
945             call.connection.responseQueue.addFirst(call);
946 
947             if (inHandler) {
948               // set the serve time when the response has to be sent later
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;              // everything went off well
960         }
961       } finally {
962         if (error && call != null) {
963           LOG.warn(getName()+", call " + call + ": output error");
964           done = true;               // error. no more data for this channel.
965           closeConnection(call.connection);
966         }
967       }
968       return done;
969     }
970 
971     //
972     // Enqueue for background thread to send responses out later.
973     //
974     private boolean enqueueInSelector(Call call) throws IOException {
975       boolean done = false;
976       incPending();
977       try {
978         // Wake up the thread blocked on select, only then can the call
979         // to channel.register() complete.
980         SocketChannel channel = call.connection.channel;
981         writeSelector.wakeup();
982         channel.register(writeSelector, SelectionKey.OP_WRITE, call);
983       } catch (ClosedChannelException e) {
984         //It's OK.  Channel might be closed else where.
985         done = true;
986       } finally {
987         decPending();
988       }
989       return done;
990     }
991 
992     //
993     // Enqueue a response from the application.
994     //
995     void doRespond(Call call) throws IOException {
996       // set the serve time when the response has to be sent later
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() {   // call waiting to be enqueued.
1012       pending++;
1013     }
1014 
1015     private synchronized void decPending() { // call done enqueueing.
1016       pending--;
1017       notify();
1018     }
1019 
1020     private synchronized void waitPending() throws InterruptedException {
1021       while (pending > 0) {
1022         wait();
1023       }
1024     }
1025   }
1026 
1027   /** Reads calls from a connection and queues them for handling. */
1028   protected class Connection {
1029     private boolean versionRead = false; //if initial signature and
1030                                          //version are read
1031     private boolean headerRead = false;  //if the connection header that
1032                                          //follows version is read.
1033     protected SocketChannel channel;
1034     private ByteBuffer data;
1035     private ByteBuffer dataLengthBuffer;
1036     protected final LinkedList<Call> responseQueue;
1037     private volatile int rpcCount = 0; // number of outstanding rpcs
1038     private long lastContact;
1039     private int dataLength;
1040     protected Socket socket;
1041     // Cache the remote host & port info so that even if the socket is
1042     // disconnected, we can say where it used to connect to.
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     /* Return true if the connection has no outstanding rpc */
1095     private boolean isIdle() {
1096       return rpcCount == 0;
1097     }
1098 
1099     /* Decrement the outstanding RPC count */
1100     protected void decRpcCount() {
1101       rpcCount--;
1102     }
1103 
1104     /* Increment the outstanding RPC count */
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         /* Read at most one RPC. If the header is not read completely yet
1116          * then iterate until we read first RPC or until there is no data left.
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           //Every connection is expected to send the header.
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             //Warning is ok since this is not supposed to happen.
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;  //ping message
1156           }
1157           data = ByteBuffer.allocate(dataLength);
1158           incRpcCount();  // Increment the rpc count
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      * Try to set up the response to indicate that the client version
1182      * is incompatible with the server. This can contain special-case
1183      * code to speak enough of past IPC protocols to pass back
1184      * an exception to the caller.
1185      * @param clientVersion the version the caller is using
1186      * @throws IOException
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         // We used to return an id of -1 which caused server to close the
1195         // connection without telling the client what the problem was.  Now
1196         // we return 0 which will keep the socket up -- bad clients, unless
1197         // they switch to suit the running server -- will fail later doing
1198         // getProtocolVersion.
1199         Call fakeCall =  new Call(0, null, this, responder);
1200         // Versions 3 and greater can interpret this exception
1201         // response in the same manner
1202         setupResponse(buffer, fakeCall, Status.FATAL,
1203             null, VersionMismatch.class.getName(), errMsg);
1204 
1205         responder.doRespond(fakeCall);
1206       }
1207     }
1208 
1209     /// Reads the connection header following version
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();                    // try to read an id
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);//read param
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);              // queue the call; maybe blocked here
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) {} // FindBugs DE_MIGHT_IGNORE
1265       if (channel.isOpen()) {
1266         try {channel.close();} catch(Exception ignored) {}
1267       }
1268       try {socket.close();} catch(Exception ignored) {}
1269     }
1270   }
1271 
1272   /** Handles queued calls . */
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         // this is just an amazing hack, but it works.
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(); // pop the queue; maybe blocked here
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             // make the call
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             // Must always clear the request context to avoid leaking
1333             // credentials between requests.
1334             RequestContext.clear();
1335           }
1336           CurCall.set(null);
1337 
1338           // Set the response for undelayed calls and delayed calls with
1339           // undelayed responses.
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) {                          // unexpected -- log it
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             // rethrow if no handler
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    * Gets the QOS level for this call.  If it is higher than the highPriorityLevel and there
1376    * are priorityHandlers available it will be processed in it's own thread set.
1377    *
1378    * @param newFunc
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   /* Constructs a server listening on the named port and address.  Parameters passed must
1398    * be of the named class.  The <code>handlerCount</handlerCount> determines
1399    * the number of handler threads that will be used to process calls.
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); // TODO hack on size
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     // Start the listener here and let it bind to the port
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     // Create the responder here
1445     responder = new Responder();
1446   }
1447 
1448   /**
1449    * Subclasses of HBaseServer can override this to provide their own
1450    * Connection implementations.
1451    */
1452   protected Connection getConnection(SocketChannel channel, long time) {
1453     return new Connection(channel, time);
1454   }
1455 
1456   /**
1457    * Setup response for the IPC Call.
1458    *
1459    * @param response buffer to serialize the response into
1460    * @param call {@link Call} to which we are setting up the response
1461    * @param status {@link Status} of the IPC call
1462    * @param rv return value for the IPC Call, if the call was successful
1463    * @param errorClass error class, if the the call failed
1464    * @param error error message, if the call failed
1465    * @throws IOException
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         // Call back to same function - this is OK since the
1481         // buffer is reset at the top, and since status is changed
1482         // to ERROR it won't infinite loop.
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   /** Sets the socket buffer size used for responding to RPCs.
1500    * @param size send size
1501    */
1502   @Override
1503   public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
1504 
1505   /** Starts the service.  Must be called before any calls will be handled. */
1506   @Override
1507   public void start() {
1508     startThreads();
1509     openServer();
1510   }
1511 
1512   /**
1513    * Open a previously started server.
1514    */
1515   @Override
1516   public void openServer() {
1517     started = true;
1518   }
1519 
1520   /**
1521    * Starts the service threads but does not allow requests to be responded yet.
1522    * Client will get {@link ServerNotRunningYetException} instead.
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   /** Stops the service.  No new calls will be handled after this is called. */
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   /** Wait for the server to be stopped.
1573    * Does not wait for all subthreads to finish.
1574    *  See {@link #stop()}.
1575    * @throws InterruptedException e
1576    */
1577   @Override
1578   public synchronized void join() throws InterruptedException {
1579     while (running) {
1580       wait();
1581     }
1582   }
1583 
1584   /**
1585    * Return the socket (ip+port) on which the RPC server is listening to.
1586    * @return the socket (ip+port) on which the RPC server is listening to.
1587    */
1588   @Override
1589   public synchronized InetSocketAddress getListenerAddress() {
1590     return listener.getAddress();
1591   }
1592 
1593   /**
1594    * The number of open RPC conections
1595    * @return the number of open rpc connections
1596    */
1597   @Override
1598   public int getNumOpenConnections() {
1599     return numConnections;
1600   }
1601 
1602   /**
1603    * The number of rpc calls in the queue.
1604    * @return The number of rpc calls in the queue.
1605    */
1606   @Override
1607   public int getCallQueueLen() {
1608     return callQueue.size();
1609   }
1610 
1611   /**
1612    * Set the handler for calling out of RPC for error conditions.
1613    * @param handler the handler implementation
1614    */
1615   @Override
1616   public void setErrorHandler(HBaseRPCErrorHandler handler) {
1617     this.errorHandler = handler;
1618   }
1619 
1620   /**
1621    * Returns the metrics instance for reporting RPC call statistics
1622    */
1623   public HBaseRpcMetrics getRpcMetrics() {
1624     return rpcMetrics;
1625   }
1626 
1627   /**
1628    * When the read or write buffer size is larger than this limit, i/o will be
1629    * done in chunks of this size. Most RPC requests and responses would be
1630    * be smaller.
1631    */
1632   private static int NIO_BUFFER_LIMIT = 8*1024; //should not be more than 64KB.
1633 
1634   /**
1635    * This is a wrapper around {@link java.nio.channels.WritableByteChannel#write(java.nio.ByteBuffer)}.
1636    * If the amount of data is large, it writes to channel in smaller chunks.
1637    * This is to avoid jdk from creating many direct buffers as the size of
1638    * buffer increases. This also minimizes extra copies in NIO layer
1639    * as a result of multiple write operations required to write a large
1640    * buffer.
1641    *
1642    * @param channel writable byte channel to write to
1643    * @param buffer buffer to write
1644    * @return number of bytes written
1645    * @throws java.io.IOException e
1646    * @see java.nio.channels.WritableByteChannel#write(java.nio.ByteBuffer)
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    * This is a wrapper around {@link java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)}.
1661    * If the amount of data is large, it writes to channel in smaller chunks.
1662    * This is to avoid jdk from creating many direct buffers as the size of
1663    * ByteBuffer increases. There should not be any performance degredation.
1664    *
1665    * @param channel writable byte channel to write on
1666    * @param buffer buffer to write
1667    * @return number of bytes written
1668    * @throws java.io.IOException e
1669    * @see java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)
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    * Helper for {@link #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)}
1684    * and {@link #channelWrite(java.nio.channels.WritableByteChannel, java.nio.ByteBuffer)}. Only
1685    * one of readCh or writeCh should be non-null.
1686    *
1687    * @param readCh read channel
1688    * @param writeCh write channel
1689    * @param buf buffer to read or write into/out of
1690    * @return bytes written
1691    * @throws java.io.IOException e
1692    * @see #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)
1693    * @see #channelWrite(java.nio.channels.WritableByteChannel, java.nio.ByteBuffer)
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 }