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 @@ -141,7 +141,11 @@ public final class ScmConfigKeys {
public static final String OZONE_CHUNK_READ_BUFFER_DEFAULT_SIZE_KEY =
"ozone.chunk.read.buffer.default.size";
public static final String OZONE_CHUNK_READ_BUFFER_DEFAULT_SIZE_DEFAULT =
"64KB";
"1MB";
public static final String OZONE_CHUNK_READ_MAPPED_BUFFER_THRESHOLD_KEY =
"ozone.chunk.read.mapped.buffer.threshold";
public static final String OZONE_CHUNK_READ_MAPPED_BUFFER_THRESHOLD_DEFAULT =
"32KB";

public static final String OZONE_SCM_CONTAINER_LAYOUT_KEY =
"ozone.scm.container.layout";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel;
import java.util.List;
import java.util.Objects;
import java.util.function.Function;
import java.util.function.Supplier;

Expand Down Expand Up @@ -58,6 +59,10 @@ static ChunkBuffer wrap(ByteBuffer buffer) {

/** Wrap the given list of {@link ByteBuffer}s as a {@link ChunkBuffer}. */
static ChunkBuffer wrap(List<ByteBuffer> buffers) {
Objects.requireNonNull(buffers, "buffers == null");
if (buffers.size() == 1) {
return wrap(buffers.get(0));
}
return new ChunkBufferImplWithByteBufferList(buffers);
}

Expand Down Expand Up @@ -91,8 +96,7 @@ default ChunkBuffer put(byte[] b) {

/** Similar to {@link ByteBuffer#put(byte[])}. */
default ChunkBuffer put(byte b) {
byte[] buf = new byte[1];
buf[0] = (byte) b;
final byte[] buf = {b};
return put(buf, 0, 1);
}

Expand All @@ -116,7 +120,6 @@ default ChunkBuffer put(ByteString b) {

/**
* Iterate the buffer from the current position to the current limit.
*
* Upon the iteration complete,
* the buffer's position will be equal to its limit.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import java.nio.channels.GatheringByteChannel;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.function.Function;

/**
Expand All @@ -50,8 +51,7 @@ public class ChunkBufferImplWithByteBufferList implements ChunkBuffer {
private int currentIndex;

ChunkBufferImplWithByteBufferList(List<ByteBuffer> buffers) {
Preconditions.checkArgument(buffers != null, "buffer == null");

Objects.requireNonNull(buffers, "buffers == null");
this.buffers = !buffers.isEmpty() ? ImmutableList.copyOf(buffers) :
EMPTY_BUFFER;
this.limit = buffers.stream().mapToInt(ByteBuffer::limit).sum();
Expand Down
10 changes: 9 additions & 1 deletion hadoop-hdds/common/src/main/resources/ozone-default.xml
Original file line number Diff line number Diff line change
Expand Up @@ -829,7 +829,7 @@
</property>
<property>
<name>ozone.chunk.read.buffer.default.size</name>
<value>64KB</value>
<value>1MB</value>
<tag>OZONE, SCM, CONTAINER, PERFORMANCE</tag>
<description>
The default read buffer size during read chunk operations when checksum
Expand All @@ -840,6 +840,14 @@
(ozone.client.bytes.per.checksum) corresponding to the chunk.
</description>
</property>
<property>
<name>ozone.chunk.read.mapped.buffer.threshold</name>
<value>32KB</value>
<tag>OZONE, SCM, CONTAINER, PERFORMANCE</tag>
<description>
The default read threshold to use memory mapped buffers.
</description>
</property>
<property>
<name>ozone.scm.container.layout</name>
<value>FILE_PER_BLOCK</value>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ public class TestChunkBufferImplWithByteBufferList {
@Test
public void rejectsNullList() {
List<ByteBuffer> list = null;
assertThrows(IllegalArgumentException.class, () -> ChunkBuffer.wrap(list));
assertThrows(NullPointerException.class, () -> ChunkBuffer.wrap(list));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -285,6 +285,17 @@ default long getTimeDuration(String name, String defaultValue,
}
}

default int getBufferSize(String name, String defaultValue) {
final double size = getStorageSize(name, defaultValue, StorageUnit.BYTES);
if (size <= 0) {
throw new IllegalArgumentException(name + " <= 0");
} else if (size > Integer.MAX_VALUE) {
throw new IllegalArgumentException(
name + " > Integer.MAX_VALUE = " + Integer.MAX_VALUE);
}
return (int) size;
}

default double getStorageSize(String name, String defaultValue,
StorageUnit targetUnit) {
String vString = get(name);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.io.InterruptedIOException;
import java.io.UncheckedIOException;
import java.nio.ByteBuffer;
import java.nio.MappedByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.channels.FileLock;
import java.nio.file.NoSuchFileException;
Expand All @@ -32,8 +33,11 @@
import java.nio.file.StandardOpenOption;
import java.nio.file.attribute.FileAttribute;
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Supplier;
Expand Down Expand Up @@ -61,7 +65,7 @@
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST;
import static org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil.onFailure;

import org.apache.ratis.util.function.CheckedConsumer;
import org.apache.ratis.util.function.CheckedFunction;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -184,26 +188,24 @@ private static long writeDataToChannel(FileChannel channel, ChunkBuffer data,
}

public static ChunkBuffer readData(long len, int bufferCapacity,
CheckedConsumer<ByteBuffer[], StorageContainerException> readMethod)
File file, long off, HddsVolume volume, int readMappedBufferThreshold)
throws StorageContainerException {
if (len > readMappedBufferThreshold) {
return readData(file, bufferCapacity, off, len, volume);
} else if (len == 0) {
return ChunkBuffer.wrap(Collections.emptyList());
}

final ByteBuffer[] buffers = BufferUtils.assignByteBuffers(len,
bufferCapacity);
readMethod.accept(buffers);
readData(file, off, len, c -> c.position(off).read(buffers), volume);
Arrays.stream(buffers).forEach(ByteBuffer::flip);
return ChunkBuffer.wrap(Arrays.asList(buffers));
}

/**
* Reads data from an existing chunk file into a list of ByteBuffers.
*
* @param file file where data lives
* @param buffers
* @param offset
* @param len
* @param volume for statistics and checker
*/
public static void readData(File file, ByteBuffer[] buffers,
long offset, long len, HddsVolume volume)
throws StorageContainerException {
private static void readData(File file, long offset, long len,
CheckedFunction<FileChannel, Long, IOException> readMethod,
HddsVolume volume) throws StorageContainerException {

final Path path = file.toPath();
final long startTime = Time.monotonicNow();
Expand All @@ -213,8 +215,7 @@ public static void readData(File file, ByteBuffer[] buffers,
bytesRead = processFileExclusively(path, () -> {
try (FileChannel channel = open(path, READ_OPTIONS, NO_ATTRIBUTES);
FileLock ignored = channel.lock(offset, len, true)) {

return channel.position(offset).read(buffers);
return readMethod.apply(channel);
} catch (IOException e) {
onFailure(volume);
throw new UncheckedIOException(e);
Expand All @@ -239,10 +240,37 @@ public static void readData(File file, ByteBuffer[] buffers,
bytesRead, offset, file);

validateReadSize(len, bytesRead);
}

for (ByteBuffer buf : buffers) {
buf.flip();
}
/**
* Read data from the given file using
* {@link FileChannel#map(FileChannel.MapMode, long, long)},
* whose javadoc recommends that it is generally only worth mapping
* relatively large files (larger than a few tens of kilobytes)
* into memory from the standpoint of performance.
*
* @return a list of {@link MappedByteBuffer} containing the data.
*/
private static ChunkBuffer readData(File file, int chunkSize,
long offset, long length, HddsVolume volume)
throws StorageContainerException {

final List<ByteBuffer> buffers = new ArrayList<>(
Math.toIntExact((length - 1) / chunkSize) + 1);
readData(file, offset, length, channel -> {
long readLen = 0;
while (readLen < length) {
final int n = Math.toIntExact(Math.min(length - readLen, chunkSize));
final ByteBuffer mapped = channel.map(
FileChannel.MapMode.READ_ONLY, offset + readLen, n);
LOG.debug("mapped: offset={}, readLen={}, n={}, {}",
offset, readLen, n, mapped.getClass());
readLen += mapped.remaining();
buffers.add(mapped);
}
return readLen;
}, volume);
return ChunkBuffer.wrap(buffers);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@

import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.StorageUnit;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdds.utils.db.BatchOperation;
Expand Down Expand Up @@ -60,6 +59,7 @@ public class BlockManagerImpl implements BlockManager {

// Default Read Buffer capacity when Checksum is not present
private final int defaultReadBufferCapacity;
private final int readMappedBufferThreshold;

/**
* Constructs a Block Manager.
Expand All @@ -69,19 +69,12 @@ public class BlockManagerImpl implements BlockManager {
public BlockManagerImpl(ConfigurationSource conf) {
Preconditions.checkNotNull(conf, "Config cannot be null");
this.config = conf;
final double size = config.getStorageSize(
this.defaultReadBufferCapacity = config.getBufferSize(
ScmConfigKeys.OZONE_CHUNK_READ_BUFFER_DEFAULT_SIZE_KEY,
ScmConfigKeys.OZONE_CHUNK_READ_BUFFER_DEFAULT_SIZE_DEFAULT,
StorageUnit.BYTES);
if (size <= 0) {
throw new IllegalArgumentException(
ScmConfigKeys.OZONE_CHUNK_READ_BUFFER_DEFAULT_SIZE_KEY + " <= 0");
} else if (size > Integer.MAX_VALUE) {
throw new IllegalArgumentException(
ScmConfigKeys.OZONE_CHUNK_READ_BUFFER_DEFAULT_SIZE_KEY
+ " > Integer.MAX_VALUE = " + Integer.MAX_VALUE);
}
this.defaultReadBufferCapacity = (int) size;
ScmConfigKeys.OZONE_CHUNK_READ_BUFFER_DEFAULT_SIZE_DEFAULT);
this.readMappedBufferThreshold = config.getBufferSize(
ScmConfigKeys.OZONE_CHUNK_READ_MAPPED_BUFFER_THRESHOLD_KEY,
ScmConfigKeys.OZONE_CHUNK_READ_MAPPED_BUFFER_THRESHOLD_DEFAULT);
}

@Override
Expand Down Expand Up @@ -267,6 +260,10 @@ public int getDefaultReadBufferCapacity() {
return defaultReadBufferCapacity;
}

public int getReadMappedBufferThreshold() {
return readMappedBufferThreshold;
}

/**
* Deletes an existing block.
* As Deletion is handled by BlockDeletingService,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,6 @@
import static org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.WriteChunkStage.COMMIT_DATA;
import static org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil.onFailure;
import static org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.limitReadSize;
import static org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.readData;
import static org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.validateChunkForOverwrite;
import static org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.verifyChunkFileExists;

Expand All @@ -75,13 +74,16 @@ public class FilePerBlockStrategy implements ChunkManager {
private final boolean doSyncWrite;
private final OpenFiles files = new OpenFiles();
private final int defaultReadBufferCapacity;
private final int readMappedBufferThreshold;
private final VolumeSet volumeSet;

public FilePerBlockStrategy(boolean sync, BlockManager manager,
VolumeSet volSet) {
doSyncWrite = sync;
this.defaultReadBufferCapacity = manager == null ? 0 :
manager.getDefaultReadBufferCapacity();
this.readMappedBufferThreshold = manager == null ? 0
: manager.getReadMappedBufferThreshold();
this.volumeSet = volSet;
}

Expand Down Expand Up @@ -192,8 +194,8 @@ public ChunkBuffer readChunk(Container container, BlockID blockID,
long offset = info.getOffset();
int bufferCapacity = ChunkManager.getBufferCapacityForChunkRead(info,
defaultReadBufferCapacity);
return readData(len, bufferCapacity,
array -> readData(chunkFile, array, offset, len, volume));
return ChunkUtils.readData(len, bufferCapacity, chunkFile, offset, volume,
readMappedBufferThreshold);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,12 @@

import com.google.common.base.Preconditions;

import com.google.common.collect.Lists;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
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.common.utils.BufferUtils;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
Expand All @@ -48,7 +46,6 @@

import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
Expand All @@ -69,6 +66,7 @@ public class FilePerChunkStrategy implements ChunkManager {
private final boolean doSyncWrite;
private final BlockManager blockManager;
private final int defaultReadBufferCapacity;
private final int readMappedBufferThreshold;
private final VolumeSet volumeSet;

public FilePerChunkStrategy(boolean sync, BlockManager manager,
Expand All @@ -77,6 +75,8 @@ public FilePerChunkStrategy(boolean sync, BlockManager manager,
blockManager = manager;
this.defaultReadBufferCapacity = manager == null ? 0 :
manager.getDefaultReadBufferCapacity();
this.readMappedBufferThreshold = manager == null ? 0
: manager.getReadMappedBufferThreshold();
this.volumeSet = volSet;
}

Expand Down Expand Up @@ -233,9 +233,6 @@ public ChunkBuffer readChunk(Container container, BlockID blockID,
int bufferCapacity = ChunkManager.getBufferCapacityForChunkRead(info,
defaultReadBufferCapacity);

ByteBuffer[] dataBuffers = BufferUtils.assignByteBuffers(len,
bufferCapacity);

long chunkFileOffset = 0;
if (info.getOffset() != 0) {
try {
Expand Down Expand Up @@ -267,16 +264,15 @@ public ChunkBuffer readChunk(Container container, BlockID blockID,
if (file.exists()) {
long offset = info.getOffset() - chunkFileOffset;
Preconditions.checkState(offset >= 0);
ChunkUtils.readData(file, dataBuffers, offset, len, volume);
return ChunkBuffer.wrap(Lists.newArrayList(dataBuffers));
return ChunkUtils.readData(len, bufferCapacity, file, offset, volume,
readMappedBufferThreshold);
}
} catch (StorageContainerException ex) {
//UNABLE TO FIND chunk is not a problem as we will try with the
//next possible location
if (ex.getResult() != UNABLE_TO_FIND_CHUNK) {
throw ex;
}
BufferUtils.clearBuffers(dataBuffers);
}
}
throw new StorageContainerException(
Expand Down
Loading