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