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