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.IOException;
23  import java.lang.reflect.UndeclaredThrowableException;
24  import java.util.ArrayList;
25  import java.util.HashMap;
26  import java.util.LinkedHashMap;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Map.Entry;
30  import java.util.Set;
31  import java.util.TreeSet;
32  import java.util.concurrent.Callable;
33  import java.util.concurrent.ConcurrentHashMap;
34  import java.util.concurrent.CopyOnWriteArraySet;
35  import java.util.concurrent.ExecutionException;
36  import java.util.concurrent.ExecutorService;
37  import java.util.concurrent.Future;
38  import java.util.concurrent.atomic.AtomicBoolean;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.DoNotRetryIOException;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HRegionLocation;
47  import org.apache.hadoop.hbase.HServerAddress;
48  import org.apache.hadoop.hbase.HTableDescriptor;
49  import org.apache.hadoop.hbase.KeyValue;
50  import org.apache.hadoop.hbase.MasterAddressTracker;
51  import org.apache.hadoop.hbase.MasterNotRunningException;
52  import org.apache.hadoop.hbase.RemoteExceptionHandler;
53  import org.apache.hadoop.hbase.TableNotFoundException;
54  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
55  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
56  import org.apache.hadoop.hbase.ipc.HBaseRPC;
57  import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
58  import org.apache.hadoop.hbase.ipc.HMasterInterface;
59  import org.apache.hadoop.hbase.ipc.HRegionInterface;
60  import org.apache.hadoop.hbase.util.Bytes;
61  import org.apache.hadoop.hbase.util.Pair;
62  import org.apache.hadoop.hbase.util.SoftValueSortedMap;
63  import org.apache.hadoop.hbase.util.Writables;
64  import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
65  import org.apache.hadoop.hbase.zookeeper.ZKTable;
66  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
67  import org.apache.hadoop.ipc.RemoteException;
68  import org.apache.zookeeper.KeeperException;
69  
70  /**
71   * A non-instantiable class that manages {@link HConnection}s.
72   * This class has a static Map of {@link HConnection} instances keyed by
73   * {@link Configuration}; all invocations of {@link #getConnection(Configuration)}
74   * that pass the same {@link Configuration} instance will be returned the same
75   * {@link  HConnection} instance (Adding properties to a Configuration
76   * instance does not change its object identity).  Sharing {@link HConnection}
77   * instances is usually what you want; all clients of the {@link HConnection}
78   * instances share the HConnections' cache of Region locations rather than each
79   * having to discover for itself the location of meta, root, etc.  It makes
80   * sense for the likes of the pool of HTables class {@link HTablePool}, for
81   * instance (If concerned that a single {@link HConnection} is insufficient
82   * for sharing amongst clients in say an heavily-multithreaded environment,
83   * in practise its not proven to be an issue.  Besides, {@link HConnection} is
84   * implemented atop Hadoop RPC and as of this writing, Hadoop RPC does a
85   * connection per cluster-member, exclusively).
86   *
87   * <p>But sharing connections
88   * makes clean up of {@link HConnection} instances a little awkward.  Currently,
89   * clients cleanup by calling
90   * {@link #deleteConnection(Configuration, boolean)}.  This will shutdown the
91   * zookeeper connection the HConnection was using and clean up all
92   * HConnection resources as well as stopping proxies to servers out on the
93   * cluster. Not running the cleanup will not end the world; it'll
94   * just stall the closeup some and spew some zookeeper connection failed
95   * messages into the log.  Running the cleanup on a {@link HConnection} that is
96   * subsequently used by another will cause breakage so be careful running
97   * cleanup.
98   * <p>To create a {@link HConnection} that is not shared by others, you can
99   * create a new {@link Configuration} instance, pass this new instance to
100  * {@link #getConnection(Configuration)}, and then when done, close it up by
101  * doing something like the following:
102  * <pre>
103  * {@code
104  * Configuration newConfig = new Configuration(originalConf);
105  * HConnection connection = HConnectionManager.getConnection(newConfig);
106  * // Use the connection to your hearts' delight and then when done...
107  * HConnectionManager.deleteConnection(newConfig, true);
108  * }
109  * </pre>
110  * <p>Cleanup used to be done inside in a shutdown hook.  On startup we'd
111  * register a shutdown hook that called {@link #deleteAllConnections(boolean)}
112  * on its way out but the order in which shutdown hooks run is not defined so
113  * were problematic for clients of HConnection that wanted to register their
114  * own shutdown hooks so we removed ours though this shifts the onus for
115  * cleanup to the client.
116  */
117 @SuppressWarnings("serial")
118 public class HConnectionManager {
119   static final int MAX_CACHED_HBASE_INSTANCES = 31;
120 
121   // A LRU Map of Configuration hashcode -> TableServers. We set instances to 31.
122   // The zk default max connections to the ensemble from the one client is 30 so
123   // should run into zk issues before hit this value of 31.
124   private static final Map<Configuration, HConnectionImplementation> HBASE_INSTANCES =
125     new LinkedHashMap<Configuration, HConnectionImplementation>
126       ((int) (MAX_CACHED_HBASE_INSTANCES/0.75F)+1, 0.75F, true) {
127       @Override
128       protected boolean removeEldestEntry(Map.Entry<Configuration, HConnectionImplementation> eldest) {
129         return size() > MAX_CACHED_HBASE_INSTANCES;
130       }
131   };
132 
133   /*
134    * Non-instantiable.
135    */
136   protected HConnectionManager() {
137     super();
138   }
139 
140   /**
141    * Get the connection that goes with the passed <code>conf</code>
142    * configuration instance.
143    * If no current connection exists, method creates a new connection for the
144    * passed <code>conf</code> instance.
145    * @param conf configuration
146    * @return HConnection object for <code>conf</code>
147    * @throws ZooKeeperConnectionException
148    */
149   public static HConnection getConnection(Configuration conf)
150   throws ZooKeeperConnectionException {
151     HConnectionImplementation connection;
152     synchronized (HBASE_INSTANCES) {
153       connection = HBASE_INSTANCES.get(conf);
154       if (connection == null) {
155         connection = new HConnectionImplementation(conf);
156         HBASE_INSTANCES.put(conf, connection);
157       }
158     }
159     return connection;
160   }
161 
162   /**
163    * Delete connection information for the instance specified by configuration.
164    * This will close connection to the zookeeper ensemble and let go of all
165    * resources.
166    * @param conf configuration whose identity is used to find {@link HConnection}
167    * instance.
168    * @param stopProxy Shuts down all the proxy's put up to cluster members
169    * including to cluster HMaster.  Calls {@link HBaseRPC#stopProxy(org.apache.hadoop.ipc.VersionedProtocol)}.
170    */
171   public static void deleteConnection(Configuration conf, boolean stopProxy) {
172     synchronized (HBASE_INSTANCES) {
173       HConnectionImplementation t = HBASE_INSTANCES.remove(conf);
174       if (t != null) {
175         t.close(stopProxy);
176       }
177     }
178   }
179 
180   /**
181    * Delete information for all connections.
182    * @param stopProxy stop the proxy as well
183    * @throws IOException
184    */
185   public static void deleteAllConnections(boolean stopProxy) {
186     synchronized (HBASE_INSTANCES) {
187       for (HConnectionImplementation t : HBASE_INSTANCES.values()) {
188         if (t != null) {
189           t.close(stopProxy);
190         }
191       }
192     }
193   }
194 
195   /**
196    * It is provided for unit test cases which verify the behavior of region
197    * location cache prefetch.
198    * @return Number of cached regions for the table.
199    * @throws ZooKeeperConnectionException
200    */
201   static int getCachedRegionCount(Configuration conf,
202       byte[] tableName)
203   throws ZooKeeperConnectionException {
204     HConnectionImplementation connection = (HConnectionImplementation)getConnection(conf);
205     return connection.getNumberOfCachedRegionLocations(tableName);
206   }
207 
208   /**
209    * It's provided for unit test cases which verify the behavior of region
210    * location cache prefetch.
211    * @return true if the region where the table and row reside is cached.
212    * @throws ZooKeeperConnectionException
213    */
214   static boolean isRegionCached(Configuration conf,
215       byte[] tableName, byte[] row) throws ZooKeeperConnectionException {
216     HConnectionImplementation connection = (HConnectionImplementation)getConnection(conf);
217     return connection.isRegionCached(tableName, row);
218   }
219 
220   /* Encapsulates connection to zookeeper and regionservers.*/
221   static class HConnectionImplementation implements HConnection {
222     static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
223     private final Class<? extends HRegionInterface> serverInterfaceClass;
224     private final long pause;
225     private final int numRetries;
226     private final int maxRPCAttempts;
227     private final int rpcTimeout;
228     private final int prefetchRegionLimit;
229 
230     private final Object masterLock = new Object();
231     private volatile boolean closed;
232     private volatile HMasterInterface master;
233     private volatile boolean masterChecked;
234     // ZooKeeper reference
235     private ZooKeeperWatcher zooKeeper;
236     // ZooKeeper-based master address tracker
237     private MasterAddressTracker masterAddressTracker;
238     private RootRegionTracker rootRegionTracker;
239     
240     private final Object metaRegionLock = new Object();
241 
242     private final Object userRegionLock = new Object();
243 
244     private final Configuration conf;
245     // Known region HServerAddress.toString() -> HRegionInterface
246 
247     private final Map<String, HRegionInterface> servers =
248       new ConcurrentHashMap<String, HRegionInterface>();
249 
250     /**
251      * Map of table to table {@link HRegionLocation}s.  The table key is made
252      * by doing a {@link Bytes#mapKey(byte[])} of the table's name.
253      */
254     private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>>
255       cachedRegionLocations =
256         new HashMap<Integer, SoftValueSortedMap<byte [], HRegionLocation>>();
257 
258     // region cache prefetch is enabled by default. this set contains all
259     // tables whose region cache prefetch are disabled.
260     private final Set<Integer> regionCachePrefetchDisabledTables =
261       new CopyOnWriteArraySet<Integer>();
262 
263     /**
264      * constructor
265      * @param conf Configuration object
266      */
267     @SuppressWarnings("unchecked")
268     public HConnectionImplementation(Configuration conf)
269     throws ZooKeeperConnectionException {
270       this.conf = conf;
271       String serverClassName = conf.get(HConstants.REGION_SERVER_CLASS,
272         HConstants.DEFAULT_REGION_SERVER_CLASS);
273       this.closed = false;
274       try {
275         this.serverInterfaceClass =
276           (Class<? extends HRegionInterface>) Class.forName(serverClassName);
277       } catch (ClassNotFoundException e) {
278         throw new UnsupportedOperationException(
279             "Unable to find region server interface " + serverClassName, e);
280       }
281 
282       this.pause = conf.getLong("hbase.client.pause", 1000);
283       this.numRetries = conf.getInt("hbase.client.retries.number", 10);
284       this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts", 1);
285       this.rpcTimeout = conf.getInt(
286           HConstants.HBASE_RPC_TIMEOUT_KEY,
287           HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
288 
289       this.prefetchRegionLimit = conf.getInt("hbase.client.prefetch.limit",
290           10);
291 
292       setupZookeeperTrackers();
293 
294       this.master = null;
295       this.masterChecked = false;
296     }
297 
298     private synchronized void setupZookeeperTrackers()
299         throws ZooKeeperConnectionException{
300       // initialize zookeeper and master address manager
301       this.zooKeeper = getZooKeeperWatcher();
302       masterAddressTracker = new MasterAddressTracker(this.zooKeeper, this);
303       zooKeeper.registerListener(masterAddressTracker);
304       masterAddressTracker.start();
305 
306       this.rootRegionTracker = new RootRegionTracker(this.zooKeeper, this);
307       this.rootRegionTracker.start();
308     }
309 
310     private synchronized void resetZooKeeperTrackers()
311         throws ZooKeeperConnectionException {
312       LOG.info("Trying to reconnect to zookeeper");
313       masterAddressTracker.stop();
314       masterAddressTracker = null;
315       rootRegionTracker.stop();
316       rootRegionTracker = null;
317       this.zooKeeper = null;
318       setupZookeeperTrackers();
319     }
320 
321     public Configuration getConfiguration() {
322       return this.conf;
323     }
324 
325     private long getPauseTime(int tries) {
326       int ntries = tries;
327       if (ntries >= HConstants.RETRY_BACKOFF.length) {
328         ntries = HConstants.RETRY_BACKOFF.length - 1;
329       }
330       return this.pause * HConstants.RETRY_BACKOFF[ntries];
331     }
332 
333     public HMasterInterface getMaster()
334     throws MasterNotRunningException, ZooKeeperConnectionException {
335 
336       // Check if we already have a good master connection
337       if (master != null) {
338         if (master.isMasterRunning()) {
339           return master;
340         }
341       }
342 
343       HServerAddress masterLocation = null;
344       synchronized (this.masterLock) {
345         for (int tries = 0;
346           !this.closed &&
347           !this.masterChecked && this.master == null &&
348           tries < numRetries;
349         tries++) {
350 
351           try {
352             masterLocation = masterAddressTracker.getMasterAddress();
353             if(masterLocation == null) {
354               LOG.info("ZooKeeper available but no active master location found");
355               throw new MasterNotRunningException();
356             }
357 
358             HMasterInterface tryMaster = (HMasterInterface)HBaseRPC.getProxy(
359                 HMasterInterface.class, HBaseRPCProtocolVersion.versionID,
360                 masterLocation.getInetSocketAddress(), this.conf, this.rpcTimeout);
361 
362             if (tryMaster.isMasterRunning()) {
363               this.master = tryMaster;
364               this.masterLock.notifyAll();
365               break;
366             }
367 
368           } catch (IOException e) {
369             if (tries == numRetries - 1) {
370               // This was our last chance - don't bother sleeping
371               LOG.info("getMaster attempt " + tries + " of " + this.numRetries +
372                 " failed; no more retrying.", e);
373               break;
374             }
375             LOG.info("getMaster attempt " + tries + " of " + this.numRetries +
376               " failed; retrying after sleep of " +
377               getPauseTime(tries), e);
378           }
379 
380           // Cannot connect to master or it is not running. Sleep & retry
381           try {
382             this.masterLock.wait(getPauseTime(tries));
383           } catch (InterruptedException e) {
384             Thread.currentThread().interrupt();
385             throw new RuntimeException("Thread was interrupted while trying to connect to master.");
386           }
387         }
388         this.masterChecked = true;
389       }
390       if (this.master == null) {
391         if (masterLocation == null) {
392           throw new MasterNotRunningException();
393         }
394         throw new MasterNotRunningException(masterLocation.toString());
395       }
396       return this.master;
397     }
398 
399     public boolean isMasterRunning()
400     throws MasterNotRunningException, ZooKeeperConnectionException {
401       if (this.master == null) {
402         getMaster();
403       }
404       boolean isRunning = master.isMasterRunning();
405       if(isRunning) {
406         return true;
407       }
408       throw new MasterNotRunningException();
409     }
410 
411     public HRegionLocation getRegionLocation(final byte [] name,
412         final byte [] row, boolean reload)
413     throws IOException {
414       return reload? relocateRegion(name, row): locateRegion(name, row);
415     }
416 
417     public HTableDescriptor[] listTables() throws IOException {
418       final TreeSet<HTableDescriptor> uniqueTables =
419         new TreeSet<HTableDescriptor>();
420       MetaScannerVisitor visitor = new MetaScannerVisitor() {
421         public boolean processRow(Result result) throws IOException {
422           try {
423             byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
424                 HConstants.REGIONINFO_QUALIFIER);
425             HRegionInfo info = null;
426             if (value != null) {
427               info = Writables.getHRegionInfo(value);
428             }
429             // Only examine the rows where the startKey is zero length
430             if (info != null && info.getStartKey().length == 0) {
431               uniqueTables.add(info.getTableDesc());
432             }
433             return true;
434           } catch (RuntimeException e) {
435             LOG.error("Result=" + result);
436             throw e;
437           }
438         }
439       };
440       MetaScanner.metaScan(conf, visitor);
441       return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
442     }
443 
444     public boolean isTableEnabled(byte[] tableName) throws IOException {
445       return testTableOnlineState(tableName, true);
446     }
447 
448     public boolean isTableDisabled(byte[] tableName) throws IOException {
449       return testTableOnlineState(tableName, false);
450     }
451 
452     public boolean isTableAvailable(final byte[] tableName) throws IOException {
453       final AtomicBoolean available = new AtomicBoolean(true);
454       MetaScannerVisitor visitor = new MetaScannerVisitor() {
455         @Override
456         public boolean processRow(Result row) throws IOException {
457           byte[] value = row.getValue(HConstants.CATALOG_FAMILY,
458               HConstants.REGIONINFO_QUALIFIER);
459           HRegionInfo info = Writables.getHRegionInfoOrNull(value);
460           if (info != null) {
461             if (Bytes.equals(tableName, info.getTableDesc().getName())) {
462               value = row.getValue(HConstants.CATALOG_FAMILY,
463                   HConstants.SERVER_QUALIFIER);
464               if (value == null) {
465                 available.set(false);
466                 return false;
467               }
468             }
469           }
470           return true;
471         }
472       };
473       MetaScanner.metaScan(conf, visitor);
474       return available.get();
475     }
476 
477     /*
478      * @param True if table is online
479      */
480     private boolean testTableOnlineState(byte [] tableName, boolean online)
481     throws IOException {
482       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
483         // The root region is always enabled
484         return online;
485       }
486       String tableNameStr = Bytes.toString(tableName);
487       try {
488         if (online) {
489           return ZKTable.isEnabledTable(this.zooKeeper, tableNameStr);
490         }
491         return ZKTable.isDisabledTable(this.zooKeeper, tableNameStr);
492       } catch (KeeperException e) {
493         throw new IOException("Enable/Disable failed", e);
494       }
495     }
496 
497     private static class HTableDescriptorFinder
498     implements MetaScanner.MetaScannerVisitor {
499         byte[] tableName;
500         HTableDescriptor result;
501         protected HTableDescriptorFinder(byte[] tableName) {
502           this.tableName = tableName;
503         }
504         public boolean processRow(Result rowResult) throws IOException {
505           HRegionInfo info = Writables.getHRegionInfoOrNull(
506               rowResult.getValue(HConstants.CATALOG_FAMILY,
507                   HConstants.REGIONINFO_QUALIFIER));
508           if (info == null) return true;
509           HTableDescriptor desc = info.getTableDesc();
510           if (Bytes.compareTo(desc.getName(), tableName) == 0) {
511             result = desc;
512             return false;
513           }
514           return true;
515         }
516         HTableDescriptor getResult() {
517           return result;
518         }
519     }
520 
521     public HTableDescriptor getHTableDescriptor(final byte[] tableName)
522     throws IOException {
523       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
524         return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
525       }
526       if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
527         return HTableDescriptor.META_TABLEDESC;
528       }
529       HTableDescriptorFinder finder = new HTableDescriptorFinder(tableName);
530       MetaScanner.metaScan(conf, finder, tableName);
531       HTableDescriptor result = finder.getResult();
532       if (result == null) {
533         throw new TableNotFoundException(Bytes.toString(tableName));
534       }
535       return result;
536     }
537 
538     @Override
539     public HRegionLocation locateRegion(final byte [] regionName)
540     throws IOException {
541       // TODO implement.  use old stuff or new stuff?
542       return null;
543     }
544 
545     @Override
546     public List<HRegionLocation> locateRegions(final byte [] tableName)
547     throws IOException {
548       // TODO implement.  use old stuff or new stuff?
549       return null;
550     }
551 
552     public HRegionLocation locateRegion(final byte [] tableName,
553         final byte [] row)
554     throws IOException{
555       return locateRegion(tableName, row, true);
556     }
557 
558     public HRegionLocation relocateRegion(final byte [] tableName,
559         final byte [] row)
560     throws IOException{
561       return locateRegion(tableName, row, false);
562     }
563 
564     private HRegionLocation locateRegion(final byte [] tableName,
565       final byte [] row, boolean useCache)
566     throws IOException {
567       if (this.closed) throw new IOException(toString() + " closed");
568       if (tableName == null || tableName.length == 0) {
569         throw new IllegalArgumentException(
570             "table name cannot be null or zero length");
571       }
572 
573       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
574         try {
575           HServerAddress hsa =
576             this.rootRegionTracker.waitRootRegionLocation(this.rpcTimeout);
577           LOG.debug("Lookedup root region location, connection=" + this +
578             "; hsa=" + hsa);
579           if (hsa == null) return null;
580           return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa);
581         } catch (InterruptedException e) {
582           Thread.currentThread().interrupt();
583           return null;
584         }
585       } else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
586         return locateRegionInMeta(HConstants.ROOT_TABLE_NAME, tableName, row,
587             useCache, metaRegionLock);
588       } else {
589         // Region not in the cache - have to go to the meta RS
590         return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
591             useCache, userRegionLock);
592       }
593     }
594 
595     /*
596      * Search .META. for the HRegionLocation info that contains the table and
597      * row we're seeking. It will prefetch certain number of regions info and
598      * save them to the global region cache.
599      */
600     private void prefetchRegionCache(final byte[] tableName,
601         final byte[] row) {
602       // Implement a new visitor for MetaScanner, and use it to walk through
603       // the .META.
604       MetaScannerVisitor visitor = new MetaScannerVisitor() {
605         public boolean processRow(Result result) throws IOException {
606           try {
607             byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
608                 HConstants.REGIONINFO_QUALIFIER);
609             HRegionInfo regionInfo = null;
610 
611             if (value != null) {
612               // convert the row result into the HRegionLocation we need!
613               regionInfo = Writables.getHRegionInfo(value);
614 
615               // possible we got a region of a different table...
616               if (!Bytes.equals(regionInfo.getTableDesc().getName(),
617                   tableName)) {
618                 return false; // stop scanning
619               }
620               if (regionInfo.isOffline()) {
621                 // don't cache offline regions
622                 return true;
623               }
624               value = result.getValue(HConstants.CATALOG_FAMILY,
625                   HConstants.SERVER_QUALIFIER);
626               if (value == null) {
627                 return true;  // don't cache it
628               }
629               final String serverAddress = Bytes.toString(value);
630 
631               // instantiate the location
632               HRegionLocation loc = new HRegionLocation(regionInfo,
633                 new HServerAddress(serverAddress));
634               // cache this meta entry
635               cacheLocation(tableName, loc);
636             }
637             return true;
638           } catch (RuntimeException e) {
639             throw new IOException(e);
640           }
641         }
642       };
643       try {
644         // pre-fetch certain number of regions info at region cache.
645         MetaScanner.metaScan(conf, visitor, tableName, row,
646             this.prefetchRegionLimit);
647       } catch (IOException e) {
648         LOG.warn("Encountered problems when prefetch META table: ", e);
649       }
650     }
651 
652     /*
653       * Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation
654       * info that contains the table and row we're seeking.
655       */
656     private HRegionLocation locateRegionInMeta(final byte [] parentTable,
657       final byte [] tableName, final byte [] row, boolean useCache,
658       Object regionLockObject)
659     throws IOException {
660       HRegionLocation location;
661       // If we are supposed to be using the cache, look in the cache to see if
662       // we already have the region.
663       if (useCache) {
664         location = getCachedLocation(tableName, row);
665         if (location != null) {
666           return location;
667         }
668       }
669 
670       // build the key of the meta region we should be looking for.
671       // the extra 9's on the end are necessary to allow "exact" matches
672       // without knowing the precise region names.
673       byte [] metaKey = HRegionInfo.createRegionName(tableName, row,
674         HConstants.NINES, false);
675       for (int tries = 0; true; tries++) {
676         if (tries >= numRetries) {
677           throw new NoServerForRegionException("Unable to find region for "
678             + Bytes.toStringBinary(row) + " after " + numRetries + " tries.");
679         }
680 
681         HRegionLocation metaLocation = null;
682         try {
683           // locate the root or meta region
684           metaLocation = locateRegion(parentTable, metaKey);
685           // If null still, go around again.
686           if (metaLocation == null) continue;
687           HRegionInterface server =
688             getHRegionConnection(metaLocation.getServerAddress());
689 
690           Result regionInfoRow = null;
691           // This block guards against two threads trying to load the meta
692           // region at the same time. The first will load the meta region and
693           // the second will use the value that the first one found.
694           synchronized (regionLockObject) {
695             // If the parent table is META, we may want to pre-fetch some
696             // region info into the global region cache for this table.
697             if (Bytes.equals(parentTable, HConstants.META_TABLE_NAME) &&
698                 (getRegionCachePrefetch(tableName)) )  {
699               prefetchRegionCache(tableName, row);
700             }
701 
702             // Check the cache again for a hit in case some other thread made the
703             // same query while we were waiting on the lock. If not supposed to
704             // be using the cache, delete any existing cached location so it won't
705             // interfere.
706             if (useCache) {
707               location = getCachedLocation(tableName, row);
708               if (location != null) {
709                 return location;
710               }
711             } else {
712               deleteCachedLocation(tableName, row);
713             }
714 
715           // Query the root or meta region for the location of the meta region
716             regionInfoRow = server.getClosestRowBefore(
717             metaLocation.getRegionInfo().getRegionName(), metaKey,
718             HConstants.CATALOG_FAMILY);
719           }
720           if (regionInfoRow == null) {
721             throw new TableNotFoundException(Bytes.toString(tableName));
722           }
723           byte[] value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
724               HConstants.REGIONINFO_QUALIFIER);
725           if (value == null || value.length == 0) {
726             throw new IOException("HRegionInfo was null or empty in " +
727               Bytes.toString(parentTable) + ", row=" + regionInfoRow);
728           }
729           // convert the row result into the HRegionLocation we need!
730           HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
731               value, new HRegionInfo());
732           // possible we got a region of a different table...
733           if (!Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) {
734             throw new TableNotFoundException(
735               "Table '" + Bytes.toString(tableName) + "' was not found.");
736           }
737           if (regionInfo.isOffline()) {
738             throw new RegionOfflineException("region offline: " +
739               regionInfo.getRegionNameAsString());
740           }
741 
742           value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
743               HConstants.SERVER_QUALIFIER);
744           String serverAddress = "";
745           if(value != null) {
746             serverAddress = Bytes.toString(value);
747           }
748           if (serverAddress.equals("")) {
749             throw new NoServerForRegionException("No server address listed " +
750               "in " + Bytes.toString(parentTable) + " for region " +
751               regionInfo.getRegionNameAsString());
752           }
753 
754           // instantiate the location
755           location = new HRegionLocation(regionInfo,
756             new HServerAddress(serverAddress));
757           cacheLocation(tableName, location);
758           return location;
759         } catch (TableNotFoundException e) {
760           // if we got this error, probably means the table just plain doesn't
761           // exist. rethrow the error immediately. this should always be coming
762           // from the HTable constructor.
763           throw e;
764         } catch (IOException e) {
765           if (e instanceof RemoteException) {
766             e = RemoteExceptionHandler.decodeRemoteException(
767                 (RemoteException) e);
768           }
769           if (tries < numRetries - 1) {
770             if (LOG.isDebugEnabled()) {
771               LOG.debug("locateRegionInMeta parentTable=" +
772                 Bytes.toString(parentTable) + ", metaLocation=" +
773                 ((metaLocation == null)? "null": metaLocation) + ", attempt=" +
774                 tries + " of " +
775                 this.numRetries + " failed; retrying after sleep of " +
776                 getPauseTime(tries) + " because: " + e.getMessage());
777             }
778           } else {
779             throw e;
780           }
781           // Only relocate the parent region if necessary
782           if(!(e instanceof RegionOfflineException ||
783               e instanceof NoServerForRegionException)) {
784             relocateRegion(parentTable, metaKey);
785           }
786         }
787         try{
788           Thread.sleep(getPauseTime(tries));
789         } catch (InterruptedException e) {
790           Thread.currentThread().interrupt();
791           throw new IOException("Giving up trying to location region in " +
792             "meta: thread is interrupted.");
793         }
794       }
795     }
796 
797     /*
798      * Search the cache for a location that fits our table and row key.
799      * Return null if no suitable region is located. TODO: synchronization note
800      *
801      * <p>TODO: This method during writing consumes 15% of CPU doing lookup
802      * into the Soft Reference SortedMap.  Improve.
803      *
804      * @param tableName
805      * @param row
806      * @return Null or region location found in cache.
807      */
808     HRegionLocation getCachedLocation(final byte [] tableName,
809         final byte [] row) {
810       SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
811         getTableLocations(tableName);
812 
813       // start to examine the cache. we can only do cache actions
814       // if there's something in the cache for this table.
815       if (tableLocations.isEmpty()) {
816         return null;
817       }
818 
819       HRegionLocation rl = tableLocations.get(row);
820       if (rl != null) {
821         if (LOG.isDebugEnabled()) {
822           LOG.debug("Cache hit for row <" +
823             Bytes.toStringBinary(row) +
824             "> in tableName " + Bytes.toString(tableName) +
825             ": location server " + rl.getServerAddress() +
826             ", location region name " +
827             rl.getRegionInfo().getRegionNameAsString());
828         }
829         return rl;
830       }
831 
832       // Cut the cache so that we only get the part that could contain
833       // regions that match our key
834       SoftValueSortedMap<byte[], HRegionLocation> matchingRegions =
835         tableLocations.headMap(row);
836 
837       // if that portion of the map is empty, then we're done. otherwise,
838       // we need to examine the cached location to verify that it is
839       // a match by end key as well.
840       if (!matchingRegions.isEmpty()) {
841         HRegionLocation possibleRegion =
842           matchingRegions.get(matchingRegions.lastKey());
843 
844         // there is a possibility that the reference was garbage collected
845         // in the instant since we checked isEmpty().
846         if (possibleRegion != null) {
847           byte[] endKey = possibleRegion.getRegionInfo().getEndKey();
848 
849           // make sure that the end key is greater than the row we're looking
850           // for, otherwise the row actually belongs in the next region, not
851           // this one. the exception case is when the endkey is
852           // HConstants.EMPTY_START_ROW, signifying that the region we're
853           // checking is actually the last region in the table.
854           if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
855               KeyValue.getRowComparator(tableName).compareRows(endKey, 0, endKey.length,
856                   row, 0, row.length) > 0) {
857             return possibleRegion;
858           }
859         }
860       }
861 
862       // Passed all the way through, so we got nothin - complete cache miss
863       return null;
864     }
865 
866     /**
867      * Delete a cached location
868      * @param tableName tableName
869      * @param row
870      */
871     void deleteCachedLocation(final byte [] tableName, final byte [] row) {
872       synchronized (this.cachedRegionLocations) {
873         SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
874             getTableLocations(tableName);
875         // start to examine the cache. we can only do cache actions
876         // if there's something in the cache for this table.
877         if (!tableLocations.isEmpty()) {
878           HRegionLocation rl = getCachedLocation(tableName, row);
879           if (rl != null) {
880             tableLocations.remove(rl.getRegionInfo().getStartKey());
881             if (LOG.isDebugEnabled()) {
882               LOG.debug("Removed " +
883                 rl.getRegionInfo().getRegionNameAsString() +
884                 " for tableName=" + Bytes.toString(tableName) +
885                 " from cache " + "because of " + Bytes.toStringBinary(row));
886             }
887           }
888         }
889       }
890     }
891 
892     /*
893      * @param tableName
894      * @return Map of cached locations for passed <code>tableName</code>
895      */
896     private SoftValueSortedMap<byte [], HRegionLocation> getTableLocations(
897         final byte [] tableName) {
898       // find the map of cached locations for this table
899       Integer key = Bytes.mapKey(tableName);
900       SoftValueSortedMap<byte [], HRegionLocation> result;
901       synchronized (this.cachedRegionLocations) {
902         result = this.cachedRegionLocations.get(key);
903         // if tableLocations for this table isn't built yet, make one
904         if (result == null) {
905           result = new SoftValueSortedMap<byte [], HRegionLocation>(
906               Bytes.BYTES_COMPARATOR);
907           this.cachedRegionLocations.put(key, result);
908         }
909       }
910       return result;
911     }
912 
913     @Override
914     public void clearRegionCache() {
915       synchronized(this.cachedRegionLocations) {
916         this.cachedRegionLocations.clear();
917       }
918     }
919 
920     @Override
921     public void clearRegionCache(final byte [] tableName) {
922       synchronized (this.cachedRegionLocations) {
923         this.cachedRegionLocations.remove(Bytes.mapKey(tableName));
924       }
925     }
926 
927     /*
928      * Put a newly discovered HRegionLocation into the cache.
929      */
930     private void cacheLocation(final byte [] tableName,
931         final HRegionLocation location) {
932       byte [] startKey = location.getRegionInfo().getStartKey();
933       SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
934         getTableLocations(tableName);
935       if (tableLocations.put(startKey, location) == null) {
936         LOG.debug("Cached location for " +
937             location.getRegionInfo().getRegionNameAsString() +
938             " is " + location.getServerAddress());
939       }
940     }
941 
942     public HRegionInterface getHRegionConnection(
943         HServerAddress regionServer, boolean getMaster)
944     throws IOException {
945       if (getMaster) {
946         getMaster();
947       }
948       HRegionInterface server;
949       synchronized (this.servers) {
950         // See if we already have a connection
951         server = this.servers.get(regionServer.toString());
952         if (server == null) { // Get a connection
953           try {
954             server = (HRegionInterface)HBaseRPC.waitForProxy(
955                 serverInterfaceClass, HBaseRPCProtocolVersion.versionID,
956                 regionServer.getInetSocketAddress(), this.conf,
957                 this.maxRPCAttempts, this.rpcTimeout, this.rpcTimeout);
958           } catch (RemoteException e) {
959             LOG.warn("RemoteException connecting to RS", e);
960             // Throw what the RemoteException was carrying.
961             throw RemoteExceptionHandler.decodeRemoteException(e);
962           }
963           this.servers.put(regionServer.toString(), server);
964         }
965       }
966       return server;
967     }
968 
969     public HRegionInterface getHRegionConnection(
970         HServerAddress regionServer)
971     throws IOException {
972       return getHRegionConnection(regionServer, false);
973     }
974 
975     /**
976      * Get the ZooKeeper instance for this TableServers instance.
977      *
978      * If ZK has not been initialized yet, this will connect to ZK.
979      * @returns zookeeper reference
980      * @throws ZooKeeperConnectionException if there's a problem connecting to zk
981      */
982     public synchronized ZooKeeperWatcher getZooKeeperWatcher()
983         throws ZooKeeperConnectionException {
984       if(zooKeeper == null) {
985         try {
986           this.zooKeeper = new ZooKeeperWatcher(conf, "hconnection", this);
987         } catch (IOException e) {
988           throw new ZooKeeperConnectionException(e);
989         }
990       }
991       return zooKeeper;
992     }
993 
994     public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
995     throws IOException, RuntimeException {
996       List<Throwable> exceptions = new ArrayList<Throwable>();
997       for(int tries = 0; tries < numRetries; tries++) {
998         try {
999           callable.instantiateServer(tries != 0);
1000           return callable.call();
1001         } catch (Throwable t) {
1002           t = translateException(t);
1003           exceptions.add(t);
1004           if (tries == numRetries - 1) {
1005             throw new RetriesExhaustedException(callable.getServerName(),
1006                 callable.getRegionName(), callable.getRow(), tries, exceptions);
1007           }
1008         }
1009         try {
1010           Thread.sleep(getPauseTime(tries));
1011         } catch (InterruptedException e) {
1012           Thread.currentThread().interrupt();
1013           throw new IOException("Giving up trying to get region server: thread is interrupted.");
1014         }
1015       }
1016       return null;
1017     }
1018 
1019     public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
1020         throws IOException, RuntimeException {
1021       try {
1022         callable.instantiateServer(false);
1023         return callable.call();
1024       } catch (Throwable t) {
1025         Throwable t2 = translateException(t);
1026         if (t2 instanceof IOException) {
1027           throw (IOException)t2;
1028         } else {
1029           throw new RuntimeException(t2);
1030         }
1031       }
1032     }
1033 
1034     void close(boolean stopProxy) {
1035       if (master != null) {
1036         if (stopProxy) {
1037           HBaseRPC.stopProxy(master);
1038         }
1039         master = null;
1040         masterChecked = false;
1041       }
1042       if (stopProxy) {
1043         for (HRegionInterface i: servers.values()) {
1044           HBaseRPC.stopProxy(i);
1045         }
1046       }
1047       if (this.zooKeeper != null) {
1048         LOG.info("Closed zookeeper sessionid=0x" +
1049           Long.toHexString(this.zooKeeper.getZooKeeper().getSessionId()));
1050         this.zooKeeper.close();
1051         this.zooKeeper = null;
1052       }
1053       this.closed = true;
1054     }
1055 
1056     private Callable<MultiResponse> createCallable(
1057         final HServerAddress address,
1058         final MultiAction multi,
1059         final byte [] tableName) {
1060       final HConnection connection = this;
1061       return new Callable<MultiResponse>() {
1062         public MultiResponse call() throws IOException {
1063           return getRegionServerWithoutRetries(
1064               new ServerCallable<MultiResponse>(connection, tableName, null) {
1065                 public MultiResponse call() throws IOException {
1066                   return server.multi(multi);
1067                 }
1068                 @Override
1069                 public void instantiateServer(boolean reload) throws IOException {
1070                   server = connection.getHRegionConnection(address);
1071                 }
1072               }
1073           );
1074         }
1075       };
1076     }
1077 
1078     public void processBatch(List<Row> list,
1079         final byte[] tableName,
1080         ExecutorService pool,
1081         Object[] results) throws IOException, InterruptedException {
1082 
1083       // results must be the same size as list
1084       if (results.length != list.size()) {
1085         throw new IllegalArgumentException("argument results must be the same size as argument list");
1086       }
1087 
1088       if (list.size() == 0) {
1089         return;
1090       }
1091 
1092       // Keep track of the most recent servers for any given item for better
1093       // exceptional reporting.
1094       HServerAddress [] lastServers = new HServerAddress[results.length];
1095       List<Row> workingList = new ArrayList<Row>(list);
1096       boolean retry = true;
1097       Throwable singleRowCause = null;
1098 
1099       for (int tries = 0; tries < numRetries && retry; ++tries) {
1100 
1101         // sleep first, if this is a retry
1102         if (tries >= 1) {
1103           long sleepTime = getPauseTime(tries);
1104           LOG.debug("Retry " +tries+ ", sleep for " +sleepTime+ "ms!");
1105           Thread.sleep(sleepTime);
1106         }
1107 
1108         // step 1: break up into regionserver-sized chunks and build the data structs
1109 
1110         Map<HServerAddress, MultiAction> actionsByServer = new HashMap<HServerAddress, MultiAction>();
1111         for (int i = 0; i < workingList.size(); i++) {
1112           Row row = workingList.get(i);
1113           if (row != null) {
1114             HRegionLocation loc = locateRegion(tableName, row.getRow(), true);
1115             HServerAddress address = loc.getServerAddress();
1116             byte[] regionName = loc.getRegionInfo().getRegionName();
1117 
1118             MultiAction actions = actionsByServer.get(address);
1119             if (actions == null) {
1120               actions = new MultiAction();
1121               actionsByServer.put(address, actions);
1122             }
1123 
1124             Action action = new Action(regionName, row, i);
1125             lastServers[i] = address;
1126             actions.add(regionName, action);
1127           }
1128         }
1129 
1130         // step 2: make the requests
1131 
1132         Map<HServerAddress,Future<MultiResponse>> futures =
1133             new HashMap<HServerAddress, Future<MultiResponse>>(actionsByServer.size());
1134 
1135         for (Entry<HServerAddress, MultiAction> e : actionsByServer.entrySet()) {
1136           futures.put(e.getKey(), pool.submit(createCallable(e.getKey(), e.getValue(), tableName)));
1137         }
1138 
1139         // step 3: collect the failures and successes and prepare for retry
1140 
1141         for (Entry<HServerAddress, Future<MultiResponse>> responsePerServer : futures.entrySet()) {
1142           HServerAddress address = responsePerServer.getKey();
1143 
1144           try {
1145             Future<MultiResponse> future = responsePerServer.getValue();
1146             MultiResponse resp = future.get();
1147 
1148             if (resp == null) {
1149               // Entire server failed
1150               LOG.debug("Failed all for server: " + address + ", removing from cache");
1151               continue;
1152             }
1153 
1154             for (Entry<byte[], List<Pair<Integer,Object>>> e : resp.getResults().entrySet()) {
1155               byte[] regionName = e.getKey();
1156               List<Pair<Integer, Object>> regionResults = e.getValue();
1157               for (Pair<Integer, Object> regionResult : regionResults) {
1158                 if (regionResult == null) {
1159                   // if the first/only record is 'null' the entire region failed.
1160                   LOG.debug("Failures for region: " +
1161                       Bytes.toStringBinary(regionName) +
1162                       ", removing from cache");
1163                 } else {
1164                   // Result might be an Exception, including DNRIOE
1165                   results[regionResult.getFirst()] = regionResult.getSecond();
1166                 }
1167               }
1168             }
1169           } catch (ExecutionException e) {
1170             LOG.debug("Failed all from " + address, e);
1171           }
1172         }
1173 
1174         // step 4: identify failures and prep for a retry (if applicable).
1175 
1176         // Find failures (i.e. null Result), and add them to the workingList (in
1177         // order), so they can be retried.
1178         retry = false;
1179         workingList.clear();
1180         for (int i = 0; i < results.length; i++) {
1181           // if null (fail) or instanceof Throwable && not instanceof DNRIOE
1182           // then retry that row. else dont.
1183           if (results[i] == null ||
1184               (results[i] instanceof Throwable &&
1185                   !(results[i] instanceof DoNotRetryIOException))) {
1186 
1187             retry = true;
1188 
1189             Row row = list.get(i);
1190             workingList.add(row);
1191             deleteCachedLocation(tableName, row.getRow());
1192           } else {
1193             // add null to workingList, so the order remains consistent with the original list argument.
1194             workingList.add(null);
1195           }
1196         }
1197       }
1198 
1199       if (retry) {
1200         // Simple little check for 1 item failures.
1201         if (singleRowCause != null) {
1202           throw new IOException(singleRowCause);
1203         }
1204       }
1205 
1206 
1207       List<Throwable> exceptions = new ArrayList<Throwable>();
1208       List<Row> actions = new ArrayList<Row>();
1209       List<HServerAddress> addresses = new ArrayList<HServerAddress>();
1210 
1211       for (int i = 0 ; i < results.length; i++) {
1212         if (results[i] == null || results[i] instanceof Throwable) {
1213           exceptions.add((Throwable)results[i]);
1214           actions.add(list.get(i));
1215           addresses.add(lastServers[i]);
1216         }
1217       }
1218 
1219       if (!exceptions.isEmpty()) {
1220         throw new RetriesExhaustedWithDetailsException(exceptions,
1221             actions,
1222             addresses);
1223       }
1224     }
1225 
1226     /**
1227      * @deprecated Use HConnectionManager::processBatch instead.
1228      */
1229     public void processBatchOfPuts(List<Put> list,
1230         final byte[] tableName,
1231         ExecutorService pool) throws IOException {
1232       Object[] results = new Object[list.size()];
1233       try {
1234         processBatch((List) list, tableName, pool, results);
1235       } catch (InterruptedException e) {
1236         throw new IOException(e);
1237       } finally {
1238 
1239         // mutate list so that it is empty for complete success, or contains only failed records
1240         // results are returned in the same order as the requests in list
1241         // walk the list backwards, so we can remove from list without impacting the indexes of earlier members
1242         for (int i = results.length - 1; i>=0; i--) {
1243           if (results[i] instanceof Result) {
1244             // successful Puts are removed from the list here.
1245             list.remove(i);
1246           }
1247         }
1248       }
1249     }
1250 
1251     private Throwable translateException(Throwable t) throws IOException {
1252       if (t instanceof UndeclaredThrowableException) {
1253         t = t.getCause();
1254       }
1255       if (t instanceof RemoteException) {
1256         t = RemoteExceptionHandler.decodeRemoteException((RemoteException)t);
1257       }
1258       if (t instanceof DoNotRetryIOException) {
1259         throw (DoNotRetryIOException)t;
1260       }
1261       return t;
1262     }
1263 
1264     /*
1265      * Return the number of cached region for a table. It will only be called
1266      * from a unit test.
1267      */
1268     int getNumberOfCachedRegionLocations(final byte[] tableName) {
1269       Integer key = Bytes.mapKey(tableName);
1270       synchronized (this.cachedRegionLocations) {
1271         SoftValueSortedMap<byte[], HRegionLocation> tableLocs =
1272           this.cachedRegionLocations.get(key);
1273 
1274         if (tableLocs == null) {
1275           return 0;
1276         }
1277         return tableLocs.values().size();
1278       }
1279     }
1280 
1281     /**
1282      * Check the region cache to see whether a region is cached yet or not.
1283      * Called by unit tests.
1284      * @param tableName tableName
1285      * @param row row
1286      * @return Region cached or not.
1287      */
1288     boolean isRegionCached(final byte[] tableName, final byte[] row) {
1289       HRegionLocation location = getCachedLocation(tableName, row);
1290       return location != null;
1291     }
1292 
1293     public void setRegionCachePrefetch(final byte[] tableName,
1294         final boolean enable) {
1295       if (!enable) {
1296         regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName));
1297       }
1298       else {
1299         regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName));
1300       }
1301     }
1302 
1303     public boolean getRegionCachePrefetch(final byte[] tableName) {
1304       return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName));
1305     }
1306 
1307     public void prewarmRegionCache(final byte[] tableName,
1308         final Map<HRegionInfo, HServerAddress> regions) {
1309       for (Map.Entry<HRegionInfo, HServerAddress> e : regions.entrySet()) {
1310         cacheLocation(tableName,
1311             new HRegionLocation(e.getKey(), e.getValue()));
1312       }
1313     }
1314 
1315     @Override
1316     public void abort(final String msg, Throwable t) {
1317       if (t instanceof KeeperException.SessionExpiredException) {
1318         try {
1319           LOG.info("This client just lost it's session with ZooKeeper, trying" +
1320               " to reconnect.");
1321           resetZooKeeperTrackers();
1322           LOG.info("Reconnected successfully. This disconnect could have been" +
1323               " caused by a network partition or a long-running GC pause," +
1324               " either way it's recommended that you verify your environment.");
1325           return;
1326         } catch (ZooKeeperConnectionException e) {
1327           LOG.error("Could not reconnect to ZooKeeper after session" +
1328               " expiration, aborting");
1329           t = e;
1330         }
1331       }
1332       if (t != null) LOG.fatal(msg, t);
1333       else LOG.fatal(msg);
1334       this.closed = true;
1335     }
1336   }
1337 }