-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Prevent too many concurrent query completed notifications #22911
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,7 @@ | |
| import com.google.common.collect.ImmutableList; | ||
| import io.airlift.configuration.Config; | ||
| import io.airlift.configuration.validation.FileExists; | ||
| import jakarta.validation.constraints.Min; | ||
| import jakarta.validation.constraints.NotNull; | ||
|
|
||
| import java.io.File; | ||
|
|
@@ -26,6 +27,7 @@ | |
| public class EventListenerConfig | ||
| { | ||
| private List<File> eventListenerFiles = ImmutableList.of(); | ||
| private int maxConcurrentQueryCompletedEvents = 100; | ||
|
|
||
| @NotNull | ||
| public List<@FileExists File> getEventListenerFiles() | ||
|
|
@@ -41,4 +43,17 @@ public EventListenerConfig setEventListenerFiles(List<String> eventListenerFiles | |
| .collect(toImmutableList()); | ||
| return this; | ||
| } | ||
|
|
||
| @Min(1) | ||
| public int getMaxConcurrentQueryCompletedEvents() | ||
| { | ||
| return maxConcurrentQueryCompletedEvents; | ||
| } | ||
|
|
||
| @Config("event-listener.max-concurrent-query-completed-events") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think "concurrent" is a good name for this. It doesn't convey the right concept or suggest a good mental model. An event is a point in time concept, so concurrent over what period? I would suggest using the term "pending" instead.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Pending might suggest there is limit on queue size of "query completed events", but there isn't "concurrent completed events" metric is only bumped when there are multiple (multithreaded) calls to event listener at the same time. Even if there could be 1000 query completed events scheduled to be run by single threaded executor, "concurrent completed events" metric will be at most 1. The backpressure in this case comes not from executor queue, but from max number of concurrent queries. If event listener is slow to consume events, then excessive events are simply ignored preventing system from blowing up.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see. In that case, I'm not sure I understand what problem this PR is trying to solve. If an event listener cannot handle a certain concurrent load, it should limit (or discard) events internally.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The problem are badly written custom event listeners. Usually they are not written by core Trino experts or concurrency experts which leads to deployment issues and poor experience
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That seems like a problem for whoever is deploying those poorly written event listeners. I'm not sure I agree with Trino having to work around plugins that are not designed to handle how Trino works. Also, as I described elsewhere, the way this is implemented in the PR, a poorly written listener will affect the ability to collect events in listeners that don't have those issues. |
||
| public EventListenerConfig setMaxConcurrentQueryCompletedEvents(int maxConcurrentQueryCompletedEvents) | ||
| { | ||
| this.maxConcurrentQueryCompletedEvents = maxConcurrentQueryCompletedEvents; | ||
| return this; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
| import com.google.common.collect.ImmutableList; | ||
| import com.google.inject.Inject; | ||
| import io.airlift.log.Logger; | ||
| import io.airlift.stats.CounterStat; | ||
| import io.airlift.stats.TimeStat; | ||
| import io.trino.spi.classloader.ThreadContextClassLoader; | ||
| import io.trino.spi.eventlistener.EventListener; | ||
|
|
@@ -38,6 +39,7 @@ | |
| import java.util.Map; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
|
|
||
| import static com.google.common.base.Preconditions.checkArgument; | ||
|
|
@@ -55,10 +57,13 @@ public class EventListenerManager | |
| private static final File CONFIG_FILE = new File("etc/event-listener.properties"); | ||
| private static final String EVENT_LISTENER_NAME_PROPERTY = "event-listener.name"; | ||
| private final List<File> configFiles; | ||
| private final int maxConcurrentQueryCompletedEvents; | ||
| private final Map<String, EventListenerFactory> eventListenerFactories = new ConcurrentHashMap<>(); | ||
| private final List<EventListener> providedEventListeners = Collections.synchronizedList(new ArrayList<>()); | ||
| private final AtomicReference<List<EventListener>> configuredEventListeners = new AtomicReference<>(ImmutableList.of()); | ||
| private final AtomicBoolean loading = new AtomicBoolean(false); | ||
| private final AtomicInteger concurrentQueryCompletedEvents = new AtomicInteger(); | ||
| private final CounterStat skippedQueryCompletedEvents = new CounterStat(); | ||
|
|
||
| private final TimeStat queryCreatedTime = new TimeStat(MILLISECONDS); | ||
| private final TimeStat queryCompletedTime = new TimeStat(MILLISECONDS); | ||
|
|
@@ -68,6 +73,7 @@ public class EventListenerManager | |
| public EventListenerManager(EventListenerConfig config) | ||
| { | ||
| this.configFiles = ImmutableList.copyOf(config.getEventListenerFiles()); | ||
| this.maxConcurrentQueryCompletedEvents = config.getMaxConcurrentQueryCompletedEvents(); | ||
| } | ||
|
|
||
| public void addEventListenerFactory(EventListenerFactory eventListenerFactory) | ||
|
|
@@ -144,7 +150,13 @@ private static Map<String, String> loadEventListenerProperties(File configFile) | |
| public void queryCompleted(Function<Boolean, QueryCompletedEvent> queryCompletedEventProvider) | ||
| { | ||
| try (TimeStat.BlockTimer _ = queryCompletedTime.time()) { | ||
| if (concurrentQueryCompletedEvents.incrementAndGet() > maxConcurrentQueryCompletedEvents) { | ||
sopel39 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The limit is being applied globally, so if you have multiple listeners and only one of the is slow, it will cause events to be missed for the other listeners. |
||
| concurrentQueryCompletedEvents.decrementAndGet(); | ||
| skippedQueryCompletedEvents.update(1); | ||
| return; | ||
| } | ||
| doQueryCompleted(queryCompletedEventProvider); | ||
| concurrentQueryCompletedEvents.decrementAndGet(); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -220,6 +232,19 @@ public TimeStat getSplitCompletedTime() | |
| return splitCompletedTime; | ||
| } | ||
|
|
||
| @Managed | ||
| public int getConcurrentQueryCompletedEvents() | ||
| { | ||
| return concurrentQueryCompletedEvents.get(); | ||
| } | ||
|
|
||
| @Managed | ||
| @Nested | ||
| public CounterStat getSkippedQueryCompletedEvents() | ||
| { | ||
| return skippedQueryCompletedEvents; | ||
| } | ||
|
|
||
| @PreDestroy | ||
| public void shutdown() | ||
| { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why 100? IMO, the default should be unlimited. The specific number would depend on which event listeners you're using and whether they can handle a given number of concurrent calls.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@martint #22994