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:
parent
9685e81958
commit
d000821c1e
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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() {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue