Prechádzať zdrojové kódy

Commands: inspect/control now takes commands from registry

This means user remote-control commands can also be used from the
command-line.

Note that you need to specify the arguments/and type of arguments
for the arguments to be correctly passed on the command-line.

There are now two decorators, which use depends on the type of
command: `@inspect_command` + `@control_command`:

```
from celery.worker.control import control_command

@control_command(
    args=[('n', int)]
    signature='[N=1]',
)
def something(state, n=1, **kwargs):
    ...
```

Here `args` is a list of args supported by the command.
The list must contain tuples of `(argument_name, type)`.

`signature` is just the command-line help used in e.g.
`celery -A proj control --help`.

Commands also support `variadic` arguments, which means that any
arguments left over will be added to a single variable.  Here demonstrated
by the ``terminate`` command which takes a signal argument and a variable
number of task_ids:

```
from celery.worker.control import control_command

@control_command(
    args=[('signal', str)],
    signature='<signal> [id1, [id2, [..., [idN]]]]',
    variadic='ids',
)
def terminate(state, signal, ids, **kwargs):
    ...
```

This command can now be called using:

`$ celery -A proj control terminate SIGKILL id1 id2 id3`
Ask Solem 8 rokov pred
rodič
commit
8b7fa9076a

+ 12 - 9
celery/app/control.py

@@ -77,23 +77,26 @@ class Inspect(object):
     def clock(self):
         return self._request('clock')
 
-    def active(self, safe=False):
-        return self._request('dump_active', safe=safe)
+    def active(self, safe=None):
+        # safe is ignored since 4.0
+        # as we now have argsrepr/kwargsrepr which means no objects
+        # will need to be serialized.
+        return self._request('active')
 
-    def scheduled(self, safe=False):
-        return self._request('dump_schedule', safe=safe)
+    def scheduled(self, safe=None):
+        return self._request('scheduled')
 
-    def reserved(self, safe=False):
-        return self._request('dump_reserved', safe=safe)
+    def reserved(self, safe=None):
+        return self._request('reserved')
 
     def stats(self):
         return self._request('stats')
 
     def revoked(self):
-        return self._request('dump_revoked')
+        return self._request('revoked')
 
     def registered(self, *taskinfoitems):
-        return self._request('dump_tasks', taskinfoitems=taskinfoitems)
+        return self._request('registered', taskinfoitems=taskinfoitems)
     registered_tasks = registered
 
     def ping(self):
@@ -106,7 +109,7 @@ class Inspect(object):
         return self._request('query_task', ids=ids)
 
     def conf(self, with_defaults=False):
-        return self._request('dump_conf', with_defaults=with_defaults)
+        return self._request('conf', with_defaults=with_defaults)
 
     def hello(self, from_node, revoked=None):
         return self._request('hello', from_node=from_node, revoked=revoked)

+ 119 - 123
celery/bin/celery.py

@@ -264,10 +264,11 @@ import sys
 from functools import partial
 from importlib import import_module
 
-from kombu.utils import json
+from kombu.utils.json import dumps, loads
+from kombu.utils.objects import cached_property
 
 from celery.app import defaults
-from celery.five import keys, string_t, values
+from celery.five import items, keys, string_t, values
 from celery.platforms import EX_OK, EX_FAILURE, EX_UNAVAILABLE, EX_USAGE
 from celery.utils import term
 from celery.utils import text
@@ -424,8 +425,8 @@ class call(Command):
                    queue=None, exchange=None, routing_key=None,
                    eta=None, expires=None):
         # arguments
-        args = json.loads(args) if isinstance(args, string_t) else args
-        kwargs = json.loads(kwargs) if isinstance(kwargs, string_t) else kwargs
+        args = loads(args) if isinstance(args, string_t) else args
+        kwargs = loads(kwargs) if isinstance(kwargs, string_t) else kwargs
 
         # Expires can be int/float.
         try:
@@ -545,8 +546,8 @@ class result(Command):
 class _RemoteControl(Command):
 
     name = None
-    choices = None
     leaf = False
+    control_group = None
 
     option_list = Command.option_list + (
         Option('--timeout', '-t', type='float',
@@ -564,38 +565,31 @@ class _RemoteControl(Command):
 
     @classmethod
     def get_command_info(self, command,
-                         indent=0, prefix='', color=None, help=False):
+                         indent=0, prefix='', color=None,
+                         help=False, app=None, choices=None):
+        if choices is None:
+            choices = self._choices_by_group(app)
+        meta = choices[command]
         if help:
-            help = '|' + text.indent(self.choices[command][1], indent + 4)
+            help = '|' + text.indent(meta.help, indent + 4)
         else:
             help = None
-        try:
-            # see if it uses args.
-            meth = getattr(self, command)
-            return text.join([
-                '|' + text.indent('{0}{1} {2}'.format(
-                    prefix, color(command), meth.__doc__), indent),
-                help,
-            ])
-
-        except AttributeError:
-            return text.join([
-                '|' + text.indent(prefix + str(color(command)), indent), help,
-            ])
+        return text.join([
+            '|' + text.indent('{0}{1} {2}'.format(
+                prefix, color(command), meta.signature or ''), indent),
+            help,
+        ])
 
     @classmethod
-    def list_commands(self, indent=0, prefix='', color=None, help=False):
+    def list_commands(self, indent=0, prefix='',
+                      color=None, help=False, app=None):
+        choices = self._choices_by_group(app)
         color = color if color else lambda x: x
         prefix = prefix + ' ' if prefix else ''
-        return '\n'.join(self.get_command_info(c, indent, prefix, color, help)
-                         for c in sorted(self.choices))
-
-    @property
-    def epilog(self):
-        return '\n'.join([
-            '[Commands]',
-            self.list_commands(indent=4, help=True)
-        ])
+        return '\n'.join(
+            self.get_command_info(c, indent, prefix, color, help,
+                                  app=app, choices=choices)
+            for c in sorted(choices))
 
     def usage(self, command):
         return '%prog {0} [options] {1} <command> [arg1 .. argN]'.format(
@@ -610,37 +604,99 @@ class _RemoteControl(Command):
                 'Missing {0.name} method. See --help'.format(self))
         return self.do_call_method(args, **kwargs)
 
-    def do_call_method(self, args, **kwargs):
+    def _ensure_fanout_supported(self):
+        with self.app.connection_for_write() as conn:
+            if not conn.supports_exchange_type('fanout'):
+                raise self.Error(
+                    'Broadcast not supported by transport {0!r}'.format(
+                        conn.info()['transport']))
+
+    def do_call_method(self, args,
+                       timeout=None, destination=None, json=False, **kwargs):
         method = args[0]
         if method == 'help':
             raise self.Error("Did you mean '{0.name} --help'?".format(self))
-        if method not in self.choices:
+        try:
+            meta = self.choices[method]
+        except KeyError:
             raise self.UsageError(
                 'Unknown {0.name} method {1}'.format(self, method))
 
-        if self.app.connection_for_write().transport.driver_type == 'sql':
-            raise self.Error('Broadcast not supported by SQL broker transport')
+        self._ensure_fanout_supported()
 
-        output_json = kwargs.get('json')
-        destination = kwargs.get('destination')
-        timeout = kwargs.get('timeout') or self.choices[method][0]
+        timeout = timeout or meta.default_timeout
         if destination and isinstance(destination, string_t):
             destination = [dest.strip() for dest in destination.split(',')]
 
-        handler = getattr(self, method, self.call)
-
-        callback = None if output_json else self.say_remote_command_reply
-
-        replies = handler(method, *args[1:], timeout=timeout,
-                          destination=destination,
-                          callback=callback)
+        replies = self.call(
+            method,
+            arguments=self.compile_arguments(meta, method, args[1:]),
+            timeout=timeout,
+            destination=destination,
+            callback=None if json else self.say_remote_command_reply,
+        )
         if not replies:
             raise self.Error('No nodes replied within time constraint.',
                              status=EX_UNAVAILABLE)
-        if output_json:
-            self.out(json.dumps(replies))
+        if json:
+            self.out(dumps(replies))
         return replies
 
+    def compile_arguments(self, meta, method, args):
+        args = list(args)
+        kw = {}
+        if meta.args:
+            kw.update({
+                k: v for k, v in self._consume_args(meta, method, args)
+            })
+        if meta.variadic:
+            kw.update({meta.variadic: args})
+        if not kw and args:
+            raise self.Error(
+                'Command {0!r} takes no arguments.'.format(method),
+                status=EX_USAGE)
+        return kw or {}
+
+    def _consume_args(self, meta, method, args):
+        i = 0
+        try:
+            for i, arg in enumerate(args):
+                try:
+                    name, typ = meta.args[i]
+                except IndexError:
+                    if meta.variadic:
+                        break
+                    raise self.Error(
+                        'Command {0!r} takes arguments: {1}'.format(
+                            method, meta.signature),
+                        status=EX_USAGE)
+                else:
+                    yield name, typ(arg) if typ is not None else arg
+        finally:
+            args[:] = args[i:]
+
+    @classmethod
+    def _choices_by_group(self, app):
+        from celery.worker.control import Panel
+        # need to import task modules for custom user-remote control commands.
+        app.loader.import_default_modules()
+
+        return {
+            name: info for name, info in items(Panel.meta)
+            if info.type == self.control_group and info.visible
+        }
+
+    @cached_property
+    def choices(self):
+        return self._choices_by_group(self.app)
+
+    @property
+    def epilog(self):
+        return '\n'.join([
+            '[Commands]',
+            self.list_commands(indent=4, help=True, app=self.app)
+        ])
+
 
 class inspect(_RemoteControl):
     """Inspect the worker at runtime.
@@ -656,37 +712,11 @@ class inspect(_RemoteControl):
     """
 
     name = 'inspect'
+    control_group = 'inspect'
 
-    choices = {
-        'active': (1.0, 'dump active tasks (being processed)'),
-        'active_queues': (1.0, 'dump queues being consumed from'),
-        'clock': (1.0, 'get value of logical clock'),
-        'conf': (1.0, 'dump worker configuration'),
-        'memdump': (1.0, 'dump memory samples (requires psutil)'),
-        'memsample': (1.0, 'sample memory (requires psutil)'),
-        'objgraph': (60.0, 'create object graph (requires objgraph)'),
-        'ping': (0.2, 'ping worker(s)'),
-        'query_task': (1.0, 'query for task information by id'),
-        'reserved': (1.0, 'dump reserved tasks (waiting to be processed)'),
-        'scheduled': (1.0, 'dump scheduled tasks (eta/countdown/retry)'),
-        'stats': (1.0, 'dump worker statistics'),
-        'registered': (1.0, 'dump of registered tasks'),
-        'report': (1.0, 'get bugreport info'),
-        'revoked': (1.0, 'dump of revoked task ids'),
-    }
-
-    def call(self, method, *args, **options):
-        i = self.app.control.inspect(**options)
-        return getattr(i, method)(*args)
-
-    def objgraph(self, type_='Request', *args, **kwargs):
-        return self.call('objgraph', type_, **kwargs)
-
-    def conf(self, with_defaults=False, *args, **kwargs):
-        return self.call('conf', with_defaults, **kwargs)
-
-    def query_task(self, *ids, **options):
-        return self.call('query_task', ids, **options)
+    def call(self, method, arguments, **options):
+        return self.app.control.inspect(**options)._request(
+            method, **arguments)
 
 
 class control(_RemoteControl):
@@ -708,49 +738,11 @@ class control(_RemoteControl):
     """
 
     name = 'control'
+    control_group = 'control'
 
-    choices = {
-        'enable_events': (1.0, 'tell worker(s) to enable events'),
-        'disable_events': (1.0, 'tell worker(s) to disable events'),
-        'add_consumer': (1.0, 'tell worker(s) to start consuming a queue'),
-        'cancel_consumer': (1.0, 'tell worker(s) to stop consuming a queue'),
-        'rate_limit': (
-            1.0, 'tell worker(s) to modify the rate limit for a task type'),
-        'time_limit': (
-            1.0, 'tell worker(s) to modify the time limit for a task type.'),
-        'pool_grow': (1.0, 'start more pool processes'),
-        'pool_shrink': (1.0, 'use less pool processes'),
-    }
-
-    def call(self, method, *args, **options):
-        return getattr(self.app.control, method)(*args, reply=True, **options)
-
-    def pool_grow(self, method, n=1, **kwargs):
-        """[N=1]"""
-        return self.call(method, int(n), **kwargs)
-
-    def pool_shrink(self, method, n=1, **kwargs):
-        """[N=1]"""
-        return self.call(method, int(n), **kwargs)
-
-    def rate_limit(self, method, task_name, rate_limit, **kwargs):
-        """<task_name> <rate_limit> (e.g. 5/s | 5/m | 5/h)>"""
-        return self.call(method, task_name, rate_limit, **kwargs)
-
-    def time_limit(self, method, task_name, soft, hard=None, **kwargs):
-        """<task_name> <soft_secs> [hard_secs]"""
-        return self.call(method, task_name,
-                         float(soft), float(hard), **kwargs)
-
-    def add_consumer(self, method, queue, exchange=None,
-                     exchange_type='direct', routing_key=None, **kwargs):
-        """<queue> [exchange [type [routing_key]]]"""
-        return self.call(method, queue, exchange,
-                         exchange_type, routing_key, **kwargs)
-
-    def cancel_consumer(self, method, queue, **kwargs):
-        """<queue>"""
-        return self.call(method, queue, **kwargs)
+    def call(self, method, arguments, **options):
+        return self.app.control.broadcast(
+            method, arguments=arguments, reply=True, **options)
 
 
 class status(Command):
@@ -1027,7 +1019,8 @@ class help(Command):
         self.parser.print_help()
         self.out(HELP.format(
             prog_name=self.prog_name,
-            commands=CeleryCommand.list_commands(colored=self.colored),
+            commands=CeleryCommand.list_commands(
+                colored=self.colored, app=self.app),
         ))
 
         return EX_USAGE
@@ -1166,7 +1159,8 @@ class CeleryCommand(Command):
             sys.exit(EX_FAILURE)
 
     @classmethod
-    def get_command_info(self, command, indent=0, color=None, colored=None):
+    def get_command_info(self, command, indent=0,
+                         color=None, colored=None, app=None):
         colored = term.colored() if colored is None else colored
         colored = colored.names[color] if color else lambda x: x
         obj = self.commands[command]
@@ -1176,11 +1170,12 @@ class CeleryCommand(Command):
         return text.join([
             ' ',
             '|' + text.indent('{0} --help'.format(cmd), indent),
-            obj.list_commands(indent, 'celery {0}'.format(command), colored),
+            obj.list_commands(indent, 'celery {0}'.format(command), colored,
+                              app=app),
         ])
 
     @classmethod
-    def list_commands(self, indent=0, colored=None):
+    def list_commands(self, indent=0, colored=None, app=None):
         colored = term.colored() if colored is None else colored
         white = colored.white
         ret = []
@@ -1188,7 +1183,8 @@ class CeleryCommand(Command):
             ret.extend([
                 text.indent('+ {0}: '.format(white(cls)), indent),
                 '\n'.join(
-                    self.get_command_info(command, indent + 4, color, colored)
+                    self.get_command_info(command, indent + 4, color, colored,
+                                          app=app)
                     for command in commands),
                 ''
             ])

+ 6 - 6
celery/tests/app/test_control.py

@@ -76,7 +76,7 @@ class test_inspect(AppCase):
     @with_mock_broadcast
     def test_active(self):
         self.i.active()
-        self.assertIn('dump_active', MockMailbox.sent)
+        self.assertIn('active', MockMailbox.sent)
 
     @with_mock_broadcast
     def test_clock(self):
@@ -86,7 +86,7 @@ class test_inspect(AppCase):
     @with_mock_broadcast
     def test_conf(self):
         self.i.conf()
-        self.assertIn('dump_conf', MockMailbox.sent)
+        self.assertIn('conf', MockMailbox.sent)
 
     @with_mock_broadcast
     def test_hello(self):
@@ -111,12 +111,12 @@ class test_inspect(AppCase):
     @with_mock_broadcast
     def test_scheduled(self):
         self.i.scheduled()
-        self.assertIn('dump_schedule', MockMailbox.sent)
+        self.assertIn('scheduled', MockMailbox.sent)
 
     @with_mock_broadcast
     def test_reserved(self):
         self.i.reserved()
-        self.assertIn('dump_reserved', MockMailbox.sent)
+        self.assertIn('reserved', MockMailbox.sent)
 
     @with_mock_broadcast
     def test_stats(self):
@@ -126,12 +126,12 @@ class test_inspect(AppCase):
     @with_mock_broadcast
     def test_revoked(self):
         self.i.revoked()
-        self.assertIn('dump_revoked', MockMailbox.sent)
+        self.assertIn('revoked', MockMailbox.sent)
 
     @with_mock_broadcast
     def test_tasks(self):
         self.i.registered()
-        self.assertIn('dump_tasks', MockMailbox.sent)
+        self.assertIn('registered', MockMailbox.sent)
 
     @with_mock_broadcast
     def test_ping(self):

+ 8 - 55
celery/tests/bin/test_celery.py

@@ -430,16 +430,16 @@ class test_inspect(AppCase):
     def test_command_info(self):
         i = inspect(app=self.app)
         self.assertTrue(i.get_command_info(
-            'ping', help=True, color=i.colored.red,
+            'ping', help=True, color=i.colored.red, app=self.app,
         ))
 
     def test_list_commands_color(self):
         i = inspect(app=self.app)
         self.assertTrue(i.list_commands(
-            help=True, color=i.colored.red,
+            help=True, color=i.colored.red, app=self.app,
         ))
         self.assertTrue(i.list_commands(
-            help=False, color=None,
+            help=False, color=None, app=self.app,
         ))
 
     def test_epilog(self):
@@ -489,12 +489,12 @@ class test_inspect(AppCase):
         callback({'foo': {'ok': 'pong'}})
         self.assertIn('OK', out.getvalue())
 
-        with patch('celery.bin.celery.json.dumps') as dumps:
+        with patch('celery.bin.celery.dumps') as dumps:
             i.run('ping', json=True)
             dumps.assert_called()
 
         instance = real.return_value = Mock()
-        instance.ping.return_value = None
+        instance._request.return_value = None
         with self.assertRaises(Error):
             i.run('ping')
 
@@ -504,18 +504,6 @@ class test_inspect(AppCase):
         i.say_chat('<-', 'hello')
         self.assertFalse(out.getvalue())
 
-    def test_objgraph(self):
-        i = inspect(app=self.app)
-        i.call = Mock(name='call')
-        i.objgraph('Message', foo=1)
-        i.call.assert_called_with('objgraph', 'Message', foo=1)
-
-    def test_conf(self):
-        i = inspect(app=self.app)
-        i.call = Mock(name='call')
-        i.conf(with_defaults=True, foo=1)
-        i.call.assert_called_with('conf', True, foo=1)
-
 
 class test_control(AppCase):
 
@@ -528,44 +516,9 @@ class test_control(AppCase):
 
     def test_call(self):
         i = self.control(False)
-        i.call('foo', 1, kw=2)
-        i.app.control.foo.assert_called_with(1, kw=2, reply=True)
-
-    def test_pool_grow(self):
-        i = self.control(True)
-        i.pool_grow('pool_grow', n=2)
-        i.call.assert_called_with('pool_grow', 2)
-
-    def test_pool_shrink(self):
-        i = self.control(True)
-        i.pool_shrink('pool_shrink', n=2)
-        i.call.assert_called_with('pool_shrink', 2)
-
-    def test_rate_limit(self):
-        i = self.control(True)
-        i.rate_limit('rate_limit', 'proj.add', '1/s')
-        i.call.assert_called_with('rate_limit', 'proj.add', '1/s')
-
-    def test_time_limit(self):
-        i = self.control(True)
-        i.time_limit('time_limit', 'proj.add', 10, 30)
-        i.call.assert_called_with('time_limit', 'proj.add', 10, 30)
-
-    def test_add_consumer(self):
-        i = self.control(True)
-        i.add_consumer(
-            'add_consumer', 'queue', 'exchange', 'topic', 'rkey',
-            durable=True,
-        )
-        i.call.assert_called_with(
-            'add_consumer', 'queue', 'exchange', 'topic', 'rkey',
-            durable=True,
-        )
-
-    def test_cancel_consumer(self):
-        i = self.control(True)
-        i.cancel_consumer('cancel_consumer', 'queue')
-        i.call.assert_called_with('cancel_consumer', 'queue')
+        i.call('foo', arguments={'kw': 2})
+        i.app.control.broadcast.assert_called_with(
+            'foo', arguments={'kw': 2}, reply=True)
 
 
 class test_multi(AppCase):

+ 5 - 0
celery/tests/tasks/test_tasks.py

@@ -37,6 +37,7 @@ class MockApplyTask(Task):
 class TasksCase(AppCase):
 
     def setup(self):
+        self.app.conf.task_protocol = 1  # XXX  Still using proto1
         self.mytask = self.app.task(shared=False)(return_True)
 
         @self.app.task(bind=True, count=0, shared=False)
@@ -121,6 +122,10 @@ class TasksCase(AppCase):
 
         self.autoretry_task = autoretry_task
 
+        # memove all messages from memory-transport
+        from kombu.transport.memory import Channel
+        Channel.queues.clear()
+
 
 class MyCustomException(Exception):
     """Random custom exception."""

+ 1 - 1
celery/utils/__init__.py

@@ -26,6 +26,6 @@ from .imports import (          # noqa
     instantiate, import_from_cwd, gen_task_name,
 )
 from .functional import chunks, noop                    # noqa
-from kombu.utils.objects import cached_property
+from kombu.utils.objects import cached_property         # noqa
 from kombu.utils.uuid import uuid   # noqa
 gen_unique_id = uuid

+ 191 - 57
celery/worker/control.py

@@ -5,26 +5,32 @@ from __future__ import absolute_import, unicode_literals
 import io
 import tempfile
 
+from collections import namedtuple
+
 from billiard.common import TERM_SIGNAME
 from kombu.utils.encoding import safe_repr
 
 from celery.exceptions import WorkerShutdown
-from celery.five import UserDict, items, string_t
+from celery.five import UserDict, items, string_t, text_t
 from celery.platforms import signals as _signals
 from celery.utils import timeutils
 from celery.utils.functional import maybe_list
 from celery.utils.log import get_logger
-from celery.utils.serialization import jsonify
+from celery.utils.serialization import jsonify, strtobool
 
 from . import state as worker_state
 from .request import Request
-from .state import revoked
 
 __all__ = ['Panel']
 
 DEFAULT_TASK_INFO_ITEMS = ('exchange', 'routing_key', 'rate_limit')
 logger = get_logger(__name__)
 
+controller_info_t = namedtuple('controller_info_t', [
+    'alias', 'type', 'visible', 'default_timeout',
+    'help', 'signature', 'args', 'variadic',
+])
+
 
 def ok(value):
     return {'ok': value}
@@ -37,22 +43,56 @@ def nok(value):
 class Panel(UserDict):
     """Global registry of remote control commands."""
     data = dict()  # global dict.
+    meta = dict()
+    by_alias = dict()
+
+    @classmethod
+    def register(cls, *args, **kwargs):
+        if args:
+            return cls._register(**kwargs)(*args)
+        return cls._register(**kwargs)
 
     @classmethod
-    def register(cls, method, name=None):
-        cls.data[name or method.__name__] = method
-        return method
+    def _register(cls, name=None, alias=None, type='control',
+                  visible=True, default_timeout=1.0, help=None,
+                  signature=None, args=None, variadic=None):
+
+        def _inner(fun):
+            control_name = name or fun.__name__
+            _help = help or (fun.__doc__ or '').strip().split('\n')[0]
+            cls.data[control_name] = fun
+            cls.meta[control_name] = controller_info_t(
+                alias, type, visible, default_timeout,
+                _help, signature, args, variadic)
+            if alias:
+                cls.data[alias] = fun
+            return fun
+        return _inner
+
+
+def control_command(**kwargs):
+    return Panel.register(type='control', **kwargs)
 
 
+def inspect_command(**kwargs):
+    return Panel.register(type='inspect', **kwargs)
+
 # -- App
 
-@Panel.register
+
+@inspect_command()
 def report(state):
+    """Information about Celery installation for bug reports."""
     return ok(state.app.bugreport())
 
 
-@Panel.register
-def dump_conf(state, with_defaults=False, **kwargs):
+@inspect_command(
+    alias='dump_conf',  # XXX < backwards compatible
+    signature='[include_defaults=False]',
+    args=[('with_defaults', strtobool)],
+)
+def conf(state, with_defaults=False, **kwargs):
+    """List configuration."""
     return jsonify(state.app.conf.table(with_defaults=with_defaults),
                    keyfilter=_wanted_config_key,
                    unknown_type_filter=safe_repr)
@@ -64,7 +104,10 @@ def _wanted_config_key(key):
 
 # -- Task
 
-@Panel.register(type='inspect')
+@inspect_command(
+    variadic='ids',
+    signature='[id1 [id2 [... [idN]]]]',
+)
 def query_task(state, ids, **kwargs):
     """Query for task information by id."""
     return {
@@ -92,7 +135,10 @@ def _state_of_task(request,
     return 'ready'
 
 
-@Panel.register(type='control')
+@control_command(
+    variadic='task_id',
+    signature='[id1 [id2 [... [idN]]]]',
+)
 def revoke(state, task_id, terminate=False, signal=None, **kwargs):
     """Revoke task by task id (or list of ids).
 
@@ -105,7 +151,7 @@ def revoke(state, task_id, terminate=False, signal=None, **kwargs):
     size = len(task_ids)
     terminated = set()
 
-    revoked.update(task_ids)
+    worker_state.revoked.update(task_ids)
     if terminate:
         signum = _signals.signum(signal or TERM_SIGNAME)
         for request in _find_requests_by_id(task_ids):
@@ -125,10 +171,22 @@ def revoke(state, task_id, terminate=False, signal=None, **kwargs):
     return ok('tasks {0} flagged as revoked'.format(idstr))
 
 
-@Panel.register(type='control')
+@control_command(
+    variadic='task_id',
+    args=[('signal', text_t)],
+    signature='<signal> [id1 [id2 [... [idN]]]]'
+)
+def terminate(state, signal, task_id, **kwargs):
+    """Terminate task by task id (or list of ids)."""
+    return revoke(state, task_id, terminate=True, signal=signal)
+
+
+@control_command(
+    args=[('task_name', text_t), ('rate_limit', text_t)],
+    signature='<task_name> <rate_limit (e.g. 5/s | 5/m | 5/h)>',
+)
 def rate_limit(state, task_name, rate_limit, **kwargs):
-    """Tell worker(s) to modify the rate limit for a task by type."""
-    """Set new rate limit for a task type.
+    """Tell worker(s) to modify the rate limit for a task by type.
 
     See Also:
         :attr:`celery.task.base.Task.rate_limit`.
@@ -137,7 +195,6 @@ def rate_limit(state, task_name, rate_limit, **kwargs):
         task_name (str): Type of task to set rate limit for.
         rate_limit (int, str): New rate limit.
     """
-
     try:
         timeutils.rate(rate_limit)
     except ValueError as exc:
@@ -161,8 +218,18 @@ def rate_limit(state, task_name, rate_limit, **kwargs):
     return ok('new rate limit set successfully')
 
 
-@Panel.register
+@control_command(
+    args=[('task_name', text_t), ('soft', float), ('hard', float)],
+    signature='<task_name> <soft_secs> [hard_secs]',
+)
 def time_limit(state, task_name=None, hard=None, soft=None, **kwargs):
+    """Tell worker(s) to modify the time limit for task by type.
+
+    Arguments:
+        task_name (str): Name of task to change.
+        hard (float): Hard time limit.
+        soft (float): Soft time limit.
+    """
     try:
         task = state.app.tasks[task_name]
     except KeyError:
@@ -181,19 +248,28 @@ def time_limit(state, task_name=None, hard=None, soft=None, **kwargs):
 # -- Events
 
 
-@Panel.register
+@inspect_command()
 def clock(state, **kwargs):
+    """Get current logical clock value."""
     return {'clock': state.app.clock.value}
 
 
-@Panel.register
+@control_command()
 def election(state, id, topic, action=None, **kwargs):
+    """Hold election.
+
+    Arguments:
+        id (str): Unique election id.
+        topic (str): Election topic.
+        action (str): Action to take for elected actor.
+    """
     if state.consumer.gossip:
         state.consumer.gossip.election(id, topic, action)
 
 
-@Panel.register
+@control_command()
 def enable_events(state):
+    """Tell worker(s) to send task-related events."""
     dispatcher = state.consumer.event_dispatcher
     if dispatcher.groups and 'task' not in dispatcher.groups:
         dispatcher.groups.add('task')
@@ -202,8 +278,9 @@ def enable_events(state):
     return ok('task events already enabled')
 
 
-@Panel.register
+@control_command()
 def disable_events(state):
+    """Tell worker(s) to stop sending task-related events."""
     dispatcher = state.consumer.event_dispatcher
     if 'task' in dispatcher.groups:
         dispatcher.groups.discard('task')
@@ -212,8 +289,9 @@ def disable_events(state):
     return ok('task events already disabled')
 
 
-@Panel.register
+@control_command()
 def heartbeat(state):
+    """Tell worker(s) to send event heartbeat immediately."""
     logger.debug('Heartbeat requested by remote.')
     dispatcher = state.consumer.event_dispatcher
     dispatcher.send('worker-heartbeat', freq=5, **worker_state.SOFTWARE_INFO)
@@ -221,8 +299,9 @@ def heartbeat(state):
 
 # -- Worker
 
-@Panel.register
+@inspect_command(visible=False)
 def hello(state, from_node, revoked=None, **kwargs):
+    """Request mingle sync-data."""
     if from_node != state.hostname:
         logger.info('sync with %s', from_node)
         if revoked:
@@ -233,22 +312,25 @@ def hello(state, from_node, revoked=None, **kwargs):
         }
 
 
-@Panel.register
+@inspect_command(default_timeout=0.2)
 def ping(state, **kwargs):
+    """Ping worker(s)."""
     return ok('pong')
 
 
-@Panel.register
+@inspect_command()
 def stats(state, **kwargs):
+    """Request worker statistics/information."""
     return state.consumer.controller.stats()
 
 
-@Panel.register
-def dump_schedule(state, safe=False, **kwargs):
-    return list(_iter_schedule_requests(state.consumer.timer, safe=safe))
+@inspect_command(alias='dump_schedule')
+def scheduled(state, **kwargs):
+    """List of currently scheduled ETA/countdown tasks."""
+    return list(_iter_schedule_requests(state.consumer.timer))
 
 
-def _iter_schedule_requests(timer, safe=False, Request=Request):
+def _iter_schedule_requests(timer, Request=Request):
     for waiting in timer.schedule.queue:
         try:
             arg0 = waiting.entry.args[0]
@@ -259,34 +341,48 @@ def _iter_schedule_requests(timer, safe=False, Request=Request):
                 yield {
                     'eta': arg0.eta.isoformat() if arg0.eta else None,
                     'priority': waiting.priority,
-                    'request': arg0.info(safe=safe),
+                    'request': arg0.info(),
                 }
 
 
-@Panel.register
-def dump_reserved(state, safe=False, **kwargs):
-    reserved = (
+@inspect_command(alias='dump_reserved')
+def reserved(state, **kwargs):
+    """List of currently reserved tasks (not including scheduled/active)."""
+    reserved_tasks = (
         state.tset(worker_state.reserved_requests) -
         state.tset(worker_state.active_requests)
     )
-    if not reserved:
+    if not reserved_tasks:
         return []
-    return [request.info(safe=safe) for request in reserved]
+    return [request.info() for request in reserved_tasks]
 
 
-@Panel.register
-def dump_active(state, safe=False, **kwargs):
-    return [request.info(safe=safe)
+@inspect_command(alias='dump_active')
+def active(state, **kwargs):
+    """List of tasks currently being executed."""
+    return [request.info()
             for request in state.tset(worker_state.active_requests)]
 
 
-@Panel.register
-def dump_revoked(state, **kwargs):
+@inspect_command(alias='dump_revoked')
+def revoked(state, **kwargs):
+    """List of revoked task-ids."""
     return list(worker_state.revoked)
 
 
-@Panel.register
-def dump_tasks(state, taskinfoitems=None, builtins=False, **kwargs):
+@inspect_command(
+    alias='dump_tasks',
+    variadic='taskinfoitems',
+    signature='[attr1 [attr2 [... [attrN]]]]',
+)
+def registered(state, taskinfoitems=None, builtins=False, **kwargs):
+    """List of registered tasks.
+
+    Arguments:
+        taskinfoitems (Sequence[str]): List of task attributes to include.
+            Defaults to ``exchange,routing_key,rate_limit``.
+        builtins (bool): Also include built-in tasks.
+    """
     reg = state.app.tasks
     taskinfoitems = taskinfoitems or DEFAULT_TASK_INFO_ITEMS
 
@@ -308,8 +404,19 @@ def dump_tasks(state, taskinfoitems=None, builtins=False, **kwargs):
 
 # -- Debugging
 
-@Panel.register
+@inspect_command(
+    default_timeout=60.0,
+    args=[('type', text_t), ('num', int), ('max_depth', int)],
+    signature='[object_type=Request] [num=200 [max_depth=10]]',
+)
 def objgraph(state, num=200, max_depth=10, type='Request'):  # pragma: no cover
+    """Create graph of uncollected objects (memory-leak debugging).
+
+    Arguments:
+        num (int): Max number of objects to graph.
+        max_depth (int): Traverse at most n levels deep.
+        type (str): Name of object to graph.  Default is ``"Request"``.
+    """
     try:
         import objgraph
     except ImportError:
@@ -326,14 +433,19 @@ def objgraph(state, num=200, max_depth=10, type='Request'):  # pragma: no cover
         return {'filename': fh.name}
 
 
-@Panel.register
-def memsample(state, **kwargs):  # pragma: no cover
+@inspect_command()
+def memsample(state, **kwargs):
+    """Sample current RSS memory usage."""
     from celery.utils.debug import sample_mem
     return sample_mem()
 
 
-@Panel.register
+@inspect_command(
+    args=[('samples', int)],
+    signature='[n_samples=10]',
+)
 def memdump(state, samples=10, **kwargs):  # pragma: no cover
+    """Dump statistics of previous memsample requests."""
     from celery.utils.debug import memdump
     out = io.StringIO()
     memdump(file=out)
@@ -342,22 +454,31 @@ def memdump(state, samples=10, **kwargs):  # pragma: no cover
 # -- Pool
 
 
-@Panel.register
+@control_command(
+    args=[('n', int)],
+    signature='[N=1]',
+)
 def pool_grow(state, n=1, **kwargs):
+    """Grow pool by n processes/threads."""
     state.consumer.pool.grow(n)
     state.consumer._update_prefetch_count(n)
     return ok('pool will grow')
 
 
-@Panel.register
+@control_command(
+    args=[('n', int)],
+    signature='[N=1]',
+)
 def pool_shrink(state, n=1, **kwargs):
+    """Shrink pool by n processes/threads."""
     state.consumer.pool.shrink(n)
     state.consumer._update_prefetch_count(-n)
     return ok('pool will shrink')
 
 
-@Panel.register
+@control_command()
 def pool_restart(state, modules=None, reload=False, reloader=None, **kwargs):
+    """Restart execution pool."""
     if state.app.conf.worker_pool_restarts:
         state.consumer.controller.reload(modules, reload, reloader=reloader)
         return ok('reload started')
@@ -365,35 +486,48 @@ def pool_restart(state, modules=None, reload=False, reloader=None, **kwargs):
         raise ValueError('Pool restarts not enabled')
 
 
-@Panel.register
+@control_command()
 def shutdown(state, msg='Got shutdown from remote', **kwargs):
+    """Shutdown worker(s)."""
     logger.warning(msg)
     raise WorkerShutdown(msg)
 
 
 # -- Queues
 
-@Panel.register
+@control_command(
+    args=[
+        ('queue', text_t),
+        ('exchange', text_t),
+        ('exchange_type', text_t),
+        ('routing_key', text_t),
+    ],
+    signature='<queue> [exchange [type [routing_key]]]',
+)
 def add_consumer(state, queue, exchange=None, exchange_type=None,
                  routing_key=None, **options):
+    """Tell worker(s) to consume from task queue by name."""
     state.consumer.call_soon(
         state.consumer.add_task_queue,
-        queue, exchange, exchange_type, routing_key, **options
-    )
+        queue, exchange, exchange_type or 'direct', routing_key, **options)
     return ok('add consumer {0}'.format(queue))
 
 
-@Panel.register
-def cancel_consumer(state, queue=None, **_):
+@control_command(
+    args=[('queue', text_t)],
+    signature='<queue>',
+)
+def cancel_consumer(state, queue, **_):
+    """Tell worker(s) to stop consuming from task queue by name."""
     state.consumer.call_soon(
         state.consumer.cancel_task_queue, queue,
     )
     return ok('no longer consuming from {0}'.format(queue))
 
 
-@Panel.register
+@inspect_command()
 def active_queues(state):
-    """Return information about the queues a worker consumes from."""
+    """List the task queues a worker are currently consuming from."""
     if state.consumer.task_consumer:
         return [dict(queue.as_dict(recurse=True))
                 for queue in state.consumer.task_consumer.queues]