Browse Source

Reorganizes utils into utils.imports, utils.text and utils.functional

Ask Solem 13 năm trước cách đây
mục cha
commit
f5f8b21926
43 tập tin đã thay đổi với 465 bổ sung451 xóa
  1. 1 1
      celery/abstract.py
  2. 2 1
      celery/app/__init__.py
  3. 4 3
      celery/app/amqp.py
  4. 2 1
      celery/app/annotations.py
  5. 3 3
      celery/app/base.py
  6. 3 3
      celery/app/task/__init__.py
  7. 1 1
      celery/apps/beat.py
  8. 3 1
      celery/apps/worker.py
  9. 2 2
      celery/backends/__init__.py
  10. 3 1
      celery/beat.py
  11. 5 4
      celery/bin/base.py
  12. 1 1
      celery/bin/camqadm.py
  13. 4 2
      celery/bin/celeryctl.py
  14. 1 1
      celery/bin/celeryd_multi.py
  15. 2 2
      celery/concurrency/__init__.py
  16. 1 68
      celery/datastructures.py
  17. 1 1
      celery/events/cursesmon.py
  18. 2 1
      celery/events/snapshot.py
  19. 2 1
      celery/events/state.py
  20. 3 2
      celery/loaders/__init__.py
  21. 4 4
      celery/loaders/base.py
  22. 1 1
      celery/loaders/default.py
  23. 3 1
      celery/routes.py
  24. 3 1
      celery/task/sets.py
  25. 1 1
      celery/tests/functional/case.py
  26. 2 2
      celery/tests/test_app/__init__.py
  27. 1 1
      celery/tests/test_app/test_annotations.py
  28. 2 1
      celery/tests/test_app/test_routes.py
  29. 1 1
      celery/tests/test_concurrency/test_concurrency_processes.py
  30. 1 1
      celery/tests/test_concurrency/test_concurrency_solo.py
  31. 24 28
      celery/tests/test_utils/__init__.py
  32. 17 0
      celery/tests/test_utils/test_utils_imports.py
  33. 2 2
      celery/tests/test_utils/test_utils_info.py
  34. 1 1
      celery/tests/utils.py
  35. 16 296
      celery/utils/__init__.py
  36. 1 1
      celery/utils/encoding.py
  37. 181 2
      celery/utils/functional.py
  38. 147 0
      celery/utils/imports.py
  39. 2 2
      celery/utils/mail.py
  40. 1 1
      celery/utils/timeutils.py
  41. 2 1
      celery/worker/__init__.py
  42. 1 1
      celery/worker/consumer.py
  43. 5 2
      celery/worker/job.py

+ 1 - 1
celery/abstract.py

@@ -15,7 +15,7 @@ from collections import defaultdict
 from importlib import import_module
 
 from .datastructures import DependencyGraph
-from .utils import instantiate
+from .utils.imports import instantiate
 
 
 class Namespace(object):

+ 2 - 1
celery/app/__init__.py

@@ -15,7 +15,8 @@ from __future__ import absolute_import
 import os
 
 from ..local import PromiseProxy
-from ..utils import cached_property, instantiate
+from ..utils import cached_property
+from ..utils.imports import instantiate
 
 from . import annotations
 from . import base

+ 4 - 3
celery/app/amqp.py

@@ -19,7 +19,8 @@ from kombu.pools import ProducerPool
 
 from .. import routes as _routes
 from .. import signals
-from ..utils import cached_property, textindent, uuid
+from ..utils import cached_property, uuid
+from ..utils import text
 
 #: List of known options to a Kombu producers send method.
 #: Used to extract the message related options out of any `dict`.
@@ -96,8 +97,8 @@ class Queues(dict):
                     name=(name + ":").ljust(12), **config)
                         for name, config in sorted(active.iteritems())]
         if indent_first:
-            return textindent("\n".join(info), indent)
-        return info[0] + "\n" + textindent("\n".join(info[1:]), indent)
+            return text.indent("\n".join(info), indent)
+        return info[0] + "\n" + text.indent("\n".join(info[1:]), indent)
 
     def select_subset(self, wanted, create_missing=True):
         """Select subset of the currently defined queues.

+ 2 - 1
celery/app/annotations.py

@@ -1,6 +1,7 @@
 from __future__ import absolute_import
 
-from ..utils import firstmethod, instantiate, mpromise
+from ..utils.functional import firstmethod, mpromise
+from ..utils.imports import instantiate
 
 _first_match = firstmethod("annotate")
 _first_match_any = firstmethod("annotate_any")

+ 3 - 3
celery/app/base.py

@@ -28,7 +28,8 @@ from .. import datastructures
 from .. import platforms
 from ..exceptions import AlwaysEagerIgnored
 from ..local import maybe_evaluate
-from ..utils import cached_property, instantiate, lpmerge, qualname
+from ..utils import cached_property, lpmerge
+from ..utils.imports import instantiate, qualname
 
 from .defaults import DEFAULTS, find_deprecated_settings, find
 
@@ -364,8 +365,7 @@ class BaseApp(object):
     def _pformat(self, value, width=80, nl_width=80, **kw):
 
         if isinstance(value, dict):
-            return "{\n %s" % (
-                                pformat(value, width=nl_width, indent=4, **kw)[1:])
+            return "{\n %s" % (pformat(value, 4, nl_width)[1:])
         elif isinstance(value, tuple):
             return "\n%s%s" % (' ' * 4,
                                 pformat(value, width=nl_width, **kw))

+ 3 - 3
celery/app/task/__init__.py

@@ -20,9 +20,9 @@ from ... import states
 from ...datastructures import ExceptionInfo
 from ...exceptions import MaxRetriesExceededError, RetryTaskError
 from ...result import EagerResult
-from ...utils import (fun_takes_kwargs, instantiate,
-                      mattrgetter, uuid, maybe_reraise)
-from ...utils.functional import maybe_list
+from ...utils import fun_takes_kwargs, uuid, maybe_reraise
+from ...utils.functional import mattrgetter, maybe_list
+from ...utils.imports import instantiate
 from ...utils.mail import ErrorMail
 from ...utils.compat import fun_of_method
 

+ 1 - 1
celery/apps/beat.py

@@ -9,7 +9,7 @@ from .. import __version__, platforms
 from .. import beat
 from ..app import app_or_default
 from ..app.abstract import configurated, from_config
-from ..utils import qualname
+from ..utils.imports import qualname
 from ..utils.log import LOG_LEVELS
 from ..utils.timeutils import humanize_seconds
 

+ 3 - 1
celery/apps/worker.py

@@ -16,8 +16,10 @@ from .. import __version__, platforms, signals
 from ..app import app_or_default
 from ..app.abstract import configurated, from_config
 from ..exceptions import ImproperlyConfigured, SystemTerminate
-from ..utils import cry, isatty, pluralize, qualname
+from ..utils import cry, isatty
+from ..utils.imports import qualname
 from ..utils.log import LOG_LEVELS, mlevel
+from ..utils.text import pluralize
 from ..worker import WorkController
 
 try:

+ 2 - 2
celery/backends/__init__.py

@@ -5,7 +5,7 @@ import sys
 
 from .. import current_app
 from ..local import Proxy
-from ..utils import get_cls_by_name
+from ..utils.imports import symbol_by_name
 from ..utils.functional import memoize
 
 UNKNOWN_BACKEND = """\
@@ -31,7 +31,7 @@ def get_backend_cls(backend=None, loader=None):
     loader = loader or current_app.loader
     aliases = dict(BACKEND_ALIASES, **loader.override_backends)
     try:
-        return get_cls_by_name(backend, aliases)
+        return symbol_by_name(backend, aliases)
     except ValueError, exc:
         raise ValueError, ValueError(UNKNOWN_BACKEND % (
                     backend, exc)), sys.exc_info()[2]

+ 3 - 1
celery/beat.py

@@ -24,6 +24,7 @@ except ImportError:
     multiprocessing = None  # noqa
 
 from kombu.utils import reprcall
+from kombu.utils.functional import maybe_promise
 
 from . import __version__
 from . import platforms
@@ -31,7 +32,8 @@ from . import signals
 from . import current_app
 from .app import app_or_default
 from .schedules import maybe_schedule, crontab
-from .utils import cached_property, instantiate, maybe_promise
+from .utils import cached_property
+from .utils.imports import instantiate
 from .utils.timeutils import humanize_seconds
 
 

+ 5 - 4
celery/bin/base.py

@@ -175,16 +175,17 @@ class Command(object):
         if config_module:
             os.environ["CELERY_CONFIG_MODULE"] = config_module
         if app:
-            self.app = self.get_cls_by_name(app)
+            self.app = self.symbol_by_name(app)
         else:
             self.app = self.get_app(loader=loader)
         if self.enable_config_from_cmdline:
             argv = self.process_cmdline_config(argv)
         return argv
 
-    def get_cls_by_name(self, name):
-        from ..utils import get_cls_by_name, import_from_cwd
-        return get_cls_by_name(name, imp=import_from_cwd)
+    def symbol_by_name(self, name):
+        from ..utils.imports import symbol_by_name, import_from_cwd
+        return symbol_by_name(name, imp=import_from_cwd)
+    get_cls_by_name = symbol_by_name  # XXX compat
 
     def process_cmdline_config(self, argv):
         try:

+ 1 - 1
celery/bin/camqadm.py

@@ -19,7 +19,7 @@ from itertools import count
 from amqplib import client_0_8 as amqp
 
 from ..app import app_or_default
-from ..utils import padlist
+from ..utils.functional import padlist
 
 from .base import Command
 

+ 4 - 2
celery/bin/celeryctl.py

@@ -17,7 +17,9 @@ from anyjson import deserialize
 from .. import __version__
 from ..app import app_or_default, current_app
 from ..platforms import EX_OK, EX_FAILURE, EX_UNAVAILABLE, EX_USAGE
-from ..utils import get_cls_by_name, pluralize, term
+from ..utils import term
+from ..utils.imports import symbol_by_name
+from ..utils.text import pluralize
 from ..utils.timeutils import maybe_iso8601
 
 from ..bin.base import Command as CeleryCommand
@@ -144,7 +146,7 @@ class Delegate(Command):
     def __init__(self, *args, **kwargs):
         super(Delegate, self).__init__(*args, **kwargs)
 
-        self.target = get_cls_by_name(self.Command)(app=self.app)
+        self.target = symbol_by_name(self.Command)(app=self.app)
         self.args = self.target.args
 
     def get_options(self):

+ 1 - 1
celery/bin/celeryd_multi.py

@@ -106,7 +106,7 @@ from kombu.utils.encoding import from_utf8
 from .. import __version__
 from ..platforms import shellsplit
 from ..utils import term
-from ..utils import pluralize
+from ..utils.text import pluralize
 
 SIGNAMES = set(sig for sig in dir(signal)
                         if sig.startswith("SIG") and "_" not in sig)

+ 2 - 2
celery/concurrency/__init__.py

@@ -1,7 +1,7 @@
 # -*- coding: utf-8 -*-
 from __future__ import absolute_import
 
-from ..utils import get_cls_by_name
+from ..utils.imports import symbol_by_name
 
 ALIASES = {
     "processes": "celery.concurrency.processes:TaskPool",
@@ -13,4 +13,4 @@ ALIASES = {
 
 
 def get_implementation(cls):
-    return get_cls_by_name(cls, ALIASES)
+    return symbol_by_name(cls, ALIASES)

+ 1 - 68
celery/datastructures.py

@@ -18,12 +18,10 @@ import traceback
 
 from collections import defaultdict
 from itertools import chain
-from threading import RLock
 
 from kombu.utils.limits import TokenBucket  # noqa
 
-from .utils import uniq
-from .utils.compat import UserDict, OrderedDict
+from .utils.functional import LRUCache, uniq  # noqa
 
 
 class CycleError(Exception):
@@ -539,68 +537,3 @@ class LimitedSet(object):
     def first(self):
         """Get the oldest member."""
         return self.chronologically[0]
-
-
-class LRUCache(UserDict):
-    """LRU Cache implementation using a doubly linked list to track access.
-
-    :keyword limit: The maximum number of keys to keep in the cache.
-        When a new key is inserted and the limit has been exceeded,
-        the *Least Recently Used* key will be discarded from the
-        cache.
-
-    """
-
-    def __init__(self, limit=None):
-        self.limit = limit
-        self.mutex = RLock()
-        self.data = OrderedDict()
-
-    def __getitem__(self, key):
-        with self.mutex:
-            value = self[key] = self.data.pop(key)
-        return value
-
-    def keys(self):
-        # userdict.keys in py3k calls __getitem__
-        return self.data.keys()
-
-    def values(self):
-        return list(self._iterate_values())
-
-    def items(self):
-        return list(self._iterate_items())
-
-    def __setitem__(self, key, value):
-        # remove least recently used key.
-        with self.mutex:
-            if self.limit and len(self.data) >= self.limit:
-                self.data.pop(iter(self.data).next())
-            self.data[key] = value
-
-    def __iter__(self):
-        return self.data.iterkeys()
-
-    def _iterate_items(self):
-        for k in self:
-            try:
-                yield (k, self.data[k])
-            except KeyError:
-                pass
-    iteritems = _iterate_items
-
-    def _iterate_values(self):
-        for k in self:
-            try:
-                yield self.data[k]
-            except KeyError:  # pragma: no cover
-                pass
-    itervalues = _iterate_values
-
-    def incr(self, key, delta=1):
-        with self.mutex:
-            # this acts as memcached does- store as a string, but return a
-            # integer as long as it exists and we can cast it
-            newval = int(self.data.pop(key)) + delta
-            self[key] = str(newval)
-        return newval

+ 1 - 1
celery/events/cursesmon.py

@@ -25,7 +25,7 @@ from math import ceil
 from .. import __version__
 from .. import states
 from ..app import app_or_default
-from ..utils import abbr, abbrtask
+from ..utils.text import abbr, abbrtask
 
 BORDER_SPACING = 4
 LEFT_BORDER_OFFSET = 3

+ 2 - 1
celery/events/snapshot.py

@@ -21,8 +21,9 @@ from kombu.utils.limits import TokenBucket
 
 from .. import platforms
 from ..app import app_or_default
-from ..utils import timer2, instantiate
+from ..utils import timer2
 from ..utils.dispatch import Signal
+from ..utils.imports import instantiate
 from ..utils.log import LOG_LEVELS
 from ..utils.timeutils import rate
 

+ 2 - 1
celery/events/state.py

@@ -27,9 +27,10 @@ import heapq
 
 from threading import Lock
 
+from kombu.utils import kwdict
+
 from .. import states
 from ..datastructures import AttributeDict, LRUCache
-from ..utils import kwdict
 
 #: Hartbeat expiry time in seconds.  The worker will be considered offline
 #: if no heartbeat is received within this time.

+ 3 - 2
celery/loaders/__init__.py

@@ -13,7 +13,8 @@
 from __future__ import absolute_import
 
 from .. import current_app
-from ..utils import deprecated, get_cls_by_name
+from ..utils import deprecated
+from ..utils.imports import symbol_by_name
 
 LOADER_ALIASES = {"app": "celery.loaders.app:AppLoader",
                   "default": "celery.loaders.default:Loader",
@@ -22,7 +23,7 @@ LOADER_ALIASES = {"app": "celery.loaders.app:AppLoader",
 
 def get_loader_cls(loader):
     """Get loader class by name/alias"""
-    return get_cls_by_name(loader, LOADER_ALIASES)
+    return symbol_by_name(loader, LOADER_ALIASES)
 
 
 @deprecated(deprecation="2.5", removal="3.0",

+ 4 - 4
celery/loaders/base.py

@@ -24,8 +24,8 @@ from kombu.utils.encoding import safe_str
 
 from ..datastructures import DictAttribute
 from ..exceptions import ImproperlyConfigured
-from ..utils import (cached_property, get_cls_by_name,
-                     import_from_cwd as _import_from_cwd)
+from ..utils import cached_property
+from ..utils.imports import import_from_cwd, symbol_by_name
 from ..utils.functional import maybe_list
 
 BUILTIN_MODULES = frozenset()
@@ -94,7 +94,7 @@ class BaseLoader(object):
         return importlib.import_module(module, package=package)
 
     def import_from_cwd(self, module, imp=None, package=None):
-        return _import_from_cwd(module,
+        return import_from_cwd(module,
                 self.import_module if imp is None else imp,
                 package=package)
 
@@ -123,7 +123,7 @@ class BaseLoader(object):
         if isinstance(obj, basestring):
             try:
                 if "." in obj:
-                    obj = get_cls_by_name(obj, imp=self.import_from_cwd)
+                    obj = symbol_by_name(obj, imp=self.import_from_cwd)
                 else:
                     obj = self.import_from_cwd(obj)
             except (ImportError, AttributeError):

+ 1 - 1
celery/loaders/default.py

@@ -17,7 +17,7 @@ import warnings
 
 from ..datastructures import AttributeDict
 from ..exceptions import NotConfigured
-from ..utils import find_module, NotAPackage
+from ..utils.imports import NotAPackage, find_module
 
 from .base import BaseLoader
 

+ 3 - 1
celery/routes.py

@@ -13,7 +13,9 @@
 from __future__ import absolute_import
 
 from .exceptions import QueueNotFound
-from .utils import firstmethod, instantiate, lpmerge, mpromise
+from .utils import lpmerge
+from .utils.functional import firstmethod, mpromise
+from .utils.imports import instantiate
 
 _first_route = firstmethod("route_for_task")
 

+ 3 - 1
celery/task/sets.py

@@ -14,10 +14,12 @@ from __future__ import with_statement
 
 from itertools import chain
 
+from kombu.utils import reprcall
+
 from .. import current_app
 from ..app import app_or_default, current_task
 from ..datastructures import AttributeDict
-from ..utils import cached_property, reprcall, uuid
+from ..utils import cached_property, uuid
 from ..utils.functional import maybe_list
 from ..utils.compat import UserList, chain_from_iterable
 

+ 1 - 1
celery/tests/functional/case.py

@@ -13,7 +13,7 @@ from time import time
 
 from celery.exceptions import TimeoutError
 from celery.task.control import ping, flatten_reply, inspect
-from celery.utils import qualname
+from celery.utils.imports import qualname
 
 from celery.tests.utils import Case
 

+ 2 - 2
celery/tests/test_app/__init__.py

@@ -17,8 +17,8 @@ from celery.utils.serialization import pickle
 from celery.tests import config
 from celery.tests.utils import (Case, mask_modules, platform_pyimp,
                                 sys_platform, pypy_version)
+from celery.utils import uuid
 from celery.utils.mail import ErrorMail
-from kombu.utils import gen_unique_id
 
 THIS_IS_A_KEY = "this is a value"
 
@@ -248,7 +248,7 @@ class test_App(Case):
 
     def test_error_mail_sender(self):
         x = ErrorMail.subject % {"name": "task_name",
-                                 "id": gen_unique_id(),
+                                 "id": uuid(),
                                  "exc": "FOOBARBAZ",
                                  "hostname": "lana"}
         self.assertTrue(x)

+ 1 - 1
celery/tests/test_app/test_annotations.py

@@ -2,7 +2,7 @@ from __future__ import absolute_import
 
 from celery.app.annotations import MapAnnotation, prepare
 from celery.task import task
-from celery.utils import qualname
+from celery.utils.imports import qualname
 
 from celery.tests.utils import Case
 

+ 2 - 1
celery/tests/test_app/test_routes.py

@@ -3,11 +3,12 @@ from __future__ import with_statement
 
 from functools import wraps
 
+from kombu.utils.functional import maybe_promise
+
 from celery import routes
 from celery import current_app
 from celery.exceptions import QueueNotFound
 from celery.task import task
-from celery.utils import maybe_promise
 from celery.tests.utils import Case
 
 

+ 1 - 1
celery/tests/test_concurrency/test_concurrency_processes.py

@@ -36,7 +36,7 @@ except ImportError:
     safe_apply_callback = None  # noqa
 
 from celery.datastructures import ExceptionInfo
-from celery.utils import noop
+from celery.utils.functional import noop
 from celery.tests.utils import Case
 
 

+ 1 - 1
celery/tests/test_concurrency/test_concurrency_solo.py

@@ -3,7 +3,7 @@ from __future__ import absolute_import
 import operator
 
 from celery.concurrency import solo
-from celery.utils import noop
+from celery.utils.functional import noop
 from celery.tests.utils import Case
 
 

+ 24 - 28
celery/tests/test_utils/__init__.py

@@ -1,8 +1,12 @@
 from __future__ import absolute_import
 from __future__ import with_statement
 
+from kombu.utils.functional import promise
+
 from celery import utils
-from celery.utils import promise, mpromise
+from celery.utils import text
+from celery.utils import functional
+from celery.utils.functional import mpromise
 from celery.utils.threads import bgThread
 from celery.tests.utils import Case
 
@@ -41,11 +45,6 @@ class test_chunks(Case):
 
 class test_utils(Case):
 
-    def test_qualname(self):
-        Class = type("Fox", (object, ), {"__module__": "quick.brown"})
-        self.assertEqual(utils.qualname(Class), "quick.brown.Fox")
-        self.assertEqual(utils.qualname(Class()), "quick.brown.Fox")
-
     def test_is_iterable(self):
         for a in "f", ["f"], ("f", ), {"f": "f"}:
             self.assertTrue(utils.is_iterable(a))
@@ -53,16 +52,17 @@ class test_utils(Case):
             self.assertFalse(utils.is_iterable(b))
 
     def test_padlist(self):
-        self.assertListEqual(utils.padlist(["George", "Costanza", "NYC"], 3),
+        self.assertListEqual(functional.padlist(
+                ["George", "Costanza", "NYC"], 3),
                 ["George", "Costanza", "NYC"])
-        self.assertListEqual(utils.padlist(["George", "Costanza"], 3),
+        self.assertListEqual(functional.padlist(["George", "Costanza"], 3),
                 ["George", "Costanza", None])
-        self.assertListEqual(utils.padlist(["George", "Costanza", "NYC"], 4,
-                                           default="Earth"),
+        self.assertListEqual(functional.padlist(
+                ["George", "Costanza", "NYC"], 4, default="Earth"),
                 ["George", "Costanza", "NYC", "Earth"])
 
     def test_firstmethod_AttributeError(self):
-        self.assertIsNone(utils.firstmethod("foo")([object()]))
+        self.assertIsNone(functional.firstmethod("foo")([object()]))
 
     def test_firstmethod_promises(self):
 
@@ -74,9 +74,9 @@ class test_utils(Case):
             def m(self):
                 return self.value
 
-        self.assertEqual("four", utils.firstmethod("m")([
+        self.assertEqual("four", functional.firstmethod("m")([
             A(), A(), A(), A("four"), A("five")]))
-        self.assertEqual("four", utils.firstmethod("m")([
+        self.assertEqual("four", functional.firstmethod("m")([
             A(), A(), A(), promise(lambda: A("four")), A("five")]))
 
     def test_first(self):
@@ -88,32 +88,28 @@ class test_utils(Case):
                 return True
             return False
 
-        self.assertEqual(5, utils.first(predicate, xrange(10)))
+        self.assertEqual(5, functional.first(predicate, xrange(10)))
         self.assertEqual(iterations[0], 6)
 
         iterations[0] = 0
-        self.assertIsNone(utils.first(predicate, xrange(10, 20)))
+        self.assertIsNone(functional.first(predicate, xrange(10, 20)))
         self.assertEqual(iterations[0], 10)
 
-    def test_get_cls_by_name__instance_returns_instance(self):
-        instance = object()
-        self.assertIs(utils.get_cls_by_name(instance), instance)
-
     def test_truncate_text(self):
-        self.assertEqual(utils.truncate_text("ABCDEFGHI", 3), "ABC...")
-        self.assertEqual(utils.truncate_text("ABCDEFGHI", 10), "ABCDEFGHI")
+        self.assertEqual(text.truncate("ABCDEFGHI", 3), "ABC...")
+        self.assertEqual(text.truncate("ABCDEFGHI", 10), "ABCDEFGHI")
 
     def test_abbr(self):
-        self.assertEqual(utils.abbr(None, 3), "???")
-        self.assertEqual(utils.abbr("ABCDEFGHI", 6), "ABC...")
-        self.assertEqual(utils.abbr("ABCDEFGHI", 20), "ABCDEFGHI")
-        self.assertEqual(utils.abbr("ABCDEFGHI", 6, None), "ABCDEF")
+        self.assertEqual(text.abbr(None, 3), "???")
+        self.assertEqual(text.abbr("ABCDEFGHI", 6), "ABC...")
+        self.assertEqual(text.abbr("ABCDEFGHI", 20), "ABCDEFGHI")
+        self.assertEqual(text.abbr("ABCDEFGHI", 6, None), "ABCDEF")
 
     def test_abbrtask(self):
-        self.assertEqual(utils.abbrtask(None, 3), "???")
-        self.assertEqual(utils.abbrtask("feeds.tasks.refresh", 10),
+        self.assertEqual(text.abbrtask(None, 3), "???")
+        self.assertEqual(text.abbrtask("feeds.tasks.refresh", 10),
                                         "[.]refresh")
-        self.assertEqual(utils.abbrtask("feeds.tasks.refresh", 30),
+        self.assertEqual(text.abbrtask("feeds.tasks.refresh", 30),
                                         "feeds.tasks.refresh")
 
     def test_cached_property(self):

+ 17 - 0
celery/tests/test_utils/test_utils_imports.py

@@ -0,0 +1,17 @@
+from __future__ import absolute_import
+
+from celery.utils.imports import qualname, symbol_by_name
+
+from celery.tests.utils import Case
+
+
+class test_import_utils(Case):
+
+    def test_qualname(self):
+        Class = type("Fox", (object, ), {"__module__": "quick.brown"})
+        self.assertEqual(qualname(Class), "quick.brown.Fox")
+        self.assertEqual(qualname(Class()), "quick.brown.Fox")
+
+    def test_symbol_by_name__instance_returns_instance(self):
+        instance = object()
+        self.assertIs(symbol_by_name(instance), instance)

+ 2 - 2
celery/tests/test_utils/test_utils_info.py

@@ -1,7 +1,7 @@
 from __future__ import absolute_import
 
 from celery import Celery
-from celery.utils import textindent
+from celery.utils.text import indent
 from celery.tests.utils import Case
 
 RANDTEXT = """\
@@ -35,7 +35,7 @@ QUEUE_FORMAT2 = """. queue2:      exchange:exchange2 (type2) binding:bind2"""
 class TestInfo(Case):
 
     def test_textindent(self):
-        self.assertEqual(textindent(RANDTEXT, 4), RANDTEXT_RES)
+        self.assertEqual(indent(RANDTEXT, 4), RANDTEXT_RES)
 
     def test_format_queues(self):
         celery = Celery(set_as_current=False)

+ 1 - 1
celery/tests/utils.py

@@ -27,8 +27,8 @@ import mock
 from nose import SkipTest
 
 from ..app import app_or_default
-from ..utils import noop
 from ..utils.compat import WhateverIO, LoggerAdapter
+from ..utils.functional import noop
 
 from .compat import catch_warnings
 

+ 16 - 296
celery/utils/__init__.py

@@ -13,28 +13,19 @@ from __future__ import absolute_import
 from __future__ import with_statement
 
 import operator
-import os
 import sys
-import imp as _imp
-import importlib
 import threading
 import traceback
 import warnings
 
-from contextlib import contextmanager
 from functools import partial, wraps
 from inspect import getargspec
-from itertools import islice
 from pprint import pprint
 
-from kombu.utils import cached_property, gen_unique_id, kwdict  # noqa
-from kombu.utils import reprcall, reprkwargs                    # noqa
-from kombu.utils.functional import promise, maybe_promise       # noqa
-uuid = gen_unique_id
-
 from ..exceptions import CPendingDeprecationWarning, CDeprecationWarning
-from .compat import StringIO, reload
-from .log import LOG_LEVELS  # noqa
+from .compat import StringIO
+
+from .imports import qualname
 
 PENDING_DEPRECATION_FMT = """
     %(description)s is scheduled for deprecation in \
@@ -82,100 +73,6 @@ def lpmerge(L, R):
     return dict(L, **dict((k, v) for k, v in R.iteritems() if v is not None))
 
 
-class mpromise(promise):
-    """Memoized promise.
-
-    The function is only evaluated once, every subsequent access
-    will return the same value.
-
-    .. attribute:: evaluated
-
-        Set to to :const:`True` after the promise has been evaluated.
-
-    """
-    evaluated = False
-    _value = None
-
-    def evaluate(self):
-        if not self.evaluated:
-            self._value = super(mpromise, self).evaluate()
-            self.evaluated = True
-        return self._value
-
-
-def noop(*args, **kwargs):
-    """No operation.
-
-    Takes any arguments/keyword arguments and does nothing.
-
-    """
-    pass
-
-
-def first(predicate, iterable):
-    """Returns the first element in `iterable` that `predicate` returns a
-    :const:`True` value for."""
-    for item in iterable:
-        if predicate(item):
-            return item
-
-
-def firstmethod(method):
-    """Returns a functions that with a list of instances,
-    finds the first instance that returns a value for the given method.
-
-    The list can also contain promises (:class:`promise`.)
-
-    """
-
-    def _matcher(seq, *args, **kwargs):
-        for cls in seq:
-            try:
-                answer = getattr(maybe_promise(cls), method)(*args, **kwargs)
-                if answer is not None:
-                    return answer
-            except AttributeError:
-                pass
-    return _matcher
-
-
-def chunks(it, n):
-    """Split an iterator into chunks with `n` elements each.
-
-    Examples
-
-        # n == 2
-        >>> x = chunks(iter([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), 2)
-        >>> list(x)
-        [[0, 1], [2, 3], [4, 5], [6, 7], [8, 9], [10]]
-
-        # n == 3
-        >>> x = chunks(iter([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), 3)
-        >>> list(x)
-        [[0, 1, 2], [3, 4, 5], [6, 7, 8], [9, 10]]
-
-    """
-    for first in it:
-        yield [first] + list(islice(it, n - 1))
-
-
-def padlist(container, size, default=None):
-    """Pad list with default elements.
-
-    Examples:
-
-        >>> first, last, city = padlist(["George", "Costanza", "NYC"], 3)
-        ("George", "Costanza", "NYC")
-        >>> first, last, city = padlist(["George", "Costanza"], 3)
-        ("George", "Costanza", None)
-        >>> first, last, city, planet = padlist(["George", "Costanza",
-                                                 "NYC"], 4, default="Earth")
-        ("George", "Costanza", "NYC", "Earth")
-
-    """
-    return list(container)[:size] + [default] * (size - len(container))
-
-
 def is_iterable(obj):
     try:
         iter(obj)
@@ -184,28 +81,6 @@ def is_iterable(obj):
     return True
 
 
-def mattrgetter(*attrs):
-    """Like :func:`operator.itemgetter` but returns :const:`None` on missing
-    attributes instead of raising :exc:`AttributeError`."""
-    return lambda obj: dict((attr, getattr(obj, attr, None))
-                                for attr in attrs)
-
-
-if sys.version_info >= (3, 3):
-
-    def qualname(obj):
-        return obj.__qualname__
-
-else:
-
-    def qualname(obj):  # noqa
-        if not hasattr(obj, "__name__") and hasattr(obj, "__class__"):
-            return qualname(obj.__class__)
-
-        return '.'.join([obj.__module__, obj.__name__])
-get_full_cls_name = qualname  # XXX Compat
-
-
 def fun_takes_kwargs(fun, kwlist=[]):
     """With a function, and a list of keyword arguments, returns arguments
     in the list which the function takes.
@@ -235,172 +110,12 @@ def fun_takes_kwargs(fun, kwlist=[]):
     return filter(partial(operator.contains, args), kwlist)
 
 
-def get_cls_by_name(name, aliases={}, imp=None, package=None,
-        sep='.', **kwargs):
-    """Get class by name.
-
-    The name should be the full dot-separated path to the class::
-
-        modulename.ClassName
-
-    Example::
-
-        celery.concurrency.processes.TaskPool
-                                    ^- class name
-
-    or using ':' to separate module and symbol::
-
-        celery.concurrency.processes:TaskPool
-
-    If `aliases` is provided, a dict containing short name/long name
-    mappings, the name is looked up in the aliases first.
-
-    Examples:
-
-        >>> get_cls_by_name("celery.concurrency.processes.TaskPool")
-        <class 'celery.concurrency.processes.TaskPool'>
-
-        >>> get_cls_by_name("default", {
-        ...     "default": "celery.concurrency.processes.TaskPool"})
-        <class 'celery.concurrency.processes.TaskPool'>
-
-        # Does not try to look up non-string names.
-        >>> from celery.concurrency.processes import TaskPool
-        >>> get_cls_by_name(TaskPool) is TaskPool
-        True
-
-    """
-    if imp is None:
-        imp = importlib.import_module
-
-    if not isinstance(name, basestring):
-        return name                                 # already a class
-
-    name = aliases.get(name) or name
-    sep = ':' if ':' in name else sep
-    module_name, _, cls_name = name.rpartition(sep)
-    if not module_name and package:
-        module_name = package
-    try:
-        module = imp(module_name, package=package, **kwargs)
-    except ValueError, exc:
-        raise ValueError, ValueError(
-                "Couldn't import %r: %s" % (name, exc)), sys.exc_info()[2]
-    return getattr(module, cls_name)
-
-get_symbol_by_name = get_cls_by_name
-
-
-def instantiate(name, *args, **kwargs):
-    """Instantiate class by name.
-
-    See :func:`get_cls_by_name`.
-
-    """
-    return get_cls_by_name(name)(*args, **kwargs)
-
-
-def truncate_text(text, maxlen=128, suffix="..."):
-    """Truncates text to a maximum number of characters."""
-    if len(text) >= maxlen:
-        return text[:maxlen].rsplit(" ", 1)[0] + suffix
-    return text
-
-
-def pluralize(n, text, suffix='s'):
-    if n > 1:
-        return text + suffix
-    return text
-
-
-def abbr(S, max, ellipsis="..."):
-    if S is None:
-        return "???"
-    if len(S) > max:
-        return ellipsis and (S[:max - len(ellipsis)] + ellipsis) or S[:max]
-    return S
-
-
-def abbrtask(S, max):
-    if S is None:
-        return "???"
-    if len(S) > max:
-        module, _, cls = S.rpartition(".")
-        module = abbr(module, max - len(cls) - 3, False)
-        return module + "[.]" + cls
-    return S
-
-
 def isatty(fh):
     # Fixes bug with mod_wsgi:
     #   mod_wsgi.Log object has no attribute isatty.
     return getattr(fh, "isatty", None) and fh.isatty()
 
 
-def textindent(t, indent=0):
-        """Indent text."""
-        return "\n".join(" " * indent + p for p in t.split("\n"))
-
-
-@contextmanager
-def cwd_in_path():
-    cwd = os.getcwd()
-    if cwd in sys.path:
-        yield
-    else:
-        sys.path.insert(0, cwd)
-        try:
-            yield cwd
-        finally:
-            try:
-                sys.path.remove(cwd)
-            except ValueError:
-                pass
-
-
-class NotAPackage(Exception):
-    pass
-
-
-def find_module(module, path=None, imp=None):
-    """Version of :func:`imp.find_module` supporting dots."""
-    if imp is None:
-        imp = importlib.import_module
-    with cwd_in_path():
-        if "." in module:
-            last = None
-            parts = module.split(".")
-            for i, part in enumerate(parts[:-1]):
-                mpart = imp(".".join(parts[:i + 1]))
-                try:
-                    path = mpart.__path__
-                except AttributeError:
-                    raise NotAPackage(module)
-                last = _imp.find_module(parts[i + 1], path)
-            return last
-        return _imp.find_module(module)
-
-
-def import_from_cwd(module, imp=None, package=None):
-    """Import module, but make sure it finds modules
-    located in the current directory.
-
-    Modules located in the current directory has
-    precedence over modules located in `sys.path`.
-    """
-    if imp is None:
-        imp = importlib.import_module
-    with cwd_in_path():
-        return imp(module, package=package)
-
-
-def reload_from_cwd(module, reloader=None):
-    if reloader is None:
-        reloader = reload
-    with cwd_in_path():
-        return reloader(module)
-
-
 def cry():  # pragma: no cover
     """Return stacktrace of all active threads.
 
@@ -435,14 +150,6 @@ def cry():  # pragma: no cover
     return out.getvalue()
 
 
-def uniq(it):
-    seen = set()
-    for obj in it:
-        if obj not in seen:
-            yield obj
-            seen.add(obj)
-
-
 def maybe_reraise():
     """Reraise if an exception is currently being handled, or return
     otherwise."""
@@ -453,3 +160,16 @@ def maybe_reraise():
     finally:
         # see http://docs.python.org/library/sys.html#sys.exc_info
         del(tb)
+
+
+# - XXX Compat
+from .log import LOG_LEVELS     # noqa
+from .imports import (          # noqa
+        qualname as get_full_cls_name,
+        symbol_by_name as gete_cls_by_name,
+        instantiate,
+        import_from_cwd
+)
+from .functional import chunks, noop            # noqa
+from kombu.utils import cached_property, uuid   # noqa
+gen_unique_id = uuid

+ 1 - 1
celery/utils/encoding.py

@@ -11,7 +11,7 @@
 """
 from __future__ import absolute_import
 
-from kombu.utils.encoding import (
+from kombu.utils.encoding import (  # noqa
         default_encode,
         default_encoding,
         bytes_t,

+ 181 - 2
celery/utils/functional.py

@@ -13,7 +13,8 @@ from __future__ import absolute_import
 from __future__ import with_statement
 
 from functools import wraps
-from threading import Lock
+from itertools import islice
+from threading import Lock, RLock
 
 try:
     from collections import Sequence
@@ -21,11 +22,78 @@ except ImportError:
     # <= Py2.5
     Sequence = (list, tuple)  # noqa
 
-from celery.datastructures import LRUCache
+from kombu.utils.functional import promise, maybe_promise
+
+from .compat import UserDict, OrderedDict
 
 KEYWORD_MARK = object()
 
 
+class LRUCache(UserDict):
+    """LRU Cache implementation using a doubly linked list to track access.
+
+    :keyword limit: The maximum number of keys to keep in the cache.
+        When a new key is inserted and the limit has been exceeded,
+        the *Least Recently Used* key will be discarded from the
+        cache.
+
+    """
+
+    def __init__(self, limit=None):
+        self.limit = limit
+        self.mutex = RLock()
+        self.data = OrderedDict()
+
+    def __getitem__(self, key):
+        with self.mutex:
+            value = self[key] = self.data.pop(key)
+        return value
+
+    def keys(self):
+        # userdict.keys in py3k calls __getitem__
+        return self.data.keys()
+
+    def values(self):
+        return list(self._iterate_values())
+
+    def items(self):
+        return list(self._iterate_items())
+
+    def __setitem__(self, key, value):
+        # remove least recently used key.
+        with self.mutex:
+            if self.limit and len(self.data) >= self.limit:
+                self.data.pop(iter(self.data).next())
+            self.data[key] = value
+
+    def __iter__(self):
+        return self.data.iterkeys()
+
+    def _iterate_items(self):
+        for k in self:
+            try:
+                yield (k, self.data[k])
+            except KeyError:
+                pass
+    iteritems = _iterate_items
+
+    def _iterate_values(self):
+        for k in self:
+            try:
+                yield self.data[k]
+            except KeyError:  # pragma: no cover
+                pass
+    itervalues = _iterate_values
+
+    def incr(self, key, delta=1):
+        with self.mutex:
+            # this acts as memcached does- store as a string, but return a
+            # integer as long as it exists and we can cast it
+            newval = int(self.data.pop(key)) + delta
+            self[key] = str(newval)
+        return newval
+
+
 def maybe_list(l):
     if l is None:
         return l
@@ -66,3 +134,114 @@ def memoize(maxsize=None, Cache=LRUCache):
         return _M
 
     return _memoize
+
+
+class mpromise(promise):
+    """Memoized promise.
+
+    The function is only evaluated once, every subsequent access
+    will return the same value.
+
+    .. attribute:: evaluated
+
+        Set to to :const:`True` after the promise has been evaluated.
+
+    """
+    evaluated = False
+    _value = None
+
+    def evaluate(self):
+        if not self.evaluated:
+            self._value = super(mpromise, self).evaluate()
+            self.evaluated = True
+        return self._value
+
+
+def noop(*args, **kwargs):
+    """No operation.
+
+    Takes any arguments/keyword arguments and does nothing.
+
+    """
+    pass
+
+
+def first(predicate, iterable):
+    """Returns the first element in `iterable` that `predicate` returns a
+    :const:`True` value for."""
+    # XXX This function is not used anymore
+    for item in iterable:
+        if predicate(item):
+            return item
+
+
+def firstmethod(method):
+    """Returns a functions that with a list of instances,
+    finds the first instance that returns a value for the given method.
+
+    The list can also contain promises (:class:`promise`.)
+
+    """
+
+    def _matcher(seq, *args, **kwargs):
+        for cls in seq:
+            try:
+                answer = getattr(maybe_promise(cls), method)(*args, **kwargs)
+                if answer is not None:
+                    return answer
+            except AttributeError:
+                pass
+    return _matcher
+
+
+def chunks(it, n):
+    """Split an iterator into chunks with `n` elements each.
+
+    Examples
+
+        # n == 2
+        >>> x = chunks(iter([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), 2)
+        >>> list(x)
+        [[0, 1], [2, 3], [4, 5], [6, 7], [8, 9], [10]]
+
+        # n == 3
+        >>> x = chunks(iter([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), 3)
+        >>> list(x)
+        [[0, 1, 2], [3, 4, 5], [6, 7, 8], [9, 10]]
+
+    """
+    # XXX This function is not used anymore, at least not by Celery itself.
+    for first in it:
+        yield [first] + list(islice(it, n - 1))
+
+
+def padlist(container, size, default=None):
+    """Pad list with default elements.
+
+    Examples:
+
+        >>> first, last, city = padlist(["George", "Costanza", "NYC"], 3)
+        ("George", "Costanza", "NYC")
+        >>> first, last, city = padlist(["George", "Costanza"], 3)
+        ("George", "Costanza", None)
+        >>> first, last, city, planet = padlist(["George", "Costanza",
+                                                 "NYC"], 4, default="Earth")
+        ("George", "Costanza", "NYC", "Earth")
+
+    """
+    return list(container)[:size] + [default] * (size - len(container))
+
+
+def mattrgetter(*attrs):
+    """Like :func:`operator.itemgetter` but returns :const:`None` on missing
+    attributes instead of raising :exc:`AttributeError`."""
+    return lambda obj: dict((attr, getattr(obj, attr, None))
+                                for attr in attrs)
+
+
+def uniq(it):
+    seen = set()
+    for obj in it:
+        if obj not in seen:
+            yield obj
+            seen.add(obj)

+ 147 - 0
celery/utils/imports.py

@@ -0,0 +1,147 @@
+from __future__ import absolute_import
+from __future__ import with_statement
+
+import imp as _imp
+import importlib
+import os
+import sys
+
+from contextlib import contextmanager
+
+from .compat import reload
+
+
+class NotAPackage(Exception):
+    pass
+
+
+if sys.version_info >= (3, 3):
+
+    def qualname(obj):
+        return obj.__qualname__
+
+else:
+
+    def qualname(obj):  # noqa
+        if not hasattr(obj, "__name__") and hasattr(obj, "__class__"):
+            return qualname(obj.__class__)
+
+        return '.'.join([obj.__module__, obj.__name__])
+
+
+def symbol_by_name(name, aliases={}, imp=None, package=None,
+        sep='.', **kwargs):
+    """Get symbol by qualified name.
+
+    The name should be the full dot-separated path to the class::
+
+        modulename.ClassName
+
+    Example::
+
+        celery.concurrency.processes.TaskPool
+                                    ^- class name
+
+    or using ':' to separate module and symbol::
+
+        celery.concurrency.processes:TaskPool
+
+    If `aliases` is provided, a dict containing short name/long name
+    mappings, the name is looked up in the aliases first.
+
+    Examples:
+
+        >>> symbol_by_name("celery.concurrency.processes.TaskPool")
+        <class 'celery.concurrency.processes.TaskPool'>
+
+        >>> symbol_by_name("default", {
+        ...     "default": "celery.concurrency.processes.TaskPool"})
+        <class 'celery.concurrency.processes.TaskPool'>
+
+        # Does not try to look up non-string names.
+        >>> from celery.concurrency.processes import TaskPool
+        >>> symbol_by_name(TaskPool) is TaskPool
+        True
+
+    """
+    if imp is None:
+        imp = importlib.import_module
+
+    if not isinstance(name, basestring):
+        return name                                 # already a class
+
+    name = aliases.get(name) or name
+    sep = ':' if ':' in name else sep
+    module_name, _, cls_name = name.rpartition(sep)
+    if not module_name and package:
+        module_name = package
+    try:
+        module = imp(module_name, package=package, **kwargs)
+    except ValueError, exc:
+        raise ValueError, ValueError(
+                "Couldn't import %r: %s" % (name, exc)), sys.exc_info()[2]
+    return getattr(module, cls_name)
+
+
+def instantiate(name, *args, **kwargs):
+    """Instantiate class by name.
+
+    See :func:`symbol_by_name`.
+
+    """
+    return symbol_by_name(name)(*args, **kwargs)
+
+
+@contextmanager
+def cwd_in_path():
+    cwd = os.getcwd()
+    if cwd in sys.path:
+        yield
+    else:
+        sys.path.insert(0, cwd)
+        try:
+            yield cwd
+        finally:
+            try:
+                sys.path.remove(cwd)
+            except ValueError:
+                pass
+
+
+def find_module(module, path=None, imp=None):
+    """Version of :func:`imp.find_module` supporting dots."""
+    if imp is None:
+        imp = importlib.import_module
+    with cwd_in_path():
+        if "." in module:
+            last = None
+            parts = module.split(".")
+            for i, part in enumerate(parts[:-1]):
+                mpart = imp(".".join(parts[:i + 1]))
+                try:
+                    path = mpart.__path__
+                except AttributeError:
+                    raise NotAPackage(module)
+                last = _imp.find_module(parts[i + 1], path)
+            return last
+        return _imp.find_module(module)
+
+
+def import_from_cwd(module, imp=None, package=None):
+    """Import module, but make sure it finds modules
+    located in the current directory.
+
+    Modules located in the current directory has
+    precedence over modules located in `sys.path`.
+    """
+    if imp is None:
+        imp = importlib.import_module
+    with cwd_in_path():
+        return imp(module, package=package)
+
+
+def reload_from_cwd(module, reloader=None):
+    if reloader is None:
+        reloader = reload
+    with cwd_in_path():
+        return reloader(module)

+ 2 - 2
celery/utils/mail.py

@@ -19,7 +19,7 @@ try:
 except ImportError:
     from email.MIMEText import MIMEText  # noqa
 
-from . import get_symbol_by_name
+from .imports import symbol_by_name
 
 supports_timeout = sys.version_info >= (2, 6)
 
@@ -168,7 +168,7 @@ celeryd at %%(hostname)s.
     def should_send(self, context, exc):
         """Returns true or false depending on if a task error mail
         should be sent for this type of error."""
-        allow_classes = tuple(map(get_symbol_by_name,  self.error_whitelist))
+        allow_classes = tuple(map(symbol_by_name,  self.error_whitelist))
         return not self.error_whitelist or isinstance(exc, allow_classes)
 
     def format_subject(self, context):

+ 1 - 1
celery/utils/timeutils.py

@@ -17,7 +17,7 @@ from datetime import datetime, timedelta
 from dateutil import tz
 from dateutil.parser import parse as parse_iso8601
 
-from . import pluralize
+from .text import pluralize
 
 try:
     import pytz

+ 2 - 1
celery/worker/__init__.py

@@ -28,7 +28,8 @@ from .. import concurrency as _concurrency
 from ..app import app_or_default
 from ..app.abstract import configurated, from_config
 from ..exceptions import SystemTerminate
-from ..utils import noop, qualname, reload_from_cwd
+from ..utils.functional import noop
+from ..utils.imports import qualname, reload_from_cwd
 
 from . import state
 from .buckets import TaskBucket, FastQueue

+ 1 - 1
celery/worker/consumer.py

@@ -87,8 +87,8 @@ from ..abstract import StartStopComponent
 from ..app import app_or_default
 from ..datastructures import AttributeDict
 from ..exceptions import InvalidTaskError
-from ..utils import noop
 from ..utils import timer2
+from ..utils.functional import noop
 
 from . import state
 from .control import Panel

+ 5 - 2
celery/worker/job.py

@@ -19,6 +19,7 @@ import sys
 
 from datetime import datetime
 
+from kombu.utils import kwdict
 from kombu.utils.encoding import safe_repr, safe_str
 
 from .. import current_app
@@ -27,7 +28,9 @@ from ..app import app_or_default
 from ..datastructures import ExceptionInfo
 from ..execute.trace import build_tracer, trace_task, report_internal_error
 from ..platforms import set_mp_process_title as setps
-from ..utils import noop, kwdict, fun_takes_kwargs, truncate_text
+from ..utils import fun_takes_kwargs
+from ..utils.functional import noop
+from ..utils.text import truncate
 from ..utils.timeutils import maybe_iso8601, timezone
 
 from . import state
@@ -406,7 +409,7 @@ class Request(object):
     def repr_result(self, result, maxlen=46):
         # 46 is the length needed to fit
         #     "the quick brown fox jumps over the lazy dog" :)
-        return truncate_text(safe_repr(result), maxlen)
+        return truncate(safe_repr(result), maxlen)
 
     def info(self, safe=False):
         return {"id": self.id,