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       if (this.useMetaReplicas) {
1291         s.setConsistency(Consistency.TIMELINE);
1292       }
1293 
1294       int localNumRetries = (retry ? numTries : 1);
1295 
1296       for (int tries = 0; true; tries++) {
1297         if (tries >= localNumRetries) {
1298           throw new NoServerForRegionException("Unable to find region for " +
1299             Bytes.toStringBinary(row) + " in " + tableName + " after " + tries + " tries.");
1300         }
1301         if (useCache) {
1302           RegionLocations locations = getCachedLocation(tableName, row);
1303           if (locations != null && locations.getRegionLocation(replicaId) != null) {
1304             return locations;
1305           }
1306         } else {
1307           // If we are not supposed to be using the cache, delete any existing cached location
1308           // so it won't interfere.
1309           // We are only supposed to clean the cache for the specific replicaId
1310           metaCache.clearCache(tableName, row, replicaId);
1311         }
1312 
1313         // Query the meta region
1314         long pauseBase = this.pause;
1315         takeUserRegionLock();
1316         try {
1317           // We don't need to check if useCache is enabled or not. Even if useCache is false
1318           // we already cleared the cache for this row before acquiring userRegion lock so if this
1319           // row is present in cache that means some other thread has populated it while we were
1320           // waiting to acquire user region lock.
1321           RegionLocations locations = getCachedLocation(tableName, row);
1322           if (locations != null && locations.getRegionLocation(replicaId) != null) {
1323             return locations;
1324           }
1325           Result regionInfoRow = null;
1326           s.resetMvccReadPoint();
1327           s.setOneRowLimit();
1328           try (ReversedClientScanner rcs =
1329               new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory,
1330                   rpcControllerFactory, getMetaLookupPool(),
1331                   metaReplicaCallTimeoutScanInMicroSecond)) {
1332             regionInfoRow = rcs.next();
1333           }
1334 
1335           if (regionInfoRow == null) {
1336             throw new TableNotFoundException(tableName);
1337           }
1338 
1339           // convert the row result into the HRegionLocation we need!
1340           locations = MetaTableAccessor.getRegionLocations(regionInfoRow);
1341           if (locations == null || locations.getRegionLocation(replicaId) == null) {
1342             throw new IOException("HRegionInfo was null in " +
1343               tableName + ", row=" + regionInfoRow);
1344           }
1345           HRegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegionInfo();
1346           if (regionInfo == null) {
1347             throw new IOException("HRegionInfo was null or empty in " +
1348               TableName.META_TABLE_NAME + ", row=" + regionInfoRow);
1349           }
1350 
1351           // possible we got a region of a different table...
1352           if (!regionInfo.getTable().equals(tableName)) {
1353             throw new TableNotFoundException(
1354                   "Table '" + tableName + "' was not found, got: " +
1355                   regionInfo.getTable() + ".");
1356           }
1357           if (regionInfo.isSplit()) {
1358             throw new RegionOfflineException(
1359                 "the only available region for the required row is a split parent," +
1360                 " the daughters should be online soon: " + regionInfo.getRegionNameAsString());
1361           }
1362           if (regionInfo.isOffline()) {
1363             throw new RegionOfflineException("the region is offline, could" +
1364               " be caused by a disable table call: " + regionInfo.getRegionNameAsString());
1365           }
1366 
1367           ServerName serverName = locations.getRegionLocation(replicaId).getServerName();
1368           if (serverName == null) {
1369             throw new NoServerForRegionException("No server address listed in " +
1370               TableName.META_TABLE_NAME + " for region " + regionInfo.getRegionNameAsString() +
1371               " containing row " + Bytes.toStringBinary(row));
1372           }
1373 
1374           if (isDeadServer(serverName)){
1375             throw new RegionServerStoppedException("hbase:meta says the region "+
1376                 regionInfo.getRegionNameAsString()+" is managed by the server " + serverName +
1377                 ", but it is dead.");
1378           }
1379           // Instantiate the location
1380           cacheLocation(tableName, locations);
1381           return locations;
1382         } catch (TableNotFoundException e) {
1383           // if we got this error, probably means the table just plain doesn't
1384           // exist. rethrow the error immediately. this should always be coming
1385           // from the HTable constructor.
1386           throw e;
1387         } catch (IOException e) {
1388           ExceptionUtil.rethrowIfInterrupt(e);
1389 
1390           if (e instanceof RemoteException) {
1391             e = ((RemoteException)e).unwrapRemoteException();
1392           }
1393           if (e instanceof CallQueueTooBigException) {
1394             // Give a special check on CallQueueTooBigException, see #HBASE-17114
1395             pauseBase = this.pauseForCQTBE;
1396           }
1397           if (tries < localNumRetries - 1) {
1398             if (LOG.isDebugEnabled()) {
1399               LOG.debug("locateRegionInMeta parentTable=" + TableName.META_TABLE_NAME +
1400                   ", metaLocation=" + ", attempt=" + tries + " of " + localNumRetries +
1401                   " failed; retrying after sleep of " +
1402                   ConnectionUtils.getPauseTime(pauseBase, tries) + " because: " + e.getMessage());
1403             }
1404           } else {
1405             throw e;
1406           }
1407           // Only relocate the parent region if necessary
1408           if(!(e instanceof RegionOfflineException ||
1409               e instanceof NoServerForRegionException)) {
1410             relocateRegion(TableName.META_TABLE_NAME, metaKey, replicaId);
1411           }
1412         } finally {
1413           userRegionLock.unlock();
1414         }
1415         try{
1416           Thread.sleep(ConnectionUtils.getPauseTime(pauseBase, tries));
1417         } catch (InterruptedException e) {
1418           throw new InterruptedIOException("Giving up trying to location region in " +
1419             "meta: thread is interrupted.");
1420         }
1421       }
1422     }
1423 
1424     void takeUserRegionLock() throws IOException {
1425       try {
1426         long waitTime = connectionConfig.getMetaOperationTimeout();
1427         if (!userRegionLock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
1428           throw new LockTimeoutException("Failed to get user region lock in"
1429             + waitTime + " ms. " + " for accessing meta region server.");
1430         }
1431       } catch (InterruptedException ie) {
1432         LOG.error("Interrupted while waiting for a lock", ie);
1433         throw ExceptionUtil.asInterrupt(ie);
1434       }
1435     }
1436 
1437     /**
1438      * Put a newly discovered HRegionLocation into the cache.
1439      * @param tableName The table name.
1440      * @param location the new location
1441      */
1442     @Override
1443     public void cacheLocation(final TableName tableName, final RegionLocations location) {
1444       metaCache.cacheLocation(tableName, location);
1445     }
1446 
1447     /**
1448      * Search the cache for a location that fits our table and row key.
1449      * Return null if no suitable region is located.
1450      *
1451      * @param tableName
1452      * @param row
1453      * @return Null or region location found in cache.
1454      */
1455     RegionLocations getCachedLocation(final TableName tableName,
1456         final byte [] row) {
1457       return metaCache.getCachedLocation(tableName, row);
1458     }
1459 
1460     public void clearRegionCache(final TableName tableName, byte[] row) {
1461       metaCache.clearCache(tableName, row);
1462     }
1463 
1464     /*
1465      * Delete all cached entries of a table that maps to a specific location.
1466      */
1467     @Override
1468     public void clearCaches(final ServerName serverName) {
1469       metaCache.clearCache(serverName);
1470     }
1471 
1472     @Override
1473     public void clearRegionCache() {
1474       metaCache.clearCache();
1475       clearMetaRegionLocation();
1476     }
1477 
1478     @Override
1479     public void clearRegionCache(final TableName tableName) {
1480       if (TableName.META_TABLE_NAME.equals(tableName)) {
1481         clearMetaRegionLocation();
1482       } else {
1483         metaCache.clearCache(tableName);
1484       }
1485     }
1486 
1487     @Override
1488     public void clearRegionCache(final byte[] tableName) {
1489       if (Bytes.equals(TableName.META_TABLE_NAME.getName(), tableName)) {
1490         clearMetaRegionLocation();
1491       } else {
1492         clearRegionCache(TableName.valueOf(tableName));
1493       }
1494     }
1495 
1496     private void clearMetaRegionLocation() {
1497       // Meta's location is cached separately from the MetaCache
1498       synchronized (metaRegionLock) {
1499         this.metaLocations = null;
1500       }
1501     }
1502 
1503     /**
1504      * Put a newly discovered HRegionLocation into the cache.
1505      * @param tableName The table name.
1506      * @param source the source of the new location, if it's not coming from meta
1507      * @param location the new location
1508      */
1509     private void cacheLocation(final TableName tableName, final ServerName source,
1510         final HRegionLocation location) {
1511       metaCache.cacheLocation(tableName, source, location);
1512     }
1513 
1514     // Map keyed by service name + regionserver to service stub implementation
1515     private final ConcurrentHashMap<String, Object> stubs =
1516       new ConcurrentHashMap<String, Object>();
1517     // Map of locks used creating service stubs per regionserver.
1518     private final ConcurrentHashMap<String, String> connectionLock =
1519       new ConcurrentHashMap<String, String>();
1520 
1521     /**
1522      * State of the MasterService connection/setup.
1523      */
1524     static class MasterServiceState {
1525       HConnection connection;
1526       MasterService.BlockingInterface stub;
1527       int userCount;
1528 
1529       MasterServiceState (final HConnection connection) {
1530         super();
1531         this.connection = connection;
1532       }
1533 
1534       @Override
1535       public String toString() {
1536         return "MasterService";
1537       }
1538 
1539       Object getStub() {
1540         return this.stub;
1541       }
1542 
1543       void clearStub() {
1544         this.stub = null;
1545       }
1546 
1547       boolean isMasterRunning() throws ServiceException {
1548         IsMasterRunningResponse response =
1549           this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
1550         return response != null? response.getIsMasterRunning(): false;
1551       }
1552     }
1553 
1554     /**
1555      * Makes a client-side stub for master services. Sub-class to specialize.
1556      * Depends on hosting class so not static.  Exists so we avoid duplicating a bunch of code
1557      * when setting up the MasterMonitorService and MasterAdminService.
1558      */
1559     abstract class StubMaker {
1560       /**
1561        * Returns the name of the service stub being created.
1562        */
1563       protected abstract String getServiceName();
1564 
1565       /**
1566        * Make stub and cache it internal so can be used later doing the isMasterRunning call.
1567        * @param channel
1568        */
1569       protected abstract Object makeStub(final BlockingRpcChannel channel);
1570 
1571       /**
1572        * Once setup, check it works by doing isMasterRunning check.
1573        * @throws ServiceException
1574        */
1575       protected abstract void isMasterRunning() throws ServiceException;
1576 
1577       /**
1578        * Create a stub. Try once only.  It is not typed because there is no common type to
1579        * protobuf services nor their interfaces.  Let the caller do appropriate casting.
1580        * @return A stub for master services.
1581        * @throws IOException
1582        * @throws KeeperException
1583        * @throws ServiceException
1584        */
1585       private Object makeStubNoRetries() throws IOException, ServiceException {
1586         ServerName sn = registry.getActiveMaster();
1587         if (sn == null) {
1588           String msg = "No active master location found";
1589           LOG.info(msg);
1590           throw new MasterNotRunningException(msg);
1591         }
1592         if (isDeadServer(sn)) {
1593           throw new MasterNotRunningException(sn + " is dead.");
1594         }
1595         // Use the security info interface name as our stub key
1596         String key = getStubKey(getServiceName(), sn);
1597         connectionLock.putIfAbsent(key, key);
1598         Object stub = null;
1599         synchronized (connectionLock.get(key)) {
1600           stub = stubs.get(key);
1601           if (stub == null) {
1602             BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout);
1603             stub = makeStub(channel);
1604             isMasterRunning();
1605             stubs.put(key, stub);
1606           }
1607         }
1608         return stub;
1609       }
1610 
1611       /**
1612        * Create a stub against the master.  Retry if necessary.
1613        * @return A stub to do <code>intf</code> against the master
1614        * @throws MasterNotRunningException
1615        */
1616       Object makeStub() throws IOException {
1617         // The lock must be at the beginning to prevent multiple master creations
1618         //  (and leaks) in a multithread context
1619         synchronized (masterAndZKLock) {
1620           Exception exceptionCaught = null;
1621           if (!closed) {
1622             try {
1623               return makeStubNoRetries();
1624             } catch (IOException e) {
1625               exceptionCaught = e;
1626             } catch (ServiceException e) {
1627               exceptionCaught = e;
1628             }
1629             throw new MasterNotRunningException(exceptionCaught);
1630           } else {
1631             throw new DoNotRetryIOException("Connection was closed while trying to get master");
1632           }
1633         }
1634       }
1635     }
1636 
1637     /**
1638      * Class to make a MasterServiceStubMaker stub.
1639      */
1640     class MasterServiceStubMaker extends StubMaker {
1641       private MasterService.BlockingInterface stub;
1642       @Override
1643       protected String getServiceName() {
1644         return MasterService.getDescriptor().getName();
1645       }
1646 
1647       @Override
1648       MasterService.BlockingInterface makeStub() throws IOException {
1649         return (MasterService.BlockingInterface)super.makeStub();
1650       }
1651 
1652       @Override
1653       protected Object makeStub(BlockingRpcChannel channel) {
1654         this.stub = MasterService.newBlockingStub(channel);
1655         return this.stub;
1656       }
1657 
1658       @Override
1659       protected void isMasterRunning() throws ServiceException {
1660         this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
1661       }
1662     }
1663 
1664     @Override
1665     public AdminService.BlockingInterface getAdmin(final ServerName serverName)
1666         throws IOException {
1667       return getAdmin(serverName, false);
1668     }
1669 
1670     @Override
1671     // Nothing is done w/ the 'master' parameter.  It is ignored.
1672     public AdminService.BlockingInterface getAdmin(final ServerName serverName,
1673       final boolean master)
1674     throws IOException {
1675       if (isDeadServer(serverName)) {
1676         throw new RegionServerStoppedException(serverName + " is dead.");
1677       }
1678       String key = getStubKey(AdminService.BlockingInterface.class.getName(), serverName);
1679       this.connectionLock.putIfAbsent(key, key);
1680       AdminService.BlockingInterface stub = null;
1681       synchronized (this.connectionLock.get(key)) {
1682         stub = (AdminService.BlockingInterface)this.stubs.get(key);
1683         if (stub == null) {
1684           BlockingRpcChannel channel =
1685               this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout);
1686           stub = AdminService.newBlockingStub(channel);
1687           this.stubs.put(key, stub);
1688         }
1689       }
1690       return stub;
1691     }
1692 
1693     @Override
1694     public ClientService.BlockingInterface getClient(final ServerName sn)
1695     throws IOException {
1696       if (isDeadServer(sn)) {
1697         throw new RegionServerStoppedException(sn + " is dead.");
1698       }
1699       String key = getStubKey(ClientService.BlockingInterface.class.getName(), sn);
1700       this.connectionLock.putIfAbsent(key, key);
1701       ClientService.BlockingInterface stub = null;
1702       synchronized (this.connectionLock.get(key)) {
1703         stub = (ClientService.BlockingInterface)this.stubs.get(key);
1704         if (stub == null) {
1705           BlockingRpcChannel channel =
1706               this.rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout);
1707           stub = ClientService.newBlockingStub(channel);
1708           // In old days, after getting stub/proxy, we'd make a call.  We are not doing that here.
1709           // Just fail on first actual call rather than in here on setup.
1710           this.stubs.put(key, stub);
1711         }
1712       }
1713       return stub;
1714     }
1715 
1716     private ZooKeeperKeepAliveConnection keepAliveZookeeper;
1717     private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0);
1718     private boolean canCloseZKW = true;
1719 
1720     // keepAlive time, in ms. No reason to make it configurable.
1721     private static final long keepAlive = 5 * 60 * 1000;
1722 
1723     /**
1724      * Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it.
1725      * @return The shared instance. Never returns null.
1726      */
1727     ZooKeeperKeepAliveConnection getKeepAliveZooKeeperWatcher()
1728       throws IOException {
1729       synchronized (masterAndZKLock) {
1730         if (keepAliveZookeeper == null) {
1731           if (this.closed) {
1732             throw new IOException(toString() + " closed");
1733           }
1734           // We don't check that our link to ZooKeeper is still valid
1735           // But there is a retry mechanism in the ZooKeeperWatcher itself
1736           keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this);
1737         }
1738         keepAliveZookeeperUserCount.addAndGet(1);
1739         keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
1740         return keepAliveZookeeper;
1741       }
1742     }
1743 
1744     void releaseZooKeeperWatcher(final ZooKeeperWatcher zkw) {
1745       if (zkw == null){
1746         return;
1747       }
1748       if (keepAliveZookeeperUserCount.addAndGet(-1) <= 0 ){
1749         keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive;
1750       }
1751     }
1752 
1753     private void closeZooKeeperWatcher() {
1754       synchronized (masterAndZKLock) {
1755         if (keepAliveZookeeper != null) {
1756           LOG.info("Closing zookeeper sessionid=0x" +
1757             Long.toHexString(
1758               keepAliveZookeeper.getRecoverableZooKeeper().getSessionId()));
1759           keepAliveZookeeper.internalClose();
1760           keepAliveZookeeper = null;
1761         }
1762         keepAliveZookeeperUserCount.set(0);
1763       }
1764     }
1765 
1766     final MasterServiceState masterServiceState = new MasterServiceState(this);
1767 
1768     @Override
1769     public MasterService.BlockingInterface getMaster() throws MasterNotRunningException {
1770       return getKeepAliveMasterService();
1771     }
1772 
1773     private void resetMasterServiceState(final MasterServiceState mss) {
1774       mss.userCount++;
1775     }
1776 
1777     @Override
1778     public MasterKeepAliveConnection getKeepAliveMasterService()
1779     throws MasterNotRunningException {
1780       synchronized (masterAndZKLock) {
1781         if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) {
1782           MasterServiceStubMaker stubMaker = new MasterServiceStubMaker();
1783           try {
1784             this.masterServiceState.stub = stubMaker.makeStub();
1785           } catch (MasterNotRunningException ex) {
1786             throw ex;
1787           } catch (IOException e) {
1788             // rethrow as MasterNotRunningException so that we can keep the method sig
1789             throw new MasterNotRunningException(e);
1790           }
1791         }
1792         resetMasterServiceState(this.masterServiceState);
1793       }
1794       // Ugly delegation just so we can add in a Close method.
1795       final MasterService.BlockingInterface stub = this.masterServiceState.stub;
1796       return new MasterKeepAliveConnection() {
1797         MasterServiceState mss = masterServiceState;
1798         @Override
1799         public MasterProtos.AbortProcedureResponse abortProcedure(
1800           RpcController controller,
1801           MasterProtos.AbortProcedureRequest request) throws ServiceException {
1802           return stub.abortProcedure(controller, request);
1803         }
1804         @Override
1805         public MasterProtos.ListProceduresResponse listProcedures(
1806             RpcController controller,
1807             MasterProtos.ListProceduresRequest request) throws ServiceException {
1808           return stub.listProcedures(controller, request);
1809         }
1810 
1811         @Override
1812         public MasterProtos.ClearDeadServersResponse clearDeadServers(
1813             RpcController controller,
1814             MasterProtos.ClearDeadServersRequest request) throws ServiceException {
1815           return stub.clearDeadServers(controller, request);
1816         }
1817 
1818         @Override
1819         public AddColumnResponse addColumn(RpcController controller, AddColumnRequest request)
1820         throws ServiceException {
1821           return stub.addColumn(controller, request);
1822         }
1823 
1824         @Override
1825         public DeleteColumnResponse deleteColumn(RpcController controller,
1826             DeleteColumnRequest request)
1827         throws ServiceException {
1828           return stub.deleteColumn(controller, request);
1829         }
1830 
1831         @Override
1832         public ModifyColumnResponse modifyColumn(RpcController controller,
1833             ModifyColumnRequest request)
1834         throws ServiceException {
1835           return stub.modifyColumn(controller, request);
1836         }
1837 
1838         @Override
1839         public MoveRegionResponse moveRegion(RpcController controller,
1840             MoveRegionRequest request) throws ServiceException {
1841           return stub.moveRegion(controller, request);
1842         }
1843 
1844         @Override
1845         public DispatchMergingRegionsResponse dispatchMergingRegions(
1846             RpcController controller, DispatchMergingRegionsRequest request)
1847             throws ServiceException {
1848           return stub.dispatchMergingRegions(controller, request);
1849         }
1850 
1851         @Override
1852         public AssignRegionResponse assignRegion(RpcController controller,
1853             AssignRegionRequest request) throws ServiceException {
1854           return stub.assignRegion(controller, request);
1855         }
1856 
1857         @Override
1858         public UnassignRegionResponse unassignRegion(RpcController controller,
1859             UnassignRegionRequest request) throws ServiceException {
1860           return stub.unassignRegion(controller, request);
1861         }
1862 
1863         @Override
1864         public OfflineRegionResponse offlineRegion(RpcController controller,
1865             OfflineRegionRequest request) throws ServiceException {
1866           return stub.offlineRegion(controller, request);
1867         }
1868 
1869         @Override
1870         public DeleteTableResponse deleteTable(RpcController controller,
1871             DeleteTableRequest request) throws ServiceException {
1872           return stub.deleteTable(controller, request);
1873         }
1874 
1875         @Override
1876         public TruncateTableResponse truncateTable(RpcController controller,
1877             TruncateTableRequest request) throws ServiceException {
1878           return stub.truncateTable(controller, request);
1879         }
1880 
1881         @Override
1882         public EnableTableResponse enableTable(RpcController controller,
1883             EnableTableRequest request) throws ServiceException {
1884           return stub.enableTable(controller, request);
1885         }
1886 
1887         @Override
1888         public DisableTableResponse disableTable(RpcController controller,
1889             DisableTableRequest request) throws ServiceException {
1890           return stub.disableTable(controller, request);
1891         }
1892 
1893         @Override
1894         public ModifyTableResponse modifyTable(RpcController controller,
1895             ModifyTableRequest request) throws ServiceException {
1896           return stub.modifyTable(controller, request);
1897         }
1898 
1899         @Override
1900         public CreateTableResponse createTable(RpcController controller,
1901             CreateTableRequest request) throws ServiceException {
1902           return stub.createTable(controller, request);
1903         }
1904 
1905         @Override
1906         public ShutdownResponse shutdown(RpcController controller,
1907             ShutdownRequest request) throws ServiceException {
1908           return stub.shutdown(controller, request);
1909         }
1910 
1911         @Override
1912         public StopMasterResponse stopMaster(RpcController controller,
1913             StopMasterRequest request) throws ServiceException {
1914           return stub.stopMaster(controller, request);
1915         }
1916 
1917         @Override
1918         public MasterProtos.IsInMaintenanceModeResponse isMasterInMaintenanceMode(
1919             final RpcController controller,
1920             final MasterProtos.IsInMaintenanceModeRequest request) throws ServiceException {
1921           return stub.isMasterInMaintenanceMode(controller, request);
1922         }
1923 
1924         @Override
1925         public BalanceResponse balance(RpcController controller,
1926             BalanceRequest request) throws ServiceException {
1927           return stub.balance(controller, request);
1928         }
1929 
1930         @Override
1931         public SetBalancerRunningResponse setBalancerRunning(
1932             RpcController controller, SetBalancerRunningRequest request)
1933             throws ServiceException {
1934           return stub.setBalancerRunning(controller, request);
1935         }
1936 
1937         @Override
1938         public NormalizeResponse normalize(RpcController controller,
1939                                        NormalizeRequest request) throws ServiceException {
1940           return stub.normalize(controller, request);
1941         }
1942 
1943         @Override
1944         public SetNormalizerRunningResponse setNormalizerRunning(
1945           RpcController controller, SetNormalizerRunningRequest request)
1946           throws ServiceException {
1947           return stub.setNormalizerRunning(controller, request);
1948         }
1949 
1950         @Override
1951         public RunCatalogScanResponse runCatalogScan(RpcController controller,
1952             RunCatalogScanRequest request) throws ServiceException {
1953           return stub.runCatalogScan(controller, request);
1954         }
1955 
1956         @Override
1957         public EnableCatalogJanitorResponse enableCatalogJanitor(
1958             RpcController controller, EnableCatalogJanitorRequest request)
1959             throws ServiceException {
1960           return stub.enableCatalogJanitor(controller, request);
1961         }
1962 
1963         @Override
1964         public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(
1965             RpcController controller, IsCatalogJanitorEnabledRequest request)
1966             throws ServiceException {
1967           return stub.isCatalogJanitorEnabled(controller, request);
1968         }
1969 
1970         @Override
1971         public RunCleanerChoreResponse runCleanerChore(RpcController controller,
1972             RunCleanerChoreRequest request) throws ServiceException {
1973           return stub.runCleanerChore(controller, request);
1974         }
1975 
1976         @Override
1977         public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController controller,
1978             SetCleanerChoreRunningRequest request) throws ServiceException {
1979           return stub.setCleanerChoreRunning(controller, request);
1980         }
1981 
1982         @Override
1983         public IsCleanerChoreEnabledResponse isCleanerChoreEnabled(
1984             RpcController controller, IsCleanerChoreEnabledRequest request)
1985             throws ServiceException {
1986           return stub.isCleanerChoreEnabled(controller, request);
1987         }
1988 
1989         @Override
1990         public CoprocessorServiceResponse execMasterService(
1991             RpcController controller, CoprocessorServiceRequest request)
1992             throws ServiceException {
1993           return stub.execMasterService(controller, request);
1994         }
1995 
1996         @Override
1997         public SnapshotResponse snapshot(RpcController controller,
1998             SnapshotRequest request) throws ServiceException {
1999           return stub.snapshot(controller, request);
2000         }
2001 
2002         @Override
2003         public GetCompletedSnapshotsResponse getCompletedSnapshots(
2004             RpcController controller, GetCompletedSnapshotsRequest request)
2005             throws ServiceException {
2006           return stub.getCompletedSnapshots(controller, request);
2007         }
2008 
2009         @Override
2010         public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
2011             DeleteSnapshotRequest request) throws ServiceException {
2012           return stub.deleteSnapshot(controller, request);
2013         }
2014 
2015         @Override
2016         public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
2017             IsSnapshotDoneRequest request) throws ServiceException {
2018           return stub.isSnapshotDone(controller, request);
2019         }
2020 
2021         @Override
2022         public RestoreSnapshotResponse restoreSnapshot(
2023             RpcController controller, RestoreSnapshotRequest request)
2024             throws ServiceException {
2025           return stub.restoreSnapshot(controller, request);
2026         }
2027 
2028         @Override
2029         public IsRestoreSnapshotDoneResponse isRestoreSnapshotDone(
2030             RpcController controller, IsRestoreSnapshotDoneRequest request)
2031             throws ServiceException {
2032           return stub.isRestoreSnapshotDone(controller, request);
2033         }
2034 
2035         @Override
2036         public ExecProcedureResponse execProcedure(
2037             RpcController controller, ExecProcedureRequest request)
2038             throws ServiceException {
2039           return stub.execProcedure(controller, request);
2040         }
2041 
2042         @Override
2043         public ExecProcedureResponse execProcedureWithRet(
2044             RpcController controller, ExecProcedureRequest request)
2045             throws ServiceException {
2046           return stub.execProcedureWithRet(controller, request);
2047         }
2048 
2049         @Override
2050         public IsProcedureDoneResponse isProcedureDone(RpcController controller,
2051             IsProcedureDoneRequest request) throws ServiceException {
2052           return stub.isProcedureDone(controller, request);
2053         }
2054 
2055         @Override
2056         public GetProcedureResultResponse getProcedureResult(RpcController controller,
2057             GetProcedureResultRequest request) throws ServiceException {
2058           return stub.getProcedureResult(controller, request);
2059         }
2060 
2061         @Override
2062         public IsMasterRunningResponse isMasterRunning(
2063             RpcController controller, IsMasterRunningRequest request)
2064             throws ServiceException {
2065           return stub.isMasterRunning(controller, request);
2066         }
2067 
2068         @Override
2069         public ModifyNamespaceResponse modifyNamespace(RpcController controller,
2070             ModifyNamespaceRequest request)
2071         throws ServiceException {
2072           return stub.modifyNamespace(controller, request);
2073         }
2074 
2075         @Override
2076         public CreateNamespaceResponse createNamespace(
2077             RpcController controller, CreateNamespaceRequest request) throws ServiceException {
2078           return stub.createNamespace(controller, request);
2079         }
2080 
2081         @Override
2082         public DeleteNamespaceResponse deleteNamespace(
2083             RpcController controller, DeleteNamespaceRequest request) throws ServiceException {
2084           return stub.deleteNamespace(controller, request);
2085         }
2086 
2087         @Override
2088         public GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller,
2089             GetNamespaceDescriptorRequest request) throws ServiceException {
2090           return stub.getNamespaceDescriptor(controller, request);
2091         }
2092 
2093         @Override
2094         public ListNamespacesResponse listNamespaces(RpcController controller,
2095             ListNamespacesRequest request) throws ServiceException {
2096           return stub.listNamespaces(controller, request);
2097         }
2098 
2099         @Override
2100         public MasterProtos.SetSnapshotCleanupResponse switchSnapshotCleanup(
2101             RpcController controller, MasterProtos.SetSnapshotCleanupRequest request)
2102             throws ServiceException {
2103           return stub.switchSnapshotCleanup(controller, request);
2104         }
2105 
2106         @Override
2107         public MasterProtos.IsSnapshotCleanupEnabledResponse isSnapshotCleanupEnabled(
2108             RpcController controller, MasterProtos.IsSnapshotCleanupEnabledRequest request)
2109             throws ServiceException {
2110           return stub.isSnapshotCleanupEnabled(controller, request);
2111         }
2112 
2113         @Override
2114         public HBaseProtos.LogEntry getLogEntries(RpcController controller,
2115             HBaseProtos.LogRequest request) throws ServiceException {
2116           return stub.getLogEntries(controller, request);
2117         }
2118 
2119         @Override
2120         public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller,
2121             ListNamespaceDescriptorsRequest request) throws ServiceException {
2122           return stub.listNamespaceDescriptors(controller, request);
2123         }
2124 
2125         @Override
2126         public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(
2127             RpcController controller, ListTableDescriptorsByNamespaceRequest request)
2128                 throws ServiceException {
2129           return stub.listTableDescriptorsByNamespace(controller, request);
2130         }
2131 
2132         @Override
2133         public ListTableNamesByNamespaceResponse listTableNamesByNamespace(
2134             RpcController controller, ListTableNamesByNamespaceRequest request)
2135                 throws ServiceException {
2136           return stub.listTableNamesByNamespace(controller, request);
2137         }
2138 
2139         @Override
2140         public void close() {
2141           release(this.mss);
2142         }
2143 
2144         @Override
2145         public GetSchemaAlterStatusResponse getSchemaAlterStatus(
2146             RpcController controller, GetSchemaAlterStatusRequest request)
2147             throws ServiceException {
2148           return stub.getSchemaAlterStatus(controller, request);
2149         }
2150 
2151         @Override
2152         public GetTableDescriptorsResponse getTableDescriptors(
2153             RpcController controller, GetTableDescriptorsRequest request)
2154             throws ServiceException {
2155           return stub.getTableDescriptors(controller, request);
2156         }
2157 
2158         @Override
2159         public GetTableNamesResponse getTableNames(
2160             RpcController controller, GetTableNamesRequest request)
2161             throws ServiceException {
2162           return stub.getTableNames(controller, request);
2163         }
2164 
2165         @Override
2166         public GetClusterStatusResponse getClusterStatus(
2167             RpcController controller, GetClusterStatusRequest request)
2168             throws ServiceException {
2169           return stub.getClusterStatus(controller, request);
2170         }
2171 
2172         @Override
2173         public SetQuotaResponse setQuota(RpcController controller, SetQuotaRequest request)
2174             throws ServiceException {
2175           return stub.setQuota(controller, request);
2176         }
2177 
2178         @Override
2179         public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(
2180             RpcController controller, MajorCompactionTimestampRequest request)
2181             throws ServiceException {
2182           return stub.getLastMajorCompactionTimestamp(controller, request);
2183         }
2184 
2185         @Override
2186         public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
2187             RpcController controller, MajorCompactionTimestampForRegionRequest request)
2188             throws ServiceException {
2189           return stub.getLastMajorCompactionTimestampForRegion(controller, request);
2190         }
2191 
2192         @Override
2193         public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
2194             IsBalancerEnabledRequest request) throws ServiceException {
2195           return stub.isBalancerEnabled(controller, request);
2196         }
2197 
2198         @Override
2199         public MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(
2200           RpcController controller,
2201           MasterProtos.SetSplitOrMergeEnabledRequest request) throws ServiceException {
2202           return stub.setSplitOrMergeEnabled(controller, request);
2203         }
2204 
2205         @Override
2206         public MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(
2207           RpcController controller,
2208           MasterProtos.IsSplitOrMergeEnabledRequest request) throws ServiceException {
2209           return stub.isSplitOrMergeEnabled(controller, request);
2210         }
2211 
2212         @Override
2213         public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
2214             IsNormalizerEnabledRequest request) throws ServiceException {
2215           return stub.isNormalizerEnabled(controller, request);
2216         }
2217 
2218         @Override
2219         public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller,
2220             SecurityCapabilitiesRequest request) throws ServiceException {
2221           return stub.getSecurityCapabilities(controller, request);
2222         }
2223       };
2224     }
2225 
2226 
2227     private static void release(MasterServiceState mss) {
2228       if (mss != null && mss.connection != null) {
2229         ((HConnectionImplementation)mss.connection).releaseMaster(mss);
2230       }
2231     }
2232 
2233     private boolean isKeepAliveMasterConnectedAndRunning(MasterServiceState mss) {
2234       if (mss.getStub() == null){
2235         return false;
2236       }
2237       try {
2238         return mss.isMasterRunning();
2239       } catch (UndeclaredThrowableException e) {
2240         // It's somehow messy, but we can receive exceptions such as
2241         //  java.net.ConnectException but they're not declared. So we catch it...
2242         LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable());
2243         return false;
2244       } catch (ServiceException se) {
2245         LOG.warn("Checking master connection", se);
2246         return false;
2247       }
2248     }
2249 
2250     void releaseMaster(MasterServiceState mss) {
2251       if (mss.getStub() == null) return;
2252       synchronized (masterAndZKLock) {
2253         --mss.userCount;
2254       }
2255     }
2256 
2257     private void closeMasterService(MasterServiceState mss) {
2258       if (mss.getStub() != null) {
2259         LOG.info("Closing master protocol: " + mss);
2260         mss.clearStub();
2261       }
2262       mss.userCount = 0;
2263     }
2264 
2265     /**
2266      * Immediate close of the shared master. Can be by the delayed close or when closing the
2267      * connection itself.
2268      */
2269     private void closeMaster() {
2270       synchronized (masterAndZKLock) {
2271         closeMasterService(masterServiceState);
2272       }
2273     }
2274 
2275     void updateCachedLocation(HRegionInfo hri, ServerName source,
2276                               ServerName serverName, long seqNum) {
2277       HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum);
2278       cacheLocation(hri.getTable(), source, newHrl);
2279     }
2280 
2281     @Override
2282     public void deleteCachedRegionLocation(final HRegionLocation location) {
2283       metaCache.clearCache(location);
2284     }
2285 
2286     @Override
2287     public void updateCachedLocations(final TableName tableName, byte[] rowkey,
2288         final Object exception, final HRegionLocation source) {
2289       assert source != null;
2290       updateCachedLocations(tableName, source.getRegionInfo().getRegionName()
2291         , rowkey, exception, source.getServerName());
2292     }
2293 
2294     /**
2295      * Update the location with the new value (if the exception is a RegionMovedException)
2296      * or delete it from the cache. Does nothing if we can be sure from the exception that
2297      * the location is still accurate, or if the cache has already been updated.
2298      * @param exception an object (to simplify user code) on which we will try to find a nested
2299      *                  or wrapped or both RegionMovedException
2300      * @param source server that is the source of the location update.
2301      */
2302     @Override
2303     public void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey,
2304       final Object exception, final ServerName source) {
2305       if (rowkey == null || tableName == null) {
2306         LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) +
2307             ", tableName=" + (tableName == null ? "null" : tableName));
2308         return;
2309       }
2310 
2311       if (source == null) {
2312         // This should not happen, but let's secure ourselves.
2313         return;
2314       }
2315 
2316       if (regionName == null) {
2317         // we do not know which region, so just remove the cache entry for the row and server
2318         if (metrics != null) {
2319           metrics.incrCacheDroppingExceptions(exception);
2320         }
2321         metaCache.clearCache(tableName, rowkey, source);
2322         return;
2323       }
2324 
2325       // Is it something we have already updated?
2326       final RegionLocations oldLocations = getCachedLocation(tableName, rowkey);
2327       HRegionLocation oldLocation = null;
2328       if (oldLocations != null) {
2329         oldLocation = oldLocations.getRegionLocationByRegionName(regionName);
2330       }
2331       if (oldLocation == null || !source.equals(oldLocation.getServerName())) {
2332         // There is no such location in the cache (it's been removed already) or
2333         // the cache has already been refreshed with a different location.  => nothing to do
2334         return;
2335       }
2336 
2337       HRegionInfo regionInfo = oldLocation.getRegionInfo();
2338       Throwable cause = ClientExceptionsUtil.findException(exception);
2339       if (cause != null) {
2340         if (!ClientExceptionsUtil.isMetaClearingException(cause)) {
2341           // We know that the region is still on this region server
2342           return;
2343         }
2344 
2345         if (cause instanceof RegionMovedException) {
2346           RegionMovedException rme = (RegionMovedException) cause;
2347           if (LOG.isTraceEnabled()) {
2348             LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " +
2349                 rme.getHostname() + ":" + rme.getPort() +
2350                 " according to " + source.getHostAndPort());
2351           }
2352           // We know that the region is not anymore on this region server, but we know
2353           //  the new location.
2354           updateCachedLocation(
2355               regionInfo, source, rme.getServerName(), rme.getLocationSeqNum());
2356           return;
2357         }
2358       }
2359 
2360       if (metrics != null) {
2361         metrics.incrCacheDroppingExceptions(exception);
2362       }
2363 
2364       // If we're here, it means that can cannot be sure about the location, so we remove it from
2365       // the cache. Do not send the source because source can be a new server in the same host:port
2366       metaCache.clearCache(regionInfo);
2367     }
2368 
2369     @Override
2370     public void updateCachedLocations(final byte[] tableName, byte[] rowkey,
2371       final Object exception, final HRegionLocation source) {
2372       updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source);
2373     }
2374 
2375     @Override
2376     @Deprecated
2377     public void processBatch(List<? extends Row> list,
2378         final TableName tableName,
2379         ExecutorService pool,
2380         Object[] results) throws IOException, InterruptedException {
2381       // This belongs in HTable!!! Not in here.  St.Ack
2382 
2383       // results must be the same size as list
2384       if (results.length != list.size()) {
2385         throw new IllegalArgumentException(
2386           "argument results must be the same size as argument list");
2387       }
2388       processBatchCallback(list, tableName, pool, results, null);
2389     }
2390 
2391     @Override
2392     @Deprecated
2393     public void processBatch(List<? extends Row> list,
2394         final byte[] tableName,
2395         ExecutorService pool,
2396         Object[] results) throws IOException, InterruptedException {
2397       processBatch(list, TableName.valueOf(tableName), pool, results);
2398     }
2399 
2400     /**
2401      * Send the queries in parallel on the different region servers. Retries on failures.
2402      * If the method returns it means that there is no error, and the 'results' array will
2403      * contain no exception. On error, an exception is thrown, and the 'results' array will
2404      * contain results and exceptions.
2405      * @deprecated since 0.96 - Use {@link HTable#processBatchCallback} instead
2406      */
2407     @Override
2408     @Deprecated
2409     public <R> void processBatchCallback(
2410       List<? extends Row> list,
2411       TableName tableName,
2412       ExecutorService pool,
2413       Object[] results,
2414       Batch.Callback<R> callback)
2415       throws IOException, InterruptedException {
2416 
2417       AsyncRequestFuture ars = this.asyncProcess.submitAll(
2418           pool, tableName, list, callback, results);
2419       ars.waitUntilDone();
2420       if (ars.hasError()) {
2421         throw ars.getErrors();
2422       }
2423     }
2424 
2425     @Override
2426     @Deprecated
2427     public <R> void processBatchCallback(
2428       List<? extends Row> list,
2429       byte[] tableName,
2430       ExecutorService pool,
2431       Object[] results,
2432       Batch.Callback<R> callback)
2433       throws IOException, InterruptedException {
2434       processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
2435     }
2436 
2437     // For tests to override.
2438     protected AsyncProcess createAsyncProcess(Configuration conf) {
2439       // No default pool available.
2440       return new AsyncProcess(this, conf, batchPool, rpcCallerFactory, false, rpcControllerFactory,
2441         writeRpcTimeout);
2442     }
2443 
2444     @Override
2445     public AsyncProcess getAsyncProcess() {
2446       return asyncProcess;
2447     }
2448 
2449     @Override
2450     public ServerStatisticTracker getStatisticsTracker() {
2451       return this.stats;
2452     }
2453 
2454     @Override
2455     public ClientBackoffPolicy getBackoffPolicy() {
2456       return this.backoffPolicy;
2457     }
2458 
2459     /*
2460      * Return the number of cached region for a table. It will only be called
2461      * from a unit test.
2462      */
2463     int getNumberOfCachedRegionLocations(final TableName tableName) {
2464       return metaCache.getNumberOfCachedRegionLocations(tableName);
2465     }
2466 
2467     @Override
2468     @Deprecated
2469     public void setRegionCachePrefetch(final TableName tableName, final boolean enable) {
2470     }
2471 
2472     @Override
2473     @Deprecated
2474     public void setRegionCachePrefetch(final byte[] tableName,
2475         final boolean enable) {
2476     }
2477 
2478     @Override
2479     @Deprecated
2480     public boolean getRegionCachePrefetch(TableName tableName) {
2481       return false;
2482     }
2483 
2484     @Override
2485     @Deprecated
2486     public boolean getRegionCachePrefetch(byte[] tableName) {
2487       return false;
2488     }
2489 
2490     @Override
2491     public void abort(final String msg, Throwable t) {
2492       if (t instanceof KeeperException.SessionExpiredException
2493         && keepAliveZookeeper != null) {
2494         synchronized (masterAndZKLock) {
2495           if (keepAliveZookeeper != null) {
2496             LOG.warn("This client just lost it's session with ZooKeeper," +
2497               " closing it." +
2498               " It will be recreated next time someone needs it", t);
2499             closeZooKeeperWatcher();
2500           }
2501         }
2502       } else {
2503         if (t != null) {
2504           LOG.fatal(msg, t);
2505         } else {
2506           LOG.fatal(msg);
2507         }
2508         this.aborted = true;
2509         close();
2510         this.closed = true;
2511       }
2512     }
2513 
2514     @Override
2515     public boolean isClosed() {
2516       return this.closed;
2517     }
2518 
2519     @Override
2520     public boolean isAborted(){
2521       return this.aborted;
2522     }
2523 
2524     @Override
2525     public int getCurrentNrHRS() throws IOException {
2526       return this.registry.getCurrentNrHRS();
2527     }
2528 
2529     /**
2530      * Increment this client's reference count.
2531      */
2532     void incCount() {
2533       ++refCount;
2534     }
2535 
2536     /**
2537      * Decrement this client's reference count.
2538      */
2539     void decCount() {
2540       if (refCount > 0) {
2541         --refCount;
2542       }
2543     }
2544 
2545     /**
2546      * Return if this client has no reference
2547      *
2548      * @return true if this client has no reference; false otherwise
2549      */
2550     boolean isZeroReference() {
2551       return refCount == 0;
2552     }
2553 
2554     void internalClose() {
2555       if (this.closed) {
2556         return;
2557       }
2558       if (this.registry != null) {
2559         this.registry.close();
2560       }
2561       closeMaster();
2562       shutdownPools();
2563       if (this.metrics != null) {
2564         this.metrics.shutdown();
2565       }
2566       this.closed = true;
2567       closeZooKeeperWatcher();
2568       this.stubs.clear();
2569       if (clusterStatusListener != null) {
2570         clusterStatusListener.close();
2571       }
2572       if (rpcClient != null) {
2573         rpcClient.close();
2574       }
2575     }
2576 
2577     @Override
2578     public void close() {
2579       if (managed) {
2580         if (aborted) {
2581           ConnectionManager.deleteStaleConnection(this);
2582         } else {
2583           ConnectionManager.deleteConnection(this, false);
2584         }
2585       } else {
2586         internalClose();
2587       }
2588     }
2589 
2590     /**
2591      * Close the connection for good, regardless of what the current value of
2592      * {@link #refCount} is. Ideally, {@link #refCount} should be zero at this
2593      * point, which would be the case if all of its consumers close the
2594      * connection. However, on the off chance that someone is unable to close
2595      * the connection, perhaps because it bailed out prematurely, the method
2596      * below will ensure that this {@link HConnection} instance is cleaned up.
2597      * Caveat: The JVM may take an unknown amount of time to call finalize on an
2598      * unreachable object, so our hope is that every consumer cleans up after
2599      * itself, like any good citizen.
2600      */
2601     @Override
2602     protected void finalize() throws Throwable {
2603       super.finalize();
2604       // Pretend as if we are about to release the last remaining reference
2605       refCount = 1;
2606       close();
2607     }
2608 
2609     /**
2610      * @deprecated Use {@link Admin#listTables()} instead
2611      */
2612     @Deprecated
2613     @Override
2614     public HTableDescriptor[] listTables() throws IOException {
2615       MasterKeepAliveConnection master = getKeepAliveMasterService();
2616       try {
2617         GetTableDescriptorsRequest req =
2618           RequestConverter.buildGetTableDescriptorsRequest((List<TableName>)null);
2619         return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
2620       } catch (ServiceException se) {
2621         throw ProtobufUtil.getRemoteException(se);
2622       } finally {
2623         master.close();
2624       }
2625     }
2626 
2627     /**
2628      * @deprecated Use {@link Admin#listTableNames()} instead
2629      */
2630     @Deprecated
2631     @Override
2632     public String[] getTableNames() throws IOException {
2633       TableName[] tableNames = listTableNames();
2634       String result[] = new String[tableNames.length];
2635       for (int i = 0; i < tableNames.length; i++) {
2636         result[i] = tableNames[i].getNameAsString();
2637       }
2638       return result;
2639     }
2640 
2641     /**
2642      * @deprecated Use {@link Admin#listTableNames()} instead
2643      */
2644     @Deprecated
2645     @Override
2646     public TableName[] listTableNames() throws IOException {
2647       MasterKeepAliveConnection master = getKeepAliveMasterService();
2648       try {
2649         return ProtobufUtil.getTableNameArray(master.getTableNames(null,
2650             GetTableNamesRequest.newBuilder().build())
2651           .getTableNamesList());
2652       } catch (ServiceException se) {
2653         throw ProtobufUtil.getRemoteException(se);
2654       } finally {
2655         master.close();
2656       }
2657     }
2658 
2659     /**
2660      * @deprecated Use {@link Admin#getTableDescriptorsByTableName(List)} instead
2661      */
2662     @Deprecated
2663     @Override
2664     public HTableDescriptor[] getHTableDescriptorsByTableName(
2665         List<TableName> tableNames) throws IOException {
2666       if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
2667       MasterKeepAliveConnection master = getKeepAliveMasterService();
2668       try {
2669         GetTableDescriptorsRequest req =
2670           RequestConverter.buildGetTableDescriptorsRequest(tableNames);
2671         return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
2672       } catch (ServiceException se) {
2673         throw ProtobufUtil.getRemoteException(se);
2674       } finally {
2675         master.close();
2676       }
2677     }
2678 
2679     /**
2680      * @deprecated Use {@link Admin#getTableDescriptorsByTableName(List)} instead
2681      */
2682     @Deprecated
2683     @Override
2684     public HTableDescriptor[] getHTableDescriptors(
2685         List<String> names) throws IOException {
2686       List<TableName> tableNames = new ArrayList<TableName>(names.size());
2687       for(String name : names) {
2688         tableNames.add(TableName.valueOf(name));
2689       }
2690 
2691       return getHTableDescriptorsByTableName(tableNames);
2692     }
2693 
2694     @Override
2695     public NonceGenerator getNonceGenerator() {
2696       return this.nonceGenerator;
2697     }
2698 
2699     /**
2700      * Connects to the master to get the table descriptor.
2701      * @param tableName table name
2702      * @throws IOException if the connection to master fails or if the table
2703      *  is not found.
2704      * @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead
2705      */
2706     @Deprecated
2707     @Override
2708     public HTableDescriptor getHTableDescriptor(final TableName tableName)
2709     throws IOException {
2710       if (tableName == null) return null;
2711       MasterKeepAliveConnection master = getKeepAliveMasterService();
2712       GetTableDescriptorsResponse htds;
2713       try {
2714         GetTableDescriptorsRequest req =
2715           RequestConverter.buildGetTableDescriptorsRequest(tableName);
2716         htds = master.getTableDescriptors(null, req);
2717       } catch (ServiceException se) {
2718         throw ProtobufUtil.getRemoteException(se);
2719       } finally {
2720         master.close();
2721       }
2722       if (!htds.getTableSchemaList().isEmpty()) {
2723         return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
2724       }
2725       throw new TableNotFoundException(tableName.getNameAsString());
2726     }
2727 
2728     /**
2729      * @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead
2730      */
2731     @Deprecated
2732     @Override
2733     public HTableDescriptor getHTableDescriptor(final byte[] tableName)
2734     throws IOException {
2735       return getHTableDescriptor(TableName.valueOf(tableName));
2736     }
2737 
2738     @Override
2739     public RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf) {
2740       return RpcRetryingCallerFactory
2741           .instantiate(conf, this.interceptor, this.getStatisticsTracker());
2742     }
2743 
2744     @Override
2745     public boolean isManaged() {
2746       return managed;
2747     }
2748 
2749     @Override
2750     public boolean hasCellBlockSupport() {
2751       return this.rpcClient.hasCellBlockSupport();
2752     }
2753 
2754     @Override
2755     public ConnectionConfiguration getConnectionConfiguration() {
2756       return this.connectionConfig;
2757     }
2758 
2759     @Override
2760     public RpcRetryingCallerFactory getRpcRetryingCallerFactory() {
2761       return this.rpcCallerFactory;
2762     }
2763 
2764     @Override
2765     public RpcControllerFactory getRpcControllerFactory() {
2766       return this.rpcControllerFactory;
2767     }
2768   }
2769 
2770   /**
2771    * The record of errors for servers.
2772    */
2773   static class ServerErrorTracker {
2774     // We need a concurrent map here, as we could have multiple threads updating it in parallel.
2775     private final ConcurrentMap<ServerName, ServerErrors> errorsByServer =
2776         new ConcurrentHashMap<ServerName, ServerErrors>();
2777     private final long canRetryUntil;
2778     private final int maxRetries;
2779     private final long startTrackingTime;
2780 
2781     public ServerErrorTracker(long timeout, int maxRetries) {
2782       this.maxRetries = maxRetries;
2783       this.canRetryUntil = EnvironmentEdgeManager.currentTime() + timeout;
2784       this.startTrackingTime = new Date().getTime();
2785     }
2786 
2787     /**
2788      * We stop to retry when we have exhausted BOTH the number of retries and the time allocated.
2789      */
2790     boolean canRetryMore(int numRetry) {
2791       // If there is a single try we must not take into account the time.
2792       return numRetry < maxRetries || (maxRetries > 1 &&
2793           EnvironmentEdgeManager.currentTime() < this.canRetryUntil);
2794     }
2795 
2796     /**
2797      * Calculates the back-off time for a retrying request to a particular server.
2798      *
2799      * @param server    The server in question.
2800      * @param basePause The default hci pause.
2801      * @return The time to wait before sending next request.
2802      */
2803     long calculateBackoffTime(ServerName server, long basePause) {
2804       long result;
2805       ServerErrors errorStats = errorsByServer.get(server);
2806       if (errorStats != null) {
2807         result = ConnectionUtils.getPauseTime(basePause, errorStats.retries.get());
2808       } else {
2809         result = 0; // yes, if the server is not in our list we don't wait before retrying.
2810       }
2811       return result;
2812     }
2813 
2814     /**
2815      * Reports that there was an error on the server to do whatever bean-counting necessary.
2816      *
2817      * @param server The server in question.
2818      */
2819     void reportServerError(ServerName server) {
2820       ServerErrors errors = errorsByServer.get(server);
2821       if (errors != null) {
2822         errors.addError();
2823       } else {
2824         errors = errorsByServer.putIfAbsent(server, new ServerErrors());
2825         if (errors != null){
2826           errors.addError();
2827         }
2828       }
2829     }
2830 
2831     long getStartTrackingTime() {
2832       return startTrackingTime;
2833     }
2834 
2835     /**
2836      * The record of errors for a server.
2837      */
2838     private static class ServerErrors {
2839       public final AtomicInteger retries = new AtomicInteger(0);
2840 
2841       public void addError() {
2842         retries.incrementAndGet();
2843       }
2844     }
2845   }
2846 }