HBASE-11145 UNEXPECTEDpatch -p1 < 11145v2.txt ! when HLog sync: Queue full

This commit is contained in:
stack 2014-09-19 14:31:10 -07:00
parent d2501d643b
commit 288ffe6563
1 changed files with 20 additions and 4 deletions

View File

@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
@ -1206,7 +1205,17 @@ class FSHLog implements HLog, Syncable {
// LinkedBlockingQueue because of
// http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html
// Could use other blockingqueues here or concurrent queues.
this.syncFutures = new LinkedBlockingQueue<SyncFuture>(maxHandlersCount);
//
// We could let the capacity be 'open' but bound it so we get alerted in pathological case
// where we cannot sync and we have a bunch of threads all backed up waiting on their syncs
// to come in. LinkedBlockingQueue actually shrinks when you remove elements so Q should
// stay neat and tidy in usual case. Let the max size be three times the maximum handlers.
// The passed in maxHandlerCount is the user-level handlers which is what we put up most of
// but HBase has other handlers running too -- opening region handlers which want to write
// the meta table when succesful (i.e. sync), closing handlers -- etc. These are usually
// much fewer in number than the user-space handlers so Q-size should be user handlers plus
// some space for these other handlers. Lets multiply by 3 for good-measure.
this.syncFutures = new LinkedBlockingQueue<SyncFuture>(maxHandlersCount * 3);
}
void offer(final long sequence, final SyncFuture [] syncFutures, final int syncFutureCount) {
@ -1874,9 +1883,16 @@ class FSHLog implements HLog, Syncable {
}
// Below expects that the offer 'transfers' responsibility for the outstanding syncs to the
// syncRunner.
// syncRunner. We should never get an exception in here. HBASE-11145 was because queue
// was sized exactly to the count of user handlers but we could have more if we factor in
// meta handlers doing opens and closes.
int index = Math.abs(this.syncRunnerIndex++) % this.syncRunners.length;
try {
this.syncRunners[index].offer(sequence, this.syncFutures, this.syncFuturesCount);
} catch (Exception e) {
cleanupOutstandingSyncsOnException(sequence, e);
throw e;
}
attainSafePoint(sequence);
this.syncFuturesCount = 0;
} catch (Throwable t) {