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 2011/10/14 01:18:44 UTC

svn commit: r1183163 [2/2] - in /hbase/branches/0.92: ./ 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/ipc/ src/main/java/org/ap...

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/HTable.java Thu Oct 13 23:18:42 2011
@@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.UnknownScannerException;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -192,7 +191,8 @@ public class HTable implements HTableInt
     }
     this.connection = HConnectionManager.getConnection(conf);
     this.scannerTimeout =
-      (int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
+      (int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
+        HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
     this.operationTimeout = HTableDescriptor.isMetaTable(tableName) ? HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT
         : conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
             HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
@@ -416,8 +416,9 @@ public class HTable implements HTableInt
       }
     };
     MetaScanner.metaScan(configuration, visitor, this.tableName);
-    return new Pair(startKeyList.toArray(new byte[startKeyList.size()][]),
-                endKeyList.toArray(new byte[endKeyList.size()][]));
+    return new Pair<byte [][], byte [][]>(
+      startKeyList.toArray(new byte[startKeyList.size()][]),
+      endKeyList.toArray(new byte[endKeyList.size()][]));
   }
 
   /**
@@ -851,7 +852,7 @@ public class HTable implements HTableInt
   @Override
   public void flushCommits() throws IOException {
     try {
-      connection.processBatchOfPuts(writeBuffer, tableName, pool);
+      this.connection.processBatchOfPuts(writeBuffer, tableName, pool);
     } finally {
       if (clearBufferOnFail) {
         writeBuffer.clear();

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java Thu Oct 13 23:18:42 2011
@@ -187,7 +187,7 @@ public class MetaScanner {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Scanning " + Bytes.toString(metaTableName) +
           " starting at row=" + Bytes.toStringBinary(startRow) + " for max=" +
-          rowUpperLimit + " rows");
+          rowUpperLimit + " rows using " + connection.toString());
       }
       callable = new ScannerCallable(connection, metaTableName, scan);
       // Open scanner

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/Result.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/Result.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/Result.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/Result.java Thu Oct 13 23:18:42 2011
@@ -20,12 +20,16 @@
 
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.collect.Ordering;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.WritableWithSize;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Writable;
 
 import java.io.DataInput;

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java Thu Oct 13 23:18:42 2011
@@ -15,9 +15,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hadoop.hbase.util.Bytes;
-
 import java.io.IOException;
+import java.util.Date;
 import java.util.List;
 
 /**
@@ -36,28 +35,53 @@ public class RetriesExhaustedException e
   }
 
   /**
+   * Datastructure that allows adding more info around Throwable incident.
+   */
+  public static class ThrowableWithExtraContext {
+    private final Throwable t;
+    private final long when;
+    private final String extras;
+
+    public ThrowableWithExtraContext(final Throwable t, final long when,
+        final String extras) {
+      this.t = t;
+      this.when = when;
+      this.extras = extras;
+    }
+ 
+    @Override
+    public String toString() {
+      return new Date(this.when).toString() + ", " + extras + ", " + t.toString();
+    }
+  }
+
+  /**
    * Create a new RetriesExhaustedException from the list of prior failures.
-   * @param serverName name of HRegionServer
-   * @param regionName name of region
-   * @param row The row we were pursuing when we ran out of retries
+   * @param callableVitals Details from the {@link ServerCallable} we were using
+   * when we got this exception.
    * @param numTries The number of tries we made
    * @param exceptions List of exceptions that failed before giving up
    */
-  public RetriesExhaustedException(String serverName, final byte [] regionName,
-      final byte []  row, int numTries, List<Throwable> exceptions) {
-    super(getMessage(serverName, regionName, row, numTries, exceptions));
+  public RetriesExhaustedException(final String callableVitals, int numTries,
+      List<Throwable> exceptions) {
+    super(getMessage(callableVitals, numTries, exceptions));
+  }
+
+  /**
+   * Create a new RetriesExhaustedException from the list of prior failures.
+   * @param numTries
+   * @param exceptions List of exceptions that failed before giving up
+   */
+  public RetriesExhaustedException(final int numTries,
+      final List<ThrowableWithExtraContext> exceptions) {
+    super(getMessage(numTries, exceptions));
   }
 
-  private static String getMessage(String serverName, final byte [] regionName,
-      final byte [] row,
-      int numTries, List<Throwable> exceptions) {
-    StringBuilder buffer = new StringBuilder("Trying to contact region server ");
-    buffer.append(serverName);
-    buffer.append(" for region ");
-    buffer.append(regionName == null? "": Bytes.toStringBinary(regionName));
-    buffer.append(", row '");
-    buffer.append(row == null? "": Bytes.toStringBinary(row));
-    buffer.append("', but failed after ");
+  private static String getMessage(String callableVitals, int numTries,
+      List<Throwable> exceptions) {
+    StringBuilder buffer = new StringBuilder("Failed contacting ");
+    buffer.append(callableVitals);
+    buffer.append(" after ");
     buffer.append(numTries + 1);
     buffer.append(" attempts.\nExceptions:\n");
     for (Throwable t : exceptions) {
@@ -66,4 +90,16 @@ public class RetriesExhaustedException e
     }
     return buffer.toString();
   }
+
+  private static String getMessage(final int numTries,
+      final List<ThrowableWithExtraContext> exceptions) {
+    StringBuilder buffer = new StringBuilder("Failed after attempts=");
+    buffer.append(numTries + 1);
+    buffer.append(", exceptions:\n");
+    for (ThrowableWithExtraContext t : exceptions) {
+      buffer.append(t.toString());
+      buffer.append("\n");
+    }
+    return buffer.toString();
+  }
 }
\ No newline at end of file

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java Thu Oct 13 23:18:42 2011
@@ -59,9 +59,9 @@ public class ScannerCallable extends Ser
    * @throws IOException
    */
   @Override
-  public void instantiateServer(boolean reload) throws IOException {
+  public void connect(boolean reload) throws IOException {
     if (!instantiated || reload) {
-      super.instantiateServer(reload);
+      super.connect(reload);
       instantiated = true;
     }
   }

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java Thu Oct 13 23:18:42 2011
@@ -31,7 +31,14 @@ import org.apache.hadoop.hbase.ipc.HRegi
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Abstract class that implements Callable, used by retryable actions.
+ * Abstract class that implements {@link Callable}.  Implementation stipulates
+ * return type and method we actually invoke on remote Server.  Usually
+ * used inside a try/catch that fields usual connection failures all wrapped
+ * up in a retry loop.
+ * <p>Call {@link #connect(boolean)} to connect to server hosting region
+ * that contains the passed row in the passed table before invoking
+ * {@link #call()}.
+ * @see HConnection#getRegionServerWithoutRetries(ServerCallable)
  * @param <T> the class that the ServerCallable handles
  */
 public abstract class ServerCallable<T> implements Callable<T> {
@@ -44,9 +51,9 @@ public abstract class ServerCallable<T> 
   protected long startTime, endTime;
 
   /**
-   * @param connection connection callable is on
-   * @param tableName table name callable is on
-   * @param row row we are querying
+   * @param connection Connection to use.
+   * @param tableName Table name to which <code>row</code> belongs.
+   * @param row The row we want in <code>tableName</code>.
    */
   public ServerCallable(HConnection connection, byte [] tableName, byte [] row) {
     this(connection, tableName, row, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
@@ -58,34 +65,37 @@ public abstract class ServerCallable<T> 
     this.row = row;
     this.callTimeout = callTimeout;
   }
+
   /**
-   *
-   * @param reload set this to true if connection should re-find the region
+   * Connect to the server hosting region with row from tablename.
+   * @param reload Set this to true if connection should re-find the region
    * @throws IOException e
    */
-  public void instantiateServer(boolean reload) throws IOException {
+  public void connect(final boolean reload) throws IOException {
     this.location = connection.getRegionLocation(tableName, row, reload);
     this.server = connection.getHRegionConnection(location.getHostname(),
       location.getPort());
   }
 
-  /** @return the server name */
+  /** @return the server name
+   * @deprecated Just use {@link #toString()} instead.
+   */
   public String getServerName() {
-    if (location == null) {
-      return null;
-    }
+    if (location == null) return null;
     return location.getHostnamePort();
   }
 
-  /** @return the region name */
+  /** @return the region name
+   * @deprecated Just use {@link #toString()} instead.
+   */
   public byte[] getRegionName() {
-    if (location == null) {
-      return null;
-    }
+    if (location == null) return null;
     return location.getRegionInfo().getRegionName();
   }
 
-  /** @return the row */
+  /** @return the row
+   * @deprecated Just use {@link #toString()} instead.
+   */
   public byte [] getRow() {
     return row;
   }

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java Thu Oct 13 23:18:42 2011
@@ -87,7 +87,7 @@ public interface HMasterInterface extend
    * @throws IOException
    */
   public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
-      throws IOException;
+  throws IOException;
 
   /**
    * Adds a column to the specified table

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Thu Oct 13 23:18:42 2011
@@ -267,11 +267,12 @@ public class AssignmentManager extends Z
    * @param tableName
    * @return Pair indicating the status of the alter command
    * @throws IOException
+   * @throws InterruptedException 
    */
   public Pair<Integer, Integer> getReopenStatus(byte[] tableName)
-      throws IOException {
-    List <HRegionInfo> hris = MetaReader.getTableRegions(
-                              this.master.getCatalogTracker(), tableName);
+  throws IOException, InterruptedException {
+    List <HRegionInfo> hris =
+      MetaReader.getTableRegions(this.master.getCatalogTracker(), tableName);
     Integer pending = 0;
     for(HRegionInfo hri : hris) {
       if(regionsToReopen.get(hri.getEncodedName()) != null) {
@@ -730,7 +731,7 @@ public class AssignmentManager extends Z
         case RS_ZK_REGION_OPENING:
           // Should see OPENING after we have asked it to OPEN or additional
           // times after already being in state of OPENING
-          if(regionState == null ||
+          if (regionState == null ||
               (!regionState.isPendingOpen() && !regionState.isOpening())) {
             LOG.warn("Received OPENING for region " +
                 prettyPrintedRegionName +
@@ -1764,7 +1765,6 @@ public class AssignmentManager extends Z
       // Presume that master has stale data.  Presume remote side just split.
       // Presume that the split message when it comes in will fix up the master's
       // in memory cluster state.
-      return;
     } catch (Throwable t) {
       if (t instanceof RemoteException) {
         t = ((RemoteException)t).unwrapRemoteException();
@@ -2082,13 +2082,13 @@ public class AssignmentManager extends Z
   Map<ServerName, List<Pair<HRegionInfo, Result>>> rebuildUserRegions()
   throws IOException, KeeperException {
     // Region assignment from META
-    List<Result> results = MetaReader.fullScanOfResults(this.catalogTracker);
+    List<Result> results = MetaReader.fullScan(this.catalogTracker);
     // Map of offline servers and their regions to be returned
     Map<ServerName, List<Pair<HRegionInfo,Result>>> offlineServers =
       new TreeMap<ServerName, List<Pair<HRegionInfo, Result>>>();
     // Iterate regions in META
     for (Result result : results) {
-      Pair<HRegionInfo, ServerName> region = MetaReader.metaRowToRegionPair(result);
+      Pair<HRegionInfo, ServerName> region = MetaReader.parseCatalogResult(result);
       if (region == null) continue;
       HRegionInfo regionInfo = region.getFirst();
       ServerName regionLocation = region.getSecond();

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Thu Oct 13 23:18:42 2011
@@ -25,7 +25,6 @@ import java.lang.reflect.InvocationTarge
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
@@ -41,21 +40,18 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
-import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
-import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.catalog.MetaReader;
-import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.MetaScanner;
-import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
@@ -64,7 +60,7 @@ import org.apache.hadoop.hbase.ipc.HBase
 import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
 import org.apache.hadoop.hbase.ipc.RpcServer;
-import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
 import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
@@ -72,18 +68,14 @@ import org.apache.hadoop.hbase.master.ha
 import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
-import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
 import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.InfoServer;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Sleeper;
@@ -205,7 +197,8 @@ implements HMasterInterface, HMasterRegi
   public HMaster(final Configuration conf)
   throws IOException, KeeperException, InterruptedException {
     this.conf = conf;
-
+    // Set how many times to retry talking to another server over HConnection.
+    HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
     // Server to handle client requests.
     String hostname = DNS.getDefaultHost(
       conf.get("hbase.master.dns.interface", "default"),
@@ -470,7 +463,9 @@ implements HMasterInterface, HMasterRegi
     // Update meta with new HRI if required. i.e migrate all HRI with HTD to
     // HRI with out HTD in meta and update the status in ROOT. This must happen
     // before we assign all user regions or else the assignment will fail.
-    updateMetaWithNewHRI();
+    // TODO: Remove this when we do 0.94.
+    org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD.
+      updateMetaWithNewHRI(this);
 
     // Fixup assignment manager status
     status.setStatus("Starting assignment manager");
@@ -500,45 +495,6 @@ implements HMasterInterface, HMasterRegi
     }
   }
 
-  public boolean isMetaHRIUpdated()
-      throws IOException {
-    boolean metaUpdated = false;
-    Get get = new Get(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
-    get.addColumn(HConstants.CATALOG_FAMILY,
-        HConstants.META_MIGRATION_QUALIFIER);
-    Result r =
-        catalogTracker.waitForRootServerConnectionDefault().get(
-        HRegionInfo.ROOT_REGIONINFO.getRegionName(), get);
-    if (r != null && r.getBytes() != null)
-    {
-      byte[] metaMigrated = r.getValue(HConstants.CATALOG_FAMILY,
-          HConstants.META_MIGRATION_QUALIFIER);
-      String migrated = Bytes.toString(metaMigrated);
-      metaUpdated = new Boolean(migrated).booleanValue();
-    } else {
-      LOG.info("metaUpdated = NULL.");
-    }
-    LOG.info("Meta updated status = " + metaUpdated);
-    return metaUpdated;
-  }
-
-
-  boolean updateMetaWithNewHRI() throws IOException {
-    if (!isMetaHRIUpdated()) {
-      LOG.info("Meta has HRI with HTDs. Updating meta now.");
-      try {
-        MetaEditor.migrateRootAndMeta(this);
-        LOG.info("ROOT and Meta updated with new HRI.");
-        return true;
-      } catch (IOException e) {
-        throw new RuntimeException("Update ROOT/Meta with new HRI failed." +
-            "Master startup aborted.");
-      }
-    }
-    LOG.info("ROOT/Meta already up-to date with new HRI.");
-    return true;
-  }
-
   /**
    * Check <code>-ROOT-</code> and <code>.META.</code> are assigned.  If not,
    * assign them.
@@ -993,6 +949,7 @@ implements HMasterInterface, HMasterRegi
            Bytes.equals(tableName, HConstants.META_TABLE_NAME);
   }
 
+  @Override
   public void deleteTable(final byte [] tableName) throws IOException {
     if (cpHost != null) {
       cpHost.preDeleteTable(tableName);
@@ -1004,16 +961,13 @@ implements HMasterInterface, HMasterRegi
     }
   }
 
-  /**
-   * Get the number of regions of the table that have been updated by the alter.
-   *
-   * @return Pair indicating the number of regions updated Pair.getFirst is the
-   *         regions that are yet to be updated Pair.getSecond is the total number
-   *         of regions of the table
-   */
   public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
   throws IOException {
-    return this.assignmentManager.getReopenStatus(tableName);
+    try {
+      return this.assignmentManager.getReopenStatus(tableName);
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted", e);
+    }
   }
 
   public void addColumn(byte [] tableName, HColumnDescriptor column)
@@ -1098,7 +1052,7 @@ implements HMasterInterface, HMasterRegi
           if (data == null || data.size() <= 0) {
             return true;
           }
-          Pair<HRegionInfo, ServerName> pair = MetaReader.metaRowToRegionPair(data);
+          Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(data);
           if (pair == null) {
             return false;
           }

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java Thu Oct 13 23:18:42 2011
@@ -111,8 +111,8 @@ public class EnableTableHandler extends 
     boolean done = false;
     // Get the regions of this table. We're done when all listed
     // tables are onlined.
-    List<HRegionInfo> regionsInMeta =
-      MetaReader.getTableRegions(this.ct, tableName, true);
+    List<HRegionInfo> regionsInMeta;
+    regionsInMeta = MetaReader.getTableRegions(this.ct, tableName, true);
     int countOfRegionsInTable = regionsInMeta.size();
     List<HRegionInfo> regions = regionsToAssign(regionsInMeta);
     int regionsCount = regions.size();

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java Thu Oct 13 23:18:42 2011
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.master.De
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Writables;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -265,14 +264,16 @@ public class ServerShutdownHandler exten
           LOG.debug("Removed " + rit.getRegion().getRegionNameAsString() +
           " from list of regions to assign because in RIT" + " region state: "
           + rit.getState());
-          hris.remove(rit.getRegion());
+          if (hris != null) hris.remove(rit.getRegion());
         }
       }
 
-      LOG.info("Reassigning " + (hris == null? 0: hris.size()) +
-          " region(s) that " + serverName +
-          " was carrying (skipping " + regionsInTransition.size() +
-      " regions(s) that are already in transition)");
+      assert regionsInTransition != null;
+      LOG.info("Reassigning " + ((hris == null)? 0: hris.size()) +
+        " region(s) that " + (serverName == null? "null": serverName)  +
+        " was carrying (skipping " +
+        regionsInTransition.size() +
+        " regions(s) that are already in transition)");
 
       // Iterate regions that were on this server and assign them
       if (hris != null) {
@@ -342,7 +343,8 @@ public class ServerShutdownHandler exten
       final AssignmentManager assignmentManager,
       final CatalogTracker catalogTracker)
   throws IOException {
-    HRegionInfo daughter = getHRegionInfo(result, qualifier);
+    HRegionInfo daughter =
+      MetaReader.parseHRegionInfoFromCatalogResult(result, qualifier);
     if (daughter == null) return;
     if (isDaughterMissing(catalogTracker, daughter)) {
       LOG.info("Fixup; missing daughter " + daughter.getRegionNameAsString());
@@ -360,21 +362,6 @@ public class ServerShutdownHandler exten
   }
 
   /**
-   * Interpret the content of the cell at {@link HConstants#CATALOG_FAMILY} and
-   * <code>qualifier</code> as an HRegionInfo and return it, or null.
-   * @param r Result instance to pull from.
-   * @param qualifier Column family qualifier
-   * @return An HRegionInfo instance or null.
-   * @throws IOException
-   */
-  private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier)
-  throws IOException {
-    byte [] bytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
-    if (bytes == null || bytes.length <= 0) return null;
-    return Writables.getHRegionInfoOrNull(bytes);
-  }
-
-  /**
    * Look for presence of the daughter OR of a split of the daughter in .META.
    * Daughter could have been split over on regionserver before a run of the
    * catalogJanitor had chance to clear reference from parent.
@@ -416,7 +403,8 @@ public class ServerShutdownHandler exten
 
     @Override
     public boolean visit(Result r) throws IOException {
-      HRegionInfo hri = getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
+      HRegionInfo hri =
+        MetaReader.parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
       if (hri == null) {
         LOG.warn("No serialized HRegionInfo in " + r);
         return true;

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Oct 13 23:18:42 2011
@@ -3060,9 +3060,9 @@ public class HRegion implements HeapSize
           EnvironmentEdgeManager.currentTimeMillis(),
           Writables.getBytes(r.getRegionInfo())));
       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
-          HConstants.META_MIGRATION_QUALIFIER,
+          org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD.META_MIGRATION_QUALIFIER,
           EnvironmentEdgeManager.currentTimeMillis(),
-          Bytes.toBytes("true")));
+          Bytes.toBytes(true)));
 
       meta.put(HConstants.CATALOG_FAMILY, edits);
     } finally {

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Oct 13 23:18:42 2011
@@ -81,6 +81,7 @@ import org.apache.hadoop.hbase.catalog.R
 import org.apache.hadoop.hbase.client.Action;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.MultiAction;
 import org.apache.hadoop.hbase.client.MultiPut;
@@ -332,6 +333,8 @@ public class HRegionServer implements HR
   throws IOException, InterruptedException {
     this.fsOk = true;
     this.conf = conf;
+    // Set how many times to retry talking to another server over HConnection.
+    HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
     this.isOnline = false;
     checkCodecs(this.conf);
 

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java Thu Oct 13 23:18:42 2011
@@ -78,4 +78,4 @@ public interface RegionServerServices ex
    */
   public Map<byte[], Boolean> getRegionsInTransitionInRS();
   
-}
+}

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Thu Oct 13 23:18:42 2011
@@ -272,17 +272,19 @@ public class ThriftServer {
     public List<TRegionInfo> getTableRegions(ByteBuffer tableName)
     throws IOError {
       try{
-        List<HRegionInfo> HRegions = this.admin.getTableRegions(tableName.array());
+        List<HRegionInfo> hris = this.admin.getTableRegions(tableName.array());
         List<TRegionInfo> regions = new ArrayList<TRegionInfo>();
 
-        for (HRegionInfo regionInfo : HRegions){
-          TRegionInfo region = new TRegionInfo();
-          region.startKey = ByteBuffer.wrap(regionInfo.getStartKey());
-          region.endKey = ByteBuffer.wrap(regionInfo.getEndKey());
-          region.id = regionInfo.getRegionId();
-          region.name = ByteBuffer.wrap(regionInfo.getRegionName());
-          region.version = regionInfo.getVersion();
-          regions.add(region);
+        if (hris != null) {
+          for (HRegionInfo regionInfo : hris){
+            TRegionInfo region = new TRegionInfo();
+            region.startKey = ByteBuffer.wrap(regionInfo.getStartKey());
+            region.endKey = ByteBuffer.wrap(regionInfo.getEndKey());
+            region.id = regionInfo.getRegionId();
+            region.name = ByteBuffer.wrap(regionInfo.getRegionName());
+            region.version = regionInfo.getVersion();
+            regions.add(region);
+          }
         }
         return regions;
       } catch (IOException e){

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java Thu Oct 13 23:18:42 2011
@@ -891,8 +891,7 @@ public class HBaseFsck {
 
           // record the latest modification of this META record
           long ts =  Collections.max(result.list(), comp).getTimestamp();
-          Pair<HRegionInfo, ServerName> pair =
-            MetaReader.metaRowToRegionPair(result);
+          Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(result);
           if (pair == null || pair.getFirst() == null) {
             emptyRegionInfoQualifiers.add(result);
             return true;

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaNodeTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaNodeTracker.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaNodeTracker.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaNodeTracker.java Thu Oct 13 23:18:42 2011
@@ -19,52 +19,29 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.catalog.CatalogTracker;
 
 /**
  * Tracks the unassigned zookeeper node used by the META table.
- *
- * A callback is made into the passed {@link CatalogTracker} when
- * <code>.META.</code> completes a new assignment.
  * <p>
  * If META is already assigned when instantiating this class, you will not
  * receive any notification for that assignment.  You will receive a
  * notification after META has been successfully assigned to a new location.
  */
 public class MetaNodeTracker extends ZooKeeperNodeTracker {
-  private static final Log LOG = LogFactory.getLog(MetaNodeTracker.class);
-
-  /** Catalog tracker to notify when META has a new assignment completed. */
-  private final CatalogTracker catalogTracker;
-
   /**
    * Creates a meta node tracker.
    * @param watcher
    * @param abortable
    */
-  public MetaNodeTracker(final ZooKeeperWatcher watcher,
-      final CatalogTracker catalogTracker, final Abortable abortable) {
+  public MetaNodeTracker(final ZooKeeperWatcher watcher, final Abortable abortable) {
     super(watcher, ZKUtil.joinZNode(watcher.assignmentZNode,
         HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()), abortable);
-    this.catalogTracker = catalogTracker;
   }
 
   @Override
   public void nodeDeleted(String path) {
     super.nodeDeleted(path);
-    if (!path.equals(node)) return;
-    LOG.info("Detected completed assignment of META, notifying catalog tracker");
-    try {
-      this.catalogTracker.waitForMetaServerConnectionDefault();
-    } catch (IOException e) {
-      LOG.warn("Tried to reset META server location after seeing the " +
-        "completion of a new META assignment but got an IOE", e);
-    }
   }
 }
\ No newline at end of file

Modified: hbase/branches/0.92/src/main/ruby/hbase/admin.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/ruby/hbase/admin.rb?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/ruby/hbase/admin.rb (original)
+++ hbase/branches/0.92/src/main/ruby/hbase/admin.rb Thu Oct 13 23:18:42 2011
@@ -263,8 +263,13 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Truncates table (deletes all records by recreating the table)
-    def truncate(table_name)
-      h_table = org.apache.hadoop.hbase.client.HTable.new(table_name)
+    def truncate(table_name, conf = nil)
+      h_table = nil
+      unless conf
+        h_table = org.apache.hadoop.hbase.client.HTable.new(table_name)
+      else
+        h_table = org.apache.hadoop.hbase.client.HTable.new(conf, table_name)
+      end
       table_description = h_table.getTableDescriptor()
       yield 'Disabling table...' if block_given?
       disable(table_name)

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java Thu Oct 13 23:18:42 2011
@@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -81,15 +80,11 @@ public class TestRegionRebalancing {
     this.table = new HTable(UTIL.getConfiguration(), this.desc.getName());
     CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration());
     ct.start();
-    Map<HRegionInfo, ServerName> regions = null;
     try {
-      regions = MetaReader.fullScan(ct);
+      MetaReader.fullScanMetaAndPrint(ct);
     } finally {
       ct.stop();
     }
-    for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
-      LOG.info(e);
-    }
     assertEquals("Test table should have right number of regions",
       HBaseTestingUtility.KEYS.length + 1/*One extra to account for start/end keys*/,
       this.table.getStartKeys().length);

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java Thu Oct 13 23:18:42 2011
@@ -35,16 +35,20 @@ import org.apache.hadoop.hbase.Abortable
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ServerCallable;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.util.Progressable;
@@ -160,27 +164,44 @@ public class TestCatalogTracker {
     t.join();
   }
 
-  @Test public void testGetMetaServerConnectionFails()
+  @Test
+  public void testGetMetaServerConnectionFails()
   throws IOException, InterruptedException, KeeperException {
-    HConnection connection = Mockito.mock(HConnection.class);
-    ConnectException connectException =
-      new ConnectException("Connection refused");
-    final HRegionInterface implementation =
-      Mockito.mock(HRegionInterface.class);
-    Mockito.when(implementation.get((byte [])Mockito.any(), (Get)Mockito.any())).
-      thenThrow(connectException);
-    Mockito.when(connection.getHRegionConnection((HServerAddress)Matchers.anyObject(), Matchers.anyBoolean())).
-      thenReturn(implementation);
-    Assert.assertNotNull(connection.getHRegionConnection(new HServerAddress(), false));
-    final CatalogTracker ct = constructAndStartCatalogTracker(connection);
+    HConnection connection =
+      HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
     try {
-      RootLocationEditor.setRootLocation(this.watcher,
-        new ServerName("example.com", 1234, System.currentTimeMillis()));
-      Assert.assertFalse(ct.verifyMetaRegionLocation(100));
+      // Mock an HRegionInterface.
+      final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
+      // If a 'get' is called on mocked interface, throw connection refused.
+      Mockito.when(implementation.get((byte[]) Mockito.any(), (Get) Mockito.any())).
+        thenThrow(new ConnectException("Connection refused"));
+      // Make it so our implementation is returned when we do a connection.
+      // Need to fake out the location lookup stuff first.
+      ServerName sn = new ServerName("example.com", 1234, System.currentTimeMillis());
+      final HRegionLocation anyLocation =
+        new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn.getHostname(),
+          sn.getPort());
+      Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
+            (byte[]) Mockito.any(), Mockito.anyBoolean())).
+        thenReturn(anyLocation);
+      Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
+          Mockito.anyInt(), Matchers.anyBoolean())).
+        thenReturn(implementation);
+      // Now start up the catalogtracker with our doctored Connection.
+      final CatalogTracker ct = constructAndStartCatalogTracker(connection);
+      try {
+        RootLocationEditor.setRootLocation(this.watcher, sn);
+        long timeout = UTIL.getConfiguration().
+          getLong("hbase.catalog.verification.timeout", 1000);
+        Assert.assertFalse(ct.verifyMetaRegionLocation(timeout));
+      } finally {
+        // Clean out root location or later tests will be confused... they
+        // presume start fresh in zk.
+        RootLocationEditor.deleteRootLocation(this.watcher);
+      }
     } finally {
-      // Clean out root location or later tests will be confused... they presume
-      // start fresh in zk.
-      RootLocationEditor.deleteRootLocation(this.watcher);
+      // Clear out our doctored connection or could mess up subsequent tests.
+      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
     }
   }
 
@@ -200,9 +221,9 @@ public class TestCatalogTracker {
       Mockito.mock(HRegionInterface.class);
     Mockito.when(implementation.getRegionInfo((byte [])Mockito.any())).
       thenThrow(connectException);
-    Mockito.when(connection.getHRegionConnection((HServerAddress)Matchers.anyObject(), Matchers.anyBoolean())).
+    Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
+      Mockito.anyInt(), Mockito.anyBoolean())).
       thenReturn(implementation);
-    Assert.assertNotNull(connection.getHRegionConnection(new HServerAddress(), false));
     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
     try {
       RootLocationEditor.setRootLocation(this.watcher,
@@ -225,8 +246,14 @@ public class TestCatalogTracker {
   @Test (expected = NotAllMetaRegionsOnlineException.class)
   public void testTimeoutWaitForMeta()
   throws IOException, InterruptedException {
-    final CatalogTracker ct = constructAndStartCatalogTracker();
-    ct.waitForMeta(100);
+    HConnection connection =
+      HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
+    try {
+      final CatalogTracker ct = constructAndStartCatalogTracker(connection);
+      ct.waitForMeta(100);
+    } finally {
+      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+    }
   }
 
   /**
@@ -259,62 +286,84 @@ public class TestCatalogTracker {
 
   /**
    * Test waiting on meta w/ no timeout specified.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
+   * @throws Exception 
    */
   @Test public void testNoTimeoutWaitForMeta()
-  throws IOException, InterruptedException, KeeperException {
+  throws Exception {
     // Mock an HConnection and a HRegionInterface implementation.  Have the
     // HConnection return the HRI.  Have the HRI return a few mocked up responses
     // to make our test work.
-    HConnection connection = Mockito.mock(HConnection.class);
-    HRegionInterface  mockHRI = Mockito.mock(HRegionInterface.class);
-    // Make the HRI return an answer no matter how Get is called.  Same for
-    // getHRegionInfo.  Thats enough for this test.
-    Mockito.when(connection.getHRegionConnection((String)Mockito.any(),
-      Matchers.anyInt())).thenReturn(mockHRI);
-
-    final CatalogTracker ct = constructAndStartCatalogTracker(connection);
-    ServerName hsa = ct.getMetaLocation();
-    Assert.assertNull(hsa);
-
-    // Now test waiting on meta location getting set.
-    Thread t = new WaitOnMetaThread(ct) {
-      @Override
-      void doWaiting() throws InterruptedException {
-        this.ct.waitForMeta();
-      }
-    };
-    startWaitAliveThenWaitItLives(t, 1000);
+    HConnection connection =
+      HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
+    try {
+      // Mock an HRegionInterface.
+      
+      final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
+      // Make it so our implementation is returned when we do a connection.
+      // Need to fake out the location lookup stuff first.
+      ServerName sn = new ServerName("example.com", 1234, System.currentTimeMillis());
+      final HRegionLocation anyLocation =
+        new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn.getHostname(),
+          sn.getPort());
+      Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
+            (byte[]) Mockito.any(), Mockito.anyBoolean())).
+        thenReturn(anyLocation);
+      // Have implementation returned which ever way getHRegionConnection is called.
+      Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
+          Mockito.anyInt(), Matchers.anyBoolean())).
+        thenReturn(implementation);
+      Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
+          Mockito.anyInt())).
+        thenReturn(implementation);
+
+      final CatalogTracker ct = constructAndStartCatalogTracker(connection);
+      ServerName hsa = ct.getMetaLocation();
+      Assert.assertNull(hsa);
+
+      // Now test waiting on meta location getting set.
+      Thread t = new WaitOnMetaThread(ct) {
+        @Override
+        void doWaiting() throws InterruptedException {
+          this.ct.waitForMeta();
+        }
+      };
+      startWaitAliveThenWaitItLives(t, 1000);
 
-    // Now the ct is up... set into the mocks some answers that make it look
-    // like things have been getting assigned.  Make it so we'll return a
-    // location (no matter what the Get is).  Same for getHRegionInfo -- always
-    // just return the meta region.
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
-    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
-      HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
-      Bytes.toBytes(SN.getHostAndPort())));
-    kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
-      HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
-      Bytes.toBytes(SN.getStartcode())));
-    final Result result = new Result(kvs);
-    Mockito.when(mockHRI.get((byte [])Mockito.any(), (Get)Mockito.any())).
-      thenReturn(result);
-    Mockito.when(mockHRI.getRegionInfo((byte [])Mockito.any())).
-      thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
-    // This should trigger wake up of meta wait (Its the removal of the meta
-    // region unassigned node that triggers catalogtrackers that a meta has
-    // been assigned.
-    String node = ct.getMetaNodeTracker().getNode();
-    ZKUtil.createAndFailSilent(this.watcher, node);
-    MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN);
-    ZKUtil.deleteNode(this.watcher, node);
-    // Join the thread... should exit shortly.
-    t.join();
-    // Now meta is available.
-    Assert.assertTrue(ct.getMetaLocation().equals(SN));
+      // Now the ct is up... set into the mocks some answers that make it look
+      // like things have been getting assigned. Make it so we'll return a
+      // location (no matter what the Get is). Same for getHRegionInfo -- always
+      // just return the meta region.
+      List<KeyValue> kvs = new ArrayList<KeyValue>();
+      kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
+        HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
+        Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
+      kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
+        HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
+        Bytes.toBytes(SN.getHostAndPort())));
+      kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
+        HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
+        Bytes.toBytes(SN.getStartcode())));
+      final Result result = new Result(kvs);
+      Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
+        thenReturn(result);
+      Mockito.when(implementation.getRegionInfo((byte[]) Mockito.any())).
+        thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
+      // This should trigger wake up of meta wait (Its the removal of the meta
+      // region unassigned node that triggers catalogtrackers that a meta has
+      // been assigned).
+      String node = ct.getMetaNodeTracker().getNode();
+      ZKUtil.createAndFailSilent(this.watcher, node);
+      MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN);
+      ZKUtil.deleteNode(this.watcher, node);
+      // Go get the new meta location. waitForMeta gets and verifies meta.
+      Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
+      // Join the thread... should exit shortly.
+      t.join();
+      // Now meta is available.
+      Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
+    } finally {
+      HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
+    }
   }
 
   private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
@@ -361,4 +410,4 @@ public class TestCatalogTracker {
       this.ct.waitForRoot();
     }
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java Thu Oct 13 23:18:42 2011
@@ -19,7 +19,10 @@
  */
 package org.apache.hadoop.hbase.catalog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.List;
@@ -34,8 +37,6 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -70,11 +71,15 @@ public class TestMetaReaderEditor {
   };
 
   @BeforeClass public static void beforeClass() throws Exception {
-    UTIL.startMiniCluster();
+    UTIL.startMiniCluster(3);
   }
 
   @Before public void setup() throws IOException, InterruptedException {
     Configuration c = new Configuration(UTIL.getConfiguration());
+    // Tests to 4 retries every 5 seconds. Make it try every 1 second so more
+    // responsive.  1 second is default as is ten retries.
+    c.setLong("hbase.client.pause", 1000);
+    c.setInt("hbase.client.retries.number", 10);
     zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE);
     ct = new CatalogTracker(zkw, c, ABORTABLE);
     ct.start();
@@ -84,6 +89,111 @@ public class TestMetaReaderEditor {
     UTIL.shutdownMiniCluster();
   }
 
+  /**
+   * Does {@link MetaReader#getRegion(CatalogTracker, byte[])} and a write
+   * against .META. while its hosted server is restarted to prove our retrying
+   * works.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test (timeout = 180000) public void testRetrying()
+  throws IOException, InterruptedException {
+    final String name = "testRetrying";
+    LOG.info("Started " + name);
+    final byte [] nameBytes = Bytes.toBytes(name);
+    HTable t = UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
+    int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
+    // Test it works getting a region from just made user table.
+    final List<HRegionInfo> regions =
+      testGettingTableRegions(this.ct, nameBytes, regionCount);
+    MetaTask reader = new MetaTask(this.ct, "reader") {
+      @Override
+      void metaTask() throws Throwable {
+        testGetRegion(this.ct, regions.get(0));
+        LOG.info("Read " + regions.get(0).getEncodedName());
+      }
+    };
+    MetaTask writer = new MetaTask(this.ct, "writer") {
+      @Override
+      void metaTask() throws Throwable {
+        MetaEditor.addRegionToMeta(this.ct, regions.get(0));
+        LOG.info("Wrote " + regions.get(0).getEncodedName());
+      }
+    };
+    reader.start();
+    writer.start();
+    // Make sure reader and writer are working.
+    assertTrue(reader.isProgressing());
+    assertTrue(writer.isProgressing());
+    // Kill server hosting meta -- twice  . See if our reader/writer ride over the
+    // meta moves.  They'll need to retry.
+    for (int i = 0; i < 2; i++) {
+      LOG.info("Restart=" + i);
+      UTIL.ensureSomeRegionServersAvailable(2);
+      int index = -1;
+      do {
+        index = UTIL.getMiniHBaseCluster().getServerWithMeta();
+      } while (index == -1);
+      UTIL.getMiniHBaseCluster().abortRegionServer(index);
+      UTIL.getMiniHBaseCluster().waitOnRegionServer(index);
+    }
+    assertTrue(reader.toString(), reader.isProgressing());
+    assertTrue(writer.toString(), writer.isProgressing());
+    reader.stop = true;
+    writer.stop = true;
+    reader.join();
+    writer.join();
+  }
+
+  /**
+   * Thread that runs a MetaReader/MetaEditor task until asked stop.
+   */
+  abstract static class MetaTask extends Thread {
+    boolean stop = false;
+    int count = 0;
+    Throwable t = null;
+    final CatalogTracker ct;
+
+    MetaTask(final CatalogTracker ct, final String name) {
+      super(name);
+      this.ct = ct;
+    }
+
+    @Override
+    public void run() {
+      try {
+        while(!this.stop) {
+          LOG.info("Before " + this.getName()+ ", count=" + this.count);
+          metaTask();
+          this.count += 1;
+          LOG.info("After " + this.getName() + ", count=" + this.count);
+          Thread.sleep(100);
+        }
+      } catch (Throwable t) {
+        LOG.info(this.getName() + " failed", t);
+        this.t = t;
+      }
+    }
+
+    boolean isProgressing() throws InterruptedException {
+      int currentCount = this.count;
+      while(currentCount == this.count) {
+        if (!isAlive()) return false;
+        if (this.t != null) return false;
+        Thread.sleep(10);
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "count=" + this.count + ", t=" +
+        (this.t == null? "null": this.t.toString());
+    }
+
+    abstract void metaTask() throws Throwable;
+  }
+
   @Test public void testGetRegionsCatalogTables()
   throws IOException, InterruptedException {
     List<HRegionInfo> regions =
@@ -114,19 +224,9 @@ public class TestMetaReaderEditor {
   @Test public void testGetRegion() throws IOException, InterruptedException {
     final String name = "testGetRegion";
     LOG.info("Started " + name);
-    final byte [] nameBytes = Bytes.toBytes(name);
-    HTable t = UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
-    int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
-
-    // Test it works getting a region from user table.
-    List<HRegionInfo> regions = MetaReader.getTableRegions(ct, nameBytes);
-    assertEquals(regionCount, regions.size());
-    Pair<HRegionInfo, ServerName> pair =
-      MetaReader.getRegion(ct, regions.get(0).getRegionName());
-    assertEquals(regions.get(0).getEncodedName(),
-      pair.getFirst().getEncodedName());
     // Test get on non-existent region.
-    pair = MetaReader.getRegion(ct, Bytes.toBytes("nonexistent-region"));
+    Pair<HRegionInfo, ServerName> pair =
+      MetaReader.getRegion(ct, Bytes.toBytes("nonexistent-region"));
     assertNull(pair);
     // Test it works getting a region from meta/root.
     pair =
@@ -137,7 +237,8 @@ public class TestMetaReaderEditor {
   }
 
   // Test for the optimization made in HBASE-3650
-  @Test public void testScanMetaForTable() throws IOException {
+  @Test public void testScanMetaForTable()
+  throws IOException, InterruptedException {
     final String name = "testScanMetaForTable";
     LOG.info("Started " + name);
 
@@ -165,4 +266,25 @@ public class TestMetaReaderEditor {
     }
     assertEquals(1, MetaReader.getTableRegions(ct, greaterName).size());
   }
-}
\ No newline at end of file
+
+  private static List<HRegionInfo> testGettingTableRegions(final CatalogTracker ct,
+      final byte [] nameBytes, final int regionCount)
+  throws IOException, InterruptedException {
+    List<HRegionInfo> regions = MetaReader.getTableRegions(ct, nameBytes);
+    assertEquals(regionCount, regions.size());
+    Pair<HRegionInfo, ServerName> pair =
+      MetaReader.getRegion(ct, regions.get(0).getRegionName());
+    assertEquals(regions.get(0).getEncodedName(),
+      pair.getFirst().getEncodedName());
+    return regions;
+  }
+
+  private static void testGetRegion(final CatalogTracker ct,
+      final HRegionInfo region)
+  throws IOException, InterruptedException {
+    Pair<HRegionInfo, ServerName> pair =
+      MetaReader.getRegion(ct, region.getRegionName());
+    assertEquals(region.getEncodedName(),
+      pair.getFirst().getEncodedName());
+  }
+}

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Thu Oct 13 23:18:42 2011
@@ -19,6 +19,10 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -27,6 +31,8 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -38,12 +44,6 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 
 /**
  * This class is for testing HCM features
@@ -225,4 +225,4 @@ public class TestHCM {
       Thread.sleep(50);
     }
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestMetaMigration.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestMetaMigration.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestMetaMigration.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/client/TestMetaMigration.java Thu Oct 13 23:18:42 2011
@@ -28,7 +28,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
 
-import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -77,7 +77,8 @@ public class TestMetaMigration {
   @Test
   public void testMetaUpdatedFlagInROOT() throws Exception {
     LOG.info("Starting testMetaUpdatedFlagInROOT");
-    boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
+    boolean metaUpdated =
+      MetaMigrationRemovingHTD.isMetaHRIUpdated(miniHBaseCluster.getMaster());
     assertEquals(true, metaUpdated);
     LOG.info("END testMetaUpdatedFlagInROOT");
   }
@@ -97,16 +98,17 @@ public class TestMetaMigration {
             Bytes.toBytes("region_b")});
     CatalogTracker ct = miniHBaseCluster.getMaster().getCatalogTracker();
     // just for this test set it to false.
-    MetaEditor.updateRootWithMetaMigrationStatus(ct, false);
+    MetaMigrationRemovingHTD.updateRootWithMetaMigrationStatus(ct, false);
     MetaReader.fullScanMetaAndPrint(ct);
-    LOG.info("MEta Print completed.testUpdatesOnMetaWithLegacyHRI");
+    LOG.info("Meta Print completed.testUpdatesOnMetaWithLegacyHRI");
 
-    List<HTableDescriptor> htds = MetaEditor.updateMetaWithNewRegionInfo(
+    List<HTableDescriptor> htds = MetaMigrationRemovingHTD.updateMetaWithNewRegionInfo(
           TEST_UTIL.getHBaseCluster().getMaster());
     MetaReader.fullScanMetaAndPrint(ct);
     assertEquals(3, htds.size());
     // Assert that the flag in ROOT is updated to reflect the correct status
-    boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
+    boolean metaUpdated =
+      MetaMigrationRemovingHTD.isMetaHRIUpdated(miniHBaseCluster.getMaster());
     assertEquals(true, metaUpdated);
     LOG.info("END testMetaWithLegacyHRI");
 
@@ -138,15 +140,16 @@ public class TestMetaMigration {
     TEST_UTIL.createMultiRegionsWithLegacyHRI(conf, htd, FAMILY, 10);
     CatalogTracker ct = miniHBaseCluster.getMaster().getCatalogTracker();
     // just for this test set it to false.
-    MetaEditor.updateRootWithMetaMigrationStatus(ct, false);
+    MetaMigrationRemovingHTD.updateRootWithMetaMigrationStatus(ct, false);
     //MetaReader.fullScanMetaAndPrint(ct);
     LOG.info("MEta Print completed.testUpdatesOnMetaWithLegacyHRI");
 
-    List<HTableDescriptor> htds = MetaEditor.updateMetaWithNewRegionInfo(
+    List<HTableDescriptor> htds = MetaMigrationRemovingHTD.updateMetaWithNewRegionInfo(
           TEST_UTIL.getHBaseCluster().getMaster());
     assertEquals(10, htds.size());
     // Assert that the flag in ROOT is updated to reflect the correct status
-    boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
+    boolean metaUpdated =
+      MetaMigrationRemovingHTD.isMetaHRIUpdated(miniHBaseCluster.getMaster());
     assertEquals(true, metaUpdated);
     LOG.info("END testMetaWithLegacyHRI");
 

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Thu Oct 13 23:18:42 2011
@@ -47,6 +47,9 @@ import org.apache.hadoop.hbase.ServerNam
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.io.Reference;
@@ -61,20 +64,27 @@ import org.mockito.Mockito;
 public class TestCatalogJanitor {
   /**
    * Pseudo server for below tests.
+   * Be sure to call stop on the way out else could leave some mess around.
    */
   class MockServer implements Server {
+    private final HConnection connection;
     private final Configuration c;
     private final CatalogTracker ct;
 
     MockServer(final HBaseTestingUtility htu)
-    throws NotAllMetaRegionsOnlineException, IOException {
+    throws NotAllMetaRegionsOnlineException, IOException, InterruptedException {
       this.c = htu.getConfiguration();
+      // Mock an HConnection and a HRegionInterface implementation.  Have the
+      // HConnection return the HRI.  Have the HRI return a few mocked up responses
+      // to make our test work.
+      this.connection = HConnectionTestingUtility.getMockedConnection(this.c);
       // Set hbase.rootdir into test dir.
       FileSystem fs = FileSystem.get(this.c);
       Path rootdir = fs.makeQualified(new Path(this.c.get(HConstants.HBASE_DIR)));
       this.c.set(HConstants.HBASE_DIR, rootdir.toString());
       this.ct = Mockito.mock(CatalogTracker.class);
       HRegionInterface hri = Mockito.mock(HRegionInterface.class);
+      Mockito.when(this.ct.getConnection()).thenReturn(this.connection);
       Mockito.when(ct.waitForMetaServerConnectionDefault()).thenReturn(hri);
     }
 
@@ -115,9 +125,13 @@ public class TestCatalogJanitor {
 
     @Override
     public void stop(String why) {
-      //no-op
+      if (this.ct != null) {
+        this.ct.stop();
+      }
+      if (this.connection != null) {
+        HConnectionManager.deleteConnection(this.connection.getConfiguration(), true);
+      }
     }
-    
   }
 
   /**
@@ -267,42 +281,53 @@ public class TestCatalogJanitor {
   }
 
   @Test
-  public void testCleanParent() throws IOException {
+  public void testCleanParent() throws IOException, InterruptedException {
     HBaseTestingUtility htu = new HBaseTestingUtility();
     setRootDirAndCleanIt(htu, "testCleanParent");
     Server server = new MockServer(htu);
-    MasterServices services = new MockMasterServices(server);
-    CatalogJanitor janitor = new CatalogJanitor(server, services);
-    // Create regions.
-    HTableDescriptor htd = createHTableDescriptor();
-    HRegionInfo parent =
-      new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
-          Bytes.toBytes("eee"));
-    HRegionInfo splita =
-      new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
-          Bytes.toBytes("ccc"));
-    HRegionInfo splitb =
-      new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
-          Bytes.toBytes("eee"));
-    // Test that when both daughter regions are in place, that we do not
-    // remove the parent.
-    Result r = createResult(parent, splita, splitb);
-    // Add a reference under splitA directory so we don't clear out the parent.
-    Path rootdir = services.getMasterFileSystem().getRootDir();
-    Path tabledir =
-      HTableDescriptor.getTableDir(rootdir, htd.getName());
-    Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(),
-      htd.getColumnFamilies()[0].getName());
-    Reference ref = new Reference(Bytes.toBytes("ccc"), Reference.Range.top);
-    long now = System.currentTimeMillis();
-    // Reference name has this format: StoreFile#REF_NAME_PARSER
-    Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
-    FileSystem fs = services.getMasterFileSystem().getFileSystem();
-    ref.write(fs, p);
-    assertFalse(janitor.cleanParent(parent, r));
-    // Remove the reference file and try again.
-    assertTrue(fs.delete(p, true));
-    assertTrue(janitor.cleanParent(parent, r));
+    try {
+      MasterServices services = new MockMasterServices(server);
+      CatalogJanitor janitor = new CatalogJanitor(server, services);
+      // Create regions.
+      HTableDescriptor htd = new HTableDescriptor("table");
+      htd.addFamily(new HColumnDescriptor("f"));
+      HRegionInfo parent =
+        new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+            Bytes.toBytes("eee"));
+      HRegionInfo splita =
+        new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+            Bytes.toBytes("ccc"));
+      HRegionInfo splitb =
+        new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
+            Bytes.toBytes("eee"));
+      // Test that when both daughter regions are in place, that we do not
+      // remove the parent.
+      List<KeyValue> kvs = new ArrayList<KeyValue>();
+      kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
+          HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
+      kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
+          HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
+      Result r = new Result(kvs);
+      // Add a reference under splitA directory so we don't clear out the parent.
+      Path rootdir = services.getMasterFileSystem().getRootDir();
+      Path tabledir =
+        HTableDescriptor.getTableDir(rootdir, htd.getName());
+      Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(),
+          htd.getColumnFamilies()[0].getName());
+      Reference ref = new Reference(Bytes.toBytes("ccc"), Reference.Range.top);
+      long now = System.currentTimeMillis();
+      // Reference name has this format: StoreFile#REF_NAME_PARSER
+      Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
+      FileSystem fs = services.getMasterFileSystem().getFileSystem();
+      Path path = ref.write(fs, p);
+      assertTrue(fs.exists(path));
+      assertFalse(janitor.cleanParent(parent, r));
+      // Remove the reference file and try again.
+      assertTrue(fs.delete(p, true));
+      assertTrue(janitor.cleanParent(parent, r));
+    } finally {
+      server.stop("shutdown");
+    }
   }
 
   /**
@@ -459,4 +484,4 @@ public class TestCatalogJanitor {
     htd.addFamily(new HColumnDescriptor("f"));
     return htd;
   }
-}
\ No newline at end of file
+}

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java Thu Oct 13 23:18:42 2011
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
@@ -98,6 +99,9 @@ public class TestDistributedLogSplitting
     LOG.info("Waiting for active/ready master");
     cluster.waitForActiveAndReadyMaster();
     master = cluster.getMaster();
+    while (cluster.getLiveRegionServerThreads().size() < num_rs) {
+      Threads.sleep(1);
+    }
   }
 
   @After
@@ -111,7 +115,7 @@ public class TestDistributedLogSplitting
     final int NUM_REGIONS_TO_CREATE = 40;
     final int NUM_ROWS_PER_REGION = 100;
 
-    startCluster(NUM_RS);
+    startCluster(NUM_RS); // NUM_RS=6.
 
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
         "distributed log splitting test", null);

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java Thu Oct 13 23:18:42 2011
@@ -86,21 +86,20 @@ public class TestMaster {
         tableRegions.get(0).getFirst().getEndKey());
 
     // Now trigger a split and stop when the split is in progress
-
-    CountDownLatch aboutToOpen = new CountDownLatch(1);
+    CountDownLatch split = new CountDownLatch(1);
     CountDownLatch proceed = new CountDownLatch(1);
-    RegionOpenListener list = new RegionOpenListener(aboutToOpen, proceed);
+    RegionSplitListener list = new RegionSplitListener(split, proceed);
     cluster.getMaster().executorService.
-      registerListener(EventType.RS_ZK_REGION_OPENED, list);
+      registerListener(EventType.RS_ZK_REGION_SPLIT, list);
 
     LOG.info("Splitting table");
     admin.split(TABLENAME);
     LOG.info("Waiting for split result to be about to open");
-    aboutToOpen.await(60, TimeUnit.SECONDS);
+    split.await(60, TimeUnit.SECONDS);
     try {
       LOG.info("Making sure we can call getTableRegions while opening");
-      tableRegions = MetaReader.getTableRegionsAndLocations(
-          m.getCatalogTracker(), Bytes.toString(TABLENAME));
+      tableRegions = MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
+        TABLENAME, false);
 
       LOG.info("Regions: " + Joiner.on(',').join(tableRegions));
       // We have three regions because one is split-in-progress
@@ -118,22 +117,21 @@ public class TestMaster {
     }
   }
 
-  static class RegionOpenListener implements EventHandlerListener {
-    CountDownLatch aboutToOpen, proceed;
+  static class RegionSplitListener implements EventHandlerListener {
+    CountDownLatch split, proceed;
 
-    public RegionOpenListener(CountDownLatch aboutToOpen, CountDownLatch proceed)
-    {
-      this.aboutToOpen = aboutToOpen;
+    public RegionSplitListener(CountDownLatch split, CountDownLatch proceed) {
+      this.split = split;
       this.proceed = proceed;
     }
 
     @Override
     public void afterProcess(EventHandler event) {
-      if (event.getEventType() != EventType.RS_ZK_REGION_OPENED) {
+      if (event.getEventType() != EventType.RS_ZK_REGION_SPLIT) {
         return;
       }
       try {
-        aboutToOpen.countDown();
+        split.countDown();
         proceed.await(60, TimeUnit.SECONDS);
       } catch (InterruptedException ie) {
         throw new RuntimeException(ie);

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java Thu Oct 13 23:18:42 2011
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -34,11 +33,18 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.executor.RegionTransitionData;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
+import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
-import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -49,8 +55,6 @@ import org.apache.hadoop.hbase.util.JVMC
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKTable;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.junit.Test;
 
 public class TestMasterFailover {
@@ -131,7 +135,7 @@ public class TestMasterFailover {
     // Stop the cluster
     TEST_UTIL.shutdownMiniCluster();
   }
-  
+
   @Test
   public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
       throws Exception {
@@ -1006,4 +1010,4 @@ public class TestMasterFailover {
   private void log(String string) {
     LOG.info("\n\n" + string + " \n\n");
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java Thu Oct 13 23:18:42 2011
@@ -128,8 +128,10 @@ public class TestMergeTable {
       assertTrue("originalTableRegions=" + originalTableRegions.size() +
         ", postMergeTableRegions=" + postMergeTableRegions.size(),
         postMergeTableRegions.size() < originalTableRegions.size());
+      LOG.info("Done with merge");
     } finally {
       UTIL.shutdownMiniCluster();
+      LOG.info("After cluster shutdown");
     }
   }
 

Modified: hbase/branches/0.92/src/test/ruby/hbase/admin_test.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/ruby/hbase/admin_test.rb?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/ruby/hbase/admin_test.rb (original)
+++ hbase/branches/0.92/src/test/ruby/hbase/admin_test.rb Thu Oct 13 23:18:42 2011
@@ -152,7 +152,7 @@ module Hbase
       assert_equal(['a:', 'b:'], table(@create_test_name).get_all_columns.sort)
      end
 
-    define_test "create hould work with hash column args" do
+    define_test "create should work with hash column args" do
       drop_test_table(@create_test_name)
       admin.create(@create_test_name, { NAME => 'a'}, { NAME => 'b'})
       assert_equal(['a:', 'b:'], table(@create_test_name).get_all_columns.sort)
@@ -160,14 +160,14 @@ module Hbase
 
     #-------------------------------------------------------------------------------
 
-#    define_test "close should work without region server name" do
-#      if admin.exists?(@create_test_name)
-#        admin.disable(@create_test_name)
-#        admin.drop(@create_test_name)
-#      end
-#      admin.create(@create_test_name, 'foo')
-#      admin.close_region(@create_test_name + ',,0')
-#    end
+    define_test "close should work without region server name" do
+      if admin.exists?(@create_test_name)
+        admin.disable(@create_test_name)
+        admin.drop(@create_test_name)
+      end
+      admin.create(@create_test_name, 'foo')
+      admin.close_region(@create_test_name + ',,0', nil)
+    end
 
     #-------------------------------------------------------------------------------
 
@@ -187,13 +187,14 @@ module Hbase
       table(@test_name).put(1, "x:a", 1)
       table(@test_name).put(2, "x:a", 2)
       assert_equal(2, table(@test_name).count)
-      admin.truncate(@test_name)
+      # This is hacky.  Need to get the configuration into admin instance
+      admin.truncate(@test_name, $TEST_CLUSTER.getConfiguration)
       assert_equal(0, table(@test_name).count)
     end
 
     define_test "truncate should yield log records" do
       logs = []
-      admin.truncate(@test_name) do |log|
+      admin.truncate(@test_name, $TEST_CLUSTER.getConfiguration) do |log|
         assert_kind_of(String, log)
         logs << log
       end

Modified: hbase/branches/0.92/src/test/ruby/shell/shell_test.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/ruby/shell/shell_test.rb?rev=1183163&r1=1183162&r2=1183163&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/ruby/shell/shell_test.rb (original)
+++ hbase/branches/0.92/src/test/ruby/shell/shell_test.rb Thu Oct 13 23:18:42 2011
@@ -25,7 +25,7 @@ require 'shell/formatter'
 class ShellTest < Test::Unit::TestCase
   def setup
     @formatter = ::Shell::Formatter::Console.new()
-    @hbase = ::Hbase::Hbase.new
+    @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
     @shell = Shell::Shell.new(@hbase, @formatter)
   end