Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand All @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -33,13 +37,46 @@
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.
* Chunks are not written to disk, Reads are returned with zero-filled buffers
*/
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)
Expand Down Expand Up @@ -72,9 +109,9 @@ public ChunkBuffer readChunk(Container container, BlockID blockID,
ChunkInfo info, DispatcherContext dispatcherContext)
throws StorageContainerException {

limitReadSize(info.getLen());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's keeping using limitReadSize(..). It limits the len <= OZONE_SCM_CHUNK_MAX_SIZE.

Then, we may create the backing file using the size OZONE_SCM_CHUNK_MAX_SIZE.

// 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
Expand Down