You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/13 19:46:32 UTC

svn commit: r1445836 - in /hbase/branches/hbase-7290/hbase-server/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/io/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/snapshot/ test/java/org/ap...

Author: jmhsieh
Date: Wed Feb 13 18:46:32 2013
New Revision: 1445836

URL: http://svn.apache.org/r1445836
Log:
HBASE-7419 revisit hfilelink file name format (Matteo Bertozzi)


Modified:
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
    hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java?rev=1445836&r1=1445835&r2=1445836&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java Wed Feb 13 18:46:32 2013
@@ -114,6 +114,9 @@ public class HRegionInfo implements Comp
   private static final int ENC_SEPARATOR = '.';
   public  static final int MD5_HEX_LENGTH   = 32;
 
+  /** A non-capture group so that this can be embedded. */
+  public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
+
   /**
    * Does region name contain its encoded name?
    * @param regionName region name

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java?rev=1445836&r1=1445835&r2=1445836&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java Wed Feb 13 18:46:32 2013
@@ -29,7 +29,9 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 
@@ -52,9 +54,24 @@ import org.apache.hadoop.hbase.util.HFil
 public class HFileLink extends FileLink {
   private static final Log LOG = LogFactory.getLog(HFileLink.class);
 
-  /** Define the HFile Link name pattern in the form of: hfile-region-table */
-  public static final Pattern LINK_NAME_PARSER =
-    Pattern.compile("^([0-9a-f\\.]+)-([0-9a-f]+)-([a-zA-Z_0-9]+[a-zA-Z0-9_\\-\\.]*)$");
+  /**
+   * A non-capture group, for HFileLink, so that this can be embedded.
+   * The HFileLink describe a link to an hfile in a different table/region
+   * and the name is in the form: table=region-hfile.
+   * <p>
+   * Table name is ([a-zA-Z_0-9][a-zA-Z_0-9.-]*), so '=' is an invalid character for the table name.
+   * Region name is ([a-f0-9]+), so '-' is an invalid character for the region name.
+   * HFile is ([0-9a-f]+(?:_SeqId_[0-9]+_)?) covering the plain hfiles (uuid)
+   * and the bulk loaded (_SeqId_[0-9]+_) hfiles.
+   */
+  public static final String LINK_NAME_REGEX =
+    String.format("%s=%s-%s", HTableDescriptor.VALID_USER_TABLE_REGEX,
+      HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFile.HFILE_NAME_REGEX);
+
+  /** Define the HFile Link name parser in the form of: table=region-hfile */
+  private static final Pattern LINK_NAME_PATTERN =
+    Pattern.compile(String.format("^(%s)=(%s)-(%s)$", HTableDescriptor.VALID_USER_TABLE_REGEX,
+      HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFile.HFILE_NAME_REGEX));
 
   private final Path archivePath;
   private final Path originPath;
@@ -118,10 +135,10 @@ public class HFileLink extends FileLink 
    * @return True if the path is a HFileLink.
    */
   public static boolean isHFileLink(String fileName) {
-    Matcher m = LINK_NAME_PARSER.matcher(fileName);
+    Matcher m = LINK_NAME_PATTERN.matcher(fileName);
     if (!m.matches()) return false;
 
-    return m.groupCount() > 2 && m.group(2) != null && m.group(3) != null;
+    return m.groupCount() > 2 && m.group(3) != null && m.group(2) != null && m.group(1) != null;
   }
 
   /**
@@ -174,15 +191,15 @@ public class HFileLink extends FileLink 
    */
   private static Path getRelativeTablePath(final Path path) {
     // hfile-region-table
-    Matcher m = LINK_NAME_PARSER.matcher(path.getName());
+    Matcher m = LINK_NAME_PATTERN.matcher(path.getName());
     if (!m.matches()) {
       throw new IllegalArgumentException(path.getName() + " is not a valid HFileLink name!");
     }
 
     // Convert the HFileLink name into a real table/region/cf/hfile path.
-    String hfileName = m.group(1);
+    String tableName = m.group(1);
     String regionName = m.group(2);
-    String tableName = m.group(3);
+    String hfileName = m.group(3);
     String familyName = path.getParent().getName();
     return new Path(new Path(tableName, regionName), new Path(familyName, hfileName));
   }
@@ -194,11 +211,11 @@ public class HFileLink extends FileLink 
    * @return the name of the referenced HFile
    */
   public static String getReferencedHFileName(final String fileName) {
-    Matcher m = LINK_NAME_PARSER.matcher(fileName);
+    Matcher m = LINK_NAME_PATTERN.matcher(fileName);
     if (!m.matches()) {
       throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
     }
-    return(m.group(1));
+    return(m.group(3));
   }
 
   /**
@@ -208,7 +225,7 @@ public class HFileLink extends FileLink 
    * @return the name of the referenced Region
    */
   public static String getReferencedRegionName(final String fileName) {
-    Matcher m = LINK_NAME_PARSER.matcher(fileName);
+    Matcher m = LINK_NAME_PATTERN.matcher(fileName);
     if (!m.matches()) {
       throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
     }
@@ -222,11 +239,11 @@ public class HFileLink extends FileLink 
    * @return the name of the referenced Table
    */
   public static String getReferencedTableName(final String fileName) {
-    Matcher m = LINK_NAME_PARSER.matcher(fileName);
+    Matcher m = LINK_NAME_PATTERN.matcher(fileName);
     if (!m.matches()) {
       throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
     }
-    return(m.group(3));
+    return(m.group(1));
   }
 
   /**
@@ -252,7 +269,7 @@ public class HFileLink extends FileLink 
    */
   public static String createHFileLinkName(final String tableName,
       final String regionName, final String hfileName) {
-    return String.format("%s-%s-%s", hfileName, regionName, tableName);
+    return String.format("%s=%s-%s", tableName, regionName, hfileName);
   }
 
   /**
@@ -340,11 +357,11 @@ public class HFileLink extends FileLink 
    */
   public static boolean createFromHFileLink(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final String hfileLinkName) throws IOException {
-    Matcher m = LINK_NAME_PARSER.matcher(hfileLinkName);
+    Matcher m = LINK_NAME_PATTERN.matcher(hfileLinkName);
     if (!m.matches()) {
       throw new IllegalArgumentException(hfileLinkName + " is not a valid HFileLink name!");
     }
-    return create(conf, fs, dstFamilyPath, m.group(3), m.group(2), m.group(1));
+    return create(conf, fs, dstFamilyPath, m.group(1), m.group(2), m.group(3));
   }
 
   /**

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1445836&r1=1445835&r2=1445836&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java Wed Feb 13 18:46:32 2013
@@ -156,8 +156,6 @@ public class StoreFile {
   // If this storefile is a link to another, this is the link instance.
   private HFileLink link;
 
-  private Configuration conf;
-
   // Block cache configuration and reference.
   private final CacheConfig cacheConf;
 
@@ -203,24 +201,25 @@ public class StoreFile {
   private Map<byte[], byte[]> metadataMap;
 
   /**
-   * Regex that will work for straight filenames (<hfile>) and for reference names
-   * (<hfile>.<parentEncRegion>).  If reference, then the regex has more than just one group.
-   * Group 1, '([0-9a-f]+(?:_SeqId_[0-9]+_0?)',  is this file's id.  Group 2 '(.+)' is the
-   * reference's parent region name.  The ?: paren expressions are non-capture markers so not
-   * included in the groups count. The _SeqId_ portion comes from bulk loaded files.
-   */
-  public static final String REF_NAME_REGEX = "^([0-9a-f]+(?:_SeqId_[0-9]+_)?)(?:\\.(.+))?$";
-  private static final Pattern REF_NAME_PARSER = Pattern.compile(REF_NAME_REGEX);
-
-  /**
-   * Regex strictly for references to hfilelinks.  (<hfile>-<region>-<table>.<parentEncRegion>).
-   * Group 1 is this file's hfilelink name.  Group 2 the referenced parent region name.  The '.'
-   * char is valid in table names but group 2's regex is greedy and interprets the table names
-   * correctly.   The _SeqId_ portion comes from bulk loaded files.
-   */
-  public static final String REF_TO_LINK_REGEX = "^([0-9a-f]+(?:_SeqId_[0-9]+_)?-[0-9a-f]+-"
-      + HTableDescriptor.VALID_USER_TABLE_REGEX + "+)\\.([^.]+)$";
-  private static final Pattern REF_TO_LINK_PARSER = Pattern.compile(REF_TO_LINK_REGEX);
+   * A non-capture group, for hfiles, so that this can be embedded.
+   * HFiles are uuid ([0-9a-z]+). Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix.
+   */
+  public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:_SeqId_[0-9]+_)?";
+
+  /** Regex that will work for hfiles */
+  private static final Pattern HFILE_NAME_PATTERN =
+    Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
+
+  /**
+   * Regex that will work for straight reference names (<hfile>.<parentEncRegion>)
+   * and hfilelink reference names (<table>=<region>-<hfile>.<parentEncRegion>)
+   * If reference, then the regex has more than just one group.
+   * Group 1, hfile/hfilelink pattern, is this file's id.
+   * Group 2 '(.+)' is the reference's parent region name.
+   */
+  private static final Pattern REF_NAME_PATTERN =
+    Pattern.compile(String.format("^(%s|%s)\\.(.+)$",
+      HFILE_NAME_REGEX, HFileLink.LINK_NAME_REGEX));
 
   // StoreFile.Reader
   private volatile Reader reader;
@@ -259,7 +258,6 @@ public class StoreFile {
       throws IOException {
     this.fs = fs;
     this.path = p;
-    this.conf = conf;
     this.cacheConf = cacheConf;
     this.dataBlockEncoder =
         dataBlockEncoder == null ? NoOpDataBlockEncoder.INSTANCE
@@ -271,6 +269,13 @@ public class StoreFile {
     } else if (isReference(p)) {
       this.reference = Reference.read(fs, p);
       this.referencePath = getReferredToFile(this.path);
+      if (HFileLink.isHFileLink(this.referencePath)) {
+        this.link = new HFileLink(conf, this.referencePath);
+      }
+      LOG.debug("Store file " + p + " is a " + reference.getFileRegion() +
+        " reference to " + this.referencePath);
+    } else if (!isHFile(p)) {
+      throw new IOException("path=" + path + " doesn't look like a valid StoreFile");
     }
 
     if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
@@ -288,7 +293,6 @@ public class StoreFile {
     } else {
       this.modificationTimeStamp = 0;
     }
-
   }
 
   /**
@@ -317,7 +321,12 @@ public class StoreFile {
    * @return <tt>true</tt> if this StoreFile is an HFileLink
    */
   boolean isLink() {
-    return this.link != null;
+    return this.link != null && this.reference == null;
+  }
+
+  private static boolean isHFile(final Path path) {
+    Matcher m = HFILE_NAME_PATTERN.matcher(path.getName());
+    return m.matches() && m.groupCount() > 0;
   }
 
   /**
@@ -325,22 +334,16 @@ public class StoreFile {
    * @return True if the path has format of a HStoreFile reference.
    */
   public static boolean isReference(final Path p) {
-    return !p.getName().startsWith("_") &&
-      isReference(p, REF_NAME_PARSER.matcher(p.getName()));
+    return isReference(p.getName());
   }
 
   /**
-   * @param p Path to check.
-   * @param m Matcher to use.
+   * @param name file name to check.
    * @return True if the path has format of a HStoreFile reference.
    */
-  public static boolean isReference(final Path p, final Matcher m) {
-    if (m == null || !m.matches()) {
-      LOG.warn("Failed match of store file name " + p.toString());
-      throw new RuntimeException("Failed match of store file name " +
-          p.toString());
-    }
-    return m.groupCount() > 1 && m.group(2) != null;
+  public static boolean isReference(final String name) {
+    Matcher m = REF_NAME_PATTERN.matcher(name);
+    return m.matches() && m.groupCount() > 1;
   }
 
   /*
@@ -350,38 +353,14 @@ public class StoreFile {
    * @return Calculated path to parent region file.
    * @throws IllegalArgumentException when path regex fails to match.
    */
-  static Path getReferredToFile(final Path p) {
-    Matcher m = REF_NAME_PARSER.matcher(p.getName());
+  public static Path getReferredToFile(final Path p) {
+    Matcher m = REF_NAME_PATTERN.matcher(p.getName());
     if (m == null || !m.matches()) {
       LOG.warn("Failed match of store file name " + p.toString());
       throw new IllegalArgumentException("Failed match of store file name " +
           p.toString());
     }
-    // Other region name is suffix on the passed Reference file name
-    String otherRegion = m.group(2);
-    // Tabledir is up two directories from where Reference was written.
-    Path tableDir = p.getParent().getParent().getParent();
-    String nameStrippedOfSuffix = m.group(1);
-    // Build up new path with the referenced region in place of our current
-    // region in the reference path.  Also strip regionname suffix from name.
-    return new Path(new Path(new Path(tableDir, otherRegion),
-      p.getParent().getName()), nameStrippedOfSuffix);
-  }
 
-  /*
-   * Return path to an hfilelink referred to by a Reference.  Presumes a directory
-   * hierarchy of <code>${hbase.rootdir}/tablename/regionname/familyname</code>.
-   * @param p Path to a Reference to hfilelink file.
-   * @return Calculated path to parent region file.
-   * @throws IllegalArgumentException when path regex fails to match.
-   */
-  static Path getReferredToLink(final Path p) {
-    Matcher m = REF_TO_LINK_PARSER.matcher(p.getName());
-    if (m == null || !m.matches()) {
-      LOG.warn("Failed match of ref to hfilelink name" + p.toString());
-      throw new IllegalArgumentException("Failed match of ref to hfilelink name " +
-          p.toString());
-    }
     // Other region name is suffix on the passed Reference file name
     String otherRegion = m.group(2);
     // Tabledir is up two directories from where Reference was written.
@@ -493,16 +472,15 @@ public class StoreFile {
    * If this estimate isn't good enough, we can improve it later.
    * @param fs  The FileSystem
    * @param reference  The reference
-   * @param reference  The referencePath
+   * @param status  The reference FileStatus
    * @return HDFS blocks distribution
    */
   static private HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(
-    FileSystem fs, Reference reference, Path referencePath) throws IOException {
-    if ( referencePath == null) {
+    FileSystem fs, Reference reference, FileStatus status) throws IOException {
+    if (status == null) {
       return null;
     }
 
-    FileStatus status = fs.getFileStatus(referencePath);
     long start = 0;
     long length = 0;
 
@@ -521,8 +499,14 @@ public class StoreFile {
    */
   private void computeHDFSBlockDistribution() throws IOException {
     if (isReference()) {
+      FileStatus status;
+      if (this.link != null) {
+        status = this.link.getFileStatus(fs);
+      } else {
+        status = fs.getFileStatus(this.referencePath);
+      }
       this.hdfsBlocksDistribution = computeRefFileHDFSBlockDistribution(
-        this.fs, this.reference, this.referencePath);
+        this.fs, this.reference, status);
     } else {
       FileStatus status;
       if (isLink()) {
@@ -547,37 +531,17 @@ public class StoreFile {
       throw new IllegalAccessError("Already open");
     }
     if (isReference()) {
-      this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
-          this.cacheConf, this.reference,
-          dataBlockEncoder.getEncodingInCache());
-    } else if (isLink()) {
-      try {
-        long size = link.getFileStatus(fs).getLen();
-        this.reader = new Reader(this.fs, this.path, link, size, this.cacheConf,
-            dataBlockEncoder.getEncodingInCache(), true);
-      } catch (FileNotFoundException fnfe) {
-        // This didn't actually link to another file!
-
-        // Handles the case where a file with the hfilelink pattern is actually a daughter
-        // reference to a hfile link.  This can occur when a cloned table's hfilelinks get split.
-        FileStatus actualRef = fs.getFileStatus(path);
-        long actualLen = actualRef.getLen();
-        if (actualLen == 0) {
-          LOG.error(path + " is a 0-len file, and actually an hfilelink missing target file!", fnfe);
-          throw fnfe;
-        }
-        LOG.debug("Size of link file is " + actualLen + "!= 0; treating as a reference to" +
-            " HFileLink " + path + "!");
-        this.reference = Reference.read(fs, this.path);
-        this.referencePath = getReferredToLink(this.path);
-        LOG.debug("Reference file "+ path + " referred to " + referencePath + "!");
-        link = new HFileLink(conf, referencePath);
-        this.reader = new HalfStoreFileReader(this.fs, this.referencePath, link,
-            this.cacheConf, this.reference,
-            dataBlockEncoder.getEncodingInCache());
-        LOG.debug("Store file " + path + " is loaded " + referencePath + " as a half store file" +
-            " reader to an HFileLink!");
+      if (this.link != null) {
+        this.reader = new HalfStoreFileReader(this.fs, this.referencePath, this.link,
+          this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
+      } else {
+        this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
+          this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
       }
+    } else if (isLink()) {
+      long size = link.getFileStatus(fs).getLen();
+      this.reader = new Reader(this.fs, this.path, link, size, this.cacheConf,
+          dataBlockEncoder.getEncodingInCache(), true);
     } else {
       this.reader = new Reader(this.fs, this.path, this.cacheConf,
           dataBlockEncoder.getEncodingInCache());
@@ -975,13 +939,14 @@ public class StoreFile {
     Reference r =
       top? Reference.createTopReference(splitRow): Reference.createBottomReference(splitRow);
     // Add the referred-to regions name as a dot separated suffix.
-    // See REF_NAME_PARSER regex above.  The referred-to regions name is
+    // See REF_NAME_REGEX regex above.  The referred-to regions name is
     // up in the path of the passed in <code>f</code> -- parentdir is family,
     // then the directory above is the region name.
     String parentRegionName = f.getPath().getParent().getParent().getName();
     // Write reference with same file id only with the other region name as
     // suffix and into the new region location (under same family).
     Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
+    LOG.info("StoreFile.split(): splitDir=" + splitDir + " p=" + p);
     return r.write(fs, p);
   }
 

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java?rev=1445836&r1=1445835&r2=1445836&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java Wed Feb 13 18:46:32 2013
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.hbase.snapshot;
 
+import java.io.InputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -44,11 +46,13 @@ import org.apache.hadoop.hbase.errorhand
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.FSVisitor;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.hadoop.io.IOUtils;
 
 /**
  * Helper to Restore/Clone a Snapshot
@@ -335,12 +339,31 @@ public class RestoreSnapshotHelper {
       final String hfileName) throws IOException {
     if (HFileLink.isHFileLink(hfileName)) {
       HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName);
+    } else if (StoreFile.isReference(hfileName)) {
+      restoreReferenceFile(familyDir, regionInfo, hfileName);
     } else {
       HFileLink.create(conf, fs, familyDir, regionInfo, hfileName);
     }
   }
 
   /**
+   * Create a new {@link Reference} as copy of the source one.
+   *
+   * @param familyDir destination directory for the store file
+   * @param regionInfo destination region info for the table
+   * @param hfileName reference file name
+   */
+  private void restoreReferenceFile(final Path familyDir, final HRegionInfo regionInfo,
+      final String hfileName) throws IOException {
+    Path inPath = new Path(new Path(new Path(snapshotDesc.getTable(),
+        regionInfo.getEncodedName()), familyDir.getName()), hfileName);
+    Path outPath = new Path(familyDir, StoreFile.getReferredToFile(inPath).getName());
+    InputStream in = new HFileLink(conf, inPath).open(fs);
+    OutputStream out = fs.create(outPath);
+    IOUtils.copyBytes(in, out, conf);
+  }
+
+  /**
    * Create a new {@link HRegionInfo} from the snapshot region info.
    * Keep the same startKey, endKey, regionId and split information but change
    * the table name.

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java?rev=1445836&r1=1445835&r2=1445836&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java Wed Feb 13 18:46:32 2013
@@ -94,8 +94,8 @@ public class TestStoreFile extends HBase
    * @throws Exception
    */
   public void testBasicHalfMapFile() throws Exception {
-    // Make up a directory hierarchy that has a regiondir and familyname.
-    Path outputDir = new Path(new Path(this.testDir, "regionname"),
+    // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname.
+    Path outputDir = new Path(new Path(this.testDir, "7e0102"),
         "familyname");
     StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf,
         this.fs, 2 * 1024)
@@ -109,7 +109,7 @@ public class TestStoreFile extends HBase
   private void writeStoreFile(final StoreFile.Writer writer) throws IOException {
     writeStoreFile(writer, Bytes.toBytes(getName()), Bytes.toBytes(getName()));
   }
-  
+
   // pick an split point (roughly halfway)
   byte[] SPLITKEY = new byte[] { (LAST_CHAR-FIRST_CHAR)/2, FIRST_CHAR};
 
@@ -141,12 +141,12 @@ public class TestStoreFile extends HBase
    */
   public void testReference()
   throws IOException {
-    Path storedir = new Path(new Path(this.testDir, "regionname"), "familyname");
-    Path dir = new Path(storedir, "1234567890");
+    // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname.
+    Path storedir = new Path(new Path(this.testDir, "7e0102"), "familyname");
     // Make a store file and write data to it.
     StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf,
         this.fs, 8 * 1024)
-            .withOutputDir(dir)
+            .withOutputDir(storedir)
             .build();
     writeStoreFile(writer);
     StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
@@ -160,7 +160,7 @@ public class TestStoreFile extends HBase
     kv = KeyValue.createKeyValueFromKey(reader.getLastKey());
     byte [] finalRow = kv.getRow();
     // Make a reference
-    Path refPath = StoreFile.split(fs, dir, hsf, midRow, true);
+    Path refPath = StoreFile.split(fs, storedir, hsf, midRow, true);
     StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
         StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
     // Now confirm that I can read from the reference and that it only gets
@@ -219,18 +219,21 @@ public class TestStoreFile extends HBase
   /**
    * Validate that we can handle valid tables with '.', '_', and '-' chars.
    */
-  public void testRefToHFileRegex() {
-    String[] legal = { "aaaa-bbbb-tablename.cccc", "aaaa-bbbb-table.with.dots.cccc",
-          "aaaa-bbbb-table-with-dashes.cccc", "aaaa-bbbb-table_with_unders.cccc",
-          "aaaa-bbbb-_table_starts_unders.cccc"};
-    for (String refToHFile : legal) {
-      LOG.info("Validating regex for '" + refToHFile + "'");
-      assertTrue(Pattern.matches(StoreFile.REF_TO_LINK_REGEX, refToHFile));
+  public void testStoreFileNames() {
+    String[] legalHFileLink = { "MyTable_02=abc012-def345", "MyTable_02.300=abc012-def345",
+      "MyTable_02-400=abc012-def345", "MyTable_02-400.200=abc012-def345",
+      "MyTable_02=abc012-def345_SeqId_1_", "MyTable_02=abc012-def345_SeqId_20_" };
+    for (String name: legalHFileLink) {
+      assertTrue("should be a valid link: " + name, HFileLink.isHFileLink(name));
+
+      String refName = name + ".6789";
+      assertTrue("should be a valid link reference: " + refName, StoreFile.isReference(refName));
     }
-    
-    String[] illegal = { "aaaa-bbbb--flkaj.cccc", "aaaa-bbbb-.flkaj.cccc" };
-    for (String bad : illegal) {
-      assertFalse(Pattern.matches(StoreFile.REF_TO_LINK_REGEX, bad));
+
+    String[] illegalHFileLink = { ".MyTable_02=abc012-def345", "-MyTable_02.300=abc012-def345",
+      "MyTable_02-400=abc0_12-def345", "MyTable_02-400.200=abc012-def345...." };
+    for (String name: illegalHFileLink) {
+      assertFalse("should not be a valid link: " + name, HFileLink.isHFileLink(name));
     }
   }
 
@@ -260,7 +263,7 @@ public class TestStoreFile extends HBase
 
     // create link to store file. <root>/clone/region/<cf>/<hfile>-<region>-<table>
     String target = "clone";
-    Path dstPath = new Path(rootDir, new Path(new Path(target, "region"), columnFamily));
+    Path dstPath = new Path(rootDir, new Path(new Path(target, "7e0102"), columnFamily));
     HFileLink.create(conf, this.fs, dstPath, hri, storeFilePath.getName());
     Path linkFilePath = new Path(dstPath,
                   HFileLink.createHFileLinkName(hri, storeFilePath.getName()));
@@ -269,9 +272,9 @@ public class TestStoreFile extends HBase
     // <root>/clone/splitA/<cf>/<reftohfilelink>,
     // <root>/clone/splitB/<cf>/<reftohfilelink>
     Path splitDirA = new Path(new Path(rootDir,
-        new Path(target, "splitA")), columnFamily);
+        new Path(target, "571A")), columnFamily);
     Path splitDirB = new Path(new Path(rootDir,
-        new Path(target, "splitB")), columnFamily);
+        new Path(target, "571B")), columnFamily);
     StoreFile f = new StoreFile(fs, linkFilePath, conf, cacheConf, BloomType.NONE,
         NoOpDataBlockEncoder.INSTANCE);
     byte[] splitRow = SPLITKEY;
@@ -825,8 +828,8 @@ public class TestStoreFile extends HBase
     long[] timestamps = new long[] {20,10,5,1};
     Scan scan = new Scan();
 
-    Path storedir = new Path(new Path(this.testDir, "regionname"),
-    "familyname");
+    // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname.
+    Path storedir = new Path(new Path(this.testDir, "7e0102"), "familyname");
     Path dir = new Path(storedir, "1234567890");
     StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf,
         this.fs, 8 * 1024)
@@ -869,8 +872,8 @@ public class TestStoreFile extends HBase
   public void testCacheOnWriteEvictOnClose() throws Exception {
     Configuration conf = this.conf;
 
-    // Find a home for our files
-    Path baseDir = new Path(new Path(this.testDir, "regionname"),"twoCOWEOC");
+    // Find a home for our files (regiondir ("7e0102") and familyname).
+    Path baseDir = new Path(new Path(this.testDir, "7e0102"),"twoCOWEOC");
 
     // Grab the block cache and get the initial hit/miss counts
     BlockCache bc = new CacheConfig(conf).getBlockCache();
@@ -1022,7 +1025,8 @@ public class TestStoreFile extends HBase
    * file info.
    */
   public void testDataBlockEncodingMetaData() throws IOException {
-    Path dir = new Path(new Path(this.testDir, "regionname"), "familyname");
+    // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname.
+    Path dir = new Path(new Path(this.testDir, "7e0102"), "familyname");
     Path path = new Path(dir, "1234567890");
 
     DataBlockEncoding dataBlockEncoderAlgo =