rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1681826881


##########
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##########
@@ -88,16 +92,26 @@ private ParallelIterator(
     @Override
     public void close() {
       // close first, avoid new task submit
-      this.closed = true;
+      this.closed.set(true);
+
+      try (Closer closer = Closer.create()) {
+        yieldedTasks.forEach(closer::register);
+        yieldedTasks.clear();
 
-      // cancel background tasks
-      for (Future<?> taskFuture : taskFutures) {
-        if (taskFuture != null && !taskFuture.isDone()) {
-          taskFuture.cancel(true);
+        // TODO close input iterables that were not started yet

Review Comment:
   I think we should either get it done or decide not to, not just leave a TODO 
about closing here.
   
   The existing code use a try-with-resources for the iterable and used a `for` 
loop. That would close iterators via the iterable that created them. The way 
our iterables should work is that there is no need to close the iterable itself 
(i.e. no resources it holds) but it is a convenient way to close any iterators 
that were opened from the iterable, which do hold resources.
   
   In this case, if we have not submitted a task there should be no need to 
close the iterable. That matches the previous behavior in main. We do need to 
make sure the yielded tasks or currently running tasks are closed. I think that 
closing the yielded tasks above and cancelling the running `Task` instances 
should take care of that. The running instances should hit the `catch` for 
`Throwable` that calls close.
   
   Should be safe to just delete the TODO then.



-- 
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

Reply via email to