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 2008/05/27 22:06:50 UTC

svn commit: r660681 - in /hadoop/hbase/branches/0.1: CHANGES.txt src/java/org/apache/hadoop/hbase/HStore.java src/java/org/apache/hadoop/hbase/HStoreFile.java

Author: stack
Date: Tue May 27 13:06:50 2008
New Revision: 660681

URL: http://svn.apache.org/viewvc?rev=660681&view=rev
Log:
HBASE-645  EOFException opening region (HBASE-550 redux)

Modified:
    hadoop/hbase/branches/0.1/CHANGES.txt
    hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStore.java
    hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStoreFile.java

Modified: hadoop/hbase/branches/0.1/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.1/CHANGES.txt?rev=660681&r1=660680&r2=660681&view=diff
==============================================================================
--- hadoop/hbase/branches/0.1/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.1/CHANGES.txt Tue May 27 13:06:50 2008
@@ -2,6 +2,7 @@
 
   BUG FIXES
    HBASE-644   DroppedSnapshotException but RegionServer doesn't restart
+   HBASE-645   EOFException opening region (HBASE-550 redux)
 
 Release 0.1.2 - 05/13/2008
 

Modified: hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStore.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStore.java?rev=660681&r1=660680&r2=660681&view=diff
==============================================================================
--- hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStore.java (original)
+++ hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStore.java Tue May 27 13:06:50 2008
@@ -19,6 +19,7 @@
  */
 package org.apache.hadoop.hbase;
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.rmi.UnexpectedException;
@@ -846,8 +847,24 @@
     // Finally, start up all the map readers! (There could be more than one
     // since we haven't compacted yet.)
     for(Map.Entry<Long, HStoreFile> e: this.storefiles.entrySet()) {
-      this.readers.put(e.getKey(),
-        e.getValue().getReader(this.fs, this.bloomFilter));
+      MapFile.Reader r = null;
+      try {
+        r = e.getValue().getReader(this.fs, this.bloomFilter);
+      } catch (EOFException eofe) {
+        LOG.warn("Failed open of reader " + e.toString() + "; attempting fix",
+          eofe);
+        try {
+          // Try fixing this file.. if we can.  
+          MapFile.fix(this.fs, e.getValue().getMapFilePath(),
+            HStoreFile.HbaseMapFile.KEY_CLASS,
+            HStoreFile.HbaseMapFile.VALUE_CLASS, false, this.conf);
+        } catch (Exception fixe) {
+          LOG.warn("Failed fix of " + e.toString() +
+            "...continuing; Probable DATA LOSS!!!", fixe);
+          continue;
+        }
+      }
+      this.readers.put(e.getKey(), r);
     }
   }
   
@@ -991,7 +1008,7 @@
       if (!fs.exists(mapfile)) {
         fs.delete(curfile.getInfoFilePath());
         LOG.warn("Mapfile " + mapfile.toString() + " does not exist. " +
-          "Cleaned up info file.  Continuing...");
+          "Cleaned up info file.  Continuing...Probable DATA LOSS!!!");
         continue;
       }
       

Modified: hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStoreFile.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStoreFile.java?rev=660681&r1=660680&r2=660681&view=diff
==============================================================================
--- hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStoreFile.java (original)
+++ hadoop/hbase/branches/0.1/src/java/org/apache/hadoop/hbase/HStoreFile.java Tue May 27 13:06:50 2008
@@ -373,7 +373,6 @@
   public synchronized MapFile.Reader getReader(final FileSystem fs,
       final Filter bloomFilter)
   throws IOException {
-    
     if (isReference()) {
       return new HStoreFile.HalfMapFileReader(fs,
           getMapFilePath(reference).toString(), conf, 
@@ -402,8 +401,7 @@
         "HStoreFile reference");
     }
     return new BloomFilterMapFile.Writer(conf, fs,
-      getMapFilePath().toString(), HStoreKey.class,
-      ImmutableBytesWritable.class, compression, bloomFilter);
+      getMapFilePath().toString(), compression, bloomFilter);
   }
 
   /**
@@ -540,6 +538,8 @@
    * Hbase customizations of MapFile.
    */
   static class HbaseMapFile extends MapFile {
+    static final Class KEY_CLASS = HStoreKey.class;
+    static final Class VALUE_CLASS = ImmutableBytesWritable.class;
 
     static class HbaseReader extends MapFile.Reader {
       
@@ -575,10 +575,9 @@
        * @throws IOException
        */
       public HbaseWriter(Configuration conf, FileSystem fs, String dirName,
-          Class<Writable> keyClass, Class<Writable> valClass,
-          SequenceFile.CompressionType compression)
+        SequenceFile.CompressionType compression)
       throws IOException {
-        super(conf, fs, dirName, keyClass, valClass, compression);
+        super(conf, fs, dirName, KEY_CLASS, VALUE_CLASS, compression);
         // Default for mapfiles is 128.  Makes random reads faster if we
         // have more keys indexed and we're not 'next'-ing around in the
         // mapfile.
@@ -665,10 +664,9 @@
        */
       @SuppressWarnings("unchecked")
       public Writer(Configuration conf, FileSystem fs, String dirName,
-          Class keyClass, Class valClass,
-          SequenceFile.CompressionType compression, final Filter filter)
+        SequenceFile.CompressionType compression, final Filter filter)
       throws IOException {
-        super(conf, fs, dirName, keyClass, valClass, compression);
+        super(conf, fs, dirName, compression);
         bloomFilter = filter;
       }