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