zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [zookeeper] eolivelli commented on a change in pull request #911: ZOOKEEPER-3365: Use Concurrent HashMap in NettyServerCnxnFactory
Date Sun, 14 Apr 2019 05:54:18 GMT
eolivelli commented on a change in pull request #911: ZOOKEEPER-3365: Use Concurrent HashMap
in NettyServerCnxnFactory
URL: https://github.com/apache/zookeeper/pull/911#discussion_r275141538
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java
 ##########
 @@ -512,44 +511,34 @@ public InetSocketAddress getLocalAddress() {
         return localAddress;
     }
 
-    private void addCnxn(NettyServerCnxn cnxn) {
+    private void addCnxn(final NettyServerCnxn cnxn) {
         cnxns.add(cnxn);
-        synchronized (ipMap){
-            InetAddress addr =
-                ((InetSocketAddress)cnxn.getChannel().remoteAddress()).getAddress();
-            Set<NettyServerCnxn> s = ipMap.get(addr);
-            if (s == null) {
-                s = new HashSet<>();
-                ipMap.put(addr, s);
+        InetAddress addr =
+            ((InetSocketAddress) cnxn.getChannel().remoteAddress()).getAddress();
+
+        ipMap.compute(addr, (a, cnxnSet) -> {
+            if (cnxnSet == null) {
+                cnxnSet = new HashSet<>();
             }
-            s.add(cnxn);
-        }
+            cnxnSet.add(cnxn);
+            return cnxnSet;
+        });
     }
-
+  
     void removeCnxnFromIpMap(NettyServerCnxn cnxn, InetAddress remoteAddress) {
-        synchronized (ipMap) {
-            Set<NettyServerCnxn> s = ipMap.get(remoteAddress);
-            if (s != null) {
-                s.remove(cnxn);
-                if (s.isEmpty()) {
-                    ipMap.remove(remoteAddress);
-                }
-                return;
-            }
+        ipMap.compute(remoteAddress, (addr, cnxnSet) -> {
+        if (cnxnSet == null) {
+            LOG.error("Unexpected remote address {} when removing cnxn {}",
+                remoteAddress, cnxn);
+            cnxnSet = Collections.emptySet();
         }
-        // Fallthrough and log errors outside the synchronized block
-        LOG.error(
-                "Unexpected null set for remote address {} when removing cnxn {}",
-                remoteAddress,
-                cnxn);
+        cnxnSet.remove(cnxn);
+        return cnxnSet.isEmpty() ? null : cnxnSet;
+      });
     }
 
-    private int getClientCnxnCount(InetAddress addr) {
-        synchronized (ipMap) {
-            Set<NettyServerCnxn> s = ipMap.get(addr);
-            if (s == null) return 0;
-            return s.size();
-        }
+    private int getClientCnxnCount(final InetAddress addr) {
+      return ipMap.getOrDefault(addr, Collections.emptySet()).size();
 
 Review comment:
   We have to synchronize access to the HashSet.
   It is not enough to protect write access operation but you have to guard even this call
to *size()*
   
   Should we use a concurrent set ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message