Ask Solem 8 år sedan
förälder
incheckning
8b1b156f39
7 ändrade filer med 342 tillägg och 239 borttagningar
  1. 1 0
      celery/app/base.py
  2. 191 183
      celery/bootsteps.py
  3. 6 5
      celery/states.py
  4. 30 0
      celery/utils/abstract.py
  5. 2 1
      celery/utils/timeutils.py
  6. 24 0
      celery/utils/typing.py
  7. 88 50
      celery/worker/__init__.py

+ 1 - 0
celery/app/base.py

@@ -98,6 +98,7 @@ class PendingConfiguration(UserDict, AttributeDictMixin):
         return self.callback(key)
 
 
+@abstract.AbstractApp.register
 class Celery:
     """Celery application.
 

+ 191 - 183
celery/bootsteps.py

@@ -2,14 +2,16 @@
 """A directed acyclic graph of reusable components."""
 from collections import deque
 from threading import Event
+from typing import Any, Callable, Mapping, Optional, Set, Sequence, Tuple
 
 from kombu.common import ignore_errors
 from kombu.utils import symbol_by_name
-from kombu.utils.encoding import bytes_to_str
 
+from .utils.abstract import AbstractApp
 from .utils.graph import DependencyGraph, GraphFormatter
 from .utils.imports import instantiate, qualname
 from .utils.log import get_logger
+from .utils.typing import Timeout
 
 try:
     from greenlet import GreenletExit
@@ -28,11 +30,155 @@ TERMINATE = 0x3
 logger = get_logger(__name__)
 
 
-def _pre(ns, fmt):
+class StepType(type):
+    """Meta-class for steps."""
+
+    def __new__(cls, name, bases, attrs):
+        module = attrs.get('__module__')
+        qname = '{0}.{1}'.format(module, name) if module else name
+        attrs.update(
+            __qualname__=qname,
+            name=attrs.get('name') or qname,
+        )
+        return super().__new__(cls, name, bases, attrs)
+
+    def __str__(self):
+        return self.name
+
+    def __repr__(self):
+        return 'step:{0.name}{{{0.requires!r}}}'.format(self)
+
+
+class Step(metaclass=StepType):
+    """A Bootstep.
+
+    The :meth:`__init__` method is called when the step
+    is bound to a parent object, and can as such be used
+    to initialize attributes in the parent object at
+    parent instantiation-time.
+    """
+
+    #: Optional step name, will use ``qualname`` if not specified.
+    name = None
+
+    #: Optional short name used for graph outputs and in logs.
+    label = None
+
+    #: Set this to true if the step is enabled based on some condition.
+    conditional = False
+
+    #: List of other steps that that must be started before this step.
+    #: Note that all dependencies must be in the same blueprint.
+    requires = ()
+
+    #: This flag is reserved for the workers Consumer,
+    #: since it is required to always be started last.
+    #: There can only be one object marked last
+    #: in every blueprint.
+    last = False
+
+    #: This provides the default for :meth:`include_if`.
+    enabled = True
+
+    def __init__(self, parent, **kwargs):
+        pass
+
+    def include_if(self, parent):
+        """An optional predicate that decides whether this
+        step should be created."""
+        return self.enabled
+
+    def instantiate(self, name, *args, **kwargs):
+        return instantiate(name, *args, **kwargs)
+
+    def _should_include(self, parent):
+        if self.include_if(parent):
+            return True, self.create(parent)
+        return False, None
+
+    def include(self, parent):
+        return self._should_include(parent)[0]
+
+    def create(self, parent):
+        """Create the step."""
+        pass
+
+    def __repr__(self):
+        return '<step: {0.alias}>'.format(self)
+
+    @property
+    def alias(self):
+        return self.label or _label(self)
+
+    def info(self, obj):
+        pass
+
+
+class StartStopStep(Step):
+
+    #: Optional obj created by the :meth:`create` method.
+    #: This is used by :class:`StartStopStep` to keep the
+    #: original service object.
+    obj = None
+
+    def start(self, parent):
+        if self.obj:
+            return self.obj.start()
+
+    def stop(self, parent):
+        if self.obj:
+            return self.obj.stop()
+
+    def close(self, parent):
+        pass
+
+    def terminate(self, parent):
+        if self.obj:
+            return getattr(self.obj, 'terminate', self.obj.stop)()
+
+    def include(self, parent):
+        inc, ret = self._should_include(parent)
+        if inc:
+            self.obj = ret
+            parent.steps.append(self)
+        return inc
+
+
+class ConsumerStep(StartStopStep):
+    requires = ('celery.worker.consumer:Connection',)
+    consumers = None
+
+    def get_consumers(self, channel):
+        raise NotImplementedError('missing get_consumers')
+
+    def start(self, c):
+        channel = c.connection.channel()
+        self.consumers = self.get_consumers(channel)
+        for consumer in self.consumers or []:
+            consumer.consume()
+
+    def stop(self, c):
+        self._close(c, True)
+
+    def shutdown(self, c):
+        self._close(c, False)
+
+    def _close(self, c, cancel_consumers=True):
+        channels = set()
+        for consumer in self.consumers or []:
+            if cancel_consumers:
+                ignore_errors(c.connection, consumer.cancel)
+            if consumer.channel:
+                channels.add(consumer.channel)
+        for channel in channels:
+            ignore_errors(c.connection, channel.close)
+
+
+def _pre(ns: Step, fmt: str) -> str:
     return '| {0}: {1}'.format(ns.alias, fmt)
 
 
-def _label(s):
+def _label(s: Step) -> str:
     return s.name.rsplit('.', 1)[-1]
 
 
@@ -47,25 +193,24 @@ class StepFormatter(GraphFormatter):
         'fillcolor': 'slategray3',
     }
 
-    def label(self, step):
+    def label(self, step: Step) -> str:
         return step and '{0}{1}'.format(
             self._get_prefix(step),
-            bytes_to_str(
-                (step.label or _label(step)).encode('utf-8', 'ignore')),
+            (step.label or _label(step)).encode('utf-8', 'ignore').decode(),
         )
 
-    def _get_prefix(self, step):
+    def _get_prefix(self, step: Step) -> str:
         if step.last:
             return self.blueprint_prefix
         if step.conditional:
             return self.conditional_prefix
         return ''
 
-    def node(self, obj, **attrs):
+    def node(self, obj: Any, **attrs) -> str:
         scheme = self.blueprint_scheme if obj.last else self.node_scheme
         return self.draw_node(obj, scheme, attrs)
 
-    def edge(self, a, b, **attrs):
+    def edge(self, a: Any, b: Any, **attrs) -> str:
         if a.last:
             attrs.update(arrowhead='none', color='darkseagreen3')
         return self.draw_edge(a, b, self.edge_scheme, attrs)
@@ -85,19 +230,23 @@ class Blueprint:
     """
     GraphFormatter = StepFormatter
 
-    name = None
-    state = None
-    started = 0
-    default_steps = set()
-    state_to_name = {
+    name = None                        # type: Optional[str]
+    state = None                       # type: Optional[int]
+    started = 0                        # type: int
+    default_steps = set()              # type: Set[Union[str, Step]]
+    state_to_name = {                  # type: Mapping[int, str]
         0: 'initializing',
         RUN: 'running',
         CLOSE: 'closing',
         TERMINATE: 'terminating',
     }
 
-    def __init__(self, steps=None, name=None, app=None,
-                 on_start=None, on_close=None, on_stopped=None):
+    def __init__(self, steps: Optional[Sequence]=None,
+                 name: Optional[str]=None,
+                 app: Optional[AbstractApp]=None,
+                 on_start: Optional[Callable[[], None]]=None,
+                 on_close: Optional[Callable[[], None]]=None,
+                 on_stopped: Optional[Callable[[], None]]=None) -> None:
         self.app = app
         self.name = name or self.name or qualname(type(self))
         self.types = set(steps or []) | set(self.default_steps)
@@ -105,9 +254,9 @@ class Blueprint:
         self.on_close = on_close
         self.on_stopped = on_stopped
         self.shutdown_complete = Event()
-        self.steps = {}
+        self.steps = {}  # type: Mapping[str, Step]
 
-    def start(self, parent):
+    def start(self, parent: Any) -> None:
         self.state = RUN
         if self.on_start:
             self.on_start()
@@ -117,26 +266,28 @@ class Blueprint:
             step.start(parent)
             logger.debug('^-- substep ok')
 
-    def human_state(self):
+    def human_state(self) -> str:
         return self.state_to_name[self.state or 0]
 
-    def info(self, parent):
+    def info(self, parent: Any) -> Mapping[str, Any]:
         info = {}
         for step in parent.steps:
             info.update(step.info(parent) or {})
         return info
 
-    def close(self, parent):
+    def close(self, parent: Any) -> None:
         if self.on_close:
             self.on_close()
         self.send_all(parent, 'close', 'closing', reverse=False)
 
-    def restart(self, parent, method='stop',
-                description='restarting', propagate=False):
+    def restart(self, parent: Any, method: str='stop',
+                description: str='restarting', propagate: bool=False) -> None:
         self.send_all(parent, method, description, propagate=propagate)
 
-    def send_all(self, parent, method,
-                 description=None, reverse=True, propagate=True, args=()):
+    def send_all(self, parent: Any, method: str,
+                 description: Optional[str]=None,
+                 reverse: bool=True, propagate: bool=True,
+                 args: Sequence=()) -> None:
         description = description or method.replace('_', ' ')
         steps = reversed(parent.steps) if reverse else parent.steps
         for step in steps:
@@ -155,7 +306,8 @@ class Blueprint:
                             description, step.alias, exc, exc_info=1,
                         )
 
-    def stop(self, parent, close=True, terminate=False):
+    def stop(self, parent: Any,
+             close: bool=True, terminate: bool=False) -> None:
         what = 'terminating' if terminate else 'stopping'
         if self.state in (CLOSE, TERMINATE):
             return
@@ -178,7 +330,7 @@ class Blueprint:
         self.state = TERMINATE
         self.shutdown_complete.set()
 
-    def join(self, timeout=None):
+    def join(self, timeout: Timeout=None) -> None:
         try:
             # Will only get here if running green,
             # makes sure all greenthreads have exited.
@@ -186,7 +338,7 @@ class Blueprint:
         except IGNORE_ERRORS:
             pass
 
-    def apply(self, parent, **kwargs):
+    def apply(self, parent: Any, **kwargs) -> 'Blueprint':
         """Apply the steps in this blueprint to an object.
 
         This will apply the ``__init__`` and ``include`` methods
@@ -214,17 +366,17 @@ class Blueprint:
             step.include(parent)
         return self
 
-    def connect_with(self, other):
+    def connect_with(self, other: 'Blueprint') -> None:
         self.graph.adjacent.update(other.graph.adjacent)
         self.graph.add_edge(type(other.order[0]), type(self.order[-1]))
 
-    def __getitem__(self, name):
+    def __getitem__(self, name: str) -> Step:
         return self.steps[name]
 
-    def _find_last(self):
+    def _find_last(self) -> Optional[Step]:
         return next((C for C in self.steps.values() if C.last), None)
 
-    def _firstpass(self, steps):
+    def _firstpass(self, steps: Mapping[str, Step]) -> None:
         for step in steps.values():
             step.requires = [symbol_by_name(dep) for dep in step.requires]
         stream = deque(step.requires for step in steps.values())
@@ -235,7 +387,7 @@ class Blueprint:
                     steps[node.name] = node
                 stream.append(node.requires)
 
-    def _finalize_steps(self, steps):
+    def _finalize_steps(self, steps: Mapping[str, Step]) -> Sequence[Step]:
         last = self._find_last()
         self._firstpass(steps)
         it = ((C, C.requires) for C in steps.values())
@@ -251,163 +403,19 @@ class Blueprint:
         except KeyError as exc:
             raise KeyError('unknown bootstep: %s' % exc)
 
-    def claim_steps(self):
+    def claim_steps(self) -> Mapping[str, Step]:
         return dict(self.load_step(step) for step in self._all_steps())
 
-    def _all_steps(self):
+    def _all_steps(self) -> Set:
         return self.types | self.app.steps[self.name.lower()]
 
-    def load_step(self, step):
+    def load_step(self, step: Step) -> Tuple[str, Step]:
         step = symbol_by_name(step)
         return step.name, step
 
-    def _debug(self, msg, *args):
-        return logger.debug(_pre(self, msg), *args)
+    def _debug(self, msg: str, *args) -> None:
+        logger.debug(_pre(self, msg), *args)
 
     @property
-    def alias(self):
+    def alias(self) -> str:
         return _label(self)
-
-
-class StepType(type):
-    """Meta-class for steps."""
-
-    def __new__(cls, name, bases, attrs):
-        module = attrs.get('__module__')
-        qname = '{0}.{1}'.format(module, name) if module else name
-        attrs.update(
-            __qualname__=qname,
-            name=attrs.get('name') or qname,
-        )
-        return super().__new__(cls, name, bases, attrs)
-
-    def __str__(self):
-        return self.name
-
-    def __repr__(self):
-        return 'step:{0.name}{{{0.requires!r}}}'.format(self)
-
-
-class Step(metaclass=StepType):
-    """A Bootstep.
-
-    The :meth:`__init__` method is called when the step
-    is bound to a parent object, and can as such be used
-    to initialize attributes in the parent object at
-    parent instantiation-time.
-    """
-
-    #: Optional step name, will use ``qualname`` if not specified.
-    name = None
-
-    #: Optional short name used for graph outputs and in logs.
-    label = None
-
-    #: Set this to true if the step is enabled based on some condition.
-    conditional = False
-
-    #: List of other steps that that must be started before this step.
-    #: Note that all dependencies must be in the same blueprint.
-    requires = ()
-
-    #: This flag is reserved for the workers Consumer,
-    #: since it is required to always be started last.
-    #: There can only be one object marked last
-    #: in every blueprint.
-    last = False
-
-    #: This provides the default for :meth:`include_if`.
-    enabled = True
-
-    def __init__(self, parent, **kwargs):
-        pass
-
-    def include_if(self, parent):
-        """An optional predicate that decides whether this
-        step should be created."""
-        return self.enabled
-
-    def instantiate(self, name, *args, **kwargs):
-        return instantiate(name, *args, **kwargs)
-
-    def _should_include(self, parent):
-        if self.include_if(parent):
-            return True, self.create(parent)
-        return False, None
-
-    def include(self, parent):
-        return self._should_include(parent)[0]
-
-    def create(self, parent):
-        """Create the step."""
-        pass
-
-    def __repr__(self):
-        return '<step: {0.alias}>'.format(self)
-
-    @property
-    def alias(self):
-        return self.label or _label(self)
-
-    def info(self, obj):
-        pass
-
-
-class StartStopStep(Step):
-
-    #: Optional obj created by the :meth:`create` method.
-    #: This is used by :class:`StartStopStep` to keep the
-    #: original service object.
-    obj = None
-
-    def start(self, parent):
-        if self.obj:
-            return self.obj.start()
-
-    def stop(self, parent):
-        if self.obj:
-            return self.obj.stop()
-
-    def close(self, parent):
-        pass
-
-    def terminate(self, parent):
-        if self.obj:
-            return getattr(self.obj, 'terminate', self.obj.stop)()
-
-    def include(self, parent):
-        inc, ret = self._should_include(parent)
-        if inc:
-            self.obj = ret
-            parent.steps.append(self)
-        return inc
-
-
-class ConsumerStep(StartStopStep):
-    requires = ('celery.worker.consumer:Connection',)
-    consumers = None
-
-    def get_consumers(self, channel):
-        raise NotImplementedError('missing get_consumers')
-
-    def start(self, c):
-        channel = c.connection.channel()
-        self.consumers = self.get_consumers(channel)
-        for consumer in self.consumers or []:
-            consumer.consume()
-
-    def stop(self, c):
-        self._close(c, True)
-
-    def shutdown(self, c):
-        self._close(c, False)
-
-    def _close(self, c, cancel_consumers=True):
-        channels = set()
-        for consumer in self.consumers or []:
-            if cancel_consumers:
-                ignore_errors(c.connection, consumer.cancel)
-            if consumer.channel:
-                channels.add(consumer.channel)
-        for channel in channels:
-            ignore_errors(c.connection, channel.close)

+ 6 - 5
celery/states.py

@@ -52,6 +52,7 @@ Misc.
 -----
 
 """
+from typing import Any, Optional
 
 __all__ = [
     'PENDING', 'RECEIVED', 'STARTED', 'SUCCESS', 'FAILURE',
@@ -79,7 +80,7 @@ PRECEDENCE_LOOKUP = dict(zip(PRECEDENCE, range(0, len(PRECEDENCE))))
 NONE_PRECEDENCE = PRECEDENCE_LOOKUP[None]
 
 
-def precedence(state):
+def precedence(state: Optional[str]) -> int:
     """Get the precedence index for state.
 
     Lower index means higher precedence.
@@ -109,16 +110,16 @@ class state(str):
         False
     """
 
-    def __gt__(self, other):
+    def __gt__(self, other: Any) -> bool:
         return precedence(self) < precedence(other)
 
-    def __ge__(self, other):
+    def __ge__(self, other: Any) -> bool:
         return precedence(self) <= precedence(other)
 
-    def __lt__(self, other):
+    def __lt__(self, other: Any) -> bool:
         return precedence(self) > precedence(other)
 
-    def __le__(self, other):
+    def __le__(self, other: Any) -> bool:
         return precedence(self) >= precedence(other)
 
 #: Task state is unknown (assumed pending since you know the id).

+ 30 - 0
celery/utils/abstract.py

@@ -27,6 +27,36 @@ class _AbstractClass(metaclass=ABCMeta):
         return other
 
 
+class AbstractApp(_AbstractClass):  # pragma: no cover
+    __required_attributes = frozenset({
+        'close' 'start', 'task', 'AsyncResult', 'finalize',
+    })
+
+    @abstractmethod
+    def close(self):
+        pass
+
+    @abstractmethod
+    def start(self):
+        pass
+
+    @abstractmethod
+    def task(self) -> 'CallableTask':
+        pass
+
+    @abstractmethod
+    def finalize(self):
+        pass
+
+    @abstractproperty
+    def conf(self):
+        pass
+
+    @abstractproperty
+    def AsyncResult(self):
+        pass
+
+
 class CallableTask(_AbstractClass, Callable):  # pragma: no cover
     __required_attributes__ = frozenset({
         'delay', 'apply_async', 'apply',

+ 2 - 1
celery/utils/timeutils.py

@@ -6,7 +6,7 @@ import time as _time
 
 from calendar import monthrange
 from datetime import date, datetime, timedelta, tzinfo
-from typing import Any, Dict, Optional, Union
+from typing import Any, Callable, Dict, Optional, Union
 
 from kombu.utils import cached_property, reprcall
 
@@ -321,6 +321,7 @@ class ffwd:
                  month: Optional[int]=None,
                  weeks: int=0,
                  weekday: Optional[int]=None,
+                 day: Optional[int]=None,
                  hour: Optional[int]=None,
                  minute: Optional[int]=None,
                  second: Optional[numbers.Number]=None,

+ 24 - 0
celery/utils/typing.py

@@ -0,0 +1,24 @@
+from datetime import datetime, timedelta
+from numbers import Number
+
+from typing import (
+    Any, AnyStr, Callable, Iterator, Mapping,
+    Optional, Sequence, SupportsInt, Tuple, Union,
+)
+
+Timeout = Optional[Number]
+
+Interval = Union[Number, timedelta]
+
+Int = Union[SupportsInt, AnyStr]
+
+DatetimeNowFun = Callable[[], datetime]
+
+#: Argument to `dict(x)` like function.
+DictArgument = Union[
+    Iterator[Tuple[str, Any]],
+    Sequence[Tuple[str, Any]],
+    Mapping,
+]
+
+ExcInfo = Tuple[Any, Any, Any]

+ 88 - 50
celery/worker/__init__.py

@@ -20,18 +20,24 @@ try:
 except ImportError:  # pragma: no cover
     resource = None  # noqa
 
+from numbers import Number
+from typing import Any, Callable, Mapping, Optional, Sequence, Union
+
 from billiard import cpu_count
 from billiard.util import Finalize
+from kombu.async import Hub
 from kombu.syn import detect_environment
 
 from celery import bootsteps
 from celery.bootsteps import RUN, TERMINATE
 from celery import concurrency as _concurrency
+from celery.concurrency.base import BasePool  # noqa
 from celery import signals
 from celery.exceptions import (
     ImproperlyConfigured, WorkerTerminate, TaskRevokedError,
 )
-from celery.platforms import EX_FAILURE, create_pidlock
+from celery.platforms import EX_FAILURE, Pidfile, create_pidlock
+from celery.utils.abstract import AbstractApp
 from celery.utils.imports import reload_from_cwd
 from celery.utils.log import mlevel, worker_logger as logger
 from celery.utils.nodenames import default_nodename, worker_direct
@@ -58,19 +64,22 @@ Trying to deselect queue subset of {0!r}, but queue {1} is not
 defined in the `task_queues` setting.
 """
 
+CsvArgument = Union[Sequence[str], str]
+PidfileArgument = Union[Pidfile, str]
+
 
 class WorkController:
     """Unmanaged worker instance."""
 
-    app = None
+    app = None        # type: Optional[AbstractApp]
 
-    pidlock = None
-    blueprint = None
-    pool = None
-    semaphore = None
+    pidlock = None    # type: Optional[Pidfile]
+    blueprint = None  # type: bootsteps.Blueprint
+    pool = None       # type: Optional[BasePool]
+    semaphore = None  # type: threading.Semaphore
 
     #: contains the exit code if a :exc:`SystemExit` event is handled.
-    exitcode = None
+    exitcode = None   # type: Optional[int]
 
     class Blueprint(bootsteps.Blueprint):
         """Worker bootstep blueprint."""
@@ -85,7 +94,10 @@ class WorkController:
             'celery.worker.components:Consumer',
         }
 
-    def __init__(self, app=None, hostname=None, **kwargs):
+    def __init__(self,
+                 app: Optional[AbstractApp]=None,
+                 hostname: Optional[str]=None,
+                 **kwargs) -> None:
         self.app = app or self.app
         self.hostname = default_nodename(hostname)
         self.app.loader.init_worker()
@@ -98,9 +110,15 @@ class WorkController:
             Finalize(self, self._send_worker_shutdown, exitpriority=10),
         ]
 
-    def setup_instance(self, queues=None, ready_callback=None, pidfile=None,
-                       include=None, use_eventloop=None, exclude_queues=None,
-                       **kwargs):
+    def setup_instance(
+            self,
+            queues: Optional[CsvArgument]=None,
+            ready_callback: Optional[Callable[[Any], Any]]=None,
+            pidfile: Optional[PidfileArgument]=None,
+            include: Optional[CsvArgument]=None,
+            use_eventloop: Optional[bool]=None,
+            exclude_queues: Optional[CsvArgument]=None,
+            **kwargs) -> None:
         self.pidfile = pidfile
         self.setup_queues(queues, exclude_queues)
         self.setup_includes(str_to_list(include))
@@ -136,33 +154,34 @@ class WorkController:
                                         on_stopped=self.on_stopped)
         self.blueprint.apply(self, **kwargs)
 
-    def on_init_blueprint(self):
+    def on_init_blueprint(self) -> None:
         pass
 
-    def on_before_init(self, **kwargs):
+    def on_before_init(self, **kwargs) -> None:
         pass
 
-    def on_after_init(self, **kwargs):
+    def on_after_init(self, **kwargs) -> None:
         pass
 
-    def on_start(self):
+    def on_start(self) -> None:
         if self.pidfile:
             self.pidlock = create_pidlock(self.pidfile)
 
-    def on_consumer_ready(self, consumer):
+    def on_consumer_ready(self, consumer: Any) -> None:
         pass
 
-    def on_close(self):
+    def on_close(self) -> None:
         self.app.loader.shutdown_worker()
 
-    def on_stopped(self):
+    def on_stopped(self) -> None:
         self.timer.stop()
         self.consumer.shutdown()
 
         if self.pidlock:
             self.pidlock.release()
 
-    def setup_queues(self, include, exclude=None):
+    def setup_queues(self, include: Optional[CsvArgument],
+                     exclude: Optional[CsvArgument]=None) -> None:
         include = str_to_list(include)
         exclude = str_to_list(exclude)
         try:
@@ -178,7 +197,7 @@ class WorkController:
         if self.app.conf.worker_direct:
             self.app.amqp.queues.select_add(worker_direct(self.hostname))
 
-    def setup_includes(self, includes):
+    def setup_includes(self, includes: Optional[CsvArgument]) -> None:
         # Update celery_include to have all known task modules, so that we
         # ensure all task modules are imported in case an execv happens.
         prev = tuple(self.app.conf.include)
@@ -190,13 +209,13 @@ class WorkController:
                         for task in self.app.tasks.values()}
         self.app.conf.include = tuple(set(prev) | task_modules)
 
-    def prepare_args(self, **kwargs):
+    def prepare_args(self, **kwargs) -> Mapping:
         return kwargs
 
-    def _send_worker_shutdown(self):
+    def _send_worker_shutdown(self) -> None:
         signals.worker_shutdown.send(sender=self)
 
-    def start(self):
+    def start(self) -> None:
         """Starts the workers main loop."""
         try:
             self.blueprint.start(self)
@@ -210,16 +229,16 @@ class WorkController:
         except KeyboardInterrupt:
             self.stop(exitcode=EX_FAILURE)
 
-    def register_with_event_loop(self, hub):
+    def register_with_event_loop(self, hub: Hub) -> None:
         self.blueprint.send_all(
             self, 'register_with_event_loop', args=(hub,),
             description='hub.register',
         )
 
-    def _process_task_sem(self, req):
+    def _process_task_sem(self, req: Any) -> Any:
         return self._quick_acquire(self._process_task, req)
 
-    def _process_task(self, req):
+    def _process_task(self, req: Any) -> None:
         """Process task by sending it to the pool of workers."""
         try:
             req.execute_using_pool(self.pool)
@@ -232,18 +251,19 @@ class WorkController:
             logger.critical('Internal error: %r\n%s',
                             exc, traceback.format_exc(), exc_info=True)
 
-    def signal_consumer_close(self):
+    def signal_consumer_close(self) -> None:
         try:
             self.consumer.close()
         except AttributeError:
             pass
 
-    def should_use_eventloop(self):
+    def should_use_eventloop(self) -> bool:
         return (detect_environment() == 'default' and
                 self._conninfo.transport.implements.async and
                 not self.app.IS_WINDOWS)
 
-    def stop(self, in_sighandler=False, exitcode=None):
+    def stop(self, in_sighandler: bool=False,
+             exitcode: Optional[int]=None) -> None:
         """Graceful shutdown of the worker server."""
         if exitcode is not None:
             self.exitcode = exitcode
@@ -252,14 +272,14 @@ class WorkController:
             if not in_sighandler or self.pool.signal_safe:
                 self._shutdown(warm=True)
 
-    def terminate(self, in_sighandler=False):
+    def terminate(self, in_sighandler: bool=False) -> None:
         """Not so graceful shutdown of the worker server."""
         if self.blueprint.state != TERMINATE:
             self.signal_consumer_close()
             if not in_sighandler or self.pool.signal_safe:
                 self._shutdown(warm=False)
 
-    def _shutdown(self, warm=True):
+    def _shutdown(self, warm: bool=True) -> None:
         # if blueprint does not exist it means that we had an
         # error before the bootsteps could be initialized.
         if self.blueprint is not None:
@@ -267,7 +287,8 @@ class WorkController:
                 self.blueprint.stop(self, terminate=not warm)
                 self.blueprint.join()
 
-    def reload(self, modules=None, reload=False, reloader=None):
+    def reload(self, modules: Optional[Sequence[str]]=None,
+               reload: bool=False, reloader: Optional[Callable]=None) -> None:
         list(self._reload_modules(
             modules, force_reload=reload, reloader=reloader))
 
@@ -279,14 +300,18 @@ class WorkController:
         except NotImplementedError:
             pass
 
-    def _reload_modules(self, modules=None, **kwargs):
+    def _reload_modules(
+            self,
+            modules: Optional[Sequence[str]]=None, **kwargs) -> Sequence[Any]:
         return (
             self._maybe_reload_module(m, **kwargs)
             for m in set(self.app.loader.task_modules
                          if modules is None else (modules or ()))
         )
 
-    def _maybe_reload_module(self, module, force_reload=False, reloader=None):
+    def _maybe_reload_module(self, module: str,
+                             force_reload: bool=False,
+                             reloader: Optional[Callable]=None) -> Any:
         if module not in sys.modules:
             logger.debug('importing module %s', module)
             return self.app.loader.import_from_cwd(module)
@@ -294,12 +319,12 @@ class WorkController:
             logger.debug('reloading module %s', module)
             return reload_from_cwd(sys.modules[module], reloader)
 
-    def info(self):
+    def info(self) -> Mapping:
         return {'total': self.state.total_count,
                 'pid': os.getpid(),
                 'clock': str(self.app.clock)}
 
-    def rusage(self):
+    def rusage(self) -> Mapping:
         if resource is None:
             raise NotImplementedError('rusage not supported by this platform')
         s = resource.getrusage(resource.RUSAGE_SELF)
@@ -322,7 +347,7 @@ class WorkController:
             'nivcsw': s.ru_nivcsw,
         }
 
-    def stats(self):
+    def stats(self) -> Mapping:
         info = self.info()
         info.update(self.blueprint.info(self))
         info.update(self.consumer.blueprint.info(self.consumer))
@@ -332,29 +357,42 @@ class WorkController:
             info['rusage'] = 'N/A'
         return info
 
-    def __repr__(self):
+    def __repr__(self) -> str:
         return '<Worker: {self.hostname} ({state})>'.format(
             self=self,
             state=self.blueprint.human_state() if self.blueprint else 'INIT',
         )
 
-    def __str__(self):
+    def __str__(self) -> str:
         return self.hostname
 
     @property
-    def state(self):
+    def state(self) -> int:
         return state
 
-    def setup_defaults(self, concurrency=None, loglevel='WARN', logfile=None,
-                       send_events=None, pool_cls=None, consumer_cls=None,
-                       timer_cls=None, timer_precision=None,
-                       pool_putlocks=None, pool_restarts=None,
-                       state_db=None, schedule_filename=None,
-                       scheduler_cls=None, task_time_limit=None,
-                       task_soft_time_limit=None, max_tasks_per_child=None,
-                       prefetch_multiplier=None, disable_rate_limits=None,
-                       worker_lost_wait=None,
-                       max_memory_per_child=None, **_kw):
+    def setup_defaults(
+            self,
+            concurrency: Optional[int]=None,
+            loglevel: Union[str, int]='WARN',
+            logfile: Optional[str]=None,
+            send_events: Optional[bool]=None,
+            pool_cls: Optional[Any]=None,
+            consumer_cls: Optional[Any]=None,
+            timer_cls: Optional[Any]=None,
+            timer_precision: Optional[Number]=None,
+            pool_putlocks: Optional[bool]=None,
+            pool_restarts: Optional[bool]=None,
+            state_db: Optional[str]=None,
+            schedule_filename: Optional[str]=None,
+            scheduler_cls: Optional[Any]=None,
+            task_time_limit: Optional[Number]=None,
+            task_soft_time_limit: Optional[Number]=None,
+            max_tasks_per_child: Optional[int]=None,
+            prefetch_multiplier: Optional[int]=None,
+            disable_rate_limits: Optional[bool]=None,
+            worker_lost_wait: Optional[Number]=None,
+            max_memory_per_child: Optional[Union[str, Number]]=None,
+            **_kw) -> None:
         either = self.app.either
         self.loglevel = loglevel
         self.logfile = logfile