View Javadoc

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