HBASE-14555 Fix Deadlock in MVCC branch-1.2 toString()

This commit is contained in:
Elliott Clark 2015-10-05 17:20:54 -07:00
parent 12e8076fba
commit 12fd46cdca
1 changed files with 10 additions and 15 deletions

View File

@ -23,10 +23,13 @@ import java.util.concurrent.atomic.AtomicLong;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.ClassSize;
import org.mortbay.log.Log;
/** /**
* Manages the read/write consistency. This provides an interface for readers to determine what * Manages the read/write consistency. This provides an interface for readers to determine what
@ -35,6 +38,8 @@ import org.mortbay.log.Log;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class MultiVersionConcurrencyControl { public class MultiVersionConcurrencyControl {
private static final Log LOG = LogFactory.getLog(MultiVersionConcurrencyControl.class);
final AtomicLong readPoint = new AtomicLong(0); final AtomicLong readPoint = new AtomicLong(0);
final AtomicLong writePoint = new AtomicLong(0); final AtomicLong writePoint = new AtomicLong(0);
private final Object readWaiters = new Object(); private final Object readWaiters = new Object();
@ -206,7 +211,7 @@ public class MultiVersionConcurrencyControl {
synchronized (readWaiters) { synchronized (readWaiters) {
while (readPoint.get() < e.getWriteNumber()) { while (readPoint.get() < e.getWriteNumber()) {
if (count % 100 == 0 && count > 0) { if (count % 100 == 0 && count > 0) {
Log.warn("STUCK: " + this); LOG.warn("STUCK: " + this);
} }
count++; count++;
try { try {
@ -225,19 +230,9 @@ public class MultiVersionConcurrencyControl {
@VisibleForTesting @VisibleForTesting
public String toString() { public String toString() {
StringBuffer sb = new StringBuffer(256); return Objects.toStringHelper(this)
sb.append("readPoint="); .add("readPoint", readPoint)
sb.append(this.readPoint.get()); .add("writePoint", writePoint).toString();
sb.append(", writePoint=");
sb.append(this.writePoint);
synchronized (this.writeQueue) {
for (WriteEntry we: this.writeQueue) {
sb.append(", [");
sb.append(we);
sb.append("]");
}
}
return sb.toString();
} }
public long getReadPoint() { public long getReadPoint() {