Przeglądaj źródła

Events are now transient and is using a topic exchange (instead of direct).

The `CELERYD_EVENT_EXCHANGE`, `CELERYD_EVENT_ROUTING_KEY`,
`CELERYD_EVENT_EXCHANGE_TYPE` settings are no longer in use.

This means events will not be stored until there is a consumer, and the events will
be gone as soon as the consumer stops.  Also it means there can be multiple
monitors running at the same time.

The routing key of an event is the type of event (e.g. `worker.started`,
`worker.hartbeat`, `task.succeeded` this means a consumer can filter on
specific types, to only be alerted of the events it cares about.

Each consumer will create a unique queue, meaning it is in effect a broadcast
exchange.

This opens up a lot of possibilites, for example the workers could listen for
worker events, to know what workers are in the neighborhood, and even restart
workers when they go down (or use this information to optimize
tasks/autoscaling).

The event exchange has been renamed from "celeryevent" to "celeryev" so it
does not colliddoes not colliddoes not colliddoes not colliddoes not colliddoes not c" to create a unique queue for each client.
Ask Solem 14 lat temu
rodzic
commit
259e049c4a
2 zmienionych plików z 33 dodań i 21 usunięć
  1. 0 4
      celery/app/defaults.py
  2. 33 17
      celery/events/__init__.py

+ 0 - 4
celery/app/defaults.py

@@ -69,10 +69,6 @@ NAMESPACES = {
         "DEFAULT_EXCHANGE_TYPE": Option("direct"),
         "DEFAULT_EXCHANGE_TYPE": Option("direct"),
         "DEFAULT_DELIVERY_MODE": Option(2, type="string"),
         "DEFAULT_DELIVERY_MODE": Option(2, type="string"),
         "EAGER_PROPAGATES_EXCEPTIONS": Option(False, type="bool"),
         "EAGER_PROPAGATES_EXCEPTIONS": Option(False, type="bool"),
-        "EVENT_QUEUE": Option("celeryevent"),
-        "EVENT_EXCHANGE": Option("celeryevent"),
-        "EVENT_EXCHANGE_TYPE": Option("direct"),
-        "EVENT_ROUTING_KEY": Option("celeryevent"),
         "EVENT_SERIALIZER": Option("json"),
         "EVENT_SERIALIZER": Option("json"),
         "IMPORTS": Option((), type="tuple"),
         "IMPORTS": Option((), type="tuple"),
         "IGNORE_RESULT": Option(False, type="bool"),
         "IGNORE_RESULT": Option(False, type="bool"),

+ 33 - 17
celery/events/__init__.py

@@ -5,10 +5,13 @@ import threading
 from collections import deque
 from collections import deque
 from itertools import count
 from itertools import count
 
 
-from kombu.compat import Publisher, Consumer
+from kombu.entity import Exchange, Queue
+from kombu.messaging import Consumer, Producer
 
 
 from celery.app import app_or_default
 from celery.app import app_or_default
+from celery.utils import gen_unique_id
 
 
+event_exchange = Exchange("celeryev", type="topic")
 
 
 def create_event(type, fields):
 def create_event(type, fields):
     std = {"type": type,
     std = {"type": type,
@@ -55,16 +58,14 @@ class EventDispatcher(object):
     def enable(self):
     def enable(self):
         conf = self.app.conf
         conf = self.app.conf
         self.enabled = True
         self.enabled = True
-        self.publisher = Publisher(self.connection,
-                                exchange=conf.CELERY_EVENT_EXCHANGE,
-                                exchange_type=conf.CELERY_EVENT_EXCHANGE_TYPE,
-                                routing_key=conf.CELERY_EVENT_ROUTING_KEY,
-                                serializer=conf.CELERY_EVENT_SERIALIZER)
+        self.publisher = Producer(self.connection.channel(),
+                                  exchange=event_exchange,
+                                  serializer=conf.CELERY_EVENT_SERIALIZER)
 
 
     def disable(self):
     def disable(self):
         self.enabled = False
         self.enabled = False
         if self.publisher is not None:
         if self.publisher is not None:
-            self.publisher.close()
+            self.publisher.channel.close()
             self.publisher = None
             self.publisher = None
 
 
     def send(self, type, **fields):
     def send(self, type, **fields):
@@ -81,7 +82,8 @@ class EventDispatcher(object):
         event = Event(type, hostname=self.hostname, **fields)
         event = Event(type, hostname=self.hostname, **fields)
         try:
         try:
             try:
             try:
-                self.publisher.send(event)
+                self.publisher.publish(event,
+                                       routing_key=type.replace("-", "."))
             except Exception, exc:
             except Exception, exc:
                 self._outbound_buffer.append((event, exc))
                 self._outbound_buffer.append((event, exc))
         finally:
         finally:
@@ -95,7 +97,7 @@ class EventDispatcher(object):
     def close(self):
     def close(self):
         """Close the event dispatcher."""
         """Close the event dispatcher."""
         self._lock.locked() and self._lock.release()
         self._lock.locked() and self._lock.release()
-        self.publisher and self.publisher.close()
+        self.publisher and self.publisher.channel.close()
 
 
 
 
 class EventReceiver(object):
 class EventReceiver(object):
@@ -111,11 +113,19 @@ class EventReceiver(object):
     """
     """
     handlers = {}
     handlers = {}
 
 
-    def __init__(self, connection, handlers=None, app=None):
+    def __init__(self, connection, handlers=None, routing_key="#",
+            app=None):
         self.app = app_or_default(app)
         self.app = app_or_default(app)
         self.connection = connection
         self.connection = connection
         if handlers is not None:
         if handlers is not None:
             self.handlers = handlers
             self.handlers = handlers
+        self.routing_key = routing_key
+        self.node_id = gen_unique_id()
+        self.queue = Queue("%s.%s" % ("celeryev", self.node_id),
+                           exchange=event_exchange,
+                           routing_key=self.routing_key,
+                           auto_delete=True,
+                           durable=False)
 
 
     def process(self, type, event):
     def process(self, type, event):
         """Process the received event by dispatching it to the appropriate
         """Process the received event by dispatching it to the appropriate
@@ -124,12 +134,17 @@ class EventReceiver(object):
         handler and handler(event)
         handler and handler(event)
 
 
     def consumer(self):
     def consumer(self):
+        """Create event consumer.
+
+        .. warning::
+
+            This creates a new channel that needs to be closed
+            by calling `consumer.channel.close()`.
+
+        """
         conf = self.app.conf
         conf = self.app.conf
-        consumer = Consumer(self.connection,
-                            queue=conf.CELERY_EVENT_QUEUE,
-                            exchange=conf.CELERY_EVENT_EXCHANGE,
-                            exchange_type=conf.CELERY_EVENT_EXCHANGE_TYPE,
-                            routing_key=conf.CELERY_EVENT_ROUTING_KEY,
+        consumer = Consumer(self.connection.channel(),
+                            queues=[self.queue],
                             no_ack=True)
                             no_ack=True)
         consumer.register_callback(self._receive)
         consumer.register_callback(self._receive)
         return consumer
         return consumer
@@ -148,14 +163,15 @@ class EventReceiver(object):
                 if limit and iteration > limit:
                 if limit and iteration > limit:
                     break
                     break
                 try:
                 try:
-                    consumer.connection.drain_events(timeout=timeout)
+                    self.connection.drain_events(timeout=timeout)
                 except socket.timeout:
                 except socket.timeout:
                     if timeout:
                     if timeout:
                         raise
                         raise
                 except socket.error:
                 except socket.error:
                     pass
                     pass
         finally:
         finally:
-            consumer.close()
+            consumer.cancel()
+            consumer.channel.close()
 
 
     def _receive(self, message_data, message):
     def _receive(self, message_data, message):
         type = message_data.pop("type").lower()
         type = message_data.pop("type").lower()