View Javadoc

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