/
BaseWindowedBolt.java
392 lines (342 loc) · 13.5 KB
/
BaseWindowedBolt.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
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
/**
* 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.topology.base;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.storm.Config;
import org.apache.storm.task.OutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.IWindowedBolt;
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.topology.TupleFieldTimestampExtractor;
import org.apache.storm.windowing.TimestampExtractor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public abstract class BaseWindowedBolt implements IWindowedBolt {
private static final Logger LOG = LoggerFactory.getLogger(BaseWindowedBolt.class);
protected final transient Map<String, Object> windowConfiguration;
protected TimestampExtractor timestampExtractor;
protected BaseWindowedBolt() {
windowConfiguration = new HashMap<>();
}
private BaseWindowedBolt withWindowLength(Count count) {
if (count == null) {
throw new IllegalArgumentException("Window length count cannot be set null");
}
if (count.value <= 0) {
throw new IllegalArgumentException("Window length must be positive [" + count + "]");
}
windowConfiguration.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, count.value);
return this;
}
private BaseWindowedBolt withWindowLength(Duration duration) {
if (duration == null) {
throw new IllegalArgumentException("Window length duration cannot be set null");
}
if (duration.value <= 0) {
throw new IllegalArgumentException("Window length must be positive [" + duration + "]");
}
windowConfiguration.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS, duration.value);
return this;
}
private BaseWindowedBolt withSlidingInterval(Count count) {
if (count == null) {
throw new IllegalArgumentException("Sliding interval count cannot be set null");
}
if (count.value <= 0) {
throw new IllegalArgumentException("Sliding interval must be positive [" + count + "]");
}
windowConfiguration.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, count.value);
return this;
}
private BaseWindowedBolt withSlidingInterval(Duration duration) {
if (duration == null) {
throw new IllegalArgumentException("Sliding interval duration cannot be set null");
}
if (duration.value <= 0) {
throw new IllegalArgumentException("Sliding interval must be positive [" + duration + "]");
}
windowConfiguration.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS, duration.value);
return this;
}
/**
* Tuple count based sliding window configuration.
*
* @param windowLength the number of tuples in the window
* @param slidingInterval the number of tuples after which the window slides
*/
public BaseWindowedBolt withWindow(Count windowLength, Count slidingInterval) {
return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
}
/**
* Tuple count and time duration based sliding window configuration.
*
* @param windowLength the number of tuples in the window
* @param slidingInterval the time duration after which the window slides
*/
public BaseWindowedBolt withWindow(Count windowLength, Duration slidingInterval) {
return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
}
/**
* Time duration and count based sliding window configuration.
*
* @param windowLength the time duration of the window
* @param slidingInterval the number of tuples after which the window slides
*/
public BaseWindowedBolt withWindow(Duration windowLength, Count slidingInterval) {
return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
}
/**
* Time duration based sliding window configuration.
*
* @param windowLength the time duration of the window
* @param slidingInterval the time duration after which the window slides
*/
public BaseWindowedBolt withWindow(Duration windowLength, Duration slidingInterval) {
return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
}
/**
* A tuple count based window that slides with every incoming tuple.
*
* @param windowLength the number of tuples in the window
*/
public BaseWindowedBolt withWindow(Count windowLength) {
return withWindowLength(windowLength).withSlidingInterval(new Count(1));
}
/**
* A time duration based window that slides with every incoming tuple.
*
* @param windowLength the time duration of the window
*/
public BaseWindowedBolt withWindow(Duration windowLength) {
return withWindowLength(windowLength).withSlidingInterval(new Count(1));
}
/**
* A count based tumbling window.
*
* @param count the number of tuples after which the window tumbles
*/
public BaseWindowedBolt withTumblingWindow(Count count) {
return withWindowLength(count).withSlidingInterval(count);
}
/**
* A time duration based tumbling window.
*
* @param duration the time duration after which the window tumbles
*/
public BaseWindowedBolt withTumblingWindow(Duration duration) {
return withWindowLength(duration).withSlidingInterval(duration);
}
/**
* Specify a field in the tuple that represents the timestamp as a long value. If this field is not present in the incoming tuple, an
* {@link IllegalArgumentException} will be thrown. The field MUST contain a timestamp in milliseconds
*
* @param fieldName the name of the field that contains the timestamp
*/
public BaseWindowedBolt withTimestampField(String fieldName) {
return withTimestampExtractor(TupleFieldTimestampExtractor.of(fieldName));
}
/**
* Specify the timestamp extractor implementation.
*
* @param timestampExtractor the {@link TimestampExtractor} implementation
*/
public BaseWindowedBolt withTimestampExtractor(TimestampExtractor timestampExtractor) {
if (timestampExtractor == null) {
throw new IllegalArgumentException("Timestamp extractor cannot be set to null");
}
if (this.timestampExtractor != null) {
throw new IllegalArgumentException("Window is already configured with a timestamp extractor: " + timestampExtractor);
}
this.timestampExtractor = timestampExtractor;
return this;
}
@Override
public TimestampExtractor getTimestampExtractor() {
return timestampExtractor;
}
/**
* Specify a stream id on which late tuples are going to be emitted. They are going to be accessible via the {@link
* org.apache.storm.topology.WindowedBoltExecutor#LATE_TUPLE_FIELD} field. It must be defined on a per-component basis, and in
* conjunction with the {@link BaseWindowedBolt#withTimestampField}, otherwise {@link IllegalArgumentException} will be thrown.
*
* @param streamId the name of the stream used to emit late tuples on
*/
public BaseWindowedBolt withLateTupleStream(String streamId) {
if (streamId == null) {
throw new IllegalArgumentException("Cannot set late tuple stream id to null");
}
windowConfiguration.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, streamId);
return this;
}
/**
* Specify the maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps cannot be out of order by
* more than this amount.
*
* @param duration the max lag duration
*/
public BaseWindowedBolt withLag(Duration duration) {
windowConfiguration.put(Config.TOPOLOGY_BOLTS_TUPLE_TIMESTAMP_MAX_LAG_MS, duration.value);
return this;
}
/**
* Specify the watermark event generation interval. For tuple based timestamps, watermark events are used to track the progress of time
*
* @param interval the interval at which watermark events are generated
*/
public BaseWindowedBolt withWatermarkInterval(Duration interval) {
if (interval == null) {
throw new IllegalArgumentException("Watermark interval cannot be set null");
}
windowConfiguration.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, interval.value);
return this;
}
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
// NOOP
}
@Override
public void cleanup() {
// NOOP
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
// NOOP
}
@Override
public Map<String, Object> getComponentConfiguration() {
return windowConfiguration;
}
/**
* Holds a count value for count based windows and sliding intervals.
*/
public static class Count implements Serializable {
public final int value;
public Count(int value) {
this.value = value;
}
/**
* Returns a {@link Count} of given value.
*
* @param value the count value
* @return the Count
*/
public static Count of(int value) {
return new Count(value);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Count count = (Count) o;
return value == count.value;
}
@Override
public int hashCode() {
return value;
}
@Override
public String toString() {
return "Count{" +
"value=" + value +
'}';
}
}
/**
* Holds a Time duration for time based windows and sliding intervals.
*/
public static class Duration implements Serializable {
public final int value;
public Duration(int value, TimeUnit timeUnit) {
if (value < 0) {
throw new IllegalArgumentException("Duration cannot be negative");
}
long longVal = timeUnit.toMillis(value);
if (longVal > (long) Integer.MAX_VALUE) {
throw new IllegalArgumentException("Duration is too long");
}
this.value = (int) longVal;
}
/**
* Returns a {@link Duration} corresponding to the the given value in milli seconds.
*
* @param milliseconds the duration in milliseconds
* @return the Duration
*/
public static Duration of(int milliseconds) {
return new Duration(milliseconds, TimeUnit.MILLISECONDS);
}
/**
* Returns a {@link Duration} corresponding to the the given value in days.
*
* @param days the number of days
* @return the Duration
*/
public static Duration days(int days) {
return new Duration(days, TimeUnit.DAYS);
}
/**
* Returns a {@link Duration} corresponding to the the given value in hours.
*
* @param hours the number of hours
* @return the Duration
*/
public static Duration hours(int hours) {
return new Duration(hours, TimeUnit.HOURS);
}
/**
* Returns a {@link Duration} corresponding to the the given value in minutes.
*
* @param minutes the number of minutes
* @return the Duration
*/
public static Duration minutes(int minutes) {
return new Duration(minutes, TimeUnit.MINUTES);
}
/**
* Returns a {@link Duration} corresponding to the the given value in seconds.
*
* @param seconds the number of seconds
* @return the Duration
*/
public static Duration seconds(int seconds) {
return new Duration(seconds, TimeUnit.SECONDS);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Duration duration = (Duration) o;
return value == duration.value;
}
@Override
public int hashCode() {
return value;
}
@Override
public String toString() {
return "Duration{" +
"value=" + value +
'}';
}
}
}