zookeeper-dev mailing list archives

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

 ##########
 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:
   I do not think it necessary to block here.  Calling the `size()` method will not break
anything and and the most it can be off by is 1 because the insert into the map is blocking.
 That is to say, imagine a `put()` call on the `Map` and the data structure has been updated
with the new record, but the size has not yet been incremented.  It's possible that this call
to `size()` happens right between those two actions.  I'm not sure if it's worth adding the
overhead of synchronization, but I don't have a strong feeling on it either way.

----------------------------------------------------------------
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