Преглед на файлове

Sphinx extension resolved to celery.attr instead of app.attr, also faq referred to outdated celery.execute module

Ask Solem преди 10 години
родител
ревизия
6ea6f24558

+ 7 - 1
celery/app/utils.py

@@ -51,7 +51,13 @@ def appstr(app):
 
 
 class Settings(ConfigurationView):
-    """Celery settings object."""
+    """Celery settings object.
+
+    .. seealso:
+
+        :ref:`configuration` for a full list of configuration keys.
+
+    """
 
     @property
     def CELERY_RESULT_BACKEND(self):

+ 20 - 3
docs/_ext/celerydocs.py

@@ -5,6 +5,7 @@ from sphinx.environment import NoUri
 APPATTRS = {
     'amqp': 'celery.app.amqp.AMQP',
     'backend': 'celery.backends.base.BaseBackend',
+    'conf': 'celery.app.utils.Settings',
     'control': 'celery.app.control.Control',
     'events': 'celery.events.Events',
     'loader': 'celery.app.loaders.base.BaseLoader',
@@ -13,15 +14,31 @@ APPATTRS = {
     'tasks': 'celery.app.registry.Registry',
 
     'AsyncResult': 'celery.result.AsyncResult',
+    'ResultSet': 'celery.result.ResultSet',
     'GroupResult': 'celery.result.GroupResult',
     'Worker': 'celery.apps.worker.Worker',
     'WorkController': 'celery.worker.WorkController',
     'Beat': 'celery.apps.beat.Beat',
     'Task': 'celery.app.task.Task',
-    'send_task': 'celery.Celery.send_task',
-    'connection': 'celery.Celery.connection',
+    'signature': 'celery.canvas.Signature',
 }
 
+APPDIRECT = {
+    'on_configure', 'on_after_configure', 'on_after_finalize',
+    'set_current', 'set_default', 'close', 'on_init', 'start',
+    'worker_main', 'task', 'gen_task_name', 'finalize',
+    'add_defaults', 'config_from_object', 'config_from_envvar',
+    'config_from_cmdline', 'setup_security', 'autodiscover_tasks',
+    'send_task', 'connection', 'connection_or_acquire',
+    'producer_or_acquire', 'prepare_config', 'now', 'mail_admins',
+    'select_queues', 'either', 'bugreport', 'create_task_cls',
+    'subclass_with_self', 'annotations', 'current_task', 'oid',
+    'timezone', '__reduce_keys__', 'fixups', 'finalized', 'configured',
+    'autofinalize', 'steps', 'user_options', 'main', 'clock',
+}
+
+APPATTRS.update({x: 'celery.Celery.{0}'.format(x) for x in APPDIRECT})
+
 ABBRS = {
     'Celery': 'celery.Celery',
 }
@@ -43,7 +60,7 @@ def shorten(S, newtarget, src_dict):
         return S[2:]
     elif S.startswith('@'):
         if src_dict is APPATTRS:
-            return '.'.join([pkg_of(newtarget), S[1:]])
+            return '.'.join(['app', S[1:]])
         return S[1:]
     return S
 

+ 3 - 4
docs/faq.rst

@@ -425,7 +425,7 @@ using the tasks current result backend.
 
 If you need to specify a custom result backend, or you want to use
 the current application's default backend you can use
-:class:`@Celery.AsyncResult`:
+:class:`@AsyncResult`:
 
     >>> result = app.AsyncResult(task_id)
     >>> result.get()
@@ -600,12 +600,11 @@ queue for exchange, so that rejected messages is moved there.
 Can I call a task by name?
 -----------------------------
 
-**Answer**: Yes. Use :func:`celery.execute.send_task`.
+**Answer**: Yes. Use :meth:`@send_task`.
 You can also call a task by name from any language
 that has an AMQP client.
 
-    >>> from celery.execute import send_task
-    >>> send_task("tasks.add", args=[2, 2], kwargs={})
+    >>> app.send_task('tasks.add', args=[2, 2], kwargs={})
     <AsyncResult: 373550e8-b9a0-4666-bc61-ace01fa4f91d>
 
 .. _faq-get-current-task-id:

+ 1 - 1
docs/getting-started/first-steps-with-celery.rst

@@ -316,7 +316,7 @@ you can also imagine your SysAdmin making simple changes to the configuration
 in the event of system trouble.
 
 You can tell your Celery instance to use a configuration module,
-by calling the :meth:`~@Celery.config_from_object` method:
+by calling the :meth:`@config_from_object` method:
 
 .. code-block:: python
 

+ 10 - 10
docs/history/changelog-3.0.rst

@@ -353,7 +353,7 @@ If you're looking for versions prior to 3.0.x you should go to :ref:`history`.
 - The ``pool_restart`` remote control command now reports
   an error if the :setting:`CELERYD_POOL_RESTARTS` setting is not set.
 
-- ``celery.conf.add_defaults`` can now be used with non-dict objects.
+- :meth:`@add_defaults`` can now be used with non-dict objects.
 
 - Fixed compatibility problems in the Proxy class (Issue #1087).
 
@@ -1108,7 +1108,7 @@ If you're looking for versions prior to 3.0.x you should go to :ref:`history`.
 
 - App instances now supports the with statement.
 
-    This calls the new :meth:`~celery.Celery.close` method at exit, which
+    This calls the new :meth:`@close` method at exit, which
     cleans up after the app like closing pool connections.
 
     Note that this is only necessary when dynamically creating apps,
@@ -1411,16 +1411,16 @@ If you're looking for versions prior to 3.0.x you should go to :ref:`history`.
                 }
             }
 
-- New :meth:`@Celery.add_defaults` method can add new default configuration
+- New :meth:`@add_defaults` method can add new default configuration
   dicts to the applications configuration.
 
     For example::
 
         config = {'FOO': 10}
 
-        celery.add_defaults(config)
+        app.add_defaults(config)
 
-    is the same as ``celery.conf.update(config)`` except that data will not be
+    is the same as ``app.conf.update(config)`` except that data will not be
     copied, and that it will not be pickled when the worker spawns child
     processes.
 
@@ -1429,7 +1429,7 @@ If you're looking for versions prior to 3.0.x you should go to :ref:`history`.
         def initialize_config():
             # insert heavy stuff that can't be done at import time here.
 
-        celery.add_defaults(initialize_config)
+        app.add_defaults(initialize_config)
 
     which means the same as the above except that it will not happen
     until the celery configuration is actually used.
@@ -1437,8 +1437,8 @@ If you're looking for versions prior to 3.0.x you should go to :ref:`history`.
     As an example, Celery can lazily use the configuration of a Flask app::
 
         flask_app = Flask()
-        celery = Celery()
-        celery.add_defaults(lambda: flask_app.config)
+        app = Celery()
+        app.add_defaults(lambda: flask_app.config)
 
 - Revoked tasks were not marked as revoked in the result backend (Issue #871).
 
@@ -1455,8 +1455,8 @@ If you're looking for versions prior to 3.0.x you should go to :ref:`history`.
 
 - New method names:
 
-    - ``Celery.default_connection()`` ➠  :meth:`~@Celery.connection_or_acquire`.
-    - ``Celery.default_producer()``   ➠  :meth:`~@Celery.producer_or_acquire`.
+    - ``Celery.default_connection()`` ➠  :meth:`~@connection_or_acquire`.
+    - ``Celery.default_producer()``   ➠  :meth:`~@producer_or_acquire`.
 
     The old names still work for backward compatibility.
 

+ 13 - 12
docs/userguide/application.rst

@@ -67,7 +67,8 @@ This is only a problem in a limited set of use cases:
     #. If the module that the task is defined in is run as a program.
     #. If the application is created in the Python shell (REPL).
 
-For example here, where the tasks module is also used to start a worker:
+For example here, where the tasks module is also used to start a worker
+with :meth:`@worker_main`:
 
 :file:`tasks.py`:
 
@@ -114,7 +115,7 @@ There are several options you can set that will change how
 Celery works.  These options can be set directly on the app instance,
 or you can use a dedicated configuration module.
 
-The configuration is available as :attr:`@Celery.conf`::
+The configuration is available as :attr:`@conf`::
 
     >>> app.conf.CELERY_TIMEZONE
     'Europe/London'
@@ -137,7 +138,7 @@ that are consulted in order:
     #. The configuration module (if any)
     #. The default configuration (:mod:`celery.app.defaults`).
 
-You can even add new default sources by using the :meth:`@Celery.add_defaults`
+You can even add new default sources by using the :meth:`@add_defaults`
 method.
 
 .. seealso::
@@ -148,13 +149,13 @@ method.
 ``config_from_object``
 ----------------------
 
-The :meth:`@Celery.config_from_object` method loads configuration
+The :meth:`@config_from_object` method loads configuration
 from a configuration object.
 
 This can be a configuration module, or any object with configuration attributes.
 
 Note that any configuration that was previous set will be reset when
-:meth:`~@Celery.config_from_object` is called.  If you want to set additional
+:meth:`~@config_from_object` is called.  If you want to set additional
 configuration you should do so after.
 
 Example 1: Using the name of a module
@@ -216,7 +217,7 @@ Example 3:  Using a configuration class/object
 ``config_from_envvar``
 ----------------------
 
-The :meth:`@Celery.config_from_envvar` takes the configuration module name
+The :meth:`@config_from_envvar` takes the configuration module name
 from an environment variable
 
 For example -- to load configuration from a module specified in the
@@ -288,9 +289,9 @@ Creating a :class:`@Celery` instance will only do the following:
     #. Create the task registry.
     #. Set itself as the current app (but not if the ``set_as_current``
        argument was disabled)
-    #. Call the :meth:`@Celery.on_init` callback (does nothing by default).
+    #. Call the :meth:`@on_init` callback (does nothing by default).
 
-The :meth:`~@Celery.task` decorator does not actually create the
+The :meth:`@task` decorator does not actually create the
 tasks at the point when it's called, instead it will defer the creation
 of the task to happen either when the task is used, or after the
 application has been *finalized*,
@@ -317,7 +318,7 @@ you use the task, or access an attribute (in this case :meth:`repr`):
     True
 
 *Finalization* of the app happens either explicitly by calling
-:meth:`@Celery.finalize` -- or implicitly by accessing the :attr:`~@Celery.tasks`
+:meth:`@finalize` -- or implicitly by accessing the :attr:`@tasks`
 attribute.
 
 Finalizing the object will:
@@ -464,8 +465,8 @@ chain breaks:
 Abstract Tasks
 ==============
 
-All tasks created using the :meth:`~@Celery.task` decorator
-will inherit from the applications base :attr:`~@Celery.Task` class.
+All tasks created using the :meth:`~@task` decorator
+will inherit from the applications base :attr:`~@Task` class.
 
 You can specify a different base class with the ``base`` argument:
 
@@ -504,7 +505,7 @@ Once a task is bound to an app it will read configuration to set default values
 and so on.
 
 It's also possible to change the default base class for an application
-by changing its :meth:`@Celery.Task` attribute:
+by changing its :meth:`@Task` attribute:
 
 .. code-block:: python
 

+ 1 - 1
docs/userguide/extending.rst

@@ -613,7 +613,7 @@ Command-specific options
 ~~~~~~~~~~~~~~~~~~~~~~~~
 
 You can add additional command-line options to the ``worker``, ``beat`` and
-``events`` commands by modifying the :attr:`~@Celery.user_options` attribute of the
+``events`` commands by modifying the :attr:`~@user_options` attribute of the
 application instance.
 
 Celery commands uses the :mod:`optparse` module to parse command-line

+ 3 - 3
docs/userguide/tasks.rst

@@ -45,7 +45,7 @@ Basics
 ======
 
 You can easily create a task from any callable by using
-the :meth:`~@Celery.task` decorator:
+the :meth:`~@task` decorator:
 
 .. code-block:: python
 
@@ -240,7 +240,7 @@ like `moduleA.tasks.taskA`, `moduleA.tasks.taskB`, `moduleB.tasks.test`
 and so on. You may want to get rid of having `tasks` in all task names.
 As pointed above, you can explicitly give names for all tasks, or you
 can change the automatic naming behavior by overriding
-:meth:`~@Celery.gen_task_name`. Continuing with the example, `celery.py`
+:meth:`@gen_task_name`. Continuing with the example, `celery.py`
 may contain:
 
 .. code-block:: python
@@ -261,7 +261,7 @@ So each task will have a name like `moduleA.taskA`, `moduleA.taskB` and
 
 .. warning::
 
-    Make sure that your `gen_task_name` is a pure function, which means
+    Make sure that your :meth:`@gen_task_name` is a pure function, which means
     that for the same input it must always return the same output.
 
 .. _task-request-info:

+ 1 - 1
docs/userguide/workers.rst

@@ -810,7 +810,7 @@ Inspecting workers
 uses remote control commands under the hood.
 
 You can also use the ``celery`` command to inspect workers,
-and it supports the same commands as the :class:`@Celery.control` interface.
+and it supports the same commands as the :class:`@control` interface.
 
 .. code-block:: python
 

+ 1 - 1
docs/whatsnew-3.0.rst

@@ -656,7 +656,7 @@ decorator executes inline like before, however for custom apps the @task
 decorator now returns a special PromiseProxy object that is only evaluated
 on access.
 
-All promises will be evaluated when `app.finalize` is called, or implicitly
+All promises will be evaluated when :meth:`@finalize` is called, or implicitly
 when the task registry is first used.
 
 

+ 3 - 3
docs/whatsnew-3.1.rst

@@ -716,7 +716,7 @@ In Other News
 -  New ability to specify additional command line options
    to the worker and beat programs.
 
-    The :attr:`@Celery.user_options` attribute can be used
+    The :attr:`@user_options` attribute can be used
     to add additional command-line arguments, and expects
     optparse-style options:
 
@@ -1053,7 +1053,7 @@ In Other News
     This is the mapping of parsed command line arguments, and can be used to
     prepare new preload arguments (``app.user_options['preload']``).
 
-- New callback: ``Celery.on_configure``.
+- New callback: :meth:`@on_configure`.
 
     This callback is called when an app is about to be configured (a
     configuration key is required).
@@ -1264,7 +1264,7 @@ Internal changes
     This removes a lot of duplicate functionality.
 
 - The ``Celery.with_default_connection`` method has been removed in favor
-  of ``with app.connection_or_acquire``.
+  of ``with app.connection_or_acquire`` (:meth:`@connection_or_acquire`)
 
 - The ``celery.results.BaseDictBackend`` class has been removed and is replaced by
   :class:`celery.results.BaseBackend`.