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