Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

perf(s3stream/wal): marshal records lazily #464

Merged
merged 2 commits into from
Oct 25, 2023
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
13 changes: 12 additions & 1 deletion s3stream/src/main/java/com/automq/stream/s3/wal/Block.java
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ public interface Block {

/**
* Append a record to this block.
* Cannot be called after {@link #data()} is called.
*
* @param recordSize The size of this record.
* @param recordSupplier The supplier of this record which receives the start offset of this record as the parameter.
Expand All @@ -58,10 +59,20 @@ default boolean isEmpty() {

/**
* The content of this block, which contains multiple records.
* The first call of this method will marshal all records in this block to a ByteBuf. It will be cached for later calls.
* It returns null if this block is empty.
*/
ByteBuf data();

/**
* The size of this block.
*/
long size();

default void release() {
data().release();
ByteBuf data = data();
if (null != data) {
data.release();
}
}
}
24 changes: 21 additions & 3 deletions s3stream/src/main/java/com/automq/stream/s3/wal/BlockImpl.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import java.util.function.Supplier;

public class BlockImpl implements Block {

Expand All @@ -41,13 +42,14 @@ public class BlockImpl implements Block {
* unless the block is empty.
*/
private final long softLimit;
private final CompositeByteBuf data = DirectByteBufAlloc.compositeByteBuffer();
private final List<CompletableFuture<WriteAheadLog.AppendResult.CallbackResult>> futures = new LinkedList<>();
private final List<Supplier<ByteBuf>> records = new LinkedList<>();
/**
* The next offset to write in this block.
* Align to {@link WALUtil#BLOCK_SIZE}
*/
private long nextOffset = 0;
private CompositeByteBuf data = null;

/**
* Create a block.
Expand All @@ -69,6 +71,7 @@ public long startOffset() {
*/
@Override
public long addRecord(long recordSize, Function<Long, ByteBuf> recordSupplier, CompletableFuture<WriteAheadLog.AppendResult.CallbackResult> future) {
assert data == null;
long requiredCapacity = nextOffset + recordSize;
if (requiredCapacity > maxSize) {
return -1;
Expand All @@ -79,8 +82,7 @@ public long addRecord(long recordSize, Function<Long, ByteBuf> recordSupplier, C
}

long recordOffset = startOffset + nextOffset;
ByteBuf record = recordSupplier.apply(recordOffset);
data.addComponent(true, record);
records.add(() -> recordSupplier.apply(recordOffset));
nextOffset += recordSize;
futures.add(future);

Expand All @@ -94,6 +96,22 @@ public List<CompletableFuture<WriteAheadLog.AppendResult.CallbackResult>> future

@Override
public ByteBuf data() {
if (null != data) {
return data;
}
if (records.isEmpty()) {
return null;
}

data = DirectByteBufAlloc.compositeByteBuffer();
for (Supplier<ByteBuf> record : records) {
data.addComponent(true, record.get());
}
return data;
}

@Override
public long size() {
return nextOffset;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ private synchronized void flushWALHeader() throws IOException {
/**
* Try to read a record at the given offset.
* The returned record should be released by the caller.
*
* @throws ReadRecordException if the record is not found or the record is corrupted
*/
private ByteBuf readRecord(long recordSectionCapacity, long recoverStartOffset) throws ReadRecordException {
Expand Down Expand Up @@ -440,19 +441,18 @@ public AppendResult append0(ByteBuf body, int crc) throws OverCapacityException
checkReadyToServe();

final long recordSize = RECORD_HEADER_SIZE + body.readableBytes();
final int recordBodyCRC = 0 == crc ? WALUtil.crc32(body) : crc;
final CompletableFuture<AppendResult.CallbackResult> appendResultFuture = new CompletableFuture<>();
long expectedWriteOffset;

Lock lock = slidingWindowService.getBlockLock();
lock.lock();
try {
Block block = slidingWindowService.getCurrentBlockLocked();
expectedWriteOffset = block.addRecord(recordSize, (offset) -> record(body, recordBodyCRC, offset), appendResultFuture);
expectedWriteOffset = block.addRecord(recordSize, (offset) -> record(body, crc, offset), appendResultFuture);
if (expectedWriteOffset < 0) {
// this block is full, create a new one
block = slidingWindowService.sealAndNewBlockLocked(block, recordSize, walHeaderCoreData.getFlushedTrimOffset(), walHeaderCoreData.getCapacity() - WAL_HEADER_TOTAL_CAPACITY);
expectedWriteOffset = block.addRecord(recordSize, (offset) -> record(body, recordBodyCRC, offset), appendResultFuture);
expectedWriteOffset = block.addRecord(recordSize, (offset) -> record(body, crc, offset), appendResultFuture);
}
} finally {
lock.unlock();
Expand All @@ -478,6 +478,7 @@ private ByteBuf recordHeader(ByteBuf body, int crc, long start) {

private ByteBuf record(ByteBuf body, int crc, long start) {
CompositeByteBuf record = DirectByteBufAlloc.compositeByteBuffer();
crc = 0 == crc ? WALUtil.crc32(body) : crc;
record.addComponents(true, recordHeader(body, crc, start), body);
return record;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ private void setCurrentBlockLocked(Block block) {
* Get the start offset of the next block.
*/
private long nextBlockStartOffset(Block block) {
return block.startOffset() + WALUtil.alignLargeByBlockSize(block.data().readableBytes());
return block.startOffset() + WALUtil.alignLargeByBlockSize(block.size());
}

/**
Expand Down Expand Up @@ -301,7 +301,7 @@ private void writeBlockData(Block block) throws IOException {
}

private boolean makeWriteOffsetMatchWindow(final Block block) throws IOException {
long newWindowEndOffset = block.startOffset() + block.data().readableBytes();
long newWindowEndOffset = block.startOffset() + block.size();
// align to block size
newWindowEndOffset = WALUtil.alignLargeByBlockSize(newWindowEndOffset);
long windowStartOffset = windowCoreData.getWindowStartOffset();
Expand Down
Loading