app.autodiscover_tasks is now lazy

This commit is contained in:
Ask Solem 2013-11-19 14:09:50 +00:00
Родитель bcb3074f55
Коммит a65b6748af
8 изменённых файлов: 70 добавлений и 10 удалений

Просмотреть файл

@ -17,12 +17,14 @@ from contextlib import contextmanager
from copy import deepcopy
from operator import attrgetter
from amqp import promise
from billiard.util import register_after_fork
from kombu.clocks import LamportClock
from kombu.common import oid_from
from kombu.utils import cached_property, uuid
from celery import platforms
from celery import signals
from celery._state import (
_task_stack, _tls, get_current_app, set_default_app,
_register_app, get_current_worker_task,
@ -257,13 +259,13 @@ class Celery(object):
del(self.conf)
return self.loader.config_from_object(obj, silent=silent)
def config_from_envvar(self, variable_name, silent=False):
def config_from_envvar(self, variable_name, silent=False, force=False):
module_name = os.environ.get(variable_name)
if not module_name:
if silent:
return False
raise ImproperlyConfigured(ERR_ENVVAR_NOT_SET.format(module_name))
return self.config_from_object(module_name, silent=silent)
return self.config_from_object(module_name, silent=silent, force=force)
def config_from_cmdline(self, argv, namespace='celery'):
self.conf.update(self.loader.cmdline_config_parser(argv, namespace))
@ -274,7 +276,16 @@ class Celery(object):
return setup_security(allowed_serializers, key, cert,
store, digest, serializer, app=self)
def autodiscover_tasks(self, packages, related_name='tasks'):
def autodiscover_tasks(self, packages, related_name='tasks', force=False):
if force:
return self._autodiscover_tasks(packages, related_name)
signals.import_modules.connect(promise(
self._autodiscover_tasks, (packages, related_name),
), weak=False, sender=self)
def _autodiscover_tasks(self, packages, related_name='tasks', **kwargs):
# argument may be lazy
packages = packages() if isinstance(packages, Callable) else packages
if self.conf.CELERY_FORCE_BILLIARD_LOGGING:
# we'll use billiard's processName instead of
# multiprocessing's one in all the loggers

Просмотреть файл

@ -20,6 +20,7 @@ from datetime import datetime
from kombu.utils import cached_property
from kombu.utils.encoding import safe_str
from celery import signals
from celery.datastructures import DictAttribute, force_mapping
from celery.five import reraise, string_t
from celery.utils.functional import maybe_list
@ -112,6 +113,7 @@ class BaseLoader(object):
)
def import_default_modules(self):
signals.import_modules.send(sender=self.app)
return [
self.import_task_module(m) for m in (
tuple(self.builtin_modules) +

Просмотреть файл

@ -52,6 +52,7 @@ task_revoked = Signal(providing_args=[
])
celeryd_init = Signal(providing_args=['instance', 'conf', 'options'])
celeryd_after_setup = Signal(providing_args=['instance', 'conf'])
import_modules = Signal(providing_args=[])
worker_init = Signal(providing_args=[])
worker_process_init = Signal(providing_args=[])
worker_process_shutdown = Signal(providing_args=[])

Просмотреть файл

@ -46,7 +46,7 @@ that defines the Celery instance:
app = Celery('proj')
app.config_from_object('django.conf:settings')
app.autodiscover_tasks(settings.INSTALLED_APPS, related_name='tasks')
app.autodiscover_tasks(lambda: settings.INSTALLED_APPS)
@app.task(bind=True)
def debug_task(self):
@ -113,7 +113,7 @@ autodiscover these modules:
.. code-block:: python
app.autodiscover_tasks(settings.INSTALLED_APPS, related_name='tasks')
app.autodiscover_tasks(lambda: settings.INSTALLED_APPS)
With the line above Celery will automatically discover tasks in reusable
apps if you follow the ``tasks.py`` convention::
@ -126,8 +126,9 @@ apps if you follow the ``tasks.py`` convention::
- app2/models.py
This way you do not have to manually add the individual modules
to the :setting:`CELERY_IMPORTS` setting.
to the :setting:`CELERY_IMPORTS` setting. The ``lambda`` so that the
autodiscovery can happen only when needed, and so that importing your
module will not evaluate the Django settings object.
Finally, the ``debug_task`` example is a task that dumps
its own request information. This is using the new ``bind=True`` task option

Просмотреть файл

@ -141,13 +141,16 @@ and creating Celery applications.
Return a string with information useful for the Celery core
developers when reporting a bug.
.. method:: Celery.config_from_object(obj, silent=False)
.. method:: Celery.config_from_object(obj, silent=False, force=False)
Reads configuration from object, where object is either
an object or the name of a module to import.
:keyword silent: If true then import errors will be ignored.
:keyword force: Force reading configuration immediately.
By default the configuration will be read only when required.
.. code-block:: python
>>> celery.config_from_object("myapp.celeryconfig")
@ -155,7 +158,8 @@ and creating Celery applications.
>>> from myapp import celeryconfig
>>> celery.config_from_object(celeryconfig)
.. method:: Celery.config_from_envvar(variable_name, silent=False)
.. method:: Celery.config_from_envvar(variable_name,
silent=False, force=False)
Read configuration from environment variable.
@ -188,6 +192,19 @@ and creating Celery applications.
Then calling ``app.autodiscover_tasks(['foo', bar', 'baz'])`` will
result in the modules ``foo.tasks`` and ``bar.tasks`` being imported.
:param packages: List of packages to search.
This argument may also be a callable, in which case the
value returned is used (for lazy evaluation).
:keyword related_name: The name of the module to find. Defaults
to "tasks", which means it look for "module.tasks" for every
module in ``packages``.
:keyword force: By default this call is lazy so that the actual
autodiscovery will not happen until an application imports the
default modules. Forcing will cause the autodiscovery to happen
immediately.
.. method:: Celery.add_defaults(d)
Add default configuration from dict ``d``.

Просмотреть файл

@ -261,6 +261,20 @@ Provides arguments:
* expired
Set to :const:`True` if the task expired.
App Signals
-----------
.. signal:: import_modules
import_modules
~~~~~~~~~~~~~~
This signal is sent when a program (worker, beat, shell) etc, asks
for modules in the :setting:`CELERY_INCLUDE` and :setting:`CELERY_IMPORTS`
settings to be imported.
Sender is the app instance.
Worker Signals
--------------

Просмотреть файл

@ -14,7 +14,7 @@ app = Celery('proj')
# Using a string here means the worker will not have to
# pickle the object when using Windows.
app.config_from_object('django.conf:settings')
app.autodiscover_tasks(settings.INSTALLED_APPS)
app.autodiscover_tasks(lambda: settings.INSTALLED_APPS)
@app.task(bind=True)

Просмотреть файл

@ -1,3 +1,17 @@
from __future__ import absolute_import
# ^^^ The above is required if you want to import from the celery
# library. If you don't have this then `from celery.schedules import`
# becomes `proj.celery.schedules` in Python 2.x since it allows
# for relative imports by default.
# Celery settings
BROKER_URL = 'amqp://guest:guest@localhost//'
#: Only add pickle to this list if your broker is secured
#: from unwanted access (see userguide/security.html)
CELERY_ACCEPT_CONTENT = ['json']
# Django settings for proj project.
DEBUG = True