HBASE-24311 Add more details in MultiVersionConcurrencyControl STUCK log message (#1636)
This commit is contained in:
parent
255470a139
commit
7313b4f5aa
|
@ -689,7 +689,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
private boolean splitRequest;
|
||||
private byte[] explicitSplitPoint = null;
|
||||
|
||||
private final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
|
||||
private final MultiVersionConcurrencyControl mvcc;
|
||||
|
||||
// Coprocessor host
|
||||
private RegionCoprocessorHost coprocessorHost;
|
||||
|
@ -767,6 +767,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
this.wal = wal;
|
||||
this.fs = fs;
|
||||
this.mvcc = new MultiVersionConcurrencyControl(getRegionInfo().getShortNameToLog());
|
||||
|
||||
// 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
|
||||
this.baseConf = confParam;
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.ToStringHelper;
|
||||
|
||||
import java.util.LinkedList;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -39,7 +40,9 @@ import org.apache.hadoop.hbase.util.ClassSize;
|
|||
@InterfaceAudience.Private
|
||||
public class MultiVersionConcurrencyControl {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(MultiVersionConcurrencyControl.class);
|
||||
private static final long READPOINT_ADVANCE_WAIT_TIME = 10L;
|
||||
|
||||
final String regionName;
|
||||
final AtomicLong readPoint = new AtomicLong(0);
|
||||
final AtomicLong writePoint = new AtomicLong(0);
|
||||
private final Object readWaiters = new Object();
|
||||
|
@ -57,12 +60,18 @@ public class MultiVersionConcurrencyControl {
|
|||
private final LinkedList<WriteEntry> writeQueue = new LinkedList<>();
|
||||
|
||||
public MultiVersionConcurrencyControl() {
|
||||
this(null);
|
||||
}
|
||||
|
||||
public MultiVersionConcurrencyControl(String regionName) {
|
||||
this.regionName = regionName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct and set read point. Write point is uninitialized.
|
||||
*/
|
||||
public MultiVersionConcurrencyControl(long startPoint) {
|
||||
this(null);
|
||||
tryAdvanceTo(startPoint, NONE);
|
||||
}
|
||||
|
||||
|
@ -224,11 +233,12 @@ public class MultiVersionConcurrencyControl {
|
|||
synchronized (readWaiters) {
|
||||
while (readPoint.get() < e.getWriteNumber()) {
|
||||
if (count % 100 == 0 && count > 0) {
|
||||
LOG.warn("STUCK: " + this);
|
||||
long totalWaitTillNow = READPOINT_ADVANCE_WAIT_TIME * count;
|
||||
LOG.warn("STUCK for : " + totalWaitTillNow + " millis. " + this);
|
||||
}
|
||||
count++;
|
||||
try {
|
||||
readWaiters.wait(10);
|
||||
readWaiters.wait(READPOINT_ADVANCE_WAIT_TIME);
|
||||
} catch (InterruptedException ie) {
|
||||
// We were interrupted... finish the loop -- i.e. cleanup --and then
|
||||
// on our way out, reset the interrupt flag.
|
||||
|
@ -244,9 +254,12 @@ public class MultiVersionConcurrencyControl {
|
|||
@VisibleForTesting
|
||||
@Override
|
||||
public String toString() {
|
||||
return MoreObjects.toStringHelper(this)
|
||||
.add("readPoint", readPoint)
|
||||
.add("writePoint", writePoint).toString();
|
||||
ToStringHelper helper = MoreObjects.toStringHelper(this).add("readPoint", readPoint)
|
||||
.add("writePoint", writePoint);
|
||||
if (this.regionName != null) {
|
||||
helper.add("regionName", this.regionName);
|
||||
}
|
||||
return helper.toString();
|
||||
}
|
||||
|
||||
public long getReadPoint() {
|
||||
|
|
Loading…
Reference in New Issue