You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2013/01/16 23:10:46 UTC

svn commit: r1434436 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/catalog/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/catalog/ test/java/org/apach...

Author: larsh
Date: Wed Jan 16 22:10:46 2013
New Revision: 1434436

URL: http://svn.apache.org/viewvc?rev=1434436&view=rev
Log:
HBASE-7578 TestCatalogTracker hangs occasionally

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java?rev=1434436&r1=1434435&r2=1434436&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java Wed Jan 16 22:10:46 2013
@@ -122,11 +122,6 @@ public class CatalogTracker {
    */
   private ServerName metaLocation;
 
-  /*
-   * Timeout waiting on root or meta to be set.
-   */
-  private final int defaultTimeout;
-
   private boolean stopped = false;
 
   static final byte [] ROOT_REGION_NAME =
@@ -162,33 +157,13 @@ public class CatalogTracker {
    * @throws IOException 
    */
   public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
-      final Abortable abortable)
+      Abortable abortable)
   throws IOException {
-    this(zk, conf, abortable,
-      conf.getInt("hbase.catalogtracker.default.timeout", 1000));
-  }
-
-  /**
-   * Constructs the catalog tracker.  Find current state of catalog tables.
-   * Begin active tracking by executing {@link #start()} post construction.
-   * @param zk If zk is null, we'll create an instance (and shut it down
-   * when {@link #stop()} is called) else we'll use what is passed.
-   * @param conf
-   * @param abortable If fatal exception we'll call abort on this.  May be null.
-   * If it is we'll use the Connection associated with the passed
-   * {@link Configuration} as our Abortable.
-   * @param defaultTimeout Timeout to use.  Pass zero for no timeout
-   * ({@link Object#wait(long)} when passed a <code>0</code> waits for ever).
-   * @throws IOException
-   */
-  public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
-      Abortable abortable, final int defaultTimeout)
-  throws IOException {
-    this(zk, conf, HConnectionManager.getConnection(conf), abortable, defaultTimeout);
+    this(zk, conf, HConnectionManager.getConnection(conf), abortable);
   }
 
   public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
-      HConnection connection, Abortable abortable, final int defaultTimeout)
+      HConnection connection, Abortable abortable)
   throws IOException {
     this.connection = connection;
     if (abortable == null) {
@@ -226,7 +201,6 @@ public class CatalogTracker {
         ct.resetMetaLocation();
       }
     };
-    this.defaultTimeout = defaultTimeout;
   }
 
   /**
@@ -364,24 +338,6 @@ public class CatalogTracker {
   }
 
   /**
-   * Gets a connection to the server hosting root, as reported by ZooKeeper,
-   * waiting for the default timeout specified on instantiation.
-   * @see #waitForRoot(long) for additional information
-   * @return connection to server hosting root
-   * @throws NotAllMetaRegionsOnlineException if timed out waiting
-   * @throws IOException
-   * @deprecated Use #getRootServerConnection(long)
-   */
-  public AdminProtocol waitForRootServerConnectionDefault()
-  throws NotAllMetaRegionsOnlineException, IOException {
-    try {
-      return getRootServerConnection(this.defaultTimeout);
-    } catch (InterruptedException e) {
-      throw new NotAllMetaRegionsOnlineException("Interrupted");
-    }
-  }
-
-  /**
    * Gets a connection to the server currently hosting <code>.META.</code> or
    * null if location is not currently available.
    * <p>
@@ -470,10 +426,10 @@ public class CatalogTracker {
    */
   public ServerName waitForMeta(long timeout)
   throws InterruptedException, IOException, NotAllMetaRegionsOnlineException {
-    long stop = System.currentTimeMillis() + timeout;
+    long stop = timeout == 0 ? Long.MAX_VALUE : System.currentTimeMillis() + timeout;
     long waitTime = Math.min(50, timeout);
     synchronized (metaAvailable) {
-      while(!stopped && (timeout == 0 || System.currentTimeMillis() < stop)) {
+      while(!stopped && System.currentTimeMillis() < stop) {
         if (getMetaServerConnection() != null) {
           return metaLocation;
         }
@@ -503,25 +459,6 @@ public class CatalogTracker {
   }
 
   /**
-   * Gets a connection to the server hosting meta, as reported by ZooKeeper,
-   * waiting up to the specified timeout for availability.
-   * Used in tests.
-   * @see #waitForMeta(long) for additional information
-   * @return connection to server hosting meta
-   * @throws NotAllMetaRegionsOnlineException if timed out or interrupted
-   * @throws IOException
-   * @deprecated Does not retry; use an HTable instance instead.
-   */
-  public AdminProtocol waitForMetaServerConnectionDefault()
-  throws NotAllMetaRegionsOnlineException, IOException {
-    try {
-      return getCachedConnection(waitForMeta(defaultTimeout));
-    } catch (InterruptedException e) {
-      throw new NotAllMetaRegionsOnlineException("Interrupted");
-    }
-  }
-
-  /**
    * Called when we figure current meta is off (called from zk callback).
    */
   public void resetMetaLocation() {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1434436&r1=1434435&r2=1434436&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Wed Jan 16 22:10:46 2013
@@ -541,8 +541,7 @@ Server {
    */
   private void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException {
-    this.catalogTracker = createCatalogTracker(this.zooKeeper, this.conf,
-        this, conf.getInt("hbase.master.catalog.timeout", 600000));
+    this.catalogTracker = createCatalogTracker(this.zooKeeper, this.conf, this);
     this.catalogTracker.start();
 
     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
@@ -585,9 +584,9 @@ Server {
    * @throws IOException
    */
   CatalogTracker createCatalogTracker(final ZooKeeperWatcher zk,
-      final Configuration conf, Abortable abortable, final int defaultTimeout)
+      final Configuration conf, Abortable abortable)
   throws IOException {
-    return new CatalogTracker(zk, conf, abortable, defaultTimeout);
+    return new CatalogTracker(zk, conf, abortable);
   }
 
   // Check if we should stop every 100ms

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1434436&r1=1434435&r2=1434436&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Wed Jan 16 22:10:46 2013
@@ -769,8 +769,7 @@ public class  HRegionServer implements C
     blockAndCheckIfStopped(this.clusterStatusTracker);
 
     // Create the catalog tracker and start it;
-    this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf,
-      this, this.conf.getInt("hbase.regionserver.catalog.timeout", 600000));
+    this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf, this);
     catalogTracker.start();
 
     // Retrieve clusterId

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java?rev=1434436&r1=1434435&r2=1434436&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java Wed Jan 16 22:10:46 2013
@@ -115,7 +115,7 @@ public class TestCatalogTracker {
   private CatalogTracker constructAndStartCatalogTracker(final HConnection c)
   throws IOException, InterruptedException {
     CatalogTracker ct = new CatalogTracker(this.watcher, UTIL.getConfiguration(),
-      c, this.abortable, 0);
+      c, this.abortable);
     ct.start();
     return ct;
   }
@@ -234,10 +234,8 @@ public class TestCatalogTracker {
           @Override
           public void run() {
             try {
-              metaSet.set(ct.waitForMetaServerConnectionDefault() !=  null);
-            } catch (NotAllMetaRegionsOnlineException e) {
-              throw new RuntimeException(e);
-            } catch (IOException e) {
+              metaSet.set(ct.waitForMetaServerConnection(100000) !=  null);
+            } catch (Exception e) {
               throw new RuntimeException(e);
             }
           }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java?rev=1434436&r1=1434435&r2=1434436&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java Wed Jan 16 22:10:46 2013
@@ -195,7 +195,7 @@ public class TestMetaReaderEditorNoClust
         when(connection).getClient(Mockito.anyString(), Mockito.anyInt());
 
       // Now start up the catalogtracker with our doctored Connection.
-      ct = new CatalogTracker(zkw, null, connection, ABORTABLE, 0);
+      ct = new CatalogTracker(zkw, null, connection, ABORTABLE);
       ct.start();
       // Scan meta for user tables and verify we got back expected answer.
       NavigableMap<HRegionInfo, Result> hris = MetaReader.getServerUserRegions(ct, sn);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1434436&r1=1434435&r2=1434436&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Wed Jan 16 22:10:46 2013
@@ -116,7 +116,7 @@ public class TestCatalogJanitor {
       this.ct = Mockito.mock(CatalogTracker.class);
       AdminProtocol hri = Mockito.mock(AdminProtocol.class);
       Mockito.when(this.ct.getConnection()).thenReturn(this.connection);
-      Mockito.when(ct.waitForMetaServerConnectionDefault()).thenReturn(hri);
+      Mockito.when(ct.waitForMetaServerConnection(Mockito.anyLong())).thenReturn(hri);
     }
 
     @Override

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java?rev=1434436&r1=1434435&r2=1434436&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java Wed Jan 16 22:10:46 2013
@@ -198,7 +198,7 @@ public class TestMasterNoCluster {
 
       @Override
       CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
-          Configuration conf, Abortable abortable, int defaultTimeout)
+          Configuration conf, Abortable abortable)
       throws IOException {
         // Insert a mock for the connection used by the CatalogTracker.  Any
         // regionserver should do.  Use TESTUTIL.getConfiguration rather than
@@ -207,7 +207,7 @@ public class TestMasterNoCluster {
         HConnection connection =
           HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
             rs0, rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
-        return new CatalogTracker(zk, conf, connection, abortable, defaultTimeout);
+        return new CatalogTracker(zk, conf, connection, abortable);
       }
     };
     master.start();
@@ -284,7 +284,7 @@ public class TestMasterNoCluster {
 
       @Override
       CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
-          Configuration conf, Abortable abortable, int defaultTimeout)
+          Configuration conf, Abortable abortable)
       throws IOException {
         // Insert a mock for the connection used by the CatalogTracker.   Use
         // TESTUTIL.getConfiguration rather than the conf from the master; the
@@ -293,7 +293,7 @@ public class TestMasterNoCluster {
         HConnection connection =
           HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
             rs0, rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
-        return new CatalogTracker(zk, conf, connection, abortable, defaultTimeout);
+        return new CatalogTracker(zk, conf, connection, abortable);
       }
     };
     master.start();