/
StormTimer.java
275 lines (245 loc) · 10 KB
/
StormTimer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
/**
* 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.storm;
import java.nio.channels.ClosedByInterruptException;
import java.util.Comparator;
import java.util.Random;
import java.util.concurrent.PriorityBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.storm.utils.Time;
import org.apache.storm.utils.Utils;
/**
* The timer defined in this file is very similar to java.util.Timer, except it integrates with Storm's time simulation capabilities. This
* lets us test code that does asynchronous work on the timer thread.
*/
public class StormTimer implements AutoCloseable {
//task to run
private StormTimerTask task = new StormTimerTask();
/**
* Makes a Timer in the form of a StormTimerTask Object.
*
* @param name name of the timer
* @param onKill function to call when timer is killed unexpectedly
*/
public StormTimer(String name, Thread.UncaughtExceptionHandler onKill) {
if (onKill == null) {
throw new RuntimeException("onKill func is null!");
}
if (name == null) {
this.task.setName("timer");
} else {
this.task.setName(name);
}
this.task.setOnKillFunc(onKill);
this.task.setActive(true);
this.task.setDaemon(true);
this.task.setPriority(Thread.MAX_PRIORITY);
this.task.start();
}
/**
* Schedule a function to be executed in the timer.
*
* @param delaySecs the number of seconds to delay before running the function
* @param func the function to run
* @param checkActive whether to check is the timer is active
* @param jitterMs add jitter to the run
*/
public void schedule(int delaySecs, Runnable func, boolean checkActive, int jitterMs) {
scheduleMs(Time.secsToMillisLong(delaySecs), func, checkActive, jitterMs);
}
public void schedule(int delaySecs, Runnable func) {
schedule(delaySecs, func, true, 0);
}
/**
* Same as schedule with millisecond resolution.
*
* @param delayMs the number of milliseconds to delay before running the function
* @param func the function to run
* @param checkActive whether to check is the timer is active
* @param jitterMs add jitter to the run
*/
public void scheduleMs(long delayMs, Runnable func, boolean checkActive, int jitterMs) {
if (func == null) {
throw new RuntimeException("function to schedule is null!");
}
if (checkActive) {
checkActive();
}
String id = Utils.uuid();
long endTimeMs = Time.currentTimeMillis() + delayMs;
if (jitterMs > 0) {
endTimeMs = this.task.random.nextInt(jitterMs) + endTimeMs;
}
task.add(new QueueEntry(endTimeMs, func, id));
}
public void scheduleMs(long delayMs, Runnable func) {
scheduleMs(delayMs, func, true, 0);
}
/**
* Schedule a function to run recurrently.
*
* @param delaySecs the number of seconds to delay before running the function
* @param recurSecs the time between each invocation
* @param func the function to run
*/
public void scheduleRecurring(int delaySecs, final int recurSecs, final Runnable func) {
schedule(delaySecs, new Runnable() {
@Override
public void run() {
func.run();
// This avoids a race condition with cancel-timer.
schedule(recurSecs, this, false, 0);
}
});
}
/**
* Schedule a function to run recurrently.
*
* @param delayMs the number of millis to delay before running the function
* @param recurMs the time between each invocation
* @param func the function to run
*/
public void scheduleRecurringMs(long delayMs, final long recurMs, final Runnable func) {
scheduleMs(delayMs, new Runnable() {
@Override
public void run() {
func.run();
// This avoids a race condition with cancel-timer.
scheduleMs(recurMs, this, true, 0);
}
});
}
/**
* Schedule a function to run recurrently with jitter.
*
* @param delaySecs the number of seconds to delay before running the function
* @param recurSecs the time between each invocation
* @param jitterMs jitter added to the run
* @param func the function to run
*/
public void scheduleRecurringWithJitter(int delaySecs, final int recurSecs, final int jitterMs, final Runnable func) {
schedule(delaySecs, new Runnable() {
@Override
public void run() {
func.run();
// This avoids a race condition with cancel-timer.
schedule(recurSecs, this, false, jitterMs);
}
});
}
/**
* check if timer is active.
*/
private void checkActive() {
if (!this.task.isActive()) {
throw new IllegalStateException("Timer is not active");
}
}
/**
* cancel timer.
*/
@Override
public void close() throws InterruptedException {
if (this.task.isActive()) {
this.task.setActive(false);
this.task.interrupt();
this.task.join();
}
}
/**
* is timer waiting. Used in timer simulation.
*/
public boolean isTimerWaiting() {
return Time.isThreadWaiting(task);
}
public static class QueueEntry {
public final Long endTimeMs;
public final Runnable func;
public final String id;
public QueueEntry(Long endTimeMs, Runnable func, String id) {
this.endTimeMs = endTimeMs;
this.func = func;
this.id = id;
}
}
public static class StormTimerTask extends Thread {
//initialCapacity set to 11 since its the default inital capacity of PriorityBlockingQueue
private PriorityBlockingQueue<QueueEntry> queue = new PriorityBlockingQueue<QueueEntry>(11, new Comparator<QueueEntry>() {
@Override
public int compare(QueueEntry o1, QueueEntry o2) {
return o1.endTimeMs.intValue() - o2.endTimeMs.intValue();
}
});
// boolean to indicate whether timer is active
private AtomicBoolean active = new AtomicBoolean(false);
// function to call when timer is killed
private Thread.UncaughtExceptionHandler onKill;
//random number generator
private Random random = new Random();
@Override
public void run() {
while (this.active.get()) {
QueueEntry queueEntry = null;
try {
queueEntry = this.queue.peek();
if ((queueEntry != null) && (Time.currentTimeMillis() >= queueEntry.endTimeMs)) {
// It is imperative to not run the function
// inside the timer lock. Otherwise, it is
// possible to deadlock if the fn deals with
// other locks, like the submit lock.
this.queue.remove(queueEntry);
queueEntry.func.run();
} else if (queueEntry != null) {
// If any events are scheduled, sleep until
// event generation. If any recurring events
// are scheduled then we will always go
// through this branch, sleeping only the
// exact necessary amount of time. We give
// an upper bound, e.g. 1000 millis, to the
// sleeping time, to limit the response time
// for detecting any new event within 1 secs.
Time.sleep(Math.min(1000, (queueEntry.endTimeMs - Time.currentTimeMillis())));
} else {
// Otherwise poll to see if any new event
// was scheduled. This is, in essence, the
// response time for detecting any new event
// schedulings when there are no scheduled
// events.
Time.sleep(1000);
}
if (Thread.interrupted()) {
this.active.set(false);
}
} catch (Throwable e) {
if (!(Utils.exceptionCauseIsInstanceOf(InterruptedException.class, e))
&& !(Utils.exceptionCauseIsInstanceOf(ClosedByInterruptException.class, e))) {
// need to set active false before calling onKill() - current implementation does not return.
this.setActive(false);
this.onKill.uncaughtException(this, e);
}
}
}
}
public void setOnKillFunc(Thread.UncaughtExceptionHandler onKill) {
this.onKill = onKill;
}
public boolean isActive() {
return this.active.get();
}
public void setActive(boolean flag) {
this.active.set(flag);
}
public void add(QueueEntry queueEntry) {
this.queue.add(queueEntry);
}
}
}