You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/04/02 01:26:19 UTC

[bookkeeper] branch master updated: Minimize number of thread local instances in CRC32CDigestManager

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 f5b4b78  Minimize number of thread local instances in CRC32CDigestManager
f5b4b78 is described below

commit f5b4b7866f34939b5cfa96bbdd3ff26a376c9677
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Sun Apr 1 18:26:11 2018 -0700

    Minimize number of thread local instances in CRC32CDigestManager
    
    CRC32CDigestManager uses thread local to store the current crc value (since the crc is computed on the headers and then on the payload).
    
    One issue is that the thread local instances are per-ledger and are never cleaned up from the thread local storage. As new ledgers get used, we'd have more memory used there.
    
    For digest we really need 1 single variable
    
    Changes:
     * Use Netty FastThreadLocal instead of JDK ThreadLocal
     * Made the thread local variable static, since one thread is computing the checksum for an entry of a given ledge, before moving to next entry.
     * Minimized thread local `.get()` access since it's more expensive than regular variable access.
     * Removed the usage of `isNewCrc` since initial value for crc32c (as well as crc32) is 0
    
    Author: Matteo Merli <mm...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1306 from merlimat/impro-crc32c
---
 .../proto/checksum/CRC32CDigestManager.java        | 37 +++++++++++-----------
 1 file changed, 19 insertions(+), 18 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java
index ca7fdec..1387028 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/CRC32CDigestManager.java
@@ -20,24 +20,28 @@ package org.apache.bookkeeper.proto.checksum;
 
 import com.scurrilous.circe.checksum.Crc32cIntChecksum;
 import com.scurrilous.circe.crc.Sse42Crc32C;
+
 import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.mutable.MutableBoolean;
+import io.netty.util.concurrent.FastThreadLocal;
+
+import lombok.extern.slf4j.Slf4j;
+
 import org.apache.commons.lang3.mutable.MutableInt;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
+@Slf4j
 class CRC32CDigestManager extends DigestManager {
-    static final Logger LOG = LoggerFactory.getLogger(CRC32CDigestManager.class);
 
-    private final ThreadLocal<MutableInt> currentCrc = ThreadLocal
-            .withInitial(() -> new MutableInt(0));
-    private final ThreadLocal<MutableBoolean> isNewCrc = ThreadLocal
-            .withInitial(() -> new MutableBoolean(true));
+    private static final FastThreadLocal<MutableInt> currentCrc = new FastThreadLocal<MutableInt>() {
+        @Override
+        protected MutableInt initialValue() throws Exception {
+            return new MutableInt(0);
+        }
+    };
 
     public CRC32CDigestManager(long ledgerId) {
         super(ledgerId);
         if (!Sse42Crc32C.isSupported()) {
-            LOG.error("Sse42Crc32C is not supported, will use less slower CRC32C implementation.");
+            log.error("Sse42Crc32C is not supported, will use less slower CRC32C implementation.");
         }
     }
 
@@ -48,18 +52,15 @@ class CRC32CDigestManager extends DigestManager {
 
     @Override
     void populateValueAndReset(ByteBuf buf) {
-        buf.writeInt(currentCrc.get().intValue());
-        isNewCrc.get().setTrue();
+        MutableInt current = currentCrc.get();
+        buf.writeInt(current.intValue());
+        current.setValue(0);
     }
 
     @Override
     void update(ByteBuf data) {
-        if (isNewCrc.get().isTrue()) {
-            isNewCrc.get().setFalse();
-            currentCrc.get().setValue(Crc32cIntChecksum.computeChecksum(data));
-        } else {
-            final int lastCrc = currentCrc.get().intValue();
-            currentCrc.get().setValue(Crc32cIntChecksum.resumeChecksum(lastCrc, data));
-        }
+        MutableInt current = currentCrc.get();
+        final int lastCrc = current.intValue();
+        current.setValue(Crc32cIntChecksum.resumeChecksum(lastCrc, data));
     }
 }

-- 
To stop receiving notification emails like this one, please contact
sijie@apache.org.