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