HBASE-21027 Inconsistent synchronization in CacheableDeserializerIdManager
Signed-off-by: Sean Busbey <busbey@apache.org>
This commit is contained in:
parent
e2fcde2d6f
commit
c6ff1de7e2
|
@ -18,9 +18,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io.hfile;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
|
@ -33,7 +34,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class CacheableDeserializerIdManager {
|
||||
private static final Map<Integer, CacheableDeserializer<Cacheable>> registeredDeserializers = new HashMap<>();
|
||||
private static final Map<Integer, CacheableDeserializer<Cacheable>> registeredDeserializers =
|
||||
new ConcurrentHashMap<>();
|
||||
private static final AtomicInteger identifier = new AtomicInteger(0);
|
||||
|
||||
/**
|
||||
|
@ -45,9 +47,8 @@ public class CacheableDeserializerIdManager {
|
|||
*/
|
||||
public static int registerDeserializer(CacheableDeserializer<Cacheable> cd) {
|
||||
int idx = identifier.incrementAndGet();
|
||||
synchronized (registeredDeserializers) {
|
||||
registeredDeserializers.put(idx, cd);
|
||||
}
|
||||
// No synchronization here because keys will be unique
|
||||
registeredDeserializers.put(idx, cd);
|
||||
return idx;
|
||||
}
|
||||
|
||||
|
@ -64,13 +65,10 @@ public class CacheableDeserializerIdManager {
|
|||
* of a file.
|
||||
*/
|
||||
public static Map<Integer,String> save() {
|
||||
Map<Integer, String> snapshot = new HashMap<>();
|
||||
synchronized (registeredDeserializers) {
|
||||
for (Map.Entry<Integer, CacheableDeserializer<Cacheable>> entry :
|
||||
registeredDeserializers.entrySet()) {
|
||||
snapshot.put(entry.getKey(), entry.getValue().getClass().getName());
|
||||
}
|
||||
}
|
||||
return snapshot;
|
||||
// No synchronization here because weakly consistent view should be good enough
|
||||
// The assumed risk is that we might not see a new serializer that comes in while iterating,
|
||||
// but with a synchronized block, we won't see it anyway
|
||||
return registeredDeserializers.entrySet().stream()
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getClass().getName()));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue