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/20 10:06:53 UTC

svn commit: r586680 - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/

Author: jimk
Date: Sat Oct 20 01:06:52 2007
New Revision: 586680

URL: http://svn.apache.org/viewvc?rev=586680&view=rev
Log:
HADOOP-2079 HADOOP-2056 Fix generated HLog, HRegion names

HLog.splitLog was generating incorrect file names, HRegion was generating file names that could be far too long especially for local file systems, HMaster had a race condition in which an old HLog would get split by two threads simultaneously. 

Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    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/HMaster.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/HStore.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.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/TestGet.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.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=586680&r1=586679&r2=586680&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Sat Oct 20 01:06:52 2007
@@ -14,6 +14,7 @@
                (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
+   HADOOP-2079 HADOOP-2056 Fix generated HLog, HRegion names
 
   IMPROVEMENTS
     HADOOP-2401 Add convenience put method that takes writable

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -159,7 +159,7 @@
             SequenceFile.Writer w = logWriters.get(regionName);
             if (w == null) {
               Path logfile = new Path(HRegion.getRegionDir(rootDir,
-                HRegionInfo.rootRegionInfo.getEncodedName()),
+                HRegionInfo.encodeRegionName(regionName)),
                 HREGION_OLDLOGFILE_NAME);
               
               if (LOG.isDebugEnabled()) {

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -326,7 +326,8 @@
       if (!hasReferencesA && !hasReferencesB) {
         LOG.info("Deleting region " + parent.getRegionName() +
         " because daughter splits no longer hold references");
-        if (!HRegion.deleteRegion(fs, dir, parent.getEncodedName())) {
+        if (!HRegion.deleteRegion(fs, dir,
+            HRegionInfo.encodeRegionName(parent.getRegionName()))) {
           LOG.warn("Deletion of " + parent.getRegionName() + " failed");
         }
         
@@ -370,7 +371,8 @@
       }
       for (Text family: split.getTableDesc().families().keySet()) {
         Path p = HStoreFile.getMapDir(fs.makeQualified(dir),
-            split.getEncodedName(), HStoreKey.extractFamily(family));
+            HRegionInfo.encodeRegionName(split.getRegionName()),
+            HStoreKey.extractFamily(family));
 
         // Look for reference files.  Call listPaths with an anonymous
         // instance of PathFilter.
@@ -418,6 +420,7 @@
         return;
       }
       HServerInfo storedInfo = null;
+      boolean deadServer = false;
       if (serverName.length() != 0) {
         Map<Text, HRegionInfo> regionsToKill = killList.get(serverName);
         if (regionsToKill != null &&
@@ -432,6 +435,9 @@
         }
         synchronized (serversToServerInfo) {
           storedInfo = serversToServerInfo.get(serverName);
+          if (storedInfo != null && deadServers.contains(serverName)) {
+            deadServer = true;
+          }
         }
       }
       if (LOG.isDebugEnabled()) {
@@ -439,13 +445,17 @@
       }
       if (!(unassignedRegions.containsKey(info.getRegionName()) ||
             pendingRegions.contains(info.getRegionName()))
-          && (storedInfo == null || storedInfo.getStartCode() != startCode)) {
+          && (storedInfo == null || 
+              (storedInfo.getStartCode() != startCode && !deadServer))) {
         // The current assignment is no good
         if (LOG.isDebugEnabled()) {
           LOG.debug("Current assignment of " + info.getRegionName() +
               " is no good");
         }
         // Recover the region server's log if there is one.
+        // This is only done from here if we are restarting and there is stale
+        // data in the meta region. Once we are on-line, dead server log
+        // recovery is handled by lease expiration and PendingServerShutdown
         if (serverName.length() != 0) {
           StringBuilder dirName = new StringBuilder("log_");
           dirName.append(serverName.replace(":", "_"));
@@ -830,6 +840,10 @@
    */
   final Map<String, HServerInfo> serversToServerInfo =
     new HashMap<String, HServerInfo>();
+  
+  /** Set of known dead servers */
+  final Set<String> deadServers =
+    Collections.synchronizedSet(new HashSet<String>());
 
   /** SortedMap server load -> Set of server names */
   SortedMap<HServerLoad, Set<String>> loadToServers;
@@ -864,8 +878,8 @@
     this.fs = FileSystem.get(conf);
     this.rand = new Random();
     
-    Path rootRegionDir =
-      HRegion.getRegionDir(dir, HRegionInfo.rootRegionInfo.getEncodedName());
+    Path rootRegionDir = HRegion.getRegionDir(dir,
+        HRegionInfo.encodeRegionName(HRegionInfo.rootRegionInfo.getRegionName()));
     LOG.info("Root region dir: " + rootRegionDir.toString());
 
     try {
@@ -2107,6 +2121,7 @@
                   Thread.currentThread().getName());
             }
           }
+          deadServers.remove(deadServerName);
           break;
 
         } catch (IOException e) {
@@ -2240,7 +2255,8 @@
 
       } else if (deleteRegion) {
         try {
-          HRegion.deleteRegion(fs, dir, regionInfo.getEncodedName());
+          HRegion.deleteRegion(fs, dir,
+              HRegionInfo.encodeRegionName(regionInfo.getRegionName()));
         } catch (IOException e) {
           e = RemoteExceptionHandler.checkIOException(e);
           LOG.error("failed delete region " + regionInfo.getRegionName(), e);
@@ -2857,7 +2873,8 @@
         // Delete the region
       
         try {
-          HRegion.deleteRegion(fs, dir, i.getEncodedName());
+          HRegion.deleteRegion(fs, dir,
+              HRegionInfo.encodeRegionName(i.getRegionName()));
         
         } catch (IOException e) {
           LOG.error("failed to delete region " + i.getRegionName(),
@@ -2924,8 +2941,9 @@
 
         // Delete the directories used by the column
 
-        fs.delete(HStoreFile.getMapDir(dir, i.getEncodedName(), columnName));
-        fs.delete(HStoreFile.getInfoDir(dir, i.getEncodedName(), columnName));
+        String encodedName = HRegionInfo.encodeRegionName(i.getRegionName());
+        fs.delete(HStoreFile.getMapDir(dir, encodedName, columnName));
+        fs.delete(HStoreFile.getInfoDir(dir, encodedName, columnName));
       }
     }
   }
@@ -2985,6 +3003,7 @@
               loadToServers.put(load, servers);
             }
           }
+          deadServers.add(server);
         }
         serversToServerInfo.notifyAll();
       }

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -121,8 +121,8 @@
     }
     
     HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
-    Path newRegionDir =
-      HRegion.getRegionDir(merges, newRegionInfo.getEncodedName());
+    Path newRegionDir = HRegion.getRegionDir(merges,
+        HRegionInfo.encodeRegionName(newRegionInfo.getRegionName()));
     if(fs.exists(newRegionDir)) {
       throw new IOException("Cannot merge; target file collision at " +
         newRegionDir);
@@ -138,9 +138,9 @@
     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.getEncodedName(),
-        colFamily, Math.abs(rand.nextLong()));
+      HStoreFile dst = new HStoreFile(conf, merges,
+          HRegionInfo.encodeRegionName(newRegionInfo.getRegionName()),
+          colFamily, Math.abs(rand.nextLong()));
       dst.mergeStoreFiles(srcFiles, fs, conf);
     }
 
@@ -215,6 +215,7 @@
   private final HLocking lock = new HLocking();
   private long desiredMaxFileSize;
   private final long minSequenceId;
+  private final String encodedRegionName;
 
   //////////////////////////////////////////////////////////////////////////////
   // Constructor
@@ -247,6 +248,8 @@
     this.fs = fs;
     this.conf = conf;
     this.regionInfo = regionInfo;
+    this.encodedRegionName =
+      HRegionInfo.encodeRegionName(this.regionInfo.getRegionName());
     this.memcache = new HMemcache();
     this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
     this.optionalFlushCount =
@@ -254,8 +257,7 @@
 
     // Declare the regionName.  This is a unique string for the region, used to 
     // build a unique filename.
-    this.regiondir =
-      HRegion.getRegionDir(rootDir, this.regionInfo.getEncodedName());
+    this.regiondir = HRegion.getRegionDir(rootDir, this.encodedRegionName);
     Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
 
     // Move prefab HStore files into place (if any).  This picks up split files
@@ -270,8 +272,8 @@
         this.regionInfo.getTableDesc().families().entrySet()) {
       Text colFamily = HStoreKey.extractFamily(e.getKey());
       
-      HStore store = new HStore(rootDir, this.regionInfo.getEncodedName(), 
-          e.getValue(), fs, oldLogFile, conf); 
+      HStore store = new HStore(rootDir, this.regionInfo.getRegionName(),
+          this.encodedRegionName, e.getValue(), fs, oldLogFile, conf); 
       
       stores.put(colFamily, store);
       
@@ -420,13 +422,15 @@
     Path splits = getSplitsDir();
     HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
         this.regionInfo.getStartKey(), midKey);
-    Path dirA = getSplitRegionDir(splits, regionAInfo.getEncodedName());
+    Path dirA = getSplitRegionDir(splits,
+        HRegionInfo.encodeRegionName(regionAInfo.getRegionName()));
     if(fs.exists(dirA)) {
       throw new IOException("Cannot split; target file collision at " + dirA);
     }
     HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
         midKey, null);
-    Path dirB = getSplitRegionDir(splits, regionBInfo.getEncodedName());
+    Path dirB = getSplitRegionDir(splits,
+        HRegionInfo.encodeRegionName(regionBInfo.getRegionName()));
     if(this.fs.exists(dirB)) {
       throw new IOException("Cannot split; target file collision at " + dirB);
     }
@@ -457,18 +461,18 @@
     for(HStoreFile h: hstoreFilesToSplit) {
       // A reference to the bottom half of the hsf store file.
       HStoreFile.Reference aReference = new HStoreFile.Reference(
-        this.regionInfo.getEncodedName(), h.getFileId(), new HStoreKey(midKey),
+        this.encodedRegionName, h.getFileId(), new HStoreKey(midKey),
         HStoreFile.Range.bottom);
       HStoreFile a = new HStoreFile(this.conf, splits,
-          regionAInfo.getEncodedName(), h.getColFamily(),
-          Math.abs(rand.nextLong()), aReference);
+          HRegionInfo.encodeRegionName(regionAInfo.getRegionName()),
+          h.getColFamily(), Math.abs(rand.nextLong()), aReference);
       // Reference to top half of the hsf store file.
       HStoreFile.Reference bReference = new HStoreFile.Reference(
-        this.regionInfo.getEncodedName(), h.getFileId(), new HStoreKey(midKey),
+        this.encodedRegionName, h.getFileId(), new HStoreKey(midKey),
         HStoreFile.Range.top);
       HStoreFile b = new HStoreFile(this.conf, splits,
-        regionBInfo.getEncodedName(), h.getColFamily(),
-        Math.abs(rand.nextLong()), bReference);
+        HRegionInfo.encodeRegionName(regionBInfo.getRegionName()),
+        h.getColFamily(), Math.abs(rand.nextLong()), bReference);
       h.splitStoreFile(a, b, this.fs);
     }
 
@@ -1824,7 +1828,8 @@
    */
   static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
       final Configuration conf, final Path initialFiles) throws IOException {
-    Path regionDir = HRegion.getRegionDir(rootDir, info.getEncodedName());
+    Path regionDir = HRegion.getRegionDir(rootDir,
+        HRegionInfo.encodeRegionName(info.getRegionName()));
     FileSystem fs = FileSystem.get(conf);
     fs.mkdirs(regionDir);
     return new HRegion(rootDir,

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -23,19 +23,50 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
 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
  * HRegions' table descriptor, etc.
  */
 public class HRegionInfo implements WritableComparable {
+  private static MessageDigest encoder = null;
+  
+  static {
+    try {
+      if (encoder == null) {
+        encoder = MessageDigest.getInstance("SHA");
+      }
+    } catch (NoSuchAlgorithmException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * @param regionName
+   * @return the encodedName
+   */
+  public static String encodeRegionName(final Text regionName) {
+    byte[] bytes = null;
+    synchronized (encoder) {
+      encoder.update(regionName.getBytes(), 0, regionName.getLength());
+      bytes = encoder.digest();
+      encoder.reset();
+    }
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < bytes.length; i++) {
+      sb.append(bytes[i]);
+    }
+    return sb.toString();
+  }
+
   /** delimiter used between portions of a region name */
-  public static final String DELIMITER = ",";
+  private static final String DELIMITER = ",";
 
   /** HRegionInfo for root region */
   public static final HRegionInfo rootRegionInfo =
@@ -61,35 +92,6 @@
     System.arraycopy(regionName.getBytes(), 0, tableName, 0, offset);
     return new Text(tableName);
   }
-  
-  /**
-   * 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;
@@ -187,16 +189,6 @@
   /** @return the regionName */
   public Text getRegionName(){
     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 */

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -24,7 +24,6 @@
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -72,6 +71,7 @@
   private static final String BLOOMFILTER_FILE_NAME = "filter";
 
   Path dir;
+  Text regionName;
   String encodedRegionName;
   HColumnDescriptor family;
   Text familyName;
@@ -131,19 +131,22 @@
    * file will be deleted (by whoever has instantiated the HStore).
    *
    * @param dir log file directory
-   * @param encodedRegionName filename friendly name of region
+   * @param regionName
+   * @param encodedName
    * @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, String encodedRegionName, HColumnDescriptor family, 
-      FileSystem fs, Path reconstructionLog, Configuration conf)
+  HStore(Path dir, Text regionName, String encodedName,
+      HColumnDescriptor family, FileSystem fs, Path reconstructionLog,
+      Configuration conf)
   throws IOException {  
     this.dir = dir;
     this.compactionDir = new Path(dir, "compaction.dir");
-    this.encodedRegionName = encodedRegionName;
+    this.regionName = regionName;
+    this.encodedRegionName = encodedName;
     this.family = family;
     this.familyName = HStoreKey.extractFamily(this.family.getName());
     this.compression = SequenceFile.CompressionType.NONE;
@@ -187,7 +190,7 @@
     // 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,
+    List<HStoreFile> hstoreFiles = HStoreFile.loadHStoreFiles(conf, dir,
         encodedRegionName, familyName, fs);
     for(HStoreFile hsf: hstoreFiles) {
       this.storefiles.put(Long.valueOf(hsf.loadInfo(fs)), hsf);
@@ -265,7 +268,6 @@
     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)) {
@@ -282,13 +284,13 @@
         // METACOLUMN info such as HBASE::CACHEFLUSH entries
         Text column = val.getColumn();
         if (column.equals(HLog.METACOLUMN)
-            || !key.getRegionName().equals(thisRegionName)
+            || !key.getRegionName().equals(regionName)
             || !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: " + thisRegionName + ", my column: " +
+                ", my region: " + regionName + ", my column: " +
                 this.familyName);
           }
           continue;

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -26,8 +26,9 @@
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Random;
-import java.util.Vector;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -133,7 +134,7 @@
    * Constructor that fully initializes the object
    * @param conf Configuration object
    * @param dir directory path
-   * @param regionName name of the region
+   * @param encodedRegionName name of the region
    * @param colFamily name of the column family
    * @param fileId file identifier
    */
@@ -377,15 +378,15 @@
    * @return List of store file instances loaded from passed dir.
    * @throws IOException
    */
-  static Vector<HStoreFile> loadHStoreFiles(Configuration conf, Path dir, 
+  static List<HStoreFile> loadHStoreFiles(Configuration conf, Path dir, 
       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, 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);
+    ArrayList<HStoreFile> results = new ArrayList<HStoreFile>(infofiles.length);
+    ArrayList<Path> mapfiles = new ArrayList<Path>(infofiles.length);
     for (int i = 0; i < infofiles.length; i++) {
       Path p = infofiles[i];
       Matcher m = REF_NAME_PARSER.matcher(p.getName());
@@ -534,7 +535,7 @@
    * @param conf configuration object
    * @throws IOException
    */
-  void mergeStoreFiles(Vector<HStoreFile> srcFiles, FileSystem fs, 
+  void mergeStoreFiles(List<HStoreFile> srcFiles, FileSystem fs, 
       @SuppressWarnings("hiding") Configuration conf)
   throws IOException {
     // Copy all the source MapFile tuples into this HSF's MapFile

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -113,7 +113,8 @@
   
   protected HRegion createNewHRegion(Path dir, Configuration c,
         HRegionInfo info) throws IOException {
-    Path regionDir = HRegion.getRegionDir(dir, info.getEncodedName());
+    Path regionDir = HRegion.getRegionDir(dir
+        , HRegionInfo.encodeRegionName(info.getRegionName()));
     FileSystem fs = dir.getFileSystem(c);
     fs.mkdirs(regionDir);
     return new HRegion(dir,

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -51,6 +51,7 @@
    * @param columnName
    * @throws IOException
    */
+  @SuppressWarnings("null")
   public static void makeMultiRegionTable(Configuration conf,
       MiniHBaseCluster cluster, FileSystem localFs, String tableName,
       String columnName)
@@ -130,7 +131,8 @@
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITA));
     HRegionInfo splitB =
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITB));
-    Path parentDir = HRegion.getRegionDir(d, parent.getEncodedName());
+    Path parentDir = HRegion.getRegionDir(d,
+        HRegionInfo.encodeRegionName(parent.getRegionName()));
     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/TestGet.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java?rev=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -89,7 +89,8 @@
       desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
       
       HRegionInfo info = new HRegionInfo(desc, null, null);
-      Path regionDir = HRegion.getRegionDir(dir, info.getEncodedName());
+      Path regionDir = HRegion.getRegionDir(dir,
+          HRegionInfo.encodeRegionName(info.getRegionName()));
       fs.mkdirs(regionDir);
       
       HLog log = new HLog(fs, new Path(regionDir, "log"), conf);

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=586680&r1=586679&r2=586680&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 Sat Oct 20 01:06:52 2007
@@ -141,7 +141,8 @@
       Path dir = new Path("/hbase");
       fs.mkdirs(dir);
       
-      Path regionDir = HRegion.getRegionDir(dir, REGION_INFO.getEncodedName());
+      Path regionDir = HRegion.getRegionDir(dir,
+          HRegionInfo.encodeRegionName(REGION_INFO.getRegionName()));
       fs.mkdirs(regionDir);
       
       HLog log = new HLog(fs, new Path(regionDir, "log"), conf);