Browse Source

Lowercase settings and settings cleanup (radical, but backwards compatible)

All settings are now in lowercase, and most of them have been renamed.

When loading settings the loader will look at the settings in the config
and decide if it's using old or new settings.
The settings will autmatically convert between old and new settings keys, depending
on the format the settings is in.

- It's not legal to mix new setting names and old setting names, that is unless
  the setting have two alternatives (old and new).

    An ImproperlyConfigured exceptions is rasised in this case, with help telling
    user exactly how to fix the problem.

- To support loading configuration from Django settings a new ``namespace``
  argument has been added to ``Celery`` and ``config_from_object``.

    This can be used from Django::

        app = Celery()
        app.config_from_object('django.conf:settings', namespace='CELERY_')

        # settings.py:
        CELERY_BROKER_URL = 'amqp://'
        CELERY_TASK_PROTOCOL = 2
        CELERY_TASK_ALWAYS_EAGER = True

    Or other apps wanting a prefix for some reason::

        app = Celery(namespace='celery_')
        app.conf.celery_task_always_eager = True
        app.conf.celery_task_routes = {'proj.tasks.add': 'math.yo'}

- Initial configuration directly on the app object is now lazy!

    You can set keys on an unfinalized app, without causing the tasks
    or the rest of the app to be evaluated:

        app = Celery()
        app.conf.update(
            task_default_delivery_mode=1,
            task_default_queue='default',
            task_default_exchange='default',
            task_default_routing_key='default',
        )
        app.conf.task_always_eager = True
        assert not app.configured  # <-- still not finalized

        app.config_from_object('celeryconfig')
        assert not app.configured  # <-- even now

        app.finalize()
        assert app.finalized       # <-- but now we are

        # and the config done first remains, unlike older versions of Celery.
        assert app.conf.task.default_queue == 'default'

        app.config_from_object(object())
        # but calling config_from_* again will reset everything.
        assert app.conf.task_default_queue == 'celery'

- ``config_from_*`` methods no longer override configuration set manually
  before the app was finalized.

    But calling again after the app is finalized, will clean out old
    configuration.
Ask Solem 9 years ago
parent
commit
53b5fdf3c5
100 changed files with 1589 additions and 1083 deletions
  1. 21 21
      celery/app/amqp.py
  2. 2 2
      celery/app/annotations.py
  3. 111 73
      celery/app/base.py
  4. 1 1
      celery/app/builtins.py
  5. 275 165
      celery/app/defaults.py
  6. 4 4
      celery/app/log.py
  7. 4 4
      celery/app/routes.py
  8. 24 25
      celery/app/task.py
  9. 2 2
      celery/app/trace.py
  10. 107 28
      celery/app/utils.py
  11. 7 11
      celery/apps/beat.py
  12. 8 9
      celery/apps/worker.py
  13. 3 3
      celery/backends/amqp.py
  14. 6 6
      celery/backends/base.py
  15. 2 2
      celery/backends/cache.py
  16. 8 8
      celery/backends/cassandra.py
  17. 1 1
      celery/backends/couchbase.py
  18. 6 6
      celery/backends/database/__init__.py
  19. 1 1
      celery/backends/mongodb.py
  20. 8 8
      celery/backends/new_cassandra.py
  21. 9 14
      celery/backends/redis.py
  22. 1 1
      celery/backends/riak.py
  23. 9 9
      celery/beat.py
  24. 1 1
      celery/bin/base.py
  25. 1 1
      celery/bin/beat.py
  26. 0 1
      celery/bin/celery.py
  27. 1 1
      celery/bin/graph.py
  28. 12 12
      celery/bin/worker.py
  29. 4 4
      celery/canvas.py
  30. 1 1
      celery/contrib/batches.py
  31. 1 1
      celery/contrib/migrate.py
  32. 37 15
      celery/datastructures.py
  33. 5 5
      celery/events/__init__.py
  34. 1 1
      celery/events/cursesmon.py
  35. 2 2
      celery/exceptions.py
  36. 9 8
      celery/loaders/base.py
  37. 2 2
      celery/schedules.py
  38. 7 7
      celery/security/__init__.py
  39. 1 1
      celery/states.py
  40. 5 5
      celery/task/base.py
  41. 1 1
      celery/task/sets.py
  42. 3 3
      celery/tests/app/test_amqp.py
  43. 168 42
      celery/tests/app/test_app.py
  44. 8 8
      celery/tests/app/test_beat.py
  45. 2 2
      celery/tests/app/test_builtins.py
  46. 23 6
      celery/tests/app/test_defaults.py
  47. 7 8
      celery/tests/app/test_loaders.py
  48. 1 1
      celery/tests/app/test_log.py
  49. 7 7
      celery/tests/app/test_routes.py
  50. 1 1
      celery/tests/backends/test_amqp.py
  51. 1 1
      celery/tests/backends/test_base.py
  52. 2 2
      celery/tests/backends/test_cache.py
  53. 6 6
      celery/tests/backends/test_cassandra.py
  54. 7 7
      celery/tests/backends/test_couchbase.py
  55. 2 2
      celery/tests/backends/test_database.py
  56. 4 4
      celery/tests/backends/test_mongodb.py
  57. 6 6
      celery/tests/backends/test_new_cassandra.py
  58. 6 6
      celery/tests/backends/test_redis.py
  59. 13 13
      celery/tests/backends/test_riak.py
  60. 3 3
      celery/tests/bin/test_base.py
  61. 4 4
      celery/tests/bin/test_worker.py
  62. 11 11
      celery/tests/case.py
  63. 2 2
      celery/tests/compat_modules/test_http.py
  64. 2 2
      celery/tests/compat_modules/test_sets.py
  65. 1 1
      celery/tests/events/test_events.py
  66. 3 3
      celery/tests/security/test_security.py
  67. 2 2
      celery/tests/tasks/test_canvas.py
  68. 3 3
      celery/tests/tasks/test_chord.py
  69. 2 2
      celery/tests/tasks/test_result.py
  70. 2 2
      celery/tests/tasks/test_tasks.py
  71. 8 4
      celery/tests/utils/test_datastructures.py
  72. 9 8
      celery/tests/worker/test_consumer.py
  73. 3 3
      celery/tests/worker/test_control.py
  74. 1 1
      celery/tests/worker/test_request.py
  75. 4 4
      celery/tests/worker/test_worker.py
  76. 10 1
      celery/utils/functional.py
  77. 35 42
      celery/worker/__init__.py
  78. 2 2
      celery/worker/components.py
  79. 10 10
      celery/worker/consumer.py
  80. 1 1
      celery/worker/control.py
  81. 1 1
      celery/worker/request.py
  82. 379 266
      docs/configuration.rst
  83. 11 5
      docs/django/first-steps-with-django.rst
  84. 5 5
      docs/faq.rst
  85. 1 1
      docs/getting-started/brokers/beanstalk.rst
  86. 1 1
      docs/getting-started/brokers/couchdb.rst
  87. 1 1
      docs/getting-started/brokers/django.rst
  88. 1 1
      docs/getting-started/brokers/ironmq.rst
  89. 1 1
      docs/getting-started/brokers/mongodb.rst
  90. 1 1
      docs/getting-started/brokers/rabbitmq.rst
  91. 7 7
      docs/getting-started/brokers/redis.rst
  92. 6 6
      docs/getting-started/brokers/sqlalchemy.rst
  93. 10 10
      docs/getting-started/brokers/sqs.rst
  94. 19 19
      docs/getting-started/first-steps-with-celery.rst
  95. 5 5
      docs/getting-started/next-steps.rst
  96. 1 1
      docs/glossary.rst
  97. 5 5
      docs/internals/app-overview.rst
  98. 23 23
      docs/internals/deprecation.rst
  99. 8 8
      docs/userguide/application.rst
  100. 7 7
      docs/userguide/calling.rst

+ 21 - 21
celery/app/amqp.py

@@ -245,7 +245,7 @@ class AMQP(object):
 
 
     @cached_property
     @cached_property
     def create_task_message(self):
     def create_task_message(self):
-        return self.task_protocols[self.app.conf.CELERY_TASK_PROTOCOL]
+        return self.task_protocols[self.app.conf.task_protocol]
 
 
     @cached_property
     @cached_property
     def send_task_message(self):
     def send_task_message(self):
@@ -257,15 +257,15 @@ class AMQP(object):
         from the current configuration."""
         from the current configuration."""
         conf = self.app.conf
         conf = self.app.conf
         if create_missing is None:
         if create_missing is None:
-            create_missing = conf.CELERY_CREATE_MISSING_QUEUES
+            create_missing = conf.task_create_missing_queues
         if ha_policy is None:
         if ha_policy is None:
-            ha_policy = conf.CELERY_QUEUE_HA_POLICY
+            ha_policy = conf.task_queue_ha_policy
         if max_priority is None:
         if max_priority is None:
-            max_priority = conf.CELERY_QUEUE_MAX_PRIORITY
-        if not queues and conf.CELERY_DEFAULT_QUEUE:
-            queues = (Queue(conf.CELERY_DEFAULT_QUEUE,
+            max_priority = conf.task_queue_max_priority
+        if not queues and conf.task_default_queue:
+            queues = (Queue(conf.task_default_queue,
                             exchange=self.default_exchange,
                             exchange=self.default_exchange,
-                            routing_key=conf.CELERY_DEFAULT_ROUTING_KEY),)
+                            routing_key=conf.task_default_routing_key),)
         autoexchange = (self.autoexchange if autoexchange is None
         autoexchange = (self.autoexchange if autoexchange is None
                         else autoexchange)
                         else autoexchange)
         return self.queues_cls(
         return self.queues_cls(
@@ -276,15 +276,15 @@ class AMQP(object):
     def Router(self, queues=None, create_missing=None):
     def Router(self, queues=None, create_missing=None):
         """Return the current task router."""
         """Return the current task router."""
         return _routes.Router(self.routes, queues or self.queues,
         return _routes.Router(self.routes, queues or self.queues,
-                              self.app.either('CELERY_CREATE_MISSING_QUEUES',
+                              self.app.either('task_create_missing_queues',
                                               create_missing), app=self.app)
                                               create_missing), app=self.app)
 
 
     def flush_routes(self):
     def flush_routes(self):
-        self._rtable = _routes.prepare(self.app.conf.CELERY_ROUTES)
+        self._rtable = _routes.prepare(self.app.conf.task_routes)
 
 
     def TaskConsumer(self, channel, queues=None, accept=None, **kw):
     def TaskConsumer(self, channel, queues=None, accept=None, **kw):
         if accept is None:
         if accept is None:
-            accept = self.app.conf.CELERY_ACCEPT_CONTENT
+            accept = self.app.conf.accept_content
         return self.Consumer(
         return self.Consumer(
             channel, accept=accept,
             channel, accept=accept,
             queues=queues or list(self.queues.consume_from.values()),
             queues=queues or list(self.queues.consume_from.values()),
@@ -442,9 +442,9 @@ class AMQP(object):
         )
         )
 
 
     def _create_task_sender(self):
     def _create_task_sender(self):
-        default_retry = self.app.conf.CELERY_TASK_PUBLISH_RETRY
-        default_policy = self.app.conf.CELERY_TASK_PUBLISH_RETRY_POLICY
-        default_delivery_mode = self.app.conf.CELERY_DEFAULT_DELIVERY_MODE
+        default_retry = self.app.conf.task_publish_retry
+        default_policy = self.app.conf.task_publish_retry_policy
+        default_delivery_mode = self.app.conf.task_default_delivery_mode
         default_queue = self.default_queue
         default_queue = self.default_queue
         queues = self.queues
         queues = self.queues
         send_before_publish = signals.before_task_publish.send
         send_before_publish = signals.before_task_publish.send
@@ -458,9 +458,9 @@ class AMQP(object):
         default_evd = self._event_dispatcher
         default_evd = self._event_dispatcher
         default_exchange = self.default_exchange
         default_exchange = self.default_exchange
 
 
-        default_rkey = self.app.conf.CELERY_DEFAULT_ROUTING_KEY
-        default_serializer = self.app.conf.CELERY_TASK_SERIALIZER
-        default_compressor = self.app.conf.CELERY_MESSAGE_COMPRESSION
+        default_rkey = self.app.conf.task_default_routing_key
+        default_serializer = self.app.conf.task_serializer
+        default_compressor = self.app.conf.result_compression
 
 
         def publish_task(producer, name, message,
         def publish_task(producer, name, message,
                          exchange=None, routing_key=None, queue=None,
                          exchange=None, routing_key=None, queue=None,
@@ -541,12 +541,12 @@ class AMQP(object):
 
 
     @cached_property
     @cached_property
     def default_queue(self):
     def default_queue(self):
-        return self.queues[self.app.conf.CELERY_DEFAULT_QUEUE]
+        return self.queues[self.app.conf.task_default_queue]
 
 
     @cached_property
     @cached_property
     def queues(self):
     def queues(self):
         """Queue name⇒ declaration mapping."""
         """Queue name⇒ declaration mapping."""
-        return self.Queues(self.app.conf.CELERY_QUEUES)
+        return self.Queues(self.app.conf.task_queues)
 
 
     @queues.setter  # noqa
     @queues.setter  # noqa
     def queues(self, queues):
     def queues(self, queues):
@@ -575,12 +575,12 @@ class AMQP(object):
 
 
     @cached_property
     @cached_property
     def default_exchange(self):
     def default_exchange(self):
-        return Exchange(self.app.conf.CELERY_DEFAULT_EXCHANGE,
-                        self.app.conf.CELERY_DEFAULT_EXCHANGE_TYPE)
+        return Exchange(self.app.conf.task_default_exchange,
+                        self.app.conf.task_default_exchange_type)
 
 
     @cached_property
     @cached_property
     def utc(self):
     def utc(self):
-        return self.app.conf.CELERY_ENABLE_UTC
+        return self.app.conf.enable_utc
 
 
     @cached_property
     @cached_property
     def _event_dispatcher(self):
     def _event_dispatcher(self):

+ 2 - 2
celery/app/annotations.py

@@ -7,7 +7,7 @@
     task classes in the configuration.
     task classes in the configuration.
 
 
     This prepares and performs the annotations in the
     This prepares and performs the annotations in the
-    :setting:`CELERY_ANNOTATIONS` setting.
+    :setting:`task_annotations` setting.
 
 
 """
 """
 from __future__ import absolute_import
 from __future__ import absolute_import
@@ -38,7 +38,7 @@ class MapAnnotation(dict):
 
 
 
 
 def prepare(annotations):
 def prepare(annotations):
-    """Expands the :setting:`CELERY_ANNOTATIONS` setting."""
+    """Expands the :setting:`task_annotations` setting."""
 
 
     def expand_annotation(annotation):
     def expand_annotation(annotation):
         if isinstance(annotation, dict):
         if isinstance(annotation, dict):

+ 111 - 73
celery/app/base.py

@@ -13,11 +13,10 @@ import threading
 import warnings
 import warnings
 
 
 from collections import defaultdict, deque
 from collections import defaultdict, deque
-from copy import deepcopy
 from operator import attrgetter
 from operator import attrgetter
 from functools import wraps
 from functools import wraps
 
 
-from amqp import promise
+from amqp import starpromise
 try:
 try:
     from billiard.util import register_after_fork
     from billiard.util import register_after_fork
 except ImportError:
 except ImportError:
@@ -33,8 +32,9 @@ from celery._state import (
     _register_app, get_current_worker_task, connect_on_app_finalize,
     _register_app, get_current_worker_task, connect_on_app_finalize,
     _announce_app_finalized,
     _announce_app_finalized,
 )
 )
+from celery.datastructures import AttributeDictMixin
 from celery.exceptions import AlwaysEagerIgnored, ImproperlyConfigured
 from celery.exceptions import AlwaysEagerIgnored, ImproperlyConfigured
-from celery.five import items, values
+from celery.five import UserDict, values
 from celery.loaders import get_loader_cls
 from celery.loaders import get_loader_cls
 from celery.local import PromiseProxy, maybe_evaluate
 from celery.local import PromiseProxy, maybe_evaluate
 from celery.utils import abstract
 from celery.utils import abstract
@@ -45,10 +45,11 @@ from celery.utils.imports import instantiate, symbol_by_name
 from celery.utils.objects import FallbackContext, mro_lookup
 from celery.utils.objects import FallbackContext, mro_lookup
 
 
 from .annotations import prepare as prepare_annotations
 from .annotations import prepare as prepare_annotations
-from .defaults import DEFAULTS, find_deprecated_settings
+from .defaults import find_deprecated_settings
 from .registry import TaskRegistry
 from .registry import TaskRegistry
 from .utils import (
 from .utils import (
-    AppPickler, Settings, bugreport, _unpickle_app, _unpickle_app_v2, appstr,
+    AppPickler, Settings,
+    bugreport, _unpickle_app, _unpickle_app_v2, appstr, detect_settings,
 )
 )
 
 
 # Load all builtin tasks
 # Load all builtin tasks
@@ -107,6 +108,18 @@ def _ensure_after_fork():
         register_after_fork(_global_after_fork, _global_after_fork)
         register_after_fork(_global_after_fork, _global_after_fork)
 
 
 
 
+class PendingConfiguration(UserDict, AttributeDictMixin):
+    callback = None
+    data = None
+
+    def __init__(self, conf, callback):
+        object.__setattr__(self, 'data', conf)
+        object.__setattr__(self, 'callback', callback)
+
+    def __getitem__(self, key):
+        return self.callback(key)
+
+
 class Celery(object):
 class Celery(object):
     """Celery application.
     """Celery application.
 
 
@@ -117,7 +130,7 @@ class Celery(object):
                      Default is :class:`celery.loaders.app.AppLoader`.
                      Default is :class:`celery.loaders.app.AppLoader`.
     :keyword backend: The result store backend class, or the name of the
     :keyword backend: The result store backend class, or the name of the
                       backend class to use. Default is the value of the
                       backend class to use. Default is the value of the
-                      :setting:`CELERY_RESULT_BACKEND` setting.
+                      :setting:`result_backend` setting.
     :keyword amqp: AMQP object or class name.
     :keyword amqp: AMQP object or class name.
     :keyword events: Events object or class name.
     :keyword events: Events object or class name.
     :keyword log: Log object or class name.
     :keyword log: Log object or class name.
@@ -181,7 +194,7 @@ class Celery(object):
                  amqp=None, events=None, log=None, control=None,
                  amqp=None, events=None, log=None, control=None,
                  set_as_current=True, tasks=None, broker=None, include=None,
                  set_as_current=True, tasks=None, broker=None, include=None,
                  changes=None, config_source=None, fixups=None, task_cls=None,
                  changes=None, config_source=None, fixups=None, task_cls=None,
-                 autofinalize=True, **kwargs):
+                 autofinalize=True, namespace=None, **kwargs):
         self.clock = LamportClock()
         self.clock = LamportClock()
         self.main = main
         self.main = main
         self.amqp_cls = amqp or self.amqp_cls
         self.amqp_cls = amqp or self.amqp_cls
@@ -195,6 +208,7 @@ class Celery(object):
         self.user_options = defaultdict(set)
         self.user_options = defaultdict(set)
         self.steps = defaultdict(set)
         self.steps = defaultdict(set)
         self.autofinalize = autofinalize
         self.autofinalize = autofinalize
+        self.namespace = namespace
 
 
         self.configured = False
         self.configured = False
         self._config_source = config_source
         self._config_source = config_source
@@ -216,12 +230,15 @@ class Celery(object):
         # these options are moved to the config to
         # these options are moved to the config to
         # simplify pickling of the app object.
         # simplify pickling of the app object.
         self._preconf = changes or {}
         self._preconf = changes or {}
-        if broker:
-            self._preconf['BROKER_URL'] = broker
-        if backend:
-            self._preconf['CELERY_RESULT_BACKEND'] = backend
-        if include:
-            self._preconf['CELERY_IMPORTS'] = include
+        self._preconf_set_by_auto = set()
+        self.__autoset('broker_url', broker)
+        self.__autoset('result_backend', backend)
+        self.__autoset('include', include)
+        self._conf = Settings(
+            PendingConfiguration(
+                self._preconf, self._get_from_conf_and_finalize),
+            prefix=self.namespace,
+        )
 
 
         # - Apply fixups.
         # - Apply fixups.
         self.fixups = set(self.builtin_fixups) if fixups is None else fixups
         self.fixups = set(self.builtin_fixups) if fixups is None else fixups
@@ -241,6 +258,11 @@ class Celery(object):
         self.on_init()
         self.on_init()
         _register_app(self)
         _register_app(self)
 
 
+    def __autoset(self, key, value):
+        if value:
+            self._preconf[key] = value
+            self._preconf_set_by_auto.add(key)
+
     def set_current(self):
     def set_current(self):
         """Makes this the current app for this thread."""
         """Makes this the current app for this thread."""
         _set_current_app(self)
         _set_current_app(self)
@@ -445,7 +467,8 @@ class Celery(object):
             return self._conf.add_defaults(fun())
             return self._conf.add_defaults(fun())
         self._pending_defaults.append(fun)
         self._pending_defaults.append(fun)
 
 
-    def config_from_object(self, obj, silent=False, force=False):
+    def config_from_object(self, obj,
+                           silent=False, force=False, namespace=None):
         """Reads configuration from object, where object is either
         """Reads configuration from object, where object is either
         an object or the name of a module to import.
         an object or the name of a module to import.
 
 
@@ -463,9 +486,11 @@ class Celery(object):
 
 
         """
         """
         self._config_source = obj
         self._config_source = obj
+        self.namespace = namespace or self.namespace
         if force or self.configured:
         if force or self.configured:
             self._conf = None
             self._conf = None
-            return self.loader.config_from_object(obj, silent=silent)
+            if self.loader.config_from_object(obj, silent=silent):
+                return self.conf
 
 
     def config_from_envvar(self, variable_name, silent=False, force=False):
     def config_from_envvar(self, variable_name, silent=False, force=False):
         """Read configuration from environment variable.
         """Read configuration from environment variable.
@@ -488,7 +513,7 @@ class Celery(object):
         return self.config_from_object(module_name, silent=silent, force=force)
         return self.config_from_object(module_name, silent=silent, force=force)
 
 
     def config_from_cmdline(self, argv, namespace='celery'):
     def config_from_cmdline(self, argv, namespace='celery'):
-        (self._conf if self.configured else self.conf).update(
+        self._conf.update(
             self.loader.cmdline_config_parser(argv, namespace)
             self.loader.cmdline_config_parser(argv, namespace)
         )
         )
 
 
@@ -505,15 +530,15 @@ class Celery(object):
         :keyword allowed_serializers: List of serializer names, or
         :keyword allowed_serializers: List of serializer names, or
             content_types that should be exempt from being disabled.
             content_types that should be exempt from being disabled.
         :keyword key: Name of private key file to use.
         :keyword key: Name of private key file to use.
-            Defaults to the :setting:`CELERY_SECURITY_KEY` setting.
+            Defaults to the :setting:`security_key` setting.
         :keyword cert: Name of certificate file to use.
         :keyword cert: Name of certificate file to use.
-            Defaults to the :setting:`CELERY_SECURITY_CERTIFICATE` setting.
+            Defaults to the :setting:`security_certificate` setting.
         :keyword store: Directory containing certificates.
         :keyword store: Directory containing certificates.
-            Defaults to the :setting:`CELERY_SECURITY_CERT_STORE` setting.
+            Defaults to the :setting:`security_cert_store` setting.
         :keyword digest: Digest algorithm used when signing messages.
         :keyword digest: Digest algorithm used when signing messages.
             Default is ``sha1``.
             Default is ``sha1``.
         :keyword serializer: Serializer used to encode messages after
         :keyword serializer: Serializer used to encode messages after
-            they have been signed.  See :setting:`CELERY_TASK_SERIALIZER` for
+            they have been signed.  See :setting:`task_serializer` for
             the serializers supported.
             the serializers supported.
             Default is ``json``.
             Default is ``json``.
 
 
@@ -559,8 +584,8 @@ class Celery(object):
         """
         """
         if force:
         if force:
             return self._autodiscover_tasks(packages, related_name)
             return self._autodiscover_tasks(packages, related_name)
-        signals.import_modules.connect(promise(
-            self._autodiscover_tasks, (packages, related_name),
+        signals.import_modules.connect(starpromise(
+            self._autodiscover_tasks, packages, related_name,
         ), weak=False, sender=self)
         ), weak=False, sender=self)
 
 
     def _autodiscover_tasks(self, packages, related_name, **kwargs):
     def _autodiscover_tasks(self, packages, related_name, **kwargs):
@@ -603,9 +628,9 @@ class Celery(object):
         producer = producer or publisher  # XXX compat
         producer = producer or publisher  # XXX compat
         router = router or amqp.router
         router = router or amqp.router
         conf = self.conf
         conf = self.conf
-        if conf.CELERY_ALWAYS_EAGER:  # pragma: no cover
+        if conf.task_always_eager:  # pragma: no cover
             warnings.warn(AlwaysEagerIgnored(
             warnings.warn(AlwaysEagerIgnored(
-                'CELERY_ALWAYS_EAGER has no effect on send_task',
+                'task_always_eager has no effect on send_task',
             ), stacklevel=2)
             ), stacklevel=2)
         options = router.route(options, route_name or name, args, kwargs)
         options = router.route(options, route_name or name, args, kwargs)
 
 
@@ -614,7 +639,7 @@ class Celery(object):
             expires, retries, chord,
             expires, retries, chord,
             maybe_list(link), maybe_list(link_error),
             maybe_list(link), maybe_list(link_error),
             reply_to or self.oid, time_limit, soft_time_limit,
             reply_to or self.oid, time_limit, soft_time_limit,
-            self.conf.CELERY_SEND_TASK_SENT_EVENT,
+            self.conf.task_send_sent_event,
             root_id, parent_id, shadow,
             root_id, parent_id, shadow,
         )
         )
 
 
@@ -646,8 +671,8 @@ class Celery(object):
         :keyword password: Password to authenticate with
         :keyword password: Password to authenticate with
         :keyword virtual_host: Virtual host to use (domain).
         :keyword virtual_host: Virtual host to use (domain).
         :keyword port: Port to connect to.
         :keyword port: Port to connect to.
-        :keyword ssl: Defaults to the :setting:`BROKER_USE_SSL` setting.
-        :keyword transport: defaults to the :setting:`BROKER_TRANSPORT`
+        :keyword ssl: Defaults to the :setting:`broker_use_ssl` setting.
+        :keyword transport: defaults to the :setting:`broker_transport`
                  setting.
                  setting.
 
 
         :returns :class:`kombu.Connection`:
         :returns :class:`kombu.Connection`:
@@ -655,23 +680,23 @@ class Celery(object):
         """
         """
         conf = self.conf
         conf = self.conf
         return self.amqp.Connection(
         return self.amqp.Connection(
-            hostname or conf.BROKER_URL,
-            userid or conf.BROKER_USER,
-            password or conf.BROKER_PASSWORD,
-            virtual_host or conf.BROKER_VHOST,
-            port or conf.BROKER_PORT,
-            transport=transport or conf.BROKER_TRANSPORT,
-            ssl=self.either('BROKER_USE_SSL', ssl),
+            hostname or conf.broker_url,
+            userid or conf.broker_user,
+            password or conf.broker_password,
+            virtual_host or conf.broker_vhost,
+            port or conf.broker_port,
+            transport=transport or conf.broker_transport,
+            ssl=self.either('broker_use_ssl', ssl),
             heartbeat=heartbeat,
             heartbeat=heartbeat,
-            login_method=login_method or conf.BROKER_LOGIN_METHOD,
+            login_method=login_method or conf.broker_login_method,
             failover_strategy=(
             failover_strategy=(
-                failover_strategy or conf.BROKER_FAILOVER_STRATEGY
+                failover_strategy or conf.broker_failover_strategy
             ),
             ),
             transport_options=dict(
             transport_options=dict(
-                conf.BROKER_TRANSPORT_OPTIONS, **transport_options or {}
+                conf.broker_transport_options, **transport_options or {}
             ),
             ),
             connect_timeout=self.either(
             connect_timeout=self.either(
-                'BROKER_CONNECTION_TIMEOUT', connect_timeout
+                'broker_connection_timeout', connect_timeout
             ),
             ),
         )
         )
     broker_connection = connection
     broker_connection = connection
@@ -712,24 +737,24 @@ class Celery(object):
     def now(self):
     def now(self):
         """Return the current time and date as a
         """Return the current time and date as a
         :class:`~datetime.datetime` object."""
         :class:`~datetime.datetime` object."""
-        return self.loader.now(utc=self.conf.CELERY_ENABLE_UTC)
+        return self.loader.now(utc=self.conf.enable_utc)
 
 
     def mail_admins(self, subject, body, fail_silently=False):
     def mail_admins(self, subject, body, fail_silently=False):
-        """Sends an email to the admins in the :setting:`ADMINS` setting."""
+        """Sends an email to the admins in the :setting:`admins` setting."""
         conf = self.conf
         conf = self.conf
-        if conf.ADMINS:
-            to = [admin_email for _, admin_email in conf.ADMINS]
+        if conf.admins:
+            to = [admin_email for _, admin_email in conf.admins]
             return self.loader.mail_admins(
             return self.loader.mail_admins(
                 subject, body, fail_silently, to=to,
                 subject, body, fail_silently, to=to,
-                sender=conf.SERVER_EMAIL,
-                host=conf.EMAIL_HOST,
-                port=conf.EMAIL_PORT,
-                user=conf.EMAIL_HOST_USER,
-                password=conf.EMAIL_HOST_PASSWORD,
-                timeout=conf.EMAIL_TIMEOUT,
-                use_ssl=conf.EMAIL_USE_SSL,
-                use_tls=conf.EMAIL_USE_TLS,
-                charset=conf.EMAIL_CHARSET,
+                sender=conf.server_email,
+                host=conf.email_host,
+                port=conf.email_port,
+                user=conf.email_host_user,
+                password=conf.email_host_password,
+                timeout=conf.email_timeout,
+                use_ssl=conf.email_use_ssl,
+                use_tls=conf.email_use_tls,
+                charset=conf.email_charset,
             )
             )
 
 
     def select_queues(self, queues=None):
     def select_queues(self, queues=None):
@@ -741,7 +766,9 @@ class Celery(object):
     def either(self, default_key, *values):
     def either(self, default_key, *values):
         """Fallback to the value of a configuration key if none of the
         """Fallback to the value of a configuration key if none of the
         `*values` are true."""
         `*values` are true."""
-        return first(None, values) or self.conf.get(default_key)
+        return first(None, [
+            first(None, values), starpromise(self.conf.get, default_key),
+        ])
 
 
     def bugreport(self):
     def bugreport(self):
         """Return a string with information useful for the Celery core
         """Return a string with information useful for the Celery core
@@ -751,7 +778,7 @@ class Celery(object):
     def _get_backend(self):
     def _get_backend(self):
         from celery.backends import get_backend_by_url
         from celery.backends import get_backend_by_url
         backend, url = get_backend_by_url(
         backend, url = get_backend_by_url(
-            self.backend_cls or self.conf.CELERY_RESULT_BACKEND,
+            self.backend_cls or self.conf.result_backend,
             self.loader)
             self.loader)
         return backend(app=self, url=url)
         return backend(app=self, url=url)
 
 
@@ -763,27 +790,32 @@ class Celery(object):
             self.on_configure()
             self.on_configure()
         if self._config_source:
         if self._config_source:
             self.loader.config_from_object(self._config_source)
             self.loader.config_from_object(self._config_source)
-        defaults = dict(deepcopy(DEFAULTS), **self._preconf)
+
         self.configured = True
         self.configured = True
-        s = self._conf = Settings(
-            {}, [self.prepare_config(self.loader.conf), defaults],
+        settings = detect_settings(
+            self.prepare_config(self.loader.conf), self._preconf,
+            ignore_keys=self._preconf_set_by_auto, prefix=self.namespace,
         )
         )
+        if self._conf is not None:
+            # replace in place, as someone may have referenced app.conf,
+            # done some changes, accessed a key, and then try to make more
+            # changes to the reference and not the finalized value.
+            self._conf.swap_with(settings)
+        else:
+            self._conf = settings
+
         # load lazy config dict initializers.
         # load lazy config dict initializers.
         pending_def = self._pending_defaults
         pending_def = self._pending_defaults
         while pending_def:
         while pending_def:
-            s.add_defaults(maybe_evaluate(pending_def.popleft()()))
+            self._conf.add_defaults(maybe_evaluate(pending_def.popleft()()))
 
 
         # load lazy periodic tasks
         # load lazy periodic tasks
         pending_beat = self._pending_periodic_tasks
         pending_beat = self._pending_periodic_tasks
         while pending_beat:
         while pending_beat:
             self._add_periodic_task(*pending_beat.popleft())
             self._add_periodic_task(*pending_beat.popleft())
 
 
-        # Settings.__setitem__ method, set Settings.change
-        if self._preconf:
-            for key, value in items(self._preconf):
-                setattr(s, key, value)
-        self.on_after_configure.send(sender=self, source=s)
-        return s
+        self.on_after_configure.send(sender=self, source=self._conf)
+        return self._conf
 
 
     def _after_fork(self, obj_):
     def _after_fork(self, obj_):
         self._maybe_close_pool()
         self._maybe_close_pool()
@@ -830,7 +862,7 @@ class Celery(object):
         }
         }
 
 
     def _add_periodic_task(self, key, entry):
     def _add_periodic_task(self, key, entry):
-        self._conf.CELERYBEAT_SCHEDULE[key] = entry
+        self._conf.beat_schedule[key] = entry
 
 
     def create_task_cls(self):
     def create_task_cls(self):
         """Creates a base task class using default configuration
         """Creates a base task class using default configuration
@@ -893,7 +925,8 @@ class Celery(object):
         when unpickling."""
         when unpickling."""
         return {
         return {
             'main': self.main,
             'main': self.main,
-            'changes': self._conf.changes if self._conf else self._preconf,
+            'changes':
+                self._conf.changes if self.configured else self._preconf,
             'loader': self.loader_cls,
             'loader': self.loader_cls,
             'backend': self.backend_cls,
             'backend': self.backend_cls,
             'amqp': self.amqp_cls,
             'amqp': self.amqp_cls,
@@ -903,11 +936,12 @@ class Celery(object):
             'fixups': self.fixups,
             'fixups': self.fixups,
             'config_source': self._config_source,
             'config_source': self._config_source,
             'task_cls': self.task_cls,
             'task_cls': self.task_cls,
+            'namespace': self.namespace,
         }
         }
 
 
     def __reduce_args__(self):
     def __reduce_args__(self):
         """Deprecated method, please use :meth:`__reduce_keys__` instead."""
         """Deprecated method, please use :meth:`__reduce_keys__` instead."""
-        return (self.main, self._conf.changes if self._conf else {},
+        return (self.main, self._conf.changes if self.configured else {},
                 self.loader_cls, self.backend_cls, self.amqp_cls,
                 self.loader_cls, self.backend_cls, self.amqp_cls,
                 self.events_cls, self.log_cls, self.control_cls,
                 self.events_cls, self.log_cls, self.control_cls,
                 False, self._config_source)
                 False, self._config_source)
@@ -938,7 +972,7 @@ class Celery(object):
 
 
     @cached_property
     @cached_property
     def annotations(self):
     def annotations(self):
-        return prepare_annotations(self.conf.CELERY_ANNOTATIONS)
+        return prepare_annotations(self.conf.task_annotations)
 
 
     @cached_property
     @cached_property
     def AsyncResult(self):
     def AsyncResult(self):
@@ -981,7 +1015,7 @@ class Celery(object):
         """
         """
         if self._pool is None:
         if self._pool is None:
             _ensure_after_fork()
             _ensure_after_fork()
-            limit = self.conf.BROKER_POOL_LIMIT
+            limit = self.conf.broker_pool_limit
             self._pool = self.connection().Pool(limit=limit)
             self._pool = self.connection().Pool(limit=limit)
         return self._pool
         return self._pool
 
 
@@ -1009,9 +1043,13 @@ class Celery(object):
     def conf(self):
     def conf(self):
         """Current configuration."""
         """Current configuration."""
         if self._conf is None:
         if self._conf is None:
-            self._load_config()
+            self._conf = self._load_config()
         return self._conf
         return self._conf
 
 
+    def _get_from_conf_and_finalize(self, key):
+        conf = self._conf = self._load_config()
+        return conf[key]
+
     @conf.setter
     @conf.setter
     def conf(self, d):  # noqa
     def conf(self, d):  # noqa
         self._conf = d
         self._conf = d
@@ -1056,14 +1094,14 @@ class Celery(object):
         """Current timezone for this app.
         """Current timezone for this app.
 
 
         This is a cached property taking the time zone from the
         This is a cached property taking the time zone from the
-        :setting:`CELERY_TIMEZONE` setting.
+        :setting:`timezone` setting.
 
 
         """
         """
         from celery.utils.timeutils import timezone
         from celery.utils.timeutils import timezone
         conf = self.conf
         conf = self.conf
-        tz = conf.CELERY_TIMEZONE
+        tz = conf.timezone
         if not tz:
         if not tz:
-            return (timezone.get_timezone('UTC') if conf.CELERY_ENABLE_UTC
+            return (timezone.get_timezone('UTC') if conf.enable_utc
                     else timezone.local)
                     else timezone.local)
-        return timezone.get_timezone(conf.CELERY_TIMEZONE)
+        return timezone.get_timezone(conf.timezone)
 App = Celery  # compat
 App = Celery  # compat

+ 1 - 1
celery/app/builtins.py

@@ -54,7 +54,7 @@ def add_unlock_chord_task(app):
     from celery.exceptions import ChordError
     from celery.exceptions import ChordError
     from celery.result import allow_join_result, result_from_tuple
     from celery.result import allow_join_result, result_from_tuple
 
 
-    default_propagate = app.conf.CELERY_CHORD_PROPAGATES
+    default_propagate = app.conf.chord_propagates
 
 
     @app.task(name='celery.chord_unlock', max_retries=None, shared=False,
     @app.task(name='celery.chord_unlock', max_retries=None, shared=False,
               default_retry_delay=1, ignore_result=True, lazy=False, bind=True)
               default_retry_delay=1, ignore_result=True, lazy=False, bind=True)

+ 275 - 165
celery/app/defaults.py

@@ -13,7 +13,7 @@ import sys
 from collections import deque, namedtuple
 from collections import deque, namedtuple
 from datetime import timedelta
 from datetime import timedelta
 
 
-from celery.five import items
+from celery.five import items, keys, values
 from celery.utils import strtobool
 from celery.utils import strtobool
 from celery.utils.functional import memoize
 from celery.utils.functional import memoize
 
 
@@ -39,13 +39,29 @@ DEFAULT_LOG_FMT = '[%(asctime)s: %(levelname)s] %(message)s'
 DEFAULT_TASK_LOG_FMT = """[%(asctime)s: %(levelname)s/%(processName)s] \
 DEFAULT_TASK_LOG_FMT = """[%(asctime)s: %(levelname)s/%(processName)s] \
 %(task_name)s[%(task_id)s]: %(message)s"""
 %(task_name)s[%(task_id)s]: %(message)s"""
 
 
+OLD_NS = {'celery_{0}'}
+OLD_NS_BEAT = {'celerybeat_{0}'}
+OLD_NS_WORKER = {'celeryd_{0}'}
+
 searchresult = namedtuple('searchresult', ('namespace', 'key', 'type'))
 searchresult = namedtuple('searchresult', ('namespace', 'key', 'type'))
 
 
 
 
+def Namespace(__old__=None, **options):
+    if __old__ is not None:
+        for opt in values(options):
+            opt.old = opt.old | __old__
+    return options
+
+
+def old_ns(ns):
+    return {'{0}_{{0}}'.format(ns)}
+
+
 class Option(object):
 class Option(object):
     alt = None
     alt = None
     deprecate_by = None
     deprecate_by = None
     remove_by = None
     remove_by = None
+    old = set()
     typemap = dict(string=str, int=int, float=float, any=lambda v: v,
     typemap = dict(string=str, int=int, float=float, any=lambda v: v,
                    bool=strtobool, dict=dict, tuple=tuple)
                    bool=strtobool, dict=dict, tuple=tuple)
 
 
@@ -62,166 +78,260 @@ class Option(object):
         return '<Option: type->{0} default->{1!r}>'.format(self.type,
         return '<Option: type->{0} default->{1!r}>'.format(self.type,
                                                            self.default)
                                                            self.default)
 
 
-NAMESPACES = {
-    'BROKER': {
-        'URL': Option(None, type='string'),
-        'CONNECTION_TIMEOUT': Option(4, type='float'),
-        'CONNECTION_RETRY': Option(True, type='bool'),
-        'CONNECTION_MAX_RETRIES': Option(100, type='int'),
-        'FAILOVER_STRATEGY': Option(None, type='string'),
-        'HEARTBEAT': Option(None, type='int'),
-        'HEARTBEAT_CHECKRATE': Option(3.0, type='int'),
-        'LOGIN_METHOD': Option(None, type='string'),
-        'POOL_LIMIT': Option(10, type='int'),
-        'USE_SSL': Option(False, type='bool'),
-        'TRANSPORT': Option(type='string'),
-        'TRANSPORT_OPTIONS': Option({}, type='dict'),
-        'HOST': Option(type='string'),
-        'PORT': Option(type='int'),
-        'USER': Option(type='string'),
-        'PASSWORD': Option(type='string'),
-        'VHOST': Option(type='string'),
-    },
-    'CASSANDRA': {
-        'COLUMN_FAMILY': Option(type='string'),
-        'DETAILED_MODE': Option(False, type='bool'),
-        'KEYSPACE': Option(type='string'),
-        'READ_CONSISTENCY': Option(type='string'),
-        'SERVERS': Option(type='list'),
-        'PORT': Option(type="string"),
-        'ENTRY_TTL': Option(type="float"),
-        'WRITE_CONSISTENCY': Option(type='string'),
-    },
-    'CELERY': {
-        'ACCEPT_CONTENT': Option(DEFAULT_ACCEPT_CONTENT, type='list'),
-        'ACKS_LATE': Option(False, type='bool'),
-        'ALWAYS_EAGER': Option(False, type='bool'),
-        'ANNOTATIONS': Option(type='any'),
-        'BROADCAST_QUEUE': Option('celeryctl'),
-        'BROADCAST_EXCHANGE': Option('celeryctl'),
-        'BROADCAST_EXCHANGE_TYPE': Option('fanout'),
-        'CACHE_BACKEND': Option(),
-        'CACHE_BACKEND_OPTIONS': Option({}, type='dict'),
-        'CHORD_PROPAGATES': Option(True, type='bool'),
-        'COUCHBASE_BACKEND_SETTINGS': Option(None, type='dict'),
-        'CREATE_MISSING_QUEUES': Option(True, type='bool'),
-        'DEFAULT_RATE_LIMIT': Option(type='string'),
-        'DISABLE_RATE_LIMITS': Option(False, type='bool'),
-        'DEFAULT_ROUTING_KEY': Option('celery'),
-        'DEFAULT_QUEUE': Option('celery'),
-        'DEFAULT_EXCHANGE': Option('celery'),
-        'DEFAULT_EXCHANGE_TYPE': Option('direct'),
-        'DEFAULT_DELIVERY_MODE': Option(2, type='string'),
-        'EAGER_PROPAGATES_EXCEPTIONS': Option(False, type='bool'),
-        'ENABLE_UTC': Option(True, type='bool'),
-        'ENABLE_REMOTE_CONTROL': Option(True, type='bool'),
-        'EVENT_SERIALIZER': Option('json'),
-        'EVENT_QUEUE_EXPIRES': Option(60.0, type='float'),
-        'EVENT_QUEUE_TTL': Option(5.0, type='float'),
-        'IMPORTS': Option((), type='tuple'),
-        'INCLUDE': Option((), type='tuple'),
-        'IGNORE_RESULT': Option(False, type='bool'),
-        'MAX_CACHED_RESULTS': Option(100, type='int'),
-        'MESSAGE_COMPRESSION': Option(type='string'),
-        'MONGODB_BACKEND_SETTINGS': Option(type='dict'),
-        'REDIS_HOST': Option(type='string'),
-        'REDIS_PORT': Option(type='int'),
-        'REDIS_DB': Option(type='int'),
-        'REDIS_PASSWORD': Option(type='string'),
-        'REDIS_MAX_CONNECTIONS': Option(type='int'),
-        'REJECT_ON_WORKER_LOST': Option(type='bool'),
-        'RESULT_BACKEND': Option(type='string'),
-        'RESULT_DB_SHORT_LIVED_SESSIONS': Option(False, type='bool'),
-        'RESULT_DB_TABLENAMES': Option(type='dict'),
-        'RESULT_DBURI': Option(),
-        'RESULT_ENGINE_OPTIONS': Option(type='dict'),
-        'RESULT_EXCHANGE': Option('celeryresults'),
-        'RESULT_EXCHANGE_TYPE': Option('direct'),
-        'RESULT_SERIALIZER': Option('json'),
-        'RESULT_PERSISTENT': Option(None, type='bool'),
-        'RIAK_BACKEND_SETTINGS': Option(type='dict'),
-        'ROUTES': Option(type='any'),
-        'SEND_EVENTS': Option(False, type='bool'),
-        'SEND_TASK_ERROR_EMAILS': Option(False, type='bool'),
-        'SEND_TASK_SENT_EVENT': Option(False, type='bool'),
-        'STORE_ERRORS_EVEN_IF_IGNORED': Option(False, type='bool'),
-        'TASK_PROTOCOL': Option(1, type='int'),
-        'TASK_PUBLISH_RETRY': Option(True, type='bool'),
-        'TASK_PUBLISH_RETRY_POLICY': Option({
-            'max_retries': 3,
-            'interval_start': 0,
-            'interval_max': 1,
-            'interval_step': 0.2}, type='dict'),
-        'TASK_RESULT_EXPIRES': Option(timedelta(days=1), type='float'),
-        'TASK_SERIALIZER': Option('json'),
-        'TIMEZONE': Option(type='string'),
-        'TRACK_STARTED': Option(False, type='bool'),
-        'REDIRECT_STDOUTS': Option(True, type='bool'),
-        'REDIRECT_STDOUTS_LEVEL': Option('WARNING'),
-        'QUEUES': Option(type='dict'),
-        'QUEUE_HA_POLICY': Option(None, type='string'),
-        'QUEUE_MAX_PRIORITY': Option(None, type='int'),
-        'SECURITY_KEY': Option(type='string'),
-        'SECURITY_CERTIFICATE': Option(type='string'),
-        'SECURITY_CERT_STORE': Option(type='string'),
-        'WORKER_DIRECT': Option(False, type='bool'),
-    },
-    'CELERYD': {
-        'AGENT': Option(None, type='string'),
-        'AUTOSCALER': Option('celery.worker.autoscale:Autoscaler'),
-        'AUTORELOADER': Option('celery.worker.autoreload:Autoreloader'),
-        'CONCURRENCY': Option(0, type='int'),
-        'TIMER': Option(type='string'),
-        'TIMER_PRECISION': Option(1.0, type='float'),
-        'FORCE_EXECV': Option(False, type='bool'),
-        'HIJACK_ROOT_LOGGER': Option(True, type='bool'),
-        'CONSUMER': Option('celery.worker.consumer:Consumer', type='string'),
-        'LOG_FORMAT': Option(DEFAULT_PROCESS_LOG_FMT),
-        'LOG_COLOR': Option(type='bool'),
-        'MAX_TASKS_PER_CHILD': Option(type='int'),
-        'MAX_MEMORY_PER_CHILD': Option(type='int'),
-        'POOL': Option(DEFAULT_POOL),
-        'POOL_PUTLOCKS': Option(True, type='bool'),
-        'POOL_RESTARTS': Option(False, type='bool'),
-        'PREFETCH_MULTIPLIER': Option(4, type='int'),
-        'STATE_DB': Option(),
-        'TASK_LOG_FORMAT': Option(DEFAULT_TASK_LOG_FMT),
-        'TASK_SOFT_TIME_LIMIT': Option(type='float'),
-        'TASK_TIME_LIMIT': Option(type='float'),
-        'WORKER_LOST_WAIT': Option(10.0, type='float')
-    },
-    'CELERYBEAT': {
-        'SCHEDULE': Option({}, type='dict'),
-        'SCHEDULER': Option('celery.beat:PersistentScheduler'),
-        'SCHEDULE_FILENAME': Option('celerybeat-schedule'),
-        'SYNC_EVERY': Option(0, type='int'),
-        'MAX_LOOP_INTERVAL': Option(0, type='float'),
-    },
-    'EMAIL': {
-        'HOST': Option('localhost'),
-        'PORT': Option(25, type='int'),
-        'HOST_USER': Option(),
-        'HOST_PASSWORD': Option(),
-        'TIMEOUT': Option(2, type='float'),
-        'USE_SSL': Option(False, type='bool'),
-        'USE_TLS': Option(False, type='bool'),
-        'CHARSET': Option('us-ascii'),
-    },
-    'SERVER_EMAIL': Option('celery@localhost'),
-    'ADMINS': Option((), type='tuple'),
-}
+NAMESPACES = Namespace(
+    accept_content=Option(DEFAULT_ACCEPT_CONTENT, type='list', old=OLD_NS),
+    admins=Option((), type='tuple'),
+    enable_utc=Option(True, type='bool'),
+    imports=Option((), type='tuple', old=OLD_NS),
+    include=Option((), type='tuple', old=OLD_NS),
+    server_email=Option('celery@localhost'),
+    timezone=Option(type='string', old=OLD_NS),
+    beat=Namespace(
+        __old__=OLD_NS_BEAT,
+
+        max_loop_interval=Option(0, type='float'),
+        schedule=Option({}, type='dict'),
+        scheduler=Option('celery.beat:PersistentScheduler'),
+        schedule_filename=Option('celerybeat-schedule'),
+        sync_every=Option(0, type='int'),
+    ),
+    broker=Namespace(
+        url=Option(None, type='string'),
+        transport=Option(type='string'),
+        transport_options=Option({}, type='dict'),
+        connection_timeout=Option(4, type='float'),
+        connection_retry=Option(True, type='bool'),
+        connection_max_retries=Option(100, type='int'),
+        failover_strategy=Option(None, type='string'),
+        heartbeat=Option(None, type='int'),
+        heartbeat_checkrate=Option(3.0, type='int'),
+        login_method=Option(None, type='string'),
+        pool_limit=Option(10, type='int'),
+        use_ssl=Option(False, type='bool'),
+
+        host=Option(type='string'),
+        port=Option(type='int'),
+        user=Option(type='string'),
+        password=Option(type='string'),
+        vhost=Option(type='string'),
+    ),
+    cache=Namespace(
+        __old__=old_ns('celery_cache'),
+
+        backend=Option(),
+        backend_options=Option({}, type='dict'),
+    ),
+    cassandra=Namespace(
+        column_family=Option(type='string'),
+        entry_ttl=Option(type="float"),
+        keyspace=Option(type='string'),
+        port=Option(type="string"),
+        read_consistency=Option(type='string'),
+        servers=Option(type='list'),
+        write_consistency=Option(type='string'),
+    ),
+    chord=Namespace(
+        __old__=old_ns('celery_chord'),
+
+        propagates=Option(True, type='bool'),
+    ),
+    couchbase=Namespace(
+        __old__=old_ns('celery_couchbase'),
+
+        backend_settings=Option(None, type='dict'),
+    ),
+    email=Namespace(
+        charset=Option('us-ascii'),
+        host=Option('localhost'),
+        host_user=Option(),
+        host_password=Option(),
+        port=Option(25, type='int'),
+        timeout=Option(2, type='float'),
+        use_ssl=Option(False, type='bool'),
+        use_tls=Option(False, type='bool'),
+    ),
+    mongodb=Namespace(
+        __old__=old_ns('celery_mongodb'),
+
+        backend_settings=Option(type='dict'),
+    ),
+    event=Namespace(
+        __old__=old_ns('celery_event'),
 
 
+        queue_expires=Option(60.0, type='float'),
+        queue_ttl=Option(5.0, type='float'),
+        serializer=Option('json'),
+    ),
+    redis=Namespace(
+        __old__=old_ns('celery_redis'),
 
 
-def flatten(d, ns=''):
-    stack = deque([(ns, d)])
+        db=Option(type='int'),
+        host=Option(type='string'),
+        max_connections=Option(type='int'),
+        password=Option(type='string'),
+        port=Option(type='int'),
+    ),
+    result=Namespace(
+        __old__=old_ns('celery_result'),
+
+        backend=Option(type='string'),
+        cache_max=Option(
+            100,
+            type='int', old={'celery_max_cached_results'},
+        ),
+        compression=Option(type='str'),
+        exchange=Option('celeryresults'),
+        exchange_type=Option('direct'),
+        expires=Option(
+            timedelta(days=1),
+            type='float', old={'celery_task_result_expires'},
+        ),
+        persistent=Option(None, type='bool'),
+        serializer=Option('json'),
+    ),
+    riak=Namespace(
+        __old__=old_ns('celery_riak'),
+
+        backend_settings=Option(type='dict'),
+    ),
+    security=Namespace(
+        __old__=old_ns('celery_security'),
+
+        certificate=Option(type='string'),
+        cert_store=Option(type='string'),
+        key=Option(type='string'),
+    ),
+    sqlalchemy=Namespace(
+        dburi=Option(old={'celery_result_dburi'}),
+        engine_options=Option(
+            type='dict', old={'celery_result_engine_options'},
+        ),
+        short_lived_sessions=Option(
+            False, type='bool', old={'celery_result_db_short_lived_sessions'},
+        ),
+        table_names=Option(type='dict', old={'celery_result_db_tablenames'}),
+    ),
+    task=Namespace(
+        __old__=OLD_NS,
+        acks_late=Option(False, type='bool'),
+        always_eager=Option(False, type='bool'),
+        annotations=Option(type='any'),
+        compression=Option(type='string', old={'celery_message_compression'}),
+        create_missing_queues=Option(True, type='bool'),
+        default_delivery_mode=Option(2, type='string'),
+        default_exchange=Option('celery'),
+        default_exchange_type=Option('direct'),
+        default_queue=Option('celery'),
+        default_rate_limit=Option(type='string'),
+        default_routing_key=Option('celery'),
+        eager_propagates_exceptions=Option(False, type='bool'),
+        ignore_result=Option(False, type='bool'),
+        protocol=Option(1, type='int', old={'celery_task_protocol'}),
+        publish_retry=Option(
+            True, type='bool', old={'celery_task_publish_retry'},
+        ),
+        publish_retry_policy=Option(
+            {'max_retries': 3,
+             'interval_start': 0,
+             'interval_max': 1,
+             'interval_step': 0.2},
+            type='dict', old={'celery_task_publish_retry_policy'},
+        ),
+        queues=Option(type='dict'),
+        queue_ha_policy=Option(None, type='string'),
+        queue_max_priority=Option(None, type='int'),
+        reject_on_worker_lost=Option(type='bool'),
+        routes=Option(type='any'),
+        send_error_emails=Option(
+            False, type='bool', old={'celery_send_task_error_emails'},
+        ),
+        send_sent_event=Option(
+            False, type='bool', old={'celery_send_task_sent_event'},
+        ),
+        serializer=Option('json', old={'celery_task_serializer'}),
+        soft_time_limit=Option(
+            type='float', old={'celeryd_task_soft_time_limit'},
+        ),
+        time_limit=Option(
+            type='float', old={'celeryd_task_time_limit'},
+        ),
+        store_errors_even_if_ignored=Option(False, type='bool'),
+        track_started=Option(False, type='bool'),
+    ),
+    worker=Namespace(
+        __old__=OLD_NS_WORKER,
+        agent=Option(None, type='string'),
+        autoscaler=Option('celery.worker.autoscale:Autoscaler'),
+        autoreloader=Option('celery.worker.autoreload:Autoreloader'),
+        concurrency=Option(0, type='int'),
+        consumer=Option('celery.worker.consumer:Consumer', type='string'),
+        direct=Option(False, type='bool', old={'celery_worker_direct'}),
+        disable_rate_limits=Option(
+            False, type='bool', old={'celery_disable_rate_limits'},
+        ),
+        enable_remote_control=Option(
+            True, type='bool', old={'celery_enable_remote_control'},
+        ),
+        force_execv=Option(False, type='bool'),
+        hijack_root_logger=Option(True, type='bool'),
+        log_color=Option(type='bool'),
+        log_format=Option(DEFAULT_PROCESS_LOG_FMT),
+        lost_wait=Option(10.0, type='float'),
+        max_memory_per_child=Option(type='int'),
+        max_tasks_per_child=Option(type='int'),
+        pool=Option(DEFAULT_POOL),
+        pool_putlocks=Option(True, type='bool'),
+        pool_restarts=Option(False, type='bool'),
+        prefetch_multiplier=Option(4, type='int'),
+        redirect_stdouts=Option(
+            True, type='bool', old={'celery_redirect_stdouts'},
+        ),
+        redirect_stdouts_level=Option(
+            'WARNING', old={'celery_redirect_stdouts_level'},
+        ),
+        send_events=Option(False, type='bool'),
+        state_db=Option(),
+        task_log_format=Option(DEFAULT_TASK_LOG_FMT),
+        timer=Option(type='string'),
+        timer_precision=Option(1.0, type='float'),
+    ),
+)
+
+
+def _flatten_keys(ns, key, opt):
+    return [(ns + key, opt)]
+
+
+def _to_compat(ns, key, opt):
+    if opt.old:
+        return [
+            (oldkey.format(key).upper(), ns + key, opt)
+            for oldkey in opt.old
+        ]
+    return [((ns + key).upper(), ns + key, opt)]
+
+
+def flatten(d, root='', keyfilter=_flatten_keys):
+    stack = deque([(root, d)])
     while stack:
     while stack:
-        name, space = stack.popleft()
-        for key, value in items(space):
-            if isinstance(value, dict):
-                stack.append((name + key + '_', value))
+        ns, options = stack.popleft()
+        for key, opt in items(options):
+            if isinstance(opt, dict):
+                stack.append((ns + key + '_', opt))
             else:
             else:
-                yield name + key, value
-DEFAULTS = {key: value.default for key, value in flatten(NAMESPACES)}
+                for ret in keyfilter(ns, key, opt):
+                    yield ret
+DEFAULTS = {
+    key: opt.default for key, opt in flatten(NAMESPACES)
+}
+__compat = list(flatten(NAMESPACES, keyfilter=_to_compat))
+_OLD_DEFAULTS = {old_key: opt.default for old_key, _, opt in __compat}
+_TO_OLD_KEY = {new_key: old_key for old_key, new_key, _ in __compat}
+_TO_NEW_KEY = {old_key: new_key for old_key, new_key, _ in __compat}
+__compat = None
+
+SETTING_KEYS = set(keys(DEFAULTS))
+_OLD_SETTING_KEYS = set(keys(_TO_NEW_KEY))
 
 
 
 
 def find_deprecated_settings(source):
 def find_deprecated_settings(source):
@@ -238,20 +348,20 @@ def find_deprecated_settings(source):
 @memoize(maxsize=None)
 @memoize(maxsize=None)
 def find(name, namespace='celery'):
 def find(name, namespace='celery'):
     # - Try specified namespace first.
     # - Try specified namespace first.
-    namespace = namespace.upper()
+    namespace = namespace.lower()
     try:
     try:
         return searchresult(
         return searchresult(
-            namespace, name.upper(), NAMESPACES[namespace][name.upper()],
+            namespace, name.lower(), NAMESPACES[namespace][name.lower()],
         )
         )
     except KeyError:
     except KeyError:
         # - Try all the other namespaces.
         # - Try all the other namespaces.
-        for ns, keys in items(NAMESPACES):
-            if ns.upper() == name.upper():
-                return searchresult(None, ns, keys)
-            elif isinstance(keys, dict):
+        for ns, opts in items(NAMESPACES):
+            if ns.lower() == name.lower():
+                return searchresult(None, ns, opts)
+            elif isinstance(opts, dict):
                 try:
                 try:
-                    return searchresult(ns, name.upper(), keys[name.upper()])
+                    return searchresult(ns, name.lower(), opts[name.lower()])
                 except KeyError:
                 except KeyError:
                     pass
                     pass
     # - See if name is a qualname last.
     # - See if name is a qualname last.
-    return searchresult(None, name.upper(), DEFAULTS[name.upper()])
+    return searchresult(None, name.lower(), DEFAULTS[name.lower()])

+ 4 - 4
celery/app/log.py

@@ -59,9 +59,9 @@ class Logging(object):
     def __init__(self, app):
     def __init__(self, app):
         self.app = app
         self.app = app
         self.loglevel = mlevel(logging.WARN)
         self.loglevel = mlevel(logging.WARN)
-        self.format = self.app.conf.CELERYD_LOG_FORMAT
-        self.task_format = self.app.conf.CELERYD_TASK_LOG_FORMAT
-        self.colorize = self.app.conf.CELERYD_LOG_COLOR
+        self.format = self.app.conf.worker_log_format
+        self.task_format = self.app.conf.worker_task_log_format
+        self.colorize = self.app.conf.worker_log_color
 
 
     def setup(self, loglevel=None, logfile=None, redirect_stdouts=False,
     def setup(self, loglevel=None, logfile=None, redirect_stdouts=False,
               redirect_level='WARNING', colorize=None, hostname=None):
               redirect_level='WARNING', colorize=None, hostname=None):
@@ -105,7 +105,7 @@ class Logging(object):
         if not receivers:
         if not receivers:
             root = logging.getLogger()
             root = logging.getLogger()
 
 
-            if self.app.conf.CELERYD_HIJACK_ROOT_LOGGER:
+            if self.app.conf.worker_hijack_root_logger:
                 root.handlers = []
                 root.handlers = []
                 get_logger('celery').handlers = []
                 get_logger('celery').handlers = []
                 get_logger('celery.task').handlers = []
                 get_logger('celery.task').handlers = []

+ 4 - 4
celery/app/routes.py

@@ -4,7 +4,7 @@
     ~~~~~~~~~~~~~
     ~~~~~~~~~~~~~
 
 
     Contains utilities for working with task routers,
     Contains utilities for working with task routers,
-    (:setting:`CELERY_ROUTES`).
+    (:setting:`task_routes`).
 
 
 """
 """
 from __future__ import absolute_import
 from __future__ import absolute_import
@@ -52,7 +52,7 @@ class Router(object):
                 return lpmerge(self.expand_destination(route), options)
                 return lpmerge(self.expand_destination(route), options)
         if 'queue' not in options:
         if 'queue' not in options:
             options = lpmerge(self.expand_destination(
             options = lpmerge(self.expand_destination(
-                              self.app.conf.CELERY_DEFAULT_QUEUE), options)
+                              self.app.conf.task_default_queue), options)
         return options
         return options
 
 
     def expand_destination(self, route):
     def expand_destination(self, route):
@@ -72,7 +72,7 @@ class Router(object):
                     route['queue'] = self.queues[queue]
                     route['queue'] = self.queues[queue]
                 except KeyError:
                 except KeyError:
                     raise QueueNotFound(
                     raise QueueNotFound(
-                        'Queue {0!r} missing from CELERY_QUEUES'.format(queue))
+                        'Queue {0!r} missing from task_queues'.format(queue))
         return route
         return route
 
 
     def lookup_route(self, task, args=None, kwargs=None):
     def lookup_route(self, task, args=None, kwargs=None):
@@ -80,7 +80,7 @@ class Router(object):
 
 
 
 
 def prepare(routes):
 def prepare(routes):
-    """Expands the :setting:`CELERY_ROUTES` setting."""
+    """Expands the :setting:`task_routes` setting."""
 
 
     def expand_route(route):
     def expand_route(route):
         if isinstance(route, dict):
         if isinstance(route, dict):

+ 24 - 25
celery/app/task.py

@@ -160,7 +160,7 @@ class Task(object):
     rate_limit = None
     rate_limit = None
 
 
     #: If enabled the worker will not store task state and return values
     #: If enabled the worker will not store task state and return values
-    #: for this task.  Defaults to the :setting:`CELERY_IGNORE_RESULT`
+    #: for this task.  Defaults to the :setting:`task_ignore_result`
     #: setting.
     #: setting.
     ignore_result = None
     ignore_result = None
 
 
@@ -173,7 +173,7 @@ class Task(object):
     #: configured to ignore results.
     #: configured to ignore results.
     store_errors_even_if_ignored = None
     store_errors_even_if_ignored = None
 
 
-    #: If enabled an email will be sent to :setting:`ADMINS` whenever a task
+    #: If enabled an email will be sent to :setting:`admins` whenever a task
     #: of this type fails.
     #: of this type fails.
     send_error_emails = None
     send_error_emails = None
 
 
@@ -182,11 +182,11 @@ class Task(object):
     serializer = None
     serializer = None
 
 
     #: Hard time limit.
     #: Hard time limit.
-    #: Defaults to the :setting:`CELERYD_TASK_TIME_LIMIT` setting.
+    #: Defaults to the :setting:`task_time_limit` setting.
     time_limit = None
     time_limit = None
 
 
     #: Soft time limit.
     #: Soft time limit.
-    #: Defaults to the :setting:`CELERYD_TASK_SOFT_TIME_LIMIT` setting.
+    #: Defaults to the :setting:`task_soft_time_limit` setting.
     soft_time_limit = None
     soft_time_limit = None
 
 
     #: The result store backend used for this task.
     #: The result store backend used for this task.
@@ -205,7 +205,7 @@ class Task(object):
     #: running.
     #: running.
     #:
     #:
     #: The application default can be overridden using the
     #: The application default can be overridden using the
-    #: :setting:`CELERY_TRACK_STARTED` setting.
+    #: :setting:`task_track_started` setting.
     track_started = None
     track_started = None
 
 
     #: When enabled messages for this task will be acknowledged **after**
     #: When enabled messages for this task will be acknowledged **after**
@@ -217,7 +217,7 @@ class Task(object):
     #: applications).
     #: applications).
     #:
     #:
     #: The application default can be overridden with the
     #: The application default can be overridden with the
-    #: :setting:`CELERY_ACKS_LATE` setting.
+    #: :setting:`task_acks_late` setting.
     acks_late = None
     acks_late = None
 
 
     #: Even if :attr:`acks_late` is enabled, the worker will
     #: Even if :attr:`acks_late` is enabled, the worker will
@@ -255,15 +255,14 @@ class Task(object):
     __bound__ = False
     __bound__ = False
 
 
     from_config = (
     from_config = (
-        ('send_error_emails', 'CELERY_SEND_TASK_ERROR_EMAILS'),
-        ('serializer', 'CELERY_TASK_SERIALIZER'),
-        ('rate_limit', 'CELERY_DEFAULT_RATE_LIMIT'),
-        ('track_started', 'CELERY_TRACK_STARTED'),
-        ('acks_late', 'CELERY_ACKS_LATE'),
-        ('reject_on_worker_lost', 'CELERY_REJECT_ON_WORKER_LOST'),
-        ('ignore_result', 'CELERY_IGNORE_RESULT'),
-        ('store_errors_even_if_ignored',
-            'CELERY_STORE_ERRORS_EVEN_IF_IGNORED'),
+        ('send_error_emails', 'task_send_error_emails'),
+        ('serializer', 'task_serializer'),
+        ('rate_limit', 'task_default_rate_limit'),
+        ('track_started', 'task_track_started'),
+        ('acks_late', 'task_acks_late'),
+        ('reject_on_worker_lost', 'task_reject_on_worker_lost'),
+        ('ignore_result', 'task_ignore_result'),
+        ('store_errors_even_if_ignored', 'task_store_errors_even_if_ignored'),
     )
     )
 
 
     #: ignored
     #: ignored
@@ -409,12 +408,12 @@ class Task(object):
 
 
         :keyword retry: If enabled sending of the task message will be retried
         :keyword retry: If enabled sending of the task message will be retried
                         in the event of connection loss or failure.  Default
                         in the event of connection loss or failure.  Default
-                        is taken from the :setting:`CELERY_TASK_PUBLISH_RETRY`
+                        is taken from the :setting:`task_publish_retry`
                         setting.  Note that you need to handle the
                         setting.  Note that you need to handle the
                         producer/connection manually for this to work.
                         producer/connection manually for this to work.
 
 
         :keyword retry_policy:  Override the retry policy used.  See the
         :keyword retry_policy:  Override the retry policy used.  See the
-                                :setting:`CELERY_TASK_PUBLISH_RETRY_POLICY`
+                                :setting:`task_publish_retry_policy`
                                 setting.
                                 setting.
 
 
         :keyword routing_key: Custom routing key used to route the task to a
         :keyword routing_key: Custom routing key used to route the task to a
@@ -423,8 +422,8 @@ class Task(object):
                               routing keys to topic exchanges.
                               routing keys to topic exchanges.
 
 
         :keyword queue: The queue to route the task to.  This must be a key
         :keyword queue: The queue to route the task to.  This must be a key
-                        present in :setting:`CELERY_QUEUES`, or
-                        :setting:`CELERY_CREATE_MISSING_QUEUES` must be
+                        present in :setting:`task_queues`, or
+                        :setting:`task_create_missing_queues` must be
                         enabled.  See :ref:`guide-routing` for more
                         enabled.  See :ref:`guide-routing` for more
                         information.
                         information.
 
 
@@ -446,7 +445,7 @@ class Task(object):
                               to use.  Can be one of ``zlib``, ``bzip2``,
                               to use.  Can be one of ``zlib``, ``bzip2``,
                               or any custom compression methods registered with
                               or any custom compression methods registered with
                               :func:`kombu.compression.register`. Defaults to
                               :func:`kombu.compression.register`. Defaults to
-                              the :setting:`CELERY_MESSAGE_COMPRESSION`
+                              the :setting:`task_compression`
                               setting.
                               setting.
         :keyword link: A single, or a list of tasks to apply if the
         :keyword link: A single, or a list of tasks to apply if the
                        task exits successfully.
                        task exits successfully.
@@ -467,14 +466,14 @@ class Task(object):
             task (a :class:`dict`)
             task (a :class:`dict`)
 
 
         :rtype :class:`celery.result.AsyncResult`: if
         :rtype :class:`celery.result.AsyncResult`: if
-            :setting:`CELERY_ALWAYS_EAGER` is not set, otherwise
+            :setting:`task_always_eager` is not set, otherwise
             :class:`celery.result.EagerResult`:
             :class:`celery.result.EagerResult`:
 
 
         Also supports all keyword arguments supported by
         Also supports all keyword arguments supported by
         :meth:`kombu.Producer.publish`.
         :meth:`kombu.Producer.publish`.
 
 
         .. note::
         .. note::
-            If the :setting:`CELERY_ALWAYS_EAGER` setting is set, it will
+            If the :setting:`task_always_eager` setting is set, it will
             be replaced by a local :func:`apply` call instead.
             be replaced by a local :func:`apply` call instead.
 
 
         """
         """
@@ -486,7 +485,7 @@ class Task(object):
             check_arguments(*(args or ()), **(kwargs or {}))
             check_arguments(*(args or ()), **(kwargs or {}))
 
 
         app = self._get_app()
         app = self._get_app()
-        if app.conf.CELERY_ALWAYS_EAGER:
+        if app.conf.task_always_eager:
             return self.apply(args, kwargs, task_id=task_id or uuid(),
             return self.apply(args, kwargs, task_id=task_id or uuid(),
                               link=link, link_error=link_error, **options)
                               link=link, link_error=link_error, **options)
         # add 'self' if this is a "task_method".
         # add 'self' if this is a "task_method".
@@ -670,7 +669,7 @@ class Task(object):
         :param args: positional arguments passed on to the task.
         :param args: positional arguments passed on to the task.
         :param kwargs: keyword arguments passed on to the task.
         :param kwargs: keyword arguments passed on to the task.
         :keyword throw: Re-raise task exceptions.  Defaults to
         :keyword throw: Re-raise task exceptions.  Defaults to
-                        the :setting:`CELERY_EAGER_PROPAGATES_EXCEPTIONS`
+                        the :setting:`task_eager_propagates_exceptions`
                         setting.
                         setting.
 
 
         :rtype :class:`celery.result.EagerResult`:
         :rtype :class:`celery.result.EagerResult`:
@@ -687,7 +686,7 @@ class Task(object):
         kwargs = kwargs or {}
         kwargs = kwargs or {}
         task_id = options.get('task_id') or uuid()
         task_id = options.get('task_id') or uuid()
         retries = options.get('retries', 0)
         retries = options.get('retries', 0)
-        throw = app.either('CELERY_EAGER_PROPAGATES_EXCEPTIONS',
+        throw = app.either('task_eager_propagates_exceptions',
                            options.pop('throw', None))
                            options.pop('throw', None))
 
 
         # Make sure we get the task instance, not class.
         # Make sure we get the task instance, not class.

+ 2 - 2
celery/app/trace.py

@@ -464,7 +464,7 @@ def _trace_task_ret(name, uuid, request, body, content_type,
     app = app or current_app._get_current_object()
     app = app or current_app._get_current_object()
     embed = None
     embed = None
     if content_type:
     if content_type:
-        accept = prepare_accept_content(app.conf.CELERY_ACCEPT_CONTENT)
+        accept = prepare_accept_content(app.conf.accept_content)
         args, kwargs, embed = loads(
         args, kwargs, embed = loads(
             body, content_type, content_encoding, accept=accept,
             body, content_type, content_encoding, accept=accept,
         )
         )
@@ -539,7 +539,7 @@ def setup_worker_optimizations(app, hostname=None):
     # set fast shortcut to task registry
     # set fast shortcut to task registry
     _localized[:] = [
     _localized[:] = [
         app._tasks,
         app._tasks,
-        prepare_accept_content(app.conf.CELERY_ACCEPT_CONTENT),
+        prepare_accept_content(app.conf.accept_content),
         hostname,
         hostname,
     ]
     ]
 
 

+ 107 - 28
celery/app/utils.py

@@ -12,18 +12,23 @@ import os
 import platform as _platform
 import platform as _platform
 import re
 import re
 
 
-from collections import Mapping
+from collections import Mapping, namedtuple
+from copy import deepcopy
 from types import ModuleType
 from types import ModuleType
 
 
 from kombu.utils.url import maybe_sanitize_url
 from kombu.utils.url import maybe_sanitize_url
 
 
 from celery.datastructures import ConfigurationView
 from celery.datastructures import ConfigurationView
-from celery.five import items, string_t, values
+from celery.exceptions import ImproperlyConfigured
+from celery.five import items, keys, string_t, values
 from celery.platforms import pyimplementation
 from celery.platforms import pyimplementation
 from celery.utils.text import pretty
 from celery.utils.text import pretty
 from celery.utils.imports import import_from_cwd, symbol_by_name, qualname
 from celery.utils.imports import import_from_cwd, symbol_by_name, qualname
 
 
-from .defaults import find
+from .defaults import (
+    _TO_NEW_KEY, _TO_OLD_KEY, _OLD_DEFAULTS, _OLD_SETTING_KEYS,
+    DEFAULTS, SETTING_KEYS, find,
+)
 
 
 __all__ = ['Settings', 'appstr', 'bugreport',
 __all__ = ['Settings', 'appstr', 'bugreport',
            'filter_hidden_settings', 'find_app']
            'filter_hidden_settings', 'find_app']
@@ -44,6 +49,28 @@ HIDDEN_SETTINGS = re.compile(
     re.IGNORECASE,
     re.IGNORECASE,
 )
 )
 
 
+E_MIX_OLD_INTO_NEW = """
+
+Cannot mix new and old setting keys, please rename the
+following settings to the new format:
+
+{renames}
+
+"""
+
+E_MIX_NEW_INTO_OLD = """
+
+Cannot mix new setting names with old setting names, please
+rename the following settings to use the old format:
+
+{renames}
+
+Or change all of the settings to use the new format :)
+
+"""
+
+FMT_REPLACE_SETTING = '{replace:<36} -> {with_}'
+
 
 
 def appstr(app):
 def appstr(app):
     """String used in __repr__ etc, to id app instances."""
     """String used in __repr__ etc, to id app instances."""
@@ -60,28 +87,14 @@ class Settings(ConfigurationView):
     """
     """
 
 
     @property
     @property
-    def CELERY_RESULT_BACKEND(self):
-        return self.first('CELERY_RESULT_BACKEND', 'CELERY_BACKEND')
-
-    @property
-    def BROKER_TRANSPORT(self):
-        return self.first('BROKER_TRANSPORT',
-                          'BROKER_BACKEND', 'CARROT_BACKEND')
-
-    @property
-    def BROKER_BACKEND(self):
-        """Deprecated compat alias to :attr:`BROKER_TRANSPORT`."""
-        return self.BROKER_TRANSPORT
-
-    @property
-    def BROKER_URL(self):
+    def broker_url(self):
         return (os.environ.get('CELERY_BROKER_URL') or
         return (os.environ.get('CELERY_BROKER_URL') or
-                self.first('BROKER_URL', 'BROKER_HOST'))
+                self.first('broker_url', 'broker_host'))
 
 
     @property
     @property
-    def CELERY_TIMEZONE(self):
+    def timezone(self):
         # this way we also support django's time zone.
         # this way we also support django's time zone.
-        return self.first('CELERY_TIMEZONE', 'TIME_ZONE')
+        return self.first('timezone', 'time_zone')
 
 
     def without_defaults(self):
     def without_defaults(self):
         """Return the current configuration, but without defaults."""
         """Return the current configuration, but without defaults."""
@@ -91,18 +104,18 @@ class Settings(ConfigurationView):
     def value_set_for(self, key):
     def value_set_for(self, key):
         return key in self.without_defaults()
         return key in self.without_defaults()
 
 
-    def find_option(self, name, namespace='celery'):
+    def find_option(self, name, namespace=''):
         """Search for option by name.
         """Search for option by name.
 
 
         Will return ``(namespace, key, type)`` tuple, e.g.::
         Will return ``(namespace, key, type)`` tuple, e.g.::
 
 
             >>> from proj.celery import app
             >>> from proj.celery import app
             >>> app.conf.find_option('disable_rate_limits')
             >>> app.conf.find_option('disable_rate_limits')
-            ('CELERY', 'DISABLE_RATE_LIMITS',
+            ('worker', 'prefetch_multiplier',
              <Option: type->bool default->False>))
              <Option: type->bool default->False>))
 
 
         :param name: Name of option, cannot be partial.
         :param name: Name of option, cannot be partial.
-        :keyword namespace: Preferred namespace (``CELERY`` by default).
+        :keyword namespace: Preferred namespace (``None`` by default).
 
 
         """
         """
         return find(name, namespace)
         return find(name, namespace)
@@ -117,7 +130,7 @@ class Settings(ConfigurationView):
         Example::
         Example::
 
 
             >>> from proj.celery import app
             >>> from proj.celery import app
-            >>> app.conf.get_by_parts('CELERY', 'DISABLE_RATE_LIMITS')
+            >>> app.conf.get_by_parts('worker', 'disable_rate_limits')
             False
             False
 
 
         """
         """
@@ -139,6 +152,72 @@ class Settings(ConfigurationView):
             for key, value in items(self.table(with_defaults, censored)))
             for key, value in items(self.table(with_defaults, censored)))
 
 
 
 
+def _new_key_to_old(key, convert=_TO_OLD_KEY.get):
+    return convert(key, key)
+
+
+def _old_key_to_new(key, convert=_TO_NEW_KEY.get):
+    return convert(key, key)
+
+
+_settings_info_t = namedtuple('settings_info_t', (
+    'defaults', 'convert', 'key_t', 'mix_error',
+))
+
+_settings_info = _settings_info_t(
+    DEFAULTS, _TO_NEW_KEY, _old_key_to_new, E_MIX_OLD_INTO_NEW,
+)
+_old_settings_info = _settings_info_t(
+    _OLD_DEFAULTS, _TO_OLD_KEY, _new_key_to_old, E_MIX_NEW_INTO_OLD,
+)
+
+
+def detect_settings(conf, preconf={}, ignore_keys=set(), prefix=None,
+                    all_keys=SETTING_KEYS, old_keys=_OLD_SETTING_KEYS):
+    source = conf
+    if conf is None:
+        source, conf = preconf, {}
+    have = set(keys(source)) - ignore_keys
+    is_in_new = have.intersection(all_keys)
+    is_in_old = have.intersection(old_keys)
+
+    if is_in_new:
+        # have new setting names
+        info, left = _settings_info, is_in_old
+        if is_in_old and len(is_in_old) > len(is_in_new):
+            # Majority of the settings are old.
+            info, left = _old_settings_info, is_in_new
+    elif is_in_old:
+        print('IS IN OLD: %r' % (is_in_old, ))
+        # have old setting names, or a majority of the names are old.
+        info, left = _old_settings_info, is_in_new
+        if is_in_new and len(is_in_new) > len(is_in_old):
+            # Majority of the settings are new
+            info, left = _settings_info, is_in_old
+    else:
+        # no settings, just use new format.
+        info, left = _settings_info, is_in_old
+
+    if prefix:
+        # always use new format if prefix is used.
+        info, left = _settings_info, set()
+
+    # only raise error for keys that the user did not provide two keys
+    # for (e.g. both ``result_expires`` and ``CELERY_TASK_RESULT_EXPIRES``).
+    really_left = {key for key in left if info.convert[key] not in have}
+    if really_left:
+        # user is mixing old/new, or new/old settings, give renaming
+        # suggestions.
+        raise ImproperlyConfigured(info.mix_error.format(renames='\n'.join(
+            FMT_REPLACE_SETTING.format(replace=key, with_=info.convert[key])
+            for key in sorted(really_left)
+        )))
+
+    preconf = {info.convert.get(k, k): v for k, v in items(preconf)}
+    defaults = dict(deepcopy(info.defaults), **preconf)
+    return Settings(preconf, [conf, defaults], info.key_t, prefix=prefix)
+
+
 class AppPickler(object):
 class AppPickler(object):
     """Old application pickler/unpickler (< 3.1)."""
     """Old application pickler/unpickler (< 3.1)."""
 
 
@@ -185,10 +264,10 @@ def filter_hidden_settings(conf):
         if isinstance(key, string_t):
         if isinstance(key, string_t):
             if HIDDEN_SETTINGS.search(key):
             if HIDDEN_SETTINGS.search(key):
                 return mask
                 return mask
-            elif 'BROKER_URL' in key.upper():
+            elif 'broker_url' in key.lower():
                 from kombu import Connection
                 from kombu import Connection
                 return Connection(value).as_uri(mask=mask)
                 return Connection(value).as_uri(mask=mask)
-            elif key.upper() in ('CELERY_RESULT_BACKEND', 'CELERY_BACKEND'):
+            elif 'backend' in key.lower():
                 return maybe_sanitize_url(value, mask=mask)
                 return maybe_sanitize_url(value, mask=mask)
 
 
         return value
         return value
@@ -220,7 +299,7 @@ def bugreport(app):
         py_v=_platform.python_version(),
         py_v=_platform.python_version(),
         driver_v=driver_v,
         driver_v=driver_v,
         transport=transport,
         transport=transport,
-        results=app.conf.CELERY_RESULT_BACKEND or 'disabled',
+        results=app.conf.result_backend or 'disabled',
         human_settings=app.conf.humanize(),
         human_settings=app.conf.humanize(),
         loader=qualname(app.loader.__class__),
         loader=qualname(app.loader.__class__),
     )
     )

+ 7 - 11
celery/apps/beat.py

@@ -48,15 +48,16 @@ class Beat(object):
                  redirect_stdouts_level=None, **kwargs):
                  redirect_stdouts_level=None, **kwargs):
         """Starts the beat task scheduler."""
         """Starts the beat task scheduler."""
         self.app = app = app or self.app
         self.app = app = app or self.app
+        either = self.app.either
         self.loglevel = loglevel
         self.loglevel = loglevel
         self.logfile = logfile
         self.logfile = logfile
-        self.schedule = self._getopt('schedule_filename', schedule)
-        self.scheduler_cls = self._getopt('scheduler', scheduler_cls)
-        self.redirect_stdouts = self._getopt(
-            'redirect_stdouts', redirect_stdouts,
+        self.schedule = either('beat_schedule_filename', schedule)
+        self.scheduler_cls = either('beat_scheduler', scheduler_cls)
+        self.redirect_stdouts = either(
+            'worker_redirect_stdouts', redirect_stdouts,
         )
         )
-        self.redirect_stdouts_level = self._getopt(
-            'redirect_stdouts_level', redirect_stdouts_level,
+        self.redirect_stdouts_level = either(
+            'worker_redirect_stdouts_level', redirect_stdouts_level,
         )
         )
 
 
         self.max_interval = max_interval
         self.max_interval = max_interval
@@ -71,11 +72,6 @@ class Beat(object):
         if not isinstance(self.loglevel, numbers.Integral):
         if not isinstance(self.loglevel, numbers.Integral):
             self.loglevel = LOG_LEVELS[self.loglevel.upper()]
             self.loglevel = LOG_LEVELS[self.loglevel.upper()]
 
 
-    def _getopt(self, key, value):
-        if value is not None:
-            return value
-        return self.app.conf.find_value_for_key(key, namespace='celerybeat')
-
     def run(self):
     def run(self):
         print(str(self.colored.cyan(
         print(str(self.colored.cyan(
             'celery beat v{0} is starting.'.format(VERSION_BANNER))))
             'celery beat v{0} is starting.'.format(VERSION_BANNER))))

+ 8 - 9
celery/apps/worker.py

@@ -57,7 +57,7 @@ If you depend on pickle then you should set a setting to disable this
 warning and to be sure that everything will continue working
 warning and to be sure that everything will continue working
 when you upgrade to Celery 4.0::
 when you upgrade to Celery 4.0::
 
 
-    CELERY_ACCEPT_CONTENT = ['pickle', 'json', 'msgpack', 'yaml']
+    accept_content = ['pickle', 'json', 'msgpack', 'yaml']
 
 
 You must only enable the serializers that you will actually use.
 You must only enable the serializers that you will actually use.
 
 
@@ -120,16 +120,16 @@ class Worker(WorkController):
             sender=self.hostname, instance=self,
             sender=self.hostname, instance=self,
             conf=self.app.conf, options=kwargs,
             conf=self.app.conf, options=kwargs,
         )
         )
-        check_privileges(self.app.conf.CELERY_ACCEPT_CONTENT)
+        check_privileges(self.app.conf.accept_content)
 
 
     def on_after_init(self, purge=False, no_color=None,
     def on_after_init(self, purge=False, no_color=None,
                       redirect_stdouts=None, redirect_stdouts_level=None,
                       redirect_stdouts=None, redirect_stdouts_level=None,
                       **kwargs):
                       **kwargs):
-        self.redirect_stdouts = self._getopt(
-            'redirect_stdouts', redirect_stdouts,
+        self.redirect_stdouts = self.app.either(
+            'worker_redirect_stdouts', redirect_stdouts,
         )
         )
-        self.redirect_stdouts_level = self._getopt(
-            'redirect_stdouts_level', redirect_stdouts_level,
+        self.redirect_stdouts_level = self.app.either(
+            'worker_redirect_stdouts_level', redirect_stdouts_level,
         )
         )
         super(Worker, self).setup_defaults(**kwargs)
         super(Worker, self).setup_defaults(**kwargs)
         self.purge = purge
         self.purge = purge
@@ -158,7 +158,7 @@ class Worker(WorkController):
             sender=self.hostname, instance=self, conf=self.app.conf,
             sender=self.hostname, instance=self, conf=self.app.conf,
         )
         )
 
 
-        if not self.app.conf.value_set_for('CELERY_ACCEPT_CONTENT'):
+        if not self.app.conf.value_set_for('accept_content'):
             warnings.warn(CDeprecationWarning(W_PICKLE_DEPRECATED))
             warnings.warn(CDeprecationWarning(W_PICKLE_DEPRECATED))
 
 
         if self.purge:
         if self.purge:
@@ -229,7 +229,7 @@ class Worker(WorkController):
             version=VERSION_BANNER,
             version=VERSION_BANNER,
             conninfo=self.app.connection().as_uri(),
             conninfo=self.app.connection().as_uri(),
             results=maybe_sanitize_url(
             results=maybe_sanitize_url(
-                self.app.conf.CELERY_RESULT_BACKEND or 'disabled',
+                self.app.conf.result_backend or 'disabled',
             ),
             ),
             concurrency=concurrency,
             concurrency=concurrency,
             platform=safe_str(_platform.platform()),
             platform=safe_str(_platform.platform()),
@@ -281,7 +281,6 @@ class Worker(WorkController):
 
 
 def _shutdown_handler(worker, sig='TERM', how='Warm',
 def _shutdown_handler(worker, sig='TERM', how='Warm',
                       exc=WorkerShutdown, callback=None, exitcode=EX_OK):
                       exc=WorkerShutdown, callback=None, exitcode=EX_OK):
-
     def _handle_request(*args):
     def _handle_request(*args):
         with in_sighandler():
         with in_sighandler():
             from celery.worker import state
             from celery.worker import state

+ 3 - 3
celery/backends/amqp.py

@@ -73,12 +73,12 @@ class AMQPBackend(BaseBackend):
         self._connection = connection
         self._connection = connection
         self.persistent = self.prepare_persistent(persistent)
         self.persistent = self.prepare_persistent(persistent)
         self.delivery_mode = 2 if self.persistent else 1
         self.delivery_mode = 2 if self.persistent else 1
-        exchange = exchange or conf.CELERY_RESULT_EXCHANGE
-        exchange_type = exchange_type or conf.CELERY_RESULT_EXCHANGE_TYPE
+        exchange = exchange or conf.result_exchange
+        exchange_type = exchange_type or conf.result_exchange_type
         self.exchange = self._create_exchange(
         self.exchange = self._create_exchange(
             exchange, exchange_type, self.delivery_mode,
             exchange, exchange_type, self.delivery_mode,
         )
         )
-        self.serializer = serializer or conf.CELERY_RESULT_SERIALIZER
+        self.serializer = serializer or conf.result_serializer
         self.auto_delete = auto_delete
         self.auto_delete = auto_delete
         self.queue_arguments = dictfilter({
         self.queue_arguments = dictfilter({
             'x-expires': maybe_s_to_ms(self.expires),
             'x-expires': maybe_s_to_ms(self.expires),

+ 6 - 6
celery/backends/base.py

@@ -96,16 +96,16 @@ class BaseBackend(object):
                  expires=None, expires_type=None, **kwargs):
                  expires=None, expires_type=None, **kwargs):
         self.app = app
         self.app = app
         conf = self.app.conf
         conf = self.app.conf
-        self.serializer = serializer or conf.CELERY_RESULT_SERIALIZER
+        self.serializer = serializer or conf.result_serializer
         (self.content_type,
         (self.content_type,
          self.content_encoding,
          self.content_encoding,
          self.encoder) = serializer_registry._encoders[self.serializer]
          self.encoder) = serializer_registry._encoders[self.serializer]
-        cmax = max_cached_results or conf.CELERY_MAX_CACHED_RESULTS
+        cmax = max_cached_results or conf.result_cache_max
         self._cache = _nulldict() if cmax == -1 else LRUCache(limit=cmax)
         self._cache = _nulldict() if cmax == -1 else LRUCache(limit=cmax)
 
 
         self.expires = self.prepare_expires(expires, expires_type)
         self.expires = self.prepare_expires(expires, expires_type)
         self.accept = prepare_accept_content(
         self.accept = prepare_accept_content(
-            conf.CELERY_ACCEPT_CONTENT if accept is None else accept,
+            conf.accept_content if accept is None else accept,
         )
         )
 
 
     def mark_as_started(self, task_id, **meta):
     def mark_as_started(self, task_id, **meta):
@@ -242,7 +242,7 @@ class BaseBackend(object):
 
 
     def prepare_expires(self, value, type=None):
     def prepare_expires(self, value, type=None):
         if value is None:
         if value is None:
-            value = self.app.conf.CELERY_TASK_RESULT_EXPIRES
+            value = self.app.conf.result_expires
         if isinstance(value, timedelta):
         if isinstance(value, timedelta):
             value = value.total_seconds()
             value = value.total_seconds()
         if value is not None and type:
         if value is not None and type:
@@ -252,7 +252,7 @@ class BaseBackend(object):
     def prepare_persistent(self, enabled=None):
     def prepare_persistent(self, enabled=None):
         if enabled is not None:
         if enabled is not None:
             return enabled
             return enabled
-        p = self.app.conf.CELERY_RESULT_PERSISTENT
+        p = self.app.conf.result_persistent
         return self.persistent if p is None else p
         return self.persistent if p is None else p
 
 
     def encode_result(self, result, status):
     def encode_result(self, result, status):
@@ -558,7 +558,7 @@ class KeyValueStoreBackend(BaseBackend):
             return
             return
         app = self.app
         app = self.app
         if propagate is None:
         if propagate is None:
-            propagate = app.conf.CELERY_CHORD_PROPAGATES
+            propagate = app.conf.chord_propagates
         gid = request.group
         gid = request.group
         if not gid:
         if not gid:
             return
             return

+ 2 - 2
celery/backends/cache.py

@@ -101,10 +101,10 @@ class CacheBackend(KeyValueStoreBackend):
                  options={}, url=None, **kwargs):
                  options={}, url=None, **kwargs):
         super(CacheBackend, self).__init__(app, **kwargs)
         super(CacheBackend, self).__init__(app, **kwargs)
 
 
-        self.options = dict(self.app.conf.CELERY_CACHE_BACKEND_OPTIONS,
+        self.options = dict(self.app.conf.cache_backend_options,
                             **options)
                             **options)
 
 
-        self.backend = url or backend or self.app.conf.CELERY_CACHE_BACKEND
+        self.backend = url or backend or self.app.conf.cache_backend
         if self.backend:
         if self.backend:
             self.backend, _, servers = self.backend.partition('://')
             self.backend, _, servers = self.backend.partition('://')
             self.servers = servers.rstrip('/').split(';')
             self.servers = servers.rstrip('/').split(';')

+ 8 - 8
celery/backends/cassandra.py

@@ -59,7 +59,7 @@ class CassandraBackend(BaseBackend):
         """Initialize Cassandra backend.
         """Initialize Cassandra backend.
 
 
         Raises :class:`celery.exceptions.ImproperlyConfigured` if
         Raises :class:`celery.exceptions.ImproperlyConfigured` if
-        the :setting:`CASSANDRA_SERVERS` setting is not set.
+        the :setting:`cassandra_servers` setting is not set.
 
 
         """
         """
         super(CassandraBackend, self).__init__(**kwargs)
         super(CassandraBackend, self).__init__(**kwargs)
@@ -71,21 +71,21 @@ class CassandraBackend(BaseBackend):
 
 
         conf = self.app.conf
         conf = self.app.conf
         self.servers = (servers or
         self.servers = (servers or
-                        conf.get('CASSANDRA_SERVERS') or
+                        conf.get('cassandra_servers') or
                         self.servers)
                         self.servers)
         self.keyspace = (keyspace or
         self.keyspace = (keyspace or
-                         conf.get('CASSANDRA_KEYSPACE') or
+                         conf.get('cassandra_keyspace') or
                          self.keyspace)
                          self.keyspace)
         self.column_family = (column_family or
         self.column_family = (column_family or
-                              conf.get('CASSANDRA_COLUMN_FAMILY') or
+                              conf.get('cassandra_column_family') or
                               self.column_family)
                               self.column_family)
-        self.cassandra_options = dict(conf.get('CASSANDRA_OPTIONS') or {},
+        self.cassandra_options = dict(conf.get('cassandra_options') or {},
                                       **cassandra_options or {})
                                       **cassandra_options or {})
         self.detailed_mode = (detailed_mode or
         self.detailed_mode = (detailed_mode or
-                              conf.get('CASSANDRA_DETAILED_MODE') or
+                              conf.get('cassandra_detailed_mode') or
                               self.detailed_mode)
                               self.detailed_mode)
-        read_cons = conf.get('CASSANDRA_READ_CONSISTENCY') or 'LOCAL_QUORUM'
-        write_cons = conf.get('CASSANDRA_WRITE_CONSISTENCY') or 'LOCAL_QUORUM'
+        read_cons = conf.get('cassandra_read_consistency') or 'LOCAL_QUORUM'
+        write_cons = conf.get('cassandra_write_consistency') or 'LOCAL_QUORUM'
         try:
         try:
             self.read_consistency = getattr(pycassa.ConsistencyLevel,
             self.read_consistency = getattr(pycassa.ConsistencyLevel,
                                             read_cons)
                                             read_cons)

+ 1 - 1
celery/backends/couchbase.py

@@ -63,7 +63,7 @@ class CouchBaseBackend(KeyValueStoreBackend):
             _, uhost, uport, uname, upass, ubucket, _ = _parse_url(url)
             _, uhost, uport, uname, upass, ubucket, _ = _parse_url(url)
             ubucket = ubucket.strip('/') if ubucket else None
             ubucket = ubucket.strip('/') if ubucket else None
 
 
-        config = self.app.conf.get('CELERY_COUCHBASE_BACKEND_SETTINGS', None)
+        config = self.app.conf.get('couchbase_backend_settings', None)
         if config is not None:
         if config is not None:
             if not isinstance(config, dict):
             if not isinstance(config, dict):
                 raise ImproperlyConfigured(
                 raise ImproperlyConfigured(

+ 6 - 6
celery/backends/database/__init__.py

@@ -80,23 +80,23 @@ class DatabaseBackend(BaseBackend):
             expires_type=maybe_timedelta, **kwargs
             expires_type=maybe_timedelta, **kwargs
         )
         )
         conf = self.app.conf
         conf = self.app.conf
-        self.dburi = url or dburi or conf.CELERY_RESULT_DBURI
+        self.dburi = url or dburi or conf.sqlalchemy_dburi
         self.engine_options = dict(
         self.engine_options = dict(
             engine_options or {},
             engine_options or {},
-            **conf.CELERY_RESULT_ENGINE_OPTIONS or {})
+            **conf.sqlalchemy_engine_options or {})
         self.short_lived_sessions = kwargs.get(
         self.short_lived_sessions = kwargs.get(
             'short_lived_sessions',
             'short_lived_sessions',
-            conf.CELERY_RESULT_DB_SHORT_LIVED_SESSIONS,
+            conf.sqlalchemy_short_lived_sessions,
         )
         )
 
 
-        tablenames = conf.CELERY_RESULT_DB_TABLENAMES or {}
+        tablenames = conf.sqlalchemy_table_names or {}
         Task.__table__.name = tablenames.get('task', 'celery_taskmeta')
         Task.__table__.name = tablenames.get('task', 'celery_taskmeta')
         TaskSet.__table__.name = tablenames.get('group', 'celery_tasksetmeta')
         TaskSet.__table__.name = tablenames.get('group', 'celery_tasksetmeta')
 
 
         if not self.dburi:
         if not self.dburi:
             raise ImproperlyConfigured(
             raise ImproperlyConfigured(
-                'Missing connection string! Do you have '
-                'CELERY_RESULT_DBURI set to a real value?')
+                'Missing connection string! Do you have the'
+                ' sqlalchemy_dburi setting set to a real value?')
 
 
     def ResultSession(self, session_manager=SessionManager()):
     def ResultSession(self, session_manager=SessionManager()):
         return session_manager.session_factory(
         return session_manager.session_factory(

+ 1 - 1
celery/backends/mongodb.py

@@ -98,7 +98,7 @@ class MongoBackend(BaseBackend):
             self.options.update(uri_data['options'])
             self.options.update(uri_data['options'])
 
 
         # update conf with specific settings
         # update conf with specific settings
-        config = self.app.conf.get('CELERY_MONGODB_BACKEND_SETTINGS')
+        config = self.app.conf.get('mongodb_backend_settings')
         if config is not None:
         if config is not None:
             if not isinstance(config, dict):
             if not isinstance(config, dict):
                 raise ImproperlyConfigured(
                 raise ImproperlyConfigured(

+ 8 - 8
celery/backends/new_cassandra.py

@@ -83,7 +83,7 @@ class CassandraBackend(BaseBackend):
         """Initialize Cassandra backend.
         """Initialize Cassandra backend.
 
 
         Raises :class:`celery.exceptions.ImproperlyConfigured` if
         Raises :class:`celery.exceptions.ImproperlyConfigured` if
-        the :setting:`CASSANDRA_SERVERS` setting is not set.
+        the :setting:`cassandra_servers` setting is not set.
 
 
         """
         """
         super(CassandraBackend, self).__init__(**kwargs)
         super(CassandraBackend, self).__init__(**kwargs)
@@ -93,24 +93,24 @@ class CassandraBackend(BaseBackend):
 
 
         conf = self.app.conf
         conf = self.app.conf
         self.servers = (servers or
         self.servers = (servers or
-                        conf.get('CASSANDRA_SERVERS', None))
+                        conf.get('cassandra_servers', None))
         self.port = (port or
         self.port = (port or
-                     conf.get('CASSANDRA_PORT', None))
+                     conf.get('cassandra_port', None))
         self.keyspace = (keyspace or
         self.keyspace = (keyspace or
-                         conf.get('CASSANDRA_KEYSPACE', None))
+                         conf.get('cassandra_keyspace', None))
         self.table = (table or
         self.table = (table or
-                      conf.get('CASSANDRA_TABLE', None))
+                      conf.get('cassandra_table', None))
 
 
         if not self.servers or not self.keyspace or not self.table:
         if not self.servers or not self.keyspace or not self.table:
             raise ImproperlyConfigured('Cassandra backend not configured.')
             raise ImproperlyConfigured('Cassandra backend not configured.')
 
 
-        expires = (entry_ttl or conf.get('CASSANDRA_ENTRY_TTL', None))
+        expires = (entry_ttl or conf.get('cassandra_entry_ttl', None))
 
 
         self.cqlexpires = (Q_EXPIRES.format(expires)
         self.cqlexpires = (Q_EXPIRES.format(expires)
                            if expires is not None else '')
                            if expires is not None else '')
 
 
-        read_cons = conf.get('CASSANDRA_READ_CONSISTENCY') or 'LOCAL_QUORUM'
-        write_cons = conf.get('CASSANDRA_WRITE_CONSISTENCY') or 'LOCAL_QUORUM'
+        read_cons = conf.get('cassandra_read_consistency') or 'LOCAL_QUORUM'
+        write_cons = conf.get('cassandra_write_consistency') or 'LOCAL_QUORUM'
 
 
         self.read_consistency = getattr(
         self.read_consistency = getattr(
             cassandra.ConsistencyLevel, read_cons,
             cassandra.ConsistencyLevel, read_cons,

+ 9 - 14
celery/backends/redis.py

@@ -60,32 +60,27 @@ class RedisBackend(KeyValueStoreBackend):
                  max_connections=None, url=None,
                  max_connections=None, url=None,
                  connection_pool=None, new_join=False, **kwargs):
                  connection_pool=None, new_join=False, **kwargs):
         super(RedisBackend, self).__init__(expires_type=int, **kwargs)
         super(RedisBackend, self).__init__(expires_type=int, **kwargs)
-        conf = self.app.conf
+        _get = self.app.conf.get
         if self.redis is None:
         if self.redis is None:
             raise ImproperlyConfigured(REDIS_MISSING)
             raise ImproperlyConfigured(REDIS_MISSING)
 
 
-        # For compatibility with the old REDIS_* configuration keys.
-        def _get(key):
-            for prefix in 'CELERY_REDIS_{0}', 'REDIS_{0}':
-                try:
-                    return conf[prefix.format(key)]
-                except KeyError:
-                    pass
         if host and '://' in host:
         if host and '://' in host:
             url = host
             url = host
             host = None
             host = None
 
 
         self.max_connections = (
         self.max_connections = (
-            max_connections or _get('MAX_CONNECTIONS') or self.max_connections
+            max_connections or
+            _get('redis_max_connections') or
+            self.max_connections
         )
         )
         self._ConnectionPool = connection_pool
         self._ConnectionPool = connection_pool
 
 
         self.connparams = {
         self.connparams = {
-            'host': _get('HOST') or 'localhost',
-            'port': _get('PORT') or 6379,
-            'db': _get('DB') or 0,
-            'password': _get('PASSWORD'),
-            'socket_timeout': _get('SOCKET_TIMEOUT'),
+            'host': _get('redis_host') or 'localhost',
+            'port': _get('redis_port') or 6379,
+            'db': _get('redis_db') or 0,
+            'password': _get('redis_password'),
+            'socket_timeout': _get('redis_socket_timeout'),
             'max_connections': self.max_connections,
             'max_connections': self.max_connections,
         }
         }
         if url:
         if url:

+ 1 - 1
celery/backends/riak.py

@@ -85,7 +85,7 @@ class RiakBackend(KeyValueStoreBackend):
             if ubucket:
             if ubucket:
                 ubucket = ubucket.strip('/')
                 ubucket = ubucket.strip('/')
 
 
-        config = self.app.conf.get('CELERY_RIAK_BACKEND_SETTINGS', None)
+        config = self.app.conf.get('riak_backend_settings', None)
         if config is not None:
         if config is not None:
             if not isinstance(config, dict):
             if not isinstance(config, dict):
                 raise ImproperlyConfigured(
                 raise ImproperlyConfigured(

+ 9 - 9
celery/beat.py

@@ -186,19 +186,19 @@ class Scheduler(object):
         self.app = app
         self.app = app
         self.data = maybe_evaluate({} if schedule is None else schedule)
         self.data = maybe_evaluate({} if schedule is None else schedule)
         self.max_interval = (max_interval or
         self.max_interval = (max_interval or
-                             app.conf.CELERYBEAT_MAX_LOOP_INTERVAL or
+                             app.conf.beat_max_loop_interval or
                              self.max_interval)
                              self.max_interval)
         self.Producer = Producer or app.amqp.Producer
         self.Producer = Producer or app.amqp.Producer
         self._heap = None
         self._heap = None
         self.sync_every_tasks = (
         self.sync_every_tasks = (
-            app.conf.CELERYBEAT_SYNC_EVERY if sync_every_tasks is None
+            app.conf.beat_sync_every if sync_every_tasks is None
             else sync_every_tasks)
             else sync_every_tasks)
         if not lazy:
         if not lazy:
             self.setup_schedule()
             self.setup_schedule()
 
 
     def install_default_entries(self, data):
     def install_default_entries(self, data):
         entries = {}
         entries = {}
-        if self.app.conf.CELERY_TASK_RESULT_EXPIRES and \
+        if self.app.conf.result_expires and \
                 not self.app.backend.supports_autoexpire:
                 not self.app.backend.supports_autoexpire:
             if 'celery.backend_cleanup' not in data:
             if 'celery.backend_cleanup' not in data:
                 entries['celery.backend_cleanup'] = {
                 entries['celery.backend_cleanup'] = {
@@ -363,7 +363,7 @@ class Scheduler(object):
                   'Trying again in %s seconds...', exc, interval)
                   'Trying again in %s seconds...', exc, interval)
 
 
         return self.connection.ensure_connection(
         return self.connection.ensure_connection(
-            _error_handler, self.app.conf.BROKER_CONNECTION_MAX_RETRIES
+            _error_handler, self.app.conf.broker_connection_max_retries
         )
         )
 
 
     def get_schedule(self):
     def get_schedule(self):
@@ -438,12 +438,12 @@ class PersistentScheduler(Scheduler):
                     self._store.clear()   # remove schedule at 3.0.9 upgrade
                     self._store.clear()   # remove schedule at 3.0.9 upgrade
             break
             break
 
 
-        tz = self.app.conf.CELERY_TIMEZONE
+        tz = self.app.conf.timezone
         stored_tz = self._store.get('tz')
         stored_tz = self._store.get('tz')
         if stored_tz is not None and stored_tz != tz:
         if stored_tz is not None and stored_tz != tz:
             warning('Reset: Timezone changed from %r to %r', stored_tz, tz)
             warning('Reset: Timezone changed from %r to %r', stored_tz, tz)
             self._store.clear()   # Timezone changed, reset db!
             self._store.clear()   # Timezone changed, reset db!
-        utc = self.app.conf.CELERY_ENABLE_UTC
+        utc = self.app.conf.enable_utc
         stored_utc = self._store.get('utc_enabled')
         stored_utc = self._store.get('utc_enabled')
         if stored_utc is not None and stored_utc != utc:
         if stored_utc is not None and stored_utc != utc:
             choices = {True: 'enabled', False: 'disabled'}
             choices = {True: 'enabled', False: 'disabled'}
@@ -451,7 +451,7 @@ class PersistentScheduler(Scheduler):
                     choices[stored_utc], choices[utc])
                     choices[stored_utc], choices[utc])
             self._store.clear()   # UTC setting changed, reset db!
             self._store.clear()   # UTC setting changed, reset db!
         entries = self._store.setdefault('entries', {})
         entries = self._store.setdefault('entries', {})
-        self.merge_inplace(self.app.conf.CELERYBEAT_SCHEDULE)
+        self.merge_inplace(self.app.conf.beat_schedule)
         self.install_default_entries(self.schedule)
         self.install_default_entries(self.schedule)
         self._store.update(__version__=__version__, tz=tz, utc_enabled=utc)
         self._store.update(__version__=__version__, tz=tz, utc_enabled=utc)
         self.sync()
         self.sync()
@@ -485,10 +485,10 @@ class Service(object):
                  scheduler_cls=None):
                  scheduler_cls=None):
         self.app = app
         self.app = app
         self.max_interval = (max_interval or
         self.max_interval = (max_interval or
-                             app.conf.CELERYBEAT_MAX_LOOP_INTERVAL)
+                             app.conf.beat_max_loop_interval)
         self.scheduler_cls = scheduler_cls or self.scheduler_cls
         self.scheduler_cls = scheduler_cls or self.scheduler_cls
         self.schedule_filename = (
         self.schedule_filename = (
-            schedule_filename or app.conf.CELERYBEAT_SCHEDULE_FILENAME)
+            schedule_filename or app.conf.beat_schedule_filename)
 
 
         self._is_shutdown = Event()
         self._is_shutdown = Event()
         self._is_stopped = Event()
         self._is_stopped = Event()

+ 1 - 1
celery/bin/base.py

@@ -219,7 +219,7 @@ class Command(object):
     enable_config_from_cmdline = False
     enable_config_from_cmdline = False
 
 
     #: Default configuration namespace.
     #: Default configuration namespace.
-    namespace = 'celery'
+    namespace = None
 
 
     #: Text to print at end of --help
     #: Text to print at end of --help
     epilog = None
     epilog = None

+ 1 - 1
celery/bin/beat.py

@@ -84,7 +84,7 @@ class beat(Command):
         return (
         return (
             (Option('--detach', action='store_true'),
             (Option('--detach', action='store_true'),
              Option('-s', '--schedule',
              Option('-s', '--schedule',
-                    default=c.CELERYBEAT_SCHEDULE_FILENAME),
+                    default=c.beat_schedule_filename),
              Option('--max-interval', type='float'),
              Option('--max-interval', type='float'),
              Option('-S', '--scheduler', dest='scheduler_cls'),
              Option('-S', '--scheduler', dest='scheduler_cls'),
              Option('-l', '--loglevel', default='WARN')) +
              Option('-l', '--loglevel', default='WARN')) +

+ 0 - 1
celery/bin/celery.py

@@ -660,7 +660,6 @@ class report(Command):
 
 
 
 
 class CeleryCommand(Command):
 class CeleryCommand(Command):
-    namespace = 'celery'
     ext_fmt = '{self.namespace}.commands'
     ext_fmt = '{self.namespace}.commands'
     commands = {
     commands = {
         'amqp': amqp,
         'amqp': amqp,

+ 1 - 1
celery/bin/graph.py

@@ -156,7 +156,7 @@ class graph(Command):
                 threads.append(reply['pool']['max-concurrency'])
                 threads.append(reply['pool']['max-concurrency'])
 
 
         wlen = len(workers)
         wlen = len(workers)
-        backend = args.get('backend', self.app.conf.CELERY_RESULT_BACKEND)
+        backend = args.get('backend', self.app.conf.result_backend)
         threads_for = {}
         threads_for = {}
         workers = maybe_abbr(workers, 'Worker')
         workers = maybe_abbr(workers, 'Worker')
         if Wmax and wlen > Wmax:
         if Wmax and wlen > Wmax:

+ 12 - 12
celery/bin/worker.py

@@ -173,7 +173,7 @@ class worker(Command):
         celery worker --autoscale=10,0
         celery worker --autoscale=10,0
     """
     """
     doc = __MODULE_DOC__  # parse help from this too
     doc = __MODULE_DOC__  # parse help from this too
-    namespace = 'celeryd'
+    namespace = 'worker'
     enable_config_from_cmdline = True
     enable_config_from_cmdline = True
     supports_args = False
     supports_args = False
 
 
@@ -200,7 +200,7 @@ class worker(Command):
         # Pools like eventlet/gevent needs to patch libs as early
         # Pools like eventlet/gevent needs to patch libs as early
         # as possible.
         # as possible.
         pool_cls = (concurrency.get_implementation(pool_cls) or
         pool_cls = (concurrency.get_implementation(pool_cls) or
-                    self.app.conf.CELERYD_POOL)
+                    self.app.conf.worker_pool)
         if self.app.IS_WINDOWS and kwargs.get('beat'):
         if self.app.IS_WINDOWS and kwargs.get('beat'):
             self.die('-B option does not work on Windows.  '
             self.die('-B option does not work on Windows.  '
                      'Please run celery beat as a separate service.')
                      'Please run celery beat as a separate service.')
@@ -231,29 +231,29 @@ class worker(Command):
         conf = self.app.conf
         conf = self.app.conf
         return (
         return (
             Option('-c', '--concurrency',
             Option('-c', '--concurrency',
-                   default=conf.CELERYD_CONCURRENCY, type='int'),
-            Option('-P', '--pool', default=conf.CELERYD_POOL, dest='pool_cls'),
+                   default=conf.worker_concurrency, type='int'),
+            Option('-P', '--pool', default=conf.worker_pool, dest='pool_cls'),
             Option('--purge', '--discard', default=False, action='store_true'),
             Option('--purge', '--discard', default=False, action='store_true'),
             Option('-l', '--loglevel', default='WARN'),
             Option('-l', '--loglevel', default='WARN'),
             Option('-n', '--hostname'),
             Option('-n', '--hostname'),
             Option('-B', '--beat', action='store_true'),
             Option('-B', '--beat', action='store_true'),
             Option('-s', '--schedule', dest='schedule_filename',
             Option('-s', '--schedule', dest='schedule_filename',
-                   default=conf.CELERYBEAT_SCHEDULE_FILENAME),
+                   default=conf.beat_schedule_filename),
             Option('--scheduler', dest='scheduler_cls'),
             Option('--scheduler', dest='scheduler_cls'),
             Option('-S', '--statedb',
             Option('-S', '--statedb',
-                   default=conf.CELERYD_STATE_DB, dest='state_db'),
-            Option('-E', '--events', default=conf.CELERY_SEND_EVENTS,
+                   default=conf.worker_state_db, dest='state_db'),
+            Option('-E', '--events', default=conf.worker_send_events,
                    action='store_true', dest='send_events'),
                    action='store_true', dest='send_events'),
             Option('--time-limit', type='float', dest='task_time_limit',
             Option('--time-limit', type='float', dest='task_time_limit',
-                   default=conf.CELERYD_TASK_TIME_LIMIT),
+                   default=conf.task_time_limit),
             Option('--soft-time-limit', dest='task_soft_time_limit',
             Option('--soft-time-limit', dest='task_soft_time_limit',
-                   default=conf.CELERYD_TASK_SOFT_TIME_LIMIT, type='float'),
+                   default=conf.task_soft_time_limit, type='float'),
             Option('--maxtasksperchild', dest='max_tasks_per_child',
             Option('--maxtasksperchild', dest='max_tasks_per_child',
-                   default=conf.CELERYD_MAX_TASKS_PER_CHILD, type='int'),
+                   default=conf.worker_max_tasks_per_child, type='int'),
             Option('--prefetch-multiplier', dest='prefetch_multiplier',
             Option('--prefetch-multiplier', dest='prefetch_multiplier',
-                   default=conf.CELERYD_PREFETCH_MULTIPLIER, type='int'),
+                   default=conf.worker_prefetch_multiplier, type='int'),
             Option('--maxmemperchild', dest='max_memory_per_child',
             Option('--maxmemperchild', dest='max_memory_per_child',
-                   default=conf.CELERYD_MAX_MEMORY_PER_CHILD, type='int'),
+                   default=conf.worker_max_memory_per_child, type='int'),
             Option('--queues', '-Q', default=[]),
             Option('--queues', '-Q', default=[]),
             Option('--exclude-queues', '-X', default=[]),
             Option('--exclude-queues', '-X', default=[]),
             Option('--include', '-I', default=[]),
             Option('--include', '-I', default=[]),

+ 4 - 4
celery/canvas.py

@@ -393,7 +393,7 @@ class chain(Signature):
     def apply_async(self, args=(), kwargs={}, **options):
     def apply_async(self, args=(), kwargs={}, **options):
         # python is best at unpacking kwargs, so .run is here to do that.
         # python is best at unpacking kwargs, so .run is here to do that.
         app = self.app
         app = self.app
-        if app.conf.CELERY_ALWAYS_EAGER:
+        if app.conf.task_always_eager:
             return self.apply(args, kwargs, **options)
             return self.apply(args, kwargs, **options)
         return self.run(args, kwargs, app=app, **(
         return self.run(args, kwargs, app=app, **(
             dict(self.options, **options) if options else self.options))
             dict(self.options, **options) if options else self.options))
@@ -688,7 +688,7 @@ class group(Signature):
     def apply_async(self, args=(), kwargs=None, add_to_parent=True,
     def apply_async(self, args=(), kwargs=None, add_to_parent=True,
                     producer=None, **options):
                     producer=None, **options):
         app = self.app
         app = self.app
-        if app.conf.CELERY_ALWAYS_EAGER:
+        if app.conf.task_always_eager:
             return self.apply(args, kwargs, **options)
             return self.apply(args, kwargs, **options)
         if not self.tasks:
         if not self.tasks:
             return self.freeze()
             return self.freeze()
@@ -846,7 +846,7 @@ class chord(Signature):
         app = self._get_app(body)
         app = self._get_app(body)
         tasks = (self.tasks.clone() if isinstance(self.tasks, group)
         tasks = (self.tasks.clone() if isinstance(self.tasks, group)
                  else group(self.tasks))
                  else group(self.tasks))
-        if app.conf.CELERY_ALWAYS_EAGER:
+        if app.conf.task_always_eager:
             return self.apply((), kwargs,
             return self.apply((), kwargs,
                               body=body, task_id=task_id, **options)
                               body=body, task_id=task_id, **options)
         return self.run(tasks, body, args, task_id=task_id, **options)
         return self.run(tasks, body, args, task_id=task_id, **options)
@@ -875,7 +875,7 @@ class chord(Signature):
             countdown=1, max_retries=None, propagate=None, eager=False,
             countdown=1, max_retries=None, propagate=None, eager=False,
             task_id=None, **options):
             task_id=None, **options):
         app = app or self._get_app(body)
         app = app or self._get_app(body)
-        propagate = (app.conf.CELERY_CHORD_PROPAGATES
+        propagate = (app.conf.chord_propagates
                      if propagate is None else propagate)
                      if propagate is None else propagate)
         group_id = uuid()
         group_id = uuid()
         root_id = body.options.get('root_id')
         root_id = body.options.get('root_id')

+ 1 - 1
celery/contrib/batches.py

@@ -8,7 +8,7 @@ Experimental task class that buffers messages and processes them as a list.
 .. warning::
 .. warning::
 
 
     For this to work you have to set
     For this to work you have to set
-    :setting:`CELERYD_PREFETCH_MULTIPLIER` to zero, or some value where
+    :setting:`worker_prefetch_multiplier` to zero, or some value where
     the final multiplied value is higher than ``flush_every``.
     the final multiplied value is higher than ``flush_every``.
 
 
     In the future we hope to add the ability to direct batching tasks
     In the future we hope to add the ability to direct batching tasks

+ 1 - 1
celery/contrib/migrate.py

@@ -141,7 +141,7 @@ def move(predicate, connection=None, exchange=None, routing_key=None,
 
 
     :keyword connection: Custom connection to use.
     :keyword connection: Custom connection to use.
     :keyword source: Optional list of source queues to use instead of the
     :keyword source: Optional list of source queues to use instead of the
-        default (which is the queues in :setting:`CELERY_QUEUES`).
+        default (which is the queues in :setting:`task_queues`).
         This list can also contain new :class:`~kombu.entity.Queue` instances.
         This list can also contain new :class:`~kombu.entity.Queue` instances.
     :keyword exchange: Default destination exchange.
     :keyword exchange: Default destination exchange.
     :keyword routing_key: Default destination routing key.
     :keyword routing_key: Default destination routing key.

+ 37 - 15
celery/datastructures.py

@@ -387,11 +387,8 @@ class DictAttribute(object):
             return default
             return default
 
 
     def setdefault(self, key, default):
     def setdefault(self, key, default):
-        try:
-            return self[key]
-        except KeyError:
+        if key not in self:
             self[key] = default
             self[key] = default
-            return default
 
 
     def __getitem__(self, key):
     def __getitem__(self, key):
         try:
         try:
@@ -451,13 +448,27 @@ class ConfigurationView(AttributeDictMixin):
     :param defaults: List of dicts containing the default configuration.
     :param defaults: List of dicts containing the default configuration.
 
 
     """
     """
+    key_t = None
     changes = None
     changes = None
     defaults = None
     defaults = None
     _order = None
     _order = None
 
 
-    def __init__(self, changes, defaults):
-        self.__dict__.update(changes=changes, defaults=defaults,
-                             _order=[changes] + defaults)
+    def __init__(self, changes, defaults=None, key_t=None, prefix=None):
+        defaults = [] if defaults is None else defaults
+        self.__dict__.update(
+            changes=changes,
+            defaults=defaults,
+            key_t=key_t,
+            _order=[changes] + defaults,
+            prefix=prefix,
+        )
+
+    def _key(self, key):
+        if self.prefix:
+            key = self.prefix + key
+            if self.prefix.isupper():
+                key = key.upper()
+        return self.key_t(key) if self.key_t is not None else key
 
 
     def add_defaults(self, d):
     def add_defaults(self, d):
         d = force_mapping(d)
         d = force_mapping(d)
@@ -465,6 +476,7 @@ class ConfigurationView(AttributeDictMixin):
         self._order.insert(1, d)
         self._order.insert(1, d)
 
 
     def __getitem__(self, key):
     def __getitem__(self, key):
+        key = self._key(key)
         for d in self._order:
         for d in self._order:
             try:
             try:
                 return d[key]
                 return d[key]
@@ -473,14 +485,14 @@ class ConfigurationView(AttributeDictMixin):
         raise KeyError(key)
         raise KeyError(key)
 
 
     def __setitem__(self, key, value):
     def __setitem__(self, key, value):
-        self.changes[key] = value
+        self.changes[self._key(key)] = value
 
 
     def first(self, *keys):
     def first(self, *keys):
-        return first(None, (self.get(key) for key in keys))
+        return first(None, (self.get(self._key(key)) for key in keys))
 
 
     def get(self, key, default=None):
     def get(self, key, default=None):
         try:
         try:
-            return self[key]
+            return self[self._key(key)]
         except KeyError:
         except KeyError:
             return default
             return default
 
 
@@ -489,16 +501,15 @@ class ConfigurationView(AttributeDictMixin):
         self.changes.clear()
         self.changes.clear()
 
 
     def setdefault(self, key, default):
     def setdefault(self, key, default):
-        try:
-            return self[key]
-        except KeyError:
+        key = self._key(key)
+        if key not in self:
             self[key] = default
             self[key] = default
-            return default
 
 
     def update(self, *args, **kwargs):
     def update(self, *args, **kwargs):
         return self.changes.update(*args, **kwargs)
         return self.changes.update(*args, **kwargs)
 
 
     def __contains__(self, key):
     def __contains__(self, key):
+        key = self._key(key)
         return any(key in m for m in self._order)
         return any(key in m for m in self._order)
 
 
     def __bool__(self):
     def __bool__(self):
@@ -521,8 +532,19 @@ class ConfigurationView(AttributeDictMixin):
         # changes takes precedence.
         # changes takes precedence.
         return chain(*[op(d) for d in reversed(self._order)])
         return chain(*[op(d) for d in reversed(self._order)])
 
 
+    def swap_with(self, other):
+        changes = other.__dict__['changes']
+        defaults = other.__dict__['defaults']
+        self.__dict__.update(
+            changes=changes,
+            defaults=defaults,
+            key_t=other.__dict__['key_t'],
+            prefix=other.__dict__['prefix'],
+            _order=[changes] + defaults
+        )
+
     def _iterate_keys(self):
     def _iterate_keys(self):
-        return uniq(self._iter(lambda d: d))
+        return uniq(self._iter(lambda d: d.keys()))
     iterkeys = _iterate_keys
     iterkeys = _iterate_keys
 
 
     def _iterate_items(self):
     def _iterate_items(self):

+ 5 - 5
celery/events/__init__.py

@@ -4,7 +4,7 @@
     ~~~~~~~~~~~~~
     ~~~~~~~~~~~~~
 
 
     Events is a stream of messages sent for certain actions occurring
     Events is a stream of messages sent for certain actions occurring
-    in the worker (and clients if :setting:`CELERY_SEND_TASK_SENT_EVENT`
+    in the worker (and clients if :setting:`task_send_sent_event`
     is enabled), used for monitoring purposes.
     is enabled), used for monitoring purposes.
 
 
 """
 """
@@ -130,7 +130,7 @@ class EventDispatcher(object):
         self.mutex = threading.Lock()
         self.mutex = threading.Lock()
         self.producer = None
         self.producer = None
         self._outbound_buffer = deque()
         self._outbound_buffer = deque()
-        self.serializer = serializer or self.app.conf.CELERY_EVENT_SERIALIZER
+        self.serializer = serializer or self.app.conf.event_serializer
         self.on_enabled = set()
         self.on_enabled = set()
         self.on_disabled = set()
         self.on_disabled = set()
         self.groups = set(groups or [])
         self.groups = set(groups or [])
@@ -321,18 +321,18 @@ class EventReceiver(ConsumerMixin):
         self.adjust_clock = self.clock.adjust
         self.adjust_clock = self.clock.adjust
         self.forward_clock = self.clock.forward
         self.forward_clock = self.clock.forward
         if accept is None:
         if accept is None:
-            accept = {self.app.conf.CELERY_EVENT_SERIALIZER, 'json'}
+            accept = {self.app.conf.event_serializer, 'json'}
         self.accept = accept
         self.accept = accept
 
 
     def _get_queue_arguments(self, ttl=None, expires=None):
     def _get_queue_arguments(self, ttl=None, expires=None):
         conf = self.app.conf
         conf = self.app.conf
         return dictfilter({
         return dictfilter({
             'x-message-ttl': maybe_s_to_ms(
             'x-message-ttl': maybe_s_to_ms(
-                ttl if ttl is not None else conf.CELERY_EVENT_QUEUE_TTL,
+                ttl if ttl is not None else conf.event_queue_ttl,
             ),
             ),
             'x-expires': maybe_s_to_ms(
             'x-expires': maybe_s_to_ms(
                 expires if expires is not None
                 expires if expires is not None
-                else conf.CELERY_EVENT_QUEUE_EXPIRES,
+                else conf.event_queue_expires,
             ),
             ),
         })
         })
 
 

+ 1 - 1
celery/events/cursesmon.py

@@ -511,7 +511,7 @@ def capture_events(app, state, display):  # pragma: no cover
         with app.connection() as conn:
         with app.connection() as conn:
             try:
             try:
                 conn.ensure_connection(on_connection_error,
                 conn.ensure_connection(on_connection_error,
-                                       app.conf.BROKER_CONNECTION_MAX_RETRIES)
+                                       app.conf.broker_connection_max_retries)
                 recv = app.events.Receiver(conn, handlers={'*': state.event})
                 recv = app.events.Receiver(conn, handlers={'*': state.event})
                 display.resetscreen()
                 display.resetscreen()
                 display.init_screen()
                 display.init_screen()

+ 2 - 2
celery/exceptions.py

@@ -120,7 +120,7 @@ class WorkerShutdown(SystemExit):
 
 
 
 
 class QueueNotFound(KeyError):
 class QueueNotFound(KeyError):
-    """Task routed to a queue not in CELERY_QUEUES."""
+    """Task routed to a queue not in ``conf.queues``."""
 
 
 
 
 class ImproperlyConfigured(ImportError):
 class ImproperlyConfigured(ImportError):
@@ -155,7 +155,7 @@ class NotConfigured(CeleryWarning):
 
 
 
 
 class AlwaysEagerIgnored(CeleryWarning):
 class AlwaysEagerIgnored(CeleryWarning):
-    """send_task ignores CELERY_ALWAYS_EAGER option"""
+    """send_task ignores :setting:`task_always_eager` option"""
 
 
 
 
 class InvalidTaskError(CeleryError):
 class InvalidTaskError(CeleryError):

+ 9 - 8
celery/loaders/base.py

@@ -40,6 +40,8 @@ CONFIG_WITH_SUFFIX = CONFIG_INVALID_NAME + """\
 Did you mean '{suggest}'?
 Did you mean '{suggest}'?
 """
 """
 
 
+unconfigured = object()
+
 
 
 class BaseLoader(object):
 class BaseLoader(object):
     """The base class for loaders.
     """The base class for loaders.
@@ -65,7 +67,7 @@ class BaseLoader(object):
     override_backends = {}
     override_backends = {}
     worker_initialized = False
     worker_initialized = False
 
 
-    _conf = None
+    _conf = unconfigured
 
 
     def __init__(self, app, **kwargs):
     def __init__(self, app, **kwargs):
         self.app = app
         self.app = app
@@ -117,8 +119,8 @@ class BaseLoader(object):
         return [
         return [
             self.import_task_module(m) for m in (
             self.import_task_module(m) for m in (
                 tuple(self.builtin_modules) +
                 tuple(self.builtin_modules) +
-                tuple(maybe_list(self.app.conf.CELERY_IMPORTS)) +
-                tuple(maybe_list(self.app.conf.CELERY_INCLUDE))
+                tuple(maybe_list(self.app.conf.imports)) +
+                tuple(maybe_list(self.app.conf.include))
             )
             )
         ]
         ]
 
 
@@ -183,7 +185,7 @@ class BaseLoader(object):
                             'list': 'json',
                             'list': 'json',
                             'dict': 'json'}):
                             'dict': 'json'}):
         from celery.app.defaults import Option, NAMESPACES
         from celery.app.defaults import Option, NAMESPACES
-        namespace = namespace.upper()
+        namespace = namespace.lower()
         typemap = dict(Option.typemap, **extra_types)
         typemap = dict(Option.typemap, **extra_types)
 
 
         def getarg(arg):
         def getarg(arg):
@@ -193,7 +195,7 @@ class BaseLoader(object):
             # ## find key/value
             # ## find key/value
             # ns.key=value|ns_key=value (case insensitive)
             # ns.key=value|ns_key=value (case insensitive)
             key, value = arg.split('=', 1)
             key, value = arg.split('=', 1)
-            key = key.upper().replace('.', '_')
+            key = key.lower().replace('.', '_')
 
 
             # ## find namespace.
             # ## find namespace.
             # .key=value|_key=value expands to default namespace.
             # .key=value|_key=value expands to default namespace.
@@ -214,7 +216,7 @@ class BaseLoader(object):
                 value = typemap[type_](value)
                 value = typemap[type_](value)
             else:
             else:
                 try:
                 try:
-                    value = NAMESPACES[ns][key].to_python(value)
+                    value = NAMESPACES[ns.lower()][key].to_python(value)
                 except ValueError as exc:
                 except ValueError as exc:
                     # display key name in error message.
                     # display key name in error message.
                     raise ValueError('{0!r}: {1}'.format(ns_key, exc))
                     raise ValueError('{0!r}: {1}'.format(ns_key, exc))
@@ -244,7 +246,6 @@ class BaseLoader(object):
             if custom_config:
             if custom_config:
                 usercfg = self._import_config_module(custom_config)
                 usercfg = self._import_config_module(custom_config)
                 return DictAttribute(usercfg)
                 return DictAttribute(usercfg)
-        return {}
 
 
     def autodiscover_tasks(self, packages, related_name='tasks'):
     def autodiscover_tasks(self, packages, related_name='tasks'):
         self.task_modules.update(
         self.task_modules.update(
@@ -254,7 +255,7 @@ class BaseLoader(object):
     @property
     @property
     def conf(self):
     def conf(self):
         """Loader configuration."""
         """Loader configuration."""
-        if self._conf is None:
+        if self._conf is unconfigured:
             self._conf = self.read_configuration()
             self._conf = self.read_configuration()
         return self._conf
         return self._conf
 
 

+ 2 - 2
celery/schedules.py

@@ -111,7 +111,7 @@ class schedule(object):
         The next time to check is used to save energy/cpu cycles,
         The next time to check is used to save energy/cpu cycles,
         it does not need to be accurate but will influence the precision
         it does not need to be accurate but will influence the precision
         of your schedule.  You must also keep in mind
         of your schedule.  You must also keep in mind
-        the value of :setting:`CELERYBEAT_MAX_LOOP_INTERVAL`,
+        the value of :setting:`beat_max_loop_interval`,
         which decides the maximum number of seconds the scheduler can
         which decides the maximum number of seconds the scheduler can
         sleep between re-checking the periodic task intervals.  So if you
         sleep between re-checking the periodic task intervals.  So if you
         have a task that changes schedule at runtime then your next_run_at
         have a task that changes schedule at runtime then your next_run_at
@@ -172,7 +172,7 @@ class schedule(object):
 
 
     @cached_property
     @cached_property
     def utc_enabled(self):
     def utc_enabled(self):
-        return self.app.conf.CELERY_ENABLE_UTC
+        return self.app.conf.enable_utc
 
 
     def to_local(self, dt):
     def to_local(self, dt):
         if not self.utc_enabled:
         if not self.utc_enabled:

+ 7 - 7
celery/security/__init__.py

@@ -25,9 +25,9 @@ Please install by:
 
 
 SETTING_MISSING = """\
 SETTING_MISSING = """\
 Sorry, but you have to configure the
 Sorry, but you have to configure the
-    * CELERY_SECURITY_KEY
-    * CELERY_SECURITY_CERTIFICATE, and the
-    * CELERY_SECURITY_CERT_STORE
+    * security_key
+    * security_certificate, and the
+    * security_cert_storE
 configuration settings to use the auth serializer.
 configuration settings to use the auth serializer.
 
 
 Please see the configuration reference for more information.
 Please see the configuration reference for more information.
@@ -46,7 +46,7 @@ def setup_security(allowed_serializers=None, key=None, cert=None, store=None,
     _disable_insecure_serializers(allowed_serializers)
     _disable_insecure_serializers(allowed_serializers)
 
 
     conf = app.conf
     conf = app.conf
-    if conf.CELERY_TASK_SERIALIZER != 'auth':
+    if conf.task_serializer != 'auth':
         return
         return
 
 
     try:
     try:
@@ -54,9 +54,9 @@ def setup_security(allowed_serializers=None, key=None, cert=None, store=None,
     except ImportError:
     except ImportError:
         raise ImproperlyConfigured(SSL_NOT_INSTALLED)
         raise ImproperlyConfigured(SSL_NOT_INSTALLED)
 
 
-    key = key or conf.CELERY_SECURITY_KEY
-    cert = cert or conf.CELERY_SECURITY_CERTIFICATE
-    store = store or conf.CELERY_SECURITY_CERT_STORE
+    key = key or conf.security_key
+    cert = cert or conf.security_certificate
+    store = store or conf.security_cert_store
 
 
     if not (key and cert and store):
     if not (key and cert and store):
         raise ImproperlyConfigured(SETTING_MISSING)
         raise ImproperlyConfigured(SETTING_MISSING)

+ 1 - 1
celery/states.py

@@ -128,7 +128,7 @@ class state(str):
 PENDING = 'PENDING'
 PENDING = 'PENDING'
 #: Task was received by a worker.
 #: Task was received by a worker.
 RECEIVED = 'RECEIVED'
 RECEIVED = 'RECEIVED'
-#: Task was started by a worker (:setting:`CELERY_TRACK_STARTED`).
+#: Task was started by a worker (:setting:`task_track_started`).
 STARTED = 'STARTED'
 STARTED = 'STARTED'
 #: Task succeeded
 #: Task succeeded
 SUCCESS = 'SUCCESS'
 SUCCESS = 'SUCCESS'

+ 5 - 5
celery/task/base.py

@@ -148,8 +148,8 @@ class Task(BaseTask):
     disable_error_emails = False
     disable_error_emails = False
 
 
     from_config = BaseTask.from_config + (
     from_config = BaseTask.from_config + (
-        ('exchange_type', 'CELERY_DEFAULT_EXCHANGE_TYPE'),
-        ('delivery_mode', 'CELERY_DEFAULT_DELIVERY_MODE'),
+        ('exchange_type', 'task_default_exchange_type'),
+        ('delivery_mode', 'task_default_delivery_mode'),
     )
     )
 
 
     # In old Celery the @task decorator didn't exist, so one would create
     # In old Celery the @task decorator didn't exist, so one would create
@@ -244,7 +244,7 @@ class Task(BaseTask):
 
 
 class PeriodicTask(Task):
 class PeriodicTask(Task):
     """A periodic task is a task that adds itself to the
     """A periodic task is a task that adds itself to the
-    :setting:`CELERYBEAT_SCHEDULE` setting."""
+    :setting:`beat_schedule` setting."""
     abstract = True
     abstract = True
     ignore_result = True
     ignore_result = True
     relative = False
     relative = False
@@ -260,7 +260,7 @@ class PeriodicTask(Task):
 
 
     @classmethod
     @classmethod
     def on_bound(cls, app):
     def on_bound(cls, app):
-        app.conf.CELERYBEAT_SCHEDULE[cls.name] = {
+        app.conf.beat_schedule[cls.name] = {
             'task': cls.name,
             'task': cls.name,
             'schedule': cls.run_every,
             'schedule': cls.run_every,
             'args': (),
             'args': (),
@@ -276,5 +276,5 @@ def task(*args, **kwargs):
 
 
 
 
 def periodic_task(*args, **options):
 def periodic_task(*args, **options):
-    """Deprecated decorator, please use :setting:`CELERYBEAT_SCHEDULE`."""
+    """Deprecated decorator, please use :setting:`beat_schedule`."""
     return task(**dict({'base': PeriodicTask}, **options))
     return task(**dict({'base': PeriodicTask}, **options))

+ 1 - 1
celery/task/sets.py

@@ -53,7 +53,7 @@ class TaskSet(list):
         """Apply TaskSet."""
         """Apply TaskSet."""
         app = self.app
         app = self.app
 
 
-        if app.conf.CELERY_ALWAYS_EAGER:
+        if app.conf.task_always_eager:
             return self.apply(taskset_id=taskset_id)
             return self.apply(taskset_id=taskset_id)
 
 
         with app.connection_or_acquire(connection) as conn:
         with app.connection_or_acquire(connection) as conn:

+ 3 - 3
celery/tests/app/test_amqp.py

@@ -11,7 +11,7 @@ class test_TaskConsumer(AppCase):
 
 
     def test_accept_content(self):
     def test_accept_content(self):
         with self.app.pool.acquire(block=True) as conn:
         with self.app.pool.acquire(block=True) as conn:
-            self.app.conf.CELERY_ACCEPT_CONTENT = ['application/json']
+            self.app.conf.accept_content = ['application/json']
             self.assertEqual(
             self.assertEqual(
                 self.app.amqp.TaskConsumer(conn).accept,
                 self.app.amqp.TaskConsumer(conn).accept,
                 {'application/json'},
                 {'application/json'},
@@ -25,7 +25,7 @@ class test_TaskConsumer(AppCase):
 class test_ProducerPool(AppCase):
 class test_ProducerPool(AppCase):
 
 
     def test_setup_nolimit(self):
     def test_setup_nolimit(self):
-        self.app.conf.BROKER_POOL_LIMIT = None
+        self.app.conf.broker_pool_limit = None
         try:
         try:
             delattr(self.app, '_pool')
             delattr(self.app, '_pool')
         except AttributeError:
         except AttributeError:
@@ -43,7 +43,7 @@ class test_ProducerPool(AppCase):
         r2 = pool.acquire()
         r2 = pool.acquire()
 
 
     def test_setup(self):
     def test_setup(self):
-        self.app.conf.BROKER_POOL_LIMIT = 2
+        self.app.conf.broker_pool_limit = 2
         try:
         try:
             delattr(self.app, '_pool')
             delattr(self.app, '_pool')
         except AttributeError:
         except AttributeError:

+ 168 - 42
celery/tests/app/test_app.py

@@ -15,8 +15,8 @@ from celery import _state
 from celery.app import base as _appbase
 from celery.app import base as _appbase
 from celery.app import defaults
 from celery.app import defaults
 from celery.exceptions import ImproperlyConfigured
 from celery.exceptions import ImproperlyConfigured
-from celery.five import items
-from celery.loaders.base import BaseLoader
+from celery.five import items, keys
+from celery.loaders.base import BaseLoader, unconfigured
 from celery.platforms import pyimplementation
 from celery.platforms import pyimplementation
 from celery.utils.serialization import pickle
 from celery.utils.serialization import pickle
 
 
@@ -225,7 +225,73 @@ class test_App(AppCase):
     @with_environ('CELERY_BROKER_URL', '')
     @with_environ('CELERY_BROKER_URL', '')
     def test_with_broker(self):
     def test_with_broker(self):
         with self.Celery(broker='foo://baribaz') as app:
         with self.Celery(broker='foo://baribaz') as app:
-            self.assertEqual(app.conf.BROKER_URL, 'foo://baribaz')
+            self.assertEqual(app.conf.broker_url, 'foo://baribaz')
+
+    def test_pending_configuration__setattr(self):
+        with self.Celery(broker='foo://bar') as app:
+            app.conf.task_default_delivery_mode = 44
+            app.conf.worker_agent = 'foo:Bar'
+            self.assertFalse(app.configured)
+            self.assertEqual(app.conf.worker_agent, 'foo:Bar')
+            self.assertEqual(app.conf.broker_url, 'foo://bar')
+            self.assertEqual(app._preconf['worker_agent'], 'foo:Bar')
+
+            self.assertTrue(app.configured)
+            reapp = pickle.loads(pickle.dumps(app))
+            self.assertEqual(reapp._preconf['worker_agent'], 'foo:Bar')
+            self.assertFalse(reapp.configured)
+            self.assertEqual(reapp.conf.worker_agent, 'foo:Bar')
+            self.assertTrue(reapp.configured)
+            self.assertEqual(reapp.conf.broker_url, 'foo://bar')
+            self.assertEqual(reapp._preconf['worker_agent'], 'foo:Bar')
+
+    def test_pending_configuration__update(self):
+        with self.Celery(broker='foo://bar') as app:
+            app.conf.update(
+                task_default_delivery_mode=44,
+                worker_agent='foo:Bar',
+            )
+            self.assertFalse(app.configured)
+            self.assertEqual(app.conf.worker_agent, 'foo:Bar')
+            self.assertEqual(app.conf.broker_url, 'foo://bar')
+            self.assertEqual(app._preconf['worker_agent'], 'foo:Bar')
+
+    def test_pending_configuration__compat_settings(self):
+        with self.Celery(broker='foo://bar', backend='foo') as app:
+            app.conf.update(
+                CELERY_ALWAYS_EAGER=4,
+                CELERY_DEFAULT_DELIVERY_MODE=63,
+                CELERYD_AGENT='foo:Barz',
+            )
+            self.assertEqual(app.conf.task_always_eager, 4)
+            self.assertEqual(app.conf.task_default_delivery_mode, 63)
+            self.assertEqual(app.conf.worker_agent, 'foo:Barz')
+            self.assertEqual(app.conf.broker_url, 'foo://bar')
+            self.assertEqual(app.conf.result_backend, 'foo')
+
+    def test_pending_configuration__setdefault(self):
+        with self.Celery(broker='foo://bar') as app:
+            app.conf.setdefault('worker_agent', 'foo:Bar')
+            self.assertFalse(app.configured)
+
+    def test_pending_configuration__iter(self):
+        with self.Celery(broker='foo://bar') as app:
+            app.conf.worker_agent = 'foo:Bar'
+            self.assertFalse(app.configured)
+            self.assertTrue(list(keys(app.conf)))
+            self.assertFalse(app.configured)
+            self.assertIn('worker_agent', app.conf)
+            self.assertFalse(app.configured)
+            self.assertTrue(dict(app.conf))
+            self.assertTrue(app.configured)
+
+    def test_pending_configuration__raises_ImproperlyConfigured(self):
+        with self.Celery() as app:
+            app.conf.worker_agent = 'foo://bar'
+            app.conf.task_default_delivery_mode = 44
+            app.conf.CELERY_ALWAYS_EAGER = True
+            with self.assertRaises(ImproperlyConfigured):
+                app.finalize()
 
 
     def test_repr(self):
     def test_repr(self):
         self.assertTrue(repr(self.app))
         self.assertTrue(repr(self.app))
@@ -236,7 +302,7 @@ class test_App(AppCase):
 
 
     def test_include_argument(self):
     def test_include_argument(self):
         with self.Celery(include=('foo', 'bar.foo')) as app:
         with self.Celery(include=('foo', 'bar.foo')) as app:
-            self.assertEqual(app.conf.CELERY_IMPORTS, ('foo', 'bar.foo'))
+            self.assertEqual(app.conf.include, ('foo', 'bar.foo'))
 
 
     def test_set_as_current(self):
     def test_set_as_current(self):
         current = _state._tls.current_app
         current = _state._tls.current_app
@@ -324,7 +390,7 @@ class test_App(AppCase):
                 return fun(*args, **kwargs)
                 return fun(*args, **kwargs)
             return _inner
             return _inner
 
 
-        self.app.conf.CELERY_ANNOTATIONS = {
+        self.app.conf.task_annotations = {
             adX.name: {'@__call__': deco}
             adX.name: {'@__call__': deco}
         }
         }
         adX.bind(self.app)
         adX.bind(self.app)
@@ -416,7 +482,7 @@ class test_App(AppCase):
     def test_config_from_object__lazy(self):
     def test_config_from_object__lazy(self):
         conf = ObjectConfig2()
         conf = ObjectConfig2()
         self.app.config_from_object(conf)
         self.app.config_from_object(conf)
-        self.assertFalse(self.app.loader._conf)
+        self.assertIs(self.app.loader._conf, unconfigured)
         self.assertIs(self.app._config_source, conf)
         self.assertIs(self.app._config_source, conf)
 
 
         self.assert_config2()
         self.assert_config2()
@@ -427,46 +493,110 @@ class test_App(AppCase):
 
 
         self.assert_config2()
         self.assert_config2()
 
 
+    def test_config_from_object__compat(self):
+
+        class Config(object):
+            CELERY_ALWAYS_EAGER = 44
+            CELERY_DEFAULT_DELIVERY_MODE = 30
+            CELERY_TASK_PUBLISH_RETRY = False
+
+        self.app.config_from_object(Config)
+        self.assertEqual(self.app.conf.task_always_eager, 44)
+        self.assertEqual(self.app.conf.CELERY_ALWAYS_EAGER, 44)
+        self.assertFalse(self.app.conf.task_publish_retry)
+        self.assertEqual(self.app.conf.task_default_routing_key, 'celery')
+
+    def test_config_from_object__supports_old_names(self):
+
+        class Config(object):
+            task_always_eager = 44
+            task_default_delivery_mode = 301
+
+        self.app.config_from_object(Config())
+        self.assertEqual(self.app.conf.CELERY_ALWAYS_EAGER, 44)
+        self.assertEqual(self.app.conf.task_always_eager, 44)
+        self.assertEqual(self.app.conf.CELERY_DEFAULT_DELIVERY_MODE, 301)
+        self.assertEqual(self.app.conf.task_default_delivery_mode, 301)
+        self.assertEqual(self.app.conf.task_default_routing_key, 'testcelery')
+
+    def test_config_from_object__namespace_uppercase(self):
+
+        class Config(object):
+            CELERY_TASK_ALWAYS_EAGER = 44
+            CELERY_TASK_DEFAULT_DELIVERY_MODE = 301
+
+        self.app.config_from_object(Config(), namespace='CELERY_')
+        self.assertEqual(self.app.conf.task_always_eager, 44)
+
+    def test_config_from_object__namespace_lowercase(self):
+
+        class Config(object):
+            celery_task_always_eager = 44
+            celery_task_default_delivery_mode = 301
+
+        self.app.config_from_object(Config(), namespace='celery_')
+        self.assertEqual(self.app.conf.task_always_eager, 44)
+
+    def test_config_from_object__mixing_new_and_old(self):
+
+        class Config(object):
+            task_always_eager = 44
+            worker_agent = 'foo:Agent'
+            worker_consumer = 'foo:Consumer'
+            beat_schedule = '/foo/schedule'
+            CELERY_DEFAULT_DELIVERY_MODE = 301
+
+        with self.assertRaises(ImproperlyConfigured) as exc:
+            self.app.config_from_object(Config(), force=True)
+            self.assertTrue(
+                exc.args[0].startswith('CELERY_DEFAULT_DELIVERY_MODE'))
+            self.assertIn('task_default_delivery_mode', exc.args[0])
+
+    def test_config_from_object__mixing_old_and_new(self):
+
+        class Config(object):
+            CELERY_ALWAYS_EAGER = 44
+            CELERYD_AGENT = 'foo:Agent'
+            CELERYD_CONSUMER = 'foo:Consumer'
+            CELERYBEAT_SCHEDULE = '/foo/schedule'
+            task_default_delivery_mode = 301
+
+        with self.assertRaises(ImproperlyConfigured) as exc:
+            self.app.config_from_object(Config(), force=True)
+            self.assertTrue(
+                exc.args[0].startswith('task_default_delivery_mode'))
+            self.assertIn('CELERY_DEFAULT_DELIVERY_MODE', exc.args[0])
+
     def test_config_from_cmdline(self):
     def test_config_from_cmdline(self):
-        cmdline = ['.always_eager=no',
-                   '.result_backend=/dev/null',
-                   'celeryd.prefetch_multiplier=368',
+        cmdline = ['task_always_eager=no',
+                   'result_backend=/dev/null',
+                   'worker_prefetch_multiplier=368',
                    '.foobarstring=(string)300',
                    '.foobarstring=(string)300',
                    '.foobarint=(int)300',
                    '.foobarint=(int)300',
-                   '.result_engine_options=(dict){"foo": "bar"}']
-        self.app.config_from_cmdline(cmdline, namespace='celery')
-        self.assertFalse(self.app.conf.CELERY_ALWAYS_EAGER)
-        self.assertEqual(self.app.conf.CELERY_RESULT_BACKEND, '/dev/null')
-        self.assertEqual(self.app.conf.CELERYD_PREFETCH_MULTIPLIER, 368)
-        self.assertEqual(self.app.conf.CELERY_FOOBARSTRING, '300')
-        self.assertEqual(self.app.conf.CELERY_FOOBARINT, 300)
-        self.assertDictEqual(self.app.conf.CELERY_RESULT_ENGINE_OPTIONS,
+                   'sqlalchemy_engine_options=(dict){"foo": "bar"}']
+        self.app.config_from_cmdline(cmdline, namespace='worker')
+        self.assertFalse(self.app.conf.task_always_eager)
+        self.assertEqual(self.app.conf.result_backend, '/dev/null')
+        self.assertEqual(self.app.conf.worker_prefetch_multiplier, 368)
+        self.assertEqual(self.app.conf.worker_foobarstring, '300')
+        self.assertEqual(self.app.conf.worker_foobarint, 300)
+        self.assertDictEqual(self.app.conf.sqlalchemy_engine_options,
                              {'foo': 'bar'})
                              {'foo': 'bar'})
 
 
-    def test_compat_setting_CELERY_BACKEND(self):
-        self.app._preconf = {}
-        self.app.conf.defaults[0]['CELERY_RESULT_BACKEND'] = None
-        self.app.config_from_object(Object(CELERY_BACKEND='set_by_us'))
-        self.assertEqual(self.app.conf.CELERY_RESULT_BACKEND, 'set_by_us')
-
-    def test_setting_BROKER_TRANSPORT_OPTIONS(self):
+    def test_setting__broker_transport_options(self):
 
 
         _args = {'foo': 'bar', 'spam': 'baz'}
         _args = {'foo': 'bar', 'spam': 'baz'}
 
 
         self.app.config_from_object(Object())
         self.app.config_from_object(Object())
-        self.assertEqual(self.app.conf.BROKER_TRANSPORT_OPTIONS, {})
+        self.assertEqual(self.app.conf.broker_transport_options, {})
 
 
-        self.app.config_from_object(Object(BROKER_TRANSPORT_OPTIONS=_args))
-        self.assertEqual(self.app.conf.BROKER_TRANSPORT_OPTIONS, _args)
+        self.app.config_from_object(Object(broker_transport_options=_args))
+        self.assertEqual(self.app.conf.broker_transport_options, _args)
 
 
     def test_Windows_log_color_disabled(self):
     def test_Windows_log_color_disabled(self):
         self.app.IS_WINDOWS = True
         self.app.IS_WINDOWS = True
         self.assertFalse(self.app.log.supports_color(True))
         self.assertFalse(self.app.log.supports_color(True))
 
 
-    def test_compat_setting_CARROT_BACKEND(self):
-        self.app.config_from_object(Object(CARROT_BACKEND='set_by_us'))
-        self.assertEqual(self.app.conf.BROKER_TRANSPORT, 'set_by_us')
-
     def test_WorkController(self):
     def test_WorkController(self):
         x = self.app.WorkController
         x = self.app.WorkController
         self.assertIs(x.app, self.app)
         self.assertIs(x.app, self.app)
@@ -537,9 +667,9 @@ class test_App(AppCase):
                 return args, kwargs
                 return args, kwargs
 
 
         self.app.loader = Loader(app=self.app)
         self.app.loader = Loader(app=self.app)
-        self.app.conf.ADMINS = None
+        self.app.conf.admins = None
         self.assertFalse(self.app.mail_admins('Subject', 'Body'))
         self.assertFalse(self.app.mail_admins('Subject', 'Body'))
-        self.app.conf.ADMINS = [('George Costanza', 'george@vandelay.com')]
+        self.app.conf.admins = [('George Costanza', 'george@vandelay.com')]
         self.assertTrue(self.app.mail_admins('Subject', 'Body'))
         self.assertTrue(self.app.mail_admins('Subject', 'Body'))
 
 
     def test_amqp_get_broker_info(self):
     def test_amqp_get_broker_info(self):
@@ -550,8 +680,8 @@ class test_App(AppCase):
              'virtual_host': '/'},
              'virtual_host': '/'},
             self.app.connection('pyamqp://').info(),
             self.app.connection('pyamqp://').info(),
         )
         )
-        self.app.conf.BROKER_PORT = 1978
-        self.app.conf.BROKER_VHOST = 'foo'
+        self.app.conf.broker_port = 1978
+        self.app.conf.broker_vhost = 'foo'
         self.assertDictContainsSubset(
         self.assertDictContainsSubset(
             {'port': 1978, 'virtual_host': 'foo'},
             {'port': 1978, 'virtual_host': 'foo'},
             self.app.connection('pyamqp://:1978/foo').info(),
             self.app.connection('pyamqp://:1978/foo').info(),
@@ -563,14 +693,14 @@ class test_App(AppCase):
     def test_amqp_failover_strategy_selection(self):
     def test_amqp_failover_strategy_selection(self):
         # Test passing in a string and make sure the string
         # Test passing in a string and make sure the string
         # gets there untouched
         # gets there untouched
-        self.app.conf.BROKER_FAILOVER_STRATEGY = 'foo-bar'
+        self.app.conf.broker_failover_strategy = 'foo-bar'
         self.assertEqual(
         self.assertEqual(
             self.app.connection('amqp:////value').failover_strategy,
             self.app.connection('amqp:////value').failover_strategy,
             'foo-bar',
             'foo-bar',
         )
         )
 
 
         # Try passing in None
         # Try passing in None
-        self.app.conf.BROKER_FAILOVER_STRATEGY = None
+        self.app.conf.broker_failover_strategy = None
         self.assertEqual(
         self.assertEqual(
             self.app.connection('amqp:////value').failover_strategy,
             self.app.connection('amqp:////value').failover_strategy,
             itertools.cycle,
             itertools.cycle,
@@ -580,16 +710,12 @@ class test_App(AppCase):
         def my_failover_strategy(it):
         def my_failover_strategy(it):
             yield True
             yield True
 
 
-        self.app.conf.BROKER_FAILOVER_STRATEGY = my_failover_strategy
+        self.app.conf.broker_failover_strategy = my_failover_strategy
         self.assertEqual(
         self.assertEqual(
             self.app.connection('amqp:////value').failover_strategy,
             self.app.connection('amqp:////value').failover_strategy,
             my_failover_strategy,
             my_failover_strategy,
         )
         )
 
 
-    def test_BROKER_BACKEND_alias(self):
-        self.assertEqual(self.app.conf.BROKER_BACKEND,
-                         self.app.conf.BROKER_TRANSPORT)
-
     def test_after_fork(self):
     def test_after_fork(self):
         p = self.app._pool = Mock()
         p = self.app._pool = Mock()
         self.app._after_fork(self.app)
         self.app._after_fork(self.app)

+ 8 - 8
celery/tests/app/test_beat.py

@@ -182,7 +182,7 @@ class test_Scheduler(AppCase):
         self.assertFalse(s._do_sync.called)
         self.assertFalse(s._do_sync.called)
 
 
     def test_should_sync_increments_sync_every_counter(self):
     def test_should_sync_increments_sync_every_counter(self):
-        self.app.conf.CELERYBEAT_SYNC_EVERY = 2
+        self.app.conf.beat_sync_every = 2
 
 
         @self.app.task(shared=False)
         @self.app.task(shared=False)
         def not_sync():
         def not_sync():
@@ -198,10 +198,10 @@ class test_Scheduler(AppCase):
         s.apply_async(s.Entry(task=not_sync.name, app=self.app))
         s.apply_async(s.Entry(task=not_sync.name, app=self.app))
         s._do_sync.assert_called_with()
         s._do_sync.assert_called_with()
 
 
-        self.app.conf.CELERYBEAT_SYNC_EVERY = 0
+        self.app.conf.beat_sync_every = 0
 
 
     def test_sync_task_counter_resets_on_do_sync(self):
     def test_sync_task_counter_resets_on_do_sync(self):
-        self.app.conf.CELERYBEAT_SYNC_EVERY = 1
+        self.app.conf.beat_sync_every = 1
 
 
         @self.app.task(shared=False)
         @self.app.task(shared=False)
         def not_sync():
         def not_sync():
@@ -214,7 +214,7 @@ class test_Scheduler(AppCase):
         s.apply_async(s.Entry(task=not_sync.name, app=self.app))
         s.apply_async(s.Entry(task=not_sync.name, app=self.app))
         self.assertEqual(s._tasks_since_sync, 0)
         self.assertEqual(s._tasks_since_sync, 0)
 
 
-        self.app.conf.CELERYBEAT_SYNC_EVERY = 0
+        self.app.conf.beat_sync_every = 0
 
 
     @patch('celery.app.base.Celery.send_task')
     @patch('celery.app.base.Celery.send_task')
     def test_send_task(self, send_task):
     def test_send_task(self, send_task):
@@ -249,20 +249,20 @@ class test_Scheduler(AppCase):
         callback(KeyError(), 5)
         callback(KeyError(), 5)
 
 
     def test_install_default_entries(self):
     def test_install_default_entries(self):
-        self.app.conf.CELERY_TASK_RESULT_EXPIRES = None
-        self.app.conf.CELERYBEAT_SCHEDULE = {}
+        self.app.conf.result_expires = None
+        self.app.conf.beat_schedule = {}
         s = mScheduler(app=self.app)
         s = mScheduler(app=self.app)
         s.install_default_entries({})
         s.install_default_entries({})
         self.assertNotIn('celery.backend_cleanup', s.data)
         self.assertNotIn('celery.backend_cleanup', s.data)
         self.app.backend.supports_autoexpire = False
         self.app.backend.supports_autoexpire = False
 
 
-        self.app.conf.CELERY_TASK_RESULT_EXPIRES = 30
+        self.app.conf.result_expires = 30
         s = mScheduler(app=self.app)
         s = mScheduler(app=self.app)
         s.install_default_entries({})
         s.install_default_entries({})
         self.assertIn('celery.backend_cleanup', s.data)
         self.assertIn('celery.backend_cleanup', s.data)
 
 
         self.app.backend.supports_autoexpire = True
         self.app.backend.supports_autoexpire = True
-        self.app.conf.CELERY_TASK_RESULT_EXPIRES = 31
+        self.app.conf.result_expires = 31
         s = mScheduler(app=self.app)
         s = mScheduler(app=self.app)
         s.install_default_entries({})
         s.install_default_entries({})
         self.assertNotIn('celery.backend_cleanup', s.data)
         self.assertNotIn('celery.backend_cleanup', s.data)

+ 2 - 2
celery/tests/app/test_builtins.py

@@ -83,7 +83,7 @@ class test_group(BuiltinsCase):
 
 
     def test_apply_async_eager(self):
     def test_apply_async_eager(self):
         self.task.apply = Mock()
         self.task.apply = Mock()
-        self.app.conf.CELERY_ALWAYS_EAGER = True
+        self.app.conf.task_always_eager = True
         self.task.apply_async((1, 2, 3, 4, 5))
         self.task.apply_async((1, 2, 3, 4, 5))
         self.assertTrue(self.task.apply.called)
         self.assertTrue(self.task.apply.called)
 
 
@@ -208,7 +208,7 @@ class test_chord(BuiltinsCase):
         self.assertEqual(resbody.options['chord'], 'some_chord_id')
         self.assertEqual(resbody.options['chord'], 'some_chord_id')
 
 
     def test_apply_eager(self):
     def test_apply_eager(self):
-        self.app.conf.CELERY_ALWAYS_EAGER = True
+        self.app.conf.task_always_eager = True
         x = chord([self.add.s(i, i) for i in range(10)], body=self.xsum.s())
         x = chord([self.add.s(i, i) for i in range(10)], body=self.xsum.s())
         r = x.apply_async()
         r = x.apply_async()
         self.assertEqual(r.get(), 90)
         self.assertEqual(r.get(), 90)

+ 23 - 6
celery/tests/app/test_defaults.py

@@ -4,11 +4,13 @@ import sys
 
 
 from importlib import import_module
 from importlib import import_module
 
 
-from celery.app.defaults import NAMESPACES
-
-from celery.tests.case import (
-    AppCase, pypy_version, sys_platform,
+from celery.app.defaults import (
+    _OLD_DEFAULTS, _OLD_SETTING_KEYS, _TO_NEW_KEY, _TO_OLD_KEY,
+    DEFAULTS, NAMESPACES, SETTING_KEYS
 )
 )
+from celery.five import values
+
+from celery.tests.case import AppCase, pypy_version, sys_platform
 
 
 
 
 class test_defaults(AppCase):
 class test_defaults(AppCase):
@@ -21,7 +23,7 @@ class test_defaults(AppCase):
             sys.modules['celery.app.defaults'] = self._prev
             sys.modules['celery.app.defaults'] = self._prev
 
 
     def test_option_repr(self):
     def test_option_repr(self):
-        self.assertTrue(repr(NAMESPACES['BROKER']['URL']))
+        self.assertTrue(repr(NAMESPACES['broker']['url']))
 
 
     def test_any(self):
     def test_any(self):
         val = object()
         val = object()
@@ -37,6 +39,21 @@ class test_defaults(AppCase):
             with pypy_version((1, 5, 0)):
             with pypy_version((1, 5, 0)):
                 self.assertEqual(self.defaults.DEFAULT_POOL, 'prefork')
                 self.assertEqual(self.defaults.DEFAULT_POOL, 'prefork')
 
 
+    def test_compat_indices(self):
+        self.assertFalse(any(key.isupper() for key in DEFAULTS))
+        self.assertFalse(any(key.islower() for key in _OLD_DEFAULTS))
+        self.assertFalse(any(key.isupper() for key in _TO_OLD_KEY))
+        self.assertFalse(any(key.islower() for key in _TO_NEW_KEY))
+        self.assertFalse(any(key.isupper() for key in SETTING_KEYS))
+        self.assertFalse(any(key.islower() for key in _OLD_SETTING_KEYS))
+        self.assertFalse(any(value.isupper() for value in values(_TO_NEW_KEY)))
+        self.assertFalse(any(value.islower() for value in values(_TO_OLD_KEY)))
+
+        for key in _TO_NEW_KEY:
+            self.assertIn(key, _OLD_SETTING_KEYS)
+        for key in _TO_OLD_KEY:
+            self.assertIn(key, SETTING_KEYS)
+
     def test_default_pool_jython(self):
     def test_default_pool_jython(self):
         with sys_platform('java 1.6.51'):
         with sys_platform('java 1.6.51'):
             self.assertEqual(self.defaults.DEFAULT_POOL, 'threads')
             self.assertEqual(self.defaults.DEFAULT_POOL, 'threads')
@@ -46,7 +63,7 @@ class test_defaults(AppCase):
 
 
         self.assertEqual(find('server_email')[2].default, 'celery@localhost')
         self.assertEqual(find('server_email')[2].default, 'celery@localhost')
         self.assertEqual(find('default_queue')[2].default, 'celery')
         self.assertEqual(find('default_queue')[2].default, 'celery')
-        self.assertEqual(find('celery_default_exchange')[2], 'celery')
+        self.assertEqual(find('task_default_exchange')[2], 'celery')
 
 
     @property
     @property
     def defaults(self):
     def defaults(self):

+ 7 - 8
celery/tests/app/test_loaders.py

@@ -22,7 +22,7 @@ from celery.tests.case import (
 class DummyLoader(base.BaseLoader):
 class DummyLoader(base.BaseLoader):
 
 
     def read_configuration(self):
     def read_configuration(self):
-        return {'foo': 'bar', 'CELERY_IMPORTS': ('os', 'sys')}
+        return {'foo': 'bar', 'imports': ('os', 'sys')}
 
 
 
 
 class test_loaders(AppCase):
 class test_loaders(AppCase):
@@ -65,10 +65,9 @@ class test_LoaderBase(AppCase):
         self.assertTrue(self.loader.now(utc=False))
         self.assertTrue(self.loader.now(utc=False))
 
 
     def test_read_configuration_no_env(self):
     def test_read_configuration_no_env(self):
-        self.assertDictEqual(
+        self.assertIsNone(
             base.BaseLoader(app=self.app).read_configuration(
             base.BaseLoader(app=self.app).read_configuration(
                 'FOO_X_S_WE_WQ_Q_WE'),
                 'FOO_X_S_WE_WQ_Q_WE'),
-            {},
         )
         )
 
 
     def test_autodiscovery(self):
     def test_autodiscovery(self):
@@ -101,7 +100,7 @@ class test_LoaderBase(AppCase):
     def test_import_default_modules(self):
     def test_import_default_modules(self):
         def modnames(l):
         def modnames(l):
             return [m.__name__ for m in l]
             return [m.__name__ for m in l]
-        self.app.conf.CELERY_IMPORTS = ('os', 'sys')
+        self.app.conf.imports = ('os', 'sys')
         self.assertEqual(
         self.assertEqual(
             sorted(modnames(self.loader.import_default_modules())),
             sorted(modnames(self.loader.import_default_modules())),
             sorted(modnames([os, sys])),
             sorted(modnames([os, sys])),
@@ -183,7 +182,7 @@ class test_DefaultLoader(AppCase):
 
 
         configname = os.environ.get('CELERY_CONFIG_MODULE') or 'celeryconfig'
         configname = os.environ.get('CELERY_CONFIG_MODULE') or 'celeryconfig'
         celeryconfig = ConfigModule(configname)
         celeryconfig = ConfigModule(configname)
-        celeryconfig.CELERY_IMPORTS = ('os', 'sys')
+        celeryconfig.imports = ('os', 'sys')
 
 
         prevconfig = sys.modules.get(configname)
         prevconfig = sys.modules.get(configname)
         sys.modules[configname] = celeryconfig
         sys.modules[configname] = celeryconfig
@@ -191,9 +190,9 @@ class test_DefaultLoader(AppCase):
             l = default.Loader(app=self.app)
             l = default.Loader(app=self.app)
             l.find_module = Mock(name='find_module')
             l.find_module = Mock(name='find_module')
             settings = l.read_configuration(fail_silently=False)
             settings = l.read_configuration(fail_silently=False)
-            self.assertTupleEqual(settings.CELERY_IMPORTS, ('os', 'sys'))
+            self.assertTupleEqual(settings.imports, ('os', 'sys'))
             settings = l.read_configuration(fail_silently=False)
             settings = l.read_configuration(fail_silently=False)
-            self.assertTupleEqual(settings.CELERY_IMPORTS, ('os', 'sys'))
+            self.assertTupleEqual(settings.imports, ('os', 'sys'))
             l.on_worker_init()
             l.on_worker_init()
         finally:
         finally:
             if prevconfig:
             if prevconfig:
@@ -239,7 +238,7 @@ class test_AppLoader(AppCase):
         self.loader = AppLoader(app=self.app)
         self.loader = AppLoader(app=self.app)
 
 
     def test_on_worker_init(self):
     def test_on_worker_init(self):
-        self.app.conf.CELERY_IMPORTS = ('subprocess',)
+        self.app.conf.imports = ('subprocess',)
         sys.modules.pop('subprocess', None)
         sys.modules.pop('subprocess', None)
         self.loader.init_worker()
         self.loader.init_worker()
         self.assertIn('subprocess', sys.modules)
         self.assertIn('subprocess', sys.modules)

+ 1 - 1
celery/tests/app/test_log.py

@@ -190,7 +190,7 @@ class test_default_logger(AppCase):
 
 
     def test_setup_logging_subsystem_misc2(self):
     def test_setup_logging_subsystem_misc2(self):
         with restore_logging():
         with restore_logging():
-            self.app.conf.CELERYD_HIJACK_ROOT_LOGGER = True
+            self.app.conf.worker_hijack_root_logger = True
             self.app.log.setup_logging_subsystem()
             self.app.log.setup_logging_subsystem()
 
 
     def test_get_default_logger(self):
     def test_get_default_logger(self):

+ 7 - 7
celery/tests/app/test_routes.py

@@ -21,7 +21,7 @@ def E(app, queues):
 
 
 
 
 def set_queues(app, **queues):
 def set_queues(app, **queues):
-    app.conf.CELERY_QUEUES = queues
+    app.conf.task_queues = queues
     app.amqp.queues = app.amqp.Queues(queues)
     app.amqp.queues = app.amqp.Queues(queues)
 
 
 
 
@@ -39,9 +39,9 @@ class RouteCase(AppCase):
             'routing_key': 'b.b.#',
             'routing_key': 'b.b.#',
         }
         }
         self.d_queue = {
         self.d_queue = {
-            'exchange': self.app.conf.CELERY_DEFAULT_EXCHANGE,
-            'exchange_type': self.app.conf.CELERY_DEFAULT_EXCHANGE_TYPE,
-            'routing_key': self.app.conf.CELERY_DEFAULT_ROUTING_KEY,
+            'exchange': self.app.conf.task_default_exchange,
+            'exchange_type': self.app.conf.task_default_exchange_type,
+            'routing_key': self.app.conf.task_default_routing_key,
         }
         }
 
 
         @self.app.task(shared=False)
         @self.app.task(shared=False)
@@ -74,7 +74,7 @@ class test_MapRoute(RouteCase):
 
 
     def test_expand_route_not_found(self):
     def test_expand_route_not_found(self):
         expand = E(self.app, self.app.amqp.Queues(
         expand = E(self.app, self.app.amqp.Queues(
-                   self.app.conf.CELERY_QUEUES, False))
+                   self.app.conf.task_queues, False))
         route = routes.MapRoute({'a': {'queue': 'x'}})
         route = routes.MapRoute({'a': {'queue': 'x'}})
         with self.assertRaises(QueueNotFound):
         with self.assertRaises(QueueNotFound):
             expand(route.route_for_task('a'))
             expand(route.route_for_task('a'))
@@ -124,7 +124,7 @@ class test_lookup_route(RouteCase):
     def test_lookup_paths_traversed(self):
     def test_lookup_paths_traversed(self):
         set_queues(
         set_queues(
             self.app, foo=self.a_queue, bar=self.b_queue,
             self.app, foo=self.a_queue, bar=self.b_queue,
-            **{self.app.conf.CELERY_DEFAULT_QUEUE: self.d_queue}
+            **{self.app.conf.task_default_queue: self.d_queue}
         )
         )
         R = routes.prepare((
         R = routes.prepare((
             {'celery.xaza': {'queue': 'bar'}},
             {'celery.xaza': {'queue': 'bar'}},
@@ -135,7 +135,7 @@ class test_lookup_route(RouteCase):
                          args=[1, 2], kwargs={})['queue'].name, 'foo')
                          args=[1, 2], kwargs={})['queue'].name, 'foo')
         self.assertEqual(
         self.assertEqual(
             router.route({}, 'celery.poza')['queue'].name,
             router.route({}, 'celery.poza')['queue'].name,
-            self.app.conf.CELERY_DEFAULT_QUEUE,
+            self.app.conf.task_default_queue,
         )
         )
 
 
 
 

+ 1 - 1
celery/tests/backends/test_amqp.py

@@ -375,7 +375,7 @@ class test_AMQPBackend(AppCase):
     def test_no_expires(self):
     def test_no_expires(self):
         b = self.create_backend(expires=None)
         b = self.create_backend(expires=None)
         app = self.app
         app = self.app
-        app.conf.CELERY_TASK_RESULT_EXPIRES = None
+        app.conf.result_expires = None
         b = self.create_backend(expires=None)
         b = self.create_backend(expires=None)
         with self.assertRaises(KeyError):
         with self.assertRaises(KeyError):
             b.queue_arguments['x-expires']
             b.queue_arguments['x-expires']

+ 1 - 1
celery/tests/backends/test_base.py

@@ -343,7 +343,7 @@ class test_KeyValueStoreBackend(AppCase):
             self.assertFalse(self.b.expire.called)
             self.assertFalse(self.b.expire.called)
             deps.delete.assert_called_with()
             deps.delete.assert_called_with()
             deps.join_native.assert_called_with(
             deps.join_native.assert_called_with(
-                propagate=self.b.app.conf.CELERY_CHORD_PROPAGATES,
+                propagate=self.b.app.conf.chord_propagates,
                 timeout=3.0,
                 timeout=3.0,
             )
             )
 
 

+ 2 - 2
celery/tests/backends/test_cache.py

@@ -31,12 +31,12 @@ class SomeClass(object):
 class test_CacheBackend(AppCase):
 class test_CacheBackend(AppCase):
 
 
     def setup(self):
     def setup(self):
-        self.app.conf.CELERY_RESULT_SERIALIZER = 'pickle'
+        self.app.conf.result_serializer = 'pickle'
         self.tb = CacheBackend(backend='memory://', app=self.app)
         self.tb = CacheBackend(backend='memory://', app=self.app)
         self.tid = uuid()
         self.tid = uuid()
 
 
     def test_no_backend(self):
     def test_no_backend(self):
-        self.app.conf.CELERY_CACHE_BACKEND = None
+        self.app.conf.cache_backend = None
         with self.assertRaises(ImproperlyConfigured):
         with self.assertRaises(ImproperlyConfigured):
             CacheBackend(backend=None, app=self.app)
             CacheBackend(backend=None, app=self.app)
 
 

+ 6 - 6
celery/tests/backends/test_cassandra.py

@@ -48,9 +48,9 @@ class test_CassandraBackend(AppCase):
 
 
     def setup(self):
     def setup(self):
         self.app.conf.update(
         self.app.conf.update(
-            CASSANDRA_SERVERS=['example.com'],
-            CASSANDRA_KEYSPACE='keyspace',
-            CASSANDRA_COLUMN_FAMILY='columns',
+            cassandra_servers=['example.com'],
+            cassandra_keyspace='keyspace',
+            cassandra_column_family='columns',
         )
         )
 
 
     def test_init_no_pycassa(self):
     def test_init_no_pycassa(self):
@@ -71,8 +71,8 @@ class test_CassandraBackend(AppCase):
             cons = mod.pycassa.ConsistencyLevel = Object()
             cons = mod.pycassa.ConsistencyLevel = Object()
             cons.LOCAL_QUORUM = 'foo'
             cons.LOCAL_QUORUM = 'foo'
 
 
-            self.app.conf.CASSANDRA_READ_CONSISTENCY = 'LOCAL_FOO'
-            self.app.conf.CASSANDRA_WRITE_CONSISTENCY = 'LOCAL_FOO'
+            self.app.conf.cassandra_read_consistency = 'LOCAL_FOO'
+            self.app.conf.cassandra_write_consistency = 'LOCAL_FOO'
 
 
             mod.CassandraBackend(app=self.app)
             mod.CassandraBackend(app=self.app)
             cons.LOCAL_FOO = 'bar'
             cons.LOCAL_FOO = 'bar'
@@ -80,7 +80,7 @@ class test_CassandraBackend(AppCase):
 
 
             # no servers raises ImproperlyConfigured
             # no servers raises ImproperlyConfigured
             with self.assertRaises(ImproperlyConfigured):
             with self.assertRaises(ImproperlyConfigured):
-                self.app.conf.CASSANDRA_SERVERS = None
+                self.app.conf.cassandra_servers = None
                 mod.CassandraBackend(
                 mod.CassandraBackend(
                     app=self.app, keyspace='b', column_family='c',
                     app=self.app, keyspace='b', column_family='c',
                 )
                 )

+ 7 - 7
celery/tests/backends/test_couchbase.py

@@ -47,13 +47,13 @@ class test_CouchBaseBackend(AppCase):
 
 
     def test_init_no_settings(self):
     def test_init_no_settings(self):
         """Test init no settings."""
         """Test init no settings."""
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = []
+        self.app.conf.couchbase_backend_settings = []
         with self.assertRaises(ImproperlyConfigured):
         with self.assertRaises(ImproperlyConfigured):
             CouchBaseBackend(app=self.app)
             CouchBaseBackend(app=self.app)
 
 
     def test_init_settings_is_None(self):
     def test_init_settings_is_None(self):
         """Test init settings is None."""
         """Test init settings is None."""
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = None
+        self.app.conf.couchbase_backend_settings = None
         CouchBaseBackend(app=self.app)
         CouchBaseBackend(app=self.app)
 
 
     def test_get_connection_connection_exists(self):
     def test_get_connection_connection_exists(self):
@@ -75,7 +75,7 @@ class test_CouchBaseBackend(AppCase):
 
 
         TODO Should test on key not exists
         TODO Should test on key not exists
         """
         """
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = {}
+        self.app.conf.couchbase_backend_settings = {}
         x = CouchBaseBackend(app=self.app)
         x = CouchBaseBackend(app=self.app)
         x._connection = Mock()
         x._connection = Mock()
         mocked_get = x._connection.get = Mock()
         mocked_get = x._connection.get = Mock()
@@ -91,7 +91,7 @@ class test_CouchBaseBackend(AppCase):
         CouchBaseBackend.set should return None and take two params
         CouchBaseBackend.set should return None and take two params
         db conn to couchbase is mocked.
         db conn to couchbase is mocked.
         """
         """
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = None
+        self.app.conf.couchbase_backend_settings = None
         x = CouchBaseBackend(app=self.app)
         x = CouchBaseBackend(app=self.app)
         x._connection = MagicMock()
         x._connection = MagicMock()
         x._connection.set = MagicMock()
         x._connection.set = MagicMock()
@@ -107,7 +107,7 @@ class test_CouchBaseBackend(AppCase):
 
 
         TODO Should test on key not exists.
         TODO Should test on key not exists.
         """
         """
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = {}
+        self.app.conf.couchbase_backend_settings = {}
         x = CouchBaseBackend(app=self.app)
         x = CouchBaseBackend(app=self.app)
         x._connection = Mock()
         x._connection = Mock()
         mocked_delete = x._connection.delete = Mock()
         mocked_delete = x._connection.delete = Mock()
@@ -120,9 +120,9 @@ class test_CouchBaseBackend(AppCase):
         """
         """
         Test config params are correct.
         Test config params are correct.
 
 
-        celery.conf.CELERY_COUCHBASE_BACKEND_SETTINGS is properly set.
+        app.conf.couchbase_backend_settings is properly set.
         """
         """
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = {
+        self.app.conf.couchbase_backend_settings = {
             'bucket': 'mycoolbucket',
             'bucket': 'mycoolbucket',
             'host': ['here.host.com', 'there.host.com'],
             'host': ['here.host.com', 'there.host.com'],
             'username': 'johndoe',
             'username': 'johndoe',

+ 2 - 2
celery/tests/backends/test_database.py

@@ -39,7 +39,7 @@ class test_DatabaseBackend(AppCase):
         if DatabaseBackend is None:
         if DatabaseBackend is None:
             raise SkipTest('sqlalchemy not installed')
             raise SkipTest('sqlalchemy not installed')
         self.uri = 'sqlite:///test.db'
         self.uri = 'sqlite:///test.db'
-        self.app.conf.CELERY_RESULT_SERIALIZER = 'pickle'
+        self.app.conf.result_serializer = 'pickle'
 
 
     def test_retry_helper(self):
     def test_retry_helper(self):
         from celery.backends.database import DatabaseError
         from celery.backends.database import DatabaseError
@@ -56,7 +56,7 @@ class test_DatabaseBackend(AppCase):
         self.assertEqual(calls[0], 5)
         self.assertEqual(calls[0], 5)
 
 
     def test_missing_dburi_raises_ImproperlyConfigured(self):
     def test_missing_dburi_raises_ImproperlyConfigured(self):
-        self.app.conf.CELERY_RESULT_DBURI = None
+        self.app.conf.sqlalchemy_dburi = None
         with self.assertRaises(ImproperlyConfigured):
         with self.assertRaises(ImproperlyConfigured):
             DatabaseBackend(app=self.app)
             DatabaseBackend(app=self.app)
 
 

+ 4 - 4
celery/tests/backends/test_mongodb.py

@@ -59,16 +59,16 @@ class test_MongoBackend(AppCase):
             module.pymongo = prev
             module.pymongo = prev
 
 
     def test_init_no_settings(self):
     def test_init_no_settings(self):
-        self.app.conf.CELERY_MONGODB_BACKEND_SETTINGS = []
+        self.app.conf.mongodb_backend_settings = []
         with self.assertRaises(ImproperlyConfigured):
         with self.assertRaises(ImproperlyConfigured):
             MongoBackend(app=self.app)
             MongoBackend(app=self.app)
 
 
     def test_init_settings_is_None(self):
     def test_init_settings_is_None(self):
-        self.app.conf.CELERY_MONGODB_BACKEND_SETTINGS = None
+        self.app.conf.mongodb_backend_settings = None
         MongoBackend(app=self.app)
         MongoBackend(app=self.app)
 
 
     def test_init_with_settings(self):
     def test_init_with_settings(self):
-        self.app.conf.CELERY_MONGODB_BACKEND_SETTINGS = None
+        self.app.conf.mongodb_backend_settings = None
         # empty settings
         # empty settings
         mb = MongoBackend(app=self.app)
         mb = MongoBackend(app=self.app)
 
 
@@ -103,7 +103,7 @@ class test_MongoBackend(AppCase):
         self.assertEqual(mb.database_name, 'celerydatabase')
         self.assertEqual(mb.database_name, 'celerydatabase')
 
 
         # same uri, change some parameters in backend settings
         # same uri, change some parameters in backend settings
-        self.app.conf.CELERY_MONGODB_BACKEND_SETTINGS = {
+        self.app.conf.mongodb_backend_settings = {
             'replicaset': 'rs1',
             'replicaset': 'rs1',
             'user': 'backenduser',
             'user': 'backenduser',
             'database': 'another_db',
             'database': 'another_db',

+ 6 - 6
celery/tests/backends/test_new_cassandra.py

@@ -20,9 +20,9 @@ class test_CassandraBackend(AppCase):
 
 
     def setup(self):
     def setup(self):
         self.app.conf.update(
         self.app.conf.update(
-            CASSANDRA_SERVERS=['example.com'],
-            CASSANDRA_KEYSPACE='celery',
-            CASSANDRA_TABLE='task_results',
+            cassandra_servers=['example.com'],
+            cassandra_keyspace='celery',
+            cassandra_table='task_results',
         )
         )
 
 
     def test_init_no_cassandra(self):
     def test_init_no_cassandra(self):
@@ -44,8 +44,8 @@ class test_CassandraBackend(AppCase):
             cons = mod.cassandra.ConsistencyLevel = Object()
             cons = mod.cassandra.ConsistencyLevel = Object()
             cons.LOCAL_QUORUM = 'foo'
             cons.LOCAL_QUORUM = 'foo'
 
 
-            self.app.conf.CASSANDRA_READ_CONSISTENCY = 'LOCAL_FOO'
-            self.app.conf.CASSANDRA_WRITE_CONSISTENCY = 'LOCAL_FOO'
+            self.app.conf.cassandra_read_consistency = 'LOCAL_FOO'
+            self.app.conf.cassandra_write_consistency = 'LOCAL_FOO'
 
 
             mod.CassandraBackend(app=self.app)
             mod.CassandraBackend(app=self.app)
             cons.LOCAL_FOO = 'bar'
             cons.LOCAL_FOO = 'bar'
@@ -53,7 +53,7 @@ class test_CassandraBackend(AppCase):
 
 
             # no servers raises ImproperlyConfigured
             # no servers raises ImproperlyConfigured
             with self.assertRaises(ImproperlyConfigured):
             with self.assertRaises(ImproperlyConfigured):
-                self.app.conf.CASSANDRA_SERVERS = None
+                self.app.conf.cassandra_servers = None
                 mod.CassandraBackend(
                 mod.CassandraBackend(
                     app=self.app, keyspace='b', column_family='c',
                     app=self.app, keyspace='b', column_family='c',
                 )
                 )

+ 6 - 6
celery/tests/backends/test_redis.py

@@ -180,15 +180,15 @@ class test_RedisBackend(AppCase):
 
 
     def test_conf_raises_KeyError(self):
     def test_conf_raises_KeyError(self):
         self.app.conf = AttributeDict({
         self.app.conf = AttributeDict({
-            'CELERY_RESULT_SERIALIZER': 'json',
-            'CELERY_MAX_CACHED_RESULTS': 1,
-            'CELERY_ACCEPT_CONTENT': ['json'],
-            'CELERY_TASK_RESULT_EXPIRES': None,
+            'result_serializer': 'json',
+            'result_cache_max': 1,
+            'result_expires': None,
+            'accept_content': ['json'],
         })
         })
         self.Backend(app=self.app, new_join=True)
         self.Backend(app=self.app, new_join=True)
 
 
     def test_expires_defaults_to_config(self):
     def test_expires_defaults_to_config(self):
-        self.app.conf.CELERY_TASK_RESULT_EXPIRES = 10
+        self.app.conf.result_expires = 10
         b = self.Backend(expires=None, app=self.app, new_join=True)
         b = self.Backend(expires=None, app=self.app, new_join=True)
         self.assertEqual(b.expires, 10)
         self.assertEqual(b.expires, 10)
 
 
@@ -216,7 +216,7 @@ class test_RedisBackend(AppCase):
         b = self.Backend(expires=None, app=self.app, new_join=True)
         b = self.Backend(expires=None, app=self.app, new_join=True)
         self.assertEqual(
         self.assertEqual(
             b.expires,
             b.expires,
-            self.app.conf.CELERY_TASK_RESULT_EXPIRES.total_seconds(),
+            self.app.conf.result_expires.total_seconds(),
         )
         )
 
 
     def test_expires_is_timedelta(self):
     def test_expires_is_timedelta(self):

+ 13 - 13
celery/tests/backends/test_riak.py

@@ -18,7 +18,7 @@ class test_RiakBackend(AppCase):
     def setup(self):
     def setup(self):
         if riak is None:
         if riak is None:
             raise SkipTest('riak is not installed.')
             raise SkipTest('riak is not installed.')
-        self.app.conf.CELERY_RESULT_BACKEND = 'riak://'
+        self.app.conf.result_backend = 'riak://'
 
 
     @property
     @property
     def backend(self):
     def backend(self):
@@ -37,7 +37,7 @@ class test_RiakBackend(AppCase):
 
 
     def test_init_no_settings(self):
     def test_init_no_settings(self):
         """Test init no settings."""
         """Test init no settings."""
-        self.app.conf.CELERY_RIAK_BACKEND_SETTINGS = []
+        self.app.conf.riak_backend_settings = []
         with self.assertRaises(ImproperlyConfigured):
         with self.assertRaises(ImproperlyConfigured):
             RiakBackend(app=self.app)
             RiakBackend(app=self.app)
 
 
@@ -45,7 +45,7 @@ class test_RiakBackend(AppCase):
         """
         """
         Test init settings is None
         Test init settings is None
         """
         """
-        self.app.conf.CELERY_RIAK_BACKEND_SETTINGS = None
+        self.app.conf.riak_backend_settings = None
         self.assertTrue(self.app.backend)
         self.assertTrue(self.app.backend)
 
 
     def test_get_client_client_exists(self):
     def test_get_client_client_exists(self):
@@ -67,7 +67,7 @@ class test_RiakBackend(AppCase):
         db conn to riak is mocked
         db conn to riak is mocked
         TODO Should test on key not exists
         TODO Should test on key not exists
         """
         """
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = {}
+        self.app.conf.couchbase_backend_settings = {}
         self.backend._client = Mock(name='_client')
         self.backend._client = Mock(name='_client')
         self.backend._bucket = Mock(name='_bucket')
         self.backend._bucket = Mock(name='_bucket')
         mocked_get = self.backend._bucket.get = Mock(name='bucket.get')
         mocked_get = self.backend._bucket.get = Mock(name='bucket.get')
@@ -84,7 +84,7 @@ class test_RiakBackend(AppCase):
         db conn to couchbase is mocked.
         db conn to couchbase is mocked.
 
 
         """
         """
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = None
+        self.app.conf.couchbase_backend_settings = None
         self.backend._client = MagicMock()
         self.backend._client = MagicMock()
         self.backend._bucket = MagicMock()
         self.backend._bucket = MagicMock()
         self.backend._bucket.set = MagicMock()
         self.backend._bucket.set = MagicMock()
@@ -100,7 +100,7 @@ class test_RiakBackend(AppCase):
         TODO Should test on key not exists
         TODO Should test on key not exists
 
 
         """
         """
-        self.app.conf.CELERY_COUCHBASE_BACKEND_SETTINGS = {}
+        self.app.conf.couchbase_backend_settings = {}
 
 
         self.backend._client = Mock(name='_client')
         self.backend._client = Mock(name='_client')
         self.backend._bucket = Mock(name='_bucket')
         self.backend._bucket = Mock(name='_bucket')
@@ -112,11 +112,11 @@ class test_RiakBackend(AppCase):
 
 
     def test_config_params(self):
     def test_config_params(self):
         """
         """
-        test celery.conf.CELERY_RIAK_BACKEND_SETTINGS
-        celery.conf.CELERY_RIAK_BACKEND_SETTINGS
+        test celery.conf.riak_backend_settingS
+        celery.conf.riak_backend_settingS
         is properly set
         is properly set
         """
         """
-        self.app.conf.CELERY_RIAK_BACKEND_SETTINGS = {
+        self.app.conf.riak_backend_settings = {
             'bucket': 'mycoolbucket',
             'bucket': 'mycoolbucket',
             'host': 'there.host.com',
             'host': 'there.host.com',
             'port': '1234',
             'port': '1234',
@@ -139,17 +139,17 @@ class test_RiakBackend(AppCase):
         """
         """
         test get backend params by url
         test get backend params by url
         """
         """
-        self.app.conf.CELERY_RESULT_BACKEND = 'riak://myhost:123/mycoolbucket'
+        self.app.conf.result_backend = 'riak://myhost:123/mycoolbucket'
         self.assertEqual(self.backend.bucket_name, 'mycoolbucket')
         self.assertEqual(self.backend.bucket_name, 'mycoolbucket')
         self.assertEqual(self.backend.host, 'myhost')
         self.assertEqual(self.backend.host, 'myhost')
         self.assertEqual(self.backend.port, 123)
         self.assertEqual(self.backend.port, 123)
 
 
     def test_non_ASCII_bucket_raises(self):
     def test_non_ASCII_bucket_raises(self):
-        """test celery.conf.CELERY_RIAK_BACKEND_SETTINGS and
-        celery.conf.CELERY_RIAK_BACKEND_SETTINGS
+        """test app.conf.riak_backend_settings and
+        app.conf.riak_backend_settings
         is properly set
         is properly set
         """
         """
-        self.app.conf.CELERY_RIAK_BACKEND_SETTINGS = {
+        self.app.conf.riak_backend_settings = {
             'bucket': 'héhé',
             'bucket': 'héhé',
             'host': 'there.host.com',
             'host': 'there.host.com',
             'port': '1234',
             'port': '1234',

+ 3 - 3
celery/tests/bin/test_base.py

@@ -281,14 +281,14 @@ class test_Command(AppCase):
     def test_with_cmdline_config(self):
     def test_with_cmdline_config(self):
         cmd = MockCommand(app=self.app)
         cmd = MockCommand(app=self.app)
         cmd.enable_config_from_cmdline = True
         cmd.enable_config_from_cmdline = True
-        cmd.namespace = 'celeryd'
+        cmd.namespace = 'worker'
         rest = cmd.setup_app_from_commandline(argv=[
         rest = cmd.setup_app_from_commandline(argv=[
             '--loglevel=INFO', '--',
             '--loglevel=INFO', '--',
             'broker.url=amqp://broker.example.com',
             'broker.url=amqp://broker.example.com',
             '.prefetch_multiplier=100'])
             '.prefetch_multiplier=100'])
-        self.assertEqual(cmd.app.conf.BROKER_URL,
+        self.assertEqual(cmd.app.conf.broker_url,
                          'amqp://broker.example.com')
                          'amqp://broker.example.com')
-        self.assertEqual(cmd.app.conf.CELERYD_PREFETCH_MULTIPLIER, 100)
+        self.assertEqual(cmd.app.conf.worker_prefetch_multiplier, 100)
         self.assertListEqual(rest, ['--loglevel=INFO'])
         self.assertListEqual(rest, ['--loglevel=INFO'])
 
 
     def test_find_app(self):
     def test_find_app(self):

+ 4 - 4
celery/tests/bin/test_worker.py

@@ -237,12 +237,12 @@ class test_Worker(WorkerAppCase):
         self.assertIn('celery', app.amqp.queues)
         self.assertIn('celery', app.amqp.queues)
         self.assertNotIn('celery', app.amqp.queues.consume_from)
         self.assertNotIn('celery', app.amqp.queues.consume_from)
 
 
-        c.CELERY_CREATE_MISSING_QUEUES = False
+        c.task_create_missing_queues = False
         del(app.amqp.queues)
         del(app.amqp.queues)
         with self.assertRaises(ImproperlyConfigured):
         with self.assertRaises(ImproperlyConfigured):
             self.Worker(app=self.app).setup_queues(['image'])
             self.Worker(app=self.app).setup_queues(['image'])
         del(app.amqp.queues)
         del(app.amqp.queues)
-        c.CELERY_CREATE_MISSING_QUEUES = True
+        c.task_create_missing_queues = True
         worker = self.Worker(app=self.app)
         worker = self.Worker(app=self.app)
         worker.setup_queues(['image'])
         worker.setup_queues(['image'])
         self.assertIn('image', app.amqp.queues.consume_from)
         self.assertIn('image', app.amqp.queues.consume_from)
@@ -283,7 +283,7 @@ class test_Worker(WorkerAppCase):
 
 
         with patch('os.getuid') as getuid:
         with patch('os.getuid') as getuid:
             getuid.return_value = 0
             getuid.return_value = 0
-            self.app.conf.CELERY_ACCEPT_CONTENT = ['pickle']
+            self.app.conf.accept_content = ['pickle']
             worker = self.Worker(app=self.app)
             worker = self.Worker(app=self.app)
             worker.on_start()
             worker.on_start()
             _exit.assert_called_with(1)
             _exit.assert_called_with(1)
@@ -297,7 +297,7 @@ class test_Worker(WorkerAppCase):
                     worker.on_start()
                     worker.on_start()
             finally:
             finally:
                 platforms.C_FORCE_ROOT = False
                 platforms.C_FORCE_ROOT = False
-            self.app.conf.CELERY_ACCEPT_CONTENT = ['json']
+            self.app.conf.accept_content = ['json']
             with self.assertWarnsRegex(
             with self.assertWarnsRegex(
                     RuntimeWarning,
                     RuntimeWarning,
                     r'absolutely not recommended'):
                     r'absolutely not recommended'):

+ 11 - 11
celery/tests/case.py

@@ -85,21 +85,21 @@ Test {0} Modified handlers for the root logger\
 
 
 CELERY_TEST_CONFIG = {
 CELERY_TEST_CONFIG = {
     #: Don't want log output when running suite.
     #: Don't want log output when running suite.
-    'CELERYD_HIJACK_ROOT_LOGGER': False,
-    'CELERY_SEND_TASK_ERROR_EMAILS': False,
-    'CELERY_DEFAULT_QUEUE': 'testcelery',
-    'CELERY_DEFAULT_EXCHANGE': 'testcelery',
-    'CELERY_DEFAULT_ROUTING_KEY': 'testcelery',
-    'CELERY_QUEUES': (
+    'worker_hijack_root_logger': False,
+    'worker_log_color': False,
+    'task_send_error_emails': False,
+    'task_default_queue': 'testcelery',
+    'task_default_exchange': 'testcelery',
+    'task_default_routing_key': 'testcelery',
+    'task_queues': (
         Queue('testcelery', routing_key='testcelery'),
         Queue('testcelery', routing_key='testcelery'),
     ),
     ),
-    'CELERY_ACCEPT_CONTENT': ('json', 'pickle'),
-    'CELERY_ENABLE_UTC': True,
-    'CELERY_TIMEZONE': 'UTC',
-    'CELERYD_LOG_COLOR': False,
+    'accept_content': ('json', 'pickle'),
+    'enable_utc': True,
+    'timezone': 'UTC',
 
 
     # Mongo results tests (only executed if installed and running)
     # Mongo results tests (only executed if installed and running)
-    'CELERY_MONGODB_BACKEND_SETTINGS': {
+    'mongodb_backend_settings': {
         'host': os.environ.get('MONGO_HOST') or 'localhost',
         'host': os.environ.get('MONGO_HOST') or 'localhost',
         'port': os.environ.get('MONGO_PORT') or 27017,
         'port': os.environ.get('MONGO_PORT') or 27017,
         'database': os.environ.get('MONGO_DB') or 'celery_unittests',
         'database': os.environ.get('MONGO_DB') or 'celery_unittests',

+ 2 - 2
celery/tests/compat_modules/test_http.py

@@ -142,7 +142,7 @@ class test_HttpDispatch(AppCase):
 class test_URL(AppCase):
 class test_URL(AppCase):
 
 
     def test_URL_get_async(self):
     def test_URL_get_async(self):
-        self.app.conf.CELERY_ALWAYS_EAGER = True
+        self.app.conf.task_always_eager = True
         with mock_urlopen(success_response(100)):
         with mock_urlopen(success_response(100)):
             d = http.URL(
             d = http.URL(
                 'http://example.com/mul', app=self.app,
                 'http://example.com/mul', app=self.app,
@@ -150,7 +150,7 @@ class test_URL(AppCase):
             self.assertEqual(d.get(), 100)
             self.assertEqual(d.get(), 100)
 
 
     def test_URL_post_async(self):
     def test_URL_post_async(self):
-        self.app.conf.CELERY_ALWAYS_EAGER = True
+        self.app.conf.task_always_eager = True
         with mock_urlopen(success_response(100)):
         with mock_urlopen(success_response(100)):
             d = http.URL(
             d = http.URL(
                 'http://example.com/mul', app=self.app,
                 'http://example.com/mul', app=self.app,

+ 2 - 2
celery/tests/compat_modules/test_sets.py

@@ -170,10 +170,10 @@ class test_TaskSet(SetsCase):
             [self.MockTask.subtask((i, i)) for i in (2, 4, 8)],
             [self.MockTask.subtask((i, i)) for i in (2, 4, 8)],
             app=self.app,
             app=self.app,
         )
         )
-        app.conf.CELERY_ALWAYS_EAGER = True
+        app.conf.task_always_eager = True
         ts.apply_async()
         ts.apply_async()
         self.assertEqual(ts.applied, 1)
         self.assertEqual(ts.applied, 1)
-        app.conf.CELERY_ALWAYS_EAGER = False
+        app.conf.task_always_eager = False
 
 
         with patch('celery.task.sets.get_current_worker_task') as gwt:
         with patch('celery.task.sets.get_current_worker_task') as gwt:
             parent = gwt.return_value = Mock()
             parent = gwt.return_value = Mock()

+ 1 - 1
celery/tests/events/test_events.py

@@ -125,7 +125,7 @@ class test_EventDispatcher(AppCase):
             self.assertTrue(dispatcher.enabled)
             self.assertTrue(dispatcher.enabled)
             self.assertTrue(dispatcher.producer.channel)
             self.assertTrue(dispatcher.producer.channel)
             self.assertEqual(dispatcher.producer.serializer,
             self.assertEqual(dispatcher.producer.serializer,
-                             self.app.conf.CELERY_EVENT_SERIALIZER)
+                             self.app.conf.event_serializer)
 
 
             created_channel = dispatcher.producer.channel
             created_channel = dispatcher.producer.channel
             dispatcher.disable()
             dispatcher.disable()

+ 3 - 3
celery/tests/security/test_security.py

@@ -57,7 +57,7 @@ class test_security(SecurityCase):
         disabled = registry._disabled_content_types
         disabled = registry._disabled_content_types
         self.assertEqual(0, len(disabled))
         self.assertEqual(0, len(disabled))
 
 
-        self.app.conf.CELERY_TASK_SERIALIZER = 'json'
+        self.app.conf.task_serializer = 'json'
         self.app.setup_security()
         self.app.setup_security()
         self.assertIn('application/x-python-serialize', disabled)
         self.assertIn('application/x-python-serialize', disabled)
         disabled.clear()
         disabled.clear()
@@ -75,7 +75,7 @@ class test_security(SecurityCase):
             finally:
             finally:
                 calls[0] += 1
                 calls[0] += 1
 
 
-        self.app.conf.CELERY_TASK_SERIALIZER = 'auth'
+        self.app.conf.task_serializer = 'auth'
         with mock_open(side_effect=effect):
         with mock_open(side_effect=effect):
             with patch('celery.security.registry') as registry:
             with patch('celery.security.registry') as registry:
                 store = Mock()
                 store = Mock()
@@ -85,7 +85,7 @@ class test_security(SecurityCase):
                 registry._set_default_serializer.assert_called_with('auth')
                 registry._set_default_serializer.assert_called_with('auth')
 
 
     def test_security_conf(self):
     def test_security_conf(self):
-        self.app.conf.CELERY_TASK_SERIALIZER = 'auth'
+        self.app.conf.task_serializer = 'auth'
         with self.assertRaises(ImproperlyConfigured):
         with self.assertRaises(ImproperlyConfigured):
             self.app.setup_security()
             self.app.setup_security()
 
 

+ 2 - 2
celery/tests/tasks/test_canvas.py

@@ -198,7 +198,7 @@ class test_chunks(CanvasCase):
         x()
         x()
         gr.apply_async.assert_called_with((), {}, route_name=self.add.name)
         gr.apply_async.assert_called_with((), {}, route_name=self.add.name)
 
 
-        self.app.conf.CELERY_ALWAYS_EAGER = True
+        self.app.conf.task_always_eager = True
         chunks.apply_chunks(app=self.app, **x['kwargs'])
         chunks.apply_chunks(app=self.app, **x['kwargs'])
 
 
 
 
@@ -216,7 +216,7 @@ class test_chain(CanvasCase):
         self.assertIsInstance(signature(dict(x)), chain)
         self.assertIsInstance(signature(dict(x)), chain)
 
 
     def test_always_eager(self):
     def test_always_eager(self):
-        self.app.conf.CELERY_ALWAYS_EAGER = True
+        self.app.conf.task_always_eager = True
         self.assertEqual(~(self.add.s(4, 4) | self.add.s(8)), 16)
         self.assertEqual(~(self.add.s(4, 4) | self.add.s(8)), 16)
 
 
     def test_apply(self):
     def test_apply(self):

+ 3 - 3
celery/tests/tasks/test_chord.py

@@ -194,18 +194,18 @@ class test_chord(ChordCase):
         def sumX(n):
         def sumX(n):
             return sum(n)
             return sum(n)
 
 
-        self.app.conf.CELERY_ALWAYS_EAGER = True
+        self.app.conf.task_always_eager = True
         x = chord(addX.s(i, i) for i in range(10))
         x = chord(addX.s(i, i) for i in range(10))
         body = sumX.s()
         body = sumX.s()
         result = x(body)
         result = x(body)
         self.assertEqual(result.get(), sum(i + i for i in range(10)))
         self.assertEqual(result.get(), sum(i + i for i in range(10)))
 
 
     def test_apply(self):
     def test_apply(self):
-        self.app.conf.CELERY_ALWAYS_EAGER = False
+        self.app.conf.task_always_eager = False
         from celery import chord
         from celery import chord
 
 
         m = Mock()
         m = Mock()
-        m.app.conf.CELERY_ALWAYS_EAGER = False
+        m.app.conf.task_always_eager = False
         m.AsyncResult = AsyncResult
         m.AsyncResult = AsyncResult
         prev, chord.run = chord.run, m
         prev, chord.run = chord.run, m
         try:
         try:

+ 2 - 2
celery/tests/tasks/test_result.py

@@ -44,7 +44,7 @@ def make_mock_group(app, size=10):
 class test_AsyncResult(AppCase):
 class test_AsyncResult(AppCase):
 
 
     def setup(self):
     def setup(self):
-        self.app.conf.CELERY_RESULT_SERIALIZER = 'pickle'
+        self.app.conf.result_serializer = 'pickle'
         self.task1 = mock_task('task1', states.SUCCESS, 'the')
         self.task1 = mock_task('task1', states.SUCCESS, 'the')
         self.task2 = mock_task('task2', states.SUCCESS, 'quick')
         self.task2 = mock_task('task2', states.SUCCESS, 'quick')
         self.task3 = mock_task('task3', states.FAILURE, KeyError('brown'))
         self.task3 = mock_task('task3', states.FAILURE, KeyError('brown'))
@@ -618,7 +618,7 @@ class test_pending_AsyncResult(AppCase):
 class test_failed_AsyncResult(test_GroupResult):
 class test_failed_AsyncResult(test_GroupResult):
 
 
     def setup(self):
     def setup(self):
-        self.app.conf.CELERY_RESULT_SERIALIZER = 'pickle'
+        self.app.conf.result_serializer = 'pickle'
         self.size = 11
         self.size = 11
         subtasks = make_mock_group(self.app, 10)
         subtasks = make_mock_group(self.app, 10)
         failed = mock_task('ts11', states.FAILURE, KeyError('Baz'))
         failed = mock_task('ts11', states.FAILURE, KeyError('Baz'))

+ 2 - 2
celery/tests/tasks/test_tasks.py

@@ -462,8 +462,8 @@ class test_apply_task(TasksCase):
         with self.assertRaises(KeyError):
         with self.assertRaises(KeyError):
             self.raising.apply(throw=True)
             self.raising.apply(throw=True)
 
 
-    def test_apply_with_CELERY_EAGER_PROPAGATES_EXCEPTIONS(self):
-        self.app.conf.CELERY_EAGER_PROPAGATES_EXCEPTIONS = True
+    def test_apply_with_task_eager_propagates_exceptions(self):
+        self.app.conf.task_eager_propagates_exceptions = True
         with self.assertRaises(KeyError):
         with self.assertRaises(KeyError):
             self.raising.apply()
             self.raising.apply()
 
 

+ 8 - 4
celery/tests/utils/test_datastructures.py

@@ -44,8 +44,10 @@ class test_DictAttribute(Case):
 
 
     def test_setdefault(self):
     def test_setdefault(self):
         x = DictAttribute(Object())
         x = DictAttribute(Object())
-        self.assertEqual(x.setdefault('foo', 'NEW'), 'NEW')
-        self.assertEqual(x.setdefault('foo', 'XYZ'), 'NEW')
+        x.setdefault('foo', 'NEW')
+        self.assertEqual(x['foo'], 'NEW')
+        x.setdefault('foo', 'XYZ')
+        self.assertEqual(x['foo'], 'NEW')
 
 
     def test_contains(self):
     def test_contains(self):
         x = DictAttribute(Object())
         x = DictAttribute(Object())
@@ -71,8 +73,10 @@ class test_ConfigurationView(Case):
                                        'both': 1}])
                                        'both': 1}])
 
 
     def test_setdefault(self):
     def test_setdefault(self):
-        self.assertEqual(self.view.setdefault('both', 36), 2)
-        self.assertEqual(self.view.setdefault('new', 36), 36)
+        self.view.setdefault('both', 36)
+        self.assertEqual(self.view['both'], 2)
+        self.view.setdefault('new', 36)
+        self.assertEqual(self.view['new'], 36)
 
 
     def test_get(self):
     def test_get(self):
         self.assertEqual(self.view.get('both'), 2)
         self.assertEqual(self.view.get('both'), 2)

+ 9 - 8
celery/tests/worker/test_consumer.py

@@ -34,10 +34,11 @@ class test_Consumer(AppCase):
             hub=None if no_hub else Mock(),
             hub=None if no_hub else Mock(),
             **kwargs
             **kwargs
         )
         )
-        consumer.blueprint = Mock()
-        consumer._restart_state = Mock()
+        consumer.blueprint = Mock(name='blueprint')
+        consumer._restart_state = Mock(name='_restart_state')
         consumer.connection = _amqp_connection()
         consumer.connection = _amqp_connection()
         consumer.connection_errors = (socket.error, OSError,)
         consumer.connection_errors = (socket.error, OSError,)
+        consumer.conninfo = consumer.connection
         return consumer
         return consumer
 
 
     def test_taskbuckets_defaultdict(self):
     def test_taskbuckets_defaultdict(self):
@@ -56,16 +57,16 @@ class test_Consumer(AppCase):
     def test_sets_heartbeat(self):
     def test_sets_heartbeat(self):
         c = self.get_consumer(amqheartbeat=10)
         c = self.get_consumer(amqheartbeat=10)
         self.assertEqual(c.amqheartbeat, 10)
         self.assertEqual(c.amqheartbeat, 10)
-        self.app.conf.BROKER_HEARTBEAT = 20
+        self.app.conf.broker_heartbeat = 20
         c = self.get_consumer(amqheartbeat=None)
         c = self.get_consumer(amqheartbeat=None)
         self.assertEqual(c.amqheartbeat, 20)
         self.assertEqual(c.amqheartbeat, 20)
 
 
     def test_gevent_bug_disables_connection_timeout(self):
     def test_gevent_bug_disables_connection_timeout(self):
         with patch('celery.worker.consumer._detect_environment') as de:
         with patch('celery.worker.consumer._detect_environment') as de:
             de.return_value = 'gevent'
             de.return_value = 'gevent'
-            self.app.conf.BROKER_CONNECTION_TIMEOUT = 33.33
+            self.app.conf.broker_connection_timeout = 33.33
             self.get_consumer()
             self.get_consumer()
-            self.assertIsNone(self.app.conf.BROKER_CONNECTION_TIMEOUT)
+            self.assertIsNone(self.app.conf.broker_connection_timeout)
 
 
     def test_limit_task(self):
     def test_limit_task(self):
         c = self.get_consumer()
         c = self.get_consumer()
@@ -116,7 +117,7 @@ class test_Consumer(AppCase):
             sleep.assert_called_with(1)
             sleep.assert_called_with(1)
 
 
     def test_no_retry_raises_error(self):
     def test_no_retry_raises_error(self):
-        self.app.conf.BROKER_CONNECTION_RETRY = False
+        self.app.conf.broker_connection_retry = False
         c = self.get_consumer()
         c = self.get_consumer()
         c.blueprint.start.side_effect = socket.error()
         c.blueprint.start.side_effect = socket.error()
         with self.assertRaises(socket.error):
         with self.assertRaises(socket.error):
@@ -280,8 +281,8 @@ class test_Mingle(AppCase):
 
 
 
 
 def _amqp_connection():
 def _amqp_connection():
-    connection = ContextMock()
-    connection.return_value = ContextMock()
+    connection = ContextMock(name='Connection')
+    connection.return_value = ContextMock(name='connection')
     connection.return_value.transport.driver_type = 'amqp'
     connection.return_value.transport.driver_type = 'amqp'
     return connection
     return connection
 
 

+ 3 - 3
celery/tests/worker/test_control.py

@@ -517,7 +517,7 @@ class test_ControlPanel(AppCase):
         with self.assertRaises(ValueError):
         with self.assertRaises(ValueError):
             panel.handle('pool_restart', {'reloader': _reload})
             panel.handle('pool_restart', {'reloader': _reload})
 
 
-        self.app.conf.CELERYD_POOL_RESTARTS = True
+        self.app.conf.worker_pool_restarts = True
         panel.handle('pool_restart', {'reloader': _reload})
         panel.handle('pool_restart', {'reloader': _reload})
         self.assertTrue(consumer.controller.pool.restart.called)
         self.assertTrue(consumer.controller.pool.restart.called)
         consumer.reset_rate_limits.assert_called_with()
         consumer.reset_rate_limits.assert_called_with()
@@ -538,7 +538,7 @@ class test_ControlPanel(AppCase):
         _import = consumer.controller.app.loader.import_from_cwd = Mock()
         _import = consumer.controller.app.loader.import_from_cwd = Mock()
         _reload = Mock()
         _reload = Mock()
 
 
-        self.app.conf.CELERYD_POOL_RESTARTS = True
+        self.app.conf.worker_pool_restarts = True
         panel.handle('pool_restart', {'modules': ['foo', 'bar'],
         panel.handle('pool_restart', {'modules': ['foo', 'bar'],
                                       'reloader': _reload})
                                       'reloader': _reload})
 
 
@@ -563,7 +563,7 @@ class test_ControlPanel(AppCase):
         _import = panel.app.loader.import_from_cwd = Mock()
         _import = panel.app.loader.import_from_cwd = Mock()
         _reload = Mock()
         _reload = Mock()
 
 
-        self.app.conf.CELERYD_POOL_RESTARTS = True
+        self.app.conf.worker_pool_restarts = True
         with patch.dict(sys.modules, {'foo': None}):
         with patch.dict(sys.modules, {'foo': None}):
             panel.handle('pool_restart', {'modules': ['foo'],
             panel.handle('pool_restart', {'modules': ['foo'],
                                           'reload': False,
                                           'reload': False,

+ 1 - 1
celery/tests/worker/test_request.py

@@ -210,7 +210,7 @@ class MockEventDispatcher(object):
 class test_Request(AppCase):
 class test_Request(AppCase):
 
 
     def setup(self):
     def setup(self):
-        self.app.conf.CELERY_RESULT_SERIALIZER = 'pickle'
+        self.app.conf.result_serializer = 'pickle'
 
 
         @self.app.task(shared=False)
         @self.app.task(shared=False)
         def add(x, y, **kw_):
         def add(x, y, **kw_):

+ 4 - 4
celery/tests/worker/test_worker.py

@@ -587,12 +587,12 @@ class test_Consumer(AppCase):
             pp('+ BLUEPRINT START 1')
             pp('+ BLUEPRINT START 1')
             l.blueprint.start(l)
             l.blueprint.start(l)
             pp('- BLUEPRINT START 1')
             pp('- BLUEPRINT START 1')
-            p = l.app.conf.BROKER_CONNECTION_RETRY
-            l.app.conf.BROKER_CONNECTION_RETRY = False
+            p = l.app.conf.broker_connection_retry
+            l.app.conf.broker_connection_retry = False
             pp('+ BLUEPRINT START 2')
             pp('+ BLUEPRINT START 2')
             l.blueprint.start(l)
             l.blueprint.start(l)
             pp('- BLUEPRINT START 2')
             pp('- BLUEPRINT START 2')
-            l.app.conf.BROKER_CONNECTION_RETRY = p
+            l.app.conf.broker_connection_retry = p
             pp('+ BLUEPRINT RESTART')
             pp('+ BLUEPRINT RESTART')
             l.blueprint.restart(l)
             l.blueprint.restart(l)
             pp('- BLUEPRINT RESTART')
             pp('- BLUEPRINT RESTART')
@@ -825,7 +825,7 @@ class test_WorkController(AppCase):
         self.worker.on_consumer_ready(Mock())
         self.worker.on_consumer_ready(Mock())
 
 
     def test_setup_queues_worker_direct(self):
     def test_setup_queues_worker_direct(self):
-        self.app.conf.CELERY_WORKER_DIRECT = True
+        self.app.conf.worker_direct = True
         self.app.amqp.__dict__['queues'] = Mock()
         self.app.amqp.__dict__['queues'] = Mock()
         self.worker.setup_queues({})
         self.worker.setup_queues({})
         self.app.amqp.queues.select_add.assert_called_with(
         self.app.amqp.queues.select_add.assert_called_with(

+ 10 - 1
celery/utils/functional.py

@@ -16,6 +16,7 @@ from functools import partial, wraps
 from inspect import getargspec, isfunction
 from inspect import getargspec, isfunction
 from itertools import islice
 from itertools import islice
 
 
+from amqp import promise
 from kombu.utils import cached_property
 from kombu.utils import cached_property
 from kombu.utils.functional import lazy, maybe_evaluate, is_list, maybe_list
 from kombu.utils.functional import lazy, maybe_evaluate, is_list, maybe_list
 
 
@@ -210,6 +211,13 @@ def noop(*args, **kwargs):
     pass
     pass
 
 
 
 
+def evaluate_promises(it):
+    for value in it:
+        if isinstance(value, promise):
+            value = value()
+        yield value
+
+
 def first(predicate, it):
 def first(predicate, it):
     """Return the first element in `iterable` that `predicate` Gives a
     """Return the first element in `iterable` that `predicate` Gives a
     :const:`True` value for.
     :const:`True` value for.
@@ -218,7 +226,8 @@ def first(predicate, it):
 
 
     """
     """
     return next(
     return next(
-        (v for v in it if (predicate(v) if predicate else v is not None)),
+        (v for v in evaluate_promises(it) if (
+            predicate(v) if predicate is not None else v is not None)),
         None,
         None,
     )
     )
 
 

+ 35 - 42
celery/worker/__init__.py

@@ -46,15 +46,15 @@ SHUTDOWN_SOCKET_TIMEOUT = 5.0
 
 
 SELECT_UNKNOWN_QUEUE = """\
 SELECT_UNKNOWN_QUEUE = """\
 Trying to select queue subset of {0!r}, but queue {1} is not
 Trying to select queue subset of {0!r}, but queue {1} is not
-defined in the CELERY_QUEUES setting.
+defined in the `task_queues` setting.
 
 
 If you want to automatically declare unknown queues you can
 If you want to automatically declare unknown queues you can
-enable the CELERY_CREATE_MISSING_QUEUES setting.
+enable the `task_create_missing_queues` setting.
 """
 """
 
 
 DESELECT_UNKNOWN_QUEUE = """\
 DESELECT_UNKNOWN_QUEUE = """\
 Trying to deselect queue subset of {0!r}, but queue {1} is not
 Trying to deselect queue subset of {0!r}, but queue {1} is not
-defined in the CELERY_QUEUES setting.
+defined in the `task_queues` setting.
 """
 """
 
 
 
 
@@ -180,20 +180,20 @@ class WorkController(object):
         except KeyError as exc:
         except KeyError as exc:
             raise ImproperlyConfigured(
             raise ImproperlyConfigured(
                 DESELECT_UNKNOWN_QUEUE.format(exclude, exc))
                 DESELECT_UNKNOWN_QUEUE.format(exclude, exc))
-        if self.app.conf.CELERY_WORKER_DIRECT:
+        if self.app.conf.worker_direct:
             self.app.amqp.queues.select_add(worker_direct(self.hostname))
             self.app.amqp.queues.select_add(worker_direct(self.hostname))
 
 
     def setup_includes(self, includes):
     def setup_includes(self, includes):
         # Update celery_include to have all known task modules, so that we
         # Update celery_include to have all known task modules, so that we
         # ensure all task modules are imported in case an execv happens.
         # ensure all task modules are imported in case an execv happens.
-        prev = tuple(self.app.conf.CELERY_INCLUDE)
+        prev = tuple(self.app.conf.include)
         if includes:
         if includes:
             prev += tuple(includes)
             prev += tuple(includes)
             [self.app.loader.import_task_module(m) for m in includes]
             [self.app.loader.import_task_module(m) for m in includes]
         self.include = includes
         self.include = includes
         task_modules = {task.__class__.__module__
         task_modules = {task.__class__.__module__
                         for task in values(self.app.tasks)}
                         for task in values(self.app.tasks)}
-        self.app.conf.CELERY_INCLUDE = tuple(set(prev) | task_modules)
+        self.app.conf.include = tuple(set(prev) | task_modules)
 
 
     def prepare_args(self, **kwargs):
     def prepare_args(self, **kwargs):
         return kwargs
         return kwargs
@@ -353,49 +353,42 @@ class WorkController(object):
                        max_tasks_per_child=None, prefetch_multiplier=None,
                        max_tasks_per_child=None, prefetch_multiplier=None,
                        disable_rate_limits=None, worker_lost_wait=None,
                        disable_rate_limits=None, worker_lost_wait=None,
                        max_memory_per_child=None, **_kw):
                        max_memory_per_child=None, **_kw):
+        either = self.app.either
         self.loglevel = loglevel
         self.loglevel = loglevel
         self.logfile = logfile
         self.logfile = logfile
-        self.concurrency = self._getopt('concurrency', concurrency)
-        self.send_events = self._getopt('send_events', send_events)
-        self.pool_cls = self._getopt('pool', pool_cls)
-        self.consumer_cls = self._getopt('consumer', consumer_cls)
-        self.timer_cls = self._getopt('timer', timer_cls)
-        self.timer_precision = self._getopt('timer_precision', timer_precision)
-        self.autoscaler_cls = self._getopt('autoscaler', autoscaler_cls)
-        self.autoreloader_cls = self._getopt('autoreloader', autoreloader_cls)
-        self.pool_putlocks = self._getopt('pool_putlocks', pool_putlocks)
-        self.pool_restarts = self._getopt('pool_restarts', pool_restarts)
-        self.force_execv = self._getopt('force_execv', force_execv)
-        self.state_db = self._getopt('state_db', state_db)
-        self.schedule_filename = self._getopt(
-            'schedule_filename', schedule_filename,
-        )
-        self.scheduler_cls = self._getopt(
-            'celerybeat_scheduler', scheduler_cls,
+
+        self.concurrency = either('worker_concurrency', concurrency)
+        self.send_events = either('worker_send_events', send_events)
+        self.pool_cls = either('worker_pool', pool_cls)
+        self.consumer_cls = either('worker_consumer', consumer_cls)
+        self.timer_cls = either('worker_timer', timer_cls)
+        self.timer_precision = either(
+            'worker_timer_precision', timer_precision,
         )
         )
-        self.task_time_limit = self._getopt(
-            'task_time_limit', task_time_limit,
+        self.autoscaler_cls = either('worker_autoscaler', autoscaler_cls)
+        self.autoreloader_cls = either('worker_autoreloader', autoreloader_cls)
+        self.pool_putlocks = either('worker_pool_putlocks', pool_putlocks)
+        self.pool_restarts = either('worker_pool_restarts', pool_restarts)
+        self.force_execv = either('worker_force_execv', force_execv)
+        self.state_db = either('worker_state_db', state_db)
+        self.schedule_filename = either(
+            'beat_schedule_filename', schedule_filename,
         )
         )
-        self.task_soft_time_limit = self._getopt(
+        self.scheduler_cls = either('beat_scheduler', scheduler_cls)
+        self.task_time_limit = either('task_time_limit', task_time_limit)
+        self.task_soft_time_limit = either(
             'task_soft_time_limit', task_soft_time_limit,
             'task_soft_time_limit', task_soft_time_limit,
         )
         )
-        self.max_tasks_per_child = self._getopt(
-            'max_tasks_per_child', max_tasks_per_child,
+        self.max_tasks_per_child = either(
+            'worker_max_tasks_per_child', max_tasks_per_child,
         )
         )
-        self.max_memory_per_child = self._getopt(
-            'max_memory_per_child', max_memory_per_child,
+        self.max_memory_per_child = either(
+            'worker_max_memory_per_child', max_memory_per_child,
         )
         )
-        self.prefetch_multiplier = int(self._getopt(
-            'prefetch_multiplier', prefetch_multiplier,
+        self.prefetch_multiplier = int(either(
+            'worker_prefetch_multiplier', prefetch_multiplier,
         ))
         ))
-        self.disable_rate_limits = self._getopt(
-            'disable_rate_limits', disable_rate_limits,
+        self.disable_rate_limits = either(
+            'worker_disable_rate_limits', disable_rate_limits,
         )
         )
-        self.worker_lost_wait = self._getopt(
-            'worker_lost_wait', worker_lost_wait,
-        )
-
-    def _getopt(self, key, value):
-        if value is not None:
-            return value
-        return self.app.conf.find_value_for_key(key, namespace='celeryd')
+        self.worker_lost_wait = either('worker_lost_wait', worker_lost_wait)

+ 2 - 2
celery/worker/components.py

@@ -31,7 +31,7 @@ use standalone beat instead.\
 """
 """
 
 
 W_POOL_SETTING = """
 W_POOL_SETTING = """
-The CELERYD_POOL setting should not be used to select the eventlet/gevent
+The worker_pool setting should not be used to select the eventlet/gevent
 pools, instead you *must use the -P* argument so that patches are applied
 pools, instead you *must use the -P* argument so that patches are applied
 as early as possible.
 as early as possible.
 """
 """
@@ -138,7 +138,7 @@ class Pool(bootsteps.StartStopStep):
             w.pool.terminate()
             w.pool.terminate()
 
 
     def create(self, w, semaphore=None, max_restarts=None):
     def create(self, w, semaphore=None, max_restarts=None):
-        if w.app.conf.CELERYD_POOL in ('eventlet', 'gevent'):
+        if w.app.conf.worker_pool in ('eventlet', 'gevent'):
             warnings.warn(UserWarning(W_POOL_SETTING))
             warnings.warn(UserWarning(W_POOL_SETTING))
         threaded = not w.use_eventloop or IS_WINDOWS
         threaded = not w.use_eventloop or IS_WINDOWS
         procs = w.min_concurrency
         procs = w.min_concurrency

+ 10 - 10
celery/worker/consumer.py

@@ -185,7 +185,7 @@ class Consumer(object):
         self._limit_order = 0
         self._limit_order = 0
         self.on_task_request = on_task_request
         self.on_task_request = on_task_request
         self.on_task_message = set()
         self.on_task_message = set()
-        self.amqheartbeat_rate = self.app.conf.BROKER_HEARTBEAT_CHECKRATE
+        self.amqheartbeat_rate = self.app.conf.broker_heartbeat_checkrate
         self.disable_rate_limits = disable_rate_limits
         self.disable_rate_limits = disable_rate_limits
         self.initial_prefetch_count = initial_prefetch_count
         self.initial_prefetch_count = initial_prefetch_count
         self.prefetch_multiplier = prefetch_multiplier
         self.prefetch_multiplier = prefetch_multiplier
@@ -199,7 +199,7 @@ class Consumer(object):
         if self.hub:
         if self.hub:
             self.amqheartbeat = amqheartbeat
             self.amqheartbeat = amqheartbeat
             if self.amqheartbeat is None:
             if self.amqheartbeat is None:
-                self.amqheartbeat = self.app.conf.BROKER_HEARTBEAT
+                self.amqheartbeat = self.app.conf.broker_heartbeat
         else:
         else:
             self.amqheartbeat = 0
             self.amqheartbeat = 0
 
 
@@ -210,7 +210,7 @@ class Consumer(object):
             # there's a gevent bug that causes timeouts to not be reset,
             # there's a gevent bug that causes timeouts to not be reset,
             # so if the connection timeout is exceeded once, it can NEVER
             # so if the connection timeout is exceeded once, it can NEVER
             # connect again.
             # connect again.
-            self.app.conf.BROKER_CONNECTION_TIMEOUT = None
+            self.app.conf.broker_connection_timeout = None
 
 
         self.steps = []
         self.steps = []
         self.blueprint = self.Blueprint(
         self.blueprint = self.Blueprint(
@@ -279,7 +279,7 @@ class Consumer(object):
             except self.connection_errors as exc:
             except self.connection_errors as exc:
                 # If we're not retrying connections, no need to catch
                 # If we're not retrying connections, no need to catch
                 # connection errors
                 # connection errors
-                if not self.app.conf.BROKER_CONNECTION_RETRY:
+                if not self.app.conf.broker_connection_retry:
                     raise
                     raise
                 if isinstance(exc, OSError) and exc.errno == errno.EMFILE:
                 if isinstance(exc, OSError) and exc.errno == errno.EMFILE:
                     raise  # Too many open files
                     raise  # Too many open files
@@ -354,7 +354,7 @@ class Consumer(object):
         """Establish the broker connection.
         """Establish the broker connection.
 
 
         Will retry establishing the connection if the
         Will retry establishing the connection if the
-        :setting:`BROKER_CONNECTION_RETRY` setting is enabled
+        :setting:`broker_connection_retry` setting is enabled
 
 
         """
         """
         conn = self.app.connection(heartbeat=self.amqheartbeat)
         conn = self.app.connection(heartbeat=self.amqheartbeat)
@@ -369,13 +369,13 @@ class Consumer(object):
 
 
         # remember that the connection is lazy, it won't establish
         # remember that the connection is lazy, it won't establish
         # until needed.
         # until needed.
-        if not self.app.conf.BROKER_CONNECTION_RETRY:
+        if not self.app.conf.broker_connection_retry:
             # retry disabled, just call connect directly.
             # retry disabled, just call connect directly.
             conn.connect()
             conn.connect()
             return conn
             return conn
 
 
         conn = conn.ensure_connection(
         conn = conn.ensure_connection(
-            _error_handler, self.app.conf.BROKER_CONNECTION_MAX_RETRIES,
+            _error_handler, self.app.conf.broker_connection_max_retries,
             callback=maybe_shutdown,
             callback=maybe_shutdown,
         )
         )
         if self.hub:
         if self.hub:
@@ -395,7 +395,7 @@ class Consumer(object):
         cset = self.task_consumer
         cset = self.task_consumer
         queues = self.app.amqp.queues
         queues = self.app.amqp.queues
         # Must use in' here, as __missing__ will automatically
         # Must use in' here, as __missing__ will automatically
-        # create queues when CELERY_CREATE_MISSING_QUEUES is enabled.
+        # create queues when :setting:`task_create_missing_queues` is enabled.
         # (Issue #1079)
         # (Issue #1079)
         if queue in queues:
         if queue in queues:
             q = queues[queue]
             q = queues[queue]
@@ -667,7 +667,7 @@ class Agent(bootsteps.StartStopStep):
     requires = (Connection,)
     requires = (Connection,)
 
 
     def __init__(self, c, **kwargs):
     def __init__(self, c, **kwargs):
-        self.agent_cls = self.enabled = c.app.conf.CELERYD_AGENT
+        self.agent_cls = self.enabled = c.app.conf.worker_agent
 
 
     def create(self, c):
     def create(self, c):
         agent = c.agent = self.instantiate(self.agent_cls, c.connection)
         agent = c.agent = self.instantiate(self.agent_cls, c.connection)
@@ -685,7 +685,7 @@ class Control(bootsteps.StartStopStep):
         self.shutdown = self.box.shutdown
         self.shutdown = self.box.shutdown
 
 
     def include_if(self, c):
     def include_if(self, c):
-        return (c.app.conf.CELERY_ENABLE_REMOTE_CONTROL and
+        return (c.app.conf.worker_enable_remote_control and
                 c.conninfo.supports_exchange_type('fanout'))
                 c.conninfo.supports_exchange_type('fanout'))
 
 
 
 

+ 1 - 1
celery/worker/control.py

@@ -321,7 +321,7 @@ def pool_shrink(state, n=1, **kwargs):
 
 
 @Panel.register
 @Panel.register
 def pool_restart(state, modules=None, reload=False, reloader=None, **kwargs):
 def pool_restart(state, modules=None, reload=False, reloader=None, **kwargs):
-    if state.app.conf.CELERYD_POOL_RESTARTS:
+    if state.app.conf.worker_pool_restarts:
         state.consumer.controller.reload(modules, reload, reloader=reloader)
         state.consumer.controller.reload(modules, reload, reloader=reloader)
         return {'ok': 'reload started'}
         return {'ok': 'reload started'}
     else:
     else:

+ 1 - 1
celery/worker/request.py

@@ -420,7 +420,7 @@ class Request(object):
     @property
     @property
     def tzlocal(self):
     def tzlocal(self):
         if self._tzlocal is None:
         if self._tzlocal is None:
-            self._tzlocal = self.app.conf.CELERY_TIMEZONE
+            self._tzlocal = self.app.conf.timezone
         return self._tzlocal
         return self._tzlocal
 
 
     @property
     @property

File diff suppressed because it is too large
+ 379 - 266
docs/configuration.rst


+ 11 - 5
docs/django/first-steps-with-django.rst

@@ -76,7 +76,13 @@ but there's probably no reason for that when using Django.
 We also add the Django settings module as a configuration source
 We also add the Django settings module as a configuration source
 for Celery.  This means that you don't have to use multiple
 for Celery.  This means that you don't have to use multiple
 configuration files, and instead configure Celery directly
 configuration files, and instead configure Celery directly
-from the Django settings.
+from the Django settings; but you can also separate them if wanted.
+
+The uppercase namespace means that all Celery configuration options
+must be specified in uppercase instead of lowercase, and start with
+``CELERY_``, so e.g. the :setting:`task_always_eager`` setting
+becomes ``CELERY_TASK_ALWAYS_EAGER``, and the :setting:`broker_url`
+setting becomes ``CELERY_BROKER_URL``.
 
 
 You can pass the object directly here, but using a string is better since
 You can pass the object directly here, but using a string is better since
 then the worker doesn't have to serialize the object when using Windows
 then the worker doesn't have to serialize the object when using Windows
@@ -84,7 +90,7 @@ or execv:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    app.config_from_object('django.conf:settings')
+    app.config_from_object('django.conf:settings', namespace='CELERY_')
 
 
 Next, a common practice for reusable apps is to define all tasks
 Next, a common practice for reusable apps is to define all tasks
 in a separate ``tasks.py`` module, and Celery does have a way to
 in a separate ``tasks.py`` module, and Celery does have a way to
@@ -106,7 +112,7 @@ of your installed apps, following the ``tasks.py`` convention::
 
 
 
 
 This way you do not have to manually add the individual modules
 This way you do not have to manually add the individual modules
-to the :setting:`CELERY_IMPORTS` setting.  The ``lambda`` so that the
+to the :setting:`CELERY_IMPORTS <imports>` setting.  The ``lambda`` so that the
 autodiscovery can happen only when needed, and so that importing your
 autodiscovery can happen only when needed, and so that importing your
 module will not evaluate the Django settings object.
 module will not evaluate the Django settings object.
 
 
@@ -176,7 +182,7 @@ To use this with your project you need to follow these four steps:
     .. code-block:: python
     .. code-block:: python
 
 
         app.conf.update(
         app.conf.update(
-            CELERY_RESULT_BACKEND='djcelery.backends.database:DatabaseBackend',
+            result_backend='djcelery.backends.database:DatabaseBackend',
         )
         )
 
 
     For the cache backend you can use:
     For the cache backend you can use:
@@ -184,7 +190,7 @@ To use this with your project you need to follow these four steps:
     .. code-block:: python
     .. code-block:: python
 
 
         app.conf.update(
         app.conf.update(
-            CELERY_RESULT_BACKEND='djcelery.backends.cache:CacheBackend',
+            result_backend='djcelery.backends.cache:CacheBackend',
         )
         )
 
 
     If you have connected Celery to your Django settings then you can
     If you have connected Celery to your Django settings then you can

+ 5 - 5
docs/faq.rst

@@ -449,10 +449,10 @@ data.
 Note that this is not just something you should be aware of with Celery, for
 Note that this is not just something you should be aware of with Celery, for
 example also Django uses pickle for its cache client.
 example also Django uses pickle for its cache client.
 
 
-For the task messages you can set the :setting:`CELERY_TASK_SERIALIZER`
+For the task messages you can set the :setting:`task_serializer`
 setting to "json" or "yaml" instead of pickle.
 setting to "json" or "yaml" instead of pickle.
 
 
-Similarly for task results you can set :setting:`CELERY_RESULT_SERIALIZER`.
+Similarly for task results you can set :setting:`result_serializer`.
 
 
 For more details of the formats used and the lookup order when
 For more details of the formats used and the lookup order when
 checking which format to use for a task see :ref:`calling-serializers`
 checking which format to use for a task see :ref:`calling-serializers`
@@ -461,7 +461,7 @@ Can messages be encrypted?
 --------------------------
 --------------------------
 
 
 **Answer**: Some AMQP brokers supports using SSL (including RabbitMQ).
 **Answer**: Some AMQP brokers supports using SSL (including RabbitMQ).
-You can enable this using the :setting:`BROKER_USE_SSL` setting.
+You can enable this using the :setting:`broker_use_ssl` setting.
 
 
 It is also possible to add additional encryption and security to messages,
 It is also possible to add additional encryption and security to messages,
 if you have a need for this then you should contact the :ref:`mailing-list`.
 if you have a need for this then you should contact the :ref:`mailing-list`.
@@ -517,7 +517,7 @@ as a message. If you don't collect these results, they will build up and
 RabbitMQ will eventually run out of memory.
 RabbitMQ will eventually run out of memory.
 
 
 Results expire after 1 day by default.  It may be a good idea
 Results expire after 1 day by default.  It may be a good idea
-to lower this value by configuring the :setting:`CELERY_TASK_RESULT_EXPIRES`
+to lower this value by configuring the :setting:`result_expires`
 setting.
 setting.
 
 
 If you don't use the results for a task, make sure you set the
 If you don't use the results for a task, make sure you set the
@@ -565,7 +565,7 @@ Tasks
 How can I reuse the same connection when calling tasks?
 How can I reuse the same connection when calling tasks?
 -------------------------------------------------------
 -------------------------------------------------------
 
 
-**Answer**: See the :setting:`BROKER_POOL_LIMIT` setting.
+**Answer**: See the :setting:`broker_pool_limit` setting.
 The connection pool is enabled by default since version 2.5.
 The connection pool is enabled by default since version 2.5.
 
 
 .. _faq-sudo-subprocess:
 .. _faq-sudo-subprocess:

+ 1 - 1
docs/getting-started/brokers/beanstalk.rst

@@ -34,7 +34,7 @@ Configuration
 Configuration is easy, set the transport, and configure the location of
 Configuration is easy, set the transport, and configure the location of
 your Beanstalk database::
 your Beanstalk database::
 
 
-    BROKER_URL = 'beanstalk://localhost:11300'
+    broker_url = 'beanstalk://localhost:11300'
 
 
 Where the URL is in the format of::
 Where the URL is in the format of::
 
 

+ 1 - 1
docs/getting-started/brokers/couchdb.rst

@@ -32,7 +32,7 @@ Configuration
 Configuration is easy, set the transport, and configure the location of
 Configuration is easy, set the transport, and configure the location of
 your CouchDB database::
 your CouchDB database::
 
 
-    BROKER_URL = 'couchdb://localhost:5984/database_name'
+    broker_url = 'couchdb://localhost:5984/database_name'
 
 
 Where the URL is in the format of::
 Where the URL is in the format of::
 
 

+ 1 - 1
docs/getting-started/brokers/django.rst

@@ -26,7 +26,7 @@ configuration values.
 
 
 #. Set your broker transport::
 #. Set your broker transport::
 
 
-    BROKER_URL = 'django://'
+    CELERY_BROKER_URL = 'django://'
 
 
 #. Add :mod:`kombu.transport.django` to `INSTALLED_APPS`::
 #. Add :mod:`kombu.transport.django` to `INSTALLED_APPS`::
 
 

+ 1 - 1
docs/getting-started/brokers/ironmq.rst

@@ -31,7 +31,7 @@ First, you'll need to import the iron_celery library right after you import Cele
 
 
 You have to specify IronMQ in the broker URL::
 You have to specify IronMQ in the broker URL::
 
 
-    BROKER_URL = 'ironmq://ABCDEFGHIJKLMNOPQRST:ZYXK7NiynGlTogH8Nj+P9nlE73sq3@'
+    broker_url = 'ironmq://ABCDEFGHIJKLMNOPQRST:ZYXK7NiynGlTogH8Nj+P9nlE73sq3@'
 
 
 where the URL format is::
 where the URL format is::
 
 

+ 1 - 1
docs/getting-started/brokers/mongodb.rst

@@ -32,7 +32,7 @@ Configuration
 Configuration is easy, set the transport, and configure the location of
 Configuration is easy, set the transport, and configure the location of
 your MongoDB database::
 your MongoDB database::
 
 
-    BROKER_URL = 'mongodb://localhost:27017/database_name'
+    broker_url = 'mongodb://localhost:27017/database_name'
 
 
 Where the URL is in the format of::
 Where the URL is in the format of::
 
 

+ 1 - 1
docs/getting-started/brokers/rabbitmq.rst

@@ -16,7 +16,7 @@ the broker instance you want to use:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    BROKER_URL = 'amqp://guest:guest@localhost:5672//'
+    broker_url = 'amqp://guest:guest@localhost:5672//'
 
 
 For a description of broker URLs and a full list of the
 For a description of broker URLs and a full list of the
 various broker configuration options available to Celery,
 various broker configuration options available to Celery,

+ 7 - 7
docs/getting-started/brokers/redis.rst

@@ -25,7 +25,7 @@ Configuration
 Configuration is easy, just configure the location of
 Configuration is easy, just configure the location of
 your Redis database::
 your Redis database::
 
 
-    BROKER_URL = 'redis://localhost:6379/0'
+    broker_url = 'redis://localhost:6379/0'
 
 
 Where the URL is in the format of::
 Where the URL is in the format of::
 
 
@@ -47,9 +47,9 @@ The visibility timeout defines the number of seconds to wait
 for the worker to acknowledge the task before the message is redelivered
 for the worker to acknowledge the task before the message is redelivered
 to another worker.  Be sure to see :ref:`redis-caveats` below.
 to another worker.  Be sure to see :ref:`redis-caveats` below.
 
 
-This option is set via the :setting:`BROKER_TRANSPORT_OPTIONS` setting::
+This option is set via the :setting:`broker_transport_options` setting::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'visibility_timeout': 3600}  # 1 hour.
+    broker_transport_options = {'visibility_timeout': 3600}  # 1 hour.
 
 
 The default visibility timeout for Redis is 1 hour.
 The default visibility timeout for Redis is 1 hour.
 
 
@@ -61,7 +61,7 @@ Results
 If you also want to store the state and return values of tasks in Redis,
 If you also want to store the state and return values of tasks in Redis,
 you should configure these settings::
 you should configure these settings::
 
 
-    CELERY_RESULT_BACKEND = 'redis://localhost:6379/0'
+    result_backend = 'redis://localhost:6379/0'
 
 
 For a complete list of options supported by the Redis result backend, see
 For a complete list of options supported by the Redis result backend, see
 :ref:`conf-redis-result-backend`
 :ref:`conf-redis-result-backend`
@@ -81,7 +81,7 @@ Broadcast messages will be seen by all virtual hosts by default.
 You have to set a transport option to prefix the messages so that
 You have to set a transport option to prefix the messages so that
 they will only be received by the active virtual host::
 they will only be received by the active virtual host::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'fanout_prefix': True}
+    broker_transport_options = {'fanout_prefix': True}
 
 
 Note that you will not be able to communicate with workers running older
 Note that you will not be able to communicate with workers running older
 versions or workers that does not have this setting enabled.
 versions or workers that does not have this setting enabled.
@@ -99,7 +99,7 @@ Workers will receive all task related events by default.
 To avoid this you must set the ``fanout_patterns`` fanout option so that
 To avoid this you must set the ``fanout_patterns`` fanout option so that
 the workers may only subscribe to worker related events::
 the workers may only subscribe to worker related events::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'fanout_patterns': True}
+    broker_transport_options = {'fanout_patterns': True}
 
 
 Note that this change is backward incompatible so all workers in the
 Note that this change is backward incompatible so all workers in the
 cluster must have this option enabled, or else they will not be able to
 cluster must have this option enabled, or else they will not be able to
@@ -131,7 +131,7 @@ as this is a concept separate from ETA/countdown.
 You can increase this timeout by configuring a transport option
 You can increase this timeout by configuring a transport option
 with the same name::
 with the same name::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'visibility_timeout': 43200}
+    broker_transport_options = {'visibility_timeout': 43200}
 
 
 The value must be an int describing the number of seconds.
 The value must be an int describing the number of seconds.
 
 

+ 6 - 6
docs/getting-started/brokers/sqlalchemy.rst

@@ -24,9 +24,9 @@ Configuration
 Celery needs to know the location of your database, which should be the usual
 Celery needs to know the location of your database, which should be the usual
 SQLAlchemy connection string, but with 'sqla+' prepended to it::
 SQLAlchemy connection string, but with 'sqla+' prepended to it::
 
 
-    BROKER_URL = 'sqla+sqlite:///celerydb.sqlite'
+    broker_url = 'sqla+sqlite:///celerydb.sqlite'
 
 
-This transport uses only the :setting:`BROKER_URL` setting, which have to be
+This transport uses only the :setting:`broker_url` setting, which have to be
 an SQLAlchemy database URI.
 an SQLAlchemy database URI.
 
 
 
 
@@ -37,16 +37,16 @@ Here's a list of examples using a selection of other `SQLAlchemy Connection Stri
 .. code-block:: python
 .. code-block:: python
 
 
     # sqlite (filename)
     # sqlite (filename)
-    BROKER_URL = 'sqla+sqlite:///celerydb.sqlite'
+    broker_url = 'sqla+sqlite:///celerydb.sqlite'
 
 
     # mysql
     # mysql
-    BROKER_URL = 'sqla+mysql://scott:tiger@localhost/foo'
+    broker_url = 'sqla+mysql://scott:tiger@localhost/foo'
 
 
     # postgresql
     # postgresql
-    BROKER_URL = 'sqla+postgresql://scott:tiger@localhost/mydatabase'
+    broker_url = 'sqla+postgresql://scott:tiger@localhost/mydatabase'
 
 
     # oracle
     # oracle
-    BROKER_URL = 'sqla+oracle://scott:tiger@127.0.0.1:1521/sidname'
+    broker_url = 'sqla+oracle://scott:tiger@127.0.0.1:1521/sidname'
 
 
 .. _`SQLAlchemy: Supported Databases`:
 .. _`SQLAlchemy: Supported Databases`:
     http://www.sqlalchemy.org/docs/core/engines.html#supported-databases
     http://www.sqlalchemy.org/docs/core/engines.html#supported-databases

+ 10 - 10
docs/getting-started/brokers/sqs.rst

@@ -32,7 +32,7 @@ Configuration
 
 
 You have to specify SQS in the broker URL::
 You have to specify SQS in the broker URL::
 
 
-    BROKER_URL = 'sqs://ABCDEFGHIJKLMNOPQRST:ZYXK7NiynGlTogH8Nj+P9nlE73sq3@'
+    broker_url = 'sqs://ABCDEFGHIJKLMNOPQRST:ZYXK7NiynGlTogH8Nj+P9nlE73sq3@'
 
 
 where the URL format is::
 where the URL format is::
 
 
@@ -57,9 +57,9 @@ Region
 ------
 ------
 
 
 The default region is ``us-east-1`` but you can select another region
 The default region is ``us-east-1`` but you can select another region
-by configuring the :setting:`BROKER_TRANSPORT_OPTIONS` setting::
+by configuring the :setting:`broker_transport_options` setting::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'region': 'eu-west-1'}
+    broker_transport_options = {'region': 'eu-west-1'}
 
 
 .. seealso::
 .. seealso::
 
 
@@ -74,9 +74,9 @@ The visibility timeout defines the number of seconds to wait
 for the worker to acknowledge the task before the message is redelivered
 for the worker to acknowledge the task before the message is redelivered
 to another worker.  Also see caveats below.
 to another worker.  Also see caveats below.
 
 
-This option is set via the :setting:`BROKER_TRANSPORT_OPTIONS` setting::
+This option is set via the :setting:`broker_transport_options` setting::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'visibility_timeout': 3600}  # 1 hour.
+    broker_transport_options = {'visibility_timeout': 3600}  # 1 hour.
 
 
 The default visibility timeout is 30 seconds.
 The default visibility timeout is 30 seconds.
 
 
@@ -91,10 +91,10 @@ sleep for one second whenever there are no more messages to read.
 You should note that **more frequent polling is also more expensive, so increasing
 You should note that **more frequent polling is also more expensive, so increasing
 the polling interval can save you money**.
 the polling interval can save you money**.
 
 
-The polling interval can be set via the :setting:`BROKER_TRANSPORT_OPTIONS`
+The polling interval can be set via the :setting:`broker_transport_options`
 setting::
 setting::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'polling_interval': 0.3}
+    broker_transport_options = {'polling_interval': 0.3}
 
 
 Very frequent polling intervals can cause *busy loops*, which results in the
 Very frequent polling intervals can cause *busy loops*, which results in the
 worker using a lot of CPU time.  If you need sub-millisecond precision you
 worker using a lot of CPU time.  If you need sub-millisecond precision you
@@ -106,9 +106,9 @@ Queue Prefix
 
 
 By default Celery will not assign any prefix to the queue names,
 By default Celery will not assign any prefix to the queue names,
 If you have other services using SQS you can configure it do so
 If you have other services using SQS you can configure it do so
-using the :setting:`BROKER_TRANSPORT_OPTIONS` setting::
+using the :setting:`broker_transport_options` setting::
 
 
-    BROKER_TRANSPORT_OPTIONS = {'queue_name_prefix': 'celery-'}
+    broker_transport_options = {'queue_name_prefix': 'celery-'}
 
 
 
 
 .. _sqs-caveats:
 .. _sqs-caveats:
@@ -137,7 +137,7 @@ Caveats
     The maximum visibility timeout supported by AWS as of this writing
     The maximum visibility timeout supported by AWS as of this writing
     is 12 hours (43200 seconds)::
     is 12 hours (43200 seconds)::
 
 
-        BROKER_TRANSPORT_OPTIONS = {'visibility_timeout': 43200}
+        broker_transport_options = {'visibility_timeout': 43200}
 
 
 - SQS does not yet support worker remote control commands.
 - SQS does not yet support worker remote control commands.
 
 

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

@@ -225,7 +225,7 @@ built-in result backends to choose from: `SQLAlchemy`_/`Django`_ ORM,
 
 
 For this example you will use the `rpc` result backend, which sends states
 For this example you will use the `rpc` result backend, which sends states
 back as transient messages.  The backend is specified via the ``backend`` argument to
 back as transient messages.  The backend is specified via the ``backend`` argument to
-:class:`@Celery`, (or via the :setting:`CELERY_RESULT_BACKEND` setting if
+:class:`@Celery`, (or via the :setting:`task_result_backend` setting if
 you choose to use a configuration module)::
 you choose to use a configuration module)::
 
 
     app = Celery('tasks', backend='rpc://', broker='amqp://')
     app = Celery('tasks', backend='rpc://', broker='amqp://')
@@ -289,22 +289,22 @@ can be configured. You can read about the options in the
 The configuration can be set on the app directly or by using a dedicated
 The configuration can be set on the app directly or by using a dedicated
 configuration module.
 configuration module.
 As an example you can configure the default serializer used for serializing
 As an example you can configure the default serializer used for serializing
-task payloads by changing the :setting:`CELERY_TASK_SERIALIZER` setting:
+task payloads by changing the :setting:`task_serializer` setting:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    app.conf.CELERY_TASK_SERIALIZER = 'json'
+    app.conf.task_serializer = 'json'
 
 
 If you are configuring many settings at once you can use ``update``:
 If you are configuring many settings at once you can use ``update``:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
     app.conf.update(
     app.conf.update(
-        CELERY_TASK_SERIALIZER='json',
-        CELERY_ACCEPT_CONTENT=['json'],  # Ignore other content
-        CELERY_RESULT_SERIALIZER='json',
-        CELERY_TIMEZONE='Europe/Oslo',
-        CELERY_ENABLE_UTC=True,
+        task_serializer='json',
+        accept_content=['json'],  # Ignore other content
+        result_serializer='json',
+        timezone='Europe/Oslo',
+        enable_utc=True,
     )
     )
 
 
 For larger projects using a dedicated configuration module is useful,
 For larger projects using a dedicated configuration module is useful,
@@ -332,14 +332,14 @@ current directory or on the Python path, it could look like this:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    BROKER_URL = 'amqp://'
-    CELERY_RESULT_BACKEND = 'rpc://'
+    broker_url = 'amqp://'
+    result_backend = 'rpc://'
 
 
-    CELERY_TASK_SERIALIZER = 'json'
-    CELERY_RESULT_SERIALIZER = 'json'
-    CELERY_ACCEPT_CONTENT=['json']
-    CELERY_TIMEZONE = 'Europe/Oslo'
-    CELERY_ENABLE_UTC = True
+    task_serializer = 'json'
+    result_serializer = 'json'
+    accept_content = ['json']
+    timezone = 'Europe/Oslo'
+    enable_utc = True
 
 
 To verify that your configuration file works properly, and doesn't
 To verify that your configuration file works properly, and doesn't
 contain any syntax errors, you can try to import it:
 contain any syntax errors, you can try to import it:
@@ -357,7 +357,7 @@ route a misbehaving task to a dedicated queue:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    CELERY_ROUTES = {
+    task_routes = {
         'tasks.add': 'low-priority',
         'tasks.add': 'low-priority',
     }
     }
 
 
@@ -369,7 +369,7 @@ instead, so that only 10 tasks of this type can be processed in a minute
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    CELERY_ANNOTATIONS = {
+    task_annotations = {
         'tasks.add': {'rate_limit': '10/m'}
         'tasks.add': {'rate_limit': '10/m'}
     }
     }
 
 
@@ -384,7 +384,7 @@ for the task at runtime:
         new rate limit set successfully
         new rate limit set successfully
 
 
 See :ref:`guide-routing` to read more about task routing,
 See :ref:`guide-routing` to read more about task routing,
-and the :setting:`CELERY_ANNOTATIONS` setting for more about annotations,
+and the :setting:`task_annotations` setting for more about annotations,
 or :ref:`guide-monitoring` for more about remote control commands,
 or :ref:`guide-monitoring` for more about remote control commands,
 and how to monitor what your workers are doing.
 and how to monitor what your workers are doing.
 
 
@@ -435,7 +435,7 @@ the task id after all).
     Enabling this option will force the worker to skip updating
     Enabling this option will force the worker to skip updating
     states.
     states.
 
 
-2) Make sure the :setting:`CELERY_IGNORE_RESULT` setting is not enabled.
+2) Make sure the :setting:`task_ignore_result` setting is not enabled.
 
 
 3) Make sure that you do not have any old workers still running.
 3) Make sure that you do not have any old workers still running.
 
 

+ 5 - 5
docs/getting-started/next-steps.rst

@@ -369,7 +369,7 @@ states. The stages of a typical task can be::
     PENDING -> STARTED -> SUCCESS
     PENDING -> STARTED -> SUCCESS
 
 
 The started state is a special state that is only recorded if the
 The started state is a special state that is only recorded if the
-:setting:`CELERY_TRACK_STARTED` setting is enabled, or if the
+:setting:`task_track_started` setting is enabled, or if the
 ``@task(track_started=True)`` option is set for the task.
 ``@task(track_started=True)`` option is set for the task.
 
 
 The pending state is actually not a recorded state, but rather
 The pending state is actually not a recorded state, but rather
@@ -605,13 +605,13 @@ Routing
 Celery supports all of the routing facilities provided by AMQP,
 Celery supports all of the routing facilities provided by AMQP,
 but it also supports simple routing where messages are sent to named queues.
 but it also supports simple routing where messages are sent to named queues.
 
 
-The :setting:`CELERY_ROUTES` setting enables you to route tasks by name
+The :setting:`task_routes` setting enables you to route tasks by name
 and keep everything centralized in one location:
 and keep everything centralized in one location:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
     app.conf.update(
     app.conf.update(
-        CELERY_ROUTES = {
+        task_routes = {
             'proj.tasks.add': {'queue': 'hipri'},
             'proj.tasks.add': {'queue': 'hipri'},
         },
         },
     )
     )
@@ -732,11 +732,11 @@ All times and dates, internally and in messages uses the UTC timezone.
 When the worker receives a message, for example with a countdown set it
 When the worker receives a message, for example with a countdown set it
 converts that UTC time to local time.  If you wish to use
 converts that UTC time to local time.  If you wish to use
 a different timezone than the system timezone then you must
 a different timezone than the system timezone then you must
-configure that using the :setting:`CELERY_TIMEZONE` setting:
+configure that using the :setting:`timezone` setting:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    app.conf.CELERY_TIMEZONE = 'Europe/London'
+    app.conf.timezone = 'Europe/London'
 
 
 Optimization
 Optimization
 ============
 ============

+ 1 - 1
docs/glossary.rst

@@ -74,7 +74,7 @@ Glossary
 
 
     prefetch multiplier
     prefetch multiplier
         The :term:`prefetch count` is configured by using the
         The :term:`prefetch count` is configured by using the
-        :setting:`CELERYD_PREFETCH_MULTIPLIER` setting, which is multiplied
+        :setting:`worker_prefetch_multiplier` setting, which is multiplied
         by the number of pool slots (threads/processes/greenthreads).
         by the number of pool slots (threads/processes/greenthreads).
 
 
     prefetch count
     prefetch count

+ 5 - 5
docs/internals/app-overview.rst

@@ -57,8 +57,8 @@ Getting access to the configuration:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    celery.conf.CELERY_ALWAYS_EAGER = True
-    celery.conf["CELERY_ALWAYS_EAGER"] = True
+    celery.conf.task_always_eager = True
+    celery.conf["task_always_eager"] = True
 
 
 
 
 Controlling workers::
 Controlling workers::
@@ -135,15 +135,15 @@ Aliases (Pending deprecation)
 * celery.conf.* -> {app.conf}
 * celery.conf.* -> {app.conf}
 
 
     **NOTE**: All configuration keys are now named the same
     **NOTE**: All configuration keys are now named the same
-    as in the configuration. So the key "CELERY_ALWAYS_EAGER"
+    as in the configuration. So the key "task_always_eager"
     is accessed as::
     is accessed as::
 
 
-        >>> app.conf.CELERY_ALWAYS_EAGER
+        >>> app.conf.task_always_eager
 
 
     instead of::
     instead of::
 
 
         >>> from celery import conf
         >>> from celery import conf
-        >>> conf.ALWAYS_EAGER
+        >>> conf.always_eager
 
 
     * .get_queues -> {app.amqp.get_queues}
     * .get_queues -> {app.amqp.get_queues}
 
 

+ 23 - 23
docs/internals/deprecation.rst

@@ -33,7 +33,7 @@ Removals for version 4.0
 
 
 .. _deprecations-v4.0:
 .. _deprecations-v4.0:
 
 
-Removals for version 4.0
+Removals for version 5.0
 ========================
 ========================
 
 
 Old Task API
 Old Task API
@@ -145,7 +145,7 @@ The task attributes:
 - ``delivery_mode``
 - ``delivery_mode``
 - ``priority``
 - ``priority``
 
 
-is deprecated and must be set by :setting:`CELERY_ROUTES` instead.
+is deprecated and must be set by :setting:`task_routes` instead.
 
 
 :mod:`celery.result`
 :mod:`celery.result`
 --------------------
 --------------------
@@ -228,11 +228,11 @@ Settings
 =====================================  =====================================
 =====================================  =====================================
 **Setting name**                       **Replace with**
 **Setting name**                       **Replace with**
 =====================================  =====================================
 =====================================  =====================================
-``BROKER_HOST``                        :setting:`BROKER_URL`
-``BROKER_PORT``                        :setting:`BROKER_URL`
-``BROKER_USER``                        :setting:`BROKER_URL`
-``BROKER_PASSWORD``                    :setting:`BROKER_URL`
-``BROKER_VHOST``                       :setting:`BROKER_URL`
+``BROKER_HOST``                        :setting:`broker_url`
+``BROKER_PORT``                        :setting:`broker_url`
+``BROKER_USER``                        :setting:`broker_url`
+``BROKER_PASSWORD``                    :setting:`broker_url`
+``BROKER_VHOST``                       :setting:`broker_url`
 =====================================  =====================================
 =====================================  =====================================
 
 
 
 
@@ -242,14 +242,14 @@ Settings
 =====================================  =====================================
 =====================================  =====================================
 **Setting name**                       **Replace with**
 **Setting name**                       **Replace with**
 =====================================  =====================================
 =====================================  =====================================
-``CELERY_REDIS_HOST``                  :setting:`CELERY_RESULT_BACKEND`
-``CELERY_REDIS_PORT``                  :setting:`CELERY_RESULT_BACKEND`
-``CELERY_REDIS_DB``                    :setting:`CELERY_RESULT_BACKEND`
-``CELERY_REDIS_PASSWORD``              :setting:`CELERY_RESULT_BACKEND`
-``REDIS_HOST``                         :setting:`CELERY_RESULT_BACKEND`
-``REDIS_PORT``                         :setting:`CELERY_RESULT_BACKEND`
-``REDIS_DB``                           :setting:`CELERY_RESULT_BACKEND`
-``REDIS_PASSWORD``                     :setting:`CELERY_RESULT_BACKEND`
+``CELERY_REDIS_HOST``                  :setting:`result_backend`
+``CELERY_REDIS_PORT``                  :setting:`result_backend`
+``CELERY_REDIS_DB``                    :setting:`result_backend`
+``CELERY_REDIS_PASSWORD``              :setting:`result_backend`
+``REDIS_HOST``                         :setting:`result_backend`
+``REDIS_PORT``                         :setting:`result_backend`
+``REDIS_DB``                           :setting:`result_backend`
+``REDIS_PASSWORD``                     :setting:`result_backend`
 =====================================  =====================================
 =====================================  =====================================
 
 
 Logging Settings
 Logging Settings
@@ -273,7 +273,7 @@ Other Settings
 **Setting name**                       **Replace with**
 **Setting name**                       **Replace with**
 =====================================  =====================================
 =====================================  =====================================
 ``CELERY_TASK_ERROR_WITELIST``         Annotate ``Task.ErrorMail``
 ``CELERY_TASK_ERROR_WITELIST``         Annotate ``Task.ErrorMail``
-``CELERY_AMQP_TASK_RESULT_EXPIRES``    :setting:`CELERY_TASK_RESULT_EXPIRES`
+``CELERY_AMQP_TASK_RESULT_EXPIRES``    :setting:`result_expires`
 =====================================  =====================================
 =====================================  =====================================
 
 
 
 
@@ -287,12 +287,12 @@ Removals for version 2.0
 =====================================  =====================================
 =====================================  =====================================
 **Setting name**                       **Replace with**
 **Setting name**                       **Replace with**
 =====================================  =====================================
 =====================================  =====================================
-`CELERY_AMQP_CONSUMER_QUEUES`          `CELERY_QUEUES`
-`CELERY_AMQP_CONSUMER_QUEUES`          `CELERY_QUEUES`
-`CELERY_AMQP_EXCHANGE`                 `CELERY_DEFAULT_EXCHANGE`
-`CELERY_AMQP_EXCHANGE_TYPE`            `CELERY_DEFAULT_AMQP_EXCHANGE_TYPE`
-`CELERY_AMQP_CONSUMER_ROUTING_KEY`     `CELERY_QUEUES`
-`CELERY_AMQP_PUBLISHER_ROUTING_KEY`    `CELERY_DEFAULT_ROUTING_KEY`
+`CELERY_AMQP_CONSUMER_QUEUES`          `task_queues`
+`CELERY_AMQP_CONSUMER_QUEUES`          `task_queues`
+`CELERY_AMQP_EXCHANGE`                 `task_default_exchange`
+`CELERY_AMQP_EXCHANGE_TYPE`            `task_default_exchange_type`
+`CELERY_AMQP_CONSUMER_ROUTING_KEY`     `task_queues`
+`CELERY_AMQP_PUBLISHER_ROUTING_KEY`    `task_default_routing_key`
 =====================================  =====================================
 =====================================  =====================================
 
 
 * :envvar:`CELERY_LOADER` definitions without class name.
 * :envvar:`CELERY_LOADER` definitions without class name.
@@ -303,4 +303,4 @@ Removals for version 2.0
 * :meth:`TaskSet.run`. Use :meth:`celery.task.base.TaskSet.apply_async`
 * :meth:`TaskSet.run`. Use :meth:`celery.task.base.TaskSet.apply_async`
     instead.
     instead.
 
 
-* The module :mod:`celery.task.rest`; use :mod:`celery.task.http` instead.
+* The module :mod:`celery.task.rest`; use :mod:`celery.task.httpY` instead.

+ 8 - 8
docs/userguide/application.rst

@@ -117,18 +117,18 @@ or you can use a dedicated configuration module.
 
 
 The configuration is available as :attr:`@conf`::
 The configuration is available as :attr:`@conf`::
 
 
-    >>> app.conf.CELERY_TIMEZONE
+    >>> app.conf.timezone
     'Europe/London'
     'Europe/London'
 
 
 where you can also set configuration values directly::
 where you can also set configuration values directly::
 
 
-    >>> app.conf.CELERY_ENABLE_UTC = True
+    >>> app.conf.enable_utc = True
 
 
 and update several keys at once by using the ``update`` method::
 and update several keys at once by using the ``update`` method::
 
 
     >>> app.conf.update(
     >>> app.conf.update(
-    ...     CELERY_ENABLE_UTC=True,
-    ...     CELERY_TIMEZONE='Europe/London',
+    ...     enable_utc=True,
+    ...     timezone='Europe/London',
     ...)
     ...)
 
 
 The configuration object consists of multiple dictionaries
 The configuration object consists of multiple dictionaries
@@ -175,8 +175,8 @@ The ``celeryconfig`` module may then look like this:
 
 
 .. code-block:: python
 .. code-block:: python
 
 
-    CELERY_ENABLE_UTC = True
-    CELERY_TIMEZONE = 'Europe/London'
+    enable_utc = True
+    timezone = 'Europe/London'
 
 
 Example 2: Using a configuration module
 Example 2: Using a configuration module
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
@@ -207,8 +207,8 @@ Example 3:  Using a configuration class/object
     app = Celery()
     app = Celery()
 
 
     class Config:
     class Config:
-        CELERY_ENABLE_UTC = True
-        CELERY_TIMEZONE = 'Europe/London'
+        enable_utc = True
+        timezone = 'Europe/London'
 
 
     app.config_from_object(Config)
     app.config_from_object(Config)
     # or using the fully qualified name of the object:
     # or using the fully qualified name of the object:

+ 7 - 7
docs/userguide/calling.rst

@@ -247,8 +247,8 @@ To disable retry you can set the ``retry`` execution option to :const:`False`:
     .. hlist::
     .. hlist::
         :columns: 2
         :columns: 2
 
 
-        - :setting:`CELERY_TASK_PUBLISH_RETRY`
-        - :setting:`CELERY_TASK_PUBLISH_RETRY_POLICY`
+        - :setting:`task_publish_retry`
+        - :setting:`task_publish_retry_policy`
 
 
 Retry Policy
 Retry Policy
 ------------
 ------------
@@ -315,7 +315,7 @@ so every message in Celery has a ``content_type`` header that
 describes the serialization method used to encode it.
 describes the serialization method used to encode it.
 
 
 The default serializer is :mod:`pickle`, but you can
 The default serializer is :mod:`pickle`, but you can
-change this using the :setting:`CELERY_TASK_SERIALIZER` setting,
+change this using the :setting:`task_serializer` setting,
 or for each individual task, or even per message.
 or for each individual task, or even per message.
 
 
 There's built-in support for :mod:`pickle`, `JSON`, `YAML`
 There's built-in support for :mod:`pickle`, `JSON`, `YAML`
@@ -382,7 +382,7 @@ to use when sending a task:
 
 
     1. The `serializer` execution option.
     1. The `serializer` execution option.
     2. The :attr:`@-Task.serializer` attribute
     2. The :attr:`@-Task.serializer` attribute
-    3. The :setting:`CELERY_TASK_SERIALIZER` setting.
+    3. The :setting:`task_serializer` setting.
 
 
 
 
 Example setting a custom serializer for a single task invocation:
 Example setting a custom serializer for a single task invocation:
@@ -405,7 +405,7 @@ to use when sending a task:
 
 
     1. The `compression` execution option.
     1. The `compression` execution option.
     2. The :attr:`@-Task.compression` attribute.
     2. The :attr:`@-Task.compression` attribute.
-    3. The :setting:`CELERY_MESSAGE_COMPRESSION` attribute.
+    3. The :setting:`task_compression` attribute.
 
 
 Example specifying the compression used when calling a task::
 Example specifying the compression used when calling a task::
 
 
@@ -424,7 +424,7 @@ Connections
 
 
     The connection pool is enabled by default since version 2.5.
     The connection pool is enabled by default since version 2.5.
 
 
-    See the :setting:`BROKER_POOL_LIMIT` setting for more information.
+    See the :setting:`broker_pool_limit` setting for more information.
 
 
 You can handle the connection manually by creating a
 You can handle the connection manually by creating a
 publisher:
 publisher:
@@ -475,7 +475,7 @@ the workers :option:`-Q` argument:
 .. seealso::
 .. seealso::
 
 
     Hard-coding queue names in code is not recommended, the best practice
     Hard-coding queue names in code is not recommended, the best practice
-    is to use configuration routers (:setting:`CELERY_ROUTES`).
+    is to use configuration routers (:setting:`task_routes`).
 
 
     To find out more about routing, please see :ref:`guide-routing`.
     To find out more about routing, please see :ref:`guide-routing`.
 
 

Some files were not shown because too many files changed in this diff