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