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