HBASE-3323 OOME in master splitting logs

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1051278 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-12-20 20:35:41 +00:00
parent 9685e81958
commit d000821c1e
10 changed files with 970 additions and 382 deletions

View File

@ -797,6 +797,7 @@ Release 0.90.0 - Unreleased
HBASE-3370 ReplicationSource.openReader fails to locate HLogs when they
aren't split yet
HBASE-3371 Race in TestReplication can make it fail
HBASE-3323 OOME in master splitting logs
IMPROVEMENTS

View File

@ -190,12 +190,13 @@ public class MasterFileSystem {
long splitTime = 0, splitLogSize = 0;
Path logDir = new Path(this.rootdir, HLog.getHLogDirectoryName(serverName));
try {
HLogSplitter splitter = HLogSplitter.createLogSplitter(conf);
HLogSplitter splitter = HLogSplitter.createLogSplitter(
conf, rootdir, logDir, oldLogDir, this.fs);
try {
splitter.splitLog(this.rootdir, logDir, oldLogDir, this.fs, conf);
splitter.splitLog();
} catch (OrphanHLogAfterSplitException e) {
LOG.warn("Retrying splitting because of:", e);
splitter.splitLog(this.rootdir, logDir, oldLogDir, this.fs, conf);
splitter.splitLog();
}
splitTime = splitter.getTime();
splitLogSize = splitter.getSize();

View File

@ -1439,8 +1439,9 @@ public class HLog implements Syncable {
throw new IOException(p + " is not a directory");
}
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(baseDir, p, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(
conf, baseDir, p, oldLogDir, fs);
logSplitter.splitLog();
}
/**

View File

@ -160,6 +160,32 @@ public class HLogKey implements WritableComparable<HLogKey> {
return result;
}
/**
* Drop this instance's tablename byte array and instead
* hold a reference to the provided tablename. This is not
* meant to be a general purpose setter - it's only used
* to collapse references to conserve memory.
*/
void internTableName(byte []tablename) {
// We should not use this as a setter - only to swap
// in a new reference to the same table name.
assert Bytes.equals(tablename, this.tablename);
this.tablename = tablename;
}
/**
* Drop this instance's region name byte array and instead
* hold a reference to the provided region name. This is not
* meant to be a general purpose setter - it's only used
* to collapse references to conserve memory.
*/
void internEncodedRegionName(byte []encodedRegionName) {
// We should not use this as a setter - only to swap
// in a new reference to the same table name.
assert Bytes.equals(this.encodedRegionName, encodedRegionName);
this.encodedRegionName = encodedRegionName;
}
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.encodedRegionName);
Bytes.writeByteArray(out, this.tablename);

View File

@ -27,6 +27,7 @@ import java.util.List;
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
@ -66,7 +67,7 @@ import org.apache.hadoop.io.Writable;
* is an old style KeyValue or the new style WALEdit.
*
*/
public class WALEdit implements Writable {
public class WALEdit implements Writable, HeapSize {
private final int VERSION_2 = -1;
@ -154,7 +155,19 @@ public class WALEdit implements Writable {
out.writeInt(scopes.get(key));
}
}
}
public long heapSize() {
long ret = 0;
for (KeyValue kv : kvs) {
ret += kv.heapSize();
}
if (scopes != null) {
ret += ClassSize.TREEMAP;
ret += ClassSize.align(scopes.size() * ClassSize.MAP_ENTRY);
// TODO this isn't quite right, need help here
}
return ret;
}
public String toString() {

View File

@ -164,10 +164,10 @@ public class TestHLog {
log.rollWriter();
}
log.close();
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, logdir, this.oldLogDir, this.fs);
List<Path> splits =
logSplitter.splitLog(hbaseDir, logdir,
this.oldLogDir, this.fs, conf);
logSplitter.splitLog();
verifySplits(splits, howmany);
log = null;
} finally {

View File

@ -24,16 +24,28 @@ import static org.junit.Assert.*;
import java.io.IOException;
import java.util.NavigableSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.MultithreadedTestUtil;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.EntryBuffers;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.RegionEntryBuffer;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import static org.mockito.Mockito.mock;
/**
* Simple testing of a few HLog methods.
*/
public class TestHLogMethods {
private static final byte[] TEST_REGION = Bytes.toBytes("test_region");;
private static final byte[] TEST_TABLE = Bytes.toBytes("test_table");
private final HBaseTestingUtility util = new HBaseTestingUtility();
/**
@ -84,4 +96,71 @@ public class TestHLogMethods {
FSDataOutputStream fdos = fs.create(new Path(testdir, name), true);
fdos.close();
}
}
@Test
public void testRegionEntryBuffer() throws Exception {
HLogSplitter.RegionEntryBuffer reb = new HLogSplitter.RegionEntryBuffer(
TEST_TABLE, TEST_REGION);
assertEquals(0, reb.heapSize());
reb.appendEntry(createTestLogEntry(1));
assertTrue(reb.heapSize() > 0);
}
@Test
public void testEntrySink() throws Exception {
Configuration conf = new Configuration();
HLogSplitter splitter = HLogSplitter.createLogSplitter(
conf, mock(Path.class), mock(Path.class), mock(Path.class),
mock(FileSystem.class));
EntryBuffers sink = splitter.new EntryBuffers(1*1024*1024);
for (int i = 0; i < 1000; i++) {
HLog.Entry entry = createTestLogEntry(i);
sink.appendEntry(entry);
}
assertTrue(sink.totalBuffered > 0);
long amountInChunk = sink.totalBuffered;
// Get a chunk
RegionEntryBuffer chunk = sink.getChunkToWrite();
assertEquals(chunk.heapSize(), amountInChunk);
// Make sure it got marked that a thread is "working on this"
assertTrue(sink.isRegionCurrentlyWriting(TEST_REGION));
// Insert some more entries
for (int i = 0; i < 500; i++) {
HLog.Entry entry = createTestLogEntry(i);
sink.appendEntry(entry);
}
// Asking for another chunk shouldn't work since the first one
// is still writing
assertNull(sink.getChunkToWrite());
// If we say we're done writing the first chunk, then we should be able
// to get the second
sink.doneWriting(chunk);
RegionEntryBuffer chunk2 = sink.getChunkToWrite();
assertNotNull(chunk2);
assertNotSame(chunk, chunk2);
long amountInChunk2 = sink.totalBuffered;
// The second chunk had fewer rows than the first
assertTrue(amountInChunk2 < amountInChunk);
sink.doneWriting(chunk2);
assertEquals(0, sink.totalBuffered);
}
private HLog.Entry createTestLogEntry(int i) {
long seq = i;
long now = i * 1000;
WALEdit edit = new WALEdit();
edit.add(KeyValueTestUtil.create("row", "fam", "qual", 1234, "val"));
HLogKey key = new HLogKey(TEST_REGION, TEST_TABLE, seq, now);
HLog.Entry entry = new HLog.Entry(key, edit);
return entry;
}
}

View File

@ -19,16 +19,14 @@
*/
package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.*;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@ -39,7 +37,9 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
@ -52,9 +52,16 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.ipc.RemoteException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
/**
* Testing {@link HLog} splitting code.
@ -119,11 +126,15 @@ public class TestHLogSplit {
@Before
public void setUp() throws Exception {
flushToConsole("Cleaning up cluster for new test\n"
+ "--------------------------");
conf = TEST_UTIL.getConfiguration();
fs = TEST_UTIL.getDFSCluster().getFileSystem();
FileStatus[] entries = fs.listStatus(new Path("/"));
flushToConsole("Num entries in /:" + entries.length);
for (FileStatus dir : entries){
fs.delete(dir.getPath(), true);
assertTrue("Deleting " + dir.getPath(),
fs.delete(dir.getPath(), true));
}
seq = 0;
regions = new ArrayList<String>();
@ -161,18 +172,23 @@ public class TestHLogSplit {
public void testSplitFailsIfNewHLogGetsCreatedAfterSplitStarted()
throws IOException {
AtomicBoolean stop = new AtomicBoolean(false);
FileStatus[] stats = fs.listStatus(new Path("/hbase/t1"));
assertTrue("Previous test should clean up table dir",
stats == null || stats.length == 0);
generateHLogs(-1);
fs.initialize(fs.getUri(), conf);
try {
(new ZombieNewLogWriterRegionServer(stop)).start();
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
} finally {
stop.set(true);
}
}
@Test
public void testSplitPreservesEdits() throws IOException{
final String REGION = "region__1";
@ -181,8 +197,9 @@ public class TestHLogSplit {
generateHLogs(1, 10, -1);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
Path originalLog = (fs.listStatus(oldLogDir))[0].getPath();
Path splitLog = getLogForRegion(hbaseDir, TABLE_NAME, REGION);
@ -202,8 +219,9 @@ public class TestHLogSplit {
// initialize will create a new DFSClient with a new client ID
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
for (String region : regions) {
@ -224,8 +242,9 @@ public class TestHLogSplit {
// initialize will create a new DFSClient with a new client ID
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
for (String region : regions) {
Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
@ -240,8 +259,9 @@ public class TestHLogSplit {
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
for (String region : regions) {
Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
@ -260,8 +280,9 @@ public class TestHLogSplit {
Corruptions.APPEND_GARBAGE, true, fs);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
for (String region : regions) {
Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
@ -278,8 +299,9 @@ public class TestHLogSplit {
Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true, fs);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
for (String region : regions) {
Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
assertEquals((NUM_WRITERS - 1) * ENTRIES, countHLog(logfile, fs, conf));
@ -296,8 +318,9 @@ public class TestHLogSplit {
corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
Corruptions.INSERT_GARBAGE_IN_THE_MIDDLE, false, fs);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
for (String region : regions) {
Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
@ -323,13 +346,13 @@ public class TestHLogSplit {
Path c1 = new Path(hlogDir, HLOG_FILE_PREFIX + "0");
conf.setClass("hbase.regionserver.hlog.reader.impl",
FaultySequenceFileLogReader.class, HLog.Reader.class);
String[] failureTypes = { "begin", "middle", "end" };
for (FaultySequenceFileLogReader.FailureType failureType : FaultySequenceFileLogReader.FailureType.values()) {
conf.set("faultysequencefilelogreader.failuretype", failureType.name());
generateHLogs(1, ENTRIES, -1);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
FileStatus[] archivedLogs = fs.listStatus(corruptDir);
assertEquals("expected a different file", c1.getName(), archivedLogs[0]
.getPath().getName());
@ -358,8 +381,9 @@ public class TestHLogSplit {
conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name());
generateHLogs(Integer.MAX_VALUE);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
} finally {
conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass,
Reader.class);
@ -383,9 +407,10 @@ public class TestHLogSplit {
conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name());
generateHLogs(-1);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
try {
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
logSplitter.splitLog();
} catch (IOException e) {
assertEquals(
"if skip.errors is false all files should remain in place",
@ -413,8 +438,9 @@ public class TestHLogSplit {
corruptHLog(c1, Corruptions.TRUNCATE, true, fs);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
Path originalLog = (fs.listStatus(oldLogDir))[0].getPath();
Path splitLog = getLogForRegion(hbaseDir, TABLE_NAME, REGION);
@ -437,8 +463,9 @@ public class TestHLogSplit {
generateHLogs(-1);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
FileStatus[] archivedLogs = fs.listStatus(oldLogDir);
@ -449,8 +476,9 @@ public class TestHLogSplit {
public void testSplit() throws IOException {
generateHLogs(-1);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
for (String region : regions) {
Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
@ -464,12 +492,16 @@ public class TestHLogSplit {
throws IOException {
generateHLogs(-1);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
FileStatus [] statuses = null;
try {
statuses = fs.listStatus(hlogDir);
assertNull(statuses);
if (statuses != null) {
Assert.fail("Files left in log dir: " +
Joiner.on(",").join(FileUtil.stat2Paths(statuses)));
}
} catch (FileNotFoundException e) {
// hadoop 0.21 throws FNFE whereas hadoop 0.20 returns null
}
@ -516,8 +548,9 @@ public class TestHLogSplit {
try {
zombie.start();
try {
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
} catch (IOException ex) {/* expected */}
int logFilesNumber = fs.listStatus(hlogDir).length;
@ -549,11 +582,12 @@ public class TestHLogSplit {
try {
InstrumentedSequenceFileLogWriter.activateFailure = true;
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
} catch (IOException e) {
assertEquals("java.io.IOException: This exception is instrumented and should only be thrown for testing", e.getMessage());
assertEquals("This exception is instrumented and should only be thrown for testing", e.getMessage());
throw e;
} finally {
InstrumentedSequenceFileLogWriter.activateFailure = false;
@ -561,7 +595,10 @@ public class TestHLogSplit {
}
// @Test
// @Test TODO this test has been disabled since it was created!
// It currently fails because the second split doesn't output anything
// -- because there are no region dirs after we move aside the first
// split result
public void testSplittingLargeNumberOfRegionsConsistency() throws IOException {
regions.removeAll(regions);
@ -572,8 +609,9 @@ public class TestHLogSplit {
generateHLogs(1, 100, -1);
fs.initialize(fs.getUri(), conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
fs.rename(oldLogDir, hlogDir);
Path firstSplitPath = new Path(hbaseDir, Bytes.toString(TABLE_NAME) + ".first");
Path splitPath = new Path(hbaseDir, Bytes.toString(TABLE_NAME));
@ -582,7 +620,9 @@ public class TestHLogSplit {
fs.initialize(fs.getUri(), conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
assertEquals(0, compareHLogSplitDirs(firstSplitPath, splitPath));
}
@ -600,11 +640,161 @@ public class TestHLogSplit {
Path regiondir = new Path(tabledir, region);
fs.delete(regiondir, true);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf);
logSplitter.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
hbaseDir, hlogDir, oldLogDir, fs);
logSplitter.splitLog();
assertFalse(fs.exists(regiondir));
}
@Test
public void testIOEOnOutputThread() throws Exception {
conf.setBoolean(HBASE_SKIP_ERRORS, false);
generateHLogs(-1);
fs.initialize(fs.getUri(), conf);
// Set up a splitter that will throw an IOE on the output side
HLogSplitter logSplitter = new HLogSplitter(
conf, hbaseDir, hlogDir, oldLogDir, fs) {
protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
throws IOException {
HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class);
Mockito.doThrow(new IOException("Injected")).when(mockWriter).append(Mockito.<HLog.Entry>any());
return mockWriter;
}
};
try {
logSplitter.splitLog();
fail("Didn't throw!");
} catch (IOException ioe) {
assertTrue(ioe.toString().contains("Injected"));
}
}
/**
* Test log split process with fake data and lots of edits to trigger threading
* issues.
*/
@Test
public void testThreading() throws Exception {
doTestThreading(20000, 128*1024*1024, 0);
}
/**
* Test blocking behavior of the log split process if writers are writing slower
* than the reader is reading.
*/
@Test
public void testThreadingSlowWriterSmallBuffer() throws Exception {
doTestThreading(200, 1024, 50);
}
/**
* Sets up a log splitter with a mock reader and writer. The mock reader generates
* a specified number of edits spread across 5 regions. The mock writer optionally
* sleeps for each edit it is fed.
* *
* After the split is complete, verifies that the statistics show the correct number
* of edits output into each region.
*
* @param numFakeEdits number of fake edits to push through pipeline
* @param bufferSize size of in-memory buffer
* @param writerSlowness writer threads will sleep this many ms per edit
*/
private void doTestThreading(final int numFakeEdits,
final int bufferSize,
final int writerSlowness) throws Exception {
Configuration localConf = new Configuration(conf);
localConf.setInt("hbase.regionserver.hlog.splitlog.buffersize", bufferSize);
// Create a fake log file (we'll override the reader to produce a stream of edits)
FSDataOutputStream out = fs.create(new Path(hlogDir, HLOG_FILE_PREFIX + ".fake"));
out.close();
// Make region dirs for our destination regions so the output doesn't get skipped
final List<String> regions = ImmutableList.of("r0", "r1", "r2", "r3", "r4");
makeRegionDirs(fs, regions);
// Create a splitter that reads and writes the data without touching disk
HLogSplitter logSplitter = new HLogSplitter(
localConf, hbaseDir, hlogDir, oldLogDir, fs) {
/* Produce a mock writer that doesn't write anywhere */
protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
throws IOException {
HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class);
Mockito.doAnswer(new Answer<Void>() {
int expectedIndex = 0;
@Override
public Void answer(InvocationOnMock invocation) {
if (writerSlowness > 0) {
try {
Thread.sleep(writerSlowness);
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
}
}
HLog.Entry entry = (Entry) invocation.getArguments()[0];
WALEdit edit = entry.getEdit();
List<KeyValue> keyValues = edit.getKeyValues();
assertEquals(1, keyValues.size());
KeyValue kv = keyValues.get(0);
// Check that the edits come in the right order.
assertEquals(expectedIndex, Bytes.toInt(kv.getRow()));
expectedIndex++;
return null;
}
}).when(mockWriter).append(Mockito.<HLog.Entry>any());
return mockWriter;
}
/* Produce a mock reader that generates fake entries */
protected Reader getReader(FileSystem fs, Path curLogFile, Configuration conf)
throws IOException {
Reader mockReader = Mockito.mock(Reader.class);
Mockito.doAnswer(new Answer<HLog.Entry>() {
int index = 0;
@Override
public HLog.Entry answer(InvocationOnMock invocation) throws Throwable {
if (index >= numFakeEdits) return null;
// Generate r0 through r4 in round robin fashion
int regionIdx = index % regions.size();
byte region[] = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
HLog.Entry ret = createTestEntry(TABLE_NAME, region,
Bytes.toBytes((int)(index / regions.size())),
FAMILY, QUALIFIER, VALUE, index);
index++;
return ret;
}
}).when(mockReader).next();
return mockReader;
}
};
logSplitter.splitLog();
// Verify number of written edits per region
Map<byte[], Long> outputCounts = logSplitter.getOutputCounts();
for (Map.Entry<byte[], Long> entry : outputCounts.entrySet()) {
LOG.info("Got " + entry.getValue() + " output edits for region " +
Bytes.toString(entry.getKey()));
assertEquals((long)entry.getValue(), numFakeEdits / regions.size());
}
assertEquals(regions.size(), outputCounts.size());
}
/**
* This thread will keep writing to the file after the split process has started
@ -677,29 +867,19 @@ public class TestHLogSplit {
if (stop.get()) {
return;
}
boolean splitStarted = false;
Path p = new Path(hbaseDir, new String(TABLE_NAME));
while (!splitStarted) {
try {
FileStatus [] statuses = fs.listStatus(p);
// In 0.20, listStatus comes back with a null if file doesn't exit.
// In 0.21, it throws FNFE.
if (statuses != null && statuses.length > 0) {
// Done.
break;
}
} catch (FileNotFoundException e) {
// Expected in hadoop 0.21
} catch (IOException e1) {
assertTrue("Failed to list status ", false);
}
flushToConsole("Juliet: split not started, sleeping a bit...");
Threads.sleep(100);
}
Path tableDir = new Path(hbaseDir, new String(TABLE_NAME));
Path regionDir = new Path(tableDir, regions.get(0));
Path recoveredEdits = new Path(regionDir, HLogSplitter.RECOVERED_EDITS);
String region = "juliet";
Path julietLog = new Path(hlogDir, HLOG_FILE_PREFIX + ".juliet");
try {
fs.mkdirs(new Path(new Path(hbaseDir, region), region));
while (!fs.exists(recoveredEdits) && !stop.get()) {
flushToConsole("Juliet: split not started, sleeping a bit...");
Threads.sleep(10);
}
fs.mkdirs(new Path(tableDir, region));
HLog.Writer writer = HLog.createWriter(fs,
julietLog, conf);
appendEntry(writer, "juliet".getBytes(), ("juliet").getBytes(),
@ -722,10 +902,15 @@ public class TestHLogSplit {
generateHLogs(NUM_WRITERS, ENTRIES, leaveOpen);
}
private void generateHLogs(int writers, int entries, int leaveOpen) throws IOException {
private void makeRegionDirs(FileSystem fs, List<String> regions) throws IOException {
for (String region : regions) {
flushToConsole("Creating dir for region " + region);
fs.mkdirs(new Path(tabledir, region));
}
}
private void generateHLogs(int writers, int entries, int leaveOpen) throws IOException {
makeRegionDirs(fs, regions);
for (int i = 0; i < writers; i++) {
writer[i] = HLog.createWriter(fs, new Path(hlogDir, HLOG_FILE_PREFIX + i), conf);
for (int j = 0; j < entries; j++) {
@ -835,14 +1020,20 @@ public class TestHLogSplit {
byte[] value, long seq)
throws IOException {
writer.append(createTestEntry(table, region, row, family, qualifier, value, seq));
writer.sync();
return seq;
}
private HLog.Entry createTestEntry(
byte[] table, byte[] region,
byte[] row, byte[] family, byte[] qualifier,
byte[] value, long seq) {
long time = System.nanoTime();
WALEdit edit = new WALEdit();
seq++;
edit.add(new KeyValue(row, family, qualifier, time, KeyValue.Type.Put, value));
writer.append(new HLog.Entry(new HLogKey(region, table, seq, time), edit));
writer.sync();
return seq;
return new HLog.Entry(new HLogKey(region, table, seq, time), edit);
}
@ -864,6 +1055,14 @@ public class TestHLogSplit {
private int compareHLogSplitDirs(Path p1, Path p2) throws IOException {
FileStatus[] f1 = fs.listStatus(p1);
FileStatus[] f2 = fs.listStatus(p2);
assertNotNull("Path " + p1 + " doesn't exist", f1);
assertNotNull("Path " + p2 + " doesn't exist", f2);
System.out.println("Files in " + p1 + ": " +
Joiner.on(",").join(FileUtil.stat2Paths(f1)));
System.out.println("Files in " + p2 + ": " +
Joiner.on(",").join(FileUtil.stat2Paths(f2)));
assertEquals(f1.length, f2.length);
for (int i = 0; i < f1.length; i++) {
// Regions now have a directory named RECOVERED_EDITS_DIR and in here

View File

@ -487,9 +487,9 @@ public class TestWALReplay {
*/
private Path runWALSplit(final Configuration c) throws IOException {
FileSystem fs = FileSystem.get(c);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(c);
List<Path> splits = logSplitter.splitLog(this.hbaseRootDir, this.logDir,
this.oldLogDir, fs, c);
HLogSplitter logSplitter = HLogSplitter.createLogSplitter(c,
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());
// Make sure the file exists