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

eolivelli pushed a commit to branch branch-4.14
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/branch-4.14 by this push:
     new 35a2e9c5ee WriteLacResponse should be processed in the same thread as 
other requ… (#3452)
35a2e9c5ee is described below

commit 35a2e9c5ee3dc1698d99e2ddbe44148a13ad35bd
Author: karanmehta93 <[email protected]>
AuthorDate: Sat Aug 20 23:46:14 2022 -0700

    WriteLacResponse should be processed in the same thread as other requ… 
(#3452)
    
    * WriteLacResponse should be processed in the same thread as other requests 
for same ledgerId
    
    In PCBC code, the WriteLacCompletion object uses LAC as the key for 
selecting the thread in which the response
    will be processed. However, according to bookkeeper code guarantees, the 
thread should be decided based on hash of ledgerId.
    
    * Empty-Commit
---
 .../main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

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 1c4a4a2f8f..e894d5a820 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
@@ -673,7 +673,7 @@ public class PerChannelBookieClient extends 
ChannelInboundHandlerAdapter {
         // writeLac is mostly like addEntry hence uses addEntryTimeout
         completionObjects.put(completionKey,
                               new WriteLacCompletion(completionKey, cb,
-                                                     ctx, lac));
+                                                     ctx, ledgerId));
 
         // Build the request
         BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder()

Reply via email to