HBASE-13971 Flushes stuck since 6 hours on a regionserver

This commit is contained in:
tedyu 2015-07-16 13:53:22 -07:00
parent be5631f377
commit 9ebafe6b08
2 changed files with 28 additions and 3 deletions

View File

@ -200,6 +200,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
"hbase.hregion.scan.loadColumnFamiliesOnDemand";
// in milliseconds
private static final String MAX_WAIT_FOR_SEQ_ID_KEY =
"hbase.hregion.max.wait.for.seq.id";
private static final int DEFAULT_MAX_WAIT_FOR_SEQ_ID = 60000;
/**
* This is the global default value for durability. All tables/mutations not
@ -331,6 +337,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
*/
private boolean isLoadingCfsOnDemandDefault = false;
private int maxWaitForSeqId;
private final AtomicInteger majorInProgress = new AtomicInteger(0);
private final AtomicInteger minorInProgress = new AtomicInteger(0);
@ -663,6 +670,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
DEFAULT_ROWLOCK_WAIT_DURATION);
maxWaitForSeqId = conf.getInt(MAX_WAIT_FOR_SEQ_ID_KEY, DEFAULT_MAX_WAIT_FOR_SEQ_ID);
this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
this.htableDescriptor = htd;
this.rsServices = rsServices;
@ -2415,7 +2423,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@VisibleForTesting
protected long getNextSequenceId(final WAL wal) throws IOException {
WALKey key = this.appendEmptyEdit(wal, null);
return key.getSequenceId();
return key.getSequenceId(maxWaitForSeqId);
}
//////////////////////////////////////////////////////////////////////////////
@ -7215,7 +7223,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
public static final long FIXED_OVERHEAD = ClassSize.align(
ClassSize.OBJECT +
ClassSize.ARRAY +
44 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
44 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
(14 * Bytes.SIZEOF_LONG) +
5 * Bytes.SIZEOF_BOOLEAN);

View File

@ -30,6 +30,7 @@ import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.util.ByteStringer;
@ -301,8 +302,24 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
*/
@Override
public long getSequenceId() throws IOException {
return getSequenceId(-1);
}
/**
* Wait for sequence number is assigned &amp; return the assigned value
* @param maxWaitForSeqId maximum duration, in milliseconds, to wait for seq number to be assigned
* @return long the new assigned sequence number
* @throws IOException
*/
public long getSequenceId(int maxWaitForSeqId) throws IOException {
try {
this.seqNumAssignedLatch.await();
if (maxWaitForSeqId < 0) {
this.seqNumAssignedLatch.await();
} else {
if (!this.seqNumAssignedLatch.await(maxWaitForSeqId, TimeUnit.MILLISECONDS)) {
throw new IOException("Timed out waiting for seq number to be assigned");
}
}
} catch (InterruptedException ie) {
LOG.warn("Thread interrupted waiting for next log sequence number");
InterruptedIOException iie = new InterruptedIOException();