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/08/07 08:22:11 UTC

svn commit: r983177 - in /hbase/branches/0.90_master_rewrite: ./ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/master/handler/ sr...

Author: stack
Date: Sat Aug  7 06:22:10 2010
New Revision: 983177

URL: http://svn.apache.org/viewvc?rev=983177&view=rev
Log:
In the main, below is about removing MasterController and using Server 
instead.  Server now implements Stoppable interface, alot of what
MasterController provided.  Other workarounds made it so could get
rid of stuff MasterController used to carry.

M src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
  Readded exception removed on last commit.  Meant to add it here at top level rather than remove it.
M src/main/java/org/apache/hadoop/hbase/Stoppable.java
  (isStopped) Added.
M  src/main/java/org/apache/hadoop/hbase/Chore.java
  Changed to use Stoppable instead of AtomicBoolean.
M  src/main/java/org/apache/hadoop/hbase/Server.java
  Implements Stoppable.
M src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
  Uses Server now rather than MasterController (latter has been removed
  Its no longer necessary).
M src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
  Moved into here the running of a cluster shutdown. Moved
  the flag that used to be over in Master in here (was an 
  AtomicBoolean).    Removed a bunch of commented out code.
M src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
  Removed MasterController in favor of Server (Server now has
  Stoppable interface, a good bit of what MasterController did)
M src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
  Removed MasterController in favor of Server
M src/main/java/org/apache/hadoop/hbase/master/HMaster.java
  Removed atomic booleans no longer needed now we have Stoppable.
  The master run loop has nothing to do now.
  Made it so thre is notion of stop cluster and then stop master
  (Needs to be tested).  Pass through shutdown of cluster to
  ServerManager. It runs it.  Master now just knows its happening.
M  src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java
M src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java
M src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java
M src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
M src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
M src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java
M src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java
  Replaced ServerController with Master. 
M src/main/java/org/apache/hadoop/hbase/HServerAddress.java
  CHANGED toString to return hostname rather than IP..
M src/main/java/org/apache/hadoop/hbase/util/Sleeper.java
  Use Stoppable instead of AtomicBoolean.
M src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
  Added implementation of new stopMaster method.
M pom.xml
  Fixed up errors and warnings in the pom.  Brought over the pom from trunk.

Added:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
Modified:
    hbase/branches/0.90_master_rewrite/pom.xml
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Chore.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HServerAddress.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Stoppable.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java

Modified: hbase/branches/0.90_master_rewrite/pom.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/pom.xml?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/pom.xml (original)
+++ hbase/branches/0.90_master_rewrite/pom.xml Sat Aug  7 06:22:10 2010
@@ -477,6 +477,7 @@
   <properties>
     <compileSource>1.6</compileSource>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <hbase.version>0.89.0-SNAPSHOT</hbase.version>
     <hadoop.version>0.20.3-append-r964955-1240</hadoop.version>
 
     <commons-cli.version>1.2</commons-cli.version>

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Chore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Chore.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Chore.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Chore.java Sat Aug  7 06:22:10 2010
@@ -19,8 +19,6 @@
  */
 package org.apache.hadoop.hbase;
 
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.util.Sleeper;
@@ -32,24 +30,24 @@ import org.apache.hadoop.hbase.util.Slee
  * Implementers just need to add checking if there is work to be done and if
  * so, do it.  Its the base of most of the chore threads in hbase.
  *
- * Don't subclass Chore if the task relies on being woken up for something to
+ * <p>Don't subclass Chore if the task relies on being woken up for something to
  * do, such as an entry being added to a queue, etc.
  */
 public abstract class Chore extends Thread {
   private final Log LOG = LogFactory.getLog(this.getClass());
   private final Sleeper sleeper;
-  protected volatile AtomicBoolean stop;
+  protected final Stoppable stopper;
 
   /**
    * @param p Period at which we should run.  Will be adjusted appropriately
    * should we find work and it takes time to complete.
-   * @param s When this flag is set to true, this thread will cleanup and exit
-   * cleanly.
+   * @param stopper When {@link Stoppable#isStopped()} is true, this thread will
+   * cleanup and exit cleanly.
    */
-  public Chore(String name, final int p, final AtomicBoolean s) {
+  public Chore(String name, final int p, final Stoppable stopper) {
     super(name);
-    this.sleeper = new Sleeper(p, s);
-    this.stop = s;
+    this.sleeper = new Sleeper(p, stopper);
+    this.stopper = stopper;
   }
 
   /**
@@ -59,7 +57,7 @@ public abstract class Chore extends Thre
   public void run() {
     try {
       boolean initialChoreComplete = false;
-      while (!this.stop.get()) {
+      while (!this.stopper.isStopped()) {
         long startTime = System.currentTimeMillis();
         try {
           if (!initialChoreComplete) {
@@ -69,15 +67,14 @@ public abstract class Chore extends Thre
           }
         } catch (Exception e) {
           LOG.error("Caught exception", e);
-          if (this.stop.get()) {
+          if (this.stopper.isStopped()) {
             continue;
           }
         }
         this.sleeper.sleep(startTime);
       }
     } catch (Throwable t) {
-      LOG.fatal("Caught error. Starting shutdown.", t);
-      this.stop.set(true);
+      LOG.fatal(getName() + "error", t);
     } finally {
       LOG.info(getName() + " exiting");
     }

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HServerAddress.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HServerAddress.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HServerAddress.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HServerAddress.java Sat Aug  7 06:22:10 2010
@@ -44,7 +44,7 @@ public class HServerAddress implements W
    */
   public HServerAddress(InetSocketAddress address) {
     this.address = address;
-    this.stringValue = address.getAddress().getHostAddress() + ":" +
+    this.stringValue = address.getAddress().getHostName() + ":" +
       address.getPort();
   }
 

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java?rev=983177&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java Sat Aug  7 06:22:10 2010
@@ -0,0 +1,43 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+/**
+ * Thrown when an operation requires the root and all meta regions to be online
+ */
+public class NotAllMetaRegionsOnlineException extends DoNotRetryIOException {
+  private static final long serialVersionUID = 6439786157874827523L;
+  /**
+   * default constructor
+   */
+  public NotAllMetaRegionsOnlineException() {
+    super();
+  }
+
+  /**
+   * @param message
+   */
+  public NotAllMetaRegionsOnlineException(String message) {
+    super(message);
+  }
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java Sat Aug  7 06:22:10 2010
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.zookeeper
  * Defines the set of shared functions implemented by HBase servers (Masters
  * and RegionServers).
  */
-public interface Server extends Abortable {
+public interface Server extends Abortable, Stoppable {
   /**
    * Returns the address of the current server.
    */

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Stoppable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Stoppable.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Stoppable.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Stoppable.java Sat Aug  7 06:22:10 2010
@@ -20,7 +20,7 @@
 package org.apache.hadoop.hbase;
 
 /**
- * Implementers are Stoppable
+ * Implementers are Stoppable.
  */
 public interface Stoppable {
   /**
@@ -28,4 +28,9 @@ public interface Stoppable {
    * @param why Why we're stopping.
    */
   public void stop(String why);
+
+  /**
+   * @return True if {@link #stop(String)} has been closed.
+   */
+  public boolean isStopped();
 }
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Sat Aug  7 06:22:10 2010
@@ -929,6 +929,21 @@ public class HBaseAdmin {
   }
 
   /**
+   * Shuts down the current HBase master only.
+   * Does not shutdown the cluster.
+   * @see #shutdown()
+   * @throws IOException if a remote or network exception occurs
+   */
+  public synchronized void stopMaster() throws IOException {
+    isMasterRunning();
+    try {
+      getMaster().stopMaster();
+    } catch (RemoteException e) {
+      throw RemoteExceptionHandler.decodeRemoteException(e);
+    }
+  }
+
+  /**
    * @return cluster status
    * @throws IOException if a remote or network exception occurs
    */

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java Sat Aug  7 06:22:10 2010
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -48,10 +49,10 @@ class ActiveMasterManager extends ZooKee
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
 
   private final HServerAddress address;
-  private final MasterController master;
+  private final Server master;
 
   ActiveMasterManager(ZooKeeperWatcher watcher, HServerAddress address,
-      MasterController master) {
+      Server master) {
     super(watcher);
     this.address = address;
     this.master = master;
@@ -59,14 +60,14 @@ class ActiveMasterManager extends ZooKee
 
   @Override
   public void nodeCreated(String path) {
-    if(path.equals(watcher.masterAddressZNode) && !master.isClosed()) {
+    if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
       handleMasterNodeChange();
     }
   }
 
   @Override
   public void nodeDeleted(String path) {
-    if(path.equals(watcher.masterAddressZNode) && !master.isClosed()) {
+    if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
       handleMasterNodeChange();
     }
   }
@@ -138,7 +139,7 @@ class ActiveMasterManager extends ZooKee
     clusterHasActiveMaster.set(true);
     thisMasterStartedCluster = false;
     synchronized(clusterHasActiveMaster) {
-      while(clusterHasActiveMaster.get() && !master.isClosed()) {
+      while(clusterHasActiveMaster.get() && !master.isStopped()) {
         try {
           clusterHasActiveMaster.wait();
         } catch (InterruptedException e) {
@@ -146,7 +147,7 @@ class ActiveMasterManager extends ZooKee
           LOG.debug("Interrupted waiting for master to die", e);
         }
       }
-      if(master.isClosed()) {
+      if(master.isStopped()) {
         return thisMasterStartedCluster;
       }
       // Try to become active master again now that there is no active master

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Sat Aug  7 06:22:10 2010
@@ -31,7 +31,6 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -42,6 +41,8 @@ import org.apache.hadoop.hbase.HServerAd
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.MetaScanner;
@@ -70,7 +71,7 @@ import org.apache.zookeeper.KeeperExcept
 public class AssignmentManager extends ZooKeeperListener {
   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
 
-  protected MasterController master;
+  protected Server master;
 
   private ServerManager serverManager;
 
@@ -113,7 +114,7 @@ public class AssignmentManager extends Z
    * @param watcher zookeeper watcher
    * @param status master status
    */
-  public AssignmentManager(ZooKeeperWatcher watcher, MasterController master,
+  public AssignmentManager(ZooKeeperWatcher watcher, Server master,
       ServerManager serverManager, CatalogTracker catalogTracker) {
     super(watcher);
     this.master = master;
@@ -122,7 +123,7 @@ public class AssignmentManager extends Z
     Configuration conf = master.getConfiguration();
     this.timeoutMonitor = new TimeoutMonitor(
         conf.getInt("hbase.master.assignment.timeoutmonitor.period", 30000),
-        master.getClosed(),
+        master,
         conf.getInt("hbase.master.assignment.timeoutmonitor.timeout", 15000));
     Threads.setDaemonThreadRunning(timeoutMonitor,
         master.getServerName() + ".timeoutMonitor");
@@ -808,12 +809,13 @@ public class AssignmentManager extends Z
      * operations.  This will deal with retries if for some reason something
      * doesn't happen within the specified timeout.
      * @param period
-     * @param stop
+   * @param stopper When {@link Stoppable#isStopped()} is true, this thread will
+   * cleanup and exit cleanly.
      * @param timeout
      */
-    public TimeoutMonitor(final int period, final AtomicBoolean stop,
+    public TimeoutMonitor(final int period, final Stoppable stopper,
         final int timeout) {
-      super("AssignmentTimeoutMonitor", period, stop);
+      super("AssignmentTimeoutMonitor", period, stopper);
       this.timeout = timeout;
     }
 

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Sat Aug  7 06:22:10 2010
@@ -25,13 +25,11 @@ import java.lang.management.ManagementFa
 import java.lang.management.RuntimeMXBean;
 import java.lang.reflect.Constructor;
 import java.net.UnknownHostException;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -45,6 +43,7 @@ import org.apache.hadoop.hbase.LocalHBas
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
@@ -76,6 +75,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.InfoServer;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Sleeper;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
@@ -100,26 +100,19 @@ import org.apache.zookeeper.Watcher;
  * this case it will tell all regionservers to go down and then wait on them
  * all reporting in that they are down.  This master will then shut itself down.
  * 
- * <p>You can also shutdown just this master.  Call {@link #close()}.
+ * <p>You can also shutdown just this master.  Call {@link #stopMaster()}.
  * 
  * @see HMasterInterface
  * @see HMasterRegionInterface
  * @see Watcher
  */
 public class HMaster extends Thread
-implements HMasterInterface, HMasterRegionInterface, MasterController {
+implements HMasterInterface, HMasterRegionInterface, Server {
+  private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
+
   // MASTER is name of the webapp and the attribute name used stuffing this
   //instance into web context.
   public static final String MASTER = "master";
-  private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
-
-  // We start out with closed flag on.  Its set to off after construction.
-  // Use AtomicBoolean rather than plain boolean because we want other threads
-  // able to set shutdown flag.  Using AtomicBoolean can pass a reference
-  // rather than have them have to know about the hosting Master class.
-  final AtomicBoolean closed = new AtomicBoolean(true);
-  // Set if we are to shutdown the cluster.
-  private final AtomicBoolean clusterShutdown = new AtomicBoolean(false);
 
   // The configuration for the Master
   private final Configuration conf;
@@ -132,8 +125,6 @@ implements HMasterInterface, HMasterRegi
   private ZooKeeperWatcher zooKeeper;
   // Manager and zk listener for master election
   private ActiveMasterManager activeMasterManager;
-  // Cluster status zk tracker and local setter
-  private ClusterStatusTracker clusterStatusTracker;
   // Region server tracker
   private RegionServerTracker regionServerTracker;
 
@@ -152,13 +143,18 @@ implements HMasterInterface, HMasterRegi
   private final AssignmentManager assignmentManager;
   // manager of catalog regions
   private final CatalogTracker catalogTracker;
+  // Cluster status zk tracker and local setter
+  private ClusterStatusTracker clusterStatusTracker;
 
   // True if this is the master that started the cluster.
   boolean clusterStarter;
 
-  /** Set on abort -- usually failure of our zk session
-   */
+  // This flag is for stopping this Master instance.
+  private boolean stopped = false;
+  // Set on abort -- usually failure of our zk session
   private volatile boolean abort = false;
+  // Gets set to the time a cluster shutdown was initiated.
+  private volatile boolean runningClusterShutdown;
 
   /**
    * Initializes the HMaster. The steps are as follows:
@@ -179,42 +175,36 @@ implements HMasterInterface, HMasterRegi
     setName(MASTER + "-" + this.address);
 
     /*
-     * 1. Determine address and initialize RPC server (but do not start)
-     *
-     *    Get the master address and create an RPC server instance.  The RPC
-     *    server ports can be ephemeral.
+     * 1. Determine address and initialize RPC server (but do not start).
+     * The RPC server ports can be ephemeral.
      */
     HServerAddress a = new HServerAddress(getMyAddress(this.conf));
     int numHandlers = conf.getInt("hbase.regionserver.handler.count", 10);
     this.rpcServer = HBaseRPC.getServer(this, a.getBindAddress(), a.getPort(),
-                                        numHandlers, false, conf);
+      numHandlers, false, conf);
     this.address = new HServerAddress(rpcServer.getListenerAddress());
 
     /*
-     * 2. Determine if this is a fresh cluster startup or failed over master
-     *
-     *    This is done by checking for the existence of any ephemeral
-     *    RegionServer nodes in ZooKeeper.  These nodes are created by RSs on
-     *    their initialization but only after they find the primary master.  As
-     *    long as this check is done before we write our address into ZK, this
-     *    will work.  Note that multiple masters could find this to be true on
-     *    startup (none have become active master yet), which is why there is
-     *    an additional check if this master does not become primary on its
-     *    first attempt.
+     * 2. Determine if this is a fresh cluster startup or failed over master.
+     *  This is done by checking for the existence of any ephemeral
+     * RegionServer nodes in ZooKeeper.  These nodes are created by RSs on
+     * their initialization but only after they find the primary master.  As
+     * long as this check is done before we write our address into ZK, this
+     * will work.  Note that multiple masters could find this to be true on
+     * startup (none have become active master yet), which is why there is an
+     * additional check if this master does not become primary on its first attempt.
      */
-    zooKeeper =
-      new ZooKeeperWatcher(conf, MASTER + "-" + getHServerAddress(), this);
+    zooKeeper = new ZooKeeperWatcher(conf, MASTER + "-" + getHServerAddress(), this);
     clusterStarter = 0 == ZKUtil.getNumberOfChildren(zooKeeper, zooKeeper.rsZNode);
 
     /*
      * 3. Initialize master components.
-     *
-     *    This includes the filesystem manager, server manager, region manager,
-     *    metrics, queues, sleeper, etc...
+     * This includes the filesystem manager, server manager, region manager,
+     * metrics, queues, sleeper, etc...
      */
+    // TODO: Do this using Dependency Injection, using PicoContainer or Spring.
     this.connection = ServerConnectionManager.getConnection(conf);
     this.metrics = new MasterMetrics(this.getName());
-    clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this);
     fileSystemManager = new MasterFileSystem(this);
     serverManager = new ServerManager(this, metrics, fileSystemManager);
     regionServerTracker = new RegionServerTracker(zooKeeper, this,
@@ -223,25 +213,22 @@ implements HMasterInterface, HMasterRegi
         conf.getInt("hbase.master.catalog.timeout", 30000));
     assignmentManager = new AssignmentManager(zooKeeper, this,
         serverManager, catalogTracker);
+    clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this);
 
     /*
      * 4. Block on becoming the active master.
+     * We race with other masters to write our address into ZooKeeper.  If we
+     * succeed, we are the primary/active master and finish initialization.
      *
-     *    We race with other masters to write our address into ZooKeeper.  If we
-     *    succeed, we are the primary/active master and finish initialization.
-     *
-     *    If we do not succeed, there is another active master and we should
-     *    now wait until it dies to try and become the next active master.  If
-     *    we do not succeed on our first attempt, this is no longer a cluster
-     *    startup.
+     * If we do not succeed, there is another active master and we should
+     * now wait until it dies to try and become the next active master.  If we
+     * do not succeed on our first attempt, this is no longer a cluster startup.
      */
-    activeMasterManager = new ActiveMasterManager(zooKeeper, address,
-        this);
+    activeMasterManager = new ActiveMasterManager(zooKeeper, address, this);
     zooKeeper.registerListener(activeMasterManager);
     zooKeeper.registerListener(assignmentManager);
     // Wait here until we are the active master
-    boolean thisMasterStartedCluster =
-      activeMasterManager.blockUntilBecomingActiveMaster();
+    clusterStarter = activeMasterManager.blockUntilBecomingActiveMaster();
 
     // TODO: We should start everything here instead of before we become
     //       active master and some after.  Requires change to RS side to not
@@ -252,10 +239,7 @@ implements HMasterInterface, HMasterRegi
     catalogTracker.start();
     clusterStatusTracker.setClusterUp();
 
-    LOG.info("Server has become the active/primary master.  Address is " +
-        this.address.toString());
-
-    // run() is executed next
+    LOG.info("Server active/primary master; " + this.address);
   }
 
   /**
@@ -287,8 +271,6 @@ implements HMasterInterface, HMasterRegi
       }
       // start up all service threads.
       startServiceThreads();
-      // set the master as opened
-      this.closed.set(false);
       // wait for minimum number of region servers to be up
       serverManager.waitForMinServers();
       // assign the root region
@@ -311,42 +293,29 @@ implements HMasterInterface, HMasterRegi
         assignmentManager.processFailover();
       }
       LOG.info("HMaster started on " + this.address.toString());
-      while (!this.closed.get()) {
-        // check if we should be shutting down
-        if (this.clusterShutdown.get()) {
-          if (this.serverManager.numServers() == 0) {
-            startShutdown();
-            break;
-          }
-          else {
-            LOG.debug("Waiting on " +
-             this.serverManager.getOnlineServers().keySet().toString());
+      Sleeper sleeper = new Sleeper(1000, this);
+      int countOfServersStillRunning = this.serverManager.numServers();
+      while (!this.stopped  && !this.abort) {
+        // Master has nothing to do
+        sleeper.sleep();
+        if (this.runningClusterShutdown) {
+          int count = this.serverManager.numServers();
+          if (count != countOfServersStillRunning) {
+            countOfServersStillRunning = count;
+            LOG.info("Regionservers still running; " +
+              countOfServersStillRunning);
           }
         }
-
-        // wait for an interruption
-        // TODO: something better?  we need to check closed and shutdown?
-        synchronized(this.clusterShutdown) {
-          try {
-            this.clusterShutdown.wait();
-          } catch(InterruptedException e) {
-            LOG.debug("Main thread interrupted", e);
-          }
-        }
-
-        // TODO: should be check file system like we used to?
-        if (!fileSystemManager.checkFileSystem()) {
-          break;
-        }
-        // Continue run loop
       }
     } catch (Throwable t) {
-      LOG.fatal("Unhandled exception. Starting shutdown.", t);
-      setClosed();
+      abort("Unhandled exception. Starting shutdown.", t);
     }
 
-    // Wait for all the remaining region servers to report in.
-    if (!this.abort) this.serverManager.letRegionServersShutdown();
+    // Wait for all the remaining region servers to report in IFF we were
+    // running a cluster shutdown AND we were NOT aborting.
+    if (!this.abort && this.runningClusterShutdown) {
+      this.serverManager.letRegionServersShutdown();
+    }
 
     // Clean up and close up shop
     if (this.infoServer != null) {
@@ -364,16 +333,7 @@ implements HMasterInterface, HMasterRegi
     LOG.info("HMaster main thread exiting");
   }
 
-  /**
-   * Sets whether this is a cluster startup or not.  Used by the
-   * {@link ActiveMasterManager} to set to false if we determine another master
-   * has become the primary.
-   * @param isClusterStartup false if another master became active before us
-   */
-  public void setClusterStartup(boolean isClusterStartup) {
-    this.clusterStarter = isClusterStartup;
-  }
-
+  @Override
   public HServerAddress getHServerAddress() {
     return address;
   }
@@ -394,7 +354,7 @@ implements HMasterInterface, HMasterRegi
 
   /** @return HServerAddress of the master server */
   public HServerAddress getMasterAddress() {
-    return this.address;
+    return getHServerAddress();
   }
 
   public long getProtocolVersion(String protocol, long clientVersion) {
@@ -417,23 +377,6 @@ implements HMasterInterface, HMasterRegi
     return this.serverManager;
   }
 
-  @Override
-  public boolean isClusterShutdown() {
-    return this.clusterShutdown.get();
-  }
-
-  public void setClosed() {
-    this.closed.set(true);
-  }
-
-  public AtomicBoolean getClosed() {
-    return this.closed;
-  }
-
-  public boolean isClosed() {
-    return this.closed.get();
-  }
-
   public ServerConnection getServerConnection() {
     return this.connection;
   }
@@ -445,12 +388,6 @@ implements HMasterInterface, HMasterRegi
   public ZooKeeperWatcher getZooKeeperWatcher() {
     return this.zooKeeper;
   }
-  /**
-   * Get the HBase root dir - needed by master_jsp.java
-   */
-  public Path getRootDir() {
-    return fileSystemManager.getRootDir();
-  }
 
   /*
    * Start up all services. If any of these threads gets an unhandled exception
@@ -463,16 +400,16 @@ implements HMasterInterface, HMasterRegi
     try {
       // Start the executor service pools
       HBaseExecutorServiceType.MASTER_OPEN_REGION.startExecutorService(
-          getServerName(),
+        getServerName(),
           conf.getInt("hbase.master.executor.openregion.threads", 5));
       HBaseExecutorServiceType.MASTER_CLOSE_REGION.startExecutorService(
-          getServerName(),
+        getServerName(),
           conf.getInt("hbase.master.executor.closeregion.threads", 5));
       HBaseExecutorServiceType.MASTER_SERVER_OPERATIONS.startExecutorService(
-          getServerName(),
+        getServerName(),
           conf.getInt("hbase.master.executor.serverops.threads", 5));
       HBaseExecutorServiceType.MASTER_TABLE_OPERATIONS.startExecutorService(
-          getServerName(),
+        getServerName(),
           conf.getInt("hbase.master.executor.tableops.threads", 5));
 
       // Put up info server.
@@ -498,24 +435,18 @@ implements HMasterInterface, HMasterRegi
         }
       }
       // Something happened during startup. Shut things down.
-      setClosed();
-      LOG.error("Failed startup", e);
+      abort("Failed startup", e);
     }
   }
 
-  /*
-   * Start shutting down the master
-   */
-  void startShutdown() {
-    setClosed();
-    this.serverManager.notifyServers();
-  }
-
   public MapWritable regionServerStartup(final HServerInfo serverInfo)
   throws IOException {
     // Set the ip into the passed in serverInfo.  Its ip is more than likely
     // not the ip that the master sees here.  See at end of this method where
     // we pass it back to the regionserver by setting "hbase.regionserver.address"
+    // Everafter, the HSI combination 'server name' is what uniquely identifies
+    // the incoming RegionServer.  No more DNS meddling of this little messing
+    // belose.
     String rsAddress = HBaseServer.getRemoteAddress();
     serverInfo.setServerAddress(new HServerAddress(rsAddress,
       serverInfo.getServerAddress().getPort()));
@@ -541,6 +472,7 @@ implements HMasterInterface, HMasterRegi
     return mw;
   }
 
+  @Override
   public HMsg [] regionServerReport(HServerInfo serverInfo, HMsg msgs[],
     HRegionInfo[] mostLoadedRegions)
   throws IOException {
@@ -561,7 +493,7 @@ implements HMasterInterface, HMasterRegi
   }
 
   public boolean isMasterRunning() {
-    return !this.closed.get();
+    return !isStopped();
   }
 
   public void createTable(HTableDescriptor desc, byte [][] splitKeys)
@@ -794,13 +726,10 @@ implements HMasterInterface, HMasterRegi
 
   @Override
   public void shutdown() {
-    LOG.info("Cluster shutdown requested. Starting to quiesce servers");
-    synchronized (clusterShutdown) {
-      clusterShutdown.set(true);
-      clusterShutdown.notifyAll();
-    }
+    this.serverManager.shutdownCluster();
+    this.runningClusterShutdown = true;
     try {
-      clusterStatusTracker.setClusterDown();
+      this.clusterStatusTracker.setClusterDown();
     } catch (KeeperException e) {
       LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
     }
@@ -808,7 +737,17 @@ implements HMasterInterface, HMasterRegi
 
   @Override
   public void stopMaster() {
-    // TODO
+    stop("Stopped by " + Thread.currentThread().getName());
+  }
+
+  @Override
+  public void stop(String why) {
+    this.stopped = true;
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
   }
 
   public void assignRegion(HRegionInfo hri) {
@@ -870,8 +809,9 @@ implements HMasterInterface, HMasterRegi
     Configuration conf = HBaseConfiguration.create();
     // Process command-line args.
     for (String cmd: args) {
-
       if (cmd.startsWith("--minServers=")) {
+        // How many servers must check in before we'll start assigning.
+        // TODO: Verify works with new master regime.
         conf.setInt("hbase.regions.server.count.min",
           Integer.valueOf(cmd.substring(13)));
         continue;
@@ -889,8 +829,7 @@ implements HMasterInterface, HMasterRegi
           // If 'local', defer to LocalHBaseCluster instance.  Starts master
           // and regionserver both in the one JVM.
           if (LocalHBaseCluster.isLocal(conf)) {
-            final MiniZooKeeperCluster zooKeeperCluster =
-              new MiniZooKeeperCluster();
+            final MiniZooKeeperCluster zooKeeperCluster = new MiniZooKeeperCluster();
             File zkDataPath = new File(conf.get("hbase.zookeeper.property.dataDir"));
             int zkClientPort = conf.getInt("hbase.zookeeper.property.clientPort", 0);
             if (zkClientPort == 0) {
@@ -916,10 +855,6 @@ implements HMasterInterface, HMasterRegi
             cluster.startup();
           } else {
             HMaster master = constructMaster(masterClass, conf);
-            if (master.clusterShutdown.get()) {
-              LOG.info("Won't bring the Master up as a shutdown is requested");
-              return;
-            }
             master.start();
           }
         } catch (Throwable t) {

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Sat Aug  7 06:22:10 2010
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
@@ -51,7 +52,7 @@ public class MasterFileSystem {
   // HBase configuration
   Configuration conf;
   // master status
-  MasterController masterStatus;
+  Server master;
   // Keep around for convenience.
   private final FileSystem fs;
   // Is the fileystem ok?
@@ -63,9 +64,9 @@ public class MasterFileSystem {
   // create the split log lock
   final Lock splitLogLock = new ReentrantLock();
 
-  public MasterFileSystem(MasterController masterStatus) throws IOException {
-    this.conf = masterStatus.getConfiguration();
-    this.masterStatus = masterStatus;
+  public MasterFileSystem(Server master) throws IOException {
+    this.conf = master.getConfiguration();
+    this.master = master;
     // Set filesystem to be that of this.rootdir else we get complaints about
     // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
     // default localfs.  Presumption is that rootdir is fully-qualified before
@@ -119,8 +120,7 @@ public class MasterFileSystem {
       try {
         FSUtils.checkFileSystemAvailable(this.fs);
       } catch (IOException e) {
-        LOG.fatal("Shutting down HBase cluster: file system not available", e);
-        masterStatus.setClosed();
+        master.abort("Shutting down HBase cluster: file system not available", e);
         this.fsOk = false;
       }
     }

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/OldLogsCleaner.java Sat Aug  7 06:22:10 2010
@@ -27,11 +27,10 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
+import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.regex.Pattern;
 
 /**
  * This Chore, everytime it runs, will clear the logs in the old logs folder
@@ -53,12 +52,13 @@ public class OldLogsCleaner extends Chor
   /**
    *
    * @param p the period of time to sleep between each run
-   * @param s the stopper boolean
+   * @param stopper When {@link Stoppable#isStopped()} is true, this thread will
+   * cleanup and exit cleanly.
    * @param conf configuration to use
    * @param fs handle to the FS
    * @param oldLogDir the path to the archived logs
    */
-  public OldLogsCleaner(final int p, final AtomicBoolean s,
+  public OldLogsCleaner(final int p, final Stoppable s,
                         Configuration conf, FileSystem fs,
                         Path oldLogDir) {
     super("OldLogsCleaner", p, s);
@@ -70,6 +70,7 @@ public class OldLogsCleaner extends Chor
     this.logCleaner = getLogCleaner();
   }
 
+  @SuppressWarnings("unchecked")
   private LogCleanerDelegate getLogCleaner() {
     try {
       Class c = Class.forName(conf.get("hbase.master.logcleanerplugin.impl",

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Sat Aug  7 06:22:10 2010
@@ -28,7 +28,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -42,6 +41,8 @@ import org.apache.hadoop.hbase.HServerIn
 import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
@@ -68,6 +69,8 @@ public class ServerManager {
 
   private final AtomicInteger quiescedServers = new AtomicInteger(0);
   private final AtomicInteger availableServers = new AtomicInteger(0);
+  // Set if we are to shutdown the cluster.
+  private volatile boolean clusterShutdown = false;
 
   /** The map of known server names to server info */
   private final Map<String, HServerInfo> onlineServers =
@@ -90,7 +93,7 @@ public class ServerManager {
   private final Set<String> deadServers =
     Collections.synchronizedSet(new HashSet<String>());
 
-  private MasterController masterStatus;
+  private Server master;
 
   private MasterMetrics masterMetrics;
 
@@ -105,8 +108,8 @@ public class ServerManager {
    * TODO: Make this a metric; dump metrics into log.
    */
   class ServerMonitor extends Chore {
-    ServerMonitor(final int period, final AtomicBoolean stop) {
-      super("ServerMonitor", period, stop);
+    ServerMonitor(final int period, final Stoppable stopper) {
+      super("ServerMonitor", period, stopper);
     }
 
     @Override
@@ -138,30 +141,29 @@ public class ServerManager {
 
   /**
    * Constructor.
-   * @param masterStatus
+   * @param master
+   * @param masterMetrics
+   * @param masterFileSystem
    */
-  public ServerManager(MasterController masterStatus,
+  public ServerManager(Server master,
       MasterMetrics masterMetrics,
-      MasterFileSystem fileSystemManager) {
-    this.masterStatus = masterStatus;
+      MasterFileSystem masterFileSystem) {
+    this.master = master;
     this.masterMetrics = masterMetrics;
-    Configuration c = masterStatus.getConfiguration();
+    Configuration c = master.getConfiguration();
     int metaRescanInterval = c.getInt("hbase.master.meta.thread.rescanfrequency",
       60 * 1000);
     this.minimumServerCount = c.getInt("hbase.regions.server.count.min", 1);
-    this.serverMonitorThread = new ServerMonitor(metaRescanInterval,
-      this.masterStatus.isClusterShutdown());
+    this.serverMonitorThread = new ServerMonitor(metaRescanInterval, master);
     String n = Thread.currentThread().getName();
     Threads.setDaemonThreadRunning(this.serverMonitorThread,
       n + ".serverMonitor");
     this.oldLogCleaner = new OldLogsCleaner(
       c.getInt("hbase.master.meta.thread.rescanfrequency",60 * 1000),
-        this.masterStatus.isClusterShutdown(), c,
-        fileSystemManager.getFileSystem(),
-        fileSystemManager.getOldLogDir());
+      master, c, masterFileSystem.getFileSystem(),
+      masterFileSystem.getOldLogDir());
     Threads.setDaemonThreadRunning(oldLogCleaner,
       n + ".oldLogCleaner");
-
   }
 
   /**
@@ -281,14 +283,13 @@ public class ServerManager {
         this.quiescedServers.incrementAndGet();
       }
     }
-    if (this.masterStatus.isClusterShutdown()) {
+    if (this.clusterShutdown) {
       if (quiescedServers.get() >= availableServers.get()) {
         // If the only servers we know about are meta servers, then we can
         // proceed with shutdown
-        LOG.info("All user tables quiesced. Proceeding with shutdown");
-        this.masterStatus.shutdown();
-      }
-      if (!this.masterStatus.isClosed()) {
+        this.master.stop("All user tables quiesced. Proceeding with shutdown");
+        notifyOnlineServers();
+      } else if (!this.master.isStopped()) {
         if (msgs.length > 0 &&
             msgs[0].isType(HMsg.Type.MSG_REPORT_QUIESCED)) {
           // Server is already quiesced, but we aren't ready to shut down
@@ -299,7 +300,7 @@ public class ServerManager {
         return new HMsg [] {HMsg.REGIONSERVER_QUIESCE};
       }
     }
-    if (this.masterStatus.isClosed()) {
+    if (this.master.isStopped()) {
       // Tell server to shut down if we are shutting down.  This should
       // happen after check of MSG_REPORT_EXITING above, since region server
       // will send us one of these messages after it gets MSG_REGIONSERVER_STOP
@@ -330,7 +331,7 @@ public class ServerManager {
 
       synchronized (this.onlineServers) {
         removeServerInfo(info.getServerName());
-        notifyServers();
+        notifyOnlineServers();
       }
 
       return new HMsg[] {HMsg.REGIONSERVER_STOP};
@@ -363,7 +364,7 @@ public class ServerManager {
           ": MSG_REPORT_EXITING");
         // Get all the regions the server was serving reassigned
         // (if we are not shutting down).
-        if (!masterStatus.getClosed().get()) {
+        if (!master.isStopped()) {
           for (int i = 1; i < msgs.length; i++) {
             LOG.info("Processing " + msgs[i] + " from " +
               serverInfo.getServerName());
@@ -418,86 +419,16 @@ public class ServerManager {
   }
 
   /**
-   * A region has split.
-   *
-   * @param region
-   * @param splitA
-   * @param splitB
-   * @param returnMsgs
-   */
-  private void processSplitRegion(HRegionInfo region, HRegionInfo a, HRegionInfo b) {
-//    synchronized (masterStatus.getRegionManager()) {
-//      // TODO: This will just delete the parent off fs?  or will this completely
-//      //       go away and we will rely on RS doing stuff and new ZK nodes
-//      // Cancel any actions pending for the affected region.
-//      // This prevents the master from sending a SPLIT message if the table
-//      // has already split by the region server.
-//      this.masterStatus.getRegionManager().endActions(region.getRegionName());
-//      assignSplitDaughter(a);
-//      assignSplitDaughter(b);
-//      if (region.isMetaTable()) {
-//        // A meta region has split.
-//        this. masterStatus.getRegionManager().offlineMetaRegionWithStartKey(region.getStartKey());
-//        this.masterStatus.getRegionManager().incrementNumMetaRegions();
-//      }
-//    }
-  }
-
-  /**
-   * Assign new daughter-of-a-split UNLESS its already been assigned.
-   * It could have been assigned already in rare case where there was a large
-   * gap between insertion of the daughter region into .META. by the
-   * splitting regionserver and receipt of the split message in master (See
-   * HBASE-1784).
-   * @param hri Region to assign.
-   */
-//  private void assignSplitDaughter(final HRegionInfo hri) {
-//    MetaRegion mr =
-//      this.masterStatus.getRegionManager().getFirstMetaRegionForRegion(hri);
-//    Get g = new Get(hri.getRegionName());
-//    g.addFamily(HConstants.CATALOG_FAMILY);
-//    try {
-//      HRegionInterface server =
-//        this.masterStatus.getServerConnection().getHRegionConnection(mr.getServer());
-//      Result r = server.get(mr.getRegionName(), g);
-//      // If size > 3 -- presume regioninfo, startcode and server -- then presume
-//      // that this daughter already assigned and return.
-//      if (r.size() >= 3) {
-//        return;
-//      }
-//    } catch (IOException e) {
-//      LOG.warn("Failed get on " + HConstants.CATALOG_FAMILY_STR +
-//        "; possible double-assignment?", e);
-//    }
-//    this.masterStatus.getRegionManager().setUnassigned(hri, false);
-//  }
-
-  /** Update a server load information because it's shutting down*/
+   * @param serverName
+   * @return True if we removed server from the list.
+   */
   private boolean removeServerInfo(final String serverName) {
-    boolean infoUpdated = false;
     HServerInfo info = this.onlineServers.remove(serverName);
-    this.availableServers.decrementAndGet();
-//    // Only update load information once.
-//    // This method can be called a couple of times during shutdown.
-//    if (info != null) {
-//      LOG.info("Removing server's info " + serverName);
-//      this.masterStatus.getRegionManager().offlineMetaServer(info.getServerAddress());
-//
-//      //HBASE-1928: Check whether this server has been transitioning the ROOT table
-//      if (this.masterStatus.getRegionManager().isRootInTransitionOnThisServer(serverName)) {
-//         this.masterStatus.getRegionManager().unsetRootRegion();
-//         this.masterStatus.getRegionManager().reassignRootRegion();
-//      }
-//
-//      //HBASE-1928: Check whether this server has been transitioning the META table
-//      HRegionInfo metaServerRegionInfo = this.masterStatus.getRegionManager().getMetaServerRegionInfo (serverName);
-//      if (metaServerRegionInfo != null) {
-//         this.masterStatus.getRegionManager().setUnassigned(metaServerRegionInfo, true);
-//      }
-//
-//      infoUpdated = true;
-//    }
-    return infoUpdated;
+    if (info != null) {
+      this.availableServers.decrementAndGet();
+      return true;
+    }
+    return false;
   }
 
   /**
@@ -557,10 +488,7 @@ public class ServerManager {
     return null;
   }
 
-  /**
-   * Wakes up threads waiting on serversToServerInfo
-   */
-  public void notifyServers() {
+  private void notifyOnlineServers() {
     synchronized (this.onlineServers) {
       this.onlineServers.notifyAll();
     }
@@ -612,7 +540,7 @@ public class ServerManager {
     this.serverConnections.remove(serverName);
     // Add to dead servers and queue a shutdown processing.
     this.deadServers.add(serverName);
-    new ServerShutdownHandler(masterStatus).submit();
+    new ServerShutdownHandler(master).submit();
     LOG.debug("Added=" + serverName +
       " to dead servers, submitted shutdown handler to be executed");
   }
@@ -710,7 +638,7 @@ public class ServerManager {
       HRegionInterface hri = serverConnections.get(info.getServerName());
       if(hri == null) {
         LOG.info("new connection");
-        hri = masterStatus.getServerConnection().getHRegionConnection(
+        hri = master.getServerConnection().getHRegionConnection(
           info.getServerAddress(), false);
         serverConnections.put(info.getServerName(), hri);
       }
@@ -745,4 +673,9 @@ public class ServerManager {
   public boolean isServerOnline(String serverName) {
     return onlineServers.containsKey(serverName);
   }
-}
+
+  public void shutdownCluster() {
+    LOG.info("Cluster shutdown requested. Starting to quiesce servers");
+    this.clusterShutdown = true;
+  }
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java Sat Aug  7 06:22:10 2010
@@ -25,15 +25,15 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.MasterController;
 
 public class DeleteTableHandler extends TableEventHandler {
   private static final Log LOG = LogFactory.getLog(DeleteTableHandler.class);
 
-  public DeleteTableHandler(byte [] tableName, MasterController server,
+  public DeleteTableHandler(byte [] tableName, Server server,
       CatalogTracker catalogTracker, MasterFileSystem fileManager) {
     super(EventType.C2M_DELETE_TABLE, tableName, server, catalogTracker,
         fileManager);
@@ -52,4 +52,4 @@ public class DeleteTableHandler extends 
     // Delete table from FS
     fileManager.deleteTable(tableName);
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java Sat Aug  7 06:22:10 2010
@@ -22,18 +22,14 @@ package org.apache.hadoop.hbase.master.h
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.MasterController;
 
 public class ModifyTableHandler extends TableEventHandler {
-  private static final Log LOG = LogFactory.getLog(ModifyTableHandler.class);
-
-  public ModifyTableHandler(byte [] tableName, MasterController server,
+  public ModifyTableHandler(byte [] tableName, Server server,
       CatalogTracker catalogTracker, MasterFileSystem fileManager) {
     super(EventType.C2M_MODIFY_TABLE, tableName, server, catalogTracker,
         fileManager);

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java Sat Aug  7 06:22:10 2010
@@ -26,10 +26,10 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.MasterController;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -40,7 +40,7 @@ public class TableAddFamilyHandler exten
   private final HColumnDescriptor familyDesc;
 
   public TableAddFamilyHandler(byte[] tableName, HColumnDescriptor familyDesc,
-      MasterController server, CatalogTracker catalogTracker,
+      Server server, CatalogTracker catalogTracker,
       MasterFileSystem fileManager) {
     super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker,
         fileManager);

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java Sat Aug  7 06:22:10 2010
@@ -25,10 +25,10 @@ import java.util.List;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.MasterController;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -39,7 +39,7 @@ public class TableDeleteFamilyHandler ex
   private final byte [] familyName;
 
   public TableDeleteFamilyHandler(byte[] tableName, byte [] familyName,
-      MasterController server, CatalogTracker catalogTracker,
+      Server server, CatalogTracker catalogTracker,
       MasterFileSystem fileManager) {
     super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker,
         fileManager);

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java Sat Aug  7 06:22:10 2010
@@ -25,13 +25,13 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.MasterController;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -49,7 +49,7 @@ public abstract class TableEventHandler 
   protected final MasterFileSystem fileManager;
 
   public TableEventHandler(EventType eventType, byte [] tableName,
-      MasterController server, CatalogTracker catalogTracker,
+      Server server, CatalogTracker catalogTracker,
       MasterFileSystem fileManager) {
     super(server, eventType);
     this.tableName = tableName;
@@ -89,4 +89,4 @@ public abstract class TableEventHandler 
 
   protected abstract void handleTableOperation(List<HRegionInfo> regions)
   throws IOException;
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java Sat Aug  7 06:22:10 2010
@@ -26,10 +26,10 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.MasterController;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -40,7 +40,7 @@ public class TableModifyFamilyHandler ex
   private final HColumnDescriptor familyDesc;
 
   public TableModifyFamilyHandler(byte[] tableName,
-      HColumnDescriptor familyDesc, MasterController server,
+      HColumnDescriptor familyDesc, Server server,
       CatalogTracker catalogTracker, MasterFileSystem fileManager) {
     super(EventType.C2M_ADD_FAMILY, tableName, server,
         catalogTracker, fileManager);

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java?rev=983177&r1=983176&r2=983177&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java Sat Aug  7 06:22:10 2010
@@ -21,8 +21,7 @@ package org.apache.hadoop.hbase.util;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
-import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.hadoop.hbase.Stoppable;
 
 /**
  * Sleeper for current thread.
@@ -33,7 +32,7 @@ import java.util.concurrent.atomic.Atomi
 public class Sleeper {
   private final Log LOG = LogFactory.getLog(this.getClass().getName());
   private final int period;
-  private final AtomicBoolean stop;
+  private final Stoppable stopper;
   private static final long MINIMAL_DELTA_FOR_LOGGING = 10000;
 
   private final Object sleepLock = new Object();
@@ -41,11 +40,12 @@ public class Sleeper {
 
   /**
    * @param sleep sleep time in milliseconds
-   * @param stop flag for when we stop
+   * @param stopper When {@link Stoppable#isStopped()} is true, this thread will
+   * cleanup and exit cleanly.
    */
-  public Sleeper(final int sleep, final AtomicBoolean stop) {
+  public Sleeper(final int sleep, final Stoppable stopper) {
     this.period = sleep;
-    this.stop = stop;
+    this.stopper = stopper;
   }
 
   /**
@@ -72,7 +72,7 @@ public class Sleeper {
    * will be docked current time minus passed <code>startTime<code>.
    */
   public void sleep(final long startTime) {
-    if (this.stop.get()) {
+    if (this.stopper.isStopped()) {
       return;
     }
     long now = System.currentTimeMillis();
@@ -101,7 +101,7 @@ public class Sleeper {
       } catch(InterruptedException iex) {
         // We we interrupted because we're meant to stop?  If not, just
         // continue ignoring the interruption
-        if (this.stop.get()) {
+        if (this.stopper.isStopped()) {
           return;
         }
       }
@@ -111,4 +111,4 @@ public class Sleeper {
     }
     triggerWake = false;
   }
-}
+}
\ No newline at end of file