You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jg...@apache.org on 2010/08/05 09:35:02 UTC

svn commit: r982489 [4/7] - in /hbase/branches/0.90_master_rewrite: ./ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/catalog/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/executor/ s...

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java?rev=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,110 @@
+/**
+ * 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.handler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerController;
+import org.apache.hadoop.hbase.executor.EventHandler;
+import org.apache.hadoop.hbase.executor.RegionTransitionData;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.zookeeper.ZKAssign;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Handles CLOSED region event on Master.
+ * <p>
+ * If table is being disabled, deletes ZK unassigned node and removes from
+ * regions in transition.
+ * <p>
+ * Otherwise, assigns the region to another server.
+ */
+public class ClosedRegionHandler extends EventHandler {
+  private static final Log LOG = LogFactory.getLog(ClosedRegionHandler.class);
+
+  private final AssignmentManager assignmentManager;
+  private final RegionTransitionData data;
+  private final HRegionInfo regionInfo;
+  private final ClosedPriority priority;
+
+  private enum ClosedPriority {
+    ROOT (1),
+    META (2),
+    USER (3);
+
+    private final int value;
+    ClosedPriority(int value) {
+      this.value = value;
+    }
+    public int getValue() {
+      return value;
+    }
+  };
+
+  public ClosedRegionHandler(ServerController server,
+      AssignmentManager assignmentManager, RegionTransitionData data,
+      HRegionInfo regionInfo) {
+    super(server, EventType.RS2ZK_REGION_CLOSED);
+    this.assignmentManager = assignmentManager;
+    this.data = data;
+    this.regionInfo = regionInfo;
+    if(regionInfo.isRootRegion()) {
+      priority = ClosedPriority.ROOT;
+    } else if(regionInfo.isMetaRegion()) {
+      priority = ClosedPriority.META;
+    } else {
+      priority = ClosedPriority.USER;
+    }
+  }
+
+  @Override
+  public int getPriority() {
+    return priority.getValue();
+  }
+
+  @Override
+  public void process() {
+    LOG.debug("Handling CLOSED event with data: " + data);
+    // Check if this table is being disabled or not
+    if(assignmentManager.isTableOfRegionDisabled(regionInfo.getRegionName())) {
+      // Disabling so should not be reassigned, just delete the CLOSED node
+      LOG.debug("Table being disabled so deleting ZK node and removing from " +
+          "regions in transition, skipping assignment");
+      try {
+        ZKAssign.deleteClosedNode(server.getZooKeeper(),
+            regionInfo.getEncodedName());
+      } catch (KeeperException.NoNodeException nne) {
+        LOG.warn("Tried to delete closed node for " + data + " but it does " +
+            "not exist");
+        return;
+      } catch (KeeperException e) {
+        LOG.fatal("Error deleting CLOSED node in ZK", e);
+        server.abort();
+      }
+      assignmentManager.regionOffline(regionInfo);
+      return;
+    }
+    // ZK Node is in CLOSED state, assign it (transition to OFFLINE done here)
+    assignmentManager.setOffline(regionInfo);
+    assignmentManager.assign(regionInfo);
+  }
+
+}

Added: 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=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,55 @@
+/**
+ * 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.handler;
+
+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.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.master.FileSystemManager;
+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) {
+    super(EventType.C2M_DELETE_TABLE, tableName, server, catalogTracker,
+        fileManager);
+  }
+
+  @Override
+  protected void handleTableOperation(List<HRegionInfo> regions)
+  throws IOException {
+    for(HRegionInfo region : regions) {
+      LOG.debug("Deleting region " + region + " from META and FS");
+      // Remove region from META
+      MetaEditor.deleteRegion(catalogTracker, region);
+      // Delete region from FS
+      fileManager.deleteRegion(region);
+    }
+    // Delete table from FS
+    fileManager.deleteTable(tableName);
+  }
+}

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java?rev=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,104 @@
+/**
+ * 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.handler;
+
+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.ServerController;
+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.AssignmentManager;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+public class DisableTableHandler extends EventHandler {
+  private static final Log LOG = LogFactory.getLog(DisableTableHandler.class);
+
+  private final byte [] tableName;
+  private final CatalogTracker catalogTracker;
+  private final AssignmentManager assignmentManager;
+
+  public DisableTableHandler(ServerController server, byte [] tableName,
+      CatalogTracker catalogTracker, AssignmentManager assignmentManager) {
+    super(server, EventType.C2M_DISABLE_TABLE);
+    this.tableName = tableName;
+    this.catalogTracker = catalogTracker;
+    this.assignmentManager = assignmentManager;
+  }
+
+  @Override
+  public void process() {
+    try {
+      LOG.info("Attemping to disable the table " + Bytes.toString(tableName));
+      handleDisableTable();
+    } catch (IOException e) {
+      LOG.error("Error trying to disable the table " + Bytes.toString(tableName),
+          e);
+    }
+  }
+
+  // Meta scan
+  @SuppressWarnings("unused")
+  private void oldDisableTable() throws IOException {
+    // Check if table exists
+    if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) {
+      throw new TableNotFoundException(Bytes.toString(tableName));
+    }
+    // Set the table as disabled so it doesn't get re-onlined
+    assignmentManager.disableTable(Bytes.toString(tableName));
+    // Get the regions of this table
+    // TODO: should we use in-memory state?  need to deal with concurrent splits
+    List<HRegionInfo> regions = MetaReader.getTableRegions(catalogTracker,
+        tableName);
+    // Verify all regions of table are offline
+    for(HRegionInfo region : regions) {
+      if(region.isOffline()) {
+        continue;
+      }
+      // Unassign any regions still online
+      assignmentManager.unassign(region);
+    }
+  }
+
+  // In-memory scan
+  private void handleDisableTable() throws IOException {
+    // Check if table exists
+    // TODO: do we want to keep this in-memory as well?  i guess this is
+    //       part of old master rewrite, schema to zk to check for table
+    //       existence and such
+    if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) {
+      throw new TableNotFoundException(Bytes.toString(tableName));
+    }
+    // Set the table as disabled so it doesn't get re-onlined
+    assignmentManager.disableTable(Bytes.toString(tableName));
+    // Get the online regions of this table
+    List<HRegionInfo> regions = assignmentManager.getRegionsOfTable(tableName);
+    // Unassign the online regions
+    for(HRegionInfo region : regions) {
+      assignmentManager.unassign(region);
+    }
+  }
+}

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java?rev=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,79 @@
+/**
+ * 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.handler;
+
+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.ServerController;
+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.AssignmentManager;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+public class EnableTableHandler extends EventHandler {
+  private static final Log LOG = LogFactory.getLog(EnableTableHandler.class);
+
+  private final byte [] tableName;
+  private final CatalogTracker catalogTracker;
+  private final AssignmentManager assignmentManager;
+
+  public EnableTableHandler(ServerController server, byte [] tableName,
+      CatalogTracker catalogTracker, AssignmentManager assignmentManager) {
+    super(server, EventType.C2M_ENABLE_TABLE);
+    this.tableName = tableName;
+    this.catalogTracker = catalogTracker;
+    this.assignmentManager = assignmentManager;
+  }
+
+  @Override
+  public void process() {
+    try {
+      LOG.info("Attemping to enable the table " + Bytes.toString(tableName));
+      handleEnableTable();
+    } catch (IOException e) {
+      LOG.error("Error trying to enable the table " + Bytes.toString(tableName),
+          e);
+    }
+  }
+
+  private void handleEnableTable() throws IOException {
+    // Check if table exists
+    if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) {
+      throw new TableNotFoundException(Bytes.toString(tableName));
+    }
+    // Get the regions of this table
+    List<HRegionInfo> regions = MetaReader.getTableRegions(catalogTracker,
+        tableName);
+    // Verify all regions of table are disabled
+    for(HRegionInfo region : regions) {
+      if(!region.isOffline()) {
+        continue;
+      }
+      assignmentManager.assign(region);
+    }
+  }
+}

Added: 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=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,52 @@
+/**
+ * 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.handler;
+
+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.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.master.FileSystemManager;
+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) {
+    super(EventType.C2M_MODIFY_TABLE, tableName, server, catalogTracker,
+        fileManager);
+  }
+
+  @Override
+  protected void handleTableOperation(List<HRegionInfo> regions)
+  throws IOException {
+    for(HRegionInfo region : regions) {
+      // Update region info in META
+      MetaEditor.updateRegionInfo(catalogTracker, region);
+      // Update region info in FS
+      fileManager.updateRegionInfo(region);
+    }
+  }
+}

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java?rev=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,97 @@
+/**
+ * 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.handler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerInfo;
+import org.apache.hadoop.hbase.ServerController;
+import org.apache.hadoop.hbase.executor.EventHandler;
+import org.apache.hadoop.hbase.executor.RegionTransitionData;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.zookeeper.ZKAssign;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Handles OPENED region event on Master.
+ */
+public class OpenedRegionHandler extends EventHandler {
+  private static final Log LOG = LogFactory.getLog(OpenedRegionHandler.class);
+
+  private final AssignmentManager assignmentManager;
+  private final RegionTransitionData data;
+  private final HRegionInfo regionInfo;
+  private final HServerInfo serverInfo;
+  private final OpenedPriority priority;
+
+  private enum OpenedPriority {
+    ROOT (1),
+    META (2),
+    USER (3);
+
+    private final int value;
+    OpenedPriority(int value) {
+      this.value = value;
+    }
+    public int getValue() {
+      return value;
+    }
+  };
+
+  public OpenedRegionHandler(ServerController server,
+      AssignmentManager assignmentManager, RegionTransitionData data,
+      HRegionInfo regionInfo, HServerInfo serverInfo) {
+    super(server, EventType.RS2ZK_REGION_OPENED);
+    this.assignmentManager = assignmentManager;
+    this.data = data;
+    this.regionInfo = regionInfo;
+    this.serverInfo = serverInfo;
+    if(regionInfo.isRootRegion()) {
+      priority = OpenedPriority.ROOT;
+    } else if(regionInfo.isMetaRegion()) {
+      priority = OpenedPriority.META;
+    } else {
+      priority = OpenedPriority.USER;
+    }
+  }
+
+  @Override
+  public int getPriority() {
+    return priority.getValue();
+  }
+
+  @Override
+  public void process() {
+    LOG.debug("Handling OPENED event with data: " + data);
+    // TODO: should we check if this table was disabled and get it closed?
+    // Remove region from in-memory transition and unassigned node from ZK
+    try {
+      ZKAssign.deleteOpenedNode(server.getZooKeeper(),
+          regionInfo.getEncodedName());
+    } catch (KeeperException e) {
+      LOG.fatal("Error deleting OPENED node in ZK", e);
+      server.abort();
+    }
+    assignmentManager.regionOnline(regionInfo, serverInfo);
+    LOG.debug("Opened region " + regionInfo);
+  }
+
+}

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java?rev=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,37 @@
+/**
+ * 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.handler;
+
+import org.apache.hadoop.hbase.ServerController;
+import org.apache.hadoop.hbase.executor.EventHandler;
+
+
+public class ServerShutdownHandler extends EventHandler {
+
+  public ServerShutdownHandler(ServerController server) {
+    super(server, EventType.M_SERVER_SHUTDOWN);
+  }
+
+  @Override
+  public void process() {
+    // TODO: implement this
+  }
+
+}

Added: 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=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,71 @@
+/**
+ * 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.handler;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+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.MasterController;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Handles adding a new family to an existing table.
+ */
+public class TableAddFamilyHandler extends TableEventHandler {
+
+  private final HColumnDescriptor familyDesc;
+
+  public TableAddFamilyHandler(byte[] tableName, HColumnDescriptor familyDesc,
+      MasterController server, CatalogTracker catalogTracker,
+      FileSystemManager fileManager) {
+    super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker,
+        fileManager);
+    this.familyDesc = familyDesc;
+  }
+
+  @Override
+  protected void handleTableOperation(List<HRegionInfo> regions)
+  throws IOException {
+    HTableDescriptor htd = regions.get(0).getTableDesc();
+    byte [] familyName = familyDesc.getName();
+    if(htd.hasFamily(familyName)) {
+      throw new InvalidFamilyOperationException(
+          "Family '" + Bytes.toString(familyName) + "' already exists so " +
+          "cannot be added");
+    }
+    for(HRegionInfo region : regions) {
+      // Update the HTD
+      region.getTableDesc().addFamily(familyDesc);
+      // Update region in META
+      MetaEditor.updateRegionInfo(catalogTracker, region);
+      // Update region info in FS
+      fileManager.updateRegionInfo(region);
+      // Add directory to FS
+      fileManager.addFamily(region, familyName);
+    }
+  }
+}

Added: 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=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,68 @@
+/**
+ * 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.handler;
+
+import java.io.IOException;
+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.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.master.FileSystemManager;
+import org.apache.hadoop.hbase.master.MasterController;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Handles adding a new family to an existing table.
+ */
+public class TableDeleteFamilyHandler extends TableEventHandler {
+
+  private final byte [] familyName;
+
+  public TableDeleteFamilyHandler(byte[] tableName, byte [] familyName,
+      MasterController server, CatalogTracker catalogTracker,
+      FileSystemManager fileManager) {
+    super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker,
+        fileManager);
+    this.familyName = familyName;
+  }
+
+  @Override
+  protected void handleTableOperation(List<HRegionInfo> regions) throws IOException {
+    HTableDescriptor htd = regions.get(0).getTableDesc();
+    if(!htd.hasFamily(familyName)) {
+      throw new InvalidFamilyOperationException(
+          "Family '" + Bytes.toString(familyName) + "' does not exist so " +
+          "cannot be deleted");
+    }
+    for(HRegionInfo region : regions) {
+      // Update the HTD
+      region.getTableDesc().removeFamily(familyName);
+      // Update region in META
+      MetaEditor.updateRegionInfo(catalogTracker, region);
+      // Update region info in FS
+      fileManager.updateRegionInfo(region);
+      // Delete directory in FS
+      fileManager.deleteFamily(region, familyName);
+    }
+  }
+}

Added: 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=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,92 @@
+/**
+ * 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.handler;
+
+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.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.FileSystemManager;
+import org.apache.hadoop.hbase.master.MasterController;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Base class for performing operations against tables.
+ * <p>
+ * Ensures all regions of the table are offline and then executes
+ * {@link #handleTableOperation(List)} with a list of regions of the
+ * table.
+ */
+public abstract class TableEventHandler extends EventHandler {
+  private static final Log LOG = LogFactory.getLog(TableEventHandler.class);
+
+  protected final byte [] tableName;
+  protected final CatalogTracker catalogTracker;
+  protected final FileSystemManager fileManager;
+
+  public TableEventHandler(EventType eventType, byte [] tableName,
+      MasterController server, CatalogTracker catalogTracker,
+      FileSystemManager fileManager) {
+    super(server, eventType);
+    this.tableName = tableName;
+    this.catalogTracker = catalogTracker;
+    this.fileManager = fileManager;
+  }
+
+  @Override
+  public void process() {
+    try {
+      LOG.info("Handling table operation " + eventType + " on table " +
+          Bytes.toString(tableName));
+      handleTableOperation(tableChecks());
+    } catch (IOException e) {
+      LOG.error("Error trying to delete the table " + Bytes.toString(tableName),
+          e);
+    }
+  }
+
+  private List<HRegionInfo> tableChecks() throws IOException {
+    // Check if table exists
+    if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) {
+      throw new TableNotFoundException(Bytes.toString(tableName));
+    }
+    // Get the regions of this table
+    // TODO: Use in-memory state of master?
+    List<HRegionInfo> regions = MetaReader.getTableRegions(catalogTracker,
+        tableName);
+    // Verify all regions of table are disabled
+    for(HRegionInfo region : regions) {
+      if(!region.isOffline()) {
+        throw new TableNotDisabledException(tableName);
+      }
+    }
+    return regions;
+  }
+
+  protected abstract void handleTableOperation(List<HRegionInfo> regions)
+  throws IOException;
+}

Added: 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=982489&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java Thu Aug  5 07:35:00 2010
@@ -0,0 +1,68 @@
+/**
+ * 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.handler;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+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.MasterController;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Handles adding a new family to an existing table.
+ */
+public class TableModifyFamilyHandler extends TableEventHandler {
+
+  private final HColumnDescriptor familyDesc;
+
+  public TableModifyFamilyHandler(byte[] tableName,
+      HColumnDescriptor familyDesc, MasterController server,
+      CatalogTracker catalogTracker, FileSystemManager fileManager) {
+    super(EventType.C2M_ADD_FAMILY, tableName, server,
+        catalogTracker, fileManager);
+    this.familyDesc = familyDesc;
+  }
+
+  @Override
+  protected void handleTableOperation(List<HRegionInfo> regions) throws IOException {
+    HTableDescriptor htd = regions.get(0).getTableDesc();
+    byte [] familyName = familyDesc.getName();
+    if(htd.hasFamily(familyName)) {
+      throw new InvalidFamilyOperationException(
+          "Family '" + Bytes.toString(familyName) + "' already exists so " +
+          "cannot be modified");
+    }
+    for(HRegionInfo region : regions) {
+      // Update the HTD
+      region.getTableDesc().addFamily(familyDesc);
+      // Update region in META
+      MetaEditor.updateRegionInfo(catalogTracker, region);
+      // Update region info in FS
+      fileManager.updateRegionInfo(region);
+    }
+  }
+}

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=982489&r1=982488&r2=982489&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Thu Aug  5 07:35:00 2010
@@ -40,7 +40,7 @@ import java.util.concurrent.locks.Reentr
 /**
  * Compact region on request and then run split if appropriate
  */
-class CompactSplitThread extends Thread {
+public class CompactSplitThread extends Thread {
   static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
 
   private HTable root = null;

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=982489&r1=982488&r2=982489&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Aug  5 07:35:00 2010
@@ -385,7 +385,7 @@ public class HRegion implements HeapSize
   /**
    * @return True if this region has references.
    */
-  boolean hasReferences() {
+  public boolean hasReferences() {
     for (Store store : this.stores.values()) {
       for (StoreFile sf : store.getStorefiles()) {
         // Found a reference, return.