Skip to content

Commit d846344

Browse files
committed
Followup multiple eventlistener plugin changes (Fix tests).
A previous PR #24456 added the needed changes to support Multiple event listener in tests, query runners and across the codebase.
1 parent 7dc32b4 commit d846344

File tree

11 files changed

+50
-46
lines changed

11 files changed

+50
-46
lines changed

presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueries.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@
2424
import org.testng.annotations.AfterClass;
2525
import org.testng.annotations.Test;
2626

27-
import java.util.Optional;
27+
import java.util.List;
2828
import java.util.Set;
2929

3030
import static com.facebook.presto.SystemSessionProperties.CTE_MATERIALIZATION_STRATEGY;
@@ -60,10 +60,10 @@ protected boolean supportsNotNullColumns()
6060
public void close()
6161
throws Exception
6262
{
63-
Optional<EventListener> eventListener = getQueryRunner().getEventListener();
64-
assertTrue(eventListener.isPresent());
65-
assertTrue(eventListener.get() instanceof TestingHiveEventListener, eventListener.get().getClass().getName());
66-
Set<QueryId> runningQueryIds = ((TestingHiveEventListener) eventListener.get()).getRunningQueries();
63+
List<EventListener> eventListener = getQueryRunner().getEventListeners();
64+
assertTrue(!eventListener.isEmpty());
65+
assertTrue(eventListener.get(0) instanceof TestingHiveEventListener, eventListener.get(0).getClass().getName());
66+
Set<QueryId> runningQueryIds = ((TestingHiveEventListener) eventListener.get(0)).getRunningQueries();
6767

6868
if (!runningQueryIds.isEmpty()) {
6969
// Await query events to propagate and finish

presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplitScheduling.java

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,12 @@
2121
import com.google.common.collect.ImmutableMap;
2222
import org.testng.annotations.Test;
2323

24+
import java.util.List;
2425
import java.util.Optional;
2526

2627
import static com.facebook.presto.hive.HiveSessionProperties.DYNAMIC_SPLIT_SIZES_ENABLED;
2728
import static io.airlift.tpch.TpchTable.ORDERS;
29+
import static org.testng.Assert.assertFalse;
2830
import static org.testng.Assert.assertTrue;
2931

3032
@Test(singleThreaded = true)
@@ -72,10 +74,10 @@ public void testDynamicSplits()
7274

7375
private TestHiveEventListenerPlugin.TestingHiveEventListener getEventListener()
7476
{
75-
Optional<EventListener> eventListener = getQueryRunner().getEventListener();
76-
assertTrue(eventListener.isPresent());
77-
assertTrue(eventListener.get() instanceof TestHiveEventListenerPlugin.TestingHiveEventListener, eventListener.get().getClass().getName());
78-
return (TestHiveEventListenerPlugin.TestingHiveEventListener) eventListener.get();
77+
List<EventListener> eventListener = getQueryRunner().getEventListeners();
78+
assertFalse(eventListener.isEmpty());
79+
assertTrue(eventListener.get(0) instanceof TestHiveEventListenerPlugin.TestingHiveEventListener, eventListener.get(0).getClass().getName());
80+
return (TestHiveEventListenerPlugin.TestingHiveEventListener) eventListener.get(0);
7981
}
8082

8183
private Session dynamicSplitsSession()

presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -628,10 +628,10 @@ public StatsCalculator getStatsCalculator()
628628
return statsCalculator;
629629
}
630630

631-
public Optional<EventListener> getEventListener()
631+
public List<EventListener> getEventListeners()
632632
{
633633
checkState(coordinator, "not a coordinator");
634-
return eventListenerManager.getEventListener();
634+
return eventListenerManager.getEventListeners();
635635
}
636636

637637
public TestingAccessControlManager getAccessControl()

presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -713,9 +713,9 @@ public StatsCalculator getStatsCalculator()
713713
}
714714

715715
@Override
716-
public Optional<EventListener> getEventListener()
716+
public List<EventListener> getEventListeners()
717717
{
718-
return Optional.empty();
718+
return ImmutableList.of();
719719
}
720720

721721
public CostCalculator getCostCalculator()

presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -60,7 +60,16 @@ public interface QueryRunner
6060

6161
StatsCalculator getStatsCalculator();
6262

63-
Optional<EventListener> getEventListener();
63+
@Deprecated
64+
default Optional<EventListener> getEventListener()
65+
{
66+
if (getEventListeners().isEmpty()) {
67+
return Optional.empty();
68+
}
69+
return Optional.of(getEventListeners().get(0));
70+
}
71+
72+
List<EventListener> getEventListeners();
6473

6574
TestingAccessControlManager getAccessControl();
6675

presto-main/src/main/java/com/facebook/presto/testing/TestingEventListenerManager.java

Lines changed: 16 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -22,16 +22,17 @@
2222
import com.facebook.presto.spi.eventlistener.QueryProgressEvent;
2323
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
2424
import com.facebook.presto.spi.eventlistener.SplitCompletedEvent;
25+
import com.google.common.collect.ImmutableList;
2526
import com.google.common.collect.ImmutableMap;
2627
import com.google.inject.Inject;
2728

28-
import java.util.Optional;
29+
import java.util.List;
2930
import java.util.concurrent.atomic.AtomicReference;
3031

3132
public class TestingEventListenerManager
3233
extends EventListenerManager
3334
{
34-
private final AtomicReference<Optional<EventListener>> configuredEventListener = new AtomicReference<>(Optional.empty());
35+
private final AtomicReference<List<EventListener>> configuredEventListeners = new AtomicReference<>(ImmutableList.of());
3536

3637
@Inject
3738
public TestingEventListenerManager(EventListenerConfig config)
@@ -42,51 +43,46 @@ public TestingEventListenerManager(EventListenerConfig config)
4243
@Override
4344
public void addEventListenerFactory(EventListenerFactory eventListenerFactory)
4445
{
45-
configuredEventListener.set(Optional.of(eventListenerFactory.create(ImmutableMap.of())));
46+
configuredEventListeners.set(ImmutableList.of(eventListenerFactory.create(ImmutableMap.of())));
4647
}
4748

4849
@Override
4950
public void queryCompleted(QueryCompletedEvent queryCompletedEvent)
5051
{
51-
if (configuredEventListener.get().isPresent()) {
52-
configuredEventListener.get().get().queryCompleted(queryCompletedEvent);
53-
}
52+
configuredEventListeners.get()
53+
.forEach(eventListener -> eventListener.queryCompleted(queryCompletedEvent));
5454
}
5555

5656
@Override
5757
public void queryCreated(QueryCreatedEvent queryCreatedEvent)
5858
{
59-
if (configuredEventListener.get().isPresent()) {
60-
configuredEventListener.get().get().queryCreated(queryCreatedEvent);
61-
}
59+
configuredEventListeners.get()
60+
.forEach(eventListener -> eventListener.queryCreated(queryCreatedEvent));
6261
}
6362

6463
@Override
6564
public void queryUpdated(QueryUpdatedEvent queryUpdatedEvent)
6665
{
67-
if (configuredEventListener.get().isPresent()) {
68-
configuredEventListener.get().get().queryUpdated(queryUpdatedEvent);
69-
}
66+
configuredEventListeners.get()
67+
.forEach(eventListener -> eventListener.queryUpdated(queryUpdatedEvent));
7068
}
7169

7270
@Override
7371
public void publishQueryProgress(QueryProgressEvent queryProgressEvent)
7472
{
75-
if (configuredEventListener.get().isPresent()) {
76-
configuredEventListener.get().get().publishQueryProgress(queryProgressEvent);
77-
}
73+
configuredEventListeners.get()
74+
.forEach(eventListener -> eventListener.publishQueryProgress(queryProgressEvent));
7875
}
7976

8077
@Override
8178
public void splitCompleted(SplitCompletedEvent splitCompletedEvent)
8279
{
83-
if (configuredEventListener.get().isPresent()) {
84-
configuredEventListener.get().get().splitCompleted(splitCompletedEvent);
85-
}
80+
configuredEventListeners.get()
81+
.forEach(eventListener -> eventListener.splitCompleted(splitCompletedEvent));
8682
}
8783

88-
public Optional<EventListener> getEventListener()
84+
public List<EventListener> getEventListeners()
8985
{
90-
return configuredEventListener.get();
86+
return configuredEventListeners.get();
9187
}
9288
}

presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,6 @@
4545
import java.util.ArrayList;
4646
import java.util.List;
4747
import java.util.Map;
48-
import java.util.Optional;
4948
import java.util.concurrent.TimeUnit;
5049
import java.util.concurrent.locks.Lock;
5150
import java.util.logging.Logger;
@@ -223,7 +222,7 @@ public StatsCalculator getStatsCalculator()
223222
}
224223

225224
@Override
226-
public Optional<EventListener> getEventListener()
225+
public List<EventListener> getEventListeners()
227226
{
228227
throw new UnsupportedOperationException();
229228
}

presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -498,9 +498,9 @@ public StatsCalculator getStatsCalculator()
498498
}
499499

500500
@Override
501-
public Optional<EventListener> getEventListener()
501+
public List<EventListener> getEventListeners()
502502
{
503-
return Optional.empty();
503+
return ImmutableList.of();
504504
}
505505

506506
@Override

presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -607,10 +607,10 @@ public StatsCalculator getStatsCalculator()
607607
}
608608

609609
@Override
610-
public Optional<EventListener> getEventListener()
610+
public List<EventListener> getEventListeners()
611611
{
612612
checkState(coordinators.size() == 1, "Expected a single coordinator");
613-
return coordinators.get(0).getEventListener();
613+
return coordinators.get(0).getEventListeners();
614614
}
615615

616616
@Override

presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,6 @@
4242

4343
import java.util.List;
4444
import java.util.Map;
45-
import java.util.Optional;
4645
import java.util.Set;
4746
import java.util.concurrent.locks.Lock;
4847
import java.util.concurrent.locks.ReadWriteLock;
@@ -178,9 +177,9 @@ public StatsCalculator getStatsCalculator()
178177
}
179178

180179
@Override
181-
public Optional<EventListener> getEventListener()
180+
public List<EventListener> getEventListeners()
182181
{
183-
return server.getEventListener();
182+
return server.getEventListeners();
184183
}
185184

186185
@Override

0 commit comments

Comments
 (0)