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): add clean up on future exception for BlockCache #781

Merged
merged 1 commit into from
Dec 4, 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
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@ public CompletableFuture<ReadDataBlock> read(long streamId, long startOffset, lo
if (ex != null) {
LOGGER.error("read {} [{}, {}) from block cache fail", streamId, startOffset, endOffset, ex);
readCf.completeExceptionally(ex);
this.inflightReadThrottle.release(uuid);
return;
}
int totalReturnedSize = ret.getRecords().stream().mapToInt(StreamRecordBatch::size).sum();
Expand Down
29 changes: 19 additions & 10 deletions s3stream/src/main/java/com/automq/stream/s3/cache/StreamReader.java
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,6 @@ public CompletableFuture<List<StreamRecordBatch>> syncReadAhead(long streamId, l
if (dataBlock.records().isEmpty()) {
return;
}

// retain records to be returned
dataBlock.records().forEach(StreamRecordBatch::retain);
recordsMap.put(dataBlockKey, dataBlock.records());
Expand All @@ -145,13 +144,18 @@ public CompletableFuture<List<StreamRecordBatch>> syncReadAhead(long streamId, l
dataBlock.records().forEach(StreamRecordBatch::retain);
blockCache.put(streamId, dataBlock.records());
dataBlock.release();

// complete and remove inflight read ahead task
}, backgroundExecutor).whenComplete((ret, ex) -> {
CompletableFuture<Void> inflightReadAheadTask = inflightReadAheadTaskMap.remove(taskKey);
if (inflightReadAheadTask != null) {
inflightReadAheadTask.complete(null);
if (ex != null) {
LOGGER.error("[S3BlockCache] sync ra fail to read data block, stream={}, {}-{}, data block: {}",
streamId, startOffset, endOffset, streamDataBlock, ex);
inflightReadAheadTask.completeExceptionally(ex);
} else {
inflightReadAheadTask.complete(null);
}
}
}, backgroundExecutor));
}));
if (reserveResult.reserveSize() > 0) {
dataBlockReadAccumulator.readDataBlock(objectReader, streamDataBlock.dataBlockIndex());
}
Expand Down Expand Up @@ -233,7 +237,6 @@ public void asyncReadAhead(long streamId, long startOffset, long endOffset, int
if (dataBlock.records().isEmpty()) {
return;
}

// retain records to be put into block cache
dataBlock.records().forEach(StreamRecordBatch::retain);
if (readIndex == 0) {
Expand All @@ -243,14 +246,20 @@ public void asyncReadAhead(long streamId, long startOffset, long endOffset, int
blockCache.put(streamId, dataBlock.records());
}
dataBlock.release();
}, backgroundExecutor).whenComplete((ret, ex) -> {
inflightReadThrottle.release(uuid);

// complete and remove inflight read ahead task
CompletableFuture<Void> inflightReadAheadTask = inflightReadAheadTaskMap.remove(taskKey);
if (inflightReadAheadTask != null) {
inflightReadAheadTask.complete(null);
if (ex != null) {
LOGGER.error("[S3BlockCache] async ra fail to read data block, stream={}, {}-{}, data block: {}",
streamId, startOffset, endOffset, streamDataBlock, ex);
inflightReadAheadTask.completeExceptionally(ex);
} else {
inflightReadAheadTask.complete(null);
}
}
}, backgroundExecutor));

}));

if (LOGGER.isDebugEnabled()) {
LOGGER.debug("[S3BlockCache] async ra acquire size: {}, uuid={}, stream={}, {}-{}, {}",
Expand Down
Loading