View Javadoc

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