HADOOP-9932. Improper synchronization in RetryCache. Contributed by Kihwal Lee.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1520126 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0e0271b5fd
commit
cd663baf16
|
@ -449,6 +449,8 @@ Release 2.1.1-beta - UNRELEASED
|
|||
HADOOP-9924. FileUtil.createJarWithClassPath() does not generate relative
|
||||
classpath correctly. (Shanyu Zhao via ivanmi)
|
||||
|
||||
HADOOP-9932. Improper synchronization in RetryCache. (kihwal)
|
||||
|
||||
Release 2.1.0-beta - 2013-08-22
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -76,6 +76,12 @@ public class RetryCache {
|
|||
this.expirationTime = expirationTime;
|
||||
}
|
||||
|
||||
CacheEntry(byte[] clientId, int callId, long expirationTime,
|
||||
boolean success) {
|
||||
this(clientId, callId, expirationTime);
|
||||
this.state = success ? SUCCESS : FAILED;
|
||||
}
|
||||
|
||||
private static int hashCode(long value) {
|
||||
return (int)(value ^ (value >>> 32));
|
||||
}
|
||||
|
@ -147,6 +153,12 @@ public class RetryCache {
|
|||
this.payload = payload;
|
||||
}
|
||||
|
||||
CacheEntryWithPayload(byte[] clientId, int callId, Object payload,
|
||||
long expirationTime, boolean success) {
|
||||
super(clientId, callId, expirationTime, success);
|
||||
this.payload = payload;
|
||||
}
|
||||
|
||||
/** Override equals to avoid findbugs warnings */
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
|
@ -253,19 +265,21 @@ public class RetryCache {
|
|||
*/
|
||||
public void addCacheEntry(byte[] clientId, int callId) {
|
||||
CacheEntry newEntry = new CacheEntry(clientId, callId, System.nanoTime()
|
||||
+ expirationTime);
|
||||
newEntry.completed(true);
|
||||
+ expirationTime, true);
|
||||
synchronized(this) {
|
||||
set.put(newEntry);
|
||||
}
|
||||
}
|
||||
|
||||
public void addCacheEntryWithPayload(byte[] clientId, int callId,
|
||||
Object payload) {
|
||||
CacheEntry newEntry = new CacheEntryWithPayload(clientId, callId, payload,
|
||||
System.nanoTime() + expirationTime);
|
||||
// since the entry is loaded from editlog, we can assume it succeeded.
|
||||
newEntry.completed(true);
|
||||
CacheEntry newEntry = new CacheEntryWithPayload(clientId, callId, payload,
|
||||
System.nanoTime() + expirationTime, true);
|
||||
synchronized(this) {
|
||||
set.put(newEntry);
|
||||
}
|
||||
}
|
||||
|
||||
private static CacheEntry newEntry(long expirationTime) {
|
||||
return new CacheEntry(Server.getClientId(), Server.getCallId(),
|
||||
|
|
Loading…
Reference in New Issue