-
Notifications
You must be signed in to change notification settings - Fork 231
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
perf(s3stream/wal): reuse the
ByteBuf
for record headers (#1877)
* refactor: manage the record headers' lifecycle in `Block` Signed-off-by: Ning Yu <[email protected]> * perf(s3stream/wal): reuse the `ByteBuf` for record headers Signed-off-by: Ning Yu <[email protected]> * perf: remove the max size limit Signed-off-by: Ning Yu <[email protected]> * test: test `FixedSizeByteBufPool` Signed-off-by: Ning Yu <[email protected]> * revert: "perf: remove the max size limit" This reverts commit ed63112. * feat: use a separate `poolSize` to limit the size of the pool Signed-off-by: Ning Yu <[email protected]> --------- Signed-off-by: Ning Yu <[email protected]>
- Loading branch information
1 parent
70108bc
commit 962fad3
Showing
9 changed files
with
300 additions
and
42 deletions.
There are no files selected for viewing
79 changes: 79 additions & 0 deletions
79
s3stream/src/main/java/com/automq/stream/FixedSizeByteBufPool.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,79 @@ | ||
/* | ||
* Copyright 2024, AutoMQ HK Limited. | ||
* | ||
* The use of this file is governed by the Business Source License, | ||
* as detailed in the file "/LICENSE.S3Stream" included in this repository. | ||
* | ||
* As of the Change Date specified in that file, in accordance with | ||
* the Business Source License, use of this software will be governed | ||
* by the Apache License, Version 2.0 | ||
*/ | ||
|
||
package com.automq.stream; | ||
|
||
import com.automq.stream.s3.ByteBufAlloc; | ||
import io.netty.buffer.ByteBuf; | ||
import java.util.Queue; | ||
import java.util.concurrent.ConcurrentLinkedQueue; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
/** | ||
* A pool of fixed-size {@link ByteBuf}. | ||
*/ | ||
public class FixedSizeByteBufPool { | ||
|
||
/** | ||
* The size of the {@link ByteBuf} in this pool. | ||
*/ | ||
private final int bufferSize; | ||
/** | ||
* The max size of the pool. | ||
* It is possible that the pool size exceeds this limit in some rare cases. | ||
*/ | ||
private final int maxPoolSize; | ||
private final Queue<ByteBuf> pool = new ConcurrentLinkedQueue<>(); | ||
/** | ||
* The current size of the pool. | ||
* We use an {@link AtomicInteger} rather than {@link Queue#size()} to avoid the cost of traversing the queue. | ||
*/ | ||
private final AtomicInteger poolSize = new AtomicInteger(0); | ||
|
||
public FixedSizeByteBufPool(int bufferSize, int maxPoolSize) { | ||
this.bufferSize = bufferSize; | ||
this.maxPoolSize = maxPoolSize; | ||
} | ||
|
||
/** | ||
* Get a {@link ByteBuf} from the pool. | ||
* If the pool is empty, a new {@link ByteBuf} will be allocated. | ||
*/ | ||
public ByteBuf get() { | ||
ByteBuf buffer = pool.poll(); | ||
if (buffer == null) { | ||
return allocate(); | ||
} | ||
poolSize.decrementAndGet(); | ||
return buffer; | ||
} | ||
|
||
private ByteBuf allocate() { | ||
return ByteBufAlloc.byteBuffer(bufferSize); | ||
} | ||
|
||
/** | ||
* Release a {@link ByteBuf} to the pool. | ||
* Note: the buffer MUST be gotten from this pool. | ||
*/ | ||
public void release(ByteBuf buffer) { | ||
assert buffer.capacity() == bufferSize; | ||
|
||
if (poolSize.get() >= maxPoolSize) { | ||
buffer.release(); | ||
return; | ||
} | ||
|
||
buffer.clear(); | ||
pool.offer(buffer); | ||
poolSize.incrementAndGet(); | ||
} | ||
} |
33 changes: 33 additions & 0 deletions
33
s3stream/src/main/java/com/automq/stream/s3/wal/common/Record.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,33 @@ | ||
/* | ||
* Copyright 2024, AutoMQ HK Limited. | ||
* | ||
* The use of this file is governed by the Business Source License, | ||
* as detailed in the file "/LICENSE.S3Stream" included in this repository. | ||
* | ||
* As of the Change Date specified in that file, in accordance with | ||
* the Business Source License, use of this software will be governed | ||
* by the Apache License, Version 2.0 | ||
*/ | ||
|
||
package com.automq.stream.s3.wal.common; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
|
||
public class Record { | ||
|
||
private final ByteBuf header; | ||
private final ByteBuf body; | ||
|
||
public Record(ByteBuf header, ByteBuf body) { | ||
this.header = header; | ||
this.body = body; | ||
} | ||
|
||
public ByteBuf header() { | ||
return header; | ||
} | ||
|
||
public ByteBuf body() { | ||
return body; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.