You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/05/02 03:58:55 UTC

svn commit: r770844 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/master/ src/java/org/apache/hadoop/hbase/regionserver/

Author: stack
Date: Fri May  1 22:34:03 2009
New Revision: 770844

URL: http://svn.apache.org/viewvc?rev=770844&view=rev
Log:
HBASE-1342 Add to filesystem info needed to rebuild .META.

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=770844&r1=770843&r2=770844&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Fri May  1 22:34:03 2009
@@ -14,6 +14,7 @@
    HBASE-1234  Change HBase StoreKey format
    HBASE-1348  Move 0.20.0 targeted TRUNK to 0.20.0 hadoop
                (Ryan Rawson and Stack)
+   HBASE-1342  Add to filesystem info needed to rebuild .META.
 
   BUG FIXES
    HBASE-1140  "ant clean test" fails (Nitay Joffe via Stack)

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=770844&r1=770843&r2=770844&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java Fri May  1 22:34:03 2009
@@ -279,7 +279,8 @@
    * @return <code>b</code>
    * @throws IllegalArgumentException If not null and not a legitimate family
    * name: i.e. 'printable' and ends in a ':' (Null passes are allowed because
-   * <code>b</code> can be null when deserializing).
+   * <code>b</code> can be null when deserializing).  Cannot start with a '.'
+   * either.
    */
   public static byte [] isLegalFamilyName(final byte [] b) {
     if (b == null) {
@@ -289,6 +290,10 @@
       throw new IllegalArgumentException("Family names must end in a colon: " +
         Bytes.toString(b));
     }
+    if (b[0] == '.') {
+      throw new IllegalArgumentException("Family names cannot start with a " +
+        "period: " + Bytes.toString(b));
+    }
     for (int i = 0; i < (b.length - 1); i++) {
       if (Character.isISOControl(b[i])) {
         throw new IllegalArgumentException("Illegal character <" + b[i] +

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java?rev=770844&r1=770843&r2=770844&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java Fri May  1 22:34:03 2009
@@ -191,7 +191,7 @@
     // If null or empty region, don't bother printing it out.
     if (this.info != null && this.info.getRegionName().length > 0) {
       sb.append(": ");
-      sb.append(this.info.getRegionNameAsString());
+      sb.append(this.info.toString());
     }
     if (this.message != null && this.message.length > 0) {
       sb.append(": " + Bytes.toString(this.message));

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java?rev=770844&r1=770843&r2=770844&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java Fri May  1 22:34:03 2009
@@ -259,7 +259,8 @@
         parent.getRegionName(), rowContent, COL_SPLITB);
     if (!hasReferencesA && !hasReferencesB) {
       LOG.info("Deleting region " + parent.getRegionNameAsString() +
-        " because daughter splits no longer hold references");
+        " (encoded=" + parent.getEncodedName() +
+        ") because daughter splits no longer hold references");
       HRegion.deleteRegion(this.master.fs, this.master.rootdir, parent);
       HRegion.removeRegionFromMETA(srvr, metaRegionName,
         parent.getRegionName());

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=770844&r1=770843&r2=770844&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri May  1 22:34:03 2009
@@ -40,6 +40,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -196,6 +197,17 @@
   private final Object splitLock = new Object();
   private long minSequenceId;
   final AtomicInteger activeScannerCount = new AtomicInteger(0);
+  
+  /**
+   * Name of the region info file that resides just under the region directory.
+   */
+  public final static String REGIONINFO_FILE = ".regioninfo";
+
+  /**
+   * REGIONINFO_FILE as byte array.
+   */
+  public final static byte [] REGIONINFO_FILE_BYTES =
+    Bytes.toBytes(REGIONINFO_FILE);
 
   /**
    * HRegion constructor.
@@ -261,6 +273,9 @@
   public void initialize( Path initialFiles, final Progressable reporter)
   throws IOException {
     Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
+    
+    // Write HRI to a file in case we need to recover .META.
+    checkRegioninfoOnFilesystem();
 
     // Move prefab HStore files into place (if any).  This picks up split files
     // and any merges from splits and merges dirs.
@@ -316,6 +331,32 @@
       " available");
   }
 
+  /*
+   * Write out an info file under the region directory.  Useful recovering
+   * mangled regions.
+   * @throws IOException
+   */
+  private void checkRegioninfoOnFilesystem() throws IOException {
+    // Name of this file has two leading and trailing underscores so it doesn't
+    // clash w/ a store/family name.  There is possibility, but assumption is
+    // that its slim (don't want to use control character in filename because
+    // 
+    Path regioninfo = new Path(this.regiondir, REGIONINFO_FILE);
+    if (this.fs.exists(regioninfo) &&
+        this.fs.getFileStatus(regioninfo).getLen() > 0) {
+      return;
+    }
+    FSDataOutputStream out = this.fs.create(regioninfo, true);
+    try {
+      this.regionInfo.write(out);
+      out.write('\n');
+      out.write('\n');
+      out.write(Bytes.toBytes(this.regionInfo.toString()));
+    } finally {
+      out.close();
+    }
+  }
+
   /**
    * @return Updates to this region need to have a sequence id that is >= to
    * the this number.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=770844&r1=770843&r2=770844&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri May  1 22:34:03 2009
@@ -1280,7 +1280,7 @@
   void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA,
       HRegionInfo newRegionB) {
     outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT, oldRegion,
-      (oldRegion.getRegionNameAsString() + " split; daughters: " +
+      ("Daughters; " +
         newRegionA.getRegionNameAsString() + ", " +
         newRegionB.getRegionNameAsString()).getBytes()));
     outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionA));