sopel39 commented on code in PR #11781: URL: https://github.com/apache/iceberg/pull/11781#discussion_r1905856982
########## core/src/main/java/org/apache/iceberg/util/ParallelIterable.java: ########## @@ -257,17 +257,17 @@ private static class Task<T> implements Supplier<Optional<Task<T>>>, Closeable { @Override public Optional<Task<T>> get() { try { + if (queue.size() >= approximateMaxQueueSize) { + // Yield when queue is over the size limit. Task will be resubmitted later and continue + // the work. + return Optional.of(this); + } + if (iterator == null) { iterator = input.iterator(); } while (iterator.hasNext()) { - if (queue.size() >= approximateMaxQueueSize) { Review Comment: Disccused offline. Generally, this change can increase memory usage when manifest files produce a huge amount of entries. Hence this change sits between unbounded queue and strictly bounded queue as implemented by @findepi. In the future we could completely eliminate `ParallelIterable` as producers know exactly (or can estimate) how man entries are in each manifest file, hence it's possible to accurately schedule manifest file reads. Alternatively, rest catalogs will provide scan planning capabilities (plus huge cache) hence engine wouldn't need to read the manifest files ########## core/src/main/java/org/apache/iceberg/util/ParallelIterable.java: ########## @@ -257,17 +263,21 @@ private static class Task<T> implements Supplier<Optional<Task<T>>>, Closeable { @Override public Optional<Task<T>> get() { try { + if (queue.size() >= approximateMaxQueueSize) { + // Tasks might hold references (via iterator) to constrained resources Review Comment: Updated comment -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org