Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 1 addition & 7 deletions django_celery_beat/tzcrontab.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,13 +38,7 @@ def is_due(self, last_run_at):
# convert last_run_at to the schedule timezone
last_run_at = last_run_at.astimezone(self.tz)

rem_delta = self.remaining_estimate(last_run_at)
rem = max(rem_delta.total_seconds(), 0)
due = rem == 0
if due:
rem_delta = self.remaining_estimate(self.now())
rem = max(rem_delta.total_seconds(), 0)
return schedstate(due, rem)
return super().is_due(last_run_at)

# Needed to support pickling
def __repr__(self):
Expand Down
176 changes: 173 additions & 3 deletions t/unit/test_schedulers.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
import os
import time
from datetime import datetime, timedelta
from datetime import timezone as dt_timezone
from itertools import count
from time import monotonic
from unittest.mock import patch
Expand All @@ -11,6 +12,8 @@
except ImportError:
from backports.zoneinfo import ZoneInfo # Python 3.8

from unittest.mock import MagicMock

import pytest
from celery.schedules import crontab, schedule, solar
from django.contrib.admin.sites import AdminSite
Expand All @@ -24,6 +27,7 @@
from django_celery_beat.models import (DAYS, ClockedSchedule, CrontabSchedule,
IntervalSchedule, PeriodicTask,
PeriodicTasks, SolarSchedule)
from django_celery_beat.tzcrontab import TzAwareCrontab
from django_celery_beat.utils import NEVER_CHECK_TIMEOUT, make_aware

_ids = count(0)
Expand Down Expand Up @@ -119,6 +123,167 @@ def create_crontab_schedule(self):
return CrontabSchedule.objects.create()


@pytest.mark.django_db
class test_TzAwareCrontab_beat_cron_starting_deadline(SchedulerCase):
"""Tests for TzAwareCrontab with beat_cron_starting_deadline."""

@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
@patch("django_celery_beat.tzcrontab.datetime")
def test_due_when_within_starting_deadline(self, mock_datetime):
"""
Test that a task is due if last_run_at is within
beat_cron_starting_deadline.
"""
# Create a mock app with 5 minute beat_cron_starting_deadline
app = MagicMock()
app.conf.beat_cron_starting_deadline = 300 # 5 minutes in seconds

# Set current time to 12:05:50
mock_now_utc = datetime(
2023, 10, 26, 12, 5, 50, tzinfo=dt_timezone.utc
)
mock_datetime.now.return_value = mock_now_utc

# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
schedule = TzAwareCrontab(app=app, tz=dt_timezone.utc, minute="*/5")

# Last run was 290 seconds ago (12:01:00) - within the deadline window
last_run_at_utc = mock_now_utc - timedelta(seconds=290)

# Next scheduled run should be 12:05:00, but current time is 12:05:50
# So the task is due, and the deadline check doesn't prevent execution
due_status, next_check_delta = schedule.is_due(last_run_at_utc)
assert due_status is True

@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
@patch("django_celery_beat.tzcrontab.datetime")
def test_not_due_when_outside_starting_deadline(self, mock_datetime):
"""
Test that a task is NOT due if last_run_at is outside (older than)
beat_cron_starting_deadline.
"""
# Create a mock app with 2 minute beat_cron_starting_deadline
app = MagicMock()
app.conf.beat_cron_starting_deadline = 120 # 2 minutes in seconds

# Set current time to 12:09:50
mock_now_utc = datetime(
2023, 10, 26, 12, 9, 50, tzinfo=dt_timezone.utc
)
mock_datetime.now.return_value = mock_now_utc

# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
schedule_utc = TzAwareCrontab(
app=app, tz=dt_timezone.utc, minute="*/5"
)

# Last run was 310 seconds ago (12:04:40) - outside the deadline window
last_run_at_utc = mock_now_utc - timedelta(seconds=310)

# Next scheduled run after 12:04:40 would be 12:05:00
# This is in the past relative to current time, so normally due
# BUT since last_run_at is before the deadline window, it's NOT due
due_status, next_check_delta = schedule_utc.is_due(last_run_at_utc)
assert due_status is False

@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
@patch("django_celery_beat.tzcrontab.datetime")
def test_not_due_with_recent_run(self, mock_datetime):
"""
Test that a task is not due if last_run_at is recent,
even with a starting_deadline set.
"""
# Create a mock app with 5 minute beat_cron_starting_deadline
app = MagicMock()
app.conf.beat_cron_starting_deadline = 300 # 5 minutes in seconds

# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
schedule = TzAwareCrontab(app=app, tz=dt_timezone.utc, minute="*/5")

# Set current time to 12:04:30 (before next scheduled execution)
mock_now_utc_early = datetime(
2023, 10, 26, 12, 4, 30, tzinfo=dt_timezone.utc
)
mock_datetime.now.return_value = mock_now_utc_early

# Last run was at 12:04:00
# The next schedule would be at 12:05:00, which is in the future
last_run_at_recent = datetime(
2023, 10, 26, 12, 4, 0, tzinfo=dt_timezone.utc
)

# Calculate if the task is due
# Since the next execution time is in the future, the task is not due
# The deadline check doesn't matter for tasks not yet scheduled to run
due_status, next_check_delta = schedule.is_due(last_run_at_recent)
assert due_status is False

@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
@patch("django_celery_beat.tzcrontab.datetime")
def test_due_with_no_starting_deadline_set(self, mock_datetime):
"""
Test that a task is due if last_run_at is old and no deadline is set.
"""
# Create a mock app with no beat_cron_starting_deadline
app = MagicMock()
app.conf.beat_cron_starting_deadline = None

# Set current time to 12:10:00
mock_now_utc = datetime(
2023, 10, 26, 12, 10, 0, tzinfo=dt_timezone.utc
)
mock_datetime.now.return_value = mock_now_utc

# Create a schedule that runs every 5 minutes (12:00, 12:05, etc.)
schedule_utc = TzAwareCrontab(
app=app, tz=dt_timezone.utc, minute="*/5"
)

# Last run was 310 seconds ago (12:04:50)
# With no deadline, age of the last run doesn't matter
last_run_at_utc = mock_now_utc - timedelta(seconds=310)

# Next scheduled time after 12:04:50 would be 12:05:00
# Current time is 12:10:00, so this is in the past
# With no deadline check, the task is due to run
due_status, next_check_delta = schedule_utc.is_due(
last_run_at_utc
)
assert due_status is True

@override_settings(DJANGO_CELERY_BEAT_TZ_AWARE=True)
@patch("django_celery_beat.tzcrontab.datetime")
def test_due_with_starting_deadline_non_utc_timezone(self, mock_datetime):
"""
Test with a non-UTC timezone for the schedule.
"""
# Create a mock app with 5 minute beat_cron_starting_deadline
app = MagicMock()
app.conf.beat_cron_starting_deadline = 300 # 5 minutes in seconds
app.timezone = ZoneInfo("America/New_York")

# Use New York timezone for the schedule
schedule_tz = ZoneInfo("America/New_York")

# Set current time to 08:05:00 New York time
mock_now_ny = datetime(2023, 10, 26, 8, 5, 0, tzinfo=schedule_tz)
mock_datetime.now.return_value = mock_now_ny

# Create a schedule that runs every 5 minutes in NY time
schedule_ny = TzAwareCrontab(app=app, tz=schedule_tz, minute="*/5")

# Last run was 290 seconds ago - within deadline window
# The deadline window starts at 08:00:00 (current time - 5 minutes)
# Since 08:00:10 is after 08:00:00, it's within the deadline window
last_run_at_ny = mock_now_ny - timedelta(seconds=290)

# Next scheduled time after 08:00:10 would be 08:05:00
# Current time is 08:05:00, so this is due
# The deadline check doesn't prevent execution
due_status, next_check_delta = schedule_ny.is_due(last_run_at_ny)
assert due_status is True


@pytest.mark.django_db
class test_ModelEntry(SchedulerCase):
Entry = EntryTrackSave
Expand Down Expand Up @@ -1465,11 +1630,14 @@ def teardown_method(self):

@patch("django_celery_beat.schedulers.aware_now")
def test_server_timezone_handling_with_zoneinfo(self, mock_aware_now):
"""Test handling when server timezone is already a ZoneInfo instance."""
"""Test handling when server timezone
Copy link

Copilot AI May 17, 2025

Choose a reason for hiding this comment

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

[nitpick] Consider reformatting the multi-line docstring in the test_server_timezone_handling_with_zoneinfo test for consistency and clarity.

Copilot uses AI. Check for mistakes.
is already a ZoneInfo instance."""

# Create a mock scheduler with only the methods we need to test
class MockScheduler:
_get_timezone_offset = schedulers.DatabaseScheduler._get_timezone_offset
_get_timezone_offset = (
schedulers.DatabaseScheduler._get_timezone_offset
)

s = MockScheduler()

Expand All @@ -1490,7 +1658,9 @@ def test_timezone_offset_with_zoneinfo_object_param(self, mock_aware_now):
"""Test handling when timezone_name parameter is a ZoneInfo object."""

class MockScheduler:
_get_timezone_offset = schedulers.DatabaseScheduler._get_timezone_offset
_get_timezone_offset = (
schedulers.DatabaseScheduler._get_timezone_offset
)

s = MockScheduler()

Expand Down