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

svn commit: r1449920 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/catalog/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/master/handler/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apa...

Author: tedyu
Date: Mon Feb 25 22:01:57 2013
New Revision: 1449920

URL: http://svn.apache.org/r1449920
Log:
HBASE-7824 Improve master start up time when there is log splitting work (Jeffrey Zhong)


Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java Mon Feb 25 22:01:57 2013
@@ -118,7 +118,7 @@ public class CatalogTracker {
    */
   private ServerName metaLocation;
 
-  private boolean stopped = false;
+  private volatile boolean stopped = false;
 
   static final byte [] ROOT_REGION_NAME =
     HRegionInfo.ROOT_REGIONINFO.getRegionName();

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Mon Feb 25 22:01:57 2013
@@ -41,9 +41,6 @@ import java.util.concurrent.atomic.Atomi
 
 import javax.management.ObjectName;
 
-import com.google.common.collect.ClassToInstanceMap;
-import com.google.common.collect.Maps;
-import com.google.common.collect.MutableClassToInstanceMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -123,6 +120,10 @@ import org.apache.hadoop.net.DNS;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 
+import com.google.common.collect.ClassToInstanceMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.MutableClassToInstanceMap;
+
 /**
  * HMaster is the "master server" for HBase. An HBase cluster has one active
  * master.  If many masters are started, all compete.  Whichever wins goes on to
@@ -552,14 +553,37 @@ Server {
     if (!masterRecovery) {
       this.assignmentManager.startTimeOutMonitor();
     }
-    // TODO: Should do this in background rather than block master startup
-    status.setStatus("Splitting logs after master startup");
-    splitLogAfterStartup(this.fileSystemManager);
 
-    // Make sure root and meta assigned before proceeding.
-    assignRootAndMeta(status);
+    // get a list for previously failed RS which need recovery work
+    Set<ServerName> failedServers = this.fileSystemManager.getFailedServersFromLogFolders();
+    ServerName preRootServer = this.catalogTracker.getRootLocation();
+    if (preRootServer != null && failedServers.contains(preRootServer)) {
+      // create recovered edits file for _ROOT_ server
+      this.fileSystemManager.splitLog(preRootServer);
+      failedServers.remove(preRootServer);
+    }
+
+    // Make sure root assigned before proceeding.
+    assignRoot(status);
+
+    // log splitting for .META. server
+    ServerName preMetaServer = this.catalogTracker.getMetaLocationOrReadLocationFromRoot();
+    if (preMetaServer != null && failedServers.contains(preMetaServer)) {
+      // create recovered edits file for .META. server
+      this.fileSystemManager.splitLog(preMetaServer);
+      failedServers.remove(preMetaServer);
+    }
+    // Make sure meta assigned before proceeding.
+    assignMeta(status, preRootServer);
+
     enableServerShutdownHandler();
 
+    // handle other dead servers in SSH
+    status.setStatus("Submit log splitting work of non-meta region servers");
+    for (ServerName curServer : failedServers) {
+      this.serverManager.processDeadServer(curServer);
+    }
+
     // Update meta with new HRI if required. i.e migrate all HRI with HTD to
     // HRI with out HTD in meta and update the status in ROOT. This must happen
     // before we assign all user regions or else the assignment will fail.
@@ -631,22 +655,13 @@ Server {
   }
 
   /**
-   * Override to change master's splitLogAfterStartup. Used testing
-   * @param mfs
-   */
-  protected void splitLogAfterStartup(final MasterFileSystem mfs) {
-    mfs.splitLogAfterStartup();
-  }
-
-  /**
-   * Check <code>-ROOT-</code> and <code>.META.</code> are assigned.  If not,
-   * assign them.
+   * Check <code>-ROOT-</code> is assigned. If not, assign it.
+   * @param status MonitoredTask
    * @throws InterruptedException
    * @throws IOException
    * @throws KeeperException
-   * @return Count of regions we assigned.
    */
-  int assignRootAndMeta(MonitoredTask status)
+  private void assignRoot(MonitoredTask status)
   throws InterruptedException, IOException, KeeperException {
     int assigned = 0;
     long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
@@ -677,16 +692,32 @@ Server {
     LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
       ", location=" + catalogTracker.getRootLocation());
 
-    // Work on meta region
+    status.setStatus("ROOT assigned.");
+  }
+
+  /**
+   * Check <code>.META.</code> is assigned. If not, assign it.
+   * @param status MonitoredTask
+   * @param previousRootServer ServerName of previous root region server before current start up
+   * @return
+   * @throws InterruptedException
+   * @throws IOException
+   * @throws KeeperException
+   */
+  private void assignMeta(MonitoredTask status, ServerName previousRootServer)
+      throws InterruptedException,
+      IOException, KeeperException {
+    int assigned = 0;
+    long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
+
     status.setStatus("Assigning META region");
-    rit = this.assignmentManager.
-      processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
+    boolean rit =
+        this.assignmentManager
+            .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
     boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
     if (!rit && !metaRegionLocation) {
-      ServerName currentMetaServer =
-        this.catalogTracker.getMetaLocationOrReadLocationFromRoot();
-      if (currentMetaServer != null
-          && !currentMetaServer.equals(currentRootServer)) {
+      ServerName currentMetaServer = this.catalogTracker.getMetaLocationOrReadLocationFromRoot();
+      if (currentMetaServer != null && !currentMetaServer.equals(previousRootServer)) {
         splitLogAndExpireIfOnline(currentMetaServer);
       }
       assignmentManager.assignMeta();
@@ -696,15 +727,14 @@ Server {
       enableSSHandWaitForMeta();
       assigned++;
     } else {
-      // Region already assigned.  We didnt' assign it.  Add to in-memory state.
+      // Region already assigned. We didnt' assign it. Add to in-memory state.
       this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
         this.catalogTracker.getMetaLocation());
     }
     enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME));
-    LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
-      ", location=" + catalogTracker.getMetaLocation());
-    status.setStatus("META and ROOT assigned.");
-    return assigned;
+    LOG.info(".META. assigned=" + assigned + ", rit=" + rit + ", location="
+        + catalogTracker.getMetaLocation());
+    status.setStatus("META assigned.");
   }
 
   private void enableSSHandWaitForMeta() throws IOException,
@@ -763,8 +793,7 @@ Server {
   }
 
   /**
-   * Split a server's log and expire it if we find it is one of the online
-   * servers.
+   * Expire a server if we find it is one of the online servers.
    * @param sn ServerName to check.
    * @throws IOException
    */
@@ -1626,12 +1655,23 @@ Server {
     }
     if (this.assignmentManager != null) this.assignmentManager.shutdown();
     if (this.serverManager != null) this.serverManager.shutdownCluster();
+
     try {
       if (this.clusterStatusTracker != null){
         this.clusterStatusTracker.setClusterDown();
       }
     } catch (KeeperException e) {
-      LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
+      if (e instanceof KeeperException.SessionExpiredException) {
+        LOG.warn("ZK session expired. Retry a new connection...");
+        try {
+          this.zooKeeper.reconnectAfterExpiration();
+          this.clusterStatusTracker.setClusterDown();
+        } catch (Exception ex) {
+          LOG.warn("Retry setClusterDown failed", ex);
+        }
+      } else {
+        LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
+      }
     }
   }
 

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Mon Feb 25 22:01:57 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.UUID;
@@ -184,30 +185,31 @@ public class MasterFileSystem {
   }
 
   /**
-   * Inspect the log directory to recover any log file without
-   * an active region server.
+   * Inspect the log directory to find dead servers which need log splitting
    */
-  void splitLogAfterStartup() {
+  Set<ServerName> getFailedServersFromLogFolders() {
     boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors",
-        HLog.SPLIT_SKIP_ERRORS_DEFAULT);
+      HLog.SPLIT_SKIP_ERRORS_DEFAULT);
+    
+    Set<ServerName> serverNames = new HashSet<ServerName>();
     Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
+
     do {
       if (master.isStopped()) {
-        LOG.warn("Master stopped while splitting logs");
+        LOG.warn("Master stopped while trying to get failed servers.");
         break;
       }
-      List<ServerName> serverNames = new ArrayList<ServerName>();
       try {
-        if (!this.fs.exists(logsDirPath)) return;
+        if (!this.fs.exists(logsDirPath)) return serverNames;
         FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
         // Get online servers after getting log folders to avoid log folder deletion of newly
         // checked in region servers . see HBASE-5916
-        Set<ServerName> onlineServers = ((HMaster) master).getServerManager().getOnlineServers()
-            .keySet();
+        Set<ServerName> onlineServers =
+            ((HMaster) master).getServerManager().getOnlineServers().keySet();
 
         if (logFolders == null || logFolders.length == 0) {
           LOG.debug("No log files to split, proceeding...");
-          return;
+          return serverNames;
         }
         for (FileStatus status : logFolders) {
           String sn = status.getPath().getName();
@@ -221,22 +223,19 @@ public class MasterFileSystem {
                 + "to a known region server, splitting");
             serverNames.add(serverName);
           } else {
-            LOG.info("Log folder " + status.getPath()
-                + " belongs to an existing region server");
+            LOG.info("Log folder " + status.getPath() + " belongs to an existing region server");
           }
         }
-        splitLog(serverNames);
         retrySplitting = false;
       } catch (IOException ioe) {
-        LOG.warn("Failed splitting of " + serverNames, ioe);
+        LOG.warn("Failed getting failed servers to be recovered.", ioe);
         if (!checkFileSystem()) {
           LOG.warn("Bad Filesystem, exiting");
           Runtime.getRuntime().halt(1);
         }
         try {
           if (retrySplitting) {
-            Thread.sleep(conf.getInt(
-              "hbase.hlog.split.failure.retry.interval", 30 * 1000));
+            Thread.sleep(conf.getInt("hbase.hlog.split.failure.retry.interval", 30 * 1000));
           }
         } catch (InterruptedException e) {
           LOG.warn("Interrupted, aborting since cannot return w/o splitting");
@@ -246,6 +245,8 @@ public class MasterFileSystem {
         }
       }
     } while (retrySplitting);
+
+    return serverNames;
   }
 
   public void splitLog(final ServerName serverName) throws IOException {

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Mon Feb 25 22:01:57 2013
@@ -425,6 +425,16 @@ public class ServerManager {
   }
 
   /**
+   * The function is to allow master to submit known dead servers into SSH
+   * @param serverName
+   */
+  void processDeadServer(final ServerName serverName) {
+    this.deadservers.add(serverName);
+    this.services.getExecutorService().submit(
+      new ServerShutdownHandler(this.master, this.services, this.deadservers, serverName, true));
+  }
+
+  /**
    * Expire the servers which died during master's initialization. It will be
    * called after HMaster#assignRootAndMeta.
    * @throws IOException

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java Mon Feb 25 22:01:57 2013
@@ -179,6 +179,10 @@ public class ServerShutdownHandler exten
   public void process() throws IOException {
     final ServerName serverName = this.serverName;
     try {
+      if (this.server.isStopped()) {
+        throw new IOException("Server is stopped");
+      }
+
       try {
         if (this.shouldSplitHlog) {
           LOG.info("Splitting logs for " + serverName);

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Mon Feb 25 22:01:57 2013
@@ -66,12 +66,12 @@ import org.apache.hadoop.hbase.HBaseConf
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HealthCheckChore;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HealthCheckChore;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MasterAddressTracker;
 import org.apache.hadoop.hbase.NotServingRegionException;
@@ -171,7 +171,6 @@ import org.apache.hadoop.util.Reflection
 import org.apache.hadoop.util.StringUtils;
 import org.apache.zookeeper.KeeperException;
 import org.codehaus.jackson.map.ObjectMapper;
-import org.joda.time.field.MillisDurationField;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
@@ -1678,7 +1677,9 @@ public class HRegionServer implements HR
   @Override
   public void stop(final String msg) {
     try {
-      this.rsHost.preStop(msg);
+      if (this.rsHost != null) {
+        this.rsHost.preStop(msg);
+      }
       this.stopped = true;
       LOG.info("STOPPED: " + msg);
       // Wakes run() if it is sleeping

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java Mon Feb 25 22:01:57 2013
@@ -45,7 +45,7 @@ public abstract class ZooKeeperNodeTrack
   /** Used to abort if a fatal error occurs */
   protected final Abortable abortable;
 
-  private boolean stopped = false;
+  private volatile boolean stopped = false;
 
   /**
    * Constructs a new ZK node tracker.

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java Mon Feb 25 22:01:57 2013
@@ -19,22 +19,29 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
+import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_wait_for_zk_delete;
+import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_final_transistion_failed;
+import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_preempt_task;
+import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
+import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_done;
+import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_err;
+import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_resigned;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableSet;
 import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -43,7 +50,13 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
@@ -54,8 +67,9 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -82,15 +96,21 @@ public class TestDistributedLogSplitting
   Configuration conf;
   HBaseTestingUtility TEST_UTIL;
 
+
   private void startCluster(int num_rs) throws Exception{
+    conf = HBaseConfiguration.create();
+    startCluster(NUM_MASTERS, num_rs, conf);
+  }
+
+  private void startCluster(int num_master, int num_rs, Configuration inConf) throws Exception {
     ZKSplitLog.Counters.resetCounters();
     LOG.info("Starting cluster");
-    conf = HBaseConfiguration.create();
+    this.conf = inConf;
     conf.getLong("hbase.splitlog.max.resubmit", 0);
     // Make the failure test faster
     conf.setInt("zookeeper.recovery.retry", 0);
     TEST_UTIL = new HBaseTestingUtility(conf);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, num_rs);
+    TEST_UTIL.startMiniCluster(num_master, num_rs);
     cluster = TEST_UTIL.getHBaseCluster();
     LOG.info("Waiting for active/ready master");
     cluster.waitForActiveAndReadyMaster();
@@ -102,6 +122,10 @@ public class TestDistributedLogSplitting
 
   @After
   public void after() throws Exception {
+    for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
+      mt.getMaster().abort("closing...", new Exception("Trace info"));
+    }
+
     TEST_UTIL.shutdownMiniCluster();
   }
 
@@ -205,6 +229,89 @@ public class TestDistributedLogSplitting
     assertEquals(NUM_LOG_LINES, count);
   }
 
+  @Test(timeout = 300000)
+  public void testMasterStartsUpWithLogSplittingWork() throws Exception {
+    LOG.info("testMasterStartsUpWithLogSplittingWork");
+    Configuration curConf = HBaseConfiguration.create();
+    curConf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
+    startCluster(2, NUM_RS, curConf);
+
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    HTable ht = installTable(zkw, "table", "f", NUM_REGIONS_TO_CREATE);
+
+    List<HRegionInfo> regions = null;
+    HRegionServer hrs = null;
+    for (int i = 0; i < NUM_RS; i++) {
+      boolean isCarryingMeta = false;
+      hrs = rsts.get(i).getRegionServer();
+      regions = hrs.getOnlineRegions();
+      for (HRegionInfo region : regions) {
+        if (region.isRootRegion() || region.isMetaRegion()) {
+          isCarryingMeta = true;
+          break;
+        }
+      }
+      if (isCarryingMeta) {
+        continue;
+      }
+      break;
+    }
+
+    LOG.info("#regions = " + regions.size());
+    Iterator<HRegionInfo> it = regions.iterator();
+    while (it.hasNext()) {
+      HRegionInfo region = it.next();
+      if (region.isMetaTable()) {
+        it.remove();
+      }
+    }
+    makeHLog(hrs.getWAL(), regions, "table", NUM_LOG_LINES, 100);
+
+    // abort master
+    abortMaster(cluster);
+
+    // abort RS
+    int numRS = cluster.getLiveRegionServerThreads().size();
+    LOG.info("Aborting region server: " + hrs.getServerName());
+    hrs.abort("testing");
+
+    // wait for the RS dies
+    long start = EnvironmentEdgeManager.currentTimeMillis();
+    while (cluster.getLiveRegionServerThreads().size() > (numRS - 1)) {
+      if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
+        assertTrue(false);
+      }
+      Thread.sleep(200);
+    }
+
+    Thread.sleep(2000);
+    LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size());
+    
+    startMasterAndWaitUntilLogSplit(cluster);
+    
+    start = EnvironmentEdgeManager.currentTimeMillis();
+    while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
+      if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
+        assertTrue("Timedout", false);
+      }
+      Thread.sleep(200);
+    }
+
+    LOG.info("Current Open Regions After Master Node Starts Up:"
+        + getAllOnlineRegions(cluster).size());
+
+    assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+
+    ht.close();
+  }
+
   /**
    * The original intention of this test was to force an abort of a region
    * server and to make sure that the failure path in the region servers is
@@ -393,33 +500,40 @@ public class TestDistributedLogSplitting
       List<HRegionInfo> hris, String tname,
       int num_edits, int edit_size) throws IOException {
 
+    // remove root and meta region
+    hris.remove(HRegionInfo.ROOT_REGIONINFO);
+    hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
     byte[] table = Bytes.toBytes(tname);
     HTableDescriptor htd = new HTableDescriptor(tname);
     byte[] value = new byte[edit_size];
     for (int i = 0; i < edit_size; i++) {
-      value[i] = (byte)('a' + (i % 26));
+      value[i] = (byte) ('a' + (i % 26));
     }
     int n = hris.size();
     int[] counts = new int[n];
-    int j = 0;
     if (n > 0) {
       for (int i = 0; i < num_edits; i += 1) {
         WALEdit e = new WALEdit();
-        byte [] row = Bytes.toBytes("r" + Integer.toString(i));
-        byte [] family = Bytes.toBytes("f");
-        byte [] qualifier = Bytes.toBytes("c" + Integer.toString(i));
-        e.add(new KeyValue(row, family, qualifier,
-            System.currentTimeMillis(), value));
-        j++;
-        log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd);
-        counts[j % n] += 1;
+        HRegionInfo curRegionInfo = hris.get(i % n);
+        byte[] startRow = curRegionInfo.getStartKey();
+        if (startRow == null || startRow.length == 0) {
+          startRow = new byte[] { 0, 0, 0, 0, 1 };
+        }
+        byte[] row = Bytes.incrementBytes(startRow, counts[i % n]);
+        row = Arrays.copyOfRange(row, 3, 8); // use last 5 bytes because
+                                             // HBaseTestingUtility.createMultiRegions use 5 bytes
+                                             // key
+        byte[] family = Bytes.toBytes("f");
+        byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
+        e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
+        log.append(curRegionInfo, table, e, System.currentTimeMillis(), htd);
+        counts[i % n] += 1;
       }
     }
     log.sync();
     log.close();
     for (int i = 0; i < n; i++) {
-      LOG.info("region " + hris.get(i).getRegionNameAsString() +
-          " has " + counts[i] + " edits");
+      LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits");
     }
     return;
   }
@@ -479,6 +593,30 @@ public class TestDistributedLogSplitting
     assertTrue(false);
   }
 
+  private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException {
+    for (MasterThread mt : cluster.getLiveMasterThreads()) {
+      if (mt.getMaster().isActiveMaster()) {
+        mt.getMaster().abort("Aborting for tests", new Exception("Trace info"));
+        mt.join();
+        break;
+      }
+    }
+    LOG.debug("Master is aborted");
+  }
+
+  private void startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster)
+      throws IOException, InterruptedException {
+    cluster.startMaster();
+    HMaster master = cluster.getMaster();
+    while (!master.isInitialized()) {
+      Thread.sleep(100);
+    }
+    ServerManager serverManager = master.getServerManager();
+    while (serverManager.areDeadServersInProgress()) {
+      Thread.sleep(100);
+    }
+  }
+
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java?rev=1449920&r1=1449919&r2=1449920&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java Mon Feb 25 22:01:57 2013
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertTru
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,19 +36,16 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.TestMasterFailover;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -102,19 +98,6 @@ public class TestRSKilledWhenMasterIniti
       super(conf);
     }
 
-    @Override
-    protected void splitLogAfterStartup(MasterFileSystem mfs) {
-      super.splitLogAfterStartup(mfs);
-      logSplit = true;
-      // If "TestingMaster.sleep" is set, sleep after log split.
-      if (getConfiguration().getBoolean("TestingMaster.sleep", false)) {
-        int duration = getConfiguration().getInt(
-            "TestingMaster.sleep.duration", 0);
-        Threads.sleep(duration);
-      }
-    }
-
-
     public boolean isLogSplitAfterStartup() {
       return logSplit;
     }
@@ -249,11 +232,13 @@ public class TestRSKilledWhenMasterIniti
   private TestingMaster startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster)
       throws IOException, InterruptedException {
     TestingMaster master = (TestingMaster) cluster.startMaster().getMaster();
-    while (!master.isLogSplitAfterStartup()) {
+    while (!master.isInitialized()) {
+      Thread.sleep(100);
+    }
+    ServerManager serverManager = cluster.getMaster().getServerManager();
+    while (serverManager.areDeadServersInProgress()) {
       Thread.sleep(100);
     }
-    LOG.debug("splitted:" + master.isLogSplitAfterStartup() + ",initialized:"
-        + master.isInitialized());
     return master;
   }