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/09/11 21:53:30 UTC
svn commit: r814003 - in /hadoop/hbase/trunk: CHANGES.txt
src/java/org/apache/hadoop/hbase/regionserver/Store.java
Author: stack
Date: Fri Sep 11 19:53:27 2009
New Revision: 814003
URL: http://svn.apache.org/viewvc?rev=814003&view=rev
Log:
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
Modified:
hadoop/hbase/trunk/CHANGES.txt
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java
Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=814003&r1=814002&r2=814003&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Fri Sep 11 19:53:27 2009
@@ -14,6 +14,9 @@
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 @@
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
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=814003&r1=814002&r2=814003&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java Fri Sep 11 19:53:27 2009
@@ -217,7 +217,10 @@
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 @@
* @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 @@
" opening " + Bytes.toString(this.storeName), e);
throw e;
}
+ return -1;
}
/*
@@ -277,20 +282,22 @@
* 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 @@
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
}
/*