From 6b5406d22d87f6ef07bb6c7df922c382909b9dfb Mon Sep 17 00:00:00 2001 From: Siyao Meng <50227127+smengcl@users.noreply.github.com> Date: Sat, 23 May 2026 15:13:59 -0700 Subject: [PATCH] HDDS-15356. Make multi-buffer chunk checksum allocation-free Generated-by: Claude Code (Opus 4.7) --- .../apache/hadoop/ozone/common/Checksum.java | 183 +++++++++++------- .../hadoop/ozone/common/ChecksumCache.java | 109 ++++++++--- .../ozone/common/utils/BufferUtils.java | 12 ++ .../ozone/common/TestChecksumCache.java | 13 +- .../ozone/common/TestChecksumMultiBuffer.java | 97 ++++++++++ .../container/keyvalue/KeyValueHandler.java | 4 +- 6 files changed, 313 insertions(+), 105 deletions(-) create mode 100644 hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumMultiBuffer.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java index ffdab4cde160..072a465eb744 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.function.Function; import java.util.function.Supplier; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; @@ -53,57 +52,105 @@ public class Checksum { */ private final ChecksumCache checksumCache; - private static Function newMessageDigestFunction( - String algorithm) { - final MessageDigest md; + private static MessageDigest newMessageDigest(String algorithm) { try { - md = MessageDigest.getInstance(algorithm); + return MessageDigest.getInstance(algorithm); } catch (NoSuchAlgorithmException e) { throw new IllegalStateException( "Failed to get MessageDigest for " + algorithm, e); } - return data -> { - md.reset(); - md.update(data); - return ByteString.copyFrom(md.digest()); - }; } public static ByteString int2ByteString(int n) { return UnsafeByteOperations.unsafeWrap(IntegerCodec.get().toByteArray(n)); } - private static Function newChecksumByteBufferFunction( - Supplier constructor) { - final ChecksumByteBuffer algorithm = constructor.get(); - return data -> { - algorithm.reset(); - algorithm.update(data); - return int2ByteString((int)algorithm.getValue()); + /** + * Streaming checksum strategy: feed multiple ByteBuffer slices via + * {@link #update}, then read the result via {@link #finish}, then + * {@link #reset} to start a new window. Used by both the no-cache and + * cache compute paths to avoid the {@code byte[bytesPerChecksum]} + * allocation that {@code ChunkBuffer.iterate} performs whenever a + * checksum window straddles multiple underlying buffers - the + * {@link ChecksumByteBuffer#update(ByteBuffer)} and + * {@link MessageDigest#update(ByteBuffer)} contracts both define + * incremental updates as byte-equivalent to a single update over the + * concatenation. + */ + interface StreamingChecksum { + void reset(); + + void update(ByteBuffer slice); + + ByteString finish(); + } + + private static StreamingChecksum streamingCrc( + Supplier ctor) { + final ChecksumByteBuffer cb = ctor.get(); + return new StreamingChecksum() { + @Override + public void reset() { + cb.reset(); + } + + @Override + public void update(ByteBuffer slice) { + cb.update(slice); + } + + @Override + public ByteString finish() { + return int2ByteString((int) cb.getValue()); + } + }; + } + + private static StreamingChecksum streamingDigest(String algorithm) { + final MessageDigest md = newMessageDigest(algorithm); + return new StreamingChecksum() { + @Override + public void reset() { + md.reset(); + } + + @Override + public void update(ByteBuffer slice) { + md.update(slice); + } + + @Override + public ByteString finish() { + return UnsafeByteOperations.unsafeWrap(md.digest()); + } }; } /** The algorithms for {@link ChecksumType}. */ enum Algorithm { - NONE(() -> data -> ByteString.EMPTY), - CRC32(() -> - newChecksumByteBufferFunction(ChecksumByteBufferFactory::crc32Impl)), - CRC32C(() -> - newChecksumByteBufferFunction(ChecksumByteBufferFactory::crc32CImpl)), - SHA256(() -> newMessageDigestFunction("SHA-256")), - MD5(() -> newMessageDigestFunction("MD5")); - - private final Supplier> constructor; + // NONE is reachable via Algorithm.valueOf(ChecksumType.NONE) only if + // computeChecksum's NONE short-circuit is bypassed; throw to surface + // such a misuse rather than silently producing empty checksums. + NONE(() -> { + throw new UnsupportedOperationException( + "ChecksumType.NONE has no StreamingChecksum"); + }), + CRC32(() -> streamingCrc(ChecksumByteBufferFactory::crc32Impl)), + CRC32C(() -> streamingCrc(ChecksumByteBufferFactory::crc32CImpl)), + SHA256(() -> streamingDigest("SHA-256")), + MD5(() -> streamingDigest("MD5")); + + private final Supplier constructor; static Algorithm valueOf(ChecksumType type) { return valueOf(type.name()); } - Algorithm(Supplier> constructor) { + Algorithm(Supplier constructor) { this.constructor = constructor; } - Function newChecksumFunction() { + StreamingChecksum newStreamingChecksum() { return constructor.get(); } } @@ -218,65 +265,71 @@ public ChecksumData computeChecksum(ChunkBuffer data) return computeChecksum(data, false); } + /** + * @implNote The position of {@code data}'s underlying buffers is not + * advanced by this method - both the no-cache and cache paths slice via + * {@link ByteBuffer#duplicate()}. + */ public ChecksumData computeChecksum(ChunkBuffer data, boolean useCache) throws OzoneChecksumException { if (checksumType == ChecksumType.NONE) { - // Since type is set to NONE, we do not need to compute the checksums return new ChecksumData(checksumType, bytesPerChecksum); } - final Function function; + final StreamingChecksum algo; try { - function = Algorithm.valueOf(checksumType).newChecksumFunction(); + algo = Algorithm.valueOf(checksumType).newStreamingChecksum(); } catch (Exception e) { - throw new OzoneChecksumException("Failed to get the checksum function for " + checksumType, e); + throw new OzoneChecksumException( + "Failed to get the checksum function for " + checksumType, e); } - final List checksumList; - if (checksumCache == null || !useCache) { - // When checksumCache is not enabled: - // Checksum is computed for each bytesPerChecksum number of bytes of data - // starting at offset 0. The last checksum might be computed for the - // remaining data with length less than bytesPerChecksum. - checksumList = new ArrayList<>(); - for (ByteBuffer b : data.iterate(bytesPerChecksum)) { - checksumList.add(computeChecksum(b, function, bytesPerChecksum)); // merge this? - } - } else { - // When checksumCache is enabled: - // We only need to update the last checksum in the cache, then pass it along. - checksumList = checksumCache.computeChecksum(data, function); - } + final List checksumList = (checksumCache == null || !useCache) + ? computeChecksumDirect(data, algo) + : checksumCache.computeChecksum(data, algo, bytesPerChecksum); return new ChecksumData(checksumType, bytesPerChecksum, checksumList); } /** - * Compute checksum using the algorithm for the data upto the max length. - * @param data input data - * @param function the checksum function - * @param maxLength the max length of data - * @return computed checksum ByteString + * Walk {@code data}'s underlying ByteBuffer list, slicing each window of + * {@link #bytesPerChecksum} bytes via {@link ByteBuffer#duplicate()} and + * feeding slices to {@code algo}. No linearization byte[] is allocated + * when a window straddles multiple buffers. */ - protected static ByteString computeChecksum(ByteBuffer data, - Function function, int maxLength) { - final int limit = data.limit(); - try { - final int maxIndex = data.position() + maxLength; - if (limit > maxIndex) { - data.limit(maxIndex); + private List computeChecksumDirect(ChunkBuffer data, + StreamingChecksum algo) { + final int dataLimit = data.limit(); + final List result = new ArrayList<>( + (dataLimit + bytesPerChecksum - 1) / bytesPerChecksum); + int windowRemaining = bytesPerChecksum; + algo.reset(); + + for (ByteBuffer src : data.asByteBufferList()) { + int srcPos = src.position(); + final int srcLim = src.limit(); + while (srcPos < srcLim) { + final int n = Math.min(srcLim - srcPos, windowRemaining); + algo.update(BufferUtils.slice(src, srcPos, n)); + srcPos += n; + windowRemaining -= n; + if (windowRemaining == 0) { + result.add(algo.finish()); + algo.reset(); + windowRemaining = bytesPerChecksum; + } } - return function.apply(data); - } finally { - data.limit(limit); } + if (windowRemaining < bytesPerChecksum) { + // Unaligned trailing window. + result.add(algo.finish()); + } + return result; } public static void verifySingleChecksum(ByteBuffer buffer, int offset, int bytesPerChecksum, ByteString checksum, ChecksumType checksumType) throws OzoneChecksumException { - final ByteBuffer duplicated = buffer.duplicate(); - duplicated.position(offset).limit(offset + bytesPerChecksum); final ChecksumData cd = new ChecksumData(checksumType, bytesPerChecksum, Collections.singletonList(checksum)); - verifyChecksum(duplicated, cd, 0); + verifyChecksum(BufferUtils.slice(buffer, offset, bytesPerChecksum), cd, 0); } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumCache.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumCache.java index 3b96b35eba6f..4732767b81b8 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumCache.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumCache.java @@ -20,7 +20,8 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import java.util.function.Function; +import org.apache.hadoop.ozone.common.Checksum.StreamingChecksum; +import org.apache.hadoop.ozone.common.utils.BufferUtils; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,58 +66,102 @@ public List getChecksums() { return checksums; } - public List computeChecksum(ChunkBuffer data, Function function) { - // Indicates how much data the current chunk buffer holds + /** + * Recompute checksums for the windows that have changed since the last + * call: bytes {@code [ciStart * bytesPerChecksum, currChunkLength)} where + * {@code ciStart = prevChunkLength / bytesPerChecksum} (the index of the + * first window whose result may have changed - either the previously- + * partial last window now has more bytes, or new full windows have been + * appended). + * + *

Walks {@code data}'s underlying buffer list directly (no + * {@code iterate()} byte[] linearization) and feeds slices to {@code algo} + * incrementally; the cached prefix is skipped via index arithmetic so + * those bytes are never re-fed. + */ + public List computeChecksum(ChunkBuffer data, + StreamingChecksum algo, int chksumSize) { + if (chksumSize != bytesPerChecksum) { + throw new IllegalArgumentException("bytesPerChecksum mismatch: cache=" + + bytesPerChecksum + " call=" + chksumSize); + } final int currChunkLength = data.limit(); if (currChunkLength == prevChunkLength) { LOG.debug("ChunkBuffer data limit same as last time ({}). No new checksums need to be computed", prevChunkLength); return checksums; } - - // Sanity check if (currChunkLength < prevChunkLength) { - // If currChunkLength <= lastChunkLength, it indicates a bug that needs to be addressed. - // It means BOS has not properly clear()ed the cache when a new chunk is started in that code path. - throw new IllegalArgumentException("ChunkBuffer data limit (" + currChunkLength + ")" + - " must not be smaller than last time (" + prevChunkLength + ")"); + // Indicates a bug: BOS did not clear() the cache before starting a new chunk. + throw new IllegalArgumentException("ChunkBuffer data limit (" + currChunkLength + ")" + + " must not be smaller than last time (" + prevChunkLength + ")"); } - // One or more checksums need to be computed - - // Start of the checksum index that need to be (re)computed + // Index of the first window that needs (re)computing. final int ciStart = prevChunkLength / bytesPerChecksum; - final int ciEnd = currChunkLength / bytesPerChecksum + (currChunkLength % bytesPerChecksum == 0 ? 0 : 1); - int i = 0; - for (ByteBuffer b : data.iterate(bytesPerChecksum)) { - if (i < ciStart) { - i++; + final int ciEnd = currChunkLength / bytesPerChecksum + + (currChunkLength % bytesPerChecksum == 0 ? 0 : 1); + // Bytes to skip (the cached full windows preceding ciStart). + long bytesToSkip = (long) ciStart * bytesPerChecksum; + + int i = ciStart; + int windowRemaining = bytesPerChecksum; + algo.reset(); + long position = 0; + + for (ByteBuffer src : data.asByteBufferList()) { + int srcPos = src.position(); + final int srcLim = src.limit(); + final int srcLen = srcLim - srcPos; + + // Fast-forward through buffers that lie entirely within the cached + // prefix. + if (position + srcLen <= bytesToSkip) { + position += srcLen; continue; } - - // variable i can either point to: - // 1. the last element in the list -- in which case the checksum needs to be updated - // 2. one after the last element -- in which case a new checksum needs to be added - assert i == checksums.size() - 1 || i == checksums.size(); - - // TODO: Furthermore for CRC32/CRC32C, it can be even more efficient by updating the last checksum byte-by-byte. - final ByteString checksum = Checksum.computeChecksum(b, function, bytesPerChecksum); - if (i == checksums.size()) { - checksums.add(checksum); - } else { - checksums.set(i, checksum); + // First buffer that crosses into the not-yet-cached region: advance + // srcPos to the boundary. + if (position < bytesToSkip) { + srcPos += (int) (bytesToSkip - position); + position = bytesToSkip; } - i++; + while (srcPos < srcLim) { + final int n = Math.min(srcLim - srcPos, windowRemaining); + algo.update(BufferUtils.slice(src, srcPos, n)); + srcPos += n; + position += n; + windowRemaining -= n; + if (windowRemaining == 0) { + storeChecksum(i++, algo.finish()); + algo.reset(); + windowRemaining = bytesPerChecksum; + } + } + } + if (windowRemaining < bytesPerChecksum) { + // Unaligned trailing window. + storeChecksum(i++, algo.finish()); } - // Sanity check if (i != ciEnd) { throw new IllegalStateException("ChecksumCache: Checksum index end does not match expectation"); } - // Update last written index prevChunkLength = currChunkLength; return checksums; } + + private void storeChecksum(int i, ByteString cs) { + // i can either point to the last cached element (recompute - the + // previously-partial trailing window) or one past it (append a new + // checksum for newly-arrived bytes). + assert i == checksums.size() - 1 || i == checksums.size(); + if (i == checksums.size()) { + checksums.add(cs); + } else { + checksums.set(i, cs); + } + } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java index 794504b1ef3d..8c1b6d377f0d 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java @@ -68,6 +68,18 @@ public static ByteBuffer[] assignByteBuffers(long totalLen, return dataBuffers; } + /** + * Return a non-copying {@link ByteBuffer#duplicate()} of {@code src} that + * covers exactly {@code [position, position + length)}. Read-only-ness + * and direct/heap kind are inherited from {@code src}. + */ + public static ByteBuffer slice(ByteBuffer src, int position, int length) { + final ByteBuffer slice = src.duplicate(); + slice.position(position); + slice.limit(position + length); + return slice; + } + /** * Return a read only ByteBuffer list for the input ByteStrings list. */ diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumCache.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumCache.java index 0b7e9a7b198c..97d3d5332ff6 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumCache.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumCache.java @@ -19,9 +19,9 @@ import java.nio.ByteBuffer; import java.util.List; -import java.util.function.Function; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.ozone.common.Checksum.Algorithm; +import org.apache.hadoop.ozone.common.Checksum.StreamingChecksum; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.params.ParameterizedTest; @@ -33,7 +33,7 @@ class TestChecksumCache { @ParameterizedTest - @EnumSource(ChecksumType.class) + @EnumSource(value = ChecksumType.class, names = {"CRC32", "CRC32C", "SHA256", "MD5"}) void testComputeChecksum(ChecksumType checksumType) throws Exception { final int bytesPerChecksum = 16; ChecksumCache checksumCache = new ChecksumCache(bytesPerChecksum); @@ -45,7 +45,7 @@ void testComputeChecksum(ChecksumType checksumType) throws Exception { byteArray[i] = (byte) (i % 128); } - final Function function = Algorithm.valueOf(checksumType).newChecksumFunction(); + final StreamingChecksum algo = Algorithm.valueOf(checksumType).newStreamingChecksum(); int iEnd = size / bytesPerChecksum + (size % bytesPerChecksum == 0 ? 0 : 1); List lastRes = null; @@ -54,9 +54,9 @@ void testComputeChecksum(ChecksumType checksumType) throws Exception { ByteBuffer byteBuffer = ByteBuffer.wrap(byteArray, 0, byteBufferLength); try (ChunkBuffer chunkBuffer = ChunkBuffer.wrap(byteBuffer.asReadOnlyBuffer())) { - List res = checksumCache.computeChecksum(chunkBuffer, function); - System.out.println(res); - // Verify that every entry in the res list except the last one is the same as the one in lastRes list + List res = checksumCache.computeChecksum(chunkBuffer, algo, bytesPerChecksum); + // Every entry except the last must be unchanged from the prior iteration + // (those windows are fully cached and never re-computed). if (i > 0) { for (int j = 0; j < res.size() - 1; j++) { Assertions.assertEquals(lastRes.get(j), res.get(j)); @@ -66,7 +66,6 @@ void testComputeChecksum(ChecksumType checksumType) throws Exception { } } - // Sanity check checksumCache.clear(); } } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumMultiBuffer.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumMultiBuffer.java new file mode 100644 index 000000000000..b2ca28efce33 --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumMultiBuffer.java @@ -0,0 +1,97 @@ +/* + * 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 org.apache.hadoop.ozone.common; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +/** + * A checksum window that straddles multiple underlying buffers must + * produce the same checksum bytes as a single contiguous buffer over the + * same data. + */ +class TestChecksumMultiBuffer { + + private static final int CHUNK_SIZE = 4 * 1024 * 1024; + private static final int BYTES_PER_CHECKSUM = 1024 * 1024; + + @ParameterizedTest + @EnumSource(value = ChecksumType.class, names = {"CRC32", "CRC32C", + "SHA256", "MD5"}) + void splitBufferProducesSameChecksumAsSingleBuffer(ChecksumType type) + throws Exception { + byte[] data = new byte[CHUNK_SIZE]; + new Random(0xCAFEBABEL).nextBytes(data); + + Checksum checksum = new Checksum(type, BYTES_PER_CHECKSUM); + + // Single contiguous buffer. + ChecksumData single = checksum.computeChecksum( + ChunkBuffer.wrap(ByteBuffer.wrap(data.clone()))); + + // Split into 16 pieces of 256KB - each 1MB checksum window straddles + // exactly 4 underlying buffers. + int piece = 256 * 1024; + List pieces = new ArrayList<>(); + for (int off = 0; off < CHUNK_SIZE; off += piece) { + pieces.add(ByteBuffer.wrap(data, off, piece).slice()); + } + ChecksumData split = checksum.computeChecksum(ChunkBuffer.wrap(pieces)); + + assertEquals(single.getChecksums().size(), split.getChecksums().size(), + "checksum count must match"); + assertEquals(single.getChecksums(), split.getChecksums(), + "single-buffer and split-buffer must produce identical checksums " + + "for " + type); + } + + @ParameterizedTest + @EnumSource(value = ChecksumType.class, names = {"CRC32", "CRC32C", + "SHA256", "MD5"}) + void unalignedTrailingWindowIsHandled(ChecksumType type) throws Exception { + // Chunk size deliberately not a multiple of bytesPerChecksum, plus + // unaligned buffer splits, to exercise the trailing-partial-window + // path in computeChecksumDirect. + int total = BYTES_PER_CHECKSUM * 3 + 12345; + byte[] data = new byte[total]; + new Random(0xDEADBEEFL).nextBytes(data); + + Checksum checksum = new Checksum(type, BYTES_PER_CHECKSUM); + + ChecksumData single = checksum.computeChecksum( + ChunkBuffer.wrap(ByteBuffer.wrap(data.clone()))); + + int piece = 333 * 1024; // intentionally awkward + List pieces = new ArrayList<>(); + for (int off = 0; off < total; off += piece) { + int len = Math.min(piece, total - off); + pieces.add(ByteBuffer.wrap(data, off, len).slice()); + } + ChecksumData split = checksum.computeChecksum(ChunkBuffer.wrap(pieces)); + + assertEquals(single.getChecksums(), split.getChecksums(), + "unaligned split must match single-buffer for " + type); + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index af7242541abd..1e68a5930c37 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -1018,7 +1018,9 @@ private void validateChunkChecksumData(ChunkBufferToByteString data, ChunkInfo i final ChunkBuffer b = (ChunkBuffer)data; Checksum.verifyChecksum(b.duplicate(b.position(), b.limit()), info.getChecksumData(), 0); } else { - Checksum.verifyChecksum(data.toByteString(byteBufferToByteString).asReadOnlyByteBuffer(), + // Skip concatenating into one ByteString - that would materialize + // a chunk-sized copy on the hot write path. + Checksum.verifyChecksum(data.toByteStringList(byteBufferToByteString), info.getChecksumData(), 0); } } catch (OzoneChecksumException ex) {