View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.ipc;
21  
22  import java.io.BufferedInputStream;
23  import java.io.BufferedOutputStream;
24  import java.io.DataInputStream;
25  import java.io.DataOutputStream;
26  import java.io.FilterInputStream;
27  import java.io.IOException;
28  import java.io.InputStream;
29  import java.io.InterruptedIOException;
30  import java.io.OutputStream;
31  import java.net.ConnectException;
32  import java.net.InetSocketAddress;
33  import java.net.Socket;
34  import java.net.SocketAddress;
35  import java.net.SocketException;
36  import java.net.SocketTimeoutException;
37  import java.net.UnknownHostException;
38  import java.nio.ByteBuffer;
39  import java.security.PrivilegedExceptionAction;
40  import java.util.HashMap;
41  import java.util.Iterator;
42  import java.util.LinkedList;
43  import java.util.Map;
44  import java.util.Map.Entry;
45  import java.util.Random;
46  import java.util.concurrent.ConcurrentSkipListMap;
47  import java.util.concurrent.atomic.AtomicBoolean;
48  import java.util.concurrent.atomic.AtomicLong;
49  
50  import javax.net.SocketFactory;
51  import javax.security.sasl.SaslException;
52  
53  import org.apache.commons.logging.Log;
54  import org.apache.commons.logging.LogFactory;
55  import org.apache.hadoop.classification.InterfaceAudience;
56  import org.apache.hadoop.classification.InterfaceStability;
57  import org.apache.hadoop.conf.Configuration;
58  import org.apache.hadoop.hbase.CellScanner;
59  import org.apache.hadoop.hbase.DoNotRetryIOException;
60  import org.apache.hadoop.hbase.HBaseIOException;
61  import org.apache.hadoop.hbase.HConstants;
62  import org.apache.hadoop.hbase.ServerName;
63  import org.apache.hadoop.hbase.codec.Codec;
64  import org.apache.hadoop.hbase.codec.KeyValueCodec;
65  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
66  import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
67  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
68  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
69  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
70  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
71  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
72  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
73  import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo;
74  import org.apache.hadoop.hbase.security.AuthMethod;
75  import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
76  import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
77  import org.apache.hadoop.hbase.security.SecurityInfo;
78  import org.apache.hadoop.hbase.security.User;
79  import org.apache.hadoop.hbase.security.UserProvider;
80  import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
81  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
82  import org.apache.hadoop.hbase.util.ExceptionUtil;
83  import org.apache.hadoop.hbase.util.Pair;
84  import org.apache.hadoop.hbase.util.PoolMap;
85  import org.apache.hadoop.hbase.util.PoolMap.PoolType;
86  import org.apache.hadoop.io.IOUtils;
87  import org.apache.hadoop.io.Text;
88  import org.apache.hadoop.io.compress.CompressionCodec;
89  import org.apache.hadoop.ipc.RemoteException;
90  import org.apache.hadoop.net.NetUtils;
91  import org.apache.hadoop.security.SecurityUtil;
92  import org.apache.hadoop.security.UserGroupInformation;
93  import org.apache.hadoop.security.token.Token;
94  import org.apache.hadoop.security.token.TokenIdentifier;
95  import org.apache.hadoop.security.token.TokenSelector;
96  import org.cloudera.htrace.Span;
97  import org.cloudera.htrace.Trace;
98  
99  import com.google.common.annotations.VisibleForTesting;
100 import com.google.protobuf.BlockingRpcChannel;
101 import com.google.protobuf.Descriptors.MethodDescriptor;
102 import com.google.protobuf.Message;
103 import com.google.protobuf.Message.Builder;
104 import com.google.protobuf.RpcController;
105 import com.google.protobuf.ServiceException;
106 import com.google.protobuf.TextFormat;
107 
108 
109 /**
110  * Does RPC against a cluster.  Manages connections per regionserver in the cluster.
111  * <p>See HBaseServer
112  */
113 @InterfaceAudience.Private
114 public class RpcClient {
115   // The LOG key is intentionally not from this package to avoid ipc logging at DEBUG (all under
116   // o.a.h.hbase is set to DEBUG as default).
117   public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.RpcClient");
118   protected final PoolMap<ConnectionId, Connection> connections;
119 
120   protected int counter;                            // counter for call ids
121   protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs
122   final protected Configuration conf;
123   final protected int maxIdleTime; // connections will be culled if it was idle for
124                            // maxIdleTime microsecs
125   final protected int maxRetries; //the max. no. of retries for socket connections
126   final protected long failureSleep; // Time to sleep before retry on failure.
127   protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
128   protected final boolean tcpKeepAlive; // if T then use keepalives
129   protected int pingInterval; // how often sends ping to the server in msecs
130   protected FailedServers failedServers;
131   private final Codec codec;
132   private final CompressionCodec compressor;
133   private final IPCUtil ipcUtil;
134 
135   protected final SocketFactory socketFactory;           // how to create sockets
136   protected String clusterId;
137   protected final SocketAddress localAddr;
138 
139   private final boolean fallbackAllowed;
140   private UserProvider userProvider;
141 
142   final private static String PING_INTERVAL_NAME = "ipc.ping.interval";
143   final private static String SOCKET_TIMEOUT = "ipc.socket.timeout";
144   final static int DEFAULT_PING_INTERVAL = 60000;  // 1 min
145   final static int DEFAULT_SOCKET_TIMEOUT = 20000; // 20 seconds
146   final static int PING_CALL_ID = -1;
147 
148   public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
149   public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
150 
151   public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY =
152       "hbase.ipc.client.fallback-to-simple-auth-allowed";
153   public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
154 
155   // thread-specific RPC timeout, which may override that of what was passed in.
156   // This is used to change dynamically the timeout (for read only) when retrying: if
157   //  the time allowed for the operation is less than the usual socket timeout, then
158   //  we lower the timeout. This is subject to race conditions, and should be used with
159   //  extreme caution.
160   private static ThreadLocal<Integer> rpcTimeout = new ThreadLocal<Integer>() {
161     @Override
162     protected Integer initialValue() {
163       return HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
164     }
165   };
166 
167   /**
168    * A class to manage a list of servers that failed recently.
169    */
170   static class FailedServers {
171     private final LinkedList<Pair<Long, String>> failedServers = new
172         LinkedList<Pair<Long, java.lang.String>>();
173     private final int recheckServersTimeout;
174 
175     FailedServers(Configuration conf) {
176       this.recheckServersTimeout = conf.getInt(
177           FAILED_SERVER_EXPIRY_KEY, FAILED_SERVER_EXPIRY_DEFAULT);
178     }
179 
180     /**
181      * Add an address to the list of the failed servers list.
182      */
183     public synchronized void addToFailedServers(InetSocketAddress address) {
184       final long expiry = EnvironmentEdgeManager.currentTimeMillis() + recheckServersTimeout;
185       failedServers.addFirst(new Pair<Long, String>(expiry, address.toString()));
186     }
187 
188     /**
189      * Check if the server should be considered as bad. Clean the old entries of the list.
190      *
191      * @return true if the server is in the failed servers list
192      */
193     public synchronized boolean isFailedServer(final InetSocketAddress address) {
194       if (failedServers.isEmpty()) {
195         return false;
196       }
197 
198       final String lookup = address.toString();
199       final long now = EnvironmentEdgeManager.currentTimeMillis();
200 
201       // iterate, looking for the search entry and cleaning expired entries
202       Iterator<Pair<Long, String>> it = failedServers.iterator();
203       while (it.hasNext()) {
204         Pair<Long, String> cur = it.next();
205         if (cur.getFirst() < now) {
206           it.remove();
207         } else {
208           if (lookup.equals(cur.getSecond())) {
209             return true;
210           }
211         }
212       }
213 
214       return false;
215     }
216   }
217 
218   @SuppressWarnings("serial")
219   @InterfaceAudience.Public
220   @InterfaceStability.Evolving
221   // Shouldn't this be a DoNotRetryException? St.Ack 10/2/2013
222   public static class FailedServerException extends HBaseIOException {
223     public FailedServerException(String s) {
224       super(s);
225     }
226   }
227 
228   /**
229    * set the ping interval value in configuration
230    *
231    * @param conf Configuration
232    * @param pingInterval the ping interval
233    */
234   // Any reason we couldn't just do tcp keepalive instead of this pingery?
235   // St.Ack 20130121
236   public static void setPingInterval(Configuration conf, int pingInterval) {
237     conf.setInt(PING_INTERVAL_NAME, pingInterval);
238   }
239 
240   /**
241    * Get the ping interval from configuration;
242    * If not set in the configuration, return the default value.
243    *
244    * @param conf Configuration
245    * @return the ping interval
246    */
247   static int getPingInterval(Configuration conf) {
248     return conf.getInt(PING_INTERVAL_NAME, DEFAULT_PING_INTERVAL);
249   }
250 
251   /**
252    * Set the socket timeout
253    * @param conf Configuration
254    * @param socketTimeout the socket timeout
255    */
256   public static void setSocketTimeout(Configuration conf, int socketTimeout) {
257     conf.setInt(SOCKET_TIMEOUT, socketTimeout);
258   }
259 
260   /**
261    * @return the socket timeout
262    */
263   static int getSocketTimeout(Configuration conf) {
264     return conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT);
265   }
266 
267   /** A call waiting for a value. */
268   protected class Call {
269     final int id;                                 // call id
270     final Message param;                          // rpc request method param object
271     /**
272      * Optionally has cells when making call.  Optionally has cells set on response.  Used
273      * passing cells to the rpc and receiving the response.
274      */
275     CellScanner cells;
276     Message response;                             // value, null if error
277     // The return type.  Used to create shell into which we deserialize the response if any.
278     Message responseDefaultType;
279     IOException error;                            // exception, null if value
280     boolean done;                                 // true when call is done
281     long startTime;
282     final MethodDescriptor md;
283 
284     protected Call(final MethodDescriptor md, Message param, final CellScanner cells,
285         final Message responseDefaultType) {
286       this.param = param;
287       this.md = md;
288       this.cells = cells;
289       this.startTime = System.currentTimeMillis();
290       this.responseDefaultType = responseDefaultType;
291       synchronized (RpcClient.this) {
292         this.id = counter++;
293       }
294     }
295 
296     @Override
297     public String toString() {
298       return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" +
299         (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") + "}";
300     }
301 
302     /** Indicate when the call is complete and the
303      * value or error are available.  Notifies by default.  */
304     protected synchronized void callComplete() {
305       this.done = true;
306       notify();                                 // notify caller
307     }
308 
309     /** Set the exception when there is an error.
310      * Notify the caller the call is done.
311      *
312      * @param error exception thrown by the call; either local or remote
313      */
314     public void setException(IOException error) {
315       this.error = error;
316       callComplete();
317     }
318 
319     /**
320      * Set the return value when there is no error.
321      * Notify the caller the call is done.
322      *
323      * @param response return value of the call.
324      * @param cells Can be null
325      */
326     public void setResponse(Message response, final CellScanner cells) {
327       this.response = response;
328       this.cells = cells;
329       callComplete();
330     }
331 
332     public long getStartTime() {
333       return this.startTime;
334     }
335   }
336 
337   protected final static Map<AuthenticationProtos.TokenIdentifier.Kind,
338       TokenSelector<? extends TokenIdentifier>> tokenHandlers =
339       new HashMap<AuthenticationProtos.TokenIdentifier.Kind, TokenSelector<? extends TokenIdentifier>>();
340   static {
341     tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
342         new AuthenticationTokenSelector());
343   }
344 
345   /**
346    * Creates a connection. Can be overridden by a subclass for testing.
347    * @param remoteId - the ConnectionId to use for the connection creation.
348    */
349   protected Connection createConnection(ConnectionId remoteId, final Codec codec,
350       final CompressionCodec compressor)
351   throws IOException {
352     return new Connection(remoteId, codec, compressor);
353   }
354 
355   /** Thread that reads responses and notifies callers.  Each connection owns a
356    * socket connected to a remote address.  Calls are multiplexed through this
357    * socket: responses may be delivered out of order. */
358   protected class Connection extends Thread {
359     private ConnectionHeader header;              // connection header
360     protected ConnectionId remoteId;
361     protected Socket socket = null;                 // connected socket
362     protected DataInputStream in;
363     protected DataOutputStream out;
364     private InetSocketAddress server;             // server ip:port
365     private String serverPrincipal;  // server's krb5 principal name
366     private AuthMethod authMethod; // authentication method
367     private boolean useSasl;
368     private Token<? extends TokenIdentifier> token;
369     private HBaseSaslRpcClient saslRpcClient;
370     private int reloginMaxBackoff; // max pause before relogin on sasl failure
371     private final Codec codec;
372     private final CompressionCodec compressor;
373 
374     // currently active calls
375     protected final ConcurrentSkipListMap<Integer, Call> calls =
376       new ConcurrentSkipListMap<Integer, Call>();
377     protected final AtomicLong lastActivity =
378       new AtomicLong(); // last I/O activity time
379     protected final AtomicBoolean shouldCloseConnection =
380       new AtomicBoolean();  // indicate if the connection is closed
381     protected IOException closeException; // close reason
382 
383     Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
384     throws IOException {
385       if (remoteId.getAddress().isUnresolved()) {
386         throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
387       }
388       this.server = remoteId.getAddress();
389       this.codec = codec;
390       this.compressor = compressor;
391 
392       UserGroupInformation ticket = remoteId.getTicket().getUGI();
393       SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
394       this.useSasl = userProvider.isHBaseSecurityEnabled();
395       if (useSasl && securityInfo != null) {
396         AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
397         if (tokenKind != null) {
398           TokenSelector<? extends TokenIdentifier> tokenSelector =
399               tokenHandlers.get(tokenKind);
400           if (tokenSelector != null) {
401             token = tokenSelector.selectToken(new Text(clusterId),
402                 ticket.getTokens());
403           } else if (LOG.isDebugEnabled()) {
404             LOG.debug("No token selector found for type "+tokenKind);
405           }
406         }
407         String serverKey = securityInfo.getServerPrincipal();
408         if (serverKey == null) {
409           throw new IOException(
410               "Can't obtain server Kerberos config key from SecurityInfo");
411         }
412         serverPrincipal = SecurityUtil.getServerPrincipal(
413             conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
414         if (LOG.isDebugEnabled()) {
415           LOG.debug("RPC Server Kerberos principal name for service="
416               + remoteId.getServiceName() + " is " + serverPrincipal);
417         }
418       }
419 
420       if (!useSasl) {
421         authMethod = AuthMethod.SIMPLE;
422       } else if (token != null) {
423         authMethod = AuthMethod.DIGEST;
424       } else {
425         authMethod = AuthMethod.KERBEROS;
426       }
427 
428       if (LOG.isDebugEnabled()) {
429         LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName +
430           ", sasl=" + useSasl);
431       }
432       reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
433       this.remoteId = remoteId;
434 
435       ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
436       builder.setServiceName(remoteId.getServiceName());
437       UserInformation userInfoPB;
438       if ((userInfoPB = getUserInfo(ticket)) != null) {
439         builder.setUserInfo(userInfoPB);
440       }
441       if (this.codec != null) {
442         builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
443       }
444       if (this.compressor != null) {
445         builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
446       }
447       this.header = builder.build();
448 
449       this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
450         remoteId.getAddress().toString() +
451         ((ticket==null)?" from an unknown user": (" from "
452         + ticket.getUserName())));
453       this.setDaemon(true);
454     }
455 
456     private UserInformation getUserInfo(UserGroupInformation ugi) {
457       if (ugi == null || authMethod == AuthMethod.DIGEST) {
458         // Don't send user for token auth
459         return null;
460       }
461       UserInformation.Builder userInfoPB = UserInformation.newBuilder();
462       if (authMethod == AuthMethod.KERBEROS) {
463         // Send effective user for Kerberos auth
464         userInfoPB.setEffectiveUser(ugi.getUserName());
465       } else if (authMethod == AuthMethod.SIMPLE) {
466         //Send both effective user and real user for simple auth
467         userInfoPB.setEffectiveUser(ugi.getUserName());
468         if (ugi.getRealUser() != null) {
469           userInfoPB.setRealUser(ugi.getRealUser().getUserName());
470         }
471       }
472       return userInfoPB.build();
473     }
474 
475     /** Update lastActivity with the current time. */
476     protected void touch() {
477       lastActivity.set(System.currentTimeMillis());
478     }
479 
480     /**
481      * Add a call to this connection's call queue and notify
482      * a listener; synchronized. If the connection is dead, the call is not added, and the
483      * caller is notified.
484      * This function can return a connection that is already marked as 'shouldCloseConnection'
485      *  It is up to the user code to check this status.
486      * @param call to add
487      */
488     @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
489       justification="Notify because new call available for processing")
490     protected synchronized void addCall(Call call) {
491       // If the connection is about to close, we manage this as if the call was already added
492       //  to the connection calls list. If not, the connection creations are serialized, as
493       //  mentioned in HBASE-6364
494       if (this.shouldCloseConnection.get()) {
495         if (this.closeException == null) {
496           call.setException(new IOException(
497               "Call " + call.id + " not added as the connection " + remoteId + " is closing"));
498         } else {
499           call.setException(this.closeException);
500         }
501         synchronized (call) {
502           call.notifyAll();
503         }
504       } else {
505         calls.put(call.id, call);
506         synchronized (call) {
507           notify();
508         }
509       }
510     }
511 
512     /** This class sends a ping to the remote side when timeout on
513      * reading. If no failure is detected, it retries until at least
514      * a byte is read.
515      */
516     protected class PingInputStream extends FilterInputStream {
517       /* constructor */
518       protected PingInputStream(InputStream in) {
519         super(in);
520       }
521 
522       /* Process timeout exception
523        * if the connection is not going to be closed, send a ping.
524        * otherwise, throw the timeout exception.
525        */
526       private void handleTimeout(SocketTimeoutException e) throws IOException {
527         if (shouldCloseConnection.get() || !running.get() || remoteId.rpcTimeout > 0) {
528           throw e;
529         }
530         sendPing();
531       }
532 
533       /** Read a byte from the stream.
534        * Send a ping if timeout on read. Retries if no failure is detected
535        * until a byte is read.
536        * @throws IOException for any IO problem other than socket timeout
537        */
538       @Override
539       public int read() throws IOException {
540         do {
541           try {
542             return super.read();
543           } catch (SocketTimeoutException e) {
544             handleTimeout(e);
545           }
546         } while (true);
547       }
548 
549       /** Read bytes into a buffer starting from offset <code>off</code>
550        * Send a ping if timeout on read. Retries if no failure is detected
551        * until a byte is read.
552        *
553        * @return the total number of bytes read; -1 if the connection is closed.
554        */
555       @Override
556       public int read(byte[] buf, int off, int len) throws IOException {
557         do {
558           try {
559             return super.read(buf, off, len);
560           } catch (SocketTimeoutException e) {
561             handleTimeout(e);
562           }
563         } while (true);
564       }
565     }
566 
567     protected synchronized void setupConnection() throws IOException {
568       short ioFailures = 0;
569       short timeoutFailures = 0;
570       while (true) {
571         try {
572           this.socket = socketFactory.createSocket();
573           this.socket.setTcpNoDelay(tcpNoDelay);
574           this.socket.setKeepAlive(tcpKeepAlive);
575           if (localAddr != null) {
576             this.socket.bind(localAddr);
577           }
578           // connection time out is 20s
579           NetUtils.connect(this.socket, remoteId.getAddress(),
580               getSocketTimeout(conf));
581           if (remoteId.rpcTimeout > 0) {
582             pingInterval = remoteId.rpcTimeout; // overwrite pingInterval
583           }
584           this.socket.setSoTimeout(pingInterval);
585           return;
586         } catch (SocketTimeoutException toe) {
587           /* The max number of retries is 45,
588            * which amounts to 20s*45 = 15 minutes retries.
589            */
590           handleConnectionFailure(timeoutFailures++, maxRetries, toe);
591         } catch (IOException ie) {
592           handleConnectionFailure(ioFailures++, maxRetries, ie);
593         }
594       }
595     }
596 
597     protected void closeConnection() {
598       if (socket == null) {
599         return;
600       }
601 
602       // close the current connection
603       try {
604         if (socket.getOutputStream() != null) {
605           socket.getOutputStream().close();
606         }
607       } catch (IOException ignored) {  // Can happen if the socket is already closed
608       }
609       try {
610         if (socket.getInputStream() != null) {
611           socket.getInputStream().close();
612         }
613       } catch (IOException ignored) {  // Can happen if the socket is already closed
614       }
615       try {
616         if (socket.getChannel() != null) {
617           socket.getChannel().close();
618         }
619       } catch (IOException ignored) {  // Can happen if the socket is already closed
620       }
621       try {
622         socket.close();
623       } catch (IOException e) {
624         LOG.warn("Not able to close a socket", e);
625       }
626 
627       // set socket to null so that the next call to setupIOstreams
628       // can start the process of connect all over again.
629       socket = null;
630     }
631 
632     /**
633      *  Handle connection failures
634      *
635      * If the current number of retries is equal to the max number of retries,
636      * stop retrying and throw the exception; Otherwise backoff N seconds and
637      * try connecting again.
638      *
639      * This Method is only called from inside setupIOstreams(), which is
640      * synchronized. Hence the sleep is synchronized; the locks will be retained.
641      *
642      * @param curRetries current number of retries
643      * @param maxRetries max number of retries allowed
644      * @param ioe failure reason
645      * @throws IOException if max number of retries is reached
646      */
647     private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
648     throws IOException {
649       closeConnection();
650 
651       // throw the exception if the maximum number of retries is reached
652       if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {
653         throw ioe;
654       }
655 
656       // otherwise back off and retry
657       try {
658         Thread.sleep(failureSleep);
659       } catch (InterruptedException ie) {
660         ExceptionUtil.rethrowIfInterrupt(ie);
661       }
662 
663       LOG.info("Retrying connect to server: " + remoteId.getAddress() +
664         " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
665         " time(s).");
666     }
667 
668     /* wait till someone signals us to start reading RPC response or
669      * it is idle too long, it is marked as to be closed,
670      * or the client is marked as not running.
671      *
672      * Return true if it is time to read a response; false otherwise.
673      */
674     protected synchronized boolean waitForWork() {
675       if (calls.isEmpty() && !shouldCloseConnection.get()  && running.get())  {
676         long timeout = maxIdleTime - (System.currentTimeMillis()-lastActivity.get());
677         if (timeout>0) {
678           try {
679             wait(timeout);
680           } catch (InterruptedException ie) {
681             Thread.currentThread().interrupt();
682           }
683         }
684       }
685 
686       if (!calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
687         return true;
688       } else if (shouldCloseConnection.get()) {
689         return false;
690       } else if (calls.isEmpty()) { // idle connection closed or stopped
691         markClosed(null);
692         return false;
693       } else { // get stopped but there are still pending requests
694         markClosed((IOException)new IOException().initCause(
695             new InterruptedException()));
696         return false;
697       }
698     }
699 
700     public InetSocketAddress getRemoteAddress() {
701       return remoteId.getAddress();
702     }
703 
704     /* Send a ping to the server if the time elapsed
705      * since last I/O activity is equal to or greater than the ping interval
706      */
707     protected synchronized void sendPing() throws IOException {
708       // Can we do tcp keepalive instead of this pinging?
709       long curTime = System.currentTimeMillis();
710       if ( curTime - lastActivity.get() >= pingInterval) {
711         lastActivity.set(curTime);
712         //noinspection SynchronizeOnNonFinalField
713         synchronized (this.out) {
714           out.writeInt(PING_CALL_ID);
715           out.flush();
716         }
717       }
718     }
719 
720     @Override
721     public void run() {
722       if (LOG.isDebugEnabled()) {
723         LOG.debug(getName() + ": starting, connections " + connections.size());
724       }
725 
726       try {
727         while (waitForWork()) { // Wait here for work - read or close connection
728           readResponse();
729         }
730       } catch (Throwable t) {
731         LOG.warn(getName() + ": unexpected exception receiving call responses", t);
732         markClosed(new IOException("Unexpected exception receiving call responses", t));
733       }
734 
735       close();
736 
737       if (LOG.isDebugEnabled())
738         LOG.debug(getName() + ": stopped, connections " + connections.size());
739     }
740 
741     private synchronized void disposeSasl() {
742       if (saslRpcClient != null) {
743         try {
744           saslRpcClient.dispose();
745           saslRpcClient = null;
746         } catch (IOException ioe) {
747           LOG.error("Error disposing of SASL client", ioe);
748         }
749       }
750     }
751 
752     private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
753       UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
754       UserGroupInformation currentUser =
755         UserGroupInformation.getCurrentUser();
756       UserGroupInformation realUser = currentUser.getRealUser();
757       return authMethod == AuthMethod.KERBEROS &&
758           loginUser != null &&
759           //Make sure user logged in using Kerberos either keytab or TGT
760           loginUser.hasKerberosCredentials() &&
761           // relogin only in case it is the login user (e.g. JT)
762           // or superuser (like oozie).
763           (loginUser.equals(currentUser) || loginUser.equals(realUser));
764     }
765 
766     private synchronized boolean setupSaslConnection(final InputStream in2,
767         final OutputStream out2) throws IOException {
768       saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed,
769           conf.get("hbase.rpc.protection", 
770               QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
771       return saslRpcClient.saslConnect(in2, out2);
772     }
773 
774     /**
775      * If multiple clients with the same principal try to connect
776      * to the same server at the same time, the server assumes a
777      * replay attack is in progress. This is a feature of kerberos.
778      * In order to work around this, what is done is that the client
779      * backs off randomly and tries to initiate the connection
780      * again.
781      * The other problem is to do with ticket expiry. To handle that,
782      * a relogin is attempted.
783      * <p>
784      * The retry logic is governed by the {@link #shouldAuthenticateOverKrb}
785      * method. In case when the user doesn't have valid credentials, we don't
786      * need to retry (from cache or ticket). In such cases, it is prudent to
787      * throw a runtime exception when we receive a SaslException from the
788      * underlying authentication implementation, so there is no retry from
789      * other high level (for eg, HCM or HBaseAdmin).
790      * </p>
791      */
792     private synchronized void handleSaslConnectionFailure(
793         final int currRetries,
794         final int maxRetries, final Exception ex, final Random rand,
795         final UserGroupInformation user)
796     throws IOException, InterruptedException{
797       user.doAs(new PrivilegedExceptionAction<Object>() {
798         public Object run() throws IOException, InterruptedException {
799           closeConnection();
800           if (shouldAuthenticateOverKrb()) {
801             if (currRetries < maxRetries) {
802               LOG.debug("Exception encountered while connecting to " +
803                   "the server : " + ex);
804               //try re-login
805               if (UserGroupInformation.isLoginKeytabBased()) {
806                 UserGroupInformation.getLoginUser().reloginFromKeytab();
807               } else {
808                 UserGroupInformation.getLoginUser().reloginFromTicketCache();
809               }
810               disposeSasl();
811               //have granularity of milliseconds
812               //we are sleeping with the Connection lock held but since this
813               //connection instance is being used for connecting to the server
814               //in question, it is okay
815               Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1));
816               return null;
817             } else {
818               String msg = "Couldn't setup connection for " +
819               UserGroupInformation.getLoginUser().getUserName() +
820               " to " + serverPrincipal;
821               LOG.warn(msg);
822               throw (IOException) new IOException(msg).initCause(ex);
823             }
824           } else {
825             LOG.warn("Exception encountered while connecting to " +
826                 "the server : " + ex);
827           }
828           if (ex instanceof RemoteException) {
829             throw (RemoteException)ex;
830           }
831           if (ex instanceof SaslException) {
832             String msg = "SASL authentication failed." +
833               " The most likely cause is missing or invalid credentials." +
834               " Consider 'kinit'.";
835             LOG.fatal(msg, ex);
836             throw new RuntimeException(msg, ex);
837           }
838           throw new IOException(ex);
839         }
840       });
841     }
842 
843     protected synchronized void setupIOstreams()
844     throws IOException, InterruptedException {
845       if (socket != null || shouldCloseConnection.get()) {
846         return;
847       }
848 
849       if (failedServers.isFailedServer(remoteId.getAddress())) {
850         if (LOG.isDebugEnabled()) {
851           LOG.debug("Not trying to connect to " + server +
852               " this server is in the failed servers list");
853         }
854         IOException e = new FailedServerException(
855             "This server is in the failed servers list: " + server);
856         markClosed(e);
857         close();
858         throw e;
859       }
860 
861       try {
862         if (LOG.isDebugEnabled()) {
863           LOG.debug("Connecting to " + server);
864         }
865         short numRetries = 0;
866         final short MAX_RETRIES = 5;
867         Random rand = null;
868         while (true) {
869           setupConnection();
870           InputStream inStream = NetUtils.getInputStream(socket);
871           // This creates a socket with a write timeout. This timeout cannot be changed,
872           //  RpcClient allows to change the timeout dynamically, but we can only
873           //  change the read timeout today.
874           OutputStream outStream = NetUtils.getOutputStream(socket, pingInterval);
875           // Write out the preamble -- MAGIC, version, and auth to use.
876           writeConnectionHeaderPreamble(outStream);
877           if (useSasl) {
878             final InputStream in2 = inStream;
879             final OutputStream out2 = outStream;
880             UserGroupInformation ticket = remoteId.getTicket().getUGI();
881             if (authMethod == AuthMethod.KERBEROS) {
882               if (ticket != null && ticket.getRealUser() != null) {
883                 ticket = ticket.getRealUser();
884               }
885             }
886             boolean continueSasl = false;
887             if (ticket == null) throw new FatalConnectionException("ticket/user is null");
888             try {
889               continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
890                 @Override
891                 public Boolean run() throws IOException {
892                   return setupSaslConnection(in2, out2);
893                 }
894               });
895             } catch (Exception ex) {
896               if (rand == null) {
897                 rand = new Random();
898               }
899               handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket);
900               continue;
901             }
902             if (continueSasl) {
903               // Sasl connect is successful. Let's set up Sasl i/o streams.
904               inStream = saslRpcClient.getInputStream(inStream);
905               outStream = saslRpcClient.getOutputStream(outStream);
906             } else {
907               // fall back to simple auth because server told us so.
908               authMethod = AuthMethod.SIMPLE;
909               useSasl = false;
910             }
911           }
912           this.in = new DataInputStream(new BufferedInputStream(new PingInputStream(inStream)));
913           this.out = new DataOutputStream(new BufferedOutputStream(outStream));
914           // Now write out the connection header
915           writeConnectionHeader();
916 
917           // update last activity time
918           touch();
919 
920           // start the receiver thread after the socket connection has been set up
921           start();
922           return;
923         }
924       } catch (Throwable t) {
925         failedServers.addToFailedServers(remoteId.address);
926         IOException e = null;
927         if (t instanceof LinkageError) {
928           // probably the hbase hadoop version does not match the running hadoop version
929           e = new DoNotRetryIOException(t);
930           markClosed(e);
931         } else if (t instanceof IOException) {
932           e = (IOException)t;
933           markClosed(e);
934         } else {
935           e = new IOException("Could not set up IO Streams", t);
936           markClosed(e);
937         }
938         close();
939         throw e;
940       }
941     }
942 
943     /**
944      * Write the RPC header: <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>
945      */
946     private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
947       // Assemble the preamble up in a buffer first and then send it.  Writing individual elements,
948       // they are getting sent across piecemeal according to wireshark and then server is messing
949       // up the reading on occasion (the passed in stream is not buffered yet).
950 
951       // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE
952       int rpcHeaderLen = HConstants.RPC_HEADER.array().length;
953       byte [] preamble = new byte [rpcHeaderLen + 2];
954       System.arraycopy(HConstants.RPC_HEADER.array(), 0, preamble, 0, rpcHeaderLen);
955       preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION;
956       preamble[rpcHeaderLen + 1] = authMethod.code;
957       outStream.write(preamble);
958       outStream.flush();
959     }
960 
961     /**
962      * Write the connection header.
963      * Out is not synchronized because only the first thread does this.
964      */
965     private void writeConnectionHeader() throws IOException {
966       synchronized (this.out) {
967         this.out.writeInt(this.header.getSerializedSize());
968         this.header.writeTo(this.out);
969         this.out.flush();
970       }
971     }
972 
973     /** Close the connection. */
974     protected synchronized void close() {
975       if (!shouldCloseConnection.get()) {
976         LOG.error(getName() + ": the connection is not in the closed state");
977         return;
978       }
979 
980       // release the resources
981       // first thing to do;take the connection out of the connection list
982       synchronized (connections) {
983         connections.removeValue(remoteId, this);
984       }
985 
986       // close the streams and therefore the socket
987       if (this.out != null) {
988         synchronized(this.out) {
989           IOUtils.closeStream(out);
990           this.out = null;
991         }
992       }
993       IOUtils.closeStream(in);
994       this.in = null;
995       disposeSasl();
996 
997       // clean up all calls
998       if (closeException == null) {
999         if (!calls.isEmpty()) {
1000           LOG.warn(getName() + ": connection is closed for no cause and calls are not empty. " +
1001               "#Calls: " + calls.size());
1002 
1003           // clean up calls anyway
1004           closeException = new IOException("Unexpected closed connection");
1005           cleanupCalls();
1006         }
1007       } else {
1008         // log the info
1009         if (LOG.isDebugEnabled()) {
1010           LOG.debug(getName() + ": closing ipc connection to " + server + ": " +
1011               closeException.getMessage(), closeException);
1012         }
1013 
1014         // cleanup calls
1015         cleanupCalls();
1016       }
1017       if (LOG.isDebugEnabled())
1018         LOG.debug(getName() + ": closed");
1019     }
1020 
1021     /**
1022      * Initiates a call by sending the parameter to the remote server.
1023      * Note: this is not called from the Connection thread, but by other
1024      * threads.
1025      * @param call
1026      * @param priority
1027      * @see #readResponse()
1028      */
1029     protected void writeRequest(Call call, final int priority) {
1030       if (shouldCloseConnection.get()) return;
1031       try {
1032         RequestHeader.Builder builder = RequestHeader.newBuilder();
1033         builder.setCallId(call.id);
1034         if (Trace.isTracing()) {
1035           Span s = Trace.currentSpan();
1036           builder.setTraceInfo(RPCTInfo.newBuilder().
1037             setParentId(s.getSpanId()).setTraceId(s.getTraceId()));
1038         }
1039         builder.setMethodName(call.md.getName());
1040         builder.setRequestParam(call.param != null);
1041         ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells);
1042         if (cellBlock != null) {
1043           CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
1044           cellBlockBuilder.setLength(cellBlock.limit());
1045           builder.setCellBlockMeta(cellBlockBuilder.build());
1046         }
1047         // Only pass priority if there one.  Let zero be same as no priority.
1048         if (priority != 0) builder.setPriority(priority);
1049         //noinspection SynchronizeOnNonFinalField
1050         RequestHeader header = builder.build();
1051         synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
1052           IPCUtil.write(this.out, header, call.param, cellBlock);
1053         }
1054         if (LOG.isDebugEnabled()) {
1055           LOG.debug(getName() + ": wrote request header " + TextFormat.shortDebugString(header));
1056         }
1057       } catch(IOException e) {
1058         markClosed(e);
1059       }
1060     }
1061 
1062     /* Receive a response.
1063      * Because only one receiver, so no synchronization on in.
1064      */
1065     protected void readResponse() {
1066       if (shouldCloseConnection.get()) return;
1067       touch();
1068       int totalSize = -1;
1069       try {
1070         // See HBaseServer.Call.setResponse for where we write out the response.
1071         // Total size of the response.  Unused.  But have to read it in anyways.
1072         totalSize = in.readInt();
1073 
1074         // Read the header
1075         ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
1076         int id = responseHeader.getCallId();
1077         if (LOG.isDebugEnabled()) {
1078           LOG.debug(getName() + ": got response header " +
1079             TextFormat.shortDebugString(responseHeader) + ", totalSize: " + totalSize + " bytes");
1080         }
1081         Call call = calls.get(id);
1082         if (call == null) {
1083           // So we got a response for which we have no corresponding 'call' here on the client-side.
1084           // We probably timed out waiting, cleaned up all references, and now the server decides
1085           // to return a response.  There is nothing we can do w/ the response at this stage. Clean
1086           // out the wire of the response so its out of the way and we can get other responses on
1087           // this connection.
1088           int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
1089           int whatIsLeftToRead = totalSize - readSoFar;
1090           LOG.debug("Unknown callId: " + id + ", skipping over this response of " +
1091             whatIsLeftToRead + " bytes");
1092           IOUtils.skipFully(in, whatIsLeftToRead);
1093         }
1094         if (responseHeader.hasException()) {
1095           ExceptionResponse exceptionResponse = responseHeader.getException();
1096           RemoteException re = createRemoteException(exceptionResponse);
1097           if (isFatalConnectionException(exceptionResponse)) {
1098             markClosed(re);
1099           } else {
1100             if (call != null) call.setException(re);
1101           }
1102         } else {
1103           Message value = null;
1104           // Call may be null because it may have timedout and been cleaned up on this side already
1105           if (call != null && call.responseDefaultType != null) {
1106             Builder builder = call.responseDefaultType.newBuilderForType();
1107             builder.mergeDelimitedFrom(in);
1108             value = builder.build();
1109           }
1110           CellScanner cellBlockScanner = null;
1111           if (responseHeader.hasCellBlockMeta()) {
1112             int size = responseHeader.getCellBlockMeta().getLength();
1113             byte [] cellBlock = new byte[size];
1114             IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
1115             cellBlockScanner = ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock);
1116           }
1117           // it's possible that this call may have been cleaned up due to a RPC
1118           // timeout, so check if it still exists before setting the value.
1119           if (call != null) call.setResponse(value, cellBlockScanner);
1120         }
1121         if (call != null) calls.remove(id);
1122       } catch (IOException e) {
1123         if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) {
1124           // Clean up open calls but don't treat this as a fatal condition,
1125           // since we expect certain responses to not make it by the specified
1126           // {@link ConnectionId#rpcTimeout}.
1127           closeException = e;
1128         } else {
1129           // Treat this as a fatal condition and close this connection
1130           markClosed(e);
1131         }
1132       } finally {
1133         if (remoteId.rpcTimeout > 0) {
1134           cleanupCalls(remoteId.rpcTimeout);
1135         }
1136       }
1137     }
1138 
1139     /**
1140      * @param e
1141      * @return True if the exception is a fatal connection exception.
1142      */
1143     private boolean isFatalConnectionException(final ExceptionResponse e) {
1144       return e.getExceptionClassName().
1145         equals(FatalConnectionException.class.getName());
1146     }
1147 
1148     /**
1149      * @param e
1150      * @return RemoteException made from passed <code>e</code>
1151      */
1152     private RemoteException createRemoteException(final ExceptionResponse e) {
1153       String innerExceptionClassName = e.getExceptionClassName();
1154       boolean doNotRetry = e.getDoNotRetry();
1155       return e.hasHostname()?
1156         // If a hostname then add it to the RemoteWithExtrasException
1157         new RemoteWithExtrasException(innerExceptionClassName,
1158           e.getStackTrace(), e.getHostname(), e.getPort(), doNotRetry):
1159         new RemoteWithExtrasException(innerExceptionClassName,
1160           e.getStackTrace(), doNotRetry);
1161     }
1162 
1163     protected synchronized void markClosed(IOException e) {
1164       if (shouldCloseConnection.compareAndSet(false, true)) {
1165         closeException = e;
1166         notifyAll();
1167       }
1168     }
1169 
1170     /* Cleanup all calls and mark them as done */
1171     protected void cleanupCalls() {
1172       cleanupCalls(0);
1173     }
1174 
1175     @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
1176       justification="Notify because timedout")
1177     protected void cleanupCalls(long rpcTimeout) {
1178       Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator();
1179       while (itor.hasNext()) {
1180         Call c = itor.next().getValue();
1181         long waitTime = System.currentTimeMillis() - c.getStartTime();
1182         if (waitTime >= rpcTimeout) {
1183           if (this.closeException == null) {
1184             // There may be no exception in the case that there are many calls
1185             // being multiplexed over this connection and these are succeeding
1186             // fine while this Call object is taking a long time to finish
1187             // over on the server; e.g. I just asked the regionserver to bulk
1188             // open 3k regions or its a big fat multiput into a heavily-loaded
1189             // server (Perhaps this only happens at the extremes?)
1190             this.closeException = new CallTimeoutException("Call id=" + c.id +
1191               ", waitTime=" + waitTime + ", rpcTimetout=" + rpcTimeout);
1192           }
1193           c.setException(this.closeException);
1194           synchronized (c) {
1195             c.notifyAll();
1196           }
1197           itor.remove();
1198         } else {
1199           break;
1200         }
1201       }
1202       try {
1203         if (!calls.isEmpty()) {
1204           Call firstCall = calls.get(calls.firstKey());
1205           long maxWaitTime = System.currentTimeMillis() - firstCall.getStartTime();
1206           if (maxWaitTime < rpcTimeout) {
1207             rpcTimeout -= maxWaitTime;
1208           }
1209         }
1210         if (!shouldCloseConnection.get()) {
1211           closeException = null;
1212           setSocketTimeout(socket, (int) rpcTimeout);
1213         }
1214       } catch (SocketException e) {
1215         LOG.debug("Couldn't lower timeout, which may result in longer than expected calls");
1216       }
1217     }
1218   }
1219 
1220   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
1221     justification="Presume sync not needed setting socket timeout")
1222   private static void setSocketTimeout(final Socket socket, final int rpcTimeout)
1223   throws java.net.SocketException {
1224     if (socket == null) return;
1225     socket.setSoTimeout(rpcTimeout);
1226   }
1227 
1228   /**
1229    * Client-side call timeout
1230    */
1231   @SuppressWarnings("serial")
1232   @InterfaceAudience.Public
1233   @InterfaceStability.Evolving
1234   public static class CallTimeoutException extends IOException {
1235     public CallTimeoutException(final String msg) {
1236       super(msg);
1237     }
1238   }
1239 
1240   /**
1241    * Construct an IPC cluster client whose values are of the {@link Message} class.
1242    * @param conf configuration
1243    * @param clusterId
1244    * @param factory socket factory
1245    */
1246   RpcClient(Configuration conf, String clusterId, SocketFactory factory) {
1247     this(conf, clusterId, factory, null);
1248   }
1249 
1250   /**
1251    * Construct an IPC cluster client whose values are of the {@link Message} class.
1252    * @param conf configuration
1253    * @param clusterId
1254    * @param factory socket factory
1255    * @param localAddr client socket bind address
1256    */
1257   RpcClient(Configuration conf, String clusterId, SocketFactory factory, SocketAddress localAddr) {
1258     this.maxIdleTime = conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
1259     this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
1260     this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
1261         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
1262     this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true);
1263     this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
1264     this.pingInterval = getPingInterval(conf);
1265     this.ipcUtil = new IPCUtil(conf);
1266     this.conf = conf;
1267     this.codec = getCodec();
1268     this.compressor = getCompressor(conf);
1269     this.socketFactory = factory;
1270     this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
1271     this.connections = new PoolMap<ConnectionId, Connection>(getPoolType(conf), getPoolSize(conf));
1272     this.failedServers = new FailedServers(conf);
1273     this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
1274         IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
1275     this.localAddr = localAddr;
1276     this.userProvider = UserProvider.instantiate(conf);
1277     // login the server principal (if using secure Hadoop)
1278     if (LOG.isDebugEnabled()) {
1279       LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor +
1280         ", tcpKeepAlive=" + this.tcpKeepAlive +
1281         ", tcpNoDelay=" + this.tcpNoDelay +
1282         ", maxIdleTime=" + this.maxIdleTime +
1283         ", maxRetries=" + this.maxRetries +
1284         ", fallbackAllowed=" + this.fallbackAllowed +
1285         ", ping interval=" + this.pingInterval + "ms" +
1286         ", bind address=" + (this.localAddr != null ? this.localAddr : "null"));
1287     }
1288   }
1289 
1290   /**
1291    * Construct an IPC client for the cluster <code>clusterId</code> with the default SocketFactory
1292    * @param conf configuration
1293    * @param clusterId
1294    */
1295   public RpcClient(Configuration conf, String clusterId) {
1296     this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), null);
1297   }
1298 
1299   /**
1300    * Construct an IPC client for the cluster <code>clusterId</code> with the default SocketFactory
1301    * @param conf configuration
1302    * @param clusterId
1303    * @param localAddr client socket bind address.
1304    */
1305   public RpcClient(Configuration conf, String clusterId, SocketAddress localAddr) {
1306     this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), localAddr);
1307   }
1308 
1309   /**
1310    * Encapsulate the ugly casting and RuntimeException conversion in private method.
1311    * @return Codec to use on this client.
1312    */
1313   Codec getCodec() {
1314     // For NO CODEC, "hbase.client.rpc.codec" must be configured with empty string AND
1315     // "hbase.client.default.rpc.codec" also -- because default is to do cell block encoding.
1316     String className = conf.get(HConstants.RPC_CODEC_CONF_KEY, getDefaultCodec(this.conf));
1317     if (className == null || className.length() == 0) return null;
1318     try {
1319       return (Codec)Class.forName(className).newInstance();
1320     } catch (Exception e) {
1321       throw new RuntimeException("Failed getting codec " + className, e);
1322     }
1323   }
1324 
1325   @VisibleForTesting
1326   public static String getDefaultCodec(final Configuration c) {
1327     // If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because
1328     // Configuration will complain -- then no default codec (and we'll pb everything).  Else
1329     // default is KeyValueCodec
1330     return c.get("hbase.client.default.rpc.codec", KeyValueCodec.class.getCanonicalName());
1331   }
1332 
1333   /**
1334    * Encapsulate the ugly casting and RuntimeException conversion in private method.
1335    * @param conf
1336    * @return The compressor to use on this client.
1337    */
1338   private static CompressionCodec getCompressor(final Configuration conf) {
1339     String className = conf.get("hbase.client.rpc.compressor", null);
1340     if (className == null || className.isEmpty()) return null;
1341     try {
1342         return (CompressionCodec)Class.forName(className).newInstance();
1343     } catch (Exception e) {
1344       throw new RuntimeException("Failed getting compressor " + className, e);
1345     }
1346   }
1347 
1348   /**
1349    * Return the pool type specified in the configuration, which must be set to
1350    * either {@link PoolType#RoundRobin} or {@link PoolType#ThreadLocal},
1351    * otherwise default to the former.
1352    *
1353    * For applications with many user threads, use a small round-robin pool. For
1354    * applications with few user threads, you may want to try using a
1355    * thread-local pool. In any case, the number of {@link RpcClient} instances
1356    * should not exceed the operating system's hard limit on the number of
1357    * connections.
1358    *
1359    * @param config configuration
1360    * @return either a {@link PoolType#RoundRobin} or
1361    *         {@link PoolType#ThreadLocal}
1362    */
1363   protected static PoolType getPoolType(Configuration config) {
1364     return PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE),
1365         PoolType.RoundRobin, PoolType.ThreadLocal);
1366   }
1367 
1368   /**
1369    * Return the pool size specified in the configuration, which is applicable only if
1370    * the pool type is {@link PoolType#RoundRobin}.
1371    *
1372    * @param config
1373    * @return the maximum pool size
1374    */
1375   protected static int getPoolSize(Configuration config) {
1376     return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1);
1377   }
1378 
1379   /** Return the socket factory of this client
1380    *
1381    * @return this client's socket factory
1382    */
1383   SocketFactory getSocketFactory() {
1384     return socketFactory;
1385   }
1386 
1387   /** Stop all threads related to this client.  No further calls may be made
1388    * using this client. */
1389   public void stop() {
1390     if (LOG.isDebugEnabled()) LOG.debug("Stopping rpc client");
1391     if (!running.compareAndSet(true, false)) return;
1392 
1393     // wake up all connections
1394     synchronized (connections) {
1395       for (Connection conn : connections.values()) {
1396         conn.interrupt();
1397       }
1398     }
1399 
1400     // wait until all connections are closed
1401     while (!connections.isEmpty()) {
1402       try {
1403         Thread.sleep(100);
1404       } catch (InterruptedException ignored) {
1405       }
1406     }
1407   }
1408 
1409   Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
1410       Message returnType, User ticket, InetSocketAddress addr, int rpcTimeout)
1411   throws InterruptedException, IOException {
1412     return call(md, param, cells, returnType, ticket, addr, rpcTimeout, HConstants.NORMAL_QOS);
1413   }
1414 
1415   /** Make a call, passing <code>param</code>, to the IPC server running at
1416    * <code>address</code> which is servicing the <code>protocol</code> protocol,
1417    * with the <code>ticket</code> credentials, returning the value.
1418    * Throws exceptions if there are network problems or if the remote code
1419    * threw an exception.
1420    * @param md
1421    * @param param
1422    * @param cells
1423    * @param addr
1424    * @param returnType
1425    * @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
1426    *          {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a
1427    *          new Connection each time.
1428    * @param rpcTimeout
1429    * @return A pair with the Message response and the Cell data (if any).
1430    * @throws InterruptedException
1431    * @throws IOException
1432    */
1433   Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
1434       Message returnType, User ticket, InetSocketAddress addr,
1435       int rpcTimeout, int priority)
1436   throws InterruptedException, IOException {
1437     Call call = new Call(md, param, cells, returnType);
1438     Connection connection =
1439       getConnection(ticket, call, addr, rpcTimeout, this.codec, this.compressor);
1440     connection.writeRequest(call, priority);                 // send the parameter
1441 
1442     //noinspection SynchronizationOnLocalVariableOrMethodParameter
1443     synchronized (call) {
1444       while (!call.done) {
1445         if (connection.shouldCloseConnection.get()) {
1446           throw new IOException("Unexpected closed connection");
1447         }
1448         call.wait(1000);                       // wait for the result
1449       }
1450 
1451       if (call.error != null) {
1452         if (call.error instanceof RemoteException) {
1453           call.error.fillInStackTrace();
1454           throw call.error;
1455         }
1456         // local exception
1457         throw wrapException(addr, call.error);
1458       }
1459       return new Pair<Message, CellScanner>(call.response, call.cells);
1460     }
1461   }
1462 
1463   /**
1464    * Take an IOException and the address we were trying to connect to
1465    * and return an IOException with the input exception as the cause.
1466    * The new exception provides the stack trace of the place where
1467    * the exception is thrown and some extra diagnostics information.
1468    * If the exception is ConnectException or SocketTimeoutException,
1469    * return a new one of the same type; Otherwise return an IOException.
1470    *
1471    * @param addr target address
1472    * @param exception the relevant exception
1473    * @return an exception to throw
1474    */
1475   protected IOException wrapException(InetSocketAddress addr,
1476                                          IOException exception) {
1477     if (exception instanceof ConnectException) {
1478       //connection refused; include the host:port in the error
1479       return (ConnectException)new ConnectException(
1480          "Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
1481     } else if (exception instanceof SocketTimeoutException) {
1482       return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
1483         " failed because " + exception).initCause(exception);
1484     } else {
1485       return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
1486         exception).initCause(exception);
1487     }
1488   }
1489 
1490   /**
1491    * Interrupt the connections to the given ip:port server. This should be called if the server
1492    *  is known as actually dead. This will not prevent current operation to be retried, and,
1493    *  depending on their own behavior, they may retry on the same server. This can be a feature,
1494    *  for example at startup. In any case, they're likely to get connection refused (if the
1495    *  process died) or no route to host: i.e. there next retries should be faster and with a
1496    *  safe exception.
1497    */
1498   public void cancelConnections(String hostname, int port, IOException ioe) {
1499     synchronized (connections) {
1500       for (Connection connection : connections.values()) {
1501         if (connection.isAlive() &&
1502             connection.getRemoteAddress().getPort() == port &&
1503             connection.getRemoteAddress().getHostName().equals(hostname)) {
1504           LOG.info("The server on " + hostname + ":" + port +
1505               " is dead - stopping the connection " + connection.remoteId);
1506           connection.closeConnection();
1507           // We could do a connection.interrupt(), but it's safer not to do it, as the
1508           //  interrupted exception behavior is not defined nor enforced enough.
1509         }
1510       }
1511     }
1512   }
1513 
1514   /* Get a connection from the pool, or create a new one and add it to the
1515    * pool.  Connections to a given host/port are reused. */
1516   protected Connection getConnection(User ticket, Call call, InetSocketAddress addr,
1517       int rpcTimeout, final Codec codec, final CompressionCodec compressor)
1518   throws IOException, InterruptedException {
1519     if (!running.get()) throw new StoppedRpcClientException();
1520     Connection connection;
1521     ConnectionId remoteId =
1522       new ConnectionId(ticket, call.md.getService().getName(), addr, rpcTimeout);
1523     synchronized (connections) {
1524       connection = connections.get(remoteId);
1525       if (connection == null) {
1526         connection = createConnection(remoteId, this.codec, this.compressor);
1527         connections.put(remoteId, connection);
1528       }
1529     }
1530     connection.addCall(call);
1531 
1532     //we don't invoke the method below inside "synchronized (connections)"
1533     //block above. The reason for that is if the server happens to be slow,
1534     //it will take longer to establish a connection and that will slow the
1535     //entire system down.
1536     //Moreover, if the connection is currently created, there will be many threads
1537     // waiting here; as setupIOstreams is synchronized. If the connection fails with a
1538     // timeout, they will all fail simultaneously. This is checked in setupIOstreams.
1539     connection.setupIOstreams();
1540     return connection;
1541   }
1542 
1543   /**
1544    * This class holds the address and the user ticket, etc. The client connections
1545    * to servers are uniquely identified by <remoteAddress, ticket, serviceName, rpcTimeout>
1546    */
1547   protected static class ConnectionId {
1548     final InetSocketAddress address;
1549     final User ticket;
1550     final int rpcTimeout;
1551     private static final int PRIME = 16777619;
1552     final String serviceName;
1553 
1554     ConnectionId(User ticket,
1555         String serviceName,
1556         InetSocketAddress address,
1557         int rpcTimeout) {
1558       this.address = address;
1559       this.ticket = ticket;
1560       this.rpcTimeout = rpcTimeout;
1561       this.serviceName = serviceName;
1562     }
1563 
1564     String getServiceName() {
1565       return this.serviceName;
1566     }
1567 
1568     InetSocketAddress getAddress() {
1569       return address;
1570     }
1571 
1572     User getTicket() {
1573       return ticket;
1574     }
1575 
1576     @Override
1577     public String toString() {
1578       return this.address.toString() + "/" + this.serviceName + "/" + this.ticket + "/" +
1579         this.rpcTimeout;
1580     }
1581 
1582     @Override
1583     public boolean equals(Object obj) {
1584      if (obj instanceof ConnectionId) {
1585        ConnectionId id = (ConnectionId) obj;
1586        return address.equals(id.address) &&
1587               ((ticket != null && ticket.equals(id.ticket)) ||
1588                (ticket == id.ticket)) && rpcTimeout == id.rpcTimeout &&
1589                this.serviceName == id.serviceName;
1590      }
1591      return false;
1592     }
1593 
1594     @Override  // simply use the default Object#hashcode() ?
1595     public int hashCode() {
1596       int hashcode = (address.hashCode() +
1597         PRIME * (PRIME * this.serviceName.hashCode() ^
1598         (ticket == null ? 0 : ticket.hashCode()) )) ^
1599         rpcTimeout;
1600       return hashcode;
1601     }
1602   }
1603 
1604   public static void setRpcTimeout(int t) {
1605     rpcTimeout.set(t);
1606   }
1607 
1608   public static int getRpcTimeout() {
1609     return rpcTimeout.get();
1610   }
1611 
1612   /**
1613    * Returns the lower of the thread-local RPC time from {@link #setRpcTimeout(int)} and the given
1614    * default timeout.
1615    */
1616   public static int getRpcTimeout(int defaultTimeout) {
1617     return Math.min(defaultTimeout, rpcTimeout.get());
1618   }
1619 
1620   public static void resetRpcTimeout() {
1621     rpcTimeout.remove();
1622   }
1623 
1624   /**
1625    * Make a blocking call. Throws exceptions if there are network problems or if the remote code
1626    * threw an exception.
1627    * @param md
1628    * @param controller
1629    * @param param
1630    * @param returnType
1631    * @param isa
1632    * @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
1633    *          {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a
1634    *          new Connection each time.
1635    * @param rpcTimeout
1636    * @return A pair with the Message response and the Cell data (if any).
1637    * @throws InterruptedException
1638    * @throws IOException
1639    */
1640   Message callBlockingMethod(MethodDescriptor md, RpcController controller,
1641       Message param, Message returnType, final User ticket, final InetSocketAddress isa,
1642       final int rpcTimeout)
1643   throws ServiceException {
1644     long startTime = 0;
1645     if (LOG.isTraceEnabled()) {
1646       startTime = System.currentTimeMillis();
1647     }
1648     PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController)controller;
1649     CellScanner cells = null;
1650     if (pcrc != null) {
1651       cells = pcrc.cellScanner();
1652       // Clear it here so we don't by mistake try and these cells processing results.
1653       pcrc.setCellScanner(null);
1654     }
1655     Pair<Message, CellScanner> val = null;
1656     try {
1657       val = call(md, param, cells, returnType, ticket, isa, rpcTimeout,
1658         pcrc != null? pcrc.getPriority(): HConstants.NORMAL_QOS);
1659       if (pcrc != null) {
1660         // Shove the results into controller so can be carried across the proxy/pb service void.
1661         if (val.getSecond() != null) pcrc.setCellScanner(val.getSecond());
1662       } else if (val.getSecond() != null) {
1663         throw new ServiceException("Client dropping data on the floor!");
1664       }
1665 
1666       if (LOG.isTraceEnabled()) {
1667         long callTime = System.currentTimeMillis() - startTime;
1668         if (LOG.isTraceEnabled()) {
1669           LOG.trace("Call: " + md.getName() + ", callTime: " + callTime + "ms");
1670         }
1671       }
1672       return val.getFirst();
1673     } catch (Throwable e) {
1674       throw new ServiceException(e);
1675     }
1676   }
1677 
1678   /**
1679    * Creates a "channel" that can be used by a blocking protobuf service.  Useful setting up
1680    * protobuf blocking stubs.
1681    * @param sn
1682    * @param ticket
1683    * @param rpcTimeout
1684    * @return A blocking rpc channel that goes via this rpc client instance.
1685    */
1686   public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn,
1687       final User ticket, final int rpcTimeout) {
1688     return new BlockingRpcChannelImplementation(this, sn, ticket, rpcTimeout);
1689   }
1690 
1691   /**
1692    * Blocking rpc channel that goes via hbase rpc.
1693    */
1694   // Public so can be subclassed for tests.
1695   public static class BlockingRpcChannelImplementation implements BlockingRpcChannel {
1696     private final InetSocketAddress isa;
1697     private volatile RpcClient rpcClient;
1698     private final int rpcTimeout;
1699     private final User ticket;
1700 
1701     protected BlockingRpcChannelImplementation(final RpcClient rpcClient, final ServerName sn,
1702         final User ticket, final int rpcTimeout) {
1703       this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort());
1704       this.rpcClient = rpcClient;
1705       // Set the rpc timeout to be the minimum of configured timeout and whatever the current
1706       // thread local setting is.
1707       this.rpcTimeout = getRpcTimeout(rpcTimeout);
1708       this.ticket = ticket;
1709     }
1710 
1711     @Override
1712     public Message callBlockingMethod(MethodDescriptor md, RpcController controller,
1713         Message param, Message returnType)
1714     throws ServiceException {
1715       return this.rpcClient.callBlockingMethod(md, controller, param, returnType, this.ticket,
1716         this.isa, this.rpcTimeout);
1717     }
1718   }
1719 }