diff --git a/CHANGELOG.md b/CHANGELOG.md index 417bb15e..5bbef5f5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,17 @@ +9.1.0 +----- + +The bulk of the changes are related to how the library entities are now +interconnected. In previous versions of `aio_pika.Channel` instances not +contains a link to the `aio_pika.Connection` instances for now is contains it. + +While I don't want custom code to work directly with the `aiormq.Channel` +instance, this was a public API and I should warn you about the change here. +The `aio_pika.Channel.channel` property is deprecated. Use +`aio_pika.Channel.get_underlay_chanel()` instead. +Now all library entities already use this method. + + 9.0.7 ----- @@ -17,9 +31,9 @@ ----- * Prevent 'Task exception was never retrieved' #524 - If future.exception() is not called (even on cancelled futures), it seems Python - will then log 'Task exception was never retrieved'. Rewriting this logic - slightly should hopefully achieve the same functionality while + If future.exception() is not called (even on cancelled futures), it seems Python + will then log 'Task exception was never retrieved'. Rewriting this logic + slightly should hopefully achieve the same functionality while preventing the Python errors. * Avoid implicitly depending on setuptools #526 diff --git a/Makefile b/Makefile index 77a43b47..38df37c9 100644 --- a/Makefile +++ b/Makefile @@ -1,6 +1,5 @@ all: test -RABBITMQ_CONTAINER_NAME:=aio_pika_rabbitmq RABBITMQ_IMAGE:=mosquito/aiormq-rabbitmq test: @@ -8,10 +7,10 @@ test: tox rabbitmq: + docker kill $(docker ps -f label=aio-pika.rabbitmq -q) || true docker pull $(RABBITMQ_IMAGE) - docker kill $(RABBITMQ_CONTAINER_NAME) || true docker run --rm -d \ - --name $(RABBITMQ_CONTAINER_NAME) \ + -l aio-pika.rabbitmq \ -p 5671:5671 \ -p 5672:5672 \ -p 15671:15671 \ diff --git a/aio_pika/abc.py b/aio_pika/abc.py index dbd2b679..31d40891 100644 --- a/aio_pika/abc.py +++ b/aio_pika/abc.py @@ -11,6 +11,7 @@ Generator, Iterator, Optional, Type, TypeVar, Union, overload, ) + if sys.version_info >= (3, 8): from typing import Literal, TypedDict else: @@ -81,11 +82,6 @@ class DeclarationResult: class AbstractTransaction: state: TransactionState - @property - @abstractmethod - def channel(self) -> "AbstractChannel": - raise NotImplementedError - @abstractmethod async def select( self, timeout: TimeoutType = None, @@ -244,7 +240,7 @@ async def __aexit__( class AbstractQueue: - channel: aiormq.abc.AbstractChannel + channel: "AbstractChannel" name: str durable: bool exclusive: bool @@ -307,7 +303,7 @@ async def unbind( @abstractmethod async def consume( self, - callback: Callable[[AbstractIncomingMessage], Any], + callback: Callable[[AbstractIncomingMessage], Awaitable[Any]], no_ack: bool = False, exclusive: bool = False, arguments: Arguments = None, @@ -409,7 +405,7 @@ class AbstractExchange(ABC): @abstractmethod def __init__( self, - channel: aiormq.abc.AbstractChannel, + channel: "AbstractChannel", name: str, type: Union[ExchangeType, str] = ExchangeType.DIRECT, *, @@ -528,9 +524,8 @@ def is_closed(self) -> bool: def close(self, exc: Optional[ExceptionType] = None) -> Awaitable[None]: raise NotImplementedError - @property @abstractmethod - def channel(self) -> aiormq.abc.AbstractChannel: + async def get_underlay_channel(self) -> aiormq.abc.AbstractChannel: raise NotImplementedError @property @@ -760,7 +755,7 @@ async def update_secret( class AbstractRobustQueue(AbstractQueue): @abstractmethod - def restore(self, channel: aiormq.abc.AbstractChannel) -> Awaitable[None]: + def restore(self) -> Awaitable[None]: raise NotImplementedError @abstractmethod @@ -791,7 +786,7 @@ async def consume( class AbstractRobustExchange(AbstractExchange): @abstractmethod - def restore(self, channel: aiormq.abc.AbstractChannel) -> Awaitable[None]: + def restore(self) -> Awaitable[None]: raise NotImplementedError @abstractmethod @@ -815,7 +810,7 @@ def reopen(self) -> Awaitable[None]: raise NotImplementedError @abstractmethod - async def restore(self, connection: aiormq.abc.AbstractConnection) -> None: + async def restore(self) -> None: raise NotImplementedError @abstractmethod diff --git a/aio_pika/channel.py b/aio_pika/channel.py index 77bcd3c6..7741b7af 100644 --- a/aio_pika/channel.py +++ b/aio_pika/channel.py @@ -1,4 +1,5 @@ import asyncio +import warnings from abc import ABC from types import TracebackType from typing import Any, AsyncContextManager, Generator, Optional, Type, Union @@ -9,9 +10,10 @@ from pamqp.common import Arguments from .abc import ( - AbstractChannel, AbstractExchange, AbstractQueue, TimeoutType, - UnderlayChannel, + AbstractChannel, AbstractConnection, AbstractExchange, AbstractQueue, + TimeoutType, UnderlayChannel, ) +from .exceptions import ChannelInvalidStateError from .exchange import Exchange, ExchangeType from .log import get_logger from .message import IncomingMessage @@ -52,7 +54,7 @@ class Channel(ChannelContext): def __init__( self, - connection: aiormq.abc.AbstractConnection, + connection: AbstractConnection, channel_number: Optional[int] = None, publisher_confirms: bool = True, on_return_raises: bool = False, @@ -73,12 +75,12 @@ def __init__( 'without "publisher_confirms"', ) - self._connection: aiormq.abc.AbstractConnection = connection + self._connection: AbstractConnection = connection # That's means user closed channel instance explicitly self._closed: bool = False - self._channel = None + self._channel: Optional[UnderlayChannel] = None self._channel_number = channel_number self.close_callbacks = CallbackCollection(self) @@ -99,9 +101,10 @@ def is_closed(self) -> bool: side or after the close() method has been called.""" if not self.is_initialized or self._closed: return True - if not self._channel: + channel = self._channel + if channel is None: return True - return self._channel.channel.is_closed + return channel.channel.is_closed async def close( self, @@ -119,28 +122,32 @@ async def close( self._closed = True await self._channel.close() - @property - def channel(self) -> aiormq.abc.AbstractChannel: + async def get_underlay_channel(self) -> aiormq.abc.AbstractChannel: if not self.is_initialized or not self._channel: raise aiormq.exceptions.ChannelInvalidStateError( "Channel was not opened", ) - if self.is_closed: - raise aiormq.exceptions.ChannelInvalidStateError( - "Channel has been closed", - ) + return self._channel.channel + @property + def channel(self) -> aiormq.abc.AbstractChannel: + warnings.warn( + "This property is deprecated, do not use this anymore.", + DeprecationWarning, + ) + if self._channel is None: + raise aiormq.exceptions.ChannelInvalidStateError return self._channel.channel @property def number(self) -> Optional[int]: - return ( - self.channel.number - if self.is_initialized - else self._channel_number - ) + if self._channel is None: + return self._channel_number + + underlay_channel: UnderlayChannel = self._channel + return underlay_channel.channel.number def __str__(self) -> str: return "{}".format(self.number or "Not initialized channel") @@ -148,16 +155,25 @@ def __str__(self) -> str: async def _open(self) -> None: await self._connection.ready() + transport = self._connection.transport + if transport is None: + raise ChannelInvalidStateError("No active transport in channel") + channel = await UnderlayChannel.create( - self._connection, + transport.connection, self._on_close, publisher_confirms=self.publisher_confirms, on_return_raises=self.on_return_raises, channel_number=self._channel_number, ) - await self._on_open(channel.channel) self._channel = channel + try: + await self._on_open() + except BaseException as e: + await channel.close(e) + self._channel = None + raise self._closed = False async def initialize(self, timeout: TimeoutType = None) -> None: @@ -169,9 +185,9 @@ async def initialize(self, timeout: TimeoutType = None) -> None: await self._open() await self._on_initialized() - async def _on_open(self, channel: aiormq.abc.AbstractChannel) -> None: + async def _on_open(self) -> None: self.default_exchange: Exchange = self.EXCHANGE_CLASS( - channel=channel, + channel=self, arguments=None, auto_delete=False, durable=False, @@ -192,7 +208,8 @@ async def _on_close(self, closing: asyncio.Future) -> None: self._channel.channel.on_return_callbacks.discard(self._on_return) async def _on_initialized(self) -> None: - self.channel.on_return_callbacks.add(self._on_return) + channel = await self.get_underlay_channel() + channel.on_return_callbacks.add(self._on_return) def _on_return(self, message: aiormq.abc.DeliveredMessage) -> None: self.return_callbacks(IncomingMessage(message, no_ack=True)) @@ -241,7 +258,7 @@ async def declare_exchange( durable = False exchange = self.EXCHANGE_CLASS( - channel=self.channel, + channel=self, name=name, type=type, durable=durable, @@ -281,7 +298,7 @@ async def get_exchange( return await self.declare_exchange(name=name, passive=True) else: return self.EXCHANGE_CLASS( - channel=self.channel, + channel=self, name=name, durable=False, auto_delete=False, @@ -321,7 +338,7 @@ async def declare_queue( """ queue: AbstractQueue = self.QUEUE_CLASS( - channel=self.channel, + channel=self, name=name, durable=durable, exclusive=exclusive, @@ -358,7 +375,7 @@ async def get_queue( return await self.declare_queue(name=name, passive=True) else: return self.QUEUE_CLASS( - channel=self.channel, + channel=self, name=name, durable=False, exclusive=False, @@ -379,7 +396,9 @@ async def set_qos( warn('Use "global_" instead of "all_channels"', DeprecationWarning) global_ = all_channels - return await self.channel.basic_qos( + channel = await self.get_underlay_channel() + + return await channel.basic_qos( prefetch_count=prefetch_count, prefetch_size=prefetch_size, global_=global_, @@ -394,7 +413,8 @@ async def queue_delete( if_empty: bool = False, nowait: bool = False, ) -> aiormq.spec.Queue.DeleteOk: - return await self.channel.queue_delete( + channel = await self.get_underlay_channel() + return await channel.queue_delete( queue=queue_name, if_unused=if_unused, if_empty=if_empty, @@ -409,7 +429,8 @@ async def exchange_delete( if_unused: bool = False, nowait: bool = False, ) -> aiormq.spec.Exchange.DeleteOk: - return await self.channel.exchange_delete( + channel = await self.get_underlay_channel() + return await channel.exchange_delete( exchange=exchange_name, if_unused=if_unused, nowait=nowait, @@ -426,7 +447,8 @@ def transaction(self) -> Transaction: return Transaction(self) async def flow(self, active: bool = True) -> aiormq.spec.Channel.FlowOk: - return await self.channel.flow(active=active) + channel = await self.get_underlay_channel() + return await channel.flow(active=active) __all__ = ("Channel",) diff --git a/aio_pika/connection.py b/aio_pika/connection.py index a088cc32..044f9a1d 100644 --- a/aio_pika/connection.py +++ b/aio_pika/connection.py @@ -4,7 +4,6 @@ from typing import Any, Callable, Dict, Optional, Tuple, Type, TypeVar, Union import aiormq.abc -from aiormq.tools import censor_url from pamqp.common import FieldTable from yarl import URL @@ -13,6 +12,7 @@ UnderlayConnection, ) from .channel import Channel +from .exceptions import ConnectionClosed from .log import get_logger from .tools import CallbackCollection @@ -34,7 +34,7 @@ def is_closed(self) -> bool: return self._closed async def close( - self, exc: Optional[aiormq.abc.ExceptionType] = asyncio.CancelledError, + self, exc: Optional[aiormq.abc.ExceptionType] = ConnectionClosed, ) -> None: transport, self.transport = self.transport, None self._close_called = True @@ -69,7 +69,10 @@ def __init__( self.connected: asyncio.Event = asyncio.Event() def __str__(self) -> str: - return str(censor_url(self.url)) + url = self.url + if url.password: + url = url.with_password("******") + return str(url) def __repr__(self) -> str: return f'<{self.__class__.__name__}: "{self}">' @@ -82,7 +85,7 @@ async def _on_connection_close(self, closing: asyncio.Future) -> None: self.connected.clear() await self.close_callbacks(exc) - async def _on_connected(self, transport: UnderlayConnection) -> None: + async def _on_connected(self) -> None: self.connected.set() async def connect(self, timeout: TimeoutType = None) -> None: @@ -94,12 +97,11 @@ async def connect(self, timeout: TimeoutType = None) -> None: You shouldn't call it explicitly. """ - transport = await UnderlayConnection.connect( + self.transport = await UnderlayConnection.connect( self.url, self._on_connection_close, timeout=timeout, **self.kwargs, ) - await self._on_connected(transport) - self.transport = transport + await self._on_connected() def channel( self, @@ -166,7 +168,7 @@ async def main(loop): log.debug("Creating AMQP channel for connection: %r", self) channel = self.CHANNEL_CLASS( - connection=self.transport.connection, + connection=self, channel_number=channel_number, publisher_confirms=publisher_confirms, on_return_raises=on_return_raises, diff --git a/aio_pika/exchange.py b/aio_pika/exchange.py index 9a7266c5..ff1e5963 100644 --- a/aio_pika/exchange.py +++ b/aio_pika/exchange.py @@ -4,8 +4,8 @@ from pamqp.common import Arguments from .abc import ( - AbstractExchange, AbstractMessage, ExchangeParamType, ExchangeType, - TimeoutType, get_exchange_name, + AbstractChannel, AbstractExchange, AbstractMessage, ExchangeParamType, + ExchangeType, TimeoutType, get_exchange_name, ) from .log import get_logger @@ -15,11 +15,11 @@ class Exchange(AbstractExchange): """ Exchange abstraction """ - channel: aiormq.abc.AbstractChannel + channel: AbstractChannel def __init__( self, - channel: aiormq.abc.AbstractChannel, + channel: AbstractChannel, name: str, type: Union[ExchangeType, str] = ExchangeType.DIRECT, *, @@ -52,7 +52,8 @@ def __repr__(self) -> str: async def declare( self, timeout: TimeoutType = None, ) -> aiormq.spec.Exchange.DeclareOk: - return await self.channel.exchange_declare( + channel = await self.channel.get_underlay_channel() + return await channel.exchange_declare( self.name, exchange_type=self._type, durable=self.durable, @@ -114,7 +115,8 @@ async def bind( arguments, ) - return await self.channel.exchange_bind( + channel = await self.channel.get_underlay_channel() + return await channel.exchange_bind( arguments=arguments, destination=self.name, routing_key=routing_key, @@ -149,7 +151,8 @@ async def unbind( arguments, ) - return await self.channel.exchange_unbind( + channel = await self.channel.get_underlay_channel() + return await channel.exchange_unbind( arguments=arguments, destination=self.name, routing_key=routing_key, @@ -187,7 +190,8 @@ async def publish( f"Can not publish to internal exchange: '{self.name}'!", ) - return await self.channel.basic_publish( + channel = await self.channel.get_underlay_channel() + return await channel.basic_publish( exchange=self.name, routing_key=routing_key, body=message.body, @@ -208,9 +212,12 @@ async def delete( """ log.info("Deleting %r", self) - return await self.channel.exchange_delete( + channel = await self.channel.get_underlay_channel() + result = await channel.exchange_delete( self.name, if_unused=if_unused, timeout=timeout, ) + del self.channel + return result __all__ = ("Exchange", "ExchangeType", "ExchangeParamType") diff --git a/aio_pika/message.py b/aio_pika/message.py index 6e4807a9..b4140752 100644 --- a/aio_pika/message.py +++ b/aio_pika/message.py @@ -15,7 +15,7 @@ AbstractMessage, AbstractProcessContext, DateType, DeliveryMode, HeadersType, MessageInfo, NoneType, ) -from .exceptions import MessageProcessError +from .exceptions import ChannelInvalidStateError, MessageProcessError from .log import get_logger @@ -410,6 +410,8 @@ def __init__(self, message: DeliveredMessage, no_ack: bool = False): @property def channel(self) -> aiormq.abc.AbstractChannel: + if self.__channel.is_closed: + raise ChannelInvalidStateError return self.__channel def process( @@ -471,7 +473,7 @@ async def ack(self, multiple: bool = False) -> None: raise MessageProcessError("Message already processed", self) if self.delivery_tag is not None: - await self.__channel.basic_ack( + await self.channel.basic_ack( delivery_tag=self.delivery_tag, multiple=multiple, ) @@ -482,7 +484,7 @@ async def ack(self, multiple: bool = False) -> None: async def reject(self, requeue: bool = False) -> None: """ When `requeue=True` the message will be returned to queue. - Otherwise message will be dropped. + Otherwise, message will be dropped. .. note:: This method looks like a blocking-method, but actually it just @@ -498,7 +500,7 @@ async def reject(self, requeue: bool = False) -> None: raise MessageProcessError("Message already processed", self) if self.delivery_tag is not None: - await self.__channel.basic_reject( + await self.channel.basic_reject( delivery_tag=self.delivery_tag, requeue=requeue, ) @@ -521,7 +523,7 @@ async def nack( raise MessageProcessError("Message already processed", self) if self.delivery_tag is not None: - await self.__channel.basic_nack( + await self.channel.basic_nack( delivery_tag=self.delivery_tag, multiple=multiple, requeue=requeue, diff --git a/aio_pika/patterns/base.py b/aio_pika/patterns/base.py index 7c2a8bda..31777d31 100644 --- a/aio_pika/patterns/base.py +++ b/aio_pika/patterns/base.py @@ -1,5 +1,9 @@ import pickle -from typing import Any, Callable +from typing import Any, Awaitable, Callable, TypeVar + + +T = TypeVar("T") +CallbackType = Callable[..., Awaitable[T]] class Method: diff --git a/aio_pika/patterns/master.py b/aio_pika/patterns/master.py index 0c79a718..5eac9e40 100644 --- a/aio_pika/patterns/master.py +++ b/aio_pika/patterns/master.py @@ -4,10 +4,9 @@ import logging from functools import partial from types import MappingProxyType -from typing import Any, Awaitable, Callable, Mapping, Optional, TypeVar +from typing import Any, Awaitable, Mapping, Optional import aiormq -from aiormq.tools import awaitable from aio_pika.abc import ( AbstractChannel, AbstractExchange, AbstractIncomingMessage, AbstractQueue, @@ -15,12 +14,11 @@ ) from aio_pika.message import Message, ReturnedMessage -from ..tools import create_task -from .base import Base, Proxy +from ..tools import create_task, ensure_awaitable +from .base import Base, CallbackType, Proxy, T log = logging.getLogger(__name__) -T = TypeVar("T") class MessageProcessingError(Exception): @@ -143,17 +141,18 @@ def deserialize(self, data: bytes) -> Any: @classmethod async def execute( - cls, func: Callable[..., Awaitable[T]], kwargs: Any, + cls, func: CallbackType, kwargs: Any, ) -> T: kwargs = kwargs or {} if not isinstance(kwargs, dict): + logging.error("Bad kwargs %r received for the %r", kwargs, func) raise RejectMessage(requeue=False) return await func(**kwargs) async def on_message( - self, func: Callable[..., Any], + self, func: CallbackType, message: AbstractIncomingMessage, ) -> None: async with message.process( @@ -174,17 +173,15 @@ async def create_queue( return await self.channel.declare_queue(queue_name, **kwargs) async def create_worker( - self, queue_name: str, func: Callable[..., Any], **kwargs: Any, + self, queue_name: str, + func: CallbackType, + **kwargs: Any, ) -> Worker: """ Creates a new :class:`Worker` instance. """ - queue = await self.create_queue(queue_name, **kwargs) - - if hasattr(func, "_is_coroutine"): - fn = func - else: - fn = awaitable(func) - consumer_tag = await queue.consume(partial(self.on_message, fn)) + consumer_tag = await queue.consume( + partial(self.on_message, ensure_awaitable(func)), + ) return Worker(queue, consumer_tag, self.loop) diff --git a/aio_pika/patterns/rpc.py b/aio_pika/patterns/rpc.py index c154a817..7b9171e9 100644 --- a/aio_pika/patterns/rpc.py +++ b/aio_pika/patterns/rpc.py @@ -5,10 +5,9 @@ import uuid from enum import Enum from functools import partial -from typing import Any, Callable, Dict, Optional, Tuple, TypeVar +from typing import Any, Callable, Dict, Optional, Tuple from aiormq.abc import ExceptionType -from aiormq.tools import awaitable from aio_pika.abc import ( AbstractChannel, AbstractExchange, AbstractIncomingMessage, AbstractQueue, @@ -18,14 +17,12 @@ from aio_pika.exchange import ExchangeType from aio_pika.message import IncomingMessage, Message, ReturnedMessage -from .base import Base, Proxy +from ..tools import ensure_awaitable +from .base import Base, CallbackType, Proxy, T log = logging.getLogger(__name__) -T = TypeVar("T") -CallbackType = Callable[..., T] - class RPCException(RuntimeError): pass @@ -390,6 +387,8 @@ async def register( arguments = kwargs.pop("arguments", {}) arguments.update({"x-dead-letter-exchange": self.DLX_NAME}) + func = ensure_awaitable(func) + kwargs["arguments"] = arguments queue = await self.channel.declare_queue(method_name, **kwargs) @@ -406,7 +405,7 @@ async def register( partial(self.on_call_message, method_name), ) - self.routes[method_name] = awaitable(func) + self.routes[method_name] = func self.queues[func] = queue async def unregister(self, func: CallbackType) -> None: @@ -457,7 +456,6 @@ async def deserialize_message( __all__ = ( - "CallbackType", "JsonRPC", "RPC", "RPCException", diff --git a/aio_pika/pool.py b/aio_pika/pool.py index 93fc4080..2a38472c 100644 --- a/aio_pika/pool.py +++ b/aio_pika/pool.py @@ -2,12 +2,10 @@ import asyncio from types import TracebackType from typing import ( - Any, AsyncContextManager, Awaitable, Callable, Coroutine, Generic, Optional, - Set, Tuple, Type, TypeVar, Union, + Any, AsyncContextManager, Awaitable, Callable, Generic, Optional, Set, + Tuple, Type, TypeVar, ) -from aiormq.tools import awaitable - from aio_pika.log import get_logger from aio_pika.tools import create_task @@ -24,11 +22,7 @@ def close(self) -> Awaitable[None]: T = TypeVar("T") ConstructorType = Callable[ ..., - Union[ - Awaitable[PoolInstance], - PoolInstance, - Coroutine[Any, Any, PoolInstance], - ], + Awaitable[PoolInstance], ] @@ -58,9 +52,7 @@ def __init__( ): self.loop = loop or asyncio.get_event_loop() self.__closed = False - self.__constructor: Callable[..., Awaitable[Any]] = awaitable( - constructor, - ) + self.__constructor: Callable[..., Awaitable[Any]] = constructor self.__constructor_args: Tuple[Any, ...] = args or () self.__created: int = 0 self.__item_set: Set[PoolInstance] = set() diff --git a/aio_pika/queue.py b/aio_pika/queue.py index 7bbe2894..05568ef6 100644 --- a/aio_pika/queue.py +++ b/aio_pika/queue.py @@ -2,21 +2,21 @@ import sys from functools import partial from types import TracebackType -from typing import Any, Callable, Optional, Type, overload +from typing import Any, Awaitable, Callable, Optional, Type, overload import aiormq from aiormq.abc import DeliveredMessage from pamqp.common import Arguments from .abc import ( - AbstractIncomingMessage, AbstractQueue, AbstractQueueIterator, ConsumerTag, - TimeoutType, get_exchange_name, + AbstractChannel, AbstractIncomingMessage, AbstractQueue, + AbstractQueueIterator, ConsumerTag, TimeoutType, get_exchange_name, ) from .exceptions import QueueEmpty from .exchange import ExchangeParamType from .log import get_logger from .message import IncomingMessage -from .tools import CallbackCollection, create_task +from .tools import CallbackCollection, create_task, ensure_awaitable if sys.version_info >= (3, 8): @@ -42,7 +42,7 @@ class Queue(AbstractQueue): def __init__( self, - channel: aiormq.abc.AbstractChannel, + channel: AbstractChannel, name: Optional[str], durable: bool, exclusive: bool, @@ -82,7 +82,8 @@ async def declare( :return: :class:`None` """ log.debug("Declaring queue: %r", self) - self.declaration_result = await self.channel.queue_declare( + channel = await self.channel.get_underlay_channel() + self.declaration_result = await channel.queue_declare( queue=self.name, durable=self.durable, exclusive=self.exclusive, @@ -135,7 +136,8 @@ async def bind( arguments, ) - return await self.channel.queue_bind( + channel = await self.channel.get_underlay_channel() + return await channel.queue_bind( self.name, exchange=get_exchange_name(exchange), routing_key=routing_key, @@ -173,7 +175,8 @@ async def unbind( arguments, ) - return await self.channel.queue_unbind( + channel = await self.channel.get_underlay_channel() + return await channel.queue_unbind( queue=self.name, exchange=get_exchange_name(exchange), routing_key=routing_key, @@ -183,7 +186,7 @@ async def unbind( async def consume( self, - callback: Callable[[AbstractIncomingMessage], Any], + callback: Callable[[AbstractIncomingMessage], Awaitable[Any]], no_ack: bool = False, exclusive: bool = False, arguments: Arguments = None, @@ -195,7 +198,7 @@ async def consume( :param timeout: :class:`asyncio.TimeoutError` will be raises when the Future was not finished after this time. - :param callback: Consuming callback. Could be a coroutine. + :param callback: Consuming callback. Should be a coroutine function. :param no_ack: if :class:`True` you don't need to call :func:`aio_pika.message.IncomingMessage.ack` @@ -214,8 +217,10 @@ async def consume( """ log.debug("Start to consuming queue: %r", self) + callback = ensure_awaitable(callback) - consume_result = await self.channel.basic_consume( + channel = await self.channel.get_underlay_channel() + consume_result = await channel.basic_consume( queue=self.name, consumer_callback=partial( consumer, @@ -259,7 +264,8 @@ async def cancel( :return: Basic.CancelOk when operation completed successfully """ - return await self.channel.basic_cancel( + channel = await self.channel.get_underlay_channel() + return await channel.basic_cancel( consumer_tag=consumer_tag, nowait=nowait, timeout=timeout, ) @@ -292,7 +298,8 @@ async def get( :return: :class:`aio_pika.message.IncomingMessage` """ - msg: DeliveredMessage = await self.channel.basic_get( + channel = await self.channel.get_underlay_channel() + msg: DeliveredMessage = await channel.basic_get( self.name, no_ack=no_ack, timeout=timeout, ) @@ -315,7 +322,8 @@ async def purge( log.info("Purging queue: %r", self) - return await self.channel.queue_purge( + channel = await self.channel.get_underlay_channel() + return await channel.queue_purge( self.name, nowait=no_wait, timeout=timeout, ) @@ -334,7 +342,8 @@ async def delete( log.info("Deleting %r", self) - return await self.channel.queue_delete( + channel = await self.channel.get_underlay_channel() + return await channel.queue_delete( self.name, if_unused=if_unused, if_empty=if_empty, diff --git a/aio_pika/robust_channel.py b/aio_pika/robust_channel.py index a357143e..8ca182e0 100644 --- a/aio_pika/robust_channel.py +++ b/aio_pika/robust_channel.py @@ -1,14 +1,16 @@ import asyncio +import warnings from collections import defaultdict from itertools import chain -from typing import Any, DefaultDict, Dict, Optional, Set, Type, Union +from typing import Any, DefaultDict, Dict, MutableSet, Optional, Type, Union from warnings import warn +from weakref import WeakSet import aiormq from .abc import ( - AbstractRobustChannel, AbstractRobustExchange, AbstractRobustQueue, - TimeoutType, + AbstractConnection, AbstractRobustChannel, AbstractRobustExchange, + AbstractRobustQueue, TimeoutType, ) from .channel import Channel from .exchange import Exchange, ExchangeType @@ -28,13 +30,15 @@ class RobustChannel(Channel, AbstractRobustChannel): # type: ignore QUEUE_CLASS: Type[Queue] = RobustQueue EXCHANGE_CLASS: Type[Exchange] = RobustExchange - _exchanges: DefaultDict[str, Set[AbstractRobustExchange]] - _queues: DefaultDict[str, Set[RobustQueue]] + RESTORE_RETRY_DELAY: int = 2 + + _exchanges: DefaultDict[str, MutableSet[AbstractRobustExchange]] + _queues: DefaultDict[str, MutableSet[RobustQueue]] default_exchange: RobustExchange def __init__( self, - connection: aiormq.abc.AbstractConnection, + connection: AbstractConnection, channel_number: Optional[int] = None, publisher_confirms: bool = True, on_return_raises: bool = False, @@ -57,37 +61,68 @@ def __init__( on_return_raises=on_return_raises, ) - self._exchanges = defaultdict(set) - self._queues = defaultdict(set) + self._exchanges = defaultdict(WeakSet) + self._queues = defaultdict(WeakSet) self._prefetch_count: int = 0 self._prefetch_size: int = 0 self._global_qos: bool = False self.reopen_callbacks: CallbackCollection = CallbackCollection(self) - self.close_callbacks.add(self.__close_callback) self.__restore_lock = asyncio.Lock() + self.__ready = asyncio.Event() + self.__restored = True + self.close_callbacks.add(self.__close_callback) - async def __close_callback(self, *_: Any) -> None: - if self._closed or self._connection.is_closed: - self.close_callbacks.discard(self.__close_callback) - return + async def ready(self) -> None: + while not self.__restored: + await self.__ready.wait() - await self.reopen() + async def get_underlay_channel(self) -> aiormq.abc.AbstractChannel: + await self._connection.ready() + return await super().get_underlay_channel() + + async def restore(self, channel: Any = None) -> None: + if channel is not None: + warnings.warn( + "Channel argument will be ignored because you " + "don't need to pass this anymore.", + DeprecationWarning, + ) - async def restore(self, connection: aiormq.abc.AbstractConnection) -> None: async with self.__restore_lock: - self._connection = connection + if self.__restored: + return + await self.reopen() + self.__restored = True + + async def __close_callback(self, _: Any, exc: BaseException) -> None: + if isinstance(exc, asyncio.CancelledError): + # This happens only if the channel is forced to close from the + # outside, for example, if the connection is closed. + # Of course, here you need to exit from this function + # as soon as possible and to avoid a recovery attempt. + return + + in_restore_state = not self.__restored + self.__restored = False + self.__ready.clear() + + if self._closed or in_restore_state: + return + + await self.restore() async def _open(self) -> None: await super()._open() await self.reopen_callbacks() - async def _on_open(self, channel: aiormq.abc.AbstractChannel) -> None: + async def _on_open(self) -> None: if not hasattr(self, "default_exchange"): - await super()._on_open(channel) + await super()._on_open() exchanges = tuple(chain(*self._exchanges.values())) queues = tuple(chain(*self._queues.values())) + channel = await self.get_underlay_channel() await channel.basic_qos( prefetch_count=self._prefetch_count, @@ -95,13 +130,15 @@ async def _on_open(self, channel: aiormq.abc.AbstractChannel) -> None: ) for exchange in exchanges: - await exchange.restore(channel) + await exchange.restore() for queue in queues: - await queue.restore(channel) + await queue.restore() if hasattr(self, "default_exchange"): - self.default_exchange.channel = channel + self.default_exchange.channel = self + + self.__ready.set() async def set_qos( self, @@ -141,6 +178,7 @@ async def declare_exchange( robust: bool = True, ) -> AbstractRobustExchange: await self._connection.ready() + await self.ready() exchange = ( await super().declare_exchange( name=name, @@ -168,6 +206,7 @@ async def exchange_delete( nowait: bool = False, ) -> aiormq.spec.Exchange.DeleteOk: await self._connection.ready() + await self.ready() result = await super().exchange_delete( exchange_name=exchange_name, timeout=timeout, @@ -190,6 +229,7 @@ async def declare_queue( robust: bool = True, ) -> AbstractRobustQueue: await self._connection.ready() + await self.ready() queue: RobustQueue = await super().declare_queue( # type: ignore name=name, durable=durable, @@ -212,6 +252,7 @@ async def queue_delete( nowait: bool = False, ) -> aiormq.spec.Queue.DeleteOk: await self._connection.ready() + await self.ready() result = await super().queue_delete( queue_name=queue_name, timeout=timeout, diff --git a/aio_pika/robust_connection.py b/aio_pika/robust_connection.py index a7c2c0f7..09db1699 100644 --- a/aio_pika/robust_connection.py +++ b/aio_pika/robust_connection.py @@ -10,7 +10,6 @@ from .abc import ( AbstractRobustChannel, AbstractRobustConnection, SSLOptions, TimeoutType, - UnderlayConnection, ) from .connection import Connection, make_url from .exceptions import CONNECTION_EXCEPTIONS @@ -81,11 +80,17 @@ async def _on_connection_close(self, closing: asyncio.Future) -> None: self.__connection_close_event.set() - async def _on_connected(self, transport: UnderlayConnection) -> None: + async def _on_connected(self) -> None: + await super()._on_connected() + + transport = self.transport + if transport is None: + raise RuntimeError("No active transport for connection %r", self) + try: for channel in self.__channels: try: - await channel.restore(transport.connection) + await channel.restore() except Exception: log.exception("Failed to reopen channel") raise @@ -98,12 +103,11 @@ async def _on_connected(self, transport: UnderlayConnection) -> None: closing.set_exception(e) await self.close_callbacks(closing) await asyncio.gather( - transport.connection.close(e), return_exceptions=True, + transport.connection.close(e), + return_exceptions=True, ) raise - await super()._on_connected(transport) - if self.connection_attempt: await self.reconnect_callbacks() @@ -205,6 +209,7 @@ async def close( self.__reconnection_task, return_exceptions=True, ) self.__reconnection_task = None + return await super().close(exc) @@ -320,7 +325,7 @@ async def main(): client_properties=client_properties, **kwargs, ), - loop=loop, ssl_context=ssl_context, **kwargs + loop=loop, ssl_context=ssl_context, **kwargs, ) await connection.connect(timeout=timeout) diff --git a/aio_pika/robust_exchange.py b/aio_pika/robust_exchange.py index 042bbf9c..18f86e74 100644 --- a/aio_pika/robust_exchange.py +++ b/aio_pika/robust_exchange.py @@ -1,11 +1,13 @@ import asyncio +import warnings from typing import Any, Dict, Union import aiormq from pamqp.common import Arguments from .abc import ( - AbstractExchange, AbstractRobustExchange, ExchangeParamType, TimeoutType, + AbstractChannel, AbstractExchange, AbstractRobustExchange, + ExchangeParamType, TimeoutType, ) from .exchange import Exchange, ExchangeType from .log import get_logger @@ -21,7 +23,7 @@ class RobustExchange(Exchange, AbstractRobustExchange): def __init__( self, - channel: aiormq.abc.AbstractChannel, + channel: AbstractChannel, name: str, type: Union[ExchangeType, str] = ExchangeType.DIRECT, *, @@ -45,11 +47,15 @@ def __init__( self._bindings = {} self.__restore_lock = asyncio.Lock() - async def restore(self, channel: aiormq.abc.AbstractChannel) -> None: + async def restore(self, channel: Any = None) -> None: + if channel is not None: + warnings.warn( + "Channel argument will be ignored because you " + "don't need to pass this anymore.", + DeprecationWarning, + ) async with self.__restore_lock: try: - self.channel = channel - # special case for default exchange if self.name == "": return @@ -59,7 +65,6 @@ async def restore(self, channel: aiormq.abc.AbstractChannel) -> None: for exchange, kwargs in tuple(self._bindings.items()): await self.bind(exchange, **kwargs) except Exception: - del self.channel raise async def bind( diff --git a/aio_pika/robust_queue.py b/aio_pika/robust_queue.py index 6929239e..554751a6 100644 --- a/aio_pika/robust_queue.py +++ b/aio_pika/robust_queue.py @@ -1,13 +1,14 @@ +import warnings from random import Random -from typing import Any, Callable, Dict, Optional, Tuple, Union +from typing import Any, Awaitable, Callable, Dict, Optional, Tuple, Union import aiormq from aiormq import ChannelInvalidStateError from pamqp.common import Arguments from .abc import ( - AbstractExchange, AbstractIncomingMessage, AbstractQueueIterator, - AbstractRobustQueue, ConsumerTag, TimeoutType, + AbstractChannel, AbstractExchange, AbstractIncomingMessage, + AbstractQueueIterator, AbstractRobustQueue, ConsumerTag, TimeoutType, ) from .exchange import ExchangeParamType from .log import get_logger @@ -32,7 +33,7 @@ def _get_random_queue_name(cls) -> str: def __init__( self, - channel: aiormq.abc.AbstractChannel, + channel: AbstractChannel, name: Optional[str], durable: bool = False, exclusive: bool = False, @@ -54,8 +55,14 @@ def __init__( self._consumers = {} self._bindings = {} - async def restore(self, channel: aiormq.abc.AbstractChannel) -> None: - self.channel = channel + async def restore(self, channel: Any = None) -> None: + if channel is not None: + warnings.warn( + "Channel argument will be ignored because you " + "don't need to pass this anymore.", + DeprecationWarning, + ) + await self.declare() bindings = tuple(self._bindings.items()) consumers = tuple(self._consumers.items()) @@ -109,7 +116,7 @@ async def unbind( async def consume( self, - callback: Callable[[AbstractIncomingMessage], Any], + callback: Callable[[AbstractIncomingMessage], Awaitable[Any]], no_ack: bool = False, exclusive: bool = False, arguments: Arguments = None, @@ -156,7 +163,7 @@ async def consume(self) -> None: try: return await super().consume() except ChannelInvalidStateError: - await self._amqp_queue.channel.connection.ready() + await self._amqp_queue.channel.get_underlay_channel() __all__ = ("RobustQueue",) diff --git a/aio_pika/tools.py b/aio_pika/tools.py index 9ff77c37..daec83ea 100644 --- a/aio_pika/tools.py +++ b/aio_pika/tools.py @@ -1,4 +1,7 @@ import asyncio +import inspect +import warnings +from functools import wraps from itertools import chain from threading import Lock from typing import ( @@ -251,11 +254,48 @@ def __call__(self, *args: Any, **kwargs: Any) -> Awaitable[Any]: return self.__task +def ensure_awaitable( + func: Callable[..., Union[T, Awaitable[T]]], +) -> Callable[..., Awaitable[T]]: + if inspect.iscoroutinefunction(func): + return func + + if inspect.isfunction(func) and not iscoroutinepartial(func): + warnings.warn( + f"You probably registering the non-coroutine function {func!r}. " + "This is deprecated and will be removed in future releases. " + "Moreover, it can block the event loop", + DeprecationWarning, + ) + + @wraps(func) + async def wrapper(*args: Any, **kwargs: Any) -> T: + nonlocal func + + result = func(*args, **kwargs) + if not hasattr(result, "__await__"): + warnings.warn( + f"Function {func!r} returned a non awaitable result." + "This may be bad for performance or may blocks the " + "event loop, you should pay attention to this. This " + "warning is here in an attempt to maintain backwards " + "compatibility and will simply be removed in " + "future releases.", + DeprecationWarning, + ) + return result + + return await result + + return wrapper + + __all__ = ( "CallbackCollection", - "CallbackType", "CallbackSetType", + "CallbackType", "OneShotCallback", "create_task", + "ensure_awaitable", "iscoroutinepartial", ) diff --git a/aio_pika/transaction.py b/aio_pika/transaction.py index a324275b..661a8ce1 100644 --- a/aio_pika/transaction.py +++ b/aio_pika/transaction.py @@ -33,7 +33,8 @@ def channel(self) -> AbstractChannel: async def select( self, timeout: TimeoutType = None, ) -> aiormq.spec.Tx.SelectOk: - result = await self.channel.channel.tx_select(timeout=timeout) + channel = await self.channel.get_underlay_channel() + result = await channel.tx_select(timeout=timeout) self.state = TransactionState.STARTED return result @@ -41,14 +42,16 @@ async def select( async def rollback( self, timeout: TimeoutType = None, ) -> commands.Tx.RollbackOk: - result = await self.channel.channel.tx_rollback(timeout=timeout) + channel = await self.channel.get_underlay_channel() + result = await channel.tx_rollback(timeout=timeout) self.state = TransactionState.ROLLED_BACK return result async def commit( self, timeout: TimeoutType = None, ) -> commands.Tx.CommitOk: - result = await self.channel.channel.tx_commit(timeout=timeout) + channel = await self.channel.get_underlay_channel() + result = await channel.tx_commit(timeout=timeout) self.state = TransactionState.COMMITED return result diff --git a/docs/source/rabbitmq-tutorial/examples/6-rpc/rpc_client.py b/docs/source/rabbitmq-tutorial/examples/6-rpc/rpc_client.py index a04d01d7..d6dc2fca 100644 --- a/docs/source/rabbitmq-tutorial/examples/6-rpc/rpc_client.py +++ b/docs/source/rabbitmq-tutorial/examples/6-rpc/rpc_client.py @@ -28,7 +28,7 @@ async def connect(self) -> "FibonacciRpcClient": return self - def on_response(self, message: AbstractIncomingMessage) -> None: + async def on_response(self, message: AbstractIncomingMessage) -> None: if message.correlation_id is None: print(f"Bad message {message!r}") return diff --git a/poetry.lock b/poetry.lock index d6550d4d..5df26e2e 100644 --- a/poetry.lock +++ b/poetry.lock @@ -73,14 +73,14 @@ files = [ [[package]] name = "argcomplete" -version = "2.1.1" +version = "2.1.2" description = "Bash tab completion for argparse" category = "dev" optional = false python-versions = ">=3.6" files = [ - {file = "argcomplete-2.1.1-py3-none-any.whl", hash = "sha256:17041f55b8c45099428df6ce6d0d282b892471a78c71375d24f227e21c13f8c5"}, - {file = "argcomplete-2.1.1.tar.gz", hash = "sha256:72e08340852d32544459c0c19aad1b48aa2c3a96de8c6e5742456b4f538ca52f"}, + {file = "argcomplete-2.1.2-py3-none-any.whl", hash = "sha256:4ba9cdaa28c361d251edce884cd50b4b1215d65cdc881bd204426cdde9f52731"}, + {file = "argcomplete-2.1.2.tar.gz", hash = "sha256:fc82ef070c607b1559b5c720529d63b54d9dcf2dcfc2632b10e6372314a34457"}, ] [package.dependencies] @@ -90,25 +90,6 @@ importlib-metadata = {version = ">=0.23,<6", markers = "python_version == \"3.7\ lint = ["flake8", "mypy"] test = ["coverage", "flake8", "mypy", "pexpect", "wheel"] -[[package]] -name = "attrs" -version = "22.2.0" -description = "Classes Without Boilerplate" -category = "dev" -optional = false -python-versions = ">=3.6" -files = [ - {file = "attrs-22.2.0-py3-none-any.whl", hash = "sha256:29e95c7f6778868dbd49170f98f8818f78f3dc5e0e37c0b1f474e3561b240836"}, - {file = "attrs-22.2.0.tar.gz", hash = "sha256:c9227bfc2f01993c03f68db37d1d15c9690188323c067c641f1a35ca58185f99"}, -] - -[package.extras] -cov = ["attrs[tests]", "coverage-enable-subprocess", "coverage[toml] (>=5.3)"] -dev = ["attrs[docs,tests]"] -docs = ["furo", "myst-parser", "sphinx", "sphinx-notfound-page", "sphinxcontrib-towncrier", "towncrier", "zope.interface"] -tests = ["attrs[tests-no-zope]", "zope.interface"] -tests-no-zope = ["cloudpickle", "cloudpickle", "hypothesis", "hypothesis", "mypy (>=0.971,<0.990)", "mypy (>=0.971,<0.990)", "pympler", "pympler", "pytest (>=4.3.0)", "pytest (>=4.3.0)", "pytest-mypy-plugins", "pytest-mypy-plugins", "pytest-xdist[psutil]", "pytest-xdist[psutil]"] - [[package]] name = "babel" version = "2.12.1" @@ -126,14 +107,14 @@ pytz = {version = ">=2015.7", markers = "python_version < \"3.9\""} [[package]] name = "certifi" -version = "2022.12.7" +version = "2023.5.7" description = "Python package for providing Mozilla's CA Bundle." category = "dev" optional = false python-versions = ">=3.6" files = [ - {file = "certifi-2022.12.7-py3-none-any.whl", hash = "sha256:4ad3232f5e926d6718ec31cfc1fcadfde020920e278684144551c91769c7bc18"}, - {file = "certifi-2022.12.7.tar.gz", hash = "sha256:35824b4c3a97115964b408844d64aa14db1cc518f6562e8d7261699d1350a9e3"}, + {file = "certifi-2023.5.7-py3-none-any.whl", hash = "sha256:c6c2e98f5c7869efca1f8916fed228dd91539f9f1b444c314c06eef02980c716"}, + {file = "certifi-2023.5.7.tar.gz", hash = "sha256:0f0d56dc5a6ad56fd4ba36484d6cc34451e1c6548c61daad8c320169f91eddc7"}, ] [[package]] @@ -388,14 +369,14 @@ files = [ [[package]] name = "exceptiongroup" -version = "1.1.0" +version = "1.1.1" description = "Backport of PEP 654 (exception groups)" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "exceptiongroup-1.1.0-py3-none-any.whl", hash = "sha256:327cbda3da756e2de031a3107b81ab7b3770a602c4d16ca618298c526f4bec1e"}, - {file = "exceptiongroup-1.1.0.tar.gz", hash = "sha256:bcb67d800a4497e1b404c2dd44fca47d3b7a5e5433dbab67f96c1a685cdfdf23"}, + {file = "exceptiongroup-1.1.1-py3-none-any.whl", hash = "sha256:232c37c63e4f682982c8b6459f33a8981039e5fb8756b2074364e5055c498c9e"}, + {file = "exceptiongroup-1.1.1.tar.gz", hash = "sha256:d484c3090ba2889ae2928419117447a14daf3c1231d5e30d0aae34f354f01785"}, ] [package.extras] @@ -403,19 +384,19 @@ test = ["pytest (>=6)"] [[package]] name = "filelock" -version = "3.9.0" +version = "3.12.0" description = "A platform independent file lock." category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "filelock-3.9.0-py3-none-any.whl", hash = "sha256:f58d535af89bb9ad5cd4df046f741f8553a418c01a7856bf0d173bbc9f6bd16d"}, - {file = "filelock-3.9.0.tar.gz", hash = "sha256:7b319f24340b51f55a2bf7a12ac0755a9b03e718311dac567a0f4f7fabd2f5de"}, + {file = "filelock-3.12.0-py3-none-any.whl", hash = "sha256:ad98852315c2ab702aeb628412cbf7e95b7ce8c3bf9565670b4eaecf1db370a9"}, + {file = "filelock-3.12.0.tar.gz", hash = "sha256:fc03ae43288c013d2ea83c8597001b1129db351aad9c57fe2409327916b8e718"}, ] [package.extras] -docs = ["furo (>=2022.12.7)", "sphinx (>=5.3)", "sphinx-autodoc-typehints (>=1.19.5)"] -testing = ["covdefaults (>=2.2.2)", "coverage (>=7.0.1)", "pytest (>=7.2)", "pytest-cov (>=4)", "pytest-timeout (>=2.1)"] +docs = ["furo (>=2023.3.27)", "sphinx (>=6.1.3)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] +testing = ["covdefaults (>=2.3)", "coverage (>=7.2.3)", "diff-cover (>=7.5)", "pytest (>=7.3.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)", "pytest-timeout (>=2.1)"] [[package]] name = "idna" @@ -765,14 +746,14 @@ tox-to-nox = ["jinja2", "tox"] [[package]] name = "packaging" -version = "23.0" +version = "23.1" description = "Core utilities for Python packages" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "packaging-23.0-py3-none-any.whl", hash = "sha256:714ac14496c3e68c99c29b00845f7a2b85f3bb6f1078fd9f72fd20f0570002b2"}, - {file = "packaging-23.0.tar.gz", hash = "sha256:b6ad297f8907de0fa2fe1ccbd26fdaf387f5f47c7275fedf8cce89f99446cf97"}, + {file = "packaging-23.1-py3-none-any.whl", hash = "sha256:994793af429502c4ea2ebf6bf664629d07c1a9fe974af92966e4b8d2df7edc61"}, + {file = "packaging-23.1.tar.gz", hash = "sha256:a392980d2b6cffa644431898be54b0045151319d1e7ec34f0cfed48767dd334f"}, ] [[package]] @@ -793,22 +774,22 @@ testing = ["coverage", "flake8", "flake8-comprehensions", "flake8-deprecated", " [[package]] name = "platformdirs" -version = "3.1.1" +version = "3.5.0" description = "A small Python package for determining appropriate platform-specific dirs, e.g. a \"user data dir\"." category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "platformdirs-3.1.1-py3-none-any.whl", hash = "sha256:e5986afb596e4bb5bde29a79ac9061aa955b94fca2399b7aaac4090860920dd8"}, - {file = "platformdirs-3.1.1.tar.gz", hash = "sha256:024996549ee88ec1a9aa99ff7f8fc819bb59e2c3477b410d90a16d32d6e707aa"}, + {file = "platformdirs-3.5.0-py3-none-any.whl", hash = "sha256:47692bc24c1958e8b0f13dd727307cff1db103fca36399f457da8e05f222fdc4"}, + {file = "platformdirs-3.5.0.tar.gz", hash = "sha256:7954a68d0ba23558d753f73437c55f89027cf8f5108c19844d4b82e5af396335"}, ] [package.dependencies] -typing-extensions = {version = ">=4.4", markers = "python_version < \"3.8\""} +typing-extensions = {version = ">=4.5", markers = "python_version < \"3.8\""} [package.extras] -docs = ["furo (>=2022.12.7)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-autodoc-typehints (>=1.22,!=1.23.4)"] -test = ["appdirs (==1.4.4)", "covdefaults (>=2.2.2)", "pytest (>=7.2.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)"] +docs = ["furo (>=2023.3.27)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] +test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=7.3.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)"] [[package]] name = "pluggy" @@ -886,14 +867,14 @@ files = [ [[package]] name = "pygments" -version = "2.14.0" +version = "2.15.1" description = "Pygments is a syntax highlighting package written in Python." category = "dev" optional = false -python-versions = ">=3.6" +python-versions = ">=3.7" files = [ - {file = "Pygments-2.14.0-py3-none-any.whl", hash = "sha256:fa7bd7bd2771287c0de303af8bfdfc731f51bd2c6a47ab69d117138893b82717"}, - {file = "Pygments-2.14.0.tar.gz", hash = "sha256:b3ed06a9e8ac9a9aae5a6f5dbe78a8a58655d17b43b93c078f094ddc476ae297"}, + {file = "Pygments-2.15.1-py3-none-any.whl", hash = "sha256:db2db3deb4b4179f399a09054b023b6a586b76499d36965813c71aa8ed7b5fd1"}, + {file = "Pygments-2.15.1.tar.gz", hash = "sha256:8ace4d3c1dd481894b2005f560ead0f9f19ee64fe983366be1a21e171d12775c"}, ] [package.extras] @@ -929,18 +910,17 @@ vulture = ["vulture"] [[package]] name = "pytest" -version = "7.2.2" +version = "7.3.1" description = "pytest: simple powerful testing with Python" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "pytest-7.2.2-py3-none-any.whl", hash = "sha256:130328f552dcfac0b1cec75c12e3f005619dc5f874f0a06e8ff7263f0ee6225e"}, - {file = "pytest-7.2.2.tar.gz", hash = "sha256:c99ab0c73aceb050f68929bc93af19ab6db0558791c6a0715723abe9d0ade9d4"}, + {file = "pytest-7.3.1-py3-none-any.whl", hash = "sha256:3799fa815351fea3a5e96ac7e503a96fa51cc9942c3753cda7651b93c1cfa362"}, + {file = "pytest-7.3.1.tar.gz", hash = "sha256:434afafd78b1d78ed0addf160ad2b77a30d35d4bdf8af234fe621919d9ed15e3"}, ] [package.dependencies] -attrs = ">=19.2.0" colorama = {version = "*", markers = "sys_platform == \"win32\""} exceptiongroup = {version = ">=1.0.0rc8", markers = "python_version < \"3.11\""} importlib-metadata = {version = ">=0.12", markers = "python_version < \"3.8\""} @@ -950,7 +930,7 @@ pluggy = ">=0.12,<2.0" tomli = {version = ">=1.0.0", markers = "python_version < \"3.11\""} [package.extras] -testing = ["argcomplete", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "xmlschema"] +testing = ["argcomplete", "attrs (>=19.2.0)", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "xmlschema"] [[package]] name = "pytest-cov" @@ -990,14 +970,14 @@ pytest = "*" [[package]] name = "pytz" -version = "2022.7.1" +version = "2023.3" description = "World timezone definitions, modern and historical" category = "dev" optional = false python-versions = "*" files = [ - {file = "pytz-2022.7.1-py2.py3-none-any.whl", hash = "sha256:78f4f37d8198e0627c5f1143240bb0206b8691d8d7ac6d78fee88b78733f8c4a"}, - {file = "pytz-2022.7.1.tar.gz", hash = "sha256:01a0681c4b9684a28304615eba55d1ab31ae00bf68ec157ec3708a8182dbbcd0"}, + {file = "pytz-2023.3-py2.py3-none-any.whl", hash = "sha256:a151b3abb88eda1d4e34a9814df37de2a80e301e68ba0fd856fb9b46bfbbbffb"}, + {file = "pytz-2023.3.tar.gz", hash = "sha256:1d8ce29db189191fb55338ee6d0387d82ab59f3d00eac103412d64e0ebd0c588"}, ] [[package]] @@ -1024,14 +1004,14 @@ use-chardet-on-py3 = ["chardet (>=3.0.2,<6)"] [[package]] name = "setuptools" -version = "67.6.0" +version = "67.7.2" description = "Easily download, build, install, upgrade, and uninstall Python packages" category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "setuptools-67.6.0-py3-none-any.whl", hash = "sha256:b78aaa36f6b90a074c1fa651168723acbf45d14cb1196b6f02c0fd07f17623b2"}, - {file = "setuptools-67.6.0.tar.gz", hash = "sha256:2ee892cd5f29f3373097f5a814697e397cf3ce313616df0af11231e2ad118077"}, + {file = "setuptools-67.7.2-py3-none-any.whl", hash = "sha256:23aaf86b85ca52ceb801d32703f12d77517b2556af839621c641fca11287952b"}, + {file = "setuptools-67.7.2.tar.gz", hash = "sha256:f104fa03692a2602fa0fec6c6a9e63b6c8a968de13e17c026957dd1f53d80990"}, ] [package.extras] @@ -1306,14 +1286,14 @@ files = [ [[package]] name = "types-docutils" -version = "0.19.1.6" +version = "0.20.0.1" description = "Typing stubs for docutils" category = "dev" optional = false python-versions = "*" files = [ - {file = "types-docutils-0.19.1.6.tar.gz", hash = "sha256:a334a703a8688910d0869464f1f6f8bd330d75843bdab4f893547bfb29417a01"}, - {file = "types_docutils-0.19.1.6-py3-none-any.whl", hash = "sha256:b7e182f371cbe20828a8750ffb150219968452445946f66f33778b08e81c7d89"}, + {file = "types-docutils-0.20.0.1.tar.gz", hash = "sha256:f682b5459a1e6e28208742adb0be8573d1ecbddd442f00d202b0278c1c4418a2"}, + {file = "types_docutils-0.20.0.1-py3-none-any.whl", hash = "sha256:6b17cbe57cb282158feb41d154cddaeabc16f1d6cff3c7308bd3056f42aa7cd2"}, ] [[package]] @@ -1345,20 +1325,21 @@ files = [ [[package]] name = "urllib3" -version = "1.26.15" +version = "2.0.2" description = "HTTP library with thread-safe connection pooling, file post, and more." category = "dev" optional = false -python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*" +python-versions = ">=3.7" files = [ - {file = "urllib3-1.26.15-py2.py3-none-any.whl", hash = "sha256:aa751d169e23c7479ce47a0cb0da579e3ede798f994f5816a74e4f4500dcea42"}, - {file = "urllib3-1.26.15.tar.gz", hash = "sha256:8a388717b9476f934a21484e8c8e61875ab60644d29b9b39e11e4b9dc1c6b305"}, + {file = "urllib3-2.0.2-py3-none-any.whl", hash = "sha256:d055c2f9d38dc53c808f6fdc8eab7360b6fdbbde02340ed25cfbcd817c62469e"}, + {file = "urllib3-2.0.2.tar.gz", hash = "sha256:61717a1095d7e155cdb737ac7bb2f4324a858a1e2e6466f6d03ff630ca68d3cc"}, ] [package.extras] -brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)", "brotlipy (>=0.6.0)"] -secure = ["certifi", "cryptography (>=1.3.4)", "idna (>=2.0.0)", "ipaddress", "pyOpenSSL (>=0.14)", "urllib3-secure-extra"] -socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"] +brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)"] +secure = ["certifi", "cryptography (>=1.9)", "idna (>=2.0.0)", "pyopenssl (>=17.1.0)", "urllib3-secure-extra"] +socks = ["pysocks (>=1.5.6,!=1.5.7,<2.0)"] +zstd = ["zstandard (>=0.18.0)"] [[package]] name = "uvloop" @@ -1407,14 +1388,14 @@ test = ["Cython (>=0.29.32,<0.30.0)", "aiohttp", "flake8 (>=3.9.2,<3.10.0)", "my [[package]] name = "virtualenv" -version = "20.20.0" +version = "20.21.1" description = "Virtual Python Environment builder" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "virtualenv-20.20.0-py3-none-any.whl", hash = "sha256:3c22fa5a7c7aa106ced59934d2c20a2ecb7f49b4130b8bf444178a16b880fa45"}, - {file = "virtualenv-20.20.0.tar.gz", hash = "sha256:a8a4b8ca1e28f864b7514a253f98c1d62b64e31e77325ba279248c65fb4fcef4"}, + {file = "virtualenv-20.21.1-py3-none-any.whl", hash = "sha256:09ddbe1af0c8ed2bb4d6ed226b9e6415718ad18aef9fa0ba023d96b7a8356049"}, + {file = "virtualenv-20.21.1.tar.gz", hash = "sha256:4c104ccde994f8b108163cf9ba58f3d11511d9403de87fb9b4f52bf33dbc8668"}, ] [package.dependencies] @@ -1424,91 +1405,91 @@ importlib-metadata = {version = ">=4.8.3", markers = "python_version < \"3.8\""} platformdirs = ">=2.4,<4" [package.extras] -docs = ["furo (>=2022.12.7)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=22.12)"] -test = ["covdefaults (>=2.2.2)", "coverage (>=7.1)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23)", "pytest (>=7.2.1)", "pytest-env (>=0.8.1)", "pytest-freezegun (>=0.4.2)", "pytest-mock (>=3.10)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)"] +docs = ["furo (>=2023.3.27)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=22.12)"] +test = ["covdefaults (>=2.3)", "coverage (>=7.2.3)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.3.1)", "pytest-env (>=0.8.1)", "pytest-freezegun (>=0.4.2)", "pytest-mock (>=3.10)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)"] [[package]] name = "yarl" -version = "1.8.2" +version = "1.9.2" description = "Yet another URL library" category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "yarl-1.8.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:bb81f753c815f6b8e2ddd2eef3c855cf7da193b82396ac013c661aaa6cc6b0a5"}, - {file = "yarl-1.8.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:47d49ac96156f0928f002e2424299b2c91d9db73e08c4cd6742923a086f1c863"}, - {file = "yarl-1.8.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:3fc056e35fa6fba63248d93ff6e672c096f95f7836938241ebc8260e062832fe"}, - {file = "yarl-1.8.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:58a3c13d1c3005dbbac5c9f0d3210b60220a65a999b1833aa46bd6677c69b08e"}, - {file = "yarl-1.8.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:10b08293cda921157f1e7c2790999d903b3fd28cd5c208cf8826b3b508026996"}, - {file = "yarl-1.8.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:de986979bbd87272fe557e0a8fcb66fd40ae2ddfe28a8b1ce4eae22681728fef"}, - {file = "yarl-1.8.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c4fcfa71e2c6a3cb568cf81aadc12768b9995323186a10827beccf5fa23d4f8"}, - {file = "yarl-1.8.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ae4d7ff1049f36accde9e1ef7301912a751e5bae0a9d142459646114c70ecba6"}, - {file = "yarl-1.8.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:bf071f797aec5b96abfc735ab97da9fd8f8768b43ce2abd85356a3127909d146"}, - {file = "yarl-1.8.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:74dece2bfc60f0f70907c34b857ee98f2c6dd0f75185db133770cd67300d505f"}, - {file = "yarl-1.8.2-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:df60a94d332158b444301c7f569659c926168e4d4aad2cfbf4bce0e8fb8be826"}, - {file = "yarl-1.8.2-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:63243b21c6e28ec2375f932a10ce7eda65139b5b854c0f6b82ed945ba526bff3"}, - {file = "yarl-1.8.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:cfa2bbca929aa742b5084fd4663dd4b87c191c844326fcb21c3afd2d11497f80"}, - {file = "yarl-1.8.2-cp310-cp310-win32.whl", hash = "sha256:b05df9ea7496df11b710081bd90ecc3a3db6adb4fee36f6a411e7bc91a18aa42"}, - {file = "yarl-1.8.2-cp310-cp310-win_amd64.whl", hash = "sha256:24ad1d10c9db1953291f56b5fe76203977f1ed05f82d09ec97acb623a7976574"}, - {file = "yarl-1.8.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:2a1fca9588f360036242f379bfea2b8b44cae2721859b1c56d033adfd5893634"}, - {file = "yarl-1.8.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f37db05c6051eff17bc832914fe46869f8849de5b92dc4a3466cd63095d23dfd"}, - {file = "yarl-1.8.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:77e913b846a6b9c5f767b14dc1e759e5aff05502fe73079f6f4176359d832581"}, - {file = "yarl-1.8.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0978f29222e649c351b173da2b9b4665ad1feb8d1daa9d971eb90df08702668a"}, - {file = "yarl-1.8.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:388a45dc77198b2460eac0aca1efd6a7c09e976ee768b0d5109173e521a19daf"}, - {file = "yarl-1.8.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2305517e332a862ef75be8fad3606ea10108662bc6fe08509d5ca99503ac2aee"}, - {file = "yarl-1.8.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:42430ff511571940d51e75cf42f1e4dbdded477e71c1b7a17f4da76c1da8ea76"}, - {file = "yarl-1.8.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3150078118f62371375e1e69b13b48288e44f6691c1069340081c3fd12c94d5b"}, - {file = "yarl-1.8.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:c15163b6125db87c8f53c98baa5e785782078fbd2dbeaa04c6141935eb6dab7a"}, - {file = "yarl-1.8.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:4d04acba75c72e6eb90745447d69f84e6c9056390f7a9724605ca9c56b4afcc6"}, - {file = "yarl-1.8.2-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:e7fd20d6576c10306dea2d6a5765f46f0ac5d6f53436217913e952d19237efc4"}, - {file = "yarl-1.8.2-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:75c16b2a900b3536dfc7014905a128a2bea8fb01f9ee26d2d7d8db0a08e7cb2c"}, - {file = "yarl-1.8.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:6d88056a04860a98341a0cf53e950e3ac9f4e51d1b6f61a53b0609df342cc8b2"}, - {file = "yarl-1.8.2-cp311-cp311-win32.whl", hash = "sha256:fb742dcdd5eec9f26b61224c23baea46c9055cf16f62475e11b9b15dfd5c117b"}, - {file = "yarl-1.8.2-cp311-cp311-win_amd64.whl", hash = "sha256:8c46d3d89902c393a1d1e243ac847e0442d0196bbd81aecc94fcebbc2fd5857c"}, - {file = "yarl-1.8.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:ceff9722e0df2e0a9e8a79c610842004fa54e5b309fe6d218e47cd52f791d7ef"}, - {file = "yarl-1.8.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3f6b4aca43b602ba0f1459de647af954769919c4714706be36af670a5f44c9c1"}, - {file = "yarl-1.8.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1684a9bd9077e922300ecd48003ddae7a7474e0412bea38d4631443a91d61077"}, - {file = "yarl-1.8.2-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ebb78745273e51b9832ef90c0898501006670d6e059f2cdb0e999494eb1450c2"}, - {file = "yarl-1.8.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3adeef150d528ded2a8e734ebf9ae2e658f4c49bf413f5f157a470e17a4a2e89"}, - {file = "yarl-1.8.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:57a7c87927a468e5a1dc60c17caf9597161d66457a34273ab1760219953f7f4c"}, - {file = "yarl-1.8.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:efff27bd8cbe1f9bd127e7894942ccc20c857aa8b5a0327874f30201e5ce83d0"}, - {file = "yarl-1.8.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:a783cd344113cb88c5ff7ca32f1f16532a6f2142185147822187913eb989f739"}, - {file = "yarl-1.8.2-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:705227dccbe96ab02c7cb2c43e1228e2826e7ead880bb19ec94ef279e9555b5b"}, - {file = "yarl-1.8.2-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:34c09b43bd538bf6c4b891ecce94b6fa4f1f10663a8d4ca589a079a5018f6ed7"}, - {file = "yarl-1.8.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:a48f4f7fea9a51098b02209d90297ac324241bf37ff6be6d2b0149ab2bd51b37"}, - {file = "yarl-1.8.2-cp37-cp37m-win32.whl", hash = "sha256:0414fd91ce0b763d4eadb4456795b307a71524dbacd015c657bb2a39db2eab89"}, - {file = "yarl-1.8.2-cp37-cp37m-win_amd64.whl", hash = "sha256:d881d152ae0007809c2c02e22aa534e702f12071e6b285e90945aa3c376463c5"}, - {file = "yarl-1.8.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:5df5e3d04101c1e5c3b1d69710b0574171cc02fddc4b23d1b2813e75f35a30b1"}, - {file = "yarl-1.8.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:7a66c506ec67eb3159eea5096acd05f5e788ceec7b96087d30c7d2865a243918"}, - {file = "yarl-1.8.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:2b4fa2606adf392051d990c3b3877d768771adc3faf2e117b9de7eb977741229"}, - {file = "yarl-1.8.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1e21fb44e1eff06dd6ef971d4bdc611807d6bd3691223d9c01a18cec3677939e"}, - {file = "yarl-1.8.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:93202666046d9edadfe9f2e7bf5e0782ea0d497b6d63da322e541665d65a044e"}, - {file = "yarl-1.8.2-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fc77086ce244453e074e445104f0ecb27530d6fd3a46698e33f6c38951d5a0f1"}, - {file = "yarl-1.8.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:64dd68a92cab699a233641f5929a40f02a4ede8c009068ca8aa1fe87b8c20ae3"}, - {file = "yarl-1.8.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1b372aad2b5f81db66ee7ec085cbad72c4da660d994e8e590c997e9b01e44901"}, - {file = "yarl-1.8.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:e6f3515aafe0209dd17fb9bdd3b4e892963370b3de781f53e1746a521fb39fc0"}, - {file = "yarl-1.8.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:dfef7350ee369197106805e193d420b75467b6cceac646ea5ed3049fcc950a05"}, - {file = "yarl-1.8.2-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:728be34f70a190566d20aa13dc1f01dc44b6aa74580e10a3fb159691bc76909d"}, - {file = "yarl-1.8.2-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:ff205b58dc2929191f68162633d5e10e8044398d7a45265f90a0f1d51f85f72c"}, - {file = "yarl-1.8.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:baf211dcad448a87a0d9047dc8282d7de59473ade7d7fdf22150b1d23859f946"}, - {file = "yarl-1.8.2-cp38-cp38-win32.whl", hash = "sha256:272b4f1599f1b621bf2aabe4e5b54f39a933971f4e7c9aa311d6d7dc06965165"}, - {file = "yarl-1.8.2-cp38-cp38-win_amd64.whl", hash = "sha256:326dd1d3caf910cd26a26ccbfb84c03b608ba32499b5d6eeb09252c920bcbe4f"}, - {file = "yarl-1.8.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:f8ca8ad414c85bbc50f49c0a106f951613dfa5f948ab69c10ce9b128d368baf8"}, - {file = "yarl-1.8.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:418857f837347e8aaef682679f41e36c24250097f9e2f315d39bae3a99a34cbf"}, - {file = "yarl-1.8.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:ae0eec05ab49e91a78700761777f284c2df119376e391db42c38ab46fd662b77"}, - {file = "yarl-1.8.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:009a028127e0a1755c38b03244c0bea9d5565630db9c4cf9572496e947137a87"}, - {file = "yarl-1.8.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3edac5d74bb3209c418805bda77f973117836e1de7c000e9755e572c1f7850d0"}, - {file = "yarl-1.8.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:da65c3f263729e47351261351b8679c6429151ef9649bba08ef2528ff2c423b2"}, - {file = "yarl-1.8.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0ef8fb25e52663a1c85d608f6dd72e19bd390e2ecaf29c17fb08f730226e3a08"}, - {file = "yarl-1.8.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:bcd7bb1e5c45274af9a1dd7494d3c52b2be5e6bd8d7e49c612705fd45420b12d"}, - {file = "yarl-1.8.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:44ceac0450e648de86da8e42674f9b7077d763ea80c8ceb9d1c3e41f0f0a9951"}, - {file = "yarl-1.8.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:97209cc91189b48e7cfe777237c04af8e7cc51eb369004e061809bcdf4e55220"}, - {file = "yarl-1.8.2-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:48dd18adcf98ea9cd721a25313aef49d70d413a999d7d89df44f469edfb38a06"}, - {file = "yarl-1.8.2-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:e59399dda559688461762800d7fb34d9e8a6a7444fd76ec33220a926c8be1516"}, - {file = "yarl-1.8.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:d617c241c8c3ad5c4e78a08429fa49e4b04bedfc507b34b4d8dceb83b4af3588"}, - {file = "yarl-1.8.2-cp39-cp39-win32.whl", hash = "sha256:cb6d48d80a41f68de41212f3dfd1a9d9898d7841c8f7ce6696cf2fd9cb57ef83"}, - {file = "yarl-1.8.2-cp39-cp39-win_amd64.whl", hash = "sha256:6604711362f2dbf7160df21c416f81fac0de6dbcf0b5445a2ef25478ecc4c778"}, - {file = "yarl-1.8.2.tar.gz", hash = "sha256:49d43402c6e3013ad0978602bf6bf5328535c48d192304b91b97a3c6790b1562"}, + {file = "yarl-1.9.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:8c2ad583743d16ddbdf6bb14b5cd76bf43b0d0006e918809d5d4ddf7bde8dd82"}, + {file = "yarl-1.9.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:82aa6264b36c50acfb2424ad5ca537a2060ab6de158a5bd2a72a032cc75b9eb8"}, + {file = "yarl-1.9.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c0c77533b5ed4bcc38e943178ccae29b9bcf48ffd1063f5821192f23a1bd27b9"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ee4afac41415d52d53a9833ebae7e32b344be72835bbb589018c9e938045a560"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9bf345c3a4f5ba7f766430f97f9cc1320786f19584acc7086491f45524a551ac"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2a96c19c52ff442a808c105901d0bdfd2e28575b3d5f82e2f5fd67e20dc5f4ea"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:891c0e3ec5ec881541f6c5113d8df0315ce5440e244a716b95f2525b7b9f3608"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c3a53ba34a636a256d767c086ceb111358876e1fb6b50dfc4d3f4951d40133d5"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:566185e8ebc0898b11f8026447eacd02e46226716229cea8db37496c8cdd26e0"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:2b0738fb871812722a0ac2154be1f049c6223b9f6f22eec352996b69775b36d4"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:32f1d071b3f362c80f1a7d322bfd7b2d11e33d2adf395cc1dd4df36c9c243095"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:e9fdc7ac0d42bc3ea78818557fab03af6181e076a2944f43c38684b4b6bed8e3"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:56ff08ab5df8429901ebdc5d15941b59f6253393cb5da07b4170beefcf1b2528"}, + {file = "yarl-1.9.2-cp310-cp310-win32.whl", hash = "sha256:8ea48e0a2f931064469bdabca50c2f578b565fc446f302a79ba6cc0ee7f384d3"}, + {file = "yarl-1.9.2-cp310-cp310-win_amd64.whl", hash = "sha256:50f33040f3836e912ed16d212f6cc1efb3231a8a60526a407aeb66c1c1956dde"}, + {file = "yarl-1.9.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:646d663eb2232d7909e6601f1a9107e66f9791f290a1b3dc7057818fe44fc2b6"}, + {file = "yarl-1.9.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:aff634b15beff8902d1f918012fc2a42e0dbae6f469fce134c8a0dc51ca423bb"}, + {file = "yarl-1.9.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a83503934c6273806aed765035716216cc9ab4e0364f7f066227e1aaea90b8d0"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b25322201585c69abc7b0e89e72790469f7dad90d26754717f3310bfe30331c2"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:22a94666751778629f1ec4280b08eb11815783c63f52092a5953faf73be24191"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8ec53a0ea2a80c5cd1ab397925f94bff59222aa3cf9c6da938ce05c9ec20428d"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:159d81f22d7a43e6eabc36d7194cb53f2f15f498dbbfa8edc8a3239350f59fe7"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:832b7e711027c114d79dffb92576acd1bd2decc467dec60e1cac96912602d0e6"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:95d2ecefbcf4e744ea952d073c6922e72ee650ffc79028eb1e320e732898d7e8"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:d4e2c6d555e77b37288eaf45b8f60f0737c9efa3452c6c44626a5455aeb250b9"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:783185c75c12a017cc345015ea359cc801c3b29a2966c2655cd12b233bf5a2be"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:b8cc1863402472f16c600e3e93d542b7e7542a540f95c30afd472e8e549fc3f7"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:822b30a0f22e588b32d3120f6d41e4ed021806418b4c9f0bc3048b8c8cb3f92a"}, + {file = "yarl-1.9.2-cp311-cp311-win32.whl", hash = "sha256:a60347f234c2212a9f0361955007fcf4033a75bf600a33c88a0a8e91af77c0e8"}, + {file = "yarl-1.9.2-cp311-cp311-win_amd64.whl", hash = "sha256:be6b3fdec5c62f2a67cb3f8c6dbf56bbf3f61c0f046f84645cd1ca73532ea051"}, + {file = "yarl-1.9.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:38a3928ae37558bc1b559f67410df446d1fbfa87318b124bf5032c31e3447b74"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ac9bb4c5ce3975aeac288cfcb5061ce60e0d14d92209e780c93954076c7c4367"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3da8a678ca8b96c8606bbb8bfacd99a12ad5dd288bc6f7979baddd62f71c63ef"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:13414591ff516e04fcdee8dc051c13fd3db13b673c7a4cb1350e6b2ad9639ad3"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bf74d08542c3a9ea97bb8f343d4fcbd4d8f91bba5ec9d5d7f792dbe727f88938"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6e7221580dc1db478464cfeef9b03b95c5852cc22894e418562997df0d074ccc"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:494053246b119b041960ddcd20fd76224149cfea8ed8777b687358727911dd33"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:52a25809fcbecfc63ac9ba0c0fb586f90837f5425edfd1ec9f3372b119585e45"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:e65610c5792870d45d7b68c677681376fcf9cc1c289f23e8e8b39c1485384185"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:1b1bba902cba32cdec51fca038fd53f8beee88b77efc373968d1ed021024cc04"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:662e6016409828ee910f5d9602a2729a8a57d74b163c89a837de3fea050c7582"}, + {file = "yarl-1.9.2-cp37-cp37m-win32.whl", hash = "sha256:f364d3480bffd3aa566e886587eaca7c8c04d74f6e8933f3f2c996b7f09bee1b"}, + {file = "yarl-1.9.2-cp37-cp37m-win_amd64.whl", hash = "sha256:6a5883464143ab3ae9ba68daae8e7c5c95b969462bbe42e2464d60e7e2698368"}, + {file = "yarl-1.9.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:5610f80cf43b6202e2c33ba3ec2ee0a2884f8f423c8f4f62906731d876ef4fac"}, + {file = "yarl-1.9.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:b9a4e67ad7b646cd6f0938c7ebfd60e481b7410f574c560e455e938d2da8e0f4"}, + {file = "yarl-1.9.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:83fcc480d7549ccebe9415d96d9263e2d4226798c37ebd18c930fce43dfb9574"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5fcd436ea16fee7d4207c045b1e340020e58a2597301cfbcfdbe5abd2356c2fb"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:84e0b1599334b1e1478db01b756e55937d4614f8654311eb26012091be109d59"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3458a24e4ea3fd8930e934c129b676c27452e4ebda80fbe47b56d8c6c7a63a9e"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:838162460b3a08987546e881a2bfa573960bb559dfa739e7800ceeec92e64417"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f4e2d08f07a3d7d3e12549052eb5ad3eab1c349c53ac51c209a0e5991bbada78"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:de119f56f3c5f0e2fb4dee508531a32b069a5f2c6e827b272d1e0ff5ac040333"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:149ddea5abf329752ea5051b61bd6c1d979e13fbf122d3a1f9f0c8be6cb6f63c"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:674ca19cbee4a82c9f54e0d1eee28116e63bc6fd1e96c43031d11cbab8b2afd5"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:9b3152f2f5677b997ae6c804b73da05a39daa6a9e85a512e0e6823d81cdad7cc"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:5415d5a4b080dc9612b1b63cba008db84e908b95848369aa1da3686ae27b6d2b"}, + {file = "yarl-1.9.2-cp38-cp38-win32.whl", hash = "sha256:f7a3d8146575e08c29ed1cd287068e6d02f1c7bdff8970db96683b9591b86ee7"}, + {file = "yarl-1.9.2-cp38-cp38-win_amd64.whl", hash = "sha256:63c48f6cef34e6319a74c727376e95626f84ea091f92c0250a98e53e62c77c72"}, + {file = "yarl-1.9.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:75df5ef94c3fdc393c6b19d80e6ef1ecc9ae2f4263c09cacb178d871c02a5ba9"}, + {file = "yarl-1.9.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:c027a6e96ef77d401d8d5a5c8d6bc478e8042f1e448272e8d9752cb0aff8b5c8"}, + {file = "yarl-1.9.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f3b078dbe227f79be488ffcfc7a9edb3409d018e0952cf13f15fd6512847f3f7"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:59723a029760079b7d991a401386390c4be5bfec1e7dd83e25a6a0881859e716"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b03917871bf859a81ccb180c9a2e6c1e04d2f6a51d953e6a5cdd70c93d4e5a2a"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c1012fa63eb6c032f3ce5d2171c267992ae0c00b9e164efe4d73db818465fac3"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a74dcbfe780e62f4b5a062714576f16c2f3493a0394e555ab141bf0d746bb955"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8c56986609b057b4839968ba901944af91b8e92f1725d1a2d77cbac6972b9ed1"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:2c315df3293cd521033533d242d15eab26583360b58f7ee5d9565f15fee1bef4"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:b7232f8dfbd225d57340e441d8caf8652a6acd06b389ea2d3222b8bc89cbfca6"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:53338749febd28935d55b41bf0bcc79d634881195a39f6b2f767870b72514caf"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:066c163aec9d3d073dc9ffe5dd3ad05069bcb03fcaab8d221290ba99f9f69ee3"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8288d7cd28f8119b07dd49b7230d6b4562f9b61ee9a4ab02221060d21136be80"}, + {file = "yarl-1.9.2-cp39-cp39-win32.whl", hash = "sha256:b124e2a6d223b65ba8768d5706d103280914d61f5cae3afbc50fc3dfcc016623"}, + {file = "yarl-1.9.2-cp39-cp39-win_amd64.whl", hash = "sha256:61016e7d582bc46a5378ffdd02cd0314fb8ba52f40f9cf4d9a5e7dbef88dee18"}, + {file = "yarl-1.9.2.tar.gz", hash = "sha256:04ab9d4b9f587c06d801c2abfe9317b77cdf996c65a90d5e84ecc45010823571"}, ] [package.dependencies] diff --git a/poetry.toml b/poetry.toml index b9b5afe4..90fd704a 100644 --- a/poetry.toml +++ b/poetry.toml @@ -3,3 +3,6 @@ cache-dir = ".cache" [virtualenvs] path = ".venv" in-project = true + +[installer] +modern-installation = false diff --git a/pyproject.toml b/pyproject.toml index eeb951a2..a919d3f2 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,6 +1,6 @@ [tool.poetry] name = "aio-pika" -version = "9.0.7" +version = "9.1.0" description = "Wrapper around the aiormq for asyncio and humans" authors = ["Dmitry Orlov "] readme = "README.rst" @@ -45,7 +45,7 @@ setuptools = [{ version = '*', python = "< 3.8" }] [tool.poetry.group.dev.dependencies] aiomisc = "^17.2" -aiomisc-pytest = "*" +aiomisc-pytest = "^1.1.1" collective-checkdocs = "^0.2" coverage = "^6.5.0" coveralls = "^3.3.1" diff --git a/tests/conftest.py b/tests/conftest.py index efff05d9..0ea1dcfb 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -16,7 +16,7 @@ @pytest.fixture -async def add_cleanup(loop): +async def add_cleanup(event_loop): entities = [] def payload(func, *args, **kwargs): @@ -34,12 +34,12 @@ def payload(func, *args, **kwargs): @pytest.fixture -async def create_task(loop): +async def create_task(event_loop): tasks = [] def payload(coroutine): nonlocal tasks - task = loop.create_task(coroutine) + task = event_loop.create_task(coroutine) tasks.append(task) return task @@ -91,8 +91,8 @@ def connection_fabric(request): @pytest.fixture -def create_connection(connection_fabric, loop, amqp_url): - return partial(connection_fabric, amqp_url, loop=loop) +def create_connection(connection_fabric, event_loop, amqp_url): + return partial(connection_fabric, amqp_url, loop=event_loop) @pytest.fixture diff --git a/tests/test_amqp.py b/tests/test_amqp.py index f82a3084..0c95e655 100644 --- a/tests/test_amqp.py +++ b/tests/test_amqp.py @@ -48,7 +48,9 @@ def declare_exchange_(declare_exchange): class TestCaseAmqp(TestCaseAmqpBase): - async def test_properties(self, loop, connection: aio_pika.Connection): + async def test_properties( + self, event_loop, connection: aio_pika.Connection, + ): assert not connection.is_closed async def test_channel_close(self, connection: aio_pika.Connection): @@ -704,7 +706,7 @@ async def test_reject_twice( async def test_consuming( self, - loop, + event_loop, channel: aio_pika.Channel, declare_exchange: Callable, declare_queue: Callable, @@ -721,7 +723,7 @@ async def test_consuming( body = bytes(shortuuid.uuid(), "utf-8") - f = loop.create_future() + f = event_loop.create_future() async def handle(message): await message.ack() @@ -741,7 +743,7 @@ async def handle(message): async def test_consuming_not_coroutine( self, - loop, + event_loop, channel: aio_pika.Channel, declare_exchange: Callable, declare_queue: Callable, @@ -759,7 +761,7 @@ async def test_consuming_not_coroutine( body = bytes(shortuuid.uuid(), "utf-8") - f = loop.create_future() + f = event_loop.create_future() async def handle(message): await message.ack() @@ -875,6 +877,12 @@ async def test_wrong_credentials( async def test_set_qos(self, channel: aio_pika.Channel): await channel.set_qos(prefetch_count=1, global_=True) + async def test_set_qos_deprecated_all_channels( + self, channel: aio_pika.Channel, + ): + with pytest.deprecated_call(): + await channel.set_qos(prefetch_count=1, all_channels=True) + async def test_exchange_delete( self, channel: aio_pika.Channel, declare_exchange, ): @@ -883,7 +891,7 @@ async def test_exchange_delete( async def test_dlx( self, - loop, + event_loop, channel: aio_pika.Channel, declare_exchange: Callable, declare_queue: Callable, @@ -893,7 +901,7 @@ async def test_dlx( routing_key = "%s_routing_key" % suffix dlx_routing_key = "%s_dlx_routing_key" % suffix - f = loop.create_future() + f = event_loop.create_future() async def dlx_handle(message): await message.ack() @@ -948,7 +956,8 @@ async def dlx_handle(message): await f async def test_expiration( - self, channel: aio_pika.Channel, loop, declare_exchange, declare_queue, + self, channel: aio_pika.Channel, event_loop, + declare_exchange, declare_queue, ): dlx_queue = await declare_queue( @@ -982,7 +991,7 @@ async def test_expiration( queue.name, ) - f = loop.create_future() + f = event_loop.create_future() await dlx_queue.consume(f.set_result, no_ack=True) @@ -1207,7 +1216,9 @@ async def test_transaction_simple_async_rollback( async with channel.transaction(): raise ValueError - async def test_async_for_queue(self, loop, connection, declare_queue): + async def test_async_for_queue( + self, event_loop, connection, declare_queue, + ): channel2 = await self.create_channel(connection) queue = await declare_queue( @@ -1226,7 +1237,7 @@ async def publisher(): Message(body=str(i).encode()), routing_key=queue.name, ) - loop.create_task(publisher()) + event_loop.create_task(publisher()) count = 0 data = list() @@ -1242,7 +1253,7 @@ async def publisher(): assert data == list(map(lambda x: str(x).encode(), range(messages))) async def test_async_for_queue_context( - self, loop, connection, declare_queue, + self, event_loop, connection, declare_queue, ): channel2 = await self.create_channel(connection) @@ -1262,7 +1273,7 @@ async def publisher(): Message(body=str(i).encode()), routing_key=queue.name, ) - loop.create_task(publisher()) + event_loop.create_task(publisher()) count = 0 data = list() @@ -1279,7 +1290,8 @@ async def publisher(): assert data == list(map(lambda x: str(x).encode(), range(messages))) async def test_async_with_connection( - self, create_connection: Callable, connection, loop, declare_queue, + self, create_connection: Callable, + connection, event_loop, declare_queue, ): async with await create_connection() as connection: @@ -1301,7 +1313,7 @@ async def publisher(): Message(body=str(i).encode()), routing_key=queue.name, ) - loop.create_task(publisher()) + event_loop.create_task(publisher()) count = 0 data = list() @@ -1370,9 +1382,9 @@ async def test_channel_locked_resource( await q2.consume(print, exclusive=True) async def test_queue_iterator_close_was_called_twice( - self, create_connection: Callable, loop, declare_queue, + self, create_connection: Callable, event_loop, declare_queue, ): - future = loop.create_future() + future = event_loop.create_future() event = asyncio.Event() queue_name = get_random_name() @@ -1403,10 +1415,10 @@ async def task_inner(): future.set_exception(e) raise - task = loop.create_task(task_inner()) + task = event_loop.create_task(task_inner()) await event.wait() - loop.call_soon(task.cancel) + event_loop.call_soon(task.cancel) with pytest.raises(asyncio.CancelledError): await task @@ -1417,7 +1429,7 @@ async def task_inner(): async def test_queue_iterator_close_with_noack( self, create_connection: Callable, - loop, + event_loop, add_cleanup: Callable, declare_queue, ): @@ -1457,7 +1469,7 @@ async def task_inner(): Message(body), routing_key=queue_name, ) - task = loop.create_task(task_inner()) + task = event_loop.create_task(task_inner()) await task @@ -1563,7 +1575,7 @@ async def run(): await channel.set_qos(10) async def test_heartbeat_disabling( - self, loop, amqp_url: URL, connection_fabric, + self, event_loop, amqp_url: URL, connection_fabric, ): url = amqp_url.update_query(heartbeat=0) connection: AbstractConnection = await connection_fabric(url) @@ -1612,10 +1624,12 @@ async def test_connection_close( finally: await exchange.delete() - async def test_basic_return(self, connection: aio_pika.Connection, loop): + async def test_basic_return( + self, connection: aio_pika.Connection, event_loop, + ): channel = await self.create_channel(connection) - f = loop.create_future() + f = event_loop.create_future() def handler(channel, message: ReturnedMessage): f.set_result(message) @@ -1634,7 +1648,7 @@ def handler(channel, message: ReturnedMessage): assert returned.body == body # handler with exception - f = loop.create_future() + f = event_loop.create_future() await channel.close() diff --git a/tests/test_amqp_robust.py b/tests/test_amqp_robust.py index 92b4d4df..51ad0c62 100644 --- a/tests/test_amqp_robust.py +++ b/tests/test_amqp_robust.py @@ -18,8 +18,8 @@ def connection_fabric(): @pytest.fixture -def create_connection(connection_fabric, loop, amqp_url): - return partial(connection_fabric, amqp_url, loop=loop) +def create_connection(connection_fabric, event_loop, amqp_url): + return partial(connection_fabric, amqp_url, loop=event_loop) class TestCaseNoRobust(TestCaseAmqp): diff --git a/tests/test_amqp_robust_proxy.py b/tests/test_amqp_robust_proxy.py index 63762e01..0938d26d 100644 --- a/tests/test_amqp_robust_proxy.py +++ b/tests/test_amqp_robust_proxy.py @@ -60,20 +60,20 @@ def connection_fabric(): @pytest.fixture -def create_direct_connection(loop, amqp_direct_url): +def create_direct_connection(event_loop, amqp_direct_url): return partial( aio_pika.connect, amqp_direct_url.update_query( name=amqp_direct_url.query["name"] + "::direct", heartbeat=30, ), - loop=loop, + loop=event_loop, ) @pytest.fixture -def create_connection(connection_fabric, loop, amqp_url): - return partial(connection_fabric, amqp_url, loop=loop) +def create_connection(connection_fabric, event_loop, amqp_url): + return partial(connection_fabric, amqp_url, loop=event_loop) @pytest.fixture @@ -148,7 +148,7 @@ def reconnect_callback(conn): @aiomisc.timeout(30) async def test_robust_reconnect( create_connection, direct_connection, - proxy: TCPProxy, loop, add_cleanup: Callable, + proxy: TCPProxy, event_loop, add_cleanup: Callable, ): read_conn = await create_connection() # type: aio_pika.RobustConnection @@ -188,7 +188,7 @@ async def reader(queue_name): ) async with queue.iterator() as q: - loop.call_soon(consumer_event.set) + event_loop.call_soon(consumer_event.set) async for message in q: shared.append(message) @@ -197,7 +197,7 @@ async def reader(queue_name): logging.info("Exit reader task") try: - reader_task = loop.create_task(reader(queue.name)) + reader_task = event_loop.create_task(reader(queue.name)) await consumer_event.wait() logging.info("Disconnect all clients") @@ -260,7 +260,7 @@ async def test_channel_locked_resource2(connection: aio_pika.RobustConnection): async def test_channel_close_when_exclusive_queue( - create_connection, create_direct_connection, proxy: TCPProxy, loop, + create_connection, create_direct_connection, proxy: TCPProxy, event_loop, ): logging.info("Creating connections") direct_conn, proxy_conn = await asyncio.gather( @@ -299,7 +299,7 @@ async def close_after(delay, closer): await closer() logging.info("Closed") - await loop.create_task(close_after(5, direct_conn.close)) + await event_loop.create_task(close_after(5, direct_conn.close)) # reconnect fired await reconnect_event.wait() @@ -334,13 +334,13 @@ async def test_context_process_abrupt_channel_close( incoming_message = await queue.get(timeout=5) # close aiormq channel to emulate abrupt connection/channel close - await channel.channel.close() + underlay_channel = await channel.get_underlay_channel() + await underlay_channel.close() + with pytest.raises(aiormq.exceptions.ChannelInvalidStateError): async with incoming_message.process(): # emulate some activity on closed channel - await channel.channel.basic_publish( - b"dummy", exchange="", routing_key="non_existent", - ) + await channel.get_underlay_channel() # emulate connection/channel restoration of connect_robust await channel.reopen() @@ -429,14 +429,15 @@ async def reader(queue: aio_pika.Queue): @aiomisc.timeout(10) async def test_channel_restore( - connection_fabric, loop, amqp_url, proxy: TCPProxy, add_cleanup: Callable, + connection_fabric, event_loop, amqp_url, proxy: TCPProxy, + add_cleanup: Callable, ): heartbeat = 10 amqp_url = amqp_url.update_query(heartbeat=heartbeat) on_reopen = asyncio.Event() - conn = await connection_fabric(amqp_url, loop=loop) + conn = await connection_fabric(amqp_url, loop=event_loop) assert isinstance(conn, aio_pika.RobustConnection) async with conn: @@ -461,11 +462,12 @@ async def test_channel_restore( @aiomisc.timeout(20) async def test_channel_reconnect( - connection_fabric, loop, amqp_url, proxy: TCPProxy, add_cleanup: Callable, + connection_fabric, event_loop, amqp_url, + proxy: TCPProxy, add_cleanup: Callable, ): on_reconnect = asyncio.Event() - conn = await connection_fabric(amqp_url, loop=loop) + conn = await connection_fabric(amqp_url, loop=event_loop) assert isinstance(conn, aio_pika.RobustConnection) conn.reconnect_callbacks.add(lambda *_: on_reconnect.set(), weak=False) @@ -524,15 +526,17 @@ async def test_channel_reconnect_after_5kb( amqp_url, amqp_direct_url, connection_fabric, - loop: asyncio.AbstractEventLoop, + event_loop: asyncio.AbstractEventLoop, proxy: TCPProxy, add_cleanup: Callable, ): connection = await aio_pika.connect_robust( amqp_url.update_query(reconnect_interval=reconnect_timeout), - loop=loop, + loop=event_loop, + ) + direct_connection = await aio_pika.connect( + amqp_direct_url, loop=event_loop, ) - direct_connection = await aio_pika.connect(amqp_direct_url, loop=loop) on_reconnect = asyncio.Event() connection.reconnect_callbacks.add( @@ -618,22 +622,22 @@ async def test_channel_reconnect_stairway( amqp_url: URL, amqp_direct_url: URL, connection_fabric, - loop: asyncio.AbstractEventLoop, + event_loop: asyncio.AbstractEventLoop, proxy: TCPProxy, add_cleanup: Callable, ): - loop.set_debug(True) + event_loop.set_debug(True) connection = await aio_pika.connect_robust( amqp_url.update_query( reconnect_interval=f"{reconnect_timeout:.2f}", name="proxy", ), - loop=loop, + loop=event_loop, ) direct_connection = await aio_pika.connect( - amqp_direct_url.update_query("name=direct"), loop=loop, + amqp_direct_url.update_query("name=direct"), loop=event_loop, ) on_reconnect = asyncio.Event() diff --git a/tests/test_amqps.py b/tests/test_amqps.py index c4575553..4ccd403f 100644 --- a/tests/test_amqps.py +++ b/tests/test_amqps.py @@ -15,7 +15,7 @@ def connection_fabric(request): @pytest.fixture -def create_connection(connection_fabric, loop, amqp_url): +def create_connection(connection_fabric, event_loop, amqp_url): ssl_context = ssl.create_default_context() ssl_context.check_hostname = False ssl_context.verify_mode = ssl.VerifyMode.CERT_NONE @@ -23,7 +23,7 @@ def create_connection(connection_fabric, loop, amqp_url): return partial( connection_fabric, amqp_url.with_scheme("amqps").with_port(5671), - loop=loop, + loop=event_loop, ssl_context=ssl_context, ) diff --git a/tests/test_kwargs.py b/tests/test_kwargs.py index 6f47668d..b8e320d0 100644 --- a/tests/test_kwargs.py +++ b/tests/test_kwargs.py @@ -47,7 +47,7 @@ class TestCase: async def get_instance(self, url, **kwargs): return await connect( - url, connection_class=self.CONNECTION_CLASS, **kwargs + url, connection_class=self.CONNECTION_CLASS, **kwargs, ) async def test_kwargs(self): @@ -62,13 +62,13 @@ async def test_kwargs_values(self): positives = VALUE_GENERATORS[parser] # type: ignore for example, expected in positives.items(): # type: ignore instance = await self.get_instance( - f"amqp://localhost/?{key}={example}" + f"amqp://localhost/?{key}={example}", ) assert hasattr(instance, key) assert getattr(instance, key) == expected instance = await self.get_instance( - "amqp://localhost", **{key: example} + "amqp://localhost", **{key: example}, ) assert hasattr(instance, key) assert getattr(instance, key) == expected diff --git a/tests/test_pool.py b/tests/test_pool.py index fe1d908a..071cb7c0 100644 --- a/tests/test_pool.py +++ b/tests/test_pool.py @@ -7,7 +7,7 @@ @pytest.mark.parametrize("max_size", [50, 10, 5, 1]) -async def test_simple(max_size, loop): +async def test_simple(max_size, event_loop): counter = 0 async def create_instance(): @@ -16,7 +16,7 @@ async def create_instance(): counter += 1 return counter - pool: Pool = Pool(create_instance, max_size=max_size, loop=loop) + pool: Pool = Pool(create_instance, max_size=max_size, loop=event_loop) async def getter(): nonlocal counter, pool @@ -55,7 +55,7 @@ def max_size(self, request): return request.param @pytest.fixture - def pool(self, max_size, instances, loop): + def pool(self, max_size, instances, event_loop): async def create_instance(): nonlocal instances @@ -63,11 +63,11 @@ async def create_instance(): instances.add(obj) return obj - return Pool(create_instance, max_size=max_size, loop=loop) + return Pool(create_instance, max_size=max_size, loop=event_loop) class TestInstance(TestInstanceBase): - async def test_close(self, pool, instances, loop, max_size): + async def test_close(self, pool, instances, event_loop, max_size): async def getter(): async with pool.acquire(): await asyncio.sleep(0.05) @@ -115,7 +115,7 @@ async def getter(): class TestCaseNoMaxSize(TestInstance): - async def test_simple(self, pool, loop): + async def test_simple(self, pool, event_loop): call_count = 200 counter = 0 diff --git a/tests/test_rpc.py b/tests/test_rpc.py index 98f2bf7d..3b3e107f 100644 --- a/tests/test_rpc.py +++ b/tests/test_rpc.py @@ -1,5 +1,7 @@ import asyncio import logging +import warnings +from functools import partial import pytest @@ -12,7 +14,7 @@ from tests import get_random_name -def rpc_func(*, foo, bar): +async def rpc_func(*, foo, bar): assert not foo assert not bar @@ -137,7 +139,9 @@ async def test_send_unknown_message( await rpc.close() - async def test_close_cancelling(self, channel: aio_pika.Channel, loop): + async def test_close_cancelling( + self, channel: aio_pika.Channel, event_loop, + ): rpc = await RPC.create(channel, auto_delete=True) async def sleeper(): @@ -150,7 +154,7 @@ async def sleeper(): tasks = set() for _ in range(10): - tasks.add(loop.create_task(rpc.call(method_name))) + tasks.add(event_loop.create_task(rpc.call(method_name))) await rpc.close() @@ -162,11 +166,14 @@ async def sleeper(): async def test_register_twice(self, channel: aio_pika.Channel): rpc = await RPC.create(channel, auto_delete=True) - await rpc.register("test.sleeper", lambda x: None, auto_delete=True) + async def bypass(_: aio_pika.abc.AbstractIncomingMessage): + return + + await rpc.register("test.sleeper", bypass, auto_delete=True) with pytest.raises(RuntimeError): await rpc.register( - "test.sleeper", lambda x: None, auto_delete=True, + "test.sleeper", bypass, auto_delete=True, ) await rpc.register("test.one", rpc_func, auto_delete=True) @@ -178,3 +185,39 @@ async def test_register_twice(self, channel: aio_pika.Channel): await rpc.unregister(rpc_func) await rpc.close() + + async def test_register_non_coroutine(self, channel: aio_pika.Channel): + rpc = await RPC.create(channel, auto_delete=True) + + def bypass(_): + return + + with pytest.deprecated_call(): + await rpc.register( + "test.non-coroutine", + bypass, # type: ignore + auto_delete=True, + ) + + async def coro(_): + return + + with pytest.warns(UserWarning) as record: + warnings.warn("Test", UserWarning) + await rpc.register( + "test.coroutine", + coro, # type: ignore + auto_delete=True, + ) + + assert len(record) == 1 + + with pytest.warns() as record: + warnings.warn("Test", UserWarning) + await rpc.register( + "test.coroutine_partial", + partial(partial(coro)), # type: ignore + auto_delete=True, + ) + + assert len(record) == 1 diff --git a/tests/test_tools.py b/tests/test_tools.py index 5190ea09..6fb059ef 100644 --- a/tests/test_tools.py +++ b/tests/test_tools.py @@ -6,7 +6,7 @@ import pytest -from aio_pika.tools import CallbackCollection +from aio_pika.tools import CallbackCollection, ensure_awaitable log = logging.getLogger(__name__) @@ -91,8 +91,10 @@ def test_callback_call(self, collection): async def test_blank_awaitable_callback(self, collection): await collection() - async def test_awaitable_callback(self, loop, collection, instance): - future = loop.create_future() + async def test_awaitable_callback( + self, event_loop, collection, instance, + ): + future = event_loop.create_future() shared = [] @@ -101,7 +103,7 @@ async def coro(arg): shared.append(arg) def task_maker(arg): - return loop.create_task(coro(arg)) + return event_loop.create_task(coro(arg)) collection.add(future.set_result) collection.add(coro) @@ -112,8 +114,10 @@ def task_maker(arg): assert shared == [instance, instance] assert await future == instance - async def test_collection_create_tasks(self, loop, collection, instance): - future = loop.create_future() + async def test_collection_create_tasks( + self, event_loop, collection, instance, + ): + future = event_loop.create_future() async def coro(arg): await asyncio.sleep(0.5) @@ -143,3 +147,49 @@ async def __call__(self, *args, **kwargs): await asyncio.wait_for(collection(), timeout=2) assert [c.counter for c in callables] == [1] * 100 + + +class TestEnsureAwaitable: + async def test_non_coroutine(self): + with pytest.deprecated_call(match="You probably registering the"): + func = ensure_awaitable(lambda x: x * x) + + with pytest.deprecated_call(match="Function"): + assert await func(2) == 4 + + with pytest.deprecated_call(match="Function"): + assert await func(4) == 16 + + async def test_coroutine(self): + async def square(x): + return x * x + func = ensure_awaitable(square) + assert await func(2) == 4 + assert await func(4) == 16 + + async def test_something_awaitable_returned(self): + + def non_coro(x): + async def coro(x): + return x * x + + return coro(x) + + with pytest.deprecated_call(match="You probably registering the"): + func = ensure_awaitable(non_coro) + + assert await func(2) == 4 + + async def test_something_non_awaitable_returned(self): + + def non_coro(x): + def coro(x): + return x * x + + return coro(x) + + with pytest.deprecated_call(match="You probably registering the"): + func = ensure_awaitable(non_coro) + + with pytest.deprecated_call(match="Function"): + assert await func(2) == 4