HBASE-5329 addRowLock() may allocate duplicate lock id, causing the client to be blocked (Ian Varley)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1376489 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
8c811a6994
commit
bbe297121d
|
@ -51,6 +51,7 @@ import java.util.SortedMap;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
import java.util.TreeSet;
|
import java.util.TreeSet;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
@ -320,7 +321,7 @@ public class HRegionServer implements ClientProtocol,
|
||||||
// Port we put up the webui on.
|
// Port we put up the webui on.
|
||||||
protected int webuiport = -1;
|
protected int webuiport = -1;
|
||||||
|
|
||||||
Map<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
|
ConcurrentMap<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
|
||||||
|
|
||||||
// A state before we go into stopped state. At this stage we're closing user
|
// A state before we go into stopped state. At this stage we're closing user
|
||||||
// space regions.
|
// space regions.
|
||||||
|
@ -2753,11 +2754,13 @@ public class HRegionServer implements ClientProtocol,
|
||||||
return this.fsOk;
|
return this.fsOk;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected long addRowLock(Integer r, HRegion region)
|
protected long addRowLock(Integer r, HRegion region) throws LeaseStillHeldException {
|
||||||
throws LeaseStillHeldException {
|
String lockName = null;
|
||||||
long lockId = nextLong();
|
long lockId;
|
||||||
String lockName = String.valueOf(lockId);
|
do {
|
||||||
rowlocks.put(lockName, r);
|
lockId = nextLong();
|
||||||
|
lockName = String.valueOf(lockId);
|
||||||
|
} while (rowlocks.putIfAbsent(lockName, r) != null);
|
||||||
this.leases.createLease(lockName, this.rowLockLeaseTimeoutPeriod, new RowLockListener(lockName,
|
this.leases.createLease(lockName, this.rowLockLeaseTimeoutPeriod, new RowLockListener(lockName,
|
||||||
region));
|
region));
|
||||||
return lockId;
|
return lockId;
|
||||||
|
|
Loading…
Reference in New Issue