|
8 | 8 |
|
9 | 9 | package org.elasticsearch.common.util.concurrent;
|
10 | 10 |
|
11 |
| -import org.apache.logging.log4j.LogManager; |
12 |
| -import org.apache.logging.log4j.Logger; |
13 |
| -import org.elasticsearch.core.Strings; |
| 11 | +import org.elasticsearch.action.ActionListener; |
| 12 | +import org.elasticsearch.core.Releasable; |
14 | 13 |
|
15 |
| -import java.util.concurrent.BlockingQueue; |
16 | 14 | import java.util.concurrent.Executor;
|
17 | 15 | import java.util.concurrent.PriorityBlockingQueue;
|
18 |
| -import java.util.concurrent.atomic.AtomicInteger; |
19 | 16 |
|
20 | 17 | /**
|
21 | 18 | * {@link PrioritizedThrottledTaskRunner} performs the enqueued tasks in the order dictated by the
|
22 | 19 | * natural ordering of the tasks, limiting the max number of concurrently running tasks. Each new task
|
23 | 20 | * that is dequeued to be run, is forked off to the given executor.
|
24 | 21 | */
|
25 | 22 | public class PrioritizedThrottledTaskRunner<T extends AbstractRunnable & Comparable<T>> {
|
26 |
| - private static final Logger logger = LogManager.getLogger(PrioritizedThrottledTaskRunner.class); |
27 |
| - |
28 |
| - private final String taskRunnerName; |
29 |
| - // The max number of tasks that this runner will schedule to concurrently run on the executor. |
30 |
| - private final int maxRunningTasks; |
31 |
| - // As we fork off dequeued tasks to the given executor, technically the following counter represents |
32 |
| - // the number of the concurrent pollAndSpawn calls currently checking the queue for a task to run. This |
33 |
| - // doesn't necessarily correspond to currently running tasks, since a pollAndSpawn could return without |
34 |
| - // actually running a task when the queue is empty. |
35 |
| - private final AtomicInteger runningTasks = new AtomicInteger(); |
36 |
| - private final BlockingQueue<T> tasks = new PriorityBlockingQueue<>(); |
37 |
| - private final Executor executor; |
38 | 23 |
|
39 |
| - public PrioritizedThrottledTaskRunner(final String name, final int maxRunningTasks, final Executor executor) { |
40 |
| - assert maxRunningTasks > 0; |
41 |
| - this.taskRunnerName = name; |
42 |
| - this.maxRunningTasks = maxRunningTasks; |
43 |
| - this.executor = executor; |
44 |
| - } |
| 24 | + private final AbstractThrottledTaskRunner<TaskWrapper<T>> runner; |
| 25 | + private final PriorityBlockingQueue<TaskWrapper<T>> queue; |
45 | 26 |
|
46 |
| - public void enqueueTask(final T task) { |
47 |
| - logger.trace("[{}] enqueuing task {}", taskRunnerName, task); |
48 |
| - tasks.add(task); |
49 |
| - // Try to run a task since now there is at least one in the queue. If the maxRunningTasks is |
50 |
| - // reached, the task is just enqueued. |
51 |
| - pollAndSpawn(); |
52 |
| - } |
| 27 | + private static class TaskWrapper<T extends AbstractRunnable & Comparable<T>> |
| 28 | + implements |
| 29 | + ActionListener<Releasable>, |
| 30 | + Comparable<TaskWrapper<T>> { |
| 31 | + |
| 32 | + private final T task; |
| 33 | + |
| 34 | + TaskWrapper(T task) { |
| 35 | + this.task = task; |
| 36 | + } |
| 37 | + |
| 38 | + @Override |
| 39 | + public int compareTo(TaskWrapper<T> o) { |
| 40 | + return task.compareTo(o.task); |
| 41 | + } |
53 | 42 |
|
54 |
| - private void pollAndSpawn() { |
55 |
| - // A pollAndSpawn attempts to run a new task. There could be many concurrent pollAndSpawn calls competing |
56 |
| - // to get a "free slot", since we attempt to run a new task on every enqueueTask call and every time an |
57 |
| - // existing task is finished. |
58 |
| - while (incrementRunningTasks()) { |
59 |
| - T task = tasks.poll(); |
60 |
| - if (task == null) { |
61 |
| - logger.trace("[{}] task queue is empty", taskRunnerName); |
62 |
| - // We have taken up a "free slot", but there are no tasks in the queue! This could happen each time a worker |
63 |
| - // sees an empty queue after running a task. Decrement to give competing pollAndSpawn calls a chance! |
64 |
| - int decremented = runningTasks.decrementAndGet(); |
65 |
| - assert decremented >= 0; |
66 |
| - // We might have blocked all competing pollAndSpawn calls. This could happen for example when |
67 |
| - // maxRunningTasks=1 and a task got enqueued just after checking the queue but before decrementing. |
68 |
| - // To be sure, return only if the queue is still empty. If the queue is not empty, this might be the |
69 |
| - // only pollAndSpawn call in progress, and returning without peeking would risk ending up with a |
70 |
| - // non-empty queue and no workers! |
71 |
| - if (tasks.peek() == null) break; |
72 |
| - } else { |
73 |
| - executor.execute(new AbstractRunnable() { |
74 |
| - private boolean rejected; // need not be volatile - if we're rejected then that happens-before calling onAfter |
75 |
| - |
76 |
| - @Override |
77 |
| - public boolean isForceExecution() { |
78 |
| - return task.isForceExecution(); |
79 |
| - } |
80 |
| - |
81 |
| - @Override |
82 |
| - public void onRejection(Exception e) { |
83 |
| - logger.trace("[{}] task {} rejected", taskRunnerName, task); |
84 |
| - rejected = true; |
85 |
| - task.onRejection(e); |
86 |
| - } |
87 |
| - |
88 |
| - @Override |
89 |
| - public void onFailure(Exception e) { |
90 |
| - logger.trace(() -> Strings.format("[%s] task %s failed", taskRunnerName, task), e); |
91 |
| - task.onFailure(e); |
92 |
| - } |
93 |
| - |
94 |
| - @Override |
95 |
| - protected void doRun() throws Exception { |
96 |
| - logger.trace("[{}] running task {}", taskRunnerName, task); |
97 |
| - task.doRun(); |
98 |
| - } |
99 |
| - |
100 |
| - @Override |
101 |
| - public void onAfter() { |
102 |
| - try { |
103 |
| - task.onAfter(); |
104 |
| - } finally { |
105 |
| - // To avoid missing to run tasks that are enqueued and waiting, we check the queue again once running |
106 |
| - // a task is finished. |
107 |
| - int decremented = runningTasks.decrementAndGet(); |
108 |
| - assert decremented >= 0; |
109 |
| - |
110 |
| - if (rejected == false) { |
111 |
| - pollAndSpawn(); |
112 |
| - } |
113 |
| - } |
114 |
| - } |
115 |
| - |
116 |
| - @Override |
117 |
| - public String toString() { |
118 |
| - return task.toString(); |
119 |
| - } |
120 |
| - }); |
| 43 | + @Override |
| 44 | + public String toString() { |
| 45 | + return task.toString(); |
| 46 | + } |
| 47 | + |
| 48 | + @Override |
| 49 | + public void onResponse(Releasable releasable) { |
| 50 | + try (releasable) { |
| 51 | + task.run(); |
| 52 | + } |
| 53 | + } |
| 54 | + |
| 55 | + @Override |
| 56 | + public void onFailure(Exception e) { |
| 57 | + assert e instanceof EsRejectedExecutionException : e; |
| 58 | + try { |
| 59 | + task.onRejection(e); |
| 60 | + } finally { |
| 61 | + task.onAfter(); |
121 | 62 | }
|
122 | 63 | }
|
123 | 64 | }
|
124 | 65 |
|
125 |
| - // Each worker thread that runs a task, first needs to get a "free slot" in order to respect maxRunningTasks. |
126 |
| - private boolean incrementRunningTasks() { |
127 |
| - int preUpdateValue = runningTasks.getAndAccumulate(maxRunningTasks, (v, maxRunning) -> v < maxRunning ? v + 1 : v); |
128 |
| - assert preUpdateValue <= maxRunningTasks; |
129 |
| - return preUpdateValue < maxRunningTasks; |
| 66 | + public PrioritizedThrottledTaskRunner(final String name, final int maxRunningTasks, final Executor executor) { |
| 67 | + this.queue = new PriorityBlockingQueue<>(); |
| 68 | + this.runner = new AbstractThrottledTaskRunner<>(name, maxRunningTasks, executor, queue); |
| 69 | + } |
| 70 | + |
| 71 | + /** |
| 72 | + * Submits a task for execution. If there are fewer than {@code maxRunningTasks} tasks currently running then this task is immediately |
| 73 | + * submitted to the executor. Otherwise this task is enqueued and will be submitted to the executor in turn on completion of some other |
| 74 | + * task. |
| 75 | + */ |
| 76 | + public void enqueueTask(final T task) { |
| 77 | + runner.enqueueTask(new TaskWrapper<>(task)); |
130 | 78 | }
|
131 | 79 |
|
132 | 80 | // Only use for testing
|
133 | 81 | public int runningTasks() {
|
134 |
| - return runningTasks.get(); |
| 82 | + return runner.runningTasks(); |
135 | 83 | }
|
136 | 84 |
|
137 | 85 | // Only use for testing
|
138 | 86 | public int queueSize() {
|
139 |
| - return tasks.size(); |
| 87 | + return queue.size(); |
140 | 88 | }
|
141 | 89 | }
|
0 commit comments