Prevent too many concurrent query completed notifications#22911
Prevent too many concurrent query completed notifications#22911sopel39 merged 1 commit intotrinodb:masterfrom
Conversation
ff0c759 to
580ff78
Compare
core/trino-main/src/main/java/io/trino/eventlistener/EventListenerManager.java
Outdated
Show resolved
Hide resolved
580ff78 to
8a74031
Compare
hashhar
left a comment
There was a problem hiding this comment.
this change doesn't help in cases where the event listener just queues up the event internally and uses a separate thread to consume them since the event listener itself will return relatively quickly.
i.e. this change is helpful for poorly written event listeners (who block on the main thread) but not for listeners which implement their own batching.
IMO we should introduce a batching/async event listener interface too since the more common problem is that the time for each individual listener adds up serially since the EventListenerManager invokes them one by one instead of all at once concurrently which would be a more useful change.
I don't feel too strongly about making the queue bounded though, so a soft LGTM from me.
Exactly. This PR is protecting Trino itself from bad listeners by bounding notification queue size. If listener is behaving badly anyway the system might still fail although it will be purely listener fault. I do think most of the custom listeners are pretty basic anyway so this change should help system stability in most of the cases.
Why do we need special interface for that rather than listener doing it itself? Also, |
hashhar
left a comment
There was a problem hiding this comment.
Also, query created events are purposefully synchronized
For queryCreated we can still fire all EventListener's concurrently (for a single event).
This is LGTM from me anyway, it's a change in the right direction.
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.
8a74031 to
f11b263
Compare
| return maxConcurrentQueryCompletedEvents; | ||
| } | ||
|
|
||
| @Config("event-listener.max-concurrent-query-completed-events") |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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 class EventListenerConfig | ||
| { | ||
| private List<File> eventListenerFiles = ImmutableList.of(); | ||
| private int maxConcurrentQueryCompletedEvents = 100; |
There was a problem hiding this comment.
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.
| public void queryCompleted(Function<Boolean, QueryCompletedEvent> queryCompletedEventProvider) | ||
| { | ||
| try (TimeStat.BlockTimer _ = queryCompletedTime.time()) { | ||
| if (concurrentQueryCompletedEvents.incrementAndGet() > maxConcurrentQueryCompletedEvents) { |
There was a problem hiding this comment.
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.
| return maxConcurrentQueryCompletedEvents; | ||
| } | ||
|
|
||
| @Config("event-listener.max-concurrent-query-completed-events") |
There was a problem hiding this comment.
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.
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.
Release notes
( ) This is not user-visible or is docs only, and no release notes are required.
( ) Release notes are required. Please propose a release note for me.
(x) Release notes are required, with the following suggested text: