Ask Solem vor 14 Jahren
Ursprung
Commit
8d85334016

+ 17 - 2
celery/schedules.py

@@ -22,7 +22,22 @@ class schedule(object):
         """Returns tuple of two items ``(is_due, next_time_to_run)``,
         where next time to run is in seconds.
 
-        See :meth:`celery.task.base.PeriodicTask.is_due` for more information.
+        e.g.
+
+        * ``(True, 20)``, means the task should be run now, and the next
+            time to run is in 20 seconds.
+
+        * ``(False, 12)``, means the task should be run in 12 seconds.
+
+        You can override this to decide the interval at runtime,
+        but keep in mind the value of :setting:`CELERYBEAT_MAX_LOOP_INTERVAL`,
+        which decides the maximum number of seconds celerybeat can sleep
+        between re-checking the periodic task intervals.  So if you
+        dynamically change the next run at value, and the max interval is
+        set to 5 minutes, it will take 5 minutes for the change to take
+        effect, so you may consider lowering the value of
+        :setting:`CELERYBEAT_MAX_LOOP_INTERVAL` if responsiveness is of
+        importance to you.
 
         """
         rem_delta = self.remaining_estimate(last_run_at)
@@ -280,7 +295,7 @@ class crontab(schedule):
         """Returns tuple of two items ``(is_due, next_time_to_run)``,
         where next time to run is in seconds.
 
-        See :meth:`celery.task.base.PeriodicTask.is_due` for more information.
+        See :meth:`celery.schedules.schedule.is_due` for more information.
 
         """
         rem_delta = self.remaining_estimate(last_run_at)

+ 1 - 16
celery/task/base.py

@@ -816,22 +816,7 @@ class PeriodicTask(Task):
         """Returns tuple of two items ``(is_due, next_time_to_run)``,
         where next time to run is in seconds.
 
-        e.g.
-
-        * ``(True, 20)``, means the task should be run now, and the next
-            time to run is in 20 seconds.
-
-        * ``(False, 12)``, means the task should be run in 12 seconds.
-
-        You can override this to decide the interval at runtime,
-        but keep in mind the value of :setting:`CELERYBEAT_MAX_LOOP_INTERVAL`,
-        which decides the maximum number of seconds celerybeat can sleep
-        between re-checking the periodic task intervals.  So if you
-        dynamically change the next run at value, and the max interval is
-        set to 5 minutes, it will take 5 minutes for the change to take
-        effect, so you may consider lowering the value of
-        :setting:`CELERYBEAT_MAX_LOOP_INTERVAL` if responsiveness is of
-        importance to you.
+        See :meth:`celery.schedules.schedule.is_due` for more information.
 
         """
         return self.run_every.is_due(last_run_at)

+ 1 - 1
docs/configuration.rst

@@ -778,7 +778,7 @@ CELERY_STORE_ERRORS_EVEN_IF_IGNORED
 If set, the worker stores all task errors in the result store even if
 :attr:`Task.ignore_result <celery.task.base.Task.ignore_result>` is on.
 
-.. setting:: CELERY_STATE_DB
+.. setting:: CELERYD_STATE_DB
 
 CELERYD_STATE_DB
 ~~~~~~~~~~~~~~~~

+ 21 - 0
docs/userguide/executing.rst

@@ -89,6 +89,27 @@ have a :class:`~datetime.datetime` object and need to modify it with a
         tomorrow = datetime.now() + timedelta(days=1)
         add.apply_async(args=[10, 10], eta=tomorrow)
 
+.. _executing-expiration:
+
+Expiration
+==========
+
+The ``expires`` argument defines an optional expiry time,
+either as seconds after task publish, or a specific date and time using
+:class:~datetime.datetime`.
+
+    >>> # Task expires after one minute from now.
+    >>> add.apply_async(args=[10, 10], expires=60)
+
+    >>> # Also supports datetime
+    >>> from datetime import datetime, timedelta
+    >>> add.apply_async(args=[10, 10], kwargs,
+    ...                 expires=datetime.now() + timedelta(days=1)
+
+
+When a worker receives a task that has been expired it will mark
+the task as :state:`REVOKED` (:exc:`~celery.exceptions.TaskRevokedError`).
+
 .. _executing-serializers:
 
 Serializers

+ 5 - 0
docs/userguide/tasks.rst

@@ -7,6 +7,11 @@
 .. contents::
     :local:
 
+
+This guide gives an overview of how tasks are defined. For a complete
+listing of task attributes and methods, please see the
+:class:`API reference <celery.task.base.Task>`.
+
 .. _task-basics:
 
 Basics

+ 13 - 0
docs/userguide/workers.rst

@@ -87,6 +87,19 @@ may perform better than having a single worker. For example 3 celeryd's with
 works best for you as this varies based on application, work load, task
 run times and other factors.
 
+.. _worker-persistent-revokes:
+
+Persistent revokes
+==================
+
+Revoking tasks works by sending a broadcast message to all the workers,
+the workers then keep a list of revoked tasks in memory.
+
+If you want tasks to remain revoked after worker restart you need to
+specify a file for these to be stored in, either by using the ``--statedb``
+argument to :mod:`~celery.bin.celeryd` or the :setting:`CELERYD_STATE_DB`
+setting. See :setting:`CELERYD_STATE_DB` for more information.
+
 .. _worker-time-limits:
 
 Time limits