Skip to content

Commit 0ced98a

Browse files
author
Ask Solem
committed
Don't use the registry for schedules. Use setup_schedule() to set up the schedule.
This is part of the on-going work to move the schedule away from the registry to a database instead. The PeriodicTask classes *will* be replaced with something much more flexible.
1 parent 5fa05de commit 0ced98a

File tree

1 file changed

+51
-39
lines changed

1 file changed

+51
-39
lines changed

celery/beat.py

Lines changed: 51 additions & 39 deletions
Original file line numberDiff line numberDiff line change
@@ -12,8 +12,8 @@
1212

1313
from celery import log
1414
from celery import conf
15-
from celery import registry as _registry
1615
from celery import platform
16+
from celery.execute import send_task
1717
from celery.messaging import establish_connection
1818
from celery.utils.info import humanize_seconds
1919

@@ -25,13 +25,28 @@ class SchedulingError(Exception):
2525
class ScheduleEntry(object):
2626
"""An entry in the scheduler.
2727
28-
:param task: see :attr:`task`.
28+
:param name: see :attr:`name`.
29+
:param schedule: see :attr:`schedule`.
30+
:param args: see :attr:`args`.
31+
:param kwargs: see :attr:`kwargs`.
2932
:keyword last_run_at: see :attr:`last_run_at`.
3033
:keyword total_run_count: see :attr:`total_run_count`.
3134
32-
.. attribute:: task
35+
.. attribute:: name
3336
34-
The task class.
37+
The task name.
38+
39+
.. attribute:: schedule
40+
41+
The schedule (run_every/crontab)
42+
43+
.. attribute:: args
44+
45+
Args to apply.
46+
47+
.. attribute:: kwargs
48+
49+
Keyword arguments to apply.
3550
3651
.. attribute:: last_run_at
3752
@@ -43,35 +58,39 @@ class ScheduleEntry(object):
4358
4459
"""
4560

46-
def __init__(self, name, last_run_at=None, total_run_count=None):
61+
def __init__(self, name, schedule, args=(), kwargs={},
62+
options={}, last_run_at=None, total_run_count=None):
4763
self.name = name
64+
self.schedule = schedule
65+
self.args = args
66+
self.kwargs = kwargs
67+
self.options = options
4868
self.last_run_at = last_run_at or datetime.now()
4969
self.total_run_count = total_run_count or 0
5070

5171
def next(self):
5272
"""Returns a new instance of the same class, but with
5373
its date and count fields updated."""
5474
return self.__class__(self.name,
75+
self.schedule,
76+
self.args,
77+
self.kwargs,
78+
self.options,
5579
datetime.now(),
5680
self.total_run_count + 1)
5781

58-
def is_due(self, task):
82+
def is_due(self):
5983
"""See :meth:`celery.task.base.PeriodicTask.is_due`."""
60-
return task.is_due(self.last_run_at)
84+
return self.schedule.is_due(self.last_run_at)
6185

6286

6387
class Scheduler(UserDict):
6488
"""Scheduler for periodic tasks.
6589
66-
:keyword registry: see :attr:`registry`.
6790
:keyword schedule: see :attr:`schedule`.
6891
:keyword logger: see :attr:`logger`.
6992
:keyword max_interval: see :attr:`max_interval`.
7093
71-
.. attribute:: registry
72-
73-
The task registry to use.
74-
7594
.. attribute:: schedule
7695
7796
The schedule dict/shelve.
@@ -86,17 +105,16 @@ class Scheduler(UserDict):
86105
87106
"""
88107

89-
def __init__(self, registry=None, schedule=None, logger=None,
108+
def __init__(self, schedule=None, logger=None,
90109
max_interval=None):
91-
self.registry = registry or _registry.TaskRegistry()
92110
self.data = schedule
93111
if self.data is None:
94112
self.data = {}
95113
self.logger = logger or log.get_default_logger()
96114
self.max_interval = max_interval or conf.CELERYBEAT_MAX_LOOP_INTERVAL
97115

98116
self.cleanup()
99-
self.schedule_registry()
117+
self.setup_schedule()
100118

101119
def tick(self):
102120
"""Run a tick, that is one iteration of the scheduler.
@@ -108,55 +126,51 @@ def tick(self):
108126
connection = establish_connection()
109127
try:
110128
for entry in self.schedule.values():
111-
is_due, next_time_to_run = self.is_due(entry)
129+
is_due, next_time_to_run = entry.is_due()
112130
if is_due:
113131
debug("Scheduler: Sending due task %s" % entry.name)
114132
try:
115-
result = self.apply_async(entry, connection=connection)
133+
result = self.apply_async(entry,
134+
connection=connection)
116135
except SchedulingError, exc:
117136
error("Scheduler: %s" % exc)
118137
else:
119-
debug("%s sent. id->%s" % (entry.name, result.task_id))
138+
debug("%s sent. id->%s" % (entry.name,
139+
result.task_id))
120140
if next_time_to_run:
121141
remaining_times.append(next_time_to_run)
122142
finally:
123143
connection.close()
124144

125145
return min(remaining_times + [self.max_interval])
126146

127-
def get_task(self, name):
128-
return self.registry[name]
129-
130-
def is_due(self, entry):
131-
return entry.is_due(self.get_task(entry.name))
132-
133147
def apply_async(self, entry, **kwargs):
134148

135149
# Update timestamps and run counts before we actually execute,
136150
# so we have that done if an exception is raised (doesn't schedule
137151
# forever.)
138152
entry = self.schedule[entry.name] = entry.next()
139-
task = self.get_task(entry.name)
140153

141154
try:
142-
result = task.apply_async(**kwargs)
155+
result = send_task(entry.name, entry.args, entry.kwargs,
156+
**entry.options)
143157
except Exception, exc:
144158
raise SchedulingError("Couldn't apply scheduled task %s: %s" % (
145-
task.name, exc))
159+
entry.name, exc))
146160
return result
147161

148-
def schedule_registry(self):
149-
"""Add the current contents of the registry to the schedule."""
150-
for name, task in self.registry.periodic().items():
151-
if name not in self.schedule:
152-
self.logger.debug("Scheduler: "
153-
"Added periodic task %s to schedule" % name)
154-
self.schedule.setdefault(name, ScheduleEntry(task.name))
162+
def setup_schedule(self):
163+
from datetime import timedelta
164+
from celery.task.schedules import schedule
165+
self.schedule["add.often"] = ScheduleEntry("tasks.add",
166+
schedule(timedelta(seconds=5)),
167+
args=(4, 4))
168+
self.schedule["sleep.often"] = ScheduleEntry("tasks.sleeptask",
169+
schedule(timedelta(minutes=1)),
170+
args=(2, ))
155171

156172
def cleanup(self):
157-
for task_name, entry in self.schedule.items():
158-
if task_name not in self.registry:
159-
self.schedule.pop(task_name, None)
173+
pass
160174

161175
@property
162176
def schedule(self):
@@ -165,7 +179,6 @@ def schedule(self):
165179

166180
class ClockService(object):
167181
scheduler_cls = Scheduler
168-
registry = _registry.tasks
169182
open_schedule = lambda self, filename: shelve.open(filename)
170183

171184
def __init__(self, logger=None,
@@ -231,7 +244,6 @@ def schedule(self):
231244
def scheduler(self):
232245
if self._scheduler is None:
233246
self._scheduler = self.scheduler_cls(schedule=self.schedule,
234-
registry=self.registry,
235247
logger=self.logger,
236248
max_interval=self.max_interval)
237249
return self._scheduler

0 commit comments

Comments
 (0)