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:
Tsz-wo Sze 2012-08-17 01:55:33 +00:00
parent c849ef9bea
commit aa9bf3b472
2 changed files with 28 additions and 14 deletions

View File

@ -230,6 +230,9 @@ Release 2.0.1-alpha - UNRELEASED
HDFS-3697. Enable fadvise readahead by default. (todd) 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 BUG FIXES
HDFS-3385. The last block of INodeFileUnderConstruction is not HDFS-3385. The last block of INodeFileUnderConstruction is not

View File

@ -17,7 +17,10 @@
*/ */
package org.apache.hadoop.hdfs.server.namenode; 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. */ /** Manage name-to-serial-number maps for users and groups. */
class SerialNumberManager { class SerialNumberManager {
@ -40,31 +43,39 @@ class SerialNumberManager {
} }
private static class SerialNumberMap<T> { private static class SerialNumberMap<T> {
private int max = 0; private AtomicInteger max = new AtomicInteger(1);
private int nextSerialNumber() {return max++;} 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>(); int get(T t) {
private Map<Integer, T> i2t = new HashMap<Integer, T>(); if (t == null) {
return 0;
synchronized int get(T t) { }
Integer sn = t2i.get(t); Integer sn = t2i.get(t);
if (sn == null) { if (sn == null) {
sn = nextSerialNumber(); sn = max.getAndIncrement();
t2i.put(t, sn); Integer old = t2i.putIfAbsent(t, sn);
if (old != null) {
return old;
}
i2t.put(sn, t); i2t.put(sn, t);
} }
return sn; return sn;
} }
synchronized T get(int i) { T get(int i) {
if (!i2t.containsKey(i)) { if (i == 0) {
return null;
}
T t = i2t.get(i);
if (t == null) {
throw new IllegalStateException("!i2t.containsKey(" + i throw new IllegalStateException("!i2t.containsKey(" + i
+ "), this=" + this); + "), this=" + this);
} }
return i2t.get(i); return t;
} }
@Override /** {@inheritDoc} */
public String toString() { public String toString() {
return "max=" + max + ",\n t2i=" + t2i + ",\n i2t=" + i2t; return "max=" + max + ",\n t2i=" + t2i + ",\n i2t=" + i2t;
} }