View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.ipc;
20  
21  import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
22  
23  import java.io.ByteArrayInputStream;
24  import java.io.ByteArrayOutputStream;
25  import java.io.DataOutputStream;
26  import java.io.IOException;
27  import java.net.BindException;
28  import java.net.InetAddress;
29  import java.net.InetSocketAddress;
30  import java.net.ServerSocket;
31  import java.net.Socket;
32  import java.net.SocketException;
33  import java.net.UnknownHostException;
34  import java.nio.ByteBuffer;
35  import java.nio.channels.CancelledKeyException;
36  import java.nio.channels.Channels;
37  import java.nio.channels.ClosedChannelException;
38  import java.nio.channels.GatheringByteChannel;
39  import java.nio.channels.ReadableByteChannel;
40  import java.nio.channels.SelectionKey;
41  import java.nio.channels.Selector;
42  import java.nio.channels.ServerSocketChannel;
43  import java.nio.channels.SocketChannel;
44  import java.nio.channels.WritableByteChannel;
45  import java.security.PrivilegedExceptionAction;
46  import java.util.ArrayList;
47  import java.util.Arrays;
48  import java.util.Collections;
49  import java.util.HashMap;
50  import java.util.Iterator;
51  import java.util.LinkedList;
52  import java.util.List;
53  import java.util.Map;
54  import java.util.Random;
55  import java.util.Set;
56  import java.util.concurrent.ConcurrentHashMap;
57  import java.util.concurrent.ConcurrentLinkedDeque;
58  import java.util.concurrent.ExecutorService;
59  import java.util.concurrent.Executors;
60  import java.util.concurrent.atomic.AtomicInteger;
61  import java.util.concurrent.locks.Lock;
62  import java.util.concurrent.locks.ReentrantLock;
63  
64  import javax.security.sasl.Sasl;
65  import javax.security.sasl.SaslException;
66  import javax.security.sasl.SaslServer;
67  
68  import org.apache.commons.logging.Log;
69  import org.apache.commons.logging.LogFactory;
70  import org.apache.hadoop.hbase.classification.InterfaceAudience;
71  import org.apache.hadoop.hbase.classification.InterfaceStability;
72  import org.apache.hadoop.conf.Configuration;
73  import org.apache.hadoop.hbase.CellScanner;
74  import org.apache.hadoop.hbase.DoNotRetryIOException;
75  import org.apache.hadoop.hbase.HBaseIOException;
76  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
77  import org.apache.hadoop.hbase.HConstants;
78  import org.apache.hadoop.hbase.HRegionInfo;
79  import org.apache.hadoop.hbase.Server;
80  import org.apache.hadoop.hbase.TableName;
81  import org.apache.hadoop.hbase.client.Operation;
82  import org.apache.hadoop.hbase.codec.Codec;
83  import org.apache.hadoop.hbase.exceptions.RegionMovedException;
84  import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
85  import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
86  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
87  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
88  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
89  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
90  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
91  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
92  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
93  import org.apache.hadoop.hbase.regionserver.HRegionServer;
94  import org.apache.hadoop.hbase.security.AccessDeniedException;
95  import org.apache.hadoop.hbase.security.AuthMethod;
96  import org.apache.hadoop.hbase.security.HBasePolicyProvider;
97  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
98  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHandler;
99  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler;
100 import org.apache.hadoop.hbase.security.SaslStatus;
101 import org.apache.hadoop.hbase.security.SaslUtil;
102 import org.apache.hadoop.hbase.security.UserProvider;
103 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
104 import org.apache.hadoop.hbase.util.Bytes;
105 import org.apache.hadoop.hbase.util.Counter;
106 import org.apache.hadoop.hbase.util.Pair;
107 import org.apache.hadoop.io.BytesWritable;
108 import org.apache.hadoop.io.IntWritable;
109 import org.apache.hadoop.io.Writable;
110 import org.apache.hadoop.io.WritableUtils;
111 import org.apache.hadoop.io.compress.CompressionCodec;
112 import org.apache.hadoop.security.UserGroupInformation;
113 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
114 import org.apache.hadoop.security.authorize.AuthorizationException;
115 import org.apache.hadoop.security.authorize.PolicyProvider;
116 import org.apache.hadoop.security.authorize.ProxyUsers;
117 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
118 import org.apache.hadoop.security.token.SecretManager;
119 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
120 import org.apache.hadoop.security.token.TokenIdentifier;
121 import org.apache.hadoop.util.StringUtils;
122 import org.codehaus.jackson.map.ObjectMapper;
123 import org.apache.htrace.TraceInfo;
124 
125 import com.google.common.util.concurrent.ThreadFactoryBuilder;
126 import com.google.protobuf.BlockingService;
127 import com.google.protobuf.CodedInputStream;
128 import com.google.protobuf.Descriptors.MethodDescriptor;
129 import com.google.protobuf.Message;
130 import com.google.protobuf.Message.Builder;
131 import com.google.protobuf.ServiceException;
132 import com.google.protobuf.TextFormat;
133 
134 /**
135  * An RPC server that hosts protobuf described Services.
136  *
137  * An RpcServer instance has a Listener that hosts the socket.  Listener has fixed number
138  * of Readers in an ExecutorPool, 10 by default.  The Listener does an accept and then
139  * round robin a Reader is chosen to do the read.  The reader is registered on Selector.  Read does
140  * total read off the channel and the parse from which it makes a Call.  The call is wrapped in a
141  * CallRunner and passed to the scheduler to be run.  Reader goes back to see if more to be done
142  * and loops till done.
143  *
144  * <p>Scheduler can be variously implemented but default simple scheduler has handlers to which it
145  * has given the queues into which calls (i.e. CallRunner instances) are inserted.  Handlers run
146  * taking from the queue.  They run the CallRunner#run method on each item gotten from queue
147  * and keep taking while the server is up.
148  *
149  * CallRunner#run executes the call.  When done, asks the included Call to put itself on new
150  * queue for Responder to pull from and return result to client.
151  *
152  * @see RpcClientImpl
153  */
154 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
155 @InterfaceStability.Evolving
156 public class RpcServer implements RpcServerInterface {
157   public static final Log LOG = LogFactory.getLog(RpcServer.class);
158 
159   private final boolean authorize;
160   private boolean isSecurityEnabled;
161 
162   public static final byte CURRENT_VERSION = 0;
163 
164   /**
165    * How many calls/handler are allowed in the queue.
166    */
167   static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
168 
169   /**
170    * The maximum size that we can hold in the RPC queue
171    */
172   private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;
173 
174   private static final String WARN_DELAYED_CALLS = "hbase.ipc.warn.delayedrpc.number";
175 
176   private static final int DEFAULT_WARN_DELAYED_CALLS = 1000;
177 
178   private final int warnDelayedCalls;
179 
180   private AtomicInteger delayedCalls;
181   private final IPCUtil ipcUtil;
182 
183   private static final String AUTH_FAILED_FOR = "Auth failed for ";
184   private static final String AUTH_SUCCESSFUL_FOR = "Auth successful for ";
185   private static final Log AUDITLOG = LogFactory.getLog("SecurityLogger." +
186     Server.class.getName());
187   protected SecretManager<TokenIdentifier> secretManager;
188   protected ServiceAuthorizationManager authManager;
189 
190   /** This is set to Call object before Handler invokes an RPC and ybdie
191    * after the call returns.
192    */
193   protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
194 
195   /** Keeps MonitoredRPCHandler per handler thread. */
196   static final ThreadLocal<MonitoredRPCHandler> MONITORED_RPC
197       = new ThreadLocal<MonitoredRPCHandler>();
198 
199   protected final InetSocketAddress bindAddress;
200   protected int port;                             // port we listen on
201   protected InetSocketAddress address;            // inet address we listen on
202   private int readThreads;                        // number of read threads
203   protected int maxIdleTime;                      // the maximum idle time after
204                                                   // which a client may be
205                                                   // disconnected
206   protected int thresholdIdleConnections;         // the number of idle
207                                                   // connections after which we
208                                                   // will start cleaning up idle
209                                                   // connections
210   int maxConnectionsToNuke;                       // the max number of
211                                                   // connections to nuke
212                                                   // during a cleanup
213 
214   protected MetricsHBaseServer metrics;
215 
216   protected final Configuration conf;
217 
218   private int maxQueueSize;
219   protected int socketSendBufferSize;
220   protected final boolean tcpNoDelay;   // if T then disable Nagle's Algorithm
221   protected final boolean tcpKeepAlive; // if T then use keepalives
222   protected final long purgeTimeout;    // in milliseconds
223 
224   /**
225    * This flag is used to indicate to sub threads when they should go down.  When we call
226    * {@link #start()}, all threads started will consult this flag on whether they should
227    * keep going.  It is set to false when {@link #stop()} is called.
228    */
229   volatile boolean running = true;
230 
231   /**
232    * This flag is set to true after all threads are up and 'running' and the server is then opened
233    * for business by the call to {@link #start()}.
234    */
235   volatile boolean started = false;
236 
237   /**
238    * This is a running count of the size of all outstanding calls by size.
239    */
240   protected final Counter callQueueSize = new Counter();
241 
242   protected final List<Connection> connectionList =
243     Collections.synchronizedList(new LinkedList<Connection>());
244   //maintain a list
245   //of client connections
246   private Listener listener = null;
247   protected Responder responder = null;
248   protected int numConnections = 0;
249 
250   protected HBaseRPCErrorHandler errorHandler = null;
251 
252   private static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";
253   private static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";
254 
255   /** Default value for above params */
256   private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
257   private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
258 
259   private static final ObjectMapper MAPPER = new ObjectMapper();
260 
261   private final int warnResponseTime;
262   private final int warnResponseSize;
263   private final Server server;
264   private final List<BlockingServiceAndInterface> services;
265 
266   private final RpcScheduler scheduler;
267 
268   private UserProvider userProvider;
269 
270   /**
271    * Datastructure that holds all necessary to a method invocation and then afterward, carries
272    * the result.
273    */
274   class Call implements RpcCallContext {
275     protected int id;                             // the client's call id
276     protected BlockingService service;
277     protected MethodDescriptor md;
278     protected RequestHeader header;
279     protected Message param;                      // the parameter passed
280     // Optional cell data passed outside of protobufs.
281     protected CellScanner cellScanner;
282     protected Connection connection;              // connection to client
283     protected long timestamp;      // the time received when response is null
284                                    // the time served when response is not null
285     /**
286      * Chain of buffers to send as response.
287      */
288     protected BufferChain response;
289     protected boolean delayResponse;
290     protected Responder responder;
291     protected boolean delayReturnValue;           // if the return value should be
292                                                   // set at call completion
293     protected long size;                          // size of current call
294     protected boolean isError;
295     protected TraceInfo tinfo;
296 
297     Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
298          Message param, CellScanner cellScanner, Connection connection, Responder responder,
299          long size, TraceInfo tinfo) {
300       this.id = id;
301       this.service = service;
302       this.md = md;
303       this.header = header;
304       this.param = param;
305       this.cellScanner = cellScanner;
306       this.connection = connection;
307       this.timestamp = System.currentTimeMillis();
308       this.response = null;
309       this.delayResponse = false;
310       this.responder = responder;
311       this.isError = false;
312       this.size = size;
313       this.tinfo = tinfo;
314     }
315 
316     @Override
317     public String toString() {
318       return toShortString() + " param: " +
319         (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") +
320         " connection: " + connection.toString();
321     }
322 
323     protected RequestHeader getHeader() {
324       return this.header;
325     }
326 
327     /*
328      * Short string representation without param info because param itself could be huge depends on
329      * the payload of a command
330      */
331     String toShortString() {
332       String serviceName = this.connection.service != null ?
333           this.connection.service.getDescriptorForType().getName() : "null";
334       return "callId: " + this.id + " service: " + serviceName +
335           " methodName: " + ((this.md != null) ? this.md.getName() : "n/a") +
336           " size: " + StringUtils.TraditionalBinaryPrefix.long2String(this.size, "", 1) +
337           " connection: " + connection.toString();
338     }
339 
340     String toTraceString() {
341       String serviceName = this.connection.service != null ?
342                            this.connection.service.getDescriptorForType().getName() : "";
343       String methodName = (this.md != null) ? this.md.getName() : "";
344       return serviceName + "." + methodName;
345     }
346 
347     protected synchronized void setSaslTokenResponse(ByteBuffer response) {
348       this.response = new BufferChain(response);
349     }
350 
351     protected synchronized void setResponse(Object m, final CellScanner cells,
352         Throwable t, String errorMsg) {
353       if (this.isError) return;
354       if (t != null) this.isError = true;
355       BufferChain bc = null;
356       try {
357         ResponseHeader.Builder headerBuilder = ResponseHeader.newBuilder();
358         // Presume it a pb Message.  Could be null.
359         Message result = (Message)m;
360         // Call id.
361         headerBuilder.setCallId(this.id);
362         if (t != null) {
363           ExceptionResponse.Builder exceptionBuilder = ExceptionResponse.newBuilder();
364           exceptionBuilder.setExceptionClassName(t.getClass().getName());
365           exceptionBuilder.setStackTrace(errorMsg);
366           exceptionBuilder.setDoNotRetry(t instanceof DoNotRetryIOException);
367           if (t instanceof RegionMovedException) {
368             // Special casing for this exception.  This is only one carrying a payload.
369             // Do this instead of build a generic system for allowing exceptions carry
370             // any kind of payload.
371             RegionMovedException rme = (RegionMovedException)t;
372             exceptionBuilder.setHostname(rme.getHostname());
373             exceptionBuilder.setPort(rme.getPort());
374           }
375           // Set the exception as the result of the method invocation.
376           headerBuilder.setException(exceptionBuilder.build());
377         }
378         ByteBuffer cellBlock =
379           ipcUtil.buildCellBlock(this.connection.codec, this.connection.compressionCodec, cells);
380         if (cellBlock != null) {
381           CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
382           // Presumes the cellBlock bytebuffer has been flipped so limit has total size in it.
383           cellBlockBuilder.setLength(cellBlock.limit());
384           headerBuilder.setCellBlockMeta(cellBlockBuilder.build());
385         }
386         Message header = headerBuilder.build();
387 
388         // Organize the response as a set of bytebuffers rather than collect it all together inside
389         // one big byte array; save on allocations.
390         ByteBuffer bbHeader = IPCUtil.getDelimitedMessageAsByteBuffer(header);
391         ByteBuffer bbResult = IPCUtil.getDelimitedMessageAsByteBuffer(result);
392         int totalSize = bbHeader.capacity() + (bbResult == null? 0: bbResult.limit()) +
393           (cellBlock == null? 0: cellBlock.limit());
394         ByteBuffer bbTotalSize = ByteBuffer.wrap(Bytes.toBytes(totalSize));
395         bc = new BufferChain(bbTotalSize, bbHeader, bbResult, cellBlock);
396         if (connection.useWrap) {
397           bc = wrapWithSasl(bc);
398         }
399       } catch (IOException e) {
400         LOG.warn("Exception while creating response " + e);
401       }
402       this.response = bc;
403     }
404 
405     private BufferChain wrapWithSasl(BufferChain bc)
406         throws IOException {
407       if (!this.connection.useSasl) return bc;
408       // Looks like no way around this; saslserver wants a byte array.  I have to make it one.
409       // THIS IS A BIG UGLY COPY.
410       byte [] responseBytes = bc.getBytes();
411       byte [] token;
412       // synchronization may be needed since there can be multiple Handler
413       // threads using saslServer to wrap responses.
414       synchronized (connection.saslServer) {
415         token = connection.saslServer.wrap(responseBytes, 0, responseBytes.length);
416       }
417       if (LOG.isTraceEnabled()) {
418         LOG.trace("Adding saslServer wrapped token of size " + token.length
419             + " as call response.");
420       }
421 
422       ByteBuffer bbTokenLength = ByteBuffer.wrap(Bytes.toBytes(token.length));
423       ByteBuffer bbTokenBytes = ByteBuffer.wrap(token);
424       return new BufferChain(bbTokenLength, bbTokenBytes);
425     }
426 
427     @Override
428     public synchronized void endDelay(Object result) throws IOException {
429       assert this.delayResponse;
430       assert this.delayReturnValue || result == null;
431       this.delayResponse = false;
432       delayedCalls.decrementAndGet();
433       if (this.delayReturnValue) {
434         this.setResponse(result, null, null, null);
435       }
436       this.responder.doRespond(this);
437     }
438 
439     @Override
440     public synchronized void endDelay() throws IOException {
441       this.endDelay(null);
442     }
443 
444     @Override
445     public synchronized void startDelay(boolean delayReturnValue) {
446       assert !this.delayResponse;
447       this.delayResponse = true;
448       this.delayReturnValue = delayReturnValue;
449       int numDelayed = delayedCalls.incrementAndGet();
450       if (numDelayed > warnDelayedCalls) {
451         LOG.warn("Too many delayed calls: limit " + warnDelayedCalls + " current " + numDelayed);
452       }
453     }
454 
455     @Override
456     public synchronized void endDelayThrowing(Throwable t) throws IOException {
457       this.setResponse(null, null, t, StringUtils.stringifyException(t));
458       this.delayResponse = false;
459       this.sendResponseIfReady();
460     }
461 
462     @Override
463     public synchronized boolean isDelayed() {
464       return this.delayResponse;
465     }
466 
467     @Override
468     public synchronized boolean isReturnValueDelayed() {
469       return this.delayReturnValue;
470     }
471 
472     @Override
473     public boolean isClientCellBlockSupport() {
474       return this.connection != null && this.connection.codec != null;
475     }
476 
477     @Override
478     public long disconnectSince() {
479       if (!connection.channel.isOpen()) {
480         return System.currentTimeMillis() - timestamp;
481       } else {
482         return -1L;
483       }
484     }
485 
486     public long getSize() {
487       return this.size;
488     }
489 
490     /**
491      * If we have a response, and delay is not set, then respond
492      * immediately.  Otherwise, do not respond to client.  This is
493      * called by the RPC code in the context of the Handler thread.
494      */
495     public synchronized void sendResponseIfReady() throws IOException {
496       if (!this.delayResponse) {
497         this.responder.doRespond(this);
498       }
499     }
500 
501     public UserGroupInformation getRemoteUser() {
502       return connection.user;
503     }
504   }
505 
506   /** Listens on the socket. Creates jobs for the handler threads*/
507   private class Listener extends Thread {
508 
509     private ServerSocketChannel acceptChannel = null; //the accept channel
510     private Selector selector = null; //the selector that we use for the server
511     private Reader[] readers = null;
512     private int currentReader = 0;
513     private Random rand = new Random();
514     private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
515                                          //-tion (for idle connections) ran
516     private long cleanupInterval = 10000; //the minimum interval between
517                                           //two cleanup runs
518     private int backlogLength;
519 
520     private ExecutorService readPool;
521 
522     public Listener(final String name) throws IOException {
523       super(name);
524       backlogLength = conf.getInt("hbase.ipc.server.listen.queue.size", 128);
525       // Create a new server socket and set to non blocking mode
526       acceptChannel = ServerSocketChannel.open();
527       acceptChannel.configureBlocking(false);
528 
529       // Bind the server socket to the binding addrees (can be different from the default interface)
530       bind(acceptChannel.socket(), bindAddress, backlogLength);
531       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
532       address = (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
533       // create a selector;
534       selector= Selector.open();
535 
536       readers = new Reader[readThreads];
537       readPool = Executors.newFixedThreadPool(readThreads,
538         new ThreadFactoryBuilder().setNameFormat(
539           "RpcServer.reader=%d,bindAddress=" + bindAddress.getHostName() +
540           ",port=" + port).setDaemon(true).build());
541       for (int i = 0; i < readThreads; ++i) {
542         Reader reader = new Reader();
543         readers[i] = reader;
544         readPool.execute(reader);
545       }
546       LOG.info(getName() + ": started " + readThreads + " reader(s).");
547 
548       // Register accepts on the server socket with the selector.
549       acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
550       this.setName("RpcServer.listener,port=" + port);
551       this.setDaemon(true);
552     }
553 
554 
555     private class Reader implements Runnable {
556       private volatile boolean adding = false;
557       private final Selector readSelector;
558 
559       Reader() throws IOException {
560         this.readSelector = Selector.open();
561       }
562       @Override
563       public void run() {
564         try {
565           doRunLoop();
566         } finally {
567           try {
568             readSelector.close();
569           } catch (IOException ioe) {
570             LOG.error(getName() + ": error closing read selector in " + getName(), ioe);
571           }
572         }
573       }
574 
575       private synchronized void doRunLoop() {
576         while (running) {
577           try {
578             readSelector.select();
579             while (adding) {
580               this.wait(1000);
581             }
582 
583             Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
584             while (iter.hasNext()) {
585               SelectionKey key = iter.next();
586               iter.remove();
587               if (key.isValid()) {
588                 if (key.isReadable()) {
589                   doRead(key);
590                 }
591               }
592             }
593           } catch (InterruptedException e) {
594             LOG.debug("Interrupted while sleeping");
595             return;
596           } catch (IOException ex) {
597             LOG.info(getName() + ": IOException in Reader", ex);
598           }
599         }
600       }
601 
602       /**
603        * This gets reader into the state that waits for the new channel
604        * to be registered with readSelector. If it was waiting in select()
605        * the thread will be woken up, otherwise whenever select() is called
606        * it will return even if there is nothing to read and wait
607        * in while(adding) for finishAdd call
608        */
609       public void startAdd() {
610         adding = true;
611         readSelector.wakeup();
612       }
613 
614       public synchronized SelectionKey registerChannel(SocketChannel channel)
615         throws IOException {
616         return channel.register(readSelector, SelectionKey.OP_READ);
617       }
618 
619       public synchronized void finishAdd() {
620         adding = false;
621         this.notify();
622       }
623     }
624 
625     /** cleanup connections from connectionList. Choose a random range
626      * to scan and also have a limit on the number of the connections
627      * that will be cleanedup per run. The criteria for cleanup is the time
628      * for which the connection was idle. If 'force' is true then all
629      * connections will be looked at for the cleanup.
630      * @param force all connections will be looked at for cleanup
631      */
632     private void cleanupConnections(boolean force) {
633       if (force || numConnections > thresholdIdleConnections) {
634         long currentTime = System.currentTimeMillis();
635         if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
636           return;
637         }
638         int start = 0;
639         int end = numConnections - 1;
640         if (!force) {
641           start = rand.nextInt() % numConnections;
642           end = rand.nextInt() % numConnections;
643           int temp;
644           if (end < start) {
645             temp = start;
646             start = end;
647             end = temp;
648           }
649         }
650         int i = start;
651         int numNuked = 0;
652         while (i <= end) {
653           Connection c;
654           synchronized (connectionList) {
655             try {
656               c = connectionList.get(i);
657             } catch (Exception e) {return;}
658           }
659           if (c.timedOut(currentTime)) {
660             if (LOG.isDebugEnabled())
661               LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
662             closeConnection(c);
663             numNuked++;
664             end--;
665             //noinspection UnusedAssignment
666             c = null;
667             if (!force && numNuked == maxConnectionsToNuke) break;
668           }
669           else i++;
670         }
671         lastCleanupRunTime = System.currentTimeMillis();
672       }
673     }
674 
675     @Override
676     public void run() {
677       LOG.info(getName() + ": starting");
678       while (running) {
679         SelectionKey key = null;
680         try {
681           selector.select(); // FindBugs IS2_INCONSISTENT_SYNC
682           Iterator<SelectionKey> iter = selector.selectedKeys().iterator();
683           while (iter.hasNext()) {
684             key = iter.next();
685             iter.remove();
686             try {
687               if (key.isValid()) {
688                 if (key.isAcceptable())
689                   doAccept(key);
690               }
691             } catch (IOException ignored) {
692               if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
693             }
694             key = null;
695           }
696         } catch (OutOfMemoryError e) {
697           if (errorHandler != null) {
698             if (errorHandler.checkOOME(e)) {
699               LOG.info(getName() + ": exiting on OutOfMemoryError");
700               closeCurrentConnection(key, e);
701               cleanupConnections(true);
702               return;
703             }
704           } else {
705             // we can run out of memory if we have too many threads
706             // log the event and sleep for a minute and give
707             // some thread(s) a chance to finish
708             LOG.warn(getName() + ": OutOfMemoryError in server select", e);
709             closeCurrentConnection(key, e);
710             cleanupConnections(true);
711             try {
712               Thread.sleep(60000);
713             } catch (InterruptedException ex) {
714               LOG.debug("Interrupted while sleeping");
715               return;
716             }
717           }
718         } catch (Exception e) {
719           closeCurrentConnection(key, e);
720         }
721         cleanupConnections(false);
722       }
723 
724       LOG.info(getName() + ": stopping");
725 
726       synchronized (this) {
727         try {
728           acceptChannel.close();
729           selector.close();
730         } catch (IOException ignored) {
731           if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
732         }
733 
734         selector= null;
735         acceptChannel= null;
736 
737         // clean up all connections
738         while (!connectionList.isEmpty()) {
739           closeConnection(connectionList.remove(0));
740         }
741       }
742     }
743 
744     private void closeCurrentConnection(SelectionKey key, Throwable e) {
745       if (key != null) {
746         Connection c = (Connection)key.attachment();
747         if (c != null) {
748           if (LOG.isDebugEnabled()) {
749             LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
750                 (e != null ? " on error " + e.getMessage() : ""));
751           }
752           closeConnection(c);
753           key.attach(null);
754         }
755       }
756     }
757 
758     InetSocketAddress getAddress() {
759       return address;
760     }
761 
762     void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
763       Connection c;
764       ServerSocketChannel server = (ServerSocketChannel) key.channel();
765 
766       SocketChannel channel;
767       while ((channel = server.accept()) != null) {
768         try {
769           channel.configureBlocking(false);
770           channel.socket().setTcpNoDelay(tcpNoDelay);
771           channel.socket().setKeepAlive(tcpKeepAlive);
772         } catch (IOException ioe) {
773           channel.close();
774           throw ioe;
775         }
776 
777         Reader reader = getReader();
778         try {
779           reader.startAdd();
780           SelectionKey readKey = reader.registerChannel(channel);
781           c = getConnection(channel, System.currentTimeMillis());
782           readKey.attach(c);
783           synchronized (connectionList) {
784             connectionList.add(numConnections, c);
785             numConnections++;
786           }
787           if (LOG.isDebugEnabled())
788             LOG.debug(getName() + ": connection from " + c.toString() +
789                 "; # active connections: " + numConnections);
790         } finally {
791           reader.finishAdd();
792         }
793       }
794     }
795 
796     void doRead(SelectionKey key) throws InterruptedException {
797       int count;
798       Connection c = (Connection) key.attachment();
799       if (c == null) {
800         return;
801       }
802       c.setLastContact(System.currentTimeMillis());
803       try {
804         count = c.readAndProcess();
805 
806         if (count > 0) {
807           c.setLastContact(System.currentTimeMillis());
808         }
809 
810       } catch (InterruptedException ieo) {
811         throw ieo;
812       } catch (Exception e) {
813         if (LOG.isDebugEnabled()) {
814           LOG.debug(getName() + ": Caught exception while reading:" + e.getMessage());
815         }
816         count = -1; //so that the (count < 0) block is executed
817       }
818       if (count < 0) {
819         if (LOG.isDebugEnabled()) {
820           LOG.debug(getName() + ": DISCONNECTING client " + c.toString() +
821               " because read count=" + count +
822               ". Number of active connections: " + numConnections);
823         }
824         closeConnection(c);
825       }
826     }
827 
828     synchronized void doStop() {
829       if (selector != null) {
830         selector.wakeup();
831         Thread.yield();
832       }
833       if (acceptChannel != null) {
834         try {
835           acceptChannel.socket().close();
836         } catch (IOException e) {
837           LOG.info(getName() + ": exception in closing listener socket. " + e);
838         }
839       }
840       readPool.shutdownNow();
841     }
842 
843     // The method that will return the next reader to work with
844     // Simplistic implementation of round robin for now
845     Reader getReader() {
846       currentReader = (currentReader + 1) % readers.length;
847       return readers[currentReader];
848     }
849   }
850 
851   // Sends responses of RPC back to clients.
852   protected class Responder extends Thread {
853     private final Selector writeSelector;
854     private final Set<Connection> writingCons =
855         Collections.newSetFromMap(new ConcurrentHashMap<Connection, Boolean>());
856 
857     Responder() throws IOException {
858       this.setName("RpcServer.responder");
859       this.setDaemon(true);
860       writeSelector = Selector.open(); // create a selector
861     }
862 
863     @Override
864     public void run() {
865       LOG.info(getName() + ": starting");
866       try {
867         doRunLoop();
868       } finally {
869         LOG.info(getName() + ": stopping");
870         try {
871           writeSelector.close();
872         } catch (IOException ioe) {
873           LOG.error(getName() + ": couldn't close write selector", ioe);
874         }
875       }
876     }
877 
878     /**
879      * Take the list of the connections that want to write, and register them
880      * in the selector.
881      */
882     private void registerWrites() {
883       Iterator<Connection> it = writingCons.iterator();
884       while (it.hasNext()) {
885         Connection c = it.next();
886         it.remove();
887         SelectionKey sk = c.channel.keyFor(writeSelector);
888         try {
889           if (sk == null) {
890             try {
891               c.channel.register(writeSelector, SelectionKey.OP_WRITE, c);
892             } catch (ClosedChannelException e) {
893               // ignore: the client went away.
894               if (LOG.isTraceEnabled()) LOG.trace("ignored", e);
895             }
896           } else {
897             sk.interestOps(SelectionKey.OP_WRITE);
898           }
899         } catch (CancelledKeyException e) {
900           // ignore: the client went away.
901           if (LOG.isTraceEnabled()) LOG.trace("ignored", e);
902         }
903       }
904     }
905 
906     /**
907      * Add a connection to the list that want to write,
908      */
909     public void registerForWrite(Connection c) {
910       if (writingCons.add(c)) {
911         writeSelector.wakeup();
912       }
913     }
914 
915     private void doRunLoop() {
916       long lastPurgeTime = 0;   // last check for old calls.
917       while (running) {
918         try {
919           registerWrites();
920           int keyCt = writeSelector.select(purgeTimeout);
921           if (keyCt == 0) {
922             continue;
923           }
924 
925           Set<SelectionKey> keys = writeSelector.selectedKeys();
926           Iterator<SelectionKey> iter = keys.iterator();
927           while (iter.hasNext()) {
928             SelectionKey key = iter.next();
929             iter.remove();
930             try {
931               if (key.isValid() && key.isWritable()) {
932                 doAsyncWrite(key);
933               }
934             } catch (IOException e) {
935               LOG.debug(getName() + ": asyncWrite", e);
936             }
937           }
938 
939           lastPurgeTime = purge(lastPurgeTime);
940 
941         } catch (OutOfMemoryError e) {
942           if (errorHandler != null) {
943             if (errorHandler.checkOOME(e)) {
944               LOG.info(getName() + ": exiting on OutOfMemoryError");
945               return;
946             }
947           } else {
948             //
949             // we can run out of memory if we have too many threads
950             // log the event and sleep for a minute and give
951             // some thread(s) a chance to finish
952             //
953             LOG.warn(getName() + ": OutOfMemoryError in server select", e);
954             try {
955               Thread.sleep(60000);
956             } catch (InterruptedException ex) {
957               LOG.debug("Interrupted while sleeping");
958               return;
959             }
960           }
961         } catch (Exception e) {
962           LOG.warn(getName() + ": exception in Responder " +
963               StringUtils.stringifyException(e), e);
964         }
965       }
966       LOG.info(getName() + ": stopped");
967     }
968 
969     /**
970      * If there were some calls that have not been sent out for a
971      * long time, we close the connection.
972      * @return the time of the purge.
973      */
974     private long purge(long lastPurgeTime) {
975       long now = System.currentTimeMillis();
976       if (now < lastPurgeTime + purgeTimeout) {
977         return lastPurgeTime;
978       }
979 
980       ArrayList<Connection> conWithOldCalls = new ArrayList<Connection>();
981       // get the list of channels from list of keys.
982       synchronized (writeSelector.keys()) {
983         for (SelectionKey key : writeSelector.keys()) {
984           Connection connection = (Connection) key.attachment();
985           if (connection == null) {
986             throw new IllegalStateException("Coding error: SelectionKey key without attachment.");
987           }
988           Call call = connection.responseQueue.peekFirst();
989           if (call != null && now > call.timestamp + purgeTimeout) {
990             conWithOldCalls.add(call.connection);
991           }
992         }
993       }
994 
995       // Seems safer to close the connection outside of the synchronized loop...
996       for (Connection connection : conWithOldCalls) {
997         closeConnection(connection);
998       }
999 
1000       return now;
1001     }
1002 
1003     private void doAsyncWrite(SelectionKey key) throws IOException {
1004       Connection connection = (Connection) key.attachment();
1005       if (connection == null) {
1006         throw new IOException("doAsyncWrite: no connection");
1007       }
1008       if (key.channel() != connection.channel) {
1009         throw new IOException("doAsyncWrite: bad channel");
1010       }
1011 
1012       if (processAllResponses(connection)) {
1013         try {
1014           // We wrote everything, so we don't need to be told when the socket is ready for
1015           //  write anymore.
1016          key.interestOps(0);
1017         } catch (CancelledKeyException e) {
1018           /* The Listener/reader might have closed the socket.
1019            * We don't explicitly cancel the key, so not sure if this will
1020            * ever fire.
1021            * This warning could be removed.
1022            */
1023           LOG.warn("Exception while changing ops : " + e);
1024         }
1025       }
1026     }
1027 
1028     /**
1029      * Process the response for this call. You need to have the lock on
1030      * {@link org.apache.hadoop.hbase.ipc.RpcServer.Connection#responseWriteLock}
1031      *
1032      * @param call the call
1033      * @return true if we proceed the call fully, false otherwise.
1034      * @throws IOException
1035      */
1036     private boolean processResponse(final Call call) throws IOException {
1037       boolean error = true;
1038       try {
1039         // Send as much data as we can in the non-blocking fashion
1040         long numBytes = channelWrite(call.connection.channel, call.response);
1041         if (numBytes < 0) {
1042           throw new HBaseIOException("Error writing on the socket " +
1043             "for the call:" + call.toShortString());
1044         }
1045         error = false;
1046       } finally {
1047         if (error) {
1048           LOG.debug(getName() + call.toShortString() + ": output error -- closing");
1049           closeConnection(call.connection);
1050         }
1051       }
1052 
1053       if (!call.response.hasRemaining()) {
1054         call.connection.decRpcCount();  // Say that we're done with this call.
1055         return true;
1056       } else {
1057         return false; // Socket can't take more, we will have to come back.
1058       }
1059     }
1060 
1061     /**
1062      * Process all the responses for this connection
1063      *
1064      * @return true if all the calls were processed or that someone else is doing it.
1065      * false if there * is still some work to do. In this case, we expect the caller to
1066      * delay us.
1067      * @throws IOException
1068      */
1069     private boolean processAllResponses(final Connection connection) throws IOException {
1070       // We want only one writer on the channel for a connection at a time.
1071       connection.responseWriteLock.lock();
1072       try {
1073         for (int i = 0; i < 20; i++) {
1074           // protection if some handlers manage to need all the responder
1075           Call call = connection.responseQueue.pollFirst();
1076           if (call == null) {
1077             return true;
1078           }
1079           if (!processResponse(call)) {
1080             connection.responseQueue.addFirst(call);
1081             return false;
1082           }
1083         }
1084       } finally {
1085         connection.responseWriteLock.unlock();
1086       }
1087 
1088       return connection.responseQueue.isEmpty();
1089     }
1090 
1091     //
1092     // Enqueue a response from the application.
1093     //
1094     void doRespond(Call call) throws IOException {
1095       boolean added = false;
1096 
1097       // If there is already a write in progress, we don't wait. This allows to free the handlers
1098       //  immediately for other tasks.
1099       if (call.connection.responseQueue.isEmpty() && call.connection.responseWriteLock.tryLock()) {
1100         try {
1101           if (call.connection.responseQueue.isEmpty()) {
1102             // If we're alone, we can try to do a direct call to the socket. It's
1103             //  an optimisation to save on context switches and data transfer between cores..
1104             if (processResponse(call)) {
1105               return; // we're done.
1106             }
1107             // Too big to fit, putting ahead.
1108             call.connection.responseQueue.addFirst(call);
1109             added = true; // We will register to the selector later, outside of the lock.
1110           }
1111         } finally {
1112           call.connection.responseWriteLock.unlock();
1113         }
1114       }
1115 
1116       if (!added) {
1117         call.connection.responseQueue.addLast(call);
1118       }
1119       call.responder.registerForWrite(call.connection);
1120 
1121       // set the serve time when the response has to be sent later
1122       call.timestamp = System.currentTimeMillis();
1123     }
1124   }
1125 
1126   @SuppressWarnings("serial")
1127   public static class CallQueueTooBigException extends IOException {
1128     CallQueueTooBigException() {
1129       super();
1130     }
1131   }
1132 
1133   /** Reads calls from a connection and queues them for handling. */
1134   @edu.umd.cs.findbugs.annotations.SuppressWarnings(
1135       value="VO_VOLATILE_INCREMENT",
1136       justification="False positive according to http://sourceforge.net/p/findbugs/bugs/1032/")
1137   public class Connection {
1138     // If initial preamble with version and magic has been read or not.
1139     private boolean connectionPreambleRead = false;
1140     // If the connection header has been read or not.
1141     private boolean connectionHeaderRead = false;
1142     protected SocketChannel channel;
1143     private ByteBuffer data;
1144     private ByteBuffer dataLengthBuffer;
1145     protected final ConcurrentLinkedDeque<Call> responseQueue = new ConcurrentLinkedDeque<Call>();
1146     private final Lock responseWriteLock = new ReentrantLock();
1147     private Counter rpcCount = new Counter(); // number of outstanding rpcs
1148     private long lastContact;
1149     private InetAddress addr;
1150     protected Socket socket;
1151     // Cache the remote host & port info so that even if the socket is
1152     // disconnected, we can say where it used to connect to.
1153     protected String hostAddress;
1154     protected int remotePort;
1155     ConnectionHeader connectionHeader;
1156     /**
1157      * Codec the client asked use.
1158      */
1159     private Codec codec;
1160     /**
1161      * Compression codec the client asked us use.
1162      */
1163     private CompressionCodec compressionCodec;
1164     BlockingService service;
1165     protected UserGroupInformation user = null;
1166     private AuthMethod authMethod;
1167     private boolean saslContextEstablished;
1168     private boolean skipInitialSaslHandshake;
1169     private ByteBuffer unwrappedData;
1170     // When is this set?  FindBugs wants to know!  Says NP
1171     private ByteBuffer unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
1172     boolean useSasl;
1173     SaslServer saslServer;
1174     private boolean useWrap = false;
1175     // Fake 'call' for failed authorization response
1176     private static final int AUTHORIZATION_FAILED_CALLID = -1;
1177     private final Call authFailedCall =
1178       new Call(AUTHORIZATION_FAILED_CALLID, null, null, null, null, null, this, null, 0, null);
1179     private ByteArrayOutputStream authFailedResponse =
1180         new ByteArrayOutputStream();
1181     // Fake 'call' for SASL context setup
1182     private static final int SASL_CALLID = -33;
1183     private final Call saslCall =
1184       new Call(SASL_CALLID, this.service, null, null, null, null, this, null, 0, null);
1185 
1186     public UserGroupInformation attemptingUser = null; // user name before auth
1187 
1188     public Connection(SocketChannel channel, long lastContact) {
1189       this.channel = channel;
1190       this.lastContact = lastContact;
1191       this.data = null;
1192       this.dataLengthBuffer = ByteBuffer.allocate(4);
1193       this.socket = channel.socket();
1194       this.addr = socket.getInetAddress();
1195       if (addr == null) {
1196         this.hostAddress = "*Unknown*";
1197       } else {
1198         this.hostAddress = addr.getHostAddress();
1199       }
1200       this.remotePort = socket.getPort();
1201       if (socketSendBufferSize != 0) {
1202         try {
1203           socket.setSendBufferSize(socketSendBufferSize);
1204         } catch (IOException e) {
1205           LOG.warn("Connection: unable to set socket send buffer size to " +
1206                    socketSendBufferSize);
1207         }
1208       }
1209     }
1210 
1211       @Override
1212     public String toString() {
1213       return getHostAddress() + ":" + remotePort;
1214     }
1215 
1216     public String getHostAddress() {
1217       return hostAddress;
1218     }
1219 
1220     public InetAddress getHostInetAddress() {
1221       return addr;
1222     }
1223 
1224     public int getRemotePort() {
1225       return remotePort;
1226     }
1227 
1228     public void setLastContact(long lastContact) {
1229       this.lastContact = lastContact;
1230     }
1231 
1232     /* Return true if the connection has no outstanding rpc */
1233     private boolean isIdle() {
1234       return rpcCount.get() == 0;
1235     }
1236 
1237     /* Decrement the outstanding RPC count */
1238     protected void decRpcCount() {
1239       rpcCount.decrement();
1240     }
1241 
1242     /* Increment the outstanding RPC count */
1243     protected void incRpcCount() {
1244       rpcCount.increment();
1245     }
1246 
1247     protected boolean timedOut(long currentTime) {
1248       return isIdle() && currentTime - lastContact > maxIdleTime;
1249     }
1250 
1251     private UserGroupInformation getAuthorizedUgi(String authorizedId)
1252         throws IOException {
1253       if (authMethod == AuthMethod.DIGEST) {
1254         TokenIdentifier tokenId = HBaseSaslRpcServer.getIdentifier(authorizedId,
1255             secretManager);
1256         UserGroupInformation ugi = tokenId.getUser();
1257         if (ugi == null) {
1258           throw new AccessDeniedException(
1259               "Can't retrieve username from tokenIdentifier.");
1260         }
1261         ugi.addTokenIdentifier(tokenId);
1262         return ugi;
1263       } else {
1264         return UserGroupInformation.createRemoteUser(authorizedId);
1265       }
1266     }
1267 
1268     private void saslReadAndProcess(byte[] saslToken) throws IOException,
1269         InterruptedException {
1270       if (saslContextEstablished) {
1271         if (LOG.isTraceEnabled())
1272           LOG.trace("Have read input token of size " + saslToken.length
1273               + " for processing by saslServer.unwrap()");
1274 
1275         if (!useWrap) {
1276           processOneRpc(saslToken);
1277         } else {
1278           byte [] plaintextData = saslServer.unwrap(saslToken, 0, saslToken.length);
1279           processUnwrappedData(plaintextData);
1280         }
1281       } else {
1282         byte[] replyToken;
1283         try {
1284           if (saslServer == null) {
1285             switch (authMethod) {
1286             case DIGEST:
1287               if (secretManager == null) {
1288                 throw new AccessDeniedException(
1289                     "Server is not configured to do DIGEST authentication.");
1290               }
1291               saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
1292                   .getMechanismName(), null, SaslUtil.SASL_DEFAULT_REALM,
1293                   SaslUtil.SASL_PROPS, new SaslDigestCallbackHandler(
1294                       secretManager, this));
1295               break;
1296             default:
1297               UserGroupInformation current = UserGroupInformation.getCurrentUser();
1298               String fullName = current.getUserName();
1299               if (LOG.isDebugEnabled()) {
1300                 LOG.debug("Kerberos principal name is " + fullName);
1301               }
1302               final String names[] = SaslUtil.splitKerberosName(fullName);
1303               if (names.length != 3) {
1304                 throw new AccessDeniedException(
1305                     "Kerberos principal name does NOT have the expected "
1306                         + "hostname part: " + fullName);
1307               }
1308               current.doAs(new PrivilegedExceptionAction<Object>() {
1309                 @Override
1310                 public Object run() throws SaslException {
1311                   saslServer = Sasl.createSaslServer(AuthMethod.KERBEROS
1312                       .getMechanismName(), names[0], names[1],
1313                       SaslUtil.SASL_PROPS, new SaslGssCallbackHandler());
1314                   return null;
1315                 }
1316               });
1317             }
1318             if (saslServer == null)
1319               throw new AccessDeniedException(
1320                   "Unable to find SASL server implementation for "
1321                       + authMethod.getMechanismName());
1322             if (LOG.isDebugEnabled()) {
1323               LOG.debug("Created SASL server with mechanism = " + authMethod.getMechanismName());
1324             }
1325           }
1326           if (LOG.isDebugEnabled()) {
1327             LOG.debug("Have read input token of size " + saslToken.length
1328                 + " for processing by saslServer.evaluateResponse()");
1329           }
1330           replyToken = saslServer.evaluateResponse(saslToken);
1331         } catch (IOException e) {
1332           IOException sendToClient = e;
1333           Throwable cause = e;
1334           while (cause != null) {
1335             if (cause instanceof InvalidToken) {
1336               sendToClient = (InvalidToken) cause;
1337               break;
1338             }
1339             cause = cause.getCause();
1340           }
1341           doRawSaslReply(SaslStatus.ERROR, null, sendToClient.getClass().getName(),
1342             sendToClient.getLocalizedMessage());
1343           metrics.authenticationFailure();
1344           String clientIP = this.toString();
1345           // attempting user could be null
1346           AUDITLOG.warn(AUTH_FAILED_FOR + clientIP + ":" + attemptingUser);
1347           throw e;
1348         }
1349         if (replyToken != null) {
1350           if (LOG.isDebugEnabled()) {
1351             LOG.debug("Will send token of size " + replyToken.length
1352                 + " from saslServer.");
1353           }
1354           doRawSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null,
1355               null);
1356         }
1357         if (saslServer.isComplete()) {
1358           String qop = (String) saslServer.getNegotiatedProperty(Sasl.QOP);
1359           useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
1360           user = getAuthorizedUgi(saslServer.getAuthorizationID());
1361           if (LOG.isDebugEnabled()) {
1362             LOG.debug("SASL server context established. Authenticated client: "
1363               + user + ". Negotiated QoP is "
1364               + saslServer.getNegotiatedProperty(Sasl.QOP));
1365           }
1366           metrics.authenticationSuccess();
1367           AUDITLOG.info(AUTH_SUCCESSFUL_FOR + user);
1368           saslContextEstablished = true;
1369         }
1370       }
1371     }
1372 
1373     /**
1374      * No protobuf encoding of raw sasl messages
1375      */
1376     private void doRawSaslReply(SaslStatus status, Writable rv,
1377         String errorClass, String error) throws IOException {
1378       ByteBufferOutputStream saslResponse = null;
1379       DataOutputStream out = null;
1380       try {
1381         // In my testing, have noticed that sasl messages are usually
1382         // in the ballpark of 100-200. That's why the initial capacity is 256.
1383         saslResponse = new ByteBufferOutputStream(256);
1384         out = new DataOutputStream(saslResponse);
1385         out.writeInt(status.state); // write status
1386         if (status == SaslStatus.SUCCESS) {
1387           rv.write(out);
1388         } else {
1389           WritableUtils.writeString(out, errorClass);
1390           WritableUtils.writeString(out, error);
1391         }
1392         saslCall.setSaslTokenResponse(saslResponse.getByteBuffer());
1393         saslCall.responder = responder;
1394         saslCall.sendResponseIfReady();
1395       } finally {
1396         if (saslResponse != null) {
1397           saslResponse.close();
1398         }
1399         if (out != null) {
1400           out.close();
1401         }
1402       }
1403     }
1404 
1405     private void disposeSasl() {
1406       if (saslServer != null) {
1407         try {
1408           saslServer.dispose();
1409           saslServer = null;
1410         } catch (SaslException ignored) {
1411         }
1412       }
1413     }
1414 
1415     private int readPreamble() throws IOException {
1416       int count;
1417       // Check for 'HBas' magic.
1418       this.dataLengthBuffer.flip();
1419       if (!Arrays.equals(HConstants.RPC_HEADER, dataLengthBuffer.array())) {
1420         return doBadPreambleHandling("Expected HEADER=" +
1421             Bytes.toStringBinary(HConstants.RPC_HEADER) +
1422             " but received HEADER=" + Bytes.toStringBinary(dataLengthBuffer.array()) +
1423             " from " + toString());
1424       }
1425       // Now read the next two bytes, the version and the auth to use.
1426       ByteBuffer versionAndAuthBytes = ByteBuffer.allocate(2);
1427       count = channelRead(channel, versionAndAuthBytes);
1428       if (count < 0 || versionAndAuthBytes.remaining() > 0) {
1429         return count;
1430       }
1431       int version = versionAndAuthBytes.get(0);
1432       byte authbyte = versionAndAuthBytes.get(1);
1433       this.authMethod = AuthMethod.valueOf(authbyte);
1434       if (version != CURRENT_VERSION) {
1435         String msg = getFatalConnectionString(version, authbyte);
1436         return doBadPreambleHandling(msg, new WrongVersionException(msg));
1437       }
1438       if (authMethod == null) {
1439         String msg = getFatalConnectionString(version, authbyte);
1440         return doBadPreambleHandling(msg, new BadAuthException(msg));
1441       }
1442       if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
1443         AccessDeniedException ae = new AccessDeniedException("Authentication is required");
1444         setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
1445         responder.doRespond(authFailedCall);
1446         throw ae;
1447       }
1448       if (!isSecurityEnabled && authMethod != AuthMethod.SIMPLE) {
1449         doRawSaslReply(SaslStatus.SUCCESS, new IntWritable(
1450             SaslUtil.SWITCH_TO_SIMPLE_AUTH), null, null);
1451         authMethod = AuthMethod.SIMPLE;
1452         // client has already sent the initial Sasl message and we
1453         // should ignore it. Both client and server should fall back
1454         // to simple auth from now on.
1455         skipInitialSaslHandshake = true;
1456       }
1457       if (authMethod != AuthMethod.SIMPLE) {
1458         useSasl = true;
1459       }
1460 
1461       dataLengthBuffer.clear();
1462       connectionPreambleRead = true;
1463       return count;
1464     }
1465 
1466     private int read4Bytes() throws IOException {
1467       if (this.dataLengthBuffer.remaining() > 0) {
1468         return channelRead(channel, this.dataLengthBuffer);
1469       } else {
1470         return 0;
1471       }
1472     }
1473 
1474 
1475     /**
1476      * Read off the wire. If there is not enough data to read, update the connection state with
1477      *  what we have and returns.
1478      * @return Returns -1 if failure (and caller will close connection), else zero or more.
1479      * @throws IOException
1480      * @throws InterruptedException
1481      */
1482     public int readAndProcess() throws IOException, InterruptedException {
1483       // Try and read in an int.  If new connection, the int will hold the 'HBas' HEADER.  If it
1484       // does, read in the rest of the connection preamble, the version and the auth method.
1485       // Else it will be length of the data to read (or -1 if a ping).  We catch the integer
1486       // length into the 4-byte this.dataLengthBuffer.
1487       int count = read4Bytes();
1488       if (count < 0 || dataLengthBuffer.remaining() > 0 ){
1489         return count;
1490       }
1491 
1492       // If we have not read the connection setup preamble, look to see if that is on the wire.
1493       if (!connectionPreambleRead) {
1494         count = readPreamble();
1495         if (!connectionPreambleRead) {
1496           return count;
1497         }
1498 
1499         count = read4Bytes();
1500         if (count < 0 || dataLengthBuffer.remaining() > 0) {
1501           return count;
1502         }
1503       }
1504 
1505       // We have read a length and we have read the preamble.  It is either the connection header
1506       // or it is a request.
1507       if (data == null) {
1508         dataLengthBuffer.flip();
1509         int dataLength = dataLengthBuffer.getInt();
1510         if (dataLength == RpcClient.PING_CALL_ID) {
1511           if (!useWrap) { //covers the !useSasl too
1512             dataLengthBuffer.clear();
1513             return 0;  //ping message
1514           }
1515         }
1516         if (dataLength < 0) { // A data length of zero is legal.
1517           throw new IllegalArgumentException("Unexpected data length "
1518               + dataLength + "!! from " + getHostAddress());
1519         }
1520         data = ByteBuffer.allocate(dataLength);
1521 
1522         // Increment the rpc count. This counter will be decreased when we write
1523         //  the response.  If we want the connection to be detected as idle properly, we
1524         //  need to keep the inc / dec correct.
1525         incRpcCount();
1526       }
1527 
1528       count = channelRead(channel, data);
1529 
1530       if (count >= 0 && data.remaining() == 0) { // count==0 if dataLength == 0
1531         process();
1532       }
1533 
1534       return count;
1535     }
1536 
1537     /**
1538      * Process the data buffer and clean the connection state for the next call.
1539      */
1540     private void process() throws IOException, InterruptedException {
1541       data.flip();
1542       try {
1543         if (skipInitialSaslHandshake) {
1544           skipInitialSaslHandshake = false;
1545           return;
1546         }
1547 
1548         if (useSasl) {
1549           saslReadAndProcess(data.array());
1550         } else {
1551           processOneRpc(data.array());
1552         }
1553 
1554       } finally {
1555         dataLengthBuffer.clear(); // Clean for the next call
1556         data = null; // For the GC
1557       }
1558     }
1559 
1560     private String getFatalConnectionString(final int version, final byte authByte) {
1561       return "serverVersion=" + CURRENT_VERSION +
1562       ", clientVersion=" + version + ", authMethod=" + authByte +
1563       ", authSupported=" + (authMethod != null) + " from " + toString();
1564     }
1565 
1566     private int doBadPreambleHandling(final String msg) throws IOException {
1567       return doBadPreambleHandling(msg, new FatalConnectionException(msg));
1568     }
1569 
1570     private int doBadPreambleHandling(final String msg, final Exception e) throws IOException {
1571       LOG.warn(msg);
1572       Call fakeCall = new Call(-1, null, null, null, null, null, this, responder, -1, null);
1573       setupResponse(null, fakeCall, e, msg);
1574       responder.doRespond(fakeCall);
1575       // Returning -1 closes out the connection.
1576       return -1;
1577     }
1578 
1579     // Reads the connection header following version
1580     private void processConnectionHeader(byte[] buf) throws IOException {
1581       this.connectionHeader = ConnectionHeader.parseFrom(buf);
1582       String serviceName = connectionHeader.getServiceName();
1583       if (serviceName == null) throw new EmptyServiceNameException();
1584       this.service = getService(services, serviceName);
1585       if (this.service == null) throw new UnknownServiceException(serviceName);
1586       setupCellBlockCodecs(this.connectionHeader);
1587       UserGroupInformation protocolUser = createUser(connectionHeader);
1588       if (!useSasl) {
1589         user = protocolUser;
1590         if (user != null) {
1591           user.setAuthenticationMethod(AuthMethod.SIMPLE.authenticationMethod);
1592         }
1593       } else {
1594         // user is authenticated
1595         user.setAuthenticationMethod(authMethod.authenticationMethod);
1596         //Now we check if this is a proxy user case. If the protocol user is
1597         //different from the 'user', it is a proxy user scenario. However,
1598         //this is not allowed if user authenticated with DIGEST.
1599         if ((protocolUser != null)
1600             && (!protocolUser.getUserName().equals(user.getUserName()))) {
1601           if (authMethod == AuthMethod.DIGEST) {
1602             // Not allowed to doAs if token authentication is used
1603             throw new AccessDeniedException("Authenticated user (" + user
1604                 + ") doesn't match what the client claims to be ("
1605                 + protocolUser + ")");
1606           } else {
1607             // Effective user can be different from authenticated user
1608             // for simple auth or kerberos auth
1609             // The user is the real user. Now we create a proxy user
1610             UserGroupInformation realUser = user;
1611             user = UserGroupInformation.createProxyUser(protocolUser
1612                 .getUserName(), realUser);
1613             // Now the user is a proxy user, set Authentication method Proxy.
1614             user.setAuthenticationMethod(AuthenticationMethod.PROXY);
1615           }
1616         }
1617       }
1618     }
1619 
1620     /**
1621      * Set up cell block codecs
1622      * @throws FatalConnectionException
1623      */
1624     private void setupCellBlockCodecs(final ConnectionHeader header)
1625     throws FatalConnectionException {
1626       // TODO: Plug in other supported decoders.
1627       if (!header.hasCellBlockCodecClass()) return;
1628       String className = header.getCellBlockCodecClass();
1629       if (className == null || className.length() == 0) return;
1630       try {
1631         this.codec = (Codec)Class.forName(className).newInstance();
1632       } catch (Exception e) {
1633         throw new UnsupportedCellCodecException(className, e);
1634       }
1635       if (!header.hasCellBlockCompressorClass()) return;
1636       className = header.getCellBlockCompressorClass();
1637       try {
1638         this.compressionCodec = (CompressionCodec)Class.forName(className).newInstance();
1639       } catch (Exception e) {
1640         throw new UnsupportedCompressionCodecException(className, e);
1641       }
1642     }
1643 
1644     private void processUnwrappedData(byte[] inBuf) throws IOException,
1645     InterruptedException {
1646       ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(inBuf));
1647       // Read all RPCs contained in the inBuf, even partial ones
1648       while (true) {
1649         int count;
1650         if (unwrappedDataLengthBuffer.remaining() > 0) {
1651           count = channelRead(ch, unwrappedDataLengthBuffer);
1652           if (count <= 0 || unwrappedDataLengthBuffer.remaining() > 0)
1653             return;
1654         }
1655 
1656         if (unwrappedData == null) {
1657           unwrappedDataLengthBuffer.flip();
1658           int unwrappedDataLength = unwrappedDataLengthBuffer.getInt();
1659 
1660           if (unwrappedDataLength == RpcClient.PING_CALL_ID) {
1661             if (LOG.isDebugEnabled())
1662               LOG.debug("Received ping message");
1663             unwrappedDataLengthBuffer.clear();
1664             continue; // ping message
1665           }
1666           unwrappedData = ByteBuffer.allocate(unwrappedDataLength);
1667         }
1668 
1669         count = channelRead(ch, unwrappedData);
1670         if (count <= 0 || unwrappedData.remaining() > 0)
1671           return;
1672 
1673         if (unwrappedData.remaining() == 0) {
1674           unwrappedDataLengthBuffer.clear();
1675           unwrappedData.flip();
1676           processOneRpc(unwrappedData.array());
1677           unwrappedData = null;
1678         }
1679       }
1680     }
1681 
1682     private void processOneRpc(byte[] buf) throws IOException, InterruptedException {
1683       if (connectionHeaderRead) {
1684         processRequest(buf);
1685       } else {
1686         processConnectionHeader(buf);
1687         this.connectionHeaderRead = true;
1688         if (!authorizeConnection()) {
1689           // Throw FatalConnectionException wrapping ACE so client does right thing and closes
1690           // down the connection instead of trying to read non-existent retun.
1691           throw new AccessDeniedException("Connection from " + this + " for service " +
1692             connectionHeader.getServiceName() + " is unauthorized for user: " + user);
1693         }
1694       }
1695     }
1696 
1697     /**
1698      * @param buf Has the request header and the request param and optionally encoded data buffer
1699      * all in this one array.
1700      * @throws IOException
1701      * @throws InterruptedException
1702      */
1703     protected void processRequest(byte[] buf) throws IOException, InterruptedException {
1704       long totalRequestSize = buf.length;
1705       int offset = 0;
1706       // Here we read in the header.  We avoid having pb
1707       // do its default 4k allocation for CodedInputStream.  We force it to use backing array.
1708       CodedInputStream cis = CodedInputStream.newInstance(buf, offset, buf.length);
1709       int headerSize = cis.readRawVarint32();
1710       offset = cis.getTotalBytesRead();
1711       RequestHeader header = RequestHeader.newBuilder().mergeFrom(buf, offset, headerSize).build();
1712       offset += headerSize;
1713       int id = header.getCallId();
1714       if (LOG.isTraceEnabled()) {
1715         LOG.trace("RequestHeader " + TextFormat.shortDebugString(header) +
1716           " totalRequestSize: " + totalRequestSize + " bytes");
1717       }
1718       // Enforcing the call queue size, this triggers a retry in the client
1719       // This is a bit late to be doing this check - we have already read in the total request.
1720       if ((totalRequestSize + callQueueSize.get()) > maxQueueSize) {
1721         final Call callTooBig =
1722           new Call(id, this.service, null, null, null, null, this,
1723             responder, totalRequestSize, null);
1724         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1725         setupResponse(responseBuffer, callTooBig, new CallQueueTooBigException(),
1726           "Call queue is full on " + getListenerAddress() +
1727           ", is hbase.ipc.server.max.callqueue.size too small?");
1728         responder.doRespond(callTooBig);
1729         return;
1730       }
1731       MethodDescriptor md = null;
1732       Message param = null;
1733       CellScanner cellScanner = null;
1734       try {
1735         if (header.hasRequestParam() && header.getRequestParam()) {
1736           md = this.service.getDescriptorForType().findMethodByName(header.getMethodName());
1737           if (md == null) throw new UnsupportedOperationException(header.getMethodName());
1738           Builder builder = this.service.getRequestPrototype(md).newBuilderForType();
1739           // To read the varint, I need an inputstream; might as well be a CIS.
1740           cis = CodedInputStream.newInstance(buf, offset, buf.length);
1741           int paramSize = cis.readRawVarint32();
1742           offset += cis.getTotalBytesRead();
1743           if (builder != null) {
1744             param = builder.mergeFrom(buf, offset, paramSize).build();
1745           }
1746           offset += paramSize;
1747         }
1748         if (header.hasCellBlockMeta()) {
1749           cellScanner = ipcUtil.createCellScanner(this.codec, this.compressionCodec,
1750             buf, offset, buf.length);
1751         }
1752       } catch (Throwable t) {
1753         String msg = getListenerAddress() + " is unable to read call parameter from client " +
1754             getHostAddress();
1755         LOG.warn(msg, t);
1756 
1757         // probably the hbase hadoop version does not match the running hadoop version
1758         if (t instanceof LinkageError) {
1759           t = new DoNotRetryIOException(t);
1760         }
1761         // If the method is not present on the server, do not retry.
1762         if (t instanceof UnsupportedOperationException) {
1763           t = new DoNotRetryIOException(t);
1764         }
1765 
1766         final Call readParamsFailedCall =
1767           new Call(id, this.service, null, null, null, null, this,
1768             responder, totalRequestSize, null);
1769         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1770         setupResponse(responseBuffer, readParamsFailedCall, t,
1771           msg + "; " + t.getMessage());
1772         responder.doRespond(readParamsFailedCall);
1773         return;
1774       }
1775 
1776       TraceInfo traceInfo = header.hasTraceInfo()
1777           ? new TraceInfo(header.getTraceInfo().getTraceId(), header.getTraceInfo().getParentId())
1778           : null;
1779       Call call = new Call(id, this.service, md, header, param, cellScanner, this, responder,
1780               totalRequestSize,
1781               traceInfo);
1782       scheduler.dispatch(new CallRunner(RpcServer.this, call, userProvider));
1783     }
1784 
1785     private boolean authorizeConnection() throws IOException {
1786       try {
1787         // If auth method is DIGEST, the token was obtained by the
1788         // real user for the effective user, therefore not required to
1789         // authorize real user. doAs is allowed only for simple or kerberos
1790         // authentication
1791         if (user != null && user.getRealUser() != null
1792             && (authMethod != AuthMethod.DIGEST)) {
1793           ProxyUsers.authorize(user, this.getHostAddress(), conf);
1794         }
1795         authorize(user, connectionHeader, getHostInetAddress());
1796         metrics.authorizationSuccess();
1797       } catch (AuthorizationException ae) {
1798         if (LOG.isDebugEnabled()) {
1799           LOG.debug("Connection authorization failed: " + ae.getMessage(), ae);
1800         }
1801         metrics.authorizationFailure();
1802         setupResponse(authFailedResponse, authFailedCall,
1803           new AccessDeniedException(ae), ae.getMessage());
1804         responder.doRespond(authFailedCall);
1805         return false;
1806       }
1807       return true;
1808     }
1809 
1810     protected synchronized void close() {
1811       disposeSasl();
1812       data = null;
1813       this.dataLengthBuffer = null;
1814       if (!channel.isOpen())
1815         return;
1816       try {socket.shutdownOutput();} catch(Exception ignored) {} // FindBugs DE_MIGHT_IGNORE
1817       if (channel.isOpen()) {
1818         try {channel.close();} catch(Exception ignored) {}
1819       }
1820       try {socket.close();} catch(Exception ignored) {}
1821     }
1822 
1823     private UserGroupInformation createUser(ConnectionHeader head) {
1824       UserGroupInformation ugi = null;
1825 
1826       if (!head.hasUserInfo()) {
1827         return null;
1828       }
1829       UserInformation userInfoProto = head.getUserInfo();
1830       String effectiveUser = null;
1831       if (userInfoProto.hasEffectiveUser()) {
1832         effectiveUser = userInfoProto.getEffectiveUser();
1833       }
1834       String realUser = null;
1835       if (userInfoProto.hasRealUser()) {
1836         realUser = userInfoProto.getRealUser();
1837       }
1838       if (effectiveUser != null) {
1839         if (realUser != null) {
1840           UserGroupInformation realUserUgi =
1841               UserGroupInformation.createRemoteUser(realUser);
1842           ugi = UserGroupInformation.createProxyUser(effectiveUser, realUserUgi);
1843         } else {
1844           ugi = UserGroupInformation.createRemoteUser(effectiveUser);
1845         }
1846       }
1847       return ugi;
1848     }
1849   }
1850 
1851   /**
1852    * Datastructure for passing a {@link BlockingService} and its associated class of
1853    * protobuf service interface.  For example, a server that fielded what is defined
1854    * in the client protobuf service would pass in an implementation of the client blocking service
1855    * and then its ClientService.BlockingInterface.class.  Used checking connection setup.
1856    */
1857   public static class BlockingServiceAndInterface {
1858     private final BlockingService service;
1859     private final Class<?> serviceInterface;
1860     public BlockingServiceAndInterface(final BlockingService service,
1861         final Class<?> serviceInterface) {
1862       this.service = service;
1863       this.serviceInterface = serviceInterface;
1864     }
1865     public Class<?> getServiceInterface() {
1866       return this.serviceInterface;
1867     }
1868     public BlockingService getBlockingService() {
1869       return this.service;
1870     }
1871   }
1872 
1873   /**
1874    * Constructs a server listening on the named port and address.
1875    * @param server hosting instance of {@link Server}. We will do authentications if an
1876    * instance else pass null for no authentication check.
1877    * @param name Used keying this rpc servers' metrics and for naming the Listener thread.
1878    * @param services A list of services.
1879    * @param bindAddress Where to listen
1880    * @param conf
1881    * @param scheduler
1882    */
1883   public RpcServer(final Server server, final String name,
1884       final List<BlockingServiceAndInterface> services,
1885       final InetSocketAddress bindAddress, Configuration conf,
1886       RpcScheduler scheduler)
1887       throws IOException {
1888 
1889     this.server = server;
1890     this.services = services;
1891     this.bindAddress = bindAddress;
1892     this.conf = conf;
1893     this.socketSendBufferSize = 0;
1894     this.maxQueueSize =
1895       this.conf.getInt("hbase.ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE);
1896     this.readThreads = conf.getInt("hbase.ipc.server.read.threadpool.size", 10);
1897     this.maxIdleTime = 2 * conf.getInt("hbase.ipc.client.connection.maxidletime", 1000);
1898     this.maxConnectionsToNuke = conf.getInt("hbase.ipc.client.kill.max", 10);
1899     this.thresholdIdleConnections = conf.getInt("hbase.ipc.client.idlethreshold", 4000);
1900     this.purgeTimeout = conf.getLong("hbase.ipc.client.call.purge.timeout",
1901       2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
1902     this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME, DEFAULT_WARN_RESPONSE_TIME);
1903     this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE, DEFAULT_WARN_RESPONSE_SIZE);
1904 
1905     // Start the listener here and let it bind to the port
1906     listener = new Listener(name);
1907     this.port = listener.getAddress().getPort();
1908 
1909     this.metrics = new MetricsHBaseServer(name, new MetricsHBaseServerWrapperImpl(this));
1910     this.tcpNoDelay = conf.getBoolean("hbase.ipc.server.tcpnodelay", true);
1911     this.tcpKeepAlive = conf.getBoolean("hbase.ipc.server.tcpkeepalive", true);
1912 
1913     this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS, DEFAULT_WARN_DELAYED_CALLS);
1914     this.delayedCalls = new AtomicInteger(0);
1915     this.ipcUtil = new IPCUtil(conf);
1916 
1917 
1918     // Create the responder here
1919     responder = new Responder();
1920     this.authorize = conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false);
1921     this.userProvider = UserProvider.instantiate(conf);
1922     this.isSecurityEnabled = userProvider.isHBaseSecurityEnabled();
1923     if (isSecurityEnabled) {
1924       HBaseSaslRpcServer.init(conf);
1925     }
1926     this.scheduler = scheduler;
1927     this.scheduler.init(new RpcSchedulerContext(this));
1928   }
1929 
1930   /**
1931    * Subclasses of HBaseServer can override this to provide their own
1932    * Connection implementations.
1933    */
1934   protected Connection getConnection(SocketChannel channel, long time) {
1935     return new Connection(channel, time);
1936   }
1937 
1938   /**
1939    * Setup response for the RPC Call.
1940    *
1941    * @param response buffer to serialize the response into
1942    * @param call {@link Call} to which we are setting up the response
1943    * @param error error message, if the call failed
1944    * @throws IOException
1945    */
1946   private void setupResponse(ByteArrayOutputStream response, Call call, Throwable t, String error)
1947   throws IOException {
1948     if (response != null) response.reset();
1949     call.setResponse(null, null, t, error);
1950   }
1951 
1952   protected void closeConnection(Connection connection) {
1953     synchronized (connectionList) {
1954       if (connectionList.remove(connection)) {
1955         numConnections--;
1956       }
1957     }
1958     connection.close();
1959   }
1960 
1961   Configuration getConf() {
1962     return conf;
1963   }
1964 
1965   /** Sets the socket buffer size used for responding to RPCs.
1966    * @param size send size
1967    */
1968   @Override
1969   public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
1970 
1971   @Override
1972   public boolean isStarted() {
1973     return this.started;
1974   }
1975 
1976   /** Starts the service.  Must be called before any calls will be handled. */
1977   @Override
1978   public synchronized void start() {
1979     if (started) return;
1980     AuthenticationTokenSecretManager mgr = createSecretManager();
1981     if (mgr != null) {
1982       setSecretManager(mgr);
1983       mgr.start();
1984     }
1985     this.authManager = new ServiceAuthorizationManager();
1986     HBasePolicyProvider.init(conf, authManager);
1987     responder.start();
1988     listener.start();
1989     scheduler.start();
1990     started = true;
1991   }
1992 
1993   @Override
1994   public void refreshAuthManager(PolicyProvider pp) {
1995     // Ignore warnings that this should be accessed in a static way instead of via an instance;
1996     // it'll break if you go via static route.
1997     this.authManager.refresh(this.conf, pp);
1998   }
1999 
2000   private AuthenticationTokenSecretManager createSecretManager() {
2001     if (!isSecurityEnabled) return null;
2002     if (server == null) return null;
2003     Configuration conf = server.getConfiguration();
2004     long keyUpdateInterval =
2005         conf.getLong("hbase.auth.key.update.interval", 24*60*60*1000);
2006     long maxAge =
2007         conf.getLong("hbase.auth.token.max.lifetime", 7*24*60*60*1000);
2008     return new AuthenticationTokenSecretManager(conf, server.getZooKeeper(),
2009         server.getServerName().toString(), keyUpdateInterval, maxAge);
2010   }
2011 
2012   public SecretManager<? extends TokenIdentifier> getSecretManager() {
2013     return this.secretManager;
2014   }
2015 
2016   @SuppressWarnings("unchecked")
2017   public void setSecretManager(SecretManager<? extends TokenIdentifier> secretManager) {
2018     this.secretManager = (SecretManager<TokenIdentifier>) secretManager;
2019   }
2020 
2021   /**
2022    * This is a server side method, which is invoked over RPC. On success
2023    * the return response has protobuf response payload. On failure, the
2024    * exception name and the stack trace are returned in the protobuf response.
2025    */
2026   @Override
2027   public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
2028       Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
2029   throws IOException {
2030     try {
2031       status.setRPC(md.getName(), new Object[]{param}, receiveTime);
2032       // TODO: Review after we add in encoded data blocks.
2033       status.setRPCPacket(param);
2034       status.resume("Servicing call");
2035       //get an instance of the method arg type
2036       long startTime = System.currentTimeMillis();
2037       PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cellScanner);
2038       Message result = service.callBlockingMethod(md, controller, param);
2039       int processingTime = (int) (System.currentTimeMillis() - startTime);
2040       int qTime = (int) (startTime - receiveTime);
2041       if (LOG.isTraceEnabled()) {
2042         LOG.trace(CurCall.get().toString() +
2043             ", response " + TextFormat.shortDebugString(result) +
2044             " queueTime: " + qTime +
2045             " processingTime: " + processingTime);
2046       }
2047       metrics.dequeuedCall(qTime);
2048       metrics.processedCall(processingTime);
2049       long responseSize = result.getSerializedSize();
2050       // log any RPC responses that are slower than the configured warn
2051       // response time or larger than configured warning size
2052       boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
2053       boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
2054       if (tooSlow || tooLarge) {
2055         // when tagging, we let TooLarge trump TooSmall to keep output simple
2056         // note that large responses will often also be slow.
2057         logResponse(new Object[]{param},
2058             md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
2059             (tooLarge ? "TooLarge" : "TooSlow"),
2060             status.getClient(), startTime, processingTime, qTime,
2061             responseSize);
2062       }
2063       return new Pair<Message, CellScanner>(result, controller.cellScanner());
2064     } catch (Throwable e) {
2065       // The above callBlockingMethod will always return a SE.  Strip the SE wrapper before
2066       // putting it on the wire.  Its needed to adhere to the pb Service Interface but we don't
2067       // need to pass it over the wire.
2068       if (e instanceof ServiceException) e = e.getCause();
2069       if (e instanceof LinkageError) throw new DoNotRetryIOException(e);
2070       if (e instanceof IOException) throw (IOException)e;
2071       LOG.error("Unexpected throwable object ", e);
2072       throw new IOException(e.getMessage(), e);
2073     }
2074   }
2075 
2076   /**
2077    * Logs an RPC response to the LOG file, producing valid JSON objects for
2078    * client Operations.
2079    * @param params The parameters received in the call.
2080    * @param methodName The name of the method invoked
2081    * @param call The string representation of the call
2082    * @param tag  The tag that will be used to indicate this event in the log.
2083    * @param clientAddress   The address of the client who made this call.
2084    * @param startTime       The time that the call was initiated, in ms.
2085    * @param processingTime  The duration that the call took to run, in ms.
2086    * @param qTime           The duration that the call spent on the queue
2087    *                        prior to being initiated, in ms.
2088    * @param responseSize    The size in bytes of the response buffer.
2089    */
2090   void logResponse(Object[] params, String methodName, String call, String tag,
2091       String clientAddress, long startTime, int processingTime, int qTime,
2092       long responseSize)
2093           throws IOException {
2094     // base information that is reported regardless of type of call
2095     Map<String, Object> responseInfo = new HashMap<String, Object>();
2096     responseInfo.put("starttimems", startTime);
2097     responseInfo.put("processingtimems", processingTime);
2098     responseInfo.put("queuetimems", qTime);
2099     responseInfo.put("responsesize", responseSize);
2100     responseInfo.put("client", clientAddress);
2101     responseInfo.put("class", server == null? "": server.getClass().getSimpleName());
2102     responseInfo.put("method", methodName);
2103     if (params.length == 2 && server instanceof HRegionServer &&
2104         params[0] instanceof byte[] &&
2105         params[1] instanceof Operation) {
2106       // if the slow process is a query, we want to log its table as well
2107       // as its own fingerprint
2108       TableName tableName = TableName.valueOf(
2109           HRegionInfo.parseRegionName((byte[]) params[0])[0]);
2110       responseInfo.put("table", tableName.getNameAsString());
2111       // annotate the response map with operation details
2112       responseInfo.putAll(((Operation) params[1]).toMap());
2113       // report to the log file
2114       LOG.warn("(operation" + tag + "): " +
2115                MAPPER.writeValueAsString(responseInfo));
2116     } else if (params.length == 1 && server instanceof HRegionServer &&
2117         params[0] instanceof Operation) {
2118       // annotate the response map with operation details
2119       responseInfo.putAll(((Operation) params[0]).toMap());
2120       // report to the log file
2121       LOG.warn("(operation" + tag + "): " +
2122                MAPPER.writeValueAsString(responseInfo));
2123     } else {
2124       // can't get JSON details, so just report call.toString() along with
2125       // a more generic tag.
2126       responseInfo.put("call", call);
2127       LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
2128     }
2129   }
2130 
2131   /** Stops the service.  No new calls will be handled after this is called. */
2132   @Override
2133   public synchronized void stop() {
2134     LOG.info("Stopping server on " + port);
2135     running = false;
2136     listener.interrupt();
2137     listener.doStop();
2138     responder.interrupt();
2139     scheduler.stop();
2140     notifyAll();
2141   }
2142 
2143   /** Wait for the server to be stopped.
2144    * Does not wait for all subthreads to finish.
2145    *  See {@link #stop()}.
2146    * @throws InterruptedException e
2147    */
2148   @Override
2149   public synchronized void join() throws InterruptedException {
2150     while (running) {
2151       wait();
2152     }
2153   }
2154 
2155   /**
2156    * Return the socket (ip+port) on which the RPC server is listening to.
2157    * @return the socket (ip+port) on which the RPC server is listening to.
2158    */
2159   @Override
2160   public synchronized InetSocketAddress getListenerAddress() {
2161     return listener.getAddress();
2162   }
2163 
2164   /**
2165    * Set the handler for calling out of RPC for error conditions.
2166    * @param handler the handler implementation
2167    */
2168   @Override
2169   public void setErrorHandler(HBaseRPCErrorHandler handler) {
2170     this.errorHandler = handler;
2171   }
2172 
2173   @Override
2174   public HBaseRPCErrorHandler getErrorHandler() {
2175     return this.errorHandler;
2176   }
2177 
2178   /**
2179    * Returns the metrics instance for reporting RPC call statistics
2180    */
2181   @Override
2182   public MetricsHBaseServer getMetrics() {
2183     return metrics;
2184   }
2185 
2186   @Override
2187   public void addCallSize(final long diff) {
2188     this.callQueueSize.add(diff);
2189   }
2190 
2191   /**
2192    * Authorize the incoming client connection.
2193    *
2194    * @param user client user
2195    * @param connection incoming connection
2196    * @param addr InetAddress of incoming connection
2197    * @throws org.apache.hadoop.security.authorize.AuthorizationException when the client isn't authorized to talk the protocol
2198    */
2199   public void authorize(UserGroupInformation user, ConnectionHeader connection, InetAddress addr)
2200   throws AuthorizationException {
2201     if (authorize) {
2202       Class<?> c = getServiceInterface(services, connection.getServiceName());
2203       this.authManager.authorize(user != null ? user : null, c, getConf(), addr);
2204     }
2205   }
2206 
2207   /**
2208    * When the read or write buffer size is larger than this limit, i/o will be
2209    * done in chunks of this size. Most RPC requests and responses would be
2210    * be smaller.
2211    */
2212   private static int NIO_BUFFER_LIMIT = 64 * 1024; //should not be more than 64KB.
2213 
2214   /**
2215    * This is a wrapper around {@link java.nio.channels.WritableByteChannel#write(java.nio.ByteBuffer)}.
2216    * If the amount of data is large, it writes to channel in smaller chunks.
2217    * This is to avoid jdk from creating many direct buffers as the size of
2218    * buffer increases. This also minimizes extra copies in NIO layer
2219    * as a result of multiple write operations required to write a large
2220    * buffer.
2221    *
2222    * @param channel writable byte channel to write to
2223    * @param bufferChain Chain of buffers to write
2224    * @return number of bytes written
2225    * @throws java.io.IOException e
2226    * @see java.nio.channels.WritableByteChannel#write(java.nio.ByteBuffer)
2227    */
2228   protected long channelWrite(GatheringByteChannel channel, BufferChain bufferChain)
2229   throws IOException {
2230     long count =  bufferChain.write(channel, NIO_BUFFER_LIMIT);
2231     if (count > 0) this.metrics.sentBytes(count);
2232     return count;
2233   }
2234 
2235   /**
2236    * This is a wrapper around {@link java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)}.
2237    * If the amount of data is large, it writes to channel in smaller chunks.
2238    * This is to avoid jdk from creating many direct buffers as the size of
2239    * ByteBuffer increases. There should not be any performance degredation.
2240    *
2241    * @param channel writable byte channel to write on
2242    * @param buffer buffer to write
2243    * @return number of bytes written
2244    * @throws java.io.IOException e
2245    * @see java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)
2246    */
2247   protected int channelRead(ReadableByteChannel channel,
2248                                    ByteBuffer buffer) throws IOException {
2249 
2250     int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
2251            channel.read(buffer) : channelIO(channel, null, buffer);
2252     if (count > 0) {
2253       metrics.receivedBytes(count);
2254     }
2255     return count;
2256   }
2257 
2258   /**
2259    * Helper for {@link #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)}
2260    * and {@link #channelWrite(GatheringByteChannel, BufferChain)}. Only
2261    * one of readCh or writeCh should be non-null.
2262    *
2263    * @param readCh read channel
2264    * @param writeCh write channel
2265    * @param buf buffer to read or write into/out of
2266    * @return bytes written
2267    * @throws java.io.IOException e
2268    * @see #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)
2269    * @see #channelWrite(GatheringByteChannel, BufferChain)
2270    */
2271   private static int channelIO(ReadableByteChannel readCh,
2272                                WritableByteChannel writeCh,
2273                                ByteBuffer buf) throws IOException {
2274 
2275     int originalLimit = buf.limit();
2276     int initialRemaining = buf.remaining();
2277     int ret = 0;
2278 
2279     while (buf.remaining() > 0) {
2280       try {
2281         int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
2282         buf.limit(buf.position() + ioSize);
2283 
2284         ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
2285 
2286         if (ret < ioSize) {
2287           break;
2288         }
2289 
2290       } finally {
2291         buf.limit(originalLimit);
2292       }
2293     }
2294 
2295     int nBytes = initialRemaining - buf.remaining();
2296     return (nBytes > 0) ? nBytes : ret;
2297   }
2298 
2299   /**
2300    * Needed for features such as delayed calls.  We need to be able to store the current call
2301    * so that we can complete it later or ask questions of what is supported by the current ongoing
2302    * call.
2303    * @return An RpcCallContext backed by the currently ongoing call (gotten from a thread local)
2304    */
2305   public static RpcCallContext getCurrentCall() {
2306     return CurCall.get();
2307   }
2308 
2309   /**
2310    * @param serviceName Some arbitrary string that represents a 'service'.
2311    * @param services Available service instances
2312    * @return Matching BlockingServiceAndInterface pair
2313    */
2314   static BlockingServiceAndInterface getServiceAndInterface(
2315       final List<BlockingServiceAndInterface> services, final String serviceName) {
2316     for (BlockingServiceAndInterface bs : services) {
2317       if (bs.getBlockingService().getDescriptorForType().getName().equals(serviceName)) {
2318         return bs;
2319       }
2320     }
2321     return null;
2322   }
2323 
2324   /**
2325    * @param serviceName Some arbitrary string that represents a 'service'.
2326    * @param services Available services and their service interfaces.
2327    * @return Service interface class for <code>serviceName</code>
2328    */
2329   static Class<?> getServiceInterface(
2330       final List<BlockingServiceAndInterface> services,
2331       final String serviceName) {
2332     BlockingServiceAndInterface bsasi =
2333         getServiceAndInterface(services, serviceName);
2334     return bsasi == null? null: bsasi.getServiceInterface();
2335   }
2336 
2337   /**
2338    * @param serviceName Some arbitrary string that represents a 'service'.
2339    * @param services Available services and their service interfaces.
2340    * @return BlockingService that goes with the passed <code>serviceName</code>
2341    */
2342   static BlockingService getService(
2343       final List<BlockingServiceAndInterface> services,
2344       final String serviceName) {
2345     BlockingServiceAndInterface bsasi =
2346         getServiceAndInterface(services, serviceName);
2347     return bsasi == null? null: bsasi.getBlockingService();
2348   }
2349 
2350   /** Returns the remote side ip address when invoked inside an RPC
2351    *  Returns null incase of an error.
2352    *  @return InetAddress
2353    */
2354   public static InetAddress getRemoteIp() {
2355     Call call = CurCall.get();
2356     if (call != null && call.connection.socket != null) {
2357       return call.connection.socket.getInetAddress();
2358     }
2359     return null;
2360   }
2361 
2362 
2363   /**
2364    * A convenience method to bind to a given address and report
2365    * better exceptions if the address is not a valid host.
2366    * @param socket the socket to bind
2367    * @param address the address to bind to
2368    * @param backlog the number of connections allowed in the queue
2369    * @throws BindException if the address can't be bound
2370    * @throws UnknownHostException if the address isn't a valid host name
2371    * @throws IOException other random errors from bind
2372    */
2373   public static void bind(ServerSocket socket, InetSocketAddress address,
2374                           int backlog) throws IOException {
2375     try {
2376       socket.bind(address, backlog);
2377     } catch (BindException e) {
2378       BindException bindException =
2379         new BindException("Problem binding to " + address + " : " +
2380             e.getMessage());
2381       bindException.initCause(e);
2382       throw bindException;
2383     } catch (SocketException e) {
2384       // If they try to bind to a different host's address, give a better
2385       // error message.
2386       if ("Unresolved address".equals(e.getMessage())) {
2387         throw new UnknownHostException("Invalid hostname for server: " +
2388                                        address.getHostName());
2389       }
2390       throw e;
2391     }
2392   }
2393 
2394   @Override
2395   public RpcScheduler getScheduler() {
2396     return scheduler;
2397   }
2398 }