View Javadoc

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