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