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

feat: block cache read ahead #134

Merged
merged 2 commits into from
Sep 14, 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 core/src/main/scala/kafka/log/s3/ObjectReader.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import java.util.concurrent.atomic.AtomicInteger;

import static kafka.log.s3.ObjectWriter.Footer.FOOTER_SIZE;
import static org.apache.kafka.metadata.stream.ObjectUtils.NOOP_OFFSET;

public class ObjectReader implements AutoCloseable {
private static final Logger LOGGER = LoggerFactory.getLogger(ObjectReader.class);
Expand Down Expand Up @@ -225,7 +226,7 @@ public List<DataBlockIndex> find(long streamId, long startOffset, long endOffset
nextMaxBytes -= Math.min(nextMaxBytes, blockSize);
}
matched = true;
if (nextStartOffset >= endOffset || nextMaxBytes == 0) {
if ((endOffset != NOOP_OFFSET && nextStartOffset >= endOffset) || nextMaxBytes == 0) {
break;
}
} else if (matched) {
Expand Down
130 changes: 90 additions & 40 deletions core/src/main/scala/kafka/log/s3/cache/BlockCache.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@


import kafka.log.s3.model.StreamRecordBatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.ArrayList;
import java.util.Collections;
Expand All @@ -35,8 +37,11 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;

public class BlockCache {
private static final Logger LOGGER = LoggerFactory.getLogger(BlockCache.class);
static final int BLOCK_SIZE = 1024 * 1024;
static final int MAX_READAHEAD_SIZE = 128 * 1024 * 1024;
private final long maxSize;
private final Map<Long, NavigableMap<Long, CacheBlock>> stream2cache = new HashMap<>();
final Map<Long, StreamCache> stream2cache = new HashMap<>();
private final LRUCache<CacheKey, Integer> inactive = new LRUCache<>();
private final LRUCache<CacheKey, Integer> active = new LRUCache<>();
private final AtomicLong size = new AtomicLong();
Expand All @@ -62,56 +67,56 @@ public void put0(long streamId, List<StreamRecordBatch> records) {
records.forEach(StreamRecordBatch::release);
return;
}
boolean overlapped = false;
records = new ArrayList<>(records);
NavigableMap<Long, CacheBlock> streamCache = stream2cache.computeIfAbsent(streamId, id -> new TreeMap<>());
StreamCache streamCache = stream2cache.computeIfAbsent(streamId, id -> new StreamCache());
long startOffset = records.get(0).getBaseOffset();
long endOffset = records.get(records.size() - 1).getLastOffset();
// TODO: generate readahead.
Map.Entry<Long, CacheBlock> floorEntry = streamCache.floorEntry(startOffset);
SortedMap<Long, CacheBlock> tailMap = streamCache.tailMap(floorEntry != null ? floorEntry.getKey() : startOffset);

// generate readahead.
Readahead readahead = genReadahead(streamId, records);

// remove overlapped part.
Map.Entry<Long, CacheBlock> floorEntry = streamCache.blocks.floorEntry(startOffset);
SortedMap<Long, CacheBlock> tailMap = streamCache.blocks.tailMap(floorEntry != null ? floorEntry.getKey() : startOffset);
for (Map.Entry<Long, CacheBlock> entry : tailMap.entrySet()) {
CacheBlock cacheBlock = entry.getValue();
if (cacheBlock.firstOffset >= endOffset) {
break;
}
// overlap is a rare case, so removeIf is fine for the performance.
if (records.removeIf(record -> {
records.removeIf(record -> {
boolean remove = record.getLastOffset() > cacheBlock.firstOffset && record.getBaseOffset() < cacheBlock.lastOffset;
if (remove) {
record.release();
}
return remove;
})) {
overlapped = true;
}
});
}

// ensure the cache size.
int size = records.stream().mapToInt(StreamRecordBatch::size).sum();
ensureCapacity(size);

// TODO: split records to 1MB blocks.
if (overlapped) {
// split to multiple cache blocks.
long expectStartOffset = -1L;
List<StreamRecordBatch> part = new ArrayList<>(records.size() / 2);
for (StreamRecordBatch record : records) {
if (expectStartOffset == -1L || record.getBaseOffset() == expectStartOffset) {
part.add(record);
} else {
put(streamId, streamCache, new CacheBlock(part));
part = new ArrayList<>(records.size() / 2);
part.add(record);
}
expectStartOffset = record.getLastOffset();
}
if (!part.isEmpty()) {
put(streamId, streamCache, new CacheBlock(part));
// split to 1MB cache blocks which one block contains sequential records.
long expectStartOffset = -1L;
List<StreamRecordBatch> part = new ArrayList<>(records.size() / 2);
int partSize = 0;
for (StreamRecordBatch record : records) {
if (expectStartOffset == -1L || record.getBaseOffset() == expectStartOffset || partSize >= BLOCK_SIZE) {
part.add(record);
partSize += record.size();
} else {
// put readahead to the first block.
put(streamId, streamCache, new CacheBlock(part, readahead));
readahead = null;
part = new ArrayList<>(records.size() / 2);
partSize = 0;
part.add(record);
}
} else {
put(streamId, streamCache, new CacheBlock(records));
expectStartOffset = record.getLastOffset();
}
if (!part.isEmpty()) {
put(streamId, streamCache, new CacheBlock(part, readahead));
}

}
Expand All @@ -131,23 +136,24 @@ public GetCacheResult get(long streamId, long startOffset, long endOffset, int m
}

public GetCacheResult get0(long streamId, long startOffset, long endOffset, int maxBytes) {
NavigableMap<Long, CacheBlock> streamCache = stream2cache.get(streamId);
StreamCache streamCache = stream2cache.get(streamId);
if (streamCache == null) {
return GetCacheResult.empty();
}
Map.Entry<Long, CacheBlock> floorEntry = streamCache.floorEntry(startOffset);
streamCache = streamCache.tailMap(floorEntry != null ? floorEntry.getKey() : startOffset, true);
Map.Entry<Long, CacheBlock> floorEntry = streamCache.blocks.floorEntry(startOffset);
NavigableMap<Long, CacheBlock> streamCacheBlocks = streamCache.blocks.tailMap(floorEntry != null ? floorEntry.getKey() : startOffset, true);
long nextStartOffset = startOffset;
int nextMaxBytes = maxBytes;
Readahead readahead = null;
LinkedList<StreamRecordBatch> records = new LinkedList<>();
for (Map.Entry<Long, CacheBlock> entry : streamCache.entrySet()) {
for (Map.Entry<Long, CacheBlock> entry : streamCacheBlocks.entrySet()) {
CacheBlock cacheBlock = entry.getValue();
if (cacheBlock.lastOffset < nextStartOffset || nextStartOffset < cacheBlock.firstOffset) {
break;
}
if (readahead == null && cacheBlock.readahead != null) {
readahead = cacheBlock.readahead;
cacheBlock.readahead = null;
}
nextMaxBytes = readFromCacheBlock(records, cacheBlock, nextStartOffset, endOffset, nextMaxBytes);
nextStartOffset = records.getLast().getLastOffset();
Expand Down Expand Up @@ -201,7 +207,12 @@ private void ensureCapacity(int size) {
if (entry == null) {
break;
}
CacheBlock cacheBlock = stream2cache.get(entry.getKey().streamId).remove(entry.getKey().startOffset);
StreamCache streamCache = stream2cache.get(entry.getKey().streamId);
if (streamCache == null) {
LOGGER.error("[BUG] Stream cache not found for streamId: {}", entry.getKey().streamId);
continue;
}
CacheBlock cacheBlock = streamCache.blocks.remove(entry.getKey().startOffset);
cacheBlock.free();
if (maxSize - this.size.addAndGet(-entry.getValue()) >= size) {
return;
Expand All @@ -210,12 +221,55 @@ private void ensureCapacity(int size) {
}
}

private void put(long streamId, NavigableMap<Long, CacheBlock> streamCache, CacheBlock cacheBlock) {
streamCache.put(cacheBlock.firstOffset, cacheBlock);
private void put(long streamId, StreamCache streamCache, CacheBlock cacheBlock) {
streamCache.blocks.put(cacheBlock.firstOffset, cacheBlock);
active.put(new CacheKey(streamId, cacheBlock.firstOffset), cacheBlock.size);
size.getAndAdd(cacheBlock.size);
}


Readahead genReadahead(long streamId, List<StreamRecordBatch> records) {
if (records.isEmpty()) {
return null;
}
long startOffset = records.get(records.size() - 1).getLastOffset();
int size = records.stream().mapToInt(StreamRecordBatch::size).sum();
size = alignBlockSize(size);
StreamCache streamCache = stream2cache.get(streamId);
if (streamCache != null && streamCache.evict) {
// exponential fallback when cache is tight.
size = alignBlockSize(size / 2);
streamCache.evict = false;
} else {
if (size < MAX_READAHEAD_SIZE / 2) {
// exponential growth
size = size * 2;
} else {
// linear growth
size += BLOCK_SIZE;
}
}
size = Math.min(Math.max(size, BLOCK_SIZE), MAX_READAHEAD_SIZE);
return new

Readahead(startOffset, size);
}

int alignBlockSize(int size) {
return (size + BLOCK_SIZE - 1) / BLOCK_SIZE * BLOCK_SIZE;
}

static class StreamCache {
NavigableMap<Long, CacheBlock> blocks;
boolean evict;

public StreamCache() {
blocks = new TreeMap<>();
evict = false;
}

}

static class CacheKey {
final long streamId;
final long startOffset;
Expand Down Expand Up @@ -256,10 +310,6 @@ public CacheBlock(List<StreamRecordBatch> records, Readahead readahead) {
this.readahead = readahead;
}

public CacheBlock(List<StreamRecordBatch> records) {
this(records, null);
}

public void free() {
records.forEach(StreamRecordBatch::release);
records = null;
Expand Down
Loading