Skip to content

Commit 70d381b

Browse files
authored
refactor: tzaware crontab is due method (#899)
* refactor: use super() in is_due() method * test: add some tests * pre commit * fix: schedulers * fix: test mistake
1 parent 26100b8 commit 70d381b

File tree

2 files changed

+174
-10
lines changed

2 files changed

+174
-10
lines changed

django_celery_beat/tzcrontab.py

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -38,13 +38,7 @@ def is_due(self, last_run_at):
3838
# convert last_run_at to the schedule timezone
3939
last_run_at = last_run_at.astimezone(self.tz)
4040

41-
rem_delta = self.remaining_estimate(last_run_at)
42-
rem = max(rem_delta.total_seconds(), 0)
43-
due = rem == 0
44-
if due:
45-
rem_delta = self.remaining_estimate(self.now())
46-
rem = max(rem_delta.total_seconds(), 0)
47-
return schedstate(due, rem)
41+
return super().is_due(last_run_at)
4842

4943
# Needed to support pickling
5044
def __repr__(self):

t/unit/test_schedulers.py

Lines changed: 173 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22
import os
33
import time
44
from datetime import datetime, timedelta
5+
from datetime import timezone as dt_timezone
56
from itertools import count
67
from time import monotonic
78
from unittest.mock import patch
@@ -11,6 +12,8 @@
1112
except ImportError:
1213
from backports.zoneinfo import ZoneInfo # Python 3.8
1314

15+
from unittest.mock import MagicMock
16+
1417
import pytest
1518
from celery.schedules import crontab, schedule, solar
1619
from django.contrib.admin.sites import AdminSite
@@ -24,6 +27,7 @@
2427
from django_celery_beat.models import (DAYS, ClockedSchedule, CrontabSchedule,
2528
IntervalSchedule, PeriodicTask,
2629
PeriodicTasks, SolarSchedule)
30+
from django_celery_beat.tzcrontab import TzAwareCrontab
2731
from django_celery_beat.utils import NEVER_CHECK_TIMEOUT, make_aware
2832

2933
_ids = count(0)
@@ -119,6 +123,167 @@ def create_crontab_schedule(self):
119123
return CrontabSchedule.objects.create()
120124

121125

126+
@pytest.mark.django_db
127+
class test_TzAwareCrontab_beat_cron_starting_deadline(SchedulerCase):
128+
"""Tests for TzAwareCrontab with beat_cron_starting_deadline."""
129+
130+
@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
131+
@patch("django_celery_beat.tzcrontab.datetime")
132+
def test_due_when_within_starting_deadline(self, mock_datetime):
133+
"""
134+
Test that a task is due if last_run_at is within
135+
beat_cron_starting_deadline.
136+
"""
137+
# Create a mock app with 5 minute beat_cron_starting_deadline
138+
app = MagicMock()
139+
app.conf.beat_cron_starting_deadline = 300 # 5 minutes in seconds
140+
141+
# Set current time to 12:05:50
142+
mock_now_utc = datetime(
143+
2023, 10, 26, 12, 5, 50, tzinfo=dt_timezone.utc
144+
)
145+
mock_datetime.now.return_value = mock_now_utc
146+
147+
# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
148+
schedule = TzAwareCrontab(app=app, tz=dt_timezone.utc, minute="*/5")
149+
150+
# Last run was 290 seconds ago (12:01:00) - within the deadline window
151+
last_run_at_utc = mock_now_utc - timedelta(seconds=290)
152+
153+
# Next scheduled run should be 12:05:00, but current time is 12:05:50
154+
# So the task is due, and the deadline check doesn't prevent execution
155+
due_status, next_check_delta = schedule.is_due(last_run_at_utc)
156+
assert due_status is True
157+
158+
@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
159+
@patch("django_celery_beat.tzcrontab.datetime")
160+
def test_not_due_when_outside_starting_deadline(self, mock_datetime):
161+
"""
162+
Test that a task is NOT due if last_run_at is outside (older than)
163+
beat_cron_starting_deadline.
164+
"""
165+
# Create a mock app with 2 minute beat_cron_starting_deadline
166+
app = MagicMock()
167+
app.conf.beat_cron_starting_deadline = 120 # 2 minutes in seconds
168+
169+
# Set current time to 12:09:50
170+
mock_now_utc = datetime(
171+
2023, 10, 26, 12, 9, 50, tzinfo=dt_timezone.utc
172+
)
173+
mock_datetime.now.return_value = mock_now_utc
174+
175+
# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
176+
schedule_utc = TzAwareCrontab(
177+
app=app, tz=dt_timezone.utc, minute="*/5"
178+
)
179+
180+
# Last run was 310 seconds ago (12:04:40) - outside the deadline window
181+
last_run_at_utc = mock_now_utc - timedelta(seconds=310)
182+
183+
# Next scheduled run after 12:04:40 would be 12:05:00
184+
# This is in the past relative to current time, so normally due
185+
# BUT since last_run_at is before the deadline window, it's NOT due
186+
due_status, next_check_delta = schedule_utc.is_due(last_run_at_utc)
187+
assert due_status is False
188+
189+
@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
190+
@patch("django_celery_beat.tzcrontab.datetime")
191+
def test_not_due_with_recent_run(self, mock_datetime):
192+
"""
193+
Test that a task is not due if last_run_at is recent,
194+
even with a starting_deadline set.
195+
"""
196+
# Create a mock app with 5 minute beat_cron_starting_deadline
197+
app = MagicMock()
198+
app.conf.beat_cron_starting_deadline = 300 # 5 minutes in seconds
199+
200+
# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
201+
schedule = TzAwareCrontab(app=app, tz=dt_timezone.utc, minute="*/5")
202+
203+
# Set current time to 12:04:30 (before next scheduled execution)
204+
mock_now_utc_early = datetime(
205+
2023, 10, 26, 12, 4, 30, tzinfo=dt_timezone.utc
206+
)
207+
mock_datetime.now.return_value = mock_now_utc_early
208+
209+
# Last run was at 12:04:00
210+
# The next schedule would be at 12:05:00, which is in the future
211+
last_run_at_recent = datetime(
212+
2023, 10, 26, 12, 4, 0, tzinfo=dt_timezone.utc
213+
)
214+
215+
# Calculate if the task is due
216+
# Since the next execution time is in the future, the task is not due
217+
# The deadline check doesn't matter for tasks not yet scheduled to run
218+
due_status, next_check_delta = schedule.is_due(last_run_at_recent)
219+
assert due_status is False
220+
221+
@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
222+
@patch("django_celery_beat.tzcrontab.datetime")
223+
def test_due_with_no_starting_deadline_set(self, mock_datetime):
224+
"""
225+
Test that a task is due if last_run_at is old and no deadline is set.
226+
"""
227+
# Create a mock app with no beat_cron_starting_deadline
228+
app = MagicMock()
229+
app.conf.beat_cron_starting_deadline = None
230+
231+
# Set current time to 12:10:00
232+
mock_now_utc = datetime(
233+
2023, 10, 26, 12, 10, 0, tzinfo=dt_timezone.utc
234+
)
235+
mock_datetime.now.return_value = mock_now_utc
236+
237+
# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
238+
schedule_utc = TzAwareCrontab(
239+
app=app, tz=dt_timezone.utc, minute="*/5"
240+
)
241+
242+
# Last run was 310 seconds ago (12:04:50)
243+
# With no deadline, age of the last run doesn't matter
244+
last_run_at_utc = mock_now_utc - timedelta(seconds=310)
245+
246+
# Next scheduled time after 12:04:50 would be 12:05:00
247+
# Current time is 12:10:00, so this is in the past
248+
# With no deadline check, the task is due to run
249+
due_status, next_check_delta = schedule_utc.is_due(
250+
last_run_at_utc
251+
)
252+
assert due_status is True
253+
254+
@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
255+
@patch("django_celery_beat.tzcrontab.datetime")
256+
def test_due_with_starting_deadline_non_utc_timezone(self, mock_datetime):
257+
"""
258+
Test with a non-UTC timezone for the schedule.
259+
"""
260+
# Create a mock app with 5 minute beat_cron_starting_deadline
261+
app = MagicMock()
262+
app.conf.beat_cron_starting_deadline = 300 # 5 minutes in seconds
263+
app.timezone = ZoneInfo("America/New_York")
264+
265+
# Use New York timezone for the schedule
266+
schedule_tz = ZoneInfo("America/New_York")
267+
268+
# Set current time to 08:05:00 New York time
269+
mock_now_ny = datetime(2023, 10, 26, 8, 5, 0, tzinfo=schedule_tz)
270+
mock_datetime.now.return_value = mock_now_ny
271+
272+
# Create a schedule that runs every 5 minutes in NY time
273+
schedule_ny = TzAwareCrontab(app=app, tz=schedule_tz, minute="*/5")
274+
275+
# Last run was 290 seconds ago - within deadline window
276+
# The deadline window starts at 08:00:00 (current time - 5 minutes)
277+
# Since 08:00:10 is after 08:00:00, it's within the deadline window
278+
last_run_at_ny = mock_now_ny - timedelta(seconds=290)
279+
280+
# Next scheduled time after 08:00:10 would be 08:05:00
281+
# Current time is 08:05:00, so this is due
282+
# The deadline check doesn't prevent execution
283+
due_status, next_check_delta = schedule_ny.is_due(last_run_at_ny)
284+
assert due_status is True
285+
286+
122287
@pytest.mark.django_db
123288
class test_ModelEntry(SchedulerCase):
124289
Entry = EntryTrackSave
@@ -1500,11 +1665,14 @@ def teardown_method(self):
15001665

15011666
@patch("django_celery_beat.schedulers.aware_now")
15021667
def test_server_timezone_handling_with_zoneinfo(self, mock_aware_now):
1503-
"""Test handling when server timezone is already a ZoneInfo instance."""
1668+
"""Test handling when server timezone
1669+
is already a ZoneInfo instance."""
15041670

15051671
# Create a mock scheduler with only the methods we need to test
15061672
class MockScheduler:
1507-
_get_timezone_offset = schedulers.DatabaseScheduler._get_timezone_offset
1673+
_get_timezone_offset = (
1674+
schedulers.DatabaseScheduler._get_timezone_offset
1675+
)
15081676

15091677
s = MockScheduler()
15101678

@@ -1525,7 +1693,9 @@ def test_timezone_offset_with_zoneinfo_object_param(self, mock_aware_now):
15251693
"""Test handling when timezone_name parameter is a ZoneInfo object."""
15261694

15271695
class MockScheduler:
1528-
_get_timezone_offset = schedulers.DatabaseScheduler._get_timezone_offset
1696+
_get_timezone_offset = (
1697+
schedulers.DatabaseScheduler._get_timezone_offset
1698+
)
15291699

15301700
s = MockScheduler()
15311701

0 commit comments

Comments
 (0)