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