Skip to content

Prevent too many concurrent query completed notifications#22911

Merged
sopel39 merged 1 commit intotrinodb:masterfrom
sopel39:ks/cap_completed
Aug 5, 2024
Merged

Prevent too many concurrent query completed notifications#22911
sopel39 merged 1 commit intotrinodb:masterfrom
sopel39:ks/cap_completed

Conversation

@sopel39
Copy link
Copy Markdown
Member

@sopel39 sopel39 commented Aug 1, 2024

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:

# Event listeners
* Skip query completed notifications when there are too many concurrent query completed request.
  Maximum number of requests can be setup with `event-listener.max-concurrent-query-completed-events`. ({issue}`issuenumber`)

@wendigo wendigo requested a review from hashhar August 1, 2024 15:03
Copy link
Copy Markdown
Member

@hashhar hashhar left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

@sopel39
Copy link
Copy Markdown
Member Author

sopel39 commented Aug 2, 2024

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.

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.

IMO we should introduce a batching/async event listener interface

Why do we need special interface for that rather than listener doing it itself?

Also, query created events are purposefully synchronized (

queryMonitor.queryCreatedEvent(stateMachine.getBasicQueryInfo(Optional.empty()));
), so I'm not sure there is an easy way around that

Copy link
Copy Markdown
Member

@hashhar hashhar left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.
@sopel39 sopel39 merged commit e700260 into trinodb:master Aug 5, 2024
@sopel39 sopel39 deleted the ks/cap_completed branch August 5, 2024 10:06
@github-actions github-actions bot added this to the 454 milestone Aug 5, 2024
return maxConcurrentQueryCompletedEvents;
}

@Config("event-listener.max-concurrent-query-completed-events")
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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.

Copy link
Copy Markdown
Member Author

@sopel39 sopel39 Aug 8, 2024

Choose a reason for hiding this comment

The 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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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.

Copy link
Copy Markdown
Member Author

@sopel39 sopel39 Aug 9, 2024

Choose a reason for hiding this comment

The 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

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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 class EventListenerConfig
{
private List<File> eventListenerFiles = ImmutableList.of();
private int maxConcurrentQueryCompletedEvents = 100;
Copy link
Copy Markdown
Member

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.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

public void queryCompleted(Function<Boolean, QueryCompletedEvent> queryCompletedEventProvider)
{
try (TimeStat.BlockTimer _ = queryCompletedTime.time()) {
if (concurrentQueryCompletedEvents.incrementAndGet() > maxConcurrentQueryCompletedEvents) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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.

return maxConcurrentQueryCompletedEvents;
}

@Config("event-listener.max-concurrent-query-completed-events")
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Development

Successfully merging this pull request may close these issues.

4 participants