From fc36487acb0189f8b26ed50737a86499f25dbdb8 Mon Sep 17 00:00:00 2001 From: Mike Drob Date: Wed, 8 Aug 2018 14:31:07 -0500 Subject: [PATCH] HBASE-21027 Inconsistent synchronization in CacheableDeserializerIdManager --- .../io/hfile/CacheableDeserializerIdManager.java | 22 +++++++++------------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializerIdManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializerIdManager.java index bcc29c2fda..709adb8875 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializerIdManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializerIdManager.java @@ -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> registeredDeserializers = new HashMap<>(); + private static final Map> registeredDeserializers = + new ConcurrentHashMap<>(); private static final AtomicInteger identifier = new AtomicInteger(0); /** @@ -45,9 +47,8 @@ public class CacheableDeserializerIdManager { */ public static int registerDeserializer(CacheableDeserializer 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,8 @@ public class CacheableDeserializerIdManager { * of a file. */ public static Map save() { - Map snapshot = new HashMap<>(); - synchronized (registeredDeserializers) { - for (Map.Entry> entry : - registeredDeserializers.entrySet()) { - snapshot.put(entry.getKey(), entry.getValue().getClass().getName()); - } - } - return snapshot; + // No synchronization here because weakly consistent view should be good enough + return registeredDeserializers.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, v -> v.getClass().getName())); } } -- 2.16.1