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 org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.hbase.DoNotRetryIOException;
26  import org.apache.hadoop.hbase.HBaseConfiguration;
27  import org.apache.hadoop.hbase.HConstants;
28  import org.apache.hadoop.hbase.HRegionInfo;
29  import org.apache.hadoop.hbase.HRegionLocation;
30  import org.apache.hadoop.hbase.HServerAddress;
31  import org.apache.hadoop.hbase.HTableDescriptor;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.MasterNotRunningException;
34  import org.apache.hadoop.hbase.NotServingRegionException;
35  import org.apache.hadoop.hbase.RemoteExceptionHandler;
36  import org.apache.hadoop.hbase.TableNotFoundException;
37  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
38  import org.apache.hadoop.hbase.ipc.HBaseRPC;
39  import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
40  import org.apache.hadoop.hbase.ipc.HMasterInterface;
41  import org.apache.hadoop.hbase.ipc.HRegionInterface;
42  import org.apache.hadoop.hbase.util.Bytes;
43  import org.apache.hadoop.hbase.util.MetaUtils;
44  import org.apache.hadoop.hbase.util.SoftValueSortedMap;
45  import org.apache.hadoop.hbase.util.Writables;
46  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
47  import org.apache.hadoop.ipc.RemoteException;
48  import org.apache.zookeeper.WatchedEvent;
49  import org.apache.zookeeper.Watcher;
50  import org.apache.zookeeper.Watcher.Event.KeeperState;
51  
52  import java.io.IOException;
53  import java.lang.reflect.UndeclaredThrowableException;
54  import java.util.ArrayList;
55  import java.util.Collections;
56  import java.util.HashMap;
57  import java.util.LinkedHashMap;
58  import java.util.List;
59  import java.util.Map;
60  import java.util.Set;
61  import java.util.TreeSet;
62  import java.util.concurrent.Callable;
63  import java.util.concurrent.ConcurrentHashMap;
64  import java.util.concurrent.ExecutionException;
65  import java.util.concurrent.ExecutorService;
66  import java.util.concurrent.Future;
67  import java.util.concurrent.atomic.AtomicBoolean;
68  import java.util.concurrent.CopyOnWriteArraySet;
69  
70  /**
71   * A non-instantiable class that manages connections to multiple tables in
72   * multiple HBase instances.
73   *
74   * Used by {@link HTable} and {@link HBaseAdmin}
75   */
76  @SuppressWarnings("serial")
77  public class HConnectionManager {
78    // Register a shutdown hook, one that cleans up RPC and closes zk sessions.
79    static {
80      Runtime.getRuntime().addShutdownHook(new Thread("HCM.shutdownHook") {
81        @Override
82        public void run() {
83          HConnectionManager.deleteAllConnections(true);
84        }
85      });
86    }
87  
88    /*
89     * Not instantiable.
90     */
91    protected HConnectionManager() {
92      super();
93    }
94  
95    private static final int MAX_CACHED_HBASE_INSTANCES=31;
96    // A LRU Map of master HBaseConfiguration -> connection information for that
97    // instance. The objects it contains are mutable and hence require
98    // synchronized access to them.  We set instances to 31.  The zk default max
99    // connections is 30 so should run into zk issues before hit this value of 31.
100   private static
101   final Map<Integer, TableServers> HBASE_INSTANCES =
102     new LinkedHashMap<Integer, TableServers>
103       ((int) (MAX_CACHED_HBASE_INSTANCES/0.75F)+1, 0.75F, true) {
104       @Override
105       protected boolean removeEldestEntry(Map.Entry<Integer, TableServers> eldest) {
106         return size() > MAX_CACHED_HBASE_INSTANCES;
107       }
108   };
109 
110   private static final Map<String, ClientZKWatcher> ZK_WRAPPERS =
111     new HashMap<String, ClientZKWatcher>();
112 
113   /**
114    * Get the connection object for the instance specified by the configuration
115    * If no current connection exists, create a new connection for that instance
116    * @param conf configuration
117    * @return HConnection object for the instance specified by the configuration
118    */
119   public static HConnection getConnection(Configuration conf) {
120     TableServers connection;
121     Integer key = HBaseConfiguration.hashCode(conf);
122     synchronized (HBASE_INSTANCES) {
123       connection = HBASE_INSTANCES.get(key);
124       if (connection == null) {
125         connection = new TableServers(conf);
126         HBASE_INSTANCES.put(key, connection);
127       }
128     }
129     return connection;
130   }
131 
132   /**
133    * Delete connection information for the instance specified by configuration
134    * @param conf configuration
135    * @param stopProxy stop the proxy as well
136    */
137   public static void deleteConnectionInfo(Configuration conf,
138       boolean stopProxy) {
139     synchronized (HBASE_INSTANCES) {
140       Integer key = HBaseConfiguration.hashCode(conf);
141       TableServers t = HBASE_INSTANCES.remove(key);
142       if (t != null) {
143         t.close(stopProxy);
144       }
145     }
146   }
147 
148   /**
149    * Delete information for all connections.
150    * @param stopProxy stop the proxy as well
151    */
152   public static void deleteAllConnections(boolean stopProxy) {
153     synchronized (HBASE_INSTANCES) {
154       for (TableServers t : HBASE_INSTANCES.values()) {
155         if (t != null) {
156           t.close(stopProxy);
157         }
158       }
159     }
160     synchronized (ZK_WRAPPERS) {
161       for (ClientZKWatcher watch : ZK_WRAPPERS.values()) {
162         watch.resetZooKeeper();
163       }
164     }
165   }
166 
167   /**
168    * Get a watcher of a zookeeper connection for a given quorum address.
169    * If the connection isn't established, a new one is created.
170    * This acts like a multiton.
171    * @param conf configuration
172    * @return ZKW watcher
173    * @throws IOException if a remote or network exception occurs
174    */
175   public static synchronized ClientZKWatcher getClientZooKeeperWatcher(
176       Configuration conf) throws IOException {
177     if (!ZK_WRAPPERS.containsKey(
178         ZooKeeperWrapper.getZookeeperClusterKey(conf))) {
179       ZK_WRAPPERS.put(ZooKeeperWrapper.getZookeeperClusterKey(conf),
180           new ClientZKWatcher(conf));
181     }
182     return ZK_WRAPPERS.get(ZooKeeperWrapper.getZookeeperClusterKey(conf));
183   }
184 
185   /**
186    * This class is responsible to handle connection and reconnection
187    * to a zookeeper quorum.
188    *
189    */
190   public static class ClientZKWatcher implements Watcher {
191 
192     static final Log LOG = LogFactory.getLog(ClientZKWatcher.class);
193     private ZooKeeperWrapper zooKeeperWrapper;
194     private Configuration conf;
195 
196     /**
197      * Takes a configuration to pass it to ZKW but won't instanciate it
198      * @param conf configuration
199      */
200     public ClientZKWatcher(Configuration conf) {
201       this.conf = conf;
202     }
203 
204     /**
205      * Called by ZooKeeper when an event occurs on our connection. We use this to
206      * detect our session expiring. When our session expires, we have lost our
207      * connection to ZooKeeper. Our handle is dead, and we need to recreate it.
208      *
209      * See http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions
210      * for more information.
211      *
212      * @param event WatchedEvent witnessed by ZooKeeper.
213      */
214     public void process(final WatchedEvent event) {
215       final KeeperState state = event.getState();
216       if (!state.equals(KeeperState.SyncConnected)) {
217         LOG.warn("No longer connected to ZooKeeper, current state: " + state);
218         resetZooKeeper();
219       }
220     }
221 
222     /**
223      * Get this watcher's ZKW, instantiate it if necessary.
224      * @return ZKW
225      * @throws java.io.IOException if a remote or network exception occurs
226      */
227     public synchronized ZooKeeperWrapper getZooKeeperWrapper() throws IOException {
228       if (zooKeeperWrapper == null) {
229         zooKeeperWrapper =
230             ZooKeeperWrapper.createInstance(conf, HConnectionManager.class.getName());
231         zooKeeperWrapper.registerListener(this);
232       }
233       return zooKeeperWrapper;
234     }
235 
236     /**
237      * Clear this connection to zookeeper.
238      */
239     private synchronized void resetZooKeeper() {
240       if (zooKeeperWrapper != null) {
241         zooKeeperWrapper.close();
242         zooKeeperWrapper = null;
243       }
244     }
245   }
246 
247   /**
248    * It is provided for unit test cases which verify the behavior of region
249    * location cache prefetch.
250    * @return Number of cached regions for the table.
251    */
252   static int getCachedRegionCount(Configuration conf,
253       byte[] tableName) {
254     TableServers connection = (TableServers)getConnection(conf);
255     return connection.getNumberOfCachedRegionLocations(tableName);
256   }
257 
258   /**
259    * It's provided for unit test cases which verify the behavior of region
260    * location cache prefetch.
261    * @return true if the region where the table and row reside is cached.
262    */
263   static boolean isRegionCached(Configuration conf,
264       byte[] tableName, byte[] row) {
265     TableServers connection = (TableServers)getConnection(conf);
266     return connection.isRegionCached(tableName, row);
267   }
268 
269   /* Encapsulates finding the servers for an HBase instance */
270   static class TableServers implements ServerConnection {
271     static final Log LOG = LogFactory.getLog(TableServers.class);
272     private final Class<? extends HRegionInterface> serverInterfaceClass;
273     private final long pause;
274     private final int numRetries;
275     private final int maxRPCAttempts;
276     private final long rpcTimeout;
277     private final int prefetchRegionLimit;
278 
279     private final Object masterLock = new Object();
280     private volatile boolean closed;
281     private volatile HMasterInterface master;
282     private volatile boolean masterChecked;
283 
284     private final Object rootRegionLock = new Object();
285     private final Object metaRegionLock = new Object();
286     private final Object userRegionLock = new Object();
287 
288     private volatile Configuration conf;
289 
290     // Known region HServerAddress.toString() -> HRegionInterface
291     private final Map<String, HRegionInterface> servers =
292       new ConcurrentHashMap<String, HRegionInterface>();
293 
294     // Used by master and region servers during safe mode only
295     private volatile HRegionLocation rootRegionLocation;
296 
297     private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>>
298       cachedRegionLocations =
299         new HashMap<Integer, SoftValueSortedMap<byte [], HRegionLocation>>();
300 
301     // region cache prefetch is enabled by default. this set contains all
302     // tables whose region cache prefetch are disabled.
303     private final Set<Integer> regionCachePrefetchDisabledTables =
304       new CopyOnWriteArraySet<Integer>();
305 
306     /**
307      * constructor
308      * @param conf Configuration object
309      */
310     @SuppressWarnings("unchecked")
311     public TableServers(Configuration conf) {
312       this.conf = conf;
313 
314       String serverClassName =
315         conf.get(HConstants.REGION_SERVER_CLASS,
316             HConstants.DEFAULT_REGION_SERVER_CLASS);
317 
318       this.closed = false;
319 
320       try {
321         this.serverInterfaceClass =
322           (Class<? extends HRegionInterface>) Class.forName(serverClassName);
323 
324       } catch (ClassNotFoundException e) {
325         throw new UnsupportedOperationException(
326             "Unable to find region server interface " + serverClassName, e);
327       }
328 
329       this.pause = conf.getLong("hbase.client.pause", 1000);
330       this.numRetries = conf.getInt("hbase.client.retries.number", 10);
331       this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts", 1);
332       this.rpcTimeout = conf.getLong(
333           HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
334           HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
335 
336       this.prefetchRegionLimit = conf.getInt("hbase.client.prefetch.limit",
337           10);
338 
339       this.master = null;
340       this.masterChecked = false;
341     }
342 
343     private long getPauseTime(int tries) {
344       int ntries = tries;
345       if (ntries >= HConstants.RETRY_BACKOFF.length)
346         ntries = HConstants.RETRY_BACKOFF.length - 1;
347       return this.pause * HConstants.RETRY_BACKOFF[ntries];
348     }
349 
350     // Used by master and region servers during safe mode only
351     public void unsetRootRegionLocation() {
352       this.rootRegionLocation = null;
353     }
354 
355     // Used by master and region servers during safe mode only
356     public void setRootRegionLocation(HRegionLocation rootRegion) {
357       if (rootRegion == null) {
358         throw new IllegalArgumentException(
359             "Cannot set root region location to null.");
360       }
361       this.rootRegionLocation = rootRegion;
362     }
363 
364     public HMasterInterface getMaster() throws MasterNotRunningException {
365       ZooKeeperWrapper zk;
366       try {
367         zk = getZooKeeperWrapper();
368       } catch (IOException e) {
369         throw new MasterNotRunningException(e);
370       }
371 
372       HServerAddress masterLocation = null;
373       synchronized (this.masterLock) {
374         for (int tries = 0;
375           !this.closed &&
376           !this.masterChecked && this.master == null &&
377           tries < numRetries;
378         tries++) {
379 
380           try {
381             masterLocation = zk.readMasterAddressOrThrow();
382 
383             HMasterInterface tryMaster = (HMasterInterface)HBaseRPC.getProxy(
384                 HMasterInterface.class, HBaseRPCProtocolVersion.versionID,
385                 masterLocation.getInetSocketAddress(), this.conf);
386 
387             if (tryMaster.isMasterRunning()) {
388               this.master = tryMaster;
389               this.masterLock.notifyAll();
390               break;
391             }
392 
393           } catch (IOException e) {
394             if (tries == numRetries - 1) {
395               // This was our last chance - don't bother sleeping
396               LOG.info("getMaster attempt " + tries + " of " + this.numRetries +
397                 " failed; no more retrying.", e);
398               break;
399             }
400             LOG.info("getMaster attempt " + tries + " of " + this.numRetries +
401               " failed; retrying after sleep of " +
402               getPauseTime(tries), e);
403           }
404 
405           // Cannot connect to master or it is not running. Sleep & retry
406           try {
407             this.masterLock.wait(getPauseTime(tries));
408           } catch (InterruptedException e) {
409             // continue
410           }
411         }
412         this.masterChecked = true;
413       }
414       if (this.master == null) {
415         if (masterLocation == null) {
416           throw new MasterNotRunningException();
417         }
418         throw new MasterNotRunningException(masterLocation.toString());
419       }
420       return this.master;
421     }
422 
423     public boolean isMasterRunning() {
424       if (this.master == null) {
425         try {
426           getMaster();
427 
428         } catch (MasterNotRunningException e) {
429           return false;
430         }
431       }
432       return true;
433     }
434 
435     public boolean tableExists(final byte [] tableName)
436     throws MasterNotRunningException {
437       getMaster();
438       if (tableName == null) {
439         throw new IllegalArgumentException("Table name cannot be null");
440       }
441       if (isMetaTableName(tableName)) {
442         return true;
443       }
444       boolean exists = false;
445       try {
446         HTableDescriptor[] tables = listTables();
447         for (HTableDescriptor table : tables) {
448           if (Bytes.equals(table.getName(), tableName)) {
449             exists = true;
450           }
451         }
452       } catch (IOException e) {
453         LOG.warn("Testing for table existence threw exception", e);
454       }
455       return exists;
456     }
457 
458     /*
459      * @param n
460      * @return Truen if passed tablename <code>n</code> is equal to the name
461      * of a catalog table.
462      */
463     private static boolean isMetaTableName(final byte [] n) {
464       return MetaUtils.isMetaTableName(n);
465     }
466 
467     public HRegionLocation getRegionLocation(final byte [] name,
468         final byte [] row, boolean reload)
469     throws IOException {
470       return reload? relocateRegion(name, row): locateRegion(name, row);
471     }
472 
473     public HTableDescriptor[] listTables() throws IOException {
474       getMaster();
475       final TreeSet<HTableDescriptor> uniqueTables =
476         new TreeSet<HTableDescriptor>();
477       MetaScannerVisitor visitor = new MetaScannerVisitor() {
478         public boolean processRow(Result result) throws IOException {
479           try {
480             byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
481                 HConstants.REGIONINFO_QUALIFIER);
482             HRegionInfo info = null;
483             if (value != null) {
484               info = Writables.getHRegionInfo(value);
485             }
486             // Only examine the rows where the startKey is zero length
487             if (info != null && info.getStartKey().length == 0) {
488               uniqueTables.add(info.getTableDesc());
489             }
490             return true;
491           } catch (RuntimeException e) {
492             LOG.error("Result=" + result);
493             throw e;
494           }
495         }
496       };
497       MetaScanner.metaScan(conf, visitor);
498 
499       return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
500     }
501 
502     public boolean isTableEnabled(byte[] tableName) throws IOException {
503       return testTableOnlineState(tableName, true);
504     }
505 
506     public boolean isTableDisabled(byte[] tableName) throws IOException {
507       return testTableOnlineState(tableName, false);
508     }
509 
510     public boolean isTableAvailable(final byte[] tableName) throws IOException {
511       final AtomicBoolean available = new AtomicBoolean(true);
512       MetaScannerVisitor visitor = new MetaScannerVisitor() {
513         @Override
514         public boolean processRow(Result row) throws IOException {
515           byte[] value = row.getValue(HConstants.CATALOG_FAMILY,
516               HConstants.REGIONINFO_QUALIFIER);
517           HRegionInfo info = Writables.getHRegionInfoOrNull(value);
518           if (info != null) {
519             if (Bytes.equals(tableName, info.getTableDesc().getName())) {
520               value = row.getValue(HConstants.CATALOG_FAMILY,
521                   HConstants.SERVER_QUALIFIER);
522               if (value == null) {
523                 available.set(false);
524                 return false;
525               }
526             }
527           }
528           return true;
529         }
530       };
531       MetaScanner.metaScan(conf, visitor);
532       return available.get();
533     }
534 
535     /*
536      * If online == true
537      *   Returns true if all regions are online
538      *   Returns false in any other case
539      * If online == false
540      *   Returns true if all regions are offline
541      *   Returns false in any other case
542      */
543     private boolean testTableOnlineState(byte[] tableName, boolean online)
544     throws IOException {
545       if (!tableExists(tableName)) {
546         throw new TableNotFoundException(Bytes.toString(tableName));
547       }
548       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
549         // The root region is always enabled
550         return true;
551       }
552       int rowsScanned = 0;
553       int rowsOffline = 0;
554       byte[] startKey =
555         HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES, false);
556       byte[] endKey;
557       HRegionInfo currentRegion;
558       Scan scan = new Scan(startKey);
559       scan.addColumn(HConstants.CATALOG_FAMILY,
560           HConstants.REGIONINFO_QUALIFIER);
561       int rows = this.conf.getInt("hbase.meta.scanner.caching", 100);
562       scan.setCaching(rows);
563       ScannerCallable s = new ScannerCallable(this,
564           (Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
565               HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME), scan);
566       try {
567         // Open scanner
568         getRegionServerWithRetries(s);
569         do {
570           currentRegion = s.getHRegionInfo();
571           Result r;
572           Result [] rrs;
573           while ((rrs = getRegionServerWithRetries(s)) != null && rrs.length > 0) {
574             r = rrs[0];
575             byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
576               HConstants.REGIONINFO_QUALIFIER);
577             if (value != null) {
578               HRegionInfo info = Writables.getHRegionInfoOrNull(value);
579               if (info != null) {
580                 if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
581                   rowsScanned += 1;
582                   rowsOffline += info.isOffline() ? 1 : 0;
583                 }
584               }
585             }
586           }
587           endKey = currentRegion.getEndKey();
588         } while (!(endKey == null ||
589             Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)));
590       } finally {
591         s.setClose();
592         // Doing below will call 'next' again and this will close the scanner
593         // Without it we leave scanners open.
594         getRegionServerWithRetries(s);
595       }
596       LOG.debug("Rowscanned=" + rowsScanned + ", rowsOffline=" + rowsOffline);
597       boolean onOffLine = online? rowsOffline == 0: rowsOffline == rowsScanned;
598       return rowsScanned > 0 && onOffLine;
599     }
600 
601     private static class HTableDescriptorFinder
602     implements MetaScanner.MetaScannerVisitor {
603         byte[] tableName;
604         HTableDescriptor result;
605         protected HTableDescriptorFinder(byte[] tableName) {
606           this.tableName = tableName;
607         }
608         public boolean processRow(Result rowResult) throws IOException {
609           HRegionInfo info = Writables.getHRegionInfo(
610               rowResult.getValue(HConstants.CATALOG_FAMILY,
611                   HConstants.REGIONINFO_QUALIFIER));
612           HTableDescriptor desc = info.getTableDesc();
613           if (Bytes.compareTo(desc.getName(), tableName) == 0) {
614             result = desc;
615             return false;
616           }
617           return true;
618         }
619         HTableDescriptor getResult() {
620           return result;
621         }
622     }
623 
624     public HTableDescriptor getHTableDescriptor(final byte[] tableName)
625     throws IOException {
626       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
627         return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
628       }
629       if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
630         return HTableDescriptor.META_TABLEDESC;
631       }
632       HTableDescriptorFinder finder = new HTableDescriptorFinder(tableName);
633       MetaScanner.metaScan(conf, finder, tableName);
634       HTableDescriptor result = finder.getResult();
635       if (result == null) {
636         throw new TableNotFoundException(Bytes.toString(tableName));
637       }
638       return result;
639     }
640 
641     public HRegionLocation locateRegion(final byte [] tableName,
642         final byte [] row)
643     throws IOException{
644       return locateRegion(tableName, row, true);
645     }
646 
647     public HRegionLocation relocateRegion(final byte [] tableName,
648         final byte [] row)
649     throws IOException{
650       return locateRegion(tableName, row, false);
651     }
652 
653     private HRegionLocation locateRegion(final byte [] tableName,
654       final byte [] row, boolean useCache)
655     throws IOException{
656       if (tableName == null || tableName.length == 0) {
657         throw new IllegalArgumentException(
658             "table name cannot be null or zero length");
659       }
660 
661       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
662         synchronized (rootRegionLock) {
663           // This block guards against two threads trying to find the root
664           // region at the same time. One will go do the find while the
665           // second waits. The second thread will not do find.
666 
667           if (!useCache || rootRegionLocation == null) {
668             this.rootRegionLocation = locateRootRegion();
669           }
670           return this.rootRegionLocation;
671         }
672       } else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
673         return locateRegionInMeta(HConstants.ROOT_TABLE_NAME, tableName, row,
674             useCache, metaRegionLock);
675       } else {
676         // Region not in the cache - have to go to the meta RS
677         return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
678             useCache, userRegionLock);
679       }
680     }
681 
682     /*
683      * Search .META. for the HRegionLocation info that contains the table and
684      * row we're seeking. It will prefetch certain number of regions info and
685      * save them to the global region cache.
686      */
687     private void prefetchRegionCache(final byte[] tableName,
688         final byte[] row) {
689       // Implement a new visitor for MetaScanner, and use it to walk through
690       // the .META.
691       MetaScannerVisitor visitor = new MetaScannerVisitor() {
692         public boolean processRow(Result result) throws IOException {
693           try {
694             byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
695                 HConstants.REGIONINFO_QUALIFIER);
696             HRegionInfo regionInfo = null;
697 
698             if (value != null) {
699               // convert the row result into the HRegionLocation we need!
700               regionInfo = Writables.getHRegionInfo(value);
701 
702               // possible we got a region of a different table...
703               if (!Bytes.equals(regionInfo.getTableDesc().getName(),
704                   tableName)) {
705                 return false; // stop scanning
706               }
707               if (regionInfo.isOffline()) {
708                 // don't cache offline regions
709                 return true;
710               }
711               value = result.getValue(HConstants.CATALOG_FAMILY,
712                   HConstants.SERVER_QUALIFIER);
713               if (value == null) {
714                 return true;  // don't cache it
715               }
716               final String serverAddress = Bytes.toString(value);
717 
718               // instantiate the location
719               HRegionLocation loc = new HRegionLocation(regionInfo,
720                 new HServerAddress(serverAddress));
721               // cache this meta entry
722               cacheLocation(tableName, loc);
723             }
724             return true;
725           } catch (RuntimeException e) {
726             throw new IOException(e);
727           }
728         }
729       };
730       try {
731         // pre-fetch certain number of regions info at region cache.
732         MetaScanner.metaScan(conf, visitor, tableName, row,
733             this.prefetchRegionLimit);
734       } catch (IOException e) {
735         LOG.warn("Encounted problems when prefetch META table: ", e);
736       }
737     }
738 
739     /*
740       * Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation
741       * info that contains the table and row we're seeking.
742       */
743     private HRegionLocation locateRegionInMeta(final byte [] parentTable,
744       final byte [] tableName, final byte [] row, boolean useCache,
745       Object regionLockObject)
746     throws IOException {
747       HRegionLocation location;
748       // If we are supposed to be using the cache, look in the cache to see if
749       // we already have the region.
750       if (useCache) {
751         location = getCachedLocation(tableName, row);
752         if (location != null) {
753           return location;
754         }
755       }
756 
757       // build the key of the meta region we should be looking for.
758       // the extra 9's on the end are necessary to allow "exact" matches
759       // without knowing the precise region names.
760       byte [] metaKey = HRegionInfo.createRegionName(tableName, row,
761         HConstants.NINES, false);
762       for (int tries = 0; true; tries++) {
763         if (tries >= numRetries) {
764           throw new NoServerForRegionException("Unable to find region for "
765             + Bytes.toStringBinary(row) + " after " + numRetries + " tries.");
766         }
767 
768         try {
769           // locate the root or meta region
770           HRegionLocation metaLocation = locateRegion(parentTable, metaKey);
771           HRegionInterface server =
772             getHRegionConnection(metaLocation.getServerAddress());
773 
774           Result regionInfoRow = null;
775           // This block guards against two threads trying to load the meta
776           // region at the same time. The first will load the meta region and
777           // the second will use the value that the first one found.
778           synchronized (regionLockObject) {
779             // If the parent table is META, we may want to pre-fetch some
780             // region info into the global region cache for this table.
781             if (Bytes.equals(parentTable, HConstants.META_TABLE_NAME) &&
782                 (getRegionCachePrefetch(tableName)) )  {
783               prefetchRegionCache(tableName, row);
784             }
785 
786             // Check the cache again for a hit in case some other thread made the
787             // same query while we were waiting on the lock. If not supposed to
788             // be using the cache, delete any existing cached location so it won't
789             // interfere.
790             if (useCache) {
791               location = getCachedLocation(tableName, row);
792               if (location != null) {
793                 return location;
794               }
795             } else {
796               deleteCachedLocation(tableName, row);
797             }
798 
799           // Query the root or meta region for the location of the meta region
800             regionInfoRow = server.getClosestRowBefore(
801             metaLocation.getRegionInfo().getRegionName(), metaKey,
802             HConstants.CATALOG_FAMILY);
803           }
804           if (regionInfoRow == null) {
805             throw new TableNotFoundException(Bytes.toString(tableName));
806           }
807           byte[] value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
808               HConstants.REGIONINFO_QUALIFIER);
809           if (value == null || value.length == 0) {
810             throw new IOException("HRegionInfo was null or empty in " +
811               Bytes.toString(parentTable) + ", row=" + regionInfoRow);
812           }
813           // convert the row result into the HRegionLocation we need!
814           HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
815               value, new HRegionInfo());
816           // possible we got a region of a different table...
817           if (!Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) {
818             throw new TableNotFoundException(
819               "Table '" + Bytes.toString(tableName) + "' was not found.");
820           }
821           if (regionInfo.isOffline()) {
822             throw new RegionOfflineException("region offline: " +
823               regionInfo.getRegionNameAsString());
824           }
825 
826           value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
827               HConstants.SERVER_QUALIFIER);
828           String serverAddress = "";
829           if(value != null) {
830             serverAddress = Bytes.toString(value);
831           }
832           if (serverAddress.equals("")) {
833             throw new NoServerForRegionException("No server address listed " +
834               "in " + Bytes.toString(parentTable) + " for region " +
835               regionInfo.getRegionNameAsString());
836           }
837 
838           // instantiate the location
839           location = new HRegionLocation(regionInfo,
840             new HServerAddress(serverAddress));
841           cacheLocation(tableName, location);
842           return location;
843         } catch (TableNotFoundException e) {
844           // if we got this error, probably means the table just plain doesn't
845           // exist. rethrow the error immediately. this should always be coming
846           // from the HTable constructor.
847           throw e;
848         } catch (IOException e) {
849           if (e instanceof RemoteException) {
850             e = RemoteExceptionHandler.decodeRemoteException(
851                 (RemoteException) e);
852           }
853           if (tries < numRetries - 1) {
854             if (LOG.isDebugEnabled()) {
855               LOG.debug("locateRegionInMeta attempt " + tries + " of " +
856                 this.numRetries + " failed; retrying after sleep of " +
857                 getPauseTime(tries) + " because: " + e.getMessage());
858             }
859           } else {
860             throw e;
861           }
862           // Only relocate the parent region if necessary
863           if(!(e instanceof RegionOfflineException ||
864               e instanceof NoServerForRegionException)) {
865             relocateRegion(parentTable, metaKey);
866           }
867         }
868         try{
869           Thread.sleep(getPauseTime(tries));
870         } catch (InterruptedException e){
871           // continue
872         }
873       }
874     }
875 
876     /*
877      * Search the cache for a location that fits our table and row key.
878      * Return null if no suitable region is located. TODO: synchronization note
879      *
880      * <p>TODO: This method during writing consumes 15% of CPU doing lookup
881      * into the Soft Reference SortedMap.  Improve.
882      *
883      * @param tableName
884      * @param row
885      * @return Null or region location found in cache.
886      */
887     HRegionLocation getCachedLocation(final byte [] tableName,
888         final byte [] row) {
889       SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
890         getTableLocations(tableName);
891 
892       // start to examine the cache. we can only do cache actions
893       // if there's something in the cache for this table.
894       if (tableLocations.isEmpty()) {
895         return null;
896       }
897 
898       HRegionLocation rl = tableLocations.get(row);
899       if (rl != null) {
900         if (LOG.isDebugEnabled()) {
901           LOG.debug("Cache hit for row <" +
902             Bytes.toStringBinary(row) +
903             "> in tableName " + Bytes.toString(tableName) +
904             ": location server " + rl.getServerAddress() +
905             ", location region name " +
906             rl.getRegionInfo().getRegionNameAsString());
907         }
908         return rl;
909       }
910 
911       // Cut the cache so that we only get the part that could contain
912       // regions that match our key
913       SoftValueSortedMap<byte[], HRegionLocation> matchingRegions =
914         tableLocations.headMap(row);
915 
916       // if that portion of the map is empty, then we're done. otherwise,
917       // we need to examine the cached location to verify that it is
918       // a match by end key as well.
919       if (!matchingRegions.isEmpty()) {
920         HRegionLocation possibleRegion =
921           matchingRegions.get(matchingRegions.lastKey());
922 
923         // there is a possibility that the reference was garbage collected
924         // in the instant since we checked isEmpty().
925         if (possibleRegion != null) {
926           byte[] endKey = possibleRegion.getRegionInfo().getEndKey();
927 
928           // make sure that the end key is greater than the row we're looking
929           // for, otherwise the row actually belongs in the next region, not
930           // this one. the exception case is when the endkey is
931           // HConstants.EMPTY_START_ROW, signifying that the region we're
932           // checking is actually the last region in the table.
933           if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
934               KeyValue.getRowComparator(tableName).compareRows(endKey, 0, endKey.length,
935                   row, 0, row.length) > 0) {
936             return possibleRegion;
937           }
938         }
939       }
940 
941       // Passed all the way through, so we got nothin - complete cache miss
942       return null;
943     }
944 
945     /*
946      * Delete a cached location, if it satisfies the table name and row
947      * requirements.
948      */
949     void deleteCachedLocation(final byte [] tableName,
950                                       final byte [] row) {
951       synchronized (this.cachedRegionLocations) {
952         SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
953             getTableLocations(tableName);
954 
955         // start to examine the cache. we can only do cache actions
956         // if there's something in the cache for this table.
957         if (!tableLocations.isEmpty()) {
958           HRegionLocation rl = getCachedLocation(tableName, row);
959           if (rl != null) {
960             tableLocations.remove(rl.getRegionInfo().getStartKey());
961             if (LOG.isDebugEnabled()) {
962               LOG.debug("Removed " +
963                   rl.getRegionInfo().getRegionNameAsString() +
964                   " for tableName=" + Bytes.toString(tableName) +
965                   " from cache " + "because of " + Bytes.toStringBinary(row));
966             }
967           }
968         }
969       }
970     }
971 
972     /*
973      * @param tableName
974      * @return Map of cached locations for passed <code>tableName</code>
975      */
976     private SoftValueSortedMap<byte [], HRegionLocation> getTableLocations(
977         final byte [] tableName) {
978       // find the map of cached locations for this table
979       Integer key = Bytes.mapKey(tableName);
980       SoftValueSortedMap<byte [], HRegionLocation> result;
981       synchronized (this.cachedRegionLocations) {
982         result = this.cachedRegionLocations.get(key);
983         // if tableLocations for this table isn't built yet, make one
984         if (result == null) {
985           result = new SoftValueSortedMap<byte [], HRegionLocation>(
986               Bytes.BYTES_COMPARATOR);
987           this.cachedRegionLocations.put(key, result);
988         }
989       }
990       return result;
991     }
992 
993     /**
994      * Allows flushing the region cache.
995      */
996     public void clearRegionCache() {
997      cachedRegionLocations.clear();
998     }
999 
1000     /*
1001      * Put a newly discovered HRegionLocation into the cache.
1002      */
1003     private void cacheLocation(final byte [] tableName,
1004         final HRegionLocation location) {
1005       byte [] startKey = location.getRegionInfo().getStartKey();
1006       SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
1007         getTableLocations(tableName);
1008       if (tableLocations.put(startKey, location) == null) {
1009         LOG.debug("Cached location for " +
1010             location.getRegionInfo().getRegionNameAsString() +
1011             " is " + location.getServerAddress());
1012       }
1013     }
1014 
1015     public HRegionInterface getHRegionConnection(
1016         HServerAddress regionServer, boolean getMaster)
1017     throws IOException {
1018       if (getMaster) {
1019         getMaster();
1020       }
1021       HRegionInterface server;
1022       synchronized (this.servers) {
1023         // See if we already have a connection
1024         server = this.servers.get(regionServer.toString());
1025         if (server == null) { // Get a connection
1026           try {
1027             server = (HRegionInterface)HBaseRPC.waitForProxy(
1028                 serverInterfaceClass, HBaseRPCProtocolVersion.versionID,
1029                 regionServer.getInetSocketAddress(), this.conf,
1030                 this.maxRPCAttempts, this.rpcTimeout);
1031           } catch (RemoteException e) {
1032             throw RemoteExceptionHandler.decodeRemoteException(e);
1033           }
1034           this.servers.put(regionServer.toString(), server);
1035         }
1036       }
1037       return server;
1038     }
1039 
1040     public HRegionInterface getHRegionConnection(
1041         HServerAddress regionServer)
1042     throws IOException {
1043       return getHRegionConnection(regionServer, false);
1044     }
1045 
1046     public synchronized ZooKeeperWrapper getZooKeeperWrapper()
1047         throws IOException {
1048       return HConnectionManager.getClientZooKeeperWatcher(conf)
1049           .getZooKeeperWrapper();
1050     }
1051 
1052     /*
1053      * Repeatedly try to find the root region in ZK
1054      * @return HRegionLocation for root region if found
1055      * @throws NoServerForRegionException - if the root region can not be
1056      * located after retrying
1057      * @throws IOException
1058      */
1059     private HRegionLocation locateRootRegion()
1060     throws IOException {
1061 
1062       // We lazily instantiate the ZooKeeper object because we don't want to
1063       // make the constructor have to throw IOException or handle it itself.
1064       ZooKeeperWrapper zk = getZooKeeperWrapper();
1065 
1066       HServerAddress rootRegionAddress = null;
1067       for (int tries = 0; tries < numRetries; tries++) {
1068         int localTimeouts = 0;
1069         // ask the master which server has the root region
1070         while (rootRegionAddress == null && localTimeouts < numRetries) {
1071           // Don't read root region until we're out of safe mode so we know
1072           // that the meta regions have been assigned.
1073           rootRegionAddress = zk.readRootRegionLocation();
1074           if (rootRegionAddress == null) {
1075             try {
1076               if (LOG.isDebugEnabled()) {
1077                 LOG.debug("Sleeping " + getPauseTime(tries) +
1078                   "ms, waiting for root region.");
1079               }
1080               Thread.sleep(getPauseTime(tries));
1081             } catch (InterruptedException iex) {
1082               // continue
1083             }
1084             localTimeouts++;
1085           }
1086         }
1087 
1088         if (rootRegionAddress == null) {
1089           throw new NoServerForRegionException(
1090               "Timed out trying to locate root region");
1091         }
1092 
1093         try {
1094           // Get a connection to the region server
1095           HRegionInterface server = getHRegionConnection(rootRegionAddress);
1096           // if this works, then we're good, and we have an acceptable address,
1097           // so we can stop doing retries and return the result.
1098           server.getRegionInfo(HRegionInfo.ROOT_REGIONINFO.getRegionName());
1099           if (LOG.isDebugEnabled()) {
1100             LOG.debug("Found ROOT at " + rootRegionAddress);
1101           }
1102           break;
1103         } catch (Throwable t) {
1104           t = translateException(t);
1105 
1106           if (tries == numRetries - 1) {
1107             throw new NoServerForRegionException("Timed out trying to locate "+
1108                 "root region because: " + t.getMessage());
1109           }
1110 
1111           // Sleep and retry finding root region.
1112           try {
1113             if (LOG.isDebugEnabled()) {
1114               LOG.debug("Root region location changed. Sleeping.");
1115             }
1116             Thread.sleep(getPauseTime(tries));
1117             if (LOG.isDebugEnabled()) {
1118               LOG.debug("Wake. Retry finding root region.");
1119             }
1120           } catch (InterruptedException iex) {
1121             // continue
1122           }
1123         }
1124 
1125         rootRegionAddress = null;
1126       }
1127 
1128       // if the address is null by this point, then the retries have failed,
1129       // and we're sort of sunk
1130       if (rootRegionAddress == null) {
1131         throw new NoServerForRegionException(
1132           "unable to locate root region server");
1133       }
1134 
1135       // return the region location
1136       return new HRegionLocation(
1137         HRegionInfo.ROOT_REGIONINFO, rootRegionAddress);
1138     }
1139 
1140     public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
1141     throws IOException, RuntimeException {
1142       List<Throwable> exceptions = new ArrayList<Throwable>();
1143       for(int tries = 0; tries < numRetries; tries++) {
1144         try {
1145           callable.instantiateServer(tries != 0);
1146           return callable.call();
1147         } catch (Throwable t) {
1148           t = translateException(t);
1149           exceptions.add(t);
1150           if (tries == numRetries - 1) {
1151             throw new RetriesExhaustedException(callable.getServerName(),
1152                 callable.getRegionName(), callable.getRow(), tries, exceptions);
1153           }
1154         }
1155         try {
1156           Thread.sleep(getPauseTime(tries));
1157         } catch (InterruptedException e) {
1158           // continue
1159         }
1160       }
1161       return null;
1162     }
1163 
1164     public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
1165         throws IOException, RuntimeException {
1166       try {
1167         callable.instantiateServer(false);
1168         return callable.call();
1169       } catch (Throwable t) {
1170         Throwable t2 = translateException(t);
1171         if (t2 instanceof IOException) {
1172           throw (IOException)t2;
1173         } else {
1174           throw new RuntimeException(t2);
1175         }
1176       }
1177     }
1178 
1179     private HRegionLocation
1180       getRegionLocationForRowWithRetries(byte[] tableName, byte[] rowKey,
1181         boolean reload)
1182     throws IOException {
1183       boolean reloadFlag = reload;
1184       List<Throwable> exceptions = new ArrayList<Throwable>();
1185       HRegionLocation location = null;
1186       int tries = 0;
1187       for (; tries < numRetries;) {
1188         try {
1189           location = getRegionLocation(tableName, rowKey, reloadFlag);
1190         } catch (Throwable t) {
1191           exceptions.add(t);
1192         }
1193         if (location != null) {
1194           break;
1195         }
1196         reloadFlag = true;
1197         tries++;
1198         try {
1199           Thread.sleep(getPauseTime(tries));
1200         } catch (InterruptedException e) {
1201           // continue
1202         }
1203       }
1204       if (location == null) {
1205         throw new RetriesExhaustedException(" -- nothing found, no 'location' returned," +
1206           " tableName=" + Bytes.toString(tableName) +
1207           ", reload=" + reload + " --",
1208           HConstants.EMPTY_BYTE_ARRAY, rowKey, tries, exceptions);
1209       }
1210       return location;
1211     }
1212 
1213     /*
1214      * Helper class for batch updates.
1215      * Holds code shared doing batch puts and batch deletes.
1216      */
1217     private abstract class Batch {
1218       final HConnection c;
1219 
1220       private Batch(final HConnection c) {
1221         this.c = c;
1222       }
1223 
1224       /**
1225        * This is the method subclasses must implement.
1226        * @param currentList current list of rows
1227        * @param tableName table we are processing
1228        * @param row row
1229        * @return Count of items processed or -1 if all.
1230        * @throws IOException if a remote or network exception occurs
1231        * @throws RuntimeException other undefined exception
1232        */
1233       abstract int doCall(final List<? extends Row> currentList,
1234         final byte [] row, final byte [] tableName)
1235       throws IOException, RuntimeException;
1236 
1237       /**
1238        * Process the passed <code>list</code>.
1239        * @param list list of rows to process
1240        * @param tableName table we are processing
1241        * @return Count of how many added or -1 if all added.
1242        * @throws IOException if a remote or network exception occurs
1243        */
1244       int process(final List<? extends Row> list, final byte[] tableName)
1245       throws IOException {
1246         byte [] region = getRegionName(tableName, list.get(0).getRow(), false);
1247         byte [] currentRegion = region;
1248         boolean isLastRow;
1249         boolean retryOnlyOne = false;
1250         List<Row> currentList = new ArrayList<Row>();
1251         int i, tries;
1252         for (i = 0, tries = 0; i < list.size() && tries < numRetries; i++) {
1253           Row row = list.get(i);
1254           currentList.add(row);
1255           // If the next record goes to a new region, then we are to clear
1256           // currentList now during this cycle.
1257           isLastRow = (i + 1) == list.size();
1258           if (!isLastRow) {
1259             region = getRegionName(tableName, list.get(i + 1).getRow(), false);
1260           }
1261           if (!Bytes.equals(currentRegion, region) || isLastRow || retryOnlyOne) {
1262             int index = doCall(currentList, row.getRow(), tableName);
1263             // index is == -1 if all processed successfully, else its index
1264             // of last record successfully processed.
1265             if (index != -1) {
1266               if (tries == numRetries - 1) {
1267                 throw new RetriesExhaustedException("Some server, retryOnlyOne=" +
1268                   retryOnlyOne + ", index=" + index + ", islastrow=" + isLastRow +
1269                   ", tries=" + tries + ", numtries=" + numRetries + ", i=" + i +
1270                   ", listsize=" + list.size() + ", region=" +
1271                   Bytes.toStringBinary(region), currentRegion, row.getRow(),
1272                   tries, new ArrayList<Throwable>());
1273               }
1274               tries = doBatchPause(currentRegion, tries);
1275               i = i - currentList.size() + index;
1276               retryOnlyOne = true;
1277               // Reload location.
1278               region = getRegionName(tableName, list.get(i + 1).getRow(), true);
1279             } else {
1280               // Reset these flags/counters on successful batch Put
1281               retryOnlyOne = false;
1282               tries = 0;
1283             }
1284             currentRegion = region;
1285             currentList.clear();
1286           }
1287         }
1288         return i;
1289       }
1290 
1291       /*
1292        * @param t
1293        * @param r
1294        * @param re
1295        * @return Region name that holds passed row <code>r</code>
1296        * @throws IOException
1297        */
1298       private byte [] getRegionName(final byte [] t, final byte [] r,
1299         final boolean re)
1300       throws IOException {
1301         HRegionLocation location = getRegionLocationForRowWithRetries(t, r, re);
1302         return location.getRegionInfo().getRegionName();
1303       }
1304 
1305       /*
1306        * Do pause processing before retrying...
1307        * @param currentRegion
1308        * @param tries
1309        * @return New value for tries.
1310        */
1311       private int doBatchPause(final byte [] currentRegion, final int tries) {
1312         int localTries = tries;
1313         long sleepTime = getPauseTime(tries);
1314         if (LOG.isDebugEnabled()) {
1315           LOG.debug("Reloading region " + Bytes.toStringBinary(currentRegion) +
1316             " location because regionserver didn't accept updates; tries=" +
1317             tries + " of max=" + numRetries + ", waiting=" + sleepTime + "ms");
1318         }
1319         try {
1320           Thread.sleep(sleepTime);
1321           localTries++;
1322         } catch (InterruptedException e) {
1323           // continue
1324         }
1325         return localTries;
1326       }
1327     }
1328 
1329     public int processBatchOfRows(final ArrayList<Put> list,
1330       final byte[] tableName)
1331     throws IOException {
1332       if (list.isEmpty()) return 0;
1333       if (list.size() > 1) Collections.sort(list);
1334       Batch b = new Batch(this) {
1335         @SuppressWarnings("unchecked")
1336         @Override
1337         int doCall(final List<? extends Row> currentList, final byte [] row,
1338           final byte [] tableName)
1339         throws IOException, RuntimeException {
1340           final List<Put> puts = (List<Put>)currentList;
1341           return getRegionServerWithRetries(new ServerCallable<Integer>(this.c,
1342               tableName, row) {
1343             public Integer call() throws IOException {
1344               return server.put(location.getRegionInfo().getRegionName(), puts);
1345             }
1346           });
1347         }
1348       };
1349       return b.process(list, tableName);
1350     }
1351 
1352     public int processBatchOfDeletes(final List<Delete> list,
1353       final byte[] tableName)
1354     throws IOException {
1355       if (list.isEmpty()) return 0;
1356       if (list.size() > 1) Collections.sort(list);
1357       Batch b = new Batch(this) {
1358         @SuppressWarnings("unchecked")
1359         @Override
1360         int doCall(final List<? extends Row> currentList, final byte [] row,
1361           final byte [] tableName)
1362         throws IOException, RuntimeException {
1363           final List<Delete> deletes = (List<Delete>)currentList;
1364           return getRegionServerWithRetries(new ServerCallable<Integer>(this.c,
1365                 tableName, row) {
1366               public Integer call() throws IOException {
1367                 return server.delete(location.getRegionInfo().getRegionName(),
1368                   deletes);
1369               }
1370             });
1371           }
1372         };
1373         return b.process(list, tableName);
1374       }
1375 
1376     void close(boolean stopProxy) {
1377       if (master != null) {
1378         if (stopProxy) {
1379           HBaseRPC.stopProxy(master);
1380         }
1381         master = null;
1382         masterChecked = false;
1383       }
1384       if (stopProxy) {
1385         for (HRegionInterface i: servers.values()) {
1386           HBaseRPC.stopProxy(i);
1387         }
1388       }
1389     }
1390 
1391     /**
1392      * Process a batch of Puts on the given executor service.
1393      *
1394      * @param list the puts to make - successful puts will be removed.
1395      * @param pool thread pool to execute requests on
1396      *
1397      * In the case of an exception, we take different actions depending on the
1398      * situation:
1399      *  - If the exception is a DoNotRetryException, we rethrow it and leave the
1400      *    'list' parameter in an indeterminate state.
1401      *  - If the 'list' parameter is a singleton, we directly throw the specific
1402      *    exception for that put.
1403      *  - Otherwise, we throw a generic exception indicating that an error occurred.
1404      *    The 'list' parameter is mutated to contain those puts that did not succeed.
1405      */
1406     public void processBatchOfPuts(List<Put> list,
1407                                    final byte[] tableName, ExecutorService pool) throws IOException {
1408       boolean singletonList = list.size() == 1;
1409       Throwable singleRowCause = null;
1410       for ( int tries = 0 ; tries < numRetries && !list.isEmpty(); ++tries) {
1411         Collections.sort(list);
1412         Map<HServerAddress, MultiPut> regionPuts =
1413             new HashMap<HServerAddress, MultiPut>();
1414         // step 1:
1415         //  break up into regionserver-sized chunks and build the data structs
1416         for ( Put put : list ) {
1417           byte [] row = put.getRow();
1418 
1419           HRegionLocation loc = locateRegion(tableName, row, true);
1420           HServerAddress address = loc.getServerAddress();
1421           byte [] regionName = loc.getRegionInfo().getRegionName();
1422 
1423           MultiPut mput = regionPuts.get(address);
1424           if (mput == null) {
1425             mput = new MultiPut(address);
1426             regionPuts.put(address, mput);
1427           }
1428           mput.add(regionName, put);
1429         }
1430 
1431         // step 2:
1432         //  make the requests
1433         // Discard the map, just use a list now, makes error recovery easier.
1434         List<MultiPut> multiPuts = new ArrayList<MultiPut>(regionPuts.values());
1435 
1436         List<Future<MultiPutResponse>> futures =
1437             new ArrayList<Future<MultiPutResponse>>(regionPuts.size());
1438         for ( MultiPut put : multiPuts ) {
1439           futures.add(pool.submit(createPutCallable(put.address,
1440               put,
1441               tableName)));
1442         }
1443         // RUN!
1444         List<Put> failed = new ArrayList<Put>();
1445 
1446         // step 3:
1447         //  collect the failures and tries from step 1.
1448         for (int i = 0; i < futures.size(); i++ ) {
1449           Future<MultiPutResponse> future = futures.get(i);
1450           MultiPut request = multiPuts.get(i);
1451           try {
1452             MultiPutResponse resp = future.get();
1453 
1454             // For each region
1455             for (Map.Entry<byte[], List<Put>> e : request.puts.entrySet()) {
1456               Integer result = resp.getAnswer(e.getKey());
1457               if (result == null) {
1458                 // failed
1459                 LOG.debug("Failed all for region: " +
1460                     Bytes.toStringBinary(e.getKey()) + ", removing from cache");
1461                 failed.addAll(e.getValue());
1462               } else if (result >= 0) {
1463                 // some failures
1464                 List<Put> lst = e.getValue();
1465                 failed.addAll(lst.subList(result, lst.size()));
1466                 LOG.debug("Failed past " + result + " for region: " +
1467                     Bytes.toStringBinary(e.getKey()) + ", removing from cache");
1468               }
1469             }
1470           } catch (InterruptedException e) {
1471             // go into the failed list.
1472             LOG.debug("Failed all from " + request.address, e);
1473             failed.addAll(request.allPuts());
1474           } catch (ExecutionException e) {
1475             Throwable cause = e.getCause();
1476             // Don't print stack trace if NSRE; NSRE is 'normal' operation.
1477             if (cause instanceof NotServingRegionException) {
1478               String msg = cause.getMessage();
1479               if (msg != null && msg.length() > 0) {
1480                 // msg is the exception as a String... we just want first line.
1481                 msg = msg.split("[\\n\\r]+\\s*at")[0];
1482               }
1483               LOG.debug("Failed execution of all on " + request.address +
1484                 " because: " + msg);
1485             } else {
1486               // all go into the failed list.
1487               LOG.debug("Failed execution of all on " + request.address,
1488                 e.getCause());
1489             }
1490             failed.addAll(request.allPuts());
1491 
1492             // Just give up, leaving the batch put list in an untouched/semi-committed state
1493             if (e.getCause() instanceof DoNotRetryIOException) {
1494               throw (DoNotRetryIOException) e.getCause();
1495             }
1496 
1497             if (singletonList) {
1498               // be richer for reporting in a 1 row case.
1499               singleRowCause = e.getCause();
1500             }
1501           }
1502         }
1503         list.clear();
1504         if (!failed.isEmpty()) {
1505           for (Put failedPut: failed) {
1506             deleteCachedLocation(tableName, failedPut.getRow());
1507           }
1508 
1509           list.addAll(failed);
1510 
1511           long sleepTime = getPauseTime(tries);
1512           LOG.debug("processBatchOfPuts had some failures, sleeping for " + sleepTime +
1513               " ms!");
1514           try {
1515             Thread.sleep(sleepTime);
1516           } catch (InterruptedException ignored) {
1517           }
1518         }
1519       }
1520       if (!list.isEmpty()) {
1521         if (singletonList && singleRowCause != null) {
1522           throw new IOException(singleRowCause);
1523         }
1524 
1525         // ran out of retries and didnt succeed everything!
1526         throw new RetriesExhaustedException("Still had " + list.size() + " puts left after retrying " +
1527             numRetries + " times.");
1528       }
1529     }
1530 
1531 
1532     private Callable<MultiPutResponse> createPutCallable(
1533         final HServerAddress address, final MultiPut puts,
1534         final byte [] tableName) {
1535       final HConnection connection = this;
1536       return new Callable<MultiPutResponse>() {
1537         public MultiPutResponse call() throws IOException {
1538           return getRegionServerWithoutRetries(
1539               new ServerCallable<MultiPutResponse>(connection, tableName, null) {
1540                 public MultiPutResponse call() throws IOException {
1541                   MultiPutResponse resp = server.multiPut(puts);
1542                   resp.request = puts;
1543                   return resp;
1544                 }
1545                 @Override
1546                 public void instantiateServer(boolean reload) throws IOException {
1547                   server = connection.getHRegionConnection(address);
1548                 }
1549               }
1550           );
1551         }
1552       };
1553     }
1554 
1555     private Throwable translateException(Throwable t) throws IOException {
1556       if (t instanceof UndeclaredThrowableException) {
1557         t = t.getCause();
1558       }
1559       if (t instanceof RemoteException) {
1560         t = RemoteExceptionHandler.decodeRemoteException((RemoteException)t);
1561       }
1562       if (t instanceof DoNotRetryIOException) {
1563         throw (DoNotRetryIOException)t;
1564       }
1565       return t;
1566     }
1567 
1568     /*
1569      * Return the number of cached region for a table. It will only be called
1570      * from a unit test.
1571      */
1572     int getNumberOfCachedRegionLocations(final byte[] tableName) {
1573       Integer key = Bytes.mapKey(tableName);
1574       synchronized (this.cachedRegionLocations) {
1575         SoftValueSortedMap<byte[], HRegionLocation> tableLocs =
1576           this.cachedRegionLocations.get(key);
1577 
1578         if (tableLocs == null) {
1579           return 0;
1580         }
1581         return tableLocs.values().size();
1582       }
1583     }
1584 
1585     /**
1586      * Check the region cache to see whether a region is cached yet or not.
1587      * Called by unit tests.
1588      * @param tableName tableName
1589      * @param row row
1590      * @return Region cached or not.
1591      */
1592     boolean isRegionCached(final byte[] tableName, final byte[] row) {
1593       HRegionLocation location = getCachedLocation(tableName, row);
1594       return location != null;
1595     }
1596 
1597     public void setRegionCachePrefetch(final byte[] tableName,
1598         final boolean enable) {
1599       if (!enable) {
1600         regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName));
1601       }
1602       else {
1603         regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName));
1604       }
1605     }
1606 
1607     public boolean getRegionCachePrefetch(final byte[] tableName) {
1608       return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName));
1609     }
1610 
1611     public void prewarmRegionCache(final byte[] tableName,
1612         final Map<HRegionInfo, HServerAddress> regions) {
1613       for (Map.Entry<HRegionInfo, HServerAddress> e : regions.entrySet()) {
1614         cacheLocation(tableName,
1615             new HRegionLocation(e.getKey(), e.getValue()));
1616       }
1617     }
1618   }
1619 }