HBASE-1430 Read the logs in batches during log splitting to avoid OOME

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@776501 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-05-19 23:43:53 +00:00
parent d6afa6748a
commit b812635201
3 changed files with 152 additions and 136 deletions

View File

@ -270,6 +270,7 @@ Release 0.20.0 - Unreleased
minutes/hours
HBASE-1420 add abliity to add and remove (table) indexes on existing
tables (Clint Morgan via Stack)
HBASE-1430 Read the logs in batches during log splitting to avoid OOME
OPTIMIZATIONS
HBASE-1412 Change values for delete column and column family in KeyValue

View File

@ -110,15 +110,22 @@ class ServerManager implements HConstants {
int numServers = serverAddressToServerInfo.size();
int numDeadServers = deadServers.size();
double averageLoad = getAverageLoad();
LOG.info(numServers + " region servers, " + numDeadServers +
" dead, average load " + averageLoad);
String deadServersList = null;
if (numDeadServers > 0) {
LOG.info("DEAD [");
StringBuilder sb = new StringBuilder("Dead Server [");
boolean first = true;
for (String server: deadServers) {
LOG.info(" " + server);
if (!first) {
sb.append(", ");
first = false;
}
LOG.info("]");
sb.append(server);
}
sb.append("]");
deadServersList = sb.toString();
}
LOG.info(numServers + " region servers, " + numDeadServers +
" dead, average load " + averageLoad + (deadServersList != null? deadServers: ""));
}
}
@ -411,7 +418,7 @@ class ServerManager implements HConstants {
for (int i = 0; i < incomingMsgs.length; i++) {
HRegionInfo region = incomingMsgs[i].getRegionInfo();
LOG.info("Received " + incomingMsgs[i] + " from " +
serverInfo.getServerName() + "; " + i + " of " + incomingMsgs.length);
serverInfo.getServerName() + "; " + (i + 1) + " of " + incomingMsgs.length);
switch (incomingMsgs[i].getType()) {
case MSG_REPORT_PROCESS_OPEN:
openingCount++;

View File

@ -774,25 +774,34 @@ public class HLog implements HConstants, Syncable {
throws IOException {
final Map<byte [], SequenceFile.Writer> logWriters =
new TreeMap<byte [], SequenceFile.Writer>(Bytes.BYTES_COMPARATOR);
try {
int maxSteps = Double.valueOf(Math.ceil((logfiles.length * 1.0) /
DEFAULT_NUMBER_CONCURRENT_LOG_READS)).intValue();
for(int step = 0; step < maxSteps; step++) {
final Map<byte[], LinkedList<HLogEntry>> logEntries =
new TreeMap<byte[], LinkedList<HLogEntry>>(Bytes.BYTES_COMPARATOR);
try {
for (int i = 0; i < logfiles.length; i++) {
// Stop at logfiles.length when it's the last step
int endIndex = step == maxSteps - 1? logfiles.length:
step * DEFAULT_NUMBER_CONCURRENT_LOG_READS +
DEFAULT_NUMBER_CONCURRENT_LOG_READS;
for (int i = (step * 10); i < endIndex; i++) {
if (LOG.isDebugEnabled()) {
LOG.debug("Splitting hlog " + (i + 1) + " of " + logfiles.length +
": " + logfiles[i].getPath() + ", length=" + logfiles[i].getLen());
": " + logfiles[i].getPath() +
", length=" + logfiles[i].getLen());
}
// Check for possibly empty file. With appends, currently Hadoop reports
// a zero length even if the file has been sync'd. Revisit if
// Check for possibly empty file. With appends, currently Hadoop
// reports a zero length even if the file has been sync'd. Revisit if
// HADOOP-4751 is committed.
long length = logfiles[i].getLen();
HLogKey key = new HLogKey();
KeyValue val = new KeyValue();
SequenceFile.Reader in = null;
int count = 0;
try {
in = new SequenceFile.Reader(fs, logfiles[i].getPath(), conf);
try {
int count = 0;
while (in.next(key, val)) {
byte [] regionName = key.getRegionName();
LinkedList<HLogEntry> queue = logEntries.get(regionName);
@ -807,6 +816,8 @@ public class HLog implements HConstants, Syncable {
LOG.debug("Pushed " + count + " entries from " +
logfiles[i].getPath());
} catch (IOException e) {
LOG.debug("IOE Pushed " + count + " entries from " +
logfiles[i].getPath());
e = RemoteExceptionHandler.checkIOException(e);
if (!(e instanceof EOFException)) {
LOG.warn("Exception processing " + logfiles[i].getPath() +
@ -864,10 +875,6 @@ public class HLog implements HConstants, Syncable {
}
w = SequenceFile.createWriter(fs, conf, logfile,
HLogKey.class, KeyValue.class, getCompressionType(conf));
// Use copy of regionName; regionName object is reused inside
// in
// HStoreKey.getRegionName so its content changes as we
// iterate.
logWriters.put(key, w);
if (LOG.isDebugEnabled()) {
LOG.debug("Creating new hlog file writer for path "
@ -916,6 +923,7 @@ public class HLog implements HConstants, Syncable {
}catch(InterruptedException ex) {
LOG.warn("Hlog writers were interrupted, possible data loss!");
}
}
} finally {
for (SequenceFile.Writer w : logWriters.values()) {
w.close();