-
Notifications
You must be signed in to change notification settings - Fork 3k
Allow stopping thread pools manually #15312
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
base: main
Are you sure you want to change the base?
Changes from all commits
db22fba
f6c1d52
5681ec6
8db36b6
7915de9
dfd4044
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -19,6 +19,9 @@ | |||||||||||||
| package org.apache.iceberg.util; | ||||||||||||||
|
|
||||||||||||||
| import java.time.Duration; | ||||||||||||||
| import java.util.ArrayDeque; | ||||||||||||||
| import java.util.Queue; | ||||||||||||||
| import java.util.concurrent.ConcurrentLinkedDeque; | ||||||||||||||
| import java.util.concurrent.ExecutorService; | ||||||||||||||
| import java.util.concurrent.Executors; | ||||||||||||||
| import java.util.concurrent.ScheduledExecutorService; | ||||||||||||||
|
|
@@ -29,8 +32,11 @@ | |||||||||||||
| import org.apache.iceberg.SystemConfigs; | ||||||||||||||
| import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; | ||||||||||||||
| import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; | ||||||||||||||
| import org.slf4j.Logger; | ||||||||||||||
| import org.slf4j.LoggerFactory; | ||||||||||||||
|
|
||||||||||||||
| public class ThreadPools { | ||||||||||||||
| private static final Logger LOG = LoggerFactory.getLogger(ThreadPools.class); | ||||||||||||||
|
|
||||||||||||||
| private ThreadPools() {} | ||||||||||||||
|
|
||||||||||||||
|
|
@@ -44,6 +50,9 @@ private ThreadPools() {} | |||||||||||||
|
|
||||||||||||||
| public static final int WORKER_THREAD_POOL_SIZE = SystemConfigs.WORKER_THREAD_POOL_SIZE.value(); | ||||||||||||||
|
|
||||||||||||||
| private static final ConcurrentLinkedDeque<ExecutorService> THREAD_POOLS_TO_SHUTDOWN = | ||||||||||||||
| new ConcurrentLinkedDeque<>(); | ||||||||||||||
|
|
||||||||||||||
| private static final ExecutorService WORKER_POOL = | ||||||||||||||
| newExitingWorkerPool("iceberg-worker-pool", WORKER_THREAD_POOL_SIZE); | ||||||||||||||
|
|
||||||||||||||
|
|
@@ -56,6 +65,14 @@ private ThreadPools() {} | |||||||||||||
| public static final int AUTH_REFRESH_THREAD_POOL_SIZE = | ||||||||||||||
| SystemConfigs.AUTH_REFRESH_THREAD_POOL_SIZE.value(); | ||||||||||||||
|
|
||||||||||||||
| private static final int SHUTDOWN_TIMEOUT_SECONDS = 120; | ||||||||||||||
|
|
||||||||||||||
| private static Thread shutdownHook; | ||||||||||||||
|
|
||||||||||||||
| static { | ||||||||||||||
| initShutdownHook(); | ||||||||||||||
| } | ||||||||||||||
|
|
||||||||||||||
| /** | ||||||||||||||
| * Return an {@link ExecutorService} that uses the "worker" thread-pool. | ||||||||||||||
| * | ||||||||||||||
|
|
@@ -149,8 +166,85 @@ 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_POOLS_TO_SHUTDOWN.add(service); | ||||||||||||||
| return service; | ||||||||||||||
| } | ||||||||||||||
|
|
||||||||||||||
| /** | ||||||||||||||
| * Force manual shutdown of the thread pools created via the {@link #newExitingWorkerPool(String, | ||||||||||||||
| * int)}. | ||||||||||||||
| */ | ||||||||||||||
| public static void shutdownStartedThreadPools() { | ||||||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think first thing we should remove the corresponding shutdown hook. |
||||||||||||||
| long startTime = System.nanoTime(); | ||||||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Probably sufficient to use System.currentTimeMillis() here. |
||||||||||||||
| ExecutorService item; | ||||||||||||||
| Queue<ExecutorService> invoked = new ArrayDeque<>(); | ||||||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||||||||
| while ((item = THREAD_POOLS_TO_SHUTDOWN.poll()) != null) { | ||||||||||||||
| item.shutdown(); | ||||||||||||||
| invoked.add(item); | ||||||||||||||
| } | ||||||||||||||
| while ((item = invoked.poll()) != null) { | ||||||||||||||
| long timeElapsed = System.nanoTime() - startTime; | ||||||||||||||
| long remainingTime = SHUTDOWN_TIMEOUT_SECONDS * 1_000_000_000L - timeElapsed; | ||||||||||||||
| if (remainingTime > 0) { | ||||||||||||||
|
Comment on lines
+188
to
+190
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||||||||
| try { | ||||||||||||||
| item.awaitTermination(remainingTime, TimeUnit.NANOSECONDS); | ||||||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we also try this?
Suggested change
|
||||||||||||||
| } catch (InterruptedException ignored) { | ||||||||||||||
| // We're shutting down anyway, so just ignore. | ||||||||||||||
| } | ||||||||||||||
| } | ||||||||||||||
| } | ||||||||||||||
| } | ||||||||||||||
|
|
||||||||||||||
| /** | ||||||||||||||
| * Initialize a shutdown hook to stop the thread pools created via the {@link | ||||||||||||||
| * #newExitingWorkerPool(String, int)}. | ||||||||||||||
| */ | ||||||||||||||
| @SuppressWarnings("ShutdownHook") | ||||||||||||||
| private static void initShutdownHook() { | ||||||||||||||
| if (shutdownHook == null) { | ||||||||||||||
| shutdownHook = | ||||||||||||||
| Executors.defaultThreadFactory() | ||||||||||||||
| .newThread( | ||||||||||||||
| new Runnable() { | ||||||||||||||
| @Override | ||||||||||||||
| public void run() { | ||||||||||||||
| shutdownStartedThreadPools(); | ||||||||||||||
| } | ||||||||||||||
| }); | ||||||||||||||
|
|
||||||||||||||
| try { | ||||||||||||||
| shutdownHook.setName("DelayedShutdownHook-iceberg"); | ||||||||||||||
| } catch (SecurityException e) { | ||||||||||||||
| LOG.warn("Cannot set thread name for the shutdown hook", e); | ||||||||||||||
| } | ||||||||||||||
svalaskevicius marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||||||||||
|
|
||||||||||||||
| 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 #shutdownStartedThreadPools()} | ||||||||||||||
| * method. | ||||||||||||||
| */ | ||||||||||||||
| @SuppressWarnings("ShutdownHook") | ||||||||||||||
| public static 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; | ||||||||||||||
| } | ||||||||||||||
| } | ||||||||||||||
|
|
||||||||||||||
| /** Creates a fixed-size thread pool that uses daemon threads. */ | ||||||||||||||
|
|
||||||||||||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.