This repository has been archived by the owner on Feb 8, 2019. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 18
/
ProcessingElement.java
903 lines (776 loc) · 32.3 KB
/
ProcessingElement.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
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
/**
* 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.s4.core;
import java.lang.Thread.UncaughtExceptionHandler;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Map;
import java.util.TimerTask;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import net.jcip.annotations.ThreadSafe;
import org.apache.s4.base.Event;
import org.apache.s4.core.ft.CheckpointId;
import org.apache.s4.core.ft.CheckpointingConfig;
import org.apache.s4.core.ft.CheckpointingConfig.CheckpointingMode;
import org.apache.s4.core.ft.CheckpointingTask;
import org.apache.s4.core.gen.OverloadDispatcher;
import org.apache.s4.core.gen.OverloadDispatcherGenerator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Preconditions;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.MapMaker;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.yammer.metrics.Metrics;
import com.yammer.metrics.core.Timer;
/**
* <p>
* Base class for implementing processing in S4. All instances are organized as follows:
* <ul>
* <li>A PE prototype is a special type of instance that, along with {@link Stream} defines the topology of the
* application graph.
* <li>PE prototypes manage the creation and destruction of PE instances.
* <li>All PE instances are clones of a PE prototype.
* <li>PE instances are associated with a unique key.
* <li>PE instances do the actual work by processing any number of input events of various types and emit output events
* of various types.
* <li>To process events, {@code ProcessingElement} dynamically matches an event type to a processing method. See
* {@link org.apache.s4.core.gen.OverloadDispatcher} . There are two types of processing methods:
* <ul>
* <li>{@code onEvent(SomeEvent event)} When implemented, input events of type {@code SomeEvent} will be dispatched to
* this method.
* <li>{@code onTrigger(AnotherEvent event)} When implemented, input events of type {@code AnotherEvent} will be
* dispatched to this method when certain conditions are met. See {@link #setTrigger(Class, int, long, TimeUnit)}.
* </ul>
* <li>
* A PE implementation must not create threads. A periodic task can be implemented by overloading the {@link #onTime()}
* method. See {@link #setTimerInterval(long, TimeUnit)}
* <li>If a reference in the PE prototype shared by the PE instances, the object must be thread safe.
* <li>The code in a PE instance is synchronized by the framework to avoid concurrency problems.
* <li>In some special cases, it may be desirable to allow concurrency in the PE instance. For example, there may be
* several event processing methods that can safely run concurrently. To enable concurrency, annotate the implementation
* of {@code ProcessingElement} with {@link ThreadSafe}.
* <li>PE instances never use the constructor. They must be initialized by implementing the {@link #onCreate()} method.
* <li>PE class fields are cloned from the prototype. References are also copied which means that if the prototype
* creates a collection object, all instances will be sharing the same collection object which is usually <em>NOT</em>
* what the programmer intended . The application developer is responsible for initializing objects in the
* {@link #onCreate()} method. For example, if each instance requires a
* <tt>List<tt/> object the PE should implement the following:
* <pre>
* public class MyPE extends ProcessingElement {
*
* private Map<String, Integer> wordCount;
*
* ...
*
* onCreate() {
* wordCount = new HashMap<String, Integer>;
* logger.trace("Created a map for instance PE with id {}, getId());
* }
* }
* </pre>
*
*
* </ul>
*
*
*
*
*/
public abstract class ProcessingElement implements Cloneable {
transient private static final Logger logger = LoggerFactory.getLogger(ProcessingElement.class);
transient private static final String SINGLETON = "singleton";
transient protected App app;
/*
* This maps holds all the instances. We make it package private to prevent concrete classes from updating the
* collection.
*/
transient LoadingCache<String, ProcessingElement> peInstances;
/* This map is initialized in the prototype and cloned to instances. */
transient Map<Class<? extends Event>, Trigger> triggers;
/* PE instance id. */
protected String id = "";
/* Private fields. */
transient private ProcessingElement pePrototype;
transient private boolean haveTriggers = false;
transient private long timerIntervalInMilliseconds = 0;
transient private ScheduledExecutorService triggerTimer;
transient private ScheduledExecutorService checkpointingTimer;
transient private boolean isPrototype = true;
transient private boolean isThreadSafe = false;
transient private String name = null;
transient private boolean isSingleton = false;
transient long eventCount = 0;
transient private OverloadDispatcher overloadDispatcher;
transient private boolean recoveryAttempted = false;
transient private boolean dirty = false;
transient private Timer processingTimer;
transient private CheckpointingConfig checkpointingConfig = new CheckpointingConfig.Builder(CheckpointingMode.NONE)
.build();
protected ProcessingElement() {
OverloadDispatcherGenerator oldg = new OverloadDispatcherGenerator(this.getClass());
Class<?> overloadDispatcherClass = oldg.generate();
try {
overloadDispatcher = (OverloadDispatcher) overloadDispatcherClass.newInstance();
} catch (Exception e) {
throw new RuntimeException(e);
}
peInstances = CacheBuilder.newBuilder().build(new CacheLoader<String, ProcessingElement>() {
@Override
public ProcessingElement load(String key) throws Exception {
return createPE(key);
}
});
triggers = new MapMaker().makeMap();
/*
* Only the PE Prototype uses the constructor. The PEPrototype field will be cloned by the instances and point
* to the prototype.
*/
this.pePrototype = this;
processingTimer = Metrics.newTimer(getClass(), getClass().getName() + "-pe-processing-time");
}
/**
* Create a PE prototype. By default, PE instances will never expire. Use {@code #configurePECache} to configure.
*
* @param app
* the app that contains this PE
*/
public ProcessingElement(App app) {
this();
setApp(app);
}
/**
* This method is called by the PE timer. By default it is synchronized with the onEvent() and onTrigger() methods.
* To execute concurrently with other methods, the {@link ProcessingElement} subclass must be annotated with
* {@link ThreadSafe}.
*
* Override this method to implement a periodic process.
*/
protected void onTime() {
}
/**
* This method is called after a PE instance is created. Use it to initialize fields that are PE instance specific.
* PE instances are created using {#clone()}.
*
* <p>
* <b>Fields initialized in the class constructor are shared by all PE instances.</b>
* </p>
*/
abstract protected void onCreate();
/**
* This method is called before a PE instance is removed. Use it to close resources and clean up.
*/
abstract protected void onRemove();
/**
* PE objects must be associated with one and only one {@code App} object.
*
* @return the app
*/
public App getApp() {
return app;
}
public void setApp(App app) {
if (this.app != null) {
throw new RuntimeException("Application was already assigne to this processing element");
}
this.app = app;
app.addPEPrototype(this, null);
}
/**
* Returns the approximate number of PE instances from the cache.
*
* @return the approximate number of PE instances.
*/
public long getNumPEInstances() {
return peInstances.size();
}
public Map<String, ProcessingElement> getPEInstances() {
return peInstances.asMap();
}
/**
* Set PE expiration and cache size.
* <p>
* PE instances will be automatically removed from the cache once a fixed duration has elapsed after the PEs
* creation, or last access.
* <p>
* Least accessed PEs will automatically be removed from the cache when the number of PEs approaches maximumSize.
* <p>
* When this method is called all existing PE instances are destroyed.
*
*
* @param maximumSize
* the approximate maximum number of PEs in the cache.
* @param duration
* the PE duration
* @param timeUnit
* the time unit
* @return the PE prototype
*/
public ProcessingElement setPECache(int maximumSize, long duration, TimeUnit timeUnit) {
Preconditions.checkArgument(isPrototype, "This method can only be used on the PE prototype. Trigger not set.");
peInstances = CacheBuilder.newBuilder().expireAfterAccess(duration, timeUnit).maximumSize(maximumSize)
.build(new CacheLoader<String, ProcessingElement>() {
@Override
public ProcessingElement load(String key) throws Exception {
return createPE(key);
}
});
return this;
}
/**
* Sets the max size of the PE cache.
*
* <p>
* Least accessed PEs will automatically be removed from the cache when the number of PEs approaches maximumSize.
* <p>
* When this method is called all existing PE instances are destroyed.
*
*
* @param maximumSize
* the approximate maximum number of PEs in the cache.
* @return the PE prototype
*/
public ProcessingElement setPECache(int maximumSize) {
Preconditions.checkArgument(isPrototype, "This method can only be used on the PE prototype. Trigger not set.");
peInstances = CacheBuilder.newBuilder().maximumSize(maximumSize)
.build(new CacheLoader<String, ProcessingElement>() {
@Override
public ProcessingElement load(String key) throws Exception {
return createPE(key);
}
});
return this;
}
/**
* This trigger is fired when the following conditions occur:
*
* <ul>
* <li>An event of eventType arrived to the PE instance
* <li>numEvents have arrived since the last time this trigger was fired -OR- time since last event is greater than
* interval.
* </ul>
*
* <p>
* When the trigger fires, the method <tt>trigger(EventType event)</tt> is called. Where <tt>EventType</tt> matches
* the argument eventType.
*
* @param eventType
* the type of event on which this trigger will fire.
* @param numEvents
* number of events since last trigger activation. Must be greater than zero. (Set to one to trigger on
* every input event.)
* @param interval
* minimum time between triggers. Set to zero if no time interval needed.
* @param timeUnit
* the TimeUnit for the argument interval. Can set to null if no time interval needed.
* @return the PE prototype
*/
public ProcessingElement setTrigger(Class<? extends Event> eventType, int numEvents, long interval,
TimeUnit timeUnit) {
Preconditions.checkArgument(isPrototype, "This method can only be used on the PE prototype. Trigger not set.");
Preconditions.checkNotNull(eventType, "Need eventType to set trigger.");
Preconditions.checkArgument(numEvents > 0 || interval > 0,
"To set trigger numEvent OR interval must be greater than zero.");
Preconditions.checkArgument(timeUnit != null || interval < 1,
"To set trigger timeUnit is needed when interval is greater than zero.");
/* Skip trigger checking overhead if there are no triggers. */
haveTriggers = true;
if (timeUnit != null && timeUnit != TimeUnit.MILLISECONDS) {
interval = timeUnit.convert(interval, TimeUnit.MILLISECONDS);
}
Trigger config = new Trigger(numEvents, interval);
triggers.put(eventType, config);
return this;
}
/**
* @return the isSingleton
*/
public boolean isSingleton() {
return isSingleton;
}
/**
* Makes this PE a singleton. A single PE instance is eagerly created and ready to receive events.
*
* @param isSingleton
* @throws ExecutionException
*/
public ProcessingElement setSingleton(boolean isSingleton) {
if (!isPrototype) {
logger.warn("This method can only be used on the PE prototype.");
return this;
}
this.isSingleton = isSingleton;
return this;
}
/**
* The duration of the periodic task controlled by the embedded timer.
*
* @param timeUnit
* the timeUnt of the returned value.
* @return the timer interval.
*/
public long getTimerInterval(TimeUnit timeUnit) {
return timeUnit.convert(timerIntervalInMilliseconds, TimeUnit.MILLISECONDS);
}
/**
* Set a timer that calls {@link #onTime()}.
*
* If {@code interval==0} the timer is disabled.
*
* @param interval
* in timeUnit
* @param timeUnit
* the timeUnit of interval
*/
public ProcessingElement setTimerInterval(long interval, TimeUnit timeUnit) {
timerIntervalInMilliseconds = TimeUnit.MILLISECONDS.convert(interval, timeUnit);
Preconditions.checkArgument(isPrototype, "This method can only be used on the PE prototype. Trigger not set.");
if (triggerTimer != null) {
triggerTimer.shutdownNow();
}
if (interval == 0) {
return this;
}
ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true)
.setUncaughtExceptionHandler(new UncaughtExceptionHandler() {
@Override
public void uncaughtException(Thread t, Throwable e) {
logger.error("Expection from timer thread", e);
}
}).setNameFormat("Timer-" + getClass().getSimpleName()).build();
triggerTimer = Executors.newSingleThreadScheduledExecutor(threadFactory);
return this;
}
/**
* Set to true if the concrete PE class has the {@link ThreadSafe} annotation. The default is false (no annotation).
* In general, application developers don't need to worry about thread safety in the concrete PEs. In some cases the
* PE needs to be thread safe to avoid deadlocks. For example , if the application graph has cycles and the queues
* are allowed to block, then some critical PEs with multiple incoming streams need to be made thread safe to avoid
* locking the entire PE instance.
*
* @return true if the PE implementation is considered thread safe.
*/
public boolean isThreadSafe() {
return isThreadSafe;
}
protected void handleInputEvent(Event event) {
// TimerContext timerContext = processingTimer.time();
Object object;
if (isThreadSafe) {
object = new Object(); // a dummy object TODO improve this.
} else {
object = this;
}
synchronized (object) {
if (!recoveryAttempted) {
recover();
recoveryAttempted = true;
}
/* Dispatch onEvent() method. */
overloadDispatcher.dispatchEvent(this, event);
/* Dispatch onTrigger() method. */
if (haveTriggers && isTrigger(event)) {
overloadDispatcher.dispatchTrigger(this, event);
}
eventCount++;
dirty = true;
if (isCheckpointable()) {
checkpoint();
}
}
// timerContext.stop();
}
protected boolean isCheckpointable() {
return getApp().checkpointingFramework.isCheckpointable(this);
}
public void checkpoint() {
getApp().getCheckpointingFramework().saveState(this);
clearDirty();
}
private boolean isTrigger(Event event) {
return isTrigger(event, event.getClass());
}
/**
* Checks the trigger for this event type. Creates an inactive trigger if no trigger is found after recursively
* exploring the event class hierarchy. An inactive trigger never triggers.
*
* @return true if trigger is reached, false if trigger is not ready yet or if trigger is inactive
*
*/
private boolean isTrigger(Event event, Class<?> triggerClass) {
/* Check if there is a trigger for this event type. Create an */
Trigger trigger = triggers.get(triggerClass);
if (trigger == null) {
if (!Event.class.isAssignableFrom(triggerClass)) {
// reached termination condition
triggers.put(event.getClass(), new Trigger());
return false;
} else {
// further explore hierarchy
return isTrigger(event, triggerClass.getSuperclass());
}
} else {
/*
* Check if it is time to activate the trigger for this event type.
*/
return trigger.checkAndUpdate();
}
}
private void removeInstanceForKeyInternal(String id) {
if (id == null)
return;
/* First let the PE instance clean after itself. */
onRemove();
/* Remove PE instance. */
peInstances.invalidate(id);
}
protected void removeAll() {
/* Close resources in prototype. */
if (triggerTimer != null) {
triggerTimer.shutdownNow();
logger.info("Trigger timer stopped.");
}
/* Remove all the instances. */
peInstances.invalidateAll();
}
protected void close() {
removeInstanceForKeyInternal(id);
}
private ProcessingElement createPE(String id) {
ProcessingElement pe = (ProcessingElement) this.clone();
pe.isPrototype = false;
pe.id = id;
pe.triggers = Maps.newHashMap(triggers);
pe.onCreate();
logger.trace("Num PE instances: {}.", getNumPEInstances());
return pe;
}
/* This method is called by App just before the application starts. */
protected void initPEPrototypeInternal() {
/* Eagerly create singleton PE. */
if (isSingleton) {
try {
peInstances.get(SINGLETON);
logger.trace("Created singleton [{}].", getInstanceForKey(SINGLETON));
} catch (ExecutionException e) {
logger.error("Problem when trying to create a PE instance.", e);
}
}
/* Start timer. */
if (triggerTimer != null) {
triggerTimer.scheduleAtFixedRate(new OnTimeTask(), 0, timerIntervalInMilliseconds, TimeUnit.MILLISECONDS);
logger.debug("Started timer for PE prototype [{}], ID [{}] with interval [{}].", new String[] {
this.getClass().getName(), id, String.valueOf(timerIntervalInMilliseconds) });
}
if (checkpointingConfig.mode == CheckpointingMode.TIME) {
ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true)
.setUncaughtExceptionHandler(new UncaughtExceptionHandler() {
@Override
public void uncaughtException(Thread t, Throwable e) {
logger.error("Expection from checkpointing thread", e);
}
}).setNameFormat("Checkpointing-trigger-" + getClass().getSimpleName()).build();
checkpointingTimer = Executors.newSingleThreadScheduledExecutor(threadFactory);
checkpointingTimer.scheduleAtFixedRate(new CheckpointingTask(this), checkpointingConfig.frequency,
checkpointingConfig.frequency, checkpointingConfig.timeUnit);
logger.debug(
"Started checkpointing timer for PE prototype [{}], ID [{}] with interval [{}] [{}].",
new String[] { this.getClass().getName(), id, String.valueOf(checkpointingConfig.frequency),
String.valueOf(checkpointingConfig.timeUnit.toString()) });
}
/* Check if this PE is annotated as thread safe. */
if (getClass().isAnnotationPresent(ThreadSafe.class) == true) {
// TODO: this doesn't seem to be working. isannotationpresent always returns false.
isThreadSafe = true;
logger.trace("Annotated with @ThreadSafe");
}
}
/**
* This method is designed to be used within the package. We make it package-private. The returned instances are all
* in the same JVM. Do not use it to access remote objects.
*
* @throws ExecutionException
*/
public ProcessingElement getInstanceForKey(String id) {
/* Check if instance for key exists, otherwise create one. */
try {
if (isSingleton) {
return peInstances.get(SINGLETON);
}
return peInstances.get(id);
} catch (ExecutionException e) {
logger.error("Problem when trying to create a PE instance for id {}", id, e);
}
return null;
}
/**
* Get all the local instances. See notes in {@link #getInstanceForKey(String) getLocalInstanceForKey}
*/
public Collection<ProcessingElement> getInstances() {
try {
if (isSingleton) {
return ImmutableList.of(peInstances.get(SINGLETON));
} else {
return peInstances.asMap().values();
}
} catch (ExecutionException e) {
logger.error("Problem when trying to create a PE instance for id {}", id, e);
return null;
}
}
/**
* This method returns a remote PE instance for key. TODO: not implemented for cluster configuration yet, use it
* only in single node configuration. for testing apps.
*
* @return pe instance for key. Null if if doesn't exist.
*/
public ProcessingElement getRemoteInstancesForKey() {
logger.warn("The getRemoteInstancesForKey() method is not implemented. Use "
+ "it to test your app in single node configuration only. Should work "
+ "transparently for remote objects once it is implemented.");
ProcessingElement pe = peInstances.asMap().get(id);
return pe;
}
/**
* This method returns an immutable map that contains all the PE instances for this prototype. PE instances may be
* located anywhere in the cluster. Be aware that this could be an expensive operation. TODO: not implemented for
* cluster configuration yet, use it only in single node configuration. for testing apps.
*/
public Map<String, ProcessingElement> getRemoteInstances() {
logger.warn("The getRemoteInstances() method is not implemented. Use "
+ "it to test your app in single node configuration only. Should work "
+ "transparently for remote objects once it is implemented.");
/*
* For now we just return a copy as a placeholder. We need to implement a custom map capable of working on an S4
* cluster as efficiently as possible.
*/
return ImmutableMap.copyOf(peInstances.asMap());
}
/**
* Unique ID for a PE instance.
*
* @return the id
*/
public String getId() {
return id;
}
/**
* The {@code ProcessingElement} prototype for this object.
*
* @return the corresponding {@code ProcessingElement} for this instance.
*/
public ProcessingElement getPrototype() {
return pePrototype;
}
/**
* This method exists simply to make <code>clone()</code> protected.
*/
@Override
protected Object clone() {
try {
Object clone = super.clone();
return clone;
} catch (CloneNotSupportedException e) {
throw new RuntimeException(e);
}
}
/**
* Helper method to be used by PE implementation classes. Sends an event to all the target streams.
*
*/
protected <T extends Event> void emit(T event, Stream<T>[] streamArray) {
for (int i = 0; i < streamArray.length; i++) {
streamArray[i].put(event);
}
}
private class OnTimeTask extends TimerTask {
@Override
public void run() {
for (Map.Entry<String, ProcessingElement> entry : getPEInstances().entrySet()) {
ProcessingElement peInstance = entry.getValue();
try {
if (isThreadSafe) {
peInstance.onTime();
} else {
synchronized (peInstance) {
peInstance.onTime();
}
}
} catch (Exception e) {
logger.error("Caught exception in timer when calling PE instance [{}] with id [{}].", peInstance,
peInstance.id);
logger.error("Timer error.", e);
}
}
}
}
/**
* @return the PE name
*/
protected String getName() {
return name;
}
/**
* @param name
* PE name
*/
protected void setName(String name) {
if (name == null)
return;
this.name = name;
if (app.peByName.containsKey(name)) {
logger.warn("Using a duplicate PE name: [{}]. This is probbaly not what you wanted.", name);
}
app.peByName.put(name, this);
}
public CheckpointingConfig getCheckpointingConfig() {
return checkpointingConfig;
}
public void setCheckpointingConfig(CheckpointingConfig checkpointingConfig) {
this.checkpointingConfig = checkpointingConfig;
}
/**
* By default, the state of a PE instance is considered dirty whenever it processed an event. Some event may
* actually leave the state of the PE unchanged. PE implementations can therefore override this method to
* accommodate specific behaviors, by managing a custom "dirty" flag.
*
* <b>If this method is overriden, {@link #clearDirty()} method must also be overriden in order to correctly reflect
* the "dirty" state of the PE.</b>
*/
public boolean isDirty() {
return dirty;
}
/**
* Dirty state is cleared after the PE has been serialized. PE implementations that maintain their "dirty" flag must
* override this method by clearing their internally managed "dirty" flag.
*
* <b>If this method is overriden, {@link #isDirty()} must also be overriden in order to correctly reflect the
* "dirty" state of the PE.</b>
*/
public void clearDirty() {
this.dirty = false;
}
public byte[] serializeState() {
return getApp().getSerDeser().serialize(this).array();
}
public ProcessingElement deserializeState(byte[] loadedState) {
return (ProcessingElement) getApp().getSerDeser().deserialize(ByteBuffer.wrap(loadedState));
}
public void restoreState(ProcessingElement oldState) {
restoreFieldsForClass(oldState.getClass(), oldState);
}
protected void recover() {
byte[] serializedState = null;
try {
serializedState = getApp().getCheckpointingFramework().fetchSerializedState(new CheckpointId(this));
} catch (RuntimeException e) {
logger.error("Cannot fetch serialized stated for [{}/{}]: {}", new String[] {
getPrototype().getClass().getName(), getId(), e.getMessage() });
}
if (serializedState == null) {
return;
}
try {
ProcessingElement peInOldState = deserializeState(serializedState);
restoreState(peInOldState);
} catch (RuntimeException e) {
logger.error("Cannot restore state for key [" + new CheckpointId(this) + "]: " + e.getMessage(), e);
}
}
private void restoreFieldsForClass(Class<?> currentInOldStateClassHierarchy, ProcessingElement oldState) {
if (!ProcessingElement.class.isAssignableFrom(currentInOldStateClassHierarchy)) {
return;
} else {
Field[] fields = oldState.getClass().getDeclaredFields();
for (Field field : fields) {
if (!Modifier.isTransient(field.getModifiers()) && !Modifier.isStatic(field.getModifiers())) {
if (!Modifier.isPublic(field.getModifiers())) {
field.setAccessible(true);
}
try {
// TODO use reflectasm
field.set(this, field.get(oldState));
} catch (IllegalArgumentException e) {
logger.error("Cannot recover old state for this PE [{}]", e);
return;
} catch (IllegalAccessException e) {
logger.error("Cannot recover old state for this PE [{}]", e);
return;
}
}
}
restoreFieldsForClass(currentInOldStateClassHierarchy.getSuperclass(), oldState);
}
}
class Trigger {
final long intervalInMilliseconds;
final int intervalInEvents;
long lastTime;
int eventCount;
// inactive triggers never trigger anything, they are used as markers
boolean active = true;
Trigger() {
this.intervalInEvents = 0;
this.intervalInMilliseconds = 0;
this.active = false;
}
Trigger(int intervalInEvents, long intervalInMilliseconds) {
this.intervalInEvents = intervalInEvents;
this.intervalInMilliseconds = intervalInMilliseconds;
}
boolean checkAndUpdate() {
if (active) {
long timeLapse = System.currentTimeMillis() - lastTime;
eventCount++;
lastTime = System.currentTimeMillis();
if (timeLapse > intervalInMilliseconds || eventCount >= intervalInEvents) {
eventCount = 0;
return true;
}
}
return false;
}
boolean isActive() {
return active;
}
}
public long getEventCount() {
return eventCount;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(getClass().getName() + "/" + getId() + " ;");
if (isSingleton) {
sb.append("singleton ;");
}
sb.append(isThreadSafe ? "IS thread-safe ;" : "Not thread-safe ;");
sb.append("timerInterval=" + timerIntervalInMilliseconds + " ;");
return sb.toString();
}
}