You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2007/08/08 22:30:20 UTC

svn commit: r564012 [1/4] - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/util/ src/test/org/apache/hadoop/hbase/

Author: stack
Date: Wed Aug  8 13:30:13 2007
New Revision: 564012

URL: http://svn.apache.org/viewvc?view=rev&rev=564012
Log:
HADOOP-1662 Make region splits faster
Splits are now near-instantaneous.  On split, daughter splits create
'references' to store files up in the parent region using new 'HalfMapFile'
class to proxy accesses against the top-half or bottom-half of 
backing MapFile.  Parent region is deleted after all references in daughter
regions have been let go.

Below includes other cleanups and at least one bug fix for fails adding
>32k records and improvements to make it more likely TestRegionServerAbort
will complete..

A src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
    Added. Tests new Reference HStoreFiles. Test new HalfMapFileReader inner
    class of HStoreFile. Test that we do the right thing when HStoreFiles
    are smaller than a MapFile index range (i.e. there is not 'MidKey').
    Test we do right thing when key is outside of a HalfMapFile.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
    getHRegionDir moved from HStoreFile to HRegion.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBatchUpdate.java
    Let out exception rather than catch and call 'fail'.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java
    Refactored so can start and stop a minihbasecluster w/o having to
    subclass this TestCase. Refactored methods in this class to use the
    newly added methods listed below.
    (MasterThread, RegionServerThread, startMaster, startRegionServers
      shutdown): Added.
    Added logging of abort, close and wait.  Also on abort/close
    was doing a remove that made it so subsequent wait had nothing to
    wait on.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
    Added tests that assert all works properly at region level on
    multiple levels of splits and then do same on a cluster.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
    Removed catch and 'fail()'.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
    Javadoc to explain how split now works. Have constructors flow
    into each other rather than replicate setup per instance. Moved
    in here operations such as delete, rename, and length of store files
    (No need of clients to remember to delete map and info files).
    (REF_NAME_PARSER, Reference, HalfMapFile, isReference,
      writeReferenceFiles, writeSplitInfo, readSplitInfo,
      createOrFail, getReader, getWriter, toString): Added.
    (getMapDir, getMapFilePath, getInfoDir, getInfoFilePath): Added
    a bunch of overrides for reference handling.
    (loadHStoreFiles): Amended to load references off disk.
    (splitStoreFiles): Redone to instead write references into
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
    Rename maps as readers and mapFiles as storefiles.
    Moved BloomFilterReader and Writer into HStoreFile. Removed
    getMapFileReader and getMapFileWriter (They are in HStoreFile now).
    (getReaders): Added.
    (HStoreSize): Added.  Data Structure to hold aggregated size
    of all HStoreFiles in HStore, the largest, its midkey, and
    if the HStore is splitable (May not be if references).
    Previous we only did largest file; less accurate.
    (getLargestFileSize): Renamed size and redone to aggregate
    sizes, etc.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    Have constructors waterfall down through each other rather than
    repeat initializations.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
    Use new HStoreSize structure.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
    Added delayed remove of HRegion (Now done in HMaster as part of
    meta scan). Change LOG.error and LOG.warn so they throw stack trace
    instead of just the Exception.toString as message.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
    (COLUMN_FAMILY_STR): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
    Added why to log of splitting.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
    Short is not big enough to hold edits tha could contain a sizable
    web page.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
    (getTableName): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
    Added constructor to BaseScanner that takes name of table we're
    scanning (ROOT or META usually). Added to scanOneRegion handling
    of split regions.  Collect splits to check while scanning and
    then outside of the scanning, so we can modify the META table
    is needed, do the checks of daughter regions and update on
    change of state.  Made LOG.warn and LOG.error print stack trace.
    (isSplitParent, cleanupSplits, hasReferences): Added. 
    Added toString to each of the PendingOperation implementations.
    In the ShutdownPendingOperation scan of meta data, removed
    check of startcode (if the server name is that of the dead
    server, it needs reassigning even if start code is good).
    Also, if server name is null -- possible if we are missing
    edits off end of log -- then the region should be reassigned
    just in case its from the dead server.  Also, if reassigning,
    clear from pendingRegions.  Server may have died after sending
    region is up but before the server confirms receipt in the
    meta scan. Added mare detail to each log.  In OpenPendingOperation
    we were trying to clear pendingRegion in wrong place -- it was
    never executed (regions were always pending). 
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
    Add split boolean.  Output offline and split status in toString.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
    Comments.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
    Moved getRegionDir here from HStoreFile.
    (COL_SPLITA, COL_SPLITB): Added.
    (closeAndSplit): Refactored to use new fast split method.
       StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
    (splitStoreFile): Moved into HStoreFile.
    (getSplitRegionDir, getSplitsDir, toString): Added.
    (needsSplit): Refactored to exploit new HStoreSize structure.
    Also manages notion of 'unsplitable' region.
    (largestHStore): Refactored.
    (removeSplitFromMETA, writeSplitToMETA, getSplit, hasReference): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Keying.java
    (intToBytes, getBytes): Added.
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Writables.java    
    Utility reading and writing Writables.

Added:
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Writables.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HBaseAdmin.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionLocation.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Keying.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/StaticTestEnvironment.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBatchUpdate.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestRegionServerAbort.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java

Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Wed Aug  8 13:30:13 2007
@@ -85,3 +85,4 @@
      side objects (HTable/HBaseAdmin/HConnection) instead of HClient.
  53. HADOOP-1528 HClient for multiple tables - expose close table function
  54. HADOOP-1466 Clean up warnings, visibility and javadoc issues in HBase.
+ 55. HADOOP-1662 Make region splits faster

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HBaseAdmin.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HBaseAdmin.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HBaseAdmin.java Wed Aug  8 13:30:13 2007
@@ -498,11 +498,8 @@
   private HRegionLocation getFirstMetaServerForTable(Text tableName)
   throws IOException {
     SortedMap<Text, HRegionLocation> metaservers =
-      connection.getTableServers(META_TABLE_NAME);
-    
+      connection.getTableServers(META_TABLE_NAME); 
     return metaservers.get((metaservers.containsKey(tableName)) ?
         tableName : metaservers.headMap(tableName).lastKey());
   }
-  
-
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java Wed Aug  8 13:30:13 2007
@@ -76,14 +76,7 @@
    * Default constructor. Must be present for Writable.
    */
   public HColumnDescriptor() {
-    this.name = new Text();
-    this.maxVersions = DEFAULT_N_VERSIONS;
-    this.compressionType = COMPRESSION_NONE;
-    this.inMemory = false;
-    this.maxValueLength = Integer.MAX_VALUE;
-    this.bloomFilterSpecified = false;
-    this.bloomFilter = null;
-    this.versionNumber = COLUMN_DESCRIPTOR_VERSION;
+    this(null);
   }
   
   /**
@@ -93,8 +86,10 @@
    * @param columnName - column family name
    */
   public HColumnDescriptor(String columnName) {
-    this();
-    this.name.set(columnName);
+    this(columnName == null || columnName.length() <= 0?
+      new Text(): new Text(columnName),
+      DEFAULT_N_VERSIONS, CompressionType.NONE, false,
+      Integer.MAX_VALUE, null);
   }
   
   /**
@@ -112,13 +107,19 @@
    * end in a <code>:</code>
    * @throws IllegalArgumentException if the number of versions is &lt;= 0
    */
-  public HColumnDescriptor(Text name, int maxVersions, CompressionType compression,
-      boolean inMemory, int maxValueLength, BloomFilterDescriptor bloomFilter) {
+  public HColumnDescriptor(final Text name, final int maxVersions,
+      final CompressionType compression, final boolean inMemory,
+      final int maxValueLength, final BloomFilterDescriptor bloomFilter) {
     String familyStr = name.toString();
-    Matcher m = LEGAL_FAMILY_NAME.matcher(familyStr);
-    if(m == null || !m.matches()) {
-      throw new IllegalArgumentException(
-          "Family names can only contain 'word characters' and must end with a ':'");
+    // Test name if not null (It can be null when deserializing after
+    // construction but before we've read in the fields);
+    if (familyStr.length() > 0) {
+      Matcher m = LEGAL_FAMILY_NAME.matcher(familyStr);
+      if(m == null || !m.matches()) {
+        throw new IllegalArgumentException("Illegal family name <" + name +
+          ">. Family names can only contain " +
+          "'word characters' and must end with a ':'");
+      }
     }
     this.name = name;
 

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java Wed Aug  8 13:30:13 2007
@@ -270,8 +270,8 @@
 
     /** {@inheritDoc} */
     public SortedMap<Text, HRegionLocation>
-    getTableServers(Text tableName) throws IOException {
-      
+      getTableServers(Text tableName)
+    throws IOException {  
       if (tableName == null || tableName.getLength() == 0) {
         throw new IllegalArgumentException(
             "table name cannot be null or zero length");
@@ -468,6 +468,7 @@
             try {
               this.tablesBeingLocated.wait(threadWakeFrequency);
             } catch (InterruptedException e) {
+              // continue
             }
           }
           if (!waited) {

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java Wed Aug  8 13:30:13 2007
@@ -100,7 +100,8 @@
   // Defines for the column names used in both ROOT and META HBase 'meta' tables.
   
   /** The ROOT and META column family */
-  static final Text COLUMN_FAMILY = new Text("info:");
+  static final String COLUMN_FAMILY_STR = "info:";
+  static final Text COLUMN_FAMILY = new Text(COLUMN_FAMILY_STR);
 
   /** Array of meta column names */
   static final Text [] COLUMN_FAMILY_ARRAY = new Text [] {COLUMN_FAMILY};

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java Wed Aug  8 13:30:13 2007
@@ -97,13 +97,15 @@
   static void splitLog(Path rootDir, Path srcDir, FileSystem fs,
     Configuration conf) throws IOException {
     Path logfiles[] = fs.listPaths(srcDir);
-    LOG.info("splitting " + logfiles.length + " log files in " +
-        srcDir.toString());
-
+    LOG.info("splitting " + logfiles.length + " log(s) in " +
+      srcDir.toString());
     HashMap<Text, SequenceFile.Writer> logWriters =
       new HashMap<Text, SequenceFile.Writer>();
     try {
       for(int i = 0; i < logfiles.length; i++) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Splitting " + logfiles[i]);
+        }
         SequenceFile.Reader in =
           new SequenceFile.Reader(fs, logfiles[i], conf);
         try {
@@ -113,7 +115,7 @@
             Text regionName = key.getRegionName();
             SequenceFile.Writer w = logWriters.get(regionName);
             if (w == null) {
-              Path logfile = new Path(HStoreFile.getHRegionDir(rootDir,
+              Path logfile = new Path(HRegion.getRegionDir(rootDir,
                   regionName), HREGION_OLDLOGFILE_NAME);
               if (LOG.isDebugEnabled()) {
                 LOG.debug("getting new log file writer for path " + logfile);
@@ -121,6 +123,9 @@
               w = SequenceFile.createWriter(fs, conf, logfile, HLogKey.class,
                   HLogEdit.class);
               logWriters.put(regionName, w);
+            }
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Edit " + key.toString());
             }
             w.append(key, val);
           }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java Wed Aug  8 13:30:13 2007
@@ -81,7 +81,7 @@
   /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
     this.column.write(out);
-    out.writeShort(this.val.length);
+    out.writeInt(this.val.length);
     out.write(this.val);
     out.writeLong(timestamp);
   }
@@ -89,7 +89,7 @@
   /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
     this.column.readFields(in);
-    this.val = new byte[in.readShort()];
+    this.val = new byte[in.readInt()];
     in.readFully(this.val);
     this.timestamp = in.readLong();
   }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java Wed Aug  8 13:30:13 2007
@@ -25,6 +25,7 @@
 import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Random;
@@ -46,12 +47,13 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.KeyedData;
+import org.apache.hadoop.hbase.util.Keying;
+import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.util.StringUtils;
 
 
 /**
@@ -76,8 +78,7 @@
     }
   }
 
-  static final Log LOG =
-    LogFactory.getLog(org.apache.hadoop.hbase.HMaster.class.getName());
+  static final Log LOG = LogFactory.getLog(HMaster.class.getName());
   
   volatile boolean closed;
   Path dir;
@@ -157,10 +158,17 @@
   abstract class BaseScanner implements Runnable {
     private final Text FIRST_ROW = new Text();
     protected boolean rootRegion;
+    protected final Text tableName;
     
     protected abstract void initialScan();
     protected abstract void maintenanceScan();
     
+    BaseScanner(final Text tableName) {
+      super();
+      this.tableName = tableName;
+      this.rootRegion = tableName.equals(ROOT_TABLE_NAME);
+    }
+    
     /**
      * {@inheritDoc}
      */
@@ -185,13 +193,15 @@
       HRegionInterface regionServer = null;
       long scannerId = -1L;
       LOG.info(Thread.currentThread().getName() + " scanning meta region " +
-          region.regionName);
-
+          region.regionName + " on " + region.server.toString());
+      // Array to hold list of split parents found.  Scan adds to list.  After
+      // scan we go check if parents can be removed.
+      Map<HRegionInfo, TreeMap<Text, byte[]>> splitParents =
+        new HashMap<HRegionInfo, TreeMap<Text, byte[]>>();
       try {
         regionServer = connection.getHRegionConnection(region.server);
         scannerId = regionServer.openScanner(region.regionName, METACOLUMNS,
           FIRST_ROW, System.currentTimeMillis(), null);
-
         int numberOfRegionsFound = 0;
         while (true) {
           TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
@@ -199,15 +209,12 @@
           if (values.length == 0) {
             break;
           }
-
           for (int i = 0; i < values.length; i++) {
             results.put(values[i].getKey().getColumn(), values[i].getData());
           }
-
           HRegionInfo info = HRegion.getRegionInfo(results);
           String serverName = HRegion.getServerName(results);
           long startCode = HRegion.getStartCode(results);
-
           if(LOG.isDebugEnabled()) {
             LOG.debug(Thread.currentThread().getName() + " scanner: " +
                 Long.valueOf(scannerId) + " regioninfo: {" + info.toString() +
@@ -216,17 +223,17 @@
 
           // Note Region has been assigned.
           checkAssigned(info, serverName, startCode);
-          
+          if (isSplitParent(info)) {
+            splitParents.put(info, results);
+          }
           numberOfRegionsFound += 1;
         }
         if(rootRegion) {
           numberOfMetaRegions.set(numberOfRegionsFound);
         }
-
       } catch (IOException e) {
         if (e instanceof RemoteException) {
           e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
-          
           if (e instanceof UnknownScannerException) {
             // Reset scannerId so we do not try closing a scanner the other side
             // has lost account of: prevents duplicated stack trace out of the 
@@ -235,7 +242,6 @@
           }
         }
         throw e;
-
       } finally {
         try {
           if (scannerId != -1L && regionServer != null) {
@@ -245,13 +251,100 @@
           if (e instanceof RemoteException) {
             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
           }
-          LOG.error(e);
+          LOG.error("Closing scanner", e);
+        }
+      }
+      // Scan is finished.  Take a look at split parents to see if any
+      // we can clean up.
+      if (splitParents.size() > 0) {
+        for (Map.Entry<HRegionInfo, TreeMap<Text, byte[]>> e:
+            splitParents.entrySet()) {
+          TreeMap<Text, byte[]> results = e.getValue();
+          cleanupSplits(e.getKey(),
+            HRegion.getSplit(results, HRegion.COL_SPLITA),
+            HRegion.getSplit(results, HRegion.COL_SPLITB));
         }
       }
       LOG.info(Thread.currentThread().getName() + " scan of meta region " +
           region.regionName + " complete");
     }
     
+    private boolean isSplitParent(final HRegionInfo info) {
+      boolean result = false;
+      // Skip if not a split region.
+      if (!info.isSplit()) {
+        return result;
+      }
+      if (!info.isOffline()) {
+        LOG.warn("Region is split but not offline: " + info.regionName);
+      }
+      return true;
+    }
+    
+    /* 
+     * @param info
+     * @param splitA
+     * @param splitB
+     * @return True if we removed <code>info</code> and this region has
+     * been cleaned up.
+     * @throws IOException
+     */
+    private boolean cleanupSplits(final HRegionInfo info,
+      final HRegionInfo splitA, final HRegionInfo splitB)
+    throws IOException {
+      boolean result = false;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Checking " + info.getRegionName() + " to see if daughter " +
+          "splits still hold references");
+      }
+      boolean noReferencesA = splitA == null;
+      boolean noReferencesB = splitB == null;
+      if (!noReferencesA) {
+        noReferencesA = hasReferences(info.getRegionName(), splitA,
+          HRegion.COL_SPLITA);
+      }
+      if (!noReferencesB) {
+        noReferencesB = hasReferences(info.getRegionName(), splitB,
+          HRegion.COL_SPLITB);
+      }
+      if (!(noReferencesA && noReferencesB)) {
+        // No references.  Remove this item from table and deleted region on
+        // disk.
+        LOG.info("Deleting region " + info.getRegionName() +
+          " because daughter splits no longer hold references");
+        HRegion.deleteRegion(fs, dir, info.getRegionName());
+        HRegion.removeRegionFromMETA(conf, this.tableName,
+          info.getRegionName());
+        result = true;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Done checking " + info.getRegionName() + ": splitA: " +
+          noReferencesA + ", splitB: "+ noReferencesB);
+      }
+      return result;
+    }
+    
+    protected boolean hasReferences(final Text regionName,
+        final HRegionInfo split, final Text column)
+    throws IOException {
+      boolean result =
+        HRegion.hasReferences(fs, fs.makeQualified(dir), split);
+      if (result) {
+        return result;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(split.getRegionName().toString()
+          +" no longer has references to " + regionName.toString());
+      }
+      HTable t = new HTable(conf, this.tableName);
+      try {
+        HRegion.removeSplitFromMETA(t, regionName, column);
+      } finally {
+        t.close();
+      }
+      return result;
+    }
+    
     protected void checkAssigned(final HRegionInfo info,
         final String serverName, final long startCode) {
       // Skip region - if ...
@@ -261,9 +354,10 @@
 
         unassignedRegions.remove(info.regionName);
         assignAttempts.remove(info.regionName);
-        
         if(LOG.isDebugEnabled()) {
-          LOG.debug("not assigning region: " + info.regionName);
+          LOG.debug("not assigning region: " + info.regionName +
+            " (offline: " + info.isOffline() + ", split: " + info.isSplit() +
+            ")");
         }
         return;
       }
@@ -273,33 +367,24 @@
         TreeMap<Text, HRegionInfo> regionsToKill = killList.get(serverName);
         if(regionsToKill != null && regionsToKill.containsKey(info.regionName)) {
           // Skip if region is on kill list
-          
           if(LOG.isDebugEnabled()) {
-            LOG.debug("not assigning region (on kill list): " + info.regionName);
+            LOG.debug("not assigning region (on kill list): " +
+              info.regionName);
           }
           return;
         }
         storedInfo = serversToServerInfo.get(serverName);
       }
-      if( !(
-          unassignedRegions.containsKey(info.regionName)
-          || pendingRegions.contains(info.regionName)
-          )
-          && (storedInfo == null
-              || storedInfo.getStartCode() != startCode)) {
-                  
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("region unassigned: " + info.regionName
-              + " serverName: " + serverName
-              + (storedInfo == null ? " storedInfo == null"
-                  : (" startCode=" + startCode + ", storedStartCode="
-                      + storedInfo.getStartCode())));
-        }
-
+      if( !(unassignedRegions.containsKey(info.regionName) ||
+            pendingRegions.contains(info.regionName))
+          && (storedInfo == null || storedInfo.getStartCode() != startCode)) {
         // The current assignment is no good; load the region.
-        
         unassignedRegions.put(info.regionName, info);
         assignAttempts.put(info.regionName, Long.valueOf(0L));
+      } else if (LOG.isDebugEnabled()) {
+          LOG.debug("Finished if " + info.getRegionName() + " is assigned: " +
+            "unassigned: " + unassignedRegions.containsKey(info.regionName) +
+            ", pending: " + pendingRegions.contains(info.regionName));
       }
     }
   }
@@ -312,7 +397,7 @@
   class RootScanner extends BaseScanner {
     /** Constructor */
     public RootScanner() {
-      rootRegion = true;
+      super(HConstants.ROOT_TABLE_NAME);
     }
 
     private void scanRoot() {
@@ -325,6 +410,7 @@
           try {
             Thread.sleep(threadWakeFrequency);
           } catch (InterruptedException e) {
+            // continue
           }
         }
         if (closed) {
@@ -337,28 +423,26 @@
                 HGlobals.rootRegionInfo.regionName, null));
           }
           break;
-
         } catch (IOException e) {
           if (e instanceof RemoteException) {
             try {
               e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
-
             } catch (IOException ex) {
               e = ex;
             }
           }
           tries += 1;
           if (tries == 1) {
-            LOG.warn(e);
-
+            LOG.warn("Scan ROOT region", e);
           } else {
-            LOG.error(e);
+            LOG.error("Scan ROOT region", e);
           }
         }
         if (!closed) {
           try {
             Thread.sleep(threadWakeFrequency);
           } catch (InterruptedException e) {
+            // continue
           }
         }
       }      
@@ -451,7 +535,7 @@
   class MetaScanner extends BaseScanner {
     /** Constructor */
     public MetaScanner() {
-      rootRegion = false;
+      super(HConstants.META_TABLE_NAME);
     }
     
     private void scanOneMetaRegion(MetaRegion region) {
@@ -461,6 +545,7 @@
           try {
             Thread.sleep(threadWakeFrequency);
           } catch (InterruptedException e) {
+            // continue
           }
         }
         if (closed) {
@@ -480,23 +565,22 @@
             try {
               e = RemoteExceptionHandler.decodeRemoteException(
                   (RemoteException) e);
-
             } catch (IOException ex) {
               e = ex;
             }
           }
           tries += 1;
           if (tries == 1) {
-            LOG.warn(e);
-
+            LOG.warn("Scan one META region", e);
           } else {
-            LOG.error(e);
+            LOG.error("Scan one META region", e);
           }
         }
         if (!closed) {
           try {
             Thread.sleep(threadWakeFrequency);                  
           } catch (InterruptedException e) {
+            //continue
           }
         }
       }
@@ -558,6 +642,7 @@
         try {
           wait(threadWakeFrequency);
         } catch (InterruptedException e) {
+          // continue
         }
       }
       return closed;
@@ -643,7 +728,7 @@
     }
 
     Path rootRegionDir =
-      HStoreFile.getHRegionDir(dir, HGlobals.rootRegionInfo.regionName);
+      HRegion.getRegionDir(dir, HGlobals.rootRegionInfo.regionName);
     LOG.info("Root region dir: " + rootRegionDir.toString());
     if(! fs.exists(rootRegionDir)) {
       LOG.info("bootstrap: creating ROOT and first META regions");
@@ -662,7 +747,7 @@
         if (e instanceof RemoteException) {
           e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
         }
-        LOG.error(e);
+        LOG.error("", e);
       }
     }
 
@@ -757,14 +842,13 @@
       if (e instanceof RemoteException) {
         try {
           e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
-          
         } catch (IOException ex) {
-          LOG.warn(ex);
+          LOG.warn("", ex);
         }
       }
       // Something happened during startup. Shut things down.
       this.closed = true;
-      LOG.error(e);
+      LOG.error("Failed startup", e);
     }
 
     // Main processing loop
@@ -779,7 +863,7 @@
       }
       try {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Processing " + op.toString());
+          LOG.debug("Main processing loop: " + op.toString());
         }
         if (!op.process()) {
           // Operation would have blocked because not all meta regions are
@@ -787,33 +871,34 @@
           // for the missing meta region(s) to come back online, but since it
           // is waiting, it cannot process the meta region online operation it
           // is waiting for. So put this operation back on the queue for now.
-          
           if (msgQueue.size() == 0) {
             // The queue is currently empty so wait for a while to see if what
             // we need comes in first
-            
             try {
               Thread.sleep(threadWakeFrequency);
             } catch (InterruptedException e) {
+              // continue
             }
           }
           try {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Put " + op.toString() + " back on queue");
+            }
             msgQueue.put(op);
           } catch (InterruptedException e) {
-            throw new RuntimeException("Putting into msgQueue was interrupted.", e);
+            throw new RuntimeException("Putting into msgQueue was " +
+              "interrupted.", e);
           }
         }
-        
       } catch (Exception ex) {
         if (ex instanceof RemoteException) {
           try {
             ex = RemoteExceptionHandler.decodeRemoteException((RemoteException) ex);
-            
           } catch (IOException e) {
-            LOG.warn(e);
+            LOG.warn("", e);
           }
         }
-        LOG.warn(ex);
+        LOG.warn("Processing pending operations: " + op.toString(), ex);
         try {
           msgQueue.put(op);
         } catch (InterruptedException e) {
@@ -846,7 +931,7 @@
     } catch (Exception iex) {
       // Print if ever there is an interrupt (Just for kicks. Remove if it
       // ever happens).
-      LOG.warn(iex);
+      LOG.warn("root scanner", iex);
     }
     try {
       // Join the thread till it finishes.
@@ -854,7 +939,7 @@
     } catch(Exception iex) {
       // Print if ever there is an interrupt (Just for kicks. Remove if it
       // ever happens).
-      LOG.warn(iex);
+      LOG.warn("meta scanner", iex);
     }
     try {
       // Join until its finished.  TODO: Maybe do in parallel in its own thread
@@ -863,7 +948,7 @@
     } catch(InterruptedException iex) {
       // Print if ever there is an interrupt (Just for kicks. Remove if it
       // ever happens).
-      LOG.warn(iex);
+      LOG.warn("server", iex);
     }
     
     LOG.info("HMaster main thread exiting");
@@ -941,11 +1026,9 @@
     if (msgs.length > 0 && msgs[0].getMsg() == HMsg.MSG_REPORT_EXITING) {
       
       // HRegionServer is shutting down. Cancel the server's lease.
-      
       if (cancelLease(s, serverLabel)) {
         // Only process the exit message if the server still has a lease.
         // Otherwise we could end up processing the server exit twice.
-
         LOG.info("Region server " + s + ": MSG_REPORT_EXITING");
       
         // Get all the regions the server was serving reassigned
@@ -1073,25 +1156,19 @@
               region.regionName);
 
           // Remove from unassigned list so we don't assign it to someone else
-
           unassignedRegions.remove(region.regionName);
           assignAttempts.remove(region.regionName);
-
           if(region.regionName.compareTo(HGlobals.rootRegionInfo.regionName) == 0) {
-
             // Store the Root Region location (in memory)
-
             rootRegionLocation = new HServerAddress(info.getServerAddress());
             break;
           }
 
           // Note that the table has been assigned and is waiting for the meta
           // table to be updated.
-          
           pendingRegions.add(region.regionName);
           
           // Queue up an update to note the region location.
-
           try {
             msgQueue.put(new PendingOpenReport(info, region));
           } catch (InterruptedException e) {
@@ -1138,28 +1215,22 @@
         break;
 
       case HMsg.MSG_REPORT_SPLIT:
-        // A region has split and the old server is serving the two new regions.
-
+        // A region has split.
         HRegionInfo newRegionA = incomingMsgs[++i].getRegionInfo();
         HRegionInfo newRegionB = incomingMsgs[++i].getRegionInfo();
-        
         LOG.info("region " + region.regionName + " split. New regions are: "
               + newRegionA.regionName + ", " + newRegionB.regionName);
-        
         if(region.tableDesc.getName().equals(META_TABLE_NAME)) {
           // A meta region has split.
-
           onlineMetaRegions.remove(region.getStartKey());
           onlineMetaRegions.put(newRegionA.getStartKey(),
-              new MetaRegion(info.getServerAddress(), newRegionA.getRegionName(),
-                  newRegionA.getStartKey()));
+            new MetaRegion(info.getServerAddress(), newRegionA.getRegionName(),
+            newRegionA.getStartKey()));
           onlineMetaRegions.put(newRegionB.getStartKey(),
-              new MetaRegion(info.getServerAddress(), newRegionB.getRegionName(),
-                  newRegionB.getStartKey()));
-          
+            new MetaRegion(info.getServerAddress(), newRegionB.getRegionName(),
+            newRegionB.getStartKey()));
           numberOfMetaRegions.incrementAndGet();
         }
-        
         break;
 
       default:
@@ -1170,7 +1241,6 @@
     }
 
     // Process the kill list
-    
     if(regionsToKill != null) {
       for(HRegionInfo i: regionsToKill.values()) {
         returnMsgs.add(new HMsg(HMsg.MSG_REGION_CLOSE, i));
@@ -1217,7 +1287,9 @@
     };
     protected final Text startRow = new Text();
 
-    PendingOperation() {}
+    PendingOperation() {
+      super();
+    }
     
     abstract boolean process() throws IOException;
   }
@@ -1231,9 +1303,9 @@
     private HServerAddress deadServer;
     private String deadServerName;
     private long oldStartCode;
-    private boolean logSplit;
-    private boolean rootChecked;
-    private boolean rootRescanned;
+    private transient boolean logSplit;
+    private transient boolean rootChecked;
+    private transient boolean rootRescanned;
     
     private class ToDoEntry {
       boolean deleteRegion;
@@ -1259,27 +1331,28 @@
       this.rootRescanned = false;
     }
     
+    @Override
+    public String toString() {
+      return "PendingServerShutdown of " + this.deadServer.toString();
+    }
+    
     /** Finds regions that the dead region server was serving */
     private void scanMetaRegion(HRegionInterface server, long scannerId,
-        Text regionName) throws IOException {
-
+        Text regionName)
+    throws IOException {
       ArrayList<ToDoEntry> toDoList = new ArrayList<ToDoEntry>();
       TreeMap<Text, HRegionInfo> regions = new TreeMap<Text, HRegionInfo>();
-
       DataInputBuffer inbuf = new DataInputBuffer();
       try {
         while(true) {
           KeyedData[] values = null;
-          
           try {
             values = server.next(scannerId);
-            
           } catch (IOException e) {
             if (e instanceof RemoteException) {
               e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
-              
             }
-            LOG.error(e);
+            LOG.error("Shutdown scanning of meta region", e);
             break;
           }
           
@@ -1292,7 +1365,6 @@
           for(int i = 0; i < values.length; i++) {
             if(row == null) {
               row = values[i].getKey().getRow();
-              
             } else {
               if(!row.equals(values[i].getKey().getRow())) {
                 LOG.error("Multiple rows in same scanner result set. firstRow="
@@ -1301,72 +1373,48 @@
             }
             results.put(values[i].getKey().getColumn(), values[i].getData());
           }
-          
-          byte [] bytes = results.get(COL_SERVER); 
-          String serverName = null;
-          if(bytes == null || bytes.length == 0) {
-            // No server
-            continue;
-          }
-          try {
-            serverName = new String(bytes, UTF8_ENCODING);
-            
-          } catch(UnsupportedEncodingException e) {
-            LOG.error(e);
-            break;
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("shutdown scanner looking at " + row.toString());
           }
 
-          if(deadServerName.compareTo(serverName) != 0) {
-            // This isn't the server you're looking for - move along
-            continue;
-          }
-
-          bytes = results.get(COL_STARTCODE);
-          if(bytes == null || bytes.length == 0) {
-            // No start code
-            continue;
-          }
-          long startCode = -1L;
+          // Check server name.  If null, be conservative and treat as though
+          // region had been on shutdown server (could be null because we
+          // missed edits in hlog because hdfs does not do write-append).
+          String serverName = null;
           try {
-            startCode =
-              Long.valueOf(new String(bytes, UTF8_ENCODING)).longValue();
+            serverName = Keying.bytesToString(results.get(COL_SERVER));
           } catch(UnsupportedEncodingException e) {
-            LOG.error(e);
+            LOG.error("Server name", e);
             break;
           }
-
-          if(oldStartCode != startCode) {
-            // Close but no cigar
+          if (serverName != null && serverName.length() > 0 &&
+              deadServerName.compareTo(serverName) != 0) {
+            // This isn't the server you're looking for - move along
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Server name " + serverName + " is not same as " +
+                deadServerName + ": Passing");
+            }
             continue;
           }
 
           // Bingo! Found it.
-
-          bytes = results.get(COL_REGIONINFO);
-          if(bytes == null || bytes.length == 0) {
-            throw new IOException("no value for " + COL_REGIONINFO);
-          }
-          inbuf.reset(bytes, bytes.length);
-          HRegionInfo info = new HRegionInfo();
+          HRegionInfo info = null;
           try {
-            info.readFields(inbuf);
-            
+            info = (HRegionInfo)Writables.
+              getWritable(results.get(COL_REGIONINFO), new HRegionInfo());
           } catch (IOException e) {
-            if (e instanceof RemoteException) {
-              e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
-            }
-            LOG.error(e);
+            LOG.error("Read fields", e);
             break;
           }
-          LOG.info(serverName + " was serving " + info.toString());
-
-          if(info.tableDesc.getName().equals(META_TABLE_NAME)) {
+          LOG.info(info.getRegionName() + " was on shutdown server <" +
+            serverName + "> (or server is null). Marking unassigned if " +
+            "meta and clearing pendingRegions");
+          if (info.tableDesc.getName().equals(META_TABLE_NAME)) {
             onlineMetaRegions.remove(info.getStartKey());
           }
           
           ToDoEntry todo = new ToDoEntry(row, info);
           toDoList.add(todo);
-          
           if(killList.containsKey(deadServerName)) {
             TreeMap<Text, HRegionInfo> regionsToKill = killList.get(deadServerName);
             if(regionsToKill.containsKey(info.regionName)) {
@@ -1374,23 +1422,21 @@
               killList.put(deadServerName, regionsToKill);
               unassignedRegions.remove(info.regionName);
               assignAttempts.remove(info.regionName);
-              
               if(regionsToDelete.contains(info.regionName)) {
                 // Delete this region
-                
                 regionsToDelete.remove(info.regionName);
                 todo.deleteRegion = true;
-                
               } else {
                 // Mark region offline
-                
                 todo.regionOffline = true;
               }
             }
           } else {
             // Get region reassigned
-
             regions.put(info.regionName, info);
+            // If was pending, remove otherwise will obstruct its getting
+            // reassigned.
+            pendingRegions.remove(info.getRegionName());
           }
         }
 
@@ -1403,7 +1449,7 @@
             if (e instanceof RemoteException) {
               e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
             }
-            LOG.error(e);
+            LOG.error("Closing scanner", e);
           }
         }
       }
@@ -1440,21 +1486,22 @@
 
     @Override
     boolean process() throws IOException {
-      LOG.info("server shutdown: " + deadServerName);
+      LOG.info("process shutdown of server " + deadServer + ": logSplit: " +
+        this.logSplit + ", rootChecked: " + this.rootChecked +
+        ", rootRescanned: " + this.rootRescanned + ", numberOfMetaRegions: " +
+        numberOfMetaRegions.get() + ", onlineMetaRegions.size(): " +
+        onlineMetaRegions.size());
 
       if(!logSplit) {
         // Process the old log file
-
-        HLog.splitLog(dir, new Path(dir, "log" + "_" + deadServer.getBindAddress()
-            + "_" + deadServer.getPort()), fs, conf);
-        
+        HLog.splitLog(dir, new Path(dir, "log" + "_" +
+          deadServer.getBindAddress() + "_" + deadServer.getPort()), fs, conf);
         logSplit = true;
       }
 
       if(!rootChecked) {
         if(rootRegionLocation != null
             && deadServer.equals(rootRegionLocation)) {
-
           rootRegionLocation = null;
           unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
               HGlobals.rootRegionInfo);
@@ -1466,7 +1513,6 @@
 
       if(!rootRescanned) {
         // Scan the ROOT region
-
         HRegionInterface server = null;
         long scannerId = -1L;
         for(int tries = 0; tries < numRetries; tries ++) {
@@ -1476,6 +1522,10 @@
           if (rootRegionLocation == null || !rootScanned) {
             // We can't proceed until the root region is online and has been
             // scanned
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("process server shutdown scanning root region " +
+                "cancelled because rootRegionLocation is null");
+            }
             return false;
           }
           server = connection.getHRegionConnection(rootRegionLocation);
@@ -1483,13 +1533,14 @@
 
           try {
             if (LOG.isDebugEnabled()) {
-              LOG.debug("scanning root region");
+              LOG.debug("process server shutdown scanning root region on " +
+                rootRegionLocation.getBindAddress());
             }
             scannerId = server.openScanner(HGlobals.rootRegionInfo.regionName,
-                columns, startRow, System.currentTimeMillis(), null);
-            scanMetaRegion(server, scannerId, HGlobals.rootRegionInfo.regionName);
+              columns, startRow, System.currentTimeMillis(), null);
+            scanMetaRegion(server, scannerId,
+              HGlobals.rootRegionInfo.regionName);
             break;
-
           } catch (IOException e) {
             if (tries == numRetries - 1) {
               if (e instanceof RemoteException) {
@@ -1499,6 +1550,11 @@
             }
           }
         }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("process server shutdown scanning root region on " +
+            rootRegionLocation.getBindAddress() + " finished " +
+            Thread.currentThread().getName());
+        }
         rootRescanned = true;
       }
 
@@ -1507,31 +1563,39 @@
           if (closed) {
             return true;
           }
-          if (!rootScanned
-              || numberOfMetaRegions.get() != onlineMetaRegions.size()) {
-            
+          if (!rootScanned ||
+              numberOfMetaRegions.get() != onlineMetaRegions.size()) {
             // We can't proceed because not all of the meta regions are online.
             // We can't block either because that would prevent the meta region
             // online message from being processed. So return false to have this
             // operation requeued.
-            
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Requeuing shutdown because rootScanned: " +
+                rootScanned + ", numberOfMetaRegions: " +
+                numberOfMetaRegions.get() + ", onlineMetaRegions.size(): " +
+                onlineMetaRegions.size());
+            }
             return false;
           }
-      
+
           for (MetaRegion r: onlineMetaRegions.values()) {
-          
             HRegionInterface server = null;
             long scannerId = -1L;
-
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("process server shutdown scanning " + r.regionName +
+                " on " + r.server + " " + Thread.currentThread().getName());
+            }
             server = connection.getHRegionConnection(r.server);
-          
             scannerId = server.openScanner(r.regionName, columns, startRow,
-                System.currentTimeMillis(), null);
+              System.currentTimeMillis(), null);
             scanMetaRegion(server, scannerId, r.regionName);
-            
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("process server shutdown finished scanning " +
+                r.regionName +
+                " on " + r.server + " " + Thread.currentThread().getName());
+            }
           }
           break;
-            
         } catch (IOException e) {
           if (tries == numRetries - 1) {
             if (e instanceof RemoteException) {
@@ -1576,6 +1640,11 @@
     }
     
     @Override
+    public String toString() {
+      return "PendingCloseReport of " + this.regionInfo.getRegionName();
+    }
+    
+    @Override
     boolean process() throws IOException {
       for (int tries = 0; tries < numRetries; tries++) {
         LOG.info("region closed: " + regionInfo.regionName);
@@ -1599,14 +1668,18 @@
           onlineMetaRegions.remove(regionInfo.getStartKey());
 
         } else {
-          if (!rootScanned
-              || numberOfMetaRegions.get() != onlineMetaRegions.size()) {
-            
+          if (!rootScanned ||
+              numberOfMetaRegions.get() != onlineMetaRegions.size()) {
             // We can't proceed because not all of the meta regions are online.
             // We can't block either because that would prevent the meta region
             // online message from being processed. So return false to have this
             // operation requeued.
-            
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Requeuing close because rootScanned=" +
+                rootScanned + ", numberOfMetaRegions=" +
+                numberOfMetaRegions.get() + ", onlineMetaRegions.size()=" +
+                onlineMetaRegions.size());
+            }
             return false;
           }
 
@@ -1671,8 +1744,7 @@
           if (e instanceof RemoteException) {
             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
           }
-          LOG.error("failed to delete region " + regionInfo.regionName);
-          LOG.error(e);
+          LOG.error("failed delete region " + regionInfo.regionName, e);
           throw e;
         }
       }
@@ -1706,23 +1778,27 @@
         this.startCode =
           String.valueOf(info.getStartCode()).getBytes(UTF8_ENCODING);
       } catch(UnsupportedEncodingException e) {
-        LOG.error(e);
+        LOG.error("Start code", e);
       }
     }
     
     @Override
+    public String toString() {
+      return "PendingOpenOperation from " + serverAddress.toString();
+    }
+    
+    @Override
     boolean process() throws IOException {
       for (int tries = 0; tries < numRetries; tries++) {
         LOG.info(region.getRegionName() + " open on " + 
             this.serverAddress.toString());
-
         // Register the newly-available Region's location.
-
         Text metaRegionName;
         HRegionInterface server;
         if (closed) {
           return true;
         }
+        
         if (rootRegion) {
           if (rootRegionLocation == null || !rootScanned) {
             // We can't proceed until the root region is online and has been
@@ -1735,19 +1811,18 @@
           }
           metaRegionName = HGlobals.rootRegionInfo.regionName;
           server = connection.getHRegionConnection(rootRegionLocation);
-
         } else {
-          if (!rootScanned
-              || numberOfMetaRegions.get() != onlineMetaRegions.size()) {
+          if (!rootScanned ||
+            numberOfMetaRegions.get() != onlineMetaRegions.size()) {
             
             // We can't proceed because not all of the meta regions are online.
             // We can't block either because that would prevent the meta region
             // online message from being processed. So return false to have this
-            // operation requeued.
-            
+            // operation requeue
             if (LOG.isDebugEnabled()) {
-              LOG.debug("rootScanned=" + rootScanned + ", numberOfMetaRegions=" +
-                  numberOfMetaRegions.get() + ", onlineMetaRegions.size()=" +
+              LOG.debug("Requeuing open because rootScanned: " +
+                  rootScanned + ", numberOfMetaRegions: " +
+                  numberOfMetaRegions.get() + ", onlineMetaRegions.size(): " +
                   onlineMetaRegions.size());
             }
             return false;
@@ -1756,7 +1831,6 @@
           MetaRegion r = null;
           if (onlineMetaRegions.containsKey(region.getRegionName())) {
             r = onlineMetaRegions.get(region.getRegionName());
-
           } else {
             r = onlineMetaRegions.get(
                 onlineMetaRegions.headMap(region.getRegionName()).lastKey());
@@ -1766,7 +1840,6 @@
         }
         LOG.info("updating row " + region.getRegionName() + " in table " +
             metaRegionName);
-
         long clientId = rand.nextLong();
         try {
           long lockid = server.startUpdate(metaRegionName, clientId,
@@ -1779,13 +1852,10 @@
           
           if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
             // It's a meta region.
-            
             MetaRegion m =
               new MetaRegion(serverAddress, region.regionName, region.startKey);
-            
             if (!initialMetaScanComplete) {
               // Put it on the queue to be scanned for the first time.
-
               try {
                 metaRegionsToScan.put(m);
               } catch (InterruptedException e) {
@@ -1794,12 +1864,12 @@
               }
             } else {
               // Add it to the online meta regions
-              
               onlineMetaRegions.put(region.startKey, m);
             }
           }
+          // If updated successfully, remove from pending list.
+          pendingRegions.remove(region.getRegionName());
           break;
-          
         } catch (IOException e) {
           if (tries == numRetries - 1) {
             if (e instanceof RemoteException) {
@@ -1808,7 +1878,6 @@
             throw e;
           }
         }
-        pendingRegions.remove(region.getRegionName());
       }
       return true;
     }
@@ -2053,8 +2122,7 @@
                   String serverName = null;
                   long startCode = -1L;
                   
-                  KeyedData[] values = null;
-                  values = server.next(scannerId);
+                  KeyedData[] values = server.next(scannerId);
                   if(values == null || values.length == 0) {
                     break;
                   }
@@ -2111,7 +2179,7 @@
                     if (e instanceof RemoteException) {
                       e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
                     }
-                    LOG.error(e);
+                    LOG.error("", e);
                   }
                 }
                 scannerId = -1L;
@@ -2157,7 +2225,7 @@
         HRegionInfo info) throws IOException;
     
     protected abstract void postProcessMeta(MetaRegion m,
-        HRegionInterface server)
+        HRegionInterface srvr)
     throws IOException;
   }
 
@@ -2196,6 +2264,13 @@
         LOG.debug("processing unserved regions");
       }
       for(HRegionInfo i: unservedRegions) {
+        if (i.offLine && i.isSplit()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Skipping region " + i.toString() + " because it is " +
+              "offline because it has been split");
+          }
+          continue;
+        }
         // Update meta table
         if(LOG.isDebugEnabled()) {
           LOG.debug("updating columns in row: " + i.regionName);
@@ -2221,8 +2296,7 @@
           if (e instanceof RemoteException) {
             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
           }
-          LOG.error("column update failed in row: " + i.regionName);
-          LOG.error(e);
+          LOG.error("column update failed in row: " + i.regionName, e);
 
         } finally {
           try {
@@ -2234,7 +2308,7 @@
             if (iex instanceof RemoteException) {
               iex = RemoteExceptionHandler.decodeRemoteException((RemoteException) iex);
             }
-            LOG.error(iex);
+            LOG.error("", iex);
           }
         }
 
@@ -2243,7 +2317,6 @@
             unassignedRegions.put(i.regionName, i);
             assignAttempts.put(i.regionName, 0L);
           }
-          
         } else {                                // Prevent region from getting assigned.
           unassignedRegions.remove(i.regionName);
           assignAttempts.remove(i.regionName);
@@ -2251,7 +2324,6 @@
       }
       
       // Process regions currently being served
-      
       if(LOG.isDebugEnabled()) {
         LOG.debug("processing regions currently being served");
       }
@@ -2284,16 +2356,14 @@
       servedRegions.clear();
     }
     
-    protected void updateRegionInfo(HRegionInterface server, Text regionName,
-        HRegionInfo i) throws IOException {
-      
+    protected void updateRegionInfo(final HRegionInterface srvr,
+        final Text regionName, final HRegionInfo i)
+    throws IOException {
       i.offLine = !online;
-      
       ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
       DataOutputStream s = new DataOutputStream(byteValue);
       i.write(s);
-
-      server.put(regionName, clientId, lockid, COL_REGIONINFO,
+      srvr.put(regionName, clientId, lockid, COL_REGIONINFO,
         byteValue.toByteArray());
     }
   }
@@ -2310,7 +2380,7 @@
     }
     
     @Override
-    protected void postProcessMeta(MetaRegion m, HRegionInterface server)
+    protected void postProcessMeta(MetaRegion m, HRegionInterface srvr)
         throws IOException {
       // For regions that are being served, mark them for deletion      
       for (TreeSet<HRegionInfo> s: servedRegions.values()) {
@@ -2328,11 +2398,10 @@
           if (e instanceof RemoteException) {
             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
           }
-          LOG.error("failed to delete region " + i.regionName);
-          LOG.error(e);
+          LOG.error("failed to delete region " + i.regionName, e);
         }
       }
-      super.postProcessMeta(m, server);
+      super.postProcessMeta(m, srvr);
     }
     
     @Override
@@ -2381,8 +2450,7 @@
         if (e instanceof RemoteException) {
           e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
         }
-        LOG.error("column update failed in row: " + i.regionName);
-        LOG.error(e);
+        LOG.error("column update failed in row: " + i.regionName, e);
 
       } finally {
         if(lockid != -1L) {
@@ -2393,7 +2461,7 @@
             if (iex instanceof RemoteException) {
               iex = RemoteExceptionHandler.decodeRemoteException((RemoteException) iex);
             }
-            LOG.error(iex);
+            LOG.error("", iex);
           }
         }
       }
@@ -2410,15 +2478,15 @@
     }
     
     @Override
-    protected void postProcessMeta(MetaRegion m, HRegionInterface server)
+    protected void postProcessMeta(MetaRegion m, HRegionInterface srvr)
     throws IOException {
 
       for(HRegionInfo i: unservedRegions) {
         i.tableDesc.families().remove(columnName);
-        updateRegionInfo(server, m.regionName, i);
+        updateRegionInfo(srvr, m.regionName, i);
         
         // Delete the directories used by the column
-
+        
         try {
           fs.delete(HStoreFile.getMapDir(dir, i.regionName, columnName));
           
@@ -2426,7 +2494,7 @@
           if (e instanceof RemoteException) {
             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
           }
-          LOG.error(e);
+          LOG.error("", e);
         }
         
         try {
@@ -2436,7 +2504,7 @@
           if (e instanceof RemoteException) {
             e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
           }
-          LOG.error(e);
+          LOG.error("", e);
         }
         
       }
@@ -2455,7 +2523,7 @@
     }
    
     @Override
-    protected void postProcessMeta(MetaRegion m, HRegionInterface server)
+    protected void postProcessMeta(MetaRegion m, HRegionInterface srvr)
         throws IOException {
 
       for(HRegionInfo i: unservedRegions) {
@@ -2465,7 +2533,7 @@
         // and create it.
         
         i.tableDesc.addFamily(newColumn);
-        updateRegionInfo(server, m.regionName, i);
+        updateRegionInfo(srvr, m.regionName, i);
       }
     }
   }
@@ -2488,15 +2556,12 @@
      */
     public void leaseExpired() {
       LOG.info(server + " lease expired");
-      
       // Remove the server from the known servers list
-      
       HServerInfo storedInfo = serversToServerInfo.remove(server);
       
       // NOTE: If the server was serving the root region, we cannot reassign it
       // here because the new server will start serving the root region before
       // the PendingServerShutdown operation has a chance to split the log file.
-      
       try {
         msgQueue.put(new PendingServerShutdown(storedInfo));
       } catch (InterruptedException e) {
@@ -2539,8 +2604,7 @@
         try {
           (new Thread(new HMaster(conf))).start();
         } catch (Throwable t) {
-          LOG.error( "Can not start master because "+
-              StringUtils.stringifyException(t) );
+          LOG.error( "Can not start master", t);
           System.exit(-1);
         }
         break;
@@ -2551,8 +2615,7 @@
           HBaseAdmin adm = new HBaseAdmin(conf);
           adm.shutdown();
         } catch (Throwable t) {
-          LOG.error( "Can not stop master because " +
-              StringUtils.stringifyException(t) );
+          LOG.error( "Can not stop master", t);
           System.exit(-1);
         }
         break;

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java Wed Aug  8 13:30:13 2007
@@ -29,6 +29,8 @@
 import java.util.Vector;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.Text;
 
@@ -37,6 +39,7 @@
  * wrapper around a TreeMap that helps us when staging the Memcache out to disk.
  */
 public class HMemcache {
+  static final Log LOG = LogFactory.getLog(HMemcache.class);
   TreeMap<HStoreKey, byte []> memcache =
     new TreeMap<HStoreKey, byte []>();
   final Vector<TreeMap<HStoreKey, byte []>> history
@@ -47,6 +50,7 @@
   
   /*
    * Approximate size in bytes of the payload carried by this memcache.
+   * Does not consider deletes nor adding again on same key.
    */
   private AtomicLong size = new AtomicLong(0);
 
@@ -157,6 +161,7 @@
   
   /**
    * @return Approximate size in bytes of payload carried by this memcache.
+   * Does not take into consideration deletes nor adding again on same key.
    */
   public long getSize() {
     return this.size.get();

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java?view=diff&rev=564012&r1=564011&r2=564012
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java Wed Aug  8 13:30:13 2007
@@ -143,17 +143,17 @@
       long currentSize = 0;
       HRegion nextRegion = null;
       long nextSize = 0;
+      Text midKey = new Text();
       for(int i = 0; i < regions.length - 1; i++) {
         if(currentRegion == null) {
           currentRegion =
             new HRegion(dir, hlog, fs, conf, regions[i], null);
-
-          currentSize = currentRegion.largestHStore();
+          currentSize = currentRegion.largestHStore(midKey).getAggregate();
         }
         nextRegion =
           new HRegion(dir, hlog, fs, conf, regions[i + 1], null);
 
-        nextSize = nextRegion.largestHStore();
+        nextSize = nextRegion.largestHStore(midKey).getAggregate();
 
         long maxFilesize =
           conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE);