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 com.google.common.base.Function;
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.hbase.util.Bytes;
28  import org.apache.hadoop.io.ObjectWritable;
29  import org.apache.hadoop.io.Writable;
30  import org.apache.hadoop.io.WritableUtils;
31  import org.apache.hadoop.security.UserGroupInformation;
32  import org.apache.hadoop.util.ReflectionUtils;
33  import org.apache.hadoop.util.StringUtils;
34  
35  import java.io.ByteArrayInputStream;
36  import java.io.ByteArrayOutputStream;
37  import java.io.DataInputStream;
38  import java.io.DataOutputStream;
39  import java.io.IOException;
40  import java.net.BindException;
41  import java.net.InetAddress;
42  import java.net.InetSocketAddress;
43  import java.net.ServerSocket;
44  import java.net.Socket;
45  import java.net.SocketException;
46  import java.net.UnknownHostException;
47  import java.nio.ByteBuffer;
48  import java.nio.channels.CancelledKeyException;
49  import java.nio.channels.ClosedChannelException;
50  import java.nio.channels.ReadableByteChannel;
51  import java.nio.channels.SelectionKey;
52  import java.nio.channels.Selector;
53  import java.nio.channels.ServerSocketChannel;
54  import java.nio.channels.SocketChannel;
55  import java.nio.channels.WritableByteChannel;
56  import java.util.ArrayList;
57  import java.util.Collections;
58  import java.util.Iterator;
59  import java.util.LinkedList;
60  import java.util.List;
61  import java.util.Random;
62  import java.util.concurrent.BlockingQueue;
63  import java.util.concurrent.ExecutorService;
64  import java.util.concurrent.Executors;
65  import java.util.concurrent.LinkedBlockingQueue;
66  import java.security.PrivilegedExceptionAction;
67  
68  /** An abstract IPC service.  IPC calls take a single {@link Writable} as a
69   * parameter, and return a {@link Writable} as their value.  A service runs on
70   * a port and is defined by a parameter class and a value class.
71   *
72   *
73   * <p>Copied local so can fix HBASE-900.
74   *
75   * @see HBaseClient
76   */
77  public abstract class HBaseServer {
78  
79    /**
80     * The first four bytes of Hadoop RPC connections
81     */
82    public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
83  
84    // 1 : Introduce ping and server does not throw away RPCs
85    // 3 : RPC was refactored in 0.19
86    public static final byte CURRENT_VERSION = 3;
87  
88    /**
89     * How many calls/handler are allowed in the queue.
90     */
91    private static final int MAX_QUEUE_SIZE_PER_HANDLER = 100;
92  
93    public static final Log LOG =
94      LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer");
95  
96    protected static final ThreadLocal<HBaseServer> SERVER =
97      new ThreadLocal<HBaseServer>();
98  
99    /** Returns the server instance called under or null.  May be called under
100    * {@link #call(Writable, long)} implementations, and under {@link Writable}
101    * methods of paramters and return values.  Permits applications to access
102    * the server context.
103    * @return HBaseServer
104    */
105   public static HBaseServer get() {
106     return SERVER.get();
107   }
108 
109   /** This is set to Call object before Handler invokes an RPC and reset
110    * after the call returns.
111    */
112   protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
113 
114   /** Returns the remote side ip address when invoked inside an RPC
115    *  Returns null incase of an error.
116    *  @return InetAddress
117    */
118   public static InetAddress getRemoteIp() {
119     Call call = CurCall.get();
120     if (call != null) {
121       return call.connection.socket.getInetAddress();
122     }
123     return null;
124   }
125   /** Returns remote address as a string when invoked inside an RPC.
126    *  Returns null in case of an error.
127    *  @return String
128    */
129   public static String getRemoteAddress() {
130     InetAddress addr = getRemoteIp();
131     return (addr == null) ? null : addr.getHostAddress();
132   }
133 
134   protected String bindAddress;
135   protected int port;                             // port we listen on
136   private int handlerCount;                       // number of handler threads
137   private int priorityHandlerCount;
138   private int readThreads;                        // number of read threads
139   protected Class<? extends Writable> paramClass; // class of call parameters
140   protected int maxIdleTime;                      // the maximum idle time after
141                                                   // which a client may be
142                                                   // disconnected
143   protected int thresholdIdleConnections;         // the number of idle
144                                                   // connections after which we
145                                                   // will start cleaning up idle
146                                                   // connections
147   int maxConnectionsToNuke;                       // the max number of
148                                                   // connections to nuke
149                                                   // during a cleanup
150 
151   protected HBaseRpcMetrics  rpcMetrics;
152 
153   protected Configuration conf;
154 
155   @SuppressWarnings({"FieldCanBeLocal"})
156   private int maxQueueSize;
157   protected int socketSendBufferSize;
158   protected final boolean tcpNoDelay;   // if T then disable Nagle's Algorithm
159   protected final boolean tcpKeepAlive; // if T then use keepalives
160 
161   volatile protected boolean running = true;         // true while server runs
162   protected BlockingQueue<Call> callQueue; // queued calls
163   protected BlockingQueue<Call> priorityCallQueue;
164 
165   private int highPriorityLevel;  // what level a high priority call is at
166 
167   protected final List<Connection> connectionList =
168     Collections.synchronizedList(new LinkedList<Connection>());
169   //maintain a list
170   //of client connections
171   private Listener listener = null;
172   protected Responder responder = null;
173   protected int numConnections = 0;
174   private Handler[] handlers = null;
175   private Handler[] priorityHandlers = null;
176   protected HBaseRPCErrorHandler errorHandler = null;
177 
178   /**
179    * A convenience method to bind to a given address and report
180    * better exceptions if the address is not a valid host.
181    * @param socket the socket to bind
182    * @param address the address to bind to
183    * @param backlog the number of connections allowed in the queue
184    * @throws BindException if the address can't be bound
185    * @throws UnknownHostException if the address isn't a valid host name
186    * @throws IOException other random errors from bind
187    */
188   public static void bind(ServerSocket socket, InetSocketAddress address,
189                           int backlog) throws IOException {
190     try {
191       socket.bind(address, backlog);
192     } catch (BindException e) {
193       BindException bindException =
194         new BindException("Problem binding to " + address + " : " +
195             e.getMessage());
196       bindException.initCause(e);
197       throw bindException;
198     } catch (SocketException e) {
199       // If they try to bind to a different host's address, give a better
200       // error message.
201       if ("Unresolved address".equals(e.getMessage())) {
202         throw new UnknownHostException("Invalid hostname for server: " +
203                                        address.getHostName());
204       }
205       throw e;
206     }
207   }
208 
209   /** A call queued for handling. */
210   private static class Call {
211     protected int id;                             // the client's call id
212     protected Writable param;                     // the parameter passed
213     protected Connection connection;              // connection to client
214     protected long timestamp;      // the time received when response is null
215                                    // the time served when response is not null
216     protected ByteBuffer response;                // the response for this call
217 
218     public Call(int id, Writable param, Connection connection) {
219       this.id = id;
220       this.param = param;
221       this.connection = connection;
222       this.timestamp = System.currentTimeMillis();
223       this.response = null;
224     }
225 
226     @Override
227     public String toString() {
228       return param.toString() + " from " + connection.toString();
229     }
230 
231     public void setResponse(ByteBuffer response) {
232       this.response = response;
233     }
234   }
235 
236   /** Listens on the socket. Creates jobs for the handler threads*/
237   private class Listener extends Thread {
238 
239     private ServerSocketChannel acceptChannel = null; //the accept channel
240     private Selector selector = null; //the selector that we use for the server
241     private Reader[] readers = null;
242     private int currentReader = 0;
243     private InetSocketAddress address; //the address we bind at
244     private Random rand = new Random();
245     private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
246                                          //-tion (for idle connections) ran
247     private long cleanupInterval = 10000; //the minimum interval between
248                                           //two cleanup runs
249     private int backlogLength = conf.getInt("ipc.server.listen.queue.size", 128);
250 
251     private ExecutorService readPool;
252 
253     public Listener() throws IOException {
254       address = new InetSocketAddress(bindAddress, port);
255       // Create a new server socket and set to non blocking mode
256       acceptChannel = ServerSocketChannel.open();
257       acceptChannel.configureBlocking(false);
258 
259       // Bind the server socket to the local host and port
260       bind(acceptChannel.socket(), address, backlogLength);
261       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
262       // create a selector;
263       selector= Selector.open();
264 
265       readers = new Reader[readThreads];
266       readPool = Executors.newFixedThreadPool(readThreads);
267       for (int i = 0; i < readThreads; ++i) {
268         Selector readSelector = Selector.open();
269         Reader reader = new Reader(readSelector);
270         readers[i] = reader;
271         readPool.execute(reader);
272       }
273 
274       // Register accepts on the server socket with the selector.
275       acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
276       this.setName("IPC Server listener on " + port);
277       this.setDaemon(true);
278     }
279 
280 
281     private class Reader implements Runnable {
282       private volatile boolean adding = false;
283       private Selector readSelector = null;
284 
285       Reader(Selector readSelector) {
286         this.readSelector = readSelector;
287       }
288       public void run() {
289         LOG.info("Starting SocketReader");
290         synchronized(this) {
291           while (running) {
292             SelectionKey key = null;
293             try {
294               readSelector.select();
295               while (adding) {
296                 this.wait(1000);
297               }
298 
299               Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
300               while (iter.hasNext()) {
301                 key = iter.next();
302                 iter.remove();
303                 if (key.isValid()) {
304                   if (key.isReadable()) {
305                     doRead(key);
306                   }
307                 }
308                 key = null;
309               }
310             } catch (InterruptedException e) {
311               if (running) {                     // unexpected -- log it
312                 LOG.info(getName() + "caught: " +
313                     StringUtils.stringifyException(e));
314               }
315             } catch (IOException ex) {
316                LOG.error("Error in Reader", ex);
317             }
318           }
319         }
320       }
321 
322       /**
323        * This gets reader into the state that waits for the new channel
324        * to be registered with readSelector. If it was waiting in select()
325        * the thread will be woken up, otherwise whenever select() is called
326        * it will return even if there is nothing to read and wait
327        * in while(adding) for finishAdd call
328        */
329       public void startAdd() {
330         adding = true;
331         readSelector.wakeup();
332       }
333 
334       public synchronized SelectionKey registerChannel(SocketChannel channel)
335         throws IOException {
336         return channel.register(readSelector, SelectionKey.OP_READ);
337       }
338 
339       public synchronized void finishAdd() {
340         adding = false;
341         this.notify();
342       }
343     }
344 
345     /** cleanup connections from connectionList. Choose a random range
346      * to scan and also have a limit on the number of the connections
347      * that will be cleanedup per run. The criteria for cleanup is the time
348      * for which the connection was idle. If 'force' is true then all
349      * connections will be looked at for the cleanup.
350      * @param force all connections will be looked at for cleanup
351      */
352     private void cleanupConnections(boolean force) {
353       if (force || numConnections > thresholdIdleConnections) {
354         long currentTime = System.currentTimeMillis();
355         if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
356           return;
357         }
358         int start = 0;
359         int end = numConnections - 1;
360         if (!force) {
361           start = rand.nextInt() % numConnections;
362           end = rand.nextInt() % numConnections;
363           int temp;
364           if (end < start) {
365             temp = start;
366             start = end;
367             end = temp;
368           }
369         }
370         int i = start;
371         int numNuked = 0;
372         while (i <= end) {
373           Connection c;
374           synchronized (connectionList) {
375             try {
376               c = connectionList.get(i);
377             } catch (Exception e) {return;}
378           }
379           if (c.timedOut(currentTime)) {
380             if (LOG.isDebugEnabled())
381               LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
382             closeConnection(c);
383             numNuked++;
384             end--;
385             //noinspection UnusedAssignment
386             c = null;
387             if (!force && numNuked == maxConnectionsToNuke) break;
388           }
389           else i++;
390         }
391         lastCleanupRunTime = System.currentTimeMillis();
392       }
393     }
394 
395     @Override
396     public void run() {
397       LOG.info(getName() + ": starting");
398       SERVER.set(HBaseServer.this);
399 
400       while (running) {
401         SelectionKey key = null;
402         try {
403           selector.select(); // FindBugs IS2_INCONSISTENT_SYNC
404           Iterator<SelectionKey> iter = selector.selectedKeys().iterator();
405           while (iter.hasNext()) {
406             key = iter.next();
407             iter.remove();
408             try {
409               if (key.isValid()) {
410                 if (key.isAcceptable())
411                   doAccept(key);
412               }
413             } catch (IOException ignored) {
414             }
415             key = null;
416           }
417         } catch (OutOfMemoryError e) {
418           if (errorHandler != null) {
419             if (errorHandler.checkOOME(e)) {
420               LOG.info(getName() + ": exiting on OOME");
421               closeCurrentConnection(key);
422               cleanupConnections(true);
423               return;
424             }
425           } else {
426             // we can run out of memory if we have too many threads
427             // log the event and sleep for a minute and give
428             // some thread(s) a chance to finish
429             LOG.warn("Out of Memory in server select", e);
430             closeCurrentConnection(key);
431             cleanupConnections(true);
432             try { Thread.sleep(60000); } catch (Exception ignored) {}
433       }
434         } catch (Exception e) {
435           closeCurrentConnection(key);
436         }
437         cleanupConnections(false);
438       }
439       LOG.info("Stopping " + this.getName());
440 
441       synchronized (this) {
442         try {
443           acceptChannel.close();
444           selector.close();
445         } catch (IOException ignored) { }
446 
447         selector= null;
448         acceptChannel= null;
449 
450         // clean up all connections
451         while (!connectionList.isEmpty()) {
452           closeConnection(connectionList.remove(0));
453         }
454       }
455     }
456 
457     private void closeCurrentConnection(SelectionKey key) {
458       if (key != null) {
459         Connection c = (Connection)key.attachment();
460         if (c != null) {
461           if (LOG.isDebugEnabled())
462             LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
463           closeConnection(c);
464         }
465       }
466     }
467 
468     InetSocketAddress getAddress() {
469       return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
470     }
471 
472     void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
473       Connection c;
474       ServerSocketChannel server = (ServerSocketChannel) key.channel();
475 
476       SocketChannel channel;
477       while ((channel = server.accept()) != null) {
478         channel.configureBlocking(false);
479         channel.socket().setTcpNoDelay(tcpNoDelay);
480         channel.socket().setKeepAlive(tcpKeepAlive);
481 
482         Reader reader = getReader();
483         try {
484           reader.startAdd();
485           SelectionKey readKey = reader.registerChannel(channel);
486         c = new Connection(channel, System.currentTimeMillis());
487         readKey.attach(c);
488         synchronized (connectionList) {
489           connectionList.add(numConnections, c);
490           numConnections++;
491         }
492         if (LOG.isDebugEnabled())
493           LOG.debug("Server connection from " + c.toString() +
494               "; # active connections: " + numConnections +
495               "; # queued calls: " + callQueue.size());
496         } finally {
497           reader.finishAdd();
498         }
499       }
500     }
501 
502     void doRead(SelectionKey key) throws InterruptedException {
503       int count = 0;
504       Connection c = (Connection)key.attachment();
505       if (c == null) {
506         return;
507       }
508       c.setLastContact(System.currentTimeMillis());
509 
510       try {
511         count = c.readAndProcess();
512       } catch (InterruptedException ieo) {
513         throw ieo;
514       } catch (Exception e) {
515         LOG.debug(getName() + ": readAndProcess threw exception " + e + ". Count of bytes read: " + count, e);
516         count = -1; //so that the (count < 0) block is executed
517       }
518       if (count < 0) {
519         if (LOG.isDebugEnabled())
520           LOG.debug(getName() + ": disconnecting client " +
521                     c.getHostAddress() + ". Number of active connections: "+
522                     numConnections);
523         closeConnection(c);
524         // c = null;
525       }
526       else {
527         c.setLastContact(System.currentTimeMillis());
528       }
529     }
530 
531     synchronized void doStop() {
532       if (selector != null) {
533         selector.wakeup();
534         Thread.yield();
535       }
536       if (acceptChannel != null) {
537         try {
538           acceptChannel.socket().close();
539         } catch (IOException e) {
540           LOG.info(getName() + ":Exception in closing listener socket. " + e);
541         }
542       }
543       readPool.shutdownNow();
544     }
545 
546     // The method that will return the next reader to work with
547     // Simplistic implementation of round robin for now
548     Reader getReader() {
549       currentReader = (currentReader + 1) % readers.length;
550       return readers[currentReader];
551     }
552   }
553 
554   // Sends responses of RPC back to clients.
555   private class Responder extends Thread {
556     private Selector writeSelector;
557     private int pending;         // connections waiting to register
558 
559     final static int PURGE_INTERVAL = 900000; // 15mins
560 
561     Responder() throws IOException {
562       this.setName("IPC Server Responder");
563       this.setDaemon(true);
564       writeSelector = Selector.open(); // create a selector
565       pending = 0;
566     }
567 
568     @Override
569     public void run() {
570       LOG.info(getName() + ": starting");
571       SERVER.set(HBaseServer.this);
572       long lastPurgeTime = 0;   // last check for old calls.
573 
574       while (running) {
575         try {
576           waitPending();     // If a channel is being registered, wait.
577           writeSelector.select(PURGE_INTERVAL);
578           Iterator<SelectionKey> iter = writeSelector.selectedKeys().iterator();
579           while (iter.hasNext()) {
580             SelectionKey key = iter.next();
581             iter.remove();
582             try {
583               if (key.isValid() && key.isWritable()) {
584                   doAsyncWrite(key);
585               }
586             } catch (IOException e) {
587               LOG.info(getName() + ": doAsyncWrite threw exception " + e);
588             }
589           }
590           long now = System.currentTimeMillis();
591           if (now < lastPurgeTime + PURGE_INTERVAL) {
592             continue;
593           }
594           lastPurgeTime = now;
595           //
596           // If there were some calls that have not been sent out for a
597           // long time, discard them.
598           //
599           LOG.debug("Checking for old call responses.");
600           ArrayList<Call> calls;
601 
602           // get the list of channels from list of keys.
603           synchronized (writeSelector.keys()) {
604             calls = new ArrayList<Call>(writeSelector.keys().size());
605             iter = writeSelector.keys().iterator();
606             while (iter.hasNext()) {
607               SelectionKey key = iter.next();
608               Call call = (Call)key.attachment();
609               if (call != null && key.channel() == call.connection.channel) {
610                 calls.add(call);
611               }
612             }
613           }
614 
615           for(Call call : calls) {
616             doPurge(call, now);
617           }
618         } catch (OutOfMemoryError e) {
619           if (errorHandler != null) {
620             if (errorHandler.checkOOME(e)) {
621               LOG.info(getName() + ": exiting on OOME");
622               return;
623             }
624           } else {
625             //
626             // we can run out of memory if we have too many threads
627             // log the event and sleep for a minute and give
628             // some thread(s) a chance to finish
629             //
630             LOG.warn("Out of Memory in server select", e);
631             try { Thread.sleep(60000); } catch (Exception ignored) {}
632       }
633         } catch (Exception e) {
634           LOG.warn("Exception in Responder " +
635                    StringUtils.stringifyException(e));
636         }
637       }
638       LOG.info("Stopping " + this.getName());
639     }
640 
641     private void doAsyncWrite(SelectionKey key) throws IOException {
642       Call call = (Call)key.attachment();
643       if (call == null) {
644         return;
645       }
646       if (key.channel() != call.connection.channel) {
647         throw new IOException("doAsyncWrite: bad channel");
648       }
649 
650       synchronized(call.connection.responseQueue) {
651         if (processResponse(call.connection.responseQueue, false)) {
652           try {
653             key.interestOps(0);
654           } catch (CancelledKeyException e) {
655             /* The Listener/reader might have closed the socket.
656              * We don't explicitly cancel the key, so not sure if this will
657              * ever fire.
658              * This warning could be removed.
659              */
660             LOG.warn("Exception while changing ops : " + e);
661           }
662         }
663       }
664     }
665 
666     //
667     // Remove calls that have been pending in the responseQueue
668     // for a long time.
669     //
670     private void doPurge(Call call, long now) {
671       synchronized (call.connection.responseQueue) {
672         Iterator<Call> iter = call.connection.responseQueue.listIterator(0);
673         while (iter.hasNext()) {
674           Call nextCall = iter.next();
675           if (now > nextCall.timestamp + PURGE_INTERVAL) {
676             closeConnection(nextCall.connection);
677             break;
678           }
679         }
680       }
681     }
682 
683     // Processes one response. Returns true if there are no more pending
684     // data for this channel.
685     //
686     @SuppressWarnings({"ConstantConditions"})
687     private boolean processResponse(final LinkedList<Call> responseQueue,
688                                     boolean inHandler) throws IOException {
689       boolean error = true;
690       boolean done = false;       // there is more data for this channel.
691       int numElements;
692       Call call = null;
693       try {
694         //noinspection SynchronizationOnLocalVariableOrMethodParameter
695         synchronized (responseQueue) {
696           //
697           // If there are no items for this channel, then we are done
698           //
699           numElements = responseQueue.size();
700           if (numElements == 0) {
701             error = false;
702             return true;              // no more data for this channel.
703           }
704           //
705           // Extract the first call
706           //
707           call = responseQueue.removeFirst();
708           SocketChannel channel = call.connection.channel;
709           if (LOG.isDebugEnabled()) {
710             LOG.debug(getName() + ": responding to #" + call.id + " from " +
711                       call.connection);
712           }
713           //
714           // Send as much data as we can in the non-blocking fashion
715           //
716           int numBytes = channelWrite(channel, call.response);
717           if (numBytes < 0) {
718             return true;
719           }
720           if (!call.response.hasRemaining()) {
721             call.connection.decRpcCount();
722             //noinspection RedundantIfStatement
723             if (numElements == 1) {    // last call fully processes.
724               done = true;             // no more data for this channel.
725             } else {
726               done = false;            // more calls pending to be sent.
727             }
728             if (LOG.isDebugEnabled()) {
729               LOG.debug(getName() + ": responding to #" + call.id + " from " +
730                         call.connection + " Wrote " + numBytes + " bytes.");
731             }
732           } else {
733             //
734             // If we were unable to write the entire response out, then
735             // insert in Selector queue.
736             //
737             call.connection.responseQueue.addFirst(call);
738 
739             if (inHandler) {
740               // set the serve time when the response has to be sent later
741               call.timestamp = System.currentTimeMillis();
742 
743               incPending();
744               try {
745                 // Wakeup the thread blocked on select, only then can the call
746                 // to channel.register() complete.
747                 writeSelector.wakeup();
748                 channel.register(writeSelector, SelectionKey.OP_WRITE, call);
749               } catch (ClosedChannelException e) {
750                 //Its ok. channel might be closed else where.
751                 done = true;
752               } finally {
753                 decPending();
754               }
755             }
756             if (LOG.isDebugEnabled()) {
757               LOG.debug(getName() + ": responding to #" + call.id + " from " +
758                         call.connection + " Wrote partial " + numBytes +
759                         " bytes.");
760             }
761           }
762           error = false;              // everything went off well
763         }
764       } finally {
765         if (error && call != null) {
766           LOG.warn(getName()+", call " + call + ": output error");
767           done = true;               // error. no more data for this channel.
768           closeConnection(call.connection);
769         }
770       }
771       return done;
772     }
773 
774     //
775     // Enqueue a response from the application.
776     //
777     void doRespond(Call call) throws IOException {
778       synchronized (call.connection.responseQueue) {
779         call.connection.responseQueue.addLast(call);
780         if (call.connection.responseQueue.size() == 1) {
781           processResponse(call.connection.responseQueue, true);
782         }
783       }
784     }
785 
786     private synchronized void incPending() {   // call waiting to be enqueued.
787       pending++;
788     }
789 
790     private synchronized void decPending() { // call done enqueueing.
791       pending--;
792       notify();
793     }
794 
795     private synchronized void waitPending() throws InterruptedException {
796       while (pending > 0) {
797         wait();
798       }
799     }
800   }
801 
802   /** Reads calls from a connection and queues them for handling. */
803   private class Connection {
804     private boolean versionRead = false; //if initial signature and
805                                          //version are read
806     private boolean headerRead = false;  //if the connection header that
807                                          //follows version is read.
808     protected SocketChannel channel;
809     private ByteBuffer data;
810     private ByteBuffer dataLengthBuffer;
811     protected final LinkedList<Call> responseQueue;
812     private volatile int rpcCount = 0; // number of outstanding rpcs
813     private long lastContact;
814     private int dataLength;
815     protected Socket socket;
816     // Cache the remote host & port info so that even if the socket is
817     // disconnected, we can say where it used to connect to.
818     private String hostAddress;
819     private int remotePort;
820     protected UserGroupInformation ticket = null;
821 
822     public Connection(SocketChannel channel, long lastContact) {
823       this.channel = channel;
824       this.lastContact = lastContact;
825       this.data = null;
826       this.dataLengthBuffer = ByteBuffer.allocate(4);
827       this.socket = channel.socket();
828       InetAddress addr = socket.getInetAddress();
829       if (addr == null) {
830         this.hostAddress = "*Unknown*";
831       } else {
832         this.hostAddress = addr.getHostAddress();
833       }
834       this.remotePort = socket.getPort();
835       this.responseQueue = new LinkedList<Call>();
836       if (socketSendBufferSize != 0) {
837         try {
838           socket.setSendBufferSize(socketSendBufferSize);
839         } catch (IOException e) {
840           LOG.warn("Connection: unable to set socket send buffer size to " +
841                    socketSendBufferSize);
842         }
843       }
844     }
845 
846     @Override
847     public String toString() {
848       return getHostAddress() + ":" + remotePort;
849     }
850 
851     public String getHostAddress() {
852       return hostAddress;
853     }
854 
855     public void setLastContact(long lastContact) {
856       this.lastContact = lastContact;
857     }
858 
859     public long getLastContact() {
860       return lastContact;
861     }
862 
863     /* Return true if the connection has no outstanding rpc */
864     private boolean isIdle() {
865       return rpcCount == 0;
866     }
867 
868     /* Decrement the outstanding RPC count */
869     protected void decRpcCount() {
870       rpcCount--;
871     }
872 
873     /* Increment the outstanding RPC count */
874     private void incRpcCount() {
875       rpcCount++;
876     }
877 
878     protected boolean timedOut(long currentTime) {
879       return isIdle() && currentTime - lastContact > maxIdleTime;
880     }
881 
882     public int readAndProcess() throws IOException, InterruptedException {
883       while (true) {
884         /* Read at most one RPC. If the header is not read completely yet
885          * then iterate until we read first RPC or until there is no data left.
886          */
887         int count;
888         if (dataLengthBuffer.remaining() > 0) {
889           count = channelRead(channel, dataLengthBuffer);
890           if (count < 0 || dataLengthBuffer.remaining() > 0)
891             return count;
892         }
893 
894         if (!versionRead) {
895           //Every connection is expected to send the header.
896           ByteBuffer versionBuffer = ByteBuffer.allocate(1);
897           count = channelRead(channel, versionBuffer);
898           if (count <= 0) {
899             return count;
900           }
901           int version = versionBuffer.get(0);
902 
903           dataLengthBuffer.flip();
904           if (!HEADER.equals(dataLengthBuffer) || version != CURRENT_VERSION) {
905             //Warning is ok since this is not supposed to happen.
906             LOG.warn("Incorrect header or version mismatch from " +
907                      hostAddress + ":" + remotePort +
908                      " got version " + version +
909                      " expected version " + CURRENT_VERSION);
910             return -1;
911           }
912           dataLengthBuffer.clear();
913           versionRead = true;
914           continue;
915         }
916 
917         if (data == null) {
918           dataLengthBuffer.flip();
919           dataLength = dataLengthBuffer.getInt();
920 
921           if (dataLength == HBaseClient.PING_CALL_ID) {
922             dataLengthBuffer.clear();
923             return 0;  //ping message
924           }
925           data = ByteBuffer.allocate(dataLength);
926           incRpcCount();  // Increment the rpc count
927         }
928 
929         count = channelRead(channel, data);
930 
931         if (data.remaining() == 0) {
932           dataLengthBuffer.clear();
933           data.flip();
934           if (headerRead) {
935             processData();
936             data = null;
937             return count;
938           }
939           processHeader();
940           headerRead = true;
941           data = null;
942           continue;
943         }
944         return count;
945       }
946     }
947 
948     /// Reads the header following version
949     private void processHeader() throws IOException {
950       /* In the current version, it is just a ticket.
951        * Later we could introduce a "ConnectionHeader" class.
952        */
953       DataInputStream in =
954         new DataInputStream(new ByteArrayInputStream(data.array()));
955       String username = WritableUtils.readString(in);
956       ticket = UserGroupInformation.createRemoteUser(username);
957     }
958 
959     private void processData() throws  IOException, InterruptedException {
960       DataInputStream dis =
961         new DataInputStream(new ByteArrayInputStream(data.array()));
962       int id = dis.readInt();                    // try to read an id
963 
964       if (LOG.isDebugEnabled())
965         LOG.debug(" got #" + id);
966 
967       Writable param = ReflectionUtils.newInstance(paramClass, conf);           // read param
968       param.readFields(dis);
969 
970       Call call = new Call(id, param, this);
971 
972       if (priorityCallQueue != null && getQosLevel(param) > highPriorityLevel) {
973         priorityCallQueue.put(call);
974       } else {
975         callQueue.put(call);              // queue the call; maybe blocked here
976       }
977     }
978 
979     protected synchronized void close() {
980       data = null;
981       dataLengthBuffer = null;
982       if (!channel.isOpen())
983         return;
984       try {socket.shutdownOutput();} catch(Exception ignored) {} // FindBugs DE_MIGHT_IGNORE
985       if (channel.isOpen()) {
986         try {channel.close();} catch(Exception ignored) {}
987       }
988       try {socket.close();} catch(Exception ignored) {}
989     }
990   }
991 
992   /** Handles queued calls . */
993   private class Handler extends Thread {
994     private final BlockingQueue<Call> myCallQueue;
995     public Handler(final BlockingQueue<Call> cq, int instanceNumber) {
996       this.myCallQueue = cq;
997       this.setDaemon(true);
998 
999       String threadName = "IPC Server handler " + instanceNumber + " on " + port;
1000       if (cq == priorityCallQueue) {
1001         // this is just an amazing hack, but it works.
1002         threadName = "PRI " + threadName;
1003       }
1004       this.setName(threadName);
1005     }
1006 
1007     @Override
1008     public void run() {
1009       LOG.info(getName() + ": starting");
1010       SERVER.set(HBaseServer.this);
1011       final int buffersize = 16 * 1024;
1012       ByteArrayOutputStream buf = new ByteArrayOutputStream(buffersize);
1013       while (running) {
1014         try {
1015           final Call call = myCallQueue.take(); // pop the queue; maybe blocked here
1016 
1017           if (LOG.isDebugEnabled())
1018             LOG.debug(getName() + ": has #" + call.id + " from " +
1019                       call.connection);
1020 
1021           String errorClass = null;
1022           String error = null;
1023           Writable value = null;
1024 
1025           CurCall.set(call);
1026           try {
1027 /*
1028 TODO: Currently we do not assume the context of the user who is
1029 requesting, since security hasn't been fully integrated in HBase.
1030 
1031             value = call.connection.ticket.doAs(
1032               new PrivilegedExceptionAction<Writable>() {
1033                 @Override
1034                 public Writable run() throws Exception {
1035                   return call(call.param, call.timestamp);
1036                 }
1037               });
1038 */
1039             value = call(call.param, call.timestamp);
1040           } catch (Throwable e) {
1041             LOG.debug(getName()+", call "+call+": error: " + e, e);
1042             errorClass = e.getClass().getName();
1043             error = StringUtils.stringifyException(e);
1044           }
1045           CurCall.set(null);
1046 
1047           if (buf.size() > buffersize) {
1048             // Allocate a new BAOS as reset only moves size back to zero but
1049             // keeps the buffer of whatever the largest write was -- see
1050             // hbase-900.
1051             buf = new ByteArrayOutputStream(buffersize);
1052           } else {
1053             buf.reset();
1054           }
1055           DataOutputStream out = new DataOutputStream(buf);
1056           out.writeInt(call.id);                // write call id
1057           out.writeBoolean(error != null);      // write error flag
1058 
1059           if (error == null) {
1060             value.write(out);
1061           } else {
1062             WritableUtils.writeString(out, errorClass);
1063             WritableUtils.writeString(out, error);
1064           }
1065           call.setResponse(ByteBuffer.wrap(buf.toByteArray()));
1066           responder.doRespond(call);
1067         } catch (InterruptedException e) {
1068           if (running) {                          // unexpected -- log it
1069             LOG.info(getName() + " caught: " +
1070                      StringUtils.stringifyException(e));
1071           }
1072         } catch (OutOfMemoryError e) {
1073           if (errorHandler != null) {
1074             if (errorHandler.checkOOME(e)) {
1075               LOG.info(getName() + ": exiting on OOME");
1076               return;
1077             }
1078           } else {
1079             // rethrow if no handler
1080             throw e;
1081           }
1082         } catch (Exception e) {
1083           LOG.info(getName() + " caught: " +
1084                    StringUtils.stringifyException(e));
1085         }
1086       }
1087       LOG.info(getName() + ": exiting");
1088     }
1089 
1090   }
1091 
1092   /**
1093    * Gets the QOS level for this call.  If it is higher than the highPriorityLevel and there
1094    * are priorityHandlers available it will be processed in it's own thread set.
1095    *
1096    * @param param
1097    * @return priority, higher is better
1098    */
1099   private Function<Writable,Integer> qosFunction = null;
1100   public void setQosFunction(Function<Writable, Integer> newFunc) {
1101     qosFunction = newFunc;
1102   }
1103 
1104   protected int getQosLevel(Writable param) {
1105     if (qosFunction == null) {
1106       return 0;
1107     }
1108 
1109     Integer res = qosFunction.apply(param);
1110     if (res == null) {
1111       return 0;
1112     }
1113     return res;
1114   }
1115 
1116   /* Constructs a server listening on the named port and address.  Parameters passed must
1117    * be of the named class.  The <code>handlerCount</handlerCount> determines
1118    * the number of handler threads that will be used to process calls.
1119    *
1120    */
1121   protected HBaseServer(String bindAddress, int port,
1122                         Class<? extends Writable> paramClass, int handlerCount,
1123                         int priorityHandlerCount, Configuration conf, String serverName,
1124                         int highPriorityLevel)
1125     throws IOException {
1126     this.bindAddress = bindAddress;
1127     this.conf = conf;
1128     this.port = port;
1129     this.paramClass = paramClass;
1130     this.handlerCount = handlerCount;
1131     this.priorityHandlerCount = priorityHandlerCount;
1132     this.socketSendBufferSize = 0;
1133     this.maxQueueSize = handlerCount * MAX_QUEUE_SIZE_PER_HANDLER;
1134      this.readThreads = conf.getInt(
1135         "ipc.server.read.threadpool.size",
1136         10);
1137     this.callQueue  = new LinkedBlockingQueue<Call>(maxQueueSize);
1138     if (priorityHandlerCount > 0) {
1139       this.priorityCallQueue = new LinkedBlockingQueue<Call>(maxQueueSize); // TODO hack on size
1140     } else {
1141       this.priorityCallQueue = null;
1142     }
1143     this.highPriorityLevel = highPriorityLevel;
1144     this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
1145     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
1146     this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
1147 
1148     // Start the listener here and let it bind to the port
1149     listener = new Listener();
1150     this.port = listener.getAddress().getPort();
1151     this.rpcMetrics = new HBaseRpcMetrics(serverName,
1152                           Integer.toString(this.port));
1153     this.tcpNoDelay = conf.getBoolean("ipc.server.tcpnodelay", false);
1154     this.tcpKeepAlive = conf.getBoolean("ipc.server.tcpkeepalive", true);
1155 
1156     // Create the responder here
1157     responder = new Responder();
1158   }
1159 
1160   protected void closeConnection(Connection connection) {
1161     synchronized (connectionList) {
1162       if (connectionList.remove(connection))
1163         numConnections--;
1164     }
1165     connection.close();
1166   }
1167 
1168   /** Sets the socket buffer size used for responding to RPCs.
1169    * @param size send size
1170    */
1171   public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
1172 
1173   /** Starts the service.  Must be called before any calls will be handled. */
1174   public synchronized void start() {
1175     responder.start();
1176     listener.start();
1177     handlers = new Handler[handlerCount];
1178 
1179     for (int i = 0; i < handlerCount; i++) {
1180       handlers[i] = new Handler(callQueue, i);
1181       handlers[i].start();
1182     }
1183 
1184     if (priorityHandlerCount > 0) {
1185       priorityHandlers = new Handler[priorityHandlerCount];
1186       for (int i = 0 ; i < priorityHandlerCount; i++) {
1187         priorityHandlers[i] = new Handler(priorityCallQueue, i);
1188         priorityHandlers[i].start();
1189       }
1190     }
1191   }
1192 
1193   /** Stops the service.  No new calls will be handled after this is called. */
1194   public synchronized void stop() {
1195     LOG.info("Stopping server on " + port);
1196     running = false;
1197     if (handlers != null) {
1198       for (Handler handler : handlers) {
1199         if (handler != null) {
1200           handler.interrupt();
1201         }
1202       }
1203     }
1204     if (priorityHandlers != null) {
1205       for (Handler handler : priorityHandlers) {
1206         if (handler != null) {
1207           handler.interrupt();
1208         }
1209       }
1210     }
1211     listener.interrupt();
1212     listener.doStop();
1213     responder.interrupt();
1214     notifyAll();
1215     if (this.rpcMetrics != null) {
1216       this.rpcMetrics.shutdown();
1217     }
1218   }
1219 
1220   /** Wait for the server to be stopped.
1221    * Does not wait for all subthreads to finish.
1222    *  See {@link #stop()}.
1223    * @throws InterruptedException e
1224    */
1225   public synchronized void join() throws InterruptedException {
1226     while (running) {
1227       wait();
1228     }
1229   }
1230 
1231   /**
1232    * Return the socket (ip+port) on which the RPC server is listening to.
1233    * @return the socket (ip+port) on which the RPC server is listening to.
1234    */
1235   public synchronized InetSocketAddress getListenerAddress() {
1236     return listener.getAddress();
1237   }
1238 
1239   /** Called for each call.
1240    * @param param writable parameter
1241    * @param receiveTime time
1242    * @return Writable
1243    * @throws IOException e
1244    */
1245   public abstract Writable call(Writable param, long receiveTime)
1246                                                 throws IOException;
1247 
1248   /**
1249    * The number of open RPC conections
1250    * @return the number of open rpc connections
1251    */
1252   public int getNumOpenConnections() {
1253     return numConnections;
1254   }
1255 
1256   /**
1257    * The number of rpc calls in the queue.
1258    * @return The number of rpc calls in the queue.
1259    */
1260   public int getCallQueueLen() {
1261     return callQueue.size();
1262   }
1263 
1264   /**
1265    * Set the handler for calling out of RPC for error conditions.
1266    * @param handler the handler implementation
1267    */
1268   public void setErrorHandler(HBaseRPCErrorHandler handler) {
1269     this.errorHandler = handler;
1270   }
1271 
1272   /**
1273    * When the read or write buffer size is larger than this limit, i/o will be
1274    * done in chunks of this size. Most RPC requests and responses would be
1275    * be smaller.
1276    */
1277   private static int NIO_BUFFER_LIMIT = 8*1024; //should not be more than 64KB.
1278 
1279   /**
1280    * This is a wrapper around {@link WritableByteChannel#write(ByteBuffer)}.
1281    * If the amount of data is large, it writes to channel in smaller chunks.
1282    * This is to avoid jdk from creating many direct buffers as the size of
1283    * buffer increases. This also minimizes extra copies in NIO layer
1284    * as a result of multiple write operations required to write a large
1285    * buffer.
1286    *
1287    * @param channel writable byte channel to write to
1288    * @param buffer buffer to write
1289    * @return number of bytes written
1290    * @throws java.io.IOException e
1291    * @see WritableByteChannel#write(ByteBuffer)
1292    */
1293   protected static int channelWrite(WritableByteChannel channel,
1294                                     ByteBuffer buffer) throws IOException {
1295     return (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1296            channel.write(buffer) : channelIO(null, channel, buffer);
1297   }
1298 
1299   /**
1300    * This is a wrapper around {@link ReadableByteChannel#read(ByteBuffer)}.
1301    * If the amount of data is large, it writes to channel in smaller chunks.
1302    * This is to avoid jdk from creating many direct buffers as the size of
1303    * ByteBuffer increases. There should not be any performance degredation.
1304    *
1305    * @param channel writable byte channel to write on
1306    * @param buffer buffer to write
1307    * @return number of bytes written
1308    * @throws java.io.IOException e
1309    * @see ReadableByteChannel#read(ByteBuffer)
1310    */
1311   protected static int channelRead(ReadableByteChannel channel,
1312                                    ByteBuffer buffer) throws IOException {
1313     return (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
1314            channel.read(buffer) : channelIO(channel, null, buffer);
1315   }
1316 
1317   /**
1318    * Helper for {@link #channelRead(ReadableByteChannel, ByteBuffer)}
1319    * and {@link #channelWrite(WritableByteChannel, ByteBuffer)}. Only
1320    * one of readCh or writeCh should be non-null.
1321    *
1322    * @param readCh read channel
1323    * @param writeCh write channel
1324    * @param buf buffer to read or write into/out of
1325    * @return bytes written
1326    * @throws java.io.IOException e
1327    * @see #channelRead(ReadableByteChannel, ByteBuffer)
1328    * @see #channelWrite(WritableByteChannel, ByteBuffer)
1329    */
1330   private static int channelIO(ReadableByteChannel readCh,
1331                                WritableByteChannel writeCh,
1332                                ByteBuffer buf) throws IOException {
1333 
1334     int originalLimit = buf.limit();
1335     int initialRemaining = buf.remaining();
1336     int ret = 0;
1337 
1338     while (buf.remaining() > 0) {
1339       try {
1340         int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
1341         buf.limit(buf.position() + ioSize);
1342 
1343         ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
1344 
1345         if (ret < ioSize) {
1346           break;
1347         }
1348 
1349       } finally {
1350         buf.limit(originalLimit);
1351       }
1352     }
1353 
1354     int nBytes = initialRemaining - buf.remaining();
1355     return (nBytes > 0) ? nBytes : ret;
1356   }
1357 }