Pārlūkot izejas kodu

Deprecated: AMQP Backend Removal: 5.0

Ask Solem 8 gadi atpakaļ
vecāks
revīzija
ad990403e9

+ 12 - 241
celery/backends/amqp.py

@@ -10,25 +10,11 @@
 """
 from __future__ import absolute_import, unicode_literals
 
-from kombu import Exchange, Queue, Producer, Consumer
-from kombu.utils import register_after_fork
+from .rpc import BaseRPCBackend
 
-from celery import states
-from celery.five import items, range
-from celery.utils.functional import dictfilter
-from celery.utils.log import get_logger
-from celery.utils.timeutils import maybe_s_to_ms
+from celery.utils import warn_deprecated
 
-from . import base
-from .async import AsyncBackendMixin, BaseResultConsumer
-
-__all__ = ['BacklogLimitExceeded', 'AMQPBackend']
-
-logger = get_logger(__name__)
-
-
-class BacklogLimitExceeded(Exception):
-    """Too much state history to fast-forward."""
+__all__ = ['AMQPBackend']
 
 
 def repair_uuid(s):
@@ -38,111 +24,14 @@ def repair_uuid(s):
     return '%s-%s-%s-%s-%s' % (s[:8], s[8:12], s[12:16], s[16:20], s[20:])
 
 
-def _on_after_fork_cleanup_backend(backend):
-    backend._after_fork()
-
-
-class NoCacheQueue(Queue):
-    can_cache_declaration = False
-
-
-class ResultConsumer(BaseResultConsumer):
-    Consumer = Consumer
-
-    _connection = None
-    _consumer = None
-
-    def __init__(self, *args, **kwargs):
-        super(ResultConsumer, self).__init__(*args, **kwargs)
-        self._create_binding = self.backend._create_binding
-
-    def start(self, initial_task_id, no_ack=True):
-        self._connection = self.app.connection()
-        initial_queue = self._create_binding(initial_task_id)
-        self._consumer = self.Consumer(
-            self._connection.default_channel, [initial_queue],
-            callbacks=[self.on_state_change], no_ack=no_ack,
-            accept=self.accept)
-        self._consumer.consume()
-
-    def drain_events(self, timeout=None):
-        return self._connection.drain_events(timeout=timeout)
-
-    def stop(self):
-        try:
-            self._consumer.cancel()
-        finally:
-            self._connection.close()
-
-    def on_after_fork(self):
-        self._consumer = None
-        if self._connection is not None:
-            self._connection.collect()
-            self._connection = None
-
-    def consume_from(self, task_id):
-        if self._consumer is None:
-            return self.start(task_id)
-        queue = self._create_binding(task_id)
-        if not self._consumer.consuming_from(queue):
-            self._consumer.add_queue(queue)
-            self._consumer.consume()
-
-    def cancel_for(self, task_id):
-        if self._consumer:
-            self._consumer.cancel_by_queue(self._create_binding(task_id).name)
-
-
-class AMQPBackend(base.Backend, AsyncBackendMixin):
+class AMQPBackend(BaseRPCBackend):
     """Publishes results by sending messages."""
 
-    Exchange = Exchange
-    Queue = NoCacheQueue
-    Consumer = Consumer
-    Producer = Producer
-    ResultConsumer = ResultConsumer
-
-    BacklogLimitExceeded = BacklogLimitExceeded
-
-    persistent = True
-    supports_autoexpire = True
-    supports_native_join = True
-
-    retry_policy = {
-        'max_retries': 20,
-        'interval_start': 0,
-        'interval_step': 1,
-        'interval_max': 1,
-    }
-
-    def __init__(self, app, connection=None, exchange=None, exchange_type=None,
-                 persistent=None, serializer=None, auto_delete=True, **kwargs):
-        super(AMQPBackend, self).__init__(app, **kwargs)
-        conf = self.app.conf
-        self._connection = connection
-        self._out_of_band = {}
-        self.persistent = self.prepare_persistent(persistent)
-        self.delivery_mode = 2 if self.persistent else 1
-        exchange = exchange or conf.result_exchange
-        exchange_type = exchange_type or conf.result_exchange_type
-        self.exchange = self._create_exchange(
-            exchange, exchange_type, self.delivery_mode,
-        )
-        self.serializer = serializer or conf.result_serializer
-        self.auto_delete = auto_delete
-        self.queue_arguments = dictfilter({
-            'x-expires': maybe_s_to_ms(self.expires),
-        })
-        self.result_consumer = self.ResultConsumer(
-            self, self.app, self.accept,
-            self._pending_results, self._pending_messages,
-        )
-        if register_after_fork is not None:
-            register_after_fork(self, _on_after_fork_cleanup_backend)
-
-    def _after_fork(self):
-        self._pending_results.clear()
-        self.result_consumer._after_fork()
+    def __init__(self, *args, **kwargs):
+        warn_deprecated(
+            'The AMQP backend', deprecation='4.0', removal='5.0',
+            alternative='Please use RPC backend or a persistent backend.')
+        super(AMQPBackend, self).__init__(*args, **kwargs)
 
     def _create_exchange(self, name, type='direct', delivery_mode=2):
         return self.Exchange(name=name,
@@ -160,7 +49,7 @@ class AMQPBackend(base.Backend, AsyncBackendMixin):
                           auto_delete=self.auto_delete,
                           queue_arguments=self.queue_arguments)
 
-    def revive(self, channel):
+    def on_task_call(self, producer, task_id):
         pass
 
     def rkey(self, task_id):
@@ -171,129 +60,11 @@ class AMQPBackend(base.Backend, AsyncBackendMixin):
             return self.rkey(task_id), request.correlation_id or task_id
         return self.rkey(task_id), task_id
 
-    def store_result(self, task_id, result, state,
-                     traceback=None, request=None, **kwargs):
-        """Send task return value and state."""
-        routing_key, correlation_id = self.destination_for(task_id, request)
-        if not routing_key:
-            return
-        with self.app.amqp.producer_pool.acquire(block=True) as producer:
-            producer.publish(
-                {'task_id': task_id, 'status': state,
-                 'result': self.encode_result(result, state),
-                 'traceback': traceback,
-                 'children': self.current_task_children(request)},
-                exchange=self.exchange,
-                routing_key=routing_key,
-                correlation_id=correlation_id,
-                serializer=self.serializer,
-                retry=True, retry_policy=self.retry_policy,
-                declare=self.on_reply_declare(task_id),
-                delivery_mode=self.delivery_mode,
-            )
-        return result
-
     def on_reply_declare(self, task_id):
         return [self._create_binding(task_id)]
 
-    def on_out_of_band_result(self, task_id, message):
-        if self.result_consumer:
-            self.result_consumer.on_out_of_band_result(message)
-        self._out_of_band[task_id] = message
-
-    def get_task_meta(self, task_id, backlog_limit=1000):
-        buffered = self._out_of_band.pop(task_id, None)
-        if buffered:
-            return self._set_cache_by_message(task_id, buffered)
-
-        # Polling and using basic_get
-        latest_by_id = {}
-        prev = None
-        for acc in self._slurp_from_queue(task_id, self.accept, backlog_limit):
-            tid = self._get_message_task_id(acc)
-            prev, latest_by_id[tid] = latest_by_id.get(tid), acc
-            if prev:
-                # backends are not expected to keep history,
-                # so we delete everything except the most recent state.
-                prev.ack()
-                prev = None
-
-        latest = latest_by_id.pop(task_id, None)
-        for tid, msg in items(latest_by_id):
-            self.on_out_of_band_result(tid, msg)
-
-        if latest:
-            latest.requeue()
-            return self._set_cache_by_message(task_id, latest)
-        else:
-            # no new state, use previous
-            try:
-                return self._cache[task_id]
-            except KeyError:
-                # result probably pending.
-                return {'status': states.PENDING, 'result': None}
-    poll = get_task_meta  # XXX compat
-
-    def _set_cache_by_message(self, task_id, message):
-        payload = self._cache[task_id] = self.meta_from_decoded(
-            message.payload)
-        return payload
-
-    def _slurp_from_queue(self, task_id, accept,
-                          limit=1000, no_ack=False):
-        with self.app.pool.acquire_channel(block=True) as (_, channel):
-            binding = self._create_binding(task_id)(channel)
-            binding.declare()
-
-            for i in range(limit):
-                msg = binding.get(accept=accept, no_ack=no_ack)
-                if not msg:
-                    break
-                yield msg
-            else:
-                raise self.BacklogLimitExceeded(task_id)
-
-    def _get_message_task_id(self, message):
-        try:
-            # try property first so we don't have to deserialize
-            # the payload.
-            return message.properties['correlation_id']
-        except (AttributeError, KeyError):
-            # message sent by old Celery version, need to deserialize.
-            return message.payload['task_id']
-
-    def reload_task_result(self, task_id):
-        raise NotImplementedError(
-            'reload_task_result is not supported by this backend.')
-
-    def reload_group_result(self, task_id):
-        """Reload group result, even if it has been previously fetched."""
-        raise NotImplementedError(
-            'reload_group_result is not supported by this backend.')
-
-    def save_group(self, group_id, result):
-        raise NotImplementedError(
-            'save_group is not supported by this backend.')
-
-    def restore_group(self, group_id, cache=True):
-        raise NotImplementedError(
-            'restore_group is not supported by this backend.')
-
-    def delete_group(self, group_id):
-        raise NotImplementedError(
-            'delete_group is not supported by this backend.')
+    def on_result_fulfilled(self, result):
+        self.result_consumer.cancel_for(result.id)
 
     def as_uri(self, include_password=True):
         return 'amqp://'
-
-    def __reduce__(self, args=(), kwargs={}):
-        kwargs.update(
-            connection=self._connection,
-            exchange=self.exchange.name,
-            exchange_type=self.exchange.type,
-            persistent=self.persistent,
-            serializer=self.serializer,
-            auto_delete=self.auto_delete,
-            expires=self.expires,
-        )
-        return super(AMQPBackend, self).__reduce__(args, kwargs)

+ 248 - 11
celery/backends/rpc.py

@@ -8,18 +8,258 @@
 """
 from __future__ import absolute_import, unicode_literals
 
-from kombu import Consumer, Exchange
+from kombu import Consumer, Exchange, Producer, Queue
 from kombu.common import maybe_declare
-from kombu.utils import cached_property
+from kombu.utils import cached_property, register_after_fork
 
 from celery import current_task
+from celery import states
 from celery._state import task_join_will_block
-from celery.backends import amqp
+from celery.five import items, range
+from celery.utils.functional import dictfilter
+from celery.utils.timeutils import maybe_s_to_ms
 
-__all__ = ['RPCBackend']
+from . import base
+from .async import AsyncBackendMixin, BaseResultConsumer
 
+__all__ = ['BacklogLimitExceeded', 'BaseRPCBackend', 'RPCBackend']
 
-class RPCBackend(amqp.AMQPBackend):
+
+class BacklogLimitExceeded(Exception):
+    """Too much state history to fast-forward."""
+
+
+class NoCacheQueue(Queue):
+    can_cache_declaration = False
+
+
+def _on_after_fork_cleanup_backend(backend):
+    backend._after_fork()
+
+
+class ResultConsumer(BaseResultConsumer):
+    Consumer = Consumer
+
+    _connection = None
+    _consumer = None
+
+    def __init__(self, *args, **kwargs):
+        super(ResultConsumer, self).__init__(*args, **kwargs)
+        self._create_binding = self.backend._create_binding
+
+    def start(self, initial_task_id, no_ack=True):
+        self._connection = self.app.connection()
+        initial_queue = self._create_binding(initial_task_id)
+        self._consumer = self.Consumer(
+            self._connection.default_channel, [initial_queue],
+            callbacks=[self.on_state_change], no_ack=no_ack,
+            accept=self.accept)
+        self._consumer.consume()
+
+    def drain_events(self, timeout=None):
+        return self._connection.drain_events(timeout=timeout)
+
+    def stop(self):
+        try:
+            self._consumer.cancel()
+        finally:
+            self._connection.close()
+
+    def on_after_fork(self):
+        self._consumer = None
+        if self._connection is not None:
+            self._connection.collect()
+            self._connection = None
+
+    def consume_from(self, task_id):
+        if self._consumer is None:
+            return self.start(task_id)
+        queue = self._create_binding(task_id)
+        if not self._consumer.consuming_from(queue):
+            self._consumer.add_queue(queue)
+            self._consumer.consume()
+
+    def cancel_for(self, task_id):
+        if self._consumer:
+            self._consumer.cancel_by_queue(self._create_binding(task_id).name)
+
+
+class BaseRPCBackend(base.Backend, AsyncBackendMixin):
+
+    Exchange = Exchange
+    Queue = NoCacheQueue
+    Consumer = Consumer
+    Producer = Producer
+    ResultConsumer = ResultConsumer
+
+    BacklogLimitExceeded = BacklogLimitExceeded
+
+    persistent = True
+    supports_autoexpire = True
+    supports_native_join = True
+
+    retry_policy = {
+        'max_retries': 20,
+        'interval_start': 0,
+        'interval_step': 1,
+        'interval_max': 1,
+    }
+
+    def __init__(self, app, connection=None, exchange=None, exchange_type=None,
+                 persistent=None, serializer=None, auto_delete=True, **kwargs):
+        super(BaseRPCBackend, self).__init__(app, **kwargs)
+        conf = self.app.conf
+        self._connection = connection
+        self._out_of_band = {}
+        self.persistent = self.prepare_persistent(persistent)
+        self.delivery_mode = 2 if self.persistent else 1
+        exchange = exchange or conf.result_exchange
+        exchange_type = exchange_type or conf.result_exchange_type
+        self.exchange = self._create_exchange(
+            exchange, exchange_type, self.delivery_mode,
+        )
+        self.serializer = serializer or conf.result_serializer
+        self.auto_delete = auto_delete
+        self.queue_arguments = dictfilter({
+            'x-expires': maybe_s_to_ms(self.expires),
+        })
+        self.result_consumer = self.ResultConsumer(
+            self, self.app, self.accept,
+            self._pending_results, self._pending_messages,
+        )
+        if register_after_fork is not None:
+            register_after_fork(self, _on_after_fork_cleanup_backend)
+
+    def _after_fork(self):
+        self._pending_results.clear()
+        self.result_consumer._after_fork()
+
+    def store_result(self, task_id, result, state,
+                     traceback=None, request=None, **kwargs):
+        """Send task return value and state."""
+        routing_key, correlation_id = self.destination_for(task_id, request)
+        if not routing_key:
+            return
+        with self.app.amqp.producer_pool.acquire(block=True) as producer:
+            producer.publish(
+                {'task_id': task_id, 'status': state,
+                 'result': self.encode_result(result, state),
+                 'traceback': traceback,
+                 'children': self.current_task_children(request)},
+                exchange=self.exchange,
+                routing_key=routing_key,
+                correlation_id=correlation_id,
+                serializer=self.serializer,
+                retry=True, retry_policy=self.retry_policy,
+                declare=self.on_reply_declare(task_id),
+                delivery_mode=self.delivery_mode,
+            )
+        return result
+
+    def on_out_of_band_result(self, task_id, message):
+        if self.result_consumer:
+            self.result_consumer.on_out_of_band_result(message)
+        self._out_of_band[task_id] = message
+
+    def get_task_meta(self, task_id, backlog_limit=1000):
+        buffered = self._out_of_band.pop(task_id, None)
+        if buffered:
+            return self._set_cache_by_message(task_id, buffered)
+
+        # Polling and using basic_get
+        latest_by_id = {}
+        prev = None
+        for acc in self._slurp_from_queue(task_id, self.accept, backlog_limit):
+            tid = self._get_message_task_id(acc)
+            prev, latest_by_id[tid] = latest_by_id.get(tid), acc
+            if prev:
+                # backends are not expected to keep history,
+                # so we delete everything except the most recent state.
+                prev.ack()
+                prev = None
+
+        latest = latest_by_id.pop(task_id, None)
+        for tid, msg in items(latest_by_id):
+            self.on_out_of_band_result(tid, msg)
+
+        if latest:
+            latest.requeue()
+            return self._set_cache_by_message(task_id, latest)
+        else:
+            # no new state, use previous
+            try:
+                return self._cache[task_id]
+            except KeyError:
+                # result probably pending.
+                return {'status': states.PENDING, 'result': None}
+    poll = get_task_meta  # XXX compat
+
+    def _set_cache_by_message(self, task_id, message):
+        payload = self._cache[task_id] = self.meta_from_decoded(
+            message.payload)
+        return payload
+
+    def _slurp_from_queue(self, task_id, accept,
+                          limit=1000, no_ack=False):
+        with self.app.pool.acquire_channel(block=True) as (_, channel):
+            binding = self._create_binding(task_id)(channel)
+            binding.declare()
+
+            for i in range(limit):
+                msg = binding.get(accept=accept, no_ack=no_ack)
+                if not msg:
+                    break
+                yield msg
+            else:
+                raise self.BacklogLimitExceeded(task_id)
+
+    def _get_message_task_id(self, message):
+        try:
+            # try property first so we don't have to deserialize
+            # the payload.
+            return message.properties['correlation_id']
+        except (AttributeError, KeyError):
+            # message sent by old Celery version, need to deserialize.
+            return message.payload['task_id']
+
+    def revive(self, channel):
+        pass
+
+    def reload_task_result(self, task_id):
+        raise NotImplementedError(
+            'reload_task_result is not supported by this backend.')
+
+    def reload_group_result(self, task_id):
+        """Reload group result, even if it has been previously fetched."""
+        raise NotImplementedError(
+            'reload_group_result is not supported by this backend.')
+
+    def save_group(self, group_id, result):
+        raise NotImplementedError(
+            'save_group is not supported by this backend.')
+
+    def restore_group(self, group_id, cache=True):
+        raise NotImplementedError(
+            'restore_group is not supported by this backend.')
+
+    def delete_group(self, group_id):
+        raise NotImplementedError(
+            'delete_group is not supported by this backend.')
+
+    def __reduce__(self, args=(), kwargs={}):
+        kwargs.update(
+            connection=self._connection,
+            exchange=self.exchange.name,
+            exchange_type=self.exchange.type,
+            persistent=self.persistent,
+            serializer=self.serializer,
+            auto_delete=self.auto_delete,
+            expires=self.expires,
+        )
+        return super(BaseRPCBackend, self).__reduce__(args, kwargs)
+
+
+class RPCBackend(BaseRPCBackend):
     persistent = False
 
     class Consumer(Consumer):
@@ -29,15 +269,12 @@ class RPCBackend(amqp.AMQPBackend):
         # uses direct to queue routing (anon exchange).
         return Exchange(None)
 
-    def on_task_call(self, producer, task_id):
-        if not task_join_will_block():
-            maybe_declare(self.binding(producer.channel), retry=True)
-
     def _create_binding(self, task_id):
         return self.binding
 
-    def _many_bindings(self, ids):
-        return [self.binding]
+    def on_task_call(self, producer, task_id):
+        if not task_join_will_block():
+            maybe_declare(self.binding(producer.channel), retry=True)
 
     def rkey(self, task_id):
         return task_id

+ 1 - 1
celery/result.py

@@ -154,7 +154,7 @@ class AsyncResult(ResultBase):
         :keyword propagate: Re-raise exception if the task failed.
         :keyword interval: Time to wait (in seconds) before retrying to
            retrieve the result.  Note that this does not have any effect
-           when using the amqp result store backend, as it does not
+           when using the RPC/redis result store backends, as they do not
            use polling.
         :keyword no_ack: Enable amqp no ack (automatically acknowledge
             message).  If this is :const:`False` then the message will

+ 0 - 6
celery/tests/backends/test_rpc.py

@@ -54,12 +54,6 @@ class test_RPCBackend(AppCase):
         self.assertFalse(queue.durable)
         self.assertTrue(queue.auto_delete)
 
-    def test_many_bindings(self):
-        self.assertListEqual(
-            self.b._many_bindings(['a', 'b']),
-            [self.b.binding],
-        )
-
     def test_create_binding(self):
         self.assertEqual(self.b._create_binding('id'), self.b.binding)
 

+ 17 - 56
docs/configuration.rst

@@ -532,10 +532,6 @@ Can be one of the following:
     Use a shared directory to store the results.
     See :ref:`conf-filesystem-result-backend`.
 
-* ``amqp``
-    Older AMQP backend (badly) emulating a database-based backend.
-    See :ref:`conf-amqp-result-backend`.
-
 * ``consul``
     Use the `Consul`_ K/V store to store the results
     See :ref:`conf-consul-result-backend`.
@@ -604,8 +600,8 @@ Default is to expire after 1 day.
 ``result_cache_max``
 ~~~~~~~~~~~~~~~~~~~~
 
-Enables client caching of results, which can be useful for the old 'amqp'
-backend where the result is unavailable as soon as one result instance
+Enables client caching of results, which can be useful for the old deprecated
+'amqp' backend where the result is unavailable as soon as one result instance
 consumes it.
 
 This is the total number of results to cache before older results are evicted.
@@ -709,6 +705,21 @@ you to customize the table names:
 RPC backend settings
 --------------------
 
+.. setting:: result_exchange
+
+``result_exchange``
+~~~~~~~~~~~~~~~~~~~
+
+Name of the exchange to publish results in.  Default is `celeryresults`.
+
+.. setting:: result_exchange_type
+
+``result_exchange_type``
+~~~~~~~~~~~~~~~~~~~~~~~~
+
+The exchange type of the result exchange.  Default is to use a `direct`
+exchange.
+
 .. setting:: result_persistent
 
 ``result_persistent``
@@ -1166,56 +1177,6 @@ The URL is formed out of the following parts:
     The default container the CouchDB server is writing to.
     Defaults to ``default``.
 
-.. _conf-amqp-result-backend:
-
-AMQP backend settings
----------------------
-
-.. admonition:: Do not use in production.
-
-    This is the old AMQP result backend that creates one queue per task,
-    if you want to send results back as message please consider using the
-    RPC backend instead, or if you need the results to be persistent
-    use a result backend designed for that purpose (e.g. Redis, or a database).
-
-.. note::
-
-    The AMQP backend requires RabbitMQ 1.1.0 or higher to automatically
-    expire results.  If you are running an older version of RabbitMQ
-    you should disable result expiration like this:
-
-        result_expires = None
-
-.. setting:: result_exchange
-
-``result_exchange``
-~~~~~~~~~~~~~~~~~~~
-
-Name of the exchange to publish results in.  Default is `celeryresults`.
-
-.. setting:: result_exchange_type
-
-``result_exchange_type``
-~~~~~~~~~~~~~~~~~~~~~~~~
-
-The exchange type of the result exchange.  Default is to use a `direct`
-exchange.
-
-``result_persistent``
-~~~~~~~~~~~~~~~~~~~~~
-
-If set to :const:`True`, result messages will be persistent.  This means the
-messages will not be lost after a broker restart.  The default is for the
-results to be transient.
-
-Example configuration
-~~~~~~~~~~~~~~~~~~~~~
-
-.. code-block:: python
-
-    result_backend = 'amqp'
-    result_expires = 18000  # 5 hours.
-
 .. _conf-filesystem-result-backend:
 
 File-system backend settings

+ 4 - 0
docs/faq.rst

@@ -501,6 +501,10 @@ When running with the AMQP result backend, every task result will be sent
 as a message. If you don't collect these results, they will build up and
 RabbitMQ will eventually run out of memory.
 
+This result backend is now deprecated so you should not be using it.
+Use either the RPC backend for rpc-style calls, or a persistent backend
+if you need multi-consumer access to results.
+
 Results expire after 1 day by default.  It may be a good idea
 to lower this value by configuring the :setting:`result_expires`
 setting.

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

@@ -203,7 +203,8 @@ Keeping Results
 If you want to keep track of the tasks' states, Celery needs to store or send
 the states somewhere.  There are several
 built-in result backends to choose from: `SQLAlchemy`_/`Django`_ ORM,
-`Memcached`_, `Redis`_, AMQP (`RabbitMQ`_), and -- or you can define your own.
+`Memcached`_, `Redis`_, :ref:`RPC <conf-rpc-result-backend>` (`RabbitMQ`_/AMQP),
+and -- or you can define your own.
 
 .. _`Memcached`: http://memcached.org
 .. _`MongoDB`: http://www.mongodb.org

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

@@ -47,7 +47,7 @@ you simply import this instance.
 - The ``backend`` argument specifies the result backend to use,
 
     It's used to keep track of task state and results.
-    While results are disabled by default I use the amqp result backend here
+    While results are disabled by default I use the RPC result backend here
     because I demonstrate how retrieving results work later, you may want to use
     a different backend for your application. They all have different
     strengths and weaknesses.  If you don't need results it's better
@@ -334,7 +334,7 @@ exception, in fact ``result.get()`` will propagate any errors by default:
     File "<stdin>", line 1, in <module>
     File "/opt/devel/celery/celery/result.py", line 113, in get
         interval=interval)
-    File "/opt/devel/celery/celery/backends/amqp.py", line 138, in wait_for
+    File "/opt/devel/celery/celery/backends/rpc.py", line 138, in wait_for
         raise meta['result']
     TypeError: add() takes exactly 2 arguments (1 given)
 

+ 1 - 1
docs/userguide/routing.rst

@@ -111,7 +111,7 @@ A queue named `"video"` will be created with the following settings:
      'exchange_type': 'direct',
      'routing_key': 'video'}
 
-The non-AMQP backends like `SQS` do not support exchanges,
+The non-AMQP transports like `SQS` do not support exchanges,
 so they require the exchange to have the same name as the queue. Using this
 design ensures it will work for them as well.
 

+ 1 - 1
docs/whatsnew-4.0.rst

@@ -1042,7 +1042,7 @@ In Other News
 
     Contributed by Samuel Jaillet.
 
-- **Result Backends**: Fix problem with RPC/AMQP backends where exception
+- **Result Backends**: Fix problem with RPC backend where exception
     was not deserialized properly with the json serializer (Issue #2518).
 
     Fix contributed by Allard Hoeve.