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

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


The following commit(s) were added to refs/heads/master by this push:
     new 86f9fd6a84f HDDS-9940. Use MappedByteBuffer in ChunkManagerDummyImpl 
(#9649)
86f9fd6a84f is described below

commit 86f9fd6a84f1e94de50ecd075dffe13a25c26aa9
Author: Russole <[email protected]>
AuthorDate: Thu Feb 12 03:09:36 2026 +0800

    HDDS-9940. Use MappedByteBuffer in ChunkManagerDummyImpl (#9649)
---
 .../container/keyvalue/helpers/ChunkUtils.java     |  3 +-
 .../keyvalue/impl/ChunkManagerDummyImpl.java       | 47 +++++++++++++++++++---
 2 files changed, 44 insertions(+), 6 deletions(-)

diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
index cf656572a93..7e9fb0b0d00 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
@@ -492,7 +492,7 @@ private static void checkSize(String of, long expected, 
long actual,
     }
   }
 
-  public static void limitReadSize(long len)
+  public static int limitReadSize(long len)
       throws StorageContainerException {
     if (len > OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE) {
       String err = String.format(
@@ -501,6 +501,7 @@ public static void limitReadSize(long len)
       LOG.error(err);
       throw new StorageContainerException(err, UNSUPPORTED_REQUEST);
     }
+    return (int) len;
   }
 
   public static StorageContainerException wrapInStorageContainerException(
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDummyImpl.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDummyImpl.java
index 3b9289fe61b..a43f433d1e7 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDummyImpl.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDummyImpl.java
@@ -17,12 +17,16 @@
 
 package org.apache.hadoop.ozone.container.keyvalue.impl;
 
-import static 
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.limitReadSize;
-
 import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
 import java.util.Objects;
 import org.apache.hadoop.hdds.client.BlockID;
 import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.ChunkBuffer;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
@@ -33,6 +37,7 @@
 import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
+import org.apache.ratis.util.Preconditions;
 
 /**
  * Implementation of ChunkManager built for running performance tests.
@@ -40,6 +45,38 @@
  */
 public class ChunkManagerDummyImpl implements ChunkManager {
 
+  private final ByteBuffer buffer;
+
+  public ChunkManagerDummyImpl() {
+    this.buffer = newMappedByteBuffer(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE);
+  }
+
+  static ByteBuffer newMappedByteBuffer(int size) {
+    final MappedByteBuffer mapped;
+    try {
+      final Path backingFile =
+          Files.createTempFile("ozone-dummy-chunk-", ".bin");
+      backingFile.toFile().deleteOnExit();
+
+      final byte[] zeros = new byte[4 << 10];
+      try (FileChannel ch = FileChannel.open(
+          backingFile, StandardOpenOption.READ, StandardOpenOption.WRITE)) {
+        for (int written = 0; written < size;) {
+          final int toWrite = Math.min(size - written, zeros.length);
+          written += ch.write(ByteBuffer.wrap(zeros, 0, toWrite));
+        }
+        mapped = ch.map(FileChannel.MapMode.READ_ONLY, 0, size);
+      }
+
+      Preconditions.assertSame(0, mapped.position(), "position");
+      Preconditions.assertSame(size, mapped.remaining(), "remaining");
+      return mapped.asReadOnlyBuffer();
+    } catch (Exception e) {
+      throw new IllegalStateException(
+          "Failed to create MappedByteBuffer for size " + size, e);
+    }
+  }
+
   @Override
   public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
       ChunkBuffer data, DispatcherContext dispatcherContext)
@@ -72,9 +109,9 @@ public ChunkBuffer readChunk(Container container, BlockID 
blockID,
       ChunkInfo info, DispatcherContext dispatcherContext)
       throws StorageContainerException {
 
-    limitReadSize(info.getLen());
-    // stats are handled in ChunkManagerImpl
-    return ChunkBuffer.wrap(ByteBuffer.allocate((int) info.getLen()));
+    final ByteBuffer dup = buffer.duplicate();
+    dup.limit(ChunkUtils.limitReadSize(info.getLen()));
+    return ChunkBuffer.wrap(dup);
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to