-
Notifications
You must be signed in to change notification settings - Fork 593
HDDS-14236. Fix BackgroundService to follow scheduleWithFixedDelay semantics #9595
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: master
Are you sure you want to change the base?
Changes from all commits
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 |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.ScheduledThreadPoolExecutor; | ||
| import java.util.concurrent.ThreadFactory; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.ratis.util.TimeDuration; | ||
| import org.slf4j.Logger; | ||
|
|
@@ -40,8 +41,9 @@ public abstract class BackgroundService { | |
| protected static final Logger LOG = | ||
| LoggerFactory.getLogger(BackgroundService.class); | ||
|
|
||
| // Executor to launch child tasks | ||
| private ScheduledThreadPoolExecutor exec; | ||
| private ThreadPoolExecutor exec; | ||
| // Scheduler for PeriodicalTask | ||
| private ScheduledThreadPoolExecutor scheduler; | ||
| private ThreadGroup threadGroup; | ||
| private final String serviceName; | ||
| private long interval; | ||
|
|
@@ -86,10 +88,16 @@ public synchronized void setPoolSize(int size) { | |
| throw new IllegalArgumentException("Pool size must be positive."); | ||
| } | ||
|
|
||
| // In ScheduledThreadPoolExecutor, maximumPoolSize is Integer.MAX_VALUE | ||
| // the corePoolSize will always less maximumPoolSize. | ||
| // So we can directly set the corePoolSize | ||
| exec.setCorePoolSize(size); | ||
| // In ThreadPoolExecutor, maximumPoolSize must always be >= corePoolSize. | ||
| // Update in the correct order based on whether we are increasing or decreasing. | ||
| final int currentCorePoolSize = exec.getCorePoolSize(); | ||
| if (size > currentCorePoolSize) { | ||
| exec.setMaximumPoolSize(size); | ||
| exec.setCorePoolSize(size); | ||
| } else if (size < currentCorePoolSize) { | ||
| exec.setCorePoolSize(size); | ||
| exec.setMaximumPoolSize(size); | ||
| } | ||
| } | ||
|
|
||
| public synchronized void setServiceTimeoutInNanos(long newTimeout) { | ||
|
|
@@ -113,12 +121,13 @@ public void runPeriodicalTaskNow() throws Exception { | |
|
|
||
| // start service | ||
| public synchronized void start() { | ||
| if (exec == null || exec.isShutdown() || exec.isTerminated()) { | ||
| if (exec == null || exec.isShutdown() || exec.isTerminated() | ||
| || scheduler == null || scheduler.isShutdown() || scheduler.isTerminated()) { | ||
| initExecutorAndThreadGroup(); | ||
| } | ||
| LOG.info("Starting service {} with interval {} {}", serviceName, | ||
| interval, unit.name().toLowerCase()); | ||
| exec.scheduleWithFixedDelay(service, 0, interval, unit); | ||
| scheduler.scheduleWithFixedDelay(service, 0, interval, unit); | ||
|
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. @xichen01 , thanks for continue this improvement task. Can we keep using the exec here? What's the extra benefit of introducing scheduler?
Contributor
Author
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.
Therefore, we introduce a separate thread
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. ”exec“ is a ScheduledExecutorService thread pool with no upper thread limit. I also just found it during the discussion of your last proposal patch's review.
Contributor
Author
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. Yes, but the queue of |
||
| } | ||
|
|
||
| protected synchronized void setInterval(long newInterval, TimeUnit newUnit) { | ||
|
|
@@ -141,15 +150,6 @@ protected void execTaskCompletion() { } | |
| public class PeriodicalTask implements Runnable { | ||
| @Override | ||
| public void run() { | ||
| // wait for previous set of tasks to complete | ||
| try { | ||
| future.join(); | ||
| } catch (RuntimeException e) { | ||
| LOG.error("Background service execution failed.", e); | ||
| } finally { | ||
| execTaskCompletion(); | ||
| } | ||
|
|
||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Running background service : {}", serviceName); | ||
| } | ||
|
|
@@ -187,20 +187,33 @@ public void run() { | |
| }, exec).exceptionally(e -> null), (Void1, Void) -> null); | ||
| } | ||
| } | ||
| // wait for all tasks to complete | ||
| try { | ||
| future.join(); | ||
| } catch (RuntimeException e) { | ||
| LOG.error("Background service execution failed.", e); | ||
| } finally { | ||
| execTaskCompletion(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // shutdown and make sure all threads are properly released. | ||
| public synchronized void shutdown() { | ||
| LOG.info("Shutting down service {}", this.serviceName); | ||
| scheduler.shutdown(); | ||
| exec.shutdown(); | ||
| try { | ||
| if (!scheduler.awaitTermination(60, TimeUnit.SECONDS)) { | ||
| scheduler.shutdownNow(); | ||
| } | ||
| if (!exec.awaitTermination(60, TimeUnit.SECONDS)) { | ||
| exec.shutdownNow(); | ||
| } | ||
| } catch (InterruptedException e) { | ||
| // Re-interrupt the thread while catching InterruptedException | ||
| Thread.currentThread().interrupt(); | ||
| scheduler.shutdownNow(); | ||
| exec.shutdownNow(); | ||
| } | ||
| if (threadGroup.activeCount() == 0 && !threadGroup.isDestroyed()) { | ||
|
|
@@ -215,7 +228,14 @@ private void initExecutorAndThreadGroup() { | |
| .setDaemon(true) | ||
| .setNameFormat(threadNamePrefix + serviceName + "#%d") | ||
| .build(); | ||
| exec = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(threadPoolSize, threadFactory); | ||
| exec = (ThreadPoolExecutor) Executors.newFixedThreadPool(threadPoolSize, threadFactory); | ||
| // Single-thread scheduler for PeriodicalTask | ||
| ThreadFactory schedulerThreadFactory = new ThreadFactoryBuilder() | ||
| .setThreadFactory(r -> new Thread(threadGroup, r)) | ||
| .setDaemon(true) | ||
| .setNameFormat(threadNamePrefix + serviceName + "-scheduler") | ||
| .build(); | ||
| scheduler = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(1, schedulerThreadFactory); | ||
| } | ||
|
|
||
| protected String getServiceName() { | ||
|
|
||
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.
execis only responsible for executing tasks, soThreadPoolExecutoris currently more suitable.