Utils: Renames .datastructures -> utils.collections
This commit is contained in:
Родитель
9a9eff0956
Коммит
707dfdbadb
|
@ -734,7 +734,7 @@ is following the conventions.
|
|||
from collections import deque
|
||||
from Queue import Queue, Empty
|
||||
|
||||
from .datastructures import TokenBucket
|
||||
from .platforms import Pidfile
|
||||
from .five import zip_longest, items, range
|
||||
from .utils import timeutils
|
||||
|
||||
|
|
|
@ -30,7 +30,6 @@ from celery._state import (
|
|||
get_current_worker_task, connect_on_app_finalize,
|
||||
_announce_app_finalized,
|
||||
)
|
||||
from celery.datastructures import AttributeDictMixin
|
||||
from celery.exceptions import AlwaysEagerIgnored, ImproperlyConfigured
|
||||
from celery.five import (
|
||||
UserDict, bytes_if_py2, python_2_unicode_compatible, values,
|
||||
|
@ -39,6 +38,7 @@ from celery.loaders import get_loader_cls
|
|||
from celery.local import PromiseProxy, maybe_evaluate
|
||||
from celery.utils import abstract
|
||||
from celery.utils import gen_task_name
|
||||
from celery.utils.collections import AttributeDictMixin
|
||||
from celery.utils.dispatch import Signal
|
||||
from celery.utils.functional import first, maybe_list, head_from_fun
|
||||
from celery.utils.timeutils import timezone
|
||||
|
|
|
@ -18,10 +18,10 @@ from types import ModuleType
|
|||
|
||||
from kombu.utils.url import maybe_sanitize_url
|
||||
|
||||
from celery.datastructures import ConfigurationView
|
||||
from celery.exceptions import ImproperlyConfigured
|
||||
from celery.five import items, keys, string_t, values
|
||||
from celery.platforms import pyimplementation
|
||||
from celery.utils.collections import ConfigurationView
|
||||
from celery.utils.text import pretty
|
||||
from celery.utils.imports import import_from_cwd, symbol_by_name, qualname
|
||||
|
||||
|
|
|
@ -31,12 +31,12 @@ from kombu.utils.url import maybe_sanitize_url
|
|||
from celery import states
|
||||
from celery import current_app, group, maybe_signature
|
||||
from celery.app import current_task
|
||||
from celery.datastructures import BufferMap
|
||||
from celery.exceptions import ChordError, TimeoutError, TaskRevokedError
|
||||
from celery.five import items
|
||||
from celery.result import (
|
||||
GroupResult, ResultBase, allow_join_result, result_from_tuple,
|
||||
)
|
||||
from celery.utils.collections import BufferMap
|
||||
from celery.utils.functional import LRUCache
|
||||
from celery.utils.log import get_logger
|
||||
from celery.utils.serialization import (
|
||||
|
|
|
@ -21,8 +21,8 @@ from kombu.utils import cached_property
|
|||
from kombu.utils.encoding import safe_str
|
||||
|
||||
from celery import signals
|
||||
from celery.datastructures import DictAttribute, force_mapping
|
||||
from celery.five import reraise, string_t
|
||||
from celery.utils.collections import DictAttribute, force_mapping
|
||||
from celery.utils.functional import maybe_list
|
||||
from celery.utils.imports import (
|
||||
import_from_cwd, symbol_by_name, NotAPackage, find_module,
|
||||
|
|
|
@ -11,9 +11,9 @@ from __future__ import absolute_import, unicode_literals
|
|||
import os
|
||||
import warnings
|
||||
|
||||
from celery.datastructures import DictAttribute
|
||||
from celery.exceptions import NotConfigured
|
||||
from celery.utils import strtobool
|
||||
from celery.utils.collections import DictAttribute
|
||||
|
||||
from .base import BaseLoader
|
||||
|
||||
|
|
|
@ -21,11 +21,11 @@ from kombu.utils import cached_property
|
|||
from . import current_app
|
||||
from .five import python_2_unicode_compatible, range, string_t
|
||||
from .utils import is_iterable
|
||||
from .utils.collections import AttributeDict
|
||||
from .utils.timeutils import (
|
||||
weekday, maybe_timedelta, remaining, humanize_seconds,
|
||||
timezone, maybe_make_aware, ffwd, localize
|
||||
)
|
||||
from .datastructures import AttributeDict
|
||||
|
||||
__all__ = ['ParseException', 'schedule', 'crontab', 'crontab_parser',
|
||||
'maybe_schedule', 'solar']
|
||||
|
|
|
@ -9,8 +9,8 @@ from celery import signature
|
|||
from celery import states
|
||||
from celery import uuid
|
||||
from celery.canvas import Signature
|
||||
from celery.datastructures import AttributeDict
|
||||
from celery.exceptions import ChordError, ImproperlyConfigured
|
||||
from celery.utils.collections import AttributeDict
|
||||
|
||||
from celery.tests.case import (
|
||||
ANY, AppCase, ContextMock, Mock, mock,
|
||||
|
|
|
@ -7,8 +7,8 @@ import socket
|
|||
from itertools import cycle
|
||||
|
||||
from celery.app.defaults import DEFAULTS
|
||||
from celery.datastructures import AttributeDict
|
||||
from celery.five import range
|
||||
from celery.utils.collections import AttributeDict
|
||||
from celery.utils.functional import noop
|
||||
from celery.utils.objects import Bunch
|
||||
|
||||
|
|
|
@ -8,7 +8,7 @@ from itertools import count
|
|||
from billiard.einfo import ExceptionInfo
|
||||
from time import time
|
||||
|
||||
from celery.datastructures import (
|
||||
from celery.utils.collections import (
|
||||
AttributeDict,
|
||||
BufferMap,
|
||||
ConfigurationView,
|
|
@ -7,13 +7,13 @@ from collections import deque
|
|||
|
||||
from billiard.exceptions import RestartFreqExceeded
|
||||
|
||||
from celery.datastructures import LimitedSet
|
||||
from celery.worker.consumer.agent import Agent
|
||||
from celery.worker.consumer.consumer import CLOSE, Consumer, dump_body
|
||||
from celery.worker.consumer.gossip import Gossip
|
||||
from celery.worker.consumer.heart import Heart
|
||||
from celery.worker.consumer.mingle import Mingle
|
||||
from celery.worker.consumer.tasks import Tasks
|
||||
from celery.utils.collections import LimitedSet
|
||||
|
||||
from celery.tests.case import (
|
||||
AppCase, ContextMock, Mock, call, patch, skip,
|
||||
|
|
|
@ -8,7 +8,6 @@ from datetime import datetime, timedelta
|
|||
|
||||
from kombu import pidbox
|
||||
|
||||
from celery.datastructures import AttributeDict
|
||||
from celery.five import Queue as FastQueue
|
||||
from celery.utils import uuid
|
||||
from celery.utils.timer2 import Timer
|
||||
|
@ -19,6 +18,7 @@ from celery.worker import state as worker_state
|
|||
from celery.worker.request import Request
|
||||
from celery.worker.state import revoked
|
||||
from celery.worker.pidbox import Pidbox, gPidbox
|
||||
from celery.utils.collections import AttributeDict
|
||||
|
||||
from celery.tests.case import AppCase, Mock, TaskMessage, call, patch
|
||||
|
||||
|
|
|
@ -5,9 +5,9 @@ import pickle
|
|||
from time import time
|
||||
|
||||
from celery import uuid
|
||||
from celery.datastructures import LimitedSet
|
||||
from celery.exceptions import WorkerShutdown, WorkerTerminate
|
||||
from celery.worker import state
|
||||
from celery.utils.collections import LimitedSet
|
||||
|
||||
from celery.tests.case import AppCase, Mock, patch
|
||||
|
||||
|
|
|
@ -1,9 +1,9 @@
|
|||
# -*- coding: utf-8 -*-
|
||||
"""
|
||||
``celery.datastructures``
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
``celery.utils.collections``
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
Custom types and data-structures.
|
||||
Custom maps, sets, sequences and other data structures.
|
||||
|
||||
"""
|
||||
from __future__ import absolute_import, unicode_literals
|
||||
|
@ -18,12 +18,10 @@ from collections import (
|
|||
from heapq import heapify, heappush, heappop
|
||||
from itertools import chain, count
|
||||
|
||||
from billiard.einfo import ExceptionInfo # noqa
|
||||
from kombu.utils.limits import TokenBucket # noqa
|
||||
|
||||
from celery.five import Empty, items, keys, python_2_unicode_compatible, values
|
||||
from celery.utils.functional import LRUCache, first, uniq # noqa
|
||||
from celery.utils.text import match_case
|
||||
|
||||
from .functional import first, uniq
|
||||
from .text import match_case
|
||||
|
||||
try:
|
||||
# pypy: dicts are ordered in recent versions
|
||||
|
@ -39,8 +37,9 @@ except ImportError:
|
|||
LazySettings = LazyObject # noqa
|
||||
|
||||
__all__ = [
|
||||
'AttributeDictMixin', 'AttributeDict', 'DictAttribute',
|
||||
'ConfigurationView', 'LimitedSet',
|
||||
'AttributeDictMixin', 'AttributeDict', 'BufferMap', 'ChainMap',
|
||||
'ConfigurationView', 'DictAttribute', 'Evictable',
|
||||
'LimitedSet', 'Messagebuffer', 'OrderedDict', 'force_mapping',
|
||||
]
|
||||
|
||||
PY3 = sys.version_info[0] >= 3
|
||||
|
@ -56,6 +55,54 @@ def force_mapping(m):
|
|||
return DictAttribute(m) if not isinstance(m, Mapping) else m
|
||||
|
||||
|
||||
class OrderedDict(_OrderedDict):
|
||||
|
||||
if PY3: # pragma: no cover
|
||||
def _LRUkey(self):
|
||||
# return value of od.keys does not support __next__,
|
||||
# but this version will also not create a copy of the list.
|
||||
return next(iter(keys(self)))
|
||||
else:
|
||||
if _dict_is_ordered: # pragma: no cover
|
||||
def _LRUkey(self):
|
||||
# iterkeys is iterable.
|
||||
return next(self.iterkeys())
|
||||
else:
|
||||
def _LRUkey(self):
|
||||
return self._OrderedDict__root[1][2]
|
||||
|
||||
if not hasattr(_OrderedDict, 'move_to_end'):
|
||||
if _dict_is_ordered: # pragma: no cover
|
||||
|
||||
def move_to_end(self, key, last=True):
|
||||
if not last:
|
||||
# we don't use this argument, and the only way to
|
||||
# implement this on PyPy seems to be O(n): creating a
|
||||
# copy with the order changed, so we just raise.
|
||||
raise NotImplementedError('no last=True on PyPy')
|
||||
self[key] = self.pop(key)
|
||||
|
||||
else:
|
||||
|
||||
def move_to_end(self, key, last=True):
|
||||
link = self._OrderedDict__map[key]
|
||||
link_prev = link[0]
|
||||
link_next = link[1]
|
||||
link_prev[1] = link_next
|
||||
link_next[0] = link_prev
|
||||
root = self._OrderedDict__root
|
||||
if last:
|
||||
last = root[0]
|
||||
link[0] = last
|
||||
link[1] = root
|
||||
last[1] = root[0] = link
|
||||
else:
|
||||
first = root[1]
|
||||
link[0] = root
|
||||
link[1] = first
|
||||
root[1] = first[0] = link
|
||||
|
||||
|
||||
class AttributeDictMixin(object):
|
||||
"""Augment classes with a Mapping interface by adding attribute access.
|
||||
|
||||
|
@ -591,54 +638,6 @@ class LimitedSet(object):
|
|||
MutableSet.register(LimitedSet)
|
||||
|
||||
|
||||
class OrderedDict(_OrderedDict):
|
||||
|
||||
if PY3: # pragma: no cover
|
||||
def _LRUkey(self):
|
||||
# return value of od.keys does not support __next__,
|
||||
# but this version will also not create a copy of the list.
|
||||
return next(iter(keys(self)))
|
||||
else:
|
||||
if _dict_is_ordered: # pragma: no cover
|
||||
def _LRUkey(self):
|
||||
# iterkeys is iterable.
|
||||
return next(self.iterkeys())
|
||||
else:
|
||||
def _LRUkey(self):
|
||||
return self._OrderedDict__root[1][2]
|
||||
|
||||
if not hasattr(_OrderedDict, 'move_to_end'):
|
||||
if _dict_is_ordered: # pragma: no cover
|
||||
|
||||
def move_to_end(self, key, last=True):
|
||||
if not last:
|
||||
# we don't use this argument, and the only way to
|
||||
# implement this on PyPy seems to be O(n): creating a
|
||||
# copy with the order changed, so we just raise.
|
||||
raise NotImplementedError('no last=True on PyPy')
|
||||
self[key] = self.pop(key)
|
||||
|
||||
else:
|
||||
|
||||
def move_to_end(self, key, last=True):
|
||||
link = self._OrderedDict__map[key]
|
||||
link_prev = link[0]
|
||||
link_next = link[1]
|
||||
link_prev[1] = link_next
|
||||
link_next[0] = link_prev
|
||||
root = self._OrderedDict__root
|
||||
if last:
|
||||
last = root[0]
|
||||
link[0] = last
|
||||
link[1] = root
|
||||
last[1] = root[0] = link
|
||||
else:
|
||||
first = root[1]
|
||||
link[0] = root
|
||||
link[1] = first
|
||||
root[1] = first[0] = link
|
||||
|
||||
|
||||
class Evictable(object):
|
||||
|
||||
Empty = Empty
|
|
@ -6,7 +6,7 @@ import threading
|
|||
from kombu.common import ignore_errors
|
||||
from kombu.utils.encoding import safe_str
|
||||
|
||||
from celery.datastructures import AttributeDict
|
||||
from celery.utils.collections import AttributeDict
|
||||
from celery.utils.functional import pass1
|
||||
from celery.utils.log import get_logger
|
||||
|
||||
|
|
|
@ -22,9 +22,9 @@ from kombu.serialization import pickle, pickle_protocol
|
|||
from kombu.utils import cached_property
|
||||
|
||||
from celery import __version__
|
||||
from celery.datastructures import LimitedSet
|
||||
from celery.exceptions import WorkerShutdown, WorkerTerminate
|
||||
from celery.five import Counter
|
||||
from celery.utils.collections import LimitedSet
|
||||
|
||||
__all__ = ['SOFTWARE_INFO', 'reserved_requests', 'active_requests',
|
||||
'total_count', 'revoked', 'task_reserved', 'maybe_shutdown',
|
||||
|
|
|
@ -763,7 +763,7 @@ is following the conventions.
|
|||
from collections import deque
|
||||
from Queue import Queue, Empty
|
||||
|
||||
from .datastructures import TokenBucket
|
||||
from .platforms import Pidfile
|
||||
from .five import zip_longest, items, range
|
||||
from .utils import timeutils
|
||||
|
||||
|
|
|
@ -82,8 +82,8 @@ Changes
|
|||
|
||||
See issue #110.
|
||||
|
||||
* celery.execute.apply: Should return exception, not :class:`~celery.datastructures.ExceptionInfo`
|
||||
on error.
|
||||
* celery.execute.apply: Should return exception, not
|
||||
:class:`~billiard.einfo.ExceptionInfo` on error.
|
||||
|
||||
See issue #111.
|
||||
|
||||
|
|
|
@ -908,7 +908,7 @@ News
|
|||
* New Task handler called after the task returns:
|
||||
:meth:`~celery.task.base.Task.after_return`.
|
||||
|
||||
* :class:`~celery.datastructures.ExceptionInfo` now passed to
|
||||
* :class:`~billiard.einfo.ExceptionInfo` now passed to
|
||||
:meth:`~celery.task.base.Task.on_retry`/
|
||||
:meth:`~celery.task.base.Task.on_failure` as ``einfo`` keyword argument.
|
||||
|
||||
|
|
|
@ -614,7 +614,7 @@ News
|
|||
|
||||
* ``timedelta_seconds``: Use ``timedelta.total_seconds`` if running on Python 2.7
|
||||
|
||||
* :class:`~celery.datastructures.TokenBucket`: Generic Token Bucket algorithm
|
||||
* :class:`~kombu.utils.limits.TokenBucket`: Generic Token Bucket algorithm
|
||||
|
||||
* :mod:`celery.events.state`: Recording of cluster state can now
|
||||
be paused and resumed, including support for buffering.
|
||||
|
|
|
@ -630,7 +630,7 @@ If you're looking for versions prior to 3.0.x you should go to :ref:`history`.
|
|||
|
||||
Fix contributed by Mher Movsisyan.
|
||||
|
||||
- :class:`celery.datastructures.LRUCache` is now pickleable.
|
||||
- :class:`celery.utils.functional.LRUCache` is now pickleable.
|
||||
|
||||
Fix contributed by Mher Movsisyan.
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ new in Celery 3.1.
|
|||
Contributed by Sebastian Kalinowski.
|
||||
|
||||
- **Utils**: The ``.discard(item)`` method of
|
||||
:class:`~celery.datastructures.LimitedSet` did not actually remove the item
|
||||
:class:`~celery.utils.collections.LimitedSet` did not actually remove the item
|
||||
(Issue #3087).
|
||||
|
||||
Fix contributed by Dave Smith.
|
||||
|
|
|
@ -1,8 +0,0 @@
|
|||
.. currentmodule:: celery.datastructures
|
||||
|
||||
.. contents::
|
||||
:local:
|
||||
|
||||
.. automodule:: celery.datastructures
|
||||
:members:
|
||||
:undoc-members:
|
|
@ -0,0 +1,8 @@
|
|||
.. currentmodule:: celery.utils.collections
|
||||
|
||||
.. contents::
|
||||
:local:
|
||||
|
||||
.. automodule:: celery.utils.collections
|
||||
:members:
|
||||
:undoc-members:
|
|
@ -40,7 +40,6 @@
|
|||
celery.app.trace
|
||||
celery.app.annotations
|
||||
celery.app.routes
|
||||
celery.datastructures
|
||||
celery.security.certificate
|
||||
celery.security.key
|
||||
celery.security.serialization
|
||||
|
@ -52,6 +51,7 @@
|
|||
celery.backends.database.session
|
||||
celery.utils
|
||||
celery.utils.abstract
|
||||
celery.utils.collections
|
||||
celery.utils.functional
|
||||
celery.utils.graph
|
||||
celery.utils.objects
|
||||
|
|
|
@ -285,7 +285,7 @@ Provides arguments:
|
|||
|
||||
* ``einfo``
|
||||
|
||||
The :class:`celery.datastructures.ExceptionInfo` instance.
|
||||
The :class:`billiard.einfo.ExceptionInfo` instance.
|
||||
|
||||
.. signal:: task_revoked
|
||||
|
||||
|
|
|
@ -1261,7 +1261,7 @@ Handlers
|
|||
:param kwargs: Original keyword arguments for the task
|
||||
that returned.
|
||||
|
||||
:keyword einfo: :class:`~celery.datastructures.ExceptionInfo`
|
||||
:keyword einfo: :class:`~billiard.einfo.ExceptionInfo`
|
||||
instance, containing the traceback (if any).
|
||||
|
||||
The return value of this handler is ignored.
|
||||
|
@ -1276,7 +1276,7 @@ Handlers
|
|||
:param kwargs: Original keyword arguments for the task
|
||||
that failed.
|
||||
|
||||
:keyword einfo: :class:`~celery.datastructures.ExceptionInfo`
|
||||
:keyword einfo: :class:`~billiard.einfo.ExceptionInfo`
|
||||
instance, containing the traceback.
|
||||
|
||||
The return value of this handler is ignored.
|
||||
|
@ -1290,7 +1290,7 @@ Handlers
|
|||
:param args: Original arguments for the retried task.
|
||||
:param kwargs: Original keyword arguments for the retried task.
|
||||
|
||||
:keyword einfo: :class:`~celery.datastructures.ExceptionInfo`
|
||||
:keyword einfo: :class:`~billiard.einfo.ExceptionInfo`
|
||||
instance, containing the traceback.
|
||||
|
||||
The return value of this handler is ignored.
|
||||
|
|
Загрузка…
Ссылка в новой задаче