svn merge -c 1374127 from trunk for HDFS-2421. Improve the concurrency of SerialNumberMap in NameNode.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1374128 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
c849ef9bea
commit
aa9bf3b472
|
@ -230,6 +230,9 @@ Release 2.0.1-alpha - UNRELEASED
|
|||
|
||||
HDFS-3697. Enable fadvise readahead by default. (todd)
|
||||
|
||||
HDFS-2421. Improve the concurrency of SerialNumberMap in NameNode.
|
||||
(Jing Zhao and Weiyan Wang via szetszwo)
|
||||
|
||||
BUG FIXES
|
||||
|
||||
HDFS-3385. The last block of INodeFileUnderConstruction is not
|
||||
|
|
|
@ -17,7 +17,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
||||
/** Manage name-to-serial-number maps for users and groups. */
|
||||
class SerialNumberManager {
|
||||
|
@ -40,33 +43,41 @@ class SerialNumberManager {
|
|||
}
|
||||
|
||||
private static class SerialNumberMap<T> {
|
||||
private int max = 0;
|
||||
private int nextSerialNumber() {return max++;}
|
||||
private AtomicInteger max = new AtomicInteger(1);
|
||||
private ConcurrentMap<T, Integer> t2i = new ConcurrentHashMap<T, Integer>();
|
||||
private ConcurrentMap<Integer, T> i2t = new ConcurrentHashMap<Integer, T>();
|
||||
|
||||
private Map<T, Integer> t2i = new HashMap<T, Integer>();
|
||||
private Map<Integer, T> i2t = new HashMap<Integer, T>();
|
||||
|
||||
synchronized int get(T t) {
|
||||
int get(T t) {
|
||||
if (t == null) {
|
||||
return 0;
|
||||
}
|
||||
Integer sn = t2i.get(t);
|
||||
if (sn == null) {
|
||||
sn = nextSerialNumber();
|
||||
t2i.put(t, sn);
|
||||
sn = max.getAndIncrement();
|
||||
Integer old = t2i.putIfAbsent(t, sn);
|
||||
if (old != null) {
|
||||
return old;
|
||||
}
|
||||
i2t.put(sn, t);
|
||||
}
|
||||
return sn;
|
||||
}
|
||||
|
||||
synchronized T get(int i) {
|
||||
if (!i2t.containsKey(i)) {
|
||||
T get(int i) {
|
||||
if (i == 0) {
|
||||
return null;
|
||||
}
|
||||
T t = i2t.get(i);
|
||||
if (t == null) {
|
||||
throw new IllegalStateException("!i2t.containsKey(" + i
|
||||
+ "), this=" + this);
|
||||
}
|
||||
return i2t.get(i);
|
||||
return t;
|
||||
}
|
||||
|
||||
@Override
|
||||
/** {@inheritDoc} */
|
||||
public String toString() {
|
||||
return "max=" + max + ",\n t2i=" + t2i + ",\n i2t=" + i2t;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue