HBASE-1683 OOME on master splitting logs; stuck, won't go down

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@797683 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-07-24 22:47:10 +00:00
parent 493d1fc998
commit db8a3e2346
3 changed files with 17 additions and 14 deletions

View File

@ -510,6 +510,7 @@ Release 0.20.0 - Unreleased
ITHBase ITHBase
HBASE-1699 Remove hbrep example as it's too out of date HBASE-1699 Remove hbrep example as it's too out of date
(Tim Sell via Stack) (Tim Sell via Stack)
HBASE-1683 OOME on master splitting logs; stuck, won't go down
OPTIMIZATIONS OPTIMIZATIONS
HBASE-1412 Change values for delete column and column family in KeyValue HBASE-1412 Change values for delete column and column family in KeyValue

View File

@ -136,14 +136,6 @@ public interface HConstants {
/** Default size of a reservation block */ /** Default size of a reservation block */
static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5; static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5;
/** Default number of threads to use when log splitting
* to rewrite the logs. More means faster but bigger mem consumption */
static final int DEFAULT_NUMBER_LOG_WRITER_THREAD = 10;
/** Default number of logs to read concurrently
* when log splitting. More means faster but bigger mem consumption */
static final int DEFAULT_NUMBER_CONCURRENT_LOG_READS = 10;
/** Maximum value length, enforced on KeyValue construction */ /** Maximum value length, enforced on KeyValue construction */
static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE; static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE;

View File

@ -838,16 +838,26 @@ public class HLog implements HConstants, Syncable {
final Map<byte [], WriterAndPath> logWriters = final Map<byte [], WriterAndPath> logWriters =
new TreeMap<byte [], WriterAndPath>(Bytes.BYTES_COMPARATOR); new TreeMap<byte [], WriterAndPath>(Bytes.BYTES_COMPARATOR);
List<Path> splits = null; List<Path> splits = null;
// Number of threads to use when log splitting to rewrite the logs.
// More means faster but bigger mem consumption.
int logWriterThreads =
conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
// Number of logs to read concurrently when log splitting.
// More means faster but bigger mem consumption */
int concurrentLogReads =
conf.getInt("hbase.regionserver.hlog.splitlog.reader.threads", 3);
try { try {
int maxSteps = Double.valueOf(Math.ceil((logfiles.length * 1.0) / int maxSteps = Double.valueOf(Math.ceil((logfiles.length * 1.0) /
DEFAULT_NUMBER_CONCURRENT_LOG_READS)).intValue(); concurrentLogReads)).intValue();
for(int step = 0; step < maxSteps; step++) { for(int step = 0; step < maxSteps; step++) {
final Map<byte[], LinkedList<HLogEntry>> logEntries = final Map<byte[], LinkedList<HLogEntry>> logEntries =
new TreeMap<byte[], LinkedList<HLogEntry>>(Bytes.BYTES_COMPARATOR); new TreeMap<byte[], LinkedList<HLogEntry>>(Bytes.BYTES_COMPARATOR);
// Stop at logfiles.length when it's the last step // Stop at logfiles.length when it's the last step
int endIndex = step == maxSteps - 1? logfiles.length: int endIndex = step == maxSteps - 1? logfiles.length:
step * DEFAULT_NUMBER_CONCURRENT_LOG_READS + step * concurrentLogReads + concurrentLogReads;
DEFAULT_NUMBER_CONCURRENT_LOG_READS;
for (int i = (step * 10); i < endIndex; i++) { for (int i = (step * 10); i < endIndex; i++) {
// Check for possibly empty file. With appends, currently Hadoop // Check for possibly empty file. With appends, currently Hadoop
// reports a zero length even if the file has been sync'd. Revisit if // reports a zero length even if the file has been sync'd. Revisit if
@ -916,7 +926,7 @@ public class HLog implements HConstants, Syncable {
} }
} }
ExecutorService threadPool = ExecutorService threadPool =
Executors.newFixedThreadPool(DEFAULT_NUMBER_LOG_WRITER_THREAD); Executors.newFixedThreadPool(logWriterThreads);
for (final byte[] key : logEntries.keySet()) { for (final byte[] key : logEntries.keySet()) {
Thread thread = new Thread(Bytes.toString(key)) { Thread thread = new Thread(Bytes.toString(key)) {
@Override @Override