View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.Closeable;
23  import java.io.IOException;
24  import java.lang.reflect.Proxy;
25  import java.lang.reflect.UndeclaredThrowableException;
26  import java.net.InetSocketAddress;
27  import java.util.ArrayList;
28  import java.util.Collections;
29  import java.util.HashMap;
30  import java.util.HashSet;
31  import java.util.LinkedHashMap;
32  import java.util.List;
33  import java.util.Map;
34  import java.util.Map.Entry;
35  import java.util.NavigableMap;
36  import java.util.Set;
37  import java.util.TreeMap;
38  import java.util.concurrent.Callable;
39  import java.util.concurrent.ConcurrentHashMap;
40  import java.util.concurrent.CopyOnWriteArraySet;
41  import java.util.concurrent.ExecutionException;
42  import java.util.concurrent.ExecutorService;
43  import java.util.concurrent.Future;
44  import java.util.concurrent.SynchronousQueue;
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  
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.DoNotRetryIOException;
54  import org.apache.hadoop.hbase.HBaseConfiguration;
55  import org.apache.hadoop.hbase.HConstants;
56  import org.apache.hadoop.hbase.HRegionInfo;
57  import org.apache.hadoop.hbase.HRegionLocation;
58  import org.apache.hadoop.hbase.HServerAddress;
59  import org.apache.hadoop.hbase.HTableDescriptor;
60  import org.apache.hadoop.hbase.KeyValue;
61  import org.apache.hadoop.hbase.MasterAddressTracker;
62  import org.apache.hadoop.hbase.MasterNotRunningException;
63  import org.apache.hadoop.hbase.RemoteExceptionHandler;
64  import org.apache.hadoop.hbase.ServerName;
65  import org.apache.hadoop.hbase.TableNotFoundException;
66  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
67  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
68  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
69  import org.apache.hadoop.hbase.client.coprocessor.Batch;
70  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
71  import org.apache.hadoop.hbase.ipc.ExecRPCInvoker;
72  import org.apache.hadoop.hbase.ipc.HBaseRPC;
73  import org.apache.hadoop.hbase.ipc.HMasterInterface;
74  import org.apache.hadoop.hbase.ipc.HRegionInterface;
75  import org.apache.hadoop.hbase.ipc.RpcEngine;
76  import org.apache.hadoop.hbase.security.User;
77  import org.apache.hadoop.hbase.util.Addressing;
78  import org.apache.hadoop.hbase.util.Bytes;
79  import org.apache.hadoop.hbase.util.Pair;
80  import org.apache.hadoop.hbase.util.SoftValueSortedMap;
81  import org.apache.hadoop.hbase.util.Threads;
82  import org.apache.hadoop.hbase.util.Writables;
83  import org.apache.hadoop.hbase.zookeeper.ClusterId;
84  import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
85  import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
86  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
87  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
88  import org.apache.hadoop.ipc.RemoteException;
89  import org.apache.zookeeper.KeeperException;
90  
91  /**
92   * A non-instantiable class that manages creation of {@link HConnection}s.
93   * <p>The simplest way to use this class is by using {@link #createConnection(Configuration)}.
94   * This creates a new {@link HConnection} that is managed by the caller.
95   * From this {@link HConnection} {@link HTableInterface} implementations are retrieved 
96   * with {@link HConnection#getTable(byte[])}. Example:
97   * <pre>
98   * {@code
99   * HConnection connection = HConnectionManager.createConnection(config);
100  * HTableInterface table = connection.getTable("table1");
101  * // use the table as needed, for a single operation and a single thread
102  * table.close();
103  * connection.close();
104  * }
105  * </pre>
106  * <p>The following logic and API will be removed in the future:
107  * <p>This class has a static Map of {@link HConnection} instances keyed by
108  * {@link Configuration}; all invocations of {@link #getConnection(Configuration)}
109  * that pass the same {@link Configuration} instance will be returned the same
110  * {@link  HConnection} instance (Adding properties to a Configuration
111  * instance does not change its object identity).  Sharing {@link HConnection}
112  * instances is usually what you want; all clients of the {@link HConnection}
113  * instances share the HConnections' cache of Region locations rather than each
114  * having to discover for itself the location of meta, root, etc.  It makes
115  * sense for the likes of the pool of HTables class {@link HTablePool}, for
116  * instance (If concerned that a single {@link HConnection} is insufficient
117  * for sharing amongst clients in say an heavily-multithreaded environment,
118  * in practise its not proven to be an issue.  Besides, {@link HConnection} is
119  * implemented atop Hadoop RPC and as of this writing, Hadoop RPC does a
120  * connection per cluster-member, exclusively).
121  *
122  * <p>But sharing connections
123  * makes clean up of {@link HConnection} instances a little awkward.  Currently,
124  * clients cleanup by calling
125  * {@link #deleteConnection(Configuration)}.  This will shutdown the
126  * zookeeper connection the HConnection was using and clean up all
127  * HConnection resources as well as stopping proxies to servers out on the
128  * cluster. Not running the cleanup will not end the world; it'll
129  * just stall the closeup some and spew some zookeeper connection failed
130  * messages into the log.  Running the cleanup on a {@link HConnection} that is
131  * subsequently used by another will cause breakage so be careful running
132  * cleanup.
133  * <p>To create a {@link HConnection} that is not shared by others, you can
134  * create a new {@link Configuration} instance, pass this new instance to
135  * {@link #getConnection(Configuration)}, and then when done, close it up by
136  * doing something like the following:
137  * <pre>
138  * {@code
139  * Configuration newConfig = new Configuration(originalConf);
140  * HConnection connection = HConnectionManager.getConnection(newConfig);
141  * // Use the connection to your hearts' delight and then when done...
142  * HConnectionManager.deleteConnection(newConfig, true);
143  * }
144  * </pre>
145  * <p>Cleanup used to be done inside in a shutdown hook.  On startup we'd
146  * register a shutdown hook that called {@link #deleteAllConnections()}
147  * on its way out but the order in which shutdown hooks run is not defined so
148  * were problematic for clients of HConnection that wanted to register their
149  * own shutdown hooks so we removed ours though this shifts the onus for
150  * cleanup to the client.
151  */
152 @SuppressWarnings("serial")
153 public class HConnectionManager {
154   // An LRU Map of HConnectionKey -> HConnection (TableServer).  All
155   // access must be synchronized.  This map is not private because tests
156   // need to be able to tinker with it.
157   static final Map<HConnectionKey, HConnectionImplementation> HBASE_INSTANCES;
158 
159   public static final int MAX_CACHED_HBASE_INSTANCES;
160 
161   private static Log LOG = LogFactory.getLog(HConnectionManager.class);
162 
163   static {
164     // We set instances to one more than the value specified for {@link
165     // HConstants#ZOOKEEPER_MAX_CLIENT_CNXNS}. By default, the zk default max
166     // connections to the ensemble from the one client is 30, so in that case we
167     // should run into zk issues before the LRU hit this value of 31.
168     MAX_CACHED_HBASE_INSTANCES = HBaseConfiguration.create().getInt(
169         HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS,
170         HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS) + 1;
171     HBASE_INSTANCES = new LinkedHashMap<HConnectionKey, HConnectionImplementation>(
172         (int) (MAX_CACHED_HBASE_INSTANCES / 0.75F) + 1, 0.75F, true) {
173        @Override
174       protected boolean removeEldestEntry(
175           Map.Entry<HConnectionKey, HConnectionImplementation> eldest) {
176          return size() > MAX_CACHED_HBASE_INSTANCES;
177        }
178     };
179   }
180 
181   /*
182    * Non-instantiable.
183    */
184   protected HConnectionManager() {
185     super();
186   }
187 
188   /**
189    * Get the connection that goes with the passed <code>conf</code>
190    * configuration instance.
191    * If no current connection exists, method creates a new connection for the
192    * passed <code>conf</code> instance.
193    * @param conf configuration
194    * @return HConnection object for <code>conf</code>
195    * @throws ZooKeeperConnectionException
196    */
197   public static HConnection getConnection(Configuration conf)
198   throws ZooKeeperConnectionException {
199     HConnectionKey connectionKey = new HConnectionKey(conf);
200     synchronized (HBASE_INSTANCES) {
201       HConnectionImplementation connection = HBASE_INSTANCES.get(connectionKey);
202       if (connection == null) {
203         connection = new HConnectionImplementation(conf, true, null);
204         HBASE_INSTANCES.put(connectionKey, connection);
205       } else if (connection.isClosed()) {
206         HConnectionManager.deleteConnection(connectionKey, true);
207         connection = new HConnectionImplementation(conf, true, null);
208         HBASE_INSTANCES.put(connectionKey, connection);
209       }
210       connection.incCount();
211       return connection;
212     }
213   }
214 
215   /**
216    * Create a new HConnection instance using the passed <code>conf</code>
217    * instance.
218    * Note: This bypasses the usual HConnection life cycle management.
219    * The caller is responsible for calling {@link HConnection#close()}
220    * on the returned connection instance.
221    *
222    * This is the recommended way to create HConnections.
223    * {@code
224    * HConnection connection = HConnectionManager.createConnection(conf);
225    * HTableInterface table = connection.getTable("mytable");
226    * table.get(...);
227    * ...
228    * table.close();
229    * connection.close();
230    * }
231    * @param conf configuration
232    * @return HConnection object for <code>conf</code>
233    * @throws ZooKeeperConnectionException
234    */
235   public static HConnection createConnection(Configuration conf)
236   throws ZooKeeperConnectionException {
237     return new HConnectionImplementation(conf, false, null);
238   }
239 
240   public static HConnection createConnection(Configuration conf, ExecutorService pool)
241       throws IOException {
242     return new HConnectionImplementation(conf, false, pool);
243   }
244 
245   /**
246    * Delete connection information for the instance specified by configuration.
247    * If there are no more references to it, this will then close connection to
248    * the zookeeper ensemble and let go of all resources.
249    *
250    * @param conf
251    *          configuration whose identity is used to find {@link HConnection}
252    *          instance.
253    * @param stopProxy
254    *          No longer used.  This parameter is ignored.
255    * @deprecated use {@link #createConnection(org.apache.hadoop.conf.Configuration)} instead
256    */
257   @Deprecated
258   public static void deleteConnection(Configuration conf, boolean stopProxy) {
259     deleteConnection(conf);
260   }
261 
262   /**
263    * Delete connection information for the instance specified by configuration.
264    * If there are no more references to it, this will then close connection to
265    * the zookeeper ensemble and let go of all resources.
266    *
267    * @param conf
268    *          configuration whose identity is used to find {@link HConnection}
269    *          instance.
270    */
271   @Deprecated
272   public static void deleteConnection(Configuration conf) {
273     deleteConnection(new HConnectionKey(conf), false);
274   }
275 
276   /**
277    * Delete stale connection information for the instance specified by configuration.
278    * This will then close connection to
279    * the zookeeper ensemble and let go of all resources.
280    *
281    * @param connection
282    */
283   public static void deleteStaleConnection(HConnection connection) {
284     deleteConnection(connection, true);
285   }
286 
287   /**
288    * Delete information for all connections.
289    * @param stopProxy No longer used.  This parameter is ignored.
290    * @deprecated use {@link #deleteAllConnections()} instead
291    */
292   @Deprecated
293   public static void deleteAllConnections(boolean stopProxy) {
294     deleteAllConnections();
295   }
296 
297   /**
298    * Delete information for all connections.
299    * @throws IOException
300    */
301   @Deprecated
302   public static void deleteAllConnections() {
303     synchronized (HBASE_INSTANCES) {
304       Set<HConnectionKey> connectionKeys = new HashSet<HConnectionKey>();
305       connectionKeys.addAll(HBASE_INSTANCES.keySet());
306       for (HConnectionKey connectionKey : connectionKeys) {
307         deleteConnection(connectionKey, false);
308       }
309       HBASE_INSTANCES.clear();
310     }
311   }
312 
313   @Deprecated
314   private static void deleteConnection(HConnection connection, boolean staleConnection) {
315     synchronized (HBASE_INSTANCES) {
316       for (Entry<HConnectionKey, HConnectionImplementation> connectionEntry : HBASE_INSTANCES
317           .entrySet()) {
318         if (connectionEntry.getValue() == connection) {
319           deleteConnection(connectionEntry.getKey(), staleConnection);
320           break;
321         }
322       }
323     }
324   }
325 
326   @Deprecated
327   private static void deleteConnection(HConnectionKey connectionKey,
328       boolean staleConnection) {
329     synchronized (HBASE_INSTANCES) {
330       HConnectionImplementation connection = HBASE_INSTANCES
331           .get(connectionKey);
332       if (connection != null) {
333         connection.decCount();
334         if (connection.isZeroReference() || staleConnection) {
335           HBASE_INSTANCES.remove(connectionKey);
336           connection.internalClose();
337         }
338       }else {
339         LOG.error("Connection not found in the list, can't delete it "+
340           "(connection key="+connectionKey+"). May be the key was modified?");
341       }
342     }
343   }
344 
345   /**
346    * It is provided for unit test cases which verify the behavior of region
347    * location cache prefetch.
348    * @return Number of cached regions for the table.
349    * @throws ZooKeeperConnectionException
350    */
351   static int getCachedRegionCount(Configuration conf,
352       final byte[] tableName)
353   throws IOException {
354     return execute(new HConnectable<Integer>(conf) {
355       @Override
356       public Integer connect(HConnection connection) {
357         return ((HConnectionImplementation) connection)
358             .getNumberOfCachedRegionLocations(tableName);
359       }
360     });
361   }
362 
363   /**
364    * It's provided for unit test cases which verify the behavior of region
365    * location cache prefetch.
366    * @return true if the region where the table and row reside is cached.
367    * @throws ZooKeeperConnectionException
368    */
369   static boolean isRegionCached(Configuration conf,
370       final byte[] tableName, final byte[] row) throws IOException {
371     return execute(new HConnectable<Boolean>(conf) {
372       @Override
373       public Boolean connect(HConnection connection) {
374         return ((HConnectionImplementation) connection).isRegionCached(tableName, row);
375       }
376     });
377   }
378 
379   /**
380    * This class makes it convenient for one to execute a command in the context
381    * of a {@link HConnection} instance based on the given {@link Configuration}.
382    *
383    * <p>
384    * If you find yourself wanting to use a {@link HConnection} for a relatively
385    * short duration of time, and do not want to deal with the hassle of creating
386    * and cleaning up that resource, then you should consider using this
387    * convenience class.
388    *
389    * @param <T>
390    *          the return type of the {@link HConnectable#connect(HConnection)}
391    *          method.
392    */
393   public static abstract class HConnectable<T> {
394     public Configuration conf;
395 
396     public HConnectable(Configuration conf) {
397       this.conf = conf;
398     }
399 
400     public abstract T connect(HConnection connection) throws IOException;
401   }
402 
403   /**
404    * This convenience method invokes the given {@link HConnectable#connect}
405    * implementation using a {@link HConnection} instance that lasts just for the
406    * duration of that invocation.
407    *
408    * @param <T> the return type of the connect method
409    * @param connectable the {@link HConnectable} instance
410    * @return the value returned by the connect method
411    * @throws IOException
412    */
413   public static <T> T execute(HConnectable<T> connectable) throws IOException {
414     if (connectable == null || connectable.conf == null) {
415       return null;
416     }
417     Configuration conf = connectable.conf;
418     HConnection connection = HConnectionManager.getConnection(conf);
419     boolean connectSucceeded = false;
420     try {
421       T returnValue = connectable.connect(connection);
422       connectSucceeded = true;
423       return returnValue;
424     } finally {
425       try {
426         connection.close();
427       } catch (Exception e) {
428         if (connectSucceeded) {
429           throw new IOException("The connection to " + connection
430               + " could not be deleted.", e);
431         }
432       }
433     }
434   }
435 
436   /**
437    * Denotes a unique key to a {@link HConnection} instance.
438    *
439    * In essence, this class captures the properties in {@link Configuration}
440    * that may be used in the process of establishing a connection. In light of
441    * that, if any new such properties are introduced into the mix, they must be
442    * added to the {@link HConnectionKey#properties} list.
443    *
444    */
445   public static class HConnectionKey {
446     public static String[] CONNECTION_PROPERTIES = new String[] {
447         HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT,
448         HConstants.ZOOKEEPER_CLIENT_PORT,
449         HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
450         HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER,
451         HConstants.HBASE_CLIENT_RPC_MAXATTEMPTS,
452         HConstants.HBASE_RPC_TIMEOUT_KEY,
453         HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
454         HConstants.HBASE_META_SCANNER_CACHING,
455         HConstants.HBASE_CLIENT_INSTANCE_ID };
456 
457     private Map<String, String> properties;
458     private String username;
459 
460     public HConnectionKey(Configuration conf) {
461       Map<String, String> m = new HashMap<String, String>();
462       if (conf != null) {
463         for (String property : CONNECTION_PROPERTIES) {
464           String value = conf.get(property);
465           if (value != null) {
466             m.put(property, value);
467           }
468         }
469       }
470       this.properties = Collections.unmodifiableMap(m);
471 
472       try {
473         UserProvider provider = UserProvider.instantiate(conf);
474         username = provider.getCurrentUserName();
475       } catch (IOException ioe) {
476         LOG.warn("Error obtaining current user, skipping username in HConnectionKey",
477             ioe);
478       }
479     }
480 
481     @Override
482     public int hashCode() {
483       final int prime = 31;
484       int result = 1;
485       if (username != null) {
486         result = username.hashCode();
487       }
488       for (String property : CONNECTION_PROPERTIES) {
489         String value = properties.get(property);
490         if (value != null) {
491           result = prime * result + value.hashCode();
492         }
493       }
494 
495       return result;
496     }
497 
498     @Override
499     public boolean equals(Object obj) {
500       if (this == obj)
501         return true;
502       if (obj == null)
503         return false;
504       if (getClass() != obj.getClass())
505         return false;
506       HConnectionKey that = (HConnectionKey) obj;
507       if (this.username != null && !this.username.equals(that.username)) {
508         return false;
509       } else if (this.username == null && that.username != null) {
510         return false;
511       }
512       if (this.properties == null) {
513         if (that.properties != null) {
514           return false;
515         }
516       } else {
517         if (that.properties == null) {
518           return false;
519         }
520         for (String property : CONNECTION_PROPERTIES) {
521           String thisValue = this.properties.get(property);
522           String thatValue = that.properties.get(property);
523           if (thisValue == thatValue) {
524             continue;
525           }
526           if (thisValue == null || !thisValue.equals(thatValue)) {
527             return false;
528           }
529         }
530       }
531       return true;
532     }
533 
534     @Override
535     public String toString() {
536       return "HConnectionKey{" +
537         "properties=" + properties +
538         ", username='" + username + '\'' +
539         '}';
540     }
541   }
542 
543   /* Encapsulates connection to zookeeper and regionservers.*/
544   static class HConnectionImplementation implements HConnection, Closeable {
545     static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
546     private final Class<? extends HRegionInterface> serverInterfaceClass;
547     private final long pause;
548     private final int numRetries;
549     private final int maxRPCAttempts;
550     private final int rpcTimeout;
551     private final int prefetchRegionLimit;
552 
553     private final Object masterLock = new Object();
554     private volatile boolean closed;
555     private volatile boolean aborted;
556     private volatile boolean resetting;
557     private volatile HMasterInterface master;
558     // ZooKeeper reference
559     private volatile ZooKeeperWatcher zooKeeper;
560     // ZooKeeper-based master address tracker
561     private volatile MasterAddressTracker masterAddressTracker;
562     private volatile RootRegionTracker rootRegionTracker;
563     private volatile ClusterId clusterId;
564 
565     private final Object metaRegionLock = new Object();
566 
567     private final Object userRegionLock = new Object();
568 	
569     private final Object resetLock = new Object();
570 
571     // thread executor shared by all HTableInterface instances created
572     // by this connection
573     private volatile ExecutorService batchPool = null;
574     private volatile boolean cleanupPool = false;
575 
576     private final Configuration conf;
577 
578     private RpcEngine rpcEngine;
579 
580     // Known region HServerAddress.toString() -> HRegionInterface
581 
582     private final Map<String, HRegionInterface> servers =
583       new ConcurrentHashMap<String, HRegionInterface>();
584     private final ConcurrentHashMap<String, String> connectionLock =
585       new ConcurrentHashMap<String, String>();
586 
587     /**
588      * Map of table to table {@link HRegionLocation}s.  The table key is made
589      * by doing a {@link Bytes#mapKey(byte[])} of the table's name.
590      */
591     private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>>
592       cachedRegionLocations =
593         new HashMap<Integer, SoftValueSortedMap<byte [], HRegionLocation>>();
594 
595     // The presence of a server in the map implies it's likely that there is an
596     // entry in cachedRegionLocations that map to this server; but the absence
597     // of a server in this map guarentees that there is no entry in cache that
598     // maps to the absent server.
599     private final Set<String> cachedServers =
600         new HashSet<String>();
601 
602     // region cache prefetch is enabled by default. this set contains all
603     // tables whose region cache prefetch are disabled.
604     private final Set<Integer> regionCachePrefetchDisabledTables =
605       new CopyOnWriteArraySet<Integer>();
606 
607     private int refCount;
608 
609     // indicates whether this connection's life cycle is managed
610     private final boolean managed;
611     /**
612      * constructor
613      * @param conf Configuration object
614      */
615     @SuppressWarnings("unchecked")
616     public HConnectionImplementation(Configuration conf, boolean managed, ExecutorService pool)
617     throws ZooKeeperConnectionException {
618       this.conf = conf;
619       this.batchPool = pool;
620       this.managed = managed;
621       String serverClassName = conf.get(HConstants.REGION_SERVER_CLASS,
622         HConstants.DEFAULT_REGION_SERVER_CLASS);
623       this.closed = false;
624       try {
625         this.serverInterfaceClass =
626           (Class<? extends HRegionInterface>) Class.forName(serverClassName);
627       } catch (ClassNotFoundException e) {
628         throw new UnsupportedOperationException(
629             "Unable to find region server interface " + serverClassName, e);
630       }
631       this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
632           HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
633       this.numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
634           HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
635       this.maxRPCAttempts = conf.getInt(
636           HConstants.HBASE_CLIENT_RPC_MAXATTEMPTS,
637           HConstants.DEFAULT_HBASE_CLIENT_RPC_MAXATTEMPTS);
638       this.rpcTimeout = conf.getInt(
639           HConstants.HBASE_RPC_TIMEOUT_KEY,
640           HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
641       this.prefetchRegionLimit = conf.getInt(
642           HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
643           HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT);
644 
645       this.master = null;
646       this.resetting = false;
647     }
648 
649     @Override
650     public HTableInterface getTable(String tableName) throws IOException {
651       return getTable(Bytes.toBytes(tableName));
652     }
653 
654     @Override
655     public HTableInterface getTable(byte[] tableName) throws IOException {
656       return getTable(tableName, getBatchPool());
657     }
658 
659     @Override
660     public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException {
661       return getTable(Bytes.toBytes(tableName), pool);
662     }
663 
664     @Override
665     public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException {
666       if (managed) {
667         throw new IOException("The connection has to be unmanaged.");
668       }
669       return new HTable(tableName, this, pool);
670     }
671 
672     private ExecutorService getBatchPool() {
673       if (batchPool == null) {
674         // shared HTable thread executor not yet initialized
675         synchronized (this) {
676           if (batchPool == null) {
677             int maxThreads = conf.getInt("hbase.hconnection.threads.max", Integer.MAX_VALUE);
678             if (maxThreads == 0) {
679               maxThreads = Runtime.getRuntime().availableProcessors();
680             }
681             long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60);
682             this.batchPool = new ThreadPoolExecutor(Runtime.getRuntime().availableProcessors(),
683                 maxThreads, keepAliveTime, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
684                 Threads.newDaemonThreadFactory("hbase-connection-shared-executor"));
685             ((ThreadPoolExecutor) this.batchPool).allowCoreThreadTimeOut(true);
686           }
687           this.cleanupPool = true;
688         }
689       }
690       return this.batchPool;
691     }
692 
693     protected ExecutorService getCurrentBatchPool() {
694       return batchPool;
695     }
696 
697     private void shutdownBatchPool() {
698       if (this.cleanupPool && this.batchPool != null && !this.batchPool.isShutdown()) {
699         this.batchPool.shutdown();
700         try {
701           if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) {
702             this.batchPool.shutdownNow();
703           }
704         } catch (InterruptedException e) {
705           this.batchPool.shutdownNow();
706         }
707       }
708     }
709 
710     private synchronized void ensureZookeeperTrackers()
711         throws ZooKeeperConnectionException {
712       // initialize zookeeper and master address manager
713       if (zooKeeper == null) {
714         zooKeeper = getZooKeeperWatcher();
715       }
716       if (clusterId == null) {
717         clusterId = new ClusterId(zooKeeper, this);
718         if (clusterId.hasId()) {
719           conf.set(HConstants.CLUSTER_ID, clusterId.getId());
720         }
721       }
722       if (masterAddressTracker == null) {
723         masterAddressTracker = new MasterAddressTracker(zooKeeper, this);
724         masterAddressTracker.start();
725       }
726       if (rootRegionTracker == null) {
727         rootRegionTracker = new RootRegionTracker(zooKeeper, this);
728         rootRegionTracker.start();
729       }
730       // RpcEngine needs access to zookeeper data, like cluster ID
731       if (rpcEngine == null) {
732         this.rpcEngine = HBaseRPC.getProtocolEngine(conf);
733       }
734     }
735 
736     private synchronized void resetZooKeeperTrackers() {
737       if (masterAddressTracker != null) {
738         masterAddressTracker.stop();
739         masterAddressTracker = null;
740       }
741       if (rootRegionTracker != null) {
742         rootRegionTracker.stop();
743         rootRegionTracker = null;
744       }
745       clusterId = null;
746       if (zooKeeper != null) {
747         zooKeeper.close();
748         zooKeeper = null;
749       }
750     }
751 
752     public Configuration getConfiguration() {
753       return this.conf;
754     }
755 
756     /**
757      * Log failure of getMaster attempt
758      * @return true if should retry
759      */
760     private boolean shouldRetryGetMaster(int tries, Exception e) {
761       if (tries == numRetries - 1) {
762         // This was our last chance - don't bother sleeping
763         LOG.info("getMaster attempt " + tries + " of " + numRetries +
764           " failed; no more retrying.", e);
765         return false;
766       }
767       LOG.info("getMaster attempt " + tries + " of " + numRetries +
768         " failed; retrying after sleep of " +
769         ConnectionUtils.getPauseTime(this.pause, tries), e);
770       return true;
771     }
772 
773     public HMasterInterface getMaster()
774     throws MasterNotRunningException, ZooKeeperConnectionException {
775       // TODO: REMOVE.  MOVE TO HBaseAdmin and redo as a Callable!!!
776 
777       // Check if we already have a good master connection
778       try {
779         if (master != null && master.isMasterRunning()) {
780           return master;
781         }
782       } catch (UndeclaredThrowableException ute) {
783         // log, but ignore, the loop below will attempt to reconnect
784         LOG.info("Exception contacting master. Retrying...", ute.getCause());
785       }
786 
787       ensureZookeeperTrackers();
788       checkIfBaseNodeAvailable();
789       ServerName sn = null;
790       synchronized (this.masterLock) {
791         try {
792           if (master != null && master.isMasterRunning()) {
793             return master;
794           }
795         } catch (UndeclaredThrowableException ute) {
796           // log, but ignore, the loop below will attempt to reconnect
797           LOG.info("Exception contacting master. Retrying...", ute.getCause());
798         }
799         this.master = null;
800 
801         for (int tries = 0;
802           !this.closed && this.master == null && tries < numRetries;
803         tries++) {
804 
805           try {
806             sn = masterAddressTracker.getMasterAddress();
807             if (sn == null) {
808               LOG.info("ZooKeeper available but no active master location found");
809               throw new MasterNotRunningException();
810             }
811 
812             InetSocketAddress isa =
813               new InetSocketAddress(sn.getHostname(), sn.getPort());
814             HMasterInterface tryMaster = rpcEngine.getProxy(
815                 HMasterInterface.class, HMasterInterface.VERSION, isa, this.conf,
816                 this.rpcTimeout);
817 
818             if (tryMaster.isMasterRunning()) {
819               this.master = tryMaster;
820               this.masterLock.notifyAll();
821               break;
822             }
823 
824           } catch (IOException e) {
825             if (!shouldRetryGetMaster(tries, e)) break;
826           } catch (UndeclaredThrowableException ute) {
827             if (!shouldRetryGetMaster(tries, ute)) break;
828           }
829 
830           // Cannot connect to master or it is not running. Sleep & retry
831           try {
832             this.masterLock.wait(ConnectionUtils.getPauseTime(this.pause, tries));
833           } catch (InterruptedException e) {
834             Thread.currentThread().interrupt();
835             throw new RuntimeException("Thread was interrupted while trying to connect to master.");
836           }
837         }
838 
839         if (this.master == null) {
840           if (sn == null) {
841             throw new MasterNotRunningException();
842           }
843           throw new MasterNotRunningException(sn.toString());
844         }
845         return this.master;
846       }
847     }
848 
849     private void checkIfBaseNodeAvailable() throws MasterNotRunningException {
850       if (false == masterAddressTracker.checkIfBaseNodeAvailable()) {
851         String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
852             + "There could be a mismatch with the one configured in the master.";
853         LOG.error(errorMsg);
854         throw new MasterNotRunningException(errorMsg);
855       }
856     }
857 
858     public boolean isMasterRunning()
859     throws MasterNotRunningException, ZooKeeperConnectionException {
860       if (this.master == null) {
861         getMaster();
862       }
863       boolean isRunning = master.isMasterRunning();
864       if(isRunning) {
865         return true;
866       }
867       throw new MasterNotRunningException();
868     }
869 
870     public HRegionLocation getRegionLocation(final byte [] name,
871         final byte [] row, boolean reload)
872     throws IOException {
873       return reload? relocateRegion(name, row): locateRegion(name, row);
874     }
875 
876     public boolean isTableEnabled(byte[] tableName) throws IOException {
877       return testTableOnlineState(tableName, true);
878     }
879 
880     public boolean isTableDisabled(byte[] tableName) throws IOException {
881       return testTableOnlineState(tableName, false);
882     }
883 
884     public boolean isTableAvailable(final byte[] tableName) throws IOException {
885       final AtomicBoolean available = new AtomicBoolean(true);
886       final AtomicInteger regionCount = new AtomicInteger(0);
887       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
888         @Override
889         public boolean processRow(Result row) throws IOException {
890           byte[] value = row.getValue(HConstants.CATALOG_FAMILY,
891               HConstants.REGIONINFO_QUALIFIER);
892           HRegionInfo info = Writables.getHRegionInfoOrNull(value);
893           if (info != null && !info.isSplitParent()) {
894             if (Bytes.equals(tableName, info.getTableName())) {
895               value = row.getValue(HConstants.CATALOG_FAMILY,
896                   HConstants.SERVER_QUALIFIER);
897               if (value == null) {
898                 available.set(false);
899                 return false;
900               }
901               regionCount.incrementAndGet();
902             }
903           }
904           return true;
905         }
906       };
907       MetaScanner.metaScan(conf, this, visitor, null);
908       return available.get() && (regionCount.get() > 0);
909     }
910 
911     /*
912      * @param True if table is online
913      */
914     private boolean testTableOnlineState(byte [] tableName, boolean online)
915     throws IOException {
916       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
917         // The root region is always enabled
918         return online;
919       }
920       ZooKeeperWatcher zkw = getZooKeeperWatcher();
921       String tableNameStr = Bytes.toString(tableName);
922       try {
923         if (online) {
924           return ZKTableReadOnly.isEnabledTable(zkw, tableNameStr);
925         }
926         return ZKTableReadOnly.isDisabledTable(zkw, tableNameStr);
927       } catch (KeeperException e) {
928         throw new IOException("Enable/Disable failed", e);
929       }
930     }
931 
932     @Override
933     public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
934       return locateRegion(HRegionInfo.getTableName(regionName),
935         HRegionInfo.getStartKey(regionName), false, true);
936     }
937 
938     @Override
939     public List<HRegionLocation> locateRegions(final byte[] tableName)
940     throws IOException {
941       return locateRegions(tableName, false, true);
942     }
943 
944     @Override
945     public List<HRegionLocation> locateRegions(final byte[] tableName, final boolean useCache,
946         final boolean offlined) throws IOException {
947       NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, this,
948           tableName, offlined);
949       final List<HRegionLocation> locations = new ArrayList<HRegionLocation>();
950       for (HRegionInfo regionInfo : regions.keySet()) {
951         locations.add(locateRegion(tableName, regionInfo.getStartKey(), useCache, true));
952       }
953       return locations;
954     }
955 
956     public HRegionLocation locateRegion(final byte [] tableName,
957         final byte [] row)
958     throws IOException{
959       return locateRegion(tableName, row, true, true);
960     }
961 
962     public HRegionLocation relocateRegion(final byte [] tableName,
963         final byte [] row)
964     throws IOException{
965 
966       // Since this is an explicit request not to use any caching, finding
967       // disabled tables should not be desirable.  This will ensure that an exception is thrown when
968       // the first time a disabled table is interacted with.
969       if (isTableDisabled(tableName)) {
970         throw new DoNotRetryIOException(Bytes.toString(tableName) + " is disabled.");
971       }
972 
973       return locateRegion(tableName, row, false, true);
974     }
975 
976     private HRegionLocation locateRegion(final byte [] tableName,
977       final byte [] row, boolean useCache, boolean retry)
978     throws IOException {
979       if (this.closed) throw new IOException(toString() + " closed");
980       if (tableName == null || tableName.length == 0) {
981         throw new IllegalArgumentException(
982             "table name cannot be null or zero length");
983       }
984       ensureZookeeperTrackers();
985       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
986         try {
987           ServerName servername = this.rootRegionTracker.waitRootRegionLocation(this.rpcTimeout);
988           LOG.debug("Looked up root region location, connection=" + this +
989             "; serverName=" + ((servername == null)? "": servername.toString()));
990           if (servername == null) return null;
991           return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
992             servername.getHostname(), servername.getPort());
993         } catch (InterruptedException e) {
994           Thread.currentThread().interrupt();
995           return null;
996         }
997       } else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
998         return locateRegionInMeta(HConstants.ROOT_TABLE_NAME, tableName, row,
999             useCache, metaRegionLock, retry);
1000       } else {
1001         // Region not in the cache - have to go to the meta RS
1002         return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
1003             useCache, userRegionLock, retry);
1004       }
1005     }
1006 
1007     /*
1008      * Search .META. for the HRegionLocation info that contains the table and
1009      * row we're seeking. It will prefetch certain number of regions info and
1010      * save them to the global region cache.
1011      */
1012     private void prefetchRegionCache(final byte[] tableName,
1013         final byte[] row) {
1014       // Implement a new visitor for MetaScanner, and use it to walk through
1015       // the .META.
1016       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
1017         public boolean processRow(Result result) throws IOException {
1018           try {
1019             byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
1020                 HConstants.REGIONINFO_QUALIFIER);
1021             HRegionInfo regionInfo = null;
1022 
1023             if (value != null) {
1024               // convert the row result into the HRegionLocation we need!
1025               regionInfo = Writables.getHRegionInfo(value);
1026 
1027               // possible we got a region of a different table...
1028               if (!Bytes.equals(regionInfo.getTableName(),
1029                   tableName)) {
1030                 return false; // stop scanning
1031               }
1032               if (regionInfo.isOffline()) {
1033                 // don't cache offline regions
1034                 return true;
1035               }
1036               value = result.getValue(HConstants.CATALOG_FAMILY,
1037                   HConstants.SERVER_QUALIFIER);
1038               if (value == null) {
1039                 return true;  // don't cache it
1040               }
1041               final String hostAndPort = Bytes.toString(value);
1042               String hostname = Addressing.parseHostname(hostAndPort);
1043               int port = Addressing.parsePort(hostAndPort);
1044               value = result.getValue(HConstants.CATALOG_FAMILY,
1045                   HConstants.STARTCODE_QUALIFIER);
1046               // instantiate the location
1047               HRegionLocation loc =
1048                 new HRegionLocation(regionInfo, hostname, port);
1049               // cache this meta entry
1050               cacheLocation(tableName, loc);
1051             }
1052             return true;
1053           } catch (RuntimeException e) {
1054             throw new IOException(e);
1055           }
1056         }
1057       };
1058       try {
1059         // pre-fetch certain number of regions info at region cache.
1060         MetaScanner.metaScan(conf, this, visitor, tableName, row,
1061             this.prefetchRegionLimit, HConstants.META_TABLE_NAME);
1062       } catch (IOException e) {
1063         LOG.warn("Encountered problems when prefetch META table: ", e);
1064       }
1065     }
1066 
1067     /*
1068       * Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation
1069       * info that contains the table and row we're seeking.
1070       */
1071     private HRegionLocation locateRegionInMeta(final byte [] parentTable,
1072       final byte [] tableName, final byte [] row, boolean useCache,
1073       Object regionLockObject, boolean retry)
1074     throws IOException {
1075       HRegionLocation location;
1076       // If we are supposed to be using the cache, look in the cache to see if
1077       // we already have the region.
1078       if (useCache) {
1079         location = getCachedLocation(tableName, row);
1080         if (location != null) {
1081           return location;
1082         }
1083       }
1084 
1085       int localNumRetries = retry ? numRetries : 1;
1086       // build the key of the meta region we should be looking for.
1087       // the extra 9's on the end are necessary to allow "exact" matches
1088       // without knowing the precise region names.
1089       byte [] metaKey = HRegionInfo.createRegionName(tableName, row,
1090         HConstants.NINES, false);
1091       for (int tries = 0; true; tries++) {
1092         if (tries >= localNumRetries) {
1093           throw new NoServerForRegionException("Unable to find region for "
1094             + Bytes.toStringBinary(row) + " after " + numRetries + " tries.");
1095         }
1096 
1097         HRegionLocation metaLocation = null;
1098         try {
1099           // locate the root or meta region
1100           metaLocation = locateRegion(parentTable, metaKey, true, false);
1101           // If null still, go around again.
1102           if (metaLocation == null) continue;
1103           HRegionInterface server =
1104             getHRegionConnection(metaLocation.getHostname(), metaLocation.getPort());
1105 
1106           Result regionInfoRow = null;
1107           // This block guards against two threads trying to load the meta
1108           // region at the same time. The first will load the meta region and
1109           // the second will use the value that the first one found.
1110           synchronized (regionLockObject) {
1111             // Check the cache again for a hit in case some other thread made the
1112             // same query while we were waiting on the lock.
1113             if (useCache) {
1114               location = getCachedLocation(tableName, row);
1115               if (location != null) {
1116                 return location;
1117               }
1118               // If the parent table is META, we may want to pre-fetch some
1119               // region info into the global region cache for this table.
1120               if (Bytes.equals(parentTable, HConstants.META_TABLE_NAME)
1121                   && (getRegionCachePrefetch(tableName))) {
1122                 prefetchRegionCache(tableName, row);
1123               }
1124               location = getCachedLocation(tableName, row);
1125               if (location != null) {
1126                 return location;
1127               }
1128             } else {
1129               // If we are not supposed to be using the cache, delete any existing cached location
1130               // so it won't interfere.
1131               deleteCachedLocation(tableName, row);
1132             }
1133 
1134             // Query the root or meta region for the location of the meta region
1135             regionInfoRow = server.getClosestRowBefore(
1136             metaLocation.getRegionInfo().getRegionName(), metaKey,
1137             HConstants.CATALOG_FAMILY);
1138           }
1139           if (regionInfoRow == null) {
1140             throw new TableNotFoundException(Bytes.toString(tableName));
1141           }
1142           byte [] value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
1143               HConstants.REGIONINFO_QUALIFIER);
1144           if (value == null || value.length == 0) {
1145             throw new IOException("HRegionInfo was null or empty in " +
1146               Bytes.toString(parentTable) + ", row=" + regionInfoRow);
1147           }
1148           // convert the row result into the HRegionLocation we need!
1149           HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
1150               value, new HRegionInfo());
1151           // possible we got a region of a different table...
1152           if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
1153             throw new TableNotFoundException(
1154                   "Table '" + Bytes.toString(tableName) + "' was not found, got: " +
1155                   Bytes.toString(regionInfo.getTableName()) + ".");
1156           }
1157           if (regionInfo.isSplit()) {
1158             throw new RegionOfflineException("the only available region for" +
1159               " the required row is a split parent," +
1160               " the daughters should be online soon: " +
1161               regionInfo.getRegionNameAsString());
1162           }
1163           if (regionInfo.isOffline()) {
1164             throw new RegionOfflineException("the region is offline, could" +
1165               " be caused by a disable table call: " +
1166               regionInfo.getRegionNameAsString());
1167           }
1168 
1169           value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
1170               HConstants.SERVER_QUALIFIER);
1171           String hostAndPort = "";
1172           if (value != null) {
1173             hostAndPort = Bytes.toString(value);
1174           }
1175           if (hostAndPort.equals("")) {
1176             throw new NoServerForRegionException("No server address listed " +
1177               "in " + Bytes.toString(parentTable) + " for region " +
1178               regionInfo.getRegionNameAsString() + " containing row " +
1179               Bytes.toStringBinary(row));
1180           }
1181 
1182           // Instantiate the location
1183           String hostname = Addressing.parseHostname(hostAndPort);
1184           int port = Addressing.parsePort(hostAndPort);
1185           location = new HRegionLocation(regionInfo, hostname, port);
1186           cacheLocation(tableName, location);
1187           return location;
1188         } catch (TableNotFoundException e) {
1189           // if we got this error, probably means the table just plain doesn't
1190           // exist. rethrow the error immediately. this should always be coming
1191           // from the HTable constructor.
1192           throw e;
1193         } catch (IOException e) {
1194           if (e instanceof RemoteException) {
1195             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
1196           }
1197           if (tries < numRetries - 1) {
1198             if (LOG.isDebugEnabled()) {
1199               LOG.debug("locateRegionInMeta parentTable=" +
1200                 Bytes.toString(parentTable) + ", metaLocation=" +
1201                 ((metaLocation == null)? "null": "{" + metaLocation + "}") +
1202                 ", attempt=" + tries + " of " +
1203                 this.numRetries + " failed; retrying after sleep of " +
1204                 ConnectionUtils.getPauseTime(this.pause, tries) + " because: " + e.getMessage());
1205             }
1206           } else {
1207             throw e;
1208           }
1209           // Only relocate the parent region if necessary
1210           if(!(e instanceof RegionOfflineException ||
1211               e instanceof NoServerForRegionException)) {
1212             relocateRegion(parentTable, metaKey);
1213           }
1214         }
1215         try{
1216           Thread.sleep(ConnectionUtils.getPauseTime(this.pause, tries));
1217         } catch (InterruptedException e) {
1218           Thread.currentThread().interrupt();
1219           throw new IOException("Giving up trying to location region in " +
1220             "meta: thread is interrupted.");
1221         }
1222       }
1223     }
1224 
1225     /*
1226      * Search the cache for a location that fits our table and row key.
1227      * Return null if no suitable region is located. TODO: synchronization note
1228      *
1229      * <p>TODO: This method during writing consumes 15% of CPU doing lookup
1230      * into the Soft Reference SortedMap.  Improve.
1231      *
1232      * @param tableName
1233      * @param row
1234      * @return Null or region location found in cache.
1235      */
1236     HRegionLocation getCachedLocation(final byte [] tableName,
1237         final byte [] row) {
1238       SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
1239         getTableLocations(tableName);
1240 
1241       // start to examine the cache. we can only do cache actions
1242       // if there's something in the cache for this table.
1243       if (tableLocations.isEmpty()) {
1244         return null;
1245       }
1246 
1247       HRegionLocation possibleRegion = tableLocations.get(row);
1248       if (possibleRegion != null) {
1249         return possibleRegion;
1250       }
1251 
1252       possibleRegion = tableLocations.lowerValueByKey(row);
1253       if (possibleRegion == null) {
1254         return null;
1255       }
1256 
1257       // make sure that the end key is greater than the row we're looking
1258       // for, otherwise the row actually belongs in the next region, not
1259       // this one. the exception case is when the endkey is
1260       // HConstants.EMPTY_END_ROW, signifying that the region we're
1261       // checking is actually the last region in the table.
1262       byte[] endKey = possibleRegion.getRegionInfo().getEndKey();
1263       if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
1264           KeyValue.getRowComparator(tableName).compareRows(
1265               endKey, 0, endKey.length, row, 0, row.length) > 0) {
1266         return possibleRegion;
1267       }
1268 
1269       // Passed all the way through, so we got nothin - complete cache miss
1270       return null;
1271     }
1272 
1273     /**
1274      * Delete a cached location
1275      * @param tableName tableName
1276      * @param row
1277      */
1278     void deleteCachedLocation(final byte [] tableName, final byte [] row) {
1279       synchronized (this.cachedRegionLocations) {
1280         Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
1281         if (!tableLocations.isEmpty()) {
1282           // start to examine the cache. we can only do cache actions
1283           // if there's something in the cache for this table.
1284           HRegionLocation rl = getCachedLocation(tableName, row);
1285           if (rl != null) {
1286             tableLocations.remove(rl.getRegionInfo().getStartKey());
1287             if (LOG.isDebugEnabled()) {
1288               LOG.debug("Removed " +
1289                 rl.getRegionInfo().getRegionNameAsString() +
1290                 " for tableName=" + Bytes.toString(tableName) +
1291                 " from cache " + "because of " + Bytes.toStringBinary(row));
1292             }
1293           }
1294         }
1295       }
1296     }
1297 
1298     @Override
1299     public void deleteCachedRegionLocation(final HRegionLocation location) {
1300       if (location == null) {
1301         return;
1302       }
1303       synchronized (this.cachedRegionLocations) {
1304         byte[] tableName = location.getRegionInfo().getTableName();
1305         Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
1306         if (!tableLocations.isEmpty()) {
1307           // Delete if there's something in the cache for this region.
1308           HRegionLocation removedLocation =
1309               tableLocations.remove(location.getRegionInfo().getStartKey());
1310           if (LOG.isDebugEnabled() && removedLocation != null) {
1311             LOG.debug("Removed " +
1312               location.getRegionInfo().getRegionNameAsString() +
1313               " for tableName=" + Bytes.toString(tableName) +
1314               " from cache");
1315           }
1316         }
1317       }
1318     }
1319 
1320     @Override
1321     public void clearCaches(String sn) {
1322       clearCachedLocationForServer(sn);
1323     }
1324 
1325     /*
1326      * Delete all cached entries of a table that maps to a specific location.
1327      *
1328      * @param tablename
1329      * @param server
1330      */
1331     private void clearCachedLocationForServer(final String server) {
1332       boolean deletedSomething = false;
1333       synchronized (this.cachedRegionLocations) {
1334         if (!cachedServers.contains(server)) {
1335           return;
1336         }
1337         for (Map<byte[], HRegionLocation> tableLocations :
1338           cachedRegionLocations.values()) {
1339           for (Entry<byte[], HRegionLocation> e : tableLocations.entrySet()) {
1340             HRegionLocation value = e.getValue();
1341             if (value != null
1342                 && value.getHostnamePort().equals(server)) {
1343               tableLocations.remove(e.getKey());
1344               deletedSomething = true;
1345             }
1346           }
1347         }
1348         cachedServers.remove(server);
1349       }
1350       if (deletedSomething && LOG.isDebugEnabled()) {
1351         LOG.debug("Removed all cached region locations that map to " + server);
1352       }
1353     }
1354 
1355     /*
1356      * @param tableName
1357      * @return Map of cached locations for passed <code>tableName</code>
1358      */
1359     private SoftValueSortedMap<byte [], HRegionLocation> getTableLocations(
1360         final byte [] tableName) {
1361       // find the map of cached locations for this table
1362       Integer key = Bytes.mapKey(tableName);
1363       SoftValueSortedMap<byte [], HRegionLocation> result;
1364       synchronized (this.cachedRegionLocations) {
1365         result = this.cachedRegionLocations.get(key);
1366         // if tableLocations for this table isn't built yet, make one
1367         if (result == null) {
1368           result = new SoftValueSortedMap<byte [], HRegionLocation>(
1369               Bytes.BYTES_COMPARATOR);
1370           this.cachedRegionLocations.put(key, result);
1371         }
1372       }
1373       return result;
1374     }
1375 
1376     @Override
1377     public void clearRegionCache() {
1378       synchronized(this.cachedRegionLocations) {
1379         this.cachedRegionLocations.clear();
1380         this.cachedServers.clear();
1381       }
1382     }
1383 
1384     @Override
1385     public void clearRegionCache(final byte [] tableName) {
1386       synchronized (this.cachedRegionLocations) {
1387         this.cachedRegionLocations.remove(Bytes.mapKey(tableName));
1388       }
1389     }
1390 
1391     /*
1392      * Put a newly discovered HRegionLocation into the cache.
1393      */
1394     private void cacheLocation(final byte [] tableName,
1395         final HRegionLocation location) {
1396       byte [] startKey = location.getRegionInfo().getStartKey();
1397       Map<byte [], HRegionLocation> tableLocations =
1398         getTableLocations(tableName);
1399       boolean hasNewCache = false;
1400       synchronized (this.cachedRegionLocations) {
1401         cachedServers.add(location.getHostnamePort());
1402         hasNewCache = (tableLocations.put(startKey, location) == null);
1403       }
1404       if (hasNewCache) {
1405         LOG.debug("Cached location for " +
1406             location.getRegionInfo().getRegionNameAsString() +
1407             " is " + location.getHostnamePort());
1408       }
1409     }
1410 
1411     public HRegionInterface getHRegionConnection(HServerAddress hsa)
1412     throws IOException {
1413       return getHRegionConnection(hsa, false);
1414     }
1415 
1416     @Override
1417     public HRegionInterface getHRegionConnection(final String hostname,
1418         final int port)
1419     throws IOException {
1420       return getHRegionConnection(hostname, port, false);
1421     }
1422 
1423     public HRegionInterface getHRegionConnection(HServerAddress hsa,
1424         boolean master)
1425     throws IOException {
1426       return getHRegionConnection(null, -1, hsa.getInetSocketAddress(), master);
1427     }
1428 
1429     @Override
1430     public HRegionInterface getHRegionConnection(final String hostname,
1431         final int port, final boolean master)
1432     throws IOException {
1433       return getHRegionConnection(hostname, port, null, master);
1434     }
1435 
1436     /**
1437      * Either the passed <code>isa</code> is null or <code>hostname</code>
1438      * can be but not both.
1439      * @param hostname
1440      * @param port
1441      * @param isa
1442      * @param master
1443      * @return Proxy.
1444      * @throws IOException
1445      */
1446     HRegionInterface getHRegionConnection(final String hostname, final int port,
1447         final InetSocketAddress isa, final boolean master)
1448     throws IOException {
1449       if (master) getMaster();
1450       HRegionInterface server;
1451       String rsName = null;
1452       if (isa != null) {
1453         rsName = Addressing.createHostAndPortStr(isa.getHostName(),
1454             isa.getPort());
1455       } else {
1456         rsName = Addressing.createHostAndPortStr(hostname, port);
1457       }
1458       ensureZookeeperTrackers();
1459       // See if we already have a connection (common case)
1460       server = this.servers.get(rsName);
1461       if (server == null) {
1462         // create a unique lock for this RS (if necessary)
1463         this.connectionLock.putIfAbsent(rsName, rsName);
1464         // get the RS lock
1465         synchronized (this.connectionLock.get(rsName)) {
1466           // do one more lookup in case we were stalled above
1467           server = this.servers.get(rsName);
1468           if (server == null) {
1469             try {
1470               // Only create isa when we need to.
1471               InetSocketAddress address = isa != null? isa:
1472                 new InetSocketAddress(hostname, port);
1473               // definitely a cache miss. establish an RPC for this RS
1474               server = HBaseRPC.waitForProxy(this.rpcEngine,
1475                   serverInterfaceClass, HRegionInterface.VERSION,
1476                   address, this.conf,
1477                   this.maxRPCAttempts, this.rpcTimeout, this.rpcTimeout);
1478               this.servers.put(Addressing.createHostAndPortStr(
1479                   address.getHostName(), address.getPort()), server);
1480             } catch (RemoteException e) {
1481               LOG.warn("RemoteException connecting to RS", e);
1482               // Throw what the RemoteException was carrying.
1483               throw e.unwrapRemoteException();
1484             }
1485           }
1486         }
1487       }
1488       return server;
1489     }
1490 
1491     /**
1492      * Get the ZooKeeper instance for this TableServers instance.
1493      *
1494      * If ZK has not been initialized yet, this will connect to ZK.
1495      * @returns zookeeper reference
1496      * @throws ZooKeeperConnectionException if there's a problem connecting to zk
1497      */
1498     @Deprecated
1499     public synchronized ZooKeeperWatcher getZooKeeperWatcher()
1500         throws ZooKeeperConnectionException {
1501       if(zooKeeper == null) {
1502         try {
1503           if (this.closed) {
1504             throw new IOException(toString() + " closed");
1505           }
1506           this.zooKeeper = new ZooKeeperWatcher(conf, "hconnection", this);
1507         } catch(ZooKeeperConnectionException zce) {
1508           throw zce;
1509         } catch (IOException e) {
1510           throw new ZooKeeperConnectionException("An error is preventing" +
1511               " HBase from connecting to ZooKeeper", e);
1512         }
1513       }
1514       return zooKeeper;
1515     }
1516 
1517     public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
1518     throws IOException, RuntimeException {
1519       return callable.withRetries();
1520     }
1521 
1522     public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
1523     throws IOException, RuntimeException {
1524       return callable.withoutRetries();
1525     }
1526 
1527     private <R> Callable<MultiResponse> createCallable(final HRegionLocation loc,
1528         final MultiAction<R> multi, final byte [] tableName) {
1529       // TODO: This does not belong in here!!! St.Ack  HConnections should
1530       // not be dealing in Callables; Callables have HConnections, not other
1531       // way around.
1532       final HConnection connection = this;
1533       return new Callable<MultiResponse>() {
1534        public MultiResponse call() throws IOException {
1535          ServerCallable<MultiResponse> callable =
1536            new ServerCallable<MultiResponse>(connection, tableName, null) {
1537              public MultiResponse call() throws IOException {
1538                return server.multi(multi);
1539              }
1540              @Override
1541              public void connect(boolean reload) throws IOException {
1542                server = connection.getHRegionConnection(loc.getHostname(), loc.getPort());
1543              }
1544            };
1545          return callable.withoutRetries();
1546        }
1547      };
1548    }
1549 
1550     public void processBatch(List<? extends Row> list,
1551         final byte[] tableName,
1552         ExecutorService pool,
1553         Object[] results) throws IOException, InterruptedException {
1554       // This belongs in HTable!!! Not in here.  St.Ack
1555 
1556       // results must be the same size as list
1557       if (results.length != list.size()) {
1558         throw new IllegalArgumentException("argument results must be the same size as argument list");
1559       }
1560 
1561       processBatchCallback(list, tableName, pool, results, null);
1562     }
1563 
1564     /**
1565      * Executes the given
1566      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call}
1567      * callable for each row in the
1568      * given list and invokes
1569      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)}
1570      * for each result returned.
1571      *
1572      * @param protocol the protocol interface being called
1573      * @param rows a list of row keys for which the callable should be invoked
1574      * @param tableName table name for the coprocessor invoked
1575      * @param pool ExecutorService used to submit the calls per row
1576      * @param callable instance on which to invoke
1577      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)}
1578      * for each row
1579      * @param callback instance on which to invoke
1580      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)}
1581      * for each result
1582      * @param <T> the protocol interface type
1583      * @param <R> the callable's return type
1584      * @throws IOException
1585      */
1586     public <T extends CoprocessorProtocol,R> void processExecs(
1587         final Class<T> protocol,
1588         List<byte[]> rows,
1589         final byte[] tableName,
1590         ExecutorService pool,
1591         final Batch.Call<T,R> callable,
1592         final Batch.Callback<R> callback)
1593       throws IOException, Throwable {
1594 
1595       Map<byte[],Future<R>> futures =
1596           new TreeMap<byte[],Future<R>>(Bytes.BYTES_COMPARATOR);
1597       for (final byte[] r : rows) {
1598         final ExecRPCInvoker invoker =
1599             new ExecRPCInvoker(conf, this, protocol, tableName, r);
1600         Future<R> future = pool.submit(
1601             new Callable<R>() {
1602               public R call() throws Exception {
1603                 T instance = (T)Proxy.newProxyInstance(conf.getClassLoader(),
1604                     new Class[]{protocol},
1605                     invoker);
1606                 R result = callable.call(instance);
1607                 byte[] region = invoker.getRegionName();
1608                 if (callback != null) {
1609                   callback.update(region, r, result);
1610                 }
1611                 return result;
1612               }
1613             });
1614         futures.put(r, future);
1615       }
1616       for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
1617         try {
1618           e.getValue().get();
1619         } catch (ExecutionException ee) {
1620           LOG.warn("Error executing for row "+Bytes.toStringBinary(e.getKey()), ee);
1621           throw ee.getCause();
1622         } catch (InterruptedException ie) {
1623           Thread.currentThread().interrupt();
1624           throw new IOException("Interrupted executing for row " +
1625               Bytes.toStringBinary(e.getKey()), ie);
1626         }
1627       }
1628     }
1629 
1630     /**
1631      * Parameterized batch processing, allowing varying return types for
1632      * different {@link Row} implementations.
1633      */
1634     public <R> void processBatchCallback(
1635         List<? extends Row> list,
1636         byte[] tableName,
1637         ExecutorService pool,
1638         Object[] results,
1639         Batch.Callback<R> callback)
1640     throws IOException, InterruptedException {
1641       // This belongs in HTable!!! Not in here.  St.Ack
1642 
1643       // results must be the same size as list
1644       if (results.length != list.size()) {
1645         throw new IllegalArgumentException(
1646             "argument results must be the same size as argument list");
1647       }
1648       if (list.isEmpty()) {
1649         return;
1650       }
1651 
1652       // Keep track of the most recent servers for any given item for better
1653       // exceptional reporting.  We keep HRegionLocation to save on parsing.
1654       // Later below when we use lastServers, we'll pull what we need from
1655       // lastServers.
1656       HRegionLocation [] lastServers = new HRegionLocation[results.length];
1657       List<Row> workingList = new ArrayList<Row>(list);
1658       boolean retry = true;
1659       // count that helps presize actions array
1660       int actionCount = 0;
1661 
1662       for (int tries = 0; tries < numRetries && retry; ++tries) {
1663 
1664         // sleep first, if this is a retry
1665         if (tries >= 1) {
1666           long sleepTime = ConnectionUtils.getPauseTime(this.pause, tries);
1667           LOG.debug("Retry " +tries+ ", sleep for " +sleepTime+ "ms!");
1668           Thread.sleep(sleepTime);
1669         }
1670         // step 1: break up into regionserver-sized chunks and build the data structs
1671         Map<HRegionLocation, MultiAction<R>> actionsByServer =
1672           new HashMap<HRegionLocation, MultiAction<R>>();
1673         for (int i = 0; i < workingList.size(); i++) {
1674           Row row = workingList.get(i);
1675           if (row != null) {
1676             HRegionLocation loc = locateRegion(tableName, row.getRow());
1677             byte[] regionName = loc.getRegionInfo().getRegionName();
1678 
1679             MultiAction<R> actions = actionsByServer.get(loc);
1680             if (actions == null) {
1681               actions = new MultiAction<R>();
1682               actionsByServer.put(loc, actions);
1683             }
1684 
1685             Action<R> action = new Action<R>(row, i);
1686             lastServers[i] = loc;
1687             actions.add(regionName, action);
1688           }
1689         }
1690 
1691         // step 2: make the requests
1692 
1693         Map<HRegionLocation, Future<MultiResponse>> futures =
1694             new HashMap<HRegionLocation, Future<MultiResponse>>(
1695                 actionsByServer.size());
1696 
1697         for (Entry<HRegionLocation, MultiAction<R>> e: actionsByServer.entrySet()) {
1698           futures.put(e.getKey(), pool.submit(createCallable(e.getKey(), e.getValue(), tableName)));
1699         }
1700 
1701         // step 3: collect the failures and successes and prepare for retry
1702 
1703         for (Entry<HRegionLocation, Future<MultiResponse>> responsePerServer
1704              : futures.entrySet()) {
1705           HRegionLocation loc = responsePerServer.getKey();
1706 
1707           try {
1708             Future<MultiResponse> future = responsePerServer.getValue();
1709             MultiResponse resp = future.get();
1710 
1711             if (resp == null) {
1712               // Entire server failed
1713               LOG.debug("Failed all for server: " + loc.getHostnamePort() +
1714                 ", removing from cache");
1715               continue;
1716             }
1717 
1718             for (Entry<byte[], List<Pair<Integer,Object>>> e : resp.getResults().entrySet()) {
1719               byte[] regionName = e.getKey();
1720               List<Pair<Integer, Object>> regionResults = e.getValue();
1721               for (Pair<Integer, Object> regionResult : regionResults) {
1722                 if (regionResult == null) {
1723                   // if the first/only record is 'null' the entire region failed.
1724                   LOG.debug("Failures for region: " +
1725                       Bytes.toStringBinary(regionName) +
1726                       ", removing from cache");
1727                 } else {
1728                   // Result might be an Exception, including DNRIOE
1729                   results[regionResult.getFirst()] = regionResult.getSecond();
1730                   if (callback != null && !(regionResult.getSecond() instanceof Throwable)) {
1731                     callback.update(e.getKey(),
1732                         list.get(regionResult.getFirst()).getRow(),
1733                         (R)regionResult.getSecond());
1734                   }
1735                 }
1736               }
1737             }
1738           } catch (ExecutionException e) {
1739             LOG.warn("Failed all from " + loc, e);
1740           }
1741         }
1742 
1743         // step 4: identify failures and prep for a retry (if applicable).
1744 
1745         // Find failures (i.e. null Result), and add them to the workingList (in
1746         // order), so they can be retried.
1747         retry = false;
1748         workingList.clear();
1749         actionCount = 0;
1750         for (int i = 0; i < results.length; i++) {
1751           // if null (fail) or instanceof Throwable && not instanceof DNRIOE
1752           // then retry that row. else dont.
1753           if (results[i] == null ||
1754               (results[i] instanceof Throwable &&
1755                   !(results[i] instanceof DoNotRetryIOException))) {
1756 
1757             retry = true;
1758             actionCount++;
1759             Row row = list.get(i);
1760             workingList.add(row);
1761             deleteCachedLocation(tableName, row.getRow());
1762           } else {
1763             if (results[i] != null && results[i] instanceof Throwable) {
1764               actionCount++;
1765             }
1766             // add null to workingList, so the order remains consistent with the original list argument.
1767             workingList.add(null);
1768           }
1769         }
1770       }
1771 
1772       List<Throwable> exceptions = new ArrayList<Throwable>(actionCount);
1773       List<Row> actions = new ArrayList<Row>(actionCount);
1774       List<String> addresses = new ArrayList<String>(actionCount);
1775 
1776       for (int i = 0 ; i < results.length; i++) {
1777         if (results[i] == null || results[i] instanceof Throwable) {
1778           exceptions.add((Throwable)results[i]);
1779           actions.add(list.get(i));
1780           addresses.add(lastServers[i].getHostnamePort());
1781         }
1782       }
1783 
1784       if (!exceptions.isEmpty()) {
1785         throw new RetriesExhaustedWithDetailsException(exceptions,
1786             actions,
1787             addresses);
1788       }
1789     }
1790 
1791     /*
1792      * Return the number of cached region for a table. It will only be called
1793      * from a unit test.
1794      */
1795     int getNumberOfCachedRegionLocations(final byte[] tableName) {
1796       Integer key = Bytes.mapKey(tableName);
1797       synchronized (this.cachedRegionLocations) {
1798         Map<byte[], HRegionLocation> tableLocs =
1799           this.cachedRegionLocations.get(key);
1800 
1801         if (tableLocs == null) {
1802           return 0;
1803         }
1804         return tableLocs.values().size();
1805       }
1806     }
1807 
1808     /**
1809      * Check the region cache to see whether a region is cached yet or not.
1810      * Called by unit tests.
1811      * @param tableName tableName
1812      * @param row row
1813      * @return Region cached or not.
1814      */
1815     boolean isRegionCached(final byte[] tableName, final byte[] row) {
1816       HRegionLocation location = getCachedLocation(tableName, row);
1817       return location != null;
1818     }
1819 
1820     public void setRegionCachePrefetch(final byte[] tableName,
1821         final boolean enable) {
1822       if (!enable) {
1823         regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName));
1824       }
1825       else {
1826         regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName));
1827       }
1828     }
1829 
1830     public boolean getRegionCachePrefetch(final byte[] tableName) {
1831       return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName));
1832     }
1833 
1834     @Override
1835     public void prewarmRegionCache(byte[] tableName,
1836         Map<HRegionInfo, HServerAddress> regions) {
1837       for (Map.Entry<HRegionInfo, HServerAddress> e : regions.entrySet()) {
1838         HServerAddress hsa = e.getValue();
1839         if (hsa == null || hsa.getInetSocketAddress() == null) continue;
1840         cacheLocation(tableName,
1841           new HRegionLocation(e.getKey(), hsa.getHostname(), hsa.getPort()));
1842       }
1843     }
1844 
1845     @Override
1846     public void abort(final String msg, Throwable t) {
1847       if (t instanceof KeeperException) {
1848         LOG.info("This client just lost it's session with ZooKeeper, will"
1849             + " automatically reconnect when needed.");
1850         if (t instanceof KeeperException.SessionExpiredException) {
1851           LOG.info("ZK session expired. This disconnect could have been" +
1852               " caused by a network partition or a long-running GC pause," +
1853               " either way it's recommended that you verify your environment.");
1854           synchronized (resetLock) {
1855             if (resetting) return;
1856             this.resetting = true;
1857           }
1858           resetZooKeeperTrackers();
1859           this.resetting = false;
1860         }
1861         return;
1862       }
1863       if (t != null) LOG.fatal(msg, t);
1864       else LOG.fatal(msg);
1865       this.aborted = true;
1866       close();
1867     }
1868 
1869     @Override
1870     public boolean isClosed() {
1871       return this.closed;
1872     }
1873 
1874     @Override
1875     public boolean isAborted(){
1876       return this.aborted;
1877     }
1878 
1879     public int getCurrentNrHRS() throws IOException {
1880       try {
1881         ZooKeeperWatcher zkw = getZooKeeperWatcher();
1882         // We go to zk rather than to master to get count of regions to avoid
1883         // HTable having a Master dependency.  See HBase-2828
1884         return ZKUtil.getNumberOfChildren(zkw,
1885             zkw.rsZNode);
1886       } catch (KeeperException ke) {
1887         throw new IOException("Unexpected ZooKeeper exception", ke);
1888       }
1889     }
1890 
1891     /**
1892      * Increment this client's reference count.
1893      */
1894     void incCount() {
1895       ++refCount;
1896     }
1897 
1898     /**
1899      * Decrement this client's reference count.
1900      */
1901     void decCount() {
1902       if (refCount > 0) {
1903         --refCount;
1904       }
1905     }
1906 
1907     /**
1908      * Return if this client has no reference
1909      *
1910      * @return true if this client has no reference; false otherwise
1911      */
1912     boolean isZeroReference() {
1913       return refCount == 0;
1914     }
1915 
1916     void internalClose() {
1917       if (this.closed) {
1918         return;
1919       }
1920       shutdownBatchPool();
1921       master = null;
1922 
1923       this.servers.clear();
1924       if (this.rpcEngine != null) {
1925         this.rpcEngine.close();
1926       }
1927 
1928       synchronized (this) {
1929         if (this.zooKeeper != null) {
1930           LOG.info("Closed zookeeper sessionid=0x" +
1931             Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1932           this.zooKeeper.close();
1933           this.zooKeeper = null;
1934         }
1935         this.closed = true;
1936       }
1937     }
1938 
1939     public void close() {
1940       if (managed) {
1941         if (aborted) {
1942           HConnectionManager.deleteStaleConnection(this);
1943         } else {
1944           HConnectionManager.deleteConnection(this, false);
1945         }
1946       } else {
1947         internalClose();
1948       }
1949       if (LOG.isTraceEnabled()) LOG.debug("" + this.zooKeeper + " closed.");
1950     }
1951 
1952     /**
1953      * Close the connection for good, regardless of what the current value of
1954      * {@link #refCount} is. Ideally, {@link #refCount} should be zero at this
1955      * point, which would be the case if all of its consumers close the
1956      * connection. However, on the off chance that someone is unable to close
1957      * the connection, perhaps because it bailed out prematurely, the method
1958      * below will ensure that this {@link HConnection} instance is cleaned up.
1959      * Caveat: The JVM may take an unknown amount of time to call finalize on an
1960      * unreachable object, so our hope is that every consumer cleans up after
1961      * itself, like any good citizen.
1962      */
1963     @Override
1964     protected void finalize() throws Throwable {
1965       // Pretend as if we are about to release the last remaining reference
1966       refCount = 1;
1967       close();
1968       LOG.debug("The connection to " + this.zooKeeper
1969           + " was closed by the finalize method.");
1970     }
1971 
1972     public HTableDescriptor[] listTables() throws IOException {
1973       HTableDescriptor[] htd = getMaster().getHTableDescriptors();
1974       return htd;
1975     }
1976 
1977     public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) throws IOException {
1978       if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
1979       if (tableNames == null || tableNames.size() == 0) return null;
1980       return getMaster().getHTableDescriptors(tableNames);
1981     }
1982 
1983     @Override
1984     public String[] getTableNames() throws IOException {
1985       return getMaster().getTableNames();
1986     }
1987 
1988     public HTableDescriptor getHTableDescriptor(final byte[] tableName)
1989     throws IOException {
1990       if (tableName == null || tableName.length == 0) return null;
1991       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
1992         return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
1993       }
1994       if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
1995         return HTableDescriptor.META_TABLEDESC;
1996       }
1997       List<String> tableNameList = new ArrayList<String>(1);
1998       tableNameList.add(Bytes.toString(tableName));
1999       HTableDescriptor[] htds = getHTableDescriptors(tableNameList);
2000       if (htds != null && htds.length > 0) {
2001         return htds[0];
2002       }
2003       throw new TableNotFoundException(Bytes.toString(tableName));
2004     }
2005   }
2006 
2007   /**
2008    * Set the number of retries to use serverside when trying to communicate
2009    * with another server over {@link HConnection}.  Used updating catalog
2010    * tables, etc.  Call this method before we create any Connections.
2011    * @param c The Configuration instance to set the retries into.
2012    * @param log Used to log what we set in here.
2013    */
2014   public static void setServerSideHConnectionRetries(final Configuration c,
2015       final Log log) {
2016     int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
2017       HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
2018     // Go big.  Multiply by 10.  If we can't get to meta after this many retries
2019     // then something seriously wrong.
2020     int serversideMultiplier =
2021       c.getInt("hbase.client.serverside.retries.multiplier", 10);
2022     int retries = hcRetries * serversideMultiplier;
2023     c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
2024     log.debug("Set serverside HConnection retries=" + retries);
2025   }
2026 }