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