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