22
22
import com .facebook .presto .spi .eventlistener .QueryProgressEvent ;
23
23
import com .facebook .presto .spi .eventlistener .QueryUpdatedEvent ;
24
24
import com .facebook .presto .spi .eventlistener .SplitCompletedEvent ;
25
+ import com .google .common .collect .ImmutableList ;
25
26
import com .google .common .collect .ImmutableMap ;
26
27
import com .google .inject .Inject ;
27
28
28
- import java .util .Optional ;
29
+ import java .util .List ;
29
30
import java .util .concurrent .atomic .AtomicReference ;
30
31
31
32
public class TestingEventListenerManager
32
33
extends EventListenerManager
33
34
{
34
- private final AtomicReference <Optional <EventListener >> configuredEventListener = new AtomicReference <>(Optional . empty ());
35
+ private final AtomicReference <List <EventListener >> configuredEventListeners = new AtomicReference <>(ImmutableList . of ());
35
36
36
37
@ Inject
37
38
public TestingEventListenerManager (EventListenerConfig config )
@@ -42,51 +43,46 @@ public TestingEventListenerManager(EventListenerConfig config)
42
43
@ Override
43
44
public void addEventListenerFactory (EventListenerFactory eventListenerFactory )
44
45
{
45
- configuredEventListener .set (Optional .of (eventListenerFactory .create (ImmutableMap .of ())));
46
+ configuredEventListeners .set (ImmutableList .of (eventListenerFactory .create (ImmutableMap .of ())));
46
47
}
47
48
48
49
@ Override
49
50
public void queryCompleted (QueryCompletedEvent queryCompletedEvent )
50
51
{
51
- if (configuredEventListener .get ().isPresent ()) {
52
- configuredEventListener .get ().get ().queryCompleted (queryCompletedEvent );
53
- }
52
+ configuredEventListeners .get ()
53
+ .forEach (eventListener -> eventListener .queryCompleted (queryCompletedEvent ));
54
54
}
55
55
56
56
@ Override
57
57
public void queryCreated (QueryCreatedEvent queryCreatedEvent )
58
58
{
59
- if (configuredEventListener .get ().isPresent ()) {
60
- configuredEventListener .get ().get ().queryCreated (queryCreatedEvent );
61
- }
59
+ configuredEventListeners .get ()
60
+ .forEach (eventListener -> eventListener .queryCreated (queryCreatedEvent ));
62
61
}
63
62
64
63
@ Override
65
64
public void queryUpdated (QueryUpdatedEvent queryUpdatedEvent )
66
65
{
67
- if (configuredEventListener .get ().isPresent ()) {
68
- configuredEventListener .get ().get ().queryUpdated (queryUpdatedEvent );
69
- }
66
+ configuredEventListeners .get ()
67
+ .forEach (eventListener -> eventListener .queryUpdated (queryUpdatedEvent ));
70
68
}
71
69
72
70
@ Override
73
71
public void publishQueryProgress (QueryProgressEvent queryProgressEvent )
74
72
{
75
- if (configuredEventListener .get ().isPresent ()) {
76
- configuredEventListener .get ().get ().publishQueryProgress (queryProgressEvent );
77
- }
73
+ configuredEventListeners .get ()
74
+ .forEach (eventListener -> eventListener .publishQueryProgress (queryProgressEvent ));
78
75
}
79
76
80
77
@ Override
81
78
public void splitCompleted (SplitCompletedEvent splitCompletedEvent )
82
79
{
83
- if (configuredEventListener .get ().isPresent ()) {
84
- configuredEventListener .get ().get ().splitCompleted (splitCompletedEvent );
85
- }
80
+ configuredEventListeners .get ()
81
+ .forEach (eventListener -> eventListener .splitCompleted (splitCompletedEvent ));
86
82
}
87
83
88
- public Optional <EventListener > getEventListener ()
84
+ public List <EventListener > getEventListeners ()
89
85
{
90
- return configuredEventListener .get ();
86
+ return configuredEventListeners .get ();
91
87
}
92
88
}
0 commit comments