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/06 21:57:48 UTC

svn commit: r983113 - in /hbase/branches/0.90_master_rewrite/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/catalog/ main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/ipc/ main/java/org/apache/hadoop...

Author: stack
Date: Fri Aug  6 19:57:47 2010
New Revision: 983113

URL: http://svn.apache.org/viewvc?rev=983113&view=rev
Log:

Renamed master FileSystemMaster as MasterFileSystem (It didn't manage. It
just has  methods used by master accessing FS).

Added Stoppable Interface.  This is probably stopgap.  May remove later.

Removed src/main/java/org/apache/hadoop/hbase/master/MasterController.java
I think its close methods move up into Server as Stoppable implmentations.
The cluster stuff is not needed outside Master (in one place in ServerManager
it needs to know if we're in a cluster shutdown situation -- I can pass in
the AtomicBoolean for that one case).

M  src/main/java/org/apache/hadoop/hbase/master/HMaster.java
Started in on the work to make it so you can shutdown cluster OR shutdown this
master instance.  Up to this the two concepts were intermixed.  Cleaning it up.

Removed src/main/java/org/apache/hadoop/hbase/master/NotAllMetaRegionsOnlineException.java
Unused.

M  src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
Made this return a boolean when returns from blockUntilBecomingMaster rather than 
expect to be ablel to set a boolean on HMaster (Only used in HMaster constructor).
Made it package private.

Removed src/main/java/org/apache/hadoop/hbase/master/InvalidColumnNameException.java
Unused

M src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
Added new stopMaster method


Added:
    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/master/MasterFileSystem.java
Removed:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/FileSystemManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/InvalidColumnNameException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterController.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/NotAllMetaRegionsOnlineException.java
Modified:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.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/ipc/HMasterInterface.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/HMaster.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.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/FSUtils.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java

Added: 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=983113&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Stoppable.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Stoppable.java Fri Aug  6 19:57:47 2010
@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+/**
+ * Implementers are Stoppable
+ */
+public interface Stoppable {
+  /**
+   * Stop this service.
+   * @param why Why we're stopping.
+   */
+  public void stop(String why);
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java?rev=983113&r1=983112&r2=983113&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java Fri Aug  6 19:57:47 2010
@@ -27,10 +27,10 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.client.ServerConnection;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
-import org.apache.hadoop.hbase.master.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -916,7 +916,7 @@ public class HBaseAdmin {
   }
 
   /**
-   * Shuts down the HBase instance
+   * Shuts down the HBase cluster
    * @throws IOException if a remote or network exception occurs
    */
   public synchronized void shutdown() throws IOException {

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java?rev=983113&r1=983112&r2=983113&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java Fri Aug  6 19:57:47 2010
@@ -112,7 +112,7 @@ public interface HMasterInterface extend
    * @throws IOException e
    */
   public void modifyTable(byte[] tableName, HTableDescriptor htd)
-    throws IOException;
+  throws IOException;
 
   /**
    * Shutdown an HBase cluster.
@@ -121,8 +121,15 @@ public interface HMasterInterface extend
   public void shutdown() throws IOException;
 
   /**
+   * Stop HBase Master only.
+   * Does not shutdown the cluster.
+   * @throws IOException e
+   */
+  public void stopMaster() throws IOException;
+
+  /**
    * Return cluster status.
    * @return status object
    */
   public ClusterStatus getClusterStatus();
-}
+}
\ No newline at end of file

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -32,17 +32,17 @@ import org.apache.zookeeper.KeeperExcept
 /**
  * Handles everything on master-side related to master election.
  *
- * Listens and responds to ZooKeeper notifications on the master znode,
- * both nodeCreated and nodeDeleted.
+ * <p>Listens and responds to ZooKeeper notifications on the master znode,
+ * both <code>nodeCreated</code> and <code>nodeDeleted</code>.
  *
- * Contains blocking methods which will hold up backup masters, waiting
+ * <p>Contains blocking methods which will hold up backup masters, waiting
  * for the active master to fail.
  *
- * This class is instantiated in the HMaster constructor and the method
+ * <p>This class is instantiated in the HMaster constructor and the method
  * {@link #blockUntilBecomingActiveMaster()} is called to wait until becoming
  * the active master of the cluster.
  */
-public class ActiveMasterManager extends ZooKeeperListener {
+class ActiveMasterManager extends ZooKeeperListener {
   private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
 
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
@@ -51,10 +51,10 @@ public class ActiveMasterManager extends
   private final MasterController master;
 
   ActiveMasterManager(ZooKeeperWatcher watcher, HServerAddress address,
-      MasterController status) {
+      MasterController master) {
     super(watcher);
     this.address = address;
-    this.master = status;
+    this.master = master;
   }
 
   @Override
@@ -77,11 +77,11 @@ public class ActiveMasterManager extends
    * that the current state of the master node matches the event at the time of
    * our next ZK request.
    *
-   * Uses the watchAndCheckExists method which watches the master address node
+   * <p>Uses the watchAndCheckExists method which watches the master address node
    * regardless of whether it exists or not.  If it does exist (there is an
    * active master), it returns true.  Otherwise it returns false.
    *
-   * A watcher is set which guarantees that this method will get called again if
+   * <p>A watcher is set which guarantees that this method will get called again if
    * there is another change in the master node.
    */
   private void handleMasterNodeChange() {
@@ -113,26 +113,30 @@ public class ActiveMasterManager extends
    *
    * This also makes sure that we are watching the master znode so will be
    * notified if another master dies.
+   * @return False if we did not start up this cluster, another
+   * master did, or if a problem (zookeeper, stop flag has been set on this
+   * Master)
    */
-  void blockUntilBecomingActiveMaster() {
+  boolean blockUntilBecomingActiveMaster() {
+    boolean thisMasterStartedCluster = true;
     // Try to become the active master, watch if there is another master
     try {
       if(ZKUtil.setAddressAndWatch(watcher, watcher.masterAddressZNode,
           address)) {
         // We are the master, return
         clusterHasActiveMaster.set(true);
-        return;
+        return thisMasterStartedCluster;
       }
     } catch (KeeperException ke) {
       master.abort("Received an unexpected KeeperException, aborting", ke);
-      return;
+      return false;
     }
     // There is another active master, this is not a cluster startup
     // and we must wait until the active master dies
     LOG.info("Another master is already the active master, waiting to become " +
-    "the next active master");
+      "the next active master");
     clusterHasActiveMaster.set(true);
-    master.setClusterStartup(false);
+    thisMasterStartedCluster = false;
     synchronized(clusterHasActiveMaster) {
       while(clusterHasActiveMaster.get() && !master.isClosed()) {
         try {
@@ -143,11 +147,12 @@ public class ActiveMasterManager extends
         }
       }
       if(master.isClosed()) {
-        return;
+        return thisMasterStartedCluster;
       }
       // Try to become active master again now that there is no active master
       blockUntilBecomingActiveMaster();
     }
+    return thisMasterStartedCluster;
   }
 
   public void stop() {
@@ -164,4 +169,4 @@ public class ActiveMasterManager extends
       watcher.error("Error deleting our own master address node", e);
     }
   }
-}
+}
\ No newline at end of file

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HServerIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -94,6 +95,13 @@ import org.apache.zookeeper.Watcher;
  * run the cluster.  All others park themselves in their constructor until
  * master or cluster shutdown or until the active master loses its lease in
  * zookeeper.  Thereafter, all running master jostle to take over master role.
+ * 
+ * <p>The Master can be asked shutdown the cluster. See {@link #shutdown()}.  In
+ * 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()}.
+ * 
  * @see HMasterInterface
  * @see HMasterRegionInterface
  * @see Watcher
@@ -110,8 +118,8 @@ implements HMasterInterface, HMasterRegi
   // 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);
-  // TODO: Is this separate flag necessary?
-  private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
+  // Set if we are to shutdown the cluster.
+  private final AtomicBoolean clusterShutdown = new AtomicBoolean(false);
 
   // The configuration for the Master
   private final Configuration conf;
@@ -134,7 +142,7 @@ implements HMasterInterface, HMasterRegi
   // Address of the HMaster
   private final HServerAddress address;
   // file system manager for the master FS operations
-  private final FileSystemManager fileSystemManager;
+  private final MasterFileSystem fileSystemManager;
 
   private final ServerConnection connection;
   // server manager to deal with region server info
@@ -146,7 +154,11 @@ implements HMasterInterface, HMasterRegi
   private final CatalogTracker catalogTracker;
 
   // True if this is the master that started the cluster.
-  boolean isClusterStartup;
+  boolean clusterStarter;
+
+  /** Set on abort -- usually failure of our zk session
+   */
+  private volatile boolean abort = false;
 
   /**
    * Initializes the HMaster. The steps are as follows:
@@ -192,8 +204,7 @@ implements HMasterInterface, HMasterRegi
      */
     zooKeeper =
       new ZooKeeperWatcher(conf, MASTER + "-" + getHServerAddress(), this);
-    isClusterStartup = 0 ==
-      ZKUtil.getNumberOfChildren(zooKeeper, zooKeeper.rsZNode);
+    clusterStarter = 0 == ZKUtil.getNumberOfChildren(zooKeeper, zooKeeper.rsZNode);
 
     /*
      * 3. Initialize master components.
@@ -204,7 +215,7 @@ implements HMasterInterface, HMasterRegi
     this.connection = ServerConnectionManager.getConnection(conf);
     this.metrics = new MasterMetrics(this.getName());
     clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this);
-    fileSystemManager = new FileSystemManager(conf, this);
+    fileSystemManager = new MasterFileSystem(this);
     serverManager = new ServerManager(this, metrics, fileSystemManager);
     regionServerTracker = new RegionServerTracker(zooKeeper, this,
         serverManager);
@@ -229,7 +240,8 @@ implements HMasterInterface, HMasterRegi
     zooKeeper.registerListener(activeMasterManager);
     zooKeeper.registerListener(assignmentManager);
     // Wait here until we are the active master
-    activeMasterManager.blockUntilBecomingActiveMaster();
+    boolean thisMasterStartedCluster =
+      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
@@ -257,8 +269,9 @@ implements HMasterInterface, HMasterRegi
   @Override
   public void run() {
     try {
-      if(isClusterStartup()) {
-        // This is a cluster startup, you are the first master.
+      if (this.clusterStarter) {
+        // This master is starting the cluster (its not a preexisting cluster
+        // that this master is joining).
         // Initialize the filesystem, which does the following:
         //   - Creates the root hbase directory in the FS if DNE
         //   - If fresh start, create first ROOT and META regions (bootstrap)
@@ -270,7 +283,7 @@ implements HMasterInterface, HMasterRegi
         // TODO: Do we want to do this before/while/after RSs check in?
         //       It seems that this method looks at active RSs but happens
         //       concurrently with when we expect them to be checking in
-        fileSystemManager.splitLogAfterStartup();
+        fileSystemManager.splitLogAfterStartup(serverManager.getOnlineServers());
       }
       // start up all service threads.
       startServiceThreads();
@@ -288,9 +301,9 @@ implements HMasterInterface, HMasterRegi
       // guarantee that the transition has completed
       assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
       // start assignment of user regions, startup or failure
-      if(isClusterStartup()) {
-        // Create or clear out unassigned node in ZK, read all regions from
-        // META and assign them out.
+      if (this.clusterStarter) {
+        // We're starting up the cluster.  Create or clear out unassigned node
+        // in ZK, read all regions from META and assign them out.
         assignmentManager.processStartup();
       } else {
         // Process existing unassigned nodes in ZK, read all regions from META,
@@ -300,7 +313,7 @@ implements HMasterInterface, HMasterRegi
       LOG.info("HMaster started on " + this.address.toString());
       while (!this.closed.get()) {
         // check if we should be shutting down
-        if (this.shutdownRequested.get()) {
+        if (this.clusterShutdown.get()) {
           if (this.serverManager.numServers() == 0) {
             startShutdown();
             break;
@@ -313,9 +326,9 @@ implements HMasterInterface, HMasterRegi
 
         // wait for an interruption
         // TODO: something better?  we need to check closed and shutdown?
-        synchronized(this.shutdownRequested) {
+        synchronized(this.clusterShutdown) {
           try {
-            this.shutdownRequested.wait();
+            this.clusterShutdown.wait();
           } catch(InterruptedException e) {
             LOG.debug("Main thread interrupted", e);
           }
@@ -333,7 +346,7 @@ implements HMasterInterface, HMasterRegi
     }
 
     // Wait for all the remaining region servers to report in.
-    this.serverManager.letRegionServersShutdown();
+    if (!this.abort) this.serverManager.letRegionServersShutdown();
 
     // Clean up and close up shop
     if (this.infoServer != null) {
@@ -352,21 +365,13 @@ implements HMasterInterface, HMasterRegi
   }
 
   /**
-   * Returns true if this master process was responsible for starting the
-   * cluster, false if not.
-   */
-  public boolean isClusterStartup() {
-    return isClusterStartup;
-  }
-
-  /**
    * 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.isClusterStartup = isClusterStartup;
+    this.clusterStarter = isClusterStartup;
   }
 
   public HServerAddress getHServerAddress() {
@@ -402,13 +407,6 @@ implements HMasterInterface, HMasterRegi
   }
 
   /**
-   * Return the file systen manager instance
-   */
-  public FileSystemManager getFileSystemManager() {
-    return fileSystemManager;
-  }
-
-  /**
    * @return Return configuration being used by this server.
    */
   public Configuration getConfiguration() {
@@ -419,8 +417,9 @@ implements HMasterInterface, HMasterRegi
     return this.serverManager;
   }
 
-  public AtomicBoolean getShutdownRequested() {
-    return this.shutdownRequested;
+  @Override
+  public boolean isClusterShutdown() {
+    return this.clusterShutdown.get();
   }
 
   public void setClosed() {
@@ -565,19 +564,6 @@ implements HMasterInterface, HMasterRegi
     return !this.closed.get();
   }
 
-  public void requestShutdown() {
-    LOG.info("Cluster shutdown requested. Starting to quiesce servers");
-    synchronized(shutdownRequested) {
-      shutdownRequested.set(true);
-      shutdownRequested.notifyAll();
-    }
-    try {
-      clusterStatusTracker.setClusterDown();
-    } catch (KeeperException e) {
-      LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
-    }
-  }
-
   public void createTable(HTableDescriptor desc, byte [][] splitKeys)
   throws IOException {
     createTable(desc, splitKeys, false);
@@ -785,6 +771,50 @@ implements HMasterInterface, HMasterRegi
     System.exit(0);
   }
 
+  @Override
+  public void abort(final String msg, final Throwable t) {
+    if (t != null) LOG.fatal(msg, t);
+    else LOG.fatal(msg);
+    this.abort = true;
+  }
+
+  @Override
+  public ZooKeeperWatcher getZooKeeper() {
+    return zooKeeper;
+  }
+
+  @Override
+  public String getServerName() {
+    return address.toString();
+  }
+
+  public CatalogTracker getCatalogTracker() {
+    return catalogTracker;
+  }
+
+  @Override
+  public void shutdown() {
+    LOG.info("Cluster shutdown requested. Starting to quiesce servers");
+    synchronized (clusterShutdown) {
+      clusterShutdown.set(true);
+      clusterShutdown.notifyAll();
+    }
+    try {
+      clusterStatusTracker.setClusterDown();
+    } catch (KeeperException e) {
+      LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
+    }
+  }
+
+  @Override
+  public void stopMaster() {
+    // TODO
+  }
+
+  public void assignRegion(HRegionInfo hri) {
+    assignmentManager.assign(hri);
+  }
+
   /**
    * Utility for constructing an instance of the passed HMaster class.
    * @param masterClass
@@ -886,7 +916,7 @@ implements HMasterInterface, HMasterRegi
             cluster.startup();
           } else {
             HMaster master = constructMaster(masterClass, conf);
-            if (master.shutdownRequested.get()) {
+            if (master.clusterShutdown.get()) {
               LOG.info("Won't bring the Master up as a shutdown is requested");
               return;
             }
@@ -931,34 +961,4 @@ implements HMasterInterface, HMasterRegi
   public static void main(String [] args) {
     doMain(args, HMaster.class);
   }
-
-  @Override
-  public void abort(final String msg, final Throwable t) {
-    if (t != null) LOG.fatal(msg, t);
-    else LOG.fatal(msg);
-    this.startShutdown();
-  }
-
-  @Override
-  public ZooKeeperWatcher getZooKeeper() {
-    return zooKeeper;
-  }
-
-  @Override
-  public String getServerName() {
-    return address.toString();
-  }
-
-  public CatalogTracker getCatalogTracker() {
-    return catalogTracker;
-  }
-
-  @Override
-  public void shutdown() {
-    requestShutdown();
-  }
-
-  public void assignRegion(HRegionInfo hri) {
-    assignmentManager.assign(hri);
-  }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java?rev=983113&r1=983112&r2=983113&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java Fri Aug  6 19:57:47 2010
@@ -20,7 +20,6 @@
 package org.apache.hadoop.hbase.master;
 
 import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
 /**
@@ -37,5 +36,4 @@ public interface LogCleanerDelegate exte
    * @return true if the log is deletable, false if not
    */
   public boolean isLogDeletable(Path filePath);
-}
-
+}
\ No newline at end of file

Added: 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=983113&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Fri Aug  6 19:57:47 2010
@@ -0,0 +1,276 @@
+/**
+ * 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.master;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerInfo;
+import org.apache.hadoop.hbase.RemoteExceptionHandler;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * This class abstracts a bunch of operations the HMaster needs to interact with
+ * the underlying file system, including splitting log files, checking file
+ * system status, etc.
+ */
+public class MasterFileSystem {
+  private static final Log LOG = LogFactory.getLog(MasterFileSystem.class.getName());
+  // HBase configuration
+  Configuration conf;
+  // master status
+  MasterController masterStatus;
+  // Keep around for convenience.
+  private final FileSystem fs;
+  // Is the fileystem ok?
+  private volatile boolean fsOk = true;
+  // The Path to the old logs dir
+  private final Path oldLogDir;
+  // root hbase directory on the FS
+  private final Path rootdir;
+  // create the split log lock
+  final Lock splitLogLock = new ReentrantLock();
+
+  public MasterFileSystem(MasterController masterStatus) throws IOException {
+    this.conf = masterStatus.getConfiguration();
+    this.masterStatus = masterStatus;
+    // 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
+    // we get to here with appropriate fs scheme.
+    this.rootdir = FSUtils.getRootDir(conf);
+    // Cover both bases, the old way of setting default fs and the new.
+    // We're supposed to run on 0.20 and 0.21 anyways.
+    conf.set("fs.default.name", this.rootdir.toString());
+    conf.set("fs.defaultFS", this.rootdir.toString());
+    // setup the filesystem variable
+    this.fs = FileSystem.get(conf);
+    // set up the archived logs path
+    this.oldLogDir = new Path(this.rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
+  }
+
+  /**
+   * <ol>
+   * <li>Check if the root region exists and is readable, if not create it</li>
+   * <li>Create a log archive directory for RS to put archived logs</li>
+   * </ol>
+   */
+  public void initialize() throws IOException {
+    // check if the root directory exists
+    checkRootDir(this.rootdir, conf, this.fs);
+
+    // Make sure the region servers can archive their old logs
+    if(!this.fs.exists(this.oldLogDir)) {
+      this.fs.mkdirs(this.oldLogDir);
+    }
+  }
+
+  public FileSystem getFileSystem() {
+    return this.fs;
+  }
+
+  /**
+   * Get the directory where old logs go
+   * @return the dir
+   */
+  public Path getOldLogDir() {
+    return this.oldLogDir;
+  }
+
+  /**
+   * Checks to see if the file system is still accessible.
+   * If not, sets closed
+   * @return false if file system is not available
+   */
+  public boolean checkFileSystem() {
+    if (this.fsOk) {
+      try {
+        FSUtils.checkFileSystemAvailable(this.fs);
+      } catch (IOException e) {
+        LOG.fatal("Shutting down HBase cluster: file system not available", e);
+        masterStatus.setClosed();
+        this.fsOk = false;
+      }
+    }
+    return this.fsOk;
+  }
+
+  /**
+   * @return HBase root dir.
+   * @throws IOException
+   */
+  public Path getRootDir() {
+    return this.rootdir;
+  }
+
+  /**
+   * Inspect the log directory to recover any log file without
+   * an active region server.
+   * @param onlineServers Map of online servers keyed by
+   * {@link HServerInfo#getServerName()}
+   */
+  void splitLogAfterStartup(final Map<String, HServerInfo> onlineServers) {
+    Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
+    try {
+      if (!this.fs.exists(logsDirPath)) {
+        return;
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Failed exists test on " + logsDirPath, e);
+    }
+    FileStatus[] logFolders;
+    try {
+      logFolders = this.fs.listStatus(logsDirPath);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed listing " + logsDirPath.toString(), e);
+    }
+    if (logFolders == null || logFolders.length == 0) {
+      LOG.debug("No log files to split, proceeding...");
+      return;
+    }
+    for (FileStatus status : logFolders) {
+      String serverName = status.getPath().getName();
+      LOG.info("Found log folder : " + serverName);
+      if(onlineServers.get(serverName) == null) {
+        LOG.info("Log folder doesn't belong " +
+          "to a known region server, splitting");
+        this.splitLogLock.lock();
+        Path logDir =
+          new Path(this.rootdir, HLog.getHLogDirectoryName(serverName));
+        try {
+          HLog.splitLog(this.rootdir, logDir, oldLogDir, this.fs, conf);
+        } catch (IOException e) {
+          LOG.error("Failed splitting " + logDir.toString(), e);
+        } finally {
+          this.splitLogLock.unlock();
+        }
+      } else {
+        LOG.info("Log folder belongs to an existing region server");
+      }
+    }
+  }
+
+  /**
+   * Get the rootdir.  Make sure its wholesome and exists before returning.
+   * @param rd
+   * @param conf
+   * @param fs
+   * @return hbase.rootdir (after checks for existence and bootstrapping if
+   * needed populating the directory with necessary bootup files).
+   * @throws IOException
+   */
+  private static Path checkRootDir(final Path rd, final Configuration c,
+    final FileSystem fs)
+  throws IOException {
+    // If FS is in safe mode wait till out of it.
+    FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
+        10 * 1000));
+    // Filesystem is good. Go ahead and check for hbase.rootdir.
+    if (!fs.exists(rd)) {
+      fs.mkdirs(rd);
+      FSUtils.setVersion(fs, rd);
+    } else {
+      FSUtils.checkVersion(fs, rd, true);
+    }
+    // Make sure the root region directory exists!
+    if (!FSUtils.rootRegionExists(fs, rd)) {
+      bootstrap(rd, c);
+    }
+    return rd;
+  }
+
+  private static void bootstrap(final Path rd, final Configuration c)
+  throws IOException {
+    LOG.info("BOOTSTRAP: creating ROOT and first META regions");
+    try {
+      // Bootstrapping, make sure blockcache is off.  Else, one will be
+      // created here in bootstap and it'll need to be cleaned up.  Better to
+      // not make it in first place.  Turn off block caching for bootstrap.
+      // Enable after.
+      HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
+      setInfoFamilyCaching(rootHRI, false);
+      HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
+      setInfoFamilyCaching(metaHRI, false);
+      HRegion root = HRegion.createHRegion(rootHRI, rd, c);
+      HRegion meta = HRegion.createHRegion(metaHRI, rd, c);
+      setInfoFamilyCaching(rootHRI, true);
+      setInfoFamilyCaching(metaHRI, true);
+      // Add first region from the META table to the ROOT region.
+      HRegion.addRegionToMETA(root, meta);
+      root.close();
+      root.getLog().closeAndDelete();
+      meta.close();
+      meta.getLog().closeAndDelete();
+    } catch (IOException e) {
+      e = RemoteExceptionHandler.checkIOException(e);
+      LOG.error("bootstrap", e);
+      throw e;
+    }
+  }
+
+  /**
+   * @param hri Set all family block caching to <code>b</code>
+   * @param b
+   */
+  private static void setInfoFamilyCaching(final HRegionInfo hri, final boolean b) {
+    for (HColumnDescriptor hcd: hri.getTableDesc().families.values()) {
+      if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
+        hcd.setBlockCacheEnabled(b);
+        hcd.setInMemory(b);
+      }
+    }
+  }
+
+  public void deleteRegion(HRegionInfo region) throws IOException {
+    fs.delete(HRegion.getRegionDir(rootdir, region), true);
+  }
+
+  public void deleteTable(byte[] tableName) throws IOException {
+    fs.delete(new Path(rootdir, Bytes.toString(tableName)), true);
+  }
+
+  public void updateRegionInfo(HRegionInfo region) {
+    // TODO implement this.  i think this is currently broken in trunk i don't
+    //      see this getting updated.
+    //      @see HRegion.checkRegioninfoOnFilesystem()
+  }
+
+  public void deleteFamily(HRegionInfo region, byte[] familyName)
+  throws IOException {
+    fs.delete(Store.getStoreHomedir(
+        new Path(rootdir, region.getTableDesc().getNameAsString()),
+        region.getEncodedName(), familyName), true);
+  }
+}
\ No newline at end of file

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -142,7 +142,7 @@ public class ServerManager {
    */
   public ServerManager(MasterController masterStatus,
       MasterMetrics masterMetrics,
-      FileSystemManager fileSystemManager) {
+      MasterFileSystem fileSystemManager) {
     this.masterStatus = masterStatus;
     this.masterMetrics = masterMetrics;
     Configuration c = masterStatus.getConfiguration();
@@ -150,13 +150,13 @@ public class ServerManager {
       60 * 1000);
     this.minimumServerCount = c.getInt("hbase.regions.server.count.min", 1);
     this.serverMonitorThread = new ServerMonitor(metaRescanInterval,
-      this.masterStatus.getShutdownRequested());
+      this.masterStatus.isClusterShutdown());
     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.getShutdownRequested(), c,
+        this.masterStatus.isClusterShutdown(), c,
         fileSystemManager.getFileSystem(),
         fileSystemManager.getOldLogDir());
     Threads.setDaemonThreadRunning(oldLogCleaner,
@@ -281,12 +281,12 @@ public class ServerManager {
         this.quiescedServers.incrementAndGet();
       }
     }
-    if (this.masterStatus.getShutdownRequested().get()) {
+    if (this.masterStatus.isClusterShutdown()) {
       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.requestShutdown();
+        this.masterStatus.shutdown();
       }
       if (!this.masterStatus.isClosed()) {
         if (msgs.length > 0 &&
@@ -573,11 +573,6 @@ public class ServerManager {
    * a MSG_REGIONSERVER_STOP.
    */
   void letRegionServersShutdown() {
-    if (!masterStatus.getFileSystemManager().checkFileSystem()) {
-      // Forget waiting for the region servers if the file system has gone
-      // away. Just exit as quickly as possible.
-      return;
-    }
     synchronized (onlineServers) {
       while (onlineServers.size() > 0) {
         LOG.info("Waiting on following regionserver(s) to go down " +

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -27,14 +27,14 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.master.FileSystemManager;
+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,
-      CatalogTracker catalogTracker, FileSystemManager fileManager) {
+      CatalogTracker catalogTracker, MasterFileSystem fileManager) {
     super(EventType.C2M_DELETE_TABLE, tableName, server, catalogTracker,
         fileManager);
   }

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -27,14 +27,14 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.master.FileSystemManager;
+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,
-      CatalogTracker catalogTracker, FileSystemManager fileManager) {
+      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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.master.FileSystemManager;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterController;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -41,7 +41,7 @@ public class TableAddFamilyHandler exten
 
   public TableAddFamilyHandler(byte[] tableName, HColumnDescriptor familyDesc,
       MasterController server, CatalogTracker catalogTracker,
-      FileSystemManager fileManager) {
+      MasterFileSystem fileManager) {
     super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker,
         fileManager);
     this.familyDesc = familyDesc;
@@ -64,8 +64,6 @@ public class TableAddFamilyHandler exten
       MetaEditor.updateRegionInfo(catalogTracker, region);
       // Update region info in FS
       fileManager.updateRegionInfo(region);
-      // Add directory to FS
-      fileManager.addFamily(region, familyName);
     }
   }
-}
+}
\ No newline at end of file

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.master.FileSystemManager;
+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 TableDeleteFamilyHandler ex
 
   public TableDeleteFamilyHandler(byte[] tableName, byte [] familyName,
       MasterController server, CatalogTracker catalogTracker,
-      FileSystemManager fileManager) {
+      MasterFileSystem fileManager) {
     super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker,
         fileManager);
     this.familyName = familyName;

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.TableNotF
 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.FileSystemManager;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterController;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -46,11 +46,11 @@ public abstract class TableEventHandler 
 
   protected final byte [] tableName;
   protected final CatalogTracker catalogTracker;
-  protected final FileSystemManager fileManager;
+  protected final MasterFileSystem fileManager;
 
   public TableEventHandler(EventType eventType, byte [] tableName,
       MasterController server, CatalogTracker catalogTracker,
-      FileSystemManager fileManager) {
+      MasterFileSystem fileManager) {
     super(server, eventType);
     this.tableName = tableName;
     this.catalogTracker = catalogTracker;

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=983113&r1=983112&r2=983113&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 Fri Aug  6 19:57:47 2010
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.master.FileSystemManager;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterController;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -41,7 +41,7 @@ public class TableModifyFamilyHandler ex
 
   public TableModifyFamilyHandler(byte[] tableName,
       HColumnDescriptor familyDesc, MasterController server,
-      CatalogTracker catalogTracker, FileSystemManager fileManager) {
+      CatalogTracker catalogTracker, MasterFileSystem fileManager) {
     super(EventType.C2M_ADD_FAMILY, tableName, server,
         catalogTracker, fileManager);
     this.familyDesc = familyDesc;

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java?rev=983113&r1=983112&r2=983113&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java Fri Aug  6 19:57:47 2010
@@ -394,7 +394,7 @@ public class FSUtils {
   public static Map<String, Integer> getTableFragmentation(
     final HMaster master)
   throws IOException {
-    Path path = master.getFileSystemManager().getRootDir();
+    Path path = getRootDir(master.getConfiguration());
     // since HMaster.getFileSystem() is package private
     FileSystem fs = path.getFileSystem(master.getConfiguration());
     return getTableFragmentation(fs, path);

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java?rev=983113&r1=983112&r2=983113&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java Fri Aug  6 19:57:47 2010
@@ -199,7 +199,7 @@ public class TestActiveMasterManager {
     }
 
     @Override
-    public FileSystemManager getFileSystemManager() {
+    public MasterFileSystem getFileSystemManager() {
       return null;
     }