HBASE-3865 Failing TestWALReplay

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1100373 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-05-06 21:03:40 +00:00
parent 6fb0428103
commit 7bc1722112
2 changed files with 9 additions and 6 deletions

View File

@ -102,6 +102,7 @@ Release 0.91.0 - Unreleased
HBASE-3862 Race conditions in aggregate calculation (John Heitmann)
HBASE-3861 MiniZooKeeperCluster should refer to maxClientCnxns (Eugene
Koontz via Andrew Purtell)
HBASE-3865 Failing TestWALReplay
IMPROVEMENTS
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)

View File

@ -382,7 +382,7 @@ public class TestWALReplay {
final Configuration newConf = HBaseConfiguration.create(this.conf);
User user = HBaseTestingUtility.getDifferentUser(newConf,
".replay.wal.secondtime");
user.runAs(new PrivilegedExceptionAction(){
user.runAs(new PrivilegedExceptionAction() {
public Object run() throws Exception {
runWALSplit(newConf);
FileSystem newFS = FileSystem.get(newConf);
@ -392,10 +392,12 @@ public class TestWALReplay {
HLog newWal = createWAL(newConf);
final AtomicInteger flushcount = new AtomicInteger(0);
try {
final HRegion region = new HRegion(basedir, newWal, newFS, newConf, hri,
null) {
protected boolean internalFlushcache(HLog wal, long myseqid)
final HRegion region =
new HRegion(basedir, newWal, newFS, newConf, hri, null) {
protected boolean internalFlushcache(
final HLog wal, final long myseqid, MonitoredTask status)
throws IOException {
LOG.info("InternalFlushCache Invoked");
boolean b = super.internalFlushcache(wal, myseqid,
Mockito.mock(MonitoredTask.class));
flushcount.incrementAndGet();
@ -404,7 +406,7 @@ public class TestWALReplay {
};
long seqid = region.initialize();
// We flushed during init.
assertTrue(flushcount.get() > 0);
assertTrue("Flushcount=" + flushcount.get(), flushcount.get() > 0);
assertTrue(seqid > wal.getSequenceNumber());
Get get = new Get(rowName);
@ -494,7 +496,7 @@ public class TestWALReplay {
this.hbaseRootDir, this.logDir, this.oldLogDir, fs);
List<Path> splits = logSplitter.splitLog();
// Split should generate only 1 file since there's only 1 region
assertEquals(1, splits.size());
assertEquals("splits=" + splits, 1, splits.size());
// Make sure the file exists
assertTrue(fs.exists(splits.get(0)));
LOG.info("Split file=" + splits.get(0));