View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.catalog;
19  
20  import java.io.EOFException;
21  import java.io.IOException;
22  import java.net.ConnectException;
23  import java.net.NoRouteToHostException;
24  import java.net.SocketException;
25  import java.net.SocketTimeoutException;
26  import java.net.UnknownHostException;
27  import java.util.concurrent.atomic.AtomicBoolean;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.Abortable;
33  import org.apache.hadoop.hbase.HRegionInfo;
34  import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
35  import org.apache.hadoop.hbase.ServerName;
36  import org.apache.hadoop.hbase.client.HConnection;
37  import org.apache.hadoop.hbase.client.HConnectionManager;
38  import org.apache.hadoop.hbase.client.RetriesExhaustedException;
39  import org.apache.hadoop.hbase.ipc.HBaseClient.FailedServerException;
40  import org.apache.hadoop.hbase.ipc.HRegionInterface;
41  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
42  import org.apache.hadoop.hbase.util.Bytes;
43  import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker;
44  import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
45  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
46  import org.apache.hadoop.ipc.RemoteException;
47  
48  /**
49   * Tracks the availability of the catalog tables <code>-ROOT-</code> and
50   * <code>.META.</code>.
51   * 
52   * This class is "read-only" in that the locations of the catalog tables cannot
53   * be explicitly set.  Instead, ZooKeeper is used to learn of the availability
54   * and location of <code>-ROOT-</code>.  <code>-ROOT-</code> is used to learn of
55   * the location of <code>.META.</code>  If not available in <code>-ROOT-</code>,
56   * ZooKeeper is used to monitor for a new location of <code>.META.</code>.
57   *
58   * <p>Call {@link #start()} to start up operation.  Call {@link #stop()}} to
59   * interrupt waits and close up shop.
60   */
61  public class CatalogTracker {
62    // TODO: This class needs a rethink.  The original intent was that it would be
63    // the one-stop-shop for root and meta locations and that it would get this
64    // info from reading and watching zk state.  The class was to be used by
65    // servers when they needed to know of root and meta movement but also by
66    // client-side (inside in HTable) so rather than figure root and meta
67    // locations on fault, the client would instead get notifications out of zk.
68    // 
69    // But this original intent is frustrated by the fact that this class has to
70    // read an hbase table, the -ROOT- table, to figure out the .META. region
71    // location which means we depend on an HConnection.  HConnection will do
72    // retrying but also, it has its own mechanism for finding root and meta
73    // locations (and for 'verifying'; it tries the location and if it fails, does
74    // new lookup, etc.).  So, at least for now, HConnection (or HTable) can't
75    // have a CT since CT needs a HConnection (Even then, do want HT to have a CT?
76    // For HT keep up a session with ZK?  Rather, shouldn't we do like asynchbase
77    // where we'd open a connection to zk, read what we need then let the
78    // connection go?).  The 'fix' is make it so both root and meta addresses
79    // are wholey up in zk -- not in zk (root) -- and in an hbase table (meta).
80    //
81    // But even then, this class does 'verification' of the location and it does
82    // this by making a call over an HConnection (which will do its own root
83    // and meta lookups).  Isn't this verification 'useless' since when we
84    // return, whatever is dependent on the result of this call then needs to
85    // use HConnection; what we have verified may change in meantime (HConnection
86    // uses the CT primitives, the root and meta trackers finding root locations).
87    //
88    // When meta is moved to zk, this class may make more sense.  In the
89    // meantime, it does not cohere.  It should just watch meta and root and not
90    // NOT do verification -- let that be out in HConnection since its going to
91    // be done there ultimately anyways.
92    //
93    // This class has spread throughout the codebase.  It needs to be reigned in.
94    // This class should be used server-side only, even if we move meta location
95    // up into zk.  Currently its used over in the client package. Its used in
96    // MetaReader and MetaEditor classes usually just to get the Configuration
97    // its using (It does this indirectly by asking its HConnection for its
98    // Configuration and even then this is just used to get an HConnection out on
99    // the other end). I made https://issues.apache.org/jira/browse/HBASE-4495 for
100   // doing CT fixup. St.Ack 09/30/2011.
101   //
102 
103   // TODO: Timeouts have never been as advertised in here and its worse now
104   // with retries; i.e. the HConnection retries and pause goes ahead whatever
105   // the passed timeout is.  Fix.
106   private static final Log LOG = LogFactory.getLog(CatalogTracker.class);
107   private final HConnection connection;
108   private final ZooKeeperWatcher zookeeper;
109   private final RootRegionTracker rootRegionTracker;
110   private final MetaNodeTracker metaNodeTracker;
111   private final AtomicBoolean metaAvailable = new AtomicBoolean(false);
112   private boolean instantiatedzkw = false;
113   private Abortable abortable;
114 
115   /*
116    * Do not clear this address once set.  Its needed when we do
117    * server shutdown processing -- we need to know who had .META. last.  If you
118    * want to know if the address is good, rely on {@link #metaAvailable} value.
119    */
120   private ServerName metaLocation;
121 
122   private volatile boolean stopped = false;
123 
124   static final byte [] ROOT_REGION_NAME =
125     HRegionInfo.ROOT_REGIONINFO.getRegionName();
126   static final byte [] META_REGION_NAME =
127     HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
128 
129   /**
130    * Constructs a catalog tracker. Find current state of catalog tables.
131    * Begin active tracking by executing {@link #start()} post construction. Does
132    * not timeout.
133    *
134    * @param conf
135    *          the {@link Configuration} from which a {@link HConnection} will be
136    *          obtained; if problem, this connections
137    *          {@link HConnection#abort(String, Throwable)} will be called.
138    * @throws IOException
139    */
140   public CatalogTracker(final Configuration conf) throws IOException {
141     this(null, conf, null);
142   }
143 
144   /**
145    * Constructs the catalog tracker.  Find current state of catalog tables.
146    * Begin active tracking by executing {@link #start()} post construction.
147    * Does not timeout.
148    * @param zk If zk is null, we'll create an instance (and shut it down
149    * when {@link #stop()} is called) else we'll use what is passed.
150    * @param conf
151    * @param abortable If fatal exception we'll call abort on this.  May be null.
152    * If it is we'll use the Connection associated with the passed
153    * {@link Configuration} as our Abortable.
154    * @throws IOException 
155    */
156   public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
157       Abortable abortable)
158   throws IOException {
159     this(zk, conf, HConnectionManager.getConnection(conf), abortable);
160   }
161 
162   CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
163       HConnection connection, Abortable abortable)
164   throws IOException {
165     this.connection = connection;
166     if (abortable == null) {
167       // A connection is abortable.
168       this.abortable = this.connection;
169     }
170     Abortable throwableAborter = new Abortable() {
171 
172       @Override
173       public void abort(String why, Throwable e) {
174         throw new RuntimeException(why, e);
175       }
176 
177       @Override
178       public boolean isAborted() {
179         return true;
180       }
181 
182     };
183     if (zk == null) {
184       // Create our own.  Set flag so we tear it down on stop.
185       this.zookeeper =
186         new ZooKeeperWatcher(conf, "catalogtracker-on-" + connection.toString(),
187           abortable);
188       instantiatedzkw = true;
189     } else {
190       this.zookeeper = zk;
191     }
192     this.rootRegionTracker = new RootRegionTracker(zookeeper, throwableAborter);
193     final CatalogTracker ct = this;
194     // Override nodeDeleted so we get notified when meta node deleted
195     this.metaNodeTracker = new MetaNodeTracker(zookeeper, throwableAborter) {
196       public void nodeDeleted(String path) {
197         if (!path.equals(node)) return;
198         ct.resetMetaLocation();
199       }
200     };
201   }
202 
203   /**
204    * Starts the catalog tracker.
205    * Determines current availability of catalog tables and ensures all further
206    * transitions of either region are tracked.
207    * @throws IOException
208    * @throws InterruptedException 
209    */
210   public void start() throws IOException, InterruptedException {
211     LOG.debug("Starting catalog tracker " + this);
212     try {
213       this.rootRegionTracker.start();
214       this.metaNodeTracker.start();
215     } catch (RuntimeException e) {
216       Throwable t = e.getCause();
217       this.abortable.abort(e.getMessage(), t);
218       throw new IOException("Attempt to start root/meta tracker failed.", t);
219     }
220   }
221 
222   /**
223    * Stop working.
224    * Interrupts any ongoing waits.
225    */
226   public void stop() {
227     if (!this.stopped) {
228       LOG.debug("Stopping catalog tracker " + this);
229       this.stopped = true;
230       this.rootRegionTracker.stop();
231       this.metaNodeTracker.stop();
232       try {
233         if (this.connection != null) {
234           this.connection.close();
235         }
236       } catch (IOException e) {
237         // Although the {@link Closeable} interface throws an {@link
238         // IOException}, in reality, the implementation would never do that.
239         LOG.error("Attempt to close catalog tracker's connection failed.", e);
240       }
241       if (this.instantiatedzkw) {
242         this.zookeeper.close();
243       }
244       // Call this and it will interrupt any ongoing waits on meta.
245       synchronized (this.metaAvailable) {
246         this.metaAvailable.notifyAll();
247       }
248     }
249   }
250 
251   /**
252    * Gets the current location for <code>-ROOT-</code> or null if location is
253    * not currently available.
254    * @return {@link ServerName} for server hosting <code>-ROOT-</code> or null
255    * if none available
256    * @throws InterruptedException 
257    */
258   public ServerName getRootLocation() throws InterruptedException {
259     return this.rootRegionTracker.getRootRegionLocation();
260   }
261 
262   /**
263    * @return {@link ServerName} for server hosting <code>.META.</code> or null
264    * if none available
265    */
266   public ServerName getMetaLocation() {
267     return this.metaLocation;
268   }
269 
270   /**
271    * Method used by master on startup trying to figure state of cluster.
272    * Returns the current meta location unless its null.  In this latter case,
273    * it has not yet been set so go check whats up in <code>-ROOT-</code> and
274    * return that.
275    * @return {@link ServerName} for server hosting <code>.META.</code> or if null,
276    * we'll read the location that is up in <code>-ROOT-</code> table (which
277    * could be null or just plain stale).
278    * @throws IOException
279    */
280   public ServerName getMetaLocationOrReadLocationFromRoot() throws IOException {
281     ServerName sn = getMetaLocation();
282     return sn != null? sn: MetaReader.getMetaRegionLocation(this);
283   }
284 
285   /**
286    * Waits indefinitely for availability of <code>-ROOT-</code>.  Used during
287    * cluster startup.
288    * @throws InterruptedException if interrupted while waiting
289    */
290   public void waitForRoot()
291   throws InterruptedException {
292     this.rootRegionTracker.blockUntilAvailable();
293   }
294 
295   /**
296    * Gets the current location for <code>-ROOT-</code> if available and waits
297    * for up to the specified timeout if not immediately available.  Returns null
298    * if the timeout elapses before root is available.
299    * @param timeout maximum time to wait for root availability, in milliseconds
300    * @return {@link ServerName} for server hosting <code>-ROOT-</code> or null
301    * if none available
302    * @throws InterruptedException if interrupted while waiting
303    * @throws NotAllMetaRegionsOnlineException if root not available before
304    * timeout
305    */
306   public ServerName waitForRoot(final long timeout)
307   throws InterruptedException, NotAllMetaRegionsOnlineException {
308     ServerName sn = rootRegionTracker.waitRootRegionLocation(timeout);
309     if (sn == null) {
310       throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
311     }
312     return sn;
313   }
314 
315   /**
316    * Gets a connection to the server hosting root, as reported by ZooKeeper,
317    * waiting up to the specified timeout for availability.
318    * @param timeout How long to wait on root location
319    * @see #waitForRoot(long) for additional information
320    * @return connection to server hosting root
321    * @throws InterruptedException
322    * @throws NotAllMetaRegionsOnlineException if timed out waiting
323    * @throws IOException
324    * @deprecated Use #getRootServerConnection(long)
325    */
326   public HRegionInterface waitForRootServerConnection(long timeout)
327   throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
328     return getRootServerConnection(timeout);
329   }
330 
331   /**
332    * Gets a connection to the server hosting root, as reported by ZooKeeper,
333    * waiting up to the specified timeout for availability.
334    * <p>WARNING: Does not retry.  Use an {@link HTable} instead.
335    * @param timeout How long to wait on root location
336    * @see #waitForRoot(long) for additional information
337    * @return connection to server hosting root
338    * @throws InterruptedException
339    * @throws NotAllMetaRegionsOnlineException if timed out waiting
340    * @throws IOException
341    */
342   HRegionInterface getRootServerConnection(long timeout)
343   throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
344     return getCachedConnection(waitForRoot(timeout));
345   }
346 
347   /**
348    * Gets a connection to the server currently hosting <code>.META.</code> or
349    * null if location is not currently available.
350    * <p>
351    * If a location is known, a connection to the cached location is returned.
352    * If refresh is true, the cached connection is verified first before
353    * returning.  If the connection is not valid, it is reset and rechecked.
354    * <p>
355    * If no location for meta is currently known, method checks ROOT for a new
356    * location, verifies META is currently there, and returns a cached connection
357    * to the server hosting META.
358    *
359    * @return connection to server hosting meta, null if location not available
360    * @throws IOException
361    * @throws InterruptedException
362    */
363   private HRegionInterface getMetaServerConnection()
364   throws IOException, InterruptedException {
365     synchronized (metaAvailable) {
366       if (metaAvailable.get()) {
367         HRegionInterface current = getCachedConnection(this.metaLocation);
368         // If we are to refresh, verify we have a good connection by making
369         // an invocation on it.
370         if (verifyRegionLocation(current, this.metaLocation, META_REGION_NAME)) {
371           return current;
372         }
373         resetMetaLocation();
374       }
375       // We got here because there is no meta available or because whats
376       // available is bad.
377 
378       // Now read the current .META. content from -ROOT-.  Note: This goes via
379       // an HConnection.  It has its own way of figuring root and meta locations
380       // which we have to wait on.
381       ServerName newLocation = MetaReader.getMetaRegionLocation(this);
382       if (newLocation == null) return null;
383 
384       HRegionInterface newConnection = getCachedConnection(newLocation);
385       if (verifyRegionLocation(newConnection, newLocation, META_REGION_NAME)) {
386         setMetaLocation(newLocation);
387         return newConnection;
388       } else {
389         if (LOG.isTraceEnabled()) {
390           LOG.trace("New .META. server: " + newLocation + " isn't valid." +
391             " Cached .META. server: " + this.metaLocation);
392         }
393       }
394       return null;
395     }
396   }
397 
398   /**
399    * Waits indefinitely for availability of <code>.META.</code>.  Used during
400    * cluster startup.  Does not verify meta, just that something has been
401    * set up in zk.
402    * @see #waitForMeta(long)
403    * @throws InterruptedException if interrupted while waiting
404    */
405   public void waitForMeta() throws InterruptedException {
406     while (!this.stopped) {
407       try {
408         if (waitForMeta(100) != null) break;
409       } catch (NotAllMetaRegionsOnlineException e) {
410         if (LOG.isTraceEnabled()) {
411           LOG.info(".META. still not available, sleeping and retrying." +
412           " Reason: " + e.getMessage());
413         }
414       } catch (IOException e) {
415         LOG.info("Retrying", e);
416       }
417     }
418   }
419 
420   /**
421    * Gets the current location for <code>.META.</code> if available and waits
422    * for up to the specified timeout if not immediately available.  Throws an
423    * exception if timed out waiting.  This method differs from {@link #waitForMeta()}
424    * in that it will go ahead and verify the location gotten from ZooKeeper and
425    * -ROOT- region by trying to use returned connection.
426    * @param timeout maximum time to wait for meta availability, in milliseconds
427    * @return {@link ServerName} for server hosting <code>.META.</code> or null
428    * if none available
429    * @throws InterruptedException if interrupted while waiting
430    * @throws IOException unexpected exception connecting to meta server
431    * @throws NotAllMetaRegionsOnlineException if meta not available before
432    * timeout
433    */
434   public ServerName waitForMeta(long timeout)
435   throws InterruptedException, IOException, NotAllMetaRegionsOnlineException {
436     long stop = timeout == 0 ? Long.MAX_VALUE : System.currentTimeMillis() + timeout;
437     long waitTime = Math.min(50, timeout);
438     synchronized (metaAvailable) {
439       while(!stopped && System.currentTimeMillis() < stop) {
440         if (getMetaServerConnection() != null) {
441           return metaLocation;
442         }
443         // perhaps -ROOT- region isn't available, let us wait a bit and retry.
444         metaAvailable.wait(waitTime);
445       }
446       if (getMetaServerConnection() == null) {
447         throw new NotAllMetaRegionsOnlineException("Timed out (" + timeout + "ms)");
448       }
449       return metaLocation;
450     }
451   }
452 
453   /**
454    * Gets a connection to the server hosting meta, as reported by ZooKeeper,
455    * waiting up to the specified timeout for availability.
456    * @see #waitForMeta(long) for additional information
457    * @return connection to server hosting meta
458    * @throws InterruptedException
459    * @throws NotAllMetaRegionsOnlineException if timed out waiting
460    * @throws IOException
461    * @deprecated Does not retry; use an HTable instance instead.
462    */
463   public HRegionInterface waitForMetaServerConnection(long timeout)
464   throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
465     return getCachedConnection(waitForMeta(timeout));
466   }
467 
468   /**
469    * Called when we figure current meta is off (called from zk callback).
470    */
471   public void resetMetaLocation() {
472     LOG.debug("Current cached META location, " + metaLocation +
473       ", is not valid, resetting");
474     synchronized(this.metaAvailable) {
475       this.metaAvailable.set(false);
476       this.metaAvailable.notifyAll();
477     }
478   }
479 
480   /**
481    * @param metaLocation
482    */
483   void setMetaLocation(final ServerName metaLocation) {
484     LOG.debug("Set new cached META location: " + metaLocation);
485     synchronized (this.metaAvailable) {
486       this.metaLocation = metaLocation;
487       this.metaAvailable.set(true);
488       // no synchronization because these are private and already under lock
489       this.metaAvailable.notifyAll();
490     }
491   }
492 
493   /**
494    * @param sn ServerName to get a connection against.
495    * @return The HRegionInterface we got when we connected to <code>sn</code>
496    * May have come from cache, may not be good, may have been setup by this
497    * invocation, or may be null.
498    * @throws IOException
499    */
500   private HRegionInterface getCachedConnection(ServerName sn)
501   throws IOException {
502     if (sn == null) {
503       return null;
504     }
505     HRegionInterface protocol = null;
506     try {
507       protocol = connection.getHRegionConnection(sn.getHostname(), sn.getPort());
508     } catch (RetriesExhaustedException e) {
509       if (e.getCause() != null && e.getCause() instanceof ConnectException) {
510         // Catch this; presume it means the cached connection has gone bad.
511       } else {
512         throw e;
513       }
514     } catch (SocketTimeoutException e) {
515       LOG.debug("Timed out connecting to " + sn);
516     } catch (NoRouteToHostException e) {
517       LOG.debug("Connecting to " + sn, e);
518     } catch (SocketException e) {
519       LOG.debug("Exception connecting to " + sn);
520     } catch (UnknownHostException e) {
521       LOG.debug("Unknown host exception connecting to  " + sn);
522     } catch (FailedServerException e) {
523       if (LOG.isDebugEnabled()) {
524         LOG.debug("Server " + sn + " is in failed server list.");
525       }
526     } catch (IOException ioe) {
527       Throwable cause = ioe.getCause();
528       if (ioe instanceof ConnectException) {
529         // Catch. Connect refused.
530       } else if (cause != null && cause instanceof EOFException) {
531         // Catch. Other end disconnected us.
532       } else if (cause != null && cause.getMessage() != null &&
533         cause.getMessage().toLowerCase().contains("connection reset")) {
534         // Catch. Connection reset.
535       } else {
536         throw ioe;
537       }
538       
539     }
540     return protocol;
541   }
542 
543   /**
544    * Verify we can connect to <code>hostingServer</code> and that its carrying
545    * <code>regionName</code>.
546    * @param hostingServer Interface to the server hosting <code>regionName</code>
547    * @param serverName The servername that goes with the <code>metaServer</code>
548    * Interface.  Used logging.
549    * @param regionName The regionname we are interested in.
550    * @return True if we were able to verify the region located at other side of
551    * the Interface.
552    * @throws IOException
553    */
554   // TODO: We should be able to get the ServerName from the HRegionInterface
555   // rather than have to pass it in.  Its made awkward by the fact that the
556   // HRI is likely a proxy against remote server so the getServerName needs
557   // to be fixed to go to a local method or to a cache before we can do this.
558   private boolean verifyRegionLocation(HRegionInterface hostingServer,
559       final ServerName address, final byte [] regionName)
560   throws IOException {
561     if (hostingServer == null) {
562       LOG.info("Passed hostingServer is null");
563       return false;
564     }
565     Throwable t = null;
566     try {
567       // Try and get regioninfo from the hosting server.
568       return hostingServer.getRegionInfo(regionName) != null;
569     } catch (ConnectException e) {
570       t = e;
571     } catch (RetriesExhaustedException e) {
572       t = e;
573     } catch (RemoteException e) {
574       IOException ioe = e.unwrapRemoteException();
575       t = ioe;
576     } catch (IOException e) {
577       Throwable cause = e.getCause();
578       if (cause != null && cause instanceof EOFException) {
579         t = cause;
580       } else if (cause != null && cause.getMessage() != null
581           && cause.getMessage().contains("Connection reset")) {
582         t = cause;
583       } else {
584         t = e;
585       }
586     }
587     LOG.info("Failed verification of " + Bytes.toStringBinary(regionName) +
588       " at address=" + address + "; " + t);
589     return false;
590   }
591 
592   /**
593    * Verify <code>-ROOT-</code> is deployed and accessible.
594    * @param timeout How long to wait on zk for root address (passed through to
595    * the internal call to {@link #waitForRootServerConnection(long)}.
596    * @return True if the <code>-ROOT-</code> location is healthy.
597    * @throws IOException
598    * @throws InterruptedException 
599    */
600   public boolean verifyRootRegionLocation(final long timeout)
601   throws InterruptedException, IOException {
602     HRegionInterface connection = null;
603     try {
604       connection = waitForRootServerConnection(timeout);
605     } catch (NotAllMetaRegionsOnlineException e) {
606       // Pass
607     } catch (ServerNotRunningYetException e) {
608       // Pass -- remote server is not up so can't be carrying root
609     } catch (UnknownHostException e) {
610       // Pass -- server name doesn't resolve so it can't be assigned anything.
611     }
612     return (connection == null)? false:
613       verifyRegionLocation(connection,
614         this.rootRegionTracker.getRootRegionLocation(), ROOT_REGION_NAME);
615   }
616 
617   /**
618    * Verify <code>.META.</code> is deployed and accessible.
619    * @param timeout How long to wait on zk for <code>.META.</code> address
620    * (passed through to the internal call to {@link #waitForMetaServerConnection(long)}.
621    * @return True if the <code>.META.</code> location is healthy.
622    * @throws IOException Some unexpected IOE.
623    * @throws InterruptedException
624    */
625   public boolean verifyMetaRegionLocation(final long timeout)
626   throws InterruptedException, IOException {
627     HRegionInterface connection = null;
628     try {
629       connection = waitForMetaServerConnection(timeout);
630     } catch (NotAllMetaRegionsOnlineException e) {
631       // Pass
632     } catch (ServerNotRunningYetException e) {
633       // Pass -- remote server is not up so can't be carrying .META.
634     } catch (UnknownHostException e) {
635       // Pass -- server name doesn't resolve so it can't be assigned anything.
636     } catch (RetriesExhaustedException e) {
637       // Pass -- failed after bunch of retries.
638       LOG.debug("Failed verify meta region location after retries", e);
639     }
640     return connection != null;
641   }
642 
643   // Used by tests.
644   MetaNodeTracker getMetaNodeTracker() {
645     return this.metaNodeTracker;
646   }
647 
648   public HConnection getConnection() {
649     return this.connection;
650   }
651 }