mxm commented on code in PR #15312:
URL: https://github.com/apache/iceberg/pull/15312#discussion_r2890304053


##########
core/src/test/java/org/apache/iceberg/util/TestThreadPools.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.util;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.time.Duration;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.junit.jupiter.api.Test;
+
+public class TestThreadPools {
+
+  @Test
+  public void testRemoveShutdownHook() {
+    try {
+      assertThat(ThreadPools.isShutdownHookRegistered()).isTrue();
+      ThreadPools.removeShutdownHook();
+      assertThat(ThreadPools.isShutdownHookRegistered()).isFalse();
+    } finally {
+      ThreadPools.initShutdownHook();
+    }
+  }
+
+  @Test
+  public void testThreadPoolManagerAddAndShutdown() throws Exception {
+    ThreadPools.ThreadPoolManager manager = new 
ThreadPools.ThreadPoolManager();
+
+    ExecutorService testExecutor = Executors.newFixedThreadPool(2);
+
+    Duration timeout = Duration.ofSeconds(5);
+    manager.addThreadPool(testExecutor, timeout);
+
+    manager.shutdownAll();
+
+    assertThat(testExecutor.isShutdown()).isTrue();
+  }
+
+  @Test
+  public void testThreadPoolManagerMultipleShutdowns() throws Exception {
+    ThreadPools.ThreadPoolManager manager = new 
ThreadPools.ThreadPoolManager();
+
+    ExecutorService executor1 = Executors.newFixedThreadPool(1);
+    ExecutorService executor2 = Executors.newFixedThreadPool(1);
+
+    Duration timeout = Duration.ofSeconds(5);
+    manager.addThreadPool(executor1, timeout);
+    manager.addThreadPool(executor2, timeout);
+
+    manager.shutdownAll();
+
+    assertThat(executor1.isShutdown()).isTrue();
+    assertThat(executor2.isShutdown()).isTrue();
+  }
+
+  @Test
+  public void testThreadPoolManagerShutdownNowCalled() throws Exception {
+    ThreadPools.ThreadPoolManager manager = new 
ThreadPools.ThreadPoolManager();
+
+    final AtomicBoolean interrupted = new AtomicBoolean(false);
+    ExecutorService slowExecutor = Executors.newFixedThreadPool(1);
+
+    manager.addThreadPool(slowExecutor, Duration.ofMillis(50));
+
+    CountDownLatch threadStarted = new CountDownLatch(1);
+    CountDownLatch threadInterrupted = new CountDownLatch(1);
+
+    slowExecutor.submit(
+        () -> {
+          try {
+            threadStarted.countDown();
+            Thread.sleep(60_000);

Review Comment:
   Should we add a comment that we're not actually waiting 60 seconds here?



##########
core/src/main/java/org/apache/iceberg/util/ThreadPools.java:
##########
@@ -180,15 +261,74 @@ public static ScheduledExecutorService 
newScheduledPool(String namePrefix, int p
    * is suitable for long-lived thread pools that should be automatically 
cleaned up on JVM
    * shutdown.
    */
-  public static ScheduledExecutorService newExitingScheduledPool(
+  public static synchronized ScheduledExecutorService newExitingScheduledPool(
       String namePrefix, int poolSize, Duration terminationTimeout) {
-    return MoreExecutors.getExitingScheduledExecutorService(
-        (ScheduledThreadPoolExecutor) newScheduledPool(namePrefix, poolSize),
-        terminationTimeout.toMillis(),
-        TimeUnit.MILLISECONDS);
+    ScheduledExecutorService service =
+        
Executors.unconfigurableScheduledExecutorService(newScheduledPool(namePrefix, 
poolSize));
+    THREAD_POOL_MANAGER.addThreadPool(service, terminationTimeout);
+    return service;
   }
 
   private static ThreadFactory newDaemonThreadFactory(String namePrefix) {
     return new ThreadFactoryBuilder().setDaemon(true).setNameFormat(namePrefix 
+ "-%d").build();
   }
+
+  /** Manages the lifecycle of thread pools that need to be shut down 
gracefully. */
+  static class ThreadPoolManager {
+    private final List<ExecutorServiceWithTimeout> threadPoolsToShutdown = 
Lists.newArrayList();
+
+    /**
+     * Add an executor service to the list of thread pools to be shut down.
+     *
+     * @param service the executor service to add
+     * @param timeout the timeout for shutdown operations
+     */
+    synchronized void addThreadPool(ExecutorService service, Duration timeout) 
{
+      threadPoolsToShutdown.add(new ExecutorServiceWithTimeout(service, 
timeout));
+    }
+
+    /** Shut down all registered thread pools. */
+    synchronized void shutdownAll() {
+      long startTime = System.nanoTime();
+      List<ExecutorServiceWithTimeout> pendingShutdown = Lists.newArrayList();
+      for (ExecutorServiceWithTimeout item : threadPoolsToShutdown) {
+        item.getService().shutdown();
+        pendingShutdown.add(item);
+      }
+      threadPoolsToShutdown.clear();
+      for (ExecutorServiceWithTimeout item : pendingShutdown) {
+        long timeElapsed = System.nanoTime() - startTime;
+        long remainingTime = item.getTimeout().toNanos() - timeElapsed;
+        if (remainingTime > 0) {
+          try {
+            if (!item.service.awaitTermination(remainingTime, 
TimeUnit.NANOSECONDS)) {
+              item.getService().shutdownNow();
+            }
+          } catch (InterruptedException ignored) {
+            // We're shutting down anyway, so just ignore.
+          }
+        } else {

Review Comment:
   NIT newline after blocks (Iceberg style)



##########
core/src/main/java/org/apache/iceberg/util/ThreadPools.java:
##########
@@ -180,15 +261,74 @@ public static ScheduledExecutorService 
newScheduledPool(String namePrefix, int p
    * is suitable for long-lived thread pools that should be automatically 
cleaned up on JVM
    * shutdown.
    */
-  public static ScheduledExecutorService newExitingScheduledPool(
+  public static synchronized ScheduledExecutorService newExitingScheduledPool(
       String namePrefix, int poolSize, Duration terminationTimeout) {
-    return MoreExecutors.getExitingScheduledExecutorService(
-        (ScheduledThreadPoolExecutor) newScheduledPool(namePrefix, poolSize),
-        terminationTimeout.toMillis(),
-        TimeUnit.MILLISECONDS);
+    ScheduledExecutorService service =
+        
Executors.unconfigurableScheduledExecutorService(newScheduledPool(namePrefix, 
poolSize));
+    THREAD_POOL_MANAGER.addThreadPool(service, terminationTimeout);
+    return service;
   }
 
   private static ThreadFactory newDaemonThreadFactory(String namePrefix) {
     return new ThreadFactoryBuilder().setDaemon(true).setNameFormat(namePrefix 
+ "-%d").build();
   }
+
+  /** Manages the lifecycle of thread pools that need to be shut down 
gracefully. */
+  static class ThreadPoolManager {
+    private final List<ExecutorServiceWithTimeout> threadPoolsToShutdown = 
Lists.newArrayList();
+
+    /**
+     * Add an executor service to the list of thread pools to be shut down.
+     *
+     * @param service the executor service to add
+     * @param timeout the timeout for shutdown operations
+     */
+    synchronized void addThreadPool(ExecutorService service, Duration timeout) 
{
+      threadPoolsToShutdown.add(new ExecutorServiceWithTimeout(service, 
timeout));
+    }
+
+    /** Shut down all registered thread pools. */
+    synchronized void shutdownAll() {
+      long startTime = System.nanoTime();
+      List<ExecutorServiceWithTimeout> pendingShutdown = Lists.newArrayList();
+      for (ExecutorServiceWithTimeout item : threadPoolsToShutdown) {
+        item.getService().shutdown();
+        pendingShutdown.add(item);
+      }
+      threadPoolsToShutdown.clear();

Review Comment:
   NIT newline after blocks (Iceberg style)



##########
core/src/main/java/org/apache/iceberg/util/ThreadPools.java:
##########
@@ -149,8 +152,86 @@ public static ExecutorService newWorkerPool(String 
namePrefix, int poolSize) {
    * that should be automatically cleaned up on JVM shutdown.
    */
   public static ExecutorService newExitingWorkerPool(String namePrefix, int 
poolSize) {
-    return MoreExecutors.getExitingExecutorService(
-        (ThreadPoolExecutor) newFixedThreadPool(namePrefix, poolSize));
+    ExecutorService service =
+        Executors.unconfigurableExecutorService(newFixedThreadPool(namePrefix, 
poolSize));
+    THREAD_POOL_MANAGER.addThreadPool(service, DEFAULT_SHUTDOWN_TIMEOUT);
+    return service;
+  }
+
+  /**
+   * Force manual shutdown of the thread pools created via the {@link 
#newExitingWorkerPool(String,
+   * int)}.
+   *
+   * <p>This method allows: (1) to stop thread pools manually, to avoid leaks 
in hot-reload
+   * environments; (2) opt-out of the standard shutdown mechanism to manage 
graceful service stops
+   * (and commit the last pending files, if the client application needs to 
react to shutdown hooks
+   * on its own).
+   *
+   * <p>Please only call this method at the end of the intended usage of the 
library, and NEVER
+   * before, as this method will stop thread pools required for normal library 
workflows.
+   */
+  public static void shutdownThreadPools() {
+    THREAD_POOL_MANAGER.shutdownAll();
+    removeShutdownHook();
+  }
+
+  /**
+   * Initialize a shutdown hook to stop the thread pools created via the {@link
+   * #newExitingWorkerPool(String, int)}.
+   */
+  @SuppressWarnings("ShutdownHook")
+  static synchronized void initShutdownHook() {
+    if (shutdownHook == null) {
+      shutdownHook =
+          Executors.defaultThreadFactory()
+              .newThread(
+                  new Runnable() {
+                    @Override
+                    public void run() {
+                      shutdownHook = null;
+                      shutdownThreadPools();
+                    }
+                  });
+
+      try {
+        shutdownHook.setName("DelayedShutdownHook-iceberg");
+      } catch (SecurityException e) {
+        LOG.warn("Cannot set thread name for the shutdown hook", e);
+      }
+
+      try {
+        Runtime.getRuntime().addShutdownHook(shutdownHook);
+      } catch (SecurityException e) {
+        LOG.warn("Cannot install a shutdown hook for thread pools clean up", 
e);
+      }
+    }
+  }
+
+  /**
+   * Stop the shutdown hook for the thread pools created via the {@link
+   * #newExitingWorkerPool(String, int)}.
+   *
+   * <p>Thread pools can still be stopped manually via the {@link 
#shutdownThreadPools()} method.
+   */
+  @SuppressWarnings("ShutdownHook")
+  public static synchronized void removeShutdownHook() {
+    if (shutdownHook != null) {
+      try {
+        Runtime.getRuntime().removeShutdownHook(shutdownHook);
+      } catch (SecurityException e) {
+        LOG.warn("Cannot remove the shutdown hook for thread pools clean up", 
e);
+      }
+      shutdownHook = null;

Review Comment:
   NIT newline after blocks (Iceberg style)



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to