HDFS-13990. Synchronization Issue With HashResolver. Contributed by BELUGA BEHR.

This commit is contained in:
Inigo Goiri 2018-10-18 10:05:27 -07:00
parent ba7e81667c
commit 1e78dfca40
1 changed files with 3 additions and 7 deletions

View File

@ -98,13 +98,9 @@ public class HashResolver implements OrderedResolver {
* namespaces using the provided set of namespace identifiers. * namespaces using the provided set of namespace identifiers.
*/ */
private ConsistentHashRing getHashResolver(final Set<String> namespaces) { private ConsistentHashRing getHashResolver(final Set<String> namespaces) {
int hash = namespaces.hashCode(); final int hash = namespaces.hashCode();
ConsistentHashRing resolver = this.hashResolverMap.get(hash); return this.hashResolverMap.computeIfAbsent(hash,
if (resolver == null) { k -> new ConsistentHashRing(namespaces));
resolver = new ConsistentHashRing(namespaces);
this.hashResolverMap.put(hash, resolver);
}
return resolver;
} }
/** /**