Skip to content

Commit 0cd047c

Browse files
authored
fix: support list more than 1000 objects by prefix (#2660) (#2666)
This commit fixes an issue where the doList method in AwsObjectStorage.java did not handle paginated results from the S3 listObjectsV2 API. The method now recursively fetches all pages of objects, ensuring that it can retrieve more than the default 1000 objects.
1 parent caaa417 commit 0cd047c

File tree

1 file changed

+27
-6
lines changed

1 file changed

+27
-6
lines changed

s3stream/src/main/java/com/automq/stream/s3/operator/AwsObjectStorage.java

Lines changed: 27 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -362,12 +362,33 @@ void doClose() {
362362

363363
@Override
364364
CompletableFuture<List<ObjectInfo>> doList(String prefix) {
365-
return readS3Client.listObjectsV2(builder -> builder.bucket(bucket).prefix(prefix))
366-
.thenApply(resp ->
367-
resp.contents()
368-
.stream()
369-
.map(object -> new ObjectInfo(bucketURI.bucketId(), object.key(), object.lastModified().toEpochMilli(), object.size()))
370-
.collect(Collectors.toList()));
365+
CompletableFuture<List<ObjectInfo>> resultFuture = new CompletableFuture<>();
366+
List<ObjectInfo> allObjects = new ArrayList<>();
367+
listNextBatch(prefix, null, allObjects, resultFuture);
368+
return resultFuture;
369+
}
370+
371+
private void listNextBatch(String prefix, String continuationToken, List<ObjectInfo> allObjects,
372+
CompletableFuture<List<ObjectInfo>> resultFuture) {
373+
readS3Client.listObjectsV2(builder -> {
374+
builder.bucket(bucket).prefix(prefix);
375+
if (continuationToken != null) {
376+
builder.continuationToken(continuationToken);
377+
}
378+
}).thenAccept(resp -> {
379+
resp.contents()
380+
.stream()
381+
.map(object -> new ObjectInfo(bucketURI.bucketId(), object.key(), object.lastModified().toEpochMilli(), object.size()))
382+
.forEach(allObjects::add);
383+
if (resp.isTruncated()) {
384+
listNextBatch(prefix, resp.nextContinuationToken(), allObjects, resultFuture);
385+
} else {
386+
resultFuture.complete(allObjects);
387+
}
388+
}).exceptionally(ex -> {
389+
resultFuture.completeExceptionally(ex);
390+
return null;
391+
});
371392
}
372393

373394
@Override

0 commit comments

Comments
 (0)