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(stream): release FetchResults if the subsequent fetch fails #2176

Merged
merged 1 commit into from
Nov 22, 2024
Merged
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 @@ -137,17 +137,35 @@ private CompletableFuture<Records> readAll0(FetchContext context, long startOffs
if (nextFetchOffset >= endOffset) {
return CompletableFuture.completedFuture(MemoryRecords.EMPTY);
}
return fetch0(context, nextFetchOffset, endOffset, maxSize)
.thenApply(rst -> PooledMemoryRecords.of(baseOffset, rst, context.readOptions().pooledBuf()));
List<FetchResult> results = new LinkedList<>();
return fetch0(context, nextFetchOffset, endOffset, maxSize, results)
.whenComplete((nil, e) -> {
if (e != null) {
results.forEach(FetchResult::free);
results.clear();
}
})
.thenApply(nil -> PooledMemoryRecords.of(baseOffset, results, context.readOptions().pooledBuf()));
}

private CompletableFuture<LinkedList<FetchResult>> fetch0(FetchContext context, long startOffset, long endOffset, int maxSize) {
/**
* Fetch records from the {@link ElasticStreamSlice}
*
* @param context fetch context
* @param startOffset start offset
* @param endOffset end offset
* @param maxSize max size of the fetched records
* @param results result list to be filled
* @return a future that completes when reaching the end offset or the max size
*/
private CompletableFuture<Void> fetch0(FetchContext context, long startOffset, long endOffset, int maxSize, List<FetchResult> results) {
if (startOffset >= endOffset || maxSize <= 0) {
return CompletableFuture.completedFuture(new LinkedList<>());
return CompletableFuture.completedFuture(null);
}
int adjustedMaxSize = Math.min(maxSize, 1024 * 1024);
return streamSlice.fetch(context, startOffset, endOffset, adjustedMaxSize)
.thenCompose(rst -> {
results.add(rst);
long nextFetchOffset = startOffset;
int readSize = 0;
for (RecordBatchWithContext recordBatchWithContext : rst.recordBatchList()) {
Expand All @@ -160,12 +178,7 @@ private CompletableFuture<LinkedList<FetchResult>> fetch0(FetchContext context,
}
readSize += recordBatchWithContext.rawPayload().remaining();
}
return fetch0(context, nextFetchOffset, endOffset, maxSize - readSize)
.thenApply(rstList -> {
// add to first since we need to reverse the order.
rstList.addFirst(rst);
return rstList;
});
return fetch0(context, nextFetchOffset, endOffset, maxSize - readSize, results);
});
}

Expand Down
Loading