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  package org.apache.hadoop.hbase.client;
20  
21  import java.io.Closeable;
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.lang.reflect.Constructor;
25  import java.lang.reflect.UndeclaredThrowableException;
26  import java.net.SocketException;
27  import java.util.ArrayList;
28  import java.util.Date;
29  import java.util.HashSet;
30  import java.util.LinkedHashMap;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.Map.Entry;
34  import java.util.NavigableMap;
35  import java.util.Set;
36  import java.util.concurrent.ConcurrentHashMap;
37  import java.util.concurrent.ConcurrentMap;
38  import java.util.concurrent.ConcurrentSkipListMap;
39  import java.util.concurrent.ConcurrentSkipListSet;
40  import java.util.concurrent.CopyOnWriteArraySet;
41  import java.util.concurrent.ExecutorService;
42  import java.util.concurrent.LinkedBlockingQueue;
43  import java.util.concurrent.ThreadPoolExecutor;
44  import java.util.concurrent.TimeUnit;
45  import java.util.concurrent.atomic.AtomicBoolean;
46  import java.util.concurrent.atomic.AtomicInteger;
47  
48  import org.apache.commons.logging.Log;
49  import org.apache.commons.logging.LogFactory;
50  import org.apache.hadoop.classification.InterfaceAudience;
51  import org.apache.hadoop.classification.InterfaceStability;
52  import org.apache.hadoop.conf.Configuration;
53  import org.apache.hadoop.hbase.Chore;
54  import org.apache.hadoop.hbase.HBaseConfiguration;
55  import org.apache.hadoop.hbase.HConstants;
56  import org.apache.hadoop.hbase.HRegionInfo;
57  import org.apache.hadoop.hbase.HRegionLocation;
58  import org.apache.hadoop.hbase.HTableDescriptor;
59  import org.apache.hadoop.hbase.MasterNotRunningException;
60  import org.apache.hadoop.hbase.RegionTooBusyException;
61  import org.apache.hadoop.hbase.ServerName;
62  import org.apache.hadoop.hbase.Stoppable;
63  import org.apache.hadoop.hbase.TableName;
64  import org.apache.hadoop.hbase.TableNotEnabledException;
65  import org.apache.hadoop.hbase.TableNotFoundException;
66  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
67  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
68  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
69  import org.apache.hadoop.hbase.client.coprocessor.Batch;
70  import org.apache.hadoop.hbase.exceptions.RegionMovedException;
71  import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
72  import org.apache.hadoop.hbase.ipc.RpcClient;
73  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
74  import org.apache.hadoop.hbase.protobuf.RequestConverter;
75  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
76  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
77  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
78  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
79  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
80  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
81  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
82  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
83  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
84  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
85  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
86  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
87  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
88  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
89  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
90  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
91  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
92  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
93  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
94  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
95  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
96  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
97  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
98  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
99  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
100 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
101 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
102 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
103 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
104 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
105 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
106 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
107 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
108 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
109 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
110 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
111 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
112 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
113 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
114 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
115 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
116 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
117 import org.apache.hadoop.hbase.security.User;
118 import org.apache.hadoop.hbase.security.UserProvider;
119 import org.apache.hadoop.hbase.util.Bytes;
120 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
121 import org.apache.hadoop.hbase.util.ExceptionUtil;
122 import org.apache.hadoop.hbase.util.Threads;
123 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
124 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
125 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
126 import org.apache.hadoop.ipc.RemoteException;
127 import org.apache.zookeeper.KeeperException;
128 
129 import com.google.common.annotations.VisibleForTesting;
130 import com.google.protobuf.BlockingRpcChannel;
131 import com.google.protobuf.RpcController;
132 import com.google.protobuf.ServiceException;
133 
134 /**
135  * A non-instantiable class that manages creation of {@link HConnection}s.
136  * <p>The simplest way to use this class is by using {@link #createConnection(Configuration)}.
137  * This creates a new {@link HConnection} to the cluster that is managed by the caller.
138  * From this {@link HConnection} {@link HTableInterface} implementations are retrieved
139  * with {@link HConnection#getTable(byte[])}. Example:
140  * <pre>
141  * {@code
142  * HConnection connection = HConnectionManager.createConnection(config);
143  * HTableInterface table = connection.getTable("table1");
144  * try {
145  *   // Use the table as needed, for a single operation and a single thread
146  * } finally {
147  *   table.close();
148  *   connection.close();
149  * }
150  * }</pre>
151  * <p>The following logic and API will be removed in the future:
152  * <p>This class has a static Map of {@link HConnection} instances keyed by
153  * {@link Configuration}; all invocations of {@link #getConnection(Configuration)}
154  * that pass the same {@link Configuration} instance will be returned the same
155  * {@link  HConnection} instance (Adding properties to a Configuration
156  * instance does not change its object identity; for more on how this is done see
157  * {@link HConnectionKey}).  Sharing {@link HConnection}
158  * instances is usually what you want; all clients of the {@link HConnection}
159  * instances share the HConnections' cache of Region locations rather than each
160  * having to discover for itself the location of meta, etc.  It makes
161  * sense for the likes of the pool of HTables class {@link HTablePool}, for
162  * instance (If concerned that a single {@link HConnection} is insufficient
163  * for sharing amongst clients in say an heavily-multithreaded environment,
164  * in practise its not proven to be an issue.  Besides, {@link HConnection} is
165  * implemented atop Hadoop RPC and as of this writing, Hadoop RPC does a
166  * connection per cluster-member, exclusively).
167  *
168  * <p>But sharing connections makes clean up of {@link HConnection} instances a little awkward.
169  * Currently, clients cleanup by calling {@link #deleteConnection(Configuration)}. This will
170  * shutdown the zookeeper connection the HConnection was using and clean up all
171  * HConnection resources as well as stopping proxies to servers out on the
172  * cluster. Not running the cleanup will not end the world; it'll
173  * just stall the closeup some and spew some zookeeper connection failed
174  * messages into the log.  Running the cleanup on a {@link HConnection} that is
175  * subsequently used by another will cause breakage so be careful running
176  * cleanup.
177  * <p>To create a {@link HConnection} that is not shared by others, you can
178  * create a new {@link Configuration} instance, pass this new instance to
179  * {@link #getConnection(Configuration)}, and then when done, close it up by
180  * doing something like the following:
181  * <pre>
182  * {@code
183  * Configuration newConfig = new Configuration(originalConf);
184  * HConnection connection = HConnectionManager.getConnection(newConfig);
185  * // Use the connection to your hearts' delight and then when done...
186  * HConnectionManager.deleteConnection(newConfig, true);
187  * }
188  * </pre>
189  * <p>Cleanup used to be done inside in a shutdown hook.  On startup we'd
190  * register a shutdown hook that called {@link #deleteAllConnections()}
191  * on its way out but the order in which shutdown hooks run is not defined so
192  * were problematic for clients of HConnection that wanted to register their
193  * own shutdown hooks so we removed ours though this shifts the onus for
194  * cleanup to the client.
195  */
196 @SuppressWarnings("serial")
197 @InterfaceAudience.Public
198 @InterfaceStability.Evolving
199 public class HConnectionManager {
200   static final Log LOG = LogFactory.getLog(HConnectionManager.class);
201 
202   public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server";
203   private static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled";
204 
205   // An LRU Map of HConnectionKey -> HConnection (TableServer).  All
206   // access must be synchronized.  This map is not private because tests
207   // need to be able to tinker with it.
208   static final Map<HConnectionKey, HConnectionImplementation> CONNECTION_INSTANCES;
209 
210   public static final int MAX_CACHED_CONNECTION_INSTANCES;
211 
212   /**
213    * Global nonceGenerator shared per client.Currently there's no reason to limit its scope.
214    * Once it's set under nonceGeneratorCreateLock, it is never unset or changed.
215    */
216   private static volatile NonceGenerator nonceGenerator = null;
217   /** The nonce generator lock. Only taken when creating HConnection, which gets a private copy. */
218   private static Object nonceGeneratorCreateLock = new Object();
219 
220   static {
221     // We set instances to one more than the value specified for {@link
222     // HConstants#ZOOKEEPER_MAX_CLIENT_CNXNS}. By default, the zk default max
223     // connections to the ensemble from the one client is 30, so in that case we
224     // should run into zk issues before the LRU hit this value of 31.
225     MAX_CACHED_CONNECTION_INSTANCES = HBaseConfiguration.create().getInt(
226       HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS) + 1;
227     CONNECTION_INSTANCES = new LinkedHashMap<HConnectionKey, HConnectionImplementation>(
228         (int) (MAX_CACHED_CONNECTION_INSTANCES / 0.75F) + 1, 0.75F, true) {
229       @Override
230       protected boolean removeEldestEntry(
231           Map.Entry<HConnectionKey, HConnectionImplementation> eldest) {
232          return size() > MAX_CACHED_CONNECTION_INSTANCES;
233        }
234     };
235   }
236 
237   /*
238    * Non-instantiable.
239    */
240   private HConnectionManager() {
241     super();
242   }
243 
244   /**
245    * @param conn The connection for which to replace the generator.
246    * @param cnm Replaces the nonce generator used, for testing.
247    * @return old nonce generator.
248    */
249   @VisibleForTesting
250   public static NonceGenerator injectNonceGeneratorForTesting(
251       HConnection conn, NonceGenerator cnm) {
252     NonceGenerator ng = conn.getNonceGenerator();
253     LOG.warn("Nonce generator is being replaced by test code for " + cnm.getClass().getName());
254     ((HConnectionImplementation)conn).nonceGenerator = cnm;
255     return ng;
256   }
257 
258   /**
259    * Get the connection that goes with the passed <code>conf</code> configuration instance.
260    * If no current connection exists, method creates a new connection and keys it using
261    * connection-specific properties from the passed {@link Configuration}; see
262    * {@link HConnectionKey}.
263    * @param conf configuration
264    * @return HConnection object for <code>conf</code>
265    * @throws ZooKeeperConnectionException
266    */
267   @Deprecated
268   public static HConnection getConnection(final Configuration conf)
269   throws IOException {
270     HConnectionKey connectionKey = new HConnectionKey(conf);
271     synchronized (CONNECTION_INSTANCES) {
272       HConnectionImplementation connection = CONNECTION_INSTANCES.get(connectionKey);
273       if (connection == null) {
274         connection = (HConnectionImplementation)createConnection(conf, true);
275         CONNECTION_INSTANCES.put(connectionKey, connection);
276       } else if (connection.isClosed()) {
277         HConnectionManager.deleteConnection(connectionKey, true);
278         connection = (HConnectionImplementation)createConnection(conf, true);
279         CONNECTION_INSTANCES.put(connectionKey, connection);
280       }
281       connection.incCount();
282       return connection;
283     }
284   }
285 
286   /**
287    * Create a new HConnection instance using the passed <code>conf</code> instance.
288    * <p>Note: This bypasses the usual HConnection life cycle management done by
289    * {@link #getConnection(Configuration)}. The caller is responsible for
290    * calling {@link HConnection#close()} on the returned connection instance.
291    *
292    * This is the recommended way to create HConnections.
293    * {@code
294    * HConnection connection = HConnectionManager.createConnection(conf);
295    * HTableInterface table = connection.getTable("mytable");
296    * table.get(...);
297    * ...
298    * table.close();
299    * connection.close();
300    * }
301    *
302    * @param conf configuration
303    * @return HConnection object for <code>conf</code>
304    * @throws ZooKeeperConnectionException
305    */
306   public static HConnection createConnection(Configuration conf)
307   throws IOException {
308     UserProvider provider = UserProvider.instantiate(conf);
309     return createConnection(conf, false, null, provider.getCurrent());
310   }
311 
312   /**
313    * Create a new HConnection instance using the passed <code>conf</code> instance.
314    * <p>Note: This bypasses the usual HConnection life cycle management done by
315    * {@link #getConnection(Configuration)}. The caller is responsible for
316    * calling {@link HConnection#close()} on the returned connection instance.
317    * This is the recommended way to create HConnections.
318    * {@code
319    * ExecutorService pool = ...;
320    * HConnection connection = HConnectionManager.createConnection(conf, pool);
321    * HTableInterface table = connection.getTable("mytable");
322    * table.get(...);
323    * ...
324    * table.close();
325    * connection.close();
326    * }
327    * @param conf configuration
328    * @param pool the thread pool to use for batch operation in HTables used via this HConnection
329    * @return HConnection object for <code>conf</code>
330    * @throws ZooKeeperConnectionException
331    */
332   public static HConnection createConnection(Configuration conf, ExecutorService pool)
333   throws IOException {
334     UserProvider provider = UserProvider.instantiate(conf);
335     return createConnection(conf, false, pool, provider.getCurrent());
336   }
337 
338   /**
339    * Create a new HConnection instance using the passed <code>conf</code> instance.
340    * <p>Note: This bypasses the usual HConnection life cycle management done by
341    * {@link #getConnection(Configuration)}. The caller is responsible for
342    * calling {@link HConnection#close()} on the returned connection instance.
343    * This is the recommended way to create HConnections.
344    * {@code
345    * ExecutorService pool = ...;
346    * HConnection connection = HConnectionManager.createConnection(conf, pool);
347    * HTableInterface table = connection.getTable("mytable");
348    * table.get(...);
349    * ...
350    * table.close();
351    * connection.close();
352    * }
353    * @param conf configuration
354    * @param user the user the connection is for
355    * @return HConnection object for <code>conf</code>
356    * @throws ZooKeeperConnectionException
357    */
358   public static HConnection createConnection(Configuration conf, User user)
359   throws IOException {
360     return createConnection(conf, false, null, user);
361   }
362 
363   /**
364    * Create a new HConnection instance using the passed <code>conf</code> instance.
365    * <p>Note: This bypasses the usual HConnection life cycle management done by
366    * {@link #getConnection(Configuration)}. The caller is responsible for
367    * calling {@link HConnection#close()} on the returned connection instance.
368    * This is the recommended way to create HConnections.
369    * {@code
370    * ExecutorService pool = ...;
371    * HConnection connection = HConnectionManager.createConnection(conf, pool);
372    * HTableInterface table = connection.getTable("mytable");
373    * table.get(...);
374    * ...
375    * table.close();
376    * connection.close();
377    * }
378    * @param conf configuration
379    * @param pool the thread pool to use for batch operation in HTables used via this HConnection
380    * @param user the user the connection is for
381    * @return HConnection object for <code>conf</code>
382    * @throws ZooKeeperConnectionException
383    */
384   public static HConnection createConnection(Configuration conf, ExecutorService pool, User user)
385   throws IOException {
386     return createConnection(conf, false, pool, user);
387   }
388 
389   @Deprecated
390   static HConnection createConnection(final Configuration conf, final boolean managed)
391       throws IOException {
392     UserProvider provider = UserProvider.instantiate(conf);
393     return createConnection(conf, managed, null, provider.getCurrent());
394   }
395 
396   @Deprecated
397   static HConnection createConnection(final Configuration conf, final boolean managed,
398       final ExecutorService pool, final User user)
399   throws IOException {
400     String className = conf.get("hbase.client.connection.impl",
401       HConnectionManager.HConnectionImplementation.class.getName());
402     Class<?> clazz = null;
403     try {
404       clazz = Class.forName(className);
405     } catch (ClassNotFoundException e) {
406       throw new IOException(e);
407     }
408     try {
409       // Default HCM#HCI is not accessible; make it so before invoking.
410       Constructor<?> constructor =
411         clazz.getDeclaredConstructor(Configuration.class,
412           boolean.class, ExecutorService.class, User.class);
413       constructor.setAccessible(true);
414       return (HConnection) constructor.newInstance(conf, managed, pool, user);
415     } catch (Exception e) {
416       throw new IOException(e);
417     }
418   }
419 
420   /**
421    * Delete connection information for the instance specified by passed configuration.
422    * If there are no more references to the designated connection connection, this method will
423    * then close connection to the zookeeper ensemble and let go of all associated resources.
424    *
425    * @param conf configuration whose identity is used to find {@link HConnection} instance.
426    * @deprecated
427    */
428   public static void deleteConnection(Configuration conf) {
429     deleteConnection(new HConnectionKey(conf), false);
430   }
431 
432   /**
433    * Cleanup a known stale connection.
434    * This will then close connection to the zookeeper ensemble and let go of all resources.
435    *
436    * @param connection
437    * @deprecated
438    */
439   public static void deleteStaleConnection(HConnection connection) {
440     deleteConnection(connection, true);
441   }
442 
443   /**
444    * Delete information for all connections. Close or not the connection, depending on the
445    *  staleConnection boolean and the ref count. By default, you should use it with
446    *  staleConnection to true.
447    * @deprecated
448    */
449   public static void deleteAllConnections(boolean staleConnection) {
450     synchronized (CONNECTION_INSTANCES) {
451       Set<HConnectionKey> connectionKeys = new HashSet<HConnectionKey>();
452       connectionKeys.addAll(CONNECTION_INSTANCES.keySet());
453       for (HConnectionKey connectionKey : connectionKeys) {
454         deleteConnection(connectionKey, staleConnection);
455       }
456       CONNECTION_INSTANCES.clear();
457     }
458   }
459 
460   /**
461    * Delete information for all connections..
462    * @deprecated kept for backward compatibility, but the behavior is broken. HBASE-8983
463    */
464   @Deprecated
465   public static void deleteAllConnections() {
466     deleteAllConnections(false);
467   }
468 
469 
470   @Deprecated
471   private static void deleteConnection(HConnection connection, boolean staleConnection) {
472     synchronized (CONNECTION_INSTANCES) {
473       for (Entry<HConnectionKey, HConnectionImplementation> e: CONNECTION_INSTANCES.entrySet()) {
474         if (e.getValue() == connection) {
475           deleteConnection(e.getKey(), staleConnection);
476           break;
477         }
478       }
479     }
480   }
481 
482   @Deprecated
483   private static void deleteConnection(HConnectionKey connectionKey, boolean staleConnection) {
484     synchronized (CONNECTION_INSTANCES) {
485       HConnectionImplementation connection = CONNECTION_INSTANCES.get(connectionKey);
486       if (connection != null) {
487         connection.decCount();
488         if (connection.isZeroReference() || staleConnection) {
489           CONNECTION_INSTANCES.remove(connectionKey);
490           connection.internalClose();
491         }
492       } else {
493         LOG.error("Connection not found in the list, can't delete it "+
494           "(connection key=" + connectionKey + "). May be the key was modified?", new Exception());
495       }
496     }
497   }
498 
499   /**
500    * It is provided for unit test cases which verify the behavior of region
501    * location cache prefetch.
502    * @return Number of cached regions for the table.
503    * @throws ZooKeeperConnectionException
504    */
505   static int getCachedRegionCount(Configuration conf, final TableName tableName)
506   throws IOException {
507     return execute(new HConnectable<Integer>(conf) {
508       @Override
509       public Integer connect(HConnection connection) {
510         return ((HConnectionImplementation)connection).getNumberOfCachedRegionLocations(tableName);
511       }
512     });
513   }
514 
515   /**
516    * It's provided for unit test cases which verify the behavior of region
517    * location cache prefetch.
518    * @return true if the region where the table and row reside is cached.
519    * @throws ZooKeeperConnectionException
520    */
521   static boolean isRegionCached(Configuration conf,
522                                 final TableName tableName,
523                                 final byte[] row)
524   throws IOException {
525     return execute(new HConnectable<Boolean>(conf) {
526       @Override
527       public Boolean connect(HConnection connection) {
528         return ((HConnectionImplementation) connection).isRegionCached(tableName, row);
529       }
530     });
531   }
532 
533   /**
534    * This convenience method invokes the given {@link HConnectable#connect}
535    * implementation using a {@link HConnection} instance that lasts just for the
536    * duration of the invocation.
537    *
538    * @param <T> the return type of the connect method
539    * @param connectable the {@link HConnectable} instance
540    * @return the value returned by the connect method
541    * @throws IOException
542    */
543   @InterfaceAudience.Private
544   public static <T> T execute(HConnectable<T> connectable) throws IOException {
545     if (connectable == null || connectable.conf == null) {
546       return null;
547     }
548     Configuration conf = connectable.conf;
549     HConnection connection = HConnectionManager.getConnection(conf);
550     boolean connectSucceeded = false;
551     try {
552       T returnValue = connectable.connect(connection);
553       connectSucceeded = true;
554       return returnValue;
555     } finally {
556       try {
557         connection.close();
558       } catch (Exception e) {
559         ExceptionUtil.rethrowIfInterrupt(e);
560         if (connectSucceeded) {
561           throw new IOException("The connection to " + connection
562               + " could not be deleted.", e);
563         }
564       }
565     }
566   }
567 
568   /** Encapsulates connection to zookeeper and regionservers.*/
569   @edu.umd.cs.findbugs.annotations.SuppressWarnings(
570       value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
571       justification="Access to the conncurrent hash map is under a lock so should be fine.")
572   static class HConnectionImplementation implements HConnection, Closeable {
573     static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
574     private final long pause;
575     private final int numTries;
576     final int rpcTimeout;
577     private NonceGenerator nonceGenerator = null;
578     private final int prefetchRegionLimit;
579 
580     private volatile boolean closed;
581     private volatile boolean aborted;
582 
583     // package protected for the tests
584     ClusterStatusListener clusterStatusListener;
585 
586     private final Object userRegionLock = new Object();
587 
588     // We have a single lock for master & zk to prevent deadlocks. Having
589     //  one lock for ZK and one lock for master is not possible:
590     //  When creating a connection to master, we need a connection to ZK to get
591     //  its address. But another thread could have taken the ZK lock, and could
592     //  be waiting for the master lock => deadlock.
593     private final Object masterAndZKLock = new Object();
594 
595     private long keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
596     private final DelayedClosing delayedClosing =
597       DelayedClosing.createAndStart(this);
598 
599     // thread executor shared by all HTableInterface instances created
600     // by this connection
601     private volatile ExecutorService batchPool = null;
602     private volatile boolean cleanupPool = false;
603 
604     private final Configuration conf;
605 
606     // Client rpc instance.
607     private RpcClient rpcClient;
608 
609     /**
610       * Map of table to table {@link HRegionLocation}s.
611       */
612     private final ConcurrentMap<TableName, ConcurrentSkipListMap<byte[], HRegionLocation>>
613         cachedRegionLocations =
614       new ConcurrentHashMap<TableName, ConcurrentSkipListMap<byte[], HRegionLocation>>();
615 
616     // The presence of a server in the map implies it's likely that there is an
617     // entry in cachedRegionLocations that map to this server; but the absence
618     // of a server in this map guarentees that there is no entry in cache that
619     // maps to the absent server.
620     // The access to this attribute must be protected by a lock on cachedRegionLocations
621     private final Set<ServerName> cachedServers = new ConcurrentSkipListSet<ServerName>();
622 
623     // region cache prefetch is enabled by default. this set contains all
624     // tables whose region cache prefetch are disabled.
625     private final Set<Integer> regionCachePrefetchDisabledTables =
626       new CopyOnWriteArraySet<Integer>();
627 
628     private int refCount;
629 
630     // indicates whether this connection's life cycle is managed (by us)
631     private boolean managed;
632 
633     private User user;
634 
635     /**
636      * Cluster registry of basic info such as clusterid and meta region location.
637      */
638      Registry registry;
639 
640      HConnectionImplementation(Configuration conf, boolean managed) throws IOException {
641        this(conf, managed, null, null);
642      }
643 
644     /**
645      * constructor
646      * @param conf Configuration object
647      * @param managed If true, does not do full shutdown on close; i.e. cleanup of connection
648      * to zk and shutdown of all services; we just close down the resources this connection was
649      * responsible for and decrement usage counters.  It is up to the caller to do the full
650      * cleanup.  It is set when we want have connection sharing going on -- reuse of zk connection,
651      * and cached region locations, established regionserver connections, etc.  When connections
652      * are shared, we have reference counting going on and will only do full cleanup when no more
653      * users of an HConnectionImplementation instance.
654      */
655     HConnectionImplementation(Configuration conf, boolean managed,
656         ExecutorService pool, User user) throws IOException {
657       this(conf);
658       this.user = user;
659       this.batchPool = pool;
660       this.managed = managed;
661       this.registry = setupRegistry();
662       retrieveClusterId();
663 
664       this.rpcClient = new RpcClient(this.conf, this.clusterId);
665 
666       // Do we publish the status?
667       boolean shouldListen = conf.getBoolean(HConstants.STATUS_PUBLISHED,
668           HConstants.STATUS_PUBLISHED_DEFAULT);
669       Class<? extends ClusterStatusListener.Listener> listenerClass =
670           conf.getClass(ClusterStatusListener.STATUS_LISTENER_CLASS,
671               ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS,
672               ClusterStatusListener.Listener.class);
673       if (shouldListen) {
674         if (listenerClass == null) {
675           LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " +
676               ClusterStatusListener.STATUS_LISTENER_CLASS + " is not set - not listening status");
677         } else {
678           clusterStatusListener = new ClusterStatusListener(
679               new ClusterStatusListener.DeadServerHandler() {
680                 @Override
681                 public void newDead(ServerName sn) {
682                   clearCaches(sn);
683                   rpcClient.cancelConnections(sn.getHostname(), sn.getPort(),
684                       new SocketException(sn.getServerName() +
685                           " is dead: closing its connection."));
686                 }
687               }, conf, listenerClass);
688         }
689       }
690     }
691 
692     /** Dummy nonce generator for disabled nonces. */
693     private static class NoNonceGenerator implements NonceGenerator {
694       @Override
695       public long getNonceGroup() {
696         return HConstants.NO_NONCE;
697       }
698       @Override
699       public long newNonce() {
700         return HConstants.NO_NONCE;
701       }
702     }
703 
704     /**
705      * For tests.
706      */
707     protected HConnectionImplementation(Configuration conf) {
708       this.conf = conf;
709       this.closed = false;
710       this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
711           HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
712       this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
713           HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
714       this.rpcTimeout = conf.getInt(
715           HConstants.HBASE_RPC_TIMEOUT_KEY,
716           HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
717       if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) {
718         synchronized (HConnectionManager.nonceGeneratorCreateLock) {
719           if (HConnectionManager.nonceGenerator == null) {
720             HConnectionManager.nonceGenerator = new PerClientRandomNonceGenerator();
721           }
722           this.nonceGenerator = HConnectionManager.nonceGenerator;
723         }
724       } else {
725         this.nonceGenerator = new NoNonceGenerator();
726       }
727 
728       this.prefetchRegionLimit = conf.getInt(
729           HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
730           HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT);
731     }
732 
733     @Override
734     public HTableInterface getTable(String tableName) throws IOException {
735       return getTable(TableName.valueOf(tableName));
736     }
737 
738     @Override
739     public HTableInterface getTable(byte[] tableName) throws IOException {
740       return getTable(TableName.valueOf(tableName));
741     }
742 
743     @Override
744     public HTableInterface getTable(TableName tableName) throws IOException {
745       return getTable(tableName, getBatchPool());
746     }
747 
748     @Override
749     public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException {
750       return getTable(TableName.valueOf(tableName), pool);
751     }
752 
753     @Override
754     public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException {
755       return getTable(TableName.valueOf(tableName), pool);
756     }
757 
758     @Override
759     public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
760       if (managed) {
761         throw new IOException("The connection has to be unmanaged.");
762       }
763       return new HTable(tableName, this, pool);
764     }
765 
766     private ExecutorService getBatchPool() {
767       if (batchPool == null) {
768         // shared HTable thread executor not yet initialized
769         synchronized (this) {
770           if (batchPool == null) {
771             int maxThreads = conf.getInt("hbase.hconnection.threads.max", 256);
772             int coreThreads = conf.getInt("hbase.hconnection.threads.core", 256);
773             if (maxThreads == 0) {
774               maxThreads = Runtime.getRuntime().availableProcessors() * 8;
775             }
776             if (coreThreads == 0) {
777               coreThreads = Runtime.getRuntime().availableProcessors() * 8;
778             }
779             long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60);
780             LinkedBlockingQueue<Runnable> workQueue =
781               new LinkedBlockingQueue<Runnable>(maxThreads *
782                 conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
783                   HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
784             ThreadPoolExecutor tpe = new ThreadPoolExecutor(
785                 coreThreads,
786                 maxThreads,
787                 keepAliveTime,
788                 TimeUnit.SECONDS,
789                 workQueue,
790                 Threads.newDaemonThreadFactory(toString() + "-shared-"));
791             tpe.allowCoreThreadTimeOut(true);
792             this.batchPool = tpe;
793           }
794           this.cleanupPool = true;
795         }
796       }
797       return this.batchPool;
798     }
799 
800     protected ExecutorService getCurrentBatchPool() {
801       return batchPool;
802     }
803 
804     private void shutdownBatchPool() {
805       if (this.cleanupPool && this.batchPool != null && !this.batchPool.isShutdown()) {
806         this.batchPool.shutdown();
807         try {
808           if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) {
809             this.batchPool.shutdownNow();
810           }
811         } catch (InterruptedException e) {
812           this.batchPool.shutdownNow();
813         }
814       }
815     }
816 
817     /**
818      * @return The cluster registry implementation to use.
819      * @throws IOException
820      */
821     private Registry setupRegistry() throws IOException {
822       String registryClass = this.conf.get("hbase.client.registry.impl",
823         ZooKeeperRegistry.class.getName());
824       Registry registry = null;
825       try {
826         registry = (Registry)Class.forName(registryClass).newInstance();
827       } catch (Throwable t) {
828         throw new IOException(t);
829       }
830       registry.init(this);
831       return registry;
832     }
833 
834     /**
835      * For tests only.
836      * @param rpcClient Client we should use instead.
837      * @return Previous rpcClient
838      */
839     RpcClient setRpcClient(final RpcClient rpcClient) {
840       RpcClient oldRpcClient = this.rpcClient;
841       this.rpcClient = rpcClient;
842       return oldRpcClient;
843     }
844 
845     /**
846      * An identifier that will remain the same for a given connection.
847      * @return
848      */
849     public String toString(){
850       return "hconnection-0x" + Integer.toHexString(hashCode());
851     }
852 
853     protected String clusterId = null;
854 
855     void retrieveClusterId() {
856       if (clusterId != null) return;
857       this.clusterId = this.registry.getClusterId();
858       if (clusterId == null) {
859         clusterId = HConstants.CLUSTER_ID_DEFAULT;
860         LOG.debug("clusterid came back null, using default " + clusterId);
861       }
862     }
863 
864     @Override
865     public Configuration getConfiguration() {
866       return this.conf;
867     }
868 
869     private void checkIfBaseNodeAvailable(ZooKeeperWatcher zkw)
870       throws MasterNotRunningException {
871       String errorMsg;
872       try {
873         if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) {
874           errorMsg = "The node " + zkw.baseZNode+" is not in ZooKeeper. "
875             + "It should have been written by the master. "
876             + "Check the value configured in 'zookeeper.znode.parent'. "
877             + "There could be a mismatch with the one configured in the master.";
878           LOG.error(errorMsg);
879           throw new MasterNotRunningException(errorMsg);
880         }
881       } catch (KeeperException e) {
882         errorMsg = "Can't get connection to ZooKeeper: " + e.getMessage();
883         LOG.error(errorMsg);
884         throw new MasterNotRunningException(errorMsg, e);
885       }
886     }
887 
888     /**
889      * @return true if the master is running, throws an exception otherwise
890      * @throws MasterNotRunningException - if the master is not running
891      * @throws ZooKeeperConnectionException
892      */
893     @Override
894     public boolean isMasterRunning()
895     throws MasterNotRunningException, ZooKeeperConnectionException {
896       // When getting the master connection, we check it's running,
897       // so if there is no exception, it means we've been able to get a
898       // connection on a running master
899       MasterKeepAliveConnection m = getKeepAliveMasterService();
900       m.close();
901       return true;
902     }
903 
904     @Override
905     public HRegionLocation getRegionLocation(final TableName tableName,
906         final byte [] row, boolean reload)
907     throws IOException {
908       return reload? relocateRegion(tableName, row): locateRegion(tableName, row);
909     }
910 
911     @Override
912     public HRegionLocation getRegionLocation(final byte[] tableName,
913         final byte [] row, boolean reload)
914     throws IOException {
915       return getRegionLocation(TableName.valueOf(tableName), row, reload);
916     }
917 
918     @Override
919     public boolean isTableEnabled(TableName tableName) throws IOException {
920       return this.registry.isTableOnlineState(tableName, true);
921     }
922 
923     @Override
924     public boolean isTableEnabled(byte[] tableName) throws IOException {
925       return isTableEnabled(TableName.valueOf(tableName));
926     }
927 
928     @Override
929     public boolean isTableDisabled(TableName tableName) throws IOException {
930       return this.registry.isTableOnlineState(tableName, false);
931     }
932 
933     @Override
934     public boolean isTableDisabled(byte[] tableName) throws IOException {
935       return isTableDisabled(TableName.valueOf(tableName));
936     }
937 
938     @Override
939     public boolean isTableAvailable(final TableName tableName) throws IOException {
940       final AtomicBoolean available = new AtomicBoolean(true);
941       final AtomicInteger regionCount = new AtomicInteger(0);
942       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
943         @Override
944         public boolean processRow(Result row) throws IOException {
945           HRegionInfo info = MetaScanner.getHRegionInfo(row);
946           if (info != null && !info.isSplitParent()) {
947             if (tableName.equals(info.getTable())) {
948               ServerName server = HRegionInfo.getServerName(row);
949               if (server == null) {
950                 available.set(false);
951                 return false;
952               }
953               regionCount.incrementAndGet();
954             } else if (tableName.compareTo(info.getTable()) < 0) {
955               // Return if we are done with the current table
956               return false;
957             }
958           }
959           return true;
960         }
961       };
962       MetaScanner.metaScan(conf, this, visitor, tableName);
963       return available.get() && (regionCount.get() > 0);
964     }
965 
966     @Override
967     public boolean isTableAvailable(final byte[] tableName) throws IOException {
968       return isTableAvailable(TableName.valueOf(tableName));
969     }
970 
971     @Override
972     public boolean isTableAvailable(final TableName tableName, final byte[][] splitKeys)
973         throws IOException {
974       final AtomicBoolean available = new AtomicBoolean(true);
975       final AtomicInteger regionCount = new AtomicInteger(0);
976       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
977         @Override
978         public boolean processRow(Result row) throws IOException {
979           HRegionInfo info = MetaScanner.getHRegionInfo(row);
980           if (info != null && !info.isSplitParent()) {
981             if (tableName.equals(info.getTable())) {
982               ServerName server = HRegionInfo.getServerName(row);
983               if (server == null) {
984                 available.set(false);
985                 return false;
986               }
987               if (!Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
988                 for (byte[] splitKey : splitKeys) {
989                   // Just check if the splitkey is available
990                   if (Bytes.equals(info.getStartKey(), splitKey)) {
991                     regionCount.incrementAndGet();
992                     break;
993                   }
994                 }
995               } else {
996                 // Always empty start row should be counted
997                 regionCount.incrementAndGet();
998               }
999             } else if (tableName.compareTo(info.getTable()) < 0) {
1000               // Return if we are done with the current table
1001               return false;
1002             }
1003           }
1004           return true;
1005         }
1006       };
1007       MetaScanner.metaScan(conf, this, visitor, tableName);
1008       // +1 needs to be added so that the empty start row is also taken into account
1009       return available.get() && (regionCount.get() == splitKeys.length + 1);
1010     }
1011 
1012     @Override
1013     public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys)
1014         throws IOException {
1015       return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1016     }
1017 
1018     @Override
1019     public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
1020       return locateRegion(HRegionInfo.getTable(regionName),
1021           HRegionInfo.getStartKey(regionName), false, true);
1022     }
1023 
1024     @Override
1025     public boolean isDeadServer(ServerName sn) {
1026       if (clusterStatusListener == null) {
1027         return false;
1028       } else {
1029         return clusterStatusListener.isDeadServer(sn);
1030       }
1031     }
1032 
1033     @Override
1034     public List<HRegionLocation> locateRegions(final TableName tableName)
1035     throws IOException {
1036       return locateRegions (tableName, false, true);
1037     }
1038 
1039     @Override
1040     public List<HRegionLocation> locateRegions(final byte[] tableName)
1041     throws IOException {
1042       return locateRegions(TableName.valueOf(tableName));
1043     }
1044 
1045     @Override
1046     public List<HRegionLocation> locateRegions(final TableName tableName,
1047         final boolean useCache, final boolean offlined) throws IOException {
1048       NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, this,
1049           tableName, offlined);
1050       final List<HRegionLocation> locations = new ArrayList<HRegionLocation>();
1051       for (HRegionInfo regionInfo : regions.keySet()) {
1052         locations.add(locateRegion(tableName, regionInfo.getStartKey(), useCache, true));
1053       }
1054       return locations;
1055     }
1056 
1057     @Override
1058     public List<HRegionLocation> locateRegions(final byte[] tableName,
1059        final boolean useCache, final boolean offlined) throws IOException {
1060       return locateRegions(TableName.valueOf(tableName), useCache, offlined);
1061     }
1062 
1063     @Override
1064     public HRegionLocation locateRegion(final TableName tableName,
1065         final byte [] row)
1066     throws IOException{
1067       return locateRegion(tableName, row, true, true);
1068     }
1069 
1070     @Override
1071     public HRegionLocation locateRegion(final byte[] tableName,
1072         final byte [] row)
1073     throws IOException{
1074       return locateRegion(TableName.valueOf(tableName), row);
1075     }
1076 
1077     @Override
1078     public HRegionLocation relocateRegion(final TableName tableName,
1079         final byte [] row) throws IOException{
1080       // Since this is an explicit request not to use any caching, finding
1081       // disabled tables should not be desirable.  This will ensure that an exception is thrown when
1082       // the first time a disabled table is interacted with.
1083       if (isTableDisabled(tableName)) {
1084         throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
1085       }
1086 
1087       return locateRegion(tableName, row, false, true);
1088     }
1089 
1090     @Override
1091     public HRegionLocation relocateRegion(final byte[] tableName,
1092         final byte [] row) throws IOException {
1093       return relocateRegion(TableName.valueOf(tableName), row);
1094     }
1095 
1096 
1097     private HRegionLocation locateRegion(final TableName tableName,
1098       final byte [] row, boolean useCache, boolean retry)
1099     throws IOException {
1100       if (this.closed) throw new IOException(toString() + " closed");
1101       if (tableName== null || tableName.getName().length == 0) {
1102         throw new IllegalArgumentException(
1103             "table name cannot be null or zero length");
1104       }
1105 
1106       if (tableName.equals(TableName.META_TABLE_NAME)) {
1107         return this.registry.getMetaRegionLocation();
1108       } else {
1109         // Region not in the cache - have to go to the meta RS
1110         return locateRegionInMeta(TableName.META_TABLE_NAME, tableName, row,
1111           useCache, userRegionLock, retry);
1112       }
1113     }
1114 
1115     /*
1116      * Search hbase:meta for the HRegionLocation info that contains the table and
1117      * row we're seeking. It will prefetch certain number of regions info and
1118      * save them to the global region cache.
1119      */
1120     private void prefetchRegionCache(final TableName tableName,
1121         final byte[] row) {
1122       // Implement a new visitor for MetaScanner, and use it to walk through
1123       // the hbase:meta
1124       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
1125         public boolean processRow(Result result) throws IOException {
1126           try {
1127             HRegionInfo regionInfo = MetaScanner.getHRegionInfo(result);
1128             if (regionInfo == null) {
1129               return true;
1130             }
1131 
1132             // possible we got a region of a different table...
1133             if (!regionInfo.getTable().equals(tableName)) {
1134               return false; // stop scanning
1135             }
1136             if (regionInfo.isOffline()) {
1137               // don't cache offline regions
1138               return true;
1139             }
1140 
1141             ServerName serverName = HRegionInfo.getServerName(result);
1142             if (serverName == null) {
1143               return true; // don't cache it
1144             }
1145             // instantiate the location
1146             long seqNum = HRegionInfo.getSeqNumDuringOpen(result);
1147             HRegionLocation loc = new HRegionLocation(regionInfo, serverName, seqNum);
1148             // cache this meta entry
1149             cacheLocation(tableName, null, loc);
1150             return true;
1151           } catch (RuntimeException e) {
1152             throw new IOException(e);
1153           }
1154         }
1155       };
1156       try {
1157         // pre-fetch certain number of regions info at region cache.
1158         MetaScanner.metaScan(conf, this, visitor, tableName, row,
1159             this.prefetchRegionLimit, TableName.META_TABLE_NAME);
1160       } catch (IOException e) {
1161         if (ExceptionUtil.isInterrupt(e)) {
1162           Thread.currentThread().interrupt();
1163         } else {
1164           LOG.warn("Encountered problems when prefetch hbase:meta table: ", e);
1165         }
1166       }
1167     }
1168 
1169     /*
1170       * Search the hbase:meta table for the HRegionLocation
1171       * info that contains the table and row we're seeking.
1172       */
1173     private HRegionLocation locateRegionInMeta(final TableName parentTable,
1174       final TableName tableName, final byte [] row, boolean useCache,
1175       Object regionLockObject, boolean retry)
1176     throws IOException {
1177       HRegionLocation location;
1178       // If we are supposed to be using the cache, look in the cache to see if
1179       // we already have the region.
1180       if (useCache) {
1181         location = getCachedLocation(tableName, row);
1182         if (location != null) {
1183           return location;
1184         }
1185       }
1186       int localNumRetries = retry ? numTries : 1;
1187       // build the key of the meta region we should be looking for.
1188       // the extra 9's on the end are necessary to allow "exact" matches
1189       // without knowing the precise region names.
1190       byte [] metaKey = HRegionInfo.createRegionName(tableName, row,
1191         HConstants.NINES, false);
1192       for (int tries = 0; true; tries++) {
1193         if (tries >= localNumRetries) {
1194           throw new NoServerForRegionException("Unable to find region for "
1195             + Bytes.toStringBinary(row) + " after " + numTries + " tries.");
1196         }
1197 
1198         HRegionLocation metaLocation = null;
1199         try {
1200           // locate the meta region
1201           metaLocation = locateRegion(parentTable, metaKey, true, false);
1202           // If null still, go around again.
1203           if (metaLocation == null) continue;
1204           ClientService.BlockingInterface service = getClient(metaLocation.getServerName());
1205 
1206           Result regionInfoRow;
1207           // This block guards against two threads trying to load the meta
1208           // region at the same time. The first will load the meta region and
1209           // the second will use the value that the first one found.
1210           if (useCache) {
1211             if (TableName.META_TABLE_NAME.equals(parentTable) &&
1212                 getRegionCachePrefetch(tableName)) {
1213               synchronized (regionLockObject) {
1214                 // Check the cache again for a hit in case some other thread made the
1215                 // same query while we were waiting on the lock.
1216                 location = getCachedLocation(tableName, row);
1217                 if (location != null) {
1218                   return location;
1219                 }
1220                 // If the parent table is META, we may want to pre-fetch some
1221                 // region info into the global region cache for this table.
1222                 prefetchRegionCache(tableName, row);
1223               }
1224             }
1225             location = getCachedLocation(tableName, row);
1226             if (location != null) {
1227               return location;
1228             }
1229           } else {
1230             // If we are not supposed to be using the cache, delete any existing cached location
1231             // so it won't interfere.
1232             forceDeleteCachedLocation(tableName, row);
1233           }
1234 
1235           // Query the meta region for the location of the meta region
1236           regionInfoRow = ProtobufUtil.getRowOrBefore(service,
1237               metaLocation.getRegionInfo().getRegionName(), metaKey,
1238               HConstants.CATALOG_FAMILY);
1239 
1240           if (regionInfoRow == null) {
1241             throw new TableNotFoundException(tableName);
1242           }
1243 
1244           // convert the row result into the HRegionLocation we need!
1245           HRegionInfo regionInfo = MetaScanner.getHRegionInfo(regionInfoRow);
1246           if (regionInfo == null) {
1247             throw new IOException("HRegionInfo was null or empty in " +
1248               parentTable + ", row=" + regionInfoRow);
1249           }
1250 
1251           // possible we got a region of a different table...
1252           if (!regionInfo.getTable().equals(tableName)) {
1253             throw new TableNotFoundException(
1254                   "Table '" + tableName + "' was not found, got: " +
1255                   regionInfo.getTable() + ".");
1256           }
1257           if (regionInfo.isSplit()) {
1258             throw new RegionOfflineException("the only available region for" +
1259               " the required row is a split parent," +
1260               " the daughters should be online soon: " +
1261               regionInfo.getRegionNameAsString());
1262           }
1263           if (regionInfo.isOffline()) {
1264             throw new RegionOfflineException("the region is offline, could" +
1265               " be caused by a disable table call: " +
1266               regionInfo.getRegionNameAsString());
1267           }
1268 
1269           ServerName serverName = HRegionInfo.getServerName(regionInfoRow);
1270           if (serverName == null) {
1271             throw new NoServerForRegionException("No server address listed " +
1272               "in " + parentTable + " for region " +
1273               regionInfo.getRegionNameAsString() + " containing row " +
1274               Bytes.toStringBinary(row));
1275           }
1276 
1277           if (isDeadServer(serverName)){
1278             throw new RegionServerStoppedException("hbase:meta says the region "+
1279                 regionInfo.getRegionNameAsString()+" is managed by the server " + serverName +
1280                 ", but it is dead.");
1281           }
1282 
1283           // Instantiate the location
1284           location = new HRegionLocation(regionInfo, serverName,
1285             HRegionInfo.getSeqNumDuringOpen(regionInfoRow));
1286           cacheLocation(tableName, null, location);
1287           return location;
1288         } catch (TableNotFoundException e) {
1289           // if we got this error, probably means the table just plain doesn't
1290           // exist. rethrow the error immediately. this should always be coming
1291           // from the HTable constructor.
1292           throw e;
1293         } catch (IOException e) {
1294           ExceptionUtil.rethrowIfInterrupt(e);
1295 
1296           if (e instanceof RemoteException) {
1297             e = ((RemoteException)e).unwrapRemoteException();
1298           }
1299           if (tries < numTries - 1) {
1300             if (LOG.isDebugEnabled()) {
1301               LOG.debug("locateRegionInMeta parentTable=" +
1302                 parentTable + ", metaLocation=" +
1303                 ((metaLocation == null)? "null": "{" + metaLocation + "}") +
1304                 ", attempt=" + tries + " of " +
1305                 this.numTries + " failed; retrying after sleep of " +
1306                 ConnectionUtils.getPauseTime(this.pause, tries) + " because: " + e.getMessage());
1307             }
1308           } else {
1309             throw e;
1310           }
1311           // Only relocate the parent region if necessary
1312           if(!(e instanceof RegionOfflineException ||
1313               e instanceof NoServerForRegionException)) {
1314             relocateRegion(parentTable, metaKey);
1315           }
1316         }
1317         try{
1318           Thread.sleep(ConnectionUtils.getPauseTime(this.pause, tries));
1319         } catch (InterruptedException e) {
1320           throw new InterruptedIOException("Giving up trying to location region in " +
1321             "meta: thread is interrupted.");
1322         }
1323       }
1324     }
1325 
1326     /*
1327      * Search the cache for a location that fits our table and row key.
1328      * Return null if no suitable region is located.
1329      *
1330      * @param tableName
1331      * @param row
1332      * @return Null or region location found in cache.
1333      */
1334     HRegionLocation getCachedLocation(final TableName tableName,
1335         final byte [] row) {
1336       ConcurrentSkipListMap<byte[], HRegionLocation> tableLocations =
1337         getTableLocations(tableName);
1338 
1339       Entry<byte[], HRegionLocation> e = tableLocations.floorEntry(row);
1340       if (e == null) {
1341         return null;
1342       }
1343       HRegionLocation possibleRegion = e.getValue();
1344 
1345       // make sure that the end key is greater than the row we're looking
1346       // for, otherwise the row actually belongs in the next region, not
1347       // this one. the exception case is when the endkey is
1348       // HConstants.EMPTY_END_ROW, signifying that the region we're
1349       // checking is actually the last region in the table.
1350       byte[] endKey = possibleRegion.getRegionInfo().getEndKey();
1351       if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
1352           tableName.getRowComparator().compareRows(
1353               endKey, 0, endKey.length, row, 0, row.length) > 0) {
1354         return possibleRegion;
1355       }
1356 
1357       // Passed all the way through, so we got nothing - complete cache miss
1358       return null;
1359     }
1360 
1361     /**
1362      * Delete a cached location, no matter what it is. Called when we were told to not use cache.
1363      * @param tableName tableName
1364      * @param row
1365      */
1366     void forceDeleteCachedLocation(final TableName tableName, final byte [] row) {
1367       HRegionLocation rl = null;
1368       Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
1369       // start to examine the cache. we can only do cache actions
1370       // if there's something in the cache for this table.
1371       rl = getCachedLocation(tableName, row);
1372       if (rl != null) {
1373         tableLocations.remove(rl.getRegionInfo().getStartKey());
1374       }
1375       if ((rl != null) && LOG.isDebugEnabled()) {
1376         LOG.debug("Removed " + rl.getHostname() + ":" + rl.getPort()
1377           + " as a location of " + rl.getRegionInfo().getRegionNameAsString() +
1378           " for tableName=" + tableName + " from cache");
1379       }
1380     }
1381 
1382     /*
1383      * Delete all cached entries of a table that maps to a specific location.
1384      */
1385     @Override
1386     public void clearCaches(final ServerName serverName) {
1387       if (!this.cachedServers.contains(serverName)) {
1388         return;
1389       }
1390 
1391       boolean deletedSomething = false;
1392       synchronized (this.cachedServers) {
1393         // We block here, because if there is an error on a server, it's likely that multiple
1394         //  threads will get the error  simultaneously. If there are hundreds of thousand of
1395         //  region location to check, it's better to do this only once. A better pattern would
1396         //  be to check if the server is dead when we get the region location.
1397         if (!this.cachedServers.contains(serverName)) {
1398           return;
1399         }
1400         for (Map<byte[], HRegionLocation> tableLocations : cachedRegionLocations.values()) {
1401           for (Entry<byte[], HRegionLocation> e : tableLocations.entrySet()) {
1402             HRegionLocation value = e.getValue();
1403             if (value != null
1404                 && serverName.equals(value.getServerName())) {
1405               tableLocations.remove(e.getKey());
1406               deletedSomething = true;
1407             }
1408           }
1409         }
1410         this.cachedServers.remove(serverName);
1411       }
1412       if (deletedSomething && LOG.isDebugEnabled()) {
1413         LOG.debug("Removed all cached region locations that map to " + serverName);
1414       }
1415     }
1416 
1417     /*
1418      * @param tableName
1419      * @return Map of cached locations for passed <code>tableName</code>
1420      */
1421     private ConcurrentSkipListMap<byte[], HRegionLocation> getTableLocations(
1422         final TableName tableName) {
1423       // find the map of cached locations for this table
1424       ConcurrentSkipListMap<byte[], HRegionLocation> result;
1425       result = this.cachedRegionLocations.get(tableName);
1426       // if tableLocations for this table isn't built yet, make one
1427       if (result == null) {
1428         result = new ConcurrentSkipListMap<byte[], HRegionLocation>(Bytes.BYTES_COMPARATOR);
1429         ConcurrentSkipListMap<byte[], HRegionLocation> old =
1430             this.cachedRegionLocations.putIfAbsent(tableName, result);
1431         if (old != null) {
1432           return old;
1433         }
1434       }
1435       return result;
1436     }
1437 
1438     @Override
1439     public void clearRegionCache() {
1440       this.cachedRegionLocations.clear();
1441       this.cachedServers.clear();
1442     }
1443 
1444     @Override
1445     public void clearRegionCache(final TableName tableName) {
1446       this.cachedRegionLocations.remove(tableName);
1447     }
1448 
1449     @Override
1450     public void clearRegionCache(final byte[] tableName) {
1451       clearRegionCache(TableName.valueOf(tableName));
1452     }
1453 
1454     /**
1455      * Put a newly discovered HRegionLocation into the cache.
1456      * @param tableName The table name.
1457      * @param source the source of the new location, if it's not coming from meta
1458      * @param location the new location
1459      */
1460     private void cacheLocation(final TableName tableName, final HRegionLocation source,
1461         final HRegionLocation location) {
1462       boolean isFromMeta = (source == null);
1463       byte [] startKey = location.getRegionInfo().getStartKey();
1464       ConcurrentMap<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
1465       HRegionLocation oldLocation = tableLocations.putIfAbsent(startKey, location);
1466       boolean isNewCacheEntry = (oldLocation == null);
1467       if (isNewCacheEntry) {
1468         cachedServers.add(location.getServerName());
1469         return;
1470       }
1471       boolean updateCache;
1472       // If the server in cache sends us a redirect, assume it's always valid.
1473       if (oldLocation.equals(source)) {
1474         updateCache = true;
1475       } else {
1476         long newLocationSeqNum = location.getSeqNum();
1477         // Meta record is stale - some (probably the same) server has closed the region
1478         // with later seqNum and told us about the new location.
1479         boolean isStaleMetaRecord = isFromMeta && (oldLocation.getSeqNum() > newLocationSeqNum);
1480         // Same as above for redirect. However, in this case, if the number is equal to previous
1481         // record, the most common case is that first the region was closed with seqNum, and then
1482         // opened with the same seqNum; hence we will ignore the redirect.
1483         // There are so many corner cases with various combinations of opens and closes that
1484         // an additional counter on top of seqNum would be necessary to handle them all.
1485         boolean isStaleRedirect = !isFromMeta && (oldLocation.getSeqNum() >= newLocationSeqNum);
1486         boolean isStaleUpdate = (isStaleMetaRecord || isStaleRedirect);
1487         updateCache = (!isStaleUpdate);
1488       }
1489       if (updateCache) {
1490         tableLocations.replace(startKey, oldLocation, location);
1491         cachedServers.add(location.getServerName());
1492       }
1493     }
1494 
1495     // Map keyed by service name + regionserver to service stub implementation
1496     private final ConcurrentHashMap<String, Object> stubs =
1497       new ConcurrentHashMap<String, Object>();
1498     // Map of locks used creating service stubs per regionserver.
1499     private final ConcurrentHashMap<String, String> connectionLock =
1500       new ConcurrentHashMap<String, String>();
1501 
1502     /**
1503      * State of the MasterService connection/setup.
1504      */
1505     static class MasterServiceState {
1506       HConnection connection;
1507       MasterService.BlockingInterface stub;
1508       int userCount;
1509       long keepAliveUntil = Long.MAX_VALUE;
1510 
1511       MasterServiceState (final HConnection connection) {
1512         super();
1513         this.connection = connection;
1514       }
1515 
1516       @Override
1517       public String toString() {
1518         return "MasterService";
1519       }
1520 
1521       Object getStub() {
1522         return this.stub;
1523       }
1524 
1525       void clearStub() {
1526         this.stub = null;
1527       }
1528 
1529       boolean isMasterRunning() throws ServiceException {
1530         IsMasterRunningResponse response =
1531           this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
1532         return response != null? response.getIsMasterRunning(): false;
1533       }
1534     }
1535 
1536     /**
1537      * Makes a client-side stub for master services. Sub-class to specialize.
1538      * Depends on hosting class so not static.  Exists so we avoid duplicating a bunch of code
1539      * when setting up the MasterMonitorService and MasterAdminService.
1540      */
1541     abstract class StubMaker {
1542       /**
1543        * Returns the name of the service stub being created.
1544        */
1545       protected abstract String getServiceName();
1546 
1547       /**
1548        * Make stub and cache it internal so can be used later doing the isMasterRunning call.
1549        * @param channel
1550        */
1551       protected abstract Object makeStub(final BlockingRpcChannel channel);
1552 
1553       /**
1554        * Once setup, check it works by doing isMasterRunning check.
1555        * @throws ServiceException
1556        */
1557       protected abstract void isMasterRunning() throws ServiceException;
1558 
1559       /**
1560        * Create a stub. Try once only.  It is not typed because there is no common type to
1561        * protobuf services nor their interfaces.  Let the caller do appropriate casting.
1562        * @return A stub for master services.
1563        * @throws IOException
1564        * @throws KeeperException
1565        * @throws ServiceException
1566        */
1567       private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException {
1568         ZooKeeperKeepAliveConnection zkw;
1569         try {
1570           zkw = getKeepAliveZooKeeperWatcher();
1571         } catch (IOException e) {
1572           ExceptionUtil.rethrowIfInterrupt(e);
1573           throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
1574         }
1575         try {
1576           checkIfBaseNodeAvailable(zkw);
1577           ServerName sn = MasterAddressTracker.getMasterAddress(zkw);
1578           if (sn == null) {
1579             String msg = "ZooKeeper available but no active master location found";
1580             LOG.info(msg);
1581             throw new MasterNotRunningException(msg);
1582           }
1583           if (isDeadServer(sn)) {
1584             throw new MasterNotRunningException(sn + " is dead.");
1585           }
1586           // Use the security info interface name as our stub key
1587           String key = getStubKey(getServiceName(), sn.getHostAndPort());
1588           connectionLock.putIfAbsent(key, key);
1589           Object stub = null;
1590           synchronized (connectionLock.get(key)) {
1591             stub = stubs.get(key);
1592             if (stub == null) {
1593               BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn,
1594                 user, rpcTimeout);
1595               stub = makeStub(channel);
1596               isMasterRunning();
1597               stubs.put(key, stub);
1598             }
1599           }
1600           return stub;
1601         } finally {
1602           zkw.close();
1603         }
1604       }
1605 
1606       /**
1607        * Create a stub against the master.  Retry if necessary.
1608        * @return A stub to do <code>intf</code> against the master
1609        * @throws MasterNotRunningException
1610        */
1611       @edu.umd.cs.findbugs.annotations.SuppressWarnings (value="SWL_SLEEP_WITH_LOCK_HELD")
1612       Object makeStub() throws MasterNotRunningException {
1613         // The lock must be at the beginning to prevent multiple master creations
1614         //  (and leaks) in a multithread context
1615         synchronized (masterAndZKLock) {
1616           Exception exceptionCaught = null;
1617           Object stub = null;
1618           int tries = 0;
1619           while (!closed && stub == null) {
1620             tries++;
1621             try {
1622               stub = makeStubNoRetries();
1623             } catch (IOException e) {
1624               exceptionCaught = e;
1625             } catch (KeeperException e) {
1626               exceptionCaught = e;
1627             } catch (ServiceException e) {
1628               exceptionCaught = e;
1629             }
1630 
1631             if (exceptionCaught != null)
1632               // It failed. If it's not the last try, we're going to wait a little
1633               if (tries < numTries && !ExceptionUtil.isInterrupt(exceptionCaught)) {
1634                 // tries at this point is 1 or more; decrement to start from 0.
1635                 long pauseTime = ConnectionUtils.getPauseTime(pause, tries - 1);
1636                 LOG.info("getMaster attempt " + tries + " of " + numTries +
1637                     " failed; retrying after sleep of " + pauseTime + ", exception=" +
1638                   exceptionCaught);
1639 
1640                 try {
1641                   Thread.sleep(pauseTime);
1642                 } catch (InterruptedException e) {
1643                   throw new MasterNotRunningException(
1644                       "Thread was interrupted while trying to connect to master.", e);
1645                 }
1646               } else {
1647                 // Enough tries, we stop now
1648                 LOG.info("getMaster attempt " + tries + " of " + numTries +
1649                     " failed; no more retrying.", exceptionCaught);
1650                 throw new MasterNotRunningException(exceptionCaught);
1651               }
1652           }
1653 
1654           if (stub == null) {
1655             // implies this.closed true
1656             throw new MasterNotRunningException("Connection was closed while trying to get master");
1657           }
1658           return stub;
1659         }
1660       }
1661     }
1662 
1663     /**
1664      * Class to make a MasterServiceStubMaker stub.
1665      */
1666     class MasterServiceStubMaker extends StubMaker {
1667       private MasterService.BlockingInterface stub;
1668       @Override
1669       protected String getServiceName() {
1670         return MasterService.getDescriptor().getName();
1671       }
1672 
1673       @Override
1674       @edu.umd.cs.findbugs.annotations.SuppressWarnings("SWL_SLEEP_WITH_LOCK_HELD")
1675       MasterService.BlockingInterface makeStub() throws MasterNotRunningException {
1676         return (MasterService.BlockingInterface)super.makeStub();
1677       }
1678 
1679       @Override
1680       protected Object makeStub(BlockingRpcChannel channel) {
1681         this.stub = MasterService.newBlockingStub(channel);
1682         return this.stub;
1683       }
1684 
1685       @Override
1686       protected void isMasterRunning() throws ServiceException {
1687         this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
1688       }
1689     }
1690 
1691     @Override
1692     public AdminService.BlockingInterface getAdmin(final ServerName serverName)
1693         throws IOException {
1694       return getAdmin(serverName, false);
1695     }
1696 
1697     @Override
1698     // Nothing is done w/ the 'master' parameter.  It is ignored.
1699     public AdminService.BlockingInterface getAdmin(final ServerName serverName,
1700       final boolean master)
1701     throws IOException {
1702       if (isDeadServer(serverName)) {
1703         throw new RegionServerStoppedException(serverName + " is dead.");
1704       }
1705       String key = getStubKey(AdminService.BlockingInterface.class.getName(),
1706         serverName.getHostAndPort());
1707       this.connectionLock.putIfAbsent(key, key);
1708       AdminService.BlockingInterface stub = null;
1709       synchronized (this.connectionLock.get(key)) {
1710         stub = (AdminService.BlockingInterface)this.stubs.get(key);
1711         if (stub == null) {
1712           BlockingRpcChannel channel = this.rpcClient.createBlockingRpcChannel(serverName,
1713             user, this.rpcTimeout);
1714           stub = AdminService.newBlockingStub(channel);
1715           this.stubs.put(key, stub);
1716         }
1717       }
1718       return stub;
1719     }
1720 
1721     @Override
1722     public ClientService.BlockingInterface getClient(final ServerName sn)
1723     throws IOException {
1724       if (isDeadServer(sn)) {
1725         throw new RegionServerStoppedException(sn + " is dead.");
1726       }
1727       String key = getStubKey(ClientService.BlockingInterface.class.getName(), sn.getHostAndPort());
1728       this.connectionLock.putIfAbsent(key, key);
1729       ClientService.BlockingInterface stub = null;
1730       synchronized (this.connectionLock.get(key)) {
1731         stub = (ClientService.BlockingInterface)this.stubs.get(key);
1732         if (stub == null) {
1733           BlockingRpcChannel channel = this.rpcClient.createBlockingRpcChannel(sn,
1734             user, this.rpcTimeout);
1735           stub = ClientService.newBlockingStub(channel);
1736           // In old days, after getting stub/proxy, we'd make a call.  We are not doing that here.
1737           // Just fail on first actual call rather than in here on setup.
1738           this.stubs.put(key, stub);
1739         }
1740       }
1741       return stub;
1742     }
1743 
1744     static String getStubKey(final String serviceName, final String rsHostnamePort) {
1745       return serviceName + "@" + rsHostnamePort;
1746     }
1747 
1748     private ZooKeeperKeepAliveConnection keepAliveZookeeper;
1749     private int keepAliveZookeeperUserCount;
1750     private boolean canCloseZKW = true;
1751 
1752     // keepAlive time, in ms. No reason to make it configurable.
1753     private static final long keepAlive = 5 * 60 * 1000;
1754 
1755     /**
1756      * Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it.
1757      * @return The shared instance. Never returns null.
1758      */
1759     ZooKeeperKeepAliveConnection getKeepAliveZooKeeperWatcher()
1760       throws IOException {
1761       synchronized (masterAndZKLock) {
1762         if (keepAliveZookeeper == null) {
1763           if (this.closed) {
1764             throw new IOException(toString() + " closed");
1765           }
1766           // We don't check that our link to ZooKeeper is still valid
1767           // But there is a retry mechanism in the ZooKeeperWatcher itself
1768           keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this);
1769         }
1770         keepAliveZookeeperUserCount++;
1771         keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
1772         return keepAliveZookeeper;
1773       }
1774     }
1775 
1776     void releaseZooKeeperWatcher(final ZooKeeperWatcher zkw) {
1777       if (zkw == null){
1778         return;
1779       }
1780       synchronized (masterAndZKLock) {
1781         --keepAliveZookeeperUserCount;
1782         if (keepAliveZookeeperUserCount <= 0 ){
1783           keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive;
1784         }
1785       }
1786     }
1787 
1788     /**
1789      * Creates a Chore thread to check the connections to master & zookeeper
1790      *  and close them when they reach their closing time (
1791      *  {@link MasterServiceState#keepAliveUntil} and
1792      *  {@link #keepZooKeeperWatcherAliveUntil}). Keep alive time is
1793      *  managed by the release functions and the variable {@link #keepAlive}
1794      */
1795     private static class DelayedClosing extends Chore implements Stoppable {
1796       private HConnectionImplementation hci;
1797       Stoppable stoppable;
1798 
1799       private DelayedClosing(
1800         HConnectionImplementation hci, Stoppable stoppable){
1801         super(
1802           "ZooKeeperWatcher and Master delayed closing for connection "+hci,
1803           60*1000, // We check every minutes
1804           stoppable);
1805         this.hci = hci;
1806         this.stoppable = stoppable;
1807       }
1808 
1809       static DelayedClosing createAndStart(HConnectionImplementation hci){
1810         Stoppable stoppable = new Stoppable() {
1811               private volatile boolean isStopped = false;
1812               @Override public void stop(String why) { isStopped = true;}
1813               @Override public boolean isStopped() {return isStopped;}
1814             };
1815 
1816         return new DelayedClosing(hci, stoppable);
1817       }
1818 
1819       protected void closeMasterProtocol(MasterServiceState protocolState) {
1820         if (System.currentTimeMillis() > protocolState.keepAliveUntil) {
1821           hci.closeMasterService(protocolState);
1822           protocolState.keepAliveUntil = Long.MAX_VALUE;
1823         }
1824       }
1825 
1826       @Override
1827       protected void chore() {
1828         synchronized (hci.masterAndZKLock) {
1829           if (hci.canCloseZKW) {
1830             if (System.currentTimeMillis() >
1831               hci.keepZooKeeperWatcherAliveUntil) {
1832 
1833               hci.closeZooKeeperWatcher();
1834               hci.keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
1835             }
1836           }
1837           closeMasterProtocol(hci.masterServiceState);
1838           closeMasterProtocol(hci.masterServiceState);
1839         }
1840       }
1841 
1842       @Override
1843       public void stop(String why) {
1844         stoppable.stop(why);
1845       }
1846 
1847       @Override
1848       public boolean isStopped() {
1849         return stoppable.isStopped();
1850       }
1851     }
1852 
1853     private void closeZooKeeperWatcher() {
1854       synchronized (masterAndZKLock) {
1855         if (keepAliveZookeeper != null) {
1856           LOG.info("Closing zookeeper sessionid=0x" +
1857             Long.toHexString(
1858               keepAliveZookeeper.getRecoverableZooKeeper().getSessionId()));
1859           keepAliveZookeeper.internalClose();
1860           keepAliveZookeeper = null;
1861         }
1862         keepAliveZookeeperUserCount = 0;
1863       }
1864     }
1865 
1866     final MasterServiceState masterServiceState = new MasterServiceState(this);
1867 
1868     @Override
1869     public MasterService.BlockingInterface getMaster() throws MasterNotRunningException {
1870       return getKeepAliveMasterService();
1871     }
1872 
1873     private void resetMasterServiceState(final MasterServiceState mss) {
1874       mss.userCount++;
1875       mss.keepAliveUntil = Long.MAX_VALUE;
1876     }
1877 
1878     @Override
1879     public MasterKeepAliveConnection getKeepAliveMasterService()
1880     throws MasterNotRunningException {
1881       synchronized (masterAndZKLock) {
1882         if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) {
1883           MasterServiceStubMaker stubMaker = new MasterServiceStubMaker();
1884           this.masterServiceState.stub = stubMaker.makeStub();
1885         }
1886         resetMasterServiceState(this.masterServiceState);
1887       }
1888       // Ugly delegation just so we can add in a Close method.
1889       final MasterService.BlockingInterface stub = this.masterServiceState.stub;
1890       return new MasterKeepAliveConnection() {
1891         MasterServiceState mss = masterServiceState;
1892         @Override
1893         public AddColumnResponse addColumn(RpcController controller, AddColumnRequest request)
1894         throws ServiceException {
1895           return stub.addColumn(controller, request);
1896         }
1897 
1898         @Override
1899         public DeleteColumnResponse deleteColumn(RpcController controller,
1900             DeleteColumnRequest request)
1901         throws ServiceException {
1902           return stub.deleteColumn(controller, request);
1903         }
1904 
1905         @Override
1906         public ModifyColumnResponse modifyColumn(RpcController controller,
1907             ModifyColumnRequest request)
1908         throws ServiceException {
1909           return stub.modifyColumn(controller, request);
1910         }
1911 
1912         @Override
1913         public MoveRegionResponse moveRegion(RpcController controller,
1914             MoveRegionRequest request) throws ServiceException {
1915           return stub.moveRegion(controller, request);
1916         }
1917 
1918         @Override
1919         public DispatchMergingRegionsResponse dispatchMergingRegions(
1920             RpcController controller, DispatchMergingRegionsRequest request)
1921             throws ServiceException {
1922           return stub.dispatchMergingRegions(controller, request);
1923         }
1924 
1925         @Override
1926         public AssignRegionResponse assignRegion(RpcController controller,
1927             AssignRegionRequest request) throws ServiceException {
1928           return stub.assignRegion(controller, request);
1929         }
1930 
1931         @Override
1932         public UnassignRegionResponse unassignRegion(RpcController controller,
1933             UnassignRegionRequest request) throws ServiceException {
1934           return stub.unassignRegion(controller, request);
1935         }
1936 
1937         @Override
1938         public OfflineRegionResponse offlineRegion(RpcController controller,
1939             OfflineRegionRequest request) throws ServiceException {
1940           return stub.offlineRegion(controller, request);
1941         }
1942 
1943         @Override
1944         public DeleteTableResponse deleteTable(RpcController controller,
1945             DeleteTableRequest request) throws ServiceException {
1946           return stub.deleteTable(controller, request);
1947         }
1948 
1949         @Override
1950         public EnableTableResponse enableTable(RpcController controller,
1951             EnableTableRequest request) throws ServiceException {
1952           return stub.enableTable(controller, request);
1953         }
1954 
1955         @Override
1956         public DisableTableResponse disableTable(RpcController controller,
1957             DisableTableRequest request) throws ServiceException {
1958           return stub.disableTable(controller, request);
1959         }
1960 
1961         @Override
1962         public ModifyTableResponse modifyTable(RpcController controller,
1963             ModifyTableRequest request) throws ServiceException {
1964           return stub.modifyTable(controller, request);
1965         }
1966 
1967         @Override
1968         public CreateTableResponse createTable(RpcController controller,
1969             CreateTableRequest request) throws ServiceException {
1970           return stub.createTable(controller, request);
1971         }
1972 
1973         @Override
1974         public ShutdownResponse shutdown(RpcController controller,
1975             ShutdownRequest request) throws ServiceException {
1976           return stub.shutdown(controller, request);
1977         }
1978 
1979         @Override
1980         public StopMasterResponse stopMaster(RpcController controller,
1981             StopMasterRequest request) throws ServiceException {
1982           return stub.stopMaster(controller, request);
1983         }
1984 
1985         @Override
1986         public BalanceResponse balance(RpcController controller,
1987             BalanceRequest request) throws ServiceException {
1988           return stub.balance(controller, request);
1989         }
1990 
1991         @Override
1992         public SetBalancerRunningResponse setBalancerRunning(
1993             RpcController controller, SetBalancerRunningRequest request)
1994             throws ServiceException {
1995           return stub.setBalancerRunning(controller, request);
1996         }
1997 
1998         @Override
1999         public RunCatalogScanResponse runCatalogScan(RpcController controller,
2000             RunCatalogScanRequest request) throws ServiceException {
2001           return stub.runCatalogScan(controller, request);
2002         }
2003 
2004         @Override
2005         public EnableCatalogJanitorResponse enableCatalogJanitor(
2006             RpcController controller, EnableCatalogJanitorRequest request)
2007             throws ServiceException {
2008           return stub.enableCatalogJanitor(controller, request);
2009         }
2010 
2011         @Override
2012         public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(
2013             RpcController controller, IsCatalogJanitorEnabledRequest request)
2014             throws ServiceException {
2015           return stub.isCatalogJanitorEnabled(controller, request);
2016         }
2017 
2018         @Override
2019         public CoprocessorServiceResponse execMasterService(
2020             RpcController controller, CoprocessorServiceRequest request)
2021             throws ServiceException {
2022           return stub.execMasterService(controller, request);
2023         }
2024 
2025         @Override
2026         public SnapshotResponse snapshot(RpcController controller,
2027             SnapshotRequest request) throws ServiceException {
2028           return stub.snapshot(controller, request);
2029         }
2030 
2031         @Override
2032         public GetCompletedSnapshotsResponse getCompletedSnapshots(
2033             RpcController controller, GetCompletedSnapshotsRequest request)
2034             throws ServiceException {
2035           return stub.getCompletedSnapshots(controller, request);
2036         }
2037 
2038         @Override
2039         public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
2040             DeleteSnapshotRequest request) throws ServiceException {
2041           return stub.deleteSnapshot(controller, request);
2042         }
2043 
2044         @Override
2045         public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
2046             IsSnapshotDoneRequest request) throws ServiceException {
2047           return stub.isSnapshotDone(controller, request);
2048         }
2049 
2050         @Override
2051         public RestoreSnapshotResponse restoreSnapshot(
2052             RpcController controller, RestoreSnapshotRequest request)
2053             throws ServiceException {
2054           return stub.restoreSnapshot(controller, request);
2055         }
2056 
2057         @Override
2058         public IsRestoreSnapshotDoneResponse isRestoreSnapshotDone(
2059             RpcController controller, IsRestoreSnapshotDoneRequest request)
2060             throws ServiceException {
2061           return stub.isRestoreSnapshotDone(controller, request);
2062         }
2063 
2064         @Override
2065         public ExecProcedureResponse execProcedure(
2066             RpcController controller, ExecProcedureRequest request)
2067             throws ServiceException {
2068           return stub.execProcedure(controller, request);
2069         }
2070 
2071         @Override
2072         public IsProcedureDoneResponse isProcedureDone(RpcController controller,
2073             IsProcedureDoneRequest request) throws ServiceException {
2074           return stub.isProcedureDone(controller, request);
2075         }
2076 
2077         @Override
2078         public IsMasterRunningResponse isMasterRunning(
2079             RpcController controller, IsMasterRunningRequest request)
2080             throws ServiceException {
2081           return stub.isMasterRunning(controller, request);
2082         }
2083 
2084         @Override
2085         public ModifyNamespaceResponse modifyNamespace(RpcController controller,
2086             ModifyNamespaceRequest request)
2087         throws ServiceException {
2088           return stub.modifyNamespace(controller, request);
2089         }
2090 
2091         @Override
2092         public CreateNamespaceResponse createNamespace(RpcController controller, CreateNamespaceRequest request) throws ServiceException {
2093           return stub.createNamespace(controller, request);
2094         }
2095 
2096         @Override
2097         public DeleteNamespaceResponse deleteNamespace(RpcController controller, DeleteNamespaceRequest request) throws ServiceException {
2098           return stub.deleteNamespace(controller, request);
2099         }
2100 
2101         @Override
2102         public GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller, GetNamespaceDescriptorRequest request) throws ServiceException {
2103           return stub.getNamespaceDescriptor(controller, request);
2104         }
2105 
2106         @Override
2107         public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller, ListNamespaceDescriptorsRequest request) throws ServiceException {
2108           return stub.listNamespaceDescriptors(controller, request);
2109         }
2110 
2111         @Override
2112         public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController controller, ListTableDescriptorsByNamespaceRequest request) throws ServiceException {
2113           return stub.listTableDescriptorsByNamespace(controller, request);
2114         }
2115 
2116         @Override
2117         public ListTableNamesByNamespaceResponse listTableNamesByNamespace(RpcController controller,
2118               ListTableNamesByNamespaceRequest request) throws ServiceException {
2119           return stub.listTableNamesByNamespace(controller, request);
2120         }
2121 
2122         @Override
2123         public void close() {
2124           release(this.mss);
2125         }
2126 
2127         @Override
2128         public GetSchemaAlterStatusResponse getSchemaAlterStatus(
2129             RpcController controller, GetSchemaAlterStatusRequest request)
2130             throws ServiceException {
2131           return stub.getSchemaAlterStatus(controller, request);
2132         }
2133 
2134         @Override
2135         public GetTableDescriptorsResponse getTableDescriptors(
2136             RpcController controller, GetTableDescriptorsRequest request)
2137             throws ServiceException {
2138           return stub.getTableDescriptors(controller, request);
2139         }
2140 
2141         @Override
2142         public GetTableNamesResponse getTableNames(
2143             RpcController controller, GetTableNamesRequest request)
2144             throws ServiceException {
2145           return stub.getTableNames(controller, request);
2146         }
2147 
2148         @Override
2149         public GetClusterStatusResponse getClusterStatus(
2150             RpcController controller, GetClusterStatusRequest request)
2151             throws ServiceException {
2152           return stub.getClusterStatus(controller, request);
2153         }
2154       };
2155     }
2156  
2157 
2158     private static void release(MasterServiceState mss) {
2159       if (mss != null && mss.connection != null) {
2160         ((HConnectionImplementation)mss.connection).releaseMaster(mss);
2161       }
2162     }
2163 
2164     private boolean isKeepAliveMasterConnectedAndRunning(MasterServiceState mss) {
2165       if (mss.getStub() == null){
2166         return false;
2167       }
2168       try {
2169         return mss.isMasterRunning();
2170       } catch (UndeclaredThrowableException e) {
2171         // It's somehow messy, but we can receive exceptions such as
2172         //  java.net.ConnectException but they're not declared. So we catch it...
2173         LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable());
2174         return false;
2175       } catch (ServiceException se) {
2176         LOG.warn("Checking master connection", se);
2177         return false;
2178       }
2179     }
2180 
2181     void releaseMaster(MasterServiceState mss) {
2182       if (mss.getStub() == null) return;
2183       synchronized (masterAndZKLock) {
2184         --mss.userCount;
2185         if (mss.userCount <= 0) {
2186           mss.keepAliveUntil = System.currentTimeMillis() + keepAlive;
2187         }
2188       }
2189     }
2190 
2191     private void closeMasterService(MasterServiceState mss) {
2192       if (mss.getStub() != null) {
2193         LOG.info("Closing master protocol: " + mss);
2194         mss.clearStub();
2195       }
2196       mss.userCount = 0;
2197     }
2198 
2199     /**
2200      * Immediate close of the shared master. Can be by the delayed close or when closing the
2201      * connection itself.
2202      */
2203     private void closeMaster() {
2204       synchronized (masterAndZKLock) {
2205         closeMasterService(masterServiceState);
2206       }
2207     }
2208 
2209     void updateCachedLocation(HRegionInfo hri, HRegionLocation source,
2210                               ServerName serverName, long seqNum) {
2211       HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum);
2212       cacheLocation(hri.getTable(), source, newHrl);
2213     }
2214 
2215    /**
2216     * Deletes the cached location of the region if necessary, based on some error from source.
2217     * @param hri The region in question.
2218     * @param source The source of the error that prompts us to invalidate cache.
2219     */
2220    void deleteCachedLocation(HRegionInfo hri, HRegionLocation source) {
2221      ConcurrentMap<byte[], HRegionLocation> tableLocations = getTableLocations(hri.getTable());
2222      tableLocations.remove(hri.getStartKey(), source);
2223    }
2224 
2225     @Override
2226     public void deleteCachedRegionLocation(final HRegionLocation location) {
2227       if (location == null) {
2228         return;
2229       }
2230 
2231       HRegionLocation removedLocation;
2232       TableName tableName = location.getRegionInfo().getTable();
2233       Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
2234       removedLocation = tableLocations.remove(location.getRegionInfo().getStartKey());
2235       if (LOG.isDebugEnabled() && removedLocation != null) {
2236         LOG.debug("Removed " +
2237             location.getRegionInfo().getRegionNameAsString() +
2238             " for tableName=" + tableName +
2239             " from cache");
2240       }
2241     }
2242 
2243     /**
2244      * Update the location with the new value (if the exception is a RegionMovedException)
2245      * or delete it from the cache. Does nothing if we can be sure from the exception that
2246      * the location is still accurate, or if the cache has already been updated.
2247      * @param exception an object (to simplify user code) on which we will try to find a nested
2248      *                  or wrapped or both RegionMovedException
2249      * @param source server that is the source of the location update.
2250      */
2251     @Override
2252     public void updateCachedLocations(final TableName tableName, byte[] rowkey,
2253       final Object exception, final HRegionLocation source) {
2254       if (rowkey == null || tableName == null) {
2255         LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) +
2256             ", tableName=" + (tableName == null ? "null" : tableName));
2257         return;
2258       }
2259 
2260       if (source == null || source.getServerName() == null){
2261         // This should not happen, but let's secure ourselves.
2262         return;
2263       }
2264 
2265       // Is it something we have already updated?
2266       final HRegionLocation oldLocation = getCachedLocation(tableName, rowkey);
2267       if (oldLocation == null || !source.getServerName().equals(oldLocation.getServerName())) {
2268         // There is no such location in the cache (it's been removed already) or
2269         // the cache has already been refreshed with a different location.  => nothing to do
2270         return;
2271       }
2272 
2273       HRegionInfo regionInfo = oldLocation.getRegionInfo();
2274       Throwable cause = findException(exception);
2275       if (cause != null) {
2276         if (cause instanceof RegionTooBusyException || cause instanceof RegionOpeningException) {
2277           // We know that the region is still on this region server
2278           return;
2279         }
2280 
2281         if (cause instanceof RegionMovedException) {
2282           RegionMovedException rme = (RegionMovedException) cause;
2283           if (LOG.isTraceEnabled()) {
2284             LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " +
2285                 rme.getHostname() + ":" + rme.getPort() +
2286                 " according to " + source.getHostnamePort());
2287           }
2288           // We know that the region is not anymore on this region server, but we know
2289           //  the new location.
2290           updateCachedLocation(
2291               regionInfo, source, rme.getServerName(), rme.getLocationSeqNum());
2292           return;
2293         }
2294       }
2295 
2296       // If we're here, it means that can cannot be sure about the location, so we remove it from
2297       //  the cache.
2298       deleteCachedLocation(regionInfo, source);
2299     }
2300 
2301     @Override
2302     public void updateCachedLocations(final byte[] tableName, byte[] rowkey,
2303       final Object exception, final HRegionLocation source) {
2304       updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source);
2305     }
2306 
2307     @Override
2308     @Deprecated
2309     public void processBatch(List<? extends Row> list,
2310         final TableName tableName,
2311         ExecutorService pool,
2312         Object[] results) throws IOException, InterruptedException {
2313       // This belongs in HTable!!! Not in here.  St.Ack
2314 
2315       // results must be the same size as list
2316       if (results.length != list.size()) {
2317         throw new IllegalArgumentException(
2318           "argument results must be the same size as argument list");
2319       }
2320       processBatchCallback(list, tableName, pool, results, null);
2321     }
2322 
2323     @Override
2324     @Deprecated
2325     public void processBatch(List<? extends Row> list,
2326         final byte[] tableName,
2327         ExecutorService pool,
2328         Object[] results) throws IOException, InterruptedException {
2329       processBatch(list, TableName.valueOf(tableName), pool, results);
2330     }
2331 
2332     /**
2333      * Send the queries in parallel on the different region servers. Retries on failures.
2334      * If the method returns it means that there is no error, and the 'results' array will
2335      * contain no exception. On error, an exception is thrown, and the 'results' array will
2336      * contain results and exceptions.
2337      * @deprecated since 0.96 - Use {@link HTable#processBatchCallback} instead
2338      */
2339     @Override
2340     @Deprecated
2341     public <R> void processBatchCallback(
2342       List<? extends Row> list,
2343       TableName tableName,
2344       ExecutorService pool,
2345       Object[] results,
2346       Batch.Callback<R> callback)
2347       throws IOException, InterruptedException {
2348 
2349       // To fulfill the original contract, we have a special callback. This callback
2350       //  will set the results in the Object array.
2351       ObjectResultFiller<R> cb = new ObjectResultFiller<R>(results, callback);
2352       AsyncProcess<?> asyncProcess = createAsyncProcess(tableName, pool, cb, conf);
2353 
2354       // We're doing a submit all. This way, the originalIndex will match the initial list.
2355       asyncProcess.submitAll(list);
2356       asyncProcess.waitUntilDone();
2357 
2358       if (asyncProcess.hasError()) {
2359         throw asyncProcess.getErrors();
2360       }
2361     }
2362 
2363     @Override
2364     @Deprecated
2365     public <R> void processBatchCallback(
2366       List<? extends Row> list,
2367       byte[] tableName,
2368       ExecutorService pool,
2369       Object[] results,
2370       Batch.Callback<R> callback)
2371       throws IOException, InterruptedException {
2372       processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
2373     }
2374 
2375     // For tests.
2376     protected <R> AsyncProcess createAsyncProcess(TableName tableName, ExecutorService pool,
2377            AsyncProcess.AsyncProcessCallback<R> callback, Configuration conf) {
2378       return new AsyncProcess<R>(this, tableName, pool, callback, conf,
2379           RpcRetryingCallerFactory.instantiate(conf));
2380     }
2381 
2382 
2383     /**
2384      * Fill the result array for the interfaces using it.
2385      */
2386     private static class ObjectResultFiller<Res>
2387         implements AsyncProcess.AsyncProcessCallback<Res> {
2388 
2389       private final Object[] results;
2390       private Batch.Callback<Res> callback;
2391 
2392       ObjectResultFiller(Object[] results, Batch.Callback<Res> callback) {
2393         this.results = results;
2394         this.callback = callback;
2395       }
2396 
2397       @Override
2398       public void success(int pos, byte[] region, Row row, Res result) {
2399         assert pos < results.length;
2400         results[pos] = result;
2401         if (callback != null) {
2402           callback.update(region, row.getRow(), result);
2403         }
2404       }
2405 
2406       @Override
2407       public boolean failure(int pos, byte[] region, Row row, Throwable t) {
2408         assert pos < results.length;
2409         results[pos] = t;
2410         //Batch.Callback<Res> was not called on failure in 0.94. We keep this.
2411         return true; // we want to have this failure in the failures list.
2412       }
2413 
2414       @Override
2415       public boolean retriableFailure(int originalIndex, Row row, byte[] region,
2416                                       Throwable exception) {
2417         return true; // we retry
2418       }
2419     }
2420 
2421 
2422     /*
2423      * Return the number of cached region for a table. It will only be called
2424      * from a unit test.
2425      */
2426     int getNumberOfCachedRegionLocations(final TableName tableName) {
2427       Map<byte[], HRegionLocation> tableLocs = this.cachedRegionLocations.get(tableName);
2428       if (tableLocs == null) {
2429         return 0;
2430       }
2431       return tableLocs.values().size();
2432     }
2433 
2434     /**
2435      * Check the region cache to see whether a region is cached yet or not.
2436      * Called by unit tests.
2437      * @param tableName tableName
2438      * @param row row
2439      * @return Region cached or not.
2440      */
2441     boolean isRegionCached(TableName tableName, final byte[] row) {
2442       HRegionLocation location = getCachedLocation(tableName, row);
2443       return location != null;
2444     }
2445 
2446     @Override
2447     public void setRegionCachePrefetch(final TableName tableName,
2448         final boolean enable) {
2449       if (!enable) {
2450         regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName.getName()));
2451       }
2452       else {
2453         regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName.getName()));
2454       }
2455     }
2456 
2457     @Override
2458     public void setRegionCachePrefetch(final byte[] tableName,
2459         final boolean enable) {
2460       setRegionCachePrefetch(TableName.valueOf(tableName), enable);
2461     }
2462 
2463     @Override
2464     public boolean getRegionCachePrefetch(TableName tableName) {
2465       return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName.getName()));
2466     }
2467 
2468     @Override
2469     public boolean getRegionCachePrefetch(byte[] tableName) {
2470       return getRegionCachePrefetch(TableName.valueOf(tableName));
2471     }
2472 
2473     @Override
2474     public void abort(final String msg, Throwable t) {
2475       if (t instanceof KeeperException.SessionExpiredException
2476         && keepAliveZookeeper != null) {
2477         synchronized (masterAndZKLock) {
2478           if (keepAliveZookeeper != null) {
2479             LOG.warn("This client just lost it's session with ZooKeeper," +
2480               " closing it." +
2481               " It will be recreated next time someone needs it", t);
2482             closeZooKeeperWatcher();
2483           }
2484         }
2485       } else {
2486         if (t != null) {
2487           LOG.fatal(msg, t);
2488         } else {
2489           LOG.fatal(msg);
2490         }
2491         this.aborted = true;
2492         close();
2493         this.closed = true;
2494       }
2495     }
2496 
2497     @Override
2498     public boolean isClosed() {
2499       return this.closed;
2500     }
2501 
2502     @Override
2503     public boolean isAborted(){
2504       return this.aborted;
2505     }
2506 
2507     @Override
2508     public int getCurrentNrHRS() throws IOException {
2509       return this.registry.getCurrentNrHRS();
2510     }
2511 
2512     /**
2513      * Increment this client's reference count.
2514      */
2515     void incCount() {
2516       ++refCount;
2517     }
2518 
2519     /**
2520      * Decrement this client's reference count.
2521      */
2522     void decCount() {
2523       if (refCount > 0) {
2524         --refCount;
2525       }
2526     }
2527 
2528     /**
2529      * Return if this client has no reference
2530      *
2531      * @return true if this client has no reference; false otherwise
2532      */
2533     boolean isZeroReference() {
2534       return refCount == 0;
2535     }
2536 
2537     void internalClose() {
2538       if (this.closed) {
2539         return;
2540       }
2541       delayedClosing.stop("Closing connection");
2542       closeMaster();
2543       shutdownBatchPool();
2544       this.closed = true;
2545       closeZooKeeperWatcher();
2546       this.stubs.clear();
2547       if (clusterStatusListener != null) {
2548         clusterStatusListener.close();
2549       }
2550       if (rpcClient != null) {
2551         rpcClient.stop();
2552       }
2553     }
2554 
2555     @Override
2556     public void close() {
2557       if (managed) {
2558         if (aborted) {
2559           HConnectionManager.deleteStaleConnection(this);
2560         } else {
2561           HConnectionManager.deleteConnection(this, false);
2562         }
2563       } else {
2564         internalClose();
2565       }
2566     }
2567 
2568     /**
2569      * Close the connection for good, regardless of what the current value of
2570      * {@link #refCount} is. Ideally, {@link #refCount} should be zero at this
2571      * point, which would be the case if all of its consumers close the
2572      * connection. However, on the off chance that someone is unable to close
2573      * the connection, perhaps because it bailed out prematurely, the method
2574      * below will ensure that this {@link HConnection} instance is cleaned up.
2575      * Caveat: The JVM may take an unknown amount of time to call finalize on an
2576      * unreachable object, so our hope is that every consumer cleans up after
2577      * itself, like any good citizen.
2578      */
2579     @Override
2580     protected void finalize() throws Throwable {
2581       super.finalize();
2582       // Pretend as if we are about to release the last remaining reference
2583       refCount = 1;
2584       close();
2585     }
2586 
2587     @Override
2588     public HTableDescriptor[] listTables() throws IOException {
2589       MasterKeepAliveConnection master = getKeepAliveMasterService();
2590       try {
2591         GetTableDescriptorsRequest req =
2592           RequestConverter.buildGetTableDescriptorsRequest((List<TableName>)null);
2593         return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
2594       } catch (ServiceException se) {
2595         throw ProtobufUtil.getRemoteException(se);
2596       } finally {
2597         master.close();
2598       }
2599     }
2600 
2601     @Override
2602     public String[] getTableNames() throws IOException {
2603       TableName[] tableNames = listTableNames();
2604       String result[] = new String[tableNames.length];
2605       for (int i = 0; i < tableNames.length; i++) {
2606         result[i] = tableNames[i].getNameAsString();
2607       }
2608       return result;
2609     }
2610 
2611     @Override
2612     public TableName[] listTableNames() throws IOException {
2613       MasterKeepAliveConnection master = getKeepAliveMasterService();
2614       try {
2615         return ProtobufUtil.getTableNameArray(master.getTableNames(null,
2616             GetTableNamesRequest.newBuilder().build())
2617           .getTableNamesList());
2618       } catch (ServiceException se) {
2619         throw ProtobufUtil.getRemoteException(se);
2620       } finally {
2621         master.close();
2622       }
2623     }
2624 
2625     @Override
2626     public HTableDescriptor[] getHTableDescriptorsByTableName(
2627         List<TableName> tableNames) throws IOException {
2628       if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
2629       MasterKeepAliveConnection master = getKeepAliveMasterService();
2630       try {
2631         GetTableDescriptorsRequest req =
2632           RequestConverter.buildGetTableDescriptorsRequest(tableNames);
2633         return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
2634       } catch (ServiceException se) {
2635         throw ProtobufUtil.getRemoteException(se);
2636       } finally {
2637         master.close();
2638       }
2639     }
2640 
2641     @Override
2642     public HTableDescriptor[] getHTableDescriptors(
2643         List<String> names) throws IOException {
2644       List<TableName> tableNames = new ArrayList(names.size());
2645       for(String name : names) {
2646         tableNames.add(TableName.valueOf(name));
2647       }
2648 
2649       return getHTableDescriptorsByTableName(tableNames);
2650     }
2651 
2652     @Override
2653     public NonceGenerator getNonceGenerator() {
2654       return this.nonceGenerator;
2655     }
2656 
2657     /**
2658      * Connects to the master to get the table descriptor.
2659      * @param tableName table name
2660      * @return
2661      * @throws IOException if the connection to master fails or if the table
2662      *  is not found.
2663      */
2664     @Override
2665     public HTableDescriptor getHTableDescriptor(final TableName tableName)
2666     throws IOException {
2667       if (tableName == null) return null;
2668       if (tableName.equals(TableName.META_TABLE_NAME)) {
2669         return HTableDescriptor.META_TABLEDESC;
2670       }
2671       MasterKeepAliveConnection master = getKeepAliveMasterService();
2672       GetTableDescriptorsResponse htds;
2673       try {
2674         GetTableDescriptorsRequest req =
2675           RequestConverter.buildGetTableDescriptorsRequest(tableName);
2676         htds = master.getTableDescriptors(null, req);
2677       } catch (ServiceException se) {
2678         throw ProtobufUtil.getRemoteException(se);
2679       } finally {
2680         master.close();
2681       }
2682       if (!htds.getTableSchemaList().isEmpty()) {
2683         return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
2684       }
2685       throw new TableNotFoundException(tableName.getNameAsString());
2686     }
2687 
2688     @Override
2689     public HTableDescriptor getHTableDescriptor(final byte[] tableName)
2690     throws IOException {
2691       return getHTableDescriptor(TableName.valueOf(tableName));
2692     }
2693   }
2694 
2695   /**
2696    * The record of errors for servers.
2697    */
2698   static class ServerErrorTracker {
2699     // We need a concurrent map here, as we could have multiple threads updating it in parallel.
2700     private final ConcurrentMap<HRegionLocation, ServerErrors> errorsByServer =
2701         new ConcurrentHashMap<HRegionLocation, ServerErrors>();
2702     private final long canRetryUntil;
2703     private final int maxRetries;
2704     private final String startTrackingTime;
2705 
2706     public ServerErrorTracker(long timeout, int maxRetries) {
2707       this.maxRetries = maxRetries;
2708       this.canRetryUntil = EnvironmentEdgeManager.currentTimeMillis() + timeout;
2709       this.startTrackingTime = new Date().toString();
2710     }
2711 
2712     /**
2713      * We stop to retry when we have exhausted BOTH the number of retries and the time allocated.
2714      */
2715     boolean canRetryMore(int numRetry) {
2716       // If there is a single try we must not take into account the time.
2717       return numRetry < maxRetries || (maxRetries > 1 &&
2718           EnvironmentEdgeManager.currentTimeMillis() < this.canRetryUntil);
2719     }
2720 
2721     /**
2722      * Calculates the back-off time for a retrying request to a particular server.
2723      *
2724      * @param server    The server in question.
2725      * @param basePause The default hci pause.
2726      * @return The time to wait before sending next request.
2727      */
2728     long calculateBackoffTime(HRegionLocation server, long basePause) {
2729       long result;
2730       ServerErrors errorStats = errorsByServer.get(server);
2731       if (errorStats != null) {
2732         result = ConnectionUtils.getPauseTime(basePause, errorStats.retries.get());
2733       } else {
2734         result = 0; // yes, if the server is not in our list we don't wait before retrying.
2735       }
2736       return result;
2737     }
2738 
2739     /**
2740      * Reports that there was an error on the server to do whatever bean-counting necessary.
2741      *
2742      * @param server The server in question.
2743      */
2744     void reportServerError(HRegionLocation server) {
2745       ServerErrors errors = errorsByServer.get(server);
2746       if (errors != null) {
2747         errors.addError();
2748       } else {
2749         errors = errorsByServer.putIfAbsent(server, new ServerErrors());
2750         if (errors != null){
2751           errors.addError();
2752         }
2753       }
2754     }
2755 
2756     String getStartTrackingTime() {
2757       return startTrackingTime;
2758     }
2759 
2760     /**
2761      * The record of errors for a server.
2762      */
2763     private static class ServerErrors {
2764       public final AtomicInteger retries = new AtomicInteger(0);
2765 
2766       public void addError() {
2767         retries.incrementAndGet();
2768       }
2769     }
2770   }
2771 
2772   /**
2773    * Look for an exception we know in the remote exception:
2774    * - hadoop.ipc wrapped exceptions
2775    * - nested exceptions
2776    * 
2777    * Looks for: RegionMovedException / RegionOpeningException / RegionTooBusyException
2778    * @return null if we didn't find the exception, the exception otherwise.
2779    */
2780   public static Throwable findException(Object exception) {
2781     if (exception == null || !(exception instanceof Throwable)) {
2782       return null;
2783     }
2784     Throwable cur = (Throwable) exception;
2785     while (cur != null) {
2786       if (cur instanceof RegionMovedException || cur instanceof RegionOpeningException
2787           || cur instanceof RegionTooBusyException) {
2788         return cur;
2789       }
2790       if (cur instanceof RemoteException) {
2791         RemoteException re = (RemoteException) cur;
2792         cur = re.unwrapRemoteException(
2793             RegionOpeningException.class, RegionMovedException.class,
2794             RegionTooBusyException.class);
2795         if (cur == null) {
2796           cur = re.unwrapRemoteException();
2797         }
2798         // unwrapRemoteException can return the exception given as a parameter when it cannot
2799         //  unwrap it. In this case, there is no need to look further
2800         // noinspection ObjectEquality
2801         if (cur == re) {
2802           return null;
2803         }
2804       } else {
2805         cur = cur.getCause();
2806       }
2807     }
2808 
2809     return null;
2810   }
2811 
2812   /**
2813    * Set the number of retries to use serverside when trying to communicate
2814    * with another server over {@link HConnection}.  Used updating catalog
2815    * tables, etc.  Call this method before we create any Connections.
2816    * @param c The Configuration instance to set the retries into.
2817    * @param log Used to log what we set in here.
2818    */
2819   public static void setServerSideHConnectionRetries(final Configuration c, final String sn,
2820       final Log log) {
2821     int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
2822       HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
2823     // Go big.  Multiply by 10.  If we can't get to meta after this many retries
2824     // then something seriously wrong.
2825     int serversideMultiplier = c.getInt("hbase.client.serverside.retries.multiplier", 10);
2826     int retries = hcRetries * serversideMultiplier;
2827     c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
2828     log.debug(sn + " HConnection server-to-server retries=" + retries);
2829   }
2830 }