Browse Source

Boosteps: Blueprint should not take "app" argument.

Ask Solem 8 years ago
parent
commit
314d456fd0
4 changed files with 36 additions and 39 deletions
  1. 2 7
      celery/bootsteps.py
  2. 2 1
      celery/worker/consumer/consumer.py
  3. 6 4
      celery/worker/worker.py
  4. 26 27
      t/unit/worker/test_bootsteps.py

+ 2 - 7
celery/bootsteps.py

@@ -80,7 +80,6 @@ class Blueprint(object):
     Arguments:
         steps Sequence[Union[str, Step]]: List of steps.
         name (str): Set explicit name for this blueprint.
-        app (~@Celery): Set the Celery app for this blueprint.
         on_start (Callable): Optional callback applied after blueprint start.
         on_close (Callable): Optional callback applied before blueprint close.
         on_stopped (Callable): Optional callback applied after
@@ -100,9 +99,8 @@ class Blueprint(object):
         TERMINATE: 'terminating',
     }
 
-    def __init__(self, steps=None, name=None, app=None,
+    def __init__(self, steps=None, name=None,
                  on_start=None, on_close=None, on_stopped=None):
-        self.app = app
         self.name = name or self.name or qualname(type(self))
         self.types = set(steps or []) | set(self.default_steps)
         self.on_start = on_start
@@ -254,10 +252,7 @@ class Blueprint(object):
             raise KeyError('unknown bootstep: %s' % exc)
 
     def claim_steps(self):
-        return dict(self.load_step(step) for step in self._all_steps())
-
-    def _all_steps(self):
-        return self.types | self.app.steps[self.name.lower()]
+        return dict(self.load_step(step) for step in self.types)
 
     def load_step(self, step):
         step = symbol_by_name(step)

+ 2 - 1
celery/worker/consumer/consumer.py

@@ -210,7 +210,8 @@ class Consumer(object):
 
         self.steps = []
         self.blueprint = self.Blueprint(
-            app=self.app, on_close=self.on_close,
+            steps=self.app.steps['consumer'],
+            on_close=self.on_close,
         )
         self.blueprint.apply(self, **dict(worker_options or {}, **kwargs))
 

+ 6 - 4
celery/worker/worker.py

@@ -135,10 +135,12 @@ class WorkController(object):
         self.pool_cls = _concurrency.get_implementation(self.pool_cls)
         self.steps = []
         self.on_init_blueprint()
-        self.blueprint = self.Blueprint(app=self.app,
-                                        on_start=self.on_start,
-                                        on_close=self.on_close,
-                                        on_stopped=self.on_stopped)
+        self.blueprint = self.Blueprint(
+            steps=self.app.steps['worker'],
+            on_start=self.on_start,
+            on_close=self.on_close,
+            on_stopped=self.on_stopped,
+        )
         self.blueprint.apply(self, **kwargs)
 
     def on_init_blueprint(self):

+ 26 - 27
t/unit/worker/test_bootsteps.py

@@ -213,7 +213,7 @@ class test_Blueprint:
     class Blueprint(bootsteps.Blueprint):
         name = 'test_Blueprint'
 
-    def test_steps_added_to_unclaimed(self, app):
+    def test_steps_added_to_unclaimed(self):
 
         class tnA(bootsteps.Step):
             name = 'test_Blueprint.A'
@@ -226,19 +226,18 @@ class test_Blueprint:
 
         class Blueprint(self.Blueprint):
             default_steps = [tnA, tnB]
-        blueprint = Blueprint(app=app)
+        blueprint = Blueprint()
 
-        assert tnA in blueprint._all_steps()
-        assert tnB in blueprint._all_steps()
-        assert xxA not in blueprint._all_steps()
+        assert tnA in blueprint.types
+        assert tnB in blueprint.types
+        assert xxA not in blueprint.types
 
-    def test_init(self, app):
-        blueprint = self.Blueprint(app=app)
-        assert blueprint.app is app
+    def test_init(self):
+        blueprint = self.Blueprint()
         assert blueprint.name == 'test_Blueprint'
 
-    def test_close__on_close_is_None(self, app):
-        blueprint = self.Blueprint(app=app)
+    def test_close__on_close_is_None(self):
+        blueprint = self.Blueprint()
         blueprint.on_close = None
         blueprint.send_all = Mock()
         blueprint.close(1)
@@ -246,30 +245,30 @@ class test_Blueprint:
             1, 'close', 'closing', reverse=False,
         )
 
-    def test_send_all_with_None_steps(self, app):
+    def test_send_all_with_None_steps(self):
         parent = Mock()
-        blueprint = self.Blueprint(app=app)
+        blueprint = self.Blueprint()
         parent.steps = [None, None, None]
         blueprint.send_all(parent, 'close', 'Closing', reverse=False)
 
-    def test_send_all_raises(self, app):
+    def test_send_all_raises(self):
         parent = Mock()
-        blueprint = self.Blueprint(app=app)
+        blueprint = self.Blueprint()
         parent.steps = [Mock()]
         parent.steps[0].foo.side_effect = KeyError()
         blueprint.send_all(parent, 'foo', propagate=False)
         with pytest.raises(KeyError):
             blueprint.send_all(parent, 'foo', propagate=True)
 
-    def test_stop_state_in_TERMINATE(self, app):
-        blueprint = self.Blueprint(app=app)
+    def test_stop_state_in_TERMINATE(self):
+        blueprint = self.Blueprint()
         blueprint.state = bootsteps.TERMINATE
         blueprint.stop(Mock())
 
-    def test_join_raises_IGNORE_ERRORS(self, app):
+    def test_join_raises_IGNORE_ERRORS(self):
         prev, bootsteps.IGNORE_ERRORS = bootsteps.IGNORE_ERRORS, (KeyError,)
         try:
-            blueprint = self.Blueprint(app=app)
+            blueprint = self.Blueprint()
             blueprint.shutdown_complete = Mock()
             blueprint.shutdown_complete.wait.side_effect = KeyError('luke')
             blueprint.join(timeout=10)
@@ -277,7 +276,7 @@ class test_Blueprint:
         finally:
             bootsteps.IGNORE_ERRORS = prev
 
-    def test_connect_with(self, app):
+    def test_connect_with(self):
 
         class b1s1(bootsteps.Step):
             pass
@@ -291,8 +290,8 @@ class test_Blueprint:
         class b2s2(bootsteps.Step):
             last = True
 
-        b1 = self.Blueprint([b1s1, b1s2], app=app)
-        b2 = self.Blueprint([b2s1, b2s2], app=app)
+        b1 = self.Blueprint([b1s1, b1s2])
+        b2 = self.Blueprint([b2s1, b2s2])
         b1.apply(Mock())
         b2.apply(Mock())
         b1.connect_with(b2)
@@ -304,12 +303,12 @@ class test_Blueprint:
         assert repr(b1s1)
         assert str(b1s1)
 
-    def test_topsort_raises_KeyError(self, app):
+    def test_topsort_raises_KeyError(self):
 
         class Step(bootsteps.Step):
             requires = ('xyxxx.fsdasewe.Unknown',)
 
-        b = self.Blueprint([Step], app=app)
+        b = self.Blueprint([Step])
         b.steps = b.claim_steps()
         with pytest.raises(ImportError):
             b._finalize_steps(b.steps)
@@ -324,7 +323,7 @@ class test_Blueprint:
             with pytest.raises(KeyError):
                 b._finalize_steps(b.steps)
 
-    def test_apply(self, app):
+    def test_apply(self):
 
         class MyBlueprint(bootsteps.Blueprint):
             name = 'test_apply'
@@ -347,7 +346,7 @@ class test_Blueprint:
             name = 'test_apply.D'
             last = True
 
-        x = MyBlueprint([A, D], app=app)
+        x = MyBlueprint([A, D])
         x.apply(self)
 
         assert isinstance(x.order[0], B)
@@ -357,11 +356,11 @@ class test_Blueprint:
         assert A in x.types
         assert x[A.name] is x.order[2]
 
-    def test_find_last_but_no_steps(self, app):
+    def test_find_last_but_no_steps(self):
 
         class MyBlueprint(bootsteps.Blueprint):
             name = 'qwejwioqjewoqiej'
 
-        x = MyBlueprint(app=app)
+        x = MyBlueprint()
         x.apply(self)
         assert x._find_last() is None