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