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
22 changes: 18 additions & 4 deletions s3stream/src/main/java/com/automq/stream/s3/cache/LogCache.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.trace.context.TraceContext;
import com.automq.stream.s3.wal.RecordOffset;
import com.automq.stream.utils.Threads;
import com.automq.stream.utils.biniarysearch.StreamRecordBatchList;

import org.slf4j.Logger;
Expand All @@ -37,7 +38,9 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
Expand All @@ -59,6 +62,8 @@ public class LogCache {
private static final Consumer<LogCacheBlock> DEFAULT_BLOCK_FREE_LISTENER = block -> {
};
private static final int MAX_BLOCKS_COUNT = 64;
private static final ExecutorService LOG_CACHE_ASYNC_EXECUTOR = Threads.newFixedFastThreadLocalThreadPoolWithMonitor(
1, "LOG_CACHE_ASYNC", true, LOGGER);
static final int MERGE_BLOCK_THRESHOLD = 8;
final List<LogCacheBlock> blocks = new ArrayList<>();
final AtomicInteger blockCount = new AtomicInteger(1);
Expand Down Expand Up @@ -259,10 +264,19 @@ Optional<LogCacheBlock> archiveCurrentBlockIfContains0(long streamId) {

}

public void markFree(LogCacheBlock block) {
public CompletableFuture<Void> markFree(LogCacheBlock block) {
block.free = true;
tryRealFree();
tryMerge();
CompletableFuture<Void> cf = new CompletableFuture<>();
LOG_CACHE_ASYNC_EXECUTOR.execute(() -> {
try {
tryMerge();
cf.complete(null);
} catch (Throwable t) {
cf.completeExceptionally(t);
}
});
return cf;
}

private void tryRealFree() {
Expand Down Expand Up @@ -295,10 +309,10 @@ private void tryRealFree() {
writeLock.unlock();
}
size.addAndGet(-freeSize);
removed.forEach(b -> {
LOG_CACHE_ASYNC_EXECUTOR.execute(() -> removed.forEach(b -> {
blockFreeListener.accept(b);
b.free();
});
}));
}

private void tryMerge() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@

import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
Expand Down Expand Up @@ -164,7 +165,7 @@ public void testMergeBlock() {
}

@Test
public void testTryMergeLogic() {
public void testTryMergeLogic() throws ExecutionException, InterruptedException {
LogCache logCache = new LogCache(Long.MAX_VALUE, 10_000L);
final long streamId = 233L;
final int blocksToCreate = LogCache.MERGE_BLOCK_THRESHOLD + 2;
Expand All @@ -187,8 +188,8 @@ public void testTryMergeLogic() {
assertEquals(leftCache.endOffset(), rightCache.startOffset());

// mark both blocks free to trigger tryMerge (called inside markFree)
logCache.markFree(left);
logCache.markFree(right);
logCache.markFree(left).get();
logCache.markFree(right).get();

int after = logCache.blocks.size();
assertEquals(before - 1, after, "two adjacent free contiguous blocks should be merged into one");
Expand Down
Loading