You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2020/05/05 19:48:48 UTC
[hbase] branch branch-1 updated: HBASE-24311 Add more details in
MultiVersionConcurrencyControl STUCK log message (#1654)
This is an automated email from the ASF dual-hosted git repository.
apurtell pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-1 by this push:
new e4dbfe9 HBASE-24311 Add more details in MultiVersionConcurrencyControl STUCK log message (#1654)
e4dbfe9 is described below
commit e4dbfe9bede043c0b18bcf4a150d674f1be3b5c9
Author: Anoop Sam John <an...@gmail.com>
AuthorDate: Tue May 5 16:25:40 2020 +0530
HBASE-24311 Add more details in MultiVersionConcurrencyControl STUCK log message (#1654)
---
.../apache/hadoop/hbase/regionserver/HRegion.java | 3 ++-
.../MultiVersionConcurrencyControl.java | 24 ++++++++++++++++------
2 files changed, 20 insertions(+), 7 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 8ef3865..84ed7b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -669,7 +669,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;
@@ -744,6 +744,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.comparator = fs.getRegionInfo().getComparator();
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;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
index cc68b03..d791ff5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
+import com.google.common.base.Objects.ToStringHelper;
import java.util.LinkedList;
import java.util.concurrent.atomic.AtomicLong;
@@ -40,7 +41,9 @@ import org.apache.hadoop.hbase.util.ClassSize;
public class MultiVersionConcurrencyControl {
private static final Log LOG = LogFactory.getLog(MultiVersionConcurrencyControl.class);
static final long NO_WRITE_NUMBER = 0;
+ 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();
@@ -58,13 +61,18 @@ public class MultiVersionConcurrencyControl {
private final LinkedList<WriteEntry> writeQueue = new LinkedList<WriteEntry>();
public MultiVersionConcurrencyControl() {
- super();
+ 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);
}
@@ -227,11 +235,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.
@@ -247,9 +256,12 @@ public class MultiVersionConcurrencyControl {
@VisibleForTesting
@Override
public String toString() {
- return Objects.toStringHelper(this)
- .add("readPoint", readPoint)
- .add("writePoint", writePoint).toString();
+ ToStringHelper helper = Objects.toStringHelper(this).add("readPoint", readPoint)
+ .add("writePoint", writePoint);
+ if (this.regionName != null) {
+ helper.add("regionName", this.regionName);
+ }
+ return helper.toString();
}
public long getReadPoint() {