You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2010/05/13 00:40:21 UTC

svn commit: r943732 - in /hadoop/hbase/trunk/core/src: main/java/org/apache/hadoop/hbase/master/ test/java/org/apache/hadoop/hbase/

Author: stack
Date: Wed May 12 22:40:20 2010
New Revision: 943732

URL: http://svn.apache.org/viewvc?rev=943732&view=rev
Log:
HBASE-2525 If one-RS only and its restarted before its shutdown is completely processed, we'll never complete shutdown processing

Modified:
    hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java
    hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java
    hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java?rev=943732&r1=943731&r2=943732&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java Wed May 12 22:40:20 2010
@@ -45,6 +45,8 @@ import java.util.Set;
  * serving, and the regions need to get reassigned.
  */
 class ProcessServerShutdown extends RegionServerOperation {
+  // Server name made of the concatenation of hostname, port and startcode
+  // formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code>
   private final String deadServer;
   private boolean isRootServer;
   private List<MetaRegion> metaRegions;
@@ -86,7 +88,7 @@ class ProcessServerShutdown extends Regi
   private void closeMetaRegions() {
     this.isRootServer =
       this.master.getRegionManager().isRootServer(this.deadServerAddress) ||
-      this.master.getRegionManager().isRootServerCandidate (deadServer);
+      this.master.getRegionManager().isRootInTransitionOnThisServer(deadServer);
     if (this.isRootServer) {
       this.master.getRegionManager().unsetRootRegion();
     }
@@ -156,7 +158,7 @@ class ProcessServerShutdown extends Regi
         // Check server name.  If null, skip (We used to consider it was on
         // shutdown server but that would mean that we'd reassign regions that
         // were already out being assigned, ones that were product of a split
-        // that happened while the shutdown was being processed.
+        // that happened while the shutdown was being processed).
         String serverAddress = BaseScanner.getServerAddress(values);
         long startCode = BaseScanner.getStartCode(values);
 
@@ -174,7 +176,6 @@ class ProcessServerShutdown extends Regi
             Bytes.toString(row));
         }
 
-//        HRegionInfo info = master.getHRegionInfo(row, values.rowResult());
         HRegionInfo info = master.getHRegionInfo(row, values);
         if (info == null) {
           emptyRows.add(row);
@@ -248,7 +249,7 @@ class ProcessServerShutdown extends Regi
 
     public Boolean call() throws IOException {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("process server shutdown scanning root region on " +
+        LOG.debug("Process server shutdown scanning root region on " +
             master.getRegionManager().getRootRegionLocation().getBindAddress());
       }
       Scan scan = new Scan();
@@ -282,11 +283,9 @@ class ProcessServerShutdown extends Regi
 
   @Override
   protected boolean process() throws IOException {
-    LOG.info("process shutdown of server " + this.deadServer +
-      ": logSplit: " +
-      logSplit + ", rootRescanned: " + rootRescanned +
-      ", numberOfMetaRegions: " +
-      master.getRegionManager().numMetaRegions() +
+    LOG.info("Process shutdown of server " + this.deadServer +
+      ": logSplit: " + logSplit + ", rootRescanned: " + rootRescanned +
+      ", numberOfMetaRegions: " + master.getRegionManager().numMetaRegions() +
       ", onlineMetaRegions.size(): " +
       master.getRegionManager().numOnlineMetaRegions());
     if (!logSplit) {
@@ -305,13 +304,10 @@ class ProcessServerShutdown extends Regi
       }
       logSplit = true;
     }
-
     LOG.info("Log split complete, meta reassignment and scanning:");
-
     if (this.isRootServer) {
       LOG.info("ProcessServerShutdown reassigning ROOT region");
       master.getRegionManager().reassignRootRegion();
-
       isRootServer = false;  // prevent double reassignment... heh.
     }
 
@@ -341,12 +337,13 @@ class ProcessServerShutdown extends Regi
       }
 
       if (LOG.isDebugEnabled()) {
-        LOG.debug("process server shutdown scanning root region on " +
+        LOG.debug("Process server shutdown scanning root region on " +
           master.getRegionManager().getRootRegionLocation().getBindAddress() +
           " finished " + Thread.currentThread().getName());
       }
       rootRescanned = true;
     }
+
     if (!metaTableAvailable()) {
       // We can't proceed because not all meta regions are online.
       // metaAvailable() has put this request on the delayedToDoQueue
@@ -367,8 +364,6 @@ class ProcessServerShutdown extends Regi
     }
 
     closeRegionsInTransition();
-
-    // Remove this server from dead servers list.  Finished splitting logs.
     this.master.getServerManager().removeDeadServer(deadServer);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Removed " + deadServer + " from deadservers Map");

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java?rev=943732&r1=943731&r2=943732&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java Wed May 12 22:40:20 2010
@@ -857,7 +857,7 @@ public class RegionManager implements HC
    * @param server Server
    * @return true if server is transitioning the ROOT table
    */
-  public boolean isRootServerCandidate(final String server) {
+  public boolean isRootInTransitionOnThisServer(final String server) {
     synchronized (this.regionsInTransition) {
       for (RegionState s : regionsInTransition.values()) {
         if (s.getRegionInfo().isRootRegion()

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=943732&r1=943731&r2=943732&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Wed May 12 22:40:20 2010
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HServerAd
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.Leases;
+import org.apache.hadoop.hbase.Leases.LeaseStillHeldException;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
@@ -170,6 +171,10 @@ public class ServerManager implements HC
     // Test for case where we get a region startup message from a regionserver
     // that has been quickly restarted but whose znode expiration handler has
     // not yet run, or from a server whose fail we are currently processing.
+    // Test its host+port combo is present in serverAddresstoServerInfo.  If it
+    // is, reject the server and trigger its expiration. The next time it comes
+    // in, it should have been removed from serverAddressToServerInfo and queued
+    // for processing by ProcessServerShutdown.
     HServerInfo info = new HServerInfo(serverInfo);
     String hostAndPort = info.getServerAddress().toString();
     HServerInfo existingServer =
@@ -183,20 +188,25 @@ public class ServerManager implements HC
       }
       throw new Leases.LeaseStillHeldException(hostAndPort);
     }
-    if (isDead(hostAndPort, true)) {
-      LOG.debug("Server start rejected; currently processing " + hostAndPort +
-        " failure");
-      throw new Leases.LeaseStillHeldException(hostAndPort);
-    }
-    if (isDead(hostAndPort, true)) {
-      LOG.debug("Server start rejected; currently processing " + hostAndPort +
-        " failure");
-      throw new Leases.LeaseStillHeldException(hostAndPort);
-    }
+    checkIsDead(info.getServerName(), "STARTUP");
     LOG.info("Received start message from: " + info.getServerName());
     recordNewServer(info);
   }
 
+  /*
+   * If this server is on the dead list, reject it with a LeaseStillHeldException
+   * @param serverName Server name formatted as host_port_startcode.
+   * @param what START or REPORT
+   * @throws LeaseStillHeldException
+   */
+  private void checkIsDead(final String serverName, final String what)
+  throws LeaseStillHeldException {
+    if (!isDead(serverName)) return;
+    LOG.debug("Server " + what + " rejected; currently processing " +
+      serverName + " as dead server");
+    throw new Leases.LeaseStillHeldException(serverName);
+  }
+
   /**
    * Adds the HSI to the RS list and creates an empty load
    * @param info The region server informations
@@ -249,11 +259,7 @@ public class ServerManager implements HC
     final HMsg msgs[], final HRegionInfo[] mostLoadedRegions)
   throws IOException {
     HServerInfo info = new HServerInfo(serverInfo);
-    if (isDead(info.getServerName())) {
-      LOG.info("Received report from region server " + info.getServerName() +
-        " previously marked dead. Rejecting report.");
-      throw new Leases.LeaseStillHeldException(info.getServerName());
-    }
+    checkIsDead(info.getServerName(), "REPORT");
     if (msgs.length > 0) {
       if (msgs[0].isType(HMsg.Type.MSG_REPORT_EXITING)) {
         processRegionServerExit(info, msgs);
@@ -665,7 +671,7 @@ public class ServerManager implements HC
       this.master.getRegionManager().offlineMetaServer(info.getServerAddress());
 
       //HBASE-1928: Check whether this server has been transitioning the ROOT table
-      if (this.master.getRegionManager().isRootServerCandidate (serverName)) {
+      if (this.master.getRegionManager().isRootInTransitionOnThisServer(serverName)) {
          this.master.getRegionManager().unsetRootRegion();
          this.master.getRegionManager().reassignRootRegion();
       }

Modified: hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=943732&r1=943731&r2=943732&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Wed May 12 22:40:20 2010
@@ -600,8 +600,10 @@ public class HBaseTestingUtility {
     ZooKeeper zk = new ZooKeeper(quorumServers,
         sessionTimeout, EmptyWatcher.instance, sessionID, password);
     zk.close();
+    final long sleep = sessionTimeout * 5L;
+    LOG.info("ZK Closed; sleeping=" + sleep);
 
-    Thread.sleep(sessionTimeout * 5L);
+    Thread.sleep(sleep);
 
     new HTable(conf, HConstants.META_TABLE_NAME);
   }

Modified: hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=943732&r1=943731&r2=943732&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hadoop/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Wed May 12 22:40:20 2010
@@ -19,6 +19,8 @@
  */
 package org.apache.hadoop.hbase;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -31,18 +33,19 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 
 public class TestZooKeeper {
+  private final Log LOG = LogFactory.getLog(this.getClass());
 
   private final static HBaseTestingUtility
       TEST_UTIL = new HBaseTestingUtility();
 
   private Configuration    conf;
-  private MiniHBaseCluster cluster;
 
   /**
    * @throws java.lang.Exception
    */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
     TEST_UTIL.startMiniCluster(1);
   }
 
@@ -60,7 +63,6 @@ public class TestZooKeeper {
   @Before
   public void setUp() throws Exception {
     conf = TEST_UTIL.getConfiguration();
-    cluster = TEST_UTIL.getHBaseCluster();
   }
 
   /**
@@ -98,13 +100,16 @@ public class TestZooKeeper {
   }
   @Test
   public void testRegionServerSessionExpired() throws Exception{
-    this.conf.setBoolean("hbase.regionserver.restart.on.zk.expire", true);
+    LOG.info("Starting testRegionServerSessionExpired");
     new HTable(conf, HConstants.META_TABLE_NAME);
+    TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).getConfiguration().
+      setBoolean("hbase.regionserver.restart.on.zk.expire", true);
     TEST_UTIL.expireRegionServerSession(0);
     testSanity();
   }
   @Test
   public void testMasterSessionExpired() throws Exception {
+    LOG.info("Starting testRegionServerSessionExpired");
     new HTable(conf, HConstants.META_TABLE_NAME);
     TEST_UTIL.expireMasterSession();
     testSanity();
@@ -136,7 +141,7 @@ public class TestZooKeeper {
   public void testMultipleZK() {
     try {
       HTable localMeta = new HTable(conf, HConstants.META_TABLE_NAME);
-      HBaseConfiguration otherConf = new HBaseConfiguration(conf);
+      Configuration otherConf = HBaseConfiguration.create(conf);
       otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
       HTable ipMeta = new HTable(conf, HConstants.META_TABLE_NAME);