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         User currentUser = User.getCurrent();
474         if (currentUser != null) {
475           username = currentUser.getName();
476         }
477       } catch (IOException ioe) {
478         LOG.warn("Error obtaining current user, skipping username in HConnectionKey",
479             ioe);
480       }
481     }
482 
483     @Override
484     public int hashCode() {
485       final int prime = 31;
486       int result = 1;
487       if (username != null) {
488         result = username.hashCode();
489       }
490       for (String property : CONNECTION_PROPERTIES) {
491         String value = properties.get(property);
492         if (value != null) {
493           result = prime * result + value.hashCode();
494         }
495       }
496 
497       return result;
498     }
499 
500     @Override
501     public boolean equals(Object obj) {
502       if (this == obj)
503         return true;
504       if (obj == null)
505         return false;
506       if (getClass() != obj.getClass())
507         return false;
508       HConnectionKey that = (HConnectionKey) obj;
509       if (this.username != null && !this.username.equals(that.username)) {
510         return false;
511       } else if (this.username == null && that.username != null) {
512         return false;
513       }
514       if (this.properties == null) {
515         if (that.properties != null) {
516           return false;
517         }
518       } else {
519         if (that.properties == null) {
520           return false;
521         }
522         for (String property : CONNECTION_PROPERTIES) {
523           String thisValue = this.properties.get(property);
524           String thatValue = that.properties.get(property);
525           if (thisValue == thatValue) {
526             continue;
527           }
528           if (thisValue == null || !thisValue.equals(thatValue)) {
529             return false;
530           }
531         }
532       }
533       return true;
534     }
535 
536     @Override
537     public String toString() {
538       return "HConnectionKey{" +
539         "properties=" + properties +
540         ", username='" + username + '\'' +
541         '}';
542     }
543   }
544 
545   /* Encapsulates connection to zookeeper and regionservers.*/
546   static class HConnectionImplementation implements HConnection, Closeable {
547     static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
548     private final Class<? extends HRegionInterface> serverInterfaceClass;
549     private final long pause;
550     private final int numRetries;
551     private final int maxRPCAttempts;
552     private final int rpcTimeout;
553     private final int prefetchRegionLimit;
554 
555     private final Object masterLock = new Object();
556     private volatile boolean closed;
557     private volatile boolean aborted;
558     private volatile boolean resetting;
559     private volatile HMasterInterface master;
560     // ZooKeeper reference
561     private volatile ZooKeeperWatcher zooKeeper;
562     // ZooKeeper-based master address tracker
563     private volatile MasterAddressTracker masterAddressTracker;
564     private volatile RootRegionTracker rootRegionTracker;
565     private volatile ClusterId clusterId;
566 
567     private final Object metaRegionLock = new Object();
568 
569     private final Object userRegionLock = new Object();
570 	
571     private final Object resetLock = new Object();
572 
573     // thread executor shared by all HTableInterface instances created
574     // by this connection
575     private volatile ExecutorService batchPool = null;
576     private volatile boolean cleanupPool = false;
577 
578     private final Configuration conf;
579 
580     private RpcEngine rpcEngine;
581 
582     // Known region HServerAddress.toString() -> HRegionInterface
583 
584     private final Map<String, HRegionInterface> servers =
585       new ConcurrentHashMap<String, HRegionInterface>();
586     private final ConcurrentHashMap<String, String> connectionLock =
587       new ConcurrentHashMap<String, String>();
588 
589     /**
590      * Map of table to table {@link HRegionLocation}s.  The table key is made
591      * by doing a {@link Bytes#mapKey(byte[])} of the table's name.
592      */
593     private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>>
594       cachedRegionLocations =
595         new HashMap<Integer, SoftValueSortedMap<byte [], HRegionLocation>>();
596 
597     // The presence of a server in the map implies it's likely that there is an
598     // entry in cachedRegionLocations that map to this server; but the absence
599     // of a server in this map guarentees that there is no entry in cache that
600     // maps to the absent server.
601     private final Set<String> cachedServers =
602         new HashSet<String>();
603 
604     // region cache prefetch is enabled by default. this set contains all
605     // tables whose region cache prefetch are disabled.
606     private final Set<Integer> regionCachePrefetchDisabledTables =
607       new CopyOnWriteArraySet<Integer>();
608 
609     private int refCount;
610 
611     // indicates whether this connection's life cycle is managed
612     private final boolean managed;
613     /**
614      * constructor
615      * @param conf Configuration object
616      */
617     @SuppressWarnings("unchecked")
618     public HConnectionImplementation(Configuration conf, boolean managed, ExecutorService pool)
619     throws ZooKeeperConnectionException {
620       this.conf = conf;
621       this.batchPool = pool;
622       this.managed = managed;
623       String serverClassName = conf.get(HConstants.REGION_SERVER_CLASS,
624         HConstants.DEFAULT_REGION_SERVER_CLASS);
625       this.closed = false;
626       try {
627         this.serverInterfaceClass =
628           (Class<? extends HRegionInterface>) Class.forName(serverClassName);
629       } catch (ClassNotFoundException e) {
630         throw new UnsupportedOperationException(
631             "Unable to find region server interface " + serverClassName, e);
632       }
633       this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
634           HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
635       this.numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
636           HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
637       this.maxRPCAttempts = conf.getInt(
638           HConstants.HBASE_CLIENT_RPC_MAXATTEMPTS,
639           HConstants.DEFAULT_HBASE_CLIENT_RPC_MAXATTEMPTS);
640       this.rpcTimeout = conf.getInt(
641           HConstants.HBASE_RPC_TIMEOUT_KEY,
642           HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
643       this.prefetchRegionLimit = conf.getInt(
644           HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
645           HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT);
646 
647       this.master = null;
648       this.resetting = false;
649     }
650 
651     @Override
652     public HTableInterface getTable(String tableName) throws IOException {
653       return getTable(Bytes.toBytes(tableName));
654     }
655 
656     @Override
657     public HTableInterface getTable(byte[] tableName) throws IOException {
658       return getTable(tableName, getBatchPool());
659     }
660 
661     @Override
662     public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException {
663       return getTable(Bytes.toBytes(tableName), pool);
664     }
665 
666     @Override
667     public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException {
668       if (managed) {
669         throw new IOException("The connection has to be unmanaged.");
670       }
671       return new HTable(tableName, this, pool);
672     }
673 
674     private ExecutorService getBatchPool() {
675       if (batchPool == null) {
676         // shared HTable thread executor not yet initialized
677         synchronized (this) {
678           if (batchPool == null) {
679             int maxThreads = conf.getInt("hbase.hconnection.threads.max", Integer.MAX_VALUE);
680             if (maxThreads == 0) {
681               maxThreads = Runtime.getRuntime().availableProcessors();
682             }
683             long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60);
684             this.batchPool = new ThreadPoolExecutor(Runtime.getRuntime().availableProcessors(),
685                 maxThreads, keepAliveTime, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
686                 Threads.newDaemonThreadFactory("hbase-connection-shared-executor"));
687             ((ThreadPoolExecutor) this.batchPool).allowCoreThreadTimeOut(true);
688           }
689           this.cleanupPool = true;
690         }
691       }
692       return this.batchPool;
693     }
694 
695     protected ExecutorService getCurrentBatchPool() {
696       return batchPool;
697     }
698 
699     private void shutdownBatchPool() {
700       if (this.cleanupPool && this.batchPool != null && !this.batchPool.isShutdown()) {
701         this.batchPool.shutdown();
702         try {
703           if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) {
704             this.batchPool.shutdownNow();
705           }
706         } catch (InterruptedException e) {
707           this.batchPool.shutdownNow();
708         }
709       }
710     }
711 
712     private synchronized void ensureZookeeperTrackers()
713         throws ZooKeeperConnectionException {
714       // initialize zookeeper and master address manager
715       if (zooKeeper == null) {
716         zooKeeper = getZooKeeperWatcher();
717       }
718       if (clusterId == null) {
719         clusterId = new ClusterId(zooKeeper, this);
720         if (clusterId.hasId()) {
721           conf.set(HConstants.CLUSTER_ID, clusterId.getId());
722         }
723       }
724       if (masterAddressTracker == null) {
725         masterAddressTracker = new MasterAddressTracker(zooKeeper, this);
726         masterAddressTracker.start();
727       }
728       if (rootRegionTracker == null) {
729         rootRegionTracker = new RootRegionTracker(zooKeeper, this);
730         rootRegionTracker.start();
731       }
732       // RpcEngine needs access to zookeeper data, like cluster ID
733       if (rpcEngine == null) {
734         this.rpcEngine = HBaseRPC.getProtocolEngine(conf);
735       }
736     }
737 
738     private synchronized void resetZooKeeperTrackers() {
739       if (masterAddressTracker != null) {
740         masterAddressTracker.stop();
741         masterAddressTracker = null;
742       }
743       if (rootRegionTracker != null) {
744         rootRegionTracker.stop();
745         rootRegionTracker = null;
746       }
747       clusterId = null;
748       if (zooKeeper != null) {
749         zooKeeper.close();
750         zooKeeper = null;
751       }
752     }
753 
754     public Configuration getConfiguration() {
755       return this.conf;
756     }
757 
758     /**
759      * Log failure of getMaster attempt
760      * @return true if should retry
761      */
762     private boolean shouldRetryGetMaster(int tries, Exception e) {
763       if (tries == numRetries - 1) {
764         // This was our last chance - don't bother sleeping
765         LOG.info("getMaster attempt " + tries + " of " + numRetries +
766           " failed; no more retrying.", e);
767         return false;
768       }
769       LOG.info("getMaster attempt " + tries + " of " + numRetries +
770         " failed; retrying after sleep of " +
771         ConnectionUtils.getPauseTime(this.pause, tries), e);
772       return true;
773     }
774 
775     public HMasterInterface getMaster()
776     throws MasterNotRunningException, ZooKeeperConnectionException {
777       // TODO: REMOVE.  MOVE TO HBaseAdmin and redo as a Callable!!!
778 
779       // Check if we already have a good master connection
780       try {
781         if (master != null && master.isMasterRunning()) {
782           return master;
783         }
784       } catch (UndeclaredThrowableException ute) {
785         // log, but ignore, the loop below will attempt to reconnect
786         LOG.info("Exception contacting master. Retrying...", ute.getCause());
787       }
788 
789       ensureZookeeperTrackers();
790       checkIfBaseNodeAvailable();
791       ServerName sn = null;
792       synchronized (this.masterLock) {
793         try {
794           if (master != null && master.isMasterRunning()) {
795             return master;
796           }
797         } catch (UndeclaredThrowableException ute) {
798           // log, but ignore, the loop below will attempt to reconnect
799           LOG.info("Exception contacting master. Retrying...", ute.getCause());
800         }
801         this.master = null;
802 
803         for (int tries = 0;
804           !this.closed && this.master == null && tries < numRetries;
805         tries++) {
806 
807           try {
808             sn = masterAddressTracker.getMasterAddress();
809             if (sn == null) {
810               LOG.info("ZooKeeper available but no active master location found");
811               throw new MasterNotRunningException();
812             }
813 
814             InetSocketAddress isa =
815               new InetSocketAddress(sn.getHostname(), sn.getPort());
816             HMasterInterface tryMaster = rpcEngine.getProxy(
817                 HMasterInterface.class, HMasterInterface.VERSION, isa, this.conf,
818                 this.rpcTimeout);
819 
820             if (tryMaster.isMasterRunning()) {
821               this.master = tryMaster;
822               this.masterLock.notifyAll();
823               break;
824             }
825 
826           } catch (IOException e) {
827             if (!shouldRetryGetMaster(tries, e)) break;
828           } catch (UndeclaredThrowableException ute) {
829             if (!shouldRetryGetMaster(tries, ute)) break;
830           }
831 
832           // Cannot connect to master or it is not running. Sleep & retry
833           try {
834             this.masterLock.wait(ConnectionUtils.getPauseTime(this.pause, tries));
835           } catch (InterruptedException e) {
836             Thread.currentThread().interrupt();
837             throw new RuntimeException("Thread was interrupted while trying to connect to master.");
838           }
839         }
840 
841         if (this.master == null) {
842           if (sn == null) {
843             throw new MasterNotRunningException();
844           }
845           throw new MasterNotRunningException(sn.toString());
846         }
847         return this.master;
848       }
849     }
850 
851     private void checkIfBaseNodeAvailable() throws MasterNotRunningException {
852       if (false == masterAddressTracker.checkIfBaseNodeAvailable()) {
853         String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
854             + "There could be a mismatch with the one configured in the master.";
855         LOG.error(errorMsg);
856         throw new MasterNotRunningException(errorMsg);
857       }
858     }
859 
860     public boolean isMasterRunning()
861     throws MasterNotRunningException, ZooKeeperConnectionException {
862       if (this.master == null) {
863         getMaster();
864       }
865       boolean isRunning = master.isMasterRunning();
866       if(isRunning) {
867         return true;
868       }
869       throw new MasterNotRunningException();
870     }
871 
872     public HRegionLocation getRegionLocation(final byte [] name,
873         final byte [] row, boolean reload)
874     throws IOException {
875       return reload? relocateRegion(name, row): locateRegion(name, row);
876     }
877 
878     public boolean isTableEnabled(byte[] tableName) throws IOException {
879       return testTableOnlineState(tableName, true);
880     }
881 
882     public boolean isTableDisabled(byte[] tableName) throws IOException {
883       return testTableOnlineState(tableName, false);
884     }
885 
886     public boolean isTableAvailable(final byte[] tableName) throws IOException {
887       final AtomicBoolean available = new AtomicBoolean(true);
888       final AtomicInteger regionCount = new AtomicInteger(0);
889       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
890         @Override
891         public boolean processRow(Result row) throws IOException {
892           byte[] value = row.getValue(HConstants.CATALOG_FAMILY,
893               HConstants.REGIONINFO_QUALIFIER);
894           HRegionInfo info = Writables.getHRegionInfoOrNull(value);
895           if (info != null) {
896             if (Bytes.equals(tableName, info.getTableName())) {
897               value = row.getValue(HConstants.CATALOG_FAMILY,
898                   HConstants.SERVER_QUALIFIER);
899               if (value == null) {
900                 available.set(false);
901                 return false;
902               }
903               regionCount.incrementAndGet();
904             }
905           }
906           return true;
907         }
908       };
909       MetaScanner.metaScan(conf, this, visitor, null);
910       return available.get() && (regionCount.get() > 0);
911     }
912 
913     /*
914      * @param True if table is online
915      */
916     private boolean testTableOnlineState(byte [] tableName, boolean online)
917     throws IOException {
918       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
919         // The root region is always enabled
920         return online;
921       }
922       ZooKeeperWatcher zkw = getZooKeeperWatcher();
923       String tableNameStr = Bytes.toString(tableName);
924       try {
925         if (online) {
926           return ZKTableReadOnly.isEnabledTable(zkw, tableNameStr);
927         }
928         return ZKTableReadOnly.isDisabledTable(zkw, tableNameStr);
929       } catch (KeeperException e) {
930         throw new IOException("Enable/Disable failed", e);
931       }
932     }
933 
934     @Override
935     public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
936       return locateRegion(HRegionInfo.getTableName(regionName),
937         HRegionInfo.getStartKey(regionName), false, true);
938     }
939 
940     @Override
941     public List<HRegionLocation> locateRegions(final byte[] tableName)
942     throws IOException {
943       return locateRegions(tableName, false, true);
944     }
945 
946     @Override
947     public List<HRegionLocation> locateRegions(final byte[] tableName, final boolean useCache,
948         final boolean offlined) throws IOException {
949       NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, this,
950           tableName, offlined);
951       final List<HRegionLocation> locations = new ArrayList<HRegionLocation>();
952       for (HRegionInfo regionInfo : regions.keySet()) {
953         locations.add(locateRegion(tableName, regionInfo.getStartKey(), useCache, true));
954       }
955       return locations;
956     }
957 
958     public HRegionLocation locateRegion(final byte [] tableName,
959         final byte [] row)
960     throws IOException{
961       return locateRegion(tableName, row, true, true);
962     }
963 
964     public HRegionLocation relocateRegion(final byte [] tableName,
965         final byte [] row)
966     throws IOException{
967 
968       // Since this is an explicit request not to use any caching, finding
969       // disabled tables should not be desirable.  This will ensure that an exception is thrown when
970       // the first time a disabled table is interacted with.
971       if (isTableDisabled(tableName)) {
972         throw new DoNotRetryIOException(Bytes.toString(tableName) + " is disabled.");
973       }
974 
975       return locateRegion(tableName, row, false, true);
976     }
977 
978     private HRegionLocation locateRegion(final byte [] tableName,
979       final byte [] row, boolean useCache, boolean retry)
980     throws IOException {
981       if (this.closed) throw new IOException(toString() + " closed");
982       if (tableName == null || tableName.length == 0) {
983         throw new IllegalArgumentException(
984             "table name cannot be null or zero length");
985       }
986       ensureZookeeperTrackers();
987       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
988         try {
989           ServerName servername = this.rootRegionTracker.waitRootRegionLocation(this.rpcTimeout);
990           LOG.debug("Looked up root region location, connection=" + this +
991             "; serverName=" + ((servername == null)? "": servername.toString()));
992           if (servername == null) return null;
993           return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
994             servername.getHostname(), servername.getPort());
995         } catch (InterruptedException e) {
996           Thread.currentThread().interrupt();
997           return null;
998         }
999       } else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
1000         return locateRegionInMeta(HConstants.ROOT_TABLE_NAME, tableName, row,
1001             useCache, metaRegionLock, retry);
1002       } else {
1003         // Region not in the cache - have to go to the meta RS
1004         return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
1005             useCache, userRegionLock, retry);
1006       }
1007     }
1008 
1009     /*
1010      * Search .META. for the HRegionLocation info that contains the table and
1011      * row we're seeking. It will prefetch certain number of regions info and
1012      * save them to the global region cache.
1013      */
1014     private void prefetchRegionCache(final byte[] tableName,
1015         final byte[] row) {
1016       // Implement a new visitor for MetaScanner, and use it to walk through
1017       // the .META.
1018       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
1019         public boolean processRow(Result result) throws IOException {
1020           try {
1021             byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
1022                 HConstants.REGIONINFO_QUALIFIER);
1023             HRegionInfo regionInfo = null;
1024 
1025             if (value != null) {
1026               // convert the row result into the HRegionLocation we need!
1027               regionInfo = Writables.getHRegionInfo(value);
1028 
1029               // possible we got a region of a different table...
1030               if (!Bytes.equals(regionInfo.getTableName(),
1031                   tableName)) {
1032                 return false; // stop scanning
1033               }
1034               if (regionInfo.isOffline()) {
1035                 // don't cache offline regions
1036                 return true;
1037               }
1038               value = result.getValue(HConstants.CATALOG_FAMILY,
1039                   HConstants.SERVER_QUALIFIER);
1040               if (value == null) {
1041                 return true;  // don't cache it
1042               }
1043               final String hostAndPort = Bytes.toString(value);
1044               String hostname = Addressing.parseHostname(hostAndPort);
1045               int port = Addressing.parsePort(hostAndPort);
1046               value = result.getValue(HConstants.CATALOG_FAMILY,
1047                   HConstants.STARTCODE_QUALIFIER);
1048               // instantiate the location
1049               HRegionLocation loc =
1050                 new HRegionLocation(regionInfo, hostname, port);
1051               // cache this meta entry
1052               cacheLocation(tableName, loc);
1053             }
1054             return true;
1055           } catch (RuntimeException e) {
1056             throw new IOException(e);
1057           }
1058         }
1059       };
1060       try {
1061         // pre-fetch certain number of regions info at region cache.
1062         MetaScanner.metaScan(conf, this, visitor, tableName, row,
1063             this.prefetchRegionLimit, HConstants.META_TABLE_NAME);
1064       } catch (IOException e) {
1065         LOG.warn("Encountered problems when prefetch META table: ", e);
1066       }
1067     }
1068 
1069     /*
1070       * Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation
1071       * info that contains the table and row we're seeking.
1072       */
1073     private HRegionLocation locateRegionInMeta(final byte [] parentTable,
1074       final byte [] tableName, final byte [] row, boolean useCache,
1075       Object regionLockObject, boolean retry)
1076     throws IOException {
1077       HRegionLocation location;
1078       // If we are supposed to be using the cache, look in the cache to see if
1079       // we already have the region.
1080       if (useCache) {
1081         location = getCachedLocation(tableName, row);
1082         if (location != null) {
1083           return location;
1084         }
1085       }
1086 
1087       int localNumRetries = retry ? numRetries : 1;
1088       // build the key of the meta region we should be looking for.
1089       // the extra 9's on the end are necessary to allow "exact" matches
1090       // without knowing the precise region names.
1091       byte [] metaKey = HRegionInfo.createRegionName(tableName, row,
1092         HConstants.NINES, false);
1093       for (int tries = 0; true; tries++) {
1094         if (tries >= localNumRetries) {
1095           throw new NoServerForRegionException("Unable to find region for "
1096             + Bytes.toStringBinary(row) + " after " + numRetries + " tries.");
1097         }
1098 
1099         HRegionLocation metaLocation = null;
1100         try {
1101           // locate the root or meta region
1102           metaLocation = locateRegion(parentTable, metaKey, true, false);
1103           // If null still, go around again.
1104           if (metaLocation == null) continue;
1105           HRegionInterface server =
1106             getHRegionConnection(metaLocation.getHostname(), metaLocation.getPort());
1107 
1108           Result regionInfoRow = null;
1109           // This block guards against two threads trying to load the meta
1110           // region at the same time. The first will load the meta region and
1111           // the second will use the value that the first one found.
1112           synchronized (regionLockObject) {
1113             // Check the cache again for a hit in case some other thread made the
1114             // same query while we were waiting on the lock.
1115             if (useCache) {
1116               location = getCachedLocation(tableName, row);
1117               if (location != null) {
1118                 return location;
1119               }
1120               // If the parent table is META, we may want to pre-fetch some
1121               // region info into the global region cache for this table.
1122               if (Bytes.equals(parentTable, HConstants.META_TABLE_NAME)
1123                   && (getRegionCachePrefetch(tableName))) {
1124                 prefetchRegionCache(tableName, row);
1125               }
1126               location = getCachedLocation(tableName, row);
1127               if (location != null) {
1128                 return location;
1129               }
1130             } else {
1131               // If we are not supposed to be using the cache, delete any existing cached location
1132               // so it won't interfere.
1133               deleteCachedLocation(tableName, row);
1134             }
1135 
1136             // Query the root or meta region for the location of the meta region
1137             regionInfoRow = server.getClosestRowBefore(
1138             metaLocation.getRegionInfo().getRegionName(), metaKey,
1139             HConstants.CATALOG_FAMILY);
1140           }
1141           if (regionInfoRow == null) {
1142             throw new TableNotFoundException(Bytes.toString(tableName));
1143           }
1144           byte [] value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
1145               HConstants.REGIONINFO_QUALIFIER);
1146           if (value == null || value.length == 0) {
1147             throw new IOException("HRegionInfo was null or empty in " +
1148               Bytes.toString(parentTable) + ", row=" + regionInfoRow);
1149           }
1150           // convert the row result into the HRegionLocation we need!
1151           HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
1152               value, new HRegionInfo());
1153           // possible we got a region of a different table...
1154           if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
1155             throw new TableNotFoundException(
1156                   "Table '" + Bytes.toString(tableName) + "' was not found, got: " +
1157                   Bytes.toString(regionInfo.getTableName()) + ".");
1158           }
1159           if (regionInfo.isSplit()) {
1160             throw new RegionOfflineException("the only available region for" +
1161               " the required row is a split parent," +
1162               " the daughters should be online soon: " +
1163               regionInfo.getRegionNameAsString());
1164           }
1165           if (regionInfo.isOffline()) {
1166             throw new RegionOfflineException("the region is offline, could" +
1167               " be caused by a disable table call: " +
1168               regionInfo.getRegionNameAsString());
1169           }
1170 
1171           value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
1172               HConstants.SERVER_QUALIFIER);
1173           String hostAndPort = "";
1174           if (value != null) {
1175             hostAndPort = Bytes.toString(value);
1176           }
1177           if (hostAndPort.equals("")) {
1178             throw new NoServerForRegionException("No server address listed " +
1179               "in " + Bytes.toString(parentTable) + " for region " +
1180               regionInfo.getRegionNameAsString() + " containing row " +
1181               Bytes.toStringBinary(row));
1182           }
1183 
1184           // Instantiate the location
1185           String hostname = Addressing.parseHostname(hostAndPort);
1186           int port = Addressing.parsePort(hostAndPort);
1187           location = new HRegionLocation(regionInfo, hostname, port);
1188           cacheLocation(tableName, location);
1189           return location;
1190         } catch (TableNotFoundException e) {
1191           // if we got this error, probably means the table just plain doesn't
1192           // exist. rethrow the error immediately. this should always be coming
1193           // from the HTable constructor.
1194           throw e;
1195         } catch (IOException e) {
1196           if (e instanceof RemoteException) {
1197             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
1198           }
1199           if (tries < numRetries - 1) {
1200             if (LOG.isDebugEnabled()) {
1201               LOG.debug("locateRegionInMeta parentTable=" +
1202                 Bytes.toString(parentTable) + ", metaLocation=" +
1203                 ((metaLocation == null)? "null": "{" + metaLocation + "}") +
1204                 ", attempt=" + tries + " of " +
1205                 this.numRetries + " failed; retrying after sleep of " +
1206                 ConnectionUtils.getPauseTime(this.pause, tries) + " because: " + e.getMessage());
1207             }
1208           } else {
1209             throw e;
1210           }
1211           // Only relocate the parent region if necessary
1212           if(!(e instanceof RegionOfflineException ||
1213               e instanceof NoServerForRegionException)) {
1214             relocateRegion(parentTable, metaKey);
1215           }
1216         }
1217         try{
1218           Thread.sleep(ConnectionUtils.getPauseTime(this.pause, tries));
1219         } catch (InterruptedException e) {
1220           Thread.currentThread().interrupt();
1221           throw new IOException("Giving up trying to location region in " +
1222             "meta: thread is interrupted.");
1223         }
1224       }
1225     }
1226 
1227     /*
1228      * Search the cache for a location that fits our table and row key.
1229      * Return null if no suitable region is located. TODO: synchronization note
1230      *
1231      * <p>TODO: This method during writing consumes 15% of CPU doing lookup
1232      * into the Soft Reference SortedMap.  Improve.
1233      *
1234      * @param tableName
1235      * @param row
1236      * @return Null or region location found in cache.
1237      */
1238     HRegionLocation getCachedLocation(final byte [] tableName,
1239         final byte [] row) {
1240       SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
1241         getTableLocations(tableName);
1242 
1243       // start to examine the cache. we can only do cache actions
1244       // if there's something in the cache for this table.
1245       if (tableLocations.isEmpty()) {
1246         return null;
1247       }
1248 
1249       HRegionLocation possibleRegion = tableLocations.get(row);
1250       if (possibleRegion != null) {
1251         return possibleRegion;
1252       }
1253 
1254       possibleRegion = tableLocations.lowerValueByKey(row);
1255       if (possibleRegion == null) {
1256         return null;
1257       }
1258 
1259       // make sure that the end key is greater than the row we're looking
1260       // for, otherwise the row actually belongs in the next region, not
1261       // this one. the exception case is when the endkey is
1262       // HConstants.EMPTY_END_ROW, signifying that the region we're
1263       // checking is actually the last region in the table.
1264       byte[] endKey = possibleRegion.getRegionInfo().getEndKey();
1265       if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
1266           KeyValue.getRowComparator(tableName).compareRows(
1267               endKey, 0, endKey.length, row, 0, row.length) > 0) {
1268         return possibleRegion;
1269       }
1270 
1271       // Passed all the way through, so we got nothin - complete cache miss
1272       return null;
1273     }
1274 
1275     /**
1276      * Delete a cached location
1277      * @param tableName tableName
1278      * @param row
1279      */
1280     void deleteCachedLocation(final byte [] tableName, final byte [] row) {
1281       synchronized (this.cachedRegionLocations) {
1282         Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
1283         if (!tableLocations.isEmpty()) {
1284           // start to examine the cache. we can only do cache actions
1285           // if there's something in the cache for this table.
1286           HRegionLocation rl = getCachedLocation(tableName, row);
1287           if (rl != null) {
1288             tableLocations.remove(rl.getRegionInfo().getStartKey());
1289             if (LOG.isDebugEnabled()) {
1290               LOG.debug("Removed " +
1291                 rl.getRegionInfo().getRegionNameAsString() +
1292                 " for tableName=" + Bytes.toString(tableName) +
1293                 " from cache " + "because of " + Bytes.toStringBinary(row));
1294             }
1295           }
1296         }
1297       }
1298     }
1299 
1300     @Override
1301     public void deleteCachedRegionLocation(final HRegionLocation location) {
1302       if (location == null) {
1303         return;
1304       }
1305       synchronized (this.cachedRegionLocations) {
1306         byte[] tableName = location.getRegionInfo().getTableName();
1307         Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
1308         if (!tableLocations.isEmpty()) {
1309           // Delete if there's something in the cache for this region.
1310           HRegionLocation removedLocation =
1311               tableLocations.remove(location.getRegionInfo().getStartKey());
1312           if (LOG.isDebugEnabled() && removedLocation != null) {
1313             LOG.debug("Removed " +
1314               location.getRegionInfo().getRegionNameAsString() +
1315               " for tableName=" + Bytes.toString(tableName) +
1316               " from cache");
1317           }
1318         }
1319       }
1320     }
1321 
1322     @Override
1323     public void clearCaches(String sn) {
1324       clearCachedLocationForServer(sn);
1325     }
1326 
1327     /*
1328      * Delete all cached entries of a table that maps to a specific location.
1329      *
1330      * @param tablename
1331      * @param server
1332      */
1333     private void clearCachedLocationForServer(final String server) {
1334       boolean deletedSomething = false;
1335       synchronized (this.cachedRegionLocations) {
1336         if (!cachedServers.contains(server)) {
1337           return;
1338         }
1339         for (Map<byte[], HRegionLocation> tableLocations :
1340           cachedRegionLocations.values()) {
1341           for (Entry<byte[], HRegionLocation> e : tableLocations.entrySet()) {
1342             HRegionLocation value = e.getValue();
1343             if (value != null
1344                 && value.getHostnamePort().equals(server)) {
1345               tableLocations.remove(e.getKey());
1346               deletedSomething = true;
1347             }
1348           }
1349         }
1350         cachedServers.remove(server);
1351       }
1352       if (deletedSomething && LOG.isDebugEnabled()) {
1353         LOG.debug("Removed all cached region locations that map to " + server);
1354       }
1355     }
1356 
1357     /*
1358      * @param tableName
1359      * @return Map of cached locations for passed <code>tableName</code>
1360      */
1361     private SoftValueSortedMap<byte [], HRegionLocation> getTableLocations(
1362         final byte [] tableName) {
1363       // find the map of cached locations for this table
1364       Integer key = Bytes.mapKey(tableName);
1365       SoftValueSortedMap<byte [], HRegionLocation> result;
1366       synchronized (this.cachedRegionLocations) {
1367         result = this.cachedRegionLocations.get(key);
1368         // if tableLocations for this table isn't built yet, make one
1369         if (result == null) {
1370           result = new SoftValueSortedMap<byte [], HRegionLocation>(
1371               Bytes.BYTES_COMPARATOR);
1372           this.cachedRegionLocations.put(key, result);
1373         }
1374       }
1375       return result;
1376     }
1377 
1378     @Override
1379     public void clearRegionCache() {
1380       synchronized(this.cachedRegionLocations) {
1381         this.cachedRegionLocations.clear();
1382         this.cachedServers.clear();
1383       }
1384     }
1385 
1386     @Override
1387     public void clearRegionCache(final byte [] tableName) {
1388       synchronized (this.cachedRegionLocations) {
1389         this.cachedRegionLocations.remove(Bytes.mapKey(tableName));
1390       }
1391     }
1392 
1393     /*
1394      * Put a newly discovered HRegionLocation into the cache.
1395      */
1396     private void cacheLocation(final byte [] tableName,
1397         final HRegionLocation location) {
1398       byte [] startKey = location.getRegionInfo().getStartKey();
1399       Map<byte [], HRegionLocation> tableLocations =
1400         getTableLocations(tableName);
1401       boolean hasNewCache = false;
1402       synchronized (this.cachedRegionLocations) {
1403         cachedServers.add(location.getHostnamePort());
1404         hasNewCache = (tableLocations.put(startKey, location) == null);
1405       }
1406       if (hasNewCache) {
1407         LOG.debug("Cached location for " +
1408             location.getRegionInfo().getRegionNameAsString() +
1409             " is " + location.getHostnamePort());
1410       }
1411     }
1412 
1413     public HRegionInterface getHRegionConnection(HServerAddress hsa)
1414     throws IOException {
1415       return getHRegionConnection(hsa, false);
1416     }
1417 
1418     @Override
1419     public HRegionInterface getHRegionConnection(final String hostname,
1420         final int port)
1421     throws IOException {
1422       return getHRegionConnection(hostname, port, false);
1423     }
1424 
1425     public HRegionInterface getHRegionConnection(HServerAddress hsa,
1426         boolean master)
1427     throws IOException {
1428       return getHRegionConnection(null, -1, hsa.getInetSocketAddress(), master);
1429     }
1430 
1431     @Override
1432     public HRegionInterface getHRegionConnection(final String hostname,
1433         final int port, final boolean master)
1434     throws IOException {
1435       return getHRegionConnection(hostname, port, null, master);
1436     }
1437 
1438     /**
1439      * Either the passed <code>isa</code> is null or <code>hostname</code>
1440      * can be but not both.
1441      * @param hostname
1442      * @param port
1443      * @param isa
1444      * @param master
1445      * @return Proxy.
1446      * @throws IOException
1447      */
1448     HRegionInterface getHRegionConnection(final String hostname, final int port,
1449         final InetSocketAddress isa, final boolean master)
1450     throws IOException {
1451       if (master) getMaster();
1452       HRegionInterface server;
1453       String rsName = null;
1454       if (isa != null) {
1455         rsName = Addressing.createHostAndPortStr(isa.getHostName(),
1456             isa.getPort());
1457       } else {
1458         rsName = Addressing.createHostAndPortStr(hostname, port);
1459       }
1460       ensureZookeeperTrackers();
1461       // See if we already have a connection (common case)
1462       server = this.servers.get(rsName);
1463       if (server == null) {
1464         // create a unique lock for this RS (if necessary)
1465         this.connectionLock.putIfAbsent(rsName, rsName);
1466         // get the RS lock
1467         synchronized (this.connectionLock.get(rsName)) {
1468           // do one more lookup in case we were stalled above
1469           server = this.servers.get(rsName);
1470           if (server == null) {
1471             try {
1472               // Only create isa when we need to.
1473               InetSocketAddress address = isa != null? isa:
1474                 new InetSocketAddress(hostname, port);
1475               // definitely a cache miss. establish an RPC for this RS
1476               server = HBaseRPC.waitForProxy(this.rpcEngine,
1477                   serverInterfaceClass, HRegionInterface.VERSION,
1478                   address, this.conf,
1479                   this.maxRPCAttempts, this.rpcTimeout, this.rpcTimeout);
1480               this.servers.put(Addressing.createHostAndPortStr(
1481                   address.getHostName(), address.getPort()), server);
1482             } catch (RemoteException e) {
1483               LOG.warn("RemoteException connecting to RS", e);
1484               // Throw what the RemoteException was carrying.
1485               throw e.unwrapRemoteException();
1486             }
1487           }
1488         }
1489       }
1490       return server;
1491     }
1492 
1493     /**
1494      * Get the ZooKeeper instance for this TableServers instance.
1495      *
1496      * If ZK has not been initialized yet, this will connect to ZK.
1497      * @returns zookeeper reference
1498      * @throws ZooKeeperConnectionException if there's a problem connecting to zk
1499      */
1500     public synchronized ZooKeeperWatcher getZooKeeperWatcher()
1501         throws ZooKeeperConnectionException {
1502       if(zooKeeper == null) {
1503         try {
1504           if (this.closed) {
1505             throw new IOException(toString() + " closed");
1506           }
1507           this.zooKeeper = new ZooKeeperWatcher(conf, "hconnection", this);
1508         } catch(ZooKeeperConnectionException zce) {
1509           throw zce;
1510         } catch (IOException e) {
1511           throw new ZooKeeperConnectionException("An error is preventing" +
1512               " HBase from connecting to ZooKeeper", e);
1513         }
1514       }
1515       return zooKeeper;
1516     }
1517 
1518     public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
1519     throws IOException, RuntimeException {
1520       return callable.withRetries();
1521     }
1522 
1523     public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
1524     throws IOException, RuntimeException {
1525       return callable.withoutRetries();
1526     }
1527 
1528     private <R> Callable<MultiResponse> createCallable(final HRegionLocation loc,
1529         final MultiAction<R> multi, final byte [] tableName) {
1530       // TODO: This does not belong in here!!! St.Ack  HConnections should
1531       // not be dealing in Callables; Callables have HConnections, not other
1532       // way around.
1533       final HConnection connection = this;
1534       return new Callable<MultiResponse>() {
1535        public MultiResponse call() throws IOException {
1536          ServerCallable<MultiResponse> callable =
1537            new ServerCallable<MultiResponse>(connection, tableName, null) {
1538              public MultiResponse call() throws IOException {
1539                return server.multi(multi);
1540              }
1541              @Override
1542              public void connect(boolean reload) throws IOException {
1543                server = connection.getHRegionConnection(loc.getHostname(), loc.getPort());
1544              }
1545            };
1546          return callable.withoutRetries();
1547        }
1548      };
1549    }
1550 
1551     public void processBatch(List<? extends Row> list,
1552         final byte[] tableName,
1553         ExecutorService pool,
1554         Object[] results) throws IOException, InterruptedException {
1555       // This belongs in HTable!!! Not in here.  St.Ack
1556 
1557       // results must be the same size as list
1558       if (results.length != list.size()) {
1559         throw new IllegalArgumentException("argument results must be the same size as argument list");
1560       }
1561 
1562       processBatchCallback(list, tableName, pool, results, null);
1563     }
1564 
1565     /**
1566      * Executes the given
1567      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call}
1568      * callable for each row in the
1569      * given list and invokes
1570      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)}
1571      * for each result returned.
1572      *
1573      * @param protocol the protocol interface being called
1574      * @param rows a list of row keys for which the callable should be invoked
1575      * @param tableName table name for the coprocessor invoked
1576      * @param pool ExecutorService used to submit the calls per row
1577      * @param callable instance on which to invoke
1578      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)}
1579      * for each row
1580      * @param callback instance on which to invoke
1581      * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)}
1582      * for each result
1583      * @param <T> the protocol interface type
1584      * @param <R> the callable's return type
1585      * @throws IOException
1586      */
1587     public <T extends CoprocessorProtocol,R> void processExecs(
1588         final Class<T> protocol,
1589         List<byte[]> rows,
1590         final byte[] tableName,
1591         ExecutorService pool,
1592         final Batch.Call<T,R> callable,
1593         final Batch.Callback<R> callback)
1594       throws IOException, Throwable {
1595 
1596       Map<byte[],Future<R>> futures =
1597           new TreeMap<byte[],Future<R>>(Bytes.BYTES_COMPARATOR);
1598       for (final byte[] r : rows) {
1599         final ExecRPCInvoker invoker =
1600             new ExecRPCInvoker(conf, this, protocol, tableName, r);
1601         Future<R> future = pool.submit(
1602             new Callable<R>() {
1603               public R call() throws Exception {
1604                 T instance = (T)Proxy.newProxyInstance(conf.getClassLoader(),
1605                     new Class[]{protocol},
1606                     invoker);
1607                 R result = callable.call(instance);
1608                 byte[] region = invoker.getRegionName();
1609                 if (callback != null) {
1610                   callback.update(region, r, result);
1611                 }
1612                 return result;
1613               }
1614             });
1615         futures.put(r, future);
1616       }
1617       for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
1618         try {
1619           e.getValue().get();
1620         } catch (ExecutionException ee) {
1621           LOG.warn("Error executing for row "+Bytes.toStringBinary(e.getKey()), ee);
1622           throw ee.getCause();
1623         } catch (InterruptedException ie) {
1624           Thread.currentThread().interrupt();
1625           throw new IOException("Interrupted executing for row " +
1626               Bytes.toStringBinary(e.getKey()), ie);
1627         }
1628       }
1629     }
1630 
1631     /**
1632      * Parameterized batch processing, allowing varying return types for
1633      * different {@link Row} implementations.
1634      */
1635     public <R> void processBatchCallback(
1636         List<? extends Row> list,
1637         byte[] tableName,
1638         ExecutorService pool,
1639         Object[] results,
1640         Batch.Callback<R> callback)
1641     throws IOException, InterruptedException {
1642       // This belongs in HTable!!! Not in here.  St.Ack
1643 
1644       // results must be the same size as list
1645       if (results.length != list.size()) {
1646         throw new IllegalArgumentException(
1647             "argument results must be the same size as argument list");
1648       }
1649       if (list.isEmpty()) {
1650         return;
1651       }
1652 
1653       // Keep track of the most recent servers for any given item for better
1654       // exceptional reporting.  We keep HRegionLocation to save on parsing.
1655       // Later below when we use lastServers, we'll pull what we need from
1656       // lastServers.
1657       HRegionLocation [] lastServers = new HRegionLocation[results.length];
1658       List<Row> workingList = new ArrayList<Row>(list);
1659       boolean retry = true;
1660       // count that helps presize actions array
1661       int actionCount = 0;
1662 
1663       for (int tries = 0; tries < numRetries && retry; ++tries) {
1664 
1665         // sleep first, if this is a retry
1666         if (tries >= 1) {
1667           long sleepTime = ConnectionUtils.getPauseTime(this.pause, tries);
1668           LOG.debug("Retry " +tries+ ", sleep for " +sleepTime+ "ms!");
1669           Thread.sleep(sleepTime);
1670         }
1671         // step 1: break up into regionserver-sized chunks and build the data structs
1672         Map<HRegionLocation, MultiAction<R>> actionsByServer =
1673           new HashMap<HRegionLocation, MultiAction<R>>();
1674         for (int i = 0; i < workingList.size(); i++) {
1675           Row row = workingList.get(i);
1676           if (row != null) {
1677             HRegionLocation loc = locateRegion(tableName, row.getRow());
1678             byte[] regionName = loc.getRegionInfo().getRegionName();
1679 
1680             MultiAction<R> actions = actionsByServer.get(loc);
1681             if (actions == null) {
1682               actions = new MultiAction<R>();
1683               actionsByServer.put(loc, actions);
1684             }
1685 
1686             Action<R> action = new Action<R>(row, i);
1687             lastServers[i] = loc;
1688             actions.add(regionName, action);
1689           }
1690         }
1691 
1692         // step 2: make the requests
1693 
1694         Map<HRegionLocation, Future<MultiResponse>> futures =
1695             new HashMap<HRegionLocation, Future<MultiResponse>>(
1696                 actionsByServer.size());
1697 
1698         for (Entry<HRegionLocation, MultiAction<R>> e: actionsByServer.entrySet()) {
1699           futures.put(e.getKey(), pool.submit(createCallable(e.getKey(), e.getValue(), tableName)));
1700         }
1701 
1702         // step 3: collect the failures and successes and prepare for retry
1703 
1704         for (Entry<HRegionLocation, Future<MultiResponse>> responsePerServer
1705              : futures.entrySet()) {
1706           HRegionLocation loc = responsePerServer.getKey();
1707 
1708           try {
1709             Future<MultiResponse> future = responsePerServer.getValue();
1710             MultiResponse resp = future.get();
1711 
1712             if (resp == null) {
1713               // Entire server failed
1714               LOG.debug("Failed all for server: " + loc.getHostnamePort() +
1715                 ", removing from cache");
1716               continue;
1717             }
1718 
1719             for (Entry<byte[], List<Pair<Integer,Object>>> e : resp.getResults().entrySet()) {
1720               byte[] regionName = e.getKey();
1721               List<Pair<Integer, Object>> regionResults = e.getValue();
1722               for (Pair<Integer, Object> regionResult : regionResults) {
1723                 if (regionResult == null) {
1724                   // if the first/only record is 'null' the entire region failed.
1725                   LOG.debug("Failures for region: " +
1726                       Bytes.toStringBinary(regionName) +
1727                       ", removing from cache");
1728                 } else {
1729                   // Result might be an Exception, including DNRIOE
1730                   results[regionResult.getFirst()] = regionResult.getSecond();
1731                   if (callback != null && !(regionResult.getSecond() instanceof Throwable)) {
1732                     callback.update(e.getKey(),
1733                         list.get(regionResult.getFirst()).getRow(),
1734                         (R)regionResult.getSecond());
1735                   }
1736                 }
1737               }
1738             }
1739           } catch (ExecutionException e) {
1740             LOG.warn("Failed all from " + loc, e);
1741           }
1742         }
1743 
1744         // step 4: identify failures and prep for a retry (if applicable).
1745 
1746         // Find failures (i.e. null Result), and add them to the workingList (in
1747         // order), so they can be retried.
1748         retry = false;
1749         workingList.clear();
1750         actionCount = 0;
1751         for (int i = 0; i < results.length; i++) {
1752           // if null (fail) or instanceof Throwable && not instanceof DNRIOE
1753           // then retry that row. else dont.
1754           if (results[i] == null ||
1755               (results[i] instanceof Throwable &&
1756                   !(results[i] instanceof DoNotRetryIOException))) {
1757 
1758             retry = true;
1759             actionCount++;
1760             Row row = list.get(i);
1761             workingList.add(row);
1762             deleteCachedLocation(tableName, row.getRow());
1763           } else {
1764             if (results[i] != null && results[i] instanceof Throwable) {
1765               actionCount++;
1766             }
1767             // add null to workingList, so the order remains consistent with the original list argument.
1768             workingList.add(null);
1769           }
1770         }
1771       }
1772 
1773       List<Throwable> exceptions = new ArrayList<Throwable>(actionCount);
1774       List<Row> actions = new ArrayList<Row>(actionCount);
1775       List<String> addresses = new ArrayList<String>(actionCount);
1776 
1777       for (int i = 0 ; i < results.length; i++) {
1778         if (results[i] == null || results[i] instanceof Throwable) {
1779           exceptions.add((Throwable)results[i]);
1780           actions.add(list.get(i));
1781           addresses.add(lastServers[i].getHostnamePort());
1782         }
1783       }
1784 
1785       if (!exceptions.isEmpty()) {
1786         throw new RetriesExhaustedWithDetailsException(exceptions,
1787             actions,
1788             addresses);
1789       }
1790     }
1791 
1792     /*
1793      * Return the number of cached region for a table. It will only be called
1794      * from a unit test.
1795      */
1796     int getNumberOfCachedRegionLocations(final byte[] tableName) {
1797       Integer key = Bytes.mapKey(tableName);
1798       synchronized (this.cachedRegionLocations) {
1799         Map<byte[], HRegionLocation> tableLocs =
1800           this.cachedRegionLocations.get(key);
1801 
1802         if (tableLocs == null) {
1803           return 0;
1804         }
1805         return tableLocs.values().size();
1806       }
1807     }
1808 
1809     /**
1810      * Check the region cache to see whether a region is cached yet or not.
1811      * Called by unit tests.
1812      * @param tableName tableName
1813      * @param row row
1814      * @return Region cached or not.
1815      */
1816     boolean isRegionCached(final byte[] tableName, final byte[] row) {
1817       HRegionLocation location = getCachedLocation(tableName, row);
1818       return location != null;
1819     }
1820 
1821     public void setRegionCachePrefetch(final byte[] tableName,
1822         final boolean enable) {
1823       if (!enable) {
1824         regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName));
1825       }
1826       else {
1827         regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName));
1828       }
1829     }
1830 
1831     public boolean getRegionCachePrefetch(final byte[] tableName) {
1832       return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName));
1833     }
1834 
1835     @Override
1836     public void prewarmRegionCache(byte[] tableName,
1837         Map<HRegionInfo, HServerAddress> regions) {
1838       for (Map.Entry<HRegionInfo, HServerAddress> e : regions.entrySet()) {
1839         HServerAddress hsa = e.getValue();
1840         if (hsa == null || hsa.getInetSocketAddress() == null) continue;
1841         cacheLocation(tableName,
1842           new HRegionLocation(e.getKey(), hsa.getHostname(), hsa.getPort()));
1843       }
1844     }
1845 
1846     @Override
1847     public void abort(final String msg, Throwable t) {
1848       if (t instanceof KeeperException) {
1849         LOG.info("This client just lost it's session with ZooKeeper, will"
1850             + " automatically reconnect when needed.");
1851         if (t instanceof KeeperException.SessionExpiredException) {
1852           LOG.info("ZK session expired. This disconnect could have been" +
1853               " caused by a network partition or a long-running GC pause," +
1854               " either way it's recommended that you verify your environment.");
1855           synchronized (resetLock) {
1856             if (resetting) return;
1857             this.resetting = true;
1858           }
1859           resetZooKeeperTrackers();
1860           this.resetting = false;
1861         }
1862         return;
1863       }
1864       if (t != null) LOG.fatal(msg, t);
1865       else LOG.fatal(msg);
1866       this.aborted = true;
1867       close();
1868     }
1869 
1870     @Override
1871     public boolean isClosed() {
1872       return this.closed;
1873     }
1874 
1875     @Override
1876     public boolean isAborted(){
1877       return this.aborted;
1878     }
1879 
1880     public int getCurrentNrHRS() throws IOException {
1881       try {
1882         ZooKeeperWatcher zkw = getZooKeeperWatcher();
1883         // We go to zk rather than to master to get count of regions to avoid
1884         // HTable having a Master dependency.  See HBase-2828
1885         return ZKUtil.getNumberOfChildren(zkw,
1886             zkw.rsZNode);
1887       } catch (KeeperException ke) {
1888         throw new IOException("Unexpected ZooKeeper exception", ke);
1889       }
1890     }
1891 
1892     /**
1893      * Increment this client's reference count.
1894      */
1895     void incCount() {
1896       ++refCount;
1897     }
1898 
1899     /**
1900      * Decrement this client's reference count.
1901      */
1902     void decCount() {
1903       if (refCount > 0) {
1904         --refCount;
1905       }
1906     }
1907 
1908     /**
1909      * Return if this client has no reference
1910      *
1911      * @return true if this client has no reference; false otherwise
1912      */
1913     boolean isZeroReference() {
1914       return refCount == 0;
1915     }
1916 
1917     void internalClose() {
1918       if (this.closed) {
1919         return;
1920       }
1921       shutdownBatchPool();
1922       master = null;
1923 
1924       this.servers.clear();
1925       if (this.rpcEngine != null) {
1926         this.rpcEngine.close();
1927       }
1928 
1929       synchronized (this) {
1930         if (this.zooKeeper != null) {
1931           LOG.info("Closed zookeeper sessionid=0x" +
1932             Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1933           this.zooKeeper.close();
1934           this.zooKeeper = null;
1935         }
1936         this.closed = true;
1937       }
1938     }
1939 
1940     public void close() {
1941       if (managed) {
1942         if (aborted) {
1943           HConnectionManager.deleteStaleConnection(this);
1944         } else {
1945           HConnectionManager.deleteConnection(this, false);
1946         }
1947       } else {
1948         internalClose();
1949       }
1950       if (LOG.isTraceEnabled()) LOG.debug("" + this.zooKeeper + " closed.");
1951     }
1952 
1953     /**
1954      * Close the connection for good, regardless of what the current value of
1955      * {@link #refCount} is. Ideally, {@link #refCount} should be zero at this
1956      * point, which would be the case if all of its consumers close the
1957      * connection. However, on the off chance that someone is unable to close
1958      * the connection, perhaps because it bailed out prematurely, the method
1959      * below will ensure that this {@link HConnection} instance is cleaned up.
1960      * Caveat: The JVM may take an unknown amount of time to call finalize on an
1961      * unreachable object, so our hope is that every consumer cleans up after
1962      * itself, like any good citizen.
1963      */
1964     @Override
1965     protected void finalize() throws Throwable {
1966       // Pretend as if we are about to release the last remaining reference
1967       refCount = 1;
1968       close();
1969       LOG.debug("The connection to " + this.zooKeeper
1970           + " was closed by the finalize method.");
1971     }
1972 
1973     public HTableDescriptor[] listTables() throws IOException {
1974       if (this.master == null) {
1975         this.master = getMaster();
1976       }
1977       HTableDescriptor[] htd = master.getHTableDescriptors();
1978       return htd;
1979     }
1980 
1981     public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) throws IOException {
1982       if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
1983       if (tableNames == null || tableNames.size() == 0) return null;
1984       if (this.master == null) {
1985         this.master = getMaster();
1986       }
1987       return master.getHTableDescriptors(tableNames);
1988     }
1989 
1990     public HTableDescriptor getHTableDescriptor(final byte[] tableName)
1991     throws IOException {
1992       if (tableName == null || tableName.length == 0) return null;
1993       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
1994         return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
1995       }
1996       if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
1997         return HTableDescriptor.META_TABLEDESC;
1998       }
1999       List<String> tableNameList = new ArrayList<String>(1);
2000       tableNameList.add(Bytes.toString(tableName));
2001       if (this.master == null) {
2002         this.master = getMaster();
2003       }
2004       HTableDescriptor[] htds = master.getHTableDescriptors(tableNameList);
2005       if (htds != null && htds.length > 0) {
2006         return htds[0];
2007       }
2008       throw new TableNotFoundException(Bytes.toString(tableName));
2009     }
2010   }
2011 
2012   /**
2013    * Set the number of retries to use serverside when trying to communicate
2014    * with another server over {@link HConnection}.  Used updating catalog
2015    * tables, etc.  Call this method before we create any Connections.
2016    * @param c The Configuration instance to set the retries into.
2017    * @param log Used to log what we set in here.
2018    */
2019   public static void setServerSideHConnectionRetries(final Configuration c,
2020       final Log log) {
2021     int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
2022       HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
2023     // Go big.  Multiply by 10.  If we can't get to meta after this many retries
2024     // then something seriously wrong.
2025     int serversideMultiplier =
2026       c.getInt("hbase.client.serverside.retries.multiplier", 10);
2027     int retries = hcRetries * serversideMultiplier;
2028     c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
2029     log.debug("Set serverside HConnection retries=" + retries);
2030   }
2031 }