Author: stack Date: Tue May 27 14:42:14 2008 New Revision: 660727 URL: http://svn.apache.org/viewvc?rev=660727&view=rev Log: HBASE-645 and HBASE-642 Splitting log in a hostile environment -- bad hdfs -- we drop write-ahead-log edits
Modified: hadoop/hbase/trunk/CHANGES.txt hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java Modified: hadoop/hbase/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=660727&r1=660726&r2=660727&view=diff ============================================================================== --- hadoop/hbase/trunk/CHANGES.txt (original) +++ hadoop/hbase/trunk/CHANGES.txt Tue May 27 14:42:14 2008 @@ -24,6 +24,8 @@ HBASE-638 Purge \r from src HBASE-644 DroppedSnapshotException but RegionServer doesn't restart HBASE-641 Improve master split logging + HBASE-642 Splitting log in a hostile environment -- bad hdfs -- we drop + write-ahead-log edits IMPROVEMENTS HBASE-559 MR example job to count table rows Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=660727&r1=660726&r2=660727&view=diff ============================================================================== --- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java (original) +++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java Tue May 27 14:42:14 2008 @@ -19,6 +19,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.io.EOFException; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.ArrayList; @@ -240,30 +241,53 @@ try { doReconstructionLog(reconstructionLog, maxSeqId, reporter); + } catch (EOFException e) { + // Presume we got here because of lack of HADOOP-1700; for now keep going + // but this is probably not what we want long term. If we got here there + // has been data-loss + LOG.warn("Exception processing reconstruction log " + reconstructionLog + + " opening " + this.storeName + + " -- continuing. Probably lack-of-HADOOP-1700 causing DATA LOSS!", e); } catch (IOException e) { - // Presume we got here because of some HDFS issue or because of a lack of - // HADOOP-1700; for now keep going but this is probably not what we want - // long term. If we got here there has been data-loss + // Presume we got here because of some HDFS issue. Don't just keep going. + // Fail to open the HStore. Probably means we'll fail over and over + // again until human intervention but alternative has us skipping logs + // and losing edits: HBASE-642. LOG.warn("Exception processing reconstruction log " + reconstructionLog + - " opening " + Bytes.toString(this.storeName) + - " -- continuing. Probably DATA LOSS!", e); + " opening " + this.storeName, e); + throw e; } // Finally, start up all the map readers! (There could be more than one // since we haven't compacted yet.) boolean first = true; for(Map.Entry<Long, HStoreFile> e: this.storefiles.entrySet()) { - if (first) { - // Use a block cache (if configured) for the first reader only - // so as to control memory usage. - this.readers.put(e.getKey(), - e.getValue().getReader(this.fs, this.bloomFilter, - family.isBlockCacheEnabled())); - first = false; - } else { - this.readers.put(e.getKey(), - e.getValue().getReader(this.fs, this.bloomFilter)); + MapFile.Reader r = null; + try { + if (first) { + // Use a block cache (if configured) for the first reader only + // so as to control memory usage. + r = e.getValue().getReader(this.fs, this.bloomFilter, + family.isBlockCacheEnabled()); + first = false; + } else { + 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); } } @@ -407,7 +431,7 @@ if (!fs.exists(mapfile)) { fs.delete(curfile.getInfoFilePath(), false); 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/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java?rev=660727&r1=660726&r2=660727&view=diff ============================================================================== --- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java (original) +++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java Tue May 27 14:42:14 2008 @@ -401,7 +401,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, @@ -452,8 +451,7 @@ "HStoreFile reference"); } return new BloomFilterMapFile.Writer(conf, fs, - getMapFilePath().toString(), HStoreKey.class, - ImmutableBytesWritable.class, compression, bloomFilter); + getMapFilePath().toString(), compression, bloomFilter); } /** @@ -624,6 +622,8 @@ * Hbase customizations of MapFile. */ static class HbaseMapFile extends MapFile { + static final Class KEY_CLASS = HStoreKey.class; + static final Class VALUE_CLASS = ImmutableBytesWritable.class; /** * A reader capable of reading and caching blocks of the data file. @@ -702,10 +702,9 @@ * @throws IOException */ public HbaseWriter(Configuration conf, FileSystem fs, String dirName, - Class<Writable> keyClass, Class<Writable> valClass, 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. @@ -807,10 +806,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; }