HADOOP-1943 LogRolling test fails: reverting changes for HADOOP-1820
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@579353 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2cf41fe535
commit
87f305b608
|
@ -45,6 +45,7 @@ Trunk (unreleased changes)
|
||||||
HADOOP-1813 OOME makes zombie of region server
|
HADOOP-1813 OOME makes zombie of region server
|
||||||
HADOOP-1814 TestCleanRegionServerExit fails too often on Hudson
|
HADOOP-1814 TestCleanRegionServerExit fails too often on Hudson
|
||||||
HADOOP-1820 Regionserver creates hlogs without bound
|
HADOOP-1820 Regionserver creates hlogs without bound
|
||||||
|
(reverted 2007/09/25)
|
||||||
HADOOP-1821 Replace all String.getBytes() with String.getBytes("UTF-8")
|
HADOOP-1821 Replace all String.getBytes() with String.getBytes("UTF-8")
|
||||||
HADOOP-1832 listTables() returns duplicate tables
|
HADOOP-1832 listTables() returns duplicate tables
|
||||||
HADOOP-1834 Scanners ignore timestamp passed on creation
|
HADOOP-1834 Scanners ignore timestamp passed on creation
|
||||||
|
|
|
@ -29,9 +29,6 @@ import org.apache.hadoop.conf.*;
|
||||||
import java.io.*;
|
import java.io.*;
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
|
||||||
import java.util.concurrent.locks.Lock;
|
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* HLog stores all the edits to the HStore.
|
* HLog stores all the edits to the HStore.
|
||||||
|
@ -56,11 +53,6 @@ import java.util.concurrent.locks.ReentrantLock;
|
||||||
* older (smaller) than the most-recent CACHEFLUSH message for every HRegion
|
* older (smaller) than the most-recent CACHEFLUSH message for every HRegion
|
||||||
* that has a message in F.
|
* that has a message in F.
|
||||||
*
|
*
|
||||||
* <p>synchronized methods can never execute in parallel. However, between the
|
|
||||||
* start of a cache flush and the completion point, appends are allowed but log
|
|
||||||
* rolling is not. To prevent log rolling taking place during this period, a
|
|
||||||
* separate reentrant lock is used.
|
|
||||||
*
|
|
||||||
* <p>TODO: Vuk Ercegovac also pointed out that keeping HBase HRegion edit logs
|
* <p>TODO: Vuk Ercegovac also pointed out that keeping HBase HRegion edit logs
|
||||||
* in HDFS is currently flawed. HBase writes edits to logs and to a memcache.
|
* in HDFS is currently flawed. HBase writes edits to logs and to a memcache.
|
||||||
* The 'atomic' write to the log is meant to serve as insurance against
|
* The 'atomic' write to the log is meant to serve as insurance against
|
||||||
|
@ -82,21 +74,20 @@ public class HLog implements HConstants {
|
||||||
|
|
||||||
SequenceFile.Writer writer;
|
SequenceFile.Writer writer;
|
||||||
TreeMap<Long, Path> outputfiles = new TreeMap<Long, Path>();
|
TreeMap<Long, Path> outputfiles = new TreeMap<Long, Path>();
|
||||||
HashMap<Text, Long> lastSeqWritten = new HashMap<Text, Long>();
|
volatile boolean insideCacheFlush = false;
|
||||||
|
|
||||||
|
TreeMap<Text, Long> regionToLastFlush = new TreeMap<Text, Long>();
|
||||||
|
|
||||||
volatile boolean closed = false;
|
volatile boolean closed = false;
|
||||||
AtomicLong logSeqNum = new AtomicLong(0);
|
volatile long logSeqNum = 0;
|
||||||
volatile long filenum = 0;
|
long filenum = 0;
|
||||||
AtomicInteger numEntries = new AtomicInteger(0);
|
AtomicInteger numEntries = new AtomicInteger(0);
|
||||||
|
|
||||||
// This lock prevents starting a log roll during a cache flush.
|
Integer rollLock = new Integer(0);
|
||||||
// synchronized is insufficient because a cache flush spans two method calls.
|
|
||||||
private final Lock cacheFlushLock = new ReentrantLock();
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Split up a bunch of log files, that are no longer being written to,
|
* Split up a bunch of log files, that are no longer being written to,
|
||||||
* into new files, one per region. Delete the old log files when finished.
|
* into new files, one per region. Delete the old log files when ready.
|
||||||
*
|
|
||||||
* @param rootDir Root directory of the HBase instance
|
* @param rootDir Root directory of the HBase instance
|
||||||
* @param srcDir Directory of log files to split:
|
* @param srcDir Directory of log files to split:
|
||||||
* e.g. <code>${ROOTDIR}/log_HOST_PORT</code>
|
* e.g. <code>${ROOTDIR}/log_HOST_PORT</code>
|
||||||
|
@ -190,51 +181,54 @@ public class HLog implements HConstants {
|
||||||
rollWriter();
|
rollWriter();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Called by HRegionServer when it opens a new region to ensure that log
|
|
||||||
* sequence numbers are always greater than the latest sequence number of
|
|
||||||
* the region being brought on-line.
|
|
||||||
*
|
|
||||||
* @param newvalue
|
|
||||||
*/
|
|
||||||
synchronized void setSequenceNumber(long newvalue) {
|
synchronized void setSequenceNumber(long newvalue) {
|
||||||
if (newvalue > logSeqNum.get()) {
|
if (newvalue > logSeqNum) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("changing sequence number from " + logSeqNum + " to " +
|
LOG.debug("changing sequence number from " + logSeqNum + " to " +
|
||||||
newvalue);
|
newvalue);
|
||||||
}
|
}
|
||||||
logSeqNum.set(newvalue);
|
logSeqNum = newvalue;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Roll the log writer. That is, start writing log messages to a new file.
|
* Roll the log writer. That is, start writing log messages to a new file.
|
||||||
*
|
*
|
||||||
* Because a log cannot be rolled during a cache flush, and a cache flush
|
* The 'rollLock' prevents us from entering rollWriter() more than
|
||||||
* spans two method calls, a special lock needs to be obtained so that a
|
* once at a time.
|
||||||
* cache flush cannot start when the log is being rolled and the log cannot
|
|
||||||
* be rolled during a cache flush.
|
|
||||||
*
|
*
|
||||||
* Note that this method cannot be synchronized because it is possible that
|
* The 'this' lock limits access to the current writer so
|
||||||
* startCacheFlush runs, obtaining the cacheFlushLock, then this method could
|
* we don't append multiple items simultaneously.
|
||||||
* start which would obtain the lock on this but block on obtaining the
|
|
||||||
* cacheFlushLock and then completeCacheFlush could be called which would
|
|
||||||
* wait for the lock on this and consequently never release the cacheFlushLock
|
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void rollWriter() throws IOException {
|
void rollWriter() throws IOException {
|
||||||
|
synchronized(rollLock) {
|
||||||
|
|
||||||
|
// Try to roll the writer to a new file. We may have to
|
||||||
|
// wait for a cache-flush to complete. In the process,
|
||||||
|
// compute a list of old log files that can be deleted.
|
||||||
|
|
||||||
|
Vector<Path> toDeleteList = new Vector<Path>();
|
||||||
|
synchronized(this) {
|
||||||
if(closed) {
|
if(closed) {
|
||||||
throw new IOException("Cannot roll log; log is closed");
|
throw new IOException("Cannot roll log; log is closed");
|
||||||
}
|
}
|
||||||
|
|
||||||
cacheFlushLock.lock(); // prevent cache flushes
|
// Make sure we do not roll the log while inside a
|
||||||
|
// cache-flush. Otherwise, the log sequence number for
|
||||||
|
// the CACHEFLUSH operation will appear in a "newer" log file
|
||||||
|
// than it should.
|
||||||
|
while(insideCacheFlush) {
|
||||||
try {
|
try {
|
||||||
// Now that we have locked out cache flushes, lock this to prevent other
|
wait();
|
||||||
// changes.
|
} catch (InterruptedException ie) {
|
||||||
|
// continue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
synchronized (this) {
|
// Close the current writer (if any), and grab a new one.
|
||||||
if (writer != null) { // Close the current writer (if any), get a new one.
|
if(writer != null) {
|
||||||
writer.close();
|
writer.close();
|
||||||
Path p = computeFilename(filenum - 1);
|
Path p = computeFilename(filenum - 1);
|
||||||
if(LOG.isDebugEnabled()) {
|
if(LOG.isDebugEnabled()) {
|
||||||
|
@ -242,13 +236,12 @@ public class HLog implements HConstants {
|
||||||
" to get a new one");
|
" to get a new one");
|
||||||
}
|
}
|
||||||
if (filenum > 0) {
|
if (filenum > 0) {
|
||||||
outputfiles.put(logSeqNum.get() - 1, p);
|
outputfiles.put(logSeqNum - 1, p);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
Path newPath = computeFilename(filenum++);
|
Path newPath = computeFilename(filenum++);
|
||||||
this.writer = SequenceFile.createWriter(fs, conf, newPath, HLogKey.class,
|
this.writer = SequenceFile.createWriter(fs, conf, newPath,
|
||||||
HLogEdit.class);
|
HLogKey.class, HLogEdit.class);
|
||||||
|
|
||||||
if(LOG.isDebugEnabled()) {
|
if(LOG.isDebugEnabled()) {
|
||||||
LOG.debug("new log writer created at " + newPath);
|
LOG.debug("new log writer created at " + newPath);
|
||||||
}
|
}
|
||||||
|
@ -258,7 +251,7 @@ public class HLog implements HConstants {
|
||||||
// over all the regions.
|
// over all the regions.
|
||||||
|
|
||||||
long oldestOutstandingSeqNum = Long.MAX_VALUE;
|
long oldestOutstandingSeqNum = Long.MAX_VALUE;
|
||||||
for (Long l: lastSeqWritten.values()) {
|
for(Long l: regionToLastFlush.values()) {
|
||||||
long curSeqNum = l.longValue();
|
long curSeqNum = l.longValue();
|
||||||
|
|
||||||
if(curSeqNum < oldestOutstandingSeqNum) {
|
if(curSeqNum < oldestOutstandingSeqNum) {
|
||||||
|
@ -266,18 +259,24 @@ public class HLog implements HConstants {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Get the set of all sequence numbers that are older than the oldest
|
// Next, remove all files with a final ID that's older
|
||||||
// pending region operation
|
// than the oldest pending region-operation.
|
||||||
|
for(Iterator<Long> it = outputfiles.keySet().iterator(); it.hasNext();) {
|
||||||
|
long maxSeqNum = it.next().longValue();
|
||||||
|
if(maxSeqNum < oldestOutstandingSeqNum) {
|
||||||
|
Path p = outputfiles.get(maxSeqNum);
|
||||||
|
it.remove();
|
||||||
|
toDeleteList.add(p);
|
||||||
|
|
||||||
TreeSet<Long> sequenceNumbers = new TreeSet<Long>();
|
} else {
|
||||||
sequenceNumbers.addAll(
|
break;
|
||||||
outputfiles.headMap(oldestOutstandingSeqNum).keySet());
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Remove all files with a final ID that's older than the oldest
|
// Actually delete them, if any!
|
||||||
// pending region-operation.
|
for(Iterator<Path> it = toDeleteList.iterator(); it.hasNext(); ) {
|
||||||
|
Path p = it.next();
|
||||||
for (Long seq: sequenceNumbers) {
|
|
||||||
Path p = outputfiles.remove(seq);
|
|
||||||
if(LOG.isDebugEnabled()) {
|
if(LOG.isDebugEnabled()) {
|
||||||
LOG.debug("removing old log file " + p.toString());
|
LOG.debug("removing old log file " + p.toString());
|
||||||
}
|
}
|
||||||
|
@ -285,10 +284,6 @@ public class HLog implements HConstants {
|
||||||
}
|
}
|
||||||
this.numEntries.set(0);
|
this.numEntries.set(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
} finally {
|
|
||||||
cacheFlushLock.unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -333,9 +328,7 @@ public class HLog implements HConstants {
|
||||||
* other systems should process the log appropriately upon each startup
|
* other systems should process the log appropriately upon each startup
|
||||||
* (and prior to initializing HLog).
|
* (and prior to initializing HLog).
|
||||||
*
|
*
|
||||||
* synchronized prevents appends during the completion of a cache flush or
|
* We need to seize a lock on the writer so that writes are atomic.
|
||||||
* for the duration of a log roll.
|
|
||||||
*
|
|
||||||
* @param regionName
|
* @param regionName
|
||||||
* @param tableName
|
* @param tableName
|
||||||
* @param row
|
* @param row
|
||||||
|
@ -344,19 +337,21 @@ public class HLog implements HConstants {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
synchronized void append(Text regionName, Text tableName, Text row,
|
synchronized void append(Text regionName, Text tableName, Text row,
|
||||||
TreeMap<Text, byte []> columns, long timestamp) throws IOException {
|
TreeMap<Text, byte []> columns, long timestamp)
|
||||||
|
throws IOException {
|
||||||
if(closed) {
|
if(closed) {
|
||||||
throw new IOException("Cannot append; log is closed");
|
throw new IOException("Cannot append; log is closed");
|
||||||
}
|
}
|
||||||
|
|
||||||
long seqNum[] = obtainSeqNum(columns.size());
|
long seqNum[] = obtainSeqNum(columns.size());
|
||||||
|
|
||||||
// The 'lastSeqWritten' map holds the sequence number of the most recent
|
// The 'regionToLastFlush' map holds the sequence id of the
|
||||||
// write for each region. When the cache is flushed, the entry for the
|
// most recent flush for every regionName. However, for regions
|
||||||
// region being flushed is removed if the sequence number of the flush
|
// that don't have any flush yet, the relevant operation is the
|
||||||
// is greater than or equal to the value in lastSeqWritten
|
// first one that's been added.
|
||||||
|
if (regionToLastFlush.get(regionName) == null) {
|
||||||
lastSeqWritten.put(regionName, seqNum[seqNum.length - 1]);
|
regionToLastFlush.put(regionName, seqNum[0]);
|
||||||
|
}
|
||||||
|
|
||||||
int counter = 0;
|
int counter = 0;
|
||||||
for (Map.Entry<Text, byte []> es: columns.entrySet()) {
|
for (Map.Entry<Text, byte []> es: columns.entrySet()) {
|
||||||
|
@ -368,39 +363,29 @@ public class HLog implements HConstants {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/** @return How many items have been added to the log */
|
||||||
* @return How many items have been added to the log
|
|
||||||
*
|
|
||||||
* Because numEntries is an AtomicInteger, no locking is required.
|
|
||||||
*/
|
|
||||||
int getNumEntries() {
|
int getNumEntries() {
|
||||||
return numEntries.get();
|
return numEntries.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Obtain a log sequence number.
|
* Obtain a log sequence number. This seizes the whole HLog
|
||||||
*
|
* lock, but it shouldn't last too long.
|
||||||
* Because it is only called from a synchronized method, no additional locking
|
|
||||||
* is required.
|
|
||||||
*/
|
*/
|
||||||
private long obtainSeqNum() {
|
synchronized long obtainSeqNum() {
|
||||||
return logSeqNum.getAndIncrement();
|
return logSeqNum++;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Obtain a specified number of sequence numbers
|
* Obtain a specified number of sequence numbers
|
||||||
*
|
*
|
||||||
* Because it is only called from a synchronized method, no additional locking
|
|
||||||
* is required.
|
|
||||||
*
|
|
||||||
* @param num - number of sequence numbers to obtain
|
* @param num - number of sequence numbers to obtain
|
||||||
* @return - array of sequence numbers
|
* @return - array of sequence numbers
|
||||||
*/
|
*/
|
||||||
private long[] obtainSeqNum(int num) {
|
synchronized long[] obtainSeqNum(int num) {
|
||||||
long sequenceNumber = logSeqNum.getAndAdd(num);
|
|
||||||
long[] results = new long[num];
|
long[] results = new long[num];
|
||||||
for (int i = 0; i < num; i++) {
|
for (int i = 0; i < num; i++) {
|
||||||
results[i] = sequenceNumber++;
|
results[i] = logSeqNum++;
|
||||||
}
|
}
|
||||||
return results;
|
return results;
|
||||||
}
|
}
|
||||||
|
@ -409,50 +394,54 @@ public class HLog implements HConstants {
|
||||||
* By acquiring a log sequence ID, we can allow log messages
|
* By acquiring a log sequence ID, we can allow log messages
|
||||||
* to continue while we flush the cache.
|
* to continue while we flush the cache.
|
||||||
*
|
*
|
||||||
* Acquire a lock so that we do not roll the log between the start
|
* Set a flag so that we do not roll the log between the start
|
||||||
* and completion of a cache-flush. Otherwise the log-seq-id for
|
* and complete of a cache-flush. Otherwise the log-seq-id for
|
||||||
* the flush will not appear in the correct logfile.
|
* the flush will not appear in the correct logfile.
|
||||||
*
|
|
||||||
* @return sequence ID to pass {@link #completeCacheFlush(Text, Text, long)}
|
* @return sequence ID to pass {@link #completeCacheFlush(Text, Text, long)}
|
||||||
* @see #completeCacheFlush(Text, Text, long)
|
* @see #completeCacheFlush(Text, Text, long)
|
||||||
* @see #abortCacheFlush()
|
* @see #abortCacheFlush()
|
||||||
*/
|
*/
|
||||||
synchronized long startCacheFlush() {
|
synchronized long startCacheFlush() {
|
||||||
cacheFlushLock.lock();
|
while (this.insideCacheFlush) {
|
||||||
|
try {
|
||||||
|
wait();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
// continue
|
||||||
|
}
|
||||||
|
}
|
||||||
|
this.insideCacheFlush = true;
|
||||||
|
notifyAll();
|
||||||
return obtainSeqNum();
|
return obtainSeqNum();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/** Complete the cache flush
|
||||||
* Complete the cache flush
|
|
||||||
*
|
|
||||||
* Protected by this.lock()
|
|
||||||
*
|
|
||||||
* @param regionName
|
* @param regionName
|
||||||
* @param tableName
|
* @param tableName
|
||||||
* @param logSeqId
|
* @param logSeqId
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
synchronized void completeCacheFlush(final Text regionName,
|
synchronized void completeCacheFlush(final Text regionName,
|
||||||
final Text tableName, final long logSeqId) throws IOException {
|
final Text tableName, final long logSeqId)
|
||||||
|
throws IOException {
|
||||||
try {
|
|
||||||
if(this.closed) {
|
if(this.closed) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
|
if (!this.insideCacheFlush) {
|
||||||
|
throw new IOException("Impossible situation: inside " +
|
||||||
|
"completeCacheFlush(), but 'insideCacheFlush' flag is false");
|
||||||
|
}
|
||||||
|
HLogKey key = new HLogKey(regionName, tableName, HLog.METAROW, logSeqId);
|
||||||
|
this.writer.append(key,
|
||||||
new HLogEdit(HLog.METACOLUMN, HGlobals.completeCacheFlush.get(),
|
new HLogEdit(HLog.METACOLUMN, HGlobals.completeCacheFlush.get(),
|
||||||
System.currentTimeMillis()));
|
System.currentTimeMillis()));
|
||||||
|
this.numEntries.getAndIncrement();
|
||||||
|
|
||||||
numEntries.getAndIncrement();
|
// Remember the most-recent flush for each region.
|
||||||
Long seq = lastSeqWritten.get(regionName);
|
// This is used to delete obsolete log files.
|
||||||
if (seq != null && logSeqId >= seq) {
|
this.regionToLastFlush.put(regionName, Long.valueOf(logSeqId));
|
||||||
lastSeqWritten.remove(regionName);
|
|
||||||
}
|
|
||||||
|
|
||||||
} finally {
|
cleanup();
|
||||||
cacheFlushLock.unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -462,8 +451,23 @@ public class HLog implements HConstants {
|
||||||
* is a restart of the regionserver so the snapshot content dropped by the
|
* is a restart of the regionserver so the snapshot content dropped by the
|
||||||
* failure gets restored to the memcache.
|
* failure gets restored to the memcache.
|
||||||
*/
|
*/
|
||||||
void abortCacheFlush() {
|
synchronized void abortCacheFlush() {
|
||||||
this.cacheFlushLock.unlock();
|
cleanup();
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized void cleanup() {
|
||||||
|
this.insideCacheFlush = false;
|
||||||
|
notifyAll();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abort a cache flush.
|
||||||
|
* This method will clear waits on {@link #insideCacheFlush} but if this
|
||||||
|
* method is called, we are losing data. TODO: Fix.
|
||||||
|
*/
|
||||||
|
synchronized void abort() {
|
||||||
|
this.insideCacheFlush = false;
|
||||||
|
notifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void usage() {
|
private static void usage() {
|
||||||
|
|
|
@ -210,7 +210,6 @@ public class HRegion implements HConstants {
|
||||||
final int memcacheFlushSize;
|
final int memcacheFlushSize;
|
||||||
final int blockingMemcacheSize;
|
final int blockingMemcacheSize;
|
||||||
protected final long threadWakeFrequency;
|
protected final long threadWakeFrequency;
|
||||||
protected final int optionalFlushCount;
|
|
||||||
private final HLocking lock = new HLocking();
|
private final HLocking lock = new HLocking();
|
||||||
private long desiredMaxFileSize;
|
private long desiredMaxFileSize;
|
||||||
private final long maxSequenceId;
|
private final long maxSequenceId;
|
||||||
|
@ -248,8 +247,6 @@ public class HRegion implements HConstants {
|
||||||
this.regionInfo = regionInfo;
|
this.regionInfo = regionInfo;
|
||||||
this.memcache = new HMemcache();
|
this.memcache = new HMemcache();
|
||||||
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
|
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||||
this.optionalFlushCount =
|
|
||||||
conf.getInt("hbase.hregion.memcache.optionalflushcount", 10);
|
|
||||||
|
|
||||||
// Declare the regionName. This is a unique string for the region, used to
|
// Declare the regionName. This is a unique string for the region, used to
|
||||||
// build a unique filename.
|
// build a unique filename.
|
||||||
|
@ -731,13 +728,11 @@ public class HRegion implements HConstants {
|
||||||
void optionallyFlush() throws IOException {
|
void optionallyFlush() throws IOException {
|
||||||
if(this.memcache.getSize() > this.memcacheFlushSize) {
|
if(this.memcache.getSize() > this.memcacheFlushSize) {
|
||||||
flushcache(false);
|
flushcache(false);
|
||||||
} else if (this.memcache.getSize() > 0) {
|
} else if (this.memcache.getSize() > 0 && this.noFlushCount >= 10) {
|
||||||
if (this.noFlushCount >= this.optionalFlushCount) {
|
|
||||||
LOG.info("Optional flush called " + this.noFlushCount +
|
LOG.info("Optional flush called " + this.noFlushCount +
|
||||||
" times when data present without flushing. Forcing one.");
|
" times when data present without flushing. Forcing one.");
|
||||||
flushcache(false);
|
flushcache(false);
|
||||||
|
if (this.memcache.getSize() > 0) {
|
||||||
} else {
|
|
||||||
// Only increment if something in the cache.
|
// Only increment if something in the cache.
|
||||||
// Gets zero'd when a flushcache is called.
|
// Gets zero'd when a flushcache is called.
|
||||||
this.noFlushCount++;
|
this.noFlushCount++;
|
||||||
|
@ -869,23 +864,12 @@ public class HRegion implements HConstants {
|
||||||
retval.memcacheSnapshot.size());
|
retval.memcacheSnapshot.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
|
||||||
// A. Flush memcache to all the HStores.
|
// A. Flush memcache to all the HStores.
|
||||||
// Keep running vector of all store files that includes both old and the
|
// Keep running vector of all store files that includes both old and the
|
||||||
// just-made new flush store file.
|
// just-made new flush store file.
|
||||||
for (HStore hstore: stores.values()) {
|
for (HStore hstore: stores.values()) {
|
||||||
hstore.flushCache(retval.memcacheSnapshot, retval.sequenceId);
|
hstore.flushCache(retval.memcacheSnapshot, retval.sequenceId);
|
||||||
}
|
}
|
||||||
} catch (IOException e) {
|
|
||||||
// An exception here means that the snapshot was not persisted.
|
|
||||||
// The hlog needs to be replayed so its content is restored to memcache.
|
|
||||||
// Currently, only a server restart will do this.
|
|
||||||
this.log.abortCacheFlush();
|
|
||||||
throw new DroppedSnapshotException(e.getMessage());
|
|
||||||
}
|
|
||||||
|
|
||||||
// If we get to here, the HStores have been written. If we get an
|
|
||||||
// error in completeCacheFlush it will release the lock it is holding
|
|
||||||
|
|
||||||
// B. Write a FLUSHCACHE-COMPLETE message to the log.
|
// B. Write a FLUSHCACHE-COMPLETE message to the log.
|
||||||
// This tells future readers that the HStores were emitted correctly,
|
// This tells future readers that the HStores were emitted correctly,
|
||||||
|
@ -893,7 +877,12 @@ public class HRegion implements HConstants {
|
||||||
// log-sequence-ids can be safely ignored.
|
// log-sequence-ids can be safely ignored.
|
||||||
this.log.completeCacheFlush(this.regionInfo.regionName,
|
this.log.completeCacheFlush(this.regionInfo.regionName,
|
||||||
regionInfo.tableDesc.getName(), logCacheFlushId);
|
regionInfo.tableDesc.getName(), logCacheFlushId);
|
||||||
|
} catch (IOException e) {
|
||||||
|
// An exception here means that the snapshot was not persisted.
|
||||||
|
// The hlog needs to be replayed so its content is restored to memcache.
|
||||||
|
// Currently, only a server restart will do this.
|
||||||
|
this.log.abortCacheFlush();
|
||||||
|
throw new DroppedSnapshotException(e.getMessage());
|
||||||
} finally {
|
} finally {
|
||||||
// C. Delete the now-irrelevant memcache snapshot; its contents have been
|
// C. Delete the now-irrelevant memcache snapshot; its contents have been
|
||||||
// dumped to disk-based HStores or, if error, clear aborted snapshot.
|
// dumped to disk-based HStores or, if error, clear aborted snapshot.
|
||||||
|
|
|
@ -1,174 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright 2007 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test log deletion as logs are rolled.
|
|
||||||
*/
|
|
||||||
public class TestLogRolling extends HBaseTestCase {
|
|
||||||
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
|
|
||||||
private MiniDFSCluster dfs;
|
|
||||||
private MiniHBaseCluster cluster;
|
|
||||||
private Path logdir;
|
|
||||||
private String tableName;
|
|
||||||
private byte[] value;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* constructor
|
|
||||||
* @throws Exception
|
|
||||||
*/
|
|
||||||
public TestLogRolling() throws Exception {
|
|
||||||
super();
|
|
||||||
this.dfs = null;
|
|
||||||
this.cluster = null;
|
|
||||||
this.logdir = null;
|
|
||||||
this.tableName = null;
|
|
||||||
this.value = null;
|
|
||||||
|
|
||||||
// We roll the log after every 256 writes
|
|
||||||
conf.setInt("hbase.regionserver.maxlogentries", 256);
|
|
||||||
|
|
||||||
// For less frequently updated regions flush after every 2 flushes
|
|
||||||
conf.setInt("hbase.hregion.memcache.optionalflushcount", 2);
|
|
||||||
|
|
||||||
// We flush the cache after every 8192 bytes
|
|
||||||
conf.setInt("hbase.hregion.memcache.flush.size", 8192);
|
|
||||||
|
|
||||||
// Make lease timeout longer, lease checks less frequent
|
|
||||||
conf.setInt("hbase.master.lease.period", 10 * 1000);
|
|
||||||
conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
|
|
||||||
|
|
||||||
// Increase the amount of time between client retries
|
|
||||||
conf.setLong("hbase.client.pause", 15 * 1000);
|
|
||||||
|
|
||||||
String className = this.getClass().getName();
|
|
||||||
StringBuilder v = new StringBuilder(className);
|
|
||||||
while (v.length() < 1000) {
|
|
||||||
v.append(className);
|
|
||||||
}
|
|
||||||
value = v.toString().getBytes(HConstants.UTF8_ENCODING);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
@Override
|
|
||||||
public void setUp() throws Exception {
|
|
||||||
super.setUp();
|
|
||||||
dfs = new MiniDFSCluster(conf, 2, true, (String[]) null);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
@Override
|
|
||||||
public void tearDown() throws Exception {
|
|
||||||
super.tearDown();
|
|
||||||
|
|
||||||
if (cluster != null) { // shutdown mini HBase cluster
|
|
||||||
cluster.shutdown();
|
|
||||||
}
|
|
||||||
|
|
||||||
assertEquals(0, countLogFiles(true));
|
|
||||||
|
|
||||||
if (dfs != null) { // shutdown mini DFS cluster
|
|
||||||
FileSystem fs = dfs.getFileSystem();
|
|
||||||
try {
|
|
||||||
dfs.shutdown();
|
|
||||||
} finally {
|
|
||||||
fs.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void startAndWriteData() throws Exception {
|
|
||||||
cluster = new MiniHBaseCluster(conf, 1, dfs);
|
|
||||||
logdir = cluster.regionThreads.get(0).getRegionServer().getLog().dir;
|
|
||||||
|
|
||||||
// When the META table can be opened, the region servers are running
|
|
||||||
@SuppressWarnings("unused")
|
|
||||||
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
|
|
||||||
|
|
||||||
// Create the test table and open it
|
|
||||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
|
||||||
desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
|
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
|
||||||
admin.createTable(desc);
|
|
||||||
HTable table = new HTable(conf, new Text(tableName));
|
|
||||||
|
|
||||||
for (int i = 1; i <= 2048; i++) { // 2048 writes should cause 8 log rolls
|
|
||||||
long lockid =
|
|
||||||
table.startUpdate(new Text("row" + String.format("%1$04d", i)));
|
|
||||||
table.put(lockid, HConstants.COLUMN_FAMILY, value);
|
|
||||||
table.commit(lockid);
|
|
||||||
|
|
||||||
if (i % 256 == 0) {
|
|
||||||
// After every 256 writes sleep to let the log roller run
|
|
||||||
|
|
||||||
try {
|
|
||||||
Thread.sleep(2000);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private int countLogFiles(boolean print) throws IOException {
|
|
||||||
Path[] logfiles = dfs.getFileSystem().listPaths(new Path[] {logdir});
|
|
||||||
if (print) {
|
|
||||||
for (int i = 0; i < logfiles.length; i++) {
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("logfile: " + logfiles[i].toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("number of log files: " + logfiles.length);
|
|
||||||
}
|
|
||||||
return logfiles.length;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Tests that logs are deleted
|
|
||||||
*
|
|
||||||
* @throws Exception
|
|
||||||
*/
|
|
||||||
public void testLogRolling() throws Exception {
|
|
||||||
tableName = getName();
|
|
||||||
// Force a region split after every 768KB
|
|
||||||
conf.setLong("hbase.hregion.max.filesize", 768L * 1024L);
|
|
||||||
startAndWriteData();
|
|
||||||
LOG.info("Finished writing. Sleeping to let cache flusher and log roller run");
|
|
||||||
try {
|
|
||||||
// Wait for log roller and cache flusher to run a few times...
|
|
||||||
Thread.sleep(30L * 1000L);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
LOG.info("Sleep interrupted", e);
|
|
||||||
}
|
|
||||||
LOG.info("Wake from sleep");
|
|
||||||
assertTrue(countLogFiles(true) <= 2);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
Loading…
Reference in New Issue