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 08:15:14 UTC

svn commit: r982881 [1/2] - in /hbase/branches/0.90_master_rewrite/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/executor/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apa...

Author: stack
Date: Fri Aug  6 06:15:12 2010
New Revision: 982881

URL: http://svn.apache.org/viewvc?rev=982881&view=rev
Log:
+ Cleaned up the top-level o.a.h.h dir.  Removed unused exceptions and moved tools
down into util, etc. For example:
++ Moved org.apache.hadoop.hbase.MiniZooKeeperCluster to org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster
++ Moved HMerge into util (and ditto for its tests).  Same for HBaseConfTool
++ Moved Lease* down into RegionServer.  Thats only place that uses them now.
++ Removed unused ValueOverMaxLengthException, ColumnNameParseException

+ Changed Abortable#abort to take a message and a Throwable explaining why the abort.
+ Removed the getLog from *Trackers; was used to log message and exception
before we called abort.  Instead, changed the Server interface so that
abort now takes a message and a Throwable.  Let it be the one to explain
why we are aborting. Changed things as follows:
 -        getLog().fatal("Unexpected exception handling nodeCreated event", e);
 -        abortable.abort();
 +        abortable.abort("Unexpected exception handling nodeDeleted event", e);
+ Renamed ServerController Server (You can change it to something else when you get back from holidays Jon).
+ Renamed o.a.h.h.rs.MasterAddressManager as o.a.h..MasterAddressTracker.  This way
it aligns with other namings and its used both by client and rs.
+ Removed commented out code that did validateValuesLength for BatchUpdates
+ Removed WritableComparable.  We use Hadoop one everywhere.


Added:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseListener.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/HBaseConfTool.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/util/TestMergeMeta.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
Removed:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/ColumnNameParseException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HBaseConfTool.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HMerge.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/LeaseException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/LeaseListener.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Leases.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/MiniZooKeeperCluster.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/ServerController.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/ValueOverMaxLengthException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/WritableComparator.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestMergeMeta.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestMergeTable.java
Modified:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Abortable.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/NamedThreadFactory.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterController.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/ClosedRegionHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseClusterTestCase.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Abortable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Abortable.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Abortable.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Abortable.java Fri Aug  6 06:15:12 2010
@@ -30,6 +30,8 @@ package org.apache.hadoop.hbase;
 public interface Abortable {
   /**
    * Abort the server or client.
+   * @param why Why we're aborting.
+   * @param e Throwable that caused abort. Can be null.
    */
-  public void abort();
+  public void abort(String why, Throwable e);
 }
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java Fri Aug  6 06:15:12 2010
@@ -38,4 +38,4 @@ public class DroppedSnapshotException ex
   public DroppedSnapshotException() {
     super();
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java Fri Aug  6 06:15:12 2010
@@ -24,6 +24,7 @@ package org.apache.hadoop.hbase;
  * HRegionServer serving the region
  */
 public class HRegionLocation implements Comparable<HRegionLocation> {
+  // TODO: Is this class necessary?  Why not just have a Pair?
   private HRegionInfo regionInfo;
   private HServerAddress serverAddress;
 

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java?rev=982881&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java Fri Aug  6 06:15:12 2010
@@ -0,0 +1,91 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
+/**
+ * Manages the location of the current active Master for this RegionServer.
+ * <p>
+ * Listens for ZooKeeper events related to the master address. The node
+ * <code>/master</code> will contain the address of the current master.
+ * This listener is interested in
+ * <code>NodeDeleted</code> and <code>NodeCreated</code> events on
+ * <code>/master</code>.
+ * <p>
+ * Utilizes {@link ZooKeeperNodeTracker} for zk interactions.
+ * <p>
+ * You can get the current master via {@link #getMasterAddress()} or the
+ * blocking method {@link #waitMasterAddress()}.
+ */
+public class MasterAddressTracker extends ZooKeeperNodeTracker {
+  /**
+   * Construct a master address listener with the specified
+   * <code>zookeeper</code> reference.
+   * <p>
+   * This constructor does not trigger any actions, you must call methods
+   * explicitly.  Normally you will just want to execute {@link #start()} to
+   * begin tracking of the master address.
+   *
+   * @param watcher zk reference and watcher
+   * @param abortable abortable in case of fatal error
+   */
+  public MasterAddressTracker(ZooKeeperWatcher watcher, Abortable abortable) {
+    super(watcher, watcher.masterAddressZNode, abortable);
+  }
+
+  /**
+   * Get the address of the current master if one is available.  Returns null
+   * if no current master.
+   *
+   * Use {@link #waitMasterAddress} if you want to block until the master is
+   * available.
+   * @return server address of current active master, or null if none available
+   */
+  public HServerAddress getMasterAddress() {
+    byte [] data = super.getData();
+    return data == null ? null : new HServerAddress(Bytes.toString(data));
+  }
+
+  /**
+   * Check if there is a master available.
+   * @return true if there is a master set, false if not.
+   */
+  public boolean hasMaster() {
+    return super.getData() != null;
+  }
+
+  /**
+   * Get the address of the current master.  If no master is available, method
+   * will block until one is available, the thread is interrupted, or timeout
+   * has passed.
+   *
+   * @param timeout maximum time to wait for master in millis, 0 for forever
+   * @return server address of current active master, null if timed out
+   * @throws InterruptedException if the thread is interrupted while waiting
+   */
+  public synchronized HServerAddress waitForMaster(long timeout)
+  throws InterruptedException {
+    byte [] data = super.blockUntilAvailable(timeout);
+    return data == null ? null : new HServerAddress(Bytes.toString(data));
+  }
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java Fri Aug  6 06:15:12 2010
@@ -26,9 +26,9 @@ import java.io.IOException;
  * and restarted so fast that the master still hasn't processed the server
  * shutdown of the first instance.
  */
+@SuppressWarnings("serial")
 public class PleaseHoldException extends IOException {
-
   public PleaseHoldException(String message) {
     super(message);
   }
-}
+}
\ No newline at end of file

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java?rev=982881&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/Server.java Fri Aug  6 06:15:12 2010
@@ -0,0 +1,56 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.ServerConnection;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
+/**
+ * Defines the set of shared functions implemented by HBase servers (Masters
+ * and RegionServers).
+ */
+public interface Server extends Abortable {
+  /**
+   * Returns the address of the current server.
+   */
+  public HServerAddress getHServerAddress();
+
+  /**
+   * Gets the configuration object for this server.
+   */
+  public Configuration getConfiguration();
+
+  /**
+   * Gets the ZooKeeper instance for this server.
+   */
+  public ZooKeeperWatcher getZooKeeper();
+
+  /**
+   * Gets the unique server name for this server.
+   * @return unique server name
+   */
+  public String getServerName();
+
+  /**
+   * Return the server RPC connection object
+   */
+  public ServerConnection getServerConnection();
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/UnknownRowLockException.java Fri Aug  6 06:15:12 2010
@@ -38,4 +38,4 @@ public class UnknownRowLockException ext
   public UnknownRowLockException(String s) {
     super(s);
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java Fri Aug  6 06:15:12 2010
@@ -26,9 +26,9 @@ import java.io.IOException;
  * already being processed as dead. This can happen when a region server loses
  * its session but didn't figure it yet.
  */
+@SuppressWarnings("serial")
 public class YouAreDeadException extends IOException {
-
   public YouAreDeadException(String message) {
     super(message);
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Fri Aug  6 06:15:12 2010
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.HRegionLo
 import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MasterAddressTracker;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.TableNotFoundException;
@@ -58,7 +59,6 @@ import org.apache.hadoop.hbase.ipc.HBase
 import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
 import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
-import org.apache.hadoop.hbase.regionserver.MasterAddressManager;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.MetaUtils;
 import org.apache.hadoop.hbase.util.SoftValueSortedMap;
@@ -201,7 +201,7 @@ public class HConnectionManager {
     // ZooKeeper reference
     private ZooKeeperWatcher zooKeeper;
     // ZooKeeper-based master address tracker
-    private MasterAddressManager masterAddressManager;
+    private MasterAddressTracker masterAddressTracker;
 
     private final Object rootRegionLock = new Object();
     private final Object metaRegionLock = new Object();
@@ -261,9 +261,9 @@ public class HConnectionManager {
 
       // initialize zookeeper and master address manager
       getZooKeeperWatcher();
-      masterAddressManager = new MasterAddressManager(zooKeeper, this);
-      zooKeeper.registerListener(masterAddressManager);
-      masterAddressManager.start();
+      masterAddressTracker = new MasterAddressTracker(zooKeeper, this);
+      zooKeeper.registerListener(masterAddressTracker);
+      masterAddressTracker.start();
 
       this.master = null;
       this.masterChecked = false;
@@ -301,14 +301,6 @@ public class HConnectionManager {
         }
       }
 
-      // If not, we need to connect to ZK to get the
-      ZooKeeperWatcher zk;
-      try {
-        zk = getZooKeeperWatcher();
-      } catch (IOException e) {
-        throw new ZooKeeperConnectionException(e);
-      }
-
       HServerAddress masterLocation = null;
       synchronized (this.masterLock) {
         for (int tries = 0;
@@ -318,7 +310,7 @@ public class HConnectionManager {
         tries++) {
 
           try {
-            masterLocation = masterAddressManager.getMasterAddress();
+            masterLocation = masterAddressTracker.getMasterAddress();
             if(masterLocation == null) {
               LOG.info("ZooKeeper available but no active master location found");
               throw new MasterNotRunningException();
@@ -1596,7 +1588,9 @@ public class HConnectionManager {
     }
 
     @Override
-    public void abort() {
+    public void abort(final String msg, Throwable t) {
+      if (t != null) LOG.fatal(msg, t);
+      else LOG.fatal(msg);
       if(zooKeeper != null) {
         zooKeeper.close();
         zooKeeper = null;

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java Fri Aug  6 06:15:12 2010
@@ -26,7 +26,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.ServerController;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.executor.HBaseExecutorService.HBaseExecutorServiceType;
 
 
@@ -49,7 +49,7 @@ public abstract class EventHandler imple
   // type of event this object represents
   protected EventType eventType;
   // server controller
-  protected ServerController server;
+  protected Server server;
   // listeners that are called before and after an event is processed
   protected static List<EventHandlerListener> eventHandlerListeners =
     Collections.synchronizedList(new ArrayList<EventHandlerListener>());
@@ -190,7 +190,7 @@ public abstract class EventHandler imple
   /**
    * Default base class constructor.
    */
-  public EventHandler(ServerController server, EventType eventType) {
+  public EventHandler(Server server, EventType eventType) {
     this.server = server;
     this.eventType = eventType;
     seqid = seqids.incrementAndGet();

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/NamedThreadFactory.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/NamedThreadFactory.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/NamedThreadFactory.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/executor/NamedThreadFactory.java Fri Aug  6 06:15:12 2010
@@ -40,4 +40,4 @@ public class NamedThreadFactory implemen
   public Thread newThread(Runnable r) {
     return new Thread(r, threadPrefix + "-" + threadId.incrementAndGet());
   }
-}
+}
\ 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=982881&r1=982880&r2=982881&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 06:15:12 2010
@@ -48,25 +48,25 @@ public class ActiveMasterManager extends
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
 
   private final HServerAddress address;
-  private final MasterController status;
+  private final MasterController master;
 
   ActiveMasterManager(ZooKeeperWatcher watcher, HServerAddress address,
       MasterController status) {
     super(watcher);
     this.address = address;
-    this.status = status;
+    this.master = status;
   }
 
   @Override
   public void nodeCreated(String path) {
-    if(path.equals(watcher.masterAddressZNode) && !status.isClosed()) {
+    if(path.equals(watcher.masterAddressZNode) && !master.isClosed()) {
       handleMasterNodeChange();
     }
   }
 
   @Override
   public void nodeDeleted(String path) {
-    if(path.equals(watcher.masterAddressZNode) && !status.isClosed()) {
+    if(path.equals(watcher.masterAddressZNode) && !master.isClosed()) {
       handleMasterNodeChange();
     }
   }
@@ -101,8 +101,7 @@ public class ActiveMasterManager extends
         }
       }
     } catch (KeeperException ke) {
-      LOG.fatal("Received an unexpected KeeperException, aborting", ke);
-      status.abort();
+      master.abort("Received an unexpected KeeperException, aborting", ke);
     }
   }
 
@@ -125,8 +124,7 @@ public class ActiveMasterManager extends
         return;
       }
     } catch (KeeperException ke) {
-      LOG.fatal("Received an unexpected KeeperException, aborting", ke);
-      status.abort();
+      master.abort("Received an unexpected KeeperException, aborting", ke);
       return;
     }
     // There is another active master, this is not a cluster startup
@@ -134,9 +132,9 @@ public class ActiveMasterManager extends
     LOG.info("Another master is already the active master, waiting to become " +
     "the next active master");
     clusterHasActiveMaster.set(true);
-    status.setClusterStartup(false);
+    master.setClusterStartup(false);
     synchronized(clusterHasActiveMaster) {
-      while(clusterHasActiveMaster.get() && !status.isClosed()) {
+      while(clusterHasActiveMaster.get() && !master.isClosed()) {
         try {
           clusterHasActiveMaster.wait();
         } catch (InterruptedException e) {
@@ -144,7 +142,7 @@ public class ActiveMasterManager extends
           LOG.debug("Interrupted waiting for master to die", e);
         }
       }
-      if(status.isClosed()) {
+      if(master.isClosed()) {
         return;
       }
       // Try to become active master again now that there is no active master

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Fri Aug  6 06:15:12 2010
@@ -205,27 +205,6 @@ public class AssignmentManager extends Z
   }
 
   /**
-   * Gets the region info for the region with the specified encoded name.
-   * <p>
-   * Currently this does a full scan of the regions map looking for a region
-   * with the specified encoded name.
-   * <p>
-   * Returns null if none found.
-   * @param regionName
-   * @return
-   * @deprecated should be able to remove this now?
-   */
-  @Deprecated
-  private HRegionInfo getRegionInfoFromEncoded(String encodedName) {
-    for(HRegionInfo regionInfo : regions.keySet()) {
-      if(regionInfo.getEncodedName().equals(encodedName)) {
-        return regionInfo;
-      }
-    }
-    return null;
-  }
-
-  /**
    * Handles various states an unassigned node can be in.
    * <p>
    * Method is called when a state change is suspected for an unassigned node.
@@ -346,8 +325,7 @@ public class AssignmentManager extends Z
           }
           handleRegion(data);
         } catch (KeeperException e) {
-          LOG.error("Unexpected ZK exception reading unassigned node data", e);
-          master.abort();
+          master.abort("Unexpected ZK exception reading unassigned node data", e);
         }
       }
     }
@@ -376,8 +354,7 @@ public class AssignmentManager extends Z
           }
           handleRegion(data);
         } catch (KeeperException e) {
-          LOG.error("Unexpected ZK exception reading unassigned node data", e);
-          master.abort();
+          master.abort("Unexpected ZK exception reading unassigned node data", e);
         }
       }
     }
@@ -408,8 +385,7 @@ public class AssignmentManager extends Z
             handleRegion(RegionTransitionData.fromBytes(newNode.getData()));
           }
         } catch(KeeperException e) {
-          LOG.error("Unexpected ZK exception reading unassigned children", e);
-          master.abort();
+          master.abort("Unexpected ZK exception reading unassigned children", e);
         }
       }
     }
@@ -519,8 +495,7 @@ public class AssignmentManager extends Z
           return;
         }
       } catch (KeeperException e) {
-        LOG.error("Unexpected ZK exception creating/setting node OFFLINE", e);
-        master.abort();
+        master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
         return;
       }
       // Pickup existing plan or make a new one

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=982881&r1=982880&r2=982881&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 06:15:12 2010
@@ -43,7 +43,6 @@ 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.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -76,9 +75,9 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.InfoServer;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.Sleeper;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -130,8 +129,6 @@ implements HMasterInterface, HMasterRegi
   // Region server tracker
   private RegionServerTracker regionServerTracker;
 
-  // A Sleeper that sleeps for threadWakeFrequency; sleep if nothing todo.
-  private final Sleeper sleeper;
   // RPC server for the HMaster
   private final HBaseServer rpcServer;
   // Address of the HMaster
@@ -215,9 +212,6 @@ implements HMasterInterface, HMasterRegi
         conf.getInt("hbase.master.catalog.timeout", 30000));
     assignmentManager = new AssignmentManager(zooKeeper, this,
         serverManager, catalogTracker);
-    // create a sleeper to sleep for a configured wait frequency
-    int threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
-    this.sleeper = new Sleeper(threadWakeFrequency, this.closed);
 
     /*
      * 4. Block on becoming the active master.
@@ -939,7 +933,9 @@ implements HMasterInterface, HMasterRegi
   }
 
   @Override
-  public void abort() {
+  public void abort(final String msg, final Throwable t) {
+    if (t != null) LOG.fatal(msg, t);
+    else LOG.fatal(msg);
     this.startShutdown();
   }
 
@@ -953,12 +949,6 @@ implements HMasterInterface, HMasterRegi
     return address.toString();
   }
 
-  @Override
-  public long getTimeout() {
-    // TODO: use configuration
-    return 5000;
-  }
-
   public CatalogTracker getCatalogTracker() {
     return catalogTracker;
   }

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterController.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterController.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/MasterController.java Fri Aug  6 06:15:12 2010
@@ -21,13 +21,13 @@ package org.apache.hadoop.hbase.master;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.hadoop.hbase.ServerController;
+import org.apache.hadoop.hbase.Server;
 
 /**
  * Defines the set of functions implemented by the HMaster related to control
  * of the master process and cluster shutdown.
  */
-public interface MasterController extends ServerController {
+public interface MasterController extends Server {
 
   // TODO: getServerManager and getFileManager exist because each references the
   //       other in a single call.  should figure how to clean this up.

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=982881&r1=982880&r2=982881&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 06:15:12 2010
@@ -43,10 +43,10 @@ import org.apache.hadoop.hbase.HServerLo
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.YouAreDeadException;
-import org.apache.hadoop.hbase.Leases.LeaseStillHeldException;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
 import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
+import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
 import org.apache.hadoop.hbase.util.Threads;
 
 /**

Modified: 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=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java Fri Aug  6 06:15:12 2010
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.master.h
 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.Server;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.master.AssignmentManager;
@@ -59,7 +59,7 @@ public class ClosedRegionHandler extends
     }
   };
 
-  public ClosedRegionHandler(ServerController server,
+  public ClosedRegionHandler(Server server,
       AssignmentManager assignmentManager, RegionTransitionData data,
       HRegionInfo regionInfo) {
     super(server, EventType.RS2ZK_REGION_CLOSED);
@@ -96,8 +96,7 @@ public class ClosedRegionHandler extends
             "not exist");
         return;
       } catch (KeeperException e) {
-        LOG.fatal("Error deleting CLOSED node in ZK", e);
-        server.abort();
+        server.abort("Error deleting CLOSED node in ZK", e);
       }
       assignmentManager.regionOffline(regionInfo);
       return;
@@ -106,5 +105,4 @@ public class ClosedRegionHandler extends
     assignmentManager.setOffline(regionInfo);
     assignmentManager.assign(regionInfo);
   }
-
-}
+}
\ No newline at end of file

Modified: 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=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java Fri Aug  6 06:15:12 2010
@@ -25,7 +25,7 @@ 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.Server;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
@@ -41,7 +41,7 @@ public class DisableTableHandler extends
   private final CatalogTracker catalogTracker;
   private final AssignmentManager assignmentManager;
 
-  public DisableTableHandler(ServerController server, byte [] tableName,
+  public DisableTableHandler(Server server, byte [] tableName,
       CatalogTracker catalogTracker, AssignmentManager assignmentManager) {
     super(server, EventType.C2M_DISABLE_TABLE);
     this.tableName = tableName;
@@ -101,4 +101,4 @@ public class DisableTableHandler extends
       assignmentManager.unassign(region);
     }
   }
-}
+}
\ No newline at end of file

Modified: 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=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java Fri Aug  6 06:15:12 2010
@@ -25,7 +25,7 @@ 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.Server;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
@@ -41,7 +41,7 @@ public class EnableTableHandler extends 
   private final CatalogTracker catalogTracker;
   private final AssignmentManager assignmentManager;
 
-  public EnableTableHandler(ServerController server, byte [] tableName,
+  public EnableTableHandler(Server server, byte [] tableName,
       CatalogTracker catalogTracker, AssignmentManager assignmentManager) {
     super(server, EventType.C2M_ENABLE_TABLE);
     this.tableName = tableName;

Modified: 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=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java Fri Aug  6 06:15:12 2010
@@ -23,7 +23,7 @@ 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.Server;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.master.AssignmentManager;
@@ -56,7 +56,7 @@ public class OpenedRegionHandler extends
     }
   };
 
-  public OpenedRegionHandler(ServerController server,
+  public OpenedRegionHandler(Server server,
       AssignmentManager assignmentManager, RegionTransitionData data,
       HRegionInfo regionInfo, HServerInfo serverInfo) {
     super(server, EventType.RS2ZK_REGION_OPENED);
@@ -87,8 +87,7 @@ public class OpenedRegionHandler extends
       ZKAssign.deleteOpenedNode(server.getZooKeeper(),
           regionInfo.getEncodedName());
     } catch (KeeperException e) {
-      LOG.fatal("Error deleting OPENED node in ZK", e);
-      server.abort();
+      server.abort("Error deleting OPENED node in ZK", e);
     }
     assignmentManager.regionOnline(regionInfo, serverInfo);
     LOG.debug("Opened region " + regionInfo);

Modified: 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=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java Fri Aug  6 06:15:12 2010
@@ -19,13 +19,13 @@
  */
 package org.apache.hadoop.hbase.master.handler;
 
-import org.apache.hadoop.hbase.ServerController;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.executor.EventHandler;
 
 
 public class ServerShutdownHandler extends EventHandler {
 
-  public ServerShutdownHandler(ServerController server) {
+  public ServerShutdownHandler(Server server) {
     super(server, EventType.M_SERVER_SHUTDOWN);
   }
 
@@ -33,5 +33,4 @@ public class ServerShutdownHandler exten
   public void process() {
     // TODO: implement this
   }
-
-}
+}
\ No newline at end of file

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=982881&r1=982880&r2=982881&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 Fri Aug  6 06:15:12 2010
@@ -19,6 +19,28 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.lang.reflect.Constructor;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -64,29 +86,6 @@ import org.apache.hadoop.util.StringUtil
 
 import com.google.common.collect.Lists;
 
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.lang.reflect.Constructor;
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 /**
  * HRegion stores data for a certain region of a table.  It stores all columns
  * for each row. A given table consists of one or more HRegions.
@@ -1643,28 +1642,6 @@ public class HRegion implements HeapSize
     }
   }
 
-//  /*
-//   * Utility method to verify values length.
-//   * @param batchUpdate The update to verify
-//   * @throws IOException Thrown if a value is too long
-//   */
-//  private void validateValuesLength(Put put)
-//  throws IOException {
-//    Map<byte[], List<KeyValue>> families = put.getFamilyMap();
-//    for(Map.Entry<byte[], List<KeyValue>> entry : families.entrySet()) {
-//      HColumnDescriptor hcd =
-//        this.regionInfo.getTableDesc().getFamily(entry.getKey());
-//      int maxLen = hcd.getMaxValueLength();
-//      for(KeyValue kv : entry.getValue()) {
-//        if(kv.getValueLength() > maxLen) {
-//          throw new ValueOverMaxLengthException("Value in column "
-//            + Bytes.toString(kv.getColumn()) + " is too long. "
-//            + kv.getValueLength() + " > " + maxLen);
-//        }
-//      }
-//    }
-//  }
-
   /*
    * Check if resources to support an update.
    *

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Aug  6 06:15:12 2010
@@ -66,9 +66,8 @@ import org.apache.hadoop.hbase.HServerIn
 import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.LeaseListener;
-import org.apache.hadoop.hbase.Leases;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
+import org.apache.hadoop.hbase.MasterAddressTracker;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.UnknownRowLockException;
@@ -76,7 +75,6 @@ import org.apache.hadoop.hbase.UnknownSc
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.HMsg.Type;
-import org.apache.hadoop.hbase.Leases.LeaseStillHeldException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -96,6 +94,7 @@ import org.apache.hadoop.hbase.ipc.HBase
 import org.apache.hadoop.hbase.ipc.HBaseServer;
 import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
+import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler;
@@ -229,7 +228,7 @@ public class HRegionServer implements HR
   private ZooKeeperWatcher zooKeeper;
 
   // master address manager and watcher
-  private MasterAddressManager masterAddressManager;
+  private MasterAddressTracker masterAddressManager;
 
   // catalog tracker
   private CatalogTracker catalogTracker;
@@ -343,7 +342,7 @@ public class HRegionServer implements HR
         + serverInfo.getServerName(), this);
 
     // create the master address manager, register with zk, and start it
-    masterAddressManager = new MasterAddressManager(zooKeeper, this);
+    masterAddressManager = new MasterAddressTracker(zooKeeper, this);
     zooKeeper.registerListener(masterAddressManager);
     masterAddressManager.start();
 
@@ -2352,13 +2351,6 @@ public class HRegionServer implements HR
     return threadWakeFrequency;
   }
 
-  // ServerStatus
-
-  @Override
-  public void abort() {
-    this.abort("Received abort call");
-  }
-
   @Override
   public HServerAddress getHServerAddress() {
     return this.address;
@@ -2509,10 +2501,4 @@ public class HRegionServer implements HR
   public ServerConnection getServerConnection() {
     return connection;
   }
-
-  @Override
-  public long getTimeout() {
-    // TODO: use configuration
-    return 5000;
-  }
 }
\ No newline at end of file

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java?rev=982881&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java Fri Aug  6 06:15:12 2010
@@ -0,0 +1,42 @@
+/**
+ * Copyright 2008 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.regionserver;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+/**
+ * Reports a problem with a lease
+ */
+public class LeaseException extends DoNotRetryIOException {
+
+  private static final long serialVersionUID = 8179703995292418650L;
+
+  /** default constructor */
+  public LeaseException() {
+    super();
+  }
+
+  /**
+   * @param message
+   */
+  public LeaseException(String message) {
+    super(message);
+  }
+}
\ No newline at end of file

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseListener.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseListener.java?rev=982881&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseListener.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseListener.java Fri Aug  6 06:15:12 2010
@@ -0,0 +1,34 @@
+/**
+ * Copyright 2007 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.regionserver;
+
+
+/**
+ * LeaseListener is an interface meant to be implemented by users of the Leases
+ * class.
+ *
+ * It receives events from the Leases class about the status of its accompanying
+ * lease.  Users of the Leases class can use a LeaseListener subclass to, for
+ * example, clean up resources after a lease has expired.
+ */
+public interface LeaseListener {
+  /** When a lease expires, this method is called. */
+  public void leaseExpired();
+}

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java?rev=982881&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java Fri Aug  6 06:15:12 2010
@@ -0,0 +1,281 @@
+/**
+ * Copyright 2007 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.regionserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.ConcurrentModificationException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.TimeUnit;
+
+import java.io.IOException;
+
+/**
+ * Leases
+ *
+ * There are several server classes in HBase that need to track external
+ * clients that occasionally send heartbeats.
+ *
+ * <p>These external clients hold resources in the server class.
+ * Those resources need to be released if the external client fails to send a
+ * heartbeat after some interval of time passes.
+ *
+ * <p>The Leases class is a general reusable class for this kind of pattern.
+ * An instance of the Leases class will create a thread to do its dirty work.
+ * You should close() the instance if you want to clean up the thread properly.
+ *
+ * <p>
+ * NOTE: This class extends Thread rather than Chore because the sleep time
+ * can be interrupted when there is something to do, rather than the Chore
+ * sleep time which is invariant.
+ */
+public class Leases extends Thread {
+  private static final Log LOG = LogFactory.getLog(Leases.class.getName());
+  private final int leasePeriod;
+  private final int leaseCheckFrequency;
+  private volatile DelayQueue<Lease> leaseQueue = new DelayQueue<Lease>();
+  protected final Map<String, Lease> leases = new HashMap<String, Lease>();
+  private volatile boolean stopRequested = false;
+
+  /**
+   * Creates a lease monitor
+   *
+   * @param leasePeriod - length of time (milliseconds) that the lease is valid
+   * @param leaseCheckFrequency - how often the lease should be checked
+   * (milliseconds)
+   */
+  public Leases(final int leasePeriod, final int leaseCheckFrequency) {
+    this.leasePeriod = leasePeriod;
+    this.leaseCheckFrequency = leaseCheckFrequency;
+  }
+
+  /**
+   * @see java.lang.Thread#run()
+   */
+  @Override
+  public void run() {
+    while (!stopRequested || (stopRequested && leaseQueue.size() > 0) ) {
+      Lease lease = null;
+      try {
+        lease = leaseQueue.poll(leaseCheckFrequency, TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        continue;
+      } catch (ConcurrentModificationException e) {
+        continue;
+      } catch (Throwable e) {
+        LOG.fatal("Unexpected exception killed leases thread", e);
+        break;
+      }
+      if (lease == null) {
+        continue;
+      }
+      // A lease expired.  Run the expired code before removing from queue
+      // since its presence in queue is used to see if lease exists still.
+      if (lease.getListener() == null) {
+        LOG.error("lease listener is null for lease " + lease.getLeaseName());
+      } else {
+        lease.getListener().leaseExpired();
+      }
+      synchronized (leaseQueue) {
+        leases.remove(lease.getLeaseName());
+      }
+    }
+    close();
+  }
+
+  /**
+   * Shuts down this lease instance when all outstanding leases expire.
+   * Like {@link #close()} but rather than violently end all leases, waits
+   * first on extant leases to finish.  Use this method if the lease holders
+   * could loose data, leak locks, etc.  Presumes client has shutdown
+   * allocation of new leases.
+   */
+  public void closeAfterLeasesExpire() {
+    this.stopRequested = true;
+  }
+
+  /**
+   * Shut down this Leases instance.  All pending leases will be destroyed,
+   * without any cancellation calls.
+   */
+  public void close() {
+    LOG.info(Thread.currentThread().getName() + " closing leases");
+    this.stopRequested = true;
+    synchronized (leaseQueue) {
+      leaseQueue.clear();
+      leases.clear();
+      leaseQueue.notifyAll();
+    }
+    LOG.info(Thread.currentThread().getName() + " closed leases");
+  }
+
+  /**
+   * Obtain a lease
+   *
+   * @param leaseName name of the lease
+   * @param listener listener that will process lease expirations
+   * @throws LeaseStillHeldException
+   */
+  public void createLease(String leaseName, final LeaseListener listener)
+  throws LeaseStillHeldException {
+    if (stopRequested) {
+      return;
+    }
+    Lease lease = new Lease(leaseName, listener,
+        System.currentTimeMillis() + leasePeriod);
+    synchronized (leaseQueue) {
+      if (leases.containsKey(leaseName)) {
+        throw new LeaseStillHeldException(leaseName);
+      }
+      leases.put(leaseName, lease);
+      leaseQueue.add(lease);
+    }
+  }
+
+  /**
+   * Thrown if we are asked create a lease but lease on passed name already
+   * exists.
+   */
+  @SuppressWarnings("serial")
+  public static class LeaseStillHeldException extends IOException {
+    private final String leaseName;
+
+    /**
+     * @param name
+     */
+    public LeaseStillHeldException(final String name) {
+      this.leaseName = name;
+    }
+
+    /** @return name of lease */
+    public String getName() {
+      return this.leaseName;
+    }
+  }
+
+  /**
+   * Renew a lease
+   *
+   * @param leaseName name of lease
+   * @throws LeaseException
+   */
+  public void renewLease(final String leaseName) throws LeaseException {
+    synchronized (leaseQueue) {
+      Lease lease = leases.get(leaseName);
+      // We need to check to see if the remove is successful as the poll in the run()
+      // method could have completed between the get and the remove which will result
+      // in a corrupt leaseQueue.
+      if (lease == null || !leaseQueue.remove(lease)) {
+        throw new LeaseException("lease '" + leaseName +
+                "' does not exist or has already expired");
+      }
+      lease.setExpirationTime(System.currentTimeMillis() + leasePeriod);
+      leaseQueue.add(lease);
+    }
+  }
+
+  /**
+   * Client explicitly cancels a lease.
+   *
+   * @param leaseName name of lease
+   * @throws LeaseException
+   */
+  public void cancelLease(final String leaseName) throws LeaseException {
+    synchronized (leaseQueue) {
+      Lease lease = leases.remove(leaseName);
+      if (lease == null) {
+        throw new LeaseException("lease '" + leaseName + "' does not exist");
+      }
+      leaseQueue.remove(lease);
+    }
+  }
+
+  /** This class tracks a single Lease. */
+  private static class Lease implements Delayed {
+    private final String leaseName;
+    private final LeaseListener listener;
+    private long expirationTime;
+
+    Lease(final String leaseName, LeaseListener listener, long expirationTime) {
+      this.leaseName = leaseName;
+      this.listener = listener;
+      this.expirationTime = expirationTime;
+    }
+
+    /** @return the lease name */
+    public String getLeaseName() {
+      return leaseName;
+    }
+
+    /** @return listener */
+    public LeaseListener getListener() {
+      return this.listener;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      return this.hashCode() == ((Lease) obj).hashCode();
+    }
+
+    @Override
+    public int hashCode() {
+      return this.leaseName.hashCode();
+    }
+
+    public long getDelay(TimeUnit unit) {
+      return unit.convert(this.expirationTime - System.currentTimeMillis(),
+          TimeUnit.MILLISECONDS);
+    }
+
+    public int compareTo(Delayed o) {
+      long delta = this.getDelay(TimeUnit.MILLISECONDS) -
+        o.getDelay(TimeUnit.MILLISECONDS);
+
+      return this.equals(o) ? 0 : (delta > 0 ? 1 : -1);
+    }
+
+    /** @param expirationTime the expirationTime to set */
+    public void setExpirationTime(long expirationTime) {
+      this.expirationTime = expirationTime;
+    }
+
+    /**
+     * Get the expiration time for that lease
+     * @return expiration time
+     */
+    public long getExpirationTime() {
+      return this.expirationTime;
+    }
+
+  }
+}

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java Fri Aug  6 06:15:12 2010
@@ -1,101 +0,0 @@
-/**
- * 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.regionserver;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HServerAddress;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Manages the location of the current active Master for this RegionServer.
- * <p>
- * Listens for ZooKeeper events related to the master address. The node /master
- * will contain the address of the current master. This listener is interested
- * in NodeDeleted and NodeCreated events on /master.
- * <p>
- * Utilizes {@link ZooKeeperNodeTracker} for zk interactions.
- * <p>
- * You can get the current master via {@link #getMasterAddress()} or the
- * blocking method {@link #waitMasterAddress()}.
- */
-public class MasterAddressManager extends ZooKeeperNodeTracker {
-  private static final Log LOG = LogFactory.getLog(MasterAddressManager.class);
-
-  /**
-   * Construct a master address listener with the specified zookeeper reference.
-   * <p>
-   * This constructor does not trigger any actions, you must call methods
-   * explicitly.  Normally you will just want to execute {@link #start()} to
-   * begin tracking of the master address.
-   *
-   * @param watcher zk reference and watcher
-   * @param abortable abortable in case of fatal error
-   */
-  public MasterAddressManager(ZooKeeperWatcher watcher, Abortable abortable) {
-    super(watcher, watcher.masterAddressZNode, abortable);
-  }
-
-  /**
-   * Get the address of the current master if one is available.  Returns null
-   * if no current master.
-   *
-   * Use {@link #waitMasterAddress} if you want to block until the master is
-   * available.
-   * @return server address of current active master, or null if none available
-   */
-  public HServerAddress getMasterAddress() {
-    byte [] data = super.getData();
-    return data == null ? null : new HServerAddress(Bytes.toString(data));
-  }
-
-  /**
-   * Check if there is a master available.
-   * @return true if there is a master set, false if not.
-   */
-  public boolean hasMaster() {
-    return super.getData() != null;
-  }
-
-  /**
-   * Get the address of the current master.  If no master is available, method
-   * will block until one is available, the thread is interrupted, or timeout
-   * has passed.
-   *
-   * @param timeout maximum time to wait for master in millis, 0 for forever
-   * @return server address of current active master, null if timed out
-   * @throws InterruptedException if the thread is interrupted while waiting
-   */
-  public synchronized HServerAddress waitForMaster(long timeout)
-  throws InterruptedException {
-    byte [] data = super.blockUntilAvailable(timeout);
-    return data == null ? null : new HServerAddress(Bytes.toString(data));
-  }
-
-  @Override
-  protected Log getLog() {
-    return LOG;
-  }
-}

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java Fri Aug  6 06:15:12 2010
@@ -23,11 +23,11 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerInfo;
-import org.apache.hadoop.hbase.ServerController;
+import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.util.Progressable;
 
-public interface RegionServerController extends ServerController {
+public interface RegionServerController extends Server {
 
   // this is unfortunate but otherwise all the implementation of region
   // open/close must happen in HRS itself and not in handlers

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java?rev=982881&r1=982880&r2=982881&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java Fri Aug  6 06:15:12 2010
@@ -120,8 +120,7 @@ public class OpenRegionHandler extends E
                 }
                 openingInteger.set(vsn);
               } catch (KeeperException e) {
-                LOG.error("ZK exception refreshing OPENING node", e);
-                server.abort();
+                server.abort("ZK exception refreshing OPENING node", e);
               }
             }
       });

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/HBaseConfTool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/HBaseConfTool.java?rev=982881&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/HBaseConfTool.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/HBaseConfTool.java Fri Aug  6 06:15:12 2010
@@ -0,0 +1,41 @@
+/*
+ * 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.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+
+/**
+ * Tool that prints out a configuration.
+ * Pass the configuration key on the command-line.
+ */
+public class HBaseConfTool {
+  public static void main(String args[]) {
+    if (args.length < 1) {
+      System.err.println("Usage: HBaseConfTool <CONFIGURATION_KEY>");
+      System.exit(1);
+      return;
+    }
+
+    Configuration conf = HBaseConfiguration.create();
+    System.out.println(conf.get(args[0]));
+  }
+}
\ No newline at end of file