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