From cfef20aa1884fb775db11c4a53fc09e22f953b89 Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Fri, 11 Sep 2009 19:53:27 +0000 Subject: [PATCH] HBASE-1795 log recovery doesnt reset the max sequence id, new logfiles can get tossed as 'duplicates' HBASE-1794 recovered log files are not inserted into the storefile map git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@814003 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 7 ++-- .../hadoop/hbase/regionserver/Store.java | 34 +++++++++++++++---- 2 files changed, 30 insertions(+), 11 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 492c11c27c5..89a21ecfdf6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -14,6 +14,9 @@ Release 0.21.0 - Unreleased HBASE-1779 ThriftServer logged error if getVer() result is empty HBASE-1778 Improve PerformanceEvaluation (Schubert Zhang via Stack) HBASE-1751 Fix KeyValue javadoc on getValue for client-side + HBASE-1795 log recovery doesnt reset the max sequence id, new logfiles can + get tossed as 'duplicates' + HBASE-1794 recovered log files are not inserted into the storefile map HBASE-1824 [stargate] default timestamp should be LATEST_TIMESTAMP IMPROVEMENTS @@ -35,12 +38,8 @@ Release 0.21.0 - Unreleased HBASE-1800 Too many ZK connections HBASE-1819 Update to 0.20.1 hadoop and zk 3.2.1 HBASE-1820 Update jruby from 1.2 to 1.3.1 - HBASE-1722 Add support for exporting HBase metrics via JMX - HBASE-1825 code cleanup, hmaster split debug logs OPTIMIZATIONS - HBASE-1765 Delay Result deserialization until asked for and permit - access to the raw binary to prevent forced deserialization Release 0.20.0 - Tue Sep 8 12:53:05 PDT 2009 diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/java/org/apache/hadoop/hbase/regionserver/Store.java index c033758114c..97b14d00a84 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -217,7 +217,10 @@ public class Store implements HConstants, HeapSize { this.storefiles.putAll(loadStoreFiles()); // Do reconstruction log. - runReconstructionLog(reconstructionLog, this.maxSeqId, reporter); + long newId = runReconstructionLog(reconstructionLog, this.maxSeqId, reporter); + if (newId != -1) { + this.maxSeqId = newId; // start with the log id we just recovered. + } } HColumnDescriptor getFamily() { @@ -245,13 +248,14 @@ public class Store implements HConstants, HeapSize { * @param reconstructionLog * @param msid * @param reporter + * @return the new max sequence id as per the log * @throws IOException */ - private void runReconstructionLog(final Path reconstructionLog, + private long runReconstructionLog(final Path reconstructionLog, final long msid, final Progressable reporter) throws IOException { try { - doReconstructionLog(reconstructionLog, msid, reporter); + return doReconstructionLog(reconstructionLog, msid, 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 @@ -268,6 +272,7 @@ public class Store implements HConstants, HeapSize { " opening " + Bytes.toString(this.storeName), e); throw e; } + return -1; } /* @@ -277,20 +282,22 @@ public class Store implements HConstants, HeapSize { * We can ignore any log message that has a sequence ID that's equal to or * lower than maxSeqID. (Because we know such log messages are already * reflected in the MapFiles.) + * + * @return the new max sequence id as per the log, or -1 if no log recovered */ - private void doReconstructionLog(final Path reconstructionLog, + private long doReconstructionLog(final Path reconstructionLog, final long maxSeqID, final Progressable reporter) throws UnsupportedEncodingException, IOException { if (reconstructionLog == null || !this.fs.exists(reconstructionLog)) { // Nothing to do. - return; + return -1; } // Check its not empty. FileStatus [] stats = this.fs.listStatus(reconstructionLog); if (stats == null || stats.length == 0) { LOG.warn("Passed reconstruction log " + reconstructionLog + " is zero-length"); - return; + return -1; } // TODO: This could grow large and blow heap out. Need to get it into // general memory usage accounting. @@ -352,8 +359,21 @@ public class Store implements HConstants, HeapSize { if (LOG.isDebugEnabled()) { LOG.debug("flushing reconstructionCache"); } - internalFlushCache(reconstructedCache, maxSeqIdInLog + 1); + + long newFileSeqNo = maxSeqIdInLog + 1; + StoreFile sf = internalFlushCache(reconstructedCache, newFileSeqNo); + // add it to the list of store files with maxSeqIdInLog+1 + if (sf == null) { + throw new IOException("Flush failed with a null store file"); + } + // Add new file to store files. Clear snapshot too while we have the + // Store write lock. + this.storefiles.put(newFileSeqNo, sf); + notifyChangedReadersObservers(); + + return newFileSeqNo; } + return -1; // the reconstructed cache was 0 sized } /*