This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new e366d97  Use concurrent open hash map to track pending requests in 
PerChannelBookieClient
e366d97 is described below

commit e366d97df11c84e29fcc4aadaab1bb23b30cf9db
Author: Matteo Merli <[email protected]>
AuthorDate: Mon Dec 4 15:56:37 2017 -0800

    Use concurrent open hash map to track pending requests in 
PerChannelBookieClient
    
    ConcurrentOpenHashMap doesn't allocate a Map.Entry for each
    insertion/lookup, so this reduces garbage in the fast path.
    
    Original change 64ab580a in yahoo-4.3 branch.
    
    Author: Matteo Merli <[email protected]>
    
    Reviewers: Sijie Guo <[email protected]>
    
    This closes #808 from ivankelly/yahoo-bp-6
---
 .../java/org/apache/bookkeeper/proto/PerChannelBookieClient.java  | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
index e6dad78..f094064 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
@@ -73,7 +73,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Queue;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -120,6 +119,7 @@ import 
org.apache.bookkeeper.tls.SecurityHandlerFactory.NodeType;
 import org.apache.bookkeeper.util.MathUtils;
 import org.apache.bookkeeper.util.OrderedSafeExecutor;
 import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.bookkeeper.util.collections.ConcurrentOpenHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -154,8 +154,8 @@ public class PerChannelBookieClient extends 
ChannelInboundHandlerAdapter {
     final int getBookieInfoTimeout;
     final int startTLSTimeout;
 
-    private final ConcurrentHashMap<CompletionKey, CompletionValue> 
completionObjects =
-        new ConcurrentHashMap<CompletionKey, CompletionValue>();
+    private final ConcurrentOpenHashMap<CompletionKey, CompletionValue> 
completionObjects =
+        new ConcurrentOpenHashMap<CompletionKey, CompletionValue>();
 
     // Map that hold duplicated read requests. The idea is to only use this 
map (synchronized) when there is a duplicate
     // read request for the same ledgerId/entryId
@@ -904,7 +904,7 @@ public class PerChannelBookieClient extends 
ChannelInboundHandlerAdapter {
                 errorOut(key, rc);
             }
         }
-        for (CompletionKey key : completionObjects.keySet()) {
+        for (CompletionKey key : completionObjects.keys()) {
             errorOut(key, rc);
         }
     }

-- 
To stop receiving notification emails like this one, please contact
['"[email protected]" <[email protected]>'].

Reply via email to