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

fix(s3stream): evict log cache by capacity #637

Merged
merged 1 commit into from
Nov 15, 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
3 changes: 2 additions & 1 deletion s3stream/src/main/java/com/automq/stream/s3/S3Storage.java
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ public S3Storage(Config config, WriteAheadLog deltaWAL, StreamManager streamMana
this.maxDeltaWALCacheSize = config.walCacheSize();
this.deltaWAL = deltaWAL;
this.blockCache = blockCache;
this.deltaWALCache = new LogCache(config.walUploadThreshold(), config.maxStreamNumPerStreamSetObject());
this.deltaWALCache = new LogCache(config.walCacheSize(), config.walUploadThreshold(), config.maxStreamNumPerStreamSetObject());
DirectByteBufAlloc.registerOOMHandlers(new LogCacheEvictOOMHandler());
this.streamManager = streamManager;
this.objectManager = objectManager;
Expand Down Expand Up @@ -235,6 +235,7 @@ public CompletableFuture<Void> append(StreamRecordBatch streamRecord) {

/**
* Append record to WAL.
*
* @return backoff status.
*/
public boolean append0(WalWriteRequest request, boolean fromBackoff) {
Expand Down
7 changes: 7 additions & 0 deletions s3stream/src/main/java/com/automq/stream/s3/S3Stream.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import com.automq.stream.utils.FutureUtil;
import io.netty.buffer.Unpooled;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -217,6 +218,12 @@ private CompletableFuture<FetchResult> fetch0(long startOffset, long endOffset,
String.format("fetch range[%s, %s) is out of stream bound [%s, %s)", startOffset, endOffset, startOffset(), confirmOffset)
));
}
if (startOffset > endOffset) {
return FutureUtil.failedFuture(new IllegalArgumentException(String.format("fetch startOffset %s is greater than endOffset %s", startOffset, endOffset)));
}
if (startOffset == endOffset) {
return CompletableFuture.completedFuture(new DefaultFetchResult(Collections.emptyList(), CacheAccessType.DELTA_WAL_CACHE_HIT));
}
return storage.read(streamId, startOffset, endOffset, maxBytes).thenApply(dataBlock -> {
List<StreamRecordBatch> records = dataBlock.getRecords();
LOGGER.trace("{} stream fetch, startOffset: {}, endOffset: {}, maxBytes: {}, records: {}", logIdent, startOffset, endOffset, maxBytes, records.size());
Expand Down
16 changes: 9 additions & 7 deletions s3stream/src/main/java/com/automq/stream/s3/cache/LogCache.java
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ public class LogCache {
private static final int DEFAULT_MAX_BLOCK_STREAM_COUNT = 10000;
private static final Consumer<LogCacheBlock> DEFAULT_BLOCK_FREE_LISTENER = block -> {
};
private final long capacity;
private final long cacheBlockMaxSize;
private final int maxCacheBlockStreamCount;
private final List<LogCacheBlock> blocks = new ArrayList<>();
Expand All @@ -50,20 +51,21 @@ public class LogCache {
private final AtomicLong size = new AtomicLong();
private final Consumer<LogCacheBlock> blockFreeListener;

public LogCache(long cacheBlockMaxSize, int maxCacheBlockStreamCount, Consumer<LogCacheBlock> blockFreeListener) {
public LogCache(long capacity, long cacheBlockMaxSize, int maxCacheBlockStreamCount, Consumer<LogCacheBlock> blockFreeListener) {
this.capacity = capacity;
this.cacheBlockMaxSize = cacheBlockMaxSize;
this.maxCacheBlockStreamCount = maxCacheBlockStreamCount;
this.activeBlock = new LogCacheBlock(cacheBlockMaxSize, maxCacheBlockStreamCount);
this.blocks.add(activeBlock);
this.blockFreeListener = blockFreeListener;
}

public LogCache(long cacheBlockMaxSize) {
this(cacheBlockMaxSize, DEFAULT_MAX_BLOCK_STREAM_COUNT, DEFAULT_BLOCK_FREE_LISTENER);
public LogCache(long capacity, long cacheBlockMaxSize) {
this(capacity, cacheBlockMaxSize, DEFAULT_MAX_BLOCK_STREAM_COUNT, DEFAULT_BLOCK_FREE_LISTENER);
}

public LogCache(long cacheBlockMaxSize, int maxCacheBlockStreamCount) {
this(cacheBlockMaxSize, maxCacheBlockStreamCount, DEFAULT_BLOCK_FREE_LISTENER);
public LogCache(long capacity, long cacheBlockMaxSize, int maxCacheBlockStreamCount) {
this(capacity, cacheBlockMaxSize, maxCacheBlockStreamCount, DEFAULT_BLOCK_FREE_LISTENER);
}

public boolean put(StreamRecordBatch recordBatch) {
Expand Down Expand Up @@ -186,11 +188,11 @@ public void markFree(LogCacheBlock block) {
}

private void tryRealFree() {
if (size.get() <= cacheBlockMaxSize * 0.9) {
if (size.get() <= capacity * 0.9) {
return;
}
blocks.removeIf(b -> {
if (size.get() <= cacheBlockMaxSize * 0.9) {
if (size.get() <= capacity * 0.9) {
return false;
}
if (b.free) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ public class LogCacheTest {

@Test
public void testPutGet() {
LogCache logCache = new LogCache(1024 * 1024);
LogCache logCache = new LogCache(1024 * 1024, 1024 * 1024);

logCache.put(new StreamRecordBatch(233L, 0L, 10L, 1, TestUtils.random(20)));
logCache.put(new StreamRecordBatch(233L, 0L, 11L, 2, TestUtils.random(20)));
Expand Down