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 ji...@apache.org on 2007/10/19 00:21:11 UTC

svn commit: r586159 [1/2] - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/filter/ src/test/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/filter/

Author: jimk
Date: Thu Oct 18 15:21:09 2007
New Revision: 586159

URL: http://svn.apache.org/viewvc?rev=586159&view=rev
Log:
HADOOP-2056 A table with row keys containing colon fails to split regions

Many modifications required because region names (which contain start key) were used as file or directory names. Now the startKey is encoded in an order preserving Base64 dialect.

Removed:
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HGlobals.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.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/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/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/HTableDescriptor.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.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/MultiRegionTable.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompaction.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.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/TestHLog.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/TestHStoreFile.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/TestScanner2.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTable.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java

Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Thu Oct 18 15:21:09 2007
@@ -13,6 +13,7 @@
    HADOOP-2059 In tests, exceptions in min dfs shutdown should not fail test
                (e.g. nightly #272)
    HADOOP-2064 TestSplit assertion and NPE failures (Patch build #952 and #953)
+   HADOOP-2056 A table with row keys containing colon fails to split regions
 
   IMPROVEMENTS
     HADOOP-2401 Add convenience put method that takes writable

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -192,7 +192,7 @@
             info = (HRegionInfo) Writables.getWritable(
                   ((ImmutableBytesWritable) e.getValue()).get(), info);
             
-            if (info.tableDesc.getName().equals(tableName)) {
+            if (info.getTableDesc().getName().equals(tableName)) {
               found = true;
             }
           }
@@ -280,7 +280,7 @@
               info = (HRegionInfo) Writables.getWritable(
                     ((ImmutableBytesWritable) e.getValue()).get(), info);
             
-              isenabled = !info.offLine;
+              isenabled = !info.isOffline();
               break;
             }
           }
@@ -380,7 +380,7 @@
               info = (HRegionInfo) Writables.getWritable(
                     ((ImmutableBytesWritable) e.getValue()).get(), info);
             
-              disabled = info.offLine;
+              disabled = info.isOffline();
               break;
             }
           }

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -261,8 +261,8 @@
                     ((ImmutableBytesWritable) e.getValue()).get(), info);
 
                 // Only examine the rows where the startKey is zero length   
-                if (info.startKey.getLength() == 0) {
-                  uniqueTables.add(info.tableDesc);
+                if (info.getStartKey().getLength() == 0) {
+                  uniqueTables.add(info.getTableDesc());
                 }
               }
             }
@@ -603,7 +603,7 @@
         HRegionInterface rootRegion = getHRegionConnection(rootRegionLocation);
 
         try {
-          rootRegion.getRegionInfo(HGlobals.rootRegionInfo.regionName);
+          rootRegion.getRegionInfo(HRegionInfo.rootRegionInfo.getRegionName());
           break;
         } catch (IOException e) {
           if (tries == numRetries - 1) {
@@ -640,7 +640,7 @@
         new TreeMap<Text, HRegionLocation>();
       
       rootServer.put(EMPTY_START_ROW,
-          new HRegionLocation(HGlobals.rootRegionInfo, rootRegionLocation));
+          new HRegionLocation(HRegionInfo.rootRegionInfo, rootRegionLocation));
       
       return rootServer;
     }
@@ -707,7 +707,7 @@
             HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
               results.get(COL_REGIONINFO), new HRegionInfo());
 
-            if (!regionInfo.tableDesc.getName().equals(tableName)) {
+            if (!regionInfo.getTableDesc().getName().equals(tableName)) {
               // We're done
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Found " + servers.size() + " servers for table " +
@@ -736,7 +736,7 @@
             }
             
             String serverAddress = Writables.bytesToString(bytes);
-            servers.put(regionInfo.startKey, new HRegionLocation(
+            servers.put(regionInfo.getStartKey(), new HRegionLocation(
                 regionInfo, new HServerAddress(serverAddress)));
           }
         } catch (IOException e) {

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -46,7 +46,8 @@
   
   /** Default master address */
   static final String DEFAULT_MASTER_ADDRESS = DEFAULT_HOST + ":60000";
-  
+
+  /** default port for master web api */
   static final int DEFAULT_MASTER_INFOPORT = 60010;
 
   /** Parameter name for hbase.regionserver address. */
@@ -54,7 +55,8 @@
   
   /** Default region server address */
   static final String DEFAULT_REGIONSERVER_ADDRESS = DEFAULT_HOST + ":60020";
-  
+
+  /** default port for region server web api */
   static final int DEFAULT_REGIONSERVER_INFOPORT = 60030;
 
   /** Parameter name for what region server interface to use. */

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -159,7 +159,9 @@
             SequenceFile.Writer w = logWriters.get(regionName);
             if (w == null) {
               Path logfile = new Path(HRegion.getRegionDir(rootDir,
-                regionName), HREGION_OLDLOGFILE_NAME);
+                HRegionInfo.rootRegionInfo.getEncodedName()),
+                HREGION_OLDLOGFILE_NAME);
+              
               if (LOG.isDebugEnabled()) {
                 LOG.debug("getting new log file writer for path " + logfile);
               }
@@ -498,7 +500,7 @@
         return;
       }
       this.writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
-        new HLogEdit(HLog.METACOLUMN, HGlobals.completeCacheFlush.get(),
+        new HLogEdit(HLog.METACOLUMN, HLogEdit.completeCacheFlush.get(),
           System.currentTimeMillis()));
       this.numEntries++;
       Long seq = this.lastSeqWritten.get(regionName);

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -19,6 +19,7 @@
  */
 package org.apache.hadoop.hbase;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.*;
 
 import java.io.*;
@@ -30,7 +31,35 @@
  * The table and row are already identified in HLogKey.
  * This just indicates the column and value.
  */
-public class HLogEdit implements Writable {
+public class HLogEdit implements Writable, HConstants {
+
+  /** Value stored for a deleted item */
+  public static ImmutableBytesWritable deleteBytes = null;
+
+  /** Value written to HLog on a complete cache flush */
+  public static ImmutableBytesWritable completeCacheFlush = null;
+
+  static {
+    try {
+      deleteBytes =
+        new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes(UTF8_ENCODING));
+    
+      completeCacheFlush =
+        new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes(UTF8_ENCODING));
+      
+    } catch (UnsupportedEncodingException e) {
+      assert(false);
+    }
+  }
+  
+  /**
+   * @param value
+   * @return True if an entry and its content is {@link #deleteBytes}.
+   */
+  public static boolean isDeleted(final byte [] value) {
+    return (value == null)? false: deleteBytes.compareTo(value) == 0;
+  }
+
   private Text column = new Text();
   private byte [] val;
   private long timestamp;

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -124,6 +124,8 @@
   // Default access so accesible from unit tests. MASTER is name of the webapp
   // and the attribute name used stuffing this instance into web context.
   InfoServer infoServer;
+  
+  /** Name of master server */
   public static final String MASTER = "master";
 
   /**
@@ -294,7 +296,7 @@
         return false;
       }
       if (!info.isOffline()) {
-        LOG.warn("Region is split but not offline: " + info.regionName);
+        LOG.warn("Region is split but not offline: " + info.getRegionName());
       }
       return true;
     }
@@ -324,7 +326,7 @@
       if (!hasReferencesA && !hasReferencesB) {
         LOG.info("Deleting region " + parent.getRegionName() +
         " because daughter splits no longer hold references");
-        if (!HRegion.deleteRegion(fs, dir, parent.getRegionName())) {
+        if (!HRegion.deleteRegion(fs, dir, parent.getEncodedName())) {
           LOG.warn("Deletion of " + parent.getRegionName() + " failed");
         }
         
@@ -368,7 +370,7 @@
       }
       for (Text family: split.getTableDesc().families().keySet()) {
         Path p = HStoreFile.getMapDir(fs.makeQualified(dir),
-            split.getRegionName(), HStoreKey.extractFamily(family));
+            split.getEncodedName(), HStoreKey.extractFamily(family));
 
         // Look for reference files.  Call listPaths with an anonymous
         // instance of PathFilter.
@@ -408,22 +410,23 @@
       final String serverName, final long startCode)
     throws IOException {
       // Skip region - if ...
-      if(info.offLine                                     // offline
-          || killedRegions.contains(info.regionName)      // queued for offline
-          || regionsToDelete.contains(info.regionName)) { // queued for delete
-        unassignedRegions.remove(info.regionName);
-        assignAttempts.remove(info.regionName);
+      if(info.isOffline()                                 // offline
+          || killedRegions.contains(info.getRegionName()) // queued for offline
+          || regionsToDelete.contains(info.getRegionName())) { // queued for delete
+        unassignedRegions.remove(info.getRegionName());
+        assignAttempts.remove(info.getRegionName());
         return;
       }
       HServerInfo storedInfo = null;
       if (serverName.length() != 0) {
         Map<Text, HRegionInfo> regionsToKill = killList.get(serverName);
         if (regionsToKill != null &&
-            regionsToKill.containsKey(info.regionName)) {
+            regionsToKill.containsKey(info.getRegionName())) {
           
           // 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.getRegionName());
           }
           return;
         }
@@ -432,14 +435,15 @@
         }
       }
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Checking " + info.regionName + " is assigned");
+        LOG.debug("Checking " + info.getRegionName() + " is assigned");
       }
-      if (!(unassignedRegions.containsKey(info.regionName) ||
-            pendingRegions.contains(info.regionName))
+      if (!(unassignedRegions.containsKey(info.getRegionName()) ||
+            pendingRegions.contains(info.getRegionName()))
           && (storedInfo == null || storedInfo.getStartCode() != startCode)) {
         // The current assignment is no good
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Current assignment of " + info.regionName + " is no good");
+          LOG.debug("Current assignment of " + info.getRegionName() +
+              " is no good");
         }
         // Recover the region server's log if there is one.
         if (serverName.length() != 0) {
@@ -464,8 +468,8 @@
           }
         }
         // Now get the region assigned
-        unassignedRegions.put(info.regionName, info);
-        assignAttempts.put(info.regionName, Long.valueOf(0L));
+        unassignedRegions.put(info.getRegionName(), info);
+        assignAttempts.put(info.getRegionName(), Long.valueOf(0L));
       }
     }
   }
@@ -503,7 +507,7 @@
           // Don't interrupt us while we're working
           synchronized(rootScannerLock) {
             scanRegion(new MetaRegion(rootRegionLocation.get(),
-                HGlobals.rootRegionInfo.regionName, null));
+                HRegionInfo.rootRegionInfo.getRegionName(), null));
           }
           succeeded = true;
           break;
@@ -552,6 +556,7 @@
   private RootScanner rootScannerThread;
   Integer rootScannerLock = new Integer(0);
 
+  /** Describes a meta region and its server */
   @SuppressWarnings("unchecked")
   public static class MetaRegion implements Comparable {
     private HServerAddress server;
@@ -575,6 +580,7 @@
       }
     }
     
+    /** {@inheritDoc} */
     @Override
     public String toString() {
       return "regionname: " + this.regionName.toString() + ", startKey: <" +
@@ -859,7 +865,7 @@
     this.rand = new Random();
     
     Path rootRegionDir =
-      HRegion.getRegionDir(dir, HGlobals.rootRegionInfo.regionName);
+      HRegion.getRegionDir(dir, HRegionInfo.rootRegionInfo.getEncodedName());
     LOG.info("Root region dir: " + rootRegionDir.toString());
 
     try {
@@ -871,10 +877,10 @@
       if (!fs.exists(rootRegionDir)) {
         LOG.info("bootstrap: creating ROOT and first META regions");
         try {
-          HRegion root = HRegion.createHRegion(HGlobals.rootRegionInfo, this.dir,
-            this.conf, null);
-          HRegion meta = HRegion.createHRegion(new HRegionInfo(1L,
-            HGlobals.metaTableDesc, null, null), this.dir, this.conf, null);
+          HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo,
+              this.dir, this.conf, null);
+          HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo,
+            this.dir, this.conf, null);
 
           // Add first region from the META table to the ROOT region.
           HRegion.addRegionToMETA(root, meta);
@@ -961,9 +967,9 @@
    */
   void unassignRootRegion() {
     this.rootRegionLocation.set(null);
-    this.unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
-        HGlobals.rootRegionInfo);
-    this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
+    this.unassignedRegions.put(HRegionInfo.rootRegionInfo.getRegionName(),
+        HRegionInfo.rootRegionInfo);
+    this.assignAttempts.put(HRegionInfo.rootRegionInfo.getRegionName(),
         Long.valueOf(0L));
     // TODO: If the old root region server had a log, it needs splitting.
   }
@@ -1300,14 +1306,14 @@
         if (!closed.get()) {
           for (int i = 1; i < msgs.length; i++) {
             HRegionInfo info = msgs[i].getRegionInfo();
-            if (info.tableDesc.getName().equals(ROOT_TABLE_NAME)) {
+            if (info.getTableDesc().getName().equals(ROOT_TABLE_NAME)) {
               rootRegionLocation.set(null);
-            } else if (info.tableDesc.getName().equals(META_TABLE_NAME)) {
+            } else if (info.getTableDesc().getName().equals(META_TABLE_NAME)) {
               onlineMetaRegions.remove(info.getStartKey());
             }
 
-            this.unassignedRegions.put(info.regionName, info);
-            this.assignAttempts.put(info.regionName, Long.valueOf(0L));
+            this.unassignedRegions.put(info.getRegionName(), info);
+            this.assignAttempts.put(info.getRegionName(), Long.valueOf(0L));
           }
         }
       }
@@ -1449,13 +1455,13 @@
       switch (incomingMsgs[i].getMsg()) {
 
       case HMsg.MSG_REPORT_OPEN:
-        HRegionInfo regionInfo = unassignedRegions.get(region.regionName);
+        HRegionInfo regionInfo = unassignedRegions.get(region.getRegionName());
 
         if (regionInfo == null) {
 
           if (LOG.isDebugEnabled()) {
             LOG.debug("region server " + info.getServerAddress().toString()
-                + " should not have opened region " + region.regionName);
+                + " should not have opened region " + region.getRegionName());
           }
 
           // This Region should not have been opened.
@@ -1467,12 +1473,12 @@
 
         } else {
           LOG.info(info.getServerAddress().toString() + " serving " +
-              region.regionName);
+              region.getRegionName());
           // Remove from unassigned list so we don't assign it to someone else
-          this.unassignedRegions.remove(region.regionName);
-          this.assignAttempts.remove(region.regionName);
-          if (region.regionName.compareTo(
-              HGlobals.rootRegionInfo.regionName) == 0) {
+          this.unassignedRegions.remove(region.getRegionName());
+          this.assignAttempts.remove(region.getRegionName());
+          if (region.getRegionName().compareTo(
+              HRegionInfo.rootRegionInfo.getRegionName()) == 0) {
             // Store the Root Region location (in memory)
             synchronized (rootRegionLocation) {
               this.rootRegionLocation.
@@ -1485,7 +1491,7 @@
           // Note that the table has been assigned and is waiting for the meta
           // table to be updated.
 
-          pendingRegions.add(region.regionName);
+          pendingRegions.add(region.getRegionName());
 
           // Queue up an update to note the region location.
 
@@ -1499,26 +1505,26 @@
 
       case HMsg.MSG_REPORT_CLOSE:
         LOG.info(info.getServerAddress().toString() + " no longer serving " +
-            region.regionName);
+            region.getRegionName());
 
-        if (region.regionName.compareTo(
-            HGlobals.rootRegionInfo.regionName) == 0) {
+        if (region.getRegionName().compareTo(
+            HRegionInfo.rootRegionInfo.getRegionName()) == 0) {
           
           // Root region
           
           rootRegionLocation.set(null);
-          unassignedRegions.put(region.regionName, region);
-          assignAttempts.put(region.regionName, Long.valueOf(0L));
+          unassignedRegions.put(region.getRegionName(), region);
+          assignAttempts.put(region.getRegionName(), Long.valueOf(0L));
 
         } else {
           boolean reassignRegion = true;
           boolean deleteRegion = false;
 
-          if (killedRegions.remove(region.regionName)) {
+          if (killedRegions.remove(region.getRegionName())) {
             reassignRegion = false;
           }
 
-          if (regionsToDelete.remove(region.regionName)) {
+          if (regionsToDelete.remove(region.getRegionName())) {
             reassignRegion = false;
             deleteRegion = true;
           }
@@ -1527,8 +1533,8 @@
           //       could create a race with the pending close if it gets 
           //       reassigned before the close is processed.
 
-          unassignedRegions.remove(region.regionName);
-          assignAttempts.remove(region.regionName);
+          unassignedRegions.remove(region.getRegionName());
+          assignAttempts.remove(region.getRegionName());
 
           try {
             msgQueue.put(new PendingCloseReport(region, reassignRegion,
@@ -1551,10 +1557,11 @@
         unassignedRegions.put(newRegionB.getRegionName(), newRegionB);
         assignAttempts.put(newRegionB.getRegionName(), Long.valueOf(0L));
 
-        LOG.info("region " + region.regionName + " split. New regions are: "
-            + newRegionA.regionName + ", " + newRegionB.regionName);
+        LOG.info("region " + region.getRegionName() +
+            " split. New regions are: " + newRegionA.getRegionName() + ", " +
+            newRegionB.getRegionName());
 
-        if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
+        if (region.getTableDesc().getName().equals(META_TABLE_NAME)) {
           // A meta region has split.
 
           onlineMetaRegions.remove(region.getStartKey());
@@ -1574,7 +1581,7 @@
     if (regionsToKill != null) {
       for (HRegionInfo i: regionsToKill.values()) {
         returnMsgs.add(new HMsg(HMsg.MSG_REGION_CLOSE, i));
-        killedRegions.add(i.regionName);
+        killedRegions.add(i.getRegionName());
       }
     }
 
@@ -1899,7 +1906,7 @@
               serverName + "> (or server is null). Marking unassigned if " +
           "meta and clearing pendingRegions");
 
-          if (info.tableDesc.getName().equals(META_TABLE_NAME)) {
+          if (info.getTableDesc().getName().equals(META_TABLE_NAME)) {
             onlineMetaRegions.remove(info.getStartKey());
           }
 
@@ -1910,14 +1917,14 @@
             HashMap<Text, HRegionInfo> regionsToKill =
               killList.get(deadServerName);
 
-            if (regionsToKill.containsKey(info.regionName)) {
-              regionsToKill.remove(info.regionName);
+            if (regionsToKill.containsKey(info.getRegionName())) {
+              regionsToKill.remove(info.getRegionName());
               killList.put(deadServerName, regionsToKill);
-              unassignedRegions.remove(info.regionName);
-              assignAttempts.remove(info.regionName);
-              if (regionsToDelete.contains(info.regionName)) {
+              unassignedRegions.remove(info.getRegionName());
+              assignAttempts.remove(info.getRegionName());
+              if (regionsToDelete.contains(info.getRegionName())) {
                 // Delete this region
-                regionsToDelete.remove(info.regionName);
+                regionsToDelete.remove(info.getRegionName());
                 todo.deleteRegion = true;
               } else {
                 // Mark region offline
@@ -1927,7 +1934,7 @@
             
           } else {
             // Get region reassigned
-            regions.put(info.regionName, info);
+            regions.put(info.getRegionName(), info);
            
             // If it was pending, remove.
             // Otherwise will obstruct its getting reassigned.
@@ -1952,7 +1959,7 @@
         if (e.deleteRegion) {
           b.delete(lockid, COL_REGIONINFO);
         } else if (e.regionOffline) {
-          e.info.offLine = true;
+          e.info.setOffline(true);
           b.put(lockid, COL_REGIONINFO, Writables.getBytes(e.info));
         }
         b.delete(lockid, COL_SERVER);
@@ -1997,10 +2004,10 @@
             deadServer.equals(rootRegionLocation.get())) {
 
           rootRegionLocation.set(null);
-          unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
-              HGlobals.rootRegionInfo);
+          unassignedRegions.put(HRegionInfo.rootRegionInfo.getRegionName(),
+              HRegionInfo.rootRegionInfo);
 
-          assignAttempts.put(HGlobals.rootRegionInfo.regionName,
+          assignAttempts.put(HRegionInfo.rootRegionInfo.getRegionName(),
               Long.valueOf(0L));
         }
         rootChecked = true;
@@ -2032,11 +2039,13 @@
               LOG.debug("process server shutdown scanning root region on " +
                   rootRegionLocation.get().getBindAddress());
             }
-            scannerId = server.openScanner(HGlobals.rootRegionInfo.regionName,
-                COLUMN_FAMILY_ARRAY, EMPTY_START_ROW,
-                System.currentTimeMillis(), null);
+            scannerId =
+              server.openScanner(HRegionInfo.rootRegionInfo.getRegionName(),
+                  COLUMN_FAMILY_ARRAY, EMPTY_START_ROW,
+                  System.currentTimeMillis(), null);
             
-            scanMetaRegion(server, scannerId, HGlobals.rootRegionInfo.regionName);
+            scanMetaRegion(server, scannerId,
+                HRegionInfo.rootRegionInfo.getRegionName());
             break;
 
           } catch (IOException e) {
@@ -2132,7 +2141,7 @@
       // If the region closing down is a meta region then we need to update
       // the ROOT table
 
-      if (this.regionInfo.tableDesc.getName().equals(META_TABLE_NAME)) {
+      if (this.regionInfo.getTableDesc().getName().equals(META_TABLE_NAME)) {
         this.rootRegion = true;
 
       } else {
@@ -2152,7 +2161,7 @@
         if (closed.get()) {
           return true;
         }
-        LOG.info("region closed: " + regionInfo.regionName);
+        LOG.info("region closed: " + regionInfo.getRegionName());
 
         // Mark the Region as unavailable in the appropriate meta table
 
@@ -2164,7 +2173,7 @@
             // scanned
             return false;
           }
-          metaRegionName = HGlobals.rootRegionInfo.regionName;
+          metaRegionName = HRegionInfo.rootRegionInfo.getRegionName();
           server = connection.getHRegionConnection(rootRegionLocation.get());
           onlineMetaRegions.remove(regionInfo.getStartKey());
 
@@ -2200,13 +2209,13 @@
 
         try {
           BatchUpdate b = new BatchUpdate(rand.nextLong());
-          long lockid = b.startUpdate(regionInfo.regionName);
+          long lockid = b.startUpdate(regionInfo.getRegionName());
 
           if (deleteRegion) {
             b.delete(lockid, COL_REGIONINFO);
 
           } else if (!reassignRegion ) {
-            regionInfo.offLine = true;
+            regionInfo.setOffline(true);
             b.put(lockid, COL_REGIONINFO, Writables.getBytes(regionInfo));
           }
           b.delete(lockid, COL_SERVER);
@@ -2224,17 +2233,17 @@
       }
 
       if (reassignRegion) {
-        LOG.info("reassign region: " + regionInfo.regionName);
+        LOG.info("reassign region: " + regionInfo.getRegionName());
 
-        unassignedRegions.put(regionInfo.regionName, regionInfo);
-        assignAttempts.put(regionInfo.regionName, Long.valueOf(0L));
+        unassignedRegions.put(regionInfo.getRegionName(), regionInfo);
+        assignAttempts.put(regionInfo.getRegionName(), Long.valueOf(0L));
 
       } else if (deleteRegion) {
         try {
-          HRegion.deleteRegion(fs, dir, regionInfo.regionName);
+          HRegion.deleteRegion(fs, dir, regionInfo.getEncodedName());
         } catch (IOException e) {
           e = RemoteExceptionHandler.checkIOException(e);
-          LOG.error("failed delete region " + regionInfo.regionName, e);
+          LOG.error("failed delete region " + regionInfo.getRegionName(), e);
           throw e;
         }
       }
@@ -2258,12 +2267,13 @@
       // If true, the region which just came on-line is a META region.
       // We need to look in the ROOT region for its information.  Otherwise,
       // its just an ordinary region. Look for it in the META table.
-      this.rootRegion = region.tableDesc.getName().equals(META_TABLE_NAME);
+      this.rootRegion = region.getTableDesc().getName().equals(META_TABLE_NAME);
       this.region = region;
       this.serverAddress = info.getServerAddress();
       this.startCode = Writables.longToBytes(info.getStartCode());
     }
 
+    /** {@inheritDoc} */
     @Override
     public String toString() {
       return "PendingOpenOperation from " + serverAddress.toString();
@@ -2292,7 +2302,7 @@
             }
             return false;
           }
-          metaRegionName = HGlobals.rootRegionInfo.regionName;
+          metaRegionName = HRegionInfo.rootRegionInfo.getRegionName();
           server = connection.getHRegionConnection(rootRegionLocation.get());
         } else {
           if (!rootScanned ||
@@ -2329,10 +2339,10 @@
             Writables.stringToBytes(serverAddress.toString()));
           b.put(lockid, COL_STARTCODE, startCode);
           server.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
-          if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
+          if (region.getTableDesc().getName().equals(META_TABLE_NAME)) {
             // It's a meta region.
             MetaRegion m = new MetaRegion(this.serverAddress,
-              this.region.regionName, this.region.startKey);
+              this.region.getRegionName(), this.region.getRegionName());
             if (!initialMetaScanComplete) {
               // Put it on the queue to be scanned for the first time.
               try {
@@ -2345,7 +2355,7 @@
             } else {
               // Add it to the online meta regions
               LOG.debug("Adding to onlineMetaRegions: " + m.toString());
-              onlineMetaRegions.put(this.region.startKey, m);
+              onlineMetaRegions.put(this.region.getRegionName(), m);
             }
           }
           // If updated successfully, remove from pending list.
@@ -2393,7 +2403,7 @@
     if (!isMasterRunning()) {
       throw new MasterNotRunningException();
     }
-    HRegionInfo newRegion = new HRegionInfo(rand.nextLong(), desc, null, null);
+    HRegionInfo newRegion = new HRegionInfo(desc, null, null);
 
     for (int tries = 0; tries < numRetries; tries++) {
       try {
@@ -2418,7 +2428,7 @@
   private Set<Text> tableInCreation = new HashSet<Text>();
 
   private void createTable(final HRegionInfo newRegion) throws IOException {
-    Text tableName = newRegion.tableDesc.getName();
+    Text tableName = newRegion.getTableDesc().getName();
     synchronized (tableInCreation) {
       if (tableInCreation.contains(tableName)) {
         throw new TableExistsException("Table " + tableName + " in process "
@@ -2432,8 +2442,8 @@
       // for the table we want to create already exists, then table already
       // created. Throw already-exists exception.
       
-      MetaRegion m = (onlineMetaRegions.containsKey(newRegion.regionName) ?
-          onlineMetaRegions.get(newRegion.regionName) :
+      MetaRegion m = (onlineMetaRegions.containsKey(newRegion.getRegionName()) ?
+          onlineMetaRegions.get(newRegion.getRegionName()) :
             onlineMetaRegions.get(onlineMetaRegions.headMap(
                 newRegion.getTableDesc().getName()).lastKey()));
           
@@ -2631,7 +2641,7 @@
                     throw new IOException(COL_REGIONINFO + " not found");
                   }
 
-                  if (info.tableDesc.getName().compareTo(tableName) > 0) {
+                  if (info.getTableDesc().getName().compareTo(tableName) > 0) {
                     break;               // Beyond any more entries for this table
                   }
 
@@ -2690,7 +2700,7 @@
     }
 
     protected boolean isEnabled(HRegionInfo info) {
-      return !info.offLine;
+      return !info.isOffline();
     }
 
     protected abstract void processScanItem(String serverName, long startCode,
@@ -2738,7 +2748,7 @@
         LOG.debug("processing unserved regions");
       }
       for (HRegionInfo i: unservedRegions) {
-        if (i.offLine && i.isSplit()) {
+        if (i.isOffline() && i.isSplit()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("Skipping region " + i.toString() + " because it is " +
                 "offline because it has been split");
@@ -2749,28 +2759,28 @@
         // Update meta table
         
         if (LOG.isDebugEnabled()) {
-          LOG.debug("updating columns in row: " + i.regionName);
+          LOG.debug("updating columns in row: " + i.getRegionName());
         }
 
         BatchUpdate b = new BatchUpdate(rand.nextLong());
-        lockid = b.startUpdate(i.regionName);
+        lockid = b.startUpdate(i.getRegionName());
         updateRegionInfo(b, i);
         b.delete(lockid, COL_SERVER);
         b.delete(lockid, COL_STARTCODE);
         server.batchUpdate(m.getRegionName(), System.currentTimeMillis(), b);
         if (LOG.isDebugEnabled()) {
-          LOG.debug("updated columns in row: " + i.regionName);
+          LOG.debug("updated columns in row: " + i.getRegionName());
         }
 
         if (online) {                           // Bring offline regions on-line
-          if (!unassignedRegions.containsKey(i.regionName)) {
-            unassignedRegions.put(i.regionName, i);
-            assignAttempts.put(i.regionName, Long.valueOf(0L));
+          if (!unassignedRegions.containsKey(i.getRegionName())) {
+            unassignedRegions.put(i.getRegionName(), i);
+            assignAttempts.put(i.getRegionName(), Long.valueOf(0L));
           }
 
         } else {                                // Prevent region from getting assigned.
-          unassignedRegions.remove(i.regionName);
-          assignAttempts.remove(i.regionName);
+          unassignedRegions.remove(i.getRegionName());
+          assignAttempts.remove(i.getRegionName());
         }
       }
 
@@ -2794,9 +2804,10 @@
         }
         for (HRegionInfo i: e.getValue()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("adding region " + i.regionName + " to local kill list");
+            LOG.debug("adding region " + i.getRegionName() +
+                " to local kill list");
           }
-          localKillList.put(i.regionName, i);
+          localKillList.put(i.getRegionName(), i);
         }
         if (localKillList.size() > 0) {
           if (LOG.isDebugEnabled()) {
@@ -2812,7 +2823,7 @@
     protected void updateRegionInfo(final BatchUpdate b, final HRegionInfo i)
       throws IOException {
       
-      i.offLine = !online;
+      i.setOffline(!online);
       b.put(lockid, COL_REGIONINFO, Writables.getBytes(i));
     }
   }
@@ -2836,7 +2847,7 @@
       
       for (HashSet<HRegionInfo> s: servedRegions.values()) {
         for (HRegionInfo i: s) {
-          regionsToDelete.add(i.regionName);
+          regionsToDelete.add(i.getRegionName());
         }
       }
 
@@ -2846,10 +2857,10 @@
         // Delete the region
       
         try {
-          HRegion.deleteRegion(fs, dir, i.regionName);
+          HRegion.deleteRegion(fs, dir, i.getEncodedName());
         
         } catch (IOException e) {
-          LOG.error("failed to delete region " + i.regionName,
+          LOG.error("failed to delete region " + i.getRegionName(),
             RemoteExceptionHandler.checkIOException(e));
         }
       }
@@ -2885,11 +2896,11 @@
         HRegionInfo i) throws IOException {
 
       BatchUpdate b = new BatchUpdate(rand.nextLong());
-      long lockid = b.startUpdate(i.regionName);
+      long lockid = b.startUpdate(i.getRegionName());
       b.put(lockid, COL_REGIONINFO, Writables.getBytes(i));
       server.batchUpdate(regionName, System.currentTimeMillis(), b);
       if (LOG.isDebugEnabled()) {
-        LOG.debug("updated columns in row: " + i.regionName);
+        LOG.debug("updated columns in row: " + i.getRegionName());
       }
     }
   }
@@ -2908,13 +2919,13 @@
       throws IOException {
 
       for (HRegionInfo i: unservedRegions) {
-        i.tableDesc.families().remove(columnName);
+        i.getTableDesc().families().remove(columnName);
         updateRegionInfo(server, m.getRegionName(), i);
 
         // Delete the directories used by the column
 
-        fs.delete(HStoreFile.getMapDir(dir, i.regionName, columnName));
-        fs.delete(HStoreFile.getInfoDir(dir, i.regionName, columnName));
+        fs.delete(HStoreFile.getMapDir(dir, i.getEncodedName(), columnName));
+        fs.delete(HStoreFile.getInfoDir(dir, i.getEncodedName(), columnName));
       }
     }
   }
@@ -2938,7 +2949,7 @@
         // When the region is brought on-line, it will find the column missing
         // and create it.
 
-        i.tableDesc.addFamily(newColumn);
+        i.getTableDesc().addFamily(newColumn);
         updateRegionInfo(server, m.getRegionName(), i);
       }
     }

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -267,7 +267,7 @@
     for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
       HStoreKey itKey = es.getKey();
       if (itKey.matchesRowCol(curKey)) {
-        if (!isDeleted(es.getValue())) {
+        if (!HLogEdit.isDeleted(es.getValue())) {
           result.add(tailMap.get(itKey));
           curKey.setVersion(itKey.getTimestamp() - 1);
         }
@@ -323,7 +323,7 @@
       if (!key.matchesRowCol(origin)) {
         break;
       }
-      if (!isDeleted(es.getValue())) {
+      if (!HLogEdit.isDeleted(es.getValue())) {
         result.add(key);
         if (versions != HConstants.ALL_VERSIONS && result.size() >= versions) {
           // We have enough results.  Return.
@@ -341,15 +341,7 @@
    * the cell has been deleted.
    */
   boolean isDeleted(final HStoreKey key) {
-    return isDeleted(this.memcache.get(key));
-  }
-
-  /**
-   * @param value
-   * @return True if an entry and its content is {@link HGlobals.deleteBytes}.
-   */
-  boolean isDeleted(final byte [] value) {
-    return (value == null)? false: HGlobals.deleteBytes.compareTo(value) == 0;
+    return HLogEdit.isDeleted(this.memcache.get(key));
   }
 
   /**

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?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- 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 Thu Oct 18 15:21:09 2007
@@ -225,8 +225,8 @@
         HRegionInfo region =
           (HRegionInfo) Writables.getWritable(bytes, new HRegionInfo());
 
-        if(!region.offLine) {
-          throw new TableNotDisabledException("region " + region.regionName
+        if(!region.isOffline()) {
+          throw new TableNotDisabledException("region " + region.getRegionName()
               + " is not disabled");
         }
         return region;
@@ -267,7 +267,7 @@
           oldRegion2
       };
       for(int r = 0; r < regionsToDelete.length; r++) {
-        if(regionsToDelete[r].equals(latestRegion.regionName)) {
+        if(regionsToDelete[r].equals(latestRegion.getRegionName())) {
           latestRegion = null;
         }
         long lockid = -1L;
@@ -290,7 +290,7 @@
       }
       ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
       DataOutputStream s = new DataOutputStream(byteValue);
-      newRegion.getRegionInfo().offLine = true;
+      newRegion.getRegionInfo().setOffline(true);
       newRegion.getRegionInfo().write(s);
       long lockid = -1L;
       try {
@@ -326,7 +326,7 @@
       // Scan root region to find all the meta regions
       
       HRegion root =
-        new HRegion(dir, hlog,fs, conf, HGlobals.rootRegionInfo, null);
+        new HRegion(dir, hlog,fs, conf, HRegionInfo.rootRegionInfo, null);
 
       HInternalScannerInterface rootScanner =
         root.getScanner(META_COLS, new Text(), System.currentTimeMillis(), null);
@@ -362,7 +362,7 @@
         HRegion newRegion) throws IOException {
       
       HRegion root =
-        new HRegion(dir, hlog, fs, conf, HGlobals.rootRegionInfo, null);
+        new HRegion(dir, hlog, fs, conf, HRegionInfo.rootRegionInfo, null);
 
       Text[] regionsToDelete = {
           oldRegion1,
@@ -394,7 +394,7 @@
       }
       ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
       DataOutputStream s = new DataOutputStream(byteValue);
-      newRegion.getRegionInfo().offLine = true;
+      newRegion.getRegionInfo().setOffline(true);
       newRegion.getRegionInfo().write(s);
       long lockid = -1L;
       try {

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java Thu Oct 18 15:21:09 2007
@@ -120,9 +120,9 @@
       fs.mkdirs(merges);
     }
     
-    HRegionInfo newRegionInfo
-      = new HRegionInfo(Math.abs(rand.nextLong()), tabledesc, startKey, endKey);
-    Path newRegionDir = HRegion.getRegionDir(merges, newRegionInfo.regionName);
+    HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
+    Path newRegionDir =
+      HRegion.getRegionDir(merges, newRegionInfo.getEncodedName());
     if(fs.exists(newRegionDir)) {
       throw new IOException("Cannot merge; target file collision at " +
         newRegionDir);
@@ -138,7 +138,8 @@
     for (Map.Entry<Text, Vector<HStoreFile>> es : byFamily.entrySet()) {
       Text colFamily = es.getKey();
       Vector<HStoreFile> srcFiles = es.getValue();
-      HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
+      HStoreFile dst =
+        new HStoreFile(conf, merges, newRegionInfo.getEncodedName(),
         colFamily, Math.abs(rand.nextLong()));
       dst.mergeStoreFiles(srcFiles, fs, conf);
     }
@@ -253,7 +254,8 @@
 
     // Declare the regionName.  This is a unique string for the region, used to 
     // build a unique filename.
-    this.regiondir = HRegion.getRegionDir(rootDir, this.regionInfo.regionName);
+    this.regiondir =
+      HRegion.getRegionDir(rootDir, this.regionInfo.getEncodedName());
     Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
 
     // Move prefab HStore files into place (if any).  This picks up split files
@@ -265,10 +267,10 @@
     // Load in all the HStores.
     long maxSeqId = -1;
     for(Map.Entry<Text, HColumnDescriptor> e :
-        this.regionInfo.tableDesc.families().entrySet()) {
+        this.regionInfo.getTableDesc().families().entrySet()) {
       Text colFamily = HStoreKey.extractFamily(e.getKey());
       
-      HStore store = new HStore(rootDir, this.regionInfo.regionName, 
+      HStore store = new HStore(rootDir, this.regionInfo.getEncodedName(), 
           e.getValue(), fs, oldLogFile, conf); 
       
       stores.put(colFamily, store);
@@ -306,7 +308,7 @@
 
     // HRegion is ready to go!
     this.writestate.compacting = false;
-    LOG.info("region " + this.regionInfo.regionName + " available");
+    LOG.info("region " + this.regionInfo.getRegionName() + " available");
   }
   
   /**
@@ -317,7 +319,7 @@
     return this.minSequenceId;
   }
 
-  /** Returns a HRegionInfo object for this region */
+  /** @return a HRegionInfo object for this region */
   public HRegionInfo getRegionInfo() {
     return this.regionInfo;
   }
@@ -360,7 +362,7 @@
    */
   Vector<HStoreFile> close(boolean abort) throws IOException {
     if (isClosed()) {
-      LOG.info("region " + this.regionInfo.regionName + " already closed");
+      LOG.info("region " + this.regionInfo.getRegionName() + " already closed");
       return null;
     }
     lock.obtainWriteLock();
@@ -393,7 +395,7 @@
         result.addAll(store.close());
       }
       this.closed.set(true);
-      LOG.info("closed " + this.regionInfo.regionName);
+      LOG.info("closed " + this.regionInfo.getRegionName());
       return result;
     } finally {
       lock.releaseWriteLock();
@@ -416,15 +418,15 @@
     checkMidKey(midKey);
     long startTime = System.currentTimeMillis();
     Path splits = getSplitsDir();
-    HRegionInfo regionAInfo = new HRegionInfo(Math.abs(rand.nextLong()),
-      this.regionInfo.tableDesc, this.regionInfo.startKey, midKey);
-    Path dirA = getSplitRegionDir(splits, regionAInfo.regionName);
+    HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
+        this.regionInfo.getStartKey(), midKey);
+    Path dirA = getSplitRegionDir(splits, regionAInfo.getEncodedName());
     if(fs.exists(dirA)) {
       throw new IOException("Cannot split; target file collision at " + dirA);
     }
-    HRegionInfo regionBInfo = new HRegionInfo(Math.abs(rand.nextLong()),
-      this.regionInfo.tableDesc, midKey, null);
-    Path dirB = getSplitRegionDir(splits, regionBInfo.regionName);
+    HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
+        midKey, null);
+    Path dirB = getSplitRegionDir(splits, regionBInfo.getEncodedName());
     if(this.fs.exists(dirB)) {
       throw new IOException("Cannot split; target file collision at " + dirB);
     }
@@ -455,18 +457,18 @@
     for(HStoreFile h: hstoreFilesToSplit) {
       // A reference to the bottom half of the hsf store file.
       HStoreFile.Reference aReference = new HStoreFile.Reference(
-        getRegionName(), h.getFileId(), new HStoreKey(midKey),
+        this.regionInfo.getEncodedName(), h.getFileId(), new HStoreKey(midKey),
         HStoreFile.Range.bottom);
       HStoreFile a = new HStoreFile(this.conf, splits,
-        regionAInfo.regionName, h.getColFamily(), Math.abs(rand.nextLong()),
-        aReference);
+          regionAInfo.getEncodedName(), h.getColFamily(),
+          Math.abs(rand.nextLong()), aReference);
       // Reference to top half of the hsf store file.
       HStoreFile.Reference bReference = new HStoreFile.Reference(
-        getRegionName(), h.getFileId(), new HStoreKey(midKey),
+        this.regionInfo.getEncodedName(), h.getFileId(), new HStoreKey(midKey),
         HStoreFile.Range.top);
       HStoreFile b = new HStoreFile(this.conf, splits,
-        regionBInfo.regionName, h.getColFamily(), Math.abs(rand.nextLong()),
-        bReference);
+        regionBInfo.getEncodedName(), h.getColFamily(),
+        Math.abs(rand.nextLong()), bReference);
       h.splitStoreFile(a, b, this.fs);
     }
 
@@ -482,25 +484,25 @@
       LOG.debug("Cleaned up " + splits.toString() + " " + deleted);
     }
     HRegion regions[] = new HRegion [] {regionA, regionB};
-    LOG.info("Region split of " + this.regionInfo.regionName + " complete; " +
-      "new regions: " + regions[0].getRegionName() + ", " +
-      regions[1].getRegionName() + ". Split took " +
-      StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
+    LOG.info("Region split of " + this.regionInfo.getRegionName() +
+        " complete; " + "new regions: " + regions[0].getRegionName() + ", " +
+        regions[1].getRegionName() + ". Split took " +
+        StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
     return regions;
   }
   
   private void checkMidKey(final Text midKey) throws IOException {
-    if(((this.regionInfo.startKey.getLength() != 0)
-        && (this.regionInfo.startKey.compareTo(midKey) > 0))
-        || ((this.regionInfo.endKey.getLength() != 0)
-            && (this.regionInfo.endKey.compareTo(midKey) < 0))) {
+    if(((this.regionInfo.getStartKey().getLength() != 0)
+        && (this.regionInfo.getStartKey().compareTo(midKey) > 0))
+        || ((this.regionInfo.getEndKey().getLength() != 0)
+            && (this.regionInfo.getEndKey().compareTo(midKey) < 0))) {
       throw new IOException("Region splitkey must lie within region " +
         "boundaries.");
     }
   }
   
-  private Path getSplitRegionDir(final Path splits, final Text regionName) {
-    return HRegion.getRegionDir(splits, regionName);
+  private Path getSplitRegionDir(final Path splits, final String region) {
+    return HRegion.getRegionDir(splits, region);
   }
   
   private Path getSplitsDir() throws IOException {
@@ -517,22 +519,22 @@
 
   /** @return start key for region */
   public Text getStartKey() {
-    return this.regionInfo.startKey;
+    return this.regionInfo.getStartKey();
   }
 
   /** @return end key for region */
   public Text getEndKey() {
-    return this.regionInfo.endKey;
+    return this.regionInfo.getEndKey();
   }
 
   /** @return region id */
   public long getRegionId() {
-    return this.regionInfo.regionId;
+    return this.regionInfo.getRegionId();
   }
 
   /** @return region name */
   public Text getRegionName() {
-    return this.regionInfo.regionName;
+    return this.regionInfo.getRegionName();
   }
 
   /** @return root directory path */
@@ -542,7 +544,7 @@
 
   /** @return HTableDescriptor for this region */
   public HTableDescriptor getTableDesc() {
-    return this.regionInfo.tableDesc;
+    return this.regionInfo.getTableDesc();
   }
 
   /** @return HLog in use for this region */
@@ -788,7 +790,7 @@
     if(!shouldFlush) {
       if(LOG.isDebugEnabled()) {
         LOG.debug("NOT flushing memcache for region " +
-          this.regionInfo.regionName);
+          this.regionInfo.getRegionName());
       }
       return;  
     }
@@ -837,7 +839,7 @@
     if(LOG.isDebugEnabled()) {
       startTime = System.currentTimeMillis();
       LOG.debug("Started memcache flush for region " +
-        this.regionInfo.regionName + ". Size " +
+        this.regionInfo.getRegionName() + ". Size " +
         StringUtils.humanReadableInt(this.memcache.getSize()));
     }
 
@@ -866,7 +868,7 @@
       long logCacheFlushId = retval.sequenceId;
       if(LOG.isDebugEnabled()) {
         LOG.debug("Snapshotted memcache for region " +
-            this.regionInfo.regionName + " with sequence id " +
+            this.regionInfo.getRegionName() + " with sequence id " +
             retval.sequenceId + " and entries " +
             retval.memcacheSnapshot.size());
       }
@@ -893,8 +895,8 @@
       //     This tells future readers that the HStores were emitted correctly,
       //     and that all updates to the log for this regionName that have lower 
       //     log-sequence-ids can be safely ignored.
-      this.log.completeCacheFlush(this.regionInfo.regionName,
-          regionInfo.tableDesc.getName(), logCacheFlushId);
+      this.log.completeCacheFlush(this.regionInfo.getRegionName(),
+          regionInfo.getTableDesc().getName(), logCacheFlushId);
 
     } finally {
       // C. Delete the now-irrelevant memcache snapshot; its contents have been 
@@ -909,7 +911,7 @@
     }
     if (LOG.isDebugEnabled()) {
       LOG.debug("Finished memcache flush for region " +
-        this.regionInfo.regionName + " in " +
+        this.regionInfo.getRegionName() + " in " +
           (System.currentTimeMillis() - startTime) + "ms");
     }
   }
@@ -1207,7 +1209,7 @@
    * @throws IOException
    */
   public void put(long lockid, Text targetCol, byte [] val) throws IOException {
-    if (HGlobals.deleteBytes.compareTo(val) == 0) {
+    if (HLogEdit.isDeleted(val)) {
       throw new IOException("Cannot insert value: " + val);
     }
     localput(lockid, targetCol, val);
@@ -1221,7 +1223,7 @@
    * @throws IOException
    */
   public void delete(long lockid, Text targetCol) throws IOException {
-    localput(lockid, targetCol, HGlobals.deleteBytes.get());
+    localput(lockid, targetCol, HLogEdit.deleteBytes.get());
   }
   
   /**
@@ -1258,7 +1260,7 @@
         List<HStoreKey> keys = getKeys(origin, versions);
         if (keys.size() > 0) {
           TreeMap<Text, byte []> edits = new TreeMap<Text, byte []>();
-          edits.put(column, HGlobals.deleteBytes.get());
+          edits.put(column, HLogEdit.deleteBytes.get());
           for (HStoreKey key: keys) {
             update(row, key.getTimestamp(), edits);
           }
@@ -1400,7 +1402,7 @@
       // Run updates one at a time so we can supply appropriate timestamp
       long now = System.currentTimeMillis();
       for (Map.Entry<Text, byte []>e: updatesByColumn.entrySet()) {
-        if (HGlobals.deleteBytes.equals(e.getValue())) {
+        if (HLogEdit.isDeleted(e.getValue())) {
           // Its a delete.  Delete latest.  deleteMultiple calls update for us.
           // Actually regets the row lock but since we already have it, should
           // be fine.
@@ -1431,8 +1433,8 @@
     if (updatesByColumn == null || updatesByColumn.size() <= 0) {
       return;
     }
-    this.log.append(regionInfo.regionName, regionInfo.tableDesc.getName(),
-        row, updatesByColumn, timestamp);
+    this.log.append(regionInfo.getRegionName(),
+        regionInfo.getTableDesc().getName(), row, updatesByColumn, timestamp);
     this.memcache.add(row, updatesByColumn, timestamp);
   }
 
@@ -1442,17 +1444,17 @@
 
   /** Make sure this is a valid row for the HRegion */
   void checkRow(Text row) throws IOException {
-    if(((regionInfo.startKey.getLength() == 0)
-        || (regionInfo.startKey.compareTo(row) <= 0))
-        && ((regionInfo.endKey.getLength() == 0)
-            || (regionInfo.endKey.compareTo(row) > 0))) {
+    if(((regionInfo.getStartKey().getLength() == 0)
+        || (regionInfo.getStartKey().compareTo(row) <= 0))
+        && ((regionInfo.getEndKey().getLength() == 0)
+            || (regionInfo.getEndKey().compareTo(row) > 0))) {
       // all's well
       
     } else {
       throw new WrongRegionException("Requested row out of range for " +
-        "HRegion " + regionInfo.regionName + ", startKey='" +
-        regionInfo.startKey + "', endKey='" + regionInfo.endKey + "', row='" +
-        row + "'");
+        "HRegion " + regionInfo.getRegionName() + ", startKey='" +
+        regionInfo.getStartKey() + "', getEndKey()='" + regionInfo.getEndKey() +
+        "', row='" + row + "'");
     }
   }
   
@@ -1463,10 +1465,10 @@
    */
   void checkColumn(Text columnName) throws IOException {
     Text family = new Text(HStoreKey.extractFamily(columnName) + ":");
-    if(! regionInfo.tableDesc.hasFamily(family)) {
+    if(! regionInfo.getTableDesc().hasFamily(family)) {
       throw new IOException("Requested column family " + family 
-          + " does not exist in HRegion " + regionInfo.regionName
-          + " for table " + regionInfo.tableDesc.getName());
+          + " does not exist in HRegion " + regionInfo.getRegionName()
+          + " for table " + regionInfo.getTableDesc().getName());
     }
   }
 
@@ -1615,14 +1617,17 @@
       }
     }
 
+    /** @return true if the scanner is a wild card scanner */
     public boolean isWildcardScanner() {
       return wildcardMatch;
     }
-    
+
+    /** @return true if the scanner is a multiple match scanner */
     public boolean isMultipleMatchScanner() {
       return multipleMatchers;
     }
 
+    /** {@inheritDoc} */
     public boolean next(HStoreKey key, SortedMap<Text, byte[]> results)
     throws IOException {
       // Filtered flag is set by filters.  If a cell has been 'filtered out'
@@ -1690,7 +1695,7 @@
                 key.getTimestamp());
               for (Map.Entry<Text, byte[]> e : resultSets[i].entrySet()) {
                 hsk.setColumn(e.getKey());
-                if (HGlobals.deleteBytes.equals(e.getValue())) {
+                if (HLogEdit.isDeleted(e.getValue())) {
                   if (!deletes.contains(hsk)) {
                     // Key changes as we cycle the for loop so add a copy to
                     // the set of deletes.
@@ -1817,10 +1822,9 @@
    * 
    * @throws IOException
    */
-  static HRegion createHRegion(final HRegionInfo info,
-    final Path rootDir, final Configuration conf, final Path initialFiles)
-  throws IOException {
-    Path regionDir = HRegion.getRegionDir(rootDir, info.regionName);
+  static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
+      final Configuration conf, final Path initialFiles) throws IOException {
+    Path regionDir = HRegion.getRegionDir(rootDir, info.getEncodedName());
     FileSystem fs = FileSystem.get(conf);
     fs.mkdirs(regionDir);
     return new HRegion(rootDir,
@@ -1851,13 +1855,13 @@
    * 
    * @param fs the file system object
    * @param baseDirectory base directory for HBase
-   * @param regionName name of the region to delete
+   * @param name region file name
    * @throws IOException
    * @return True if deleted.
    */
-  static boolean deleteRegion(FileSystem fs, Path baseDirectory,
-      Text regionName) throws IOException {
-    Path p = HRegion.getRegionDir(fs.makeQualified(baseDirectory), regionName);
+  static boolean deleteRegion(FileSystem fs, Path baseDirectory, String name)
+    throws IOException {
+    Path p = HRegion.getRegionDir(fs.makeQualified(baseDirectory), name);
     return fs.delete(p);
   }
 
@@ -1865,10 +1869,10 @@
    * Computes the Path of the HRegion
    * 
    * @param dir hbase home directory
-   * @param regionName name of the region
+   * @param name region file name
    * @return Path of HRegion directory
    */
-  public static Path getRegionDir(final Path dir, final Text regionName) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
+  public static Path getRegionDir(final Path dir, final String name) {
+    return new Path(dir, new Path(HREGIONDIR_PREFIX + name));
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java Thu Oct 18 15:21:09 2007
@@ -26,6 +26,8 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 
+import org.apache.hadoop.hbase.util.Base64;
+
 /**
  * HRegion information.
  * Contains HRegion id, start and end keys, a reference to this
@@ -33,7 +35,15 @@
  */
 public class HRegionInfo implements WritableComparable {
   /** delimiter used between portions of a region name */
-  public static final char DELIMITER = ',';
+  public static final String DELIMITER = ",";
+
+  /** HRegionInfo for root region */
+  public static final HRegionInfo rootRegionInfo =
+    new HRegionInfo(0L, HTableDescriptor.rootTableDesc);
+
+  /** HRegionInfo for first meta region */
+  public static final HRegionInfo firstMetaRegionInfo =
+    new HRegionInfo(1L, HTableDescriptor.metaTableDesc);
   
   /**
    * Extracts table name prefix from a region name.
@@ -42,61 +52,92 @@
    * @return The table prefix of a region name.
    */
   public static Text getTableNameFromRegionName(final Text regionName) {
-    int index = -1;
-    byte [] bytes = regionName.getBytes();
-    for (int i = 0; i < bytes.length; i++) {
-      if (((char) bytes[i]) == DELIMITER) {
-        index = i;
-        break;
-      }
-    }
-    if (index == -1) {
+    int offset = regionName.find(DELIMITER);
+    if (offset == -1) {
       throw new IllegalArgumentException(regionName.toString() + " does not " +
-        "contain " + DELIMITER + " character");
+        "contain '" + DELIMITER + "' character");
     }
-    byte [] tableName = new byte[index];
-    System.arraycopy(bytes, 0, tableName, 0, index);
+    byte [] tableName = new byte[offset];
+    System.arraycopy(regionName.getBytes(), 0, tableName, 0, offset);
     return new Text(tableName);
   }
-
-  Text regionName;
-  long regionId;
-  Text startKey;
-  Text endKey;
-  boolean offLine;
-  boolean split;
-  HTableDescriptor tableDesc;
   
+  /**
+   * Converts an encoded region name to its unencoded form
+   * 
+   * @param encodedName 
+   * @return unencoded region name
+   */
+  public static Text decodeRegionName(String encodedName) {
+    int offset = encodedName.indexOf(DELIMITER);
+    if (offset == -1) {
+      throw new IllegalArgumentException(
+          "encoded region name does not contain '" + DELIMITER + "': " +
+          encodedName);
+    }
+    String regionName = encodedName.substring(0, offset++);
+    String remainder = encodedName.substring(offset);
+    offset = remainder.indexOf(DELIMITER);
+    if (offset == -1) {
+      throw new IllegalArgumentException(
+          "improperly formatted encoded region name " + encodedName);
+    }
+    Text startKey = new Text();
+    if (offset != 0) {
+      startKey.set(Base64.decode(remainder.substring(0, offset), Base64.ORDERED));
+    }
+    offset += 1;
+    return new Text(regionName + DELIMITER + startKey.toString() + DELIMITER +
+      remainder.substring(offset));
+  }
+
+  private Text endKey;
+  private boolean offLine;
+  private long regionId;
+  private Text regionName;
+  private boolean split;
+  private Text startKey;
+  private HTableDescriptor tableDesc;
+
+  /** Used to construct the HRegionInfo for the root and first meta regions */
+  private HRegionInfo(long regionId, HTableDescriptor tableDesc) {
+    this.regionId = regionId;
+    this.tableDesc = tableDesc;
+    this.endKey = new Text();
+    this.offLine = false;
+    this.regionName = new Text(tableDesc.getName().toString() + DELIMITER +
+        DELIMITER + regionId);
+    this.split = false;
+    this.startKey = new Text();
+  }
+
   /** Default constructor - creates empty object */
   public HRegionInfo() {
-    this.regionId = 0;
-    this.tableDesc = new HTableDescriptor();
-    this.startKey = new Text();
     this.endKey = new Text();
-    this.regionName = new Text();
     this.offLine = false;
+    this.regionId = 0;
+    this.regionName = new Text();
     this.split = false;
+    this.startKey = new Text();
+    this.tableDesc = new HTableDescriptor();
   }
   
   /**
    * Construct HRegionInfo with explicit parameters
    * 
-   * @param regionId the region id
    * @param tableDesc the table descriptor
    * @param startKey first key in region
    * @param endKey end of key range
    * @throws IllegalArgumentException
    */
-  public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey,
-      Text endKey)
-  throws IllegalArgumentException {
-    this(regionId, tableDesc, startKey, endKey, false);
+  public HRegionInfo(HTableDescriptor tableDesc, Text startKey, Text endKey)
+    throws IllegalArgumentException {
+    this(tableDesc, startKey, endKey, false);
   }
 
   /**
    * Construct HRegionInfo with explicit parameters
    * 
-   * @param regionId the region id
    * @param tableDesc the table descriptor
    * @param startKey first key in region
    * @param endKey end of key range
@@ -104,34 +145,33 @@
    * regions that may or may not hold references to this region.
    * @throws IllegalArgumentException
    */
-  public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey,
-      Text endKey, final boolean split)
-  throws IllegalArgumentException {
-    
-    this.regionId = regionId;
-    
+  public HRegionInfo(HTableDescriptor tableDesc, Text startKey, Text endKey,
+      final boolean split) throws IllegalArgumentException {
+
     if(tableDesc == null) {
       throw new IllegalArgumentException("tableDesc cannot be null");
     }
-    
-    this.tableDesc = tableDesc;
-    
-    this.startKey = new Text();
-    if(startKey != null) {
-      this.startKey.set(startKey);
-    }
-    
+
     this.endKey = new Text();
     if(endKey != null) {
       this.endKey.set(endKey);
     }
     
-    this.regionName = new Text(tableDesc.getName().toString() + DELIMITER +
-      (startKey == null ? "" : startKey.toString()) + DELIMITER +
-      regionId);
-    
     this.offLine = false;
+    this.regionId = System.currentTimeMillis();
+    
+    this.regionName = new Text(tableDesc.getName().toString() + DELIMITER +
+        (startKey == null ? "" : startKey.toString()) + DELIMITER +
+        regionId);
+      
     this.split = split;
+
+    this.startKey = new Text();
+    if(startKey != null) {
+      this.startKey.set(startKey);
+    }
+    
+    this.tableDesc = tableDesc;
   }
   
   /** @return the endKey */
@@ -149,6 +189,16 @@
     return regionName;
   }
 
+  /**
+   * @return the encodedName
+   */
+  public String getEncodedName() {
+    return tableDesc.getName().toString() + DELIMITER +
+    (startKey == null || startKey.getLength() == 0 ? "" : 
+      Base64.encodeBytes(startKey.getBytes(), Base64.ORDERED)) + DELIMITER +
+    regionId;
+  }
+
   /** @return the startKey */
   public Text getStartKey(){
     return startKey;
@@ -167,6 +217,13 @@
   }
   
   /**
+   * @param split set split status
+   */
+  public void setSplit(boolean split) {
+    this.split = split;
+  }
+
+  /**
    * @return True if this region is offline.
    */
   public boolean isOffline() {
@@ -174,6 +231,13 @@
   }
 
   /**
+   * @param offLine set online - offline status
+   */
+  public void setOffline(boolean offLine) {
+    this.offLine = offLine;
+  }
+
+  /**
    * {@inheritDoc}
    */
   @Override
@@ -215,26 +279,26 @@
    * {@inheritDoc}
    */
   public void write(DataOutput out) throws IOException {
-    out.writeLong(regionId);
-    tableDesc.write(out);
-    startKey.write(out);
     endKey.write(out);
-    regionName.write(out);
     out.writeBoolean(offLine);
+    out.writeLong(regionId);
+    regionName.write(out);
     out.writeBoolean(split);
+    startKey.write(out);
+    tableDesc.write(out);
   }
   
   /**
    * {@inheritDoc}
    */
   public void readFields(DataInput in) throws IOException {
-    this.regionId = in.readLong();
-    this.tableDesc.readFields(in);
-    this.startKey.readFields(in);
     this.endKey.readFields(in);
-    this.regionName.readFields(in);
     this.offLine = in.readBoolean();
+    this.regionId = in.readLong();
+    this.regionName.readFields(in);
     this.split = in.readBoolean();
+    this.startKey.readFields(in);
+    this.tableDesc.readFields(in);
   }
   
   //

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java Thu Oct 18 15:21:09 2007
@@ -122,6 +122,8 @@
   // is name of the webapp and the attribute name used stuffing this instance
   // into web context.
   InfoServer infoServer;
+  
+  /** region server process name */
   public static final String REGIONSERVER = "regionserver";
 
   // Check to see if regions should be split
@@ -220,7 +222,7 @@
       // splitting a 'normal' region, and the ROOT table needs to be
       // updated if we are splitting a META region.
       HTable t = null;
-      if (region.getRegionInfo().tableDesc.getName().equals(META_TABLE_NAME)) {
+      if (region.getRegionInfo().getTableDesc().getName().equals(META_TABLE_NAME)) {
         // We need to update the root region
         if (this.root == null) {
           this.root = new HTable(conf, ROOT_TABLE_NAME);
@@ -238,8 +240,8 @@
       // Mark old region as offline and split in META.
       // NOTE: there is no need for retry logic here. HTable does it for us.
       long lockid = t.startUpdate(oldRegionInfo.getRegionName());
-      oldRegionInfo.offLine = true;
-      oldRegionInfo.split = true;
+      oldRegionInfo.setOffline(true);
+      oldRegionInfo.setSplit(true);
       t.put(lockid, COL_REGIONINFO, Writables.getBytes(oldRegionInfo));
       t.put(lockid, COL_SPLITA, Writables.getBytes(
         newRegions[0].getRegionInfo()));
@@ -927,7 +929,7 @@
   }
   
   void openRegion(final HRegionInfo regionInfo) throws IOException {
-    HRegion region = onlineRegions.get(regionInfo.regionName);
+    HRegion region = onlineRegions.get(regionInfo.getRegionName());
     if(region == null) {
       region = new HRegion(new Path(this.conf.get(HConstants.HBASE_DIR)),
         this.log, FileSystem.get(conf), conf, regionInfo, null);
@@ -947,7 +949,7 @@
     this.lock.writeLock().lock();
     HRegion region = null;
     try {
-      region = onlineRegions.remove(hri.regionName);
+      region = onlineRegions.remove(hri.getRegionName());
     } finally {
       this.lock.writeLock().unlock();
     }
@@ -1154,6 +1156,7 @@
   // remote scanner interface
   //
 
+  /** {@inheritDoc} */
   public long openScanner(Text regionName, Text[] cols, Text firstRow,
       final long timestamp, final RowFilterInterface filter)
     throws IOException {
@@ -1284,6 +1287,7 @@
     return Collections.unmodifiableSortedMap(this.onlineRegions);
   }
 
+  /** @return the request count */
   public AtomicInteger getRequestCount() {
     return this.requestCount;
   }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java Thu Oct 18 15:21:09 2007
@@ -72,7 +72,7 @@
   private static final String BLOOMFILTER_FILE_NAME = "filter";
 
   Path dir;
-  Text regionName;
+  String encodedRegionName;
   HColumnDescriptor family;
   Text familyName;
   SequenceFile.CompressionType compression;
@@ -131,24 +131,23 @@
    * file will be deleted (by whoever has instantiated the HStore).
    *
    * @param dir log file directory
-   * @param regionName name of region
+   * @param encodedRegionName filename friendly name of region
    * @param family name of column family
    * @param fs file system object
    * @param reconstructionLog existing log file to apply if any
    * @param conf configuration object
    * @throws IOException
    */
-  HStore(Path dir, Text regionName, HColumnDescriptor family, 
+  HStore(Path dir, String encodedRegionName, HColumnDescriptor family, 
       FileSystem fs, Path reconstructionLog, Configuration conf)
   throws IOException {  
     this.dir = dir;
     this.compactionDir = new Path(dir, "compaction.dir");
-    this.regionName = regionName;
+    this.encodedRegionName = encodedRegionName;
     this.family = family;
     this.familyName = HStoreKey.extractFamily(this.family.getName());
     this.compression = SequenceFile.CompressionType.NONE;
-    this.storeName = this.regionName.toString() + "/" +
-      this.familyName.toString();
+    this.storeName = this.encodedRegionName + "/" + this.familyName.toString();
     
     if(family.getCompression() != HColumnDescriptor.CompressionType.NONE) {
       if(family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) {
@@ -163,15 +162,16 @@
     
     this.fs = fs;
     this.conf = conf;
-    this.mapdir = HStoreFile.getMapDir(dir, regionName, familyName);
+    this.mapdir = HStoreFile.getMapDir(dir, encodedRegionName, familyName);
     fs.mkdirs(mapdir);
-    this.loginfodir = HStoreFile.getInfoDir(dir, regionName, familyName);
+    this.loginfodir = HStoreFile.getInfoDir(dir, encodedRegionName, familyName);
     fs.mkdirs(loginfodir);
     if(family.getBloomFilter() == null) {
       this.filterDir = null;
       this.bloomFilter = null;
     } else {
-      this.filterDir = HStoreFile.getFilterDir(dir, regionName, familyName);
+      this.filterDir =
+        HStoreFile.getFilterDir(dir, encodedRegionName, familyName);
       fs.mkdirs(filterDir);
       loadOrCreateBloomFilter();
     }
@@ -187,8 +187,8 @@
     // MapFiles are in a reliable state.  Every entry in 'mapdir' must have a 
     // corresponding one in 'loginfodir'. Without a corresponding log info
     // file, the entry in 'mapdir' must be deleted.
-    Collection<HStoreFile> hstoreFiles 
-      = HStoreFile.loadHStoreFiles(conf, dir, regionName, familyName, fs);
+    Collection<HStoreFile> hstoreFiles = HStoreFile.loadHStoreFiles(conf, dir,
+        encodedRegionName, familyName, fs);
     for(HStoreFile hsf: hstoreFiles) {
       this.storefiles.put(Long.valueOf(hsf.loadInfo(fs)), hsf);
     }
@@ -265,6 +265,7 @@
     SequenceFile.Reader login =
       new SequenceFile.Reader(this.fs, reconstructionLog, this.conf);
     try {
+      Text thisRegionName = HRegionInfo.decodeRegionName(encodedRegionName);
       HLogKey key = new HLogKey();
       HLogEdit val = new HLogEdit();
       while (login.next(key, val)) {
@@ -281,13 +282,13 @@
         // METACOLUMN info such as HBASE::CACHEFLUSH entries
         Text column = val.getColumn();
         if (column.equals(HLog.METACOLUMN)
-            || !key.getRegionName().equals(this.regionName)
+            || !key.getRegionName().equals(thisRegionName)
             || !HStoreKey.extractFamily(column).equals(this.familyName)) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("Passing on edit " + key.getRegionName() + ", " +
                 column.toString() + ": " + 
                 new String(val.getVal(), UTF8_ENCODING) +
-                ", my region: " + this.regionName + ", my column: " +
+                ", my region: " + thisRegionName + ", my column: " +
                 this.familyName);
           }
           continue;
@@ -452,7 +453,7 @@
     synchronized(flushLock) {
       // A. Write the TreeMap out to the disk
       HStoreFile flushedFile = HStoreFile.obtainNewHStoreFile(conf, dir,
-        regionName, familyName, fs);
+        encodedRegionName, familyName, fs);
       String name = flushedFile.toString();
       MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression,
         this.bloomFilter);
@@ -575,8 +576,8 @@
   throws IOException {
     long maxId = maxSeenSeqID;
     synchronized(compactLock) {
-      Path curCompactStore =
-        HStoreFile.getHStoreDir(this.compactionDir, regionName, familyName);
+      Path curCompactStore = HStoreFile.getHStoreDir(this.compactionDir,
+          encodedRegionName, familyName);
       if(LOG.isDebugEnabled()) {
         LOG.debug("started compaction of " + storefiles.size() + " files in " +
           curCompactStore.toString());
@@ -590,8 +591,8 @@
       }
       try {
         List<HStoreFile> toCompactFiles = getFilesToCompact();
-        HStoreFile compactedOutputFile =
-          new HStoreFile(conf, this.compactionDir, regionName, familyName, -1);
+        HStoreFile compactedOutputFile = new HStoreFile(conf,
+            this.compactionDir, encodedRegionName, familyName, -1);
         if (toCompactFiles.size() < 1 ||
             (toCompactFiles.size() == 1 &&
               !toCompactFiles.get(0).isReference())) {
@@ -943,7 +944,7 @@
       // If a null value, shouldn't be in here.  Mark it as deleted cell.
       return true;
     }
-    if (!HGlobals.deleteBytes.equals(value)) {
+    if (!HLogEdit.isDeleted(value)) {
       return false;
     }
     // Cell has delete value.  Save it into deletes.
@@ -981,8 +982,8 @@
    */
   void processReadyCompaction() throws IOException {
     // 1. Acquiring the write-lock
-    Path curCompactStore =
-      HStoreFile.getHStoreDir(this.compactionDir, regionName, familyName);
+    Path curCompactStore = HStoreFile.getHStoreDir(this.compactionDir,
+        encodedRegionName, familyName);
     this.lock.obtainWriteLock();
     try {
       Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
@@ -1010,12 +1011,12 @@
       }
 
       // 3. Moving the new MapFile into place.
-      HStoreFile compactedFile 
-        = new HStoreFile(conf, this.compactionDir, regionName, familyName, -1);
+      HStoreFile compactedFile = new HStoreFile(conf, this.compactionDir,
+          encodedRegionName, familyName, -1);
       // obtainNewHStoreFile does its best to generate a filename that does not
       // currently exist.
-      HStoreFile finalCompactedFile 
-        = HStoreFile.obtainNewHStoreFile(conf, dir, regionName, familyName, fs);
+      HStoreFile finalCompactedFile = HStoreFile.obtainNewHStoreFile(conf, dir,
+          encodedRegionName, familyName, fs);
       if(LOG.isDebugEnabled()) {
         LOG.debug("moving " + compactedFile.toString() + " in " +
             this.compactionDir.toString() +
@@ -1093,7 +1094,7 @@
             Text readcol = readkey.getColumn();
             if (results.get(readcol) == null
                 && key.matchesWithoutColumn(readkey)) {
-              if(readval.equals(HGlobals.deleteBytes)) {
+              if(HLogEdit.isDeleted(readval.get())) {
                 break;
               }
               results.put(new Text(readcol), readval.get());