You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by mm...@apache.org on 2022/06/08 01:29:16 UTC

[bookkeeper] branch branch-4.15 updated: Try to use Java9 CRC32C when JNI based CRC is not available (#3309)

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

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


The following commit(s) were added to refs/heads/branch-4.15 by this push:
     new dba6a493c1 Try to use Java9 CRC32C when JNI based CRC is not available (#3309)
dba6a493c1 is described below

commit dba6a493c1b92f1b848ea7c3d23cdd764f6bbe0e
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Tue Jun 7 18:26:50 2022 -0700

    Try to use Java9 CRC32C when JNI based CRC is not available (#3309)
    
    * Try to use Java9 CRC32C when JNI based CRC is not available
    
    * Fixed shading test dependency
    
    * Fixed dlog shading deps
---
 .../proto/checksum/CRC32CDigestManager.java        |  11 ---
 .../circe/checksum/Crc32cIntChecksum.java          |  40 ++------
 .../circe/checksum/Crc32cLongChecksum.java         |  92 -----------------
 .../com/scurrilous/circe/checksum/IntHash.java     |  26 +++++
 .../scurrilous/circe/checksum/Java8IntHash.java    |  46 +++++++++
 .../scurrilous/circe/checksum/Java9IntHash.java    | 110 +++++++++++++++++++++
 .../com/scurrilous/circe/checksum/JniIntHash.java  |  46 +++++++++
 .../scurrilous/circe/checksum/ChecksumTest.java    |  32 +-----
 microbenchmarks/run.sh                             |   4 +-
 .../proto/checksum/DigestTypeBenchmark.java        |  28 +++---
 shaded/distributedlog-core-shaded/pom.xml          |   4 -
 11 files changed, 257 insertions(+), 182 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 f12b5479f8..c3be132c39 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
@@ -19,21 +19,15 @@ package org.apache.bookkeeper.proto.checksum;
 */
 
 import com.scurrilous.circe.checksum.Crc32cIntChecksum;
-import com.scurrilous.circe.crc.Sse42Crc32C;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.util.concurrent.FastThreadLocal;
-
 import lombok.extern.slf4j.Slf4j;
-
 import org.apache.commons.lang3.mutable.MutableInt;
 
 @Slf4j
 class CRC32CDigestManager extends DigestManager {
 
-    private static boolean nonSupportedMessagePrinted = false;
-
     private static final FastThreadLocal<MutableInt> currentCrc = new FastThreadLocal<MutableInt>() {
         @Override
         protected MutableInt initialValue() throws Exception {
@@ -43,11 +37,6 @@ class CRC32CDigestManager extends DigestManager {
 
     public CRC32CDigestManager(long ledgerId, boolean useV2Protocol, ByteBufAllocator allocator) {
         super(ledgerId, useV2Protocol, allocator);
-
-        if (!Sse42Crc32C.isSupported() && !nonSupportedMessagePrinted) {
-            log.warn("Sse42Crc32C is not supported, will use a slower CRC32C implementation.");
-            nonSupportedMessagePrinted = true;
-        }
     }
 
     @Override
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cIntChecksum.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cIntChecksum.java
index 528244b12b..c286b6d51b 100644
--- a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cIntChecksum.java
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cIntChecksum.java
@@ -18,30 +18,20 @@
  */
 package com.scurrilous.circe.checksum;
 
-import static com.scurrilous.circe.params.CrcParameters.CRC32C;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.scurrilous.circe.IncrementalIntHash;
 import com.scurrilous.circe.crc.Sse42Crc32C;
-import com.scurrilous.circe.crc.StandardCrcProvider;
 import io.netty.buffer.ByteBuf;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class Crc32cIntChecksum {
 
-    private static final Logger log = LoggerFactory.getLogger(Crc32cIntChecksum.class);
-
-    @VisibleForTesting
-    final static IncrementalIntHash CRC32C_HASH;
+    private final static IntHash CRC32C_HASH;
 
     static {
         if (Sse42Crc32C.isSupported()) {
-            CRC32C_HASH = new Crc32cSse42Provider().getIncrementalInt(CRC32C);
-            log.info("SSE4.2 CRC32C provider initialized");
+            CRC32C_HASH = new JniIntHash();
+        } else if (Java9IntHash.HAS_JAVA9_CRC32C) {
+            CRC32C_HASH = new Java9IntHash();
         } else {
-            CRC32C_HASH = new StandardCrcProvider().getIncrementalInt(CRC32C);
-            log.warn("Failed to load Circe JNI library. Falling back to Java based CRC32c provider");
+            CRC32C_HASH = new Java8IntHash();
         }
     }
 
@@ -53,17 +43,9 @@ public class Crc32cIntChecksum {
      * @return
      */
     public static int computeChecksum(ByteBuf payload) {
-        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
-            return CRC32C_HASH.calculate(payload.memoryAddress() + payload.readerIndex(), payload.readableBytes());
-        } else if (payload.hasArray()) {
-            return CRC32C_HASH.calculate(payload.array(), payload.arrayOffset() + payload.readerIndex(),
-                payload.readableBytes());
-        } else {
-            return CRC32C_HASH.calculate(payload.nioBuffer());
-        }
+        return CRC32C_HASH.calculate(payload);
     }
 
-
     /**
      * Computes incremental checksum with input previousChecksum and input payload
      *
@@ -72,15 +54,7 @@ public class Crc32cIntChecksum {
      * @return
      */
     public static int resumeChecksum(int previousChecksum, ByteBuf payload) {
-        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
-            return CRC32C_HASH.resume(previousChecksum, payload.memoryAddress() + payload.readerIndex(),
-                payload.readableBytes());
-        } else if (payload.hasArray()) {
-            return CRC32C_HASH.resume(previousChecksum, payload.array(), payload.arrayOffset() + payload.readerIndex(),
-                payload.readableBytes());
-        } else {
-            return CRC32C_HASH.resume(previousChecksum, payload.nioBuffer());
-        }
+        return CRC32C_HASH.resume(previousChecksum, payload);
     }
 
 }
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cLongChecksum.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cLongChecksum.java
deleted file mode 100644
index c283e5680b..0000000000
--- a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Crc32cLongChecksum.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package com.scurrilous.circe.checksum;
-
-import static com.scurrilous.circe.params.CrcParameters.CRC32C;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.scurrilous.circe.IncrementalIntHash;
-import com.scurrilous.circe.crc.Sse42Crc32C;
-import com.scurrilous.circe.crc.StandardCrcProvider;
-import io.netty.buffer.ByteBuf;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class Crc32cLongChecksum {
-
-    private static final Logger log = LoggerFactory.getLogger(Crc32cLongChecksum.class);
-
-    @VisibleForTesting
-    final static IncrementalIntHash CRC32C_HASH;
-
-    static {
-        if (Sse42Crc32C.isSupported()) {
-            CRC32C_HASH = new Crc32cSse42Provider().getIncrementalInt(CRC32C);
-            if (log.isDebugEnabled()) {
-                log.debug("SSE4.2 CRC32C provider initialized");
-            }
-        } else {
-            CRC32C_HASH = new StandardCrcProvider().getIncrementalInt(CRC32C);
-            log.warn("Failed to load Circe JNI library. Falling back to Java based CRC32c provider");
-        }
-    }
-
-    /**
-     * Computes crc32c checksum: if it is able to load crc32c native library then it computes using that native library
-     * which is faster as it computes using hardware machine instruction else it computes using crc32c algo.
-     *
-     * @param payload
-     * @return
-     */
-    public static long computeChecksum(ByteBuf payload) {
-        int crc;
-        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
-            crc = CRC32C_HASH.calculate(payload.memoryAddress() + payload.readerIndex(), payload.readableBytes());
-        } else if (payload.hasArray()) {
-            crc = CRC32C_HASH.calculate(payload.array(), payload.arrayOffset() + payload.readerIndex(),
-                payload.readableBytes());
-        } else {
-            crc = CRC32C_HASH.calculate(payload.nioBuffer());
-        }
-        return crc & 0xffffffffL;
-    }
-
-
-    /**
-     * Computes incremental checksum with input previousChecksum and input payload
-     *
-     * @param previousChecksum : previously computed checksum
-     * @param payload
-     * @return
-     */
-    public static long resumeChecksum(long previousChecksum, ByteBuf payload) {
-        int crc = (int) previousChecksum;
-        if (payload.hasMemoryAddress() && (CRC32C_HASH instanceof Sse42Crc32C)) {
-            crc = CRC32C_HASH.resume(crc, payload.memoryAddress() + payload.readerIndex(),
-                payload.readableBytes());
-        } else if (payload.hasArray()) {
-            crc = CRC32C_HASH.resume(crc, payload.array(), payload.arrayOffset() + payload.readerIndex(),
-                payload.readableBytes());
-        } else {
-            crc = CRC32C_HASH.resume(crc, payload.nioBuffer());
-        }
-        return crc & 0xffffffffL;
-    }
-
-}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/IntHash.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/IntHash.java
new file mode 100644
index 0000000000..e0e0939cef
--- /dev/null
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/IntHash.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import io.netty.buffer.ByteBuf;
+
+public interface IntHash {
+    int calculate(ByteBuf buffer);
+    int resume(int current, ByteBuf buffer);
+}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java8IntHash.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java8IntHash.java
new file mode 100644
index 0000000000..756a1db1c2
--- /dev/null
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java8IntHash.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import static com.scurrilous.circe.params.CrcParameters.CRC32C;
+import com.scurrilous.circe.IncrementalIntHash;
+import com.scurrilous.circe.crc.StandardCrcProvider;
+import io.netty.buffer.ByteBuf;
+import java.nio.ByteBuffer;
+import java.util.zip.Checksum;
+
+public class Java8IntHash implements IntHash {
+
+    private final IncrementalIntHash hash = new StandardCrcProvider().getIncrementalInt(CRC32C);
+
+    @Override
+    public int calculate(ByteBuf buffer) {
+        return resume(0, buffer);
+    }
+
+    @Override
+    public int resume(int current, ByteBuf buffer) {
+        if (buffer.hasArray()) {
+            return hash.resume(current, buffer.array(), buffer.arrayOffset() + buffer.readerIndex(),
+                    buffer.readableBytes());
+        } else {
+            return hash.resume(current, buffer.nioBuffer());
+        }
+    }
+}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java9IntHash.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java9IntHash.java
new file mode 100644
index 0000000000..729fa477ef
--- /dev/null
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java9IntHash.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.util.concurrent.FastThreadLocal;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class Java9IntHash implements IntHash {
+    static final boolean HAS_JAVA9_CRC32C;
+    private static final Method UPDATE_BYTES;
+    private static final Method UPDATE_DIRECT_BYTEBUFFER;
+
+    private static final String CRC32C_CLASS_NAME = "java.util.zip.CRC32C";
+
+    private static final FastThreadLocal<byte[]> TL_BUFFER = new FastThreadLocal<byte[]>() {
+        @Override
+        protected byte[] initialValue() {
+            return new byte[4096];
+        }
+    };
+
+    static {
+        boolean hasJava9CRC32C = false;
+        Method updateBytes = null;
+        Method updateDirectByteBuffer = null;
+
+        try {
+            Class<?> c = Class.forName(CRC32C_CLASS_NAME);
+            updateBytes = c.getDeclaredMethod("updateBytes", int.class, byte[].class, int.class, int.class);
+            updateBytes.setAccessible(true);
+            updateDirectByteBuffer =
+                    c.getDeclaredMethod("updateDirectByteBuffer", int.class, long.class, int.class, int.class);
+            updateDirectByteBuffer.setAccessible(true);
+
+            hasJava9CRC32C = true;
+        } catch (Exception e) {
+            log.debug("Unable to use reflected methods: ", e);
+            updateBytes = null;
+            updateDirectByteBuffer = null;
+        }
+
+        HAS_JAVA9_CRC32C = hasJava9CRC32C;
+        UPDATE_BYTES = updateBytes;
+        UPDATE_DIRECT_BYTEBUFFER = updateDirectByteBuffer;
+    }
+
+    @Override
+    public int calculate(ByteBuf buffer) {
+        return resume(0, buffer);
+    }
+
+    private int resume(int current, long address, int offset, int length) {
+        try {
+            return (int) UPDATE_DIRECT_BYTEBUFFER.invoke(null, current, address, offset, offset + length);
+        } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private int resume(int current, byte[] array, int offset, int length) {
+        try {
+            return (int) UPDATE_BYTES.invoke(null, current, array, offset, offset + length);
+        } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public int resume(int current, ByteBuf buffer) {
+        int negCrc = ~current;
+
+        if (buffer.hasMemoryAddress()) {
+            negCrc = resume(negCrc, buffer.memoryAddress(), buffer.readerIndex(), buffer.readableBytes());
+        } else if (buffer.hasArray()) {
+            int offset = buffer.arrayOffset() + buffer.readerIndex();
+            negCrc = resume(negCrc, buffer.array(), offset, buffer.readableBytes());
+        } else {
+            byte[] b = TL_BUFFER.get();
+            int toRead = buffer.readableBytes();
+            while (toRead > 0) {
+                int length = Math.min(toRead, b.length);
+                buffer.readBytes(b, 0, length);
+                negCrc = resume(negCrc, b, 0, length);
+                toRead -= length;
+            }
+        }
+
+        return ~negCrc;
+    }
+}
diff --git a/circe-checksum/src/main/java/com/scurrilous/circe/checksum/JniIntHash.java b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/JniIntHash.java
new file mode 100644
index 0000000000..cc29496396
--- /dev/null
+++ b/circe-checksum/src/main/java/com/scurrilous/circe/checksum/JniIntHash.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package com.scurrilous.circe.checksum;
+
+import static com.scurrilous.circe.params.CrcParameters.CRC32C;
+import com.scurrilous.circe.IncrementalIntHash;
+import io.netty.buffer.ByteBuf;
+
+public class JniIntHash implements IntHash {
+
+    private final IncrementalIntHash hash = new Crc32cSse42Provider().getIncrementalInt(CRC32C);
+
+    @Override
+    public int calculate(ByteBuf buffer) {
+        return resume(0, buffer);
+    }
+
+    @Override
+    public int resume(int current, ByteBuf buffer) {
+        if (buffer.hasMemoryAddress()) {
+            return hash.resume(current, buffer.memoryAddress() + buffer.readerIndex(),
+                    buffer.readableBytes());
+        } else if (buffer.hasArray()) {
+            return hash.resume(current, buffer.array(), buffer.arrayOffset() + buffer.readerIndex(),
+                    buffer.readableBytes());
+        } else {
+            return hash.resume(current, buffer.nioBuffer());
+        }
+    }
+}
diff --git a/circe-checksum/src/test/java/com/scurrilous/circe/checksum/ChecksumTest.java b/circe-checksum/src/test/java/com/scurrilous/circe/checksum/ChecksumTest.java
index 67fee3ce37..f085b18925 100644
--- a/circe-checksum/src/test/java/com/scurrilous/circe/checksum/ChecksumTest.java
+++ b/circe-checksum/src/test/java/com/scurrilous/circe/checksum/ChecksumTest.java
@@ -53,17 +53,17 @@ public class ChecksumTest {
     public void testCrc32cValueIncremental() {
         final byte[] bytes = "Some String".getBytes();
 
-        int checksum = Crc32cIntChecksum.CRC32C_HASH.calculate(bytes, 0, bytes.length);
+        int checksum = Crc32cIntChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes));
         assertEquals(608512271, checksum);
 
-        checksum = Crc32cIntChecksum.CRC32C_HASH.calculate(bytes, 0, 1);
+        checksum = Crc32cIntChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes, 0, 1));
         for (int i = 1; i < bytes.length; i++) {
-            checksum = Crc32cIntChecksum.CRC32C_HASH.resume(checksum, bytes, i, 1);
+            checksum = Crc32cIntChecksum.resumeChecksum(checksum, Unpooled.wrappedBuffer(bytes, i, 1));
         }
         assertEquals(608512271, checksum);
 
-        checksum = Crc32cIntChecksum.CRC32C_HASH.calculate(bytes, 0, 4);
-        checksum = Crc32cIntChecksum.CRC32C_HASH.resume(checksum, bytes, 4, 7);
+        checksum = Crc32cIntChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes, 0, 4));
+        checksum = Crc32cIntChecksum.resumeChecksum(checksum, Unpooled.wrappedBuffer(bytes, 4, 7));
         assertEquals(608512271, checksum);
 
 
@@ -91,28 +91,6 @@ public class ChecksumTest {
         assertEquals(608512271L, checksum);
     }
 
-    @Test
-    public void testCrc32cLongValueIncremental() {
-        final byte[] bytes = "Some String".getBytes();
-
-        long checksum = Crc32cLongChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes));
-        assertEquals(608512271, checksum);
-
-        checksum = Crc32cLongChecksum.computeChecksum(Unpooled.wrappedBuffer(bytes, 0, 1));
-        for (int i = 1; i < bytes.length; i++) {
-            checksum = Crc32cLongChecksum.resumeChecksum(
-                checksum, Unpooled.wrappedBuffer(bytes, i, 1));
-        }
-        assertEquals(608512271, checksum);
-
-        ByteBuf buffer = Unpooled.wrappedBuffer(bytes, 0, 4);
-        checksum = Crc32cLongChecksum.computeChecksum(buffer);
-        checksum = Crc32cLongChecksum.resumeChecksum(
-            checksum, Unpooled.wrappedBuffer(bytes, 4, bytes.length - 4));
-
-        assertEquals(608512271, checksum);
-    }
-
     @Test
     public void testCRC32CIncrementalLong() {
         IncrementalLongHash crc32c = new StandardCrcProvider().getIncrementalLong(CRC32C);
diff --git a/microbenchmarks/run.sh b/microbenchmarks/run.sh
index 6d1348c72d..a95d8d2085 100755
--- a/microbenchmarks/run.sh
+++ b/microbenchmarks/run.sh
@@ -15,5 +15,7 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 
-java -Xms1G -Xmx1G -Djdk.nio.maxCachedBufferSize=0 -Djava.net.preferIPv4Stack=true -Duser.timezone=UTC -XX:-MaxFDLimit -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+ResizeTLAB -XX:-ResizePLAB -XX:MetaspaceSize=128m -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -XX:+ParallelRefProcEnabled -XX:StackShadowPages=20 -XX:+UseCompressedOops -XX:+DisableExplicitGC -XX:StringTableSize=1000003 -XX:InitiatingHeapOccupancyPercent=40 -jar target/benchmarks.jar $@ -prof gc -prof stack:lines=5 [...]
+java -Xms1G -Xmx1G -Djdk.nio.maxCachedBufferSize=0 -Djava.net.preferIPv4Stack=true -Duser.timezone=UTC \
+    --add-opens java.base/java.util.zip=ALL-UNNAMED \
+    -XX:-MaxFDLimit -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+ResizeTLAB -XX:-ResizePLAB -XX:MetaspaceSize=128m -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -XX:+ParallelRefProcEnabled -XX:StackShadowPages=20 -XX:+UseCompressedOops -XX:+DisableExplicitGC -XX:StringTableSize=1000003 -XX:InitiatingHeapOccupancyPercent=40 -jar target/benchmarks.jar $@ -prof gc -prof stack:lines=5;time=1;top=3 
 
diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java
index cc65edea5e..dca06e1d0e 100644
--- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java
+++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java
@@ -61,8 +61,8 @@ public class DigestTypeBenchmark {
      * BufferType.
      */
     public enum BufferType {
-        ARRAY_BACKED,
-        NOT_ARRAY_BACKED,
+//        ARRAY_BACKED,
+//        NOT_ARRAY_BACKED,
         BYTE_BUF_DEFAULT_ALLOC
     }
 
@@ -70,8 +70,8 @@ public class DigestTypeBenchmark {
      * Digest.
      */
     public enum Digest {
-        MAC,
-        CRC32,
+//        MAC,
+//        CRC32,
         CRC32_C,
     }
 
@@ -91,7 +91,7 @@ public class DigestTypeBenchmark {
         public BufferType bufferType;
         @Param
         public Digest digest;
-        @Param({"1024", "4086", "8192", "16384", "65536"})
+        @Param({"64", "1024", "4086", "8192", "16384", "65536"})
         public int entrySize;
 
         private DigestManager crc32;
@@ -139,10 +139,10 @@ public class DigestTypeBenchmark {
 
         public ByteBuf getByteBuff(BufferType bType) {
             switch (bType) {
-                case ARRAY_BACKED:
-                    return arrayBackedBuffer;
-                case NOT_ARRAY_BACKED:
-                    return notArrayBackedBuffer;
+//                case ARRAY_BACKED:
+//                    return arrayBackedBuffer;
+//                case NOT_ARRAY_BACKED:
+//                    return notArrayBackedBuffer;
                 case BYTE_BUF_DEFAULT_ALLOC:
                     return byteBufDefaultAlloc;
                 default:
@@ -152,12 +152,12 @@ public class DigestTypeBenchmark {
 
         public DigestManager getDigestManager(Digest digest) {
             switch (digest) {
-                case CRC32:
-                    return crc32;
+//                case CRC32:
+//                    return crc32;
                 case CRC32_C:
                     return crc32c;
-                case MAC:
-                    return mac;
+//                case MAC:
+//                    return mac;
                 default:
                     throw new IllegalArgumentException("unknown digest " + digest);
             }
@@ -168,7 +168,7 @@ public class DigestTypeBenchmark {
     @BenchmarkMode(Mode.Throughput)
     @OutputTimeUnit(TimeUnit.MILLISECONDS)
     @Warmup(iterations = 2, time = 3, timeUnit = TimeUnit.SECONDS)
-    @Measurement(iterations = 5, time = 12, timeUnit = TimeUnit.SECONDS)
+    @Measurement(iterations = 2, time = 10, timeUnit = TimeUnit.SECONDS)
     @Threads(2)
     @Fork(value = 1, warmups = 1)
     public void digestManager(MyState state) {
diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml
index 2871b08a6f..2ad6399452 100644
--- a/shaded/distributedlog-core-shaded/pom.xml
+++ b/shaded/distributedlog-core-shaded/pom.xml
@@ -43,10 +43,6 @@
           <groupId>log4j</groupId>
           <artifactId>log4j</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>io.netty</groupId>
-          <artifactId>netty-common</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>io.netty</groupId>
           <artifactId>netty-buffer</artifactId>