Skip to content

Commit e700260

Browse files
committed
Prevent too many concurrent query completed notifications
queryCompleted notifications are handled by unbounded executor, hence it could cause all sort of issues when there is spike in query traffic and event listener is slow to report events.
1 parent 8f4dc1c commit e700260

File tree

5 files changed

+218
-2
lines changed

5 files changed

+218
-2
lines changed

core/trino-main/src/main/java/io/trino/eventlistener/EventListenerConfig.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
import com.google.common.collect.ImmutableList;
1717
import io.airlift.configuration.Config;
1818
import io.airlift.configuration.validation.FileExists;
19+
import jakarta.validation.constraints.Min;
1920
import jakarta.validation.constraints.NotNull;
2021

2122
import java.io.File;
@@ -26,6 +27,7 @@
2627
public class EventListenerConfig
2728
{
2829
private List<File> eventListenerFiles = ImmutableList.of();
30+
private int maxConcurrentQueryCompletedEvents = 100;
2931

3032
@NotNull
3133
public List<@FileExists File> getEventListenerFiles()
@@ -41,4 +43,17 @@ public EventListenerConfig setEventListenerFiles(List<String> eventListenerFiles
4143
.collect(toImmutableList());
4244
return this;
4345
}
46+
47+
@Min(1)
48+
public int getMaxConcurrentQueryCompletedEvents()
49+
{
50+
return maxConcurrentQueryCompletedEvents;
51+
}
52+
53+
@Config("event-listener.max-concurrent-query-completed-events")
54+
public EventListenerConfig setMaxConcurrentQueryCompletedEvents(int maxConcurrentQueryCompletedEvents)
55+
{
56+
this.maxConcurrentQueryCompletedEvents = maxConcurrentQueryCompletedEvents;
57+
return this;
58+
}
4459
}

core/trino-main/src/main/java/io/trino/eventlistener/EventListenerManager.java

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
import com.google.common.collect.ImmutableList;
1818
import com.google.inject.Inject;
1919
import io.airlift.log.Logger;
20+
import io.airlift.stats.CounterStat;
2021
import io.airlift.stats.TimeStat;
2122
import io.trino.spi.classloader.ThreadContextClassLoader;
2223
import io.trino.spi.eventlistener.EventListener;
@@ -38,6 +39,7 @@
3839
import java.util.Map;
3940
import java.util.concurrent.ConcurrentHashMap;
4041
import java.util.concurrent.atomic.AtomicBoolean;
42+
import java.util.concurrent.atomic.AtomicInteger;
4143
import java.util.concurrent.atomic.AtomicReference;
4244

4345
import static com.google.common.base.Preconditions.checkArgument;
@@ -55,10 +57,13 @@ public class EventListenerManager
5557
private static final File CONFIG_FILE = new File("etc/event-listener.properties");
5658
private static final String EVENT_LISTENER_NAME_PROPERTY = "event-listener.name";
5759
private final List<File> configFiles;
60+
private final int maxConcurrentQueryCompletedEvents;
5861
private final Map<String, EventListenerFactory> eventListenerFactories = new ConcurrentHashMap<>();
5962
private final List<EventListener> providedEventListeners = Collections.synchronizedList(new ArrayList<>());
6063
private final AtomicReference<List<EventListener>> configuredEventListeners = new AtomicReference<>(ImmutableList.of());
6164
private final AtomicBoolean loading = new AtomicBoolean(false);
65+
private final AtomicInteger concurrentQueryCompletedEvents = new AtomicInteger();
66+
private final CounterStat skippedQueryCompletedEvents = new CounterStat();
6267

6368
private final TimeStat queryCreatedTime = new TimeStat(MILLISECONDS);
6469
private final TimeStat queryCompletedTime = new TimeStat(MILLISECONDS);
@@ -68,6 +73,7 @@ public class EventListenerManager
6873
public EventListenerManager(EventListenerConfig config)
6974
{
7075
this.configFiles = ImmutableList.copyOf(config.getEventListenerFiles());
76+
this.maxConcurrentQueryCompletedEvents = config.getMaxConcurrentQueryCompletedEvents();
7177
}
7278

7379
public void addEventListenerFactory(EventListenerFactory eventListenerFactory)
@@ -144,7 +150,13 @@ private static Map<String, String> loadEventListenerProperties(File configFile)
144150
public void queryCompleted(Function<Boolean, QueryCompletedEvent> queryCompletedEventProvider)
145151
{
146152
try (TimeStat.BlockTimer _ = queryCompletedTime.time()) {
153+
if (concurrentQueryCompletedEvents.incrementAndGet() > maxConcurrentQueryCompletedEvents) {
154+
concurrentQueryCompletedEvents.decrementAndGet();
155+
skippedQueryCompletedEvents.update(1);
156+
return;
157+
}
147158
doQueryCompleted(queryCompletedEventProvider);
159+
concurrentQueryCompletedEvents.decrementAndGet();
148160
}
149161
}
150162

@@ -220,6 +232,19 @@ public TimeStat getSplitCompletedTime()
220232
return splitCompletedTime;
221233
}
222234

235+
@Managed
236+
public int getConcurrentQueryCompletedEvents()
237+
{
238+
return concurrentQueryCompletedEvents.get();
239+
}
240+
241+
@Managed
242+
@Nested
243+
public CounterStat getSkippedQueryCompletedEvents()
244+
{
245+
return skippedQueryCompletedEvents;
246+
}
247+
223248
@PreDestroy
224249
public void shutdown()
225250
{

core/trino-main/src/test/java/io/trino/eventlistener/TestEventListenerConfig.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ public class TestEventListenerConfig
3232
public void testDefaults()
3333
{
3434
assertRecordedDefaults(ConfigAssertions.recordDefaults(EventListenerConfig.class)
35+
.setMaxConcurrentQueryCompletedEvents(100)
3536
.setEventListenerFiles(ImmutableList.of()));
3637
}
3738

@@ -42,10 +43,13 @@ public void testExplicitPropertyMappings()
4243
Path config1 = Files.createTempFile(null, null);
4344
Path config2 = Files.createTempFile(null, null);
4445

45-
Map<String, String> properties = ImmutableMap.of("event-listener.config-files", config1.toString() + "," + config2.toString());
46+
Map<String, String> properties = ImmutableMap.of(
47+
"event-listener.config-files", config1.toString() + "," + config2.toString(),
48+
"event-listener.max-concurrent-query-completed-events", "1");
4649

4750
EventListenerConfig expected = new EventListenerConfig()
48-
.setEventListenerFiles(ImmutableList.of(config1.toFile().getPath(), config2.toFile().getPath()));
51+
.setEventListenerFiles(ImmutableList.of(config1.toFile().getPath(), config2.toFile().getPath()))
52+
.setMaxConcurrentQueryCompletedEvents(1);
4953

5054
assertFullMapping(properties, expected);
5155
}

core/trino-main/src/test/java/io/trino/eventlistener/TestEventListenerManager.java

Lines changed: 168 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,14 +14,139 @@
1414
package io.trino.eventlistener;
1515

1616
import io.trino.spi.eventlistener.EventListener;
17+
import io.trino.spi.eventlistener.QueryCompletedEvent;
18+
import io.trino.spi.eventlistener.QueryContext;
19+
import io.trino.spi.eventlistener.QueryIOMetadata;
20+
import io.trino.spi.eventlistener.QueryMetadata;
21+
import io.trino.spi.eventlistener.QueryStatistics;
22+
import io.trino.spi.session.ResourceEstimates;
1723
import org.junit.jupiter.api.Test;
1824

25+
import java.net.URI;
26+
import java.time.Instant;
27+
import java.util.Collections;
28+
import java.util.List;
29+
import java.util.Map;
30+
import java.util.Optional;
31+
import java.util.Set;
32+
import java.util.concurrent.CountDownLatch;
33+
import java.util.concurrent.ExecutorService;
34+
import java.util.concurrent.TimeUnit;
1935
import java.util.concurrent.atomic.AtomicBoolean;
2036

37+
import static io.trino.spi.type.TimeZoneKey.UTC_KEY;
38+
import static java.time.Duration.ofMillis;
39+
import static java.util.concurrent.Executors.newFixedThreadPool;
2140
import static org.assertj.core.api.Assertions.assertThat;
2241

2342
class TestEventListenerManager
2443
{
44+
private static final QueryMetadata QUERY_METADATA = new QueryMetadata(
45+
"minimal_query",
46+
Optional.empty(),
47+
"query",
48+
Optional.empty(),
49+
Optional.empty(),
50+
"queryState",
51+
// not stored
52+
List.of(),
53+
// not stored
54+
List.of(),
55+
URI.create("http://localhost"),
56+
Optional.empty(),
57+
Optional.empty(),
58+
Optional.empty());
59+
60+
private static final QueryStatistics QUERY_STATISTICS = new QueryStatistics(
61+
ofMillis(101),
62+
ofMillis(102),
63+
ofMillis(103),
64+
ofMillis(104),
65+
Optional.empty(),
66+
Optional.empty(),
67+
Optional.empty(),
68+
Optional.empty(),
69+
Optional.empty(),
70+
Optional.empty(),
71+
Optional.empty(),
72+
Optional.empty(),
73+
Optional.empty(),
74+
Optional.empty(),
75+
Optional.empty(),
76+
Optional.empty(),
77+
115L,
78+
116L,
79+
117L,
80+
118L,
81+
119L,
82+
1191L,
83+
1192L,
84+
120L,
85+
121L,
86+
122L,
87+
123L,
88+
124L,
89+
125L,
90+
126L,
91+
127L,
92+
1271L,
93+
128.0,
94+
129.0,
95+
// not stored
96+
Collections.emptyList(),
97+
130,
98+
false,
99+
// not stored
100+
Collections.emptyList(),
101+
// not stored
102+
Collections.emptyList(),
103+
// not stored
104+
Collections.emptyList(),
105+
Collections.emptyList(),
106+
Collections.emptyList(),
107+
// not stored
108+
Optional.empty());
109+
110+
private static final QueryContext QUERY_CONTEXT = new QueryContext(
111+
"user",
112+
"originalUser",
113+
Optional.empty(),
114+
Set.of(),
115+
Set.of(),
116+
Optional.empty(),
117+
Optional.empty(),
118+
Optional.empty(),
119+
Optional.empty(),
120+
Set.of(),
121+
// not stored
122+
Set.of(),
123+
Optional.empty(),
124+
UTC_KEY.getId(),
125+
Optional.empty(),
126+
Optional.empty(),
127+
Optional.empty(),
128+
Map.of(),
129+
// not stored
130+
new ResourceEstimates(Optional.empty(), Optional.empty(), Optional.empty()),
131+
"serverAddress",
132+
"serverVersion",
133+
"environment",
134+
Optional.empty(),
135+
"NONE");
136+
137+
private static final QueryIOMetadata QUERY_IO_METADATA = new QueryIOMetadata(List.of(), Optional.empty());
138+
139+
private static final QueryCompletedEvent QUERY_COMPLETED_EVENT = new QueryCompletedEvent(
140+
QUERY_METADATA,
141+
QUERY_STATISTICS,
142+
QUERY_CONTEXT,
143+
QUERY_IO_METADATA,
144+
Optional.empty(),
145+
List.of(),
146+
Instant.now(),
147+
Instant.now(),
148+
Instant.now());
149+
25150
@Test
26151
public void testShutdownIsForwardedToListeners()
27152
{
@@ -42,4 +167,47 @@ public void shutdown()
42167

43168
assertThat(wasCalled.get()).isTrue();
44169
}
170+
171+
@Test
172+
public void testMaxConcurrentQueryCompletedEvents()
173+
throws InterruptedException
174+
{
175+
EventListenerManager eventListenerManager = new EventListenerManager(new EventListenerConfig().setMaxConcurrentQueryCompletedEvents(1));
176+
eventListenerManager.addEventListener(new BlockingEventListener());
177+
eventListenerManager.loadEventListeners();
178+
ExecutorService executor = newFixedThreadPool(2);
179+
CountDownLatch countDownLatch = new CountDownLatch(1);
180+
try {
181+
Runnable queryCompletedEvent = () -> {
182+
eventListenerManager.queryCompleted(_ -> QUERY_COMPLETED_EVENT);
183+
countDownLatch.countDown();
184+
};
185+
executor.submit(queryCompletedEvent);
186+
executor.submit(queryCompletedEvent);
187+
188+
countDownLatch.await();
189+
assertThat(eventListenerManager.getSkippedQueryCompletedEvents().getTotalCount()).isEqualTo(1);
190+
assertThat(eventListenerManager.getConcurrentQueryCompletedEvents()).isEqualTo(1);
191+
}
192+
finally {
193+
executor.shutdownNow();
194+
assertThat(executor.awaitTermination(10, TimeUnit.SECONDS)).isTrue();
195+
}
196+
}
197+
198+
private static final class BlockingEventListener
199+
implements EventListener
200+
{
201+
@Override
202+
public void queryCompleted(QueryCompletedEvent queryCompletedEvent)
203+
{
204+
try {
205+
// sleep forever
206+
Thread.sleep(100_000);
207+
}
208+
catch (InterruptedException e) {
209+
throw new RuntimeException(e);
210+
}
211+
}
212+
}
45213
}

docs/src/main/sphinx/develop/event-listener.md

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,10 @@ custom-property1=custom-value1
4545
custom-property2=custom-value2
4646
```
4747

48+
Maximum number of concurrent query completed events
49+
can be configured using `event-listener.max-concurrent-query-completed-events` property
50+
(`100` by default). Excessive events are dropped.
51+
4852
(multiple-listeners)=
4953
## Multiple event listeners
5054

0 commit comments

Comments
 (0)