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 [2/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/

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java Thu Oct 18 15:21:09 2007
@@ -118,7 +118,7 @@
   private static Random rand = new Random();
 
   private Path dir;
-  private Text regionName;
+  private String encodedRegionName;
   private Text colFamily;
   private long fileId;
   private final Configuration conf;
@@ -126,7 +126,7 @@
 
   /** Shutdown constructor used by Writable */
   HStoreFile(Configuration conf) {
-    this(conf, new Path(Path.CUR_DIR), new Text(), new Text(), 0);
+    this(conf, new Path(Path.CUR_DIR), "", new Text(), 0);
   }
   
   /**
@@ -137,25 +137,25 @@
    * @param colFamily name of the column family
    * @param fileId file identifier
    */
-  HStoreFile(final Configuration conf, final Path dir, final Text regionName, 
-      final Text colFamily, final long fileId) {
-    this(conf, dir, regionName, colFamily, fileId, null);
+  HStoreFile(final Configuration conf, final Path dir, 
+      final String encodedRegionName, final Text colFamily, final long fileId) {
+    this(conf, dir, encodedRegionName, colFamily, fileId, null);
   }
 
   /**
    * Constructor that fully initializes the object
    * @param conf Configuration object
    * @param dir directory path
-   * @param regionName name of the region
+   * @param encodedRegionName file name friendly name of the region
    * @param colFamily name of the column family
    * @param fileId file identifier
    * @param ref Reference to another HStoreFile.
    */
-  HStoreFile(Configuration conf, Path dir, Text regionName, 
+  HStoreFile(Configuration conf, Path dir, String encodedRegionName, 
       Text colFamily, long fileId, final Reference ref) {
     this.conf = conf;
     this.dir = dir;
-    this.regionName = new Text(regionName);
+    this.encodedRegionName = encodedRegionName;
     this.colFamily = new Text(colFamily);
     this.fileId = fileId;
     // If a reference, construction does not write the pointer files.  Thats
@@ -168,14 +168,14 @@
    * Data structure to hold reference to a store file over in another region.
    */
   static class Reference implements Writable {
-    Text regionName;
-    long fileid;
-    Range region;
-    HStoreKey midkey;
+    private String encodedRegionName;
+    private long fileid;
+    private Range region;
+    private HStoreKey midkey;
     
-    Reference(final Text rn, final long fid, final HStoreKey m,
+    Reference(final String ern, final long fid, final HStoreKey m,
         final Range fr) {
-      this.regionName = rn;
+      this.encodedRegionName = ern;
       this.fileid = fid;
       this.region = fr;
       this.midkey = m;
@@ -197,21 +197,21 @@
       return this.midkey;
     }
     
-    Text getRegionName() {
-      return this.regionName;
+    String getEncodedRegionName() {
+      return this.encodedRegionName;
     }
    
     /** {@inheritDoc} */
     @Override
     public String toString() {
-      return this.regionName + "/" + this.fileid + "/" + this.region;
+      return this.encodedRegionName + "/" + this.fileid + "/" + this.region;
     }
 
     // Make it serializable.
 
     /** {@inheritDoc} */
     public void write(DataOutput out) throws IOException {
-      this.regionName.write(out);
+      out.writeUTF(this.encodedRegionName);
       out.writeLong(this.fileid);
       // Write true if we're doing top of the file.
       out.writeBoolean(isTopFileRegion(this.region));
@@ -220,8 +220,7 @@
 
     /** {@inheritDoc} */
     public void readFields(DataInput in) throws IOException {
-      this.regionName = new Text();
-      this.regionName.readFields(in);
+      this.encodedRegionName = in.readUTF();
       this.fileid = in.readLong();
       boolean tmp = in.readBoolean();
       // If true, set region to top.
@@ -244,8 +243,8 @@
     return this.reference;
   }
 
-  Text getRegionName() {
-    return this.regionName;
+  String getEncodedRegionName() {
+    return this.encodedRegionName;
   }
 
   /** @return the column family */
@@ -262,43 +261,45 @@
   /** @return path for MapFile */
   Path getMapFilePath() {
     return isReference()?
-      getMapFilePath(this.regionName, this.fileId,
-        this.reference.getRegionName()):
-      getMapFilePath(this.regionName, this.fileId);
+      getMapFilePath(this.encodedRegionName, this.fileId,
+        this.reference.getEncodedRegionName()):
+      getMapFilePath(this.encodedRegionName, this.fileId);
   }
 
   private Path getMapFilePath(final Reference r) {
     return r == null?
       getMapFilePath():
-      getMapFilePath(r.getRegionName(), r.getFileId());
+      getMapFilePath(r.getEncodedRegionName(), r.getFileId());
   }
 
-  private Path getMapFilePath(final Text name, final long fid) {
-    return new Path(HStoreFile.getMapDir(dir, name, colFamily), 
+  private Path getMapFilePath(final String encodedName, final long fid) {
+    return new Path(HStoreFile.getMapDir(dir, encodedName, colFamily), 
       createHStoreFilename(fid, null));
   }
   
-  private Path getMapFilePath(final Text name, final long fid, final Text rn) {
-    return new Path(HStoreFile.getMapDir(dir, name, colFamily), 
-      createHStoreFilename(fid, rn));
+  private Path getMapFilePath(final String encodedName, final long fid,
+      final String ern) {
+    return new Path(HStoreFile.getMapDir(dir, encodedName, colFamily), 
+      createHStoreFilename(fid, ern));
   }
 
   /** @return path for info file */
   Path getInfoFilePath() {
     return isReference()?
-      getInfoFilePath(this.regionName, this.fileId,
-        this.reference.getRegionName()):
-      getInfoFilePath(this.regionName, this.fileId);
+      getInfoFilePath(this.encodedRegionName, this.fileId,
+        this.reference.getEncodedRegionName()):
+      getInfoFilePath(this.encodedRegionName, this.fileId);
   }
   
-  private Path getInfoFilePath(final Text name, final long fid) {
-    return new Path(HStoreFile.getInfoDir(dir, name, colFamily), 
+  private Path getInfoFilePath(final String encodedName, final long fid) {
+    return new Path(HStoreFile.getInfoDir(dir, encodedName, colFamily), 
       createHStoreFilename(fid, null));
   }
   
-  private Path getInfoFilePath(final Text name, final long fid, final Text rn) {
-    return new Path(HStoreFile.getInfoDir(dir, name, colFamily), 
-      createHStoreFilename(fid, rn));
+  private Path getInfoFilePath(final String encodedName, final long fid,
+      final String ern) {
+    return new Path(HStoreFile.getInfoDir(dir, encodedName, colFamily), 
+      createHStoreFilename(fid, ern));
   }
 
   // Static methods to build partial paths to internal directories.  Useful for 
@@ -308,35 +309,35 @@
   }
   
   private static String createHStoreFilename(final long fid,
-      final Text regionName) {
+      final String encodedRegionName) {
     return Long.toString(fid) +
-      ((regionName != null)? "." + regionName.toString(): "");
+      ((encodedRegionName != null) ? "." + encodedRegionName : "");
   }
   
   private static String createHStoreInfoFilename(final long fid) {
     return createHStoreFilename(fid, null);
   }
   
-  static Path getMapDir(Path dir, Text regionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
+  static Path getMapDir(Path dir, String encodedRegionName, Text colFamily) {
+    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName, 
         new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
   }
 
   /** @return the info directory path */
-  static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
+  static Path getInfoDir(Path dir, String encodedRegionName, Text colFamily) {
+    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName, 
         new Path(colFamily.toString(), HSTORE_INFO_DIR)));
   }
 
   /** @return the bloom filter directory path */
-  static Path getFilterDir(Path dir, Text regionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
+  static Path getFilterDir(Path dir, String encodedRegionName, Text colFamily) {
+    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName,
         new Path(colFamily.toString(), HSTORE_FILTER_DIR)));
   }
 
   /** @return the HStore directory path */
-  static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
+  static Path getHStoreDir(Path dir, String encodedRegionName, Text colFamily) {
+    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName, 
         colFamily.toString()));
   }
 
@@ -347,9 +348,10 @@
    * will keep generating names until it generates a name that does not exist.
    */
   static HStoreFile obtainNewHStoreFile(Configuration conf, Path dir, 
-      Text regionName, Text colFamily, FileSystem fs) throws IOException {
+      String encodedRegionName, Text colFamily, FileSystem fs)
+      throws IOException {
     
-    Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
+    Path mapdir = HStoreFile.getMapDir(dir, encodedRegionName, colFamily);
     long fileId = Math.abs(rand.nextLong());
 
     Path testpath1 = new Path(mapdir, createHStoreFilename(fileId));
@@ -359,7 +361,7 @@
       testpath1 = new Path(mapdir, createHStoreFilename(fileId));
       testpath2 = new Path(mapdir, createHStoreInfoFilename(fileId));
     }
-    return new HStoreFile(conf, dir, regionName, colFamily, fileId);
+    return new HStoreFile(conf, dir, encodedRegionName, colFamily, fileId);
   }
 
   /*
@@ -376,11 +378,11 @@
    * @throws IOException
    */
   static Vector<HStoreFile> loadHStoreFiles(Configuration conf, Path dir, 
-      Text regionName, Text colFamily, FileSystem fs)
+      String encodedRegionName, Text colFamily, FileSystem fs)
   throws IOException {
     // Look first at info files.  If a reference, these contain info we need
     // to create the HStoreFile.
-    Path infodir = HStoreFile.getInfoDir(dir, regionName, colFamily);
+    Path infodir = HStoreFile.getInfoDir(dir, encodedRegionName, colFamily);
     Path infofiles[] = fs.listPaths(new Path[] {infodir});
     Vector<HStoreFile> results = new Vector<HStoreFile>(infofiles.length);
     Vector<Path> mapfiles = new Vector<Path>(infofiles.length);
@@ -392,10 +394,10 @@
       HStoreFile curfile = null;
       if (isReference) {
         Reference reference = readSplitInfo(infofiles[i], fs);
-        curfile = new HStoreFile(conf, dir, regionName, colFamily, fid,
+        curfile = new HStoreFile(conf, dir, encodedRegionName, colFamily, fid,
           reference);
       } else {
-        curfile = new HStoreFile(conf, dir, regionName, colFamily, fid);
+        curfile = new HStoreFile(conf, dir, encodedRegionName, colFamily, fid);
       }
       Path mapfile = curfile.getMapFilePath();
       if (!fs.exists(mapfile)) {
@@ -415,7 +417,7 @@
       mapfiles.add(qualified);
     }
     
-    Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
+    Path mapdir = HStoreFile.getMapDir(dir, encodedRegionName, colFamily);
     // List paths by experience returns fully qualified names -- at least when
     // running on a mini hdfs cluster.
     Path datfiles[] = fs.listPaths(new Path[] {mapdir});
@@ -486,14 +488,13 @@
    * @param fs
    * @throws IOException
    */
-  private void writeSplitInfo(final FileSystem fs)
-  throws IOException {
+  private void writeSplitInfo(final FileSystem fs) throws IOException {
     Path p = getInfoFilePath();
     if (fs.exists(p)) {
       throw new IOException("File already exists " + p.toString());
     }
     FSDataOutputStream out = fs.create(p);
-    getReference().getRegionName().write(out);
+    out.writeUTF(getReference().getEncodedRegionName());
     getReference().getMidkey().write(out);
     out.writeLong(getReference().getFileId());
     out.writeBoolean(isTopFileRegion(getReference().getFileRegion()));
@@ -506,8 +507,7 @@
   static Reference readSplitInfo(final Path p, final FileSystem fs)
   throws IOException {
     FSDataInputStream in = fs.open(p);
-    Text rn = new Text();
-    rn.readFields(in);
+    String rn = in.readUTF();
     HStoreKey midkey = new HStoreKey();
     midkey.readFields(in);
     long fid = in.readLong();
@@ -580,10 +580,10 @@
    * @throws IOException
    */
   long loadInfo(FileSystem fs) throws IOException {
-    Path p = isReference()?
-      getInfoFilePath(this.reference.getRegionName(),
-        this.reference.getFileId()):
-      getInfoFilePath();
+    Path p = isReference() ?
+        getInfoFilePath(this.reference.getEncodedRegionName(),
+            this.reference.getFileId()) :
+              getInfoFilePath();
     DataInputStream in = new DataInputStream(fs.open(p));
     try {
       byte flag = in.readByte();
@@ -930,7 +930,7 @@
   throws IOException {
     return isReference()?
       new HStoreFile.HalfMapFileReader(fs,
-        getMapFilePath(getReference().getRegionName(),
+        getMapFilePath(getReference().getEncodedRegionName(),
           getReference().getFileId()).toString(),
         this.conf, getReference().getFileRegion(), getReference().getMidkey(),
         bloomFilter):
@@ -975,7 +975,7 @@
   /** {@inheritDoc} */
   @Override
   public String toString() {
-    return this.regionName.toString() + "/" + this.colFamily.toString() +
+    return this.encodedRegionName.toString() + "/" + this.colFamily.toString() +
       "/" + this.fileId +
       (isReference()? "/" + this.reference.toString(): "");
   }
@@ -990,7 +990,7 @@
   @Override
   public int hashCode() {
     int result = this.dir.hashCode();
-    result ^= this.regionName.hashCode();
+    result ^= this.encodedRegionName.hashCode();
     result ^= this.colFamily.hashCode();
     result ^= Long.valueOf(this.fileId).hashCode();
     return result;
@@ -1001,7 +1001,7 @@
   /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
     out.writeUTF(dir.toString());
-    this.regionName.write(out);
+    out.writeUTF(this.encodedRegionName);
     this.colFamily.write(out);
     out.writeLong(fileId);
     out.writeBoolean(isReference());
@@ -1013,7 +1013,7 @@
   /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
     this.dir = new Path(in.readUTF());
-    this.regionName.readFields(in);
+    this.encodedRegionName = in.readUTF();
     this.colFamily.readFields(in);
     this.fileId = in.readLong();
     this.reference = null;
@@ -1031,7 +1031,7 @@
     HStoreFile other = (HStoreFile) o;
     int result = this.dir.compareTo(other.dir);    
     if(result == 0) {
-      this.regionName.compareTo(other.regionName);
+      this.encodedRegionName.compareTo(other.encodedRegionName);
     }
     if(result == 0) {
       result = this.colFamily.compareTo(other.colFamily);

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java Thu Oct 18 15:21:09 2007
@@ -38,6 +38,21 @@
  * column families.
  */
 public class HTableDescriptor implements WritableComparable {
+  /** table descriptor for root table */
+  public static final HTableDescriptor rootTableDesc =
+    new HTableDescriptor(HConstants.ROOT_TABLE_NAME,
+        new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1,
+            HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE,
+            null));
+  
+  /** table descriptor for meta table */
+  public static final HTableDescriptor metaTableDesc =
+    new HTableDescriptor(HConstants.META_TABLE_NAME,
+        new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1,
+            HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE,
+            null));
+  
+
   private Text name;
   // TODO: Does this need to be a treemap?  Can it be a HashMap?
   private final TreeMap<Text, HColumnDescriptor> families;
@@ -51,6 +66,13 @@
    */
   private static final Pattern LEGAL_TABLE_NAME =
     Pattern.compile("^[\\w-.]+$");
+
+  /** Used to construct the table descriptors for root and meta tables */
+  private HTableDescriptor(Text name, HColumnDescriptor family) {
+    this.name = new Text(name);
+    this.families = new TreeMap<Text, HColumnDescriptor>();
+    families.put(family.getName(), family);
+  }
 
   /** Constructs an empty object */
   public HTableDescriptor() {

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java Thu Oct 18 15:21:09 2007
@@ -33,9 +33,10 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HGlobals;
 import org.apache.hadoop.io.Text;
 
+import org.apache.hadoop.hbase.HLogEdit;
+
 /**
  * Implementation of RowFilterInterface that can filter by rowkey regular
  * expression and/or individual column values (equals comparison only).
@@ -176,7 +177,7 @@
       }
     }
     if (nullColumns.contains(colKey)) {
-      if (data != null && !Arrays.equals(HGlobals.deleteBytes.get(), data)) {
+      if (data != null && !HLogEdit.isDeleted(data)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("filter returning true for rowKey: " + rowKey + 
             " colKey: " + colKey);
@@ -198,7 +199,7 @@
   public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
     for (Entry<Text, byte[]> col : columns.entrySet()) {
       if (nullColumns.contains(col.getKey())
-          && !Arrays.equals(HGlobals.deleteBytes.get(), col.getValue())) {
+          && !HLogEdit.isDeleted(col.getValue())) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("filterNotNull returning true for colKey: " + col.getKey()
             + ", column should be null.");

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java Thu Oct 18 15:21:09 2007
@@ -88,12 +88,8 @@
       // Now create the root and meta regions and insert the data regions
       // created above into the meta
       
-      HRegion root =
-        createNewHRegion(dir, conf, HGlobals.rootTableDesc, 0L, null, null);
-      
-      HRegion meta =
-        createNewHRegion(dir, conf, HGlobals.metaTableDesc, 1L, null, null);
-    
+      HRegion root = createNewHRegion(dir, conf, HRegionInfo.rootRegionInfo);
+      HRegion meta = createNewHRegion(dir, conf, HRegionInfo.firstMetaRegionInfo);
       HRegion.addRegionToMETA(root, meta);
       
       for(int i = 0; i < regions.length; i++) {
@@ -134,8 +130,7 @@
   private HRegion createAregion(Text startKey, Text endKey, int firstRow,
       int nrows) throws IOException {
     
-    HRegion region =
-      createNewHRegion(dir, conf, desc, rand.nextLong(), startKey, endKey);
+    HRegion region = createNewHRegion(dir, conf, desc, startKey, endKey);
     
     System.out.println("created region " + region.getRegionName());
 
@@ -155,7 +150,7 @@
     region.compactStores();
     region.close();
     region.getLog().closeAndDelete();
-    region.getRegionInfo().offLine = true;
+    region.getRegionInfo().setOffline(true);
     return region;
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java Thu Oct 18 15:21:09 2007
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
+import java.io.UnsupportedEncodingException;
 
 import junit.framework.TestCase;
 
@@ -40,8 +41,10 @@
   protected FileSystem localFs = null;
   protected static final char FIRST_CHAR = 'a';
   protected static final char LAST_CHAR = 'z';
+  protected static final String PUNCTUATION = "~`@#$%^&*()-_+=:;',.<>/?[]{}|";
   protected static final byte [] START_KEY_BYTES =
     {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
+  protected String START_KEY;
   protected static final int MAXVERSIONS = 3;
   
   static {
@@ -50,10 +53,18 @@
   
   protected volatile Configuration conf;
 
-  /** constructor */
+  /**
+   * constructor
+   */
   public HBaseTestCase() {
     super();
     conf = new HBaseConfiguration();
+    try {
+      START_KEY =
+        new String(START_KEY_BYTES, HConstants.UTF8_ENCODING) + PUNCTUATION;
+    } catch (UnsupportedEncodingException e) {
+      fail();
+    }
   }
   
   /**
@@ -62,6 +73,12 @@
   public HBaseTestCase(String name) {
     super(name);
     conf = new HBaseConfiguration();
+    try {
+      START_KEY =
+        new String(START_KEY_BYTES, HConstants.UTF8_ENCODING) + PUNCTUATION;
+    } catch (UnsupportedEncodingException e) {
+      fail();
+    }
   }
   
   /** {@inheritDoc} */
@@ -90,10 +107,13 @@
   }
 
   protected HRegion createNewHRegion(Path dir, Configuration c,
-    HTableDescriptor desc, long regionId, Text startKey, Text endKey)
-  throws IOException {
-    HRegionInfo info = new HRegionInfo(regionId, desc, startKey, endKey);
-    Path regionDir = HRegion.getRegionDir(dir, info.regionName);
+      HTableDescriptor desc, Text startKey, Text endKey) throws IOException {
+    return createNewHRegion(dir, c, new HRegionInfo(desc, startKey, endKey));
+  }
+  
+  protected HRegion createNewHRegion(Path dir, Configuration c,
+        HRegionInfo info) throws IOException {
+    Path regionDir = HRegion.getRegionDir(dir, info.getEncodedName());
     FileSystem fs = dir.getFileSystem(c);
     fs.mkdirs(regionDir);
     return new HRegion(dir,
@@ -189,7 +209,9 @@
       for (char d = secondCharStart; d <= LAST_CHAR; d++) {
         for (char e = thirdCharStart; e <= LAST_CHAR; e++) {
           byte [] bytes = new byte [] {(byte)c, (byte)d, (byte)e};
-          Text t = new Text(new String(bytes, HConstants.UTF8_ENCODING));
+          String s = new String(bytes, HConstants.UTF8_ENCODING) + PUNCTUATION;
+          bytes = s.getBytes(HConstants.UTF8_ENCODING);
+          Text t = new Text(s);
           if (endKey != null && endKey.getLength() > 0
               && endKey.compareTo(t) <= 0) {
             break EXIT;

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MultiRegionTable.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MultiRegionTable.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MultiRegionTable.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MultiRegionTable.java Thu Oct 18 15:21:09 2007
@@ -130,7 +130,7 @@
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITA));
     HRegionInfo splitB =
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITB));
-    Path parentDir = HRegion.getRegionDir(d, parent.getRegionName());
+    Path parentDir = HRegion.getRegionDir(d, parent.getEncodedName());
     assertTrue(fs.exists(parentDir));
     LOG.info("Split happened. Parent is " + parent.getRegionName() +
         " and daughters are " + splitA.getRegionName() + ", " +

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompaction.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompaction.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompaction.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompaction.java Thu Oct 18 15:21:09 2007
@@ -35,21 +35,28 @@
   private HLog hlog = null;
   private HRegion r = null;
   private static final String COLUMN_FAMILY = COLFAMILY_NAME1;
-  private static final Text STARTROW = new Text(START_KEY_BYTES);
+  private final Text STARTROW;
   private static final Text COLUMN_FAMILY_TEXT = new Text(COLUMN_FAMILY);
   private static final Text COLUMN_FAMILY_TEXT_MINUS_COLON =
     new Text(COLUMN_FAMILY.substring(0, COLUMN_FAMILY.length() - 1));
   private static final int COMPACTION_THRESHOLD = MAXVERSIONS;
   
+  /** constructor */
+  public TestCompaction() {
+    STARTROW = new Text(START_KEY);
+  }
+  
+  /** {@inheritDoc} */
   @Override
   public void setUp() throws Exception {
     super.setUp();
     this.hlog = new HLog(this.localFs, this.testDir, this.conf);
     HTableDescriptor htd = createTableDescriptor(getName());
-    HRegionInfo hri = new HRegionInfo(1, htd, null, null);
+    HRegionInfo hri = new HRegionInfo(htd, null, null);
     this.r = new HRegion(testDir, hlog, this.localFs, this.conf, hri, null);
   }
   
+  /** {@inheritDoc} */
   @Override
   public void tearDown() throws Exception {
     this.r.close();
@@ -111,9 +118,7 @@
     // we added when we flushed. But could be 3 only if the flush happened
     // before the compaction started though we tried to have the threads run
     // concurrently (On hudson this happens).
-    byte [] secondRowBytes = new byte[START_KEY_BYTES.length];
-    System.arraycopy(START_KEY_BYTES, 0, secondRowBytes, 0,
-      START_KEY_BYTES.length);
+    byte [] secondRowBytes = START_KEY.getBytes(HConstants.UTF8_ENCODING);
     // Increment the least significant character so we get to next row.
     secondRowBytes[START_KEY_BYTES.length - 1]++;
     Text secondRow = new Text(secondRowBytes);

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.java Thu Oct 18 15:21:09 2007
@@ -48,25 +48,25 @@
    * Sort of HRegionInfo.
    */
   public void testHRegionInfo() {
-    HRegionInfo a = new HRegionInfo(1, new HTableDescriptor("a"), null, null);
-    HRegionInfo b = new HRegionInfo(2, new HTableDescriptor("b"), null, null);
+    HRegionInfo a = new HRegionInfo(new HTableDescriptor("a"), null, null);
+    HRegionInfo b = new HRegionInfo(new HTableDescriptor("b"), null, null);
     assertTrue(a.compareTo(b) != 0);
     HTableDescriptor t = new HTableDescriptor("t");
     Text midway = new Text("midway");
-    a = new HRegionInfo(1, t, null, midway);
-    b = new HRegionInfo(2, t, midway, null);
+    a = new HRegionInfo(t, null, midway);
+    b = new HRegionInfo(t, midway, null);
     assertTrue(a.compareTo(b) < 0);
     assertTrue(b.compareTo(a) > 0);
     assertEquals(a, a);
     assertTrue(a.compareTo(a) == 0);
-    a = new HRegionInfo(1, t, new Text("a"), new Text("d"));
-    b = new HRegionInfo(2, t, new Text("e"), new Text("g"));
+    a = new HRegionInfo(t, new Text("a"), new Text("d"));
+    b = new HRegionInfo(t, new Text("e"), new Text("g"));
     assertTrue(a.compareTo(b) < 0);
-    a = new HRegionInfo(1, t, new Text("aaaa"), new Text("dddd"));
-    b = new HRegionInfo(2, t, new Text("e"), new Text("g"));
+    a = new HRegionInfo(t, new Text("aaaa"), new Text("dddd"));
+    b = new HRegionInfo(t, new Text("e"), new Text("g"));
     assertTrue(a.compareTo(b) < 0);
-    a = new HRegionInfo(1, t, new Text("aaaa"), new Text("dddd"));
-    b = new HRegionInfo(2, t, new Text("aaaa"), new Text("eeee"));
+    a = new HRegionInfo(t, new Text("aaaa"), new Text("dddd"));
+    b = new HRegionInfo(t, new Text("aaaa"), new Text("eeee"));
     assertTrue(a.compareTo(b) < 0);
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java Thu Oct 18 15:21:09 2007
@@ -40,7 +40,7 @@
   
   private static final Text CONTENTS = new Text("contents:");
   private static final Text ROW_KEY =
-    new Text(HGlobals.rootRegionInfo.regionName);
+    new Text(HRegionInfo.rootRegionInfo.getRegionName());
   private static final String SERVER_ADDRESS = "foo.bar.com:1234";
 
   
@@ -88,8 +88,8 @@
       desc.addFamily(new HColumnDescriptor(CONTENTS.toString()));
       desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
       
-      HRegionInfo info = new HRegionInfo(0L, desc, null, null);
-      Path regionDir = HRegion.getRegionDir(dir, info.regionName);
+      HRegionInfo info = new HRegionInfo(desc, null, null);
+      Path regionDir = HRegion.getRegionDir(dir, info.getEncodedName());
       fs.mkdirs(regionDir);
       
       HLog log = new HLog(fs, new Path(regionDir, "log"), conf);
@@ -105,10 +105,10 @@
       r.put(lockid, CONTENTS, bytes.toByteArray());
 
       bytes.reset();
-      HGlobals.rootRegionInfo.write(s);
+      HRegionInfo.rootRegionInfo.write(s);
       
       r.put(lockid, HConstants.COL_REGIONINFO, 
-          Writables.getBytes(HGlobals.rootRegionInfo));
+          Writables.getBytes(HRegionInfo.rootRegionInfo));
       
       r.commit(lockid, System.currentTimeMillis());
       

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java Thu Oct 18 15:21:09 2007
@@ -79,7 +79,7 @@
         assertEquals(tableName, key.getTablename());
         assertEquals(HLog.METAROW, key.getRow());
         assertEquals(HLog.METACOLUMN, val.getColumn());
-        assertEquals(0, HGlobals.completeCacheFlush.compareTo(val.getVal()));
+        assertEquals(0, HLogEdit.completeCacheFlush.compareTo(val.getVal()));
         System.out.println(key + " " + val);
       }
     } finally {

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java Thu Oct 18 15:21:09 2007
@@ -104,7 +104,7 @@
     desc.addFamily(new HColumnDescriptor("contents:"));
     desc.addFamily(new HColumnDescriptor("anchor:"));
     region = new HRegion(parentdir, log, fs, conf, 
-        new HRegionInfo(1, desc, null, null), null);
+        new HRegionInfo(desc, null, null), null);
   }
 
   // Test basic functionality. Writes to contents:basic and anchor:anchornum-*

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java Thu Oct 18 15:21:09 2007
@@ -20,7 +20,6 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
-import java.io.PrintWriter;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,7 +31,6 @@
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.util.ReflectionUtils;
 
 /**
  * Test HStoreFile
@@ -125,7 +123,7 @@
   public void testReference()
   throws IOException {
     // Make a store file and write data to it.
-    HStoreFile hsf = new HStoreFile(this.conf, this.dir, new Text(getName()),
+    HStoreFile hsf = new HStoreFile(this.conf, this.dir, getName(),
         new Text("colfamily"), 1234567890L);
     MapFile.Writer writer =
       hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, null);
@@ -140,10 +138,10 @@
     Text finalKey = hsk.getRow();
     // Make a reference for the bottom half of the just written file.
     HStoreFile.Reference reference =
-      new HStoreFile.Reference(hsf.getRegionName(), hsf.getFileId(), midkey,
-          HStoreFile.Range.top);
+      new HStoreFile.Reference(hsf.getEncodedRegionName(), hsf.getFileId(),
+          midkey, HStoreFile.Range.top);
     HStoreFile refHsf = new HStoreFile(this.conf, new Path(DIR, getName()),
-        new Text(getName() + "_reference"), hsf.getColFamily(), 456,
+        getName() + "_reference", hsf.getColFamily(), 456,
         reference);
     // Assert that reference files are written and that we can write and
     // read the info reference file at least.
@@ -152,8 +150,8 @@
     assertTrue(this.fs.exists(refHsf.getInfoFilePath()));
     HStoreFile.Reference otherReference =
       HStoreFile.readSplitInfo(refHsf.getInfoFilePath(), this.fs);
-    assertEquals(reference.getRegionName().toString(),
-        otherReference.getRegionName().toString());
+    assertEquals(reference.getEncodedRegionName(),
+        otherReference.getEncodedRegionName());
     assertEquals(reference.getFileId(),
         otherReference.getFileId());
     assertEquals(reference.getMidkey().toString(),

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java Thu Oct 18 15:21:09 2007
@@ -46,9 +46,9 @@
     HConstants.COL_STARTCODE
   };
   
-  private static final Text ROW_KEY = new Text(HGlobals.rootRegionInfo.regionName);
-  private static final HRegionInfo REGION_INFO = 
-    new HRegionInfo(0L, HGlobals.rootTableDesc, null, null);
+  private static final Text ROW_KEY =
+    new Text(HRegionInfo.rootRegionInfo.getRegionName());
+  private static final HRegionInfo REGION_INFO = HRegionInfo.rootRegionInfo;
   
   private static final long START_CODE = Long.MAX_VALUE;
 
@@ -59,11 +59,11 @@
     HRegionInfo info =
       (HRegionInfo) Writables.getWritable(regionBytes, new HRegionInfo());
     
-    assertEquals(REGION_INFO.regionId, info.regionId);
-    assertEquals(0, info.startKey.getLength());
-    assertEquals(0, info.endKey.getLength());
-    assertEquals(0, info.regionName.compareTo(REGION_INFO.regionName));
-    assertEquals(0, info.tableDesc.compareTo(REGION_INFO.tableDesc));
+    assertEquals(REGION_INFO.getRegionId(), info.getRegionId());
+    assertEquals(0, info.getStartKey().getLength());
+    assertEquals(0, info.getEndKey().getLength());
+    assertEquals(0, info.getRegionName().compareTo(REGION_INFO.getRegionName()));
+    assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc()));
   }
   
   /** Use a scanner to get the region info and then validate the results */
@@ -109,9 +109,10 @@
         }
 
       } finally {
-        if(scanner != null) {
-          scanner.close();
-          scanner = null;
+        HInternalScannerInterface s = scanner;
+        scanner = null;
+        if(s != null) {
+          s.close();
         }
       }
     }
@@ -140,7 +141,7 @@
       Path dir = new Path("/hbase");
       fs.mkdirs(dir);
       
-      Path regionDir = HRegion.getRegionDir(dir, REGION_INFO.regionName);
+      Path regionDir = HRegion.getRegionDir(dir, REGION_INFO.getEncodedName());
       fs.mkdirs(regionDir);
       
       HLog log = new HLog(fs, new Path(regionDir, "log"), conf);
@@ -153,7 +154,7 @@
 
       ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
       DataOutputStream s = new DataOutputStream(byteStream);
-      HGlobals.rootRegionInfo.write(s);
+      HRegionInfo.rootRegionInfo.write(s);
       region.put(lockid, HConstants.COL_REGIONINFO, byteStream.toByteArray());
       region.commit(lockid, System.currentTimeMillis());
 

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java Thu Oct 18 15:21:09 2007
@@ -244,19 +244,19 @@
     assertEquals("Expected one region", regions.size(), 1);
     HRegionInfo region = regions.get(0);
     assertTrue("Expected region named for test",
-      region.regionName.toString().startsWith(getName()));
+      region.getRegionName().toString().startsWith(getName()));
     // Now do what happens at split time; remove old region and then add two
     // new ones in its place.
     removeRegionFromMETA(new HTable(conf, HConstants.META_TABLE_NAME),
-      region.regionName);
-    HTableDescriptor desc = region.tableDesc;
+      region.getRegionName());
+    HTableDescriptor desc = region.getTableDesc();
     Path homedir = new Path(getName());
     List<HRegion> newRegions = new ArrayList<HRegion>(2);
     newRegions.add(HRegion.createHRegion(
-      new HRegionInfo(2L, desc, null, new Text("midway")),
+      new HRegionInfo(desc, null, new Text("midway")),
       homedir, this.conf, null));
     newRegions.add(HRegion.createHRegion(
-      new HRegionInfo(3L, desc, new Text("midway"), null),
+      new HRegionInfo(desc, new Text("midway"), null),
         homedir, this.conf, null));
     try {
       for (HRegion r : newRegions) {

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java Thu Oct 18 15:21:09 2007
@@ -70,7 +70,7 @@
     HLog hlog = new HLog(this.localFs, this.testDir, this.conf);
     try {
       HTableDescriptor htd = createTableDescriptor(getName());
-      HRegionInfo hri = new HRegionInfo(1, htd, null, null);
+      HRegionInfo hri = new HRegionInfo(htd, null, null);
       region = new HRegion(testDir, hlog, this.localFs, this.conf, hri, null);
       basicSplit(region);
     } finally {
@@ -89,11 +89,10 @@
     HRegion [] regions = split(region);
     // Assert can get rows out of new regions.  Should be able to get first
     // row from first region and the midkey from second region.
-    byte [] b = new byte [] {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
-    assertGet(regions[0], COLFAMILY_NAME3, new Text(b));
+    assertGet(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
     assertGet(regions[1], COLFAMILY_NAME3, midkey);
     // Test I can get scanner and that it starts at right place.
-    assertScan(regions[0], COLFAMILY_NAME3, new Text(b));
+    assertScan(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
     assertScan(regions[1], COLFAMILY_NAME3, midkey);
     // Now prove can't split regions that have references.
     Text [] midkeys = new Text[regions.length];
@@ -143,6 +142,7 @@
     // The splits should have been even.  Test I can get some arbitrary row out
     // of each.
     int interval = (LAST_CHAR - FIRST_CHAR) / 3;
+    byte[] b = START_KEY.getBytes(HConstants.UTF8_ENCODING);
     for (HRegion r: sortedMap.values()) {
       assertGet(r, COLFAMILY_NAME3,
           new Text(new String(b, HConstants.UTF8_ENCODING)));

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTable.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTable.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTable.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTable.java Thu Oct 18 15:21:09 2007
@@ -33,23 +33,23 @@
     final HBaseAdmin admin = new HBaseAdmin(conf);
     String msg = null;
     try {
-      admin.createTable(HGlobals.rootTableDesc);
+      admin.createTable(HTableDescriptor.rootTableDesc);
     } catch (IllegalArgumentException e) {
       msg = e.toString();
     }
     assertTrue("Unexcepted exception message " + msg, msg != null &&
       msg.startsWith(IllegalArgumentException.class.getName()) &&
-      msg.contains(HGlobals.rootTableDesc.getName().toString()));
+      msg.contains(HTableDescriptor.rootTableDesc.getName().toString()));
     
     msg = null;
     try {
-      admin.createTable(HGlobals.metaTableDesc);
+      admin.createTable(HTableDescriptor.metaTableDesc);
     } catch(IllegalArgumentException e) {
       msg = e.toString();
     }
     assertTrue("Unexcepted exception message " + msg, msg != null &&
       msg.startsWith(IllegalArgumentException.class.getName()) &&
-      msg.contains(HGlobals.metaTableDesc.getName().toString()));
+      msg.contains(HTableDescriptor.metaTableDesc.getName().toString()));
     
     // Try doing a duplicate database create.
     msg = null;

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java Thu Oct 18 15:21:09 2007
@@ -320,7 +320,7 @@
     HTableDescriptor htd = createTableDescriptor(getName());
     htd.addFamily(new HColumnDescriptor(COLUMN, VERSIONS,
       CompressionType.NONE, false, Integer.MAX_VALUE, null));
-    HRegionInfo hri = new HRegionInfo(1, htd, null, null);
+    HRegionInfo hri = new HRegionInfo(htd, null, null);
     return new HRegion(testDir, hlog, this.localFs, this.conf, hri, null);
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java Thu Oct 18 15:21:09 2007
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase;
 
-import org.apache.hadoop.io.Text;
-
 import junit.framework.TestCase;
 
 /**
@@ -38,32 +36,28 @@
     assertEquals("HServerInfo", "address: " + hostport + ", startcode: -1" +
         ", load: (requests: 0 regions: 0)", info.toString());
   }
-  
+
   /**
-   * Tests toString method on HRegionInfo
-   * @throws Exception
+   * Test HTableDescriptor.toString();
    */
-  public void testHRegionInfo() throws Exception {
-    HTableDescriptor htd = new HTableDescriptor("hank");
-    htd.addFamily(new HColumnDescriptor("hankfamily:"));
-    htd.addFamily(new HColumnDescriptor(new Text("hankotherfamily:"), 10,
-      HColumnDescriptor.CompressionType.BLOCK, true, 1000, null));
+  public void testHTableDescriptor() {
+    HTableDescriptor htd = HTableDescriptor.rootTableDesc;
     System. out.println(htd.toString());
-    assertEquals("Table descriptor", "name: hank, families: " +
-      "{hankfamily:={name: hankfamily, max versions: 3, compression: NONE, " +
-      "in memory: false, max length: 2147483647, bloom filter: none}, " +
-      "hankotherfamily:={name: hankotherfamily, max versions: 10, " +
-      "compression: BLOCK, in memory: true, max length: 1000, " +
-      "bloom filter: none}}", htd.toString());
-    HRegionInfo hri = new HRegionInfo(-1, htd, new Text(), new Text("10"));
+    assertEquals("Table descriptor", "name: -ROOT-, families: {info:={name: " +
+        "info, max versions: 1, compression: NONE, in memory: false, max " +
+        "length: 2147483647, bloom filter: none}}", htd.toString());
+  }
+  
+  /**
+   * Tests HRegionInfo.toString()
+   */
+  public void testHRegionInfo() {
+    HRegionInfo hri = HRegionInfo.rootRegionInfo;
     System.out.println(hri.toString());
     assertEquals("HRegionInfo", 
-      "regionname: hank,,-1, startKey: <>, tableDesc: {name: hank, " +
-      "families: {hankfamily:={name: hankfamily, max versions: 3, " +
-      "compression: NONE, in memory: false, max length: 2147483647, " +
-      "bloom filter: none}, hankotherfamily:={name: hankotherfamily, " +
-      "max versions: 10, compression: BLOCK, in memory: true, " +
-      "max length: 1000, bloom filter: none}}}",
+      "regionname: -ROOT-,,0, startKey: <>, tableDesc: {name: -ROOT-, " +
+      "families: {info:={name: info, max versions: 1, compression: NONE, " +
+      "in memory: false, max length: 2147483647, bloom filter: none}}}",
       hri.toString());
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java?rev=586159&r1=586158&r2=586159&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java Thu Oct 18 15:21:09 2007
@@ -30,7 +30,7 @@
 import junit.framework.TestCase;
 
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HGlobals;
+import org.apache.hadoop.hbase.HLogEdit;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -186,7 +186,7 @@
     // that maps to a null value.
     // Testing row with columnKeys: a-e, e maps to null
     colvalues.put(new Text(new String(new char[] { LAST_CHAR })), 
-      HGlobals.deleteBytes.get());
+      HLogEdit.deleteBytes.get());
     assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.", 
       filter.filterNotNull(colvalues));
   }