workers: Rewrite missedmessage_emails with a worker thread.

The previous implementation leaked database connections, as a new
thread (and thus a new thread-local database connection) was made for
each timer execution.  While these connections were relatively
lightweight in Python, they also incur memory overhead in the
PostgreSQL server itself.  The logic for managing the timer was also
unclear, and the unavoidable deadlock in the stopping logic was rather
unfortunate.

Rewrite with one explicit worker thread which handles the delayed
message sending.  The RabbitMQ consumer creates the database rows, and
notifies the worker to start its 5s timeout.  Because it is controlled
by a condition variable, it does not hold the lock while waiting, and
can be notified to exit.
This commit is contained in:
Alex Vandiver
2023-03-30 20:28:34 +00:00
committed by Tim Abbott
parent 4478f8cc55
commit 3efc0c9af3
2 changed files with 307 additions and 249 deletions

View File

@@ -10,13 +10,13 @@ import os
import signal
import socket
import tempfile
import threading
import time
import urllib
from abc import ABC, abstractmethod
from collections import deque
from email.message import EmailMessage
from functools import wraps
from threading import RLock, Timer
from types import FrameType
from typing import (
Any,
@@ -566,48 +566,72 @@ class UserPresenceWorker(QueueProcessingWorker):
@assign_queue("missedmessage_emails")
class MissedMessageWorker(QueueProcessingWorker):
# Aggregate all messages received over the last BATCH_DURATION
# seconds to let someone finish sending a batch of messages and/or
# editing them before they are sent out as emails to recipients.
# Aggregate all messages received over the last several seconds
# (configurable by each recipient) to let someone finish sending a
# batch of messages and/or editing them before they are sent out
# as emails to recipients.
#
# The timer is running whenever; we poll at most every TIMER_FREQUENCY
# seconds, to avoid excessive activity.
TIMER_FREQUENCY = 5
timer_event: Optional[Timer] = None
# The batch interval is best-effort -- we poll at most every
# CHECK_FREQUENCY_SECONDS, to avoid excessive activity.
CHECK_FREQUENCY_SECONDS = 5
# This lock protects access to all of the data structures declared
# above. A lock is required because maybe_send_batched_emails, as
# the argument to Timer, runs in a separate thread from the rest
# of the consumer. This is a _re-entrant_ lock because we may
# need to take the lock when we already have it during shutdown
# (see the stop method).
lock = RLock()
worker_thread: Optional[threading.Thread] = None
# Because the background `maybe_send_batched_email` thread can
# hold the lock for an indeterminate amount of time, the `consume`
# can block on that for longer than 30s, the default worker
# timeout. Allow arbitrarily-long worker `consume` calls.
MAX_CONSUME_SECONDS = None
# This condition variable mediates the stopping and has_timeout
# pieces of state, below it.
cv = threading.Condition()
stopping = False
has_timeout = False
# The main thread, which handles the RabbitMQ connection and creates
# database rows from them.
def consume(self, event: Dict[str, Any]) -> None:
with self.lock:
logging.debug("Received missedmessage_emails event: %s", event)
logging.debug("Processing missedmessage_emails event: %s", event)
# When we consume an event, check if there are existing pending emails
# for that user, and if so use the same scheduled timestamp.
user_profile_id: int = event["user_profile_id"]
user_profile = get_user_profile_by_id(user_profile_id)
batch_duration_seconds = user_profile.email_notifications_batching_period_seconds
batch_duration = datetime.timedelta(seconds=batch_duration_seconds)
# When we consume an event, check if there are existing pending emails
# for that user, and if so use the same scheduled timestamp.
user_profile_id: int = event["user_profile_id"]
user_profile = get_user_profile_by_id(user_profile_id)
batch_duration_seconds = user_profile.email_notifications_batching_period_seconds
batch_duration = datetime.timedelta(seconds=batch_duration_seconds)
try:
pending_email = ScheduledMessageNotificationEmail.objects.filter(
user_profile_id=user_profile_id
)[0]
scheduled_timestamp = pending_email.scheduled_timestamp
except IndexError:
scheduled_timestamp = timezone_now() + batch_duration
try:
pending_email = ScheduledMessageNotificationEmail.objects.filter(
user_profile_id=user_profile_id
)[0]
scheduled_timestamp = pending_email.scheduled_timestamp
except IndexError:
scheduled_timestamp = timezone_now() + batch_duration
with self.cv:
# We now hold the lock, so there are three places the
# worker thread can be:
#
# 1. In maybe_send_batched_emails, and will have to take
# the lock (and thus block insertions of new rows
# here) to decide if there are any rows and if it thus
# needs a timeout.
#
# 2. In the cv.wait_for with a timeout because there were
# rows already. There's nothing for us to do, since
# the newly-inserted row will get checked upon that
# timeout.
#
# 3. In the cv.wait_for without a timeout, because there
# weren't any rows (which we're about to change).
#
# Notifying in (1) is irrelevant, since the thread is not
# waiting. If we over-notify by doing so for both (2) and
# (3), the behaviour is correct but slightly inefficient,
# as the thread will be needlessly awoken and will just
# re-wait. However, if we fail to awake case (3), the
# worker thread will never wake up, and the
# ScheduledMessageNotificationEmail internal queue will
# back up.
#
# Use the self.has_timeout property (which is protected by
# the lock) to determine which of cases (2) or (3) we are
# in, and as such if we need to notify after making the
# row.
try:
ScheduledMessageNotificationEmail.objects.create(
user_profile_id=user_profile_id,
@@ -616,108 +640,126 @@ class MissedMessageWorker(QueueProcessingWorker):
scheduled_timestamp=scheduled_timestamp,
mentioned_user_group_id=event.get("mentioned_user_group_id"),
)
self.ensure_timer()
if not self.has_timeout:
self.cv.notify()
except IntegrityError:
logging.debug(
"ScheduledMessageNotificationEmail row could not be created. The message may have been deleted. Skipping event."
)
def ensure_timer(self) -> None:
# The caller is responsible for ensuring self.lock is held when it calls this.
if self.timer_event is not None:
return
def start(self) -> None:
with self.cv:
self.stopping = False
self.worker_thread = threading.Thread(target=lambda: self.work())
self.worker_thread.start()
super().start()
self.timer_event = Timer(
self.TIMER_FREQUENCY, MissedMessageWorker.maybe_send_batched_emails, [self]
)
self.timer_event.start()
def work(self) -> None:
while True:
with self.cv:
if self.stopping:
return
# There are three conditions which we wait for:
#
# 1. We are being explicitly asked to stop; see the
# notify() call in stop()
#
# 2. We have no ScheduledMessageNotificationEmail
# objects currently (has_timeout = False) and the
# first one was just enqueued; see the notify()
# call in consume(). We break out so that we can
# come back around the loop and re-wait with a
# timeout (see next condition).
#
# 3. One or more ScheduledMessageNotificationEmail
# exist in the database, so we need to re-check
# them regularly; this happens by hitting the
# timeout and calling maybe_send_batched_emails().
# There is no explicit notify() for this.
timeout: Optional[int] = None
if ScheduledMessageNotificationEmail.objects.exists():
timeout = self.CHECK_FREQUENCY_SECONDS
self.has_timeout = timeout is not None
def wait_condition() -> bool:
if self.stopping:
# Condition (1)
return True
if timeout is None:
# Condition (2). We went to sleep with no
# ScheduledMessageNotificationEmail existing,
# and one has just been made. We re-check
# that is still true now that we have the
# lock, and if we see it, we stop waiting.
return ScheduledMessageNotificationEmail.objects.exists()
# This should only happen at the start or end of
# the wait, when we haven't been notified, but are
# re-checking the condition.
return False
was_notified = self.cv.wait_for(wait_condition, timeout=timeout)
# Being notified means that we are in conditions (1) or
# (2), above. In neither case do we need to look at if
# there are batches to send -- (2) means that the
# ScheduledMessageNotificationEmail was _just_ created, so
# there is no need to check it now.
if not was_notified:
self.maybe_send_batched_emails()
def maybe_send_batched_emails(self) -> None:
with self.lock:
# self.timer_event just triggered execution of this
# function in a thread, so now that we hold the lock, we
# clear the timer_event attribute to record that no Timer
# is active. If it is already None, stop() is shutting us
# down.
if self.timer_event is None:
return
self.timer_event = None
current_time = timezone_now()
current_time = timezone_now()
with transaction.atomic():
events_to_process = ScheduledMessageNotificationEmail.objects.filter(
scheduled_timestamp__lte=current_time
).select_related()
with transaction.atomic():
events_to_process = ScheduledMessageNotificationEmail.objects.filter(
scheduled_timestamp__lte=current_time
).select_related()
# Batch the entries by user
events_by_recipient: Dict[int, List[Dict[str, Any]]] = {}
for event in events_to_process:
entry = dict(
user_profile_id=event.user_profile_id,
message_id=event.message_id,
trigger=event.trigger,
mentioned_user_group_id=event.mentioned_user_group_id,
)
if event.user_profile_id in events_by_recipient:
events_by_recipient[event.user_profile_id].append(entry)
else:
events_by_recipient[event.user_profile_id] = [entry]
# Batch the entries by user
events_by_recipient: Dict[int, List[Dict[str, Any]]] = {}
for event in events_to_process:
entry = dict(
user_profile_id=event.user_profile_id,
message_id=event.message_id,
trigger=event.trigger,
mentioned_user_group_id=event.mentioned_user_group_id,
)
if event.user_profile_id in events_by_recipient:
events_by_recipient[event.user_profile_id].append(entry)
else:
events_by_recipient[event.user_profile_id] = [entry]
for user_profile_id in events_by_recipient:
events: List[Dict[str, Any]] = events_by_recipient[user_profile_id]
for user_profile_id in events_by_recipient:
events: List[Dict[str, Any]] = events_by_recipient[user_profile_id]
logging.info(
"Batch-processing %s missedmessage_emails events for user %s",
logging.info(
"Batch-processing %s missedmessage_emails events for user %s",
len(events),
user_profile_id,
)
try:
# Because we process events in batches, an
# escaped exception here would lead to
# duplicate messages being sent for other
# users in the same events_to_process batch,
# and no guarantee of forward progress.
handle_missedmessage_emails(user_profile_id, events)
except Exception:
logging.exception(
"Failed to process %d missedmessage_emails for user %s",
len(events),
user_profile_id,
stack_info=True,
)
try:
# Because we process events in batches, an
# escaped exception here would lead to
# duplicate messages being sent for other
# users in the same events_to_process batch,
# and no guarantee of forward progress.
handle_missedmessage_emails(user_profile_id, events)
except Exception:
logging.exception(
"Failed to process %d missedmessage_emails for user %s",
len(events),
user_profile_id,
stack_info=True,
)
events_to_process.delete()
# By only restarting the timer if there are actually events in
# the queue, we ensure this queue processor is idle when there
# are no missed-message emails to process. This avoids
# constant CPU usage when there is no work to do.
if ScheduledMessageNotificationEmail.objects.exists():
self.ensure_timer()
events_to_process.delete()
def stop(self) -> None:
# This may be called from a signal handler when we _already_
# have the lock. Python doesn't give us a way to check if our
# thread has the lock, so we instead use a re-entrant lock to
# always take it.
with self.lock:
# With the lock,we can safely inspect the timer_event and
# cancel it if it is still pending.
if self.timer_event is not None:
# We cancel and then join the timer with a timeout to
# prevent deadlock, where we took the lock, the timer
# then ran out and started maybe_send_batched_emails,
# and then it started waiting for the lock. The timer
# isn't running anymore so can't be canceled, and the
# thread is blocked on the lock, so will never join().
self.timer_event.cancel()
self.timer_event.join(timeout=1)
# In case we did hit this deadlock, we signal to
# maybe_send_batched_emails that it should abort by,
# before releasing the lock, unsetting the timer.
self.timer_event = None
with self.cv:
self.stopping = True
self.cv.notify()
if self.worker_thread is not None:
self.worker_thread.join()
super().stop()