diff --git a/CHANGES b/CHANGES index dbc27dbacc..4b3c8cf27e 100644 --- a/CHANGES +++ b/CHANGES @@ -1,3 +1,4 @@ + * Add AnyIO based async client (and by extension, add support for Trio) * Support transactions in ClusterPipeline * Removing support for RedisGraph module. RedisGraph support is deprecated since Redis Stack 7.2 (https://redis.com/blog/redisgraph-eol/) * Fix lock.extend() typedef to accept float TTL extension @@ -64,7 +65,7 @@ * Fix Sentinel.execute_command doesn't execute across the entire sentinel cluster bug (#2458) * Added a replacement for the default cluster node in the event of failure (#2463) * Fix for Unhandled exception related to self.host with unix socket (#2496) - * Improve error output for master discovery + * Improve error output for master discovery * Make `ClusterCommandsProtocol` an actual Protocol * Add `sum` to DUPLICATE_POLICY documentation of `TS.CREATE`, `TS.ADD` and `TS.ALTER` * Prevent async ClusterPipeline instances from becoming "false-y" in case of empty command stack (#3061) diff --git a/dev_requirements.txt b/dev_requirements.txt index ad7330598d..26137efb9f 100644 --- a/dev_requirements.txt +++ b/dev_requirements.txt @@ -14,3 +14,4 @@ uvloop vulture>=2.3.0 numpy>=1.24.0 redis-entraid==0.4.0b2 +anyio[trio] >= 4.5.2 diff --git a/pyproject.toml b/pyproject.toml index 5cd40c0212..f4e14afc8a 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -32,6 +32,9 @@ classifiers = [ dependencies = ['async-timeout>=4.0.3; python_full_version<"3.11.3"'] [project.optional-dependencies] +anyio = [ + "anyio >= 4.5.2", +] hiredis = [ "hiredis>=3.0.0", ] @@ -68,13 +71,13 @@ markers = [ "onlycluster: marks tests to be run only with cluster mode redis", "onlynoncluster: marks tests to be run only with standalone redis", "ssl: marker for only the ssl tests", - "asyncio: marker for async tests", + "asyncio: marker for asyncio tests", "replica: replica tests", "experimental: run only experimental tests", "cp_integration: credential provider integration tests", ] asyncio_default_fixture_loop_scope = "function" -asyncio_mode = "auto" +asyncio_mode = "strict" timeout = 30 filterwarnings = [ "always", diff --git a/redis/__init__.py b/redis/__init__.py index cd3ee12adb..50f1e95a21 100644 --- a/redis/__init__.py +++ b/redis/__init__.py @@ -1,4 +1,5 @@ from redis import asyncio # noqa +from redis import anyio # noqa from redis.backoff import default_backoff from redis.client import Redis, StrictRedis from redis.cluster import RedisCluster diff --git a/redis/anyio/__init__.py b/redis/anyio/__init__.py new file mode 100644 index 0000000000..32254326bf --- /dev/null +++ b/redis/anyio/__init__.py @@ -0,0 +1,64 @@ +from redis.anyio.client import Redis, StrictRedis +from redis.anyio.cluster import RedisCluster +from redis.anyio.connection import ( + BlockingConnectionPool, + Connection, + ConnectionPool, + SSLConnection, + UnixDomainSocketConnection, +) +from redis.anyio.sentinel import ( + Sentinel, + SentinelConnectionPool, + SentinelManagedConnection, + SentinelManagedSSLConnection, +) +from redis.anyio.utils import from_url +from redis.backoff import default_backoff +from redis.exceptions import ( + AuthenticationError, + AuthenticationWrongNumberOfArgsError, + BusyLoadingError, + ChildDeadlockedError, + ConnectionError, + DataError, + InvalidResponse, + OutOfMemoryError, + PubSubError, + ReadOnlyError, + RedisError, + ResponseError, + TimeoutError, + WatchError, +) + +__all__ = [ + "AuthenticationError", + "AuthenticationWrongNumberOfArgsError", + "BlockingConnectionPool", + "BusyLoadingError", + "ChildDeadlockedError", + "Connection", + "ConnectionError", + "ConnectionPool", + "DataError", + "from_url", + "default_backoff", + "InvalidResponse", + "PubSubError", + "OutOfMemoryError", + "ReadOnlyError", + "Redis", + "RedisCluster", + "RedisError", + "ResponseError", + "Sentinel", + "SentinelConnectionPool", + "SentinelManagedConnection", + "SentinelManagedSSLConnection", + "SSLConnection", + "StrictRedis", + "TimeoutError", + "UnixDomainSocketConnection", + "WatchError", +] diff --git a/redis/anyio/_commands/__init__.py b/redis/anyio/_commands/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/redis/anyio/_commands/cluster.py b/redis/anyio/_commands/cluster.py new file mode 100644 index 0000000000..a431b5b193 --- /dev/null +++ b/redis/anyio/_commands/cluster.py @@ -0,0 +1,209 @@ +from __future__ import annotations + +from collections.abc import Sequence +from typing import ( + Any, + AsyncIterator, + Iterable, + List, + Mapping, + Optional, +) + +from ...commands.cluster import ( + READ_COMMANDS, + ClusterDataAccessCommands, + ClusterManagementCommands, + ClusterMultiKeyCommands, +) +from ...commands.core import ( + AsyncACLCommands, + AsyncDataAccessCommands, + AsyncFunctionCommands, + AsyncManagementCommands, + AsyncModuleCommands, + AsyncScriptCommands, +) +from ...commands.helpers import list_or_args +from ...commands.redismodules import AsyncRedisModuleCommands +from ...typing import AnyKeyT, EncodableT, KeysT, KeyT, PatternT +from ..utils import gather + + +class AsyncClusterMultiKeyCommands(ClusterMultiKeyCommands): + """ + A class containing commands that handle more than one key + """ + + async def mget_nonatomic(self, keys: KeysT, *args: KeyT) -> List[Optional[Any]]: + """ + Splits the keys into different slots and then calls MGET + for the keys of every slot. This operation will not be atomic + if keys belong to more than one slot. + + Returns a list of values ordered identically to ``keys`` + + For more information see https://redis.io/commands/mget + """ + + # Concatenate all keys into a list + keys = list_or_args(keys, args) + + # Split keys into slots + slots_to_keys = self._partition_keys_by_slot(keys) + + # Execute commands using a pipeline + res = await self._execute_pipeline_by_slot("MGET", slots_to_keys) + + # Reorder keys in the order the user provided & return + return self._reorder_keys_by_command(keys, slots_to_keys, res) + + async def mset_nonatomic(self, mapping: Mapping[AnyKeyT, EncodableT]) -> List[bool]: + """ + Sets key/values based on a mapping. Mapping is a dictionary of + key/value pairs. Both keys and values should be strings or types that + can be cast to a string via str(). + + Splits the keys into different slots and then calls MSET + for the keys of every slot. This operation will not be atomic + if keys belong to more than one slot. + + For more information see https://redis.io/commands/mset + """ + + # Partition the keys by slot + slots_to_pairs = self._partition_pairs_by_slot(mapping) + + # Execute commands using a pipeline & return list of replies + return await self._execute_pipeline_by_slot("MSET", slots_to_pairs) + + async def _split_command_across_slots(self, command: str, *keys: KeyT) -> int: + """ + Runs the given command once for the keys + of each slot. Returns the sum of the return values. + """ + + # Partition the keys by slot + slots_to_keys = self._partition_keys_by_slot(keys) + + # Sum up the reply from each command + return sum(await self._execute_pipeline_by_slot(command, slots_to_keys)) + + async def _execute_pipeline_by_slot( + self, command: str, slots_to_args: Mapping[int, Iterable[EncodableT]] + ) -> List[Any]: + read_from_replicas = self.read_from_replicas and command in READ_COMMANDS + pipe = self.pipeline() + [ + pipe.execute_command( + command, + *slot_args, + target_nodes=[ + self.nodes_manager.get_node_from_slot(slot, read_from_replicas) + ], + ) + for slot, slot_args in slots_to_args.items() + ] + return await pipe.execute() + + +class AsyncClusterManagementCommands( + ClusterManagementCommands, AsyncManagementCommands +): + """ + A class for Redis Cluster management commands + + The class inherits from Redis's core ManagementCommands class and do the + required adjustments to work with cluster mode + """ + + async def cluster_delslots(self, *slots: EncodableT) -> Sequence[bool]: + """ + Set hash slots as unbound in the cluster. + It determines by it self what node the slot is in and sends it there + + Returns a list of the results for each processed slot. + + For more information see https://redis.io/commands/cluster-delslots + """ + return await gather( + *(self.execute_command("CLUSTER DELSLOTS", slot) for slot in slots) + ) + + +class AsyncClusterDataAccessCommands( + ClusterDataAccessCommands, AsyncDataAccessCommands +): + """ + A class for Redis Cluster Data Access Commands + + The class inherits from Redis's core DataAccessCommand class and do the + required adjustments to work with cluster mode + """ + + async def scan_iter( + self, + match: Optional[PatternT] = None, + count: Optional[int] = None, + _type: Optional[str] = None, + **kwargs, + ) -> AsyncIterator: + # Do the first query with cursor=0 for all nodes + cursors, data = await self.scan(match=match, count=count, _type=_type, **kwargs) + for value in data: + yield value + + cursors = {name: cursor for name, cursor in cursors.items() if cursor != 0} + if cursors: + # Get nodes by name + nodes = {name: self.get_node(node_name=name) for name in cursors.keys()} + + # Iterate over each node till its cursor is 0 + kwargs.pop("target_nodes", None) + while cursors: + for name, cursor in cursors.items(): + cur, data = await self.scan( + cursor=cursor, + match=match, + count=count, + _type=_type, + target_nodes=nodes[name], + **kwargs, + ) + for value in data: + yield value + cursors[name] = cur[name] + + cursors = { + name: cursor for name, cursor in cursors.items() if cursor != 0 + } + + +class AsyncRedisClusterCommands( + AsyncClusterMultiKeyCommands, + AsyncClusterManagementCommands, + AsyncACLCommands, + AsyncClusterDataAccessCommands, + AsyncScriptCommands, + AsyncFunctionCommands, + AsyncModuleCommands, + AsyncRedisModuleCommands, +): + """ + A class for all Redis Cluster commands + + For key-based commands, the target node(s) will be internally determined + by the keys' hash slot. + Non-key-based commands can be executed with the 'target_nodes' argument to + target specific nodes. By default, if target_nodes is not specified, the + command will be executed on the default cluster node. + + :param :target_nodes: type can be one of the followings: + - nodes flag: ALL_NODES, PRIMARIES, REPLICAS, RANDOM + - 'ClusterNode' + - 'list(ClusterNodes)' + - 'dict(any:clusterNodes)' + + for example: + r.cluster_info(target_nodes=RedisCluster.ALL_NODES) + """ diff --git a/redis/anyio/_parsers/__init__.py b/redis/anyio/_parsers/__init__.py new file mode 100644 index 0000000000..c64eb172f6 --- /dev/null +++ b/redis/anyio/_parsers/__init__.py @@ -0,0 +1,11 @@ +from .base import _AnyIORESPBase +from .hiredis import _AnyIOHiredisParser +from .resp2 import _AnyIORESP2Parser +from .resp3 import _AnyIORESP3Parser + +__all__ = [ + "_AnyIORESPBase", + "_AnyIOHiredisParser", + "_AnyIORESP2Parser", + "_AnyIORESP3Parser", +] diff --git a/redis/anyio/_parsers/base.py b/redis/anyio/_parsers/base.py new file mode 100644 index 0000000000..c0d6251b55 --- /dev/null +++ b/redis/anyio/_parsers/base.py @@ -0,0 +1,115 @@ +from __future__ import annotations + +import sys +from abc import abstractmethod + +from anyio import IncompleteRead, move_on_after +from anyio.abc import ByteStream +from anyio.streams.buffered import BufferedByteReceiveStream + +from redis._parsers import BaseParser, Encoder +from redis.exceptions import RedisError, ResponseError +from redis.typing import EncodableT + +from ..._parsers.socket import SERVER_CLOSED_CONNECTION_ERROR + + +class AnyIOBaseParser(BaseParser): + """Base parsing class for the python-backed async parser""" + + def __init__(self, socket_read_size: int): + self._socket_read_size = socket_read_size + self._stream: ByteStream | None = None + self._connected = False + + async def can_read_destructive(self) -> bool: + raise NotImplementedError() + + @abstractmethod + async def read_response( + self, disable_decoding: bool = False + ) -> EncodableT | ResponseError | list[EncodableT] | None: + pass + + +class _AnyIORESPBase(AnyIOBaseParser): + """Base class for async resp parsing""" + + def __init__(self, socket_read_size: int): + super().__init__(socket_read_size) + self.encoder: Encoder | None = None + self._buffer = b"" + self._chunks = [] + self._pos = 0 + + def _clear(self): + self._buffer = b"" + self._chunks.clear() + self._pos = 0 + + def on_connect(self, connection): + """Called when the stream connects""" + if connection._stream is None: + raise RedisError("Buffer is closed.") + + self._stream = BufferedByteReceiveStream(connection._stream) + self.encoder = connection.encoder + self._clear() + self._connected = True + + def on_disconnect(self): + """Called when the stream disconnects""" + self._connected = False + + async def can_read_destructive(self) -> bool: + if not self._connected: + raise RedisError("Buffer is closed.") + + if self._stream.buffer: + return True + + with move_on_after(0): + await self._read(1) + return True + + return False + + async def _read(self, length: int) -> bytes: + """ + Read `length` bytes of data. These are assumed to be followed + by a '\r\n' terminator which is subsequently discarded. + """ + want = length + 2 + end = self._pos + want + if len(self._buffer) >= end: + result = self._buffer[self._pos : end - 2] + else: + tail = self._buffer[self._pos :] + try: + data = await self._stream.receive_exactly(want - len(tail)) + except IncompleteRead as error: + raise ConnectionError(SERVER_CLOSED_CONNECTION_ERROR) from error + result = (tail + data)[:-2] + self._chunks.append(data) + self._pos += want + return result + + async def _readline(self) -> bytes: + """ + read an unknown number of bytes up to the next '\r\n' + line separator, which is discarded. + """ + found = self._buffer.find(b"\r\n", self._pos) + if found >= 0: + result = self._buffer[self._pos : found] + else: + tail = self._buffer[self._pos :] + try: + data = await self._stream.receive_until(b"\r\n", sys.maxsize) + except IncompleteRead as error: + raise ConnectionError(SERVER_CLOSED_CONNECTION_ERROR) from error + + result = tail + data + self._chunks.append(data + b"\r\n") + self._pos += len(result) + 2 + return result diff --git a/redis/anyio/_parsers/commands.py b/redis/anyio/_parsers/commands.py new file mode 100644 index 0000000000..fcc5251496 --- /dev/null +++ b/redis/anyio/_parsers/commands.py @@ -0,0 +1,120 @@ +from __future__ import annotations + +from typing import TYPE_CHECKING, Any + +from redis._parsers.commands import AbstractCommandsParser +from redis.exceptions import RedisError, ResponseError +from redis.utils import str_if_bytes + +if TYPE_CHECKING: + from redis.anyio.cluster import ClusterNode + + +class AnyIOCommandsParser(AbstractCommandsParser): + """ + Parses Redis commands to get command keys. + + COMMAND output is used to determine key locations. + Commands that do not have a predefined key location are flagged with 'movablekeys', + and these commands' keys are determined by the command 'COMMAND GETKEYS'. + + NOTE: Due to a bug in redis<7.0, this does not work properly + for EVAL or EVALSHA when the `numkeys` arg is 0. + - issue: https://github.com/redis/redis/issues/9493 + - fix: https://github.com/redis/redis/pull/9733 + + So, don't use this with EVAL or EVALSHA. + """ + + def __init__(self) -> None: + self.commands: dict[str, int | dict[str, Any]] = {} + + async def initialize(self, node: ClusterNode | None = None) -> None: + if node: + self.node = node + + commands = await self.node.execute_command("COMMAND") + self.commands = {cmd.lower(): command for cmd, command in commands.items()} + + # As soon as this PR is merged into Redis, we should reimplement + # our logic to use COMMAND INFO changes to determine the key positions + # https://github.com/redis/redis/pull/8324 + async def get_keys(self, *args: Any) -> tuple[str, ...]: + """ + Get the keys from the passed command. + + NOTE: Due to a bug in redis<7.0, this function does not work properly + for EVAL or EVALSHA when the `numkeys` arg is 0. + - issue: https://github.com/redis/redis/issues/9493 + - fix: https://github.com/redis/redis/pull/9733 + + So, don't use this function with EVAL or EVALSHA. + """ + if len(args) < 2: + # The command has no keys in it + return () + + cmd_name = args[0].lower() + if cmd_name not in self.commands: + # try to split the command name and to take only the main command, + # e.g. 'memory' for 'memory usage' + cmd_name_split = cmd_name.split() + cmd_name = cmd_name_split[0] + if cmd_name in self.commands: + # save the splitted command to args + args = cmd_name_split + list(args[1:]) + else: + # We'll try to reinitialize the commands cache, if the engine + # version has changed, the commands may not be current + await self.initialize() + if cmd_name not in self.commands: + raise RedisError( + f"{cmd_name.upper()} command doesn't exist in Redis commands" + ) + + command = self.commands.get(cmd_name) + if "movablekeys" in command["flags"]: + keys = await self._get_moveable_keys(*args) + elif "pubsub" in command["flags"] or command["name"] == "pubsub": + keys = self._get_pubsub_keys(*args) + else: + if ( + command["step_count"] == 0 + and command["first_key_pos"] == 0 + and command["last_key_pos"] == 0 + ): + is_subcmd = False + if "subcommands" in command: + subcmd_name = f"{cmd_name}|{args[1].lower()}" + for subcmd in command["subcommands"]: + if str_if_bytes(subcmd[0]) == subcmd_name: + command = self.parse_subcommand(subcmd) + is_subcmd = True + + # The command doesn't have keys in it + if not is_subcmd: + return () + last_key_pos = command["last_key_pos"] + if last_key_pos < 0: + last_key_pos = len(args) - abs(last_key_pos) + keys_pos = list( + range(command["first_key_pos"], last_key_pos + 1, command["step_count"]) + ) + keys = [args[pos] for pos in keys_pos] + + return keys + + async def _get_moveable_keys(self, *args: Any) -> tuple[str, ...] | None: + try: + keys = await self.node.execute_command("COMMAND GETKEYS", *args) + except ResponseError as e: + message = str(e) + if ( + "Invalid arguments" in message + or "The command has no key arguments" in message + ): + return None + else: + raise e + + return keys diff --git a/redis/anyio/_parsers/hiredis.py b/redis/anyio/_parsers/hiredis.py new file mode 100644 index 0000000000..f423fda50b --- /dev/null +++ b/redis/anyio/_parsers/hiredis.py @@ -0,0 +1,113 @@ +from __future__ import annotations + +from collections.abc import Callable +from typing import TypedDict + +from anyio import BrokenResourceError, EndOfStream, move_on_after + +from ..._parsers.socket import ( + SERVER_CLOSED_CONNECTION_ERROR, +) +from ...exceptions import ConnectionError, InvalidResponse, RedisError +from ...typing import EncodableT +from ...utils import HIREDIS_AVAILABLE +from .base import AnyIOBaseParser + +# Used to signal that hiredis-py does not have enough data to parse. +# Using `False` or `None` is not reliable, given that the parser can +# return `False` or `None` for legitimate reasons from RESP payloads. +NOT_ENOUGH_DATA = object() + + +class _HiredisReaderArgs(TypedDict, total=False): + protocolError: Callable[[str], Exception] + replyError: Callable[[str], Exception] + encoding: str | None + errors: str | None + notEnoughData: object + + +class _AnyIOHiredisParser(AnyIOBaseParser): + """AnyIO implementation of parser class for connections using Hiredis""" + + def __init__(self, socket_read_size: int): + if not HIREDIS_AVAILABLE: + raise RedisError("Hiredis is not available.") + + super().__init__(socket_read_size=socket_read_size) + self._reader = None + + def on_connect(self, connection): + import hiredis + + self._stream = connection._stream + kwargs: _HiredisReaderArgs = { + "protocolError": InvalidResponse, + "replyError": self.parse_error, + "notEnoughData": NOT_ENOUGH_DATA, + } + if connection.encoder.decode_responses: + kwargs["encoding"] = connection.encoder.encoding + kwargs["errors"] = connection.encoder.encoding_errors + + self._reader = hiredis.Reader(**kwargs) + self._connected = True + + def on_disconnect(self): + self._connected = False + + async def can_read_destructive(self) -> bool: + if not self._connected: + raise ConnectionError(SERVER_CLOSED_CONNECTION_ERROR) + + if self._reader.gets() is not NOT_ENOUGH_DATA: + return True + + with move_on_after(0): + await self.read_from_socket() + return True + + return False + + async def read_from_socket(self) -> None: + try: + buffer = await self._stream.receive(self._socket_read_size) + except (EndOfStream, BrokenResourceError): + raise ConnectionError(SERVER_CLOSED_CONNECTION_ERROR) from None + + self._reader.feed(buffer) + + async def read_response( + self, disable_decoding: bool = False + ) -> EncodableT | list[EncodableT]: + # If `on_disconnect()` has been called, prohibit any more reads + # even if they could happen because data might be present. + # We still allow reads in progress to finish + if not self._connected: + raise ConnectionError(SERVER_CLOSED_CONNECTION_ERROR) + + if disable_decoding: + response = self._reader.gets(False) + else: + response = self._reader.gets() + + while response is NOT_ENOUGH_DATA: + await self.read_from_socket() + if disable_decoding: + response = self._reader.gets(False) + else: + response = self._reader.gets() + + # if the response is a ConnectionError or the response is a list and + # the first item is a ConnectionError, raise it as something bad + # happened + if isinstance(response, ConnectionError): + raise response + elif ( + isinstance(response, list) + and response + and isinstance(response[0], ConnectionError) + ): + raise response[0] + + return response diff --git a/redis/anyio/_parsers/resp2.py b/redis/anyio/_parsers/resp2.py new file mode 100644 index 0000000000..bbe55e6b5f --- /dev/null +++ b/redis/anyio/_parsers/resp2.py @@ -0,0 +1,72 @@ +from __future__ import annotations + +from typing import Any + +from ..._parsers.socket import SERVER_CLOSED_CONNECTION_ERROR +from ...exceptions import ConnectionError, InvalidResponse, ResponseError +from ...typing import EncodableT +from .base import _AnyIORESPBase + + +class _AnyIORESP2Parser(_AnyIORESPBase): + """Async class for the RESP2 protocol""" + + async def read_response(self, disable_decoding: bool = False): + if not self._connected: + raise ConnectionError(SERVER_CLOSED_CONNECTION_ERROR) + if self._chunks: + # augment parsing buffer with previously read data + self._buffer += b"".join(self._chunks) + self._chunks.clear() + self._pos = 0 + response = await self._read_response(disable_decoding=disable_decoding) + # Successfully parsing a response allows us to clear our parsing buffer + self._clear() + return response + + async def _read_response( + self, disable_decoding: bool = False + ) -> EncodableT | ResponseError | None: + raw = await self._readline() + response: Any + byte, response = raw[:1], raw[1:] + + # server returned an error + if byte == b"-": + response = response.decode("utf-8", errors="replace") + error = self.parse_error(response) + # if the error is a ConnectionError, raise immediately so the user + # is notified + if isinstance(error, ConnectionError): + self._clear() # Successful parse + raise error + # otherwise, we're dealing with a ResponseError that might belong + # inside a pipeline response. the connection's read_response() + # and/or the pipeline's execute() will raise this error if + # necessary, so just return the exception instance here. + return error + # single value + elif byte == b"+": + pass + # int value + elif byte == b":": + return int(response) + # bulk response + elif byte == b"$" and response == b"-1": + return None + elif byte == b"$": + response = await self._read(int(response)) + # multi-bulk response + elif byte == b"*" and response == b"-1": + return None + elif byte == b"*": + response = [ + (await self._read_response(disable_decoding)) + for _ in range(int(response)) # noqa + ] + else: + raise InvalidResponse(f"Protocol Error: {raw!r}") + + if disable_decoding is False: + response = self.encoder.decode(response) + return response diff --git a/redis/anyio/_parsers/resp3.py b/redis/anyio/_parsers/resp3.py new file mode 100644 index 0000000000..382eb7bfe6 --- /dev/null +++ b/redis/anyio/_parsers/resp3.py @@ -0,0 +1,151 @@ +from __future__ import annotations + +from logging import getLogger +from typing import Any + +from ..._parsers.socket import SERVER_CLOSED_CONNECTION_ERROR +from ...exceptions import ConnectionError, InvalidResponse, ResponseError +from ...typing import EncodableT +from .base import _AnyIORESPBase + +_INVALIDATION_MESSAGE = [b"invalidate", "invalidate"] + + +class _AnyIORESP3Parser(_AnyIORESPBase): + def __init__(self, socket_read_size): + super().__init__(socket_read_size) + self.pubsub_push_handler_func = self.handle_pubsub_push_response + self.invalidation_push_handler_func = None + + async def handle_pubsub_push_response(self, response): + logger = getLogger("push_response") + logger.debug("Push response: %s", response) + return response + + async def read_response( + self, disable_decoding: bool = False, push_request: bool = False + ): + if self._chunks: + # augment parsing buffer with previously read data + self._buffer += b"".join(self._chunks) + self._chunks.clear() + self._pos = 0 + response = await self._read_response( + disable_decoding=disable_decoding, push_request=push_request + ) + # Successfully parsing a response allows us to clear our parsing buffer + self._clear() + return response + + async def _read_response( + self, disable_decoding: bool = False, push_request: bool = False + ) -> EncodableT | ResponseError | None: + if not self._stream or not self.encoder: + raise ConnectionError(SERVER_CLOSED_CONNECTION_ERROR) + raw = await self._readline() + response: Any + byte, response = raw[:1], raw[1:] + + # if byte not in (b"-", b"+", b":", b"$", b"*"): + # raise InvalidResponse(f"Protocol Error: {raw!r}") + + # server returned an error + if byte in (b"-", b"!"): + if byte == b"!": + response = await self._read(int(response)) + response = response.decode("utf-8", errors="replace") + error = self.parse_error(response) + # if the error is a ConnectionError, raise immediately so the user + # is notified + if isinstance(error, ConnectionError): + self._clear() # Successful parse + raise error + # otherwise, we're dealing with a ResponseError that might belong + # inside a pipeline response. the connection's read_response() + # and/or the pipeline's execute() will raise this error if + # necessary, so just return the exception instance here. + return error + # single value + elif byte == b"+": + pass + # null value + elif byte == b"_": + return None + # int and big int values + elif byte in (b":", b"("): + return int(response) + # double value + elif byte == b",": + return float(response) + # bool value + elif byte == b"#": + return response == b"t" + # bulk response + elif byte == b"$": + response = await self._read(int(response)) + # verbatim string response + elif byte == b"=": + response = (await self._read(int(response)))[4:] + # array response + elif byte == b"*": + response = [ + (await self._read_response(disable_decoding=disable_decoding)) + for _ in range(int(response)) + ] + # set response + elif byte == b"~": + # redis can return unhashable types (like dict) in a set, + # so we always convert to a list, to have predictable return types + response = [ + (await self._read_response(disable_decoding=disable_decoding)) + for _ in range(int(response)) + ] + # map response + elif byte == b"%": + # We cannot use a dict-comprehension to parse stream. + # Evaluation order of key:val expression in dict comprehension only + # became defined to be left-right in version 3.8 + resp_dict = {} + for _ in range(int(response)): + key = await self._read_response(disable_decoding=disable_decoding) + resp_dict[key] = await self._read_response( + disable_decoding=disable_decoding, push_request=push_request + ) + response = resp_dict + # push response + elif byte == b">": + response = [ + ( + await self._read_response( + disable_decoding=disable_decoding, push_request=push_request + ) + ) + for _ in range(int(response)) + ] + response = await self.handle_push_response( + response, disable_decoding, push_request + ) + if not push_request: + return await self._read_response( + disable_decoding=disable_decoding, push_request=push_request + ) + else: + return response + else: + raise InvalidResponse(f"Protocol Error: {raw!r}") + + if isinstance(response, bytes) and disable_decoding is False: + response = self.encoder.decode(response) + return response + + async def handle_push_response(self, response, disable_decoding, push_request): + if response[0] not in _INVALIDATION_MESSAGE: + return await self.pubsub_push_handler_func(response) + if self.invalidation_push_handler_func: + return await self.invalidation_push_handler_func(response) + + def set_pubsub_push_handler(self, pubsub_push_handler_func): + self.pubsub_push_handler_func = pubsub_push_handler_func + + def set_invalidation_push_handler(self, invalidation_push_handler_func): + self.invalidation_push_handler_func = invalidation_push_handler_func diff --git a/redis/anyio/client.py b/redis/anyio/client.py new file mode 100644 index 0000000000..2c259f496d --- /dev/null +++ b/redis/anyio/client.py @@ -0,0 +1,1600 @@ +from __future__ import annotations + +import copy +import inspect +import re +import warnings +from typing import ( + TYPE_CHECKING, + Any, + AsyncIterator, + Awaitable, + Callable, + Dict, + Iterable, + List, + Mapping, + MutableMapping, + Optional, + Protocol, + Set, + Tuple, + Type, + TypedDict, + TypeVar, + Union, + cast, +) + +import anyio.lowlevel + +from redis._parsers.helpers import ( + _RedisCallbacks, + _RedisCallbacksRESP2, + _RedisCallbacksRESP3, + bool_ok, +) +from redis.anyio.connection import ( + Connection, + ConnectionPool, + SSLConnection, + UnixDomainSocketConnection, +) +from redis.anyio.lock import Lock +from redis.anyio.retry import Retry +from redis.client import ( + EMPTY_RESPONSE, + NEVER_DECODE, + AbstractRedis, + CaseInsensitiveDict, +) +from redis.commands import ( + AsyncCoreCommands, + AsyncRedisModuleCommands, + AsyncSentinelCommands, + list_or_args, +) +from redis.credentials import CredentialProvider +from redis.event import ( + AfterPooledConnectionsInstantiationEvent, + AfterPubSubConnectionInstantiationEvent, + AfterSingleConnectionInstantiationEvent, + ClientType, + EventDispatcher, +) +from redis.exceptions import ( + ConnectionError, + ExecAbortError, + PubSubError, + RedisError, + ResponseError, + TimeoutError, + WatchError, +) +from redis.typing import ChannelT, EncodableT, KeyT +from redis.utils import ( + HIREDIS_AVAILABLE, + SSL_AVAILABLE, + _set_info_logger, + deprecated_function, + get_lib_version, + safe_str, + str_if_bytes, +) + +if TYPE_CHECKING and SSL_AVAILABLE: + from ssl import TLSVersion +else: + TLSVersion = None + +PubSubHandler = Callable[[Dict[str, str]], Awaitable[None]] +_KeyT = TypeVar("_KeyT", bound=KeyT) +_ArgT = TypeVar("_ArgT", KeyT, EncodableT) +_RedisT = TypeVar("_RedisT", bound="Redis") +_NormalizeKeysT = TypeVar("_NormalizeKeysT", bound=Mapping[ChannelT, object]) +if TYPE_CHECKING: + from redis.commands.core import Script + + +class ResponseCallbackProtocol(Protocol): + def __call__(self, response: Any, **kwargs): ... + + +class AsyncResponseCallbackProtocol(Protocol): + async def __call__(self, response: Any, **kwargs): ... + + +ResponseCallbackT = Union[ResponseCallbackProtocol, AsyncResponseCallbackProtocol] + + +class Redis( + AbstractRedis, AsyncRedisModuleCommands, AsyncCoreCommands, AsyncSentinelCommands +): + """ + Implementation of the Redis protocol. + + This abstract class provides a Python interface to all Redis commands + and an implementation of the Redis protocol. + + Pipelines derive from this, implementing how + the commands are sent and received to the Redis server. Based on + configuration, an instance will either use a ConnectionPool, or + Connection object to talk to redis. + """ + + response_callbacks: MutableMapping[Union[str, bytes], ResponseCallbackT] + + @classmethod + def from_url( + cls, + url: str, + single_connection_client: bool = False, + auto_close_connection_pool: Optional[bool] = None, + **kwargs, + ): + """ + Return a Redis client object configured from the given URL + + For example:: + + redis://[[username]:[password]]@localhost:6379/0 + rediss://[[username]:[password]]@localhost:6379/0 + unix://[username@]/path/to/socket.sock?db=0[&password=password] + + Three URL schemes are supported: + + - `redis://` creates a TCP socket connection. See more at: + + - `rediss://` creates a SSL wrapped TCP socket connection. See more at: + + - ``unix://``: creates a Unix Domain Socket connection. + + The username, password, hostname, path and all querystring values + are passed through urllib.parse.unquote in order to replace any + percent-encoded values with their corresponding characters. + + There are several ways to specify a database number. The first value + found will be used: + + 1. A ``db`` querystring option, e.g. redis://localhost?db=0 + + 2. If using the redis:// or rediss:// schemes, the path argument + of the url, e.g. redis://localhost/0 + + 3. A ``db`` keyword argument to this function. + + If none of these options are specified, the default db=0 is used. + + All querystring options are cast to their appropriate Python types. + Boolean arguments can be specified with string values "True"/"False" + or "Yes"/"No". Values that cannot be properly cast cause a + ``ValueError`` to be raised. Once parsed, the querystring arguments + and keyword arguments are passed to the ``ConnectionPool``'s + class initializer. In the case of conflicting arguments, querystring + arguments always win. + + """ + connection_pool = ConnectionPool.from_url(url, **kwargs) + client = cls( + connection_pool=connection_pool, + single_connection_client=single_connection_client, + ) + if auto_close_connection_pool is not None: + warnings.warn( + DeprecationWarning( + '"auto_close_connection_pool" is deprecated ' + "since version 5.0.1. " + "Please create a ConnectionPool explicitly and " + "provide to the Redis() constructor instead." + ) + ) + else: + auto_close_connection_pool = True + client.auto_close_connection_pool = auto_close_connection_pool + return client + + @classmethod + def from_pool( + cls: Type["Redis"], + connection_pool: ConnectionPool, + ) -> "Redis": + """ + Return a Redis client from the given connection pool. + The Redis client will take ownership of the connection pool and + close it when the Redis client is closed. + """ + client = cls( + connection_pool=connection_pool, + ) + client.auto_close_connection_pool = True + return client + + def __init__( + self, + *, + host: str = "localhost", + port: int = 6379, + db: Union[str, int] = 0, + password: Optional[str] = None, + socket_timeout: Optional[float] = None, + socket_connect_timeout: Optional[float] = None, + socket_keepalive: Optional[bool] = None, + socket_keepalive_options: Optional[Mapping[int, Union[int, bytes]]] = None, + connection_pool: Optional[ConnectionPool] = None, + unix_socket_path: Optional[str] = None, + encoding: str = "utf-8", + encoding_errors: str = "strict", + decode_responses: bool = False, + retry_on_timeout: bool = False, + retry_on_error: Optional[list] = None, + ssl: bool = False, + ssl_keyfile: Optional[str] = None, + ssl_certfile: Optional[str] = None, + ssl_cert_reqs: str = "required", + ssl_ca_certs: Optional[str] = None, + ssl_ca_data: Optional[str] = None, + ssl_check_hostname: bool = False, + ssl_min_version: Optional[TLSVersion] = None, + ssl_ciphers: Optional[str] = None, + max_connections: Optional[int] = None, + single_connection_client: bool = False, + health_check_interval: int = 0, + client_name: Optional[str] = None, + lib_name: Optional[str] = "redis-py", + lib_version: Optional[str] = get_lib_version(), + username: Optional[str] = None, + retry: Optional[Retry] = None, + auto_close_connection_pool: Optional[bool] = None, + redis_connect_func=None, + credential_provider: Optional[CredentialProvider] = None, + protocol: Optional[int] = 2, + event_dispatcher: Optional[EventDispatcher] = None, + ): + """ + Initialize a new Redis client. + To specify a retry policy for specific errors, first set + `retry_on_error` to a list of the error/s to retry on, then set + `retry` to a valid `Retry` object. + To retry on TimeoutError, `retry_on_timeout` can also be set to `True`. + """ + kwargs: Dict[str, Any] + if event_dispatcher is None: + self._event_dispatcher = EventDispatcher() + else: + self._event_dispatcher = event_dispatcher + # auto_close_connection_pool only has an effect if connection_pool is + # None. It is assumed that if connection_pool is not None, the user + # wants to manage the connection pool themselves. + if auto_close_connection_pool is not None: + warnings.warn( + DeprecationWarning( + '"auto_close_connection_pool" is deprecated ' + "since version 5.0.1. " + "Please create a ConnectionPool explicitly and " + "provide to the Redis() constructor instead." + ) + ) + else: + auto_close_connection_pool = True + + if not connection_pool: + # Create internal connection pool, expected to be closed by Redis instance + if not retry_on_error: + retry_on_error = [] + if retry_on_timeout is True: + retry_on_error.append(TimeoutError) + kwargs = { + "db": db, + "username": username, + "password": password, + "credential_provider": credential_provider, + "socket_timeout": socket_timeout, + "encoding": encoding, + "encoding_errors": encoding_errors, + "decode_responses": decode_responses, + "retry_on_timeout": retry_on_timeout, + "retry_on_error": retry_on_error, + "retry": copy.deepcopy(retry), + "max_connections": max_connections, + "health_check_interval": health_check_interval, + "client_name": client_name, + "lib_name": lib_name, + "lib_version": lib_version, + "redis_connect_func": redis_connect_func, + "protocol": protocol, + } + # based on input, setup appropriate connection args + if unix_socket_path is not None: + kwargs.update( + { + "path": unix_socket_path, + "connection_class": UnixDomainSocketConnection, + } + ) + else: + # TCP specific options + kwargs.update( + { + "host": host, + "port": port, + "socket_connect_timeout": socket_connect_timeout, + "socket_keepalive": socket_keepalive, + "socket_keepalive_options": socket_keepalive_options, + } + ) + + if ssl: + kwargs.update( + { + "connection_class": SSLConnection, + "ssl_keyfile": ssl_keyfile, + "ssl_certfile": ssl_certfile, + "ssl_cert_reqs": ssl_cert_reqs, + "ssl_ca_certs": ssl_ca_certs, + "ssl_ca_data": ssl_ca_data, + "ssl_check_hostname": ssl_check_hostname, + "ssl_min_version": ssl_min_version, + "ssl_ciphers": ssl_ciphers, + } + ) + # This arg only used if no pool is passed in + self.auto_close_connection_pool = auto_close_connection_pool + connection_pool = ConnectionPool(**kwargs) + self._event_dispatcher.dispatch( + AfterPooledConnectionsInstantiationEvent( + [connection_pool], ClientType.ASYNC, credential_provider + ) + ) + else: + # If a pool is passed in, do not close it + self.auto_close_connection_pool = False + self._event_dispatcher.dispatch( + AfterPooledConnectionsInstantiationEvent( + [connection_pool], ClientType.ASYNC, credential_provider + ) + ) + + self.connection_pool = connection_pool + self.single_connection_client = single_connection_client + self.connection: Optional[Connection] = None + + self.response_callbacks = CaseInsensitiveDict(_RedisCallbacks) + + if self.connection_pool.connection_kwargs.get("protocol") in ["3", 3]: + self.response_callbacks.update(_RedisCallbacksRESP3) + else: + self.response_callbacks.update(_RedisCallbacksRESP2) + + # If using a single connection client, we need to lock creation-of and use-of + # the client in order to avoid race conditions such as using task groups on a + # set of redis commands + self._single_conn_lock = anyio.Lock() + + def __repr__(self): + return ( + f"<{self.__class__.__module__}.{self.__class__.__name__}" + f"({self.connection_pool!r})>" + ) + + def __await__(self): + return self.initialize().__await__() + + async def initialize(self: _RedisT) -> _RedisT: + if self.single_connection_client: + async with self._single_conn_lock: + if self.connection is None: + self.connection = await self.connection_pool.get_connection() + + self._event_dispatcher.dispatch( + AfterSingleConnectionInstantiationEvent( + self.connection, ClientType.ASYNC, self._single_conn_lock + ) + ) + return self + + def set_response_callback(self, command: str, callback: ResponseCallbackT): + """Set a custom Response Callback""" + self.response_callbacks[command] = callback + + def get_encoder(self): + """Get the connection pool's encoder""" + return self.connection_pool.get_encoder() + + def get_connection_kwargs(self): + """Get the connection's key-word arguments""" + return self.connection_pool.connection_kwargs + + def get_retry(self) -> Optional["Retry"]: + return self.get_connection_kwargs().get("retry") + + def set_retry(self, retry: "Retry") -> None: + self.get_connection_kwargs().update({"retry": retry}) + self.connection_pool.set_retry(retry) + + def load_external_module(self, funcname, func): + """ + This function can be used to add externally defined redis modules, + and their namespaces to the redis client. + + funcname - A string containing the name of the function to create + func - The function, being added to this class. + + ex: Assume that one has a custom redis module named foomod that + creates command named 'foo.dothing' and 'foo.anotherthing' in redis. + To load function functions into this namespace: + + from redis import Redis + from foomodule import F + r = Redis() + r.load_external_module("foo", F) + r.foo().dothing('your', 'arguments') + + For a concrete example see the reimport of the redisjson module in + tests/test_connection.py::test_loading_external_modules + """ + setattr(self, funcname, func) + + def pipeline( + self, transaction: bool = True, shard_hint: Optional[str] = None + ) -> "Pipeline": + """ + Return a new pipeline object that can queue multiple commands for + later execution. ``transaction`` indicates whether all commands + should be executed atomically. Apart from making a group of operations + atomic, pipelines are useful for reducing the back-and-forth overhead + between the client and server. + """ + return Pipeline( + self.connection_pool, self.response_callbacks, transaction, shard_hint + ) + + async def transaction( + self, + func: Callable[["Pipeline"], Union[Any, Awaitable[Any]]], + *watches: KeyT, + shard_hint: Optional[str] = None, + value_from_callable: bool = False, + watch_delay: Optional[float] = None, + ): + """ + Convenience method for executing the callable `func` as a transaction + while watching all keys specified in `watches`. The 'func' callable + should expect a single argument which is a Pipeline object. + """ + pipe: Pipeline + async with self.pipeline(True, shard_hint) as pipe: + while True: + try: + if watches: + await pipe.watch(*watches) + func_value = func(pipe) + if inspect.isawaitable(func_value): + func_value = await func_value + exec_value = await pipe.execute() + return func_value if value_from_callable else exec_value + except WatchError: + if watch_delay is not None and watch_delay > 0: + await anyio.sleep(watch_delay) + continue + + def lock( + self, + name: KeyT, + timeout: Optional[float] = None, + sleep: float = 0.1, + blocking: bool = True, + blocking_timeout: Optional[float] = None, + lock_class: Optional[Type[Lock]] = None, + thread_local: bool = True, + raise_on_release_error: bool = True, + ) -> Lock: + """ + Return a new Lock object using key ``name`` that mimics + the behavior of threading.Lock. + + If specified, ``timeout`` indicates a maximum life for the lock. + By default, it will remain locked until release() is called. + + ``sleep`` indicates the amount of time to sleep per loop iteration + when the lock is in blocking mode and another client is currently + holding the lock. + + ``blocking`` indicates whether calling ``acquire`` should block until + the lock has been acquired or to fail immediately, causing ``acquire`` + to return False and the lock not being acquired. Defaults to True. + Note this value can be overridden by passing a ``blocking`` + argument to ``acquire``. + + ``blocking_timeout`` indicates the maximum amount of time in seconds to + spend trying to acquire the lock. A value of ``None`` indicates + continue trying forever. ``blocking_timeout`` can be specified as a + float or integer, both representing the number of seconds to wait. + + ``lock_class`` forces the specified lock implementation. Note that as + of redis-py 3.0, the only lock class we implement is ``Lock`` (which is + a Lua-based lock). So, it's unlikely you'll need this parameter, unless + you have created your own custom lock class. + + ``thread_local`` indicates whether the lock token is placed in + thread-local storage. By default, the token is placed in thread local + storage so that a thread only sees its token, not a token set by + another thread. Consider the following timeline: + + time: 0, thread-1 acquires `my-lock`, with a timeout of 5 seconds. + thread-1 sets the token to "abc" + time: 1, thread-2 blocks trying to acquire `my-lock` using the + Lock instance. + time: 5, thread-1 has not yet completed. redis expires the lock + key. + time: 5, thread-2 acquired `my-lock` now that it's available. + thread-2 sets the token to "xyz" + time: 6, thread-1 finishes its work and calls release(). if the + token is *not* stored in thread local storage, then + thread-1 would see the token value as "xyz" and would be + able to successfully release the thread-2's lock. + + ``raise_on_release_error`` indicates whether to raise an exception when + the lock is no longer owned when exiting the context manager. By default, + this is True, meaning an exception will be raised. If False, the warning + will be logged and the exception will be suppressed. + + In some use cases it's necessary to disable thread local storage. For + example, if you have code where one thread acquires a lock and passes + that lock instance to a worker thread to release later. If thread + local storage isn't disabled in this case, the worker thread won't see + the token set by the thread that acquired the lock. Our assumption + is that these cases aren't common and as such default to using + thread local storage.""" + if lock_class is None: + lock_class = Lock + return lock_class( + self, + name, + timeout=timeout, + sleep=sleep, + blocking=blocking, + blocking_timeout=blocking_timeout, + thread_local=thread_local, + raise_on_release_error=raise_on_release_error, + ) + + def pubsub(self, **kwargs) -> "PubSub": + """ + Return a Publish/Subscribe object. With this object, you can + subscribe to channels and listen for messages that get published to + them. + """ + return PubSub( + self.connection_pool, event_dispatcher=self._event_dispatcher, **kwargs + ) + + def monitor(self) -> "Monitor": + return Monitor(self.connection_pool) + + def client(self) -> "Redis": + return self.__class__( + connection_pool=self.connection_pool, single_connection_client=True + ) + + async def __aenter__(self: _RedisT) -> _RedisT: + return await self.initialize() + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() + + _DEL_MESSAGE = "Unclosed Redis client" + + # passing _warnings and _grl as argument default since they may be gone + # by the time __del__ is called at shutdown + def __del__( + self, + _warn: Any = warnings.warn, + ) -> None: + if self.connection is not None: + _warn(f"Unclosed client session {self!r}", ResourceWarning, source=self) + + async def aclose(self, close_connection_pool: Optional[bool] = None) -> None: + """ + Closes Redis client connection + + Args: + close_connection_pool: + decides whether to close the connection pool used by this Redis client, + overriding Redis.auto_close_connection_pool. + By default, let Redis.auto_close_connection_pool decide + whether to close the connection pool. + """ + conn = self.connection + if conn: + self.connection = None + await self.connection_pool.release(conn) + if close_connection_pool or ( + close_connection_pool is None and self.auto_close_connection_pool + ): + await self.connection_pool.disconnect() + + @deprecated_function(version="5.0.1", reason="Use aclose() instead", name="close") + async def close(self, close_connection_pool: Optional[bool] = None) -> None: + """ + Alias for aclose(), for backwards compatibility + """ + await self.aclose(close_connection_pool) + + async def _send_command_parse_response(self, conn, command_name, *args, **options): + """ + Send a command and parse the response + """ + await conn.send_command(*args) + return await self.parse_response(conn, command_name, **options) + + async def _disconnect_raise(self, conn: Connection, error: Exception): + """ + Close the connection and raise an exception + if retry_on_error is not set or the error + is not one of the specified error types + """ + await conn.disconnect() + if ( + conn.retry_on_error is None + or isinstance(error, tuple(conn.retry_on_error)) is False + ): + raise error + + # COMMAND EXECUTION AND PROTOCOL PARSING + async def execute_command(self, *args, **options): + """Execute a command and return a parsed response""" + await self.initialize() + pool = self.connection_pool + command_name = args[0] + conn = self.connection or await pool.get_connection() + + if self.single_connection_client: + await self._single_conn_lock.acquire() + try: + return await conn.retry.call_with_retry( + lambda: self._send_command_parse_response( + conn, command_name, *args, **options + ), + lambda error: self._disconnect_raise(conn, error), + ) + finally: + if self.single_connection_client: + self._single_conn_lock.release() + if not self.connection: + await pool.release(conn) + + async def parse_response( + self, connection: Connection, command_name: Union[str, bytes], **options + ): + """Parses a response from the Redis server""" + try: + if NEVER_DECODE in options: + response = await connection.read_response(disable_decoding=True) + options.pop(NEVER_DECODE) + else: + response = await connection.read_response() + except ResponseError: + if EMPTY_RESPONSE in options: + return options[EMPTY_RESPONSE] + raise + + if EMPTY_RESPONSE in options: + options.pop(EMPTY_RESPONSE) + + # Remove keys entry, it needs only for cache. + options.pop("keys", None) + + if command_name in self.response_callbacks: + # Mypy bug: https://github.com/python/mypy/issues/10977 + command_name = cast(str, command_name) + retval = self.response_callbacks[command_name](response, **options) + return await retval if inspect.isawaitable(retval) else retval + return response + + +StrictRedis = Redis + + +class MonitorCommandInfo(TypedDict): + time: float + db: int + client_address: str + client_port: str + client_type: str + command: str + + +class Monitor: + """ + Monitor is useful for handling the MONITOR command to the redis server. + next_command() method returns one command from monitor + listen() method yields commands from monitor. + """ + + monitor_re = re.compile(r"\[(\d+) (.*?)\] (.*)") + command_re = re.compile(r'"(.*?)(? MonitorCommandInfo: + """Parse the response from a monitor command""" + await self.connect() + response = await self.connection.read_response() + if isinstance(response, bytes): + response = self.connection.encoder.decode(response, force=True) + command_time, command_data = response.split(" ", 1) + m = self.monitor_re.match(command_data) + db_id, client_info, command = m.groups() + command = " ".join(self.command_re.findall(command)) + # Redis escapes double quotes because each piece of the command + # string is surrounded by double quotes. We don't have that + # requirement so remove the escaping and leave the quote. + command = command.replace('\\"', '"') + + if client_info == "lua": + client_address = "lua" + client_port = "" + client_type = "lua" + elif client_info.startswith("unix"): + client_address = "unix" + client_port = client_info[5:] + client_type = "unix" + else: + # use rsplit as ipv6 addresses contain colons + client_address, client_port = client_info.rsplit(":", 1) + client_type = "tcp" + return { + "time": float(command_time), + "db": int(db_id), + "client_address": client_address, + "client_port": client_port, + "client_type": client_type, + "command": command, + } + + async def listen(self) -> AsyncIterator[MonitorCommandInfo]: + """Listen for commands coming to the server.""" + while True: + yield await self.next_command() + + +class PubSub: + """ + PubSub provides publish, subscribe and listen support to Redis channels. + + After subscribing to one or more channels, the listen() method will block + until a message arrives on one of the subscribed channels. That message + will be returned and it's safe to start listening again. + """ + + PUBLISH_MESSAGE_TYPES = ("message", "pmessage") + UNSUBSCRIBE_MESSAGE_TYPES = ("unsubscribe", "punsubscribe") + HEALTH_CHECK_MESSAGE = "redis-py-health-check" + + def __init__( + self, + connection_pool: ConnectionPool, + shard_hint: Optional[str] = None, + ignore_subscribe_messages: bool = False, + encoder=None, + push_handler_func: Optional[Callable] = None, + event_dispatcher: Optional["EventDispatcher"] = None, + ): + if event_dispatcher is None: + self._event_dispatcher = EventDispatcher() + else: + self._event_dispatcher = event_dispatcher + self.connection_pool = connection_pool + self.shard_hint = shard_hint + self.ignore_subscribe_messages = ignore_subscribe_messages + self.connection = None + # we need to know the encoding options for this connection in order + # to lookup channel and pattern names for callback handlers. + self.encoder = encoder + self.push_handler_func = push_handler_func + if self.encoder is None: + self.encoder = self.connection_pool.get_encoder() + if self.encoder.decode_responses: + self.health_check_response = [ + ["pong", self.HEALTH_CHECK_MESSAGE], + self.HEALTH_CHECK_MESSAGE, + ] + else: + self.health_check_response = [ + [b"pong", self.encoder.encode(self.HEALTH_CHECK_MESSAGE)], + self.encoder.encode(self.HEALTH_CHECK_MESSAGE), + ] + if self.push_handler_func is None: + _set_info_logger() + self.channels = {} + self.pending_unsubscribe_channels = set() + self.patterns = {} + self.pending_unsubscribe_patterns = set() + self._lock = anyio.Lock() + + async def __aenter__(self): + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() + + def __del__(self): + if self.connection: + self.connection.deregister_connect_callback(self.on_connect) + + async def aclose(self): + # In case a connection property does not yet exist + # (due to a crash earlier in the Redis() constructor), return + # immediately as there is nothing to clean-up. + if not hasattr(self, "connection"): + return + async with self._lock: + if self.connection: + await self.connection.disconnect() + self.connection.deregister_connect_callback(self.on_connect) + await self.connection_pool.release(self.connection) + self.connection = None + self.channels = {} + self.pending_unsubscribe_channels = set() + self.patterns = {} + self.pending_unsubscribe_patterns = set() + + @deprecated_function(version="5.0.1", reason="Use aclose() instead", name="close") + async def close(self) -> None: + """Alias for aclose(), for backwards compatibility""" + await self.aclose() + + @deprecated_function(version="5.0.1", reason="Use aclose() instead", name="reset") + async def reset(self) -> None: + """Alias for aclose(), for backwards compatibility""" + await self.aclose() + + async def on_connect(self, connection: Connection): + """Re-subscribe to any channels and patterns previously subscribed to""" + # NOTE: for python3, we can't pass bytestrings as keyword arguments + # so we need to decode channel/pattern names back to unicode strings + # before passing them to [p]subscribe. + self.pending_unsubscribe_channels.clear() + self.pending_unsubscribe_patterns.clear() + if self.channels: + channels = {} + for k, v in self.channels.items(): + channels[self.encoder.decode(k, force=True)] = v + await self.subscribe(**channels) + if self.patterns: + patterns = {} + for k, v in self.patterns.items(): + patterns[self.encoder.decode(k, force=True)] = v + await self.psubscribe(**patterns) + + @property + def subscribed(self): + """Indicates if there are subscriptions to any channels or patterns""" + return bool(self.channels or self.patterns) + + async def execute_command(self, *args: EncodableT): + """Execute a publish/subscribe command""" + + # NOTE: don't parse the response in this function -- it could pull a + # legitimate message off the stack if the connection is already + # subscribed to one or more channels + + await self.connect() + connection = self.connection + kwargs = {"check_health": not self.subscribed} + await self._execute(connection, connection.send_command, *args, **kwargs) + + async def connect(self): + """ + Ensure that the PubSub is connected + """ + if self.connection is None: + self.connection = await self.connection_pool.get_connection() + # register a callback that re-subscribes to any channels we + # were listening to when we were disconnected + self.connection.register_connect_callback(self.on_connect) + else: + await self.connection.connect() + if self.push_handler_func is not None and not HIREDIS_AVAILABLE: + self.connection._parser.set_pubsub_push_handler(self.push_handler_func) + + self._event_dispatcher.dispatch( + AfterPubSubConnectionInstantiationEvent( + self.connection, self.connection_pool, ClientType.ASYNC, self._lock + ) + ) + + async def _disconnect_raise_connect(self, conn, error): + """ + Close the connection and raise an exception + if retry_on_error is not set or the error is not one + of the specified error types. Otherwise, try to + reconnect + """ + await conn.disconnect() + if ( + conn.retry_on_error is None + or isinstance(error, tuple(conn.retry_on_error)) is False + ): + raise error + await conn.connect() + + async def _execute(self, conn, command, *args, **kwargs): + """ + Connect manually upon disconnection. If the Redis server is down, + this will fail and raise a ConnectionError as desired. + After reconnection, the ``on_connect`` callback should have been + called by the # connection to resubscribe us to any channels and + patterns we were previously listening to + """ + return await conn.retry.call_with_retry( + lambda: command(*args, **kwargs), + lambda error: self._disconnect_raise_connect(conn, error), + ) + + async def parse_response(self, block: bool = True, timeout: float = 0): + """Parse the response from a publish/subscribe command""" + conn = self.connection + if conn is None: + raise RuntimeError( + "pubsub connection not set: " + "did you forget to call subscribe() or psubscribe()?" + ) + + await self.check_health() + + if not conn.is_connected: + await conn.connect() + + read_timeout = None if block else timeout + response = await self._execute( + conn, + conn.read_response, + timeout=read_timeout, + disconnect_on_error=False, + push_request=True, + ) + + if conn.health_check_interval and response in self.health_check_response: + # ignore the health check message as user might not expect it + return None + return response + + async def check_health(self): + conn = self.connection + if conn is None: + raise RuntimeError( + "pubsub connection not set: " + "did you forget to call subscribe() or psubscribe()?" + ) + + if conn.health_check_interval and anyio.current_time() > conn.next_health_check: + await conn.send_command( + "PING", self.HEALTH_CHECK_MESSAGE, check_health=False + ) + + def _normalize_keys(self, data: _NormalizeKeysT) -> _NormalizeKeysT: + """ + normalize channel/pattern names to be either bytes or strings + based on whether responses are automatically decoded. this saves us + from coercing the value for each message coming in. + """ + encode = self.encoder.encode + decode = self.encoder.decode + return {decode(encode(k)): v for k, v in data.items()} # type: ignore[return-value] # noqa: E501 + + async def psubscribe(self, *args: ChannelT, **kwargs: PubSubHandler): + """ + Subscribe to channel patterns. Patterns supplied as keyword arguments + expect a pattern name as the key and a callable as the value. A + pattern's callable will be invoked automatically when a message is + received on that pattern rather than producing a message via + ``listen()``. + """ + parsed_args = list_or_args((args[0],), args[1:]) if args else args + new_patterns: Dict[ChannelT, PubSubHandler] = dict.fromkeys(parsed_args) + # Mypy bug: https://github.com/python/mypy/issues/10970 + new_patterns.update(kwargs) # type: ignore[arg-type] + ret_val = await self.execute_command("PSUBSCRIBE", *new_patterns.keys()) + # update the patterns dict AFTER we send the command. we don't want to + # subscribe twice to these patterns, once for the command and again + # for the reconnection. + new_patterns = self._normalize_keys(new_patterns) + self.patterns.update(new_patterns) + self.pending_unsubscribe_patterns.difference_update(new_patterns) + return ret_val + + def punsubscribe(self, *args: ChannelT) -> Awaitable: + """ + Unsubscribe from the supplied patterns. If empty, unsubscribe from + all patterns. + """ + patterns: Iterable[ChannelT] + if args: + parsed_args = list_or_args((args[0],), args[1:]) + patterns = self._normalize_keys(dict.fromkeys(parsed_args)).keys() + else: + parsed_args = [] + patterns = self.patterns + self.pending_unsubscribe_patterns.update(patterns) + return self.execute_command("PUNSUBSCRIBE", *parsed_args) + + async def subscribe(self, *args: ChannelT, **kwargs: Callable): + """ + Subscribe to channels. Channels supplied as keyword arguments expect + a channel name as the key and a callable as the value. A channel's + callable will be invoked automatically when a message is received on + that channel rather than producing a message via ``listen()`` or + ``get_message()``. + """ + parsed_args = list_or_args((args[0],), args[1:]) if args else () + new_channels = dict.fromkeys(parsed_args) + # Mypy bug: https://github.com/python/mypy/issues/10970 + new_channels.update(kwargs) # type: ignore[arg-type] + ret_val = await self.execute_command("SUBSCRIBE", *new_channels.keys()) + # update the channels dict AFTER we send the command. we don't want to + # subscribe twice to these channels, once for the command and again + # for the reconnection. + new_channels = self._normalize_keys(new_channels) + self.channels.update(new_channels) + self.pending_unsubscribe_channels.difference_update(new_channels) + return ret_val + + def unsubscribe(self, *args) -> Awaitable: + """ + Unsubscribe from the supplied channels. If empty, unsubscribe from + all channels + """ + if args: + parsed_args = list_or_args(args[0], args[1:]) + channels = self._normalize_keys(dict.fromkeys(parsed_args)) + else: + parsed_args = [] + channels = self.channels + self.pending_unsubscribe_channels.update(channels) + return self.execute_command("UNSUBSCRIBE", *parsed_args) + + async def listen(self) -> AsyncIterator: + """Listen for messages on channels this client has been subscribed to""" + while self.subscribed: + response = await self.handle_message(await self.parse_response(block=True)) + if response is not None: + yield response + + async def get_message( + self, ignore_subscribe_messages: bool = False, timeout: Optional[float] = None + ): + """ + Get the next message if one is available, otherwise None. + + If timeout is specified, the system will wait for `timeout` seconds + before returning. Timeout should be specified as a floating point + number or None to wait indefinitely. + """ + response = await self.parse_response(block=(timeout is None), timeout=timeout) + if response: + return await self.handle_message(response, ignore_subscribe_messages) + return None + + def ping(self, message=None) -> Awaitable: + """ + Ping the Redis server + """ + args = ["PING", message] if message is not None else ["PING"] + return self.execute_command(*args) + + async def handle_message(self, response, ignore_subscribe_messages=False): + """ + Parses a pub/sub message. If the channel or pattern was subscribed to + with a message handler, the handler is invoked instead of a parsed + message being returned. + """ + if response is None: + return None + if isinstance(response, bytes): + response = [b"pong", response] if response != b"PONG" else [b"pong", b""] + message_type = str_if_bytes(response[0]) + if message_type == "pmessage": + message = { + "type": message_type, + "pattern": response[1], + "channel": response[2], + "data": response[3], + } + elif message_type == "pong": + message = { + "type": message_type, + "pattern": None, + "channel": None, + "data": response[1], + } + else: + message = { + "type": message_type, + "pattern": None, + "channel": response[1], + "data": response[2], + } + + # if this is an unsubscribe message, remove it from memory + if message_type in self.UNSUBSCRIBE_MESSAGE_TYPES: + if message_type == "punsubscribe": + pattern = response[1] + if pattern in self.pending_unsubscribe_patterns: + self.pending_unsubscribe_patterns.remove(pattern) + self.patterns.pop(pattern, None) + else: + channel = response[1] + if channel in self.pending_unsubscribe_channels: + self.pending_unsubscribe_channels.remove(channel) + self.channels.pop(channel, None) + + if message_type in self.PUBLISH_MESSAGE_TYPES: + # if there's a message handler, invoke it + if message_type == "pmessage": + handler = self.patterns.get(message["pattern"], None) + else: + handler = self.channels.get(message["channel"], None) + if handler: + if inspect.iscoroutinefunction(handler): + await handler(message) + else: + handler(message) + return None + elif message_type != "pong": + # this is a subscribe/unsubscribe message. ignore if we don't + # want them + if ignore_subscribe_messages or self.ignore_subscribe_messages: + return None + + return message + + async def run( + self, + *, + exception_handler: Optional["PSWorkerThreadExcHandlerT"] = None, + poll_timeout: float = 1.0, + ) -> None: + """Process pub/sub messages using registered callbacks. + + This is the equivalent of :py:meth:`redis.PubSub.run_in_thread` in + redis-py, but it is a coroutine. To launch it as a separate task, use a task + group: + + >>> async with anyio.create_task_group() as tg: + tg.start_soon(pubsub.run) + + To shut it down, cancel the task group: + + >>> tg.cancel_scope.cancel() + """ + for channel, handler in self.channels.items(): + if handler is None: + raise PubSubError(f"Channel: '{channel}' has no handler registered") + for pattern, handler in self.patterns.items(): + if handler is None: + raise PubSubError(f"Pattern: '{pattern}' has no handler registered") + + await self.connect() + while True: + try: + await self.get_message( + ignore_subscribe_messages=True, timeout=poll_timeout + ) + except anyio.get_cancelled_exc_class(): + raise + except BaseException as e: + if exception_handler is None: + raise + res = exception_handler(e, self) + if inspect.isawaitable(res): + await res + # Ensure that other tasks on the event loop get a chance to run + # if we didn't have to block for I/O anywhere. + await anyio.lowlevel.checkpoint() + + +class PubsubWorkerExceptionHandler(Protocol): + def __call__(self, e: BaseException, pubsub: PubSub): ... + + +class AsyncPubsubWorkerExceptionHandler(Protocol): + async def __call__(self, e: BaseException, pubsub: PubSub): ... + + +PSWorkerThreadExcHandlerT = Union[ + PubsubWorkerExceptionHandler, AsyncPubsubWorkerExceptionHandler +] + + +CommandT = Tuple[Tuple[Union[str, bytes], ...], Mapping[str, Any]] +CommandStackT = List[CommandT] + + +class Pipeline(Redis): # lgtm [py/init-calls-subclass] + """ + Pipelines provide a way to transmit multiple commands to the Redis server + in one transmission. This is convenient for batch processing, such as + saving all the values in a list to Redis. + + All commands executed within a pipeline are wrapped with MULTI and EXEC + calls. This guarantees all commands executed in the pipeline will be + executed atomically. + + Any command raising an exception does *not* halt the execution of + subsequent commands in the pipeline. Instead, the exception is caught + and its instance is placed into the response list returned by execute(). + Code iterating over the response list should be able to deal with an + instance of an exception as a potential value. In general, these will be + ResponseError exceptions, such as those raised when issuing a command + on a key of a different datatype. + """ + + UNWATCH_COMMANDS = {"DISCARD", "EXEC", "UNWATCH"} + + def __init__( + self, + connection_pool: ConnectionPool, + response_callbacks: MutableMapping[Union[str, bytes], ResponseCallbackT], + transaction: bool, + shard_hint: Optional[str], + ): + self.connection_pool = connection_pool + self.connection = None + self.response_callbacks = response_callbacks + self.is_transaction = transaction + self.shard_hint = shard_hint + self.watching = False + self.command_stack: CommandStackT = [] + self.scripts: Set["Script"] = set() + self.explicit_transaction = False + + async def __aenter__(self: _RedisT) -> _RedisT: + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.reset() + + def __await__(self): + return self._async_self().__await__() + + _DEL_MESSAGE = "Unclosed Pipeline client" + + def __len__(self): + return len(self.command_stack) + + def __bool__(self): + """Pipeline instances should always evaluate to True""" + return True + + async def _async_self(self): + return self + + async def reset(self): + self.command_stack = [] + self.scripts = set() + # make sure to reset the connection state in the event that we were + # watching something + if self.watching and self.connection: + try: + # call this manually since our unwatch or + # immediate_execute_command methods can call reset() + await self.connection.send_command("UNWATCH") + await self.connection.read_response() + except ConnectionError: + # disconnect will also remove any previous WATCHes + if self.connection: + await self.connection.disconnect() + # clean up the other instance attributes + self.watching = False + self.explicit_transaction = False + # we can safely return the connection to the pool here since we're + # sure we're no longer WATCHing anything + if self.connection: + await self.connection_pool.release(self.connection) + self.connection = None + + async def aclose(self) -> None: + """Alias for reset(), a standard method name for cleanup""" + await self.reset() + + def multi(self): + """ + Start a transactional block of the pipeline after WATCH commands + are issued. End the transactional block with `execute`. + """ + if self.explicit_transaction: + raise RedisError("Cannot issue nested calls to MULTI") + if self.command_stack: + raise RedisError( + "Commands without an initial WATCH have already been issued" + ) + self.explicit_transaction = True + + def execute_command( + self, *args, **kwargs + ) -> Union["Pipeline", Awaitable["Pipeline"]]: + if (self.watching or args[0] == "WATCH") and not self.explicit_transaction: + return self.immediate_execute_command(*args, **kwargs) + return self.pipeline_execute_command(*args, **kwargs) + + async def _disconnect_reset_raise(self, conn, error): + """ + Close the connection, reset watching state and + raise an exception if we were watching, + if retry_on_error is not set or the error is not one + of the specified error types. + """ + await conn.disconnect() + # if we were already watching a variable, the watch is no longer + # valid since this connection has died. raise a WatchError, which + # indicates the user should retry this transaction. + if self.watching: + await self.aclose() + raise WatchError( + "A ConnectionError occurred on while watching one or more keys" + ) + # if retry_on_error is not set or the error is not one + # of the specified error types, raise it + if ( + conn.retry_on_error is None + or isinstance(error, tuple(conn.retry_on_error)) is False + ): + await self.aclose() + raise + + async def immediate_execute_command(self, *args, **options): + """ + Execute a command immediately, but don't auto-retry on a + ConnectionError if we're already WATCHing a variable. Used when + issuing WATCH or subsequent commands retrieving their values but before + MULTI is called. + """ + command_name = args[0] + conn = self.connection + # if this is the first call, we need a connection + if not conn: + conn = await self.connection_pool.get_connection() + self.connection = conn + + return await conn.retry.call_with_retry( + lambda: self._send_command_parse_response( + conn, command_name, *args, **options + ), + lambda error: self._disconnect_reset_raise(conn, error), + ) + + def pipeline_execute_command(self, *args, **options): + """ + Stage a command to be executed when execute() is next called + + Returns the current Pipeline object back so commands can be + chained together, such as: + + pipe = pipe.set('foo', 'bar').incr('baz').decr('bang') + + At some other point, you can then run: pipe.execute(), + which will execute all commands queued in the pipe. + """ + self.command_stack.append((args, options)) + return self + + async def _execute_transaction( # noqa: C901 + self, connection: Connection, commands: CommandStackT, raise_on_error + ): + pre: CommandT = (("MULTI",), {}) + post: CommandT = (("EXEC",), {}) + cmds = (pre, *commands, post) + all_cmds = connection.pack_commands( + args for args, options in cmds if EMPTY_RESPONSE not in options + ) + await connection.send_packed_command(all_cmds) + errors = [] + + # parse off the response for MULTI + # NOTE: we need to handle ResponseErrors here and continue + # so that we read all the additional command messages from + # the socket + try: + await self.parse_response(connection, "_") + except ResponseError as err: + errors.append((0, err)) + + # and all the other commands + for i, command in enumerate(commands): + if EMPTY_RESPONSE in command[1]: + errors.append((i, command[1][EMPTY_RESPONSE])) + else: + try: + await self.parse_response(connection, "_") + except ResponseError as err: + self.annotate_exception(err, i + 1, command[0]) + errors.append((i, err)) + + # parse the EXEC. + try: + response = await self.parse_response(connection, "_") + except ExecAbortError as err: + if errors: + raise errors[0][1] from err + raise + + # EXEC clears any watched keys + self.watching = False + + if response is None: + raise WatchError("Watched variable changed.") from None + + # put any parse errors into the response + for i, e in errors: + response.insert(i, e) + + if len(response) != len(commands): + if self.connection: + await self.connection.disconnect() + raise ResponseError( + "Wrong number of response items from pipeline execution" + ) from None + + # find any errors in the response and raise if necessary + if raise_on_error: + self.raise_first_error(commands, response) + + # We have to run response callbacks manually + data = [] + for r, cmd in zip(response, commands): + if not isinstance(r, Exception): + args, options = cmd + command_name = args[0] + + # Remove keys entry, it needs only for cache. + options.pop("keys", None) + + if command_name in self.response_callbacks: + r = self.response_callbacks[command_name](r, **options) + if inspect.isawaitable(r): + r = await r + data.append(r) + return data + + async def _execute_pipeline( + self, connection: Connection, commands: CommandStackT, raise_on_error: bool + ): + # build up all commands into a single request to increase network perf + all_cmds = connection.pack_commands([args for args, _ in commands]) + await connection.send_packed_command(all_cmds) + + response = [] + for args, options in commands: + try: + response.append( + await self.parse_response(connection, args[0], **options) + ) + except ResponseError as e: + response.append(e) + + if raise_on_error: + self.raise_first_error(commands, response) + return response + + def raise_first_error(self, commands: CommandStackT, response: Iterable[Any]): + for i, r in enumerate(response): + if isinstance(r, ResponseError): + self.annotate_exception(r, i + 1, commands[i][0]) + raise r + + def annotate_exception( + self, exception: Exception, number: int, command: Iterable[object] + ) -> None: + cmd = " ".join(map(safe_str, command)) + msg = f"Command # {number} ({cmd}) of pipeline caused error: {exception.args}" + exception.args = (msg,) + exception.args[1:] + + async def parse_response( + self, connection: Connection, command_name: Union[str, bytes], **options + ): + result = await super().parse_response(connection, command_name, **options) + if command_name in self.UNWATCH_COMMANDS: + self.watching = False + elif command_name == "WATCH": + self.watching = True + return result + + async def load_scripts(self): + # make sure all scripts that are about to be run on this pipeline exist + scripts = list(self.scripts) + immediate = self.immediate_execute_command + shas = [s.sha for s in scripts] + # we can't use the normal script_* methods because they would just + # get buffered in the pipeline. + exists = await immediate("SCRIPT EXISTS", *shas) + if not all(exists): + for s, exist in zip(scripts, exists): + if not exist: + s.sha = await immediate("SCRIPT LOAD", s.script) + + async def _disconnect_raise_reset(self, conn: Connection, error: Exception): + """ + Close the connection, raise an exception if we were watching, + and raise an exception if retry_on_error is not set or the + error is not one of the specified error types. + """ + await conn.disconnect() + # if we were watching a variable, the watch is no longer valid + # since this connection has died. raise a WatchError, which + # indicates the user should retry this transaction. + if self.watching: + raise WatchError( + "A ConnectionError occurred on while watching one or more keys" + ) + # if retry_on_error is not set or the error is not one + # of the specified error types, raise it + if ( + conn.retry_on_error is None + or isinstance(error, tuple(conn.retry_on_error)) is False + ): + await self.reset() + raise + + async def execute(self, raise_on_error: bool = True) -> List[Any]: + """Execute all the commands in the current pipeline""" + stack = self.command_stack + if not stack and not self.watching: + return [] + if self.scripts: + await self.load_scripts() + if self.is_transaction or self.explicit_transaction: + execute = self._execute_transaction + else: + execute = self._execute_pipeline + + conn = self.connection + if not conn: + conn = await self.connection_pool.get_connection() + # assign to self.connection so reset() releases the connection + # back to the pool after we're done + self.connection = conn + conn = cast(Connection, conn) + + try: + return await conn.retry.call_with_retry( + lambda: execute(conn, stack, raise_on_error), + lambda error: self._disconnect_raise_reset(conn, error), + ) + finally: + await self.reset() + + async def discard(self): + """Flushes all previously queued commands + See: https://redis.io/commands/DISCARD + """ + await self.execute_command("DISCARD") + + async def watch(self, *names: KeyT): + """Watches the values at keys ``names``""" + if self.explicit_transaction: + raise RedisError("Cannot issue a WATCH after a MULTI") + return await self.execute_command("WATCH", *names) + + async def unwatch(self): + """Unwatches all previously specified keys""" + return self.watching and await self.execute_command("UNWATCH") or True diff --git a/redis/anyio/cluster.py b/redis/anyio/cluster.py new file mode 100644 index 0000000000..ce8f83e303 --- /dev/null +++ b/redis/anyio/cluster.py @@ -0,0 +1,1720 @@ +from __future__ import annotations + +import collections +import random +import socket +import sys +import warnings +from contextlib import AsyncExitStack +from types import TracebackType +from typing import ( + Any, + Callable, + Deque, + Dict, + List, + Mapping, + Optional, + Tuple, + Type, + TypeVar, + Union, +) + +import anyio + +from redis._parsers import AsyncCommandsParser, Encoder +from redis._parsers.helpers import ( + _RedisCallbacks, + _RedisCallbacksRESP2, + _RedisCallbacksRESP3, +) +from redis.anyio._commands.cluster import AsyncRedisClusterCommands +from redis.anyio.client import ResponseCallbackT +from redis.anyio.connection import Connection, SSLConnection, parse_url +from redis.anyio.lock import Lock +from redis.anyio.retry import Retry +from redis.anyio.utils import gather +from redis.auth.token import TokenInterface +from redis.backoff import default_backoff +from redis.client import EMPTY_RESPONSE, NEVER_DECODE, AbstractRedis +from redis.cluster import ( + PIPELINE_BLOCKED_COMMANDS, + PRIMARY, + REPLICA, + SLOT_ID, + AbstractRedisCluster, + LoadBalancer, + LoadBalancingStrategy, + block_pipeline_command, + get_node_name, + parse_cluster_slots, +) +from redis.commands import READ_COMMANDS +from redis.crc import REDIS_CLUSTER_HASH_SLOTS, key_slot +from redis.credentials import CredentialProvider +from redis.event import AfterAsyncClusterInstantiationEvent, EventDispatcher +from redis.exceptions import ( + AskError, + BusyLoadingError, + ClusterDownError, + ClusterError, + ConnectionError, + DataError, + MaxConnectionsError, + MovedError, + RedisClusterException, + RedisError, + ResponseError, + SlotNotCoveredError, + TimeoutError, + TryAgainError, +) +from redis.typing import AnyKeyT, EncodableT, KeyT +from redis.utils import ( + SSL_AVAILABLE, + deprecated_args, + get_lib_version, + safe_str, + str_if_bytes, + truncate_text, +) + +if sys.version_info >= (3, 11): + from typing import Self +else: + from exceptiongroup import ExceptionGroup + from typing_extensions import Self + +if SSL_AVAILABLE: + from ssl import TLSVersion, VerifyMode +else: + TLSVersion = None + VerifyMode = None + +TargetNodesT = TypeVar( + "TargetNodesT", str, "ClusterNode", List["ClusterNode"], Dict[Any, "ClusterNode"] +) + + +class RedisCluster(AbstractRedis, AbstractRedisCluster, AsyncRedisClusterCommands): + """ + Create a new RedisCluster client. + + Pass one of parameters: + + - `host` & `port` + - `startup_nodes` + + | Use this object as an async context manager to initialize it and then close the + | connections. + + Many commands support the target_nodes kwarg. It can be one of the + :attr:`NODE_FLAGS`: + + - :attr:`PRIMARIES` + - :attr:`REPLICAS` + - :attr:`ALL_NODES` + - :attr:`RANDOM` + - :attr:`DEFAULT_NODE` + + Note: This client is not thread/process/fork safe. + + :param host: + | Can be used to point to a startup node + :param port: + | Port used if **host** is provided + :param startup_nodes: + | :class:`~.ClusterNode` to used as a startup node + :param require_full_coverage: + | When set to ``False``: the client will not require a full coverage of + the slots. However, if not all slots are covered, and at least one node + has ``cluster-require-full-coverage`` set to ``yes``, the server will throw + a :class:`~.ClusterDownError` for some key-based commands. + | When set to ``True``: all slots must be covered to construct the cluster + client. If not all slots are covered, :class:`~.RedisClusterException` will be + thrown. + | See: + https://redis.io/docs/manual/scaling/#redis-cluster-configuration-parameters + :param read_from_replicas: + | @deprecated - please use load_balancing_strategy instead + | Enable read from replicas in READONLY mode. + When set to true, read commands will be assigned between the primary and + its replications in a Round-Robin manner. + The data read from replicas is eventually consistent with the data in primary nodes. + :param load_balancing_strategy: + | Enable read from replicas in READONLY mode and defines the load balancing + strategy that will be used for cluster node selection. + The data read from replicas is eventually consistent with the data in primary nodes. + :param reinitialize_steps: + | Specifies the number of MOVED errors that need to occur before reinitializing + the whole cluster topology. If a MOVED error occurs and the cluster does not + need to be reinitialized on this current error handling, only the MOVED slot + will be patched with the redirected node. + To reinitialize the cluster on every MOVED error, set reinitialize_steps to 1. + To avoid reinitializing the cluster on moved errors, set reinitialize_steps to + 0. + :param cluster_error_retry_attempts: + | Number of times to retry before raising an error when :class:`~.TimeoutError` + or :class:`~.ConnectionError` or :class:`~.ClusterDownError` are encountered + :param connection_error_retry_attempts: + | Number of times to retry before reinitializing when :class:`~.TimeoutError` + or :class:`~.ConnectionError` are encountered. + The default backoff strategy will be set if Retry object is not passed (see + default_backoff in backoff.py). To change it, pass a custom Retry object + using the "retry" keyword. + :param max_connections: + | Maximum number of connections per node. If there are no free connections & the + maximum number of connections are already created, a + :class:`~.MaxConnectionsError` is raised. This error may be retried as defined + by :attr:`connection_error_retry_attempts` + :param address_remap: + | An optional callable which, when provided with an internal network + address of a node, e.g. a `(host, port)` tuple, will return the address + where the node is reachable. This can be used to map the addresses at + which the nodes _think_ they are, to addresses at which a client may + reach them, such as when they sit behind a proxy. + + | Rest of the arguments will be passed to the + :class:`~redis.anyio.connection.Connection` instances when created + + :raises RedisClusterException: + if any arguments are invalid or unknown. Eg: + + - `db` != 0 or None + - `path` argument for unix socket connection + - none of the `host`/`port` & `startup_nodes` were provided + + """ + + @classmethod + def from_url(cls, url: str, **kwargs: Any) -> "RedisCluster": + """ + Return a Redis client object configured from the given URL. + + For example:: + + redis://[[username]:[password]]@localhost:6379/0 + rediss://[[username]:[password]]@localhost:6379/0 + + Three URL schemes are supported: + + - `redis://` creates a TCP socket connection. See more at: + + - `rediss://` creates a SSL wrapped TCP socket connection. See more at: + + + The username, password, hostname, path and all querystring values are passed + through ``urllib.parse.unquote`` in order to replace any percent-encoded values + with their corresponding characters. + + All querystring options are cast to their appropriate Python types. Boolean + arguments can be specified with string values "True"/"False" or "Yes"/"No". + Values that cannot be properly cast cause a ``ValueError`` to be raised. Once + parsed, the querystring arguments and keyword arguments are passed to + :class:`~redis.anyio.connection.Connection` when created. + In the case of conflicting arguments, querystring arguments are used. + """ + kwargs.update(parse_url(url)) + if kwargs.pop("connection_class", None) is SSLConnection: + kwargs["ssl"] = True + return cls(**kwargs) + + __slots__ = ( + "_initialize", + "_lock", + "_exit_stack", + "cluster_error_retry_attempts", + "command_flags", + "commands_parser", + "connection_error_retry_attempts", + "connection_kwargs", + "encoder", + "node_flags", + "nodes_manager", + "read_from_replicas", + "reinitialize_counter", + "reinitialize_steps", + "response_callbacks", + "result_callbacks", + ) + + @deprecated_args( + args_to_warn=["read_from_replicas"], + reason="Please configure the 'load_balancing_strategy' instead", + version="5.0.3", + ) + def __init__( + self, + host: Optional[str] = None, + port: Union[str, int] = 6379, + # Cluster related kwargs + startup_nodes: Optional[List["ClusterNode"]] = None, + require_full_coverage: bool = True, + read_from_replicas: bool = False, + load_balancing_strategy: Optional[LoadBalancingStrategy] = None, + reinitialize_steps: int = 5, + cluster_error_retry_attempts: int = 3, + connection_error_retry_attempts: int = 3, + max_connections: int = 2**31, + # Client related kwargs + db: Union[str, int] = 0, + path: Optional[str] = None, + credential_provider: Optional[CredentialProvider] = None, + username: Optional[str] = None, + password: Optional[str] = None, + client_name: Optional[str] = None, + lib_name: Optional[str] = "redis-py", + lib_version: Optional[str] = get_lib_version(), + # Encoding related kwargs + encoding: str = "utf-8", + encoding_errors: str = "strict", + decode_responses: bool = False, + # Connection related kwargs + health_check_interval: float = 0, + socket_connect_timeout: Optional[float] = None, + socket_keepalive: bool = False, + socket_keepalive_options: Optional[Mapping[int, Union[int, bytes]]] = None, + socket_timeout: Optional[float] = None, + retry: Optional["Retry"] = None, + retry_on_error: Optional[List[Type[Exception]]] = None, + # SSL related kwargs + ssl: bool = False, + ssl_ca_certs: Optional[str] = None, + ssl_ca_data: Optional[str] = None, + ssl_cert_reqs: Union[str, VerifyMode] = "required", + ssl_certfile: Optional[str] = None, + ssl_check_hostname: bool = False, + ssl_keyfile: Optional[str] = None, + ssl_min_version: Optional[TLSVersion] = None, + ssl_ciphers: Optional[str] = None, + protocol: Optional[int] = 2, + address_remap: Optional[Callable[[Tuple[str, int]], Tuple[str, int]]] = None, + event_dispatcher: Optional[EventDispatcher] = None, + ) -> None: + if db: + raise RedisClusterException( + "Argument 'db' must be 0 or None in cluster mode" + ) + + if path: + raise RedisClusterException( + "Unix domain socket is not supported in cluster mode" + ) + + if (not host or not port) and not startup_nodes: + raise RedisClusterException( + "RedisCluster requires at least one node to discover the cluster.\n" + "Please provide one of the following or use RedisCluster.from_url:\n" + ' - host and port: RedisCluster(host="localhost", port=6379)\n' + " - startup_nodes: RedisCluster(startup_nodes=[" + 'ClusterNode("localhost", 6379), ClusterNode("localhost", 6380)])' + ) + + kwargs: Dict[str, Any] = { + "max_connections": max_connections, + "connection_class": Connection, + # Client related kwargs + "credential_provider": credential_provider, + "username": username, + "password": password, + "client_name": client_name, + "lib_name": lib_name, + "lib_version": lib_version, + # Encoding related kwargs + "encoding": encoding, + "encoding_errors": encoding_errors, + "decode_responses": decode_responses, + # Connection related kwargs + "health_check_interval": health_check_interval, + "socket_connect_timeout": socket_connect_timeout, + "socket_keepalive": socket_keepalive, + "socket_keepalive_options": socket_keepalive_options, + "socket_timeout": socket_timeout, + "retry": retry, + "protocol": protocol, + } + + if ssl: + # SSL related kwargs + kwargs.update( + { + "connection_class": SSLConnection, + "ssl_ca_certs": ssl_ca_certs, + "ssl_ca_data": ssl_ca_data, + "ssl_cert_reqs": ssl_cert_reqs, + "ssl_certfile": ssl_certfile, + "ssl_check_hostname": ssl_check_hostname, + "ssl_keyfile": ssl_keyfile, + "ssl_min_version": ssl_min_version, + "ssl_ciphers": ssl_ciphers, + } + ) + + if read_from_replicas or load_balancing_strategy: + # Call our on_connect function to configure READONLY mode + kwargs["redis_connect_func"] = self.on_connect + + self.retry = retry + if retry or retry_on_error or connection_error_retry_attempts > 0: + # Set a retry object for all cluster nodes + self.retry = retry or Retry( + default_backoff(), connection_error_retry_attempts + ) + if not retry_on_error: + # Default errors for retrying + retry_on_error = [ConnectionError, TimeoutError] + self.retry.update_supported_errors(retry_on_error) + kwargs.update({"retry": self.retry}) + + kwargs["response_callbacks"] = _RedisCallbacks.copy() + if kwargs.get("protocol") in ["3", 3]: + kwargs["response_callbacks"].update(_RedisCallbacksRESP3) + else: + kwargs["response_callbacks"].update(_RedisCallbacksRESP2) + self.connection_kwargs = kwargs + + if startup_nodes: + passed_nodes = [] + for node in startup_nodes: + passed_nodes.append( + ClusterNode(node.host, node.port, **self.connection_kwargs) + ) + startup_nodes = passed_nodes + else: + startup_nodes = [] + if host and port: + startup_nodes.append(ClusterNode(host, port, **self.connection_kwargs)) + + if event_dispatcher is None: + self._event_dispatcher = EventDispatcher() + else: + self._event_dispatcher = event_dispatcher + + self.nodes_manager = NodesManager( + startup_nodes, + require_full_coverage, + kwargs, + address_remap=address_remap, + event_dispatcher=self._event_dispatcher, + ) + self.encoder = Encoder(encoding, encoding_errors, decode_responses) + self.read_from_replicas = read_from_replicas + self.load_balancing_strategy = load_balancing_strategy + self.reinitialize_steps = reinitialize_steps + self.cluster_error_retry_attempts = cluster_error_retry_attempts + self.connection_error_retry_attempts = connection_error_retry_attempts + self.reinitialize_counter = 0 + self.commands_parser = AsyncCommandsParser() + self.node_flags = self.__class__.NODE_FLAGS.copy() + self.command_flags = self.__class__.COMMAND_FLAGS.copy() + self.response_callbacks = kwargs["response_callbacks"] + self.result_callbacks = self.__class__.RESULT_CALLBACKS.copy() + self.result_callbacks["CLUSTER SLOTS"] = ( + lambda cmd, res, **kwargs: parse_cluster_slots( + list(res.values())[0], **kwargs + ) + ) + + self._initialize = True + self._lock = anyio.Lock() + self._exit_stack: Optional[AsyncExitStack] = None + + async def initialize(self) -> "RedisCluster": + """Get all nodes from startup nodes & creates connections if not initialized.""" + if self._initialize: + async with self._lock: + if self._initialize: + try: + await self.nodes_manager.initialize() + await self.commands_parser.initialize( + self.nodes_manager.default_node + ) + self._initialize = False + except BaseException: + await self.nodes_manager.aclose() + await self.nodes_manager.aclose("startup_nodes") + raise + return self + + async def aclose(self) -> None: + """Close all connections & client if initialized.""" + if not self._initialize: + async with self._lock: + if not self._initialize: + self._initialize = True + await self.nodes_manager.aclose() + await self.nodes_manager.aclose("startup_nodes") + + async def __aenter__(self) -> Self: + # Get all nodes from startup nodes & create connections if not initialized + async with AsyncExitStack() as stack: + await stack.enter_async_context(self.nodes_manager) + await self.initialize() + stack.push_async_callback(self.aclose) + self._exit_stack = stack.pop_all() + + return self + + async def __aexit__( + self, + exc_type: type[BaseException] | None, + exc_value: BaseException | None, + exc_tb: TracebackType | None, + ) -> None: + exit_stack, self._exit_stack = self._exit_stack, None + await exit_stack.__aexit__(exc_type, exc_value, exc_tb) + + def __del__( + self, + _warn: Any = warnings.warn, + ) -> None: + if getattr(self, "_exit_stack", None) is not None: + _warn( + f"Unclosed RedisCluster client {self!r}", ResourceWarning, source=self + ) + + async def on_connect(self, connection: Connection) -> None: + await connection.on_connect() + + # Sending READONLY command to server to configure connection as + # readonly. Since each cluster node may change its server type due + # to a failover, we should establish a READONLY connection + # regardless of the server type. If this is a primary connection, + # READONLY would not affect executing write commands. + await connection.send_command("READONLY") + if str_if_bytes(await connection.read_response()) != "OK": + raise ConnectionError("READONLY command failed") + + def get_nodes(self) -> List["ClusterNode"]: + """Get all nodes of the cluster.""" + return list(self.nodes_manager.nodes_cache.values()) + + def get_primaries(self) -> List["ClusterNode"]: + """Get the primary nodes of the cluster.""" + return self.nodes_manager.get_nodes_by_server_type(PRIMARY) + + def get_replicas(self) -> List["ClusterNode"]: + """Get the replica nodes of the cluster.""" + return self.nodes_manager.get_nodes_by_server_type(REPLICA) + + def get_random_node(self) -> "ClusterNode": + """Get a random node of the cluster.""" + return random.choice(list(self.nodes_manager.nodes_cache.values())) + + def get_default_node(self) -> "ClusterNode": + """Get the default node of the client.""" + return self.nodes_manager.default_node + + def set_default_node(self, node: "ClusterNode") -> None: + """ + Set the default node of the client. + + :raises DataError: if None is passed or node does not exist in cluster. + """ + if not node or not self.get_node(node_name=node.name): + raise DataError("The requested node does not exist in the cluster.") + + self.nodes_manager.default_node = node + + def get_node( + self, + host: Optional[str] = None, + port: Optional[int] = None, + node_name: Optional[str] = None, + ) -> Optional["ClusterNode"]: + """Get node by (host, port) or node_name.""" + return self.nodes_manager.get_node(host, port, node_name) + + def get_node_from_key( + self, key: str, replica: bool = False + ) -> Optional["ClusterNode"]: + """ + Get the cluster node corresponding to the provided key. + + :param key: + :param replica: + | Indicates if a replica should be returned + | + None will returned if no replica holds this key + + :raises SlotNotCoveredError: if the key is not covered by any slot. + """ + slot = self.keyslot(key) + slot_cache = self.nodes_manager.slots_cache.get(slot) + if not slot_cache: + raise SlotNotCoveredError(f'Slot "{slot}" is not covered by the cluster.') + + if replica: + if len(self.nodes_manager.slots_cache[slot]) < 2: + return None + node_idx = 1 + else: + node_idx = 0 + + return slot_cache[node_idx] + + def keyslot(self, key: EncodableT) -> int: + """ + Find the keyslot for a given key. + + See: https://redis.io/docs/manual/scaling/#redis-cluster-data-sharding + """ + return key_slot(self.encoder.encode(key)) + + def get_encoder(self) -> Encoder: + """Get the encoder object of the client.""" + return self.encoder + + def get_connection_kwargs(self) -> Dict[str, Optional[Any]]: + """Get the kwargs passed to :class:`~redis.anyio.connection.Connection`.""" + return self.connection_kwargs + + def get_retry(self) -> Optional["Retry"]: + return self.retry + + def set_retry(self, retry: "Retry") -> None: + self.retry = retry + for node in self.get_nodes(): + node.connection_kwargs.update({"retry": retry}) + for conn in node._connections: + conn.retry = retry + + def set_response_callback(self, command: str, callback: ResponseCallbackT) -> None: + """Set a custom response callback.""" + self.response_callbacks[command] = callback + + async def _determine_nodes( + self, command: str, *args: Any, node_flag: Optional[str] = None + ) -> List["ClusterNode"]: + # Determine which nodes should be executed the command on. + # Returns a list of target nodes. + if not node_flag: + # get the nodes group for this command if it was predefined + node_flag = self.command_flags.get(command) + + if node_flag in self.node_flags: + if node_flag == self.__class__.DEFAULT_NODE: + # return the cluster's default node + return [self.nodes_manager.default_node] + if node_flag == self.__class__.PRIMARIES: + # return all primaries + return self.nodes_manager.get_nodes_by_server_type(PRIMARY) + if node_flag == self.__class__.REPLICAS: + # return all replicas + return self.nodes_manager.get_nodes_by_server_type(REPLICA) + if node_flag == self.__class__.ALL_NODES: + # return all nodes + return list(self.nodes_manager.nodes_cache.values()) + if node_flag == self.__class__.RANDOM: + # return a random node + return [random.choice(list(self.nodes_manager.nodes_cache.values()))] + + # get the node that holds the key's slot + return [ + self.nodes_manager.get_node_from_slot( + await self._determine_slot(command, *args), + self.read_from_replicas and command in READ_COMMANDS, + self.load_balancing_strategy if command in READ_COMMANDS else None, + ) + ] + + async def _determine_slot(self, command: str, *args: Any) -> int: + if self.command_flags.get(command) == SLOT_ID: + # The command contains the slot ID + return int(args[0]) + + # Get the keys in the command + + # EVAL and EVALSHA are common enough that it's wasteful to go to the + # redis server to parse the keys. Besides, there is a bug in redis<7.0 + # where `self._get_command_keys()` fails anyway. So, we special case + # EVAL/EVALSHA. + # - issue: https://github.com/redis/redis/issues/9493 + # - fix: https://github.com/redis/redis/pull/9733 + if command.upper() in ("EVAL", "EVALSHA"): + # command syntax: EVAL "script body" num_keys ... + if len(args) < 2: + raise RedisClusterException( + f"Invalid args in command: {command, *args}" + ) + keys = args[2 : 2 + int(args[1])] + # if there are 0 keys, that means the script can be run on any node + # so we can just return a random slot + if not keys: + return random.randrange(0, REDIS_CLUSTER_HASH_SLOTS) + else: + keys = await self.commands_parser.get_keys(command, *args) + if not keys: + # FCALL can call a function with 0 keys, that means the function + # can be run on any node so we can just return a random slot + if command.upper() in ("FCALL", "FCALL_RO"): + return random.randrange(0, REDIS_CLUSTER_HASH_SLOTS) + raise RedisClusterException( + "No way to dispatch this command to Redis Cluster. " + "Missing key.\nYou can execute the command by specifying " + f"target nodes.\nCommand: {args}" + ) + + # single key command + if len(keys) == 1: + return self.keyslot(keys[0]) + + # multi-key command; we need to make sure all keys are mapped to + # the same slot + slots = {self.keyslot(key) for key in keys} + if len(slots) != 1: + raise RedisClusterException( + f"{command} - all keys must map to the same key slot" + ) + + return slots.pop() + + def _is_node_flag(self, target_nodes: Any) -> bool: + return isinstance(target_nodes, str) and target_nodes in self.node_flags + + def _parse_target_nodes(self, target_nodes: Any) -> List["ClusterNode"]: + if isinstance(target_nodes, list): + nodes = target_nodes + elif isinstance(target_nodes, ClusterNode): + # Supports passing a single ClusterNode as a variable + nodes = [target_nodes] + elif isinstance(target_nodes, dict): + # Supports dictionaries of the format {node_name: node}. + # It enables to execute commands with multi nodes as follows: + # rc.cluster_save_config(rc.get_primaries()) + nodes = list(target_nodes.values()) + else: + raise TypeError( + "target_nodes type can be one of the following: " + "node_flag (PRIMARIES, REPLICAS, RANDOM, ALL_NODES)," + "ClusterNode, list, or dict. " + f"The passed type is {type(target_nodes)}" + ) + return nodes + + async def execute_command(self, *args: EncodableT, **kwargs: Any) -> Any: + """ + Execute a raw command on the appropriate cluster node or target_nodes. + + It will retry the command as specified by :attr:`cluster_error_retry_attempts` & + then raise an exception. + + :param args: + | Raw command args + :param kwargs: + + - target_nodes: :attr:`NODE_FLAGS` or :class:`~.ClusterNode` + or List[:class:`~.ClusterNode`] or Dict[Any, :class:`~.ClusterNode`] + - Rest of the kwargs are passed to the Redis connection + + :raises RedisClusterException: if target_nodes is not provided & the command + can't be mapped to a slot + """ + command = args[0] + target_nodes = [] + target_nodes_specified = False + retry_attempts = self.cluster_error_retry_attempts + + passed_targets = kwargs.pop("target_nodes", None) + if passed_targets and not self._is_node_flag(passed_targets): + target_nodes = self._parse_target_nodes(passed_targets) + target_nodes_specified = True + retry_attempts = 0 + + # Add one for the first execution + execute_attempts = 1 + retry_attempts + for _ in range(execute_attempts): + if self._initialize: + await self.initialize() + if ( + len(target_nodes) == 1 + and target_nodes[0] == self.get_default_node() + ): + # Replace the default cluster node + self.replace_default_node() + try: + if not target_nodes_specified: + # Determine the nodes to execute the command on + target_nodes = await self._determine_nodes( + *args, node_flag=passed_targets + ) + if not target_nodes: + raise RedisClusterException( + f"No targets were found to execute {args} command on" + ) + + if len(target_nodes) == 1: + # Return the processed result + ret = await self._execute_command(target_nodes[0], *args, **kwargs) + if command in self.result_callbacks: + return self.result_callbacks[command]( + command, {target_nodes[0].name: ret}, **kwargs + ) + return ret + else: + keys = [node.name for node in target_nodes] + values = await gather( + *( + self._execute_command(node, *args, **kwargs) + for node in target_nodes + ) + ) + if command in self.result_callbacks: + return self.result_callbacks[command]( + command, dict(zip(keys, values)), **kwargs + ) + return dict(zip(keys, values)) + except Exception as e: + if retry_attempts > 0 and type(e) in self.__class__.ERRORS_ALLOW_RETRY: + # The nodes and slots cache were should be reinitialized. + # Try again with the new cluster setup. + retry_attempts -= 1 + continue + else: + # raise the exception + raise e + + async def _execute_command( + self, target_node: "ClusterNode", *args: Union[KeyT, EncodableT], **kwargs: Any + ) -> Any: + asking = moved = False + redirect_addr = None + ttl = self.RedisClusterRequestTTL + + while ttl > 0: + ttl -= 1 + try: + if asking: + target_node = self.get_node(node_name=redirect_addr) + await target_node.execute_command("ASKING") + asking = False + elif moved: + # MOVED occurred and the slots cache was updated, + # refresh the target node + slot = await self._determine_slot(*args) + target_node = self.nodes_manager.get_node_from_slot( + slot, + self.read_from_replicas and args[0] in READ_COMMANDS, + self.load_balancing_strategy + if args[0] in READ_COMMANDS + else None, + ) + moved = False + + return await target_node.execute_command(*args, **kwargs) + except (BusyLoadingError, MaxConnectionsError): + raise + except (ConnectionError, TimeoutError): + # Connection retries are being handled in the node's + # Retry object. + # Remove the failed node from the startup nodes before we try + # to reinitialize the cluster + self.nodes_manager.startup_nodes.pop(target_node.name, None) + # Hard force of reinitialize of the node/slots setup + # and try again with the new setup + await self.aclose() + raise + except ClusterDownError: + # ClusterDownError can occur during a failover and to get + # self-healed, we will try to reinitialize the cluster layout + # and retry executing the command + await self.aclose() + await anyio.sleep(0.25) + raise + except MovedError as e: + # First, we will try to patch the slots/nodes cache with the + # redirected node output and try again. If MovedError exceeds + # 'reinitialize_steps' number of times, we will force + # reinitializing the tables, and then try again. + # 'reinitialize_steps' counter will increase faster when + # the same client object is shared between multiple threads. To + # reduce the frequency you can set this variable in the + # RedisCluster constructor. + self.reinitialize_counter += 1 + if ( + self.reinitialize_steps + and self.reinitialize_counter % self.reinitialize_steps == 0 + ): + await self.aclose() + # Reset the counter + self.reinitialize_counter = 0 + else: + self.nodes_manager._moved_exception = e + moved = True + except AskError as e: + redirect_addr = get_node_name(host=e.host, port=e.port) + asking = True + except TryAgainError: + if ttl < self.RedisClusterRequestTTL / 2: + await anyio.sleep(0.05) + + raise ClusterError("TTL exhausted.") + + def pipeline( + self, transaction: Optional[Any] = None, shard_hint: Optional[Any] = None + ) -> "ClusterPipeline": + """ + Create & return a new :class:`~.ClusterPipeline` object. + + Cluster implementation of pipeline does not support transaction or shard_hint. + + :raises RedisClusterException: if transaction or shard_hint are truthy values + """ + if shard_hint: + raise RedisClusterException("shard_hint is deprecated in cluster mode") + + if transaction: + raise RedisClusterException("transaction is deprecated in cluster mode") + + return ClusterPipeline(self) + + def lock( + self, + name: KeyT, + timeout: Optional[float] = None, + sleep: float = 0.1, + blocking: bool = True, + blocking_timeout: Optional[float] = None, + lock_class: Optional[Type[Lock]] = None, + thread_local: bool = True, + raise_on_release_error: bool = True, + ) -> Lock: + """ + Return a new Lock object using key ``name`` that mimics + the behavior of threading.Lock. + + If specified, ``timeout`` indicates a maximum life for the lock. + By default, it will remain locked until release() is called. + + ``sleep`` indicates the amount of time to sleep per loop iteration + when the lock is in blocking mode and another client is currently + holding the lock. + + ``blocking`` indicates whether calling ``acquire`` should block until + the lock has been acquired or to fail immediately, causing ``acquire`` + to return False and the lock not being acquired. Defaults to True. + Note this value can be overridden by passing a ``blocking`` + argument to ``acquire``. + + ``blocking_timeout`` indicates the maximum amount of time in seconds to + spend trying to acquire the lock. A value of ``None`` indicates + continue trying forever. ``blocking_timeout`` can be specified as a + float or integer, both representing the number of seconds to wait. + + ``lock_class`` forces the specified lock implementation. Note that as + of redis-py 3.0, the only lock class we implement is ``Lock`` (which is + a Lua-based lock). So, it's unlikely you'll need this parameter, unless + you have created your own custom lock class. + + ``thread_local`` indicates whether the lock token is placed in + thread-local storage. By default, the token is placed in thread local + storage so that a thread only sees its token, not a token set by + another thread. Consider the following timeline: + + time: 0, thread-1 acquires `my-lock`, with a timeout of 5 seconds. + thread-1 sets the token to "abc" + time: 1, thread-2 blocks trying to acquire `my-lock` using the + Lock instance. + time: 5, thread-1 has not yet completed. redis expires the lock + key. + time: 5, thread-2 acquired `my-lock` now that it's available. + thread-2 sets the token to "xyz" + time: 6, thread-1 finishes its work and calls release(). if the + token is *not* stored in thread local storage, then + thread-1 would see the token value as "xyz" and would be + able to successfully release the thread-2's lock. + + ``raise_on_release_error`` indicates whether to raise an exception when + the lock is no longer owned when exiting the context manager. By default, + this is True, meaning an exception will be raised. If False, the warning + will be logged and the exception will be suppressed. + + In some use cases it's necessary to disable thread local storage. For + example, if you have code where one thread acquires a lock and passes + that lock instance to a worker thread to release later. If thread + local storage isn't disabled in this case, the worker thread won't see + the token set by the thread that acquired the lock. Our assumption + is that these cases aren't common and as such default to using + thread local storage.""" + if lock_class is None: + lock_class = Lock + return lock_class( + self, + name, + timeout=timeout, + sleep=sleep, + blocking=blocking, + blocking_timeout=blocking_timeout, + thread_local=thread_local, + raise_on_release_error=raise_on_release_error, + ) + + +class ClusterNode: + """ + Create a new ClusterNode. + + Each ClusterNode manages multiple :class:`~redis.anyio.connection.Connection` + objects for the (host, port). + """ + + __slots__ = ( + "_connections", + "_free", + "_lock", + "_event_dispatcher", + "connection_class", + "connection_kwargs", + "host", + "max_connections", + "name", + "port", + "response_callbacks", + "server_type", + ) + + def __init__( + self, + host: str, + port: Union[str, int], + server_type: Optional[str] = None, + *, + max_connections: int = 2**31, + connection_class: Type[Connection] = Connection, + **connection_kwargs: Any, + ) -> None: + if host == "localhost": + host = socket.gethostbyname(host) + + connection_kwargs["host"] = host + connection_kwargs["port"] = port + self.host = host + self.port = port + self.name = get_node_name(host, port) + self.server_type = server_type + + self.max_connections = max_connections + self.connection_class = connection_class + self.connection_kwargs = connection_kwargs + self.response_callbacks = connection_kwargs.pop("response_callbacks", {}) + + self._connections: List[Connection] = [] + self._free: Deque[Connection] = collections.deque(maxlen=self.max_connections) + self._event_dispatcher = self.connection_kwargs.get("event_dispatcher", None) + if self._event_dispatcher is None: + self._event_dispatcher = EventDispatcher() + + def __repr__(self) -> str: + return ( + f"[host={self.host}, port={self.port}, " + f"name={self.name}, server_type={self.server_type}]" + ) + + def __eq__(self, obj: Any) -> bool: + return isinstance(obj, ClusterNode) and obj.name == self.name + + _DEL_MESSAGE = "Unclosed ClusterNode object" + + def __del__( + self, + _warn: Any = warnings.warn, + ) -> None: + for connection in self._connections: + if connection.is_connected: + _warn(f"{self._DEL_MESSAGE} {self!r}", ResourceWarning, source=self) + + async def disconnect(self) -> None: + async with anyio.create_task_group() as tg: + for connection in self._connections: + tg.start_soon(connection.disconnect) + + def acquire_connection(self) -> Connection: + try: + return self._free.popleft() + except IndexError: + if len(self._connections) < self.max_connections: + connection = self.connection_class(**self.connection_kwargs) + self._connections.append(connection) + return connection + + raise MaxConnectionsError() + + async def parse_response( + self, connection: Connection, command: str, **kwargs: Any + ) -> Any: + try: + if NEVER_DECODE in kwargs: + response = await connection.read_response(disable_decoding=True) + kwargs.pop(NEVER_DECODE) + else: + response = await connection.read_response() + except ResponseError: + if EMPTY_RESPONSE in kwargs: + return kwargs[EMPTY_RESPONSE] + raise + + if EMPTY_RESPONSE in kwargs: + kwargs.pop(EMPTY_RESPONSE) + + # Remove keys entry, it needs only for cache. + kwargs.pop("keys", None) + + # Return response + if command in self.response_callbacks: + return self.response_callbacks[command](response, **kwargs) + + return response + + async def execute_command(self, *args: Any, **kwargs: Any) -> Any: + # Acquire connection + connection = self.acquire_connection() + + # Execute command + await connection.send_packed_command(connection.pack_command(*args), False) + + # Read response + try: + return await self.parse_response(connection, args[0], **kwargs) + finally: + # Release connection + self._free.append(connection) + + async def execute_pipeline(self, commands: List["PipelineCommand"]) -> bool: + # Acquire connection + connection = self.acquire_connection() + + # Execute command + await connection.send_packed_command( + connection.pack_commands(cmd.args for cmd in commands), False + ) + + # Read responses + ret = False + for cmd in commands: + try: + cmd.result = await self.parse_response( + connection, cmd.args[0], **cmd.kwargs + ) + except Exception as e: + cmd.result = e + ret = True + + # Release connection + self._free.append(connection) + + return ret + + async def re_auth_callback(self, token: TokenInterface): + tmp_queue = collections.deque() + while self._free: + conn = self._free.popleft() + await conn.retry.call_with_retry( + lambda: conn.send_command( + "AUTH", token.try_get("oid"), token.get_value() + ), + lambda error: self._mock(error), + ) + await conn.retry.call_with_retry( + lambda: conn.read_response(), lambda error: self._mock(error) + ) + tmp_queue.append(conn) + + while tmp_queue: + conn = tmp_queue.popleft() + self._free.append(conn) + + async def _mock(self, error: RedisError): + """ + Dummy functions, needs to be passed as error callback to retry object. + :param error: + :return: + """ + pass + + +class NodesManager: + __slots__ = ( + "_moved_exception", + "_event_dispatcher", + "_task_group", + "_exit_stack", + "connection_kwargs", + "default_node", + "nodes_cache", + "read_load_balancer", + "require_full_coverage", + "slots_cache", + "startup_nodes", + "address_remap", + ) + + def __init__( + self, + startup_nodes: List["ClusterNode"], + require_full_coverage: bool, + connection_kwargs: Dict[str, Any], + address_remap: Optional[Callable[[Tuple[str, int]], Tuple[str, int]]] = None, + event_dispatcher: Optional[EventDispatcher] = None, + ) -> None: + self.startup_nodes = {node.name: node for node in startup_nodes} + self.require_full_coverage = require_full_coverage + self.connection_kwargs = connection_kwargs + self.address_remap = address_remap + + self.default_node: "ClusterNode" = None + self.nodes_cache: Dict[str, "ClusterNode"] = {} + self.slots_cache: Dict[int, List["ClusterNode"]] = {} + self.read_load_balancer = LoadBalancer() + self._moved_exception: MovedError = None + if event_dispatcher is None: + self._event_dispatcher = EventDispatcher() + else: + self._event_dispatcher = event_dispatcher + + def get_node( + self, + host: Optional[str] = None, + port: Optional[int] = None, + node_name: Optional[str] = None, + ) -> Optional["ClusterNode"]: + if host and port: + # the user passed host and port + if host == "localhost": + host = socket.gethostbyname(host) + return self.nodes_cache.get(get_node_name(host=host, port=port)) + elif node_name: + return self.nodes_cache.get(node_name) + else: + raise DataError( + "get_node requires one of the following: 1. node name 2. host and port" + ) + + def set_nodes( + self, + old: Dict[str, "ClusterNode"], + new: Dict[str, "ClusterNode"], + remove_old: bool = False, + ) -> None: + if remove_old: + for name in list(old.keys()): + if name not in new: + self._task_group.start_soon(old.pop(name).disconnect) + + for name, node in new.items(): + if name in old: + if old[name] is node: + continue + + self._task_group.start_soon(old[name].disconnect) + + old[name] = node + + def _update_moved_slots(self) -> None: + e = self._moved_exception + redirected_node = self.get_node(host=e.host, port=e.port) + if redirected_node: + # The node already exists + if redirected_node.server_type != PRIMARY: + # Update the node's server type + redirected_node.server_type = PRIMARY + else: + # This is a new node, we will add it to the nodes cache + redirected_node = ClusterNode( + e.host, e.port, PRIMARY, **self.connection_kwargs + ) + self.set_nodes(self.nodes_cache, {redirected_node.name: redirected_node}) + if redirected_node in self.slots_cache[e.slot_id]: + # The MOVED error resulted from a failover, and the new slot owner + # had previously been a replica. + old_primary = self.slots_cache[e.slot_id][0] + # Update the old primary to be a replica and add it to the end of + # the slot's node list + old_primary.server_type = REPLICA + self.slots_cache[e.slot_id].append(old_primary) + # Remove the old replica, which is now a primary, from the slot's + # node list + self.slots_cache[e.slot_id].remove(redirected_node) + # Override the old primary with the new one + self.slots_cache[e.slot_id][0] = redirected_node + if self.default_node == old_primary: + # Update the default node with the new primary + self.default_node = redirected_node + else: + # The new slot owner is a new server, or a server from a different + # shard. We need to remove all current nodes from the slot's list + # (including replications) and add just the new node. + self.slots_cache[e.slot_id] = [redirected_node] + # Reset moved_exception + self._moved_exception = None + + def get_node_from_slot( + self, + slot: int, + read_from_replicas: bool = False, + load_balancing_strategy=None, + ) -> "ClusterNode": + if self._moved_exception: + self._update_moved_slots() + + if read_from_replicas is True and load_balancing_strategy is None: + load_balancing_strategy = LoadBalancingStrategy.ROUND_ROBIN + + try: + if len(self.slots_cache[slot]) > 1 and load_balancing_strategy: + # get the server index using the strategy defined in load_balancing_strategy + primary_name = self.slots_cache[slot][0].name + node_idx = self.read_load_balancer.get_server_index( + primary_name, len(self.slots_cache[slot]), load_balancing_strategy + ) + return self.slots_cache[slot][node_idx] + return self.slots_cache[slot][0] + except (IndexError, TypeError): + raise SlotNotCoveredError( + f'Slot "{slot}" not covered by the cluster. ' + f'"require_full_coverage={self.require_full_coverage}"' + ) + + def get_nodes_by_server_type(self, server_type: str) -> List["ClusterNode"]: + return [ + node + for node in self.nodes_cache.values() + if node.server_type == server_type + ] + + async def initialize(self) -> None: + self.read_load_balancer.reset() + tmp_nodes_cache: Dict[str, "ClusterNode"] = {} + tmp_slots: Dict[int, List["ClusterNode"]] = {} + disagreements = [] + startup_nodes_reachable = False + fully_covered = False + exception = None + for startup_node in tuple(self.startup_nodes.values()): + try: + # Make sure cluster mode is enabled on this node + try: + self._event_dispatcher.dispatch( + AfterAsyncClusterInstantiationEvent( + self.nodes_cache, + self.connection_kwargs.get("credential_provider", None), + ) + ) + cluster_slots = await startup_node.execute_command("CLUSTER SLOTS") + except ResponseError: + raise RedisClusterException( + "Cluster mode is not enabled on this node" + ) + startup_nodes_reachable = True + except Exception as e: + # Try the next startup node. + # The exception is saved and raised only if we have no more nodes. + exception = e + continue + + # CLUSTER SLOTS command results in the following output: + # [[slot_section[from_slot,to_slot,master,replica1,...,replicaN]]] + # where each node contains the following list: [IP, port, node_id] + # Therefore, cluster_slots[0][2][0] will be the IP address of the + # primary node of the first slot section. + # If there's only one server in the cluster, its ``host`` is '' + # Fix it to the host in startup_nodes + if ( + len(cluster_slots) == 1 + and not cluster_slots[0][2][0] + and len(self.startup_nodes) == 1 + ): + cluster_slots[0][2][0] = startup_node.host + + for slot in cluster_slots: + for i in range(2, len(slot)): + slot[i] = [str_if_bytes(val) for val in slot[i]] + primary_node = slot[2] + host = primary_node[0] + if host == "": + host = startup_node.host + port = int(primary_node[1]) + host, port = self.remap_host_port(host, port) + + nodes_for_slot = [] + + target_node = tmp_nodes_cache.get(get_node_name(host, port)) + if not target_node: + target_node = ClusterNode( + host, port, PRIMARY, **self.connection_kwargs + ) + # add this node to the nodes cache + tmp_nodes_cache[target_node.name] = target_node + nodes_for_slot.append(target_node) + + replica_nodes = slot[3:] + for replica_node in replica_nodes: + host = replica_node[0] + port = replica_node[1] + host, port = self.remap_host_port(host, port) + + target_replica_node = tmp_nodes_cache.get(get_node_name(host, port)) + if not target_replica_node: + target_replica_node = ClusterNode( + host, port, REPLICA, **self.connection_kwargs + ) + # add this node to the nodes cache + tmp_nodes_cache[target_replica_node.name] = target_replica_node + nodes_for_slot.append(target_replica_node) + + for i in range(int(slot[0]), int(slot[1]) + 1): + if i not in tmp_slots: + tmp_slots[i] = nodes_for_slot + else: + # Validate that 2 nodes want to use the same slot cache + # setup + tmp_slot = tmp_slots[i][0] + if tmp_slot.name != target_node.name: + disagreements.append( + f"{tmp_slot.name} vs {target_node.name} on slot: {i}" + ) + + if len(disagreements) > 5: + raise RedisClusterException( + f"startup_nodes could not agree on a valid " + f"slots cache: {', '.join(disagreements)}" + ) + + # Validate if all slots are covered or if we should try next startup node + fully_covered = True + for i in range(REDIS_CLUSTER_HASH_SLOTS): + if i not in tmp_slots: + fully_covered = False + break + if fully_covered: + break + + if not startup_nodes_reachable: + raise RedisClusterException( + f"Redis Cluster cannot be connected. Please provide at least " + f"one reachable node: {str(exception)}" + ) from exception + + # Check if the slots are not fully covered + if not fully_covered and self.require_full_coverage: + # Despite the requirement that the slots be covered, there + # isn't a full coverage + raise RedisClusterException( + f"All slots are not covered after query all startup_nodes. " + f"{len(tmp_slots)} of {REDIS_CLUSTER_HASH_SLOTS} " + f"covered..." + ) + + # Set the tmp variables to the real variables + self.slots_cache = tmp_slots + self.set_nodes(self.nodes_cache, tmp_nodes_cache, remove_old=True) + # Populate the startup nodes with all discovered nodes + self.set_nodes(self.startup_nodes, self.nodes_cache, remove_old=True) + + # Set the default node + self.default_node = self.get_nodes_by_server_type(PRIMARY)[0] + # If initialize was called after a MovedError, clear it + self._moved_exception = None + + async def __aenter__(self): + async with AsyncExitStack() as stack: + self._task_group = await stack.enter_async_context( + anyio.create_task_group() + ) + stack.push_async_callback(self.aclose) + self._exit_stack = stack.pop_all() + return self + + async def __aexit__( + self, + exc_type: type[BaseException] | None, + exc_value: BaseException | None, + exc_tb: TracebackType | None, + ) -> bool | None: + try: + return await self._exit_stack.__aexit__(exc_type, exc_value, exc_tb) + except ExceptionGroup as excgrp: + while isinstance(excgrp, ExceptionGroup): + if len(excgrp.exceptions) > 1: + break + + if excgrp.exceptions[0] is exc_value: + return False + + excgrp = excgrp.exceptions[0] + + raise + + async def aclose(self, attr: str = "nodes_cache") -> None: + self.default_node = None + await gather(*(node.disconnect() for node in getattr(self, attr).values())) + + def remap_host_port(self, host: str, port: int) -> Tuple[str, int]: + """ + Remap the host and port returned from the cluster to a different + internal value. Useful if the client is not connecting directly + to the cluster. + """ + if self.address_remap: + return self.address_remap((host, port)) + return host, port + + +class ClusterPipeline(AbstractRedis, AbstractRedisCluster, AsyncRedisClusterCommands): + """ + Create a new ClusterPipeline object. + + Usage:: + + result = await ( + rc.pipeline() + .set("A", 1) + .get("A") + .hset("K", "F", "V") + .hgetall("K") + .mset_nonatomic({"A": 2, "B": 3}) + .get("A") + .get("B") + .delete("A", "B", "K") + .execute() + ) + # result = [True, "1", 1, {"F": "V"}, True, True, "2", "3", 1, 1, 1] + + Note: For commands `DELETE`, `EXISTS`, `TOUCH`, `UNLINK`, `mset_nonatomic`, which + are split across multiple nodes, you'll get multiple results for them in the array. + + Retryable errors: + - :class:`~.ClusterDownError` + - :class:`~.ConnectionError` + - :class:`~.TimeoutError` + + Redirection errors: + - :class:`~.TryAgainError` + - :class:`~.MovedError` + - :class:`~.AskError` + + :param client: + | Existing :class:`~.RedisCluster` client + """ + + __slots__ = ("_command_stack", "_client") + + def __init__(self, client: RedisCluster) -> None: + self._client = client + + self._command_stack: List["PipelineCommand"] = [] + + async def initialize(self) -> "ClusterPipeline": + if self._client._initialize: + await self._client.initialize() + self._command_stack = [] + return self + + async def __aenter__(self) -> "ClusterPipeline": + return await self.initialize() + + async def __aexit__(self, exc_type: None, exc_value: None, traceback: None) -> None: + self._command_stack = [] + + def __enter__(self) -> "ClusterPipeline": + self._command_stack = [] + return self + + def __exit__(self, exc_type: None, exc_value: None, traceback: None) -> None: + self._command_stack = [] + + def __bool__(self) -> bool: + "Pipeline instances should always evaluate to True on Python 3+" + return True + + def __len__(self) -> int: + return len(self._command_stack) + + def execute_command( + self, *args: Union[KeyT, EncodableT], **kwargs: Any + ) -> "ClusterPipeline": + """ + Append a raw command to the pipeline. + + :param args: + | Raw command args + :param kwargs: + + - target_nodes: :attr:`NODE_FLAGS` or :class:`~.ClusterNode` + or List[:class:`~.ClusterNode`] or Dict[Any, :class:`~.ClusterNode`] + - Rest of the kwargs are passed to the Redis connection + """ + self._command_stack.append( + PipelineCommand(len(self._command_stack), *args, **kwargs) + ) + return self + + async def execute( + self, raise_on_error: bool = True, allow_redirections: bool = True + ) -> List[Any]: + """ + Execute the pipeline. + + It will retry the commands as specified by :attr:`cluster_error_retry_attempts` + & then raise an exception. + + :param raise_on_error: + | Raise the first error if there are any errors + :param allow_redirections: + | Whether to retry each failed command individually in case of redirection + errors + + :raises RedisClusterException: if target_nodes is not provided & the command + can't be mapped to a slot + """ + if not self._command_stack: + return [] + + try: + retry_attempts = self._client.cluster_error_retry_attempts + while True: + try: + if self._client._initialize: + await self._client.initialize() + return await self._execute( + self._client, + self._command_stack, + raise_on_error=raise_on_error, + allow_redirections=allow_redirections, + ) + + except self.__class__.ERRORS_ALLOW_RETRY as e: + if retry_attempts > 0: + # Try again with the new cluster setup. All other errors + # should be raised. + retry_attempts -= 1 + await self._client.aclose() + await anyio.sleep(0.25) + else: + # All other errors should be raised. + raise e + finally: + self._command_stack = [] + + async def _execute( + self, + client: "RedisCluster", + stack: List["PipelineCommand"], + raise_on_error: bool = True, + allow_redirections: bool = True, + ) -> List[Any]: + todo = [ + cmd for cmd in stack if not cmd.result or isinstance(cmd.result, Exception) + ] + + nodes = {} + for cmd in todo: + passed_targets = cmd.kwargs.pop("target_nodes", None) + if passed_targets and not client._is_node_flag(passed_targets): + target_nodes = client._parse_target_nodes(passed_targets) + else: + target_nodes = await client._determine_nodes( + *cmd.args, node_flag=passed_targets + ) + if not target_nodes: + raise RedisClusterException( + f"No targets were found to execute {cmd.args} command on" + ) + if len(target_nodes) > 1: + raise RedisClusterException(f"Too many targets for command {cmd.args}") + node = target_nodes[0] + if node.name not in nodes: + nodes[node.name] = (node, []) + nodes[node.name][1].append(cmd) + + errors = await gather( + *(node[0].execute_pipeline(node[1]) for node in nodes.values()) + ) + + if any(errors): + if allow_redirections: + # send each errored command individually + for cmd in todo: + if isinstance(cmd.result, (TryAgainError, MovedError, AskError)): + try: + cmd.result = await client.execute_command( + *cmd.args, **cmd.kwargs + ) + except Exception as e: + cmd.result = e + + if raise_on_error: + for cmd in todo: + result = cmd.result + if isinstance(result, Exception): + command = " ".join(map(safe_str, cmd.args)) + msg = ( + f"Command # {cmd.position + 1} " + f"({truncate_text(command)}) " + f"of pipeline caused error: {result.args}" + ) + result.args = (msg,) + result.args[1:] + raise result + + default_cluster_node = client.get_default_node() + + # Check whether the default node was used. In some cases, + # 'client.get_default_node()' may return None. The check below + # prevents a potential AttributeError. + if default_cluster_node is not None: + default_node = nodes.get(default_cluster_node.name) + if default_node is not None: + # This pipeline execution used the default node, check if we need + # to replace it. + # Note: when the error is raised we'll reset the default node in the + # caller function. + for cmd in default_node[1]: + # Check if it has a command that failed with a relevant + # exception + if type(cmd.result) in self.__class__.ERRORS_ALLOW_RETRY: + client.replace_default_node() + break + + return [cmd.result for cmd in stack] + + def _split_command_across_slots( + self, command: str, *keys: KeyT + ) -> "ClusterPipeline": + for slot_keys in self._client._partition_keys_by_slot(keys).values(): + self.execute_command(command, *slot_keys) + + return self + + def mset_nonatomic( + self, mapping: Mapping[AnyKeyT, EncodableT] + ) -> "ClusterPipeline": + encoder = self._client.encoder + + slots_pairs = {} + for pair in mapping.items(): + slot = key_slot(encoder.encode(pair[0])) + slots_pairs.setdefault(slot, []).extend(pair) + + for pairs in slots_pairs.values(): + self.execute_command("MSET", *pairs) + + return self + + +for command in PIPELINE_BLOCKED_COMMANDS: + command = command.replace(" ", "_").lower() + if command == "mset_nonatomic": + continue + + setattr(ClusterPipeline, command, block_pipeline_command(command)) + + +class PipelineCommand: + def __init__(self, position: int, *args: Any, **kwargs: Any) -> None: + self.args = args + self.kwargs = kwargs + self.position = position + self.result: Union[Any, Exception] = None + + def __repr__(self) -> str: + return f"[{self.position}] {self.args} ({self.kwargs})" diff --git a/redis/anyio/compat.py b/redis/anyio/compat.py new file mode 100644 index 0000000000..75ab03c254 --- /dev/null +++ b/redis/anyio/compat.py @@ -0,0 +1,11 @@ +try: + from contextlib import aclosing +except ImportError: + from contextlib import asynccontextmanager + + @asynccontextmanager + async def aclosing(thing): + try: + yield thing + finally: + await thing.aclose() diff --git a/redis/anyio/connection.py b/redis/anyio/connection.py new file mode 100644 index 0000000000..08b61a4155 --- /dev/null +++ b/redis/anyio/connection.py @@ -0,0 +1,1293 @@ +from __future__ import annotations + +import builtins +import copy +import enum +import inspect +import socket +import warnings +import weakref +from abc import abstractmethod +from itertools import chain +from types import MappingProxyType +from typing import ( + Any, + Callable, + Iterable, + List, + Mapping, + Optional, + Protocol, + Set, + Tuple, + Type, + TypedDict, + TypeVar, + Union, +) +from urllib.parse import ParseResult, parse_qs, unquote, urlparse + +import anyio +from anyio import BrokenResourceError, aclose_forcefully +from anyio.abc import SocketAttribute, SocketStream + +from ..utils import SSL_AVAILABLE +from ._parsers.base import AnyIOBaseParser +from .utils import wait_for_condition + +if SSL_AVAILABLE: + import ssl + from ssl import SSLContext, TLSVersion +else: + ssl = None + TLSVersion = None + SSLContext = None + +from redis.anyio.retry import Retry +from redis.backoff import NoBackoff +from redis.connection import DEFAULT_RESP_VERSION +from redis.credentials import CredentialProvider, UsernamePasswordCredentialProvider +from redis.exceptions import ( + AuthenticationError, + AuthenticationWrongNumberOfArgsError, + ConnectionError, + DataError, + RedisError, + ResponseError, + TimeoutError, +) +from redis.typing import EncodableT +from redis.utils import HIREDIS_AVAILABLE, get_lib_version, str_if_bytes + +from .._parsers import ( + Encoder, +) +from ..auth.token import TokenInterface +from ..event import AsyncAfterConnectionReleasedEvent, EventDispatcher +from ..utils import deprecated_args, format_error_message +from ._parsers import ( + _AnyIOHiredisParser, + _AnyIORESP2Parser, + _AnyIORESP3Parser, +) + +SYM_STAR = b"*" +SYM_DOLLAR = b"$" +SYM_CRLF = b"\r\n" +SYM_LF = b"\n" +SYM_EMPTY = b"" + + +class _Sentinel(enum.Enum): + sentinel = object() + + +SENTINEL = _Sentinel.sentinel + + +DefaultParser: type[_AnyIORESP2Parser | _AnyIORESP3Parser, _AnyIOHiredisParser] +if HIREDIS_AVAILABLE: + DefaultParser = _AnyIOHiredisParser +else: + DefaultParser = _AnyIORESP2Parser + + +class ConnectCallbackProtocol(Protocol): + def __call__(self, connection: "AbstractConnection"): ... + + +class AsyncConnectCallbackProtocol(Protocol): + async def __call__(self, connection: "AbstractConnection"): ... + + +ConnectCallbackT = Union[ConnectCallbackProtocol, AsyncConnectCallbackProtocol] + + +class AbstractConnection: + """Manages communication to and from a Redis server""" + + __slots__ = ( + "db", + "username", + "client_name", + "lib_name", + "lib_version", + "credential_provider", + "password", + "socket_timeout", + "socket_connect_timeout", + "redis_connect_func", + "retry_on_timeout", + "retry_on_error", + "health_check_interval", + "next_health_check", + "last_active_at", + "encoder", + "ssl_context", + "protocol", + "_stream", + "_parser", + "_connect_callbacks", + "_buffer_cutoff", + "_lock", + "_socket_read_size", + "__dict__", + ) + + def __init__( + self, + *, + db: Union[str, int] = 0, + password: Optional[str] = None, + socket_timeout: Optional[float] = None, + socket_connect_timeout: Optional[float] = None, + retry_on_timeout: bool = False, + retry_on_error: Union[list, _Sentinel] = SENTINEL, + encoding: str = "utf-8", + encoding_errors: str = "strict", + decode_responses: bool = False, + parser_class: Type[AnyIOBaseParser] = DefaultParser, + socket_read_size: int = 65536, + health_check_interval: float = 0, + client_name: Optional[str] = None, + lib_name: Optional[str] = "redis-py", + lib_version: Optional[str] = get_lib_version(), + username: Optional[str] = None, + retry: Optional[Retry] = None, + redis_connect_func: Optional[ConnectCallbackT] = None, + encoder_class: Type[Encoder] = Encoder, + credential_provider: Optional[CredentialProvider] = None, + protocol: Optional[int] = 2, + event_dispatcher: Optional[EventDispatcher] = None, + ): + if (username or password) and credential_provider is not None: + raise DataError( + "'username' and 'password' cannot be passed along with 'credential_" + "provider'. Please provide only one of the following arguments: \n" + "1. 'password' and (optional) 'username'\n" + "2. 'credential_provider'" + ) + if event_dispatcher is None: + self._event_dispatcher = EventDispatcher() + else: + self._event_dispatcher = event_dispatcher + self.db = db + self.client_name = client_name + self.lib_name = lib_name + self.lib_version = lib_version + self.credential_provider = credential_provider + self.password = password + self.username = username + self.socket_timeout = socket_timeout + if socket_connect_timeout is None: + socket_connect_timeout = socket_timeout + self.socket_connect_timeout = socket_connect_timeout + self.retry_on_timeout = retry_on_timeout + if retry_on_error is SENTINEL: + retry_on_error = [] + if retry_on_timeout: + retry_on_error.append(TimeoutError) + retry_on_error.append(socket.timeout) + self.retry_on_error = retry_on_error + if retry or retry_on_error: + if not retry: + self.retry = Retry(NoBackoff(), 1) + else: + # deep-copy the Retry object as it is mutable + self.retry = copy.deepcopy(retry) + # Update the retry's supported errors with the specified errors + self.retry.update_supported_errors(retry_on_error) + else: + self.retry = Retry(NoBackoff(), 0) + self.health_check_interval = health_check_interval + self.next_health_check: float = -1 + self.encoder = encoder_class(encoding, encoding_errors, decode_responses) + self.redis_connect_func = redis_connect_func + self._stream: SocketStream | None = None + self._socket_read_size = socket_read_size + self.set_parser(parser_class) + self._connect_callbacks: List[weakref.WeakMethod[ConnectCallbackT]] = [] + self._buffer_cutoff = 6000 + self._re_auth_token: Optional[TokenInterface] = None + + try: + p = int(protocol) + except TypeError: + p = DEFAULT_RESP_VERSION + except ValueError: + raise ConnectionError("protocol must be an integer") + + if p < 2 or p > 3: + raise ConnectionError("protocol must be either 2 or 3") + + self.protocol = protocol + + # passing _warnings and _grl as argument default since they may be gone + # by the time __del__ is called at shutdown + def __del__( + self, + _warn: Any = warnings.warn, + ) -> None: + # For some reason, the individual streams don't get properly garbage + # collected and therefore produce no resource warnings. We add one + # here, in the same style as those from the stdlib. + if getattr(self, "_stream", None): + _warn(f"unclosed Connection {self!r}", ResourceWarning, source=self) + + def __repr__(self): + repr_args = ",".join((f"{k}={v}" for k, v in self.repr_pieces())) + return f"<{self.__class__.__module__}.{self.__class__.__name__}({repr_args})>" + + @abstractmethod + def repr_pieces(self): + pass + + @property + def is_connected(self): + return self._stream is not None + + def register_connect_callback(self, callback): + """ + Register a callback to be called when the connection is established either + initially or reconnected. This allows listeners to issue commands that + are ephemeral to the connection, for example pub/sub subscription or + key tracking. The callback must be a _method_ and will be kept as + a weak reference. + """ + wm = weakref.WeakMethod(callback) + if wm not in self._connect_callbacks: + self._connect_callbacks.append(wm) + + def deregister_connect_callback(self, callback): + """ + De-register a previously registered callback. It will no-longer receive + notifications on connection events. Calling this is not required when the + listener goes away, since the callbacks are kept as weak methods. + """ + try: + self._connect_callbacks.remove(weakref.WeakMethod(callback)) + except ValueError: + pass + + def set_parser(self, parser_class: Type[AnyIOBaseParser]) -> None: + """ + Creates a new instance of parser_class with socket size: + _socket_read_size and assigns it to the parser for the connection + :param parser_class: The required parser class + """ + self._parser = parser_class(socket_read_size=self._socket_read_size) + + async def connect(self): + """Connects to the Redis server if not already connected""" + await self.connect_check_health(check_health=True) + + async def connect_check_health(self, check_health: bool = True): + """Connects to the Redis server if not already connected""" + if self.is_connected: + return + try: + await self.retry.call_with_retry( + lambda: self._connect(), lambda error: self.disconnect() + ) + except builtins.TimeoutError: + raise TimeoutError("Timeout connecting to server") from None + except OSError as exc: + e = exc.__cause__ if isinstance(exc.__cause__, OSError) else exc + raise ConnectionError(self._error_message(e)) from exc + except Exception as exc: + raise ConnectionError(exc) from exc + + try: + if not self.redis_connect_func: + # Use the default on_connect function + await self.on_connect_check_health(check_health=check_health) + else: + # Use the passed function redis_connect_func + ( + await self.redis_connect_func(self) + if inspect.iscoroutinefunction(self.redis_connect_func) + else self.redis_connect_func(self) + ) + except RedisError: + # clean up after any error in on_connect + await self.disconnect() + raise + + # run any user callbacks. right now the only internal callback + # is for pubsub channel/pattern resubscription + # first, remove any dead weakrefs + self._connect_callbacks = [ref for ref in self._connect_callbacks if ref()] + for ref in self._connect_callbacks: + callback = ref() + task = callback(self) + if task and inspect.isawaitable(task): + await task + + @abstractmethod + async def _connect(self): + pass + + @abstractmethod + def _host_error(self) -> str: + pass + + def _error_message(self, exception: BaseException) -> str: + return format_error_message(self._host_error(), exception) + + def get_protocol(self): + return self.protocol + + async def on_connect(self) -> None: + """Initialize the connection, authenticate and select a database""" + await self.on_connect_check_health(check_health=True) + + async def on_connect_check_health(self, check_health: bool = True) -> None: + """Initialize the connection, authenticate and select a database""" + self._parser.on_connect(self) + parser = self._parser + + auth_args = None + # if credential provider or username and/or password are set, authenticate + if self.credential_provider or (self.username or self.password): + cred_provider = ( + self.credential_provider + or UsernamePasswordCredentialProvider(self.username, self.password) + ) + auth_args = await cred_provider.get_credentials_async() + + # if resp version is specified and we have auth args, + # we need to send them via HELLO + if auth_args and self.protocol not in [2, "2"]: + if isinstance(self._parser, _AnyIORESP2Parser): + self.set_parser(_AnyIORESP3Parser) + # update cluster exception classes + self._parser.EXCEPTION_CLASSES = parser.EXCEPTION_CLASSES + self._parser.on_connect(self) + if len(auth_args) == 1: + auth_args = ["default", auth_args[0]] + # avoid checking health here -- PING will fail if we try + # to check the health prior to the AUTH + await self.send_command( + "HELLO", self.protocol, "AUTH", *auth_args, check_health=False + ) + response = await self.read_response() + if response.get(b"proto") != int(self.protocol) and response.get( + "proto" + ) != int(self.protocol): + raise ConnectionError("Invalid RESP version") + # avoid checking health here -- PING will fail if we try + # to check the health prior to the AUTH + elif auth_args: + await self.send_command("AUTH", *auth_args, check_health=False) + + try: + auth_response = await self.read_response() + except AuthenticationWrongNumberOfArgsError: + # a username and password were specified but the Redis + # server seems to be < 6.0.0 which expects a single password + # arg. retry auth with just the password. + # https://github.com/andymccurdy/redis-py/issues/1274 + await self.send_command("AUTH", auth_args[-1], check_health=False) + auth_response = await self.read_response() + + if str_if_bytes(auth_response) != "OK": + raise AuthenticationError("Invalid Username or Password") + + # if resp version is specified, switch to it + elif self.protocol not in [2, "2"]: + if isinstance(self._parser, _AnyIORESP2Parser): + self.set_parser(_AnyIORESP3Parser) + # update cluster exception classes + self._parser.EXCEPTION_CLASSES = parser.EXCEPTION_CLASSES + self._parser.on_connect(self) + await self.send_command("HELLO", self.protocol, check_health=check_health) + response = await self.read_response() + # if response.get(b"proto") != self.protocol and response.get( + # "proto" + # ) != self.protocol: + # raise ConnectionError("Invalid RESP version") + + # if a client_name is given, set it + if self.client_name: + await self.send_command( + "CLIENT", + "SETNAME", + self.client_name, + check_health=check_health, + ) + if str_if_bytes(await self.read_response()) != "OK": + raise ConnectionError("Error setting client name") + + # set the library name and version, pipeline for lower startup latency + if self.lib_name: + await self.send_command( + "CLIENT", + "SETINFO", + "LIB-NAME", + self.lib_name, + check_health=check_health, + ) + if self.lib_version: + await self.send_command( + "CLIENT", + "SETINFO", + "LIB-VER", + self.lib_version, + check_health=check_health, + ) + # if a database is specified, switch to it. Also pipeline this + if self.db: + await self.send_command("SELECT", self.db, check_health=check_health) + + # read responses from pipeline + for _ in (sent for sent in (self.lib_name, self.lib_version) if sent): + try: + await self.read_response() + except ResponseError: + pass + + if self.db: + if str_if_bytes(await self.read_response()) != "OK": + raise ConnectionError("Invalid Database") + + async def disconnect(self, nowait: bool = False) -> None: + """Disconnects from the Redis server""" + if nowait: + stream, self._stream = self._stream, None + await aclose_forcefully(stream) + return + + try: + with anyio.fail_after(self.socket_connect_timeout): + self._parser.on_disconnect() + if not self.is_connected: + return + + try: + await self._stream.aclose() + finally: + self._stream = None + except TimeoutError: + raise TimeoutError( + f"Timed out closing connection after {self.socket_connect_timeout}" + ) from None + + async def _send_ping(self): + """Send PING, expect PONG in return""" + await self.send_command("PING", check_health=False) + if str_if_bytes(await self.read_response()) != "PONG": + raise ConnectionError("Bad response from PING health check") + + async def _ping_failed(self, error): + """Function to call when PING fails""" + await self.disconnect() + + async def check_health(self): + """Check the health of the connection with a PING/PONG""" + if self.health_check_interval and anyio.current_time() > self.next_health_check: + await self.retry.call_with_retry(self._send_ping, self._ping_failed) + + async def _send_packed_command(self, command: Iterable[bytes]) -> None: + for chunk in command: + await self._stream.send(chunk) + + async def send_packed_command( + self, command: Union[bytes, str, Iterable[bytes]], check_health: bool = True + ) -> None: + if not self.is_connected: + await self.connect() + elif check_health: + await self.check_health() + + try: + if isinstance(command, str): + command = command.encode() + if isinstance(command, bytes): + command = [command] + if self.socket_timeout: + with anyio.move_on_after(self.socket_timeout): + await self._send_packed_command(command) + else: + for chunk in command: + await self._stream.send(chunk) + except TimeoutError: + await self.disconnect(nowait=True) + raise TimeoutError("Timeout writing to socket") from None + except BrokenResourceError as e: + await self.disconnect(nowait=True) + raise ConnectionError("Error while writing to socket") from e + except BaseException: + # BaseExceptions can be raised when a socket send operation is not + # finished, e.g. due to a timeout. Ideally, a caller could then re-try + # to send un-sent data. However, the send_packed_command() API + # does not support it so there is no point in keeping the connection open. + await self.disconnect(nowait=True) + raise + + async def send_command(self, *args: Any, **kwargs: Any) -> None: + """Pack and send a command to the Redis server""" + await self.send_packed_command( + self.pack_command(*args), check_health=kwargs.get("check_health", True) + ) + + async def can_read_destructive(self): + """Poll the socket to see if there's data that can be read.""" + try: + return await self._parser.can_read_destructive() + except OSError as e: + await self.disconnect(nowait=True) + host_error = self._host_error() + raise ConnectionError(f"Error while reading from {host_error}: {e.args}") + + async def read_response( + self, + disable_decoding: bool = False, + timeout: Optional[float] = None, + *, + disconnect_on_error: bool = True, + push_request: Optional[bool] = False, + ): + """Read the response from a previously sent command""" + read_timeout = timeout if timeout is not None else self.socket_timeout + host_error = self._host_error() + try: + if ( + read_timeout is not None + and self.protocol in ["3", 3] + and not HIREDIS_AVAILABLE + ): + with anyio.fail_after(read_timeout): + response = await self._parser.read_response( + disable_decoding=disable_decoding, push_request=push_request + ) + elif read_timeout is not None: + with anyio.fail_after(read_timeout): + response = await self._parser.read_response( + disable_decoding=disable_decoding + ) + elif self.protocol in ["3", 3] and not HIREDIS_AVAILABLE: + response = await self._parser.read_response( + disable_decoding=disable_decoding, push_request=push_request + ) + else: + response = await self._parser.read_response( + disable_decoding=disable_decoding + ) + except builtins.TimeoutError: + if timeout is not None: + # user requested timeout, return None. Operation can be retried + return None + # it was a self.socket_timeout error. + if disconnect_on_error: + await self.disconnect(nowait=True) + raise TimeoutError(f"Timeout reading from {host_error}") + except OSError as e: + if disconnect_on_error: + await self.disconnect(nowait=True) + raise ConnectionError(f"Error while reading from {host_error} : {e.args}") + except BaseException: + # Also by default close in case of BaseException. A lot of code + # relies on this behaviour when doing Command/Response pairs. + # See #1128. + if disconnect_on_error: + await self.disconnect(nowait=True) + raise + + if self.health_check_interval: + next_time = anyio.current_time() + self.health_check_interval + self.next_health_check = next_time + + if isinstance(response, ResponseError): + raise response from None + return response + + def pack_command(self, *args: EncodableT) -> List[bytes]: + """Pack a series of arguments into the Redis protocol""" + output = [] + # the client might have included 1 or more literal arguments in + # the command name, e.g., 'CONFIG GET'. The Redis server expects these + # arguments to be sent separately, so split the first argument + # manually. These arguments should be bytestrings so that they are + # not encoded. + assert not isinstance(args[0], float) + if isinstance(args[0], str): + args = tuple(args[0].encode().split()) + args[1:] + elif b" " in args[0]: + args = tuple(args[0].split()) + args[1:] + + buff = SYM_EMPTY.join((SYM_STAR, str(len(args)).encode(), SYM_CRLF)) + + buffer_cutoff = self._buffer_cutoff + for arg in map(self.encoder.encode, args): + # to avoid large string mallocs, chunk the command into the + # output list if we're sending large values or memoryviews + arg_length = len(arg) + if ( + len(buff) > buffer_cutoff + or arg_length > buffer_cutoff + or isinstance(arg, memoryview) + ): + buff = SYM_EMPTY.join( + (buff, SYM_DOLLAR, str(arg_length).encode(), SYM_CRLF) + ) + output.append(buff) + output.append(arg) + buff = SYM_CRLF + else: + buff = SYM_EMPTY.join( + ( + buff, + SYM_DOLLAR, + str(arg_length).encode(), + SYM_CRLF, + arg, + SYM_CRLF, + ) + ) + output.append(buff) + return output + + def pack_commands(self, commands: Iterable[Iterable[EncodableT]]) -> List[bytes]: + """Pack multiple commands into the Redis protocol""" + output: List[bytes] = [] + pieces: List[bytes] = [] + buffer_length = 0 + buffer_cutoff = self._buffer_cutoff + + for cmd in commands: + for chunk in self.pack_command(*cmd): + chunklen = len(chunk) + if ( + buffer_length > buffer_cutoff + or chunklen > buffer_cutoff + or isinstance(chunk, memoryview) + ): + if pieces: + output.append(SYM_EMPTY.join(pieces)) + buffer_length = 0 + pieces = [] + + if chunklen > buffer_cutoff or isinstance(chunk, memoryview): + output.append(chunk) + else: + pieces.append(chunk) + buffer_length += chunklen + + if pieces: + output.append(SYM_EMPTY.join(pieces)) + return output + + def set_re_auth_token(self, token: TokenInterface): + self._re_auth_token = token + + async def re_auth(self): + if self._re_auth_token is not None: + await self.send_command( + "AUTH", + self._re_auth_token.try_get("oid"), + self._re_auth_token.get_value(), + ) + await self.read_response() + self._re_auth_token = None + + +class Connection(AbstractConnection): + "Manages TCP communication to and from a Redis server" + + def __init__( + self, + *, + host: str = "localhost", + port: Union[str, int] = 6379, + socket_keepalive: bool = False, + socket_keepalive_options: Optional[Mapping[int, Union[int, bytes]]] = None, + socket_type: int = 0, + **kwargs, + ): + self.host = host + self.port = int(port) + self.socket_keepalive = socket_keepalive + self.socket_keepalive_options = socket_keepalive_options or {} + self.socket_type = socket_type + super().__init__(**kwargs) + + def repr_pieces(self): + pieces = [("host", self.host), ("port", self.port), ("db", self.db)] + if self.client_name: + pieces.append(("client_name", self.client_name)) + return pieces + + def _connection_arguments(self) -> Mapping: + return { + "remote_host": self.host, + "remote_port": self.port, + "tls_standard_compatible": False, + } + + async def _connect(self): + """Create a TCP socket connection""" + with anyio.fail_after(self.socket_connect_timeout): + self._stream = await anyio.connect_tcp(**self._connection_arguments()) + + sock = self._stream.extra(SocketAttribute.raw_socket) + try: + if self.socket_keepalive: + sock.setsockopt(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1) + for k, v in self.socket_keepalive_options.items(): + sock.setsockopt(socket.SOL_TCP, k, v) + except (OSError, TypeError): + # `socket_keepalive_options` might contain invalid options + # causing an error. Do not leave the connection open. + await anyio.aclose_forcefully(self._stream) + raise + + def _host_error(self) -> str: + return f"{self.host}:{self.port}" + + +class SSLConnection(Connection): + """Manages SSL connections to and from the Redis server(s). + This class extends the Connection class, adding SSL functionality, and making + use of ssl.SSLContext (https://docs.python.org/3/library/ssl.html#ssl.SSLContext) + """ + + def __init__( + self, + ssl_keyfile: Optional[str] = None, + ssl_certfile: Optional[str] = None, + ssl_cert_reqs: str = "required", + ssl_ca_certs: Optional[str] = None, + ssl_ca_data: Optional[str] = None, + ssl_check_hostname: bool = False, + ssl_min_version: Optional[TLSVersion] = None, + ssl_ciphers: Optional[str] = None, + **kwargs, + ): + if not SSL_AVAILABLE: + raise RedisError("Python wasn't built with SSL support") + + self.ssl_context: RedisSSLContext = RedisSSLContext( + keyfile=ssl_keyfile, + certfile=ssl_certfile, + cert_reqs=ssl_cert_reqs, + ca_certs=ssl_ca_certs, + ca_data=ssl_ca_data, + check_hostname=ssl_check_hostname, + min_version=ssl_min_version, + ciphers=ssl_ciphers, + ) + super().__init__(**kwargs) + + def _connection_arguments(self) -> Mapping: + kwargs = super()._connection_arguments() + kwargs["ssl_context"] = self.ssl_context.get() + return kwargs + + @property + def keyfile(self): + return self.ssl_context.keyfile + + @property + def certfile(self): + return self.ssl_context.certfile + + @property + def cert_reqs(self): + return self.ssl_context.cert_reqs + + @property + def ca_certs(self): + return self.ssl_context.ca_certs + + @property + def ca_data(self): + return self.ssl_context.ca_data + + @property + def check_hostname(self): + return self.ssl_context.check_hostname + + @property + def min_version(self): + return self.ssl_context.min_version + + +class RedisSSLContext: + __slots__ = ( + "keyfile", + "certfile", + "cert_reqs", + "ca_certs", + "ca_data", + "context", + "check_hostname", + "min_version", + "ciphers", + ) + + def __init__( + self, + keyfile: Optional[str] = None, + certfile: Optional[str] = None, + cert_reqs: Optional[str] = None, + ca_certs: Optional[str] = None, + ca_data: Optional[str] = None, + check_hostname: bool = True, + min_version: Optional[TLSVersion] = None, + ciphers: Optional[str] = None, + ): + if not SSL_AVAILABLE: + raise RedisError("Python wasn't built with SSL support") + + self.keyfile = keyfile + self.certfile = certfile + if cert_reqs is None: + self.cert_reqs = ssl.CERT_NONE + elif isinstance(cert_reqs, str): + CERT_REQS = { # noqa: N806 + "none": ssl.CERT_NONE, + "optional": ssl.CERT_OPTIONAL, + "required": ssl.CERT_REQUIRED, + } + if cert_reqs not in CERT_REQS: + raise RedisError( + f"Invalid SSL Certificate Requirements Flag: {cert_reqs}" + ) + self.cert_reqs = CERT_REQS[cert_reqs] + self.ca_certs = ca_certs + self.ca_data = ca_data + self.check_hostname = ( + check_hostname if self.cert_reqs != ssl.CERT_NONE else False + ) + self.min_version = min_version + self.ciphers = ciphers + self.context: Optional[SSLContext] = None + + def get(self) -> SSLContext: + if not self.context: + context = ssl.create_default_context() + context.check_hostname = self.check_hostname + context.verify_mode = self.cert_reqs + if self.certfile and self.keyfile: + context.load_cert_chain(certfile=self.certfile, keyfile=self.keyfile) + if self.ca_certs or self.ca_data: + context.load_verify_locations(cafile=self.ca_certs, cadata=self.ca_data) + if self.min_version is not None: + context.minimum_version = self.min_version + if self.ciphers is not None: + context.set_ciphers(self.ciphers) + self.context = context + return self.context + + +class UnixDomainSocketConnection(AbstractConnection): + "Manages UDS communication to and from a Redis server" + + def __init__(self, *, path: str = "", **kwargs): + self.path = path + super().__init__(**kwargs) + + def repr_pieces(self) -> Iterable[Tuple[str, Union[str, int]]]: + pieces = [("path", self.path), ("db", self.db)] + if self.client_name: + pieces.append(("client_name", self.client_name)) + return pieces + + async def _connect(self): + with anyio.fail_after(self.socket_connect_timeout): + self._stream = await anyio.connect_unix(self.path) + + await self.on_connect() + + def _host_error(self) -> str: + return self.path + + +FALSE_STRINGS = ("0", "F", "FALSE", "N", "NO") + + +def to_bool(value) -> Optional[bool]: + if value is None or value == "": + return None + if isinstance(value, str) and value.upper() in FALSE_STRINGS: + return False + return bool(value) + + +URL_QUERY_ARGUMENT_PARSERS: Mapping[str, Callable[..., object]] = MappingProxyType( + { + "db": int, + "socket_timeout": float, + "socket_connect_timeout": float, + "socket_keepalive": to_bool, + "retry_on_timeout": to_bool, + "max_connections": int, + "health_check_interval": int, + "ssl_check_hostname": to_bool, + "timeout": float, + } +) + + +class ConnectKwargs(TypedDict, total=False): + username: str + password: str + connection_class: Type[AbstractConnection] + host: str + port: int + db: int + path: str + + +def parse_url(url: str) -> ConnectKwargs: + parsed: ParseResult = urlparse(url) + kwargs: ConnectKwargs = {} + + for name, value_list in parse_qs(parsed.query).items(): + if value_list and len(value_list) > 0: + value = unquote(value_list[0]) + parser = URL_QUERY_ARGUMENT_PARSERS.get(name) + if parser: + try: + kwargs[name] = parser(value) + except (TypeError, ValueError): + raise ValueError(f"Invalid value for '{name}' in connection URL.") + else: + kwargs[name] = value + + if parsed.username: + kwargs["username"] = unquote(parsed.username) + if parsed.password: + kwargs["password"] = unquote(parsed.password) + + # We only support redis://, rediss:// and unix:// schemes. + if parsed.scheme == "unix": + if parsed.path: + kwargs["path"] = unquote(parsed.path) + kwargs["connection_class"] = UnixDomainSocketConnection + + elif parsed.scheme in ("redis", "rediss"): + if parsed.hostname: + kwargs["host"] = unquote(parsed.hostname) + if parsed.port: + kwargs["port"] = int(parsed.port) + + # If there's a path argument, use it as the db argument if a + # querystring value wasn't specified + if parsed.path and "db" not in kwargs: + try: + kwargs["db"] = int(unquote(parsed.path).replace("/", "")) + except (AttributeError, ValueError): + pass + + if parsed.scheme == "rediss": + kwargs["connection_class"] = SSLConnection + else: + valid_schemes = "redis://, rediss://, unix://" + raise ValueError( + f"Redis URL must specify one of the following schemes ({valid_schemes})" + ) + + return kwargs + + +_CP = TypeVar("_CP", bound="ConnectionPool") + + +class ConnectionPool: + """ + Create a connection pool. ``If max_connections`` is set, then this + object raises :py:class:`~redis.ConnectionError` when the pool's + limit is reached. + + By default, TCP connections are created unless ``connection_class`` + is specified. Use :py:class:`~redis.UnixDomainSocketConnection` for + unix sockets. + + Any additional keyword arguments are passed to the constructor of + ``connection_class``. + """ + + @classmethod + def from_url(cls: Type[_CP], url: str, **kwargs) -> _CP: + """ + Return a connection pool configured from the given URL. + + For example:: + + redis://[[username]:[password]]@localhost:6379/0 + rediss://[[username]:[password]]@localhost:6379/0 + unix://[username@]/path/to/socket.sock?db=0[&password=password] + + Three URL schemes are supported: + + - `redis://` creates a TCP socket connection. See more at: + + - `rediss://` creates a SSL wrapped TCP socket connection. See more at: + + - ``unix://``: creates a Unix Domain Socket connection. + + The username, password, hostname, path and all querystring values + are passed through urllib.parse.unquote in order to replace any + percent-encoded values with their corresponding characters. + + There are several ways to specify a database number. The first value + found will be used: + + 1. A ``db`` querystring option, e.g. redis://localhost?db=0 + + 2. If using the redis:// or rediss:// schemes, the path argument + of the url, e.g. redis://localhost/0 + + 3. A ``db`` keyword argument to this function. + + If none of these options are specified, the default db=0 is used. + + All querystring options are cast to their appropriate Python types. + Boolean arguments can be specified with string values "True"/"False" + or "Yes"/"No". Values that cannot be properly cast cause a + ``ValueError`` to be raised. Once parsed, the querystring arguments + and keyword arguments are passed to the ``ConnectionPool``'s + class initializer. In the case of conflicting arguments, querystring + arguments always win. + """ + url_options = parse_url(url) + kwargs.update(url_options) + return cls(**kwargs) + + def __init__( + self, + connection_class: Type[AbstractConnection] = Connection, + max_connections: Optional[int] = None, + **connection_kwargs, + ): + max_connections = max_connections or 2**31 + if not isinstance(max_connections, int) or max_connections < 0: + raise ValueError('"max_connections" must be a positive integer') + + self.connection_class = connection_class + self.connection_kwargs = connection_kwargs + self.max_connections = max_connections + + self._available_connections: List[AbstractConnection] = [] + self._in_use_connections: Set[AbstractConnection] = set() + self.encoder_class = self.connection_kwargs.get("encoder_class", Encoder) + self._lock = anyio.Lock() + self._event_dispatcher = self.connection_kwargs.get("event_dispatcher", None) + if self._event_dispatcher is None: + self._event_dispatcher = EventDispatcher() + + def __repr__(self): + return ( + f"<{self.__class__.__module__}.{self.__class__.__name__}" + f"({self.connection_class(**self.connection_kwargs)!r})>" + ) + + def reset(self): + self._available_connections = [] + self._in_use_connections = weakref.WeakSet() + + def can_get_connection(self) -> bool: + """Return True if a connection can be retrieved from the pool.""" + return ( + self._available_connections + or len(self._in_use_connections) < self.max_connections + ) + + @deprecated_args( + args_to_warn=["*"], + reason="Use get_connection() without args instead", + version="5.0.3", + ) + async def get_connection(self, command_name=None, *keys, **options): + async with self._lock: + """Get a connected connection from the pool""" + connection = self.get_available_connection() + try: + await self.ensure_connection(connection) + except BaseException: + await self.release(connection) + raise + + return connection + + def get_available_connection(self): + """Get a connection from the pool, without making sure it is connected""" + try: + connection = self._available_connections.pop() + except IndexError: + if len(self._in_use_connections) >= self.max_connections: + raise ConnectionError("Too many connections") from None + connection = self.make_connection() + self._in_use_connections.add(connection) + return connection + + def get_encoder(self): + """Return an encoder based on encoding settings""" + kwargs = self.connection_kwargs + return self.encoder_class( + encoding=kwargs.get("encoding", "utf-8"), + encoding_errors=kwargs.get("encoding_errors", "strict"), + decode_responses=kwargs.get("decode_responses", False), + ) + + def make_connection(self): + """Create a new connection. Can be overridden by child classes.""" + return self.connection_class(**self.connection_kwargs) + + async def ensure_connection(self, connection: AbstractConnection): + """Ensure that the connection object is connected and valid""" + await connection.connect() + # connections that the pool provides should be ready to send + # a command. if not, the connection was either returned to the + # pool before all data has been read or the socket has been + # closed. either way, reconnect and verify everything is good. + try: + if await connection.can_read_destructive(): + raise ConnectionError("Connection has data") from None + except (ConnectionError, TimeoutError, OSError): + await connection.disconnect() + await connection.connect() + if await connection.can_read_destructive(): + raise ConnectionError("Connection not ready") from None + + async def release(self, connection: AbstractConnection): + """Releases the connection back to the pool""" + # Connections should always be returned to the correct pool, + # not doing so is an error that will cause an exception here. + self._in_use_connections.remove(connection) + self._available_connections.append(connection) + await self._event_dispatcher.dispatch_async( + AsyncAfterConnectionReleasedEvent(connection) + ) + + async def disconnect(self, inuse_connections: bool = True): + """ + Disconnects connections in the pool + + If ``inuse_connections`` is True, disconnect connections that are + current in use, potentially by other tasks. Otherwise only disconnect + connections that are idle in the pool. + """ + if inuse_connections: + connections: Iterable[AbstractConnection] = chain( + self._available_connections, self._in_use_connections + ) + else: + connections = self._available_connections + + async with anyio.create_task_group() as tg: + for connection in connections: + tg.start_soon(connection.disconnect) + + async def aclose(self) -> None: + """Close the pool, disconnecting all connections""" + await self.disconnect() + + def set_retry(self, retry: "Retry") -> None: + for conn in self._available_connections: + conn.retry = retry + for conn in self._in_use_connections: + conn.retry = retry + + async def re_auth_callback(self, token: TokenInterface): + async with self._lock: + for conn in self._available_connections: + await conn.retry.call_with_retry( + lambda: conn.send_command( + "AUTH", token.try_get("oid"), token.get_value() + ), + lambda error: self._mock(error), + ) + await conn.retry.call_with_retry( + lambda: conn.read_response(), lambda error: self._mock(error) + ) + for conn in self._in_use_connections: + conn.set_re_auth_token(token) + + async def _mock(self, error: RedisError): + """ + Dummy functions, needs to be passed as error callback to retry object. + :param error: + :return: + """ + pass + + +class BlockingConnectionPool(ConnectionPool): + """ + A blocking connection pool:: + + >>> from redis.anyio import Redis, BlockingConnectionPool + >>> client = Redis.from_pool(BlockingConnectionPool()) + + It performs the same function as the default + :py:class:`~redis.anyio.ConnectionPool` implementation, in that, + it maintains a pool of reusable connections that can be shared by + multiple async redis clients. + + The difference is that, in the event that a client tries to get a + connection from the pool when all of connections are in use, rather than + raising a :py:class:`~redis.ConnectionError` (as the default + :py:class:`~redis.anyio.ConnectionPool` implementation does), it + blocks the current `Task` for a specified number of seconds until + a connection becomes available. + + Use ``max_connections`` to increase / decrease the pool size:: + + >>> pool = BlockingConnectionPool(max_connections=10) + + Use ``timeout`` to tell it either how many seconds to wait for a connection + to become available, or to block forever: + + >>> # Block forever. + >>> pool = BlockingConnectionPool(timeout=None) + + >>> # Raise a ``ConnectionError`` after five seconds if a connection is + >>> # not available. + >>> pool = BlockingConnectionPool(timeout=5) + """ + + def __init__( + self, + max_connections: int = 50, + timeout: Optional[int] = 20, + connection_class: Type[AbstractConnection] = Connection, + **connection_kwargs, + ): + super().__init__( + connection_class=connection_class, + max_connections=max_connections, + **connection_kwargs, + ) + self._condition = anyio.Condition() + self.timeout = timeout + + @deprecated_args( + args_to_warn=["*"], + reason="Use get_connection() without args instead", + version="5.0.3", + ) + async def get_connection(self, command_name=None, *keys, **options): + """Gets a connection from the pool, blocking until one is available""" + try: + async with self._condition: + with anyio.fail_after(self.timeout): + await wait_for_condition(self._condition, self.can_get_connection) + connection = super().get_available_connection() + except builtins.TimeoutError as err: + raise ConnectionError("No connection available.") from err + + # We now perform the connection check outside of the lock. + try: + await self.ensure_connection(connection) + return connection + except BaseException: + await self.release(connection) + raise + + async def release(self, connection: AbstractConnection): + """Releases the connection back to the pool.""" + async with self._condition: + await super().release(connection) + self._condition.notify() diff --git a/redis/anyio/lock.py b/redis/anyio/lock.py new file mode 100644 index 0000000000..012965ab6e --- /dev/null +++ b/redis/anyio/lock.py @@ -0,0 +1,335 @@ +import logging +import threading +import uuid +from types import SimpleNamespace +from typing import TYPE_CHECKING, Awaitable, Optional, Union + +import anyio + +from redis.exceptions import LockError, LockNotOwnedError +from redis.typing import Number + +if TYPE_CHECKING: + from redis.anyio import Redis, RedisCluster + +logger = logging.getLogger(__name__) + + +class Lock: + """ + A shared, distributed Lock. Using Redis for locking allows the Lock + to be shared across processes and/or machines. + + It's left to the user to resolve deadlock issues and make sure + multiple clients play nicely together. + """ + + lua_release = None + lua_extend = None + lua_reacquire = None + + # KEYS[1] - lock name + # ARGV[1] - token + # return 1 if the lock was released, otherwise 0 + LUA_RELEASE_SCRIPT = """ + local token = redis.call('get', KEYS[1]) + if not token or token ~= ARGV[1] then + return 0 + end + redis.call('del', KEYS[1]) + return 1 + """ + + # KEYS[1] - lock name + # ARGV[1] - token + # ARGV[2] - additional milliseconds + # ARGV[3] - "0" if the additional time should be added to the lock's + # existing ttl or "1" if the existing ttl should be replaced + # return 1 if the locks time was extended, otherwise 0 + LUA_EXTEND_SCRIPT = """ + local token = redis.call('get', KEYS[1]) + if not token or token ~= ARGV[1] then + return 0 + end + local expiration = redis.call('pttl', KEYS[1]) + if not expiration then + expiration = 0 + end + if expiration < 0 then + return 0 + end + + local newttl = ARGV[2] + if ARGV[3] == "0" then + newttl = ARGV[2] + expiration + end + redis.call('pexpire', KEYS[1], newttl) + return 1 + """ + + # KEYS[1] - lock name + # ARGV[1] - token + # ARGV[2] - milliseconds + # return 1 if the locks time was reacquired, otherwise 0 + LUA_REACQUIRE_SCRIPT = """ + local token = redis.call('get', KEYS[1]) + if not token or token ~= ARGV[1] then + return 0 + end + redis.call('pexpire', KEYS[1], ARGV[2]) + return 1 + """ + + def __init__( + self, + redis: Union["Redis", "RedisCluster"], + name: Union[str, bytes, memoryview], + timeout: Optional[float] = None, + sleep: float = 0.1, + blocking: bool = True, + blocking_timeout: Optional[Number] = None, + thread_local: bool = True, + raise_on_release_error: bool = True, + ): + """ + Create a new Lock instance named ``name`` using the Redis client + supplied by ``redis``. + + ``timeout`` indicates a maximum life for the lock in seconds. + By default, it will remain locked until release() is called. + ``timeout`` can be specified as a float or integer, both representing + the number of seconds to wait. + + ``sleep`` indicates the amount of time to sleep in seconds per loop + iteration when the lock is in blocking mode and another client is + currently holding the lock. + + ``blocking`` indicates whether calling ``acquire`` should block until + the lock has been acquired or to fail immediately, causing ``acquire`` + to return False and the lock not being acquired. Defaults to True. + Note this value can be overridden by passing a ``blocking`` + argument to ``acquire``. + + ``blocking_timeout`` indicates the maximum amount of time in seconds to + spend trying to acquire the lock. A value of ``None`` indicates + continue trying forever. ``blocking_timeout`` can be specified as a + float or integer, both representing the number of seconds to wait. + + ``thread_local`` indicates whether the lock token is placed in + thread-local storage. By default, the token is placed in thread local + storage so that a thread only sees its token, not a token set by + another thread. Consider the following timeline: + + time: 0, thread-1 acquires `my-lock`, with a timeout of 5 seconds. + thread-1 sets the token to "abc" + time: 1, thread-2 blocks trying to acquire `my-lock` using the + Lock instance. + time: 5, thread-1 has not yet completed. redis expires the lock + key. + time: 5, thread-2 acquired `my-lock` now that it's available. + thread-2 sets the token to "xyz" + time: 6, thread-1 finishes its work and calls release(). if the + token is *not* stored in thread local storage, then + thread-1 would see the token value as "xyz" and would be + able to successfully release the thread-2's lock. + + ``raise_on_release_error`` indicates whether to raise an exception when + the lock is no longer owned when exiting the context manager. By default, + this is True, meaning an exception will be raised. If False, the warning + will be logged and the exception will be suppressed. + + In some use cases it's necessary to disable thread local storage. For + example, if you have code where one thread acquires a lock and passes + that lock instance to a worker thread to release later. If thread + local storage isn't disabled in this case, the worker thread won't see + the token set by the thread that acquired the lock. Our assumption + is that these cases aren't common and as such default to using + thread local storage. + """ + self.redis = redis + self.name = name + self.timeout = timeout + self.sleep = sleep + self.blocking = blocking + self.blocking_timeout = blocking_timeout + self.thread_local = bool(thread_local) + self.local = threading.local() if self.thread_local else SimpleNamespace() + self.raise_on_release_error = raise_on_release_error + self.local.token = None + self.register_scripts() + + def register_scripts(self): + cls = self.__class__ + client = self.redis + if cls.lua_release is None: + cls.lua_release = client.register_script(cls.LUA_RELEASE_SCRIPT) + if cls.lua_extend is None: + cls.lua_extend = client.register_script(cls.LUA_EXTEND_SCRIPT) + if cls.lua_reacquire is None: + cls.lua_reacquire = client.register_script(cls.LUA_REACQUIRE_SCRIPT) + + async def __aenter__(self): + if await self.acquire(): + return self + raise LockError("Unable to acquire lock within the time specified") + + async def __aexit__(self, exc_type, exc_value, traceback): + try: + await self.release() + except LockError: + if self.raise_on_release_error: + raise + logger.warning( + "Lock was unlocked or no longer owned when exiting context manager." + ) + + async def acquire( + self, + blocking: Optional[bool] = None, + blocking_timeout: Optional[Number] = None, + token: Optional[Union[str, bytes]] = None, + ): + """ + Use Redis to hold a shared, distributed lock named ``name``. + Returns True once the lock is acquired. + + If ``blocking`` is False, always return immediately. If the lock + was acquired, return True, otherwise return False. + + ``blocking_timeout`` specifies the maximum number of seconds to + wait trying to acquire the lock. + + ``token`` specifies the token value to be used. If provided, token + must be a bytes object or a string that can be encoded to a bytes + object with the default encoding. If a token isn't specified, a UUID + will be generated. + """ + sleep = self.sleep + if token is None: + token = uuid.uuid1().hex.encode() + else: + try: + encoder = self.redis.connection_pool.get_encoder() + except AttributeError: + # Cluster + encoder = self.redis.get_encoder() + token = encoder.encode(token) + if blocking is None: + blocking = self.blocking + if blocking_timeout is None: + blocking_timeout = self.blocking_timeout + stop_trying_at = None + if blocking_timeout is not None: + stop_trying_at = anyio.current_time() + blocking_timeout + while True: + if await self.do_acquire(token): + self.local.token = token + return True + if not blocking: + return False + next_try_at = anyio.current_time() + sleep + if stop_trying_at is not None and next_try_at > stop_trying_at: + return False + await anyio.sleep(sleep) + + async def do_acquire(self, token: Union[str, bytes]) -> bool: + if self.timeout: + # convert to milliseconds + timeout = int(self.timeout * 1000) + else: + timeout = None + if await self.redis.set(self.name, token, nx=True, px=timeout): + return True + return False + + async def locked(self) -> bool: + """ + Returns True if this key is locked by any process, otherwise False. + """ + return await self.redis.get(self.name) is not None + + async def owned(self) -> bool: + """ + Returns True if this key is locked by this lock, otherwise False. + """ + stored_token = await self.redis.get(self.name) + # need to always compare bytes to bytes + # TODO: this can be simplified when the context manager is finished + if stored_token and not isinstance(stored_token, bytes): + try: + encoder = self.redis.connection_pool.get_encoder() + except AttributeError: + # Cluster + encoder = self.redis.get_encoder() + stored_token = encoder.encode(stored_token) + return self.local.token is not None and stored_token == self.local.token + + def release(self) -> Awaitable[None]: + """Releases the already acquired lock""" + expected_token = self.local.token + if expected_token is None: + raise LockError( + "Cannot release a lock that's not owned or is already unlocked.", + lock_name=self.name, + ) + self.local.token = None + return self.do_release(expected_token) + + async def do_release(self, expected_token: bytes) -> None: + if not bool( + await self.lua_release( + keys=[self.name], args=[expected_token], client=self.redis + ) + ): + raise LockNotOwnedError("Cannot release a lock that's no longer owned") + + def extend( + self, additional_time: Number, replace_ttl: bool = False + ) -> Awaitable[bool]: + """ + Adds more time to an already acquired lock. + + ``additional_time`` can be specified as an integer or a float, both + representing the number of seconds to add. + + ``replace_ttl`` if False (the default), add `additional_time` to + the lock's existing ttl. If True, replace the lock's ttl with + `additional_time`. + """ + if self.local.token is None: + raise LockError("Cannot extend an unlocked lock") + if self.timeout is None: + raise LockError("Cannot extend a lock with no timeout") + return self.do_extend(additional_time, replace_ttl) + + async def do_extend(self, additional_time, replace_ttl) -> bool: + additional_time = int(additional_time * 1000) + if not bool( + await self.lua_extend( + keys=[self.name], + args=[self.local.token, additional_time, replace_ttl and "1" or "0"], + client=self.redis, + ) + ): + raise LockNotOwnedError("Cannot extend a lock that's no longer owned") + return True + + def reacquire(self) -> Awaitable[bool]: + """ + Resets a TTL of an already acquired lock back to a timeout value. + """ + if self.local.token is None: + raise LockError("Cannot reacquire an unlocked lock") + if self.timeout is None: + raise LockError("Cannot reacquire a lock with no timeout") + return self.do_reacquire() + + async def do_reacquire(self) -> bool: + timeout = int(self.timeout * 1000) + if not bool( + await self.lua_reacquire( + keys=[self.name], args=[self.local.token, timeout], client=self.redis + ) + ): + raise LockNotOwnedError("Cannot reacquire a lock that's no longer owned") + return True diff --git a/redis/anyio/retry.py b/redis/anyio/retry.py new file mode 100644 index 0000000000..b6261b1ed3 --- /dev/null +++ b/redis/anyio/retry.py @@ -0,0 +1,68 @@ +from typing import TYPE_CHECKING, Any, Awaitable, Callable, Tuple, Type, TypeVar + +import anyio + +from redis.exceptions import ConnectionError, RedisError, TimeoutError + +if TYPE_CHECKING: + from redis.backoff import AbstractBackoff + + +T = TypeVar("T") + + +class Retry: + """Retry a specific number of times after a failure""" + + __slots__ = "_backoff", "_retries", "_supported_errors" + + def __init__( + self, + backoff: "AbstractBackoff", + retries: int, + supported_errors: Tuple[Type[RedisError], ...] = ( + ConnectionError, + TimeoutError, + ), + ): + """ + Initialize a `Retry` object with a `Backoff` object + that retries a maximum of `retries` times. + `retries` can be negative to retry forever. + You can specify the types of supported errors which trigger + a retry with the `supported_errors` parameter. + """ + self._backoff = backoff + self._retries = retries + self._supported_errors = supported_errors + + def update_supported_errors(self, specified_errors: list): + """ + Updates the supported errors with the specified error types + """ + self._supported_errors = tuple( + set(self._supported_errors + tuple(specified_errors)) + ) + + async def call_with_retry( + self, do: Callable[[], Awaitable[T]], fail: Callable[[RedisError], Any] + ) -> T: + """ + Execute an operation that might fail and returns its result, or + raise the exception that was thrown depending on the `Backoff` object. + `do`: the operation to call. Expects no argument. + `fail`: the failure handler, expects the last error that was thrown + """ + self._backoff.reset() + failures = 0 + while True: + try: + return await do() + except self._supported_errors as error: + failures += 1 + await fail(error) + if self._retries >= 0 and failures > self._retries: + raise error + backoff = self._backoff.compute(failures) + if backoff > 0: + await anyio.sleep(backoff) diff --git a/redis/anyio/sentinel.py b/redis/anyio/sentinel.py new file mode 100644 index 0000000000..5f82043669 --- /dev/null +++ b/redis/anyio/sentinel.py @@ -0,0 +1,390 @@ +import random +import weakref +from typing import AsyncIterator, Iterable, Mapping, Optional, Sequence, Tuple, Type + +import anyio + +from redis.anyio.client import Redis +from redis.anyio.compat import aclosing +from redis.anyio.connection import ( + Connection, + ConnectionPool, + EncodableT, + SSLConnection, +) +from redis.commands import AsyncSentinelCommands +from redis.exceptions import ConnectionError, ReadOnlyError, ResponseError, TimeoutError +from redis.utils import str_if_bytes + + +class MasterNotFoundError(ConnectionError): + pass + + +class SlaveNotFoundError(ConnectionError): + pass + + +class SentinelManagedConnection(Connection): + def __init__(self, **kwargs): + self.connection_pool = kwargs.pop("connection_pool") + super().__init__(**kwargs) + + def __repr__(self): + s = f"<{self.__class__.__module__}.{self.__class__.__name__}" + if self.host: + host_info = f",host={self.host},port={self.port}" + s += host_info + return s + ")>" + + async def connect_to(self, address): + self.host, self.port = address + await super().connect() + if self.connection_pool.check_connection: + await self.send_command("PING") + if str_if_bytes(await self.read_response()) != "PONG": + raise ConnectionError("PING failed") + + async def _connect_retry(self): + if self._stream: + return # already connected + if self.connection_pool.is_master: + await self.connect_to(await self.connection_pool.get_master_address()) + else: + async with aclosing(self.connection_pool.rotate_slaves()) as slaves: + async for slave in slaves: + try: + return await self.connect_to(slave) + except ConnectionError: + continue + + raise SlaveNotFoundError # Never be here + + async def connect(self): + return await self.retry.call_with_retry( + self._connect_retry, + lambda error: anyio.sleep(0), + ) + + async def read_response( + self, + disable_decoding: bool = False, + timeout: Optional[float] = None, + *, + disconnect_on_error: Optional[float] = True, + push_request: Optional[bool] = False, + ): + try: + return await super().read_response( + disable_decoding=disable_decoding, + timeout=timeout, + disconnect_on_error=disconnect_on_error, + push_request=push_request, + ) + except ReadOnlyError: + if self.connection_pool.is_master: + # When talking to a master, a ReadOnlyError when likely + # indicates that the previous master that we're still connected + # to has been demoted to a slave and there's a new master. + # calling disconnect will force the connection to re-query + # sentinel during the next connect() attempt. + await self.disconnect() + raise ConnectionError("The previous master is now a slave") + raise + + +class SentinelManagedSSLConnection(SentinelManagedConnection, SSLConnection): + pass + + +class SentinelConnectionPool(ConnectionPool): + """ + Sentinel backed connection pool. + + If ``check_connection`` flag is set to True, SentinelManagedConnection + sends a PING command right after establishing the connection. + """ + + def __init__(self, service_name, sentinel_manager, **kwargs): + kwargs["connection_class"] = kwargs.get( + "connection_class", + ( + SentinelManagedSSLConnection + if kwargs.pop("ssl", False) + else SentinelManagedConnection + ), + ) + self.is_master = kwargs.pop("is_master", True) + self.check_connection = kwargs.pop("check_connection", False) + super().__init__(**kwargs) + self.connection_kwargs["connection_pool"] = weakref.proxy(self) + self.service_name = service_name + self.sentinel_manager = sentinel_manager + self.master_address = None + self.slave_rr_counter = None + + def __repr__(self): + return ( + f"<{self.__class__.__module__}.{self.__class__.__name__}" + f"(service={self.service_name}({self.is_master and 'master' or 'slave'}))>" + ) + + def reset(self): + super().reset() + self.master_address = None + self.slave_rr_counter = None + + def owns_connection(self, connection: Connection): + check = not self.is_master or ( + self.is_master and self.master_address == (connection.host, connection.port) + ) + return check and super().owns_connection(connection) + + async def get_master_address(self): + master_address = await self.sentinel_manager.discover_master(self.service_name) + if self.is_master: + if self.master_address != master_address: + self.master_address = master_address + # disconnect any idle connections so that they reconnect + # to the new master the next time that they are used. + await self.disconnect(inuse_connections=False) + return master_address + + async def rotate_slaves(self) -> AsyncIterator: + """Round-robin slave balancer""" + slaves = await self.sentinel_manager.discover_slaves(self.service_name) + if slaves: + if self.slave_rr_counter is None: + self.slave_rr_counter = random.randint(0, len(slaves) - 1) + for _ in range(len(slaves)): + self.slave_rr_counter = (self.slave_rr_counter + 1) % len(slaves) + slave = slaves[self.slave_rr_counter] + yield slave + # Fallback to the master connection + try: + yield await self.get_master_address() + except MasterNotFoundError: + pass + raise SlaveNotFoundError(f"No slave found for {self.service_name!r}") + + +class Sentinel(AsyncSentinelCommands): + """ + Redis Sentinel cluster client + + >>> from redis.sentinel import Sentinel + >>> sentinel = Sentinel([('localhost', 26379)], socket_timeout=0.1) + >>> master = sentinel.master_for('mymaster', socket_timeout=0.1) + >>> await master.set('foo', 'bar') + >>> slave = sentinel.slave_for('mymaster', socket_timeout=0.1) + >>> await slave.get('foo') + b'bar' + + ``sentinels`` is a list of sentinel nodes. Each node is represented by + a pair (hostname, port). + + ``min_other_sentinels`` defined a minimum number of peers for a sentinel. + When querying a sentinel, if it doesn't meet this threshold, responses + from that sentinel won't be considered valid. + + ``sentinel_kwargs`` is a dictionary of connection arguments used when + connecting to sentinel instances. Any argument that can be passed to + a normal Redis connection can be specified here. If ``sentinel_kwargs`` is + not specified, any socket_timeout and socket_keepalive options specified + in ``connection_kwargs`` will be used. + + ``connection_kwargs`` are keyword arguments that will be used when + establishing a connection to a Redis server. + """ + + def __init__( + self, + sentinels, + min_other_sentinels=0, + sentinel_kwargs=None, + force_master_ip=None, + **connection_kwargs, + ): + # if sentinel_kwargs isn't defined, use the socket_* options from + # connection_kwargs + if sentinel_kwargs is None: + sentinel_kwargs = { + k: v for k, v in connection_kwargs.items() if k.startswith("socket_") + } + self.sentinel_kwargs = sentinel_kwargs + + self.sentinels = [ + Redis(host=hostname, port=port, **self.sentinel_kwargs) + for hostname, port in sentinels + ] + self.min_other_sentinels = min_other_sentinels + self.connection_kwargs = connection_kwargs + self._force_master_ip = force_master_ip + + async def execute_command(self, *args, **kwargs): + """ + Execute Sentinel command in sentinel nodes. + once - If set to True, then execute the resulting command on a single + node at random, rather than across the entire sentinel cluster. + """ + once = bool(kwargs.get("once", False)) + if "once" in kwargs.keys(): + kwargs.pop("once") + + if once: + await random.choice(self.sentinels).execute_command(*args, **kwargs) + else: + async with anyio.create_task_group() as tg: + for sentinel in self.sentinels: + tg.start_soon(lambda: sentinel.execute_command(*args, **kwargs)) + + return True + + def __repr__(self): + sentinel_addresses = [] + for sentinel in self.sentinels: + sentinel_addresses.append( + f"{sentinel.connection_pool.connection_kwargs['host']}:" + f"{sentinel.connection_pool.connection_kwargs['port']}" + ) + return ( + f"<{self.__class__}.{self.__class__.__name__}" + f"(sentinels=[{','.join(sentinel_addresses)}])>" + ) + + def check_master_state(self, state: dict, service_name: str) -> bool: + if not state["is_master"] or state["is_sdown"] or state["is_odown"]: + return False + # Check if our sentinel doesn't see other nodes + if state["num-other-sentinels"] < self.min_other_sentinels: + return False + return True + + async def discover_master(self, service_name: str): + """ + Asks sentinel servers for the Redis master's address corresponding + to the service labeled ``service_name``. + + Returns a pair (address, port) or raises MasterNotFoundError if no + master is found. + """ + collected_errors = list() + for sentinel_no, sentinel in enumerate(self.sentinels): + try: + masters = await sentinel.sentinel_masters() + except (ConnectionError, TimeoutError) as e: + collected_errors.append(f"{sentinel} - {e!r}") + continue + state = masters.get(service_name) + if state and self.check_master_state(state, service_name): + # Put this sentinel at the top of the list + self.sentinels[0], self.sentinels[sentinel_no] = ( + sentinel, + self.sentinels[0], + ) + + ip = ( + self._force_master_ip + if self._force_master_ip is not None + else state["ip"] + ) + return ip, state["port"] + + error_info = "" + if len(collected_errors) > 0: + error_info = f" : {', '.join(collected_errors)}" + raise MasterNotFoundError(f"No master found for {service_name!r}{error_info}") + + def filter_slaves( + self, slaves: Iterable[Mapping] + ) -> Sequence[Tuple[EncodableT, EncodableT]]: + """Remove slaves that are in an ODOWN or SDOWN state""" + slaves_alive = [] + for slave in slaves: + if slave["is_odown"] or slave["is_sdown"]: + continue + slaves_alive.append((slave["ip"], slave["port"])) + return slaves_alive + + async def discover_slaves( + self, service_name: str + ) -> Sequence[Tuple[EncodableT, EncodableT]]: + """Returns a list of alive slaves for service ``service_name``""" + for sentinel in self.sentinels: + try: + slaves = await sentinel.sentinel_slaves(service_name) + except (ConnectionError, ResponseError, TimeoutError): + continue + slaves = self.filter_slaves(slaves) + if slaves: + return slaves + return [] + + def master_for( + self, + service_name: str, + redis_class: Type[Redis] = Redis, + connection_pool_class: Type[SentinelConnectionPool] = SentinelConnectionPool, + **kwargs, + ): + """ + Returns a redis client instance for the ``service_name`` master. + + A :py:class:`~redis.sentinel.SentinelConnectionPool` class is + used to retrieve the master's address before establishing a new + connection. + + NOTE: If the master's address has changed, any cached connections to + the old master are closed. + + By default clients will be a :py:class:`~redis.Redis` instance. + Specify a different class to the ``redis_class`` argument if you + desire something different. + + The ``connection_pool_class`` specifies the connection pool to + use. The :py:class:`~redis.sentinel.SentinelConnectionPool` + will be used by default. + + All other keyword arguments are merged with any connection_kwargs + passed to this class and passed to the connection pool as keyword + arguments to be used to initialize Redis connections. + """ + kwargs["is_master"] = True + connection_kwargs = dict(self.connection_kwargs) + connection_kwargs.update(kwargs) + + connection_pool = connection_pool_class(service_name, self, **connection_kwargs) + # The Redis object "owns" the pool + return redis_class.from_pool(connection_pool) + + def slave_for( + self, + service_name: str, + redis_class: Type[Redis] = Redis, + connection_pool_class: Type[SentinelConnectionPool] = SentinelConnectionPool, + **kwargs, + ): + """ + Returns redis client instance for the ``service_name`` slave(s). + + A SentinelConnectionPool class is used to retrieve the slave's + address before establishing a new connection. + + By default clients will be a :py:class:`~redis.Redis` instance. + Specify a different class to the ``redis_class`` argument if you + desire something different. + + The ``connection_pool_class`` specifies the connection pool to use. + The SentinelConnectionPool will be used by default. + + All other keyword arguments are merged with any connection_kwargs + passed to this class and passed to the connection pool as keyword + arguments to be used to initialize Redis connections. + """ + kwargs["is_master"] = False + connection_kwargs = dict(self.connection_kwargs) + connection_kwargs.update(kwargs) + + connection_pool = connection_pool_class(service_name, self, **connection_kwargs) + # The Redis object "owns" the pool + return redis_class.from_pool(connection_pool) diff --git a/redis/anyio/utils.py b/redis/anyio/utils.py new file mode 100644 index 0000000000..00ea2fb824 --- /dev/null +++ b/redis/anyio/utils.py @@ -0,0 +1,55 @@ +from __future__ import annotations + +from collections.abc import Callable, Coroutine, Sequence +from typing import TYPE_CHECKING, Any + +import anyio + +if TYPE_CHECKING: + from redis.anyio.client import Pipeline, Redis + + +def from_url(url, **kwargs): + """ + Returns an active Redis client generated from the given database URL. + + Will attempt to extract the database id from the path url fragment, if + none is provided. + """ + from redis.anyio.client import Redis + + return Redis.from_url(url, **kwargs) + + +class pipeline: # noqa: N801 + def __init__(self, redis_obj: "Redis"): + self.p: "Pipeline" = redis_obj.pipeline() + + async def __aenter__(self) -> "Pipeline": + return self.p + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.p.execute() + del self.p + + +async def gather(*coros: Coroutine[Any, Any, Any]) -> Sequence[Any]: + results = [None] * len(coros) + + async def run_coro(coro: Coroutine[Any, Any, Any], index: int) -> None: + results[index] = await coro + + async with anyio.create_task_group() as tg: + for i, coro in enumerate(coros): + tg.start_soon(run_coro, coro, i) + + return results + + +async def wait_for_condition(cond: anyio.Condition, predicate: Callable) -> None: + result = predicate() + while not result: + await cond.wait() + result = predicate() + + return result diff --git a/redis/commands/core.py b/redis/commands/core.py index 378898272f..0eae2456d2 100644 --- a/redis/commands/core.py +++ b/redis/commands/core.py @@ -53,6 +53,7 @@ from .helpers import list_or_args if TYPE_CHECKING: + import redis.anyio.client import redis.asyncio.client import redis.client @@ -1449,7 +1450,11 @@ class BitFieldOperation: def __init__( self, - client: Union["redis.client.Redis", "redis.asyncio.client.Redis"], + client: Union[ + "redis.client.Redis", + "redis.asyncio.client.Redis", + "redis.anyio.client.Redis", + ], key: str, default_overflow: Optional[str] = None, ): @@ -1593,7 +1598,11 @@ def bitcount( return self.execute_command("BITCOUNT", *params, keys=[key]) def bitfield( - self: Union["redis.client.Redis", "redis.asyncio.client.Redis"], + self: Union[ + "redis.client.Redis", + "redis.asyncio.client.Redis", + "redis.anyio.client.Redis", + ], key: KeyT, default_overflow: Optional[str] = None, ) -> BitFieldOperation: @@ -1606,7 +1615,11 @@ def bitfield( return BitFieldOperation(self, key, default_overflow=default_overflow) def bitfield_ro( - self: Union["redis.client.Redis", "redis.asyncio.client.Redis"], + self: Union[ + "redis.client.Redis", + "redis.asyncio.client.Redis", + "redis.anyio.client.Redis", + ], key: KeyT, encoding: str, offset: BitfieldOffsetT, @@ -5671,7 +5684,7 @@ class AsyncScript: def __init__( self, - registered_client: "redis.asyncio.client.Redis", + registered_client: "redis.asyncio.client.Redis | redis.anyio.client.Redis", script: ScriptTextT, ): self.registered_client = registered_client @@ -5693,7 +5706,9 @@ async def __call__( self, keys: Union[Sequence[KeyT], None] = None, args: Union[Iterable[EncodableT], None] = None, - client: Union["redis.asyncio.client.Redis", None] = None, + client: Union[ + "redis.asyncio.client.Redis", "redis.anyio.client.Redis", None + ] = None, ): """Execute the script, passing any required ``args``""" keys = keys or [] @@ -5702,9 +5717,10 @@ async def __call__( client = self.registered_client args = tuple(keys) + tuple(args) # make sure the Redis server knows about the script - from redis.asyncio.client import Pipeline + from redis.anyio.client import Pipeline as AnyioPipeline + from redis.asyncio.client import Pipeline as AsyncioPipeline - if isinstance(client, Pipeline): + if isinstance(client, (AsyncioPipeline, AnyioPipeline)): # Make sure the pipeline can register the script before executing. client.scripts.add(self) try: @@ -5933,7 +5949,7 @@ async def script_debug(self, *args) -> None: return super().script_debug() def register_script( - self: "redis.asyncio.client.Redis", + self: "redis.asyncio.client.Redis | redis.anyio.client.Redis", script: ScriptTextT, ) -> AsyncScript: """ diff --git a/redis/commands/helpers.py b/redis/commands/helpers.py index 859a43aea9..e1f1c9a5bb 100644 --- a/redis/commands/helpers.py +++ b/redis/commands/helpers.py @@ -112,7 +112,7 @@ def decode_dict_keys(obj): def get_protocol_version(client): - if isinstance(client, redis.Redis) or isinstance(client, redis.asyncio.Redis): + if isinstance(client, (redis.Redis, redis.asyncio.Redis, redis.anyio.Redis)): return client.connection_pool.connection_kwargs.get("protocol") elif isinstance(client, redis.cluster.AbstractRedisCluster): return client.nodes_manager.connection_kwargs.get("protocol") diff --git a/tests/conftest.py b/tests/conftest.py index 7eaccb1acb..5aba13a914 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -658,7 +658,7 @@ def wait_for_command(client, monitor, command, key=None): def is_resp2_connection(r): - if isinstance(r, redis.Redis) or isinstance(r, redis.asyncio.Redis): + if isinstance(r, (redis.Redis, redis.asyncio.Redis, redis.anyio.Redis)): protocol = r.connection_pool.connection_kwargs.get("protocol") elif isinstance(r, redis.cluster.AbstractRedisCluster): protocol = r.nodes_manager.connection_kwargs.get("protocol") @@ -666,7 +666,7 @@ def is_resp2_connection(r): def get_protocol_version(r): - if isinstance(r, redis.Redis) or isinstance(r, redis.asyncio.Redis): + if isinstance(r, (redis.Redis, redis.asyncio.Redis, redis.anyio.Redis)): return r.connection_pool.connection_kwargs.get("protocol") elif isinstance(r, redis.cluster.AbstractRedisCluster): return r.nodes_manager.connection_kwargs.get("protocol") diff --git a/tests/test_anyio/__init__.py b/tests/test_anyio/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/tests/test_anyio/conftest.py b/tests/test_anyio/conftest.py new file mode 100644 index 0000000000..c52115f7db --- /dev/null +++ b/tests/test_anyio/conftest.py @@ -0,0 +1,258 @@ +from __future__ import annotations + +import random +from collections.abc import Callable +from contextlib import AbstractAsyncContextManager, AsyncExitStack, asynccontextmanager +from unittest.mock import AsyncMock, patch + +import pytest + +import redis.anyio as redis +from packaging.version import Version +from redis.anyio import Sentinel +from redis.anyio.client import Monitor +from redis.anyio.connection import Connection, parse_url +from redis.anyio.retry import Retry +from redis.backoff import NoBackoff +from redis.credentials import CredentialProvider +from tests.conftest import REDIS_INFO, get_credential_provider + + +async def _get_info(redis_url): + client = redis.Redis.from_url(redis_url) + info = await client.info() + await client.connection_pool.disconnect() + return info + + +@pytest.fixture( + params=[ + pytest.param( + True, + marks=pytest.mark.skipif( + 'config.REDIS_INFO["cluster_enabled"]', reason="cluster mode enabled" + ), + id="single", + ), + pytest.param(False, id="pool"), + ], +) +async def create_redis( + request, +) -> Callable[..., AbstractAsyncContextManager[redis.Redis]]: + """Wrapper around redis.create_redis.""" + single_connection = request.param + + @asynccontextmanager + async def client_factory( + url: str = request.config.getoption("--redis-url"), + cls=redis.Redis, + flushdb=True, + **kwargs, + ): + if "protocol" not in url and kwargs.get("protocol") is None: + kwargs["protocol"] = request.config.getoption("--protocol") + + async with AsyncExitStack() as exit_stack: + cluster_mode = REDIS_INFO["cluster_enabled"] + if not cluster_mode: + single = ( + kwargs.pop("single_connection_client", False) or single_connection + ) + url_options = parse_url(url) + url_options.update(kwargs) + pool = redis.ConnectionPool(**url_options) + exit_stack.push_async_callback(pool.disconnect) + client = await exit_stack.enter_async_context(cls(connection_pool=pool)) + else: + client = await exit_stack.enter_async_context( + redis.RedisCluster.from_url(url, **kwargs) + ) + single = False + + if single: + client = await exit_stack.enter_async_context(client.client()) + + yield client + + if not cluster_mode: + if flushdb and "username" not in kwargs: + try: + await client.flushdb() + except redis.ConnectionError: + # handle cases where a test disconnected a client + # just manually retry the flushdb + await client.flushdb() + else: + if flushdb: + try: + await client.flushdb(target_nodes="primaries") + except redis.ConnectionError: + # handle cases where a test disconnected a client + # just manually retry the flushdb + await client.flushdb(target_nodes="primaries") + + return client_factory + + +@pytest.fixture +async def r(create_redis): + async with create_redis() as client: + yield client + + +@pytest.fixture +async def r2(create_redis): + """A second client for tests that need multiple""" + async with create_redis() as client: + yield client + + +@pytest.fixture +async def decoded_r(create_redis): + async with create_redis(decode_responses=True) as client: + yield client + + +@pytest.fixture +async def sentinel_setup(local_cache, request): + sentinel_ips = request.config.getoption("--sentinels") + sentinel_endpoints = [ + (ip.strip(), int(port.strip())) + for ip, port in (endpoint.split(":") for endpoint in sentinel_ips.split(",")) + ] + kwargs = request.param.get("kwargs", {}) if hasattr(request, "param") else {} + force_master_ip = request.param.get("force_master_ip", None) + sentinel = Sentinel( + sentinel_endpoints, + force_master_ip=force_master_ip, + socket_timeout=0.1, + client_cache=local_cache, + protocol=3, + **kwargs, + ) + yield sentinel + for s in sentinel.sentinels: + await s.aclose() + + +@pytest.fixture +async def master(request, sentinel_setup): + master_service = request.config.getoption("--master-service") + master = sentinel_setup.master_for(master_service) + yield master + await master.aclose() + + +def _gen_cluster_mock_resp(r, response): + connection = AsyncMock(spec=Connection) + connection.retry = Retry(NoBackoff(), 0) + connection.read_response.return_value = response + with patch.object(r, "connection", connection): + yield r + + +@pytest.fixture +async def mock_cluster_resp_ok(create_redis, **kwargs): + async with create_redis(**kwargs) as r: + for mocked in _gen_cluster_mock_resp(r, "OK"): + yield mocked + + +@pytest.fixture +async def mock_cluster_resp_int(create_redis, **kwargs): + async with create_redis(**kwargs) as r: + for mocked in _gen_cluster_mock_resp(r, 2): + yield mocked + + +@pytest.fixture +async def mock_cluster_resp_info(create_redis, **kwargs): + async with create_redis(**kwargs) as r: + response = ( + "cluster_state:ok\r\ncluster_slots_assigned:16384\r\n" + "cluster_slots_ok:16384\r\ncluster_slots_pfail:0\r\n" + "cluster_slots_fail:0\r\ncluster_known_nodes:7\r\n" + "cluster_size:3\r\ncluster_current_epoch:7\r\n" + "cluster_my_epoch:2\r\ncluster_stats_messages_sent:170262\r\n" + "cluster_stats_messages_received:105653\r\n" + ) + for mocked in _gen_cluster_mock_resp(r, response): + yield mocked + + +@pytest.fixture +async def mock_cluster_resp_nodes(create_redis, **kwargs): + async with create_redis(**kwargs) as r: + response = ( + "c8253bae761cb1ecb2b61857d85dfe455a0fec8b 172.17.0.7:7006 " + "slave aa90da731f673a99617dfe930306549a09f83a6b 0 " + "1447836263059 5 connected\n" + "9bd595fe4821a0e8d6b99d70faa660638a7612b3 172.17.0.7:7008 " + "master - 0 1447836264065 0 connected\n" + "aa90da731f673a99617dfe930306549a09f83a6b 172.17.0.7:7003 " + "myself,master - 0 0 2 connected 5461-10922\n" + "1df047e5a594f945d82fc140be97a1452bcbf93e 172.17.0.7:7007 " + "slave 19efe5a631f3296fdf21a5441680f893e8cc96ec 0 " + "1447836262556 3 connected\n" + "4ad9a12e63e8f0207025eeba2354bcf4c85e5b22 172.17.0.7:7005 " + "master - 0 1447836262555 7 connected 0-5460\n" + "19efe5a631f3296fdf21a5441680f893e8cc96ec 172.17.0.7:7004 " + "master - 0 1447836263562 3 connected 10923-16383\n" + "fbb23ed8cfa23f17eaf27ff7d0c410492a1093d6 172.17.0.7:7002 " + "master,fail - 1447829446956 1447829444948 1 disconnected\n" + ) + for mocked in _gen_cluster_mock_resp(r, response): + yield mocked + + +@pytest.fixture +async def mock_cluster_resp_slaves(create_redis, **kwargs): + async with create_redis(**kwargs) as r: + response = ( + "['1df047e5a594f945d82fc140be97a1452bcbf93e 172.17.0.7:7007 " + "slave 19efe5a631f3296fdf21a5441680f893e8cc96ec 0 " + "1447836789290 3 connected']" + ) + for mocked in _gen_cluster_mock_resp(r, response): + yield mocked + + +@pytest.fixture +async def credential_provider(request) -> CredentialProvider: + return get_credential_provider(request) + + +async def wait_for_command( + client: redis.Redis, monitor: Monitor, command: str, key: str | None = None +): + # issue a command with a key name that's local to this process. + # if we find a command with our key before the command we're waiting + # for, something went wrong + if key is None: + # generate key + redis_version = REDIS_INFO["version"] + if Version(redis_version) >= Version("5.0.0"): + id_str = str(await client.client_id()) + else: + id_str = f"{random.randrange(2**32):08x}" + key = f"__REDIS-PY-{id_str}__" + await client.get(key) + while True: + monitor_response = await monitor.next_command() + if command in monitor_response["command"]: + return monitor_response + if key in monitor_response["command"]: + return None + + +# helpers to get the connection arguments for this run +@pytest.fixture() +def redis_url(request): + return request.config.getoption("--redis-url") + + +@pytest.fixture() +def connect_args(request): + url = request.config.getoption("--redis-url") + return parse_url(url) diff --git a/tests/test_anyio/mocks.py b/tests/test_anyio/mocks.py new file mode 100644 index 0000000000..fa558fccfb --- /dev/null +++ b/tests/test_anyio/mocks.py @@ -0,0 +1,58 @@ +# Helper Mocking classes for the tests. +from anyio import EndOfStream +from anyio.abc import ByteReceiveStream +from anyio.streams.buffered import BufferedByteReceiveStream + + +class MockMemoryByteStreamReceiver(ByteReceiveStream): + def __init__(self, data: bytes): + self.data = data + self.pos = 0 + + async def receive(self, max_bytes: int = 65536) -> bytes: + result = self.data[self.pos : self.pos + max_bytes] + self.pos += len(result) + if result: + return result + + raise EndOfStream + + async def aclose(self) -> None: + self.pos = len(self.data) + + +class MockStream(BufferedByteReceiveStream): + """ + A class simulating an AnyIO input buffer, optionally raising a + special exception every other read. + """ + + class TestError(BaseException): + pass + + def __init__(self, data: bytes, interrupt_every: int = 0): + super().__init__(MockMemoryByteStreamReceiver(data)) + self.counter = 0 + self.interrupt_every = interrupt_every + + def tick(self): + self.counter += 1 + if not self.interrupt_every: + return + if (self.counter % self.interrupt_every) == 0: + raise self.TestError() + + async def receive(self, max_bytes: int = 65536) -> bytes: + self.tick() + data = await super().receive(5) + return data + + async def receive_exactly(self, nbytes: int) -> bytes: + self.tick() + data = await super().receive_exactly(nbytes) + return data + + async def receive_until(self, delimiter: bytes, max_bytes: int) -> bytes: + self.tick() + data = await super().receive_until(delimiter, max_bytes) + return data diff --git a/tests/test_anyio/test_bloom.py b/tests/test_anyio/test_bloom.py new file mode 100644 index 0000000000..33fe789248 --- /dev/null +++ b/tests/test_anyio/test_bloom.py @@ -0,0 +1,531 @@ +from math import inf + +import pytest +import redis.anyio as redis +from redis.exceptions import RedisError +from tests.conftest import ( + assert_resp_response, + is_resp2_connection, + skip_ifmodversion_lt, +) + +pytestmark = pytest.mark.anyio + + +@pytest.fixture +async def decoded_r(create_redis, stack_url): + async with create_redis(decode_responses=True, url=stack_url) as client: + yield client + + +def intlist(obj): + return [int(v) for v in obj] + + +@pytest.mark.redismod +async def test_create(decoded_r: redis.Redis): + """Test CREATE/RESERVE calls""" + assert await decoded_r.bf().create("bloom", 0.01, 1000) + assert await decoded_r.bf().create("bloom_e", 0.01, 1000, expansion=1) + assert await decoded_r.bf().create("bloom_ns", 0.01, 1000, noScale=True) + assert await decoded_r.cf().create("cuckoo", 1000) + assert await decoded_r.cf().create("cuckoo_e", 1000, expansion=1) + assert await decoded_r.cf().create("cuckoo_bs", 1000, bucket_size=4) + assert await decoded_r.cf().create("cuckoo_mi", 1000, max_iterations=10) + assert await decoded_r.cms().initbydim("cmsDim", 100, 5) + assert await decoded_r.cms().initbyprob("cmsProb", 0.01, 0.01) + assert await decoded_r.topk().reserve("topk", 5, 100, 5, 0.9) + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_create(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("tDigest", 100) + + +@pytest.mark.redismod +async def test_bf_add(decoded_r: redis.Redis): + assert await decoded_r.bf().create("bloom", 0.01, 1000) + assert 1 == await decoded_r.bf().add("bloom", "foo") + assert 0 == await decoded_r.bf().add("bloom", "foo") + assert [0] == intlist(await decoded_r.bf().madd("bloom", "foo")) + assert [0, 1] == await decoded_r.bf().madd("bloom", "foo", "bar") + assert [0, 0, 1] == await decoded_r.bf().madd("bloom", "foo", "bar", "baz") + assert 1 == await decoded_r.bf().exists("bloom", "foo") + assert 0 == await decoded_r.bf().exists("bloom", "noexist") + assert [1, 0] == intlist(await decoded_r.bf().mexists("bloom", "foo", "noexist")) + + +@pytest.mark.redismod +async def test_bf_insert(decoded_r: redis.Redis): + assert await decoded_r.bf().create("bloom", 0.01, 1000) + assert [1] == intlist(await decoded_r.bf().insert("bloom", ["foo"])) + assert [0, 1] == intlist(await decoded_r.bf().insert("bloom", ["foo", "bar"])) + assert [1] == intlist(await decoded_r.bf().insert("captest", ["foo"], capacity=10)) + assert [1] == intlist(await decoded_r.bf().insert("errtest", ["foo"], error=0.01)) + assert 1 == await decoded_r.bf().exists("bloom", "foo") + assert 0 == await decoded_r.bf().exists("bloom", "noexist") + assert [1, 0] == intlist(await decoded_r.bf().mexists("bloom", "foo", "noexist")) + info = await decoded_r.bf().info("bloom") + assert_resp_response( + decoded_r, + 2, + info.get("insertedNum"), + info.get("Number of items inserted"), + ) + assert_resp_response( + decoded_r, + 1000, + info.get("capacity"), + info.get("Capacity"), + ) + assert_resp_response( + decoded_r, + 1, + info.get("filterNum"), + info.get("Number of filters"), + ) + + +@pytest.mark.redismod +async def test_bf_scandump_and_loadchunk(decoded_r: redis.Redis): + # Store a filter + await decoded_r.bf().create("myBloom", "0.0001", "1000") + + # test is probabilistic and might fail. It is OK to change variables if + # certain to not break anything + async def do_verify(): + res = 0 + for x in range(1000): + await decoded_r.bf().add("myBloom", x) + rv = await decoded_r.bf().exists("myBloom", x) + assert rv + rv = await decoded_r.bf().exists("myBloom", f"nonexist_{x}") + res += rv == x + assert res < 5 + + await do_verify() + cmds = [] + + cur = await decoded_r.bf().scandump("myBloom", 0) + first = cur[0] + cmds.append(cur) + + while True: + cur = await decoded_r.bf().scandump("myBloom", first) + first = cur[0] + if first == 0: + break + else: + cmds.append(cur) + prev_info = await decoded_r.bf().execute_command("bf.debug", "myBloom") + + # Remove the filter + await decoded_r.bf().client.delete("myBloom") + + # Now, load all the commands: + for cmd in cmds: + await decoded_r.bf().loadchunk("myBloom", *cmd) + + cur_info = await decoded_r.bf().execute_command("bf.debug", "myBloom") + assert prev_info == cur_info + await do_verify() + + await decoded_r.bf().client.delete("myBloom") + await decoded_r.bf().create("myBloom", "0.0001", "10000000") + + +@pytest.mark.redismod +async def test_bf_info(decoded_r: redis.Redis): + expansion = 4 + # Store a filter + await decoded_r.bf().create("nonscaling", "0.0001", "1000", noScale=True) + info = await decoded_r.bf().info("nonscaling") + assert_resp_response( + decoded_r, + None, + info.get("expansionRate"), + info.get("Expansion rate"), + ) + + await decoded_r.bf().create("expanding", "0.0001", "1000", expansion=expansion) + info = await decoded_r.bf().info("expanding") + assert_resp_response( + decoded_r, + 4, + info.get("expansionRate"), + info.get("Expansion rate"), + ) + + try: + # noScale mean no expansion + await decoded_r.bf().create( + "myBloom", "0.0001", "1000", expansion=expansion, noScale=True + ) + assert False + except RedisError: + assert True + + +@pytest.mark.redismod +async def test_bf_card(decoded_r: redis.Redis): + # return 0 if the key does not exist + assert await decoded_r.bf().card("not_exist") == 0 + + # Store a filter + assert await decoded_r.bf().add("bf1", "item_foo") == 1 + assert await decoded_r.bf().card("bf1") == 1 + + # Error when key is of a type other than Bloom filtedecoded_r. + with pytest.raises(redis.ResponseError): + await decoded_r.set("setKey", "value") + await decoded_r.bf().card("setKey") + + +@pytest.mark.redismod +async def test_cf_add_and_insert(decoded_r: redis.Redis): + assert await decoded_r.cf().create("cuckoo", 1000) + assert await decoded_r.cf().add("cuckoo", "filter") + assert not await decoded_r.cf().addnx("cuckoo", "filter") + assert 1 == await decoded_r.cf().addnx("cuckoo", "newItem") + assert [1] == await decoded_r.cf().insert("captest", ["foo"]) + assert [1] == await decoded_r.cf().insert("captest", ["foo"], capacity=1000) + assert [1] == await decoded_r.cf().insertnx("captest", ["bar"]) + assert [1] == await decoded_r.cf().insertnx("captest", ["food"], nocreate="1") + assert [0, 0, 1] == await decoded_r.cf().insertnx("captest", ["foo", "bar", "baz"]) + assert [0] == await decoded_r.cf().insertnx("captest", ["bar"], capacity=1000) + assert [1] == await decoded_r.cf().insert("empty1", ["foo"], capacity=1000) + assert [1] == await decoded_r.cf().insertnx("empty2", ["bar"], capacity=1000) + info = await decoded_r.cf().info("captest") + assert_resp_response( + decoded_r, 5, info.get("insertedNum"), info.get("Number of items inserted") + ) + assert_resp_response( + decoded_r, 0, info.get("deletedNum"), info.get("Number of items deleted") + ) + assert_resp_response( + decoded_r, 1, info.get("filterNum"), info.get("Number of filters") + ) + + +@pytest.mark.redismod +async def test_cf_exists_and_del(decoded_r: redis.Redis): + assert await decoded_r.cf().create("cuckoo", 1000) + assert await decoded_r.cf().add("cuckoo", "filter") + assert await decoded_r.cf().exists("cuckoo", "filter") + assert not await decoded_r.cf().exists("cuckoo", "notexist") + assert 1 == await decoded_r.cf().count("cuckoo", "filter") + assert 0 == await decoded_r.cf().count("cuckoo", "notexist") + assert await decoded_r.cf().delete("cuckoo", "filter") + assert 0 == await decoded_r.cf().count("cuckoo", "filter") + + +@pytest.mark.redismod +async def test_cms(decoded_r: redis.Redis): + assert await decoded_r.cms().initbydim("dim", 1000, 5) + assert await decoded_r.cms().initbyprob("prob", 0.01, 0.01) + assert await decoded_r.cms().incrby("dim", ["foo"], [5]) + assert [0] == await decoded_r.cms().query("dim", "notexist") + assert [5] == await decoded_r.cms().query("dim", "foo") + assert [10, 15] == await decoded_r.cms().incrby("dim", ["foo", "bar"], [5, 15]) + assert [10, 15] == await decoded_r.cms().query("dim", "foo", "bar") + info = await decoded_r.cms().info("dim") + assert info["width"] + assert 1000 == info["width"] + assert 5 == info["depth"] + assert 25 == info["count"] + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +async def test_cms_merge(decoded_r: redis.Redis): + assert await decoded_r.cms().initbydim("A", 1000, 5) + assert await decoded_r.cms().initbydim("B", 1000, 5) + assert await decoded_r.cms().initbydim("C", 1000, 5) + assert await decoded_r.cms().incrby("A", ["foo", "bar", "baz"], [5, 3, 9]) + assert await decoded_r.cms().incrby("B", ["foo", "bar", "baz"], [2, 3, 1]) + assert [5, 3, 9] == await decoded_r.cms().query("A", "foo", "bar", "baz") + assert [2, 3, 1] == await decoded_r.cms().query("B", "foo", "bar", "baz") + assert await decoded_r.cms().merge("C", 2, ["A", "B"]) + assert [7, 6, 10] == await decoded_r.cms().query("C", "foo", "bar", "baz") + assert await decoded_r.cms().merge("C", 2, ["A", "B"], ["1", "2"]) + assert [9, 9, 11] == await decoded_r.cms().query("C", "foo", "bar", "baz") + assert await decoded_r.cms().merge("C", 2, ["A", "B"], ["2", "3"]) + assert [16, 15, 21] == await decoded_r.cms().query("C", "foo", "bar", "baz") + + +@pytest.mark.redismod +async def test_topk(decoded_r: redis.Redis): + # test list with empty buckets + assert await decoded_r.topk().reserve("topk", 3, 50, 4, 0.9) + assert [ + None, + None, + None, + "A", + "C", + "D", + None, + None, + "E", + None, + "B", + "C", + None, + None, + None, + "D", + None, + ] == await decoded_r.topk().add( + "topk", + "A", + "B", + "C", + "D", + "E", + "A", + "A", + "B", + "C", + "G", + "D", + "B", + "D", + "A", + "E", + "E", + 1, + ) + assert [1, 1, 0, 0, 1, 0, 0] == await decoded_r.topk().query( + "topk", "A", "B", "C", "D", "E", "F", "G" + ) + with pytest.deprecated_call(): + assert [4, 3, 2, 3, 3, 0, 1] == await decoded_r.topk().count( + "topk", "A", "B", "C", "D", "E", "F", "G" + ) + + # test full list + assert await decoded_r.topk().reserve("topklist", 3, 50, 3, 0.9) + assert await decoded_r.topk().add( + "topklist", + "A", + "B", + "C", + "D", + "E", + "A", + "A", + "B", + "C", + "G", + "D", + "B", + "D", + "A", + "E", + "E", + ) + assert ["A", "B", "E"] == await decoded_r.topk().list("topklist") + res = await decoded_r.topk().list("topklist", withcount=True) + assert ["A", 4, "B", 3, "E", 3] == res + info = await decoded_r.topk().info("topklist") + assert 3 == info["k"] + assert 50 == info["width"] + assert 3 == info["depth"] + assert 0.9 == round(float(info["decay"]), 1) + + +@pytest.mark.redismod +async def test_topk_list_with_special_words(decoded_r: redis.Redis): + # test list with empty buckets + assert await decoded_r.topk().reserve("topklist:specialwords", 5, 20, 4, 0.9) + assert await decoded_r.topk().add( + "topklist:specialwords", + "infinity", + "B", + "nan", + "D", + "-infinity", + "infinity", + "infinity", + "B", + "nan", + "G", + "D", + "B", + "D", + "infinity", + "-infinity", + "-infinity", + ) + assert ["infinity", "B", "D", "-infinity", "nan"] == await decoded_r.topk().list( + "topklist:specialwords" + ) + + +@pytest.mark.redismod +async def test_topk_incrby(decoded_r: redis.Redis): + await decoded_r.flushdb() + assert await decoded_r.topk().reserve("topk", 3, 10, 3, 1) + assert [None, None, None] == await decoded_r.topk().incrby( + "topk", ["bar", "baz", "42"], [3, 6, 2] + ) + res = await decoded_r.topk().incrby("topk", ["42", "xyzzy"], [8, 4]) + assert [None, "bar"] == res + with pytest.deprecated_call(): + assert [3, 6, 10, 4, 0] == await decoded_r.topk().count( + "topk", "bar", "baz", "42", "xyzzy", 4 + ) + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_reset(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("tDigest", 10) + # reset on empty histogram + assert await decoded_r.tdigest().reset("tDigest") + # insert data-points into sketch + assert await decoded_r.tdigest().add("tDigest", list(range(10))) + + assert await decoded_r.tdigest().reset("tDigest") + # assert we have 0 unmerged nodes + info = await decoded_r.tdigest().info("tDigest") + assert_resp_response( + decoded_r, 0, info.get("unmerged_nodes"), info.get("Unmerged nodes") + ) + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +async def test_tdigest_merge(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("to-tDigest", 10) + assert await decoded_r.tdigest().create("from-tDigest", 10) + # insert data-points into sketch + assert await decoded_r.tdigest().add("from-tDigest", [1.0] * 10) + assert await decoded_r.tdigest().add("to-tDigest", [2.0] * 10) + # merge from-tdigest into to-tdigest + assert await decoded_r.tdigest().merge("to-tDigest", 1, "from-tDigest") + # we should now have 110 weight on to-histogram + info = await decoded_r.tdigest().info("to-tDigest") + if is_resp2_connection(decoded_r): + assert 20 == float(info["merged_weight"]) + float(info["unmerged_weight"]) + else: + assert 20 == float(info["Merged weight"]) + float(info["Unmerged weight"]) + # test override + assert await decoded_r.tdigest().create("from-override", 10) + assert await decoded_r.tdigest().create("from-override-2", 10) + assert await decoded_r.tdigest().add("from-override", [3.0] * 10) + assert await decoded_r.tdigest().add("from-override-2", [4.0] * 10) + assert await decoded_r.tdigest().merge( + "to-tDigest", 2, "from-override", "from-override-2", override=True + ) + assert 3.0 == await decoded_r.tdigest().min("to-tDigest") + assert 4.0 == await decoded_r.tdigest().max("to-tDigest") + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_min_and_max(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("tDigest", 100) + # insert data-points into sketch + assert await decoded_r.tdigest().add("tDigest", [1, 2, 3]) + # min/max + assert 3 == await decoded_r.tdigest().max("tDigest") + assert 1 == await decoded_r.tdigest().min("tDigest") + + +@pytest.mark.experimental +@pytest.mark.redismod +@skip_ifmodversion_lt("2.4.0", "bf") +async def test_tdigest_quantile(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("tDigest", 500) + # insert data-points into sketch + assert await decoded_r.tdigest().add( + "tDigest", list([x * 0.01 for x in range(1, 10000)]) + ) + # assert min min/max have same result as quantile 0 and 1 + assert ( + await decoded_r.tdigest().max("tDigest") + == (await decoded_r.tdigest().quantile("tDigest", 1))[0] + ) + assert ( + await decoded_r.tdigest().min("tDigest") + == (await decoded_r.tdigest().quantile("tDigest", 0.0))[0] + ) + + assert 1.0 == round((await decoded_r.tdigest().quantile("tDigest", 0.01))[0], 2) + assert 99.0 == round((await decoded_r.tdigest().quantile("tDigest", 0.99))[0], 2) + + # test multiple quantiles + assert await decoded_r.tdigest().create("t-digest", 100) + assert await decoded_r.tdigest().add("t-digest", [1, 2, 3, 4, 5]) + res = await decoded_r.tdigest().quantile("t-digest", 0.5, 0.8) + assert [3.0, 5.0] == res + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_cdf(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("tDigest", 100) + # insert data-points into sketch + assert await decoded_r.tdigest().add("tDigest", list(range(1, 10))) + assert 0.1 == round((await decoded_r.tdigest().cdf("tDigest", 1.0))[0], 1) + assert 0.9 == round((await decoded_r.tdigest().cdf("tDigest", 9.0))[0], 1) + res = await decoded_r.tdigest().cdf("tDigest", 1.0, 9.0) + assert [0.1, 0.9] == [round(x, 1) for x in res] + + +@pytest.mark.experimental +@pytest.mark.redismod +@skip_ifmodversion_lt("2.4.0", "bf") +async def test_tdigest_trimmed_mean(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("tDigest", 100) + # insert data-points into sketch + assert await decoded_r.tdigest().add("tDigest", list(range(1, 10))) + assert 5 == await decoded_r.tdigest().trimmed_mean("tDigest", 0.1, 0.9) + assert 4.5 == await decoded_r.tdigest().trimmed_mean("tDigest", 0.4, 0.5) + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_rank(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("t-digest", 500) + assert await decoded_r.tdigest().add("t-digest", list(range(0, 20))) + assert -1 == (await decoded_r.tdigest().rank("t-digest", -1))[0] + assert 0 == (await decoded_r.tdigest().rank("t-digest", 0))[0] + assert 10 == (await decoded_r.tdigest().rank("t-digest", 10))[0] + assert [-1, 20, 9] == await decoded_r.tdigest().rank("t-digest", -20, 20, 9) + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_revrank(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("t-digest", 500) + assert await decoded_r.tdigest().add("t-digest", list(range(0, 20))) + assert -1 == (await decoded_r.tdigest().revrank("t-digest", 20))[0] + assert 19 == (await decoded_r.tdigest().revrank("t-digest", 0))[0] + assert [-1, 19, 9] == await decoded_r.tdigest().revrank("t-digest", 21, 0, 10) + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_byrank(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("t-digest", 500) + assert await decoded_r.tdigest().add("t-digest", list(range(1, 11))) + assert 1 == (await decoded_r.tdigest().byrank("t-digest", 0))[0] + assert 10 == (await decoded_r.tdigest().byrank("t-digest", 9))[0] + assert (await decoded_r.tdigest().byrank("t-digest", 100))[0] == inf + with pytest.raises(redis.ResponseError): + (await decoded_r.tdigest().byrank("t-digest", -1))[0] + + +@pytest.mark.experimental +@pytest.mark.redismod +async def test_tdigest_byrevrank(decoded_r: redis.Redis): + assert await decoded_r.tdigest().create("t-digest", 500) + assert await decoded_r.tdigest().add("t-digest", list(range(1, 11))) + assert 10 == (await decoded_r.tdigest().byrevrank("t-digest", 0))[0] + assert 1 == (await decoded_r.tdigest().byrevrank("t-digest", 9))[0] + assert (await decoded_r.tdigest().byrevrank("t-digest", 100))[0] == -inf + with pytest.raises(redis.ResponseError): + (await decoded_r.tdigest().byrevrank("t-digest", -1))[0] diff --git a/tests/test_anyio/test_cluster.py b/tests/test_anyio/test_cluster.py new file mode 100644 index 0000000000..e758c0900c --- /dev/null +++ b/tests/test_anyio/test_cluster.py @@ -0,0 +1,3113 @@ +from __future__ import annotations + +import binascii +import datetime +import ssl +import sys +import warnings +from collections.abc import AsyncGenerator, Callable +from contextlib import AbstractAsyncContextManager, asynccontextmanager +from typing import Any, NoReturn +from unittest import mock +from unittest.mock import AsyncMock, patch +from urllib.parse import urlparse + +import anyio +import pytest +from _pytest.fixtures import FixtureRequest +from anyio.abc import SocketStream, TaskStatus + +from redis._parsers import AsyncCommandsParser +from redis.anyio.cluster import ClusterNode, NodesManager, RedisCluster +from redis.anyio.connection import Connection, SSLConnection +from redis.anyio.retry import Retry +from redis.anyio.utils import gather +from redis.backoff import ExponentialBackoff, NoBackoff, default_backoff +from redis.cluster import ( + PIPELINE_BLOCKED_COMMANDS, + PRIMARY, + REPLICA, + LoadBalancingStrategy, + get_node_name, +) +from redis.crc import REDIS_CLUSTER_HASH_SLOTS, key_slot +from redis.exceptions import ( + AskError, + ClusterDownError, + ConnectionError, + DataError, + MaxConnectionsError, + MovedError, + NoPermissionError, + RedisClusterException, + RedisError, + ResponseError, +) +from redis.utils import str_if_bytes +from tests.conftest import ( + assert_resp_response, + is_resp2_connection, + skip_if_redis_enterprise, + skip_if_server_version_lt, + skip_unless_arch_bits, +) + +from ..ssl_utils import get_tls_certificates + +if sys.version_info < (3, 11): + from exceptiongroup import ExceptionGroup + +pytestmark = [pytest.mark.onlycluster, pytest.mark.anyio] + + +default_host = "127.0.0.1" +default_port = 7000 +default_cluster_slots = [ + [0, 8191, ["127.0.0.1", 7000, "node_0"], ["127.0.0.1", 7003, "node_3"]], + [8192, 16383, ["127.0.0.1", 7001, "node_1"], ["127.0.0.1", 7002, "node_2"]], +] + + +class NodeProxy: + """A class to proxy a node connection to a different port""" + + def __init__(self, addr, redis_addr): + self.addr = addr + self.redis_addr = redis_addr + self.n_connections = 0 + + async def run(self, *, task_status: TaskStatus[None]) -> NoReturn: + stream = await anyio.connect_tcp(*self.redis_addr) + await stream.aclose() + + local_host, local_port = self.addr + async with await anyio.create_tcp_listener( + local_host=local_host, local_port=local_port, reuse_port=True + ) as listener: + task_status.started() + await listener.serve(self.handle) + + async def handle(self, client_stream: SocketStream) -> None: + # establish connection to redis + proxied_stream = await anyio.connect_tcp(*self.redis_addr) + async with client_stream, proxied_stream: + self.n_connections += 1 + async with anyio.create_task_group() as tg: + tg.start_soon(self.pipe, client_stream, proxied_stream) + tg.start_soon(self.pipe, proxied_stream, client_stream) + + async def pipe(self, in_stream: SocketStream, out_stream: SocketStream): + while True: + async for data in in_stream: + await out_stream.send(data) + + +@pytest.fixture +async def slowlog(r: RedisCluster) -> AsyncGenerator[None]: + """ + Set the slowlog threshold to 0, and the + max length to 128. This will force every + command into the slowlog and allow us + to test it + """ + # Save old values + current_config = await r.config_get(target_nodes=r.get_primaries()[0]) + old_slower_than_value = current_config["slowlog-log-slower-than"] + old_max_length_value = current_config["slowlog-max-len"] + + # Set the new values + await r.config_set("slowlog-log-slower-than", 0) + await r.config_set("slowlog-max-len", 128) + + yield + + await r.config_set("slowlog-log-slower-than", old_slower_than_value) + await r.config_set("slowlog-max-len", old_max_length_value) + + +@asynccontextmanager +async def get_mocked_redis_client( + cluster_slots_raise_error=False, *args, **kwargs +) -> AsyncGenerator[RedisCluster]: + """ + Return a stable RedisCluster object that have deterministic + nodes and slots setup to remove the problem of different IP addresses + on different installations and machines. + """ + cluster_slots = kwargs.pop("cluster_slots", default_cluster_slots) + coverage_res = kwargs.pop("coverage_result", "yes") + cluster_enabled = kwargs.pop("cluster_enabled", True) + + async def mocked_execute_command(self, *_args, **_kwargs): + if _args[0] == "CLUSTER SLOTS": + if cluster_slots_raise_error: + raise ResponseError() + else: + mock_cluster_slots = cluster_slots + return mock_cluster_slots + elif _args[0] == "COMMAND": + return {"get": [], "set": []} + elif _args[0] == "INFO": + return {"cluster_enabled": cluster_enabled} + elif len(_args) > 1 and _args[1] == "cluster-require-full-coverage": + return {"cluster-require-full-coverage": coverage_res} + + pytest.fail("Got an unexpected command: " + " ".join(_args)) + + async def cmd_init_mock(self, r: ClusterNode) -> None: + self.commands = { + "get": { + "name": "get", + "arity": 2, + "flags": ["readonly", "fast"], + "first_key_pos": 1, + "last_key_pos": 1, + "step_count": 1, + } + } + + # Create a subclass of RedisCluster that overrides __del__ + class MockedRedisCluster(RedisCluster): + def __del__(self): + # Override to prevent connection cleanup attempts + pass + + @property + def connection_pool(self): + # Required abstract property implementation + return ( + self.nodes_manager.get_default_node().redis_connection.connection_pool + ) + + execute_command_patcher = patch.object( + ClusterNode, "execute_command", mocked_execute_command + ) + execute_command_patcher.start() + with patch.object(AsyncCommandsParser, "initialize", cmd_init_mock): + try: + async with MockedRedisCluster(*args, **kwargs) as cluster: + execute_command_patcher.stop() + yield cluster + finally: + execute_command_patcher.stop() + + +def mock_node_resp(node: ClusterNode, response: Any) -> ClusterNode: + connection = AsyncMock(spec=Connection) + connection.is_connected = True + connection.read_response.return_value = response + while node._free: + node._free.pop() + node._free.append(connection) + return node + + +def mock_node_resp_exc(node: ClusterNode, exc: Exception) -> ClusterNode: + connection = AsyncMock(spec=Connection) + connection.is_connected = True + connection.read_response.side_effect = exc + while node._free: + node._free.pop() + node._free.append(connection) + return node + + +def mock_all_nodes_resp(rc: RedisCluster, response: Any) -> RedisCluster: + for node in rc.get_nodes(): + mock_node_resp(node, response) + return rc + + +async def moved_redirection_helper( + create_redis: Callable[..., RedisCluster], failover: bool = False +) -> None: + """ + Test that the client handles MOVED response after a failover. + Redirection after a failover means that the redirection address is of a + replica that was promoted to a primary. + + At first call it should return a MOVED ResponseError that will point + the client to the next server it should talk to. + + Verify that: + 1. it tries to talk to the redirected node + 2. it updates the slot's primary to the redirected node + + For a failover, also verify: + 3. the redirected node's server type updated to 'primary' + 4. the server type of the previous slot owner updated to 'replica' + """ + async with create_redis(cls=RedisCluster, flushdb=False) as rc: + slot = 12182 + redirect_node = None + # Get the current primary that holds this slot + prev_primary = rc.nodes_manager.get_node_from_slot(slot) + if failover: + if len(rc.nodes_manager.slots_cache[slot]) < 2: + warnings.warn("Skipping this test since it requires to have a replica") + return + redirect_node = rc.nodes_manager.slots_cache[slot][1] + else: + # Use one of the primaries to be the redirected node + redirect_node = rc.get_primaries()[0] + r_host = redirect_node.host + r_port = redirect_node.port + with patch.object( + ClusterNode, "execute_command", autospec=True + ) as execute_command: + + def moved_redirect_effect(self, *args, **options): + def ok_response(self, *args, **options): + assert self.host == r_host + assert self.port == r_port + + return "MOCK_OK" + + execute_command.side_effect = ok_response + raise MovedError(f"{slot} {r_host}:{r_port}") + + execute_command.side_effect = moved_redirect_effect + assert await rc.execute_command("SET", "foo", "bar") == "MOCK_OK" + slot_primary = rc.nodes_manager.slots_cache[slot][0] + assert slot_primary == redirect_node + if failover: + assert rc.get_node(host=r_host, port=r_port).server_type == PRIMARY + assert prev_primary.server_type == REPLICA + + +class TestRedisClusterObj: + """ + Tests for the RedisCluster class + """ + + async def test_host_port_startup_node(self) -> None: + """ + Test that it is possible to use host & port arguments as startup node + args + """ + async with get_mocked_redis_client( + host=default_host, port=default_port + ) as cluster: + assert cluster.get_node(host=default_host, port=default_port) is not None + + async def test_startup_nodes(self) -> None: + """ + Test that it is possible to use startup_nodes + argument to init the cluster + """ + port_1 = 7000 + port_2 = 7001 + startup_nodes = [ + ClusterNode(default_host, port_1), + ClusterNode(default_host, port_2), + ] + async with get_mocked_redis_client(startup_nodes=startup_nodes) as cluster: + assert ( + cluster.get_node(host=default_host, port=port_1) is not None + and cluster.get_node(host=default_host, port=port_2) is not None + ) + + startup_node = ClusterNode("127.0.0.1", 16379) + async with RedisCluster(startup_nodes=[startup_node], client_name="test") as rc: + assert await rc.set("A", 1) + assert await rc.get("A") == b"1" + assert all( + [ + name == "test" + for name in ( + await rc.client_getname(target_nodes=rc.ALL_NODES) + ).values() + ] + ) + + async def test_cluster_set_get_retry_object(self, request: FixtureRequest): + retry = Retry(NoBackoff(), 2) + url = request.config.getoption("--redis-url") + async with RedisCluster.from_url(url, retry=retry) as r: + assert r.get_retry()._retries == retry._retries + assert isinstance(r.get_retry()._backoff, NoBackoff) + for node in r.get_nodes(): + n_retry = node.connection_kwargs.get("retry") + assert n_retry is not None + assert n_retry._retries == retry._retries + assert isinstance(n_retry._backoff, NoBackoff) + rand_cluster_node = r.get_random_node() + existing_conn = rand_cluster_node.acquire_connection() + # Change retry policy + new_retry = Retry(ExponentialBackoff(), 3) + r.set_retry(new_retry) + assert r.get_retry()._retries == new_retry._retries + assert isinstance(r.get_retry()._backoff, ExponentialBackoff) + for node in r.get_nodes(): + n_retry = node.connection_kwargs.get("retry") + assert n_retry is not None + assert n_retry._retries == new_retry._retries + assert isinstance(n_retry._backoff, ExponentialBackoff) + assert existing_conn.retry._retries == new_retry._retries + new_conn = rand_cluster_node.acquire_connection() + assert new_conn.retry._retries == new_retry._retries + + async def test_cluster_retry_object(self, request: FixtureRequest) -> None: + url = request.config.getoption("--redis-url") + async with RedisCluster.from_url(url) as rc_default: + # Test default retry + retry = rc_default.connection_kwargs.get("retry") + + # FIXME: Workaround for https://github.com/redis/redis-py/issues/3030 + host = rc_default.get_default_node().host + + assert isinstance(retry, Retry) + assert retry._retries == 3 + assert isinstance(retry._backoff, type(default_backoff())) + assert rc_default.get_node(host, 16379).connection_kwargs.get( + "retry" + ) == rc_default.get_node(host, 16380).connection_kwargs.get("retry") + + retry = Retry(ExponentialBackoff(10, 5), 5) + async with RedisCluster.from_url(url, retry=retry) as rc_custom_retry: + # Test custom retry + assert ( + rc_custom_retry.get_node(host, 16379).connection_kwargs.get("retry") + == retry + ) + + async with RedisCluster.from_url( + url, connection_error_retry_attempts=0 + ) as rc_no_retries: + # Test no connection retries + assert ( + rc_no_retries.get_node(host, 16379).connection_kwargs.get("retry") + is None + ) + + async with RedisCluster.from_url( + url, retry=Retry(NoBackoff(), 0) + ) as rc_no_retries: + assert ( + rc_no_retries.get_node(host, 16379) + .connection_kwargs.get("retry") + ._retries + == 0 + ) + + async def test_empty_startup_nodes(self) -> None: + """ + Test that exception is raised when empty providing empty startup_nodes + """ + with pytest.raises(RedisClusterException) as ex: + RedisCluster(startup_nodes=[]) + + assert str(ex.value).startswith( + "RedisCluster requires at least one node to discover the cluster" + ), str_if_bytes(ex.value) + + async def test_from_url(self, request: FixtureRequest) -> None: + url = request.config.getoption("--redis-url") + + async with RedisCluster.from_url(url) as rc: + await rc.set("a", 1) + assert await rc.get("a") == b"1" + + rc = RedisCluster.from_url("rediss://localhost:16379") + assert rc.connection_kwargs["connection_class"] is SSLConnection + + async def test_max_connections( + self, create_redis: Callable[..., RedisCluster] + ) -> None: + async with create_redis(cls=RedisCluster, max_connections=10) as rc: + for node in rc.get_nodes(): + assert node.max_connections == 10 + + with patch.object(Connection, "read_response") as read_response: + + async def read_response_mocked(*args: Any, **kwargs: Any) -> None: + await anyio.sleep(10) + + read_response.side_effect = read_response_mocked + + with pytest.raises(ExceptionGroup) as exc_info: + await gather( + *( + rc.ping(target_nodes=RedisCluster.DEFAULT_NODE) + for _ in range(11) + ) + ) + + assert all( + isinstance(exc, MaxConnectionsError) + for exc in exc_info.value.exceptions + ) + + async def test_execute_command_errors(self, r: RedisCluster) -> None: + """ + Test that if no key is provided then exception should be raised. + """ + with pytest.raises(RedisClusterException) as ex: + await r.execute_command("GET") + assert str(ex.value).startswith( + "No way to dispatch this command to Redis Cluster. Missing key." + ) + + async def test_execute_command_node_flag_primaries(self, r: RedisCluster) -> None: + """ + Test command execution with nodes flag PRIMARIES + """ + primaries = r.get_primaries() + replicas = r.get_replicas() + mock_all_nodes_resp(r, "PONG") + assert await r.ping(target_nodes=RedisCluster.PRIMARIES) is True + for primary in primaries: + conn = primary._free.pop() + assert conn.read_response.called is True + for replica in replicas: + conn = replica._free.pop() + assert conn.read_response.called is not True + + async def test_execute_command_node_flag_replicas(self, r: RedisCluster) -> None: + """ + Test command execution with nodes flag REPLICAS + """ + replicas = r.get_replicas() + assert len(replicas) != 0, "This test requires Cluster with 1 replica" + + primaries = r.get_primaries() + mock_all_nodes_resp(r, "PONG") + assert await r.ping(target_nodes=RedisCluster.REPLICAS) is True + for replica in replicas: + conn = replica._free.pop() + assert conn.read_response.called is True + for primary in primaries: + conn = primary._free.pop() + assert conn.read_response.called is not True + + async def test_execute_command_node_flag_all_nodes(self, r: RedisCluster) -> None: + """ + Test command execution with nodes flag ALL_NODES + """ + mock_all_nodes_resp(r, "PONG") + assert await r.ping(target_nodes=RedisCluster.ALL_NODES) is True + for node in r.get_nodes(): + conn = node._free.pop() + assert conn.read_response.called is True + + async def test_execute_command_node_flag_random(self, r: RedisCluster) -> None: + """ + Test command execution with nodes flag RANDOM + """ + mock_all_nodes_resp(r, "PONG") + assert await r.ping(target_nodes=RedisCluster.RANDOM) is True + called_count = 0 + for node in r.get_nodes(): + conn = node._free.pop() + if conn.read_response.called is True: + called_count += 1 + assert called_count == 1 + + async def test_execute_command_default_node(self, r: RedisCluster) -> None: + """ + Test command execution without node flag is being executed on the + default node + """ + def_node = r.get_default_node() + mock_node_resp(def_node, "PONG") + assert await r.ping() is True + conn = def_node._free.pop() + assert conn.read_response.called + + async def test_ask_redirection(self, r: RedisCluster) -> None: + """ + Test that the server handles ASK response. + + At first call it should return a ASK ResponseError that will point + the client to the next server it should talk to. + + Important thing to verify is that it tries to talk to the second node. + """ + redirect_node = r.get_nodes()[0] + with patch.object( + ClusterNode, "execute_command", autospec=True + ) as execute_command: + + def ask_redirect_effect(self, *args, **options): + def ok_response(self, *args, **options): + assert self.host == redirect_node.host + assert self.port == redirect_node.port + + return "MOCK_OK" + + execute_command.side_effect = ok_response + raise AskError(f"12182 {redirect_node.host}:{redirect_node.port}") + + execute_command.side_effect = ask_redirect_effect + + assert await r.execute_command("SET", "foo", "bar") == "MOCK_OK" + + async def test_moved_redirection( + self, create_redis: Callable[..., RedisCluster] + ) -> None: + """ + Test that the client handles MOVED response. + """ + await moved_redirection_helper(create_redis, failover=False) + + async def test_moved_redirection_after_failover( + self, create_redis: Callable[..., RedisCluster] + ) -> None: + """ + Test that the client handles MOVED response after a failover. + """ + await moved_redirection_helper(create_redis, failover=True) + + async def test_refresh_using_specific_nodes( + self, create_redis: Callable[..., RedisCluster] + ) -> None: + """ + Test making calls on specific nodes when the cluster has failed over to + another node + """ + node_7006 = ClusterNode(host=default_host, port=7006, server_type=PRIMARY) + node_7007 = ClusterNode(host=default_host, port=7007, server_type=PRIMARY) + with patch.object( + ClusterNode, "execute_command", autospec=True + ) as execute_command: + with patch.object(NodesManager, "initialize", autospec=True) as initialize: + with patch.multiple( + Connection, + send_packed_command=mock.DEFAULT, + connect=mock.DEFAULT, + can_read_destructive=mock.DEFAULT, + ) as mocks: + # simulate 7006 as a failed node + def execute_command_mock(self, *args, **options): + if self.port == 7006: + execute_command.failed_calls += 1 + raise ClusterDownError( + "CLUSTERDOWN The cluster is " + "down. Use CLUSTER INFO for " + "more information" + ) + elif self.port == 7007: + execute_command.successful_calls += 1 + + def initialize_mock(self): + # start with all slots mapped to 7006 + self.nodes_cache = {node_7006.name: node_7006} + self.default_node = node_7006 + self.slots_cache = {} + + for i in range(0, 16383): + self.slots_cache[i] = [node_7006] + + # After the first connection fails, a reinitialize + # should follow the cluster to 7007 + def map_7007(self): + self.nodes_cache = {node_7007.name: node_7007} + self.default_node = node_7007 + self.slots_cache = {} + + for i in range(0, 16383): + self.slots_cache[i] = [node_7007] + + # Change initialize side effect for the second call + initialize.side_effect = map_7007 + + execute_command.side_effect = execute_command_mock + execute_command.successful_calls = 0 + execute_command.failed_calls = 0 + initialize.side_effect = initialize_mock + mocks["can_read_destructive"].return_value = False + mocks["send_packed_command"].return_value = "MOCK_OK" + mocks["connect"].return_value = None + with patch.object( + AsyncCommandsParser, "initialize", autospec=True + ) as cmd_parser_initialize: + + def cmd_init_mock(self, r: ClusterNode) -> None: + self.commands = { + "get": { + "name": "get", + "arity": 2, + "flags": ["readonly", "fast"], + "first_key_pos": 1, + "last_key_pos": 1, + "step_count": 1, + } + } + + cmd_parser_initialize.side_effect = cmd_init_mock + + async with create_redis(cls=RedisCluster, flushdb=False) as rc: + assert len(rc.get_nodes()) == 1 + assert rc.get_node(node_name=node_7006.name) is not None + + await rc.get("foo") + + # Cluster should now point to 7007, and there should be + # one failed and one successful call + assert len(rc.get_nodes()) == 1 + assert rc.get_node(node_name=node_7007.name) is not None + assert rc.get_node(node_name=node_7006.name) is None + assert execute_command.failed_calls == 1 + assert execute_command.successful_calls == 1 + + @pytest.mark.parametrize( + "read_from_replicas,load_balancing_strategy,mocks_srv_ports", + [ + (True, None, [7001, 7002, 7001]), + (True, LoadBalancingStrategy.ROUND_ROBIN, [7001, 7002, 7001]), + (True, LoadBalancingStrategy.ROUND_ROBIN_REPLICAS, [7002, 7002, 7002]), + (True, LoadBalancingStrategy.RANDOM_REPLICA, [7002, 7002, 7002]), + (False, LoadBalancingStrategy.ROUND_ROBIN, [7001, 7002, 7001]), + (False, LoadBalancingStrategy.ROUND_ROBIN_REPLICAS, [7002, 7002, 7002]), + (False, LoadBalancingStrategy.RANDOM_REPLICA, [7002, 7002, 7002]), + ], + ) + async def test_reading_with_load_balancing_strategies( + self, + read_from_replicas: bool, + load_balancing_strategy: LoadBalancingStrategy, + mocks_srv_ports: list[int], + ) -> None: + with patch.multiple( + Connection, + send_command=mock.DEFAULT, + read_response=mock.DEFAULT, + _connect=mock.DEFAULT, + can_read_destructive=mock.DEFAULT, + on_connect=mock.DEFAULT, + ) as mocks: + with patch.object( + ClusterNode, "execute_command", autospec=True + ) as execute_command: + + async def execute_command_mock_first(self, *args, **options): + await self.connection_class(**self.connection_kwargs).connect() + # Primary + assert self.port == mocks_srv_ports[0] + execute_command.side_effect = execute_command_mock_second + return "MOCK_OK" + + def execute_command_mock_second(self, *args, **options): + # Replica + assert self.port == mocks_srv_ports[1] + execute_command.side_effect = execute_command_mock_third + return "MOCK_OK" + + def execute_command_mock_third(self, *args, **options): + # Primary + assert self.port == mocks_srv_ports[2] + return "MOCK_OK" + + # We don't need to create a real cluster connection but we + # do want RedisCluster.on_connect function to get called, + # so we'll mock some of the Connection's functions to allow it + execute_command.side_effect = execute_command_mock_first + mocks["send_command"].return_value = True + mocks["read_response"].return_value = "OK" + mocks["_connect"].return_value = True + mocks["can_read_destructive"].return_value = False + mocks["on_connect"].return_value = True + + # Create a cluster with reading from replications + with pytest.warns( + DeprecationWarning, match="input argument/s 'read_from_replicas'" + ): + async with get_mocked_redis_client( + host=default_host, + port=default_port, + read_from_replicas=read_from_replicas, + load_balancing_strategy=load_balancing_strategy, + ) as read_cluster: + assert read_cluster.read_from_replicas is read_from_replicas + assert ( + read_cluster.load_balancing_strategy + is load_balancing_strategy + ) + # Check that we read from the slot's nodes in a round robin + # matter. + # 'foo' belongs to slot 12182 and the slot's nodes are: + # [(127.0.0.1,7001,primary), (127.0.0.1,7002,replica)] + await read_cluster.get("foo") + await read_cluster.get("foo") + await read_cluster.get("foo") + mocks["send_command"].assert_has_calls([mock.call("READONLY")]) + + async def test_keyslot(self, r: RedisCluster) -> None: + """ + Test that method will compute correct key in all supported cases + """ + assert r.keyslot("foo") == 12182 + assert r.keyslot("{foo}bar") == 12182 + assert r.keyslot("{foo}") == 12182 + assert r.keyslot(1337) == 4314 + + assert r.keyslot(125) == r.keyslot(b"125") + assert r.keyslot(125) == r.keyslot("\x31\x32\x35") + assert r.keyslot("大奖") == r.keyslot(b"\xe5\xa4\xa7\xe5\xa5\x96") + assert r.keyslot("大奖") == r.keyslot(b"\xe5\xa4\xa7\xe5\xa5\x96") + assert r.keyslot(1337.1234) == r.keyslot("1337.1234") + assert r.keyslot(1337) == r.keyslot("1337") + assert r.keyslot(b"abc") == r.keyslot("abc") + + async def test_get_node_name(self) -> None: + assert ( + get_node_name(default_host, default_port) + == f"{default_host}:{default_port}" + ) + + async def test_all_nodes(self, r: RedisCluster) -> None: + """ + Set a list of nodes and it should be possible to iterate over all + """ + nodes = [node for node in r.nodes_manager.nodes_cache.values()] + + for i, node in enumerate(r.get_nodes()): + assert node in nodes + + async def test_all_nodes_masters(self, r: RedisCluster) -> None: + """ + Set a list of nodes with random primaries/replicas config and it shold + be possible to iterate over all of them. + """ + nodes = [ + node + for node in r.nodes_manager.nodes_cache.values() + if node.server_type == PRIMARY + ] + + for node in r.get_primaries(): + assert node in nodes + + @pytest.mark.parametrize("error", RedisCluster.ERRORS_ALLOW_RETRY) + async def test_cluster_down_overreaches_retry_attempts( + self, + error: type[TimeoutError] | type[ClusterDownError] | type[ConnectionError], + ) -> None: + """ + When error that allows retry is thrown, test that we retry executing + the command as many times as configured in cluster_error_retry_attempts + and then raise the exception + """ + with patch.object(RedisCluster, "_execute_command") as execute_command: + + def raise_error(target_node, *args, **kwargs): + execute_command.failed_calls += 1 + raise error("mocked error") + + execute_command.side_effect = raise_error + + async with get_mocked_redis_client( + host=default_host, port=default_port + ) as rc: + with pytest.raises(error): + await rc.get("bar") + assert ( + execute_command.failed_calls == rc.cluster_error_retry_attempts + ) + + async def test_set_default_node_success(self, r: RedisCluster) -> None: + """ + test successful replacement of the default cluster node + """ + default_node = r.get_default_node() + # get a different node + new_def_node = None + for node in r.get_nodes(): + if node != default_node: + new_def_node = node + break + r.set_default_node(new_def_node) + assert r.get_default_node() == new_def_node + + async def test_set_default_node_failure(self, r: RedisCluster) -> None: + """ + test failed replacement of the default cluster node + """ + default_node = r.get_default_node() + new_def_node = ClusterNode("1.1.1.1", 1111) + with pytest.raises(DataError): + r.set_default_node(None) + with pytest.raises(DataError): + r.set_default_node(new_def_node) + assert r.get_default_node() == default_node + + async def test_get_node_from_key(self, r: RedisCluster) -> None: + """ + Test that get_node_from_key function returns the correct node + """ + key = "bar" + slot = r.keyslot(key) + slot_nodes = r.nodes_manager.slots_cache.get(slot) + primary = slot_nodes[0] + assert r.get_node_from_key(key, replica=False) == primary + replica = r.get_node_from_key(key, replica=True) + if replica is not None: + assert replica.server_type == REPLICA + assert replica in slot_nodes + + @skip_if_redis_enterprise() + async def test_not_require_full_coverage_cluster_down_error( + self, r: RedisCluster + ) -> None: + """ + When require_full_coverage is set to False (default client config) and not + all slots are covered, if one of the nodes has 'cluster-require_full_coverage' + config set to 'yes' some key-based commands should throw ClusterDownError + """ + node = r.get_node_from_key("foo") + missing_slot = r.keyslot("foo") + assert await r.set("foo", "bar") is True + try: + assert all(await r.cluster_delslots(missing_slot)) + with pytest.raises(ClusterDownError): + await r.exists("foo") + except ResponseError as e: + assert "CLUSTERDOWN" in str(e) + finally: + try: + # Add back the missing slot + assert await r.cluster_addslots(node, missing_slot) is True + # Make sure we are not getting ClusterDownError anymore + assert await r.exists("foo") == 1 + except ResponseError as e: + if f"Slot {missing_slot} is already busy" in str(e): + # It can happen if the test failed to delete this slot + pass + else: + raise e + + async def test_can_run_concurrent_commands(self, request: FixtureRequest) -> None: + url = request.config.getoption("--redis-url") + async with RedisCluster.from_url(url) as rc: + assert all( + await gather( + *( + rc.echo("i", target_nodes=RedisCluster.ALL_NODES) + for i in range(100) + ) + ) + ) + + async def test_replace_cluster_node(self, r: RedisCluster) -> None: + prev_default_node = r.get_default_node() + r.replace_default_node() + assert r.get_default_node() != prev_default_node + r.replace_default_node(prev_default_node) + assert r.get_default_node() == prev_default_node + + async def test_default_node_is_replaced_after_exception(self, r): + curr_default_node = r.get_default_node() + # CLUSTER NODES command is being executed on the default node + nodes = await r.cluster_nodes() + assert "myself" in nodes.get(curr_default_node.name).get("flags") + # Mock connection error for the default node + mock_node_resp_exc(curr_default_node, ConnectionError("error")) + # Test that the command succeed from a different node + nodes = await r.cluster_nodes() + assert "myself" not in nodes.get(curr_default_node.name).get("flags") + assert r.get_default_node() != curr_default_node + # Rollback to the old default node + r.replace_default_node(curr_default_node) + + async def test_address_remap(self, create_redis, master_host): + """Test that we can create a rediscluster object with + a host-port remapper and map connections through proxy objects + """ + + # we remap the first n nodes + offset = 1000 + n = 6 + hostname, master_port = master_host + ports = [master_port + i for i in range(n)] + + def address_remap(address): + # remap first three nodes to our local proxy + # old = host, port + host, port = address + if int(port) in ports: + host, port = "127.0.0.1", int(port) + offset + # print(f"{old} {host, port}") + return host, port + + # create the proxies + proxies = [ + NodeProxy(("127.0.0.1", port + offset), (hostname, port)) for port in ports + ] + async with anyio.create_task_group() as tg: + for proxy in proxies: + await tg.start(proxy.run) + + # create cluster: + async with create_redis( + cls=RedisCluster, flushdb=False, address_remap=address_remap + ) as r: + assert await r.ping() is True + assert await r.set("byte_string", b"giraffe") + assert await r.get("byte_string") == b"giraffe" + + tg.cancel_scope.cancel() + + # verify that the proxies were indeed used + n_used = sum((1 if p.n_connections else 0) for p in proxies) + assert n_used > 1 + + +class TestClusterRedisCommands: + """ + Tests for RedisCluster unique commands + """ + + async def test_get_and_set(self, r: RedisCluster) -> None: + # get and set can't be tested independently of each other + assert await r.get("a") is None + byte_string = b"value" + integer = 5 + unicode_string = chr(3456) + "abcd" + chr(3421) + assert await r.set("byte_string", byte_string) + assert await r.set("integer", 5) + assert await r.set("unicode_string", unicode_string) + assert await r.get("byte_string") == byte_string + assert await r.get("integer") == str(integer).encode() + assert (await r.get("unicode_string")).decode("utf-8") == unicode_string + + @pytest.mark.parametrize( + "load_balancing_strategy", + [ + LoadBalancingStrategy.ROUND_ROBIN, + LoadBalancingStrategy.ROUND_ROBIN_REPLICAS, + LoadBalancingStrategy.RANDOM_REPLICA, + ], + ) + async def test_get_and_set_with_load_balanced_client( + self, create_redis, load_balancing_strategy: LoadBalancingStrategy + ) -> None: + async with create_redis( + cls=RedisCluster, + load_balancing_strategy=load_balancing_strategy, + ) as r: + # get and set can't be tested independently of each other + assert await r.get("a") is None + + byte_string = b"value" + assert await r.set("byte_string", byte_string) + + # run the get command for the same key several times + # to iterate over the read nodes + assert await r.get("byte_string") == byte_string + assert await r.get("byte_string") == byte_string + assert await r.get("byte_string") == byte_string + + async def test_mget_nonatomic(self, r: RedisCluster) -> None: + assert await r.mget_nonatomic([]) == [] + assert await r.mget_nonatomic(["a", "b"]) == [None, None] + await r.set("a", "1") + await r.set("b", "2") + await r.set("c", "3") + + assert await r.mget_nonatomic("a", "other", "b", "c") == [ + b"1", + None, + b"2", + b"3", + ] + + async def test_mset_nonatomic(self, r: RedisCluster) -> None: + d = {"a": b"1", "b": b"2", "c": b"3", "d": b"4"} + assert await r.mset_nonatomic(d) + for k, v in d.items(): + assert await r.get(k) == v + + async def test_config_set(self, r: RedisCluster) -> None: + assert await r.config_set("slowlog-log-slower-than", 0) + + async def test_cluster_config_resetstat(self, r: RedisCluster) -> None: + await r.ping(target_nodes="all") + all_info = await r.info(target_nodes="all") + prior_commands_processed = -1 + for node_info in all_info.values(): + prior_commands_processed = node_info["total_commands_processed"] + assert prior_commands_processed >= 1 + await r.config_resetstat(target_nodes="all") + all_info = await r.info(target_nodes="all") + for node_info in all_info.values(): + reset_commands_processed = node_info["total_commands_processed"] + assert reset_commands_processed < prior_commands_processed + + async def test_client_setname(self, r: RedisCluster) -> None: + node = r.get_random_node() + await r.client_setname("redis_py_test", target_nodes=node) + client_name = await r.client_getname(target_nodes=node) + assert_resp_response(r, client_name, "redis_py_test", b"redis_py_test") + + async def test_exists(self, r: RedisCluster) -> None: + d = {"a": b"1", "b": b"2", "c": b"3", "d": b"4"} + await r.mset_nonatomic(d) + assert await r.exists(*d.keys()) == len(d) + + async def test_delete(self, r: RedisCluster) -> None: + d = {"a": b"1", "b": b"2", "c": b"3", "d": b"4"} + await r.mset_nonatomic(d) + assert await r.delete(*d.keys()) == len(d) + assert await r.delete(*d.keys()) == 0 + + async def test_touch(self, r: RedisCluster) -> None: + d = {"a": b"1", "b": b"2", "c": b"3", "d": b"4"} + await r.mset_nonatomic(d) + assert await r.touch(*d.keys()) == len(d) + + async def test_unlink(self, r: RedisCluster) -> None: + d = {"a": b"1", "b": b"2", "c": b"3", "d": b"4"} + await r.mset_nonatomic(d) + assert await r.unlink(*d.keys()) == len(d) + # Unlink is non-blocking so we sleep before + # verifying the deletion + await anyio.sleep(0.1) + assert await r.unlink(*d.keys()) == 0 + + @skip_if_redis_enterprise() + async def test_cluster_myid(self, r: RedisCluster) -> None: + node = r.get_random_node() + myid = await r.cluster_myid(node) + assert len(myid) == 40 + + @skip_if_server_version_lt("7.2.0") + @skip_if_redis_enterprise() + async def test_cluster_myshardid(self, r: RedisCluster) -> None: + node = r.get_random_node() + myshardid = await r.cluster_myshardid(node) + assert len(myshardid) == 40 + + @skip_if_redis_enterprise() + async def test_cluster_slots(self, r: RedisCluster) -> None: + mock_all_nodes_resp(r, default_cluster_slots) + cluster_slots = await r.cluster_slots() + assert isinstance(cluster_slots, dict) + assert len(default_cluster_slots) == len(cluster_slots) + assert cluster_slots.get((0, 8191)) is not None + assert cluster_slots.get((0, 8191)).get("primary") == ("127.0.0.1", 7000) + + @skip_if_redis_enterprise() + async def test_cluster_addslots(self, r: RedisCluster) -> None: + node = r.get_random_node() + mock_node_resp(node, "OK") + assert await r.cluster_addslots(node, 1, 2, 3) is True + + @skip_if_server_version_lt("7.0.0") + @skip_if_redis_enterprise() + async def test_cluster_addslotsrange(self, r: RedisCluster): + node = r.get_random_node() + mock_node_resp(node, "OK") + assert await r.cluster_addslotsrange(node, 1, 5) + + @skip_if_redis_enterprise() + async def test_cluster_countkeysinslot(self, r: RedisCluster) -> None: + node = r.nodes_manager.get_node_from_slot(1) + mock_node_resp(node, 2) + assert await r.cluster_countkeysinslot(1) == 2 + + async def test_cluster_count_failure_report(self, r: RedisCluster) -> None: + mock_all_nodes_resp(r, 0) + assert await r.cluster_count_failure_report("node_0") == 0 + + @skip_if_redis_enterprise() + async def test_cluster_delslots(self) -> None: + cluster_slots = [ + [0, 8191, ["127.0.0.1", 7000, "node_0"]], + [8192, 16383, ["127.0.0.1", 7001, "node_1"]], + ] + async with get_mocked_redis_client( + host=default_host, port=default_port, cluster_slots=cluster_slots + ) as r: + mock_all_nodes_resp(r, "OK") + node0 = r.get_node(default_host, 7000) + node1 = r.get_node(default_host, 7001) + assert await r.cluster_delslots(0, 8192) == [True, True] + assert node0._free.pop().read_response.called + assert node1._free.pop().read_response.called + + @skip_if_server_version_lt("7.0.0") + @skip_if_redis_enterprise() + async def test_cluster_delslotsrange(self): + async with get_mocked_redis_client(host=default_host, port=default_port) as r: + mock_all_nodes_resp(r, "OK") + node = r.get_random_node() + await r.cluster_addslots(node, 1, 2, 3, 4, 5) + assert await r.cluster_delslotsrange(1, 5) + assert node._free.pop().read_response.called + + @skip_if_redis_enterprise() + async def test_cluster_failover(self, r: RedisCluster) -> None: + node = r.get_random_node() + mock_node_resp(node, "OK") + assert await r.cluster_failover(node) is True + assert await r.cluster_failover(node, "FORCE") is True + assert await r.cluster_failover(node, "TAKEOVER") is True + with pytest.raises(RedisError): + await r.cluster_failover(node, "FORCT") + + @skip_if_redis_enterprise() + async def test_cluster_info(self, r: RedisCluster) -> None: + info = await r.cluster_info() + assert isinstance(info, dict) + assert info["cluster_state"] == "ok" + + @skip_if_redis_enterprise() + async def test_cluster_keyslot(self, r: RedisCluster) -> None: + mock_all_nodes_resp(r, 12182) + assert await r.cluster_keyslot("foo") == 12182 + + @skip_if_redis_enterprise() + async def test_cluster_meet(self, r: RedisCluster) -> None: + node = r.get_default_node() + mock_node_resp(node, "OK") + assert await r.cluster_meet("127.0.0.1", 6379) is True + + @skip_if_redis_enterprise() + async def test_cluster_nodes(self, r: RedisCluster) -> None: + response = ( + "c8253bae761cb1ecb2b61857d85dfe455a0fec8b 172.17.0.7:7006 " + "slave aa90da731f673a99617dfe930306549a09f83a6b 0 " + "1447836263059 5 connected\n" + "9bd595fe4821a0e8d6b99d70faa660638a7612b3 172.17.0.7:7008 " + "master - 0 1447836264065 0 connected\n" + "aa90da731f673a99617dfe930306549a09f83a6b 172.17.0.7:7003 " + "myself,master - 0 0 2 connected 5461-10922\n" + "1df047e5a594f945d82fc140be97a1452bcbf93e 172.17.0.7:7007 " + "slave 19efe5a631f3296fdf21a5441680f893e8cc96ec 0 " + "1447836262556 3 connected\n" + "4ad9a12e63e8f0207025eeba2354bcf4c85e5b22 172.17.0.7:7005 " + "master - 0 1447836262555 7 connected 0-5460\n" + "19efe5a631f3296fdf21a5441680f893e8cc96ec 172.17.0.7:7004 " + "master - 0 1447836263562 3 connected 10923-16383\n" + "fbb23ed8cfa23f17eaf27ff7d0c410492a1093d6 172.17.0.7:7002 " + "master,fail - 1447829446956 1447829444948 1 disconnected\n" + ) + mock_all_nodes_resp(r, response) + nodes = await r.cluster_nodes() + assert len(nodes) == 7 + assert nodes.get("172.17.0.7:7006") is not None + assert ( + nodes.get("172.17.0.7:7006").get("node_id") + == "c8253bae761cb1ecb2b61857d85dfe455a0fec8b" + ) + + @skip_if_redis_enterprise() + async def test_cluster_nodes_importing_migrating(self, r: RedisCluster) -> None: + response = ( + "488ead2fcce24d8c0f158f9172cb1f4a9e040fe5 127.0.0.1:16381@26381 " + "master - 0 1648975557664 3 connected 10923-16383\n" + "8ae2e70812db80776f739a72374e57fc4ae6f89d 127.0.0.1:16380@26380 " + "master - 0 1648975555000 2 connected 1 5461-10922 [" + "2-<-ed8007ccfa2d91a7b76f8e6fba7ba7e257034a16]\n" + "ed8007ccfa2d91a7b76f8e6fba7ba7e257034a16 127.0.0.1:16379@26379 " + "myself,master - 0 1648975556000 1 connected 0 2-5460 [" + "2->-8ae2e70812db80776f739a72374e57fc4ae6f89d]\n" + ) + mock_all_nodes_resp(r, response) + nodes = await r.cluster_nodes() + assert len(nodes) == 3 + node_16379 = nodes.get("127.0.0.1:16379") + node_16380 = nodes.get("127.0.0.1:16380") + node_16381 = nodes.get("127.0.0.1:16381") + assert node_16379.get("migrations") == [ + { + "slot": "2", + "node_id": "8ae2e70812db80776f739a72374e57fc4ae6f89d", + "state": "migrating", + } + ] + assert node_16379.get("slots") == [["0"], ["2", "5460"]] + assert node_16380.get("migrations") == [ + { + "slot": "2", + "node_id": "ed8007ccfa2d91a7b76f8e6fba7ba7e257034a16", + "state": "importing", + } + ] + assert node_16380.get("slots") == [["1"], ["5461", "10922"]] + assert node_16381.get("slots") == [["10923", "16383"]] + assert node_16381.get("migrations") == [] + + @skip_if_redis_enterprise() + async def test_cluster_replicate(self, r: RedisCluster) -> None: + node = r.get_random_node() + all_replicas = r.get_replicas() + mock_all_nodes_resp(r, "OK") + assert await r.cluster_replicate(node, "c8253bae761cb61857d") is True + results = await r.cluster_replicate(all_replicas, "c8253bae761cb61857d") + if isinstance(results, dict): + for res in results.values(): + assert res is True + else: + assert results is True + + @skip_if_redis_enterprise() + async def test_cluster_reset(self, r: RedisCluster) -> None: + mock_all_nodes_resp(r, "OK") + assert await r.cluster_reset() is True + assert await r.cluster_reset(False) is True + all_results = await r.cluster_reset(False, target_nodes="all") + for res in all_results.values(): + assert res is True + + @skip_if_redis_enterprise() + async def test_cluster_save_config(self, r: RedisCluster) -> None: + node = r.get_random_node() + all_nodes = r.get_nodes() + mock_all_nodes_resp(r, "OK") + assert await r.cluster_save_config(node) is True + all_results = await r.cluster_save_config(all_nodes) + for res in all_results.values(): + assert res is True + + @skip_if_redis_enterprise() + async def test_cluster_get_keys_in_slot(self, r: RedisCluster) -> None: + response = ["{foo}1", "{foo}2"] + node = r.nodes_manager.get_node_from_slot(12182) + mock_node_resp(node, response) + keys = await r.cluster_get_keys_in_slot(12182, 4) + assert keys == response + + @skip_if_redis_enterprise() + async def test_cluster_set_config_epoch(self, r: RedisCluster) -> None: + mock_all_nodes_resp(r, "OK") + assert await r.cluster_set_config_epoch(3) is True + all_results = await r.cluster_set_config_epoch(3, target_nodes="all") + for res in all_results.values(): + assert res is True + + @skip_if_redis_enterprise() + async def test_cluster_setslot(self, r: RedisCluster) -> None: + node = r.get_random_node() + mock_node_resp(node, "OK") + assert await r.cluster_setslot(node, "node_0", 1218, "IMPORTING") is True + assert await r.cluster_setslot(node, "node_0", 1218, "NODE") is True + assert await r.cluster_setslot(node, "node_0", 1218, "MIGRATING") is True + with pytest.raises(RedisError): + await r.cluster_failover(node, "STABLE") + with pytest.raises(RedisError): + await r.cluster_failover(node, "STATE") + + async def test_cluster_setslot_stable(self, r: RedisCluster) -> None: + node = r.nodes_manager.get_node_from_slot(12182) + mock_node_resp(node, "OK") + assert await r.cluster_setslot_stable(12182) is True + assert node._free.pop().read_response.called + + @skip_if_redis_enterprise() + async def test_cluster_replicas(self, r: RedisCluster) -> None: + response = [ + b"01eca22229cf3c652b6fca0d09ff6941e0d2e3 " + b"127.0.0.1:6377@16377 slave " + b"52611e796814b78e90ad94be9d769a4f668f9a 0 " + b"1634550063436 4 connected", + b"r4xfga22229cf3c652b6fca0d09ff69f3e0d4d " + b"127.0.0.1:6378@16378 slave " + b"52611e796814b78e90ad94be9d769a4f668f9a 0 " + b"1634550063436 4 connected", + ] + mock_all_nodes_resp(r, response) + replicas = await r.cluster_replicas("52611e796814b78e90ad94be9d769a4f668f9a") + assert replicas.get("127.0.0.1:6377") is not None + assert replicas.get("127.0.0.1:6378") is not None + assert ( + replicas.get("127.0.0.1:6378").get("node_id") + == "r4xfga22229cf3c652b6fca0d09ff69f3e0d4d" + ) + + @skip_if_server_version_lt("7.0.0") + async def test_cluster_links(self, r: RedisCluster): + node = r.get_random_node() + res = await r.cluster_links(node) + if is_resp2_connection(r): + links_to = sum(x.count(b"to") for x in res) + links_for = sum(x.count(b"from") for x in res) + assert links_to == links_for + for i in range(0, len(res) - 1, 2): + assert res[i][3] == res[i + 1][3] + else: + links_to = len(list(filter(lambda x: x[b"direction"] == b"to", res))) + links_for = len(list(filter(lambda x: x[b"direction"] == b"from", res))) + assert links_to == links_for + for i in range(0, len(res) - 1, 2): + assert res[i][b"node"] == res[i + 1][b"node"] + + @skip_if_redis_enterprise() + async def test_readonly(self) -> None: + async with get_mocked_redis_client(host=default_host, port=default_port) as r: + mock_all_nodes_resp(r, "OK") + assert await r.readonly() is True + all_replicas_results = await r.readonly(target_nodes="replicas") + for res in all_replicas_results.values(): + assert res is True + for replica in r.get_replicas(): + assert replica._free.pop().read_response.called + + @skip_if_redis_enterprise() + async def test_readwrite(self) -> None: + async with get_mocked_redis_client(host=default_host, port=default_port) as r: + mock_all_nodes_resp(r, "OK") + assert await r.readwrite() is True + all_replicas_results = await r.readwrite(target_nodes="replicas") + for res in all_replicas_results.values(): + assert res is True + for replica in r.get_replicas(): + assert replica._free.pop().read_response.called + + @skip_if_redis_enterprise() + async def test_bgsave(self, r: RedisCluster) -> None: + try: + assert await r.bgsave() + await anyio.sleep(0.3) + assert await r.bgsave(True) + except ResponseError as e: + if "Background save already in progress" not in e.__str__(): + raise + + async def test_info(self, r: RedisCluster) -> None: + # Map keys to same slot + await r.set("x{1}", 1) + await r.set("y{1}", 2) + await r.set("z{1}", 3) + # Get node that handles the slot + slot = r.keyslot("x{1}") + node = r.nodes_manager.get_node_from_slot(slot) + # Run info on that node + info = await r.info(target_nodes=node) + assert isinstance(info, dict) + assert info["db0"]["keys"] == 3 + + async def _init_slowlog_test(self, r: RedisCluster, node: ClusterNode) -> str: + slowlog_lim = await r.config_get("slowlog-log-slower-than", target_nodes=node) + assert ( + await r.config_set("slowlog-log-slower-than", 0, target_nodes=node) is True + ) + return slowlog_lim["slowlog-log-slower-than"] + + async def _teardown_slowlog_test( + self, r: RedisCluster, node: ClusterNode, prev_limit: str + ) -> None: + assert ( + await r.config_set("slowlog-log-slower-than", prev_limit, target_nodes=node) + is True + ) + + async def test_slowlog_get( + self, r: RedisCluster, slowlog: list[dict[str, int | bytes]] | None + ) -> None: + unicode_string = chr(3456) + "abcd" + chr(3421) + node = r.get_node_from_key(unicode_string) + slowlog_limit = await self._init_slowlog_test(r, node) + assert await r.slowlog_reset(target_nodes=node) + await r.get(unicode_string) + slowlog = await r.slowlog_get(target_nodes=node) + assert isinstance(slowlog, list) + commands = [log["command"] for log in slowlog] + + get_command = b" ".join((b"GET", unicode_string.encode("utf-8"))) + assert get_command in commands + assert b"SLOWLOG RESET" in commands + + # the order should be ['GET ', 'SLOWLOG RESET'], + # but if other clients are executing commands at the same time, there + # could be commands, before, between, or after, so just check that + # the two we care about are in the appropriate order. + assert commands.index(get_command) < commands.index(b"SLOWLOG RESET") + + # make sure other attributes are typed correctly + assert isinstance(slowlog[0]["start_time"], int) + assert isinstance(slowlog[0]["duration"], int) + # rollback the slowlog limit to its original value + await self._teardown_slowlog_test(r, node, slowlog_limit) + + async def test_slowlog_get_limit( + self, r: RedisCluster, slowlog: list[dict[str, int | bytes]] | None + ) -> None: + assert await r.slowlog_reset() + node = r.get_node_from_key("foo") + slowlog_limit = await self._init_slowlog_test(r, node) + await r.get("foo") + slowlog = await r.slowlog_get(1, target_nodes=node) + assert isinstance(slowlog, list) + # only one command, based on the number we passed to slowlog_get() + assert len(slowlog) == 1 + await self._teardown_slowlog_test(r, node, slowlog_limit) + + async def test_slowlog_length(self, r: RedisCluster, slowlog: None) -> None: + await r.get("foo") + node = r.nodes_manager.get_node_from_slot(key_slot(b"foo")) + slowlog_len = await r.slowlog_len(target_nodes=node) + assert isinstance(slowlog_len, int) + + async def test_time(self, r: RedisCluster) -> None: + t = await r.time(target_nodes=r.get_primaries()[0]) + assert len(t) == 2 + assert isinstance(t[0], int) + assert isinstance(t[1], int) + + @skip_if_server_version_lt("4.0.0") + async def test_memory_usage(self, r: RedisCluster) -> None: + await r.set("foo", "bar") + assert isinstance(await r.memory_usage("foo"), int) + + @skip_if_server_version_lt("4.0.0") + @skip_if_redis_enterprise() + async def test_memory_malloc_stats(self, r: RedisCluster) -> None: + assert await r.memory_malloc_stats() + + @skip_if_server_version_lt("4.0.0") + @skip_if_redis_enterprise() + async def test_memory_stats(self, r: RedisCluster) -> None: + # put a key into the current db to make sure that "db." + # has data + await r.set("foo", "bar") + node = r.nodes_manager.get_node_from_slot(key_slot(b"foo")) + stats = await r.memory_stats(target_nodes=node) + assert isinstance(stats, dict) + for key, value in stats.items(): + if key.startswith("db."): + assert not isinstance(value, list) + + @skip_if_server_version_lt("4.0.0") + async def test_memory_help(self, r: RedisCluster) -> None: + with pytest.raises(NotImplementedError): + await r.memory_help() + + @skip_if_server_version_lt("4.0.0") + async def test_memory_doctor(self, r: RedisCluster) -> None: + with pytest.raises(NotImplementedError): + await r.memory_doctor() + + @skip_if_redis_enterprise() + async def test_lastsave(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + assert isinstance(await r.lastsave(target_nodes=node), datetime.datetime) + + async def test_cluster_echo(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + assert await r.echo("foo bar", target_nodes=node) == b"foo bar" + + @skip_if_server_version_lt("1.0.0") + async def test_debug_segfault(self, r: RedisCluster) -> None: + with pytest.raises(NotImplementedError): + await r.debug_segfault() + + async def test_config_resetstat(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + await r.ping(target_nodes=node) + prior_commands_processed = int( + (await r.info(target_nodes=node))["total_commands_processed"] + ) + assert prior_commands_processed >= 1 + await r.config_resetstat(target_nodes=node) + reset_commands_processed = int( + (await r.info(target_nodes=node))["total_commands_processed"] + ) + assert reset_commands_processed < prior_commands_processed + + @skip_if_server_version_lt("6.2.0") + async def test_client_trackinginfo(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + res = await r.client_trackinginfo(target_nodes=node) + assert len(res) > 2 + assert "prefixes" in res or b"prefixes" in res + + @skip_if_server_version_lt("2.9.50") + async def test_client_pause(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + assert await r.client_pause(1, target_nodes=node) + assert await r.client_pause(timeout=1, target_nodes=node) + with pytest.raises(RedisError): + await r.client_pause(timeout="not an integer", target_nodes=node) + + @skip_if_server_version_lt("6.2.0") + @skip_if_redis_enterprise() + async def test_client_unpause(self, r: RedisCluster) -> None: + assert await r.client_unpause() + + @skip_if_server_version_lt("5.0.0") + async def test_client_id(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + assert await r.client_id(target_nodes=node) > 0 + + @skip_if_server_version_lt("5.0.0") + async def test_client_unblock(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + myid = await r.client_id(target_nodes=node) + assert not await r.client_unblock(myid, target_nodes=node) + assert not await r.client_unblock(myid, error=True, target_nodes=node) + assert not await r.client_unblock(myid, error=False, target_nodes=node) + + @skip_if_server_version_lt("6.0.0") + async def test_client_getredir(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + assert isinstance(await r.client_getredir(target_nodes=node), int) + assert await r.client_getredir(target_nodes=node) == -1 + + @skip_if_server_version_lt("6.2.0") + async def test_client_info(self, r: RedisCluster) -> None: + node = r.get_primaries()[0] + info = await r.client_info(target_nodes=node) + assert isinstance(info, dict) + assert "addr" in info + + @skip_if_server_version_lt("2.6.9") + async def test_client_kill( + self, r: RedisCluster, create_redis: Callable[..., RedisCluster] + ) -> None: + node = r.get_primaries()[0] + async with create_redis(cls=RedisCluster, flushdb=False) as r2: + await r.client_setname("redis-py-c1", target_nodes="all") + await r2.client_setname("redis-py-c2", target_nodes="all") + clients = [ + client + for client in await r.client_list(target_nodes=node) + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 2 + clients_by_name = {client.get("name"): client for client in clients} + + client_addr = clients_by_name["redis-py-c2"].get("addr") + assert await r.client_kill(client_addr, target_nodes=node) is True + + clients = [ + client + for client in await r.client_list(target_nodes=node) + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 1 + assert clients[0].get("name") == "redis-py-c1" + + @skip_if_server_version_lt("2.6.0") + async def test_cluster_bitop_not_empty_string(self, r: RedisCluster) -> None: + await r.set("{foo}a", "") + await r.bitop("not", "{foo}r", "{foo}a") + assert await r.get("{foo}r") is None + + @skip_if_server_version_lt("2.6.0") + async def test_cluster_bitop_not(self, r: RedisCluster) -> None: + test_str = b"\xaa\x00\xff\x55" + correct = ~0xAA00FF55 & 0xFFFFFFFF + await r.set("{foo}a", test_str) + await r.bitop("not", "{foo}r", "{foo}a") + assert int(binascii.hexlify(await r.get("{foo}r")), 16) == correct + + @skip_if_server_version_lt("2.6.0") + async def test_cluster_bitop_not_in_place(self, r: RedisCluster) -> None: + test_str = b"\xaa\x00\xff\x55" + correct = ~0xAA00FF55 & 0xFFFFFFFF + await r.set("{foo}a", test_str) + await r.bitop("not", "{foo}a", "{foo}a") + assert int(binascii.hexlify(await r.get("{foo}a")), 16) == correct + + @skip_if_server_version_lt("2.6.0") + async def test_cluster_bitop_single_string(self, r: RedisCluster) -> None: + test_str = b"\x01\x02\xff" + await r.set("{foo}a", test_str) + await r.bitop("and", "{foo}res1", "{foo}a") + await r.bitop("or", "{foo}res2", "{foo}a") + await r.bitop("xor", "{foo}res3", "{foo}a") + assert await r.get("{foo}res1") == test_str + assert await r.get("{foo}res2") == test_str + assert await r.get("{foo}res3") == test_str + + @skip_if_server_version_lt("2.6.0") + async def test_cluster_bitop_string_operands(self, r: RedisCluster) -> None: + await r.set("{foo}a", b"\x01\x02\xff\xff") + await r.set("{foo}b", b"\x01\x02\xff") + await r.bitop("and", "{foo}res1", "{foo}a", "{foo}b") + await r.bitop("or", "{foo}res2", "{foo}a", "{foo}b") + await r.bitop("xor", "{foo}res3", "{foo}a", "{foo}b") + assert int(binascii.hexlify(await r.get("{foo}res1")), 16) == 0x0102FF00 + assert int(binascii.hexlify(await r.get("{foo}res2")), 16) == 0x0102FFFF + assert int(binascii.hexlify(await r.get("{foo}res3")), 16) == 0x000000FF + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_copy(self, r: RedisCluster) -> None: + assert await r.copy("{foo}a", "{foo}b") == 0 + await r.set("{foo}a", "bar") + assert await r.copy("{foo}a", "{foo}b") == 1 + assert await r.get("{foo}a") == b"bar" + assert await r.get("{foo}b") == b"bar" + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_copy_and_replace(self, r: RedisCluster) -> None: + await r.set("{foo}a", "foo1") + await r.set("{foo}b", "foo2") + assert await r.copy("{foo}a", "{foo}b") == 0 + assert await r.copy("{foo}a", "{foo}b", replace=True) == 1 + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_lmove(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "one", "two", "three", "four") + assert await r.lmove("{foo}a", "{foo}b") + assert await r.lmove("{foo}a", "{foo}b", "right", "left") + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_blmove(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "one", "two", "three", "four") + assert await r.blmove("{foo}a", "{foo}b", 5) + assert await r.blmove("{foo}a", "{foo}b", 1, "RIGHT", "LEFT") + + async def test_cluster_msetnx(self, r: RedisCluster) -> None: + d = {"{foo}a": b"1", "{foo}b": b"2", "{foo}c": b"3"} + assert await r.msetnx(d) + d2 = {"{foo}a": b"x", "{foo}d": b"4"} + assert not await r.msetnx(d2) + for k, v in d.items(): + assert await r.get(k) == v + assert await r.get("{foo}d") is None + + async def test_cluster_rename(self, r: RedisCluster) -> None: + await r.set("{foo}a", "1") + assert await r.rename("{foo}a", "{foo}b") + assert await r.get("{foo}a") is None + assert await r.get("{foo}b") == b"1" + + async def test_cluster_renamenx(self, r: RedisCluster) -> None: + await r.set("{foo}a", "1") + await r.set("{foo}b", "2") + assert not await r.renamenx("{foo}a", "{foo}b") + assert await r.get("{foo}a") == b"1" + assert await r.get("{foo}b") == b"2" + + # LIST COMMANDS + async def test_cluster_blpop(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "1", "2") + await r.rpush("{foo}b", "3", "4") + assert_resp_response( + r, + await r.blpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"3"), + [b"{foo}b", b"3"], + ) + assert_resp_response( + r, + await r.blpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"4"), + [b"{foo}b", b"4"], + ) + assert_resp_response( + r, + await r.blpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"1"), + [b"{foo}a", b"1"], + ) + assert_resp_response( + r, + await r.blpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"2"), + [b"{foo}a", b"2"], + ) + assert await r.blpop(["{foo}b", "{foo}a"], timeout=1) is None + await r.rpush("{foo}c", "1") + assert_resp_response( + r, await r.blpop("{foo}c", timeout=1), (b"{foo}c", b"1"), [b"{foo}c", b"1"] + ) + + async def test_cluster_brpop(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "1", "2") + await r.rpush("{foo}b", "3", "4") + assert_resp_response( + r, + await r.brpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"4"), + [b"{foo}b", b"4"], + ) + assert_resp_response( + r, + await r.brpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"3"), + [b"{foo}b", b"3"], + ) + assert_resp_response( + r, + await r.brpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"2"), + [b"{foo}a", b"2"], + ) + assert_resp_response( + r, + await r.brpop(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"1"), + [b"{foo}a", b"1"], + ) + assert await r.brpop(["{foo}b", "{foo}a"], timeout=1) is None + await r.rpush("{foo}c", "1") + assert_resp_response( + r, await r.brpop("{foo}c", timeout=1), (b"{foo}c", b"1"), [b"{foo}c", b"1"] + ) + + async def test_cluster_brpoplpush(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "1", "2") + await r.rpush("{foo}b", "3", "4") + assert await r.brpoplpush("{foo}a", "{foo}b") == b"2" + assert await r.brpoplpush("{foo}a", "{foo}b") == b"1" + assert await r.brpoplpush("{foo}a", "{foo}b", timeout=1) is None + assert await r.lrange("{foo}a", 0, -1) == [] + assert await r.lrange("{foo}b", 0, -1) == [b"1", b"2", b"3", b"4"] + + async def test_cluster_brpoplpush_empty_string(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "") + assert await r.brpoplpush("{foo}a", "{foo}b") == b"" + + async def test_cluster_rpoplpush(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "a1", "a2", "a3") + await r.rpush("{foo}b", "b1", "b2", "b3") + assert await r.rpoplpush("{foo}a", "{foo}b") == b"a3" + assert await r.lrange("{foo}a", 0, -1) == [b"a1", b"a2"] + assert await r.lrange("{foo}b", 0, -1) == [b"a3", b"b1", b"b2", b"b3"] + + async def test_cluster_sdiff(self, r: RedisCluster) -> None: + await r.sadd("{foo}a", "1", "2", "3") + assert await r.sdiff("{foo}a", "{foo}b") == {b"1", b"2", b"3"} + await r.sadd("{foo}b", "2", "3") + assert await r.sdiff("{foo}a", "{foo}b") == {b"1"} + + async def test_cluster_sdiffstore(self, r: RedisCluster) -> None: + await r.sadd("{foo}a", "1", "2", "3") + assert await r.sdiffstore("{foo}c", "{foo}a", "{foo}b") == 3 + assert await r.smembers("{foo}c") == {b"1", b"2", b"3"} + await r.sadd("{foo}b", "2", "3") + assert await r.sdiffstore("{foo}c", "{foo}a", "{foo}b") == 1 + assert await r.smembers("{foo}c") == {b"1"} + + async def test_cluster_sinter(self, r: RedisCluster) -> None: + await r.sadd("{foo}a", "1", "2", "3") + assert await r.sinter("{foo}a", "{foo}b") == set() + await r.sadd("{foo}b", "2", "3") + assert await r.sinter("{foo}a", "{foo}b") == {b"2", b"3"} + + async def test_cluster_sinterstore(self, r: RedisCluster) -> None: + await r.sadd("{foo}a", "1", "2", "3") + assert await r.sinterstore("{foo}c", "{foo}a", "{foo}b") == 0 + assert await r.smembers("{foo}c") == set() + await r.sadd("{foo}b", "2", "3") + assert await r.sinterstore("{foo}c", "{foo}a", "{foo}b") == 2 + assert await r.smembers("{foo}c") == {b"2", b"3"} + + async def test_cluster_smove(self, r: RedisCluster) -> None: + await r.sadd("{foo}a", "a1", "a2") + await r.sadd("{foo}b", "b1", "b2") + assert await r.smove("{foo}a", "{foo}b", "a1") + assert await r.smembers("{foo}a") == {b"a2"} + assert await r.smembers("{foo}b") == {b"b1", b"b2", b"a1"} + + async def test_cluster_sunion(self, r: RedisCluster) -> None: + await r.sadd("{foo}a", "1", "2") + await r.sadd("{foo}b", "2", "3") + assert set(await r.sunion("{foo}a", "{foo}b")) == {b"1", b"2", b"3"} + + async def test_cluster_sunionstore(self, r: RedisCluster) -> None: + await r.sadd("{foo}a", "1", "2") + await r.sadd("{foo}b", "2", "3") + assert await r.sunionstore("{foo}c", "{foo}a", "{foo}b") == 3 + assert set(await r.smembers("{foo}c")) == {b"1", b"2", b"3"} + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_zdiff(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("{foo}b", {"a1": 1, "a2": 2}) + assert await r.zdiff(["{foo}a", "{foo}b"]) == [b"a3"] + response = await r.zdiff(["{foo}a", "{foo}b"], withscores=True) + assert_resp_response(r, response, [b"a3", b"3"], [[b"a3", 3.0]]) + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_zdiffstore(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("{foo}b", {"a1": 1, "a2": 2}) + assert await r.zdiffstore("{foo}out", ["{foo}a", "{foo}b"]) + assert await r.zrange("{foo}out", 0, -1) == [b"a3"] + response = await r.zrange("{foo}out", 0, -1, withscores=True) + assert_resp_response(r, response, [(b"a3", 3.0)], [[b"a3", 3.0]]) + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_zinter(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zinter(["{foo}a", "{foo}b", "{foo}c"]) == [b"a3", b"a1"] + # invalid aggregation + with pytest.raises(DataError): + await r.zinter( + ["{foo}a", "{foo}b", "{foo}c"], aggregate="foo", withscores=True + ) + # aggregate with SUM + response = await r.zinter(["{foo}a", "{foo}b", "{foo}c"], withscores=True) + assert_resp_response( + r, response, [(b"a3", 8), (b"a1", 9)], [[b"a3", 8], [b"a1", 9]] + ) + # aggregate with MAX + response = await r.zinter( + ["{foo}a", "{foo}b", "{foo}c"], aggregate="MAX", withscores=True + ) + assert_resp_response( + r, response, [(b"a3", 5), (b"a1", 6)], [[b"a3", 5], [b"a1", 6]] + ) + # aggregate with MIN + response = await r.zinter( + ["{foo}a", "{foo}b", "{foo}c"], aggregate="MIN", withscores=True + ) + assert_resp_response( + r, response, [(b"a1", 1), (b"a3", 1)], [[b"a1", 1], [b"a3", 1]] + ) + # with weights + res = await r.zinter({"{foo}a": 1, "{foo}b": 2, "{foo}c": 3}, withscores=True) + assert_resp_response( + r, res, [(b"a3", 20), (b"a1", 23)], [[b"a3", 20], [b"a1", 23]] + ) + + async def test_cluster_zinterstore_sum(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zinterstore("{foo}d", ["{foo}a", "{foo}b", "{foo}c"]) == 2 + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a3", 8), (b"a1", 9)], + [[b"a3", 8.0], [b"a1", 9.0]], + ) + + async def test_cluster_zinterstore_max(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert ( + await r.zinterstore( + "{foo}d", ["{foo}a", "{foo}b", "{foo}c"], aggregate="MAX" + ) + == 2 + ) + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a3", 5), (b"a1", 6)], + [[b"a3", 5.0], [b"a1", 6.0]], + ) + + async def test_cluster_zinterstore_min(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("{foo}b", {"a1": 2, "a2": 3, "a3": 5}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert ( + await r.zinterstore( + "{foo}d", ["{foo}a", "{foo}b", "{foo}c"], aggregate="MIN" + ) + == 2 + ) + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a1", 1), (b"a3", 3)], + [[b"a1", 1.0], [b"a3", 3.0]], + ) + + async def test_cluster_zinterstore_with_weight(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert ( + await r.zinterstore("{foo}d", {"{foo}a": 1, "{foo}b": 2, "{foo}c": 3}) == 2 + ) + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a3", 20), (b"a1", 23)], + [[b"a3", 20.0], [b"a1", 23.0]], + ) + + @skip_if_server_version_lt("4.9.0") + async def test_cluster_bzpopmax(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2}) + await r.zadd("{foo}b", {"b1": 10, "b2": 20}) + assert_resp_response( + r, + await r.bzpopmax(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"b2", 20), + [b"{foo}b", b"b2", 20], + ) + assert_resp_response( + r, + await r.bzpopmax(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"b1", 10), + [b"{foo}b", b"b1", 10], + ) + assert_resp_response( + r, + await r.bzpopmax(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"a2", 2), + [b"{foo}a", b"a2", 2], + ) + assert_resp_response( + r, + await r.bzpopmax(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"a1", 1), + [b"{foo}a", b"a1", 1], + ) + assert await r.bzpopmax(["{foo}b", "{foo}a"], timeout=1) is None + await r.zadd("{foo}c", {"c1": 100}) + assert_resp_response( + r, + await r.bzpopmax("{foo}c", timeout=1), + (b"{foo}c", b"c1", 100), + [b"{foo}c", b"c1", 100], + ) + + @skip_if_server_version_lt("4.9.0") + async def test_cluster_bzpopmin(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2}) + await r.zadd("{foo}b", {"b1": 10, "b2": 20}) + assert_resp_response( + r, + await r.bzpopmin(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"b1", 10), + [b"{foo}b", b"b1", 10], + ) + assert_resp_response( + r, + await r.bzpopmin(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}b", b"b2", 20), + [b"{foo}b", b"b2", 20], + ) + assert_resp_response( + r, + await r.bzpopmin(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"a1", 1), + [b"{foo}a", b"a1", 1], + ) + assert_resp_response( + r, + await r.bzpopmin(["{foo}b", "{foo}a"], timeout=1), + (b"{foo}a", b"a2", 2), + [b"{foo}a", b"a2", 2], + ) + assert await r.bzpopmin(["{foo}b", "{foo}a"], timeout=1) is None + await r.zadd("{foo}c", {"c1": 100}) + assert_resp_response( + r, + await r.bzpopmin("{foo}c", timeout=1), + (b"{foo}c", b"c1", 100), + [b"{foo}c", b"c1", 100], + ) + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_zrangestore(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zrangestore("{foo}b", "{foo}a", 0, 1) + assert await r.zrange("{foo}b", 0, -1) == [b"a1", b"a2"] + assert await r.zrangestore("{foo}b", "{foo}a", 1, 2) + assert await r.zrange("{foo}b", 0, -1) == [b"a2", b"a3"] + assert_resp_response( + r, + await r.zrange("{foo}b", 0, -1, withscores=True), + [(b"a2", 2), (b"a3", 3)], + [[b"a2", 2.0], [b"a3", 3.0]], + ) + # reversed order + assert await r.zrangestore("{foo}b", "{foo}a", 1, 2, desc=True) + assert await r.zrange("{foo}b", 0, -1) == [b"a1", b"a2"] + # by score + assert await r.zrangestore( + "{foo}b", "{foo}a", 2, 1, byscore=True, offset=0, num=1, desc=True + ) + assert await r.zrange("{foo}b", 0, -1) == [b"a2"] + # by lex + assert await r.zrangestore( + "{foo}b", "{foo}a", "[a2", "(a3", bylex=True, offset=0, num=1 + ) + assert await r.zrange("{foo}b", 0, -1) == [b"a2"] + + @skip_if_server_version_lt("6.2.0") + async def test_cluster_zunion(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + # sum + assert await r.zunion(["{foo}a", "{foo}b", "{foo}c"]) == [ + b"a2", + b"a4", + b"a3", + b"a1", + ] + assert_resp_response( + r, + await r.zunion(["{foo}a", "{foo}b", "{foo}c"], withscores=True), + [(b"a2", 3), (b"a4", 4), (b"a3", 8), (b"a1", 9)], + [[b"a2", 3.0], [b"a4", 4.0], [b"a3", 8.0], [b"a1", 9.0]], + ) + # max + assert_resp_response( + r, + await r.zunion( + ["{foo}a", "{foo}b", "{foo}c"], aggregate="MAX", withscores=True + ), + [(b"a2", 2), (b"a4", 4), (b"a3", 5), (b"a1", 6)], + [[b"a2", 2.0], [b"a4", 4.0], [b"a3", 5.0], [b"a1", 6.0]], + ) + # min + assert_resp_response( + r, + await r.zunion( + ["{foo}a", "{foo}b", "{foo}c"], aggregate="MIN", withscores=True + ), + [(b"a1", 1), (b"a2", 1), (b"a3", 1), (b"a4", 4)], + [[b"a1", 1.0], [b"a2", 1.0], [b"a3", 1.0], [b"a4", 4.0]], + ) + # with weight + assert_resp_response( + r, + await r.zunion({"{foo}a": 1, "{foo}b": 2, "{foo}c": 3}, withscores=True), + [(b"a2", 5), (b"a4", 12), (b"a3", 20), (b"a1", 23)], + [[b"a2", 5.0], [b"a4", 12.0], [b"a3", 20.0], [b"a1", 23.0]], + ) + + async def test_cluster_zunionstore_sum(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zunionstore("{foo}d", ["{foo}a", "{foo}b", "{foo}c"]) == 4 + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a2", 3), (b"a4", 4), (b"a3", 8), (b"a1", 9)], + [[b"a2", 3.0], [b"a4", 4.0], [b"a3", 8.0], [b"a1", 9.0]], + ) + + async def test_cluster_zunionstore_max(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert ( + await r.zunionstore( + "{foo}d", ["{foo}a", "{foo}b", "{foo}c"], aggregate="MAX" + ) + == 4 + ) + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a2", 2), (b"a4", 4), (b"a3", 5), (b"a1", 6)], + [[b"a2", 2.0], [b"a4", 4.0], [b"a3", 5.0], [b"a1", 6.0]], + ) + + async def test_cluster_zunionstore_min(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 4}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert ( + await r.zunionstore( + "{foo}d", ["{foo}a", "{foo}b", "{foo}c"], aggregate="MIN" + ) + == 4 + ) + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a1", 1), (b"a2", 2), (b"a3", 3), (b"a4", 4)], + [[b"a1", 1.0], [b"a2", 2.0], [b"a3", 3.0], [b"a4", 4.0]], + ) + + async def test_cluster_zunionstore_with_weight(self, r: RedisCluster) -> None: + await r.zadd("{foo}a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("{foo}b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("{foo}c", {"a1": 6, "a3": 5, "a4": 4}) + assert ( + await r.zunionstore("{foo}d", {"{foo}a": 1, "{foo}b": 2, "{foo}c": 3}) == 4 + ) + assert_resp_response( + r, + await r.zrange("{foo}d", 0, -1, withscores=True), + [(b"a2", 5), (b"a4", 12), (b"a3", 20), (b"a1", 23)], + [[b"a2", 5.0], [b"a4", 12.0], [b"a3", 20.0], [b"a1", 23.0]], + ) + + @skip_if_server_version_lt("2.8.9") + async def test_cluster_pfcount(self, r: RedisCluster) -> None: + members = {b"1", b"2", b"3"} + await r.pfadd("{foo}a", *members) + assert await r.pfcount("{foo}a") == len(members) + members_b = {b"2", b"3", b"4"} + await r.pfadd("{foo}b", *members_b) + assert await r.pfcount("{foo}b") == len(members_b) + assert await r.pfcount("{foo}a", "{foo}b") == len(members_b.union(members)) + + @skip_if_server_version_lt("2.8.9") + async def test_cluster_pfmerge(self, r: RedisCluster) -> None: + mema = {b"1", b"2", b"3"} + memb = {b"2", b"3", b"4"} + memc = {b"5", b"6", b"7"} + await r.pfadd("{foo}a", *mema) + await r.pfadd("{foo}b", *memb) + await r.pfadd("{foo}c", *memc) + await r.pfmerge("{foo}d", "{foo}c", "{foo}a") + assert await r.pfcount("{foo}d") == 6 + await r.pfmerge("{foo}d", "{foo}b") + assert await r.pfcount("{foo}d") == 7 + + async def test_cluster_sort_store(self, r: RedisCluster) -> None: + await r.rpush("{foo}a", "2", "3", "1") + assert await r.sort("{foo}a", store="{foo}sorted_values") == 3 + assert await r.lrange("{foo}sorted_values", 0, -1) == [b"1", b"2", b"3"] + + # GEO COMMANDS + @skip_if_server_version_lt("6.2.0") + async def test_cluster_geosearchstore(self, r: RedisCluster) -> None: + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("{foo}barcelona", values) + await r.geosearchstore( + "{foo}places_barcelona", + "{foo}barcelona", + longitude=2.191, + latitude=41.433, + radius=1000, + ) + assert await r.zrange("{foo}places_barcelona", 0, -1) == [b"place1"] + + @skip_unless_arch_bits(64) + @skip_if_server_version_lt("6.2.0") + async def test_geosearchstore_dist(self, r: RedisCluster) -> None: + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("{foo}barcelona", values) + await r.geosearchstore( + "{foo}places_barcelona", + "{foo}barcelona", + longitude=2.191, + latitude=41.433, + radius=1000, + storedist=True, + ) + # instead of save the geo score, the distance is saved. + assert await r.zscore("{foo}places_barcelona", "place1") == 88.05060698409301 + + @skip_if_server_version_lt("3.2.0") + async def test_cluster_georadius_store(self, r: RedisCluster) -> None: + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("{foo}barcelona", values) + await r.georadius( + "{foo}barcelona", 2.191, 41.433, 1000, store="{foo}places_barcelona" + ) + assert await r.zrange("{foo}places_barcelona", 0, -1) == [b"place1"] + + @skip_unless_arch_bits(64) + @skip_if_server_version_lt("3.2.0") + async def test_cluster_georadius_store_dist(self, r: RedisCluster) -> None: + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("{foo}barcelona", values) + await r.georadius( + "{foo}barcelona", 2.191, 41.433, 1000, store_dist="{foo}places_barcelona" + ) + # instead of save the geo score, the distance is saved. + assert await r.zscore("{foo}places_barcelona", "place1") == 88.05060698409301 + + async def test_cluster_dbsize(self, r: RedisCluster) -> None: + d = {"a": b"1", "b": b"2", "c": b"3", "d": b"4"} + assert await r.mset_nonatomic(d) + assert await r.dbsize(target_nodes="primaries") == len(d) + + async def test_cluster_keys(self, r: RedisCluster) -> None: + assert await r.keys() == [] + keys_with_underscores = {b"test_a", b"test_b"} + keys = keys_with_underscores.union({b"testc"}) + for key in keys: + await r.set(key, 1) + assert ( + set(await r.keys(pattern="test_*", target_nodes="primaries")) + == keys_with_underscores + ) + assert set(await r.keys(pattern="test*", target_nodes="primaries")) == keys + + # SCAN COMMANDS + @skip_if_server_version_lt("2.8.0") + async def test_cluster_scan(self, r: RedisCluster) -> None: + await r.set("a", 1) + await r.set("b", 2) + await r.set("c", 3) + + for target_nodes, nodes in zip( + ["primaries", "replicas"], [r.get_primaries(), r.get_replicas()] + ): + cursors, keys = await r.scan(target_nodes=target_nodes) + assert sorted(keys) == [b"a", b"b", b"c"] + assert sorted(cursors.keys()) == sorted(node.name for node in nodes) + assert all(cursor == 0 for cursor in cursors.values()) + + cursors, keys = await r.scan(match="a*", target_nodes=target_nodes) + assert sorted(keys) == [b"a"] + assert sorted(cursors.keys()) == sorted(node.name for node in nodes) + assert all(cursor == 0 for cursor in cursors.values()) + + @skip_if_server_version_lt("6.0.0") + async def test_cluster_scan_type(self, r: RedisCluster) -> None: + await r.sadd("a-set", 1) + await r.sadd("b-set", 1) + await r.sadd("c-set", 1) + await r.hset("a-hash", "foo", 2) + await r.lpush("a-list", "aux", 3) + + for target_nodes, nodes in zip( + ["primaries", "replicas"], [r.get_primaries(), r.get_replicas()] + ): + cursors, keys = await r.scan(_type="SET", target_nodes=target_nodes) + assert sorted(keys) == [b"a-set", b"b-set", b"c-set"] + assert sorted(cursors.keys()) == sorted(node.name for node in nodes) + assert all(cursor == 0 for cursor in cursors.values()) + + cursors, keys = await r.scan( + _type="SET", match="a*", target_nodes=target_nodes + ) + assert sorted(keys) == [b"a-set"] + assert sorted(cursors.keys()) == sorted(node.name for node in nodes) + assert all(cursor == 0 for cursor in cursors.values()) + + @skip_if_server_version_lt("2.8.0") + async def test_cluster_scan_iter(self, r: RedisCluster) -> None: + keys_all = [] + keys_1 = [] + for i in range(100): + s = str(i) + await r.set(s, 1) + keys_all.append(s.encode("utf-8")) + if s.startswith("1"): + keys_1.append(s.encode("utf-8")) + keys_all.sort() + keys_1.sort() + + for target_nodes in ["primaries", "replicas"]: + keys = [key async for key in r.scan_iter(target_nodes=target_nodes)] + assert sorted(keys) == keys_all + + keys = [ + key async for key in r.scan_iter(match="1*", target_nodes=target_nodes) + ] + assert sorted(keys) == keys_1 + + async def test_cluster_randomkey(self, r: RedisCluster) -> None: + node = r.get_node_from_key("{foo}") + assert await r.randomkey(target_nodes=node) is None + for key in ("{foo}a", "{foo}b", "{foo}c"): + await r.set(key, 1) + assert await r.randomkey(target_nodes=node) in (b"{foo}a", b"{foo}b", b"{foo}c") + + @skip_if_server_version_lt("6.0.0") + @skip_if_redis_enterprise() + async def test_acl_log( + self, r: RedisCluster, create_redis: Callable[..., RedisCluster] + ) -> None: + key = "{cache}:" + node = r.get_node_from_key(key) + username = "redis-py-user" + + await r.acl_setuser( + username, + enabled=True, + reset=True, + commands=["+get", "+set", "+select", "+cluster", "+command", "+info"], + keys=["{cache}:*"], + nopass=True, + target_nodes="primaries", + ) + await r.acl_log_reset(target_nodes=node) + + async with create_redis( + cls=RedisCluster, flushdb=False, username=username + ) as user_client: + # Valid operation and key + assert await user_client.set("{cache}:0", 1) + assert await user_client.get("{cache}:0") == b"1" + + # Invalid key + with pytest.raises(NoPermissionError): + await user_client.get("{cache}violated_cache:0") + + # Invalid operation + with pytest.raises(NoPermissionError): + await user_client.hset("{cache}:0", "hkey", "hval") + + assert isinstance(await r.acl_log(target_nodes=node), list) + assert len(await r.acl_log(target_nodes=node)) == 3 + assert len(await r.acl_log(count=1, target_nodes=node)) == 1 + assert isinstance((await r.acl_log(target_nodes=node))[0], dict) + assert "client-info" in (await r.acl_log(count=1, target_nodes=node))[0] + assert await r.acl_log_reset(target_nodes=node) + + await r.acl_deluser(username, target_nodes="primaries") + + +class TestNodesManager: + """ + Tests for the NodesManager class + """ + + async def test_load_balancer(self, r: RedisCluster) -> None: + n_manager = r.nodes_manager + lb = n_manager.read_load_balancer + slot_1 = 1257 + slot_2 = 8975 + node_1 = ClusterNode(default_host, 6379, PRIMARY) + node_2 = ClusterNode(default_host, 6378, REPLICA) + node_3 = ClusterNode(default_host, 6377, REPLICA) + node_4 = ClusterNode(default_host, 6376, PRIMARY) + node_5 = ClusterNode(default_host, 6375, REPLICA) + n_manager.slots_cache = { + slot_1: [node_1, node_2, node_3], + slot_2: [node_4, node_5], + } + primary1_name = n_manager.slots_cache[slot_1][0].name + primary2_name = n_manager.slots_cache[slot_2][0].name + list1_size = len(n_manager.slots_cache[slot_1]) + list2_size = len(n_manager.slots_cache[slot_2]) + + # default load balancer strategy: LoadBalancerStrategy.ROUND_ROBIN + # slot 1 + assert lb.get_server_index(primary1_name, list1_size) == 0 + assert lb.get_server_index(primary1_name, list1_size) == 1 + assert lb.get_server_index(primary1_name, list1_size) == 2 + assert lb.get_server_index(primary1_name, list1_size) == 0 + + # slot 2 + assert lb.get_server_index(primary2_name, list2_size) == 0 + assert lb.get_server_index(primary2_name, list2_size) == 1 + assert lb.get_server_index(primary2_name, list2_size) == 0 + + lb.reset() + assert lb.get_server_index(primary1_name, list1_size) == 0 + assert lb.get_server_index(primary2_name, list2_size) == 0 + + # reset the indexes before load balancing strategy test + lb.reset() + # load balancer strategy: LoadBalancerStrategy.ROUND_ROBIN_REPLICAS + for i in [1, 2, 1]: + srv_index = lb.get_server_index( + primary1_name, + list1_size, + load_balancing_strategy=LoadBalancingStrategy.ROUND_ROBIN_REPLICAS, + ) + assert srv_index == i + + # reset the indexes before load balancing strategy test + lb.reset() + # load balancer strategy: LoadBalancerStrategy.RANDOM_REPLICA + for i in range(5): + srv_index = lb.get_server_index( + primary1_name, + list1_size, + load_balancing_strategy=LoadBalancingStrategy.RANDOM_REPLICA, + ) + + assert srv_index > 0 and srv_index <= 2 + + async def test_init_slots_cache_not_all_slots_covered(self) -> None: + """ + Test that if not all slots are covered it should raise an exception + """ + # Missing slot 5460 + cluster_slots = [ + [0, 5459, ["127.0.0.1", 7000], ["127.0.0.1", 7003]], + [5461, 10922, ["127.0.0.1", 7001], ["127.0.0.1", 7004]], + [10923, 16383, ["127.0.0.1", 7002], ["127.0.0.1", 7005]], + ] + with pytest.raises( + RedisClusterException, + match="^All slots are not covered after query all startup_nodes.", + ): + async with get_mocked_redis_client( + host=default_host, + port=default_port, + cluster_slots=cluster_slots, + require_full_coverage=True, + ): + pass + + async def test_init_slots_cache_not_require_full_coverage_success(self) -> None: + """ + When require_full_coverage is set to False and not all slots are + covered the cluster client initialization should succeed + """ + # Missing slot 5460 + cluster_slots = [ + [0, 5459, ["127.0.0.1", 7000], ["127.0.0.1", 7003]], + [5461, 10922, ["127.0.0.1", 7001], ["127.0.0.1", 7004]], + [10923, 16383, ["127.0.0.1", 7002], ["127.0.0.1", 7005]], + ] + + async with get_mocked_redis_client( + host=default_host, + port=default_port, + cluster_slots=cluster_slots, + require_full_coverage=False, + ) as rc: + assert 5460 not in rc.nodes_manager.slots_cache + + async def test_init_slots_cache(self) -> None: + """ + Test that slots cache can in initialized and all slots are covered + """ + good_slots_resp = [ + [0, 5460, ["127.0.0.1", 7000], ["127.0.0.2", 7003]], + [5461, 10922, ["127.0.0.1", 7001], ["127.0.0.2", 7004]], + [10923, 16383, ["127.0.0.1", 7002], ["127.0.0.2", 7005]], + ] + + async with get_mocked_redis_client( + host=default_host, port=default_port, cluster_slots=good_slots_resp + ) as rc: + n_manager = rc.nodes_manager + assert len(n_manager.slots_cache) == REDIS_CLUSTER_HASH_SLOTS + for slot_info in good_slots_resp: + all_hosts = ["127.0.0.1", "127.0.0.2"] + all_ports = [7000, 7001, 7002, 7003, 7004, 7005] + slot_start = slot_info[0] + slot_end = slot_info[1] + for i in range(slot_start, slot_end + 1): + assert len(n_manager.slots_cache[i]) == len(slot_info[2:]) + assert n_manager.slots_cache[i][0].host in all_hosts + assert n_manager.slots_cache[i][1].host in all_hosts + assert n_manager.slots_cache[i][0].port in all_ports + assert n_manager.slots_cache[i][1].port in all_ports + + assert len(n_manager.nodes_cache) == 6 + + async def test_init_slots_cache_cluster_mode_disabled(self) -> None: + """ + Test that creating a RedisCluster failes if one of the startup nodes + has cluster mode disabled + """ + with pytest.raises(RedisClusterException) as e: + async with get_mocked_redis_client( + cluster_slots_raise_error=True, + host=default_host, + port=default_port, + cluster_enabled=False, + ): + pass + assert "Cluster mode is not enabled on this node" in str(e.value) + + async def test_empty_startup_nodes(self) -> None: + """ + It should not be possible to create a node manager with no nodes + specified + """ + with pytest.raises(RedisClusterException): + async with NodesManager([], False, {}) as nodes_manager: + await nodes_manager.initialize() + + async def test_wrong_startup_nodes_type(self) -> None: + """ + If something other then a list type iterable is provided it should fail + """ + with pytest.raises(RedisClusterException): + async with NodesManager({}, False, {}) as nodes_manager: + await nodes_manager.initialize() + + async def test_init_slots_cache_slots_collision(self) -> None: + """ + Test that if 2 nodes do not agree on the same slots setup it should + raise an error. In this test both nodes will say that the first + slots block should be bound to different servers. + """ + + async def mocked_execute_command(self, *args, **kwargs): + """ + Helper function to return custom slots cache data from + different redis nodes + """ + if self.port == 7000: + result = [ + [0, 5460, ["127.0.0.1", 7000], ["127.0.0.1", 7003]], + [5461, 10922, ["127.0.0.1", 7001], ["127.0.0.1", 7004]], + ] + elif self.port == 7001: + result = [ + [0, 5460, ["127.0.0.1", 7001], ["127.0.0.1", 7003]], + [5461, 10922, ["127.0.0.1", 7000], ["127.0.0.1", 7004]], + ] + else: + result = [] + + if args[0] == "CLUSTER SLOTS": + return result + elif args[0] == "INFO": + return {"cluster_enabled": True} + elif args[1] == "cluster-require-full-coverage": + return {"cluster-require-full-coverage": "yes"} + + with patch.object(ClusterNode, "execute_command", mocked_execute_command): + with pytest.raises( + RedisClusterException, + match="^startup_nodes could not agree on a valid slots cache", + ): + node_1 = ClusterNode("127.0.0.1", 7000) + node_2 = ClusterNode("127.0.0.1", 7001) + async with RedisCluster(startup_nodes=[node_1, node_2]): + ... + + async def test_cluster_one_instance(self) -> None: + """ + If the cluster exists of only 1 node then there is some hacks that must + be validated they work. + """ + node = ClusterNode(default_host, default_port) + cluster_slots = [[0, 16383, ["", default_port]]] + async with get_mocked_redis_client( + startup_nodes=[node], cluster_slots=cluster_slots + ) as rc: + n = rc.nodes_manager + assert len(n.nodes_cache) == 1 + n_node = rc.get_node(node_name=node.name) + assert n_node is not None + assert n_node == node + assert n_node.server_type == PRIMARY + assert len(n.slots_cache) == REDIS_CLUSTER_HASH_SLOTS + for i in range(0, REDIS_CLUSTER_HASH_SLOTS): + assert n.slots_cache[i] == [n_node] + + async def test_init_with_down_node(self) -> None: + """ + If I can't connect to one of the nodes, everything should still work. + But if I can't connect to any of the nodes, exception should be thrown. + """ + with patch.object( + ClusterNode, "execute_command", autospec=True + ) as execute_command: + + async def mocked_execute_command(self, *args, **kwargs): + if self.port == 7000: + raise ConnectionError("mock connection error for 7000") + + if args[0] == "CLUSTER SLOTS": + return [ + [0, 8191, ["127.0.0.1", 7001, "node_1"]], + [8192, 16383, ["127.0.0.1", 7002, "node_2"]], + ] + elif args[0] == "INFO": + return {"cluster_enabled": True} + elif args[1] == "cluster-require-full-coverage": + return {"cluster-require-full-coverage": "yes"} + + execute_command.side_effect = mocked_execute_command + + node_1 = ClusterNode("127.0.0.1", 7000) + node_2 = ClusterNode("127.0.0.1", 7001) + + # If all startup nodes fail to connect, connection error should be + # thrown + with pytest.raises(RedisClusterException) as e: + async with RedisCluster(startup_nodes=[node_1]): + ... + assert "Redis Cluster cannot be connected" in str(e.value) + + with patch.object( + AsyncCommandsParser, "initialize", autospec=True + ) as cmd_parser_initialize: + + def cmd_init_mock(self, r: ClusterNode) -> None: + self.commands = { + "GET": { + "name": "get", + "arity": 2, + "flags": ["readonly", "fast"], + "first_key_pos": 1, + "last_key_pos": 1, + "step_count": 1, + } + } + + cmd_parser_initialize.side_effect = cmd_init_mock + # When at least one startup node is reachable, the cluster + # initialization should succeeds + async with RedisCluster(startup_nodes=[node_1, node_2]) as rc: + assert rc.get_node(host=default_host, port=7001) is not None + assert rc.get_node(host=default_host, port=7002) is not None + + +class TestClusterPipeline: + """Tests for the ClusterPipeline class.""" + + async def test_blocked_arguments(self, r: RedisCluster) -> None: + """Test handling for blocked pipeline arguments.""" + with pytest.raises(RedisClusterException) as ex: + r.pipeline(transaction=True) + + assert str(ex.value) == "transaction is deprecated in cluster mode" + + with pytest.raises(RedisClusterException) as ex: + r.pipeline(shard_hint=True) + + assert str(ex.value) == "shard_hint is deprecated in cluster mode" + + async def test_blocked_methods(self, r: RedisCluster) -> None: + """Test handling for blocked pipeline commands.""" + pipeline = r.pipeline() + for command in PIPELINE_BLOCKED_COMMANDS: + command = command.replace(" ", "_").lower() + if command == "mset_nonatomic": + continue + + with pytest.raises(RedisClusterException) as exc: + getattr(pipeline, command)() + + assert str(exc.value) == ( + f"ERROR: Calling pipelined function {command} is blocked " + "when running redis in cluster mode..." + ) + + async def test_empty_stack(self, r: RedisCluster) -> None: + """If a pipeline is executed with no commands it should return a empty list.""" + p = r.pipeline() + result = await p.execute() + assert result == [] + + async def test_redis_cluster_pipeline(self, r: RedisCluster) -> None: + """Test that we can use a pipeline with the RedisCluster class""" + result = await ( + r.pipeline() + .set("A", 1) + .get("A") + .hset("K", "F", "V") + .hgetall("K") + .mset_nonatomic({"A": 2, "B": 3}) + .get("A") + .get("B") + .delete("A", "B", "K") + .execute() + ) + assert result == [True, b"1", 1, {b"F": b"V"}, True, True, b"2", b"3", 1, 1, 1] + + async def test_cluster_pipeline_execution_zero_cluster_err_retries( + self, r: RedisCluster + ) -> None: + """ + Test that we can run successfully cluster pipeline execute at least once when + cluster_error_retry_attempts is set to 0 + """ + r.cluster_error_retry_attempts = 0 + result = await r.pipeline().set("A", 1).get("A").delete("A").execute() + assert result == [True, b"1", 1] + + async def test_multi_key_operation_with_a_single_slot( + self, r: RedisCluster + ) -> None: + """Test multi key operation with a single slot.""" + pipe = r.pipeline() + pipe.set("a{foo}", 1) + pipe.set("b{foo}", 2) + pipe.set("c{foo}", 3) + pipe.get("a{foo}") + pipe.get("b{foo}") + pipe.get("c{foo}") + + res = await pipe.execute() + assert res == [True, True, True, b"1", b"2", b"3"] + + async def test_multi_key_operation_with_multi_slots(self, r: RedisCluster) -> None: + """Test multi key operation with more than one slot.""" + pipe = r.pipeline() + pipe.set("a{foo}", 1) + pipe.set("b{foo}", 2) + pipe.set("c{foo}", 3) + pipe.set("bar", 4) + pipe.set("bazz", 5) + pipe.get("a{foo}") + pipe.get("b{foo}") + pipe.get("c{foo}") + pipe.get("bar") + pipe.get("bazz") + res = await pipe.execute() + assert res == [True, True, True, True, True, b"1", b"2", b"3", b"4", b"5"] + + async def test_cluster_down_error(self, r: RedisCluster) -> None: + """ + Test that the pipeline retries cluster_error_retry_attempts times before raising + an error. + """ + key = "foo" + node = r.get_node_from_key(key, False) + + parse_response_orig = node.parse_response + with patch.object( + ClusterNode, "parse_response", autospec=True + ) as parse_response_mock: + + async def parse_response( + self, connection: Connection, command: str, **kwargs: Any + ) -> Any: + if command == "GET": + raise ClusterDownError("error") + return await parse_response_orig(connection, command, **kwargs) + + parse_response_mock.side_effect = parse_response + + # For each ClusterDownError, we launch 4 commands: INFO, CLUSTER SLOTS, + # COMMAND, GET. Before any errors, the first 3 commands are already run + async with r.pipeline() as pipe: + with pytest.raises(ClusterDownError): + await pipe.get(key).execute() + assert ( + node.parse_response.await_count + == 3 * r.cluster_error_retry_attempts + 1 + ) + + async def test_connection_error_not_raised(self, r: RedisCluster) -> None: + """Test ConnectionError handling with raise_on_error=False.""" + key = "foo" + node = r.get_node_from_key(key, False) + + parse_response_orig = node.parse_response + with patch.object( + ClusterNode, "parse_response", autospec=True + ) as parse_response_mock: + + async def parse_response( + self, connection: Connection, command: str, **kwargs: Any + ) -> Any: + if command == "GET": + raise ConnectionError("error") + return await parse_response_orig(connection, command, **kwargs) + + parse_response_mock.side_effect = parse_response + + async with r.pipeline() as pipe: + res = await pipe.get(key).get(key).execute(raise_on_error=False) + assert node.parse_response.await_count + assert isinstance(res[0], ConnectionError) + + async def test_connection_error_raised(self, r: RedisCluster) -> None: + """Test ConnectionError handling with raise_on_error=True.""" + key = "foo" + node = r.get_node_from_key(key, False) + + parse_response_orig = node.parse_response + with patch.object( + ClusterNode, "parse_response", autospec=True + ) as parse_response_mock: + + async def parse_response( + self, connection: Connection, command: str, **kwargs: Any + ) -> Any: + if command == "GET": + raise ConnectionError("error") + return await parse_response_orig(connection, command, **kwargs) + + parse_response_mock.side_effect = parse_response + + async with r.pipeline() as pipe: + with pytest.raises(ConnectionError): + await pipe.get(key).get(key).execute(raise_on_error=True) + + async def test_asking_error(self, r: RedisCluster) -> None: + """Test AskError handling.""" + key = "foo" + first_node = r.get_node_from_key(key, False) + ask_node = None + for node in r.get_nodes(): + if node != first_node: + ask_node = node + break + ask_msg = f"{r.keyslot(key)} {ask_node.host}:{ask_node.port}" + + async with r.pipeline() as pipe: + mock_node_resp_exc(first_node, AskError(ask_msg)) + mock_node_resp(ask_node, "MOCK_OK") + res = await pipe.get(key).execute() + assert first_node._free.pop().read_response.await_count + assert ask_node._free.pop().read_response.await_count + assert res == ["MOCK_OK"] + + async def test_moved_redirection_on_slave_with_default( + self, r: RedisCluster + ) -> None: + """Test MovedError handling.""" + key = "foo" + await r.set("foo", "bar") + # set read_from_replicas to True + r.read_from_replicas = True + primary = r.get_node_from_key(key, False) + moved_error = f"{r.keyslot(key)} {primary.host}:{primary.port}" + + parse_response_orig = primary.parse_response + with patch.object( + ClusterNode, "parse_response", autospec=True + ) as parse_response_mock: + + async def parse_response( + self, connection: Connection, command: str, **kwargs: Any + ) -> Any: + if command == "GET" and self.port != primary.port: + raise MovedError(moved_error) + + return await parse_response_orig(connection, command, **kwargs) + + parse_response_mock.side_effect = parse_response + + async with r.pipeline() as readwrite_pipe: + assert r.reinitialize_counter == 0 + readwrite_pipe.get(key).get(key) + assert r.reinitialize_counter == 0 + assert await readwrite_pipe.execute() == [b"bar", b"bar"] + + async def test_readonly_pipeline_from_readonly_client( + self, r: RedisCluster + ) -> None: + """Test that the pipeline uses replicas for read_from_replicas clients.""" + # Create a cluster with reading from replications + r.read_from_replicas = True + key = "bar" + await r.set(key, "foo") + + async with r.pipeline() as pipe: + mock_all_nodes_resp(r, "MOCK_OK") + assert await pipe.get(key).get(key).execute() == ["MOCK_OK", "MOCK_OK"] + slot_nodes = r.nodes_manager.slots_cache[r.keyslot(key)] + executed_on_replica = False + for node in slot_nodes: + if node.server_type == REPLICA: + if node._free.pop().read_response.await_count: + executed_on_replica = True + break + assert executed_on_replica + + @pytest.mark.parametrize( + "load_balancing_strategy", + [ + LoadBalancingStrategy.ROUND_ROBIN_REPLICAS, + LoadBalancingStrategy.RANDOM_REPLICA, + ], + ) + async def test_readonly_pipeline_with_reading_from_replicas_strategies( + self, r: RedisCluster, load_balancing_strategy: LoadBalancingStrategy + ) -> None: + """ + Test that the pipeline uses replicas for different replica-based + load balancing strategies. + """ + # Set the load balancing strategy + r.load_balancing_strategy = load_balancing_strategy + key = "bar" + await r.set(key, "foo") + + async with r.pipeline() as pipe: + mock_all_nodes_resp(r, "MOCK_OK") + assert await pipe.get(key).get(key).execute() == ["MOCK_OK", "MOCK_OK"] + slot_nodes = r.nodes_manager.slots_cache[r.keyslot(key)] + executed_on_replicas_only = True + for node in slot_nodes: + if node.server_type == PRIMARY: + if node._free.pop().read_response.await_count > 0: + executed_on_replicas_only = False + break + assert executed_on_replicas_only + + async def test_can_run_concurrent_pipelines(self, r: RedisCluster) -> None: + """Test that the pipeline can be used concurrently.""" + await gather( + *(self.test_redis_cluster_pipeline(r) for i in range(100)), + *(self.test_multi_key_operation_with_a_single_slot(r) for i in range(100)), + *(self.test_multi_key_operation_with_multi_slots(r) for i in range(100)), + ) + + @pytest.mark.onlycluster + async def test_pipeline_with_default_node_error_command(self, create_redis): + """ + Test that the default node is being replaced when it raises a relevant exception + """ + async with create_redis(cls=RedisCluster, flushdb=False) as r: + curr_default_node = r.get_default_node() + err = ConnectionError("error") + cmd_count = await r.command_count() + mock_node_resp_exc(curr_default_node, err) + async with r.pipeline(transaction=False) as pipe: + pipe.command_count() + result = await pipe.execute(raise_on_error=False) + assert result[0] == err + assert r.get_default_node() != curr_default_node + pipe.command_count() + result = await pipe.execute(raise_on_error=False) + assert result[0] == cmd_count + + +@pytest.mark.ssl +class TestSSL: + """ + Tests for SSL connections. + + This relies on the --redis-ssl-url for building the client and connecting to the + appropriate port. + """ + + @pytest.fixture + def create_client( + self, request: FixtureRequest + ) -> Callable[..., AbstractAsyncContextManager[RedisCluster]]: + ssl_url = request.config.option.redis_ssl_url + ssl_host, ssl_port = urlparse(ssl_url)[1].split(":") + self.client_cert, self.client_key, self.ca_cert = get_tls_certificates( + "cluster" + ) + + @asynccontextmanager + async def _create_client( + mocked: bool = True, **kwargs: Any + ) -> AsyncGenerator[RedisCluster]: + if mocked: + + async def execute_command(self, *args, **kwargs): + if args[0] == "INFO": + return {"cluster_enabled": True} + if args[0] == "CLUSTER SLOTS": + return [[0, 16383, [ssl_host, ssl_port, "ssl_node"]]] + if args[0] == "COMMAND": + return { + "ping": { + "name": "ping", + "arity": -1, + "flags": ["stale", "fast"], + "first_key_pos": 0, + "last_key_pos": 0, + "step_count": 0, + } + } + raise NotImplementedError() + + patcher = patch.object(ClusterNode, "execute_command", autospec=True) + execute_command_mock = patcher.start() + execute_command_mock.side_effect = execute_command + async with RedisCluster(host=ssl_host, port=ssl_port, **kwargs) as rc: + patcher.stop() + assert len(rc.get_nodes()) == 1 + node = rc.get_default_node() + assert node.port == int(ssl_port) + yield rc + else: + async with RedisCluster(host=ssl_host, port=ssl_port, **kwargs) as rc: + yield rc + + return _create_client + + async def test_ssl_connection_without_ssl( + self, create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]] + ) -> None: + with pytest.raises(RedisClusterException) as e: + async with create_client(mocked=False, ssl=False): + pass + e = e.value.__cause__ + assert "Connection closed by server" in str(e) + + async def test_ssl_with_invalid_cert( + self, create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]] + ) -> None: + with pytest.raises(RedisClusterException) as e: + async with create_client(mocked=False, ssl=True): + pass + e = e.value.__cause__.__context__ + assert "SSL: CERTIFICATE_VERIFY_FAILED" in str(e) + + async def test_ssl_connection( + self, create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]] + ) -> None: + async with create_client(ssl=True, ssl_cert_reqs="none") as rc: + assert await rc.ping() + + @pytest.mark.parametrize( + "ssl_ciphers", + [ + "AES256-SHA:DHE-RSA-AES256-SHA:AES128-SHA:DHE-RSA-AES128-SHA", + "ECDHE-ECDSA-AES256-GCM-SHA384", + "ECDHE-RSA-AES128-GCM-SHA256", + ], + ) + async def test_ssl_connection_tls12_custom_ciphers( + self, + ssl_ciphers, + create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]], + ) -> None: + async with create_client( + ssl=True, + ssl_cert_reqs="none", + ssl_min_version=ssl.TLSVersion.TLSv1_2, + ssl_ciphers=ssl_ciphers, + ) as rc: + assert await rc.ping() + + async def test_ssl_connection_tls12_custom_ciphers_invalid( + self, create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]] + ) -> None: + async with create_client( + ssl=True, + ssl_cert_reqs="none", + ssl_min_version=ssl.TLSVersion.TLSv1_2, + ssl_ciphers="foo:bar", + ) as rc: + with pytest.raises(RedisClusterException) as e: + assert await rc.ping() + assert "Redis Cluster cannot be connected" in str(e.value) + + @pytest.mark.parametrize( + "ssl_ciphers", + [ + "TLS_CHACHA20_POLY1305_SHA256", + "TLS_AES_256_GCM_SHA384:TLS_CHACHA20_POLY1305_SHA256", + ], + ) + async def test_ssl_connection_tls13_custom_ciphers( + self, + ssl_ciphers, + create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]], + ) -> None: + # TLSv1.3 does not support changing the ciphers + async with create_client( + ssl=True, + ssl_cert_reqs="none", + ssl_min_version=ssl.TLSVersion.TLSv1_2, + ssl_ciphers=ssl_ciphers, + ) as rc: + with pytest.raises(RedisClusterException) as e: + assert await rc.ping() + assert "Redis Cluster cannot be connected" in str(e.value) + + async def test_validating_self_signed_certificate( + self, create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]] + ) -> None: + async with create_client( + ssl=True, + ssl_ca_certs=self.ca_cert, + ssl_cert_reqs="required", + ssl_certfile=self.client_cert, + ssl_keyfile=self.client_key, + ) as rc: + assert await rc.ping() + + async def test_validating_self_signed_string_certificate( + self, create_client: Callable[..., AbstractAsyncContextManager[RedisCluster]] + ) -> None: + with open(self.ca_cert) as f: + cert_data = f.read() + + async with create_client( + ssl=True, + ssl_ca_data=cert_data, + ssl_cert_reqs="required", + ssl_certfile=self.client_cert, + ssl_keyfile=self.client_key, + ) as rc: + assert await rc.ping() diff --git a/tests/test_anyio/test_commands.py b/tests/test_anyio/test_commands.py new file mode 100644 index 0000000000..578a84c3c2 --- /dev/null +++ b/tests/test_anyio/test_commands.py @@ -0,0 +1,3593 @@ +""" +Tests async overrides of commands from their mixins +""" + +import binascii +import datetime +import re +from string import ascii_letters + +import anyio +import pytest + +import redis +from redis import exceptions +from redis._parsers.helpers import ( + _RedisCallbacks, + _RedisCallbacksRESP2, + _RedisCallbacksRESP3, + parse_info, +) +from redis.client import EMPTY_RESPONSE, NEVER_DECODE +from redis.commands.json.path import Path +from redis.commands.search.field import TextField +from redis.commands.search.query import Query +from tests.conftest import ( + assert_resp_response, + assert_resp_response_in, + is_resp2_connection, + skip_if_server_version_gte, + skip_if_server_version_lt, + skip_unless_arch_bits, +) + +pytestmark = pytest.mark.anyio + +REDIS_6_VERSION = "5.9.0" + + +@pytest.fixture +async def r_teardown(r: redis.Redis): + """ + A special fixture which removes the provided names from the database after use + """ + usernames = [] + + def factory(username): + usernames.append(username) + return r + + yield factory + try: + client_info = await r.client_info() + except exceptions.NoPermissionError: + client_info = {} + if "default" != client_info.get("user", ""): + await r.auth("", "default") + for username in usernames: + await r.acl_deluser(username) + + +@pytest.fixture +async def slowlog(r: redis.Redis): + current_config = await r.config_get() + old_slower_than_value = current_config["slowlog-log-slower-than"] + old_max_legnth_value = current_config["slowlog-max-len"] + + await r.config_set("slowlog-log-slower-than", 0) + await r.config_set("slowlog-max-len", 128) + + yield + + await r.config_set("slowlog-log-slower-than", old_slower_than_value) + await r.config_set("slowlog-max-len", old_max_legnth_value) + + +async def redis_server_time(client: redis.Redis): + seconds, milliseconds = await client.time() + timestamp = float(f"{seconds}.{milliseconds}") + return datetime.datetime.fromtimestamp(timestamp) + + +async def get_stream_message(client: redis.Redis, stream: str, message_id: str): + """Fetch a stream message and format it as a (message_id, fields) pair""" + response = await client.xrange(stream, min=message_id, max=message_id) + assert len(response) == 1 + return response[0] + + +# RESPONSE CALLBACKS +@pytest.mark.onlynoncluster +class TestResponseCallbacks: + """Tests for the response callback system""" + + async def test_response_callbacks(self, r: redis.Redis): + callbacks = _RedisCallbacks + if is_resp2_connection(r): + callbacks.update(_RedisCallbacksRESP2) + else: + callbacks.update(_RedisCallbacksRESP3) + assert r.response_callbacks == callbacks + assert id(r.response_callbacks) != id(_RedisCallbacks) + r.set_response_callback("GET", lambda x: "static") + await r.set("a", "foo") + assert await r.get("a") == "static" + + async def test_case_insensitive_command_names(self, r: redis.Redis): + assert r.response_callbacks["ping"] == r.response_callbacks["PING"] + + +class TestRedisCommands: + async def test_command_on_invalid_key_type(self, r: redis.Redis): + await r.lpush("a", "1") + with pytest.raises(redis.ResponseError): + await r.get("a") + + # SERVER INFORMATION + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_cat_no_category(self, r: redis.Redis): + categories = await r.acl_cat() + assert isinstance(categories, list) + assert "read" in categories or b"read" in categories + + @pytest.mark.redismod + @skip_if_server_version_lt("7.9.0") + async def test_acl_cat_contain_modules_no_category(self, r: redis.Redis): + modules_list = [ + "search", + "bloom", + "json", + "cuckoo", + "timeseries", + "cms", + "topk", + "tdigest", + ] + categories = await r.acl_cat() + assert isinstance(categories, list) + for module_cat in modules_list: + assert module_cat in categories or module_cat.encode() in categories + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_cat_with_category(self, r: redis.Redis): + commands = await r.acl_cat("read") + assert isinstance(commands, list) + assert "get" in commands or b"get" in commands + + @pytest.mark.redismod + @skip_if_server_version_lt("7.9.0") + async def test_acl_modules_cat_with_category(self, r: redis.Redis): + search_commands = await r.acl_cat("search") + assert isinstance(search_commands, list) + assert "FT.SEARCH" in search_commands or b"FT.SEARCH" in search_commands + + bloom_commands = await r.acl_cat("bloom") + assert isinstance(bloom_commands, list) + assert "bf.add" in bloom_commands or b"bf.add" in bloom_commands + + json_commands = await r.acl_cat("json") + assert isinstance(json_commands, list) + assert "json.get" in json_commands or b"json.get" in json_commands + + cuckoo_commands = await r.acl_cat("cuckoo") + assert isinstance(cuckoo_commands, list) + assert "cf.insert" in cuckoo_commands or b"cf.insert" in cuckoo_commands + + cms_commands = await r.acl_cat("cms") + assert isinstance(cms_commands, list) + assert "cms.query" in cms_commands or b"cms.query" in cms_commands + + topk_commands = await r.acl_cat("topk") + assert isinstance(topk_commands, list) + assert "topk.list" in topk_commands or b"topk.list" in topk_commands + + tdigest_commands = await r.acl_cat("tdigest") + assert isinstance(tdigest_commands, list) + assert "tdigest.rank" in tdigest_commands or b"tdigest.rank" in tdigest_commands + + timeseries_commands = await r.acl_cat("timeseries") + assert isinstance(timeseries_commands, list) + assert "ts.range" in timeseries_commands or b"ts.range" in timeseries_commands + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_deluser(self, r_teardown): + username = "redis-py-user" + r = r_teardown(username) + + assert await r.acl_deluser(username) == 0 + assert await r.acl_setuser(username, enabled=False, reset=True) + assert await r.acl_deluser(username) == 1 + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_genpass(self, r: redis.Redis): + password = await r.acl_genpass() + assert isinstance(password, (str, bytes)) + + @skip_if_server_version_lt("7.0.0") + async def test_acl_getuser_setuser(self, r_teardown): + username = "redis-py-user" + r = r_teardown(username) + # test enabled=False + assert await r.acl_setuser(username, enabled=False, reset=True) + acl = await r.acl_getuser(username) + assert acl["categories"] == ["-@all"] + assert acl["commands"] == [] + assert acl["keys"] == [] + assert acl["passwords"] == [] + assert "off" in acl["flags"] + assert acl["enabled"] is False + + # test nopass=True + assert await r.acl_setuser(username, enabled=True, reset=True, nopass=True) + acl = await r.acl_getuser(username) + assert acl["categories"] == ["-@all"] + assert acl["commands"] == [] + assert acl["keys"] == [] + assert acl["passwords"] == [] + assert "on" in acl["flags"] + assert "nopass" in acl["flags"] + assert acl["enabled"] is True + + # test all args + assert await r.acl_setuser( + username, + enabled=True, + reset=True, + passwords=["+pass1", "+pass2"], + categories=["+set", "+@hash", "-geo"], + commands=["+get", "+mget", "-hset"], + keys=["cache:*", "objects:*"], + ) + acl = await r.acl_getuser(username) + assert set(acl["categories"]) == {"-@all", "+@set", "+@hash", "-@geo"} + assert set(acl["commands"]) == {"+get", "+mget", "-hset"} + assert acl["enabled"] is True + assert "on" in acl["flags"] + assert set(acl["keys"]) == {"~cache:*", "~objects:*"} + assert len(acl["passwords"]) == 2 + + # test reset=False keeps existing ACL and applies new ACL on top + assert await r.acl_setuser( + username, + enabled=True, + reset=True, + passwords=["+pass1"], + categories=["+@set"], + commands=["+get"], + keys=["cache:*"], + ) + assert await r.acl_setuser( + username, + enabled=True, + passwords=["+pass2"], + categories=["+@hash"], + commands=["+mget"], + keys=["objects:*"], + ) + acl = await r.acl_getuser(username) + assert set(acl["commands"]) == {"+get", "+mget"} + assert acl["enabled"] is True + assert "on" in acl["flags"] + assert set(acl["keys"]) == {"~cache:*", "~objects:*"} + assert len(acl["passwords"]) == 2 + + # test removal of passwords + assert await r.acl_setuser( + username, enabled=True, reset=True, passwords=["+pass1", "+pass2"] + ) + assert len((await r.acl_getuser(username))["passwords"]) == 2 + assert await r.acl_setuser(username, enabled=True, passwords=["-pass2"]) + assert len((await r.acl_getuser(username))["passwords"]) == 1 + + # Resets and tests that hashed passwords are set properly. + hashed_password = ( + "5e884898da28047151d0e56f8dc6292773603d0d6aabbdd62a11ef721d1542d8" + ) + assert await r.acl_setuser( + username, enabled=True, reset=True, hashed_passwords=["+" + hashed_password] + ) + acl = await r.acl_getuser(username) + assert acl["passwords"] == [hashed_password] + + # test removal of hashed passwords + assert await r.acl_setuser( + username, + enabled=True, + reset=True, + hashed_passwords=["+" + hashed_password], + passwords=["+pass1"], + ) + assert len((await r.acl_getuser(username))["passwords"]) == 2 + assert await r.acl_setuser( + username, enabled=True, hashed_passwords=["-" + hashed_password] + ) + assert len((await r.acl_getuser(username))["passwords"]) == 1 + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_list(self, r_teardown): + username = "redis-py-user" + r = r_teardown(username) + start = await r.acl_list() + assert await r.acl_setuser(username, enabled=False, reset=True) + users = await r.acl_list() + assert len(users) == len(start) + 1 + + @skip_if_server_version_lt(REDIS_6_VERSION) + @pytest.mark.onlynoncluster + async def test_acl_log(self, r_teardown, create_redis): + username = "redis-py-user" + r = r_teardown(username) + await r.acl_setuser( + username, + enabled=True, + reset=True, + commands=["+get", "+set", "+select"], + keys=["cache:*"], + nopass=True, + ) + await r.acl_log_reset() + + async with create_redis(username=username) as user_client: + # Valid operation and key + assert await user_client.set("cache:0", 1) + assert await user_client.get("cache:0") == b"1" + + # Invalid key + with pytest.raises(exceptions.NoPermissionError): + await user_client.get("violated_cache:0") + + # Invalid operation + with pytest.raises(exceptions.NoPermissionError): + await user_client.hset("cache:0", "hkey", "hval") + + assert isinstance(await r.acl_log(), list) + assert len(await r.acl_log()) == 3 + assert len(await r.acl_log(count=1)) == 1 + assert isinstance((await r.acl_log())[0], dict) + expected = (await r.acl_log(count=1))[0] + assert_resp_response_in(r, "client-info", expected, expected.keys()) + assert await r.acl_log_reset() + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_setuser_categories_without_prefix_fails(self, r_teardown): + username = "redis-py-user" + r = r_teardown(username) + + with pytest.raises(exceptions.DataError): + await r.acl_setuser(username, categories=["list"]) + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_setuser_commands_without_prefix_fails(self, r_teardown): + username = "redis-py-user" + r = r_teardown(username) + + with pytest.raises(exceptions.DataError): + await r.acl_setuser(username, commands=["get"]) + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_setuser_add_passwords_and_nopass_fails(self, r_teardown): + username = "redis-py-user" + r = r_teardown(username) + + with pytest.raises(exceptions.DataError): + await r.acl_setuser(username, passwords="+mypass", nopass=True) + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_users(self, r: redis.Redis): + users = await r.acl_users() + assert isinstance(users, list) + assert len(users) > 0 + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_acl_whoami(self, r: redis.Redis): + username = await r.acl_whoami() + assert isinstance(username, (str, bytes)) + + @pytest.mark.redismod + @skip_if_server_version_lt("7.9.0") + async def test_acl_modules_commands(self, r_teardown): + username = "redis-py-user" + password = "pass-for-test-user" + + r = r_teardown(username) + await r.flushdb() + + await r.ft().create_index((TextField("txt"),)) + await r.hset("doc1", mapping={"txt": "foo baz"}) + await r.hset("doc2", mapping={"txt": "foo bar"}) + + await r.acl_setuser( + username, + enabled=True, + reset=True, + passwords=[f"+{password}"], + categories=["-all"], + commands=[ + "+FT.SEARCH", + "-FT.DROPINDEX", + "+json.set", + "+json.get", + "-json.clear", + "+bf.reserve", + "-bf.info", + "+cf.reserve", + "+cms.initbydim", + "+topk.reserve", + "+tdigest.create", + "+ts.create", + "-ts.info", + ], + keys=["*"], + ) + + await r.auth(password, username) + + assert await r.ft().search(Query("foo ~bar")) + with pytest.raises(exceptions.NoPermissionError): + await r.ft().dropindex() + + await r.json().set("foo", Path.root_path(), "bar") + assert await r.json().get("foo") == "bar" + with pytest.raises(exceptions.NoPermissionError): + await r.json().clear("foo") + + assert await r.bf().create("bloom", 0.01, 1000) + assert await r.cf().create("cuckoo", 1000) + assert await r.cms().initbydim("cmsDim", 100, 5) + assert await r.topk().reserve("topk", 5, 100, 5, 0.9) + assert await r.tdigest().create("to-tDigest", 10) + with pytest.raises(exceptions.NoPermissionError): + await r.bf().info("bloom") + + assert await r.ts().create(1, labels={"Redis": "Labs"}) + with pytest.raises(exceptions.NoPermissionError): + await r.ts().info(1) + + @pytest.mark.redismod + @skip_if_server_version_lt("7.9.0") + async def test_acl_modules_category_commands(self, r_teardown): + username = "redis-py-user" + password = "pass-for-test-user" + + r = r_teardown(username) + await r.flushdb() + + # validate modules categories acl config + await r.acl_setuser( + username, + enabled=True, + reset=True, + passwords=[f"+{password}"], + categories=[ + "-all", + "+@search", + "+@json", + "+@bloom", + "+@cuckoo", + "+@topk", + "+@cms", + "+@timeseries", + "+@tdigest", + ], + keys=["*"], + ) + await r.ft().create_index((TextField("txt"),)) + await r.hset("doc1", mapping={"txt": "foo baz"}) + await r.hset("doc2", mapping={"txt": "foo bar"}) + + await r.auth(password, username) + + assert await r.ft().search(Query("foo ~bar")) + assert await r.ft().dropindex() + + assert await r.json().set("foo", Path.root_path(), "bar") + assert await r.json().get("foo") == "bar" + + assert await r.bf().create("bloom", 0.01, 1000) + assert await r.bf().info("bloom") + assert await r.cf().create("cuckoo", 1000) + assert await r.cms().initbydim("cmsDim", 100, 5) + assert await r.topk().reserve("topk", 5, 100, 5, 0.9) + assert await r.tdigest().create("to-tDigest", 10) + + assert await r.ts().create(1, labels={"Redis": "Labs"}) + assert await r.ts().info(1) + + @pytest.mark.onlynoncluster + async def test_client_list(self, r: redis.Redis): + clients = await r.client_list() + assert isinstance(clients[0], dict) + assert "addr" in clients[0] + + @skip_if_server_version_lt("5.0.0") + async def test_client_list_type(self, r: redis.Redis): + with pytest.raises(exceptions.RedisError): + await r.client_list(_type="not a client type") + for client_type in ["normal", "master", "replica", "pubsub"]: + clients = await r.client_list(_type=client_type) + assert isinstance(clients, list) + + @skip_if_server_version_lt("5.0.0") + @pytest.mark.onlynoncluster + async def test_client_id(self, r: redis.Redis): + assert await r.client_id() > 0 + + @skip_if_server_version_lt("5.0.0") + @pytest.mark.onlynoncluster + async def test_client_unblock(self, r: redis.Redis): + myid = await r.client_id() + assert not await r.client_unblock(myid) + assert not await r.client_unblock(myid, error=True) + assert not await r.client_unblock(myid, error=False) + + @skip_if_server_version_lt("2.6.9") + @pytest.mark.onlynoncluster + async def test_client_getname(self, r: redis.Redis): + assert await r.client_getname() is None + + @skip_if_server_version_lt("2.6.9") + @pytest.mark.onlynoncluster + async def test_client_setname(self, r: redis.Redis): + assert await r.client_setname("redis_py_test") + assert_resp_response( + r, await r.client_getname(), "redis_py_test", b"redis_py_test" + ) + + @skip_if_server_version_lt("7.2.0") + async def test_client_setinfo(self, r: redis.Redis): + await r.ping() + info = await r.client_info() + assert info["lib-name"] == "redis-py" + assert info["lib-ver"] == redis.__version__ + assert await r.client_setinfo("lib-name", "test") + assert await r.client_setinfo("lib-ver", "123") + info = await r.client_info() + assert info["lib-name"] == "test" + assert info["lib-ver"] == "123" + r2 = redis.anyio.Redis(lib_name="test2", lib_version="1234") + info = await r2.client_info() + assert info["lib-name"] == "test2" + assert info["lib-ver"] == "1234" + await r2.aclose() + r3 = redis.anyio.Redis(lib_name=None, lib_version=None) + info = await r3.client_info() + assert info["lib-name"] == "" + assert info["lib-ver"] == "" + await r3.aclose() + + @skip_if_server_version_lt("2.6.9") + @pytest.mark.onlynoncluster + async def test_client_kill(self, r: redis.Redis, r2): + await r.client_setname("redis-py-c1") + await r2.client_setname("redis-py-c2") + clients = [ + client + for client in await r.client_list() + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 2 + + clients_by_name = {client.get("name"): client for client in clients} + + client_addr = clients_by_name["redis-py-c2"].get("addr") + assert await r.client_kill(client_addr) is True + + clients = [ + client + for client in await r.client_list() + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 1 + assert clients[0].get("name") == "redis-py-c1" + + @skip_if_server_version_lt("2.8.12") + async def test_client_kill_filter_invalid_params(self, r: redis.Redis): + # empty + with pytest.raises(exceptions.DataError): + await r.client_kill_filter() + + # invalid skipme + with pytest.raises(exceptions.DataError): + await r.client_kill_filter(skipme="yeah") # type: ignore + + # invalid type + with pytest.raises(exceptions.DataError): + await r.client_kill_filter(_type="caster") # type: ignore + + @skip_if_server_version_lt("2.8.12") + @pytest.mark.onlynoncluster + async def test_client_kill_filter_by_id(self, r: redis.Redis, r2): + await r.client_setname("redis-py-c1") + await r2.client_setname("redis-py-c2") + clients = [ + client + for client in await r.client_list() + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 2 + + clients_by_name = {client.get("name"): client for client in clients} + + client_2_id = clients_by_name["redis-py-c2"].get("id") + resp = await r.client_kill_filter(_id=client_2_id) + assert resp == 1 + + clients = [ + client + for client in await r.client_list() + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 1 + assert clients[0].get("name") == "redis-py-c1" + + @skip_if_server_version_lt("2.8.12") + @pytest.mark.onlynoncluster + async def test_client_kill_filter_by_addr(self, r: redis.Redis, r2): + await r.client_setname("redis-py-c1") + await r2.client_setname("redis-py-c2") + clients = [ + client + for client in await r.client_list() + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 2 + + clients_by_name = {client.get("name"): client for client in clients} + + client_2_addr = clients_by_name["redis-py-c2"].get("addr") + resp = await r.client_kill_filter(addr=client_2_addr) + assert resp == 1 + + clients = [ + client + for client in await r.client_list() + if client.get("name") in ["redis-py-c1", "redis-py-c2"] + ] + assert len(clients) == 1 + assert clients[0].get("name") == "redis-py-c1" + + @skip_if_server_version_lt("2.6.9") + async def test_client_list_after_client_setname(self, r: redis.Redis): + await r.client_setname("redis_py_test") + clients = await r.client_list() + # we don't know which client ours will be + assert "redis_py_test" in [c["name"] for c in clients] + + @skip_if_server_version_lt("2.9.50") + @pytest.mark.onlynoncluster + async def test_client_pause(self, r: redis.Redis): + assert await r.client_pause(1) + assert await r.client_pause(timeout=1) + with pytest.raises(exceptions.RedisError): + await r.client_pause(timeout="not an integer") + + @skip_if_server_version_lt("7.2.0") + @pytest.mark.onlynoncluster + async def test_client_no_touch(self, r: redis.Redis): + assert await r.client_no_touch("ON") == b"OK" + assert await r.client_no_touch("OFF") == b"OK" + with pytest.raises(TypeError): + await r.client_no_touch() + + async def test_config_get(self, r: redis.Redis): + data = await r.config_get() + assert "maxmemory" in data + assert data["maxmemory"].isdigit() + + @pytest.mark.onlynoncluster + async def test_config_resetstat(self, r: redis.Redis): + await r.ping() + prior_commands_processed = int((await r.info())["total_commands_processed"]) + assert prior_commands_processed >= 1 + await r.config_resetstat() + reset_commands_processed = int((await r.info())["total_commands_processed"]) + assert reset_commands_processed < prior_commands_processed + + async def test_config_set(self, r: redis.Redis): + await r.config_set("timeout", 70) + assert (await r.config_get())["timeout"] == "70" + assert await r.config_set("timeout", 0) + assert (await r.config_get())["timeout"] == "0" + + @pytest.mark.redismod + @skip_if_server_version_lt("7.9.0") + async def test_config_get_for_modules(self, r: redis.Redis): + search_module_configs = await r.config_get("search-*") + assert "search-timeout" in search_module_configs + + ts_module_configs = await r.config_get("ts-*") + assert "ts-retention-policy" in ts_module_configs + + bf_module_configs = await r.config_get("bf-*") + assert "bf-error-rate" in bf_module_configs + + cf_module_configs = await r.config_get("cf-*") + assert "cf-initial-size" in cf_module_configs + + @pytest.mark.redismod + @skip_if_server_version_lt("7.9.0") + async def test_config_set_for_search_module(self, r: redis.Redis): + config = await r.config_get("*") + initial_default_search_dialect = config["search-default-dialect"] + try: + default_dialect_new = "3" + assert await r.config_set("search-default-dialect", default_dialect_new) + assert (await r.config_get("*"))[ + "search-default-dialect" + ] == default_dialect_new + assert ( + ((await r.ft().config_get("*"))[b"DEFAULT_DIALECT"]).decode() + == default_dialect_new + ) + except AssertionError as ex: + raise ex + finally: + assert await r.config_set( + "search-default-dialect", initial_default_search_dialect + ) + with pytest.raises(exceptions.ResponseError): + await r.config_set("search-max-doctablesize", 2000000) + + @pytest.mark.onlynoncluster + async def test_dbsize(self, r: redis.Redis): + await r.set("a", "foo") + await r.set("b", "bar") + assert await r.dbsize() == 2 + + @pytest.mark.onlynoncluster + async def test_echo(self, r: redis.Redis): + assert await r.echo("foo bar") == b"foo bar" + + @pytest.mark.onlynoncluster + async def test_info(self, r: redis.Redis): + await r.set("a", "foo") + await r.set("b", "bar") + info = await r.info() + assert isinstance(info, dict) + assert "arch_bits" in info.keys() + assert "redis_version" in info.keys() + + @pytest.mark.onlynoncluster + async def test_lastsave(self, r: redis.Redis): + assert isinstance(await r.lastsave(), datetime.datetime) + + async def test_object(self, r: redis.Redis): + await r.set("a", "foo") + assert isinstance(await r.object("refcount", "a"), int) + assert isinstance(await r.object("idletime", "a"), int) + assert await r.object("encoding", "a") in (b"raw", b"embstr") + assert await r.object("idletime", "invalid-key") is None + + async def test_ping(self, r: redis.Redis): + assert await r.ping() + + @pytest.mark.onlynoncluster + async def test_slowlog_get(self, r: redis.Redis, slowlog): + assert await r.slowlog_reset() + unicode_string = chr(3456) + "abcd" + chr(3421) + await r.get(unicode_string) + slowlog = await r.slowlog_get() + assert isinstance(slowlog, list) + commands = [log["command"] for log in slowlog] + + get_command = b" ".join((b"GET", unicode_string.encode("utf-8"))) + assert get_command in commands + assert b"SLOWLOG RESET" in commands + # the order should be ['GET ', 'SLOWLOG RESET'], + # but if other clients are executing commands at the same time, there + # could be commands, before, between, or after, so just check that + # the two we care about are in the appropriate order. + assert commands.index(get_command) < commands.index(b"SLOWLOG RESET") + + # make sure other attributes are typed correctly + assert isinstance(slowlog[0]["start_time"], int) + assert isinstance(slowlog[0]["duration"], int) + + @pytest.mark.onlynoncluster + async def test_slowlog_get_limit(self, r: redis.Redis, slowlog): + assert await r.slowlog_reset() + await r.get("foo") + slowlog = await r.slowlog_get(1) + assert isinstance(slowlog, list) + # only one command, based on the number we passed to slowlog_get() + assert len(slowlog) == 1 + + @pytest.mark.onlynoncluster + async def test_slowlog_length(self, r: redis.Redis, slowlog): + await r.get("foo") + assert isinstance(await r.slowlog_len(), int) + + @skip_if_server_version_lt("2.6.0") + async def test_time(self, r: redis.Redis): + t = await r.time() + assert len(t) == 2 + assert isinstance(t[0], int) + assert isinstance(t[1], int) + + async def test_never_decode_option(self, r: redis.Redis): + opts = {NEVER_DECODE: []} + await r.delete("a") + assert await r.execute_command("EXISTS", "a", **opts) == 0 + + async def test_empty_response_option(self, r: redis.Redis): + opts = {EMPTY_RESPONSE: []} + await r.delete("a") + assert await r.execute_command("EXISTS", "a", **opts) == 0 + + # BASIC KEY COMMANDS + async def test_append(self, r: redis.Redis): + assert await r.append("a", "a1") == 2 + assert await r.get("a") == b"a1" + assert await r.append("a", "a2") == 4 + assert await r.get("a") == b"a1a2" + + @skip_if_server_version_lt("2.6.0") + async def test_bitcount(self, r: redis.Redis): + await r.setbit("a", 5, True) + assert await r.bitcount("a") == 1 + await r.setbit("a", 6, True) + assert await r.bitcount("a") == 2 + await r.setbit("a", 5, False) + assert await r.bitcount("a") == 1 + await r.setbit("a", 9, True) + await r.setbit("a", 17, True) + await r.setbit("a", 25, True) + await r.setbit("a", 33, True) + assert await r.bitcount("a") == 5 + assert await r.bitcount("a", 0, -1) == 5 + assert await r.bitcount("a", 2, 3) == 2 + assert await r.bitcount("a", 2, -1) == 3 + assert await r.bitcount("a", -2, -1) == 2 + assert await r.bitcount("a", 1, 1) == 1 + + @skip_if_server_version_lt("2.6.0") + @pytest.mark.onlynoncluster + async def test_bitop_not_empty_string(self, r: redis.Redis): + await r.set("a", "") + await r.bitop("not", "r", "a") + assert await r.get("r") is None + + @skip_if_server_version_lt("2.6.0") + @pytest.mark.onlynoncluster + async def test_bitop_not(self, r: redis.Redis): + test_str = b"\xaa\x00\xff\x55" + correct = ~0xAA00FF55 & 0xFFFFFFFF + await r.set("a", test_str) + await r.bitop("not", "r", "a") + assert int(binascii.hexlify(await r.get("r")), 16) == correct + + @skip_if_server_version_lt("2.6.0") + @pytest.mark.onlynoncluster + async def test_bitop_not_in_place(self, r: redis.Redis): + test_str = b"\xaa\x00\xff\x55" + correct = ~0xAA00FF55 & 0xFFFFFFFF + await r.set("a", test_str) + await r.bitop("not", "a", "a") + assert int(binascii.hexlify(await r.get("a")), 16) == correct + + @skip_if_server_version_lt("2.6.0") + @pytest.mark.onlynoncluster + async def test_bitop_single_string(self, r: redis.Redis): + test_str = b"\x01\x02\xff" + await r.set("a", test_str) + await r.bitop("and", "res1", "a") + await r.bitop("or", "res2", "a") + await r.bitop("xor", "res3", "a") + assert await r.get("res1") == test_str + assert await r.get("res2") == test_str + assert await r.get("res3") == test_str + + @skip_if_server_version_lt("2.6.0") + @pytest.mark.onlynoncluster + async def test_bitop_string_operands(self, r: redis.Redis): + await r.set("a", b"\x01\x02\xff\xff") + await r.set("b", b"\x01\x02\xff") + await r.bitop("and", "res1", "a", "b") + await r.bitop("or", "res2", "a", "b") + await r.bitop("xor", "res3", "a", "b") + assert int(binascii.hexlify(await r.get("res1")), 16) == 0x0102FF00 + assert int(binascii.hexlify(await r.get("res2")), 16) == 0x0102FFFF + assert int(binascii.hexlify(await r.get("res3")), 16) == 0x000000FF + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("2.8.7") + async def test_bitpos(self, r: redis.Redis): + key = "key:bitpos" + await r.set(key, b"\xff\xf0\x00") + assert await r.bitpos(key, 0) == 12 + assert await r.bitpos(key, 0, 2, -1) == 16 + assert await r.bitpos(key, 0, -2, -1) == 12 + await r.set(key, b"\x00\xff\xf0") + assert await r.bitpos(key, 1, 0) == 8 + assert await r.bitpos(key, 1, 1) == 8 + await r.set(key, b"\x00\x00\x00") + assert await r.bitpos(key, 1) == -1 + + @skip_if_server_version_lt("2.8.7") + async def test_bitpos_wrong_arguments(self, r: redis.Redis): + key = "key:bitpos:wrong:args" + await r.set(key, b"\xff\xf0\x00") + with pytest.raises(exceptions.RedisError): + await r.bitpos(key, 0, end=1) == 12 + with pytest.raises(exceptions.RedisError): + await r.bitpos(key, 7) == 12 + + async def test_decr(self, r: redis.Redis): + assert await r.decr("a") == -1 + assert await r.get("a") == b"-1" + assert await r.decr("a") == -2 + assert await r.get("a") == b"-2" + assert await r.decr("a", amount=5) == -7 + assert await r.get("a") == b"-7" + + async def test_decrby(self, r: redis.Redis): + assert await r.decrby("a", amount=2) == -2 + assert await r.decrby("a", amount=3) == -5 + assert await r.get("a") == b"-5" + + async def test_delete(self, r: redis.Redis): + assert await r.delete("a") == 0 + await r.set("a", "foo") + assert await r.delete("a") == 1 + + async def test_delete_with_multiple_keys(self, r: redis.Redis): + await r.set("a", "foo") + await r.set("b", "bar") + assert await r.delete("a", "b") == 2 + assert await r.get("a") is None + assert await r.get("b") is None + + async def test_delitem(self, r: redis.Redis): + await r.set("a", "foo") + await r.delete("a") + assert await r.get("a") is None + + @skip_if_server_version_lt("4.0.0") + async def test_unlink(self, r: redis.Redis): + assert await r.unlink("a") == 0 + await r.set("a", "foo") + assert await r.unlink("a") == 1 + assert await r.get("a") is None + + @skip_if_server_version_lt("4.0.0") + async def test_unlink_with_multiple_keys(self, r: redis.Redis): + await r.set("a", "foo") + await r.set("b", "bar") + assert await r.unlink("a", "b") == 2 + assert await r.get("a") is None + assert await r.get("b") is None + + @skip_if_server_version_lt("2.6.0") + async def test_dump_and_restore(self, r: redis.Redis): + await r.set("a", "foo") + dumped = await r.dump("a") + await r.delete("a") + await r.restore("a", 0, dumped) + assert await r.get("a") == b"foo" + + @skip_if_server_version_lt("3.0.0") + async def test_dump_and_restore_and_replace(self, r: redis.Redis): + await r.set("a", "bar") + dumped = await r.dump("a") + with pytest.raises(redis.ResponseError): + await r.restore("a", 0, dumped) + + await r.restore("a", 0, dumped, replace=True) + assert await r.get("a") == b"bar" + + @skip_if_server_version_lt("5.0.0") + async def test_dump_and_restore_absttl(self, r: redis.Redis): + await r.set("a", "foo") + dumped = await r.dump("a") + await r.delete("a") + ttl = int( + (await redis_server_time(r) + datetime.timedelta(minutes=1)).timestamp() + * 1000 + ) + await r.restore("a", ttl, dumped, absttl=True) + assert await r.get("a") == b"foo" + assert 0 < await r.ttl("a") <= 61 + + async def test_exists(self, r: redis.Redis): + assert await r.exists("a") == 0 + await r.set("a", "foo") + await r.set("b", "bar") + assert await r.exists("a") == 1 + assert await r.exists("a", "b") == 2 + + async def test_exists_contains(self, r: redis.Redis): + assert not await r.exists("a") + await r.set("a", "foo") + assert await r.exists("a") + + async def test_expire(self, r: redis.Redis): + assert not await r.expire("a", 10) + await r.set("a", "foo") + assert await r.expire("a", 10) + assert 0 < await r.ttl("a") <= 10 + assert await r.persist("a") + assert await r.ttl("a") == -1 + + async def test_expireat_datetime(self, r: redis.Redis): + expire_at = await redis_server_time(r) + datetime.timedelta(minutes=1) + await r.set("a", "foo") + assert await r.expireat("a", expire_at) + assert 0 < await r.ttl("a") <= 61 + + async def test_expireat_no_key(self, r: redis.Redis): + expire_at = await redis_server_time(r) + datetime.timedelta(minutes=1) + assert not await r.expireat("a", expire_at) + + async def test_expireat_unixtime(self, r: redis.Redis): + expire_at = await redis_server_time(r) + datetime.timedelta(minutes=1) + await r.set("a", "foo") + expire_at_seconds = int(expire_at.timestamp()) + assert await r.expireat("a", expire_at_seconds) + assert 0 < await r.ttl("a") <= 61 + + async def test_get_and_set(self, r: redis.Redis): + # get and set can't be tested independently of each other + assert await r.get("a") is None + byte_string = b"value" + integer = 5 + unicode_string = chr(3456) + "abcd" + chr(3421) + assert await r.set("byte_string", byte_string) + assert await r.set("integer", 5) + assert await r.set("unicode_string", unicode_string) + assert await r.get("byte_string") == byte_string + assert await r.get("integer") == str(integer).encode() + assert (await r.get("unicode_string")).decode("utf-8") == unicode_string + + async def test_get_set_bit(self, r: redis.Redis): + # no value + assert not await r.getbit("a", 5) + # set bit 5 + assert not await r.setbit("a", 5, True) + assert await r.getbit("a", 5) + # unset bit 4 + assert not await r.setbit("a", 4, False) + assert not await r.getbit("a", 4) + # set bit 4 + assert not await r.setbit("a", 4, True) + assert await r.getbit("a", 4) + # set bit 5 again + assert await r.setbit("a", 5, True) + assert await r.getbit("a", 5) + + async def test_getrange(self, r: redis.Redis): + await r.set("a", "foo") + assert await r.getrange("a", 0, 0) == b"f" + assert await r.getrange("a", 0, 2) == b"foo" + assert await r.getrange("a", 3, 4) == b"" + + async def test_getset(self, r: redis.Redis): + assert await r.getset("a", "foo") is None + assert await r.getset("a", "bar") == b"foo" + assert await r.get("a") == b"bar" + + async def test_incr(self, r: redis.Redis): + assert await r.incr("a") == 1 + assert await r.get("a") == b"1" + assert await r.incr("a") == 2 + assert await r.get("a") == b"2" + assert await r.incr("a", amount=5) == 7 + assert await r.get("a") == b"7" + + async def test_incrby(self, r: redis.Redis): + assert await r.incrby("a") == 1 + assert await r.incrby("a", 4) == 5 + assert await r.get("a") == b"5" + + @skip_if_server_version_lt("2.6.0") + async def test_incrbyfloat(self, r: redis.Redis): + assert await r.incrbyfloat("a") == 1.0 + assert await r.get("a") == b"1" + assert await r.incrbyfloat("a", 1.1) == 2.1 + assert float(await r.get("a")) == float(2.1) + + @pytest.mark.onlynoncluster + async def test_keys(self, r: redis.Redis): + assert await r.keys() == [] + keys_with_underscores = {b"test_a", b"test_b"} + keys = keys_with_underscores.union({b"testc"}) + for key in keys: + await r.set(key, 1) + assert set(await r.keys(pattern="test_*")) == keys_with_underscores + assert set(await r.keys(pattern="test*")) == keys + + @pytest.mark.onlynoncluster + async def test_mget(self, r: redis.Redis): + assert await r.mget([]) == [] + assert await r.mget(["a", "b"]) == [None, None] + await r.set("a", "1") + await r.set("b", "2") + await r.set("c", "3") + assert await r.mget("a", "other", "b", "c") == [b"1", None, b"2", b"3"] + + @pytest.mark.onlynoncluster + async def test_mset(self, r: redis.Redis): + d = {"a": b"1", "b": b"2", "c": b"3"} + assert await r.mset(d) + for k, v in d.items(): + assert await r.get(k) == v + + @pytest.mark.onlynoncluster + async def test_msetnx(self, r: redis.Redis): + d = {"a": b"1", "b": b"2", "c": b"3"} + assert await r.msetnx(d) + d2 = {"a": b"x", "d": b"4"} + assert not await r.msetnx(d2) + for k, v in d.items(): + assert await r.get(k) == v + assert await r.get("d") is None + + @skip_if_server_version_lt("2.6.0") + async def test_pexpire(self, r: redis.Redis): + assert not await r.pexpire("a", 60000) + await r.set("a", "foo") + assert await r.pexpire("a", 60000) + assert 0 < await r.pttl("a") <= 60000 + assert await r.persist("a") + assert await r.pttl("a") == -1 + + @skip_if_server_version_lt("2.6.0") + async def test_pexpireat_datetime(self, r: redis.Redis): + expire_at = await redis_server_time(r) + datetime.timedelta(minutes=1) + await r.set("a", "foo") + assert await r.pexpireat("a", expire_at) + assert 0 < await r.pttl("a") <= 61000 + + @skip_if_server_version_lt("2.6.0") + async def test_pexpireat_no_key(self, r: redis.Redis): + expire_at = await redis_server_time(r) + datetime.timedelta(minutes=1) + assert not await r.pexpireat("a", expire_at) + + @skip_if_server_version_lt("2.6.0") + async def test_pexpireat_unixtime(self, r: redis.Redis): + expire_at = await redis_server_time(r) + datetime.timedelta(minutes=1) + await r.set("a", "foo") + expire_at_milliseconds = int(expire_at.timestamp() * 1000) + assert await r.pexpireat("a", expire_at_milliseconds) + assert 0 < await r.pttl("a") <= 61000 + + @skip_if_server_version_lt("2.6.0") + async def test_psetex(self, r: redis.Redis): + assert await r.psetex("a", 1000, "value") + assert await r.get("a") == b"value" + assert 0 < await r.pttl("a") <= 1000 + + @skip_if_server_version_lt("2.6.0") + async def test_psetex_timedelta(self, r: redis.Redis): + expire_at = datetime.timedelta(milliseconds=1000) + assert await r.psetex("a", expire_at, "value") + assert await r.get("a") == b"value" + assert 0 < await r.pttl("a") <= 1000 + + @skip_if_server_version_lt("2.6.0") + async def test_pttl(self, r: redis.Redis): + assert not await r.pexpire("a", 10000) + await r.set("a", "1") + assert await r.pexpire("a", 10000) + assert 0 < await r.pttl("a") <= 10000 + assert await r.persist("a") + assert await r.pttl("a") == -1 + + @skip_if_server_version_lt("2.8.0") + async def test_pttl_no_key(self, r: redis.Redis): + """PTTL on servers 2.8 and after return -2 when the key doesn't exist""" + assert await r.pttl("a") == -2 + + @skip_if_server_version_lt("6.2.0") + async def test_hrandfield(self, r): + assert await r.hrandfield("key") is None + await r.hset("key", mapping={"a": 1, "b": 2, "c": 3, "d": 4, "e": 5}) + assert await r.hrandfield("key") is not None + assert len(await r.hrandfield("key", 2)) == 2 + # with values + assert_resp_response(r, len(await r.hrandfield("key", 2, True)), 4, 2) + # without duplications + assert len(await r.hrandfield("key", 10)) == 5 + # with duplications + assert len(await r.hrandfield("key", -10)) == 10 + + @pytest.mark.onlynoncluster + async def test_randomkey(self, r: redis.Redis): + assert await r.randomkey() is None + for key in ("a", "b", "c"): + await r.set(key, 1) + assert await r.randomkey() in (b"a", b"b", b"c") + + @pytest.mark.onlynoncluster + async def test_rename(self, r: redis.Redis): + await r.set("a", "1") + assert await r.rename("a", "b") + assert await r.get("a") is None + assert await r.get("b") == b"1" + + @pytest.mark.onlynoncluster + async def test_renamenx(self, r: redis.Redis): + await r.set("a", "1") + await r.set("b", "2") + assert not await r.renamenx("a", "b") + assert await r.get("a") == b"1" + assert await r.get("b") == b"2" + + @skip_if_server_version_lt("2.6.0") + async def test_set_nx(self, r: redis.Redis): + assert await r.set("a", "1", nx=True) + assert not await r.set("a", "2", nx=True) + assert await r.get("a") == b"1" + + @skip_if_server_version_lt("2.6.0") + async def test_set_xx(self, r: redis.Redis): + assert not await r.set("a", "1", xx=True) + assert await r.get("a") is None + await r.set("a", "bar") + assert await r.set("a", "2", xx=True) + assert await r.get("a") == b"2" + + @skip_if_server_version_lt("2.6.0") + async def test_set_px(self, r: redis.Redis): + assert await r.set("a", "1", px=10000) + assert await r.get("a") == b"1" + assert 0 < await r.pttl("a") <= 10000 + assert 0 < await r.ttl("a") <= 10 + + @skip_if_server_version_lt("2.6.0") + async def test_set_px_timedelta(self, r: redis.Redis): + expire_at = datetime.timedelta(milliseconds=1000) + assert await r.set("a", "1", px=expire_at) + assert 0 < await r.pttl("a") <= 1000 + assert 0 < await r.ttl("a") <= 1 + + @skip_if_server_version_lt("2.6.0") + async def test_set_ex(self, r: redis.Redis): + assert await r.set("a", "1", ex=10) + assert 0 < await r.ttl("a") <= 10 + + @skip_if_server_version_lt("2.6.0") + async def test_set_ex_timedelta(self, r: redis.Redis): + expire_at = datetime.timedelta(seconds=60) + assert await r.set("a", "1", ex=expire_at) + assert 0 < await r.ttl("a") <= 60 + + @skip_if_server_version_lt("2.6.0") + async def test_set_multipleoptions(self, r: redis.Redis): + await r.set("a", "val") + assert await r.set("a", "1", xx=True, px=10000) + assert 0 < await r.ttl("a") <= 10 + + @skip_if_server_version_lt(REDIS_6_VERSION) + async def test_set_keepttl(self, r: redis.Redis): + await r.set("a", "val") + assert await r.set("a", "1", xx=True, px=10000) + assert 0 < await r.ttl("a") <= 10 + await r.set("a", "2", keepttl=True) + assert await r.get("a") == b"2" + assert 0 < await r.ttl("a") <= 10 + + async def test_setex(self, r: redis.Redis): + assert await r.setex("a", 60, "1") + assert await r.get("a") == b"1" + assert 0 < await r.ttl("a") <= 60 + + async def test_setnx(self, r: redis.Redis): + assert await r.setnx("a", "1") + assert await r.get("a") == b"1" + assert not await r.setnx("a", "2") + assert await r.get("a") == b"1" + + async def test_setrange(self, r: redis.Redis): + assert await r.setrange("a", 5, "foo") == 8 + assert await r.get("a") == b"\0\0\0\0\0foo" + await r.set("a", "abcdefghijh") + assert await r.setrange("a", 6, "12345") == 11 + assert await r.get("a") == b"abcdef12345" + + async def test_strlen(self, r: redis.Redis): + await r.set("a", "foo") + assert await r.strlen("a") == 3 + + async def test_substr(self, r: redis.Redis): + await r.set("a", "0123456789") + assert await r.substr("a", 0) == b"0123456789" + assert await r.substr("a", 2) == b"23456789" + assert await r.substr("a", 3, 5) == b"345" + assert await r.substr("a", 3, -2) == b"345678" + + async def test_ttl(self, r: redis.Redis): + await r.set("a", "1") + assert await r.expire("a", 10) + assert 0 < await r.ttl("a") <= 10 + assert await r.persist("a") + assert await r.ttl("a") == -1 + + @skip_if_server_version_lt("2.8.0") + async def test_ttl_nokey(self, r: redis.Redis): + """TTL on servers 2.8 and after return -2 when the key doesn't exist""" + assert await r.ttl("a") == -2 + + async def test_type(self, r: redis.Redis): + assert await r.type("a") == b"none" + await r.set("a", "1") + assert await r.type("a") == b"string" + await r.delete("a") + await r.lpush("a", "1") + assert await r.type("a") == b"list" + await r.delete("a") + await r.sadd("a", "1") + assert await r.type("a") == b"set" + await r.delete("a") + await r.zadd("a", {"1": 1}) + assert await r.type("a") == b"zset" + + # LIST COMMANDS + @pytest.mark.onlynoncluster + async def test_blpop(self, r: redis.Redis): + await r.rpush("a", "1", "2") + await r.rpush("b", "3", "4") + assert_resp_response( + r, await r.blpop(["b", "a"], timeout=1), (b"b", b"3"), [b"b", b"3"] + ) + assert_resp_response( + r, await r.blpop(["b", "a"], timeout=1), (b"b", b"4"), [b"b", b"4"] + ) + assert_resp_response( + r, await r.blpop(["b", "a"], timeout=1), (b"a", b"1"), [b"a", b"1"] + ) + assert_resp_response( + r, await r.blpop(["b", "a"], timeout=1), (b"a", b"2"), [b"a", b"2"] + ) + assert await r.blpop(["b", "a"], timeout=1) is None + await r.rpush("c", "1") + assert_resp_response( + r, await r.blpop("c", timeout=1), (b"c", b"1"), [b"c", b"1"] + ) + + @pytest.mark.onlynoncluster + async def test_brpop(self, r: redis.Redis): + await r.rpush("a", "1", "2") + await r.rpush("b", "3", "4") + assert_resp_response( + r, await r.brpop(["b", "a"], timeout=1), (b"b", b"4"), [b"b", b"4"] + ) + assert_resp_response( + r, await r.brpop(["b", "a"], timeout=1), (b"b", b"3"), [b"b", b"3"] + ) + assert_resp_response( + r, await r.brpop(["b", "a"], timeout=1), (b"a", b"2"), [b"a", b"2"] + ) + assert_resp_response( + r, await r.brpop(["b", "a"], timeout=1), (b"a", b"1"), [b"a", b"1"] + ) + assert await r.brpop(["b", "a"], timeout=1) is None + await r.rpush("c", "1") + assert_resp_response( + r, await r.brpop("c", timeout=1), (b"c", b"1"), [b"c", b"1"] + ) + + @pytest.mark.onlynoncluster + async def test_brpoplpush(self, r: redis.Redis): + await r.rpush("a", "1", "2") + await r.rpush("b", "3", "4") + assert await r.brpoplpush("a", "b") == b"2" + assert await r.brpoplpush("a", "b") == b"1" + assert await r.brpoplpush("a", "b", timeout=1) is None + assert await r.lrange("a", 0, -1) == [] + assert await r.lrange("b", 0, -1) == [b"1", b"2", b"3", b"4"] + + @pytest.mark.onlynoncluster + async def test_brpoplpush_empty_string(self, r: redis.Redis): + await r.rpush("a", "") + assert await r.brpoplpush("a", "b") == b"" + + async def test_lindex(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3") + assert await r.lindex("a", "0") == b"1" + assert await r.lindex("a", "1") == b"2" + assert await r.lindex("a", "2") == b"3" + + async def test_linsert(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3") + assert await r.linsert("a", "after", "2", "2.5") == 4 + assert await r.lrange("a", 0, -1) == [b"1", b"2", b"2.5", b"3"] + assert await r.linsert("a", "before", "2", "1.5") == 5 + assert await r.lrange("a", 0, -1) == [b"1", b"1.5", b"2", b"2.5", b"3"] + + async def test_llen(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3") + assert await r.llen("a") == 3 + + async def test_lpop(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3") + assert await r.lpop("a") == b"1" + assert await r.lpop("a") == b"2" + assert await r.lpop("a") == b"3" + assert await r.lpop("a") is None + + async def test_lpush(self, r: redis.Redis): + assert await r.lpush("a", "1") == 1 + assert await r.lpush("a", "2") == 2 + assert await r.lpush("a", "3", "4") == 4 + assert await r.lrange("a", 0, -1) == [b"4", b"3", b"2", b"1"] + + async def test_lpushx(self, r: redis.Redis): + assert await r.lpushx("a", "1") == 0 + assert await r.lrange("a", 0, -1) == [] + await r.rpush("a", "1", "2", "3") + assert await r.lpushx("a", "4") == 4 + assert await r.lrange("a", 0, -1) == [b"4", b"1", b"2", b"3"] + + async def test_lrange(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3", "4", "5") + assert await r.lrange("a", 0, 2) == [b"1", b"2", b"3"] + assert await r.lrange("a", 2, 10) == [b"3", b"4", b"5"] + assert await r.lrange("a", 0, -1) == [b"1", b"2", b"3", b"4", b"5"] + + async def test_lrem(self, r: redis.Redis): + await r.rpush("a", "Z", "b", "Z", "Z", "c", "Z", "Z") + # remove the first 'Z' item + assert await r.lrem("a", 1, "Z") == 1 + assert await r.lrange("a", 0, -1) == [b"b", b"Z", b"Z", b"c", b"Z", b"Z"] + # remove the last 2 'Z' items + assert await r.lrem("a", -2, "Z") == 2 + assert await r.lrange("a", 0, -1) == [b"b", b"Z", b"Z", b"c"] + # remove all 'Z' items + assert await r.lrem("a", 0, "Z") == 2 + assert await r.lrange("a", 0, -1) == [b"b", b"c"] + + async def test_lset(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3") + assert await r.lrange("a", 0, -1) == [b"1", b"2", b"3"] + assert await r.lset("a", 1, "4") + assert await r.lrange("a", 0, 2) == [b"1", b"4", b"3"] + + async def test_ltrim(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3") + assert await r.ltrim("a", 0, 1) + assert await r.lrange("a", 0, -1) == [b"1", b"2"] + + async def test_rpop(self, r: redis.Redis): + await r.rpush("a", "1", "2", "3") + assert await r.rpop("a") == b"3" + assert await r.rpop("a") == b"2" + assert await r.rpop("a") == b"1" + assert await r.rpop("a") is None + + @pytest.mark.onlynoncluster + async def test_rpoplpush(self, r: redis.Redis): + await r.rpush("a", "a1", "a2", "a3") + await r.rpush("b", "b1", "b2", "b3") + assert await r.rpoplpush("a", "b") == b"a3" + assert await r.lrange("a", 0, -1) == [b"a1", b"a2"] + assert await r.lrange("b", 0, -1) == [b"a3", b"b1", b"b2", b"b3"] + + async def test_rpush(self, r: redis.Redis): + assert await r.rpush("a", "1") == 1 + assert await r.rpush("a", "2") == 2 + assert await r.rpush("a", "3", "4") == 4 + assert await r.lrange("a", 0, -1) == [b"1", b"2", b"3", b"4"] + + @skip_if_server_version_lt("6.0.6") + async def test_lpos(self, r: redis.Redis): + assert await r.rpush("a", "a", "b", "c", "1", "2", "3", "c", "c") == 8 + assert await r.lpos("a", "a") == 0 + assert await r.lpos("a", "c") == 2 + + assert await r.lpos("a", "c", rank=1) == 2 + assert await r.lpos("a", "c", rank=2) == 6 + assert await r.lpos("a", "c", rank=4) is None + assert await r.lpos("a", "c", rank=-1) == 7 + assert await r.lpos("a", "c", rank=-2) == 6 + + assert await r.lpos("a", "c", count=0) == [2, 6, 7] + assert await r.lpos("a", "c", count=1) == [2] + assert await r.lpos("a", "c", count=2) == [2, 6] + assert await r.lpos("a", "c", count=100) == [2, 6, 7] + + assert await r.lpos("a", "c", count=0, rank=2) == [6, 7] + assert await r.lpos("a", "c", count=2, rank=-1) == [7, 6] + + assert await r.lpos("axxx", "c", count=0, rank=2) == [] + assert await r.lpos("axxx", "c") is None + + assert await r.lpos("a", "x", count=2) == [] + assert await r.lpos("a", "x") is None + + assert await r.lpos("a", "a", count=0, maxlen=1) == [0] + assert await r.lpos("a", "c", count=0, maxlen=1) == [] + assert await r.lpos("a", "c", count=0, maxlen=3) == [2] + assert await r.lpos("a", "c", count=0, maxlen=3, rank=-1) == [7, 6] + assert await r.lpos("a", "c", count=0, maxlen=7, rank=2) == [6] + + async def test_rpushx(self, r: redis.Redis): + assert await r.rpushx("a", "b") == 0 + assert await r.lrange("a", 0, -1) == [] + await r.rpush("a", "1", "2", "3") + assert await r.rpushx("a", "4") == 4 + assert await r.lrange("a", 0, -1) == [b"1", b"2", b"3", b"4"] + + # SCAN COMMANDS + @skip_if_server_version_lt("2.8.0") + @pytest.mark.onlynoncluster + async def test_scan(self, r: redis.Redis): + await r.set("a", 1) + await r.set("b", 2) + await r.set("c", 3) + cursor, keys = await r.scan() + assert cursor == 0 + assert set(keys) == {b"a", b"b", b"c"} + _, keys = await r.scan(match="a") + assert set(keys) == {b"a"} + + @skip_if_server_version_lt(REDIS_6_VERSION) + @pytest.mark.onlynoncluster + async def test_scan_type(self, r: redis.Redis): + await r.sadd("a-set", 1) + await r.hset("a-hash", "foo", 2) + await r.lpush("a-list", "aux", 3) + _, keys = await r.scan(match="a*", _type="SET") + assert set(keys) == {b"a-set"} + + @skip_if_server_version_lt("2.8.0") + @pytest.mark.onlynoncluster + async def test_scan_iter(self, r: redis.Redis): + await r.set("a", 1) + await r.set("b", 2) + await r.set("c", 3) + keys = [k async for k in r.scan_iter()] + assert set(keys) == {b"a", b"b", b"c"} + keys = [k async for k in r.scan_iter(match="a")] + assert set(keys) == {b"a"} + + @skip_if_server_version_lt("2.8.0") + async def test_sscan(self, r: redis.Redis): + await r.sadd("a", 1, 2, 3) + cursor, members = await r.sscan("a") + assert cursor == 0 + assert set(members) == {b"1", b"2", b"3"} + _, members = await r.sscan("a", match=b"1") + assert set(members) == {b"1"} + + @skip_if_server_version_lt("2.8.0") + async def test_sscan_iter(self, r: redis.Redis): + await r.sadd("a", 1, 2, 3) + members = [k async for k in r.sscan_iter("a")] + assert set(members) == {b"1", b"2", b"3"} + members = [k async for k in r.sscan_iter("a", match=b"1")] + assert set(members) == {b"1"} + + @skip_if_server_version_lt("2.8.0") + async def test_hscan(self, r: redis.Redis): + await r.hset("a", mapping={"a": 1, "b": 2, "c": 3}) + cursor, dic = await r.hscan("a") + assert cursor == 0 + assert dic == {b"a": b"1", b"b": b"2", b"c": b"3"} + _, dic = await r.hscan("a", match="a") + assert dic == {b"a": b"1"} + _, dic = await r.hscan("a_notset", match="a") + assert dic == {} + + @skip_if_server_version_lt("7.3.240") + async def test_hscan_novalues(self, r: redis.Redis): + await r.hset("a", mapping={"a": 1, "b": 2, "c": 3}) + cursor, keys = await r.hscan("a", no_values=True) + assert cursor == 0 + assert sorted(keys) == [b"a", b"b", b"c"] + _, keys = await r.hscan("a", match="a", no_values=True) + assert keys == [b"a"] + _, keys = await r.hscan("a_notset", match="a", no_values=True) + assert keys == [] + + @skip_if_server_version_lt("2.8.0") + async def test_hscan_iter(self, r: redis.Redis): + await r.hset("a", mapping={"a": 1, "b": 2, "c": 3}) + dic = {k: v async for k, v in r.hscan_iter("a")} + assert dic == {b"a": b"1", b"b": b"2", b"c": b"3"} + dic = {k: v async for k, v in r.hscan_iter("a", match="a")} + assert dic == {b"a": b"1"} + dic = {k: v async for k, v in r.hscan_iter("a_notset", match="a")} + assert dic == {} + + @skip_if_server_version_lt("7.3.240") + async def test_hscan_iter_novalues(self, r: redis.Redis): + await r.hset("a", mapping={"a": 1, "b": 2, "c": 3}) + keys = list([k async for k in r.hscan_iter("a", no_values=True)]) + assert sorted(keys) == [b"a", b"b", b"c"] + keys = list([k async for k in r.hscan_iter("a", match="a", no_values=True)]) + assert keys == [b"a"] + keys = list( + [k async for k in r.hscan_iter("a", match="a_notset", no_values=True)] + ) + assert keys == [] + + @skip_if_server_version_lt("2.8.0") + async def test_zscan(self, r: redis.Redis): + await r.zadd("a", {"a": 1, "b": 2, "c": 3}) + cursor, pairs = await r.zscan("a") + assert cursor == 0 + assert set(pairs) == {(b"a", 1), (b"b", 2), (b"c", 3)} + _, pairs = await r.zscan("a", match="a") + assert set(pairs) == {(b"a", 1)} + + @skip_if_server_version_lt("2.8.0") + async def test_zscan_iter(self, r: redis.Redis): + await r.zadd("a", {"a": 1, "b": 2, "c": 3}) + pairs = [k async for k in r.zscan_iter("a")] + assert set(pairs) == {(b"a", 1), (b"b", 2), (b"c", 3)} + pairs = [k async for k in r.zscan_iter("a", match="a")] + assert set(pairs) == {(b"a", 1)} + + # SET COMMANDS + async def test_sadd(self, r: redis.Redis): + members = {b"1", b"2", b"3"} + await r.sadd("a", *members) + assert set(await r.smembers("a")) == members + + async def test_scard(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3") + assert await r.scard("a") == 3 + + @pytest.mark.onlynoncluster + async def test_sdiff(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3") + assert await r.sdiff("a", "b") == {b"1", b"2", b"3"} + await r.sadd("b", "2", "3") + assert await r.sdiff("a", "b") == {b"1"} + + @pytest.mark.onlynoncluster + async def test_sdiffstore(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3") + assert await r.sdiffstore("c", "a", "b") == 3 + assert await r.smembers("c") == {b"1", b"2", b"3"} + await r.sadd("b", "2", "3") + assert await r.sdiffstore("c", "a", "b") == 1 + assert await r.smembers("c") == {b"1"} + + @pytest.mark.onlynoncluster + async def test_sinter(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3") + assert await r.sinter("a", "b") == set() + await r.sadd("b", "2", "3") + assert await r.sinter("a", "b") == {b"2", b"3"} + + @pytest.mark.onlynoncluster + async def test_sinterstore(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3") + assert await r.sinterstore("c", "a", "b") == 0 + assert await r.smembers("c") == set() + await r.sadd("b", "2", "3") + assert await r.sinterstore("c", "a", "b") == 2 + assert await r.smembers("c") == {b"2", b"3"} + + async def test_sismember(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3") + assert await r.sismember("a", "1") + assert await r.sismember("a", "2") + assert await r.sismember("a", "3") + assert not await r.sismember("a", "4") + + async def test_smembers(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3") + assert set(await r.smembers("a")) == {b"1", b"2", b"3"} + + @pytest.mark.onlynoncluster + async def test_smove(self, r: redis.Redis): + await r.sadd("a", "a1", "a2") + await r.sadd("b", "b1", "b2") + assert await r.smove("a", "b", "a1") + assert await r.smembers("a") == {b"a2"} + assert await r.smembers("b") == {b"b1", b"b2", b"a1"} + + async def test_spop(self, r: redis.Redis): + s = [b"1", b"2", b"3"] + await r.sadd("a", *s) + value = await r.spop("a") + assert value in s + assert set(await r.smembers("a")) == set(s) - {value} + + @skip_if_server_version_lt("3.2.0") + async def test_spop_multi_value(self, r: redis.Redis): + s = [b"1", b"2", b"3"] + await r.sadd("a", *s) + values = await r.spop("a", 2) + assert len(values) == 2 + + for value in values: + assert value in s + + response = await r.spop("a", 1) + assert set(response) == set(s) - set(values) + + async def test_srandmember(self, r: redis.Redis): + s = [b"1", b"2", b"3"] + await r.sadd("a", *s) + assert await r.srandmember("a") in s + + @skip_if_server_version_lt("2.6.0") + async def test_srandmember_multi_value(self, r: redis.Redis): + s = [b"1", b"2", b"3"] + await r.sadd("a", *s) + randoms = await r.srandmember("a", number=2) + assert len(randoms) == 2 + assert set(randoms).intersection(s) == set(randoms) + + async def test_srem(self, r: redis.Redis): + await r.sadd("a", "1", "2", "3", "4") + assert await r.srem("a", "5") == 0 + assert await r.srem("a", "2", "4") == 2 + assert set(await r.smembers("a")) == {b"1", b"3"} + + @pytest.mark.onlynoncluster + async def test_sunion(self, r: redis.Redis): + await r.sadd("a", "1", "2") + await r.sadd("b", "2", "3") + assert set(await r.sunion("a", "b")) == {b"1", b"2", b"3"} + + @pytest.mark.onlynoncluster + async def test_sunionstore(self, r: redis.Redis): + await r.sadd("a", "1", "2") + await r.sadd("b", "2", "3") + assert await r.sunionstore("c", "a", "b") == 3 + assert set(await r.smembers("c")) == {b"1", b"2", b"3"} + + # SORTED SET COMMANDS + async def test_zadd(self, r: redis.Redis): + mapping = {"a1": 1.0, "a2": 2.0, "a3": 3.0} + await r.zadd("a", mapping) + response = await r.zrange("a", 0, -1, withscores=True) + assert_resp_response( + r, + response, + [(b"a1", 1.0), (b"a2", 2.0), (b"a3", 3.0)], + [[b"a1", 1.0], [b"a2", 2.0], [b"a3", 3.0]], + ) + + # error cases + with pytest.raises(exceptions.DataError): + await r.zadd("a", {}) + + # cannot use both nx and xx options + with pytest.raises(exceptions.DataError): + await r.zadd("a", mapping, nx=True, xx=True) + + # cannot use the incr options with more than one value + with pytest.raises(exceptions.DataError): + await r.zadd("a", mapping, incr=True) + + async def test_zadd_nx(self, r: redis.Redis): + assert await r.zadd("a", {"a1": 1}) == 1 + assert await r.zadd("a", {"a1": 99, "a2": 2}, nx=True) == 1 + response = await r.zrange("a", 0, -1, withscores=True) + assert_resp_response( + r, response, [(b"a1", 1.0), (b"a2", 2.0)], [[b"a1", 1.0], [b"a2", 2.0]] + ) + + async def test_zadd_xx(self, r: redis.Redis): + assert await r.zadd("a", {"a1": 1}) == 1 + assert await r.zadd("a", {"a1": 99, "a2": 2}, xx=True) == 0 + response = await r.zrange("a", 0, -1, withscores=True) + assert_resp_response(r, response, [(b"a1", 99.0)], [[b"a1", 99.0]]) + + async def test_zadd_ch(self, r: redis.Redis): + assert await r.zadd("a", {"a1": 1}) == 1 + assert await r.zadd("a", {"a1": 99, "a2": 2}, ch=True) == 2 + response = await r.zrange("a", 0, -1, withscores=True) + assert_resp_response( + r, response, [(b"a2", 2.0), (b"a1", 99.0)], [[b"a2", 2.0], [b"a1", 99.0]] + ) + + async def test_zadd_incr(self, r: redis.Redis): + assert await r.zadd("a", {"a1": 1}) == 1 + assert await r.zadd("a", {"a1": 4.5}, incr=True) == 5.5 + + async def test_zadd_incr_with_xx(self, r: redis.Redis): + # this asks zadd to incr 'a1' only if it exists, but it clearly + # doesn't. Redis returns a null value in this case and so should + # redis-py + assert await r.zadd("a", {"a1": 1}, xx=True, incr=True) is None + + async def test_zcard(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zcard("a") == 3 + + async def test_zcount(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zcount("a", "-inf", "+inf") == 3 + assert await r.zcount("a", 1, 2) == 2 + assert await r.zcount("a", "(" + str(1), 2) == 1 + assert await r.zcount("a", 1, "(" + str(2)) == 1 + assert await r.zcount("a", 10, 20) == 0 + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("6.2.0") + async def test_zdiff(self, r): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("b", {"a1": 1, "a2": 2}) + assert await r.zdiff(["a", "b"]) == [b"a3"] + response = await r.zdiff(["a", "b"], withscores=True) + assert_resp_response(r, response, [b"a3", b"3"], [[b"a3", 3.0]]) + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("6.2.0") + async def test_zdiffstore(self, r): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("b", {"a1": 1, "a2": 2}) + assert await r.zdiffstore("out", ["a", "b"]) + assert await r.zrange("out", 0, -1) == [b"a3"] + response = await r.zrange("out", 0, -1, withscores=True) + assert_resp_response(r, response, [(b"a3", 3.0)], [[b"a3", 3.0]]) + + async def test_zincrby(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zincrby("a", 1, "a2") == 3.0 + assert await r.zincrby("a", 5, "a3") == 8.0 + assert await r.zscore("a", "a2") == 3.0 + assert await r.zscore("a", "a3") == 8.0 + + @skip_if_server_version_lt("2.8.9") + async def test_zlexcount(self, r: redis.Redis): + await r.zadd("a", {"a": 0, "b": 0, "c": 0, "d": 0, "e": 0, "f": 0, "g": 0}) + assert await r.zlexcount("a", "-", "+") == 7 + assert await r.zlexcount("a", "[b", "[f") == 5 + + @pytest.mark.onlynoncluster + async def test_zinterstore_sum(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zinterstore("d", ["a", "b", "c"]) == 2 + response = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, response, [(b"a3", 8), (b"a1", 9)], [[b"a3", 8.0], [b"a1", 9.0]] + ) + + @pytest.mark.onlynoncluster + async def test_zinterstore_max(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zinterstore("d", ["a", "b", "c"], aggregate="MAX") == 2 + response = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, response, [(b"a3", 5), (b"a1", 6)], [[b"a3", 5], [b"a1", 6]] + ) + + @pytest.mark.onlynoncluster + async def test_zinterstore_min(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("b", {"a1": 2, "a2": 3, "a3": 5}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zinterstore("d", ["a", "b", "c"], aggregate="MIN") == 2 + response = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, response, [(b"a1", 1), (b"a3", 3)], [[b"a1", 1], [b"a3", 3]] + ) + + @pytest.mark.onlynoncluster + async def test_zinterstore_with_weight(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zinterstore("d", {"a": 1, "b": 2, "c": 3}) == 2 + response = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, response, [(b"a3", 20), (b"a1", 23)], [[b"a3", 20], [b"a1", 23]] + ) + + @skip_if_server_version_lt("4.9.0") + async def test_zpopmax(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + response = await r.zpopmax("a") + assert_resp_response(r, response, [(b"a3", 3)], [b"a3", 3.0]) + + # with count + response = await r.zpopmax("a", count=2) + assert_resp_response( + r, response, [(b"a2", 2), (b"a1", 1)], [[b"a2", 2], [b"a1", 1]] + ) + + @skip_if_server_version_lt("4.9.0") + async def test_zpopmin(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + response = await r.zpopmin("a") + assert_resp_response(r, response, [(b"a1", 1)], [b"a1", 1.0]) + + # with count + response = await r.zpopmin("a", count=2) + assert_resp_response( + r, response, [(b"a2", 2), (b"a3", 3)], [[b"a2", 2], [b"a3", 3]] + ) + + @skip_if_server_version_lt("4.9.0") + @pytest.mark.onlynoncluster + async def test_bzpopmax(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2}) + await r.zadd("b", {"b1": 10, "b2": 20}) + assert_resp_response( + r, + await r.bzpopmax(["b", "a"], timeout=1), + (b"b", b"b2", 20), + [b"b", b"b2", 20], + ) + assert_resp_response( + r, + await r.bzpopmax(["b", "a"], timeout=1), + (b"b", b"b1", 10), + [b"b", b"b1", 10], + ) + assert_resp_response( + r, + await r.bzpopmax(["b", "a"], timeout=1), + (b"a", b"a2", 2), + [b"a", b"a2", 2], + ) + assert_resp_response( + r, + await r.bzpopmax(["b", "a"], timeout=1), + (b"a", b"a1", 1), + [b"a", b"a1", 1], + ) + assert await r.bzpopmax(["b", "a"], timeout=1) is None + await r.zadd("c", {"c1": 100}) + assert_resp_response( + r, await r.bzpopmax("c", timeout=1), (b"c", b"c1", 100), [b"c", b"c1", 100] + ) + + @skip_if_server_version_lt("4.9.0") + @pytest.mark.onlynoncluster + async def test_bzpopmin(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2}) + await r.zadd("b", {"b1": 10, "b2": 20}) + assert_resp_response( + r, + await r.bzpopmin(["b", "a"], timeout=1), + (b"b", b"b1", 10), + [b"b", b"b1", 10], + ) + assert_resp_response( + r, + await r.bzpopmin(["b", "a"], timeout=1), + (b"b", b"b2", 20), + [b"b", b"b2", 20], + ) + assert_resp_response( + r, + await r.bzpopmin(["b", "a"], timeout=1), + (b"a", b"a1", 1), + [b"a", b"a1", 1], + ) + assert_resp_response( + r, + await r.bzpopmin(["b", "a"], timeout=1), + (b"a", b"a2", 2), + [b"a", b"a2", 2], + ) + assert await r.bzpopmin(["b", "a"], timeout=1) is None + await r.zadd("c", {"c1": 100}) + assert_resp_response( + r, await r.bzpopmin("c", timeout=1), (b"c", b"c1", 100), [b"c", b"c1", 100] + ) + + async def test_zrange(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zrange("a", 0, 1) == [b"a1", b"a2"] + assert await r.zrange("a", 1, 2) == [b"a2", b"a3"] + + # withscores + response = await r.zrange("a", 0, 1, withscores=True) + assert_resp_response( + r, response, [(b"a1", 1.0), (b"a2", 2.0)], [[b"a1", 1.0], [b"a2", 2.0]] + ) + response = await r.zrange("a", 1, 2, withscores=True) + assert_resp_response( + r, response, [(b"a2", 2.0), (b"a3", 3.0)], [[b"a2", 2.0], [b"a3", 3.0]] + ) + + # custom score function + # assert await r.zrange("a", 0, 1, withscores=True, score_cast_func=int) == [ + # (b"a1", 1), + # (b"a2", 2), + # ] + + @skip_if_server_version_lt("2.8.9") + async def test_zrangebylex(self, r: redis.Redis): + await r.zadd("a", {"a": 0, "b": 0, "c": 0, "d": 0, "e": 0, "f": 0, "g": 0}) + assert await r.zrangebylex("a", "-", "[c") == [b"a", b"b", b"c"] + assert await r.zrangebylex("a", "-", "(c") == [b"a", b"b"] + assert await r.zrangebylex("a", "[aaa", "(g") == [b"b", b"c", b"d", b"e", b"f"] + assert await r.zrangebylex("a", "[f", "+") == [b"f", b"g"] + assert await r.zrangebylex("a", "-", "+", start=3, num=2) == [b"d", b"e"] + + @skip_if_server_version_lt("2.9.9") + async def test_zrevrangebylex(self, r: redis.Redis): + await r.zadd("a", {"a": 0, "b": 0, "c": 0, "d": 0, "e": 0, "f": 0, "g": 0}) + assert await r.zrevrangebylex("a", "[c", "-") == [b"c", b"b", b"a"] + assert await r.zrevrangebylex("a", "(c", "-") == [b"b", b"a"] + assert await r.zrevrangebylex("a", "(g", "[aaa") == [ + b"f", + b"e", + b"d", + b"c", + b"b", + ] + assert await r.zrevrangebylex("a", "+", "[f") == [b"g", b"f"] + assert await r.zrevrangebylex("a", "+", "-", start=3, num=2) == [b"d", b"c"] + + async def test_zrangebyscore(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zrangebyscore("a", 2, 4) == [b"a2", b"a3", b"a4"] + + # slicing with start/num + assert await r.zrangebyscore("a", 2, 4, start=1, num=2) == [b"a3", b"a4"] + + # withscores + response = await r.zrangebyscore("a", 2, 4, withscores=True) + assert_resp_response( + r, + response, + [(b"a2", 2.0), (b"a3", 3.0), (b"a4", 4.0)], + [[b"a2", 2.0], [b"a3", 3.0], [b"a4", 4.0]], + ) + + # custom score function + response = await r.zrangebyscore( + "a", 2, 4, withscores=True, score_cast_func=int + ) + assert_resp_response( + r, + response, + [(b"a2", 2), (b"a3", 3), (b"a4", 4)], + [[b"a2", 2], [b"a3", 3], [b"a4", 4]], + ) + + async def test_zrank(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zrank("a", "a1") == 0 + assert await r.zrank("a", "a2") == 1 + assert await r.zrank("a", "a6") is None + + @skip_if_server_version_lt("7.2.0") + async def test_zrank_withscore(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zrank("a", "a1") == 0 + assert await r.zrank("a", "a2") == 1 + assert await r.zrank("a", "a6") is None + assert_resp_response( + r, await r.zrank("a", "a3", withscore=True), [2, b"3"], [2, 3.0] + ) + assert await r.zrank("a", "a6", withscore=True) is None + + async def test_zrem(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zrem("a", "a2") == 1 + assert await r.zrange("a", 0, -1) == [b"a1", b"a3"] + assert await r.zrem("a", "b") == 0 + assert await r.zrange("a", 0, -1) == [b"a1", b"a3"] + + async def test_zrem_multiple_keys(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zrem("a", "a1", "a2") == 2 + assert await r.zrange("a", 0, 5) == [b"a3"] + + @skip_if_server_version_lt("2.8.9") + async def test_zremrangebylex(self, r: redis.Redis): + await r.zadd("a", {"a": 0, "b": 0, "c": 0, "d": 0, "e": 0, "f": 0, "g": 0}) + assert await r.zremrangebylex("a", "-", "[c") == 3 + assert await r.zrange("a", 0, -1) == [b"d", b"e", b"f", b"g"] + assert await r.zremrangebylex("a", "[f", "+") == 2 + assert await r.zrange("a", 0, -1) == [b"d", b"e"] + assert await r.zremrangebylex("a", "[h", "+") == 0 + assert await r.zrange("a", 0, -1) == [b"d", b"e"] + + async def test_zremrangebyrank(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zremrangebyrank("a", 1, 3) == 3 + assert await r.zrange("a", 0, 5) == [b"a1", b"a5"] + + async def test_zremrangebyscore(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zremrangebyscore("a", 2, 4) == 3 + assert await r.zrange("a", 0, -1) == [b"a1", b"a5"] + assert await r.zremrangebyscore("a", 2, 4) == 0 + assert await r.zrange("a", 0, -1) == [b"a1", b"a5"] + + async def test_zrevrange(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zrevrange("a", 0, 1) == [b"a3", b"a2"] + assert await r.zrevrange("a", 1, 2) == [b"a2", b"a1"] + + # withscores + response = await r.zrevrange("a", 0, 1, withscores=True) + assert_resp_response( + r, response, [(b"a3", 3.0), (b"a2", 2.0)], [[b"a3", 3.0], [b"a2", 2.0]] + ) + response = await r.zrevrange("a", 1, 2, withscores=True) + assert_resp_response( + r, response, [(b"a2", 2.0), (b"a1", 1.0)], [[b"a2", 2.0], [b"a1", 1.0]] + ) + + # custom score function + response = await r.zrevrange("a", 0, 1, withscores=True, score_cast_func=int) + assert_resp_response( + r, response, [(b"a3", 3), (b"a2", 2)], [[b"a3", 3], [b"a2", 2]] + ) + + async def test_zrevrangebyscore(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zrevrangebyscore("a", 4, 2) == [b"a4", b"a3", b"a2"] + + # slicing with start/num + assert await r.zrevrangebyscore("a", 4, 2, start=1, num=2) == [b"a3", b"a2"] + + # withscores + response = await r.zrevrangebyscore("a", 4, 2, withscores=True) + assert_resp_response( + r, + response, + [(b"a4", 4.0), (b"a3", 3.0), (b"a2", 2.0)], + [[b"a4", 4.0], [b"a3", 3.0], [b"a2", 2.0]], + ) + + # custom score function + response = await r.zrevrangebyscore( + "a", 4, 2, withscores=True, score_cast_func=int + ) + assert_resp_response( + r, + response, + [(b"a4", 4), (b"a3", 3), (b"a2", 2)], + [[b"a4", 4], [b"a3", 3], [b"a2", 2]], + ) + + async def test_zrevrank(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zrevrank("a", "a1") == 4 + assert await r.zrevrank("a", "a2") == 3 + assert await r.zrevrank("a", "a6") is None + + @skip_if_server_version_lt("7.2.0") + async def test_zrevrank_withscore(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3, "a4": 4, "a5": 5}) + assert await r.zrevrank("a", "a1") == 4 + assert await r.zrevrank("a", "a2") == 3 + assert await r.zrevrank("a", "a6") is None + assert_resp_response( + r, await r.zrevrank("a", "a3", withscore=True), [2, b"3"], [2, 3.0] + ) + assert await r.zrevrank("a", "a6", withscore=True) is None + + async def test_zscore(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + assert await r.zscore("a", "a1") == 1.0 + assert await r.zscore("a", "a2") == 2.0 + assert await r.zscore("a", "a4") is None + + @pytest.mark.onlynoncluster + async def test_zunionstore_sum(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zunionstore("d", ["a", "b", "c"]) == 4 + response = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, + response, + [(b"a2", 3.0), (b"a4", 4.0), (b"a3", 8.0), (b"a1", 9.0)], + [[b"a2", 3.0], [b"a4", 4.0], [b"a3", 8.0], [b"a1", 9.0]], + ) + + @pytest.mark.onlynoncluster + async def test_zunionstore_max(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zunionstore("d", ["a", "b", "c"], aggregate="MAX") == 4 + respponse = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, + respponse, + [(b"a2", 2.0), (b"a4", 4.0), (b"a3", 5.0), (b"a1", 6.0)], + [[b"a2", 2.0], [b"a4", 4.0], [b"a3", 5.0], [b"a1", 6.0]], + ) + + @pytest.mark.onlynoncluster + async def test_zunionstore_min(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 2, "a3": 3}) + await r.zadd("b", {"a1": 2, "a2": 2, "a3": 4}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zunionstore("d", ["a", "b", "c"], aggregate="MIN") == 4 + response = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, + response, + [(b"a1", 1.0), (b"a2", 2.0), (b"a3", 3.0), (b"a4", 4.0)], + [[b"a1", 1.0], [b"a2", 2.0], [b"a3", 3.0], [b"a4", 4.0]], + ) + + @pytest.mark.onlynoncluster + async def test_zunionstore_with_weight(self, r: redis.Redis): + await r.zadd("a", {"a1": 1, "a2": 1, "a3": 1}) + await r.zadd("b", {"a1": 2, "a2": 2, "a3": 2}) + await r.zadd("c", {"a1": 6, "a3": 5, "a4": 4}) + assert await r.zunionstore("d", {"a": 1, "b": 2, "c": 3}) == 4 + response = await r.zrange("d", 0, -1, withscores=True) + assert_resp_response( + r, + response, + [(b"a2", 5.0), (b"a4", 12.0), (b"a3", 20.0), (b"a1", 23.0)], + [[b"a2", 5.0], [b"a4", 12.0], [b"a3", 20.0], [b"a1", 23.0]], + ) + + # HYPERLOGLOG TESTS + @skip_if_server_version_lt("2.8.9") + async def test_pfadd(self, r: redis.Redis): + members = {b"1", b"2", b"3"} + assert await r.pfadd("a", *members) == 1 + assert await r.pfadd("a", *members) == 0 + assert await r.pfcount("a") == len(members) + + @skip_if_server_version_lt("2.8.9") + @pytest.mark.onlynoncluster + async def test_pfcount(self, r: redis.Redis): + members = {b"1", b"2", b"3"} + await r.pfadd("a", *members) + assert await r.pfcount("a") == len(members) + members_b = {b"2", b"3", b"4"} + await r.pfadd("b", *members_b) + assert await r.pfcount("b") == len(members_b) + assert await r.pfcount("a", "b") == len(members_b.union(members)) + + @skip_if_server_version_lt("2.8.9") + @pytest.mark.onlynoncluster + async def test_pfmerge(self, r: redis.Redis): + mema = {b"1", b"2", b"3"} + memb = {b"2", b"3", b"4"} + memc = {b"5", b"6", b"7"} + await r.pfadd("a", *mema) + await r.pfadd("b", *memb) + await r.pfadd("c", *memc) + await r.pfmerge("d", "c", "a") + assert await r.pfcount("d") == 6 + await r.pfmerge("d", "b") + assert await r.pfcount("d") == 7 + + # HASH COMMANDS + async def test_hget_and_hset(self, r: redis.Redis): + await r.hset("a", mapping={"1": 1, "2": 2, "3": 3}) + assert await r.hget("a", "1") == b"1" + assert await r.hget("a", "2") == b"2" + assert await r.hget("a", "3") == b"3" + + # field was updated, redis returns 0 + assert await r.hset("a", "2", 5) == 0 + assert await r.hget("a", "2") == b"5" + + # field is new, redis returns 1 + assert await r.hset("a", "4", 4) == 1 + assert await r.hget("a", "4") == b"4" + + # key inside of hash that doesn't exist returns null value + assert await r.hget("a", "b") is None + + # keys with bool(key) == False + assert await r.hset("a", 0, 10) == 1 + assert await r.hset("a", "", 10) == 1 + + async def test_hset_with_multi_key_values(self, r: redis.Redis): + await r.hset("a", mapping={"1": 1, "2": 2, "3": 3}) + assert await r.hget("a", "1") == b"1" + assert await r.hget("a", "2") == b"2" + assert await r.hget("a", "3") == b"3" + + await r.hset("b", "foo", "bar", mapping={"1": 1, "2": 2}) + assert await r.hget("b", "1") == b"1" + assert await r.hget("b", "2") == b"2" + assert await r.hget("b", "foo") == b"bar" + + async def test_hset_without_data(self, r: redis.Redis): + with pytest.raises(exceptions.DataError): + await r.hset("x") + + async def test_hdel(self, r: redis.Redis): + await r.hset("a", mapping={"1": 1, "2": 2, "3": 3}) + assert await r.hdel("a", "2") == 1 + assert await r.hget("a", "2") is None + assert await r.hdel("a", "1", "3") == 2 + assert await r.hlen("a") == 0 + + async def test_hexists(self, r: redis.Redis): + await r.hset("a", mapping={"1": 1, "2": 2, "3": 3}) + assert await r.hexists("a", "1") + assert not await r.hexists("a", "4") + + async def test_hgetall(self, r: redis.Redis): + h = {b"a1": b"1", b"a2": b"2", b"a3": b"3"} + await r.hset("a", mapping=h) + assert await r.hgetall("a") == h + + async def test_hincrby(self, r: redis.Redis): + assert await r.hincrby("a", "1") == 1 + assert await r.hincrby("a", "1", amount=2) == 3 + assert await r.hincrby("a", "1", amount=-2) == 1 + + @skip_if_server_version_lt("2.6.0") + async def test_hincrbyfloat(self, r: redis.Redis): + assert await r.hincrbyfloat("a", "1") == 1.0 + assert await r.hincrbyfloat("a", "1") == 2.0 + assert await r.hincrbyfloat("a", "1", 1.2) == 3.2 + + async def test_hkeys(self, r: redis.Redis): + h = {b"a1": b"1", b"a2": b"2", b"a3": b"3"} + await r.hset("a", mapping=h) + local_keys = list(h.keys()) + remote_keys = await r.hkeys("a") + assert sorted(local_keys) == sorted(remote_keys) + + async def test_hlen(self, r: redis.Redis): + await r.hset("a", mapping={"1": 1, "2": 2, "3": 3}) + assert await r.hlen("a") == 3 + + async def test_hmget(self, r: redis.Redis): + assert await r.hset("a", mapping={"a": 1, "b": 2, "c": 3}) + assert await r.hmget("a", "a", "b", "c") == [b"1", b"2", b"3"] + + async def test_hmset(self, r: redis.Redis): + h = {b"a": b"1", b"b": b"2", b"c": b"3"} + with pytest.warns(DeprecationWarning): + assert await r.hmset("a", h) + assert await r.hgetall("a") == h + + async def test_hsetnx(self, r: redis.Redis): + # Initially set the hash field + assert await r.hsetnx("a", "1", 1) + assert await r.hget("a", "1") == b"1" + assert not await r.hsetnx("a", "1", 2) + assert await r.hget("a", "1") == b"1" + + async def test_hvals(self, r: redis.Redis): + h = {b"a1": b"1", b"a2": b"2", b"a3": b"3"} + await r.hset("a", mapping=h) + local_vals = list(h.values()) + remote_vals = await r.hvals("a") + assert sorted(local_vals) == sorted(remote_vals) + + @skip_if_server_version_lt("3.2.0") + async def test_hstrlen(self, r: redis.Redis): + await r.hset("a", mapping={"1": "22", "2": "333"}) + assert await r.hstrlen("a", "1") == 2 + assert await r.hstrlen("a", "2") == 3 + + # SORT + async def test_sort_basic(self, r: redis.Redis): + await r.rpush("a", "3", "2", "1", "4") + assert await r.sort("a") == [b"1", b"2", b"3", b"4"] + + async def test_sort_limited(self, r: redis.Redis): + await r.rpush("a", "3", "2", "1", "4") + assert await r.sort("a", start=1, num=2) == [b"2", b"3"] + + @pytest.mark.onlynoncluster + async def test_sort_by(self, r: redis.Redis): + await r.set("score:1", 8) + await r.set("score:2", 3) + await r.set("score:3", 5) + await r.rpush("a", "3", "2", "1") + assert await r.sort("a", by="score:*") == [b"2", b"3", b"1"] + + @pytest.mark.onlynoncluster + async def test_sort_get(self, r: redis.Redis): + await r.set("user:1", "u1") + await r.set("user:2", "u2") + await r.set("user:3", "u3") + await r.rpush("a", "2", "3", "1") + assert await r.sort("a", get="user:*") == [b"u1", b"u2", b"u3"] + + @pytest.mark.onlynoncluster + async def test_sort_get_multi(self, r: redis.Redis): + await r.set("user:1", "u1") + await r.set("user:2", "u2") + await r.set("user:3", "u3") + await r.rpush("a", "2", "3", "1") + assert await r.sort("a", get=("user:*", "#")) == [ + b"u1", + b"1", + b"u2", + b"2", + b"u3", + b"3", + ] + + @pytest.mark.onlynoncluster + async def test_sort_get_groups_two(self, r: redis.Redis): + await r.set("user:1", "u1") + await r.set("user:2", "u2") + await r.set("user:3", "u3") + await r.rpush("a", "2", "3", "1") + assert await r.sort("a", get=("user:*", "#"), groups=True) == [ + (b"u1", b"1"), + (b"u2", b"2"), + (b"u3", b"3"), + ] + + @pytest.mark.onlynoncluster + async def test_sort_groups_string_get(self, r: redis.Redis): + await r.set("user:1", "u1") + await r.set("user:2", "u2") + await r.set("user:3", "u3") + await r.rpush("a", "2", "3", "1") + with pytest.raises(exceptions.DataError): + await r.sort("a", get="user:*", groups=True) + + @pytest.mark.onlynoncluster + async def test_sort_groups_just_one_get(self, r: redis.Redis): + await r.set("user:1", "u1") + await r.set("user:2", "u2") + await r.set("user:3", "u3") + await r.rpush("a", "2", "3", "1") + with pytest.raises(exceptions.DataError): + await r.sort("a", get=["user:*"], groups=True) + + async def test_sort_groups_no_get(self, r: redis.Redis): + await r.set("user:1", "u1") + await r.set("user:2", "u2") + await r.set("user:3", "u3") + await r.rpush("a", "2", "3", "1") + with pytest.raises(exceptions.DataError): + await r.sort("a", groups=True) + + @pytest.mark.onlynoncluster + async def test_sort_groups_three_gets(self, r: redis.Redis): + await r.set("user:1", "u1") + await r.set("user:2", "u2") + await r.set("user:3", "u3") + await r.set("door:1", "d1") + await r.set("door:2", "d2") + await r.set("door:3", "d3") + await r.rpush("a", "2", "3", "1") + assert await r.sort("a", get=("user:*", "door:*", "#"), groups=True) == [ + (b"u1", b"d1", b"1"), + (b"u2", b"d2", b"2"), + (b"u3", b"d3", b"3"), + ] + + async def test_sort_desc(self, r: redis.Redis): + await r.rpush("a", "2", "3", "1") + assert await r.sort("a", desc=True) == [b"3", b"2", b"1"] + + async def test_sort_alpha(self, r: redis.Redis): + await r.rpush("a", "e", "c", "b", "d", "a") + assert await r.sort("a", alpha=True) == [b"a", b"b", b"c", b"d", b"e"] + + @pytest.mark.onlynoncluster + async def test_sort_store(self, r: redis.Redis): + await r.rpush("a", "2", "3", "1") + assert await r.sort("a", store="sorted_values") == 3 + assert await r.lrange("sorted_values", 0, -1) == [b"1", b"2", b"3"] + + @pytest.mark.onlynoncluster + async def test_sort_all_options(self, r: redis.Redis): + await r.set("user:1:username", "zeus") + await r.set("user:2:username", "titan") + await r.set("user:3:username", "hermes") + await r.set("user:4:username", "hercules") + await r.set("user:5:username", "apollo") + await r.set("user:6:username", "athena") + await r.set("user:7:username", "hades") + await r.set("user:8:username", "dionysus") + + await r.set("user:1:favorite_drink", "yuengling") + await r.set("user:2:favorite_drink", "rum") + await r.set("user:3:favorite_drink", "vodka") + await r.set("user:4:favorite_drink", "milk") + await r.set("user:5:favorite_drink", "pinot noir") + await r.set("user:6:favorite_drink", "water") + await r.set("user:7:favorite_drink", "gin") + await r.set("user:8:favorite_drink", "apple juice") + + await r.rpush("gods", "5", "8", "3", "1", "2", "7", "6", "4") + num = await r.sort( + "gods", + start=2, + num=4, + by="user:*:username", + get="user:*:favorite_drink", + desc=True, + alpha=True, + store="sorted", + ) + assert num == 4 + assert await r.lrange("sorted", 0, 10) == [ + b"vodka", + b"milk", + b"gin", + b"apple juice", + ] + + async def test_sort_issue_924(self, r: redis.Redis): + # Tests for issue https://github.com/andymccurdy/redis-py/issues/924 + await r.execute_command("SADD", "issue#924", 1) + await r.execute_command("SORT", "issue#924") + + @pytest.mark.onlynoncluster + async def test_cluster_addslots(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("ADDSLOTS", 1) is True + + @pytest.mark.onlynoncluster + async def test_cluster_count_failure_reports(self, mock_cluster_resp_int): + assert isinstance( + await mock_cluster_resp_int.cluster("COUNT-FAILURE-REPORTS", "node"), int + ) + + @pytest.mark.onlynoncluster + async def test_cluster_countkeysinslot(self, mock_cluster_resp_int): + assert isinstance( + await mock_cluster_resp_int.cluster("COUNTKEYSINSLOT", 2), int + ) + + @pytest.mark.onlynoncluster + async def test_cluster_delslots(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("DELSLOTS", 1) is True + + @pytest.mark.onlynoncluster + async def test_cluster_failover(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("FAILOVER", 1) is True + + @pytest.mark.onlynoncluster + async def test_cluster_forget(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("FORGET", 1) is True + + @pytest.mark.onlynoncluster + async def test_cluster_info(self, mock_cluster_resp_info): + assert isinstance(await mock_cluster_resp_info.cluster("info"), dict) + + @pytest.mark.onlynoncluster + async def test_cluster_keyslot(self, mock_cluster_resp_int): + assert isinstance(await mock_cluster_resp_int.cluster("keyslot", "asdf"), int) + + @pytest.mark.onlynoncluster + async def test_cluster_meet(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("meet", "ip", "port", 1) is True + + @pytest.mark.onlynoncluster + async def test_cluster_nodes(self, mock_cluster_resp_nodes): + assert isinstance(await mock_cluster_resp_nodes.cluster("nodes"), dict) + + @pytest.mark.onlynoncluster + async def test_cluster_replicate(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("replicate", "nodeid") is True + + @pytest.mark.onlynoncluster + async def test_cluster_reset(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("reset", "hard") is True + + @pytest.mark.onlynoncluster + async def test_cluster_saveconfig(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.cluster("saveconfig") is True + + @pytest.mark.onlynoncluster + async def test_cluster_setslot(self, mock_cluster_resp_ok): + assert ( + await mock_cluster_resp_ok.cluster("setslot", 1, "IMPORTING", "nodeid") + is True + ) + + @pytest.mark.onlynoncluster + async def test_cluster_slaves(self, mock_cluster_resp_slaves): + assert isinstance( + await mock_cluster_resp_slaves.cluster("slaves", "nodeid"), dict + ) + + @skip_if_server_version_lt("3.0.0") + @skip_if_server_version_gte("7.0.0") + @pytest.mark.onlynoncluster + async def test_readwrite(self, r: redis.Redis): + assert await r.readwrite() + + @skip_if_server_version_lt("3.0.0") + @pytest.mark.onlynoncluster + async def test_readonly_invalid_cluster_state(self, r: redis.Redis): + with pytest.raises(exceptions.RedisError): + await r.readonly() + + @skip_if_server_version_lt("3.0.0") + @pytest.mark.onlynoncluster + async def test_readonly(self, mock_cluster_resp_ok): + assert await mock_cluster_resp_ok.readonly() is True + + # GEO COMMANDS + @skip_if_server_version_lt("3.2.0") + async def test_geoadd(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + assert await r.geoadd("barcelona", values) == 2 + assert await r.zcard("barcelona") == 2 + + @skip_if_server_version_lt("3.2.0") + async def test_geoadd_invalid_params(self, r: redis.Redis): + with pytest.raises(exceptions.RedisError): + await r.geoadd("barcelona", (1, 2)) + + @skip_if_server_version_lt("3.2.0") + async def test_geodist(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + assert await r.geoadd("barcelona", values) == 2 + assert await r.geodist("barcelona", "place1", "place2") == 3067.4157 + + @skip_if_server_version_lt("3.2.0") + async def test_geodist_units(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + assert await r.geodist("barcelona", "place1", "place2", "km") == 3.0674 + + @skip_if_server_version_lt("3.2.0") + async def test_geodist_missing_one_member(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + await r.geoadd("barcelona", values) + assert await r.geodist("barcelona", "place1", "missing_member", "km") is None + + @skip_if_server_version_lt("3.2.0") + async def test_geodist_invalid_units(self, r: redis.Redis): + with pytest.raises(exceptions.RedisError): + assert await r.geodist("x", "y", "z", "inches") + + @skip_if_server_version_lt("3.2.0") + async def test_geohash(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + assert_resp_response( + r, + await r.geohash("barcelona", "place1", "place2", "place3"), + ["sp3e9yg3kd0", "sp3e9cbc3t0", None], + [b"sp3e9yg3kd0", b"sp3e9cbc3t0", None], + ) + + @skip_if_server_version_lt("3.2.0") + async def test_geopos(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + # redis uses 52 bits precision, hereby small errors may be introduced. + assert_resp_response( + r, + await r.geopos("barcelona", "place1", "place2"), + [ + (2.19093829393386841, 41.43379028184083523), + (2.18737632036209106, 41.40634178640635099), + ], + [ + [2.19093829393386841, 41.43379028184083523], + [2.18737632036209106, 41.40634178640635099], + ], + ) + + @skip_if_server_version_lt("4.0.0") + async def test_geopos_no_value(self, r: redis.Redis): + assert await r.geopos("barcelona", "place1", "place2") == [None, None] + + @skip_if_server_version_lt("3.2.0") + @skip_if_server_version_gte("4.0.0") + async def test_old_geopos_no_value(self, r: redis.Redis): + assert await r.geopos("barcelona", "place1", "place2") == [] + + @skip_if_server_version_lt("3.2.0") + async def test_georadius(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + b"\x80place2", + ) + + await r.geoadd("barcelona", values) + assert await r.georadius("barcelona", 2.191, 41.433, 1000) == [b"place1"] + assert await r.georadius("barcelona", 2.187, 41.406, 1000) == [b"\x80place2"] + + @skip_if_server_version_lt("3.2.0") + async def test_georadius_no_values(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + assert await r.georadius("barcelona", 1, 2, 1000) == [] + + @skip_if_server_version_lt("3.2.0") + async def test_georadius_units(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + assert await r.georadius("barcelona", 2.191, 41.433, 1, unit="km") == [ + b"place1" + ] + + @skip_unless_arch_bits(64) + @skip_if_server_version_lt("3.2.0") + async def test_georadius_with(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + + # test a bunch of combinations to test the parse response + # function. + assert await r.georadius( + "barcelona", + 2.191, + 41.433, + 1, + unit="km", + withdist=True, + withcoord=True, + withhash=True, + ) == [ + [ + b"place1", + 0.0881, + 3471609698139488, + (2.19093829393386841, 41.43379028184083523), + ] + ] + + assert await r.georadius( + "barcelona", 2.191, 41.433, 1, unit="km", withdist=True, withcoord=True + ) == [[b"place1", 0.0881, (2.19093829393386841, 41.43379028184083523)]] + + assert await r.georadius( + "barcelona", 2.191, 41.433, 1, unit="km", withhash=True, withcoord=True + ) == [ + [b"place1", 3471609698139488, (2.19093829393386841, 41.43379028184083523)] + ] + + # test no values. + assert ( + await r.georadius( + "barcelona", + 2, + 1, + 1, + unit="km", + withdist=True, + withcoord=True, + withhash=True, + ) + == [] + ) + + @skip_if_server_version_lt("3.2.0") + async def test_georadius_count(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + assert await r.georadius("barcelona", 2.191, 41.433, 3000, count=1) == [ + b"place1" + ] + + @skip_if_server_version_lt("3.2.0") + async def test_georadius_sort(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + assert await r.georadius("barcelona", 2.191, 41.433, 3000, sort="ASC") == [ + b"place1", + b"place2", + ] + assert await r.georadius("barcelona", 2.191, 41.433, 3000, sort="DESC") == [ + b"place2", + b"place1", + ] + + @skip_if_server_version_lt("3.2.0") + @pytest.mark.onlynoncluster + async def test_georadius_store(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + await r.georadius("barcelona", 2.191, 41.433, 1000, store="places_barcelona") + assert await r.zrange("places_barcelona", 0, -1) == [b"place1"] + + @skip_unless_arch_bits(64) + @skip_if_server_version_lt("3.2.0") + @pytest.mark.onlynoncluster + async def test_georadius_store_dist(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + "place2", + ) + + await r.geoadd("barcelona", values) + await r.georadius( + "barcelona", 2.191, 41.433, 1000, store_dist="places_barcelona" + ) + # instead of save the geo score, the distance is saved. + assert await r.zscore("places_barcelona", "place1") == 88.05060698409301 + + @skip_unless_arch_bits(64) + @skip_if_server_version_lt("3.2.0") + async def test_georadiusmember(self, r: redis.Redis): + values = (2.1909389952632, 41.433791470673, "place1") + ( + 2.1873744593677, + 41.406342043777, + b"\x80place2", + ) + + await r.geoadd("barcelona", values) + assert await r.georadiusbymember("barcelona", "place1", 4000) == [ + b"\x80place2", + b"place1", + ] + assert await r.georadiusbymember("barcelona", "place1", 10) == [b"place1"] + + assert await r.georadiusbymember( + "barcelona", "place1", 4000, withdist=True, withcoord=True, withhash=True + ) == [ + [ + b"\x80place2", + 3067.4157, + 3471609625421029, + (2.187376320362091, 41.40634178640635), + ], + [ + b"place1", + 0.0, + 3471609698139488, + (2.1909382939338684, 41.433790281840835), + ], + ] + + @skip_if_server_version_lt("5.0.0") + async def test_xack(self, r: redis.Redis): + stream = "stream" + group = "group" + consumer = "consumer" + # xack on a stream that doesn't exist + assert await r.xack(stream, group, "0-0") == 0 + + m1 = await r.xadd(stream, {"one": "one"}) + m2 = await r.xadd(stream, {"two": "two"}) + m3 = await r.xadd(stream, {"three": "three"}) + + # xack on a group that doesn't exist + assert await r.xack(stream, group, m1) == 0 + + await r.xgroup_create(stream, group, 0) + await r.xreadgroup(group, consumer, streams={stream: ">"}) + # xack returns the number of ack'd elements + assert await r.xack(stream, group, m1) == 1 + assert await r.xack(stream, group, m2, m3) == 2 + + @skip_if_server_version_lt("5.0.0") + async def test_xadd(self, r: redis.Redis): + stream = "stream" + message_id = await r.xadd(stream, {"foo": "bar"}) + assert re.match(rb"[0-9]+\-[0-9]+", message_id) + + # explicit message id + message_id = b"9999999999999999999-0" + assert message_id == await r.xadd(stream, {"foo": "bar"}, id=message_id) + + # with maxlen, the list evicts the first message + await r.xadd(stream, {"foo": "bar"}, maxlen=2, approximate=False) + assert await r.xlen(stream) == 2 + + @skip_if_server_version_lt("5.0.0") + async def test_xclaim(self, r: redis.Redis): + stream = "stream" + group = "group" + consumer1 = "consumer1" + consumer2 = "consumer2" + + message_id = await r.xadd(stream, {"john": "wick"}) + message = await get_stream_message(r, stream, message_id) + await r.xgroup_create(stream, group, 0) + + # trying to claim a message that isn't already pending doesn't + # do anything + response = await r.xclaim( + stream, group, consumer2, min_idle_time=0, message_ids=(message_id,) + ) + assert response == [] + + # read the group as consumer1 to initially claim the messages + await r.xreadgroup(group, consumer1, streams={stream: ">"}) + + # claim the message as consumer2 + response = await r.xclaim( + stream, group, consumer2, min_idle_time=0, message_ids=(message_id,) + ) + assert response[0] == message + + # reclaim the message as consumer1, but use the justid argument + # which only returns message ids + assert await r.xclaim( + stream, + group, + consumer1, + min_idle_time=0, + message_ids=(message_id,), + justid=True, + ) == [message_id] + + @skip_if_server_version_lt("7.0.0") + async def test_xclaim_trimmed(self, r: redis.Redis): + # xclaim should not raise an exception if the item is not there + stream = "stream" + group = "group" + + await r.xgroup_create(stream, group, id="$", mkstream=True) + + # add a couple of new items + sid1 = await r.xadd(stream, {"item": 0}) + sid2 = await r.xadd(stream, {"item": 0}) + + # read them from consumer1 + await r.xreadgroup(group, "consumer1", {stream: ">"}) + + # add a 3rd and trim the stream down to 2 items + await r.xadd(stream, {"item": 3}, maxlen=2, approximate=False) + + # xclaim them from consumer2 + # the item that is still in the stream should be returned + item = await r.xclaim(stream, group, "consumer2", 0, [sid1, sid2]) + assert len(item) == 1 + assert item[0][0] == sid2 + + @skip_if_server_version_lt("5.0.0") + async def test_xdel(self, r: redis.Redis): + stream = "stream" + + # deleting from an empty stream doesn't do anything + assert await r.xdel(stream, 1) == 0 + + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"foo": "bar"}) + m3 = await r.xadd(stream, {"foo": "bar"}) + + # xdel returns the number of deleted elements + assert await r.xdel(stream, m1) == 1 + assert await r.xdel(stream, m2, m3) == 2 + + @skip_if_server_version_lt("7.0.0") + async def test_xgroup_create(self, r: redis.Redis): + # tests xgroup_create and xinfo_groups + stream = "stream" + group = "group" + await r.xadd(stream, {"foo": "bar"}) + + # no group is setup yet, no info to obtain + assert await r.xinfo_groups(stream) == [] + + assert await r.xgroup_create(stream, group, 0) + expected = [ + { + "name": group.encode(), + "consumers": 0, + "pending": 0, + "last-delivered-id": b"0-0", + "entries-read": None, + "lag": 1, + } + ] + assert await r.xinfo_groups(stream) == expected + + @skip_if_server_version_lt("7.0.0") + async def test_xgroup_create_mkstream(self, r: redis.Redis): + # tests xgroup_create and xinfo_groups + stream = "stream" + group = "group" + + # an error is raised if a group is created on a stream that + # doesn't already exist + with pytest.raises(exceptions.ResponseError): + await r.xgroup_create(stream, group, 0) + + # however, with mkstream=True, the underlying stream is created + # automatically + assert await r.xgroup_create(stream, group, 0, mkstream=True) + expected = [ + { + "name": group.encode(), + "consumers": 0, + "pending": 0, + "last-delivered-id": b"0-0", + "entries-read": None, + "lag": 0, + } + ] + assert await r.xinfo_groups(stream) == expected + + @skip_if_server_version_lt("5.0.0") + async def test_xgroup_delconsumer(self, r: redis.Redis): + stream = "stream" + group = "group" + consumer = "consumer" + await r.xadd(stream, {"foo": "bar"}) + await r.xadd(stream, {"foo": "bar"}) + await r.xgroup_create(stream, group, 0) + + # a consumer that hasn't yet read any messages doesn't do anything + assert await r.xgroup_delconsumer(stream, group, consumer) == 0 + + # read all messages from the group + await r.xreadgroup(group, consumer, streams={stream: ">"}) + + # deleting the consumer should return 2 pending messages + assert await r.xgroup_delconsumer(stream, group, consumer) == 2 + + @skip_if_server_version_lt("5.0.0") + async def test_xgroup_destroy(self, r: redis.Redis): + stream = "stream" + group = "group" + await r.xadd(stream, {"foo": "bar"}) + + # destroying a nonexistent group returns False + assert not await r.xgroup_destroy(stream, group) + + await r.xgroup_create(stream, group, 0) + assert await r.xgroup_destroy(stream, group) + + @skip_if_server_version_lt("7.0.0") + async def test_xgroup_setid(self, r: redis.Redis): + stream = "stream" + group = "group" + message_id = await r.xadd(stream, {"foo": "bar"}) + + await r.xgroup_create(stream, group, 0) + # advance the last_delivered_id to the message_id + await r.xgroup_setid(stream, group, message_id, entries_read=2) + expected = [ + { + "name": group.encode(), + "consumers": 0, + "pending": 0, + "last-delivered-id": message_id, + "entries-read": 2, + "lag": -1, + } + ] + assert await r.xinfo_groups(stream) == expected + + @skip_if_server_version_lt("7.2.0") + async def test_xinfo_consumers(self, r: redis.Redis): + stream = "stream" + group = "group" + consumer1 = "consumer1" + consumer2 = "consumer2" + await r.xadd(stream, {"foo": "bar"}) + await r.xadd(stream, {"foo": "bar"}) + await r.xadd(stream, {"foo": "bar"}) + + await r.xgroup_create(stream, group, 0) + await r.xreadgroup(group, consumer1, streams={stream: ">"}, count=1) + await r.xreadgroup(group, consumer2, streams={stream: ">"}) + info = await r.xinfo_consumers(stream, group) + assert len(info) == 2 + expected = [ + {"name": consumer1.encode(), "pending": 1}, + {"name": consumer2.encode(), "pending": 2}, + ] + + # we can't determine the idle and inactive time, so just make sure it's an int + assert isinstance(info[0].pop("idle"), int) + assert isinstance(info[1].pop("idle"), int) + assert isinstance(info[0].pop("inactive"), int) + assert isinstance(info[1].pop("inactive"), int) + assert info == expected + + @skip_if_server_version_lt("5.0.0") + async def test_xinfo_stream(self, r: redis.Redis): + stream = "stream" + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"foo": "bar"}) + info = await r.xinfo_stream(stream) + + assert info["length"] == 2 + assert info["first-entry"] == await get_stream_message(r, stream, m1) + assert info["last-entry"] == await get_stream_message(r, stream, m2) + + await r.xtrim(stream, 0) + info = await r.xinfo_stream(stream) + assert info["length"] == 0 + assert info["first-entry"] is None + assert info["last-entry"] is None + + @skip_if_server_version_lt("6.0.0") + async def test_xinfo_stream_full(self, r: redis.Redis): + stream = "stream" + group = "group" + + await r.xadd(stream, {"foo": "bar"}) + info = await r.xinfo_stream(stream, full=True) + assert info["length"] == 1 + assert len(info["groups"]) == 0 + + await r.xgroup_create(stream, group, 0) + info = await r.xinfo_stream(stream, full=True) + assert info["length"] == 1 + + await r.xreadgroup(group, "consumer", streams={stream: ">"}) + info = await r.xinfo_stream(stream, full=True) + consumer = info["groups"][0]["consumers"][0] + assert isinstance(consumer, dict) + + @skip_if_server_version_lt("5.0.0") + async def test_xlen(self, r: redis.Redis): + stream = "stream" + assert await r.xlen(stream) == 0 + await r.xadd(stream, {"foo": "bar"}) + await r.xadd(stream, {"foo": "bar"}) + assert await r.xlen(stream) == 2 + + @skip_if_server_version_lt("5.0.0") + async def test_xpending(self, r: redis.Redis): + stream = "stream" + group = "group" + consumer1 = "consumer1" + consumer2 = "consumer2" + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"foo": "bar"}) + await r.xgroup_create(stream, group, 0) + + # xpending on a group that has no consumers yet + expected = {"pending": 0, "min": None, "max": None, "consumers": []} + assert await r.xpending(stream, group) == expected + + # read 1 message from the group with each consumer + await r.xreadgroup(group, consumer1, streams={stream: ">"}, count=1) + await r.xreadgroup(group, consumer2, streams={stream: ">"}, count=1) + + expected = { + "pending": 2, + "min": m1, + "max": m2, + "consumers": [ + {"name": consumer1.encode(), "pending": 1}, + {"name": consumer2.encode(), "pending": 1}, + ], + } + assert await r.xpending(stream, group) == expected + + @skip_if_server_version_lt("5.0.0") + async def test_xpending_range(self, r: redis.Redis): + stream = "stream" + group = "group" + consumer1 = "consumer1" + consumer2 = "consumer2" + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"foo": "bar"}) + await r.xgroup_create(stream, group, 0) + + # xpending range on a group that has no consumers yet + assert await r.xpending_range(stream, group, min="-", max="+", count=5) == [] + + # read 1 message from the group with each consumer + await r.xreadgroup(group, consumer1, streams={stream: ">"}, count=1) + await r.xreadgroup(group, consumer2, streams={stream: ">"}, count=1) + + response = await r.xpending_range(stream, group, min="-", max="+", count=5) + assert len(response) == 2 + assert response[0]["message_id"] == m1 + assert response[0]["consumer"] == consumer1.encode() + assert response[1]["message_id"] == m2 + assert response[1]["consumer"] == consumer2.encode() + + @skip_if_server_version_lt("5.0.0") + async def test_xrange(self, r: redis.Redis): + stream = "stream" + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"foo": "bar"}) + m3 = await r.xadd(stream, {"foo": "bar"}) + m4 = await r.xadd(stream, {"foo": "bar"}) + + def get_ids(results): + return [result[0] for result in results] + + results = await r.xrange(stream, min=m1) + assert get_ids(results) == [m1, m2, m3, m4] + + results = await r.xrange(stream, min=m2, max=m3) + assert get_ids(results) == [m2, m3] + + results = await r.xrange(stream, max=m3) + assert get_ids(results) == [m1, m2, m3] + + results = await r.xrange(stream, max=m2, count=1) + assert get_ids(results) == [m1] + + @skip_if_server_version_lt("5.0.0") + async def test_xread(self, r: redis.Redis): + stream = "stream" + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"bing": "baz"}) + + strem_name = stream.encode() + expected_entries = [ + await get_stream_message(r, stream, m1), + await get_stream_message(r, stream, m2), + ] + # xread starting at 0 returns both messages + res = await r.xread(streams={stream: 0}) + assert_resp_response( + r, res, [[strem_name, expected_entries]], {strem_name: [expected_entries]} + ) + + expected_entries = [await get_stream_message(r, stream, m1)] + # xread starting at 0 and count=1 returns only the first message + res = await r.xread(streams={stream: 0}, count=1) + assert_resp_response( + r, res, [[strem_name, expected_entries]], {strem_name: [expected_entries]} + ) + + expected_entries = [await get_stream_message(r, stream, m2)] + # xread starting at m1 returns only the second message + res = await r.xread(streams={stream: m1}) + assert_resp_response( + r, res, [[strem_name, expected_entries]], {strem_name: [expected_entries]} + ) + + @skip_if_server_version_lt("5.0.0") + async def test_xreadgroup(self, r: redis.Redis): + stream = "stream" + group = "group" + consumer = "consumer" + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"bing": "baz"}) + await r.xgroup_create(stream, group, 0) + + strem_name = stream.encode() + expected_entries = [ + await get_stream_message(r, stream, m1), + await get_stream_message(r, stream, m2), + ] + + # xread starting at 0 returns both messages + res = await r.xreadgroup(group, consumer, streams={stream: ">"}) + assert_resp_response( + r, res, [[strem_name, expected_entries]], {strem_name: [expected_entries]} + ) + + await r.xgroup_destroy(stream, group) + await r.xgroup_create(stream, group, 0) + + expected_entries = [await get_stream_message(r, stream, m1)] + + # xread with count=1 returns only the first message + res = await r.xreadgroup(group, consumer, streams={stream: ">"}, count=1) + assert_resp_response( + r, res, [[strem_name, expected_entries]], {strem_name: [expected_entries]} + ) + + await r.xgroup_destroy(stream, group) + + # create the group using $ as the last id meaning subsequent reads + # will only find messages added after this + await r.xgroup_create(stream, group, "$") + + # xread starting after the last message returns an empty message list + res = await r.xreadgroup(group, consumer, streams={stream: ">"}) + assert_resp_response(r, res, [], {}) + + # xreadgroup with noack does not have any items in the PEL + await r.xgroup_destroy(stream, group) + await r.xgroup_create(stream, group, "0") + res = await r.xreadgroup(group, consumer, streams={stream: ">"}, noack=True) + empty_res = await r.xreadgroup(group, consumer, streams={stream: "0"}) + if is_resp2_connection(r): + assert len(res[0][1]) == 2 + # now there should be nothing pending + assert len(empty_res[0][1]) == 0 + else: + assert len(res[strem_name][0]) == 2 + # now there should be nothing pending + assert len(empty_res[strem_name][0]) == 0 + + await r.xgroup_destroy(stream, group) + await r.xgroup_create(stream, group, "0") + # delete all the messages in the stream + expected_entries = [(m1, {}), (m2, {})] + await r.xreadgroup(group, consumer, streams={stream: ">"}) + await r.xtrim(stream, 0) + res = await r.xreadgroup(group, consumer, streams={stream: "0"}) + assert_resp_response( + r, res, [[strem_name, expected_entries]], {strem_name: [expected_entries]} + ) + + @skip_if_server_version_lt("5.0.0") + async def test_xrevrange(self, r: redis.Redis): + stream = "stream" + m1 = await r.xadd(stream, {"foo": "bar"}) + m2 = await r.xadd(stream, {"foo": "bar"}) + m3 = await r.xadd(stream, {"foo": "bar"}) + m4 = await r.xadd(stream, {"foo": "bar"}) + + def get_ids(results): + return [result[0] for result in results] + + results = await r.xrevrange(stream, max=m4) + assert get_ids(results) == [m4, m3, m2, m1] + + results = await r.xrevrange(stream, max=m3, min=m2) + assert get_ids(results) == [m3, m2] + + results = await r.xrevrange(stream, min=m3) + assert get_ids(results) == [m4, m3] + + results = await r.xrevrange(stream, min=m2, count=1) + assert get_ids(results) == [m4] + + @skip_if_server_version_lt("5.0.0") + async def test_xtrim(self, r: redis.Redis): + stream = "stream" + + # trimming an empty key doesn't do anything + assert await r.xtrim(stream, 1000) == 0 + + await r.xadd(stream, {"foo": "bar"}) + await r.xadd(stream, {"foo": "bar"}) + await r.xadd(stream, {"foo": "bar"}) + await r.xadd(stream, {"foo": "bar"}) + + # trimming an amount large than the number of messages + # doesn't do anything + assert await r.xtrim(stream, 5, approximate=False) == 0 + + # 1 message is trimmed + assert await r.xtrim(stream, 3, approximate=False) == 1 + + @pytest.mark.onlynoncluster + async def test_bitfield_operations(self, r: redis.Redis): + # comments show affected bits + await r.execute_command("SELECT", 10) + bf = r.bitfield("a") + resp = await ( + bf.set("u8", 8, 255) # 00000000 11111111 + .get("u8", 0) # 00000000 + .get("u4", 8) # 1111 + .get("u4", 12) # 1111 + .get("u4", 13) # 111 0 + .execute() + ) + assert resp == [0, 0, 15, 15, 14] + + # .set() returns the previous value... + resp = await ( + bf.set("u8", 4, 1) # 0000 0001 + .get("u16", 0) # 00000000 00011111 + .set("u16", 0, 0) # 00000000 00000000 + .execute() + ) + assert resp == [15, 31, 31] + + # incrby adds to the value + resp = await ( + bf.incrby("u8", 8, 254) # 00000000 11111110 + .incrby("u8", 8, 1) # 00000000 11111111 + .get("u16", 0) # 00000000 11111111 + .execute() + ) + assert resp == [254, 255, 255] + + # Verify overflow protection works as a method: + await r.delete("a") + resp = await ( + bf.set("u8", 8, 254) # 00000000 11111110 + .overflow("fail") + .incrby("u8", 8, 2) # incrby 2 would overflow, None returned + .incrby("u8", 8, 1) # 00000000 11111111 + .incrby("u8", 8, 1) # incrby 1 would overflow, None returned + .get("u16", 0) # 00000000 11111111 + .execute() + ) + assert resp == [0, None, 255, None, 255] + + # Verify overflow protection works as arg to incrby: + await r.delete("a") + resp = await ( + bf.set("u8", 8, 255) # 00000000 11111111 + .incrby("u8", 8, 1) # 00000000 00000000 wrap default + .set("u8", 8, 255) # 00000000 11111111 + .incrby("u8", 8, 1, "FAIL") # 00000000 11111111 fail + .incrby("u8", 8, 1) # 00000000 11111111 still fail + .get("u16", 0) # 00000000 11111111 + .execute() + ) + assert resp == [0, 0, 0, None, None, 255] + + # test default default_overflow + await r.delete("a") + bf = r.bitfield("a", default_overflow="FAIL") + resp = await ( + bf.set("u8", 8, 255) # 00000000 11111111 + .incrby("u8", 8, 1) # 00000000 11111111 fail default + .get("u16", 0) # 00000000 11111111 + .execute() + ) + assert resp == [0, None, 255] + + @skip_if_server_version_lt("6.0.0") + async def test_bitfield_ro(self, r: redis.Redis): + bf = r.bitfield("a") + resp = await bf.set("u8", 8, 255).execute() + assert resp == [0] + + resp = await r.bitfield_ro("a", "u8", 0) + assert resp == [0] + + items = [("u4", 8), ("u4", 12), ("u4", 13)] + resp = await r.bitfield_ro("a", "u8", 0, items) + assert resp == [0, 15, 15, 14] + + @skip_if_server_version_lt("4.0.0") + async def test_memory_stats(self, r: redis.Redis): + # put a key into the current db to make sure that "db." + # has data + await r.set("foo", "bar") + stats = await r.memory_stats() + assert isinstance(stats, dict) + for key, value in stats.items(): + if key.startswith("db."): + assert not isinstance(value, list) + + @skip_if_server_version_lt("4.0.0") + async def test_memory_usage(self, r: redis.Redis): + await r.set("foo", "bar") + assert isinstance(await r.memory_usage("foo"), int) + + @skip_if_server_version_lt("4.0.0") + async def test_module_list(self, r: redis.Redis): + assert isinstance(await r.module_list(), list) + for x in await r.module_list(): + assert isinstance(x, dict) + + @pytest.mark.onlynoncluster + async def test_interrupted_command(self, r: redis.Redis): + """ + Regression test for issue #1128: An Un-handled BaseException + will leave the socket with un-read response to a previous + command. + """ + ready = anyio.Event() + + async def helper(): + with pytest.raises(anyio.get_cancelled_exc_class()): + # blocking pop + ready.set() + await r.brpop(["nonexist"]) + + # if all is well, we can continue. The following should not hang. + with anyio.fail_after(5, shield=True): + await r.set("status", "down") + + # If all is well, the task should finish right away, otherwise fail with Timeout + with anyio.fail_after(1.0): + async with anyio.create_task_group() as tg: + tg.start_soon(helper) + await ready.wait() + await anyio.wait_all_tasks_blocked() + tg.cancel_scope.cancel() + + +@pytest.mark.onlynoncluster +class TestBinarySave: + async def test_binary_get_set(self, r: redis.Redis): + assert await r.set(" foo bar ", "123") + assert await r.get(" foo bar ") == b"123" + + assert await r.set(" foo\r\nbar\r\n ", "456") + assert await r.get(" foo\r\nbar\r\n ") == b"456" + + assert await r.set(" \r\n\t\x07\x13 ", "789") + assert await r.get(" \r\n\t\x07\x13 ") == b"789" + + assert sorted(await r.keys("*")) == [ + b" \r\n\t\x07\x13 ", + b" foo\r\nbar\r\n ", + b" foo bar ", + ] + + assert await r.delete(" foo bar ") + assert await r.delete(" foo\r\nbar\r\n ") + assert await r.delete(" \r\n\t\x07\x13 ") + + async def test_binary_lists(self, r: redis.Redis): + mapping = { + b"foo bar": [b"1", b"2", b"3"], + b"foo\r\nbar\r\n": [b"4", b"5", b"6"], + b"foo\tbar\x07": [b"7", b"8", b"9"], + } + # fill in lists + for key, value in mapping.items(): + await r.rpush(key, *value) + + # check that KEYS returns all the keys as they are + assert sorted(await r.keys("*")) == sorted(mapping.keys()) + + # check that it is possible to get list content by key name + for key, value in mapping.items(): + assert await r.lrange(key, 0, -1) == value + + async def test_22_info(self, r: redis.Redis): + """ + Older Redis versions contained 'allocation_stats' in INFO that + was the cause of a number of bugs when parsing. + """ + info = ( + "allocation_stats:6=1,7=1,8=7141,9=180,10=92,11=116,12=5330," + "13=123,14=3091,15=11048,16=225842,17=1784,18=814,19=12020," + "20=2530,21=645,22=15113,23=8695,24=142860,25=318,26=3303," + "27=20561,28=54042,29=37390,30=1884,31=18071,32=31367,33=160," + "34=169,35=201,36=10155,37=1045,38=15078,39=22985,40=12523," + "41=15588,42=265,43=1287,44=142,45=382,46=945,47=426,48=171," + "49=56,50=516,51=43,52=41,53=46,54=54,55=75,56=647,57=332," + "58=32,59=39,60=48,61=35,62=62,63=32,64=221,65=26,66=30," + "67=36,68=41,69=44,70=26,71=144,72=169,73=24,74=37,75=25," + "76=42,77=21,78=126,79=374,80=27,81=40,82=43,83=47,84=46," + "85=114,86=34,87=37,88=7240,89=34,90=38,91=18,92=99,93=20," + "94=18,95=17,96=15,97=22,98=18,99=69,100=17,101=22,102=15," + "103=29,104=39,105=30,106=70,107=22,108=21,109=26,110=52," + "111=45,112=33,113=67,114=41,115=44,116=48,117=53,118=54," + "119=51,120=75,121=44,122=57,123=44,124=66,125=56,126=52," + "127=81,128=108,129=70,130=50,131=51,132=53,133=45,134=62," + "135=12,136=13,137=7,138=15,139=21,140=11,141=20,142=6,143=7," + "144=11,145=6,146=16,147=19,148=1112,149=1,151=83,154=1," + "155=1,156=1,157=1,160=1,161=1,162=2,166=1,169=1,170=1,171=2," + "172=1,174=1,176=2,177=9,178=34,179=73,180=30,181=1,185=3," + "187=1,188=1,189=1,192=1,196=1,198=1,200=1,201=1,204=1,205=1," + "207=1,208=1,209=1,214=2,215=31,216=78,217=28,218=5,219=2," + "220=1,222=1,225=1,227=1,234=1,242=1,250=1,252=1,253=1," + ">=256=203" + ) + parsed = parse_info(info) + assert "allocation_stats" in parsed + assert "6" in parsed["allocation_stats"] + assert ">=256" in parsed["allocation_stats"] + + async def test_large_responses(self, r: redis.Redis): + """The PythonParser has some special cases for return values > 1MB""" + # load up 5MB of data into a key + data = "".join([ascii_letters] * (5000000 // len(ascii_letters))) + await r.set("a", data) + assert await r.get("a") == data.encode() + + async def test_floating_point_encoding(self, r: redis.Redis): + """ + High precision floating point values sent to the server should keep + precision. + """ + timestamp = 1349673917.939762 + await r.zadd("a", {"a1": timestamp}) + assert await r.zscore("a", "a1") == timestamp diff --git a/tests/test_anyio/test_connect.py b/tests/test_anyio/test_connect.py new file mode 100644 index 0000000000..d0e3ee8e26 --- /dev/null +++ b/tests/test_anyio/test_connect.py @@ -0,0 +1,234 @@ +import re +import socket +import ssl +import sys + +import anyio +import pytest +from anyio.abc import ByteStream +from anyio.streams.tls import TLSListener + +from redis.anyio.connection import ( + Connection, + SSLConnection, + UnixDomainSocketConnection, +) +from redis.exceptions import ConnectionError + +from ..ssl_utils import CertificateType, get_tls_certificates + +if sys.version_info < (3, 11): + from exceptiongroup import ExceptionGroup + +pytestmark = pytest.mark.anyio + +_CLIENT_NAME = "test-suite-client" +_CMD_SEP = b"\r\n" +_SUCCESS_RESP = b"+OK" + _CMD_SEP +_ERROR_RESP = b"-ERR" + _CMD_SEP +_SUPPORTED_CMDS = {f"CLIENT SETNAME {_CLIENT_NAME}": _SUCCESS_RESP} + + +@pytest.fixture +def tcp_address(): + with socket.socket() as sock: + sock.bind(("127.0.0.1", 0)) + return sock.getsockname() + + +@pytest.fixture +def uds_address(tmpdir): + return tmpdir / "uds.sock" + + +async def test_tcp_connect(tcp_address): + host, port = tcp_address + conn = Connection(host=host, port=port, client_name=_CLIENT_NAME, socket_timeout=10) + await _assert_connect(conn, tcp_address) + + +async def test_uds_connect(uds_address): + path = str(uds_address) + conn = UnixDomainSocketConnection( + path=path, client_name=_CLIENT_NAME, socket_timeout=10 + ) + await _assert_connect(conn, path) + + +@pytest.mark.ssl +@pytest.mark.parametrize( + "ssl_ciphers", + [ + "AES256-SHA:DHE-RSA-AES256-SHA:AES128-SHA:DHE-RSA-AES128-SHA", + "ECDHE-ECDSA-AES256-GCM-SHA384", + "ECDHE-RSA-AES128-GCM-SHA256", + ], +) +async def test_tcp_ssl_tls12_custom_ciphers(tcp_address, ssl_ciphers): + host, port = tcp_address + + server_certs = get_tls_certificates(cert_type=CertificateType.server) + + conn = SSLConnection( + host=host, + port=port, + client_name=_CLIENT_NAME, + ssl_ca_certs=server_certs.ca_certfile, + socket_timeout=10, + ssl_min_version=ssl.TLSVersion.TLSv1_2, + ssl_ciphers=ssl_ciphers, + ) + await _assert_connect( + conn, tcp_address, certfile=server_certs.certfile, keyfile=server_certs.keyfile + ) + await conn.disconnect() + + +@pytest.mark.ssl +@pytest.mark.parametrize( + "ssl_min_version", + [ + ssl.TLSVersion.TLSv1_2, + pytest.param( + ssl.TLSVersion.TLSv1_3, + marks=pytest.mark.skipif(not ssl.HAS_TLSv1_3, reason="requires TLSv1.3"), + ), + ], +) +async def test_tcp_ssl_connect(tcp_address, ssl_min_version): + host, port = tcp_address + + server_certs = get_tls_certificates(cert_type=CertificateType.server) + + conn = SSLConnection( + host=host, + port=port, + client_name=_CLIENT_NAME, + ssl_ca_certs=server_certs.ca_certfile, + socket_timeout=10, + ssl_min_version=ssl_min_version, + ) + await _assert_connect( + conn, tcp_address, certfile=server_certs.certfile, keyfile=server_certs.keyfile + ) + await conn.disconnect() + + +@pytest.mark.ssl +@pytest.mark.skipif(not ssl.HAS_TLSv1_3, reason="requires TLSv1.3") +async def test_tcp_ssl_version_mismatch(tcp_address): + host, port = tcp_address + certfile, keyfile, _ = get_tls_certificates(cert_type=CertificateType.server) + conn = SSLConnection( + host=host, + port=port, + client_name=_CLIENT_NAME, + ssl_ca_certs=certfile, + socket_timeout=1, + ssl_min_version=ssl.TLSVersion.TLSv1_3, + ) + with pytest.raises(ConnectionError): + await _assert_connect( + conn, + tcp_address, + certfile=certfile, + keyfile=keyfile, + maximum_ssl_version=ssl.TLSVersion.TLSv1_2, + ) + + await conn.disconnect() + + +async def _assert_connect( + conn, + server_address, + certfile=None, + keyfile=None, + minimum_ssl_version=ssl.TLSVersion.TLSv1_2, + maximum_ssl_version=ssl.TLSVersion.TLSv1_3, +): + stop_event = anyio.Event() + finished = anyio.Event() + + async def _handler(stream: ByteStream): + try: + async with stream: + return await _redis_request_handler(stream, stop_event) + finally: + finished.set() + + if isinstance(server_address, str): + listener = await anyio.create_unix_listener(path=server_address) + elif certfile: + host, port = server_address + context = ssl.create_default_context(ssl.Purpose.CLIENT_AUTH) + context.minimum_version = minimum_ssl_version + context.maximum_version = maximum_ssl_version + context.load_cert_chain(certfile=certfile, keyfile=keyfile) + tcp_listener = await anyio.create_tcp_listener(local_host=host, local_port=port) + listener = TLSListener( + tcp_listener, ssl_context=context, standard_compatible=False + ) + else: + host, port = server_address + listener = await anyio.create_tcp_listener(local_host=host, local_port=port) + + try: + async with listener, anyio.create_task_group() as tg: + tg.start_soon(listener.serve, _handler) + try: + await conn.connect() + await conn.disconnect() + except ConnectionError: + finished.set() + raise + finally: + stop_event.set() + await finished.wait() + tg.cancel_scope.cancel() + except ExceptionGroup as excgrp: + if len(excgrp.exceptions) == 1 and isinstance( + excgrp.exceptions[0], ConnectionError + ): + raise excgrp.exceptions[0] from None + + raise + + +async def _redis_request_handler(stream: ByteStream, stop_event: anyio.Event): + command = None + command_ptr = None + fragment_length = None + while not stop_event.is_set(): + try: + buffer = await stream.receive() + except anyio.EndOfStream: + break + + parts = re.split(_CMD_SEP, buffer) + for fragment in parts: + fragment = fragment.decode() + if not fragment: + continue + + if fragment.startswith("*") and command is None: + command = [None for _ in range(int(fragment[1:]))] + command_ptr = 0 + fragment_length = None + continue + + if fragment.startswith("$") and command[command_ptr] is None: + fragment_length = int(fragment[1:]) + continue + + assert len(fragment) == fragment_length + command[command_ptr] = fragment + command_ptr += 1 + + if command_ptr < len(command): + continue + + command = " ".join(command) + resp = _SUPPORTED_CMDS.get(command, _ERROR_RESP) + await stream.send(resp) + command = None diff --git a/tests/test_anyio/test_connection.py b/tests/test_anyio/test_connection.py new file mode 100644 index 0000000000..ccdccf1a1a --- /dev/null +++ b/tests/test_anyio/test_connection.py @@ -0,0 +1,520 @@ +import builtins +import gc +import socket +import types +from errno import ECONNREFUSED +from unittest import mock +from unittest.mock import AsyncMock, Mock, patch + +import anyio +import pytest +from anyio.abc import SocketStream +from anyio.streams.buffered import BufferedByteReceiveStream + +import redis +from redis.anyio._parsers import ( + _AnyIOHiredisParser, + _AnyIORESP2Parser, + _AnyIORESP3Parser, + _AnyIORESPBase, +) +from redis.anyio import ConnectionPool, Redis +from redis.anyio.connection import ( + Connection, + SSLConnection, + UnixDomainSocketConnection, + parse_url, +) +from redis.anyio.retry import Retry +from redis.anyio.utils import gather, wait_for_condition +from redis.backoff import NoBackoff +from redis.exceptions import ConnectionError, InvalidResponse, TimeoutError +from redis.utils import HIREDIS_AVAILABLE +from tests.conftest import skip_if_server_version_lt + +from .mocks import MockStream + +pytestmark = pytest.mark.anyio + + +@pytest.mark.onlynoncluster +async def test_invalid_response(create_redis, monkeypatch): + async with create_redis(single_connection_client=True) as r: + monkeypatch.setattr(r, "flushdb", AsyncMock()) + raw = b"x" + fake_stream = BufferedByteReceiveStream(MockStream(raw + b"\r\n")) + + parser: _AnyIORESPBase = r.connection._parser + + if isinstance(parser, _AnyIORESPBase): + exp_err = f"Protocol Error: {raw!r}" + else: + exp_err = f'Protocol error, got "{raw.decode()}" as reply type byte' + + with patch.object(parser, "_stream", fake_stream): + with pytest.raises(InvalidResponse, match=exp_err): + await parser.read_response() + + +@pytest.mark.onlynoncluster +async def test_single_connection(): + """Test that concurrent requests on a single client are synchronised.""" + r = Redis(single_connection_client=True) + + init_call_count = 0 + command_call_count = 0 + in_use = False + + class Retry_: + async def call_with_retry(self, _, __): + # If we remove the single-client lock, this error gets raised as two + # coroutines will be vying for the `in_use` flag due to the two + # asymmetric sleep calls + nonlocal command_call_count + nonlocal in_use + if in_use is True: + raise ValueError("Commands should be executed one at a time.") + in_use = True + await anyio.sleep(0.01) + command_call_count += 1 + await anyio.sleep(0.03) + in_use = False + return "foo" + + mock_conn = mock.AsyncMock(spec=Connection) + mock_conn.retry = Retry_() + + async def get_conn(): + # Validate only one client is created in single-client mode when + # concurrent requests are made + nonlocal init_call_count + await anyio.sleep(0.01) + init_call_count += 1 + return mock_conn + + with patch.object(r.connection_pool, "get_connection", get_conn): + with patch.object(r.connection_pool, "release"): + await gather(r.set("a", "b"), r.set("c", "d")) + + assert init_call_count == 1 + assert command_call_count == 2 + r.connection = None # it was a Mock + await r.aclose() + + +@skip_if_server_version_lt("4.0.0") +@pytest.mark.redismod +@pytest.mark.onlynoncluster +async def test_loading_external_modules(r): + def inner(): + pass + + r.load_external_module("myfuncname", inner) + assert getattr(r, "myfuncname") == inner + assert isinstance(getattr(r, "myfuncname"), types.FunctionType) + + # and call it + from redis.commands import RedisModuleCommands + + j = RedisModuleCommands.json + r.load_external_module("sometestfuncname", j) + + # d = {'hello': 'world!'} + # mod = j(r) + # mod.set("fookey", ".", d) + # assert mod.get('fookey') == d + + +async def test_socket_param_regression(r): + """A regression test for issue #1060""" + conn = UnixDomainSocketConnection() + _ = await conn.disconnect() is True + + +async def test_can_run_concurrent_commands(r): + if getattr(r, "connection", None) is not None: + # Concurrent commands are only supported on pooled or cluster connections + # since there is no synchronization on a single connection. + pytest.skip("pool only") + assert await r.ping() is True + assert all(await gather(*(r.ping() for _ in range(10)))) + + +async def test_connect_retry_on_timeout_error(connect_args): + """Test that the _connect function is retried in case of a timeout""" + conn = Connection( + retry_on_timeout=True, retry=Retry(NoBackoff(), 3), **connect_args + ) + origin_connect = conn._connect + conn._connect = mock.AsyncMock() + + async def mock_connect(): + # connect only on the last retry + if conn._connect.call_count <= 2: + raise socket.timeout + else: + return await origin_connect() + + conn._connect.side_effect = mock_connect + await conn.connect() + assert conn._connect.call_count == 3 + await conn.disconnect() + + +async def test_connect_without_retry_on_os_error(): + """Test that the _connect function is not being retried in case of a OSError""" + with patch.object(Connection, "_connect") as _connect: + _connect.side_effect = OSError("") + conn = Connection(retry_on_timeout=True, retry=Retry(NoBackoff(), 2)) + with pytest.raises(ConnectionError): + await conn.connect() + assert _connect.call_count == 1 + + +async def test_connect_timeout_error_without_retry(): + """Test that the _connect function is not being retried if retry_on_timeout is + set to False""" + conn = Connection(retry_on_timeout=False) + conn._connect = mock.AsyncMock() + conn._connect.side_effect = builtins.TimeoutError + + with pytest.raises(TimeoutError, match="Timeout connecting to server"): + await conn.connect() + assert conn._connect.call_count == 1 + + +@pytest.mark.onlynoncluster +async def test_connection_parse_response_resume(r: redis.Redis): + """ + This test verifies that the Connection parser, + be that PythonParser or HiredisParser, + can be interrupted at IO time and then resume parsing. + """ + conn = Connection(**r.connection_pool.connection_kwargs) + await conn.connect() + message = ( + b"*3\r\n$7\r\nmessage\r\n$8\r\nchannel1\r\n" + b"$25\r\nhi\r\nthere\r\n+how\r\nare\r\nyou\r\n" + ) + + conn._parser._stream = MockStream(message, interrupt_every=2) + for i in range(100): + try: + response = await conn.read_response(disconnect_on_error=False) + break + except MockStream.TestError: + pass + else: + pytest.fail("didn't receive a response") + + assert response + assert i > 0 + await conn.disconnect() + + +@pytest.mark.onlynoncluster +@pytest.mark.parametrize( + "parser_class", + [_AnyIORESP2Parser, _AnyIORESP3Parser, _AnyIOHiredisParser], + ids=["_AnyIORESP2Parser", "_AnyIORESP3Parser", "_AnyIOHiredisParser"], +) +async def test_connection_disconnect_race(parser_class, connect_args): + """ + This test reproduces the case in issue #2349 + where a connection is closed while the parser is reading to feed the + internal buffer.The stream `read()` will succeed, but when it returns, + another task has already called `disconnect()` and is waiting for + close to finish. When we attempts to feed the buffer, we will fail + since the buffer is no longer there. + + This test verifies that a read in progress can finish even + if the `disconnect()` method is called. + """ + if parser_class == _AnyIOHiredisParser and not HIREDIS_AVAILABLE: + pytest.skip("Hiredis not available") + + connect_args["parser_class"] = parser_class + + conn = Connection(**connect_args) + + cond = anyio.Condition() + # 0 == initial + # 1 == reader is reading + # 2 == closer has closed and is waiting for close to finish + state = 0 + + # Mock read function, which wait for a close to happen before returning + # Can either be invoked as two `read()` calls (HiredisParser) + # or as a `readline()` followed by `readexact()` (PythonParser) + chunks = [b"$13\r\n", b"Hello, World!\r\n"] + + async def receive(_=None): + nonlocal state + async with cond: + if state == 0: + state = 1 # we are reading + cond.notify() + # wait until the closing task has done + await wait_for_condition(cond, lambda: state == 2) + return chunks.pop(0) + + # function closes the connection while reader is still blocked reading + async def do_close(): + nonlocal state + async with cond: + await wait_for_condition(cond, lambda: state == 1) + state = 2 + cond.notify() + await conn.disconnect() + + async def do_read(): + return await conn.read_response() + + stream = Mock(spec=SocketStream) + stream.receive.side_effect = receive + + # get dummy stream objects for the connection + with patch.object(anyio, "connect_tcp", return_value=stream): + # disable the initial version handshake + with patch.multiple( + conn, send_command=mock.DEFAULT, read_response=mock.DEFAULT + ): + await conn.connect() + + vals = await gather(do_read(), do_close()) + assert vals == [b"Hello, World!", None] + + +@pytest.mark.onlynoncluster +def test_create_single_connection_client_from_url(): + client = Redis.from_url("redis://localhost:6379/0?", single_connection_client=True) + assert client.single_connection_client is True + + +@pytest.mark.parametrize("from_url", (True, False), ids=("from_url", "from_args")) +async def test_pool_auto_close(request, from_url): + """Verify that basic Redis instances have auto_close_connection_pool set to True""" + + url: str = request.config.getoption("--redis-url") + url_args = parse_url(url) + + async def get_redis_connection(): + if from_url: + return Redis.from_url(url) + return Redis(**url_args) + + r1 = await get_redis_connection() + assert r1.auto_close_connection_pool is True + await r1.aclose() + + +async def test_close_is_aclose(request): + """Verify close() calls aclose()""" + calls = 0 + + async def mock_aclose(self): + nonlocal calls + calls += 1 + + url: str = request.config.getoption("--redis-url") + r1 = await Redis.from_url(url) + with patch.object(r1, "aclose", mock_aclose): + with pytest.deprecated_call(): + await r1.close() + assert calls == 1 + + with pytest.deprecated_call(): + await r1.close() + + +async def test_pool_from_url_deprecation(request): + url: str = request.config.getoption("--redis-url") + + with pytest.deprecated_call(): + return Redis.from_url(url, auto_close_connection_pool=False) + + +async def test_pool_auto_close_disable(request): + """Verify that auto_close_connection_pool can be disabled (deprecated)""" + + url: str = request.config.getoption("--redis-url") + url_args = parse_url(url) + + async def get_redis_connection(): + url_args["auto_close_connection_pool"] = False + with pytest.deprecated_call(): + return Redis(**url_args) + + r1 = await get_redis_connection() + assert r1.auto_close_connection_pool is False + await r1.connection_pool.disconnect() + await r1.aclose() + + +@pytest.mark.parametrize("from_url", (True, False), ids=("from_url", "from_args")) +async def test_redis_connection_pool(request, from_url): + """Verify that basic Redis instances using `connection_pool` + have auto_close_connection_pool set to False""" + + url: str = request.config.getoption("--redis-url") + url_args = parse_url(url) + + pool = None + + async def get_redis_connection(): + nonlocal pool + if from_url: + pool = ConnectionPool.from_url(url) + else: + pool = ConnectionPool(**url_args) + return Redis(connection_pool=pool) + + called = 0 + + async def mock_disconnect(_): + nonlocal called + called += 1 + + with patch.object(ConnectionPool, "disconnect", mock_disconnect): + async with await get_redis_connection() as r1: + assert r1.auto_close_connection_pool is False + + assert called == 0 + await pool.disconnect() + + +@pytest.mark.parametrize("from_url", (True, False), ids=("from_url", "from_args")) +async def test_redis_from_pool(request, from_url): + """Verify that basic Redis instances created using `from_pool()` + have auto_close_connection_pool set to True""" + + url: str = request.config.getoption("--redis-url") + url_args = parse_url(url) + + pool = None + + async def get_redis_connection(): + nonlocal pool + if from_url: + pool = ConnectionPool.from_url(url) + else: + pool = ConnectionPool(**url_args) + return Redis.from_pool(pool) + + called = 0 + + async def mock_disconnect(_): + nonlocal called + called += 1 + + with patch.object(ConnectionPool, "disconnect", mock_disconnect): + async with await get_redis_connection() as r1: + assert r1.auto_close_connection_pool is True + + assert called == 1 + await pool.disconnect() + + +@pytest.mark.parametrize("auto_close", (True, False)) +async def test_redis_pool_auto_close_arg(request, auto_close): + """test that redis instance where pool is provided have + auto_close_connection_pool set to False, regardless of arg""" + + url: str = request.config.getoption("--redis-url") + pool = ConnectionPool.from_url(url) + + async def get_redis_connection(): + with pytest.deprecated_call(): + client = Redis(connection_pool=pool, auto_close_connection_pool=auto_close) + return client + + called = 0 + + async def mock_disconnect(_): + nonlocal called + called += 1 + + with patch.object(ConnectionPool, "disconnect", mock_disconnect): + async with await get_redis_connection() as r1: + assert r1.auto_close_connection_pool is False + + assert called == 0 + await pool.disconnect() + + +async def test_client_garbage_collection(request): + """ + Test that a Redis client will call _close() on any + connection that it holds at time of destruction + """ + + url: str = request.config.getoption("--redis-url") + pool = ConnectionPool.from_url(url) + + # create a client with a connection from the pool + client = Redis(connection_pool=pool, single_connection_client=True) + await client.initialize() + with pytest.warns(ResourceWarning, match="^Unclosed client session"): + await pool.aclose() + del client + gc.collect() + + +async def test_connection_garbage_collection(request): + """ + Test that a Connection object will emit a ResourceWarning when not closed properly. + """ + + url: str = request.config.getoption("--redis-url") + conn = ConnectionPool.from_url(url).make_connection() + conn._stream = Mock(spec=SocketStream) + with pytest.warns(ResourceWarning): + del conn + gc.collect() + + +@pytest.mark.parametrize( + "conn, error, expected_message", + [ + (SSLConnection(), OSError(), "Error connecting to localhost:6379."), + (SSLConnection(), OSError(12), "Error 12 connecting to localhost:6379."), + ( + SSLConnection(), + OSError(12, "Some Error"), + "Error 12 connecting to localhost:6379. Some Error.", + ), + ( + UnixDomainSocketConnection(path="unix:///tmp/redis.sock"), + OSError(), + "Error connecting to unix:///tmp/redis.sock.", + ), + ( + UnixDomainSocketConnection(path="unix:///tmp/redis.sock"), + OSError(12), + "Error 12 connecting to unix:///tmp/redis.sock.", + ), + ( + UnixDomainSocketConnection(path="unix:///tmp/redis.sock"), + OSError(12, "Some Error"), + "Error 12 connecting to unix:///tmp/redis.sock. Some Error.", + ), + ], +) +async def test_format_error_message(conn, error, expected_message): + """Test that the _error_message function formats errors correctly""" + error_message = conn._error_message(error) + assert error_message == expected_message + + +async def test_network_connection_failure(): + exp_err = rf"^Error {ECONNREFUSED} connecting to 127.0.0.1:9999.(.+)$" + with pytest.raises(ConnectionError, match=exp_err): + redis = Redis(host="127.0.0.1", port=9999) + await redis.set("a", "b") + + +async def test_unix_socket_connection_failure(): + exp_err = "Error 2 connecting to unix:///tmp/a.sock. No such file or directory." + with pytest.raises(ConnectionError, match=exp_err): + redis = Redis(unix_socket_path="unix:///tmp/a.sock") + await redis.set("a", "b") diff --git a/tests/test_anyio/test_connection_pool.py b/tests/test_anyio/test_connection_pool.py new file mode 100644 index 0000000000..16adc077a1 --- /dev/null +++ b/tests/test_anyio/test_connection_pool.py @@ -0,0 +1,894 @@ +import re +from contextlib import asynccontextmanager +from unittest import mock + +import anyio +import pytest +import redis.anyio as redis +from redis.anyio.compat import aclosing +from redis.anyio.connection import Connection, to_bool +from redis.anyio.utils import gather +from redis.auth.token import TokenInterface +from tests.conftest import skip_if_redis_enterprise, skip_if_server_version_lt + +from .test_pubsub import wait_for_message + +pytestmark = pytest.mark.anyio + + +@pytest.mark.onlynoncluster +class TestRedisAutoReleaseConnectionPool: + @pytest.fixture + async def r(self, create_redis) -> redis.Redis: + """This is necessary since r and r2 create ConnectionPools behind the scenes""" + async with create_redis() as client: + client.auto_close_connection_pool = True + yield client + + @staticmethod + def get_total_connected_connections(pool): + return len(pool._available_connections) + len(pool._in_use_connections) + + @staticmethod + async def create_two_conn(r: redis.Redis): + if not r.single_connection_client: # Single already initialized connection + r.connection = await r.connection_pool.get_connection() + return await r.connection_pool.get_connection() + + @staticmethod + def has_no_connected_connections(pool: redis.ConnectionPool): + return not any( + x.is_connected + for x in pool._available_connections + list(pool._in_use_connections) + ) + + async def test_auto_disconnect_redis_created_pool(self, r: redis.Redis): + new_conn = await self.create_two_conn(r) + assert new_conn != r.connection + assert self.get_total_connected_connections(r.connection_pool) == 2 + await r.aclose() + assert self.has_no_connected_connections(r.connection_pool) + + async def test_do_not_auto_disconnect_redis_created_pool(self, r2: redis.Redis): + assert r2.auto_close_connection_pool is False, ( + "The connection pool should not be disconnected as a manually created " + "connection pool was passed in in conftest.py" + ) + new_conn = await self.create_two_conn(r2) + assert self.get_total_connected_connections(r2.connection_pool) == 2 + await r2.aclose() + assert r2.connection_pool._in_use_connections == {new_conn} + assert new_conn.is_connected + assert len(r2.connection_pool._available_connections) == 1 + assert r2.connection_pool._available_connections[0].is_connected + + async def test_auto_release_override_true_manual_created_pool(self, r: redis.Redis): + assert r.auto_close_connection_pool is True, "This is from the class fixture" + await self.create_two_conn(r) + await r.aclose() + assert self.get_total_connected_connections(r.connection_pool) == 2, ( + "The connection pool should not be disconnected as a manually created " + "connection pool was passed in in conftest.py" + ) + assert self.has_no_connected_connections(r.connection_pool) + + @pytest.mark.parametrize("auto_close_conn_pool", [True, False]) + async def test_close_override(self, r: redis.Redis, auto_close_conn_pool): + r.auto_close_connection_pool = auto_close_conn_pool + await self.create_two_conn(r) + await r.aclose(close_connection_pool=True) + assert self.has_no_connected_connections(r.connection_pool) + + @pytest.mark.parametrize("auto_close_conn_pool", [True, False]) + async def test_negate_auto_close_client_pool( + self, r: redis.Redis, auto_close_conn_pool + ): + r.auto_close_connection_pool = auto_close_conn_pool + new_conn = await self.create_two_conn(r) + await r.aclose(close_connection_pool=False) + assert not self.has_no_connected_connections(r.connection_pool) + assert r.connection_pool._in_use_connections == {new_conn} + assert r.connection_pool._available_connections[0].is_connected + assert self.get_total_connected_connections(r.connection_pool) == 2 + + +class DummyConnection(Connection): + description_format = "DummyConnection<>" + + def __init__(self, **kwargs): + self.kwargs = kwargs + + def repr_pieces(self): + return [("id", id(self)), ("kwargs", self.kwargs)] + + async def connect(self): + pass + + async def disconnect(self, nowait: bool = False) -> None: + pass + + async def can_read_destructive(self, timeout: float = 0): + return False + + def set_re_auth_token(self, token: TokenInterface): + pass + + async def re_auth(self): + pass + + +class TestConnectionPool: + @asynccontextmanager + async def get_pool( + self, + connection_kwargs=None, + max_connections=None, + connection_class=redis.Connection, + ): + connection_kwargs = connection_kwargs or {} + pool = redis.ConnectionPool( + connection_class=connection_class, + max_connections=max_connections, + **connection_kwargs, + ) + try: + yield pool + finally: + await pool.disconnect(inuse_connections=True) + + async def test_connection_creation(self): + connection_kwargs = {"foo": "bar", "biz": "baz"} + async with self.get_pool( + connection_kwargs=connection_kwargs, connection_class=DummyConnection + ) as pool: + connection = await pool.get_connection() + assert isinstance(connection, DummyConnection) + assert connection.kwargs == connection_kwargs + + async def test_aclosing(self): + connection_kwargs = {"foo": "bar", "biz": "baz"} + pool = redis.ConnectionPool( + connection_class=DummyConnection, + max_connections=None, + **connection_kwargs, + ) + async with aclosing(pool): + pass + + async def test_multiple_connections(self, master_host): + connection_kwargs = {"host": master_host[0]} + async with self.get_pool(connection_kwargs=connection_kwargs) as pool: + c1 = await pool.get_connection() + c2 = await pool.get_connection() + assert c1 != c2 + + async def test_max_connections(self, master_host): + connection_kwargs = {"host": master_host[0]} + async with self.get_pool( + max_connections=2, connection_kwargs=connection_kwargs + ) as pool: + await pool.get_connection() + await pool.get_connection() + with pytest.raises(redis.ConnectionError): + await pool.get_connection() + + async def test_reuse_previously_released_connection(self, master_host): + connection_kwargs = {"host": master_host[0]} + async with self.get_pool(connection_kwargs=connection_kwargs) as pool: + c1 = await pool.get_connection() + await pool.release(c1) + c2 = await pool.get_connection() + assert c1 == c2 + + async def test_repr_contains_db_info_tcp(self): + connection_kwargs = { + "host": "localhost", + "port": 6379, + "db": 1, + "client_name": "test-client", + } + async with self.get_pool( + connection_kwargs=connection_kwargs, connection_class=redis.Connection + ) as pool: + expected = "host=localhost,port=6379,db=1,client_name=test-client" + assert expected in repr(pool) + + async def test_repr_contains_db_info_unix(self): + connection_kwargs = {"path": "/abc", "db": 1, "client_name": "test-client"} + async with self.get_pool( + connection_kwargs=connection_kwargs, + connection_class=redis.UnixDomainSocketConnection, + ) as pool: + expected = "path=/abc,db=1,client_name=test-client" + assert expected in repr(pool) + + +class TestBlockingConnectionPool: + @asynccontextmanager + async def get_pool(self, connection_kwargs=None, max_connections=10, timeout=20): + connection_kwargs = connection_kwargs or {} + pool = redis.BlockingConnectionPool( + connection_class=DummyConnection, + max_connections=max_connections, + timeout=timeout, + **connection_kwargs, + ) + try: + yield pool + finally: + await pool.disconnect(inuse_connections=True) + + async def test_connection_creation(self, master_host): + connection_kwargs = { + "foo": "bar", + "biz": "baz", + "host": master_host[0], + "port": master_host[1], + } + async with self.get_pool(connection_kwargs=connection_kwargs) as pool: + connection = await pool.get_connection() + assert isinstance(connection, DummyConnection) + assert connection.kwargs == connection_kwargs + + async def test_disconnect(self, master_host): + """A regression test for #1047""" + connection_kwargs = { + "foo": "bar", + "biz": "baz", + "host": master_host[0], + "port": master_host[1], + } + async with self.get_pool(connection_kwargs=connection_kwargs) as pool: + await pool.get_connection() + await pool.disconnect() + + async def test_multiple_connections(self, master_host): + connection_kwargs = {"host": master_host[0], "port": master_host[1]} + async with self.get_pool(connection_kwargs=connection_kwargs) as pool: + c1 = await pool.get_connection() + c2 = await pool.get_connection() + assert c1 != c2 + + async def test_connection_pool_blocks_until_timeout(self, master_host): + """When out of connections, block for timeout seconds, then raise""" + connection_kwargs = {"host": master_host[0]} + async with self.get_pool( + max_connections=1, timeout=0.1, connection_kwargs=connection_kwargs + ) as pool: + c1 = await pool.get_connection() + + start = anyio.current_time() + with pytest.raises(redis.ConnectionError): + await pool.get_connection() + + # we should have waited at least some period of time + assert anyio.current_time() - start >= 0.05 + await c1.disconnect() + + async def test_connection_pool_blocks_until_conn_available(self, master_host): + """ + When out of connections, block until another connection is released + to the pool + """ + connection_kwargs = {"host": master_host[0], "port": master_host[1]} + async with self.get_pool( + max_connections=1, timeout=2, connection_kwargs=connection_kwargs + ) as pool: + c1 = await pool.get_connection() + + async def target(): + await anyio.sleep(0.1) + await pool.release(c1) + + start = anyio.current_time() + await gather(target(), pool.get_connection()) + stop = anyio.current_time() + assert (stop - start) <= 0.2 + + async def test_reuse_previously_released_connection(self, master_host): + connection_kwargs = {"host": master_host[0]} + async with self.get_pool(connection_kwargs=connection_kwargs) as pool: + c1 = await pool.get_connection() + await pool.release(c1) + c2 = await pool.get_connection() + assert c1 == c2 + + def test_repr_contains_db_info_tcp(self): + pool = redis.ConnectionPool( + host="localhost", port=6379, client_name="test-client" + ) + expected = "host=localhost,port=6379,db=0,client_name=test-client" + assert expected in repr(pool) + + def test_repr_contains_db_info_unix(self): + pool = redis.ConnectionPool( + connection_class=redis.UnixDomainSocketConnection, + path="abc", + client_name="test-client", + ) + expected = "path=abc,db=0,client_name=test-client" + assert expected in repr(pool) + + +class TestConnectionPoolURLParsing: + def test_hostname(self): + pool = redis.ConnectionPool.from_url("redis://my.host") + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "my.host"} + + def test_quoted_hostname(self): + pool = redis.ConnectionPool.from_url("redis://my %2F host %2B%3D+") + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "my / host +=+"} + + def test_port(self): + pool = redis.ConnectionPool.from_url("redis://localhost:6380") + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "localhost", "port": 6380} + + @skip_if_server_version_lt("6.0.0") + def test_username(self): + pool = redis.ConnectionPool.from_url("redis://myuser:@localhost") + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "localhost", "username": "myuser"} + + @skip_if_server_version_lt("6.0.0") + def test_quoted_username(self): + pool = redis.ConnectionPool.from_url( + "redis://%2Fmyuser%2F%2B name%3D%24+:@localhost" + ) + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == { + "host": "localhost", + "username": "/myuser/+ name=$+", + } + + def test_password(self): + pool = redis.ConnectionPool.from_url("redis://:mypassword@localhost") + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "localhost", "password": "mypassword"} + + def test_quoted_password(self): + pool = redis.ConnectionPool.from_url( + "redis://:%2Fmypass%2F%2B word%3D%24+@localhost" + ) + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == { + "host": "localhost", + "password": "/mypass/+ word=$+", + } + + @skip_if_server_version_lt("6.0.0") + def test_username_and_password(self): + pool = redis.ConnectionPool.from_url("redis://myuser:mypass@localhost") + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == { + "host": "localhost", + "username": "myuser", + "password": "mypass", + } + + def test_db_as_argument(self): + pool = redis.ConnectionPool.from_url("redis://localhost", db=1) + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "localhost", "db": 1} + + def test_db_in_path(self): + pool = redis.ConnectionPool.from_url("redis://localhost/2", db=1) + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "localhost", "db": 2} + + def test_db_in_querystring(self): + pool = redis.ConnectionPool.from_url("redis://localhost/2?db=3", db=1) + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "localhost", "db": 3} + + def test_extra_typed_querystring_options(self): + pool = redis.ConnectionPool.from_url( + "redis://localhost/2?socket_timeout=20&socket_connect_timeout=10" + "&socket_keepalive=&retry_on_timeout=Yes&max_connections=10" + ) + + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == { + "host": "localhost", + "db": 2, + "socket_timeout": 20.0, + "socket_connect_timeout": 10.0, + "retry_on_timeout": True, + } + assert pool.max_connections == 10 + + def test_boolean_parsing(self): + for expected, value in ( + (None, None), + (None, ""), + (False, 0), + (False, "0"), + (False, "f"), + (False, "F"), + (False, "False"), + (False, "n"), + (False, "N"), + (False, "No"), + (True, 1), + (True, "1"), + (True, "y"), + (True, "Y"), + (True, "Yes"), + ): + assert expected is to_bool(value) + + def test_client_name_in_querystring(self): + pool = redis.ConnectionPool.from_url("redis://location?client_name=test-client") + assert pool.connection_kwargs["client_name"] == "test-client" + + def test_invalid_extra_typed_querystring_options(self): + with pytest.raises(ValueError): + redis.ConnectionPool.from_url( + "redis://localhost/2?socket_timeout=_&socket_connect_timeout=abc" + ) + + def test_extra_querystring_options(self): + pool = redis.ConnectionPool.from_url("redis://localhost?a=1&b=2") + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == {"host": "localhost", "a": "1", "b": "2"} + + def test_calling_from_subclass_returns_correct_instance(self): + pool = redis.BlockingConnectionPool.from_url("redis://localhost") + assert isinstance(pool, redis.BlockingConnectionPool) + + def test_client_creates_connection_pool(self): + r = redis.Redis.from_url("redis://myhost") + assert r.connection_pool.connection_class == redis.Connection + assert r.connection_pool.connection_kwargs == {"host": "myhost"} + + def test_invalid_scheme_raises_error(self): + with pytest.raises(ValueError) as cm: + redis.ConnectionPool.from_url("localhost") + assert str(cm.value) == ( + "Redis URL must specify one of the following schemes " + "(redis://, rediss://, unix://)" + ) + + +class TestBlockingConnectionPoolURLParsing: + def test_extra_typed_querystring_options(self): + pool = redis.BlockingConnectionPool.from_url( + "redis://localhost/2?socket_timeout=20&socket_connect_timeout=10" + "&socket_keepalive=&retry_on_timeout=Yes&max_connections=10&timeout=13.37" + ) + + assert pool.connection_class == redis.Connection + assert pool.connection_kwargs == { + "host": "localhost", + "db": 2, + "socket_timeout": 20.0, + "socket_connect_timeout": 10.0, + "retry_on_timeout": True, + } + assert pool.max_connections == 10 + assert pool.timeout == 13.37 + + def test_invalid_extra_typed_querystring_options(self): + with pytest.raises(ValueError): + redis.BlockingConnectionPool.from_url( + "redis://localhost/2?timeout=_not_a_float_" + ) + + +class TestConnectionPoolUnixSocketURLParsing: + def test_defaults(self): + pool = redis.ConnectionPool.from_url("unix:///socket") + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == {"path": "/socket"} + + @skip_if_server_version_lt("6.0.0") + def test_username(self): + pool = redis.ConnectionPool.from_url("unix://myuser:@/socket") + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == {"path": "/socket", "username": "myuser"} + + @skip_if_server_version_lt("6.0.0") + def test_quoted_username(self): + pool = redis.ConnectionPool.from_url( + "unix://%2Fmyuser%2F%2B name%3D%24+:@/socket" + ) + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == { + "path": "/socket", + "username": "/myuser/+ name=$+", + } + + def test_password(self): + pool = redis.ConnectionPool.from_url("unix://:mypassword@/socket") + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == {"path": "/socket", "password": "mypassword"} + + def test_quoted_password(self): + pool = redis.ConnectionPool.from_url( + "unix://:%2Fmypass%2F%2B word%3D%24+@/socket" + ) + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == { + "path": "/socket", + "password": "/mypass/+ word=$+", + } + + def test_quoted_path(self): + pool = redis.ConnectionPool.from_url( + "unix://:mypassword@/my%2Fpath%2Fto%2F..%2F+_%2B%3D%24ocket" + ) + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == { + "path": "/my/path/to/../+_+=$ocket", + "password": "mypassword", + } + + def test_db_as_argument(self): + pool = redis.ConnectionPool.from_url("unix:///socket", db=1) + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == {"path": "/socket", "db": 1} + + def test_db_in_querystring(self): + pool = redis.ConnectionPool.from_url("unix:///socket?db=2", db=1) + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == {"path": "/socket", "db": 2} + + def test_client_name_in_querystring(self): + pool = redis.ConnectionPool.from_url("redis://location?client_name=test-client") + assert pool.connection_kwargs["client_name"] == "test-client" + + def test_extra_querystring_options(self): + pool = redis.ConnectionPool.from_url("unix:///socket?a=1&b=2") + assert pool.connection_class == redis.UnixDomainSocketConnection + assert pool.connection_kwargs == {"path": "/socket", "a": "1", "b": "2"} + + +class TestSSLConnectionURLParsing: + def test_host(self): + pool = redis.ConnectionPool.from_url("rediss://my.host") + assert pool.connection_class == redis.SSLConnection + assert pool.connection_kwargs == {"host": "my.host"} + + def test_cert_reqs_options(self): + import ssl + + class DummyConnectionPool(redis.ConnectionPool): + def get_connection(self): + return self.make_connection() + + pool = DummyConnectionPool.from_url("rediss://?ssl_cert_reqs=none") + assert pool.get_connection().cert_reqs == ssl.CERT_NONE + + pool = DummyConnectionPool.from_url("rediss://?ssl_cert_reqs=optional") + assert pool.get_connection().cert_reqs == ssl.CERT_OPTIONAL + + pool = DummyConnectionPool.from_url("rediss://?ssl_cert_reqs=required") + assert pool.get_connection().cert_reqs == ssl.CERT_REQUIRED + + pool = DummyConnectionPool.from_url("rediss://?ssl_check_hostname=False") + assert pool.get_connection().check_hostname is False + + pool = DummyConnectionPool.from_url("rediss://?ssl_check_hostname=True") + assert pool.get_connection().check_hostname is True + + +class TestConnection: + async def test_on_connect_error(self): + """ + An error in Connection.on_connect should disconnect from the server + see for details: https://github.com/andymccurdy/redis-py/issues/368 + """ + # this assumes the Redis server being tested against doesn't have + # 9999 databases ;) + bad_connection = redis.Redis(db=9999) + # an error should be raised on connect + with pytest.raises(redis.RedisError): + await bad_connection.info() + pool = bad_connection.connection_pool + assert len(pool._available_connections) == 1 + assert not pool._available_connections[0]._stream + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("2.8.8") + @skip_if_redis_enterprise() + async def test_busy_loading_disconnects_socket(self, r): + """ + If Redis raises a LOADING error, the connection should be + disconnected and a BusyLoadingError raised + """ + with pytest.raises(redis.BusyLoadingError): + await r.execute_command("DEBUG", "ERROR", "LOADING fake message") + if r.connection: + assert not r.connection._stream + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("2.8.8") + @skip_if_redis_enterprise() + async def test_busy_loading_from_pipeline_immediate_command(self, r): + """ + BusyLoadingErrors should raise from Pipelines that execute a + command immediately, like WATCH does. + """ + pipe = r.pipeline() + with pytest.raises(redis.BusyLoadingError): + await pipe.immediate_execute_command( + "DEBUG", "ERROR", "LOADING fake message" + ) + pool = r.connection_pool + assert not pipe.connection + assert len(pool._available_connections) == 1 + assert not pool._available_connections[0]._stream + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("2.8.8") + @skip_if_redis_enterprise() + async def test_busy_loading_from_pipeline(self, r): + """ + BusyLoadingErrors should be raised from a pipeline execution + regardless of the raise_on_error flag. + """ + pipe = r.pipeline() + pipe.execute_command("DEBUG", "ERROR", "LOADING fake message") + with pytest.raises(redis.BusyLoadingError): + await pipe.execute() + pool = r.connection_pool + assert not pipe.connection + assert len(pool._available_connections) == 1 + assert not pool._available_connections[0]._stream + + @skip_if_server_version_lt("2.8.8") + @skip_if_redis_enterprise() + async def test_read_only_error(self, r): + """READONLY errors get turned into ReadOnlyError exceptions""" + with pytest.raises(redis.ReadOnlyError): + await r.execute_command("DEBUG", "ERROR", "READONLY blah blah") + + @skip_if_redis_enterprise() + async def test_oom_error(self, r): + """OOM errors get turned into OutOfMemoryError exceptions""" + with pytest.raises(redis.OutOfMemoryError): + # note: don't use the DEBUG OOM command since it's not the same + # as the db being full + await r.execute_command("DEBUG", "ERROR", "OOM blah blah") + + def test_connect_from_url_tcp(self): + connection = redis.Redis.from_url("redis://localhost") + pool = connection.connection_pool + + assert re.match( + r"< .*?([^\.]+) \( < .*?([^\.]+) \( (.+) \) > \) >", repr(pool), re.VERBOSE + ).groups() == ( + "ConnectionPool", + "Connection", + "host=localhost,port=6379,db=0", + ) + + def test_connect_from_url_unix(self): + connection = redis.Redis.from_url("unix:///path/to/socket") + pool = connection.connection_pool + + assert re.match( + r"< .*?([^\.]+) \( < .*?([^\.]+) \( (.+) \) > \) >", repr(pool), re.VERBOSE + ).groups() == ( + "ConnectionPool", + "UnixDomainSocketConnection", + "path=/path/to/socket,db=0", + ) + + @skip_if_redis_enterprise() + async def test_connect_no_auth_supplied_when_required(self, r): + """ + AuthenticationError should be raised when the server requires a + password but one isn't supplied. + """ + with pytest.raises(redis.AuthenticationError): + await r.execute_command( + "DEBUG", "ERROR", "ERR Client sent AUTH, but no password is set" + ) + + @skip_if_redis_enterprise() + async def test_connect_invalid_password_supplied(self, r): + """AuthenticationError should be raised when sending the wrong password""" + with pytest.raises(redis.AuthenticationError): + await r.execute_command("DEBUG", "ERROR", "ERR invalid password") + + +@pytest.mark.onlynoncluster +class TestMultiConnectionClient: + @pytest.fixture + async def r(self, create_redis, server): + redis = await create_redis(single_connection_client=False) + yield redis + await redis.flushall() + + +@pytest.mark.onlynoncluster +@pytest.mark.xfail(strict=False) +class TestHealthCheck: + interval = 60 + + @pytest.fixture + async def r(self, create_redis): + redis = await create_redis(health_check_interval=self.interval) + yield redis + await redis.flushall() + + def assert_interval_advanced(self, connection): + diff = connection.next_health_check - anyio.current_time() + assert self.interval >= diff > (self.interval - 1) + + async def test_health_check_runs(self, r): + if r.connection: + r.connection.next_health_check = anyio.current_time() - 1 + await r.connection.check_health() + self.assert_interval_advanced(r.connection) + + async def test_arbitrary_command_invokes_health_check(self, r): + # invoke a command to make sure the connection is entirely setup + if r.connection: + await r.get("foo") + r.connection.next_health_check = anyio.current_time() + with mock.patch.object( + r.connection, "send_command", wraps=r.connection.send_command + ) as m: + await r.get("foo") + m.assert_called_with("PING", check_health=False) + + self.assert_interval_advanced(r.connection) + + async def test_arbitrary_command_advances_next_health_check(self, r): + if r.connection: + await r.get("foo") + next_health_check = r.connection.next_health_check + # ensure that the event loop's `time()` advances a bit + await anyio.sleep(0.001) + await r.get("foo") + assert next_health_check < r.connection.next_health_check + + async def test_health_check_not_invoked_within_interval(self, r): + if r.connection: + await r.get("foo") + with mock.patch.object( + r.connection, "send_command", wraps=r.connection.send_command + ) as m: + await r.get("foo") + ping_call_spec = (("PING",), {"check_health": False}) + assert ping_call_spec not in m.call_args_list + + async def test_health_check_in_pipeline(self, r): + async with r.pipeline(transaction=False) as pipe: + pipe.connection = await pipe.connection_pool.get_connection() + pipe.connection.next_health_check = 0 + with mock.patch.object( + pipe.connection, "send_command", wraps=pipe.connection.send_command + ) as m: + responses = await pipe.set("foo", "bar").get("foo").execute() + m.assert_any_call("PING", check_health=False) + assert responses == [True, b"bar"] + + async def test_health_check_in_transaction(self, r): + async with r.pipeline(transaction=True) as pipe: + pipe.connection = await pipe.connection_pool.get_connection() + pipe.connection.next_health_check = 0 + with mock.patch.object( + pipe.connection, "send_command", wraps=pipe.connection.send_command + ) as m: + responses = await pipe.set("foo", "bar").get("foo").execute() + m.assert_any_call("PING", check_health=False) + assert responses == [True, b"bar"] + + async def test_health_check_in_watched_pipeline(self, r): + await r.set("foo", "bar") + async with r.pipeline(transaction=False) as pipe: + pipe.connection = await pipe.connection_pool.get_connection() + pipe.connection.next_health_check = 0 + with mock.patch.object( + pipe.connection, "send_command", wraps=pipe.connection.send_command + ) as m: + await pipe.watch("foo") + # the health check should be called when watching + m.assert_called_with("PING", check_health=False) + self.assert_interval_advanced(pipe.connection) + assert await pipe.get("foo") == b"bar" + + # reset the mock to clear the call list and schedule another + # health check + m.reset_mock() + pipe.connection.next_health_check = 0 + + pipe.multi() + responses = await pipe.set("foo", "not-bar").get("foo").execute() + assert responses == [True, b"not-bar"] + m.assert_any_call("PING", check_health=False) + + async def test_health_check_in_pubsub_before_subscribe(self, r): + """A health check happens before the first [p]subscribe""" + p = r.pubsub() + p.connection = await p.connection_pool.get_connection() + p.connection.next_health_check = 0 + with mock.patch.object( + p.connection, "send_command", wraps=p.connection.send_command + ) as m: + assert not p.subscribed + await p.subscribe("foo") + # the connection is not yet in pubsub mode, so the normal + # ping/pong within connection.send_command should check + # the health of the connection + m.assert_any_call("PING", check_health=False) + self.assert_interval_advanced(p.connection) + + subscribe_message = await wait_for_message(p) + assert subscribe_message["type"] == "subscribe" + + async def test_health_check_in_pubsub_after_subscribed(self, r): + """ + Pubsub can handle a new subscribe when it's time to check the + connection health + """ + p = r.pubsub() + p.connection = await p.connection_pool.get_connection() + p.connection.next_health_check = 0 + with mock.patch.object( + p.connection, "send_command", wraps=p.connection.send_command + ) as m: + await p.subscribe("foo") + subscribe_message = await wait_for_message(p) + assert subscribe_message["type"] == "subscribe" + self.assert_interval_advanced(p.connection) + # because we weren't subscribed when sending the subscribe + # message to 'foo', the connection's standard check_health ran + # prior to subscribing. + m.assert_any_call("PING", check_health=False) + + p.connection.next_health_check = 0 + m.reset_mock() + + await p.subscribe("bar") + # the second subscribe issues exactly only command (the subscribe) + # and the health check is not invoked + m.assert_called_once_with("SUBSCRIBE", "bar", check_health=False) + + # since no message has been read since the health check was + # reset, it should still be 0 + assert p.connection.next_health_check == 0 + + subscribe_message = await wait_for_message(p) + assert subscribe_message["type"] == "subscribe" + assert await wait_for_message(p) is None + # now that the connection is subscribed, the pubsub health + # check should have taken over and include the HEALTH_CHECK_MESSAGE + m.assert_any_call("PING", p.HEALTH_CHECK_MESSAGE, check_health=False) + self.assert_interval_advanced(p.connection) + + async def test_health_check_in_pubsub_poll(self, r): + """ + Polling a pubsub connection that's subscribed will regularly + check the connection's health. + """ + p = r.pubsub() + p.connection = await p.connection_pool.get_connection() + with mock.patch.object( + p.connection, "send_command", wraps=p.connection.send_command + ) as m: + await p.subscribe("foo") + subscribe_message = await wait_for_message(p) + assert subscribe_message["type"] == "subscribe" + self.assert_interval_advanced(p.connection) + + # polling the connection before the health check interval + # doesn't result in another health check + m.reset_mock() + next_health_check = p.connection.next_health_check + assert await wait_for_message(p) is None + assert p.connection.next_health_check == next_health_check + m.assert_not_called() + + # reset the health check and poll again + # we should not receive a pong message, but the next_health_check + # should be advanced + p.connection.next_health_check = 0 + assert await wait_for_message(p) is None + m.assert_called_with("PING", p.HEALTH_CHECK_MESSAGE, check_health=False) + self.assert_interval_advanced(p.connection) diff --git a/tests/test_anyio/test_credentials.py b/tests/test_anyio/test_credentials.py new file mode 100644 index 0000000000..8737652c17 --- /dev/null +++ b/tests/test_anyio/test_credentials.py @@ -0,0 +1,720 @@ +import functools +import random +import string +from typing import Optional, Tuple, Union + +import pytest + +import redis +from anyio import Lock as AsyncLock +from anyio import sleep as async_sleep +from mock.mock import Mock, call +from redis import AuthenticationError, DataError, ResponseError +from redis.anyio import Connection, ConnectionPool, Redis +from redis.anyio.retry import Retry +from redis.auth.err import RequestTokenErr +from redis.backoff import NoBackoff +from redis.credentials import CredentialProvider, UsernamePasswordCredentialProvider +from redis.exceptions import ConnectionError +from redis.utils import str_if_bytes +from tests.conftest import get_endpoint, skip_if_redis_enterprise +from tests.entraid_utils import AuthType +from tests.test_anyio.conftest import get_credential_provider + +try: + from redis_entraid.cred_provider import EntraIdCredentialsProvider +except ImportError: + EntraIdCredentialsProvider = None + +pytestmark = pytest.mark.anyio + + +@pytest.fixture +def endpoint(request): + endpoint_name = request.config.getoption("--endpoint-name") + + try: + return get_endpoint(endpoint_name) + except FileNotFoundError as e: + pytest.skip( + f"Skipping scenario test because endpoints file is missing: {str(e)}" + ) + + +@pytest.fixture +async def r_credential(request, create_redis, endpoint): + credential_provider = request.param.get("cred_provider_class", None) + + if credential_provider is not None: + credential_provider = get_credential_provider(request) + + kwargs = { + "credential_provider": credential_provider, + } + + async with create_redis(url=endpoint, **kwargs) as client: + yield client + + +@pytest.fixture +async def r_acl_teardown(r: redis.Redis): + """ + A special fixture which removes the provided names from the database after use + """ + usernames = [] + + def factory(username): + usernames.append(username) + return r + + yield factory + for username in usernames: + await r.acl_deluser(username) + + +@pytest.fixture +async def r_required_pass_teardown(r: redis.Redis): + """ + A special fixture which removes the provided password from the database after use + """ + passwords = [] + + def factory(username): + passwords.append(username) + return r + + yield factory + for password in passwords: + try: + await r.auth(password) + except (ResponseError, AuthenticationError): + await r.auth("default", "") + await r.config_set("requirepass", "") + + +class NoPassCredProvider(CredentialProvider): + def get_credentials(self) -> Union[Tuple[str], Tuple[str, str]]: + return "username", "" + + +class AsyncRandomAuthCredProvider(CredentialProvider): + def __init__(self, user: Optional[str], endpoint: str): + self.user = user + self.endpoint = endpoint + + @functools.lru_cache(maxsize=10) + def get_credentials(self) -> Union[Tuple[str, str], Tuple[str]]: + def get_random_string(length): + letters = string.ascii_lowercase + result_str = "".join(random.choice(letters) for i in range(length)) + return result_str + + if self.user: + auth_token: str = get_random_string(5) + self.user + "_" + self.endpoint + return self.user, auth_token + else: + auth_token: str = get_random_string(5) + self.endpoint + return (auth_token,) + + +async def init_acl_user(r, username, password): + # reset the user + await r.acl_deluser(username) + if password: + assert ( + await r.acl_setuser( + username, + enabled=True, + passwords=["+" + password], + keys="~*", + commands=[ + "+ping", + "+command", + "+info", + "+select", + "+flushdb", + "+cluster", + ], + ) + is True + ) + else: + assert ( + await r.acl_setuser( + username, + enabled=True, + keys="~*", + commands=[ + "+ping", + "+command", + "+info", + "+select", + "+flushdb", + "+cluster", + ], + nopass=True, + ) + is True + ) + + +async def init_required_pass(r, password): + await r.config_set("requirepass", password) + + +class TestCredentialsProvider: + @skip_if_redis_enterprise() + async def test_only_pass_without_creds_provider( + self, r_required_pass_teardown, create_redis + ): + # test for default user (`username` is supposed to be optional) + password = "password" + r = r_required_pass_teardown(password) + await init_required_pass(r, password) + assert await r.auth(password) is True + + async with create_redis(flushdb=False, password=password) as r2: + assert await r2.ping() is True + + @skip_if_redis_enterprise() + async def test_user_and_pass_without_creds_provider( + self, r_acl_teardown, create_redis + ): + """ + Test backward compatibility with username and password + """ + # test for other users + username = "username" + password = "password" + r = r_acl_teardown(username) + await init_acl_user(r, username, password) + async with create_redis( + flushdb=False, username=username, password=password + ) as r2: + assert await r2.ping() is True + + @pytest.mark.parametrize("username", ["username", None]) + @skip_if_redis_enterprise() + @pytest.mark.onlynoncluster + async def test_credential_provider_with_supplier( + self, r_acl_teardown, r_required_pass_teardown, create_redis, username + ): + creds_provider = AsyncRandomAuthCredProvider( + user=username, + endpoint="localhost", + ) + + auth_args = creds_provider.get_credentials() + password = auth_args[-1] + + if username: + r = r_acl_teardown(username) + await init_acl_user(r, username, password) + else: + r = r_required_pass_teardown(password) + await init_required_pass(r, password) + + async with create_redis( + flushdb=False, credential_provider=creds_provider + ) as r2: + assert await r2.ping() is True + + async def test_async_credential_provider_no_password_success( + self, r_acl_teardown, create_redis + ): + username = "username" + r = r_acl_teardown(username) + await init_acl_user(r, username, "") + async with create_redis( + flushdb=False, + credential_provider=NoPassCredProvider(), + ) as r2: + assert await r2.ping() is True + + @pytest.mark.onlynoncluster + async def test_credential_provider_no_password_error( + self, r_acl_teardown, create_redis + ): + username = "username" + r = r_acl_teardown(username) + await init_acl_user(r, username, "password") + with pytest.raises(AuthenticationError) as e: + async with create_redis( + flushdb=False, + credential_provider=NoPassCredProvider(), + single_connection_client=True, + ): + pass + + assert e.match("invalid username-password") + assert await r.acl_deluser(username) + + @pytest.mark.onlynoncluster + async def test_password_and_username_together_with_cred_provider_raise_error( + self, r_acl_teardown, create_redis + ): + username = "username" + r = r_acl_teardown(username) + await init_acl_user(r, username, "password") + cred_provider = UsernamePasswordCredentialProvider( + username="username", password="password" + ) + with pytest.raises(DataError) as e: + async with create_redis( + flushdb=False, + username="username", + password="password", + credential_provider=cred_provider, + single_connection_client=True, + ): + pass + + assert e.match( + "'username' and 'password' cannot be passed along with " + "'credential_provider'." + ) + + @pytest.mark.onlynoncluster + async def test_change_username_password_on_existing_connection( + self, r_acl_teardown, create_redis + ): + username = "origin_username" + password = "origin_password" + new_username = "new_username" + new_password = "new_password" + r = r_acl_teardown(username) + await init_acl_user(r, username, password) + async with create_redis( + flushdb=False, username=username, password=password + ) as r2: + assert await r2.ping() is True + conn = await r2.connection_pool.get_connection() + await conn.send_command("PING") + assert str_if_bytes(await conn.read_response()) == "PONG" + assert conn.username == username + assert conn.password == password + await init_acl_user(r, new_username, new_password) + conn.password = new_password + conn.username = new_username + await conn.send_command("PING") + assert str_if_bytes(await conn.read_response()) == "PONG" + + +class TestUsernamePasswordCredentialProvider: + async def test_user_pass_credential_provider_acl_user_and_pass( + self, r_acl_teardown, create_redis + ): + username = "username" + password = "password" + r = r_acl_teardown(username) + provider = UsernamePasswordCredentialProvider(username, password) + assert provider.username == username + assert provider.password == password + assert provider.get_credentials() == (username, password) + await init_acl_user(r, provider.username, provider.password) + async with create_redis(flushdb=False, credential_provider=provider) as r2: + assert await r2.ping() is True + + async def test_user_pass_provider_only_password( + self, r_required_pass_teardown, create_redis + ): + password = "password" + provider = UsernamePasswordCredentialProvider(password=password) + r = r_required_pass_teardown(password) + assert provider.username == "" + assert provider.password == password + assert provider.get_credentials() == (password,) + + await init_required_pass(r, password) + + async with create_redis(flushdb=False, credential_provider=provider) as r2: + assert await r2.auth(provider.password) is True + assert await r2.ping() is True + + +@pytest.mark.onlynoncluster +@pytest.mark.skipif(not EntraIdCredentialsProvider, reason="requires redis-entraid") +class TestStreamingCredentialProvider: + @pytest.mark.parametrize( + "credential_provider", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"expiration_refresh_ratio": 0.00005}, + "mock_idp": True, + } + ], + indirect=True, + ) + @pytest.mark.parametrize("anyio_backend", ["asyncio"], indirect=True) + async def test_async_re_auth_all_connections(self, credential_provider): + mock_connection = Mock(spec=Connection) + mock_connection.retry = Retry(NoBackoff(), 0) + mock_another_connection = Mock(spec=Connection) + mock_pool = Mock(spec=ConnectionPool) + mock_pool.connection_kwargs = { + "credential_provider": credential_provider, + } + mock_pool.get_connection.return_value = mock_connection + mock_pool._available_connections = [mock_connection, mock_another_connection] + mock_pool._lock = AsyncLock() + auth_token = None + + async def re_auth_callback(token): + nonlocal auth_token + auth_token = token + async with mock_pool._lock: + for conn in mock_pool._available_connections: + await conn.send_command( + "AUTH", token.try_get("oid"), token.get_value() + ) + await conn.read_response() + + mock_pool.re_auth_callback = re_auth_callback + + async with Redis( + connection_pool=mock_pool, + credential_provider=credential_provider, + ): + await credential_provider.get_credentials_async() + await async_sleep(0.5) + + mock_connection.send_command.assert_has_calls( + [call("AUTH", auth_token.try_get("oid"), auth_token.get_value())] + ) + mock_another_connection.send_command.assert_has_calls( + [call("AUTH", auth_token.try_get("oid"), auth_token.get_value())] + ) + + @pytest.mark.parametrize( + "credential_provider", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"expiration_refresh_ratio": 0.00005}, + "mock_idp": True, + } + ], + indirect=True, + ) + @pytest.mark.parametrize("anyio_backend", ["asyncio"], indirect=True) + async def test_async_re_auth_partial_connections(self, credential_provider): + mock_connection = Mock(spec=Connection) + mock_connection.retry = Retry(NoBackoff(), 3) + mock_another_connection = Mock(spec=Connection) + mock_another_connection.retry = Retry(NoBackoff(), 3) + mock_failed_connection = Mock(spec=Connection) + mock_failed_connection.read_response.side_effect = ConnectionError( + "Failed auth" + ) + mock_failed_connection.retry = Retry(NoBackoff(), 3) + mock_pool = Mock(spec=ConnectionPool) + mock_pool.connection_kwargs = { + "credential_provider": credential_provider, + } + mock_pool.get_connection.return_value = mock_connection + mock_pool._available_connections = [ + mock_connection, + mock_another_connection, + mock_failed_connection, + ] + mock_pool._lock = AsyncLock() + + async def _raise(error: Exception): + pass + + async def re_auth_callback(token): + async with mock_pool._lock: + for conn in mock_pool._available_connections: + await conn.retry.call_with_retry( + lambda: conn.send_command( + "AUTH", token.try_get("oid"), token.get_value() + ), + lambda error: _raise(error), + ) + await conn.retry.call_with_retry( + lambda: conn.read_response(), lambda error: _raise(error) + ) + + mock_pool.re_auth_callback = re_auth_callback + + async with Redis( + connection_pool=mock_pool, + credential_provider=credential_provider, + ): + credential_provider._listener.on_error = _raise + await credential_provider.get_credentials_async() + await async_sleep(0.5) + + # Workaround for the lack of proper resource management + credential_provider._token_mgr.stop() + await async_sleep(0.1) + + mock_connection.read_response.assert_has_calls([call()]) + mock_another_connection.read_response.assert_has_calls([call()]) + mock_failed_connection.read_response.assert_has_calls([call(), call(), call()]) + + @pytest.mark.parametrize( + "credential_provider", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"expiration_refresh_ratio": 0.00005}, + "mock_idp": True, + } + ], + indirect=True, + ) + @pytest.mark.parametrize("anyio_backend", ["asyncio"], indirect=True) + async def test_re_auth_pub_sub_in_resp3(self, credential_provider): + mock_pubsub_connection = Mock(spec=Connection) + mock_pubsub_connection.get_protocol.return_value = 3 + mock_pubsub_connection.credential_provider = credential_provider + mock_pubsub_connection.retry = Retry(NoBackoff(), 3) + mock_another_connection = Mock(spec=Connection) + mock_another_connection.retry = Retry(NoBackoff(), 3) + + mock_pool = Mock(spec=ConnectionPool) + mock_pool.connection_kwargs = { + "credential_provider": credential_provider, + } + mock_pool.get_connection.side_effect = [ + mock_pubsub_connection, + mock_another_connection, + ] + mock_pool._available_connections = [mock_another_connection] + mock_pool._lock = AsyncLock() + auth_token = None + + async def re_auth_callback(token): + nonlocal auth_token + auth_token = token + async with mock_pool._lock: + for conn in mock_pool._available_connections: + await conn.send_command( + "AUTH", token.try_get("oid"), token.get_value() + ) + await conn.read_response() + + mock_pool.re_auth_callback = re_auth_callback + + r = Redis( + connection_pool=mock_pool, + credential_provider=credential_provider, + ) + p = r.pubsub() + await p.subscribe("test") + await credential_provider.get_credentials_async() + await async_sleep(0.5) + + mock_pubsub_connection.send_command.assert_has_calls( + [ + call("SUBSCRIBE", "test", check_health=True), + call("AUTH", auth_token.try_get("oid"), auth_token.get_value()), + ] + ) + mock_another_connection.send_command.assert_has_calls( + [call("AUTH", auth_token.try_get("oid"), auth_token.get_value())] + ) + + @pytest.mark.parametrize( + "credential_provider", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"expiration_refresh_ratio": 0.00005}, + "mock_idp": True, + } + ], + indirect=True, + ) + @pytest.mark.parametrize("anyio_backend", ["asyncio"], indirect=True) + async def test_do_not_re_auth_pub_sub_in_resp2(self, credential_provider): + mock_pubsub_connection = Mock(spec=Connection) + mock_pubsub_connection.get_protocol.return_value = 2 + mock_pubsub_connection.credential_provider = credential_provider + mock_pubsub_connection.retry = Retry(NoBackoff(), 3) + mock_another_connection = Mock(spec=Connection) + mock_another_connection.retry = Retry(NoBackoff(), 3) + + mock_pool = Mock(spec=ConnectionPool) + mock_pool.connection_kwargs = { + "credential_provider": credential_provider, + } + mock_pool.get_connection.side_effect = [ + mock_pubsub_connection, + mock_another_connection, + ] + mock_pool._available_connections = [mock_another_connection] + mock_pool._lock = AsyncLock() + auth_token = None + + async def re_auth_callback(token): + nonlocal auth_token + auth_token = token + async with mock_pool._lock: + for conn in mock_pool._available_connections: + await conn.send_command( + "AUTH", token.try_get("oid"), token.get_value() + ) + await conn.read_response() + + mock_pool.re_auth_callback = re_auth_callback + + r = Redis( + connection_pool=mock_pool, + credential_provider=credential_provider, + ) + p = r.pubsub() + await p.subscribe("test") + await credential_provider.get_credentials_async() + await async_sleep(0.5) + + mock_pubsub_connection.send_command.assert_has_calls( + [ + call("SUBSCRIBE", "test", check_health=True), + ] + ) + mock_another_connection.send_command.assert_has_calls( + [call("AUTH", auth_token.try_get("oid"), auth_token.get_value())] + ) + + @pytest.mark.parametrize( + "credential_provider", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"expiration_refresh_ratio": 0.00005}, + "mock_idp": True, + } + ], + indirect=True, + ) + async def test_fails_on_token_renewal(self, credential_provider): + credential_provider._token_mgr._idp.request_token.side_effect = [ + RequestTokenErr, + RequestTokenErr, + RequestTokenErr, + RequestTokenErr, + ] + mock_connection = Mock(spec=Connection) + mock_connection.retry = Retry(NoBackoff(), 0) + mock_another_connection = Mock(spec=Connection) + mock_pool = Mock(spec=ConnectionPool) + mock_pool.connection_kwargs = { + "credential_provider": credential_provider, + } + mock_pool.get_connection.return_value = mock_connection + mock_pool._available_connections = [mock_connection, mock_another_connection] + + await Redis( + connection_pool=mock_pool, + credential_provider=credential_provider, + ) + + with pytest.raises(RequestTokenErr): + await credential_provider.get_credentials() + + +@pytest.mark.onlynoncluster +@pytest.mark.cp_integration +@pytest.mark.skipif(not EntraIdCredentialsProvider, reason="requires redis-entraid") +class TestEntraIdCredentialsProvider: + @pytest.mark.parametrize( + "r_credential", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + }, + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"block_for_initial": True}, + }, + { + "cred_provider_class": EntraIdCredentialsProvider, + "idp_kwargs": {"auth_type": AuthType.DEFAULT_AZURE_CREDENTIAL}, + }, + ], + ids=["blocked", "non-blocked", "DefaultAzureCredential"], + indirect=True, + ) + @pytest.mark.onlynoncluster + @pytest.mark.cp_integration + async def test_async_auth_pool_with_credential_provider(self, r_credential: Redis): + assert await r_credential.ping() is True + + @pytest.mark.parametrize( + "r_credential", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + }, + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"block_for_initial": True}, + }, + ], + ids=["blocked", "non-blocked"], + indirect=True, + ) + @pytest.mark.onlynoncluster + @pytest.mark.cp_integration + async def test_async_pipeline_with_credential_provider(self, r_credential: Redis): + pipe = r_credential.pipeline() + + await pipe.set("key", "value") + await pipe.get("key") + + assert await pipe.execute() == [True, b"value"] + + @pytest.mark.parametrize( + "r_credential", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + }, + ], + indirect=True, + ) + @pytest.mark.onlynoncluster + @pytest.mark.cp_integration + async def test_async_auth_pubsub_with_credential_provider( + self, r_credential: Redis + ): + p = r_credential.pubsub() + await p.subscribe("entraid") + + await r_credential.publish("entraid", "test") + await r_credential.publish("entraid", "test") + + msg1 = await p.get_message() + + assert msg1["type"] == "subscribe" + + +@pytest.mark.onlycluster +@pytest.mark.cp_integration +@pytest.mark.skipif(not EntraIdCredentialsProvider, reason="requires redis-entraid") +class TestClusterEntraIdCredentialsProvider: + @pytest.mark.parametrize( + "r_credential", + [ + { + "cred_provider_class": EntraIdCredentialsProvider, + }, + { + "cred_provider_class": EntraIdCredentialsProvider, + "cred_provider_kwargs": {"block_for_initial": True}, + }, + { + "cred_provider_class": EntraIdCredentialsProvider, + "idp_kwargs": {"auth_type": AuthType.DEFAULT_AZURE_CREDENTIAL}, + }, + ], + ids=["blocked", "non-blocked", "DefaultAzureCredential"], + indirect=True, + ) + @pytest.mark.onlycluster + @pytest.mark.cp_integration + async def test_async_auth_pool_with_credential_provider(self, r_credential: Redis): + assert await r_credential.ping() is True diff --git a/tests/test_anyio/test_cwe_404.py b/tests/test_anyio/test_cwe_404.py new file mode 100644 index 0000000000..4ae75432aa --- /dev/null +++ b/tests/test_anyio/test_cwe_404.py @@ -0,0 +1,231 @@ +import contextlib +from contextlib import AsyncExitStack + +import anyio +import pytest +from anyio.abc import ByteReceiveStream, ByteSendStream, ByteStream, TaskGroup + +from redis.anyio import Redis +from redis.anyio.cluster import RedisCluster +from redis.anyio.utils import gather + +pytestmark = pytest.mark.anyio + + +class DelayProxy: + def __init__(self, addr, redis_addr, delay: float = 0.0): + self.addr = addr + self.redis_addr = redis_addr + self.delay = delay + self.send_event = anyio.Event() + + async def __aenter__(self): + # test that we can connect to redis + with anyio.fail_after(2): + async with await anyio.connect_tcp(*self.redis_addr): + pass + + async with AsyncExitStack() as stack: + listener = await anyio.create_tcp_listener( + local_port=self.addr[1], reuse_port=True + ) + await stack.enter_async_context(listener) + task_group = await stack.enter_async_context(anyio.create_task_group()) + task_group.start_soon(listener.serve, self.handle) + stack.callback(task_group.cancel_scope.cancel) + self._stack = stack.pop_all() + + return self + + async def __aexit__(self, exc_type, exc_val, exc_tb): + return await self._stack.__aexit__(exc_type, exc_val, exc_tb) + + @contextlib.contextmanager + def set_delay(self, delay: float = 0.0): + """ + Allow to override the delay for parts of tests which aren't time dependent, + to speed up execution. + """ + old_delay = self.delay + self.delay = delay + try: + yield + finally: + self.delay = old_delay + + async def handle(self, stream: ByteStream) -> None: + # establish connection to redis + redis_stream = await anyio.connect_tcp(*self.redis_addr) + async with stream, redis_stream, anyio.create_task_group() as tg: + tg.start_soon(self.pipe, stream, redis_stream, True) + tg.start_soon(self.pipe, redis_stream, stream, False) + + async def pipe( + self, + reader: ByteReceiveStream, + writer: ByteSendStream, + set_event_on_receive: bool, + ): + async for data in reader: + if set_event_on_receive: + self.send_event.set() + + await anyio.sleep(self.delay) + await writer.send(data) + + +@pytest.mark.onlynoncluster +@pytest.mark.parametrize("delay", argvalues=[0.05, 0.5, 1, 2]) +async def test_standalone(delay, master_host): + # create a tcp socket proxy that relays data to Redis and back, + # inserting 0.1 seconds of delay + async with DelayProxy(addr=("127.0.0.1", 5380), redis_addr=master_host) as dp: + for b in [True, False]: + # note that we connect to proxy, rather than to Redis directly + async with Redis( + host="127.0.0.1", port=5380, single_connection_client=b + ) as r: + await r.set("foo", "foo") + await r.set("bar", "bar") + + async def op(r): + with dp.set_delay(delay * 2): + return await r.get( + "foo" + ) # <-- this is the operation we want to cancel + + dp.send_event = anyio.Event() + async with anyio.create_task_group() as tg: + tg.start_soon(op, r) + # Wait until the task has sent, and then some, to make sure it has + # settled on the read. + await dp.send_event.wait() + await anyio.sleep(0.01) # a little extra time for prudence + tg.cancel_scope.cancel() + + # make sure that our previous request, cancelled while waiting for + # a repsponse, didn't leave the connection open andin a bad state + assert await r.get("bar") == b"bar" + assert await r.ping() + assert await r.get("foo") == b"foo" + + +@pytest.mark.onlynoncluster +@pytest.mark.parametrize("delay", argvalues=[0.05, 0.5, 1, 2]) +async def test_standalone_pipeline(delay, master_host): + async with DelayProxy(addr=("127.0.0.1", 5380), redis_addr=master_host) as dp: + for b in [True, False]: + async with Redis( + host="127.0.0.1", port=5380, single_connection_client=b + ) as r: + await r.set("foo", "foo") + await r.set("bar", "bar") + + pipe = r.pipeline() + + pipe2 = r.pipeline() + pipe2.get("bar") + pipe2.ping() + pipe2.get("foo") + + async def op(r): + with dp.set_delay(delay * 2): + return await r.get( + "foo" + ) # <-- this is the operation we want to cancel + + dp.send_event = anyio.Event() + async with anyio.create_task_group() as tg: + tg.start_soon(op, r) + # wait until task has settled on the read + await dp.send_event.wait() + await anyio.sleep(0.01) + tg.cancel_scope.cancel() + + # we have now cancelled the pieline in the middle of a request, + # make sure that the connection is still usable + pipe.get("bar") + pipe.ping() + pipe.get("foo") + await pipe.reset() + + # check that the pipeline is empty after reset + assert await pipe.execute() == [] + + # validating that the pipeline can be used as it could previously + pipe.get("bar") + pipe.ping() + pipe.get("foo") + assert await pipe.execute() == [b"bar", True, b"foo"] + assert await pipe2.execute() == [b"bar", True, b"foo"] + + +@pytest.mark.onlycluster +async def test_cluster(master_host): + delay = 0.1 + cluster_port = 16379 + remap_base = 7372 + n_nodes = 6 + hostname, _ = master_host + + def remap(address): + host, port = address + return host, remap_base + port - cluster_port + + proxies = [] + for i in range(n_nodes): + port = cluster_port + i + remapped = remap_base + i + forward_addr = hostname, port + proxy = DelayProxy(addr=(hostname, remapped), redis_addr=forward_addr) + proxies.append(proxy) + + def all_clear(): + for p in proxies: + p.send_event = anyio.Event() + + async def wait_for_send(): + async def wait_and_cancel_rest(proxy: DelayProxy, tg: TaskGroup) -> None: + await proxy.send_event.wait() + tg.cancel_scope.cancel() + + async with anyio.create_task_group() as tg: + for p in proxies: + tg.start_soon(wait_and_cancel_rest, p, tg) + + @contextlib.contextmanager + def set_delay(delay: float): + with contextlib.ExitStack() as stack: + for p in proxies: + stack.enter_context(p.set_delay(delay)) + yield + + async with contextlib.AsyncExitStack() as stack: + for p in proxies: + await stack.enter_async_context(p) + + async with RedisCluster.from_url( + f"redis://{hostname}:{remap_base}", address_remap=remap + ) as r: + await r.set("foo", "foo") + await r.set("bar", "bar") + + async def op(r): + with set_delay(delay): + return await r.get("foo") + + all_clear() + async with anyio.create_task_group() as tg: + # Wait for whichever DelayProxy gets the request first + tg.start_soon(op, r) + await wait_for_send() + await anyio.sleep(0.01) + tg.cancel_scope.cancel() + + # try a number of requests to exercise all the connections + async def doit(): + assert await r.get("bar") == b"bar" + assert await r.ping() + assert await r.get("foo") == b"foo" + + await gather(*[doit() for _ in range(10)]) diff --git a/tests/test_anyio/test_encoding.py b/tests/test_anyio/test_encoding.py new file mode 100644 index 0000000000..d31ea379fd --- /dev/null +++ b/tests/test_anyio/test_encoding.py @@ -0,0 +1,113 @@ +import pytest +import redis.anyio as redis +from redis.exceptions import DataError + +pytestmark = pytest.mark.anyio + + +@pytest.mark.onlynoncluster +class TestEncoding: + @pytest.fixture + async def r(self, create_redis): + async with create_redis(decode_responses=True) as client: + yield client + await client.flushall() + + @pytest.fixture + async def r_no_decode(self, create_redis): + async with create_redis(decode_responses=False) as client: + yield client + await client.flushall() + + async def test_simple_encoding(self, r_no_decode: redis.Redis): + unicode_string = chr(3456) + "abcd" + chr(3421) + await r_no_decode.set("unicode-string", unicode_string.encode("utf-8")) + cached_val = await r_no_decode.get("unicode-string") + assert isinstance(cached_val, bytes) + assert unicode_string == cached_val.decode("utf-8") + + async def test_simple_encoding_and_decoding(self, r: redis.Redis): + unicode_string = chr(3456) + "abcd" + chr(3421) + await r.set("unicode-string", unicode_string) + cached_val = await r.get("unicode-string") + assert isinstance(cached_val, str) + assert unicode_string == cached_val + + async def test_memoryview_encoding(self, r_no_decode: redis.Redis): + unicode_string = chr(3456) + "abcd" + chr(3421) + unicode_string_view = memoryview(unicode_string.encode("utf-8")) + await r_no_decode.set("unicode-string-memoryview", unicode_string_view) + cached_val = await r_no_decode.get("unicode-string-memoryview") + # The cached value won't be a memoryview because it's a copy from Redis + assert isinstance(cached_val, bytes) + assert unicode_string == cached_val.decode("utf-8") + + async def test_memoryview_encoding_and_decoding(self, r: redis.Redis): + unicode_string = chr(3456) + "abcd" + chr(3421) + unicode_string_view = memoryview(unicode_string.encode("utf-8")) + await r.set("unicode-string-memoryview", unicode_string_view) + cached_val = await r.get("unicode-string-memoryview") + assert isinstance(cached_val, str) + assert unicode_string == cached_val + + async def test_list_encoding(self, r: redis.Redis): + unicode_string = chr(3456) + "abcd" + chr(3421) + result = [unicode_string, unicode_string, unicode_string] + await r.rpush("a", *result) + assert await r.lrange("a", 0, -1) == result + + +@pytest.mark.onlynoncluster +class TestEncodingErrors: + async def test_ignore(self, create_redis): + async with create_redis(decode_responses=True, encoding_errors="ignore") as r: + await r.set("a", b"foo\xff") + assert await r.get("a") == "foo" + + async def test_replace(self, create_redis): + async with create_redis(decode_responses=True, encoding_errors="replace") as r: + await r.set("a", b"foo\xff") + assert await r.get("a") == "foo\ufffd" + + +@pytest.mark.onlynoncluster +class TestMemoryviewsAreNotPacked: + async def test_memoryviews_are_not_packed(self, r): + arg = memoryview(b"some_arg") + arg_list = ["SOME_COMMAND", arg] + c = r.connection or await r.connection_pool.get_connection() + cmd = c.pack_command(*arg_list) + assert cmd[1] is arg + cmds = c.pack_commands([arg_list, arg_list]) + assert cmds[1] is arg + assert cmds[3] is arg + + +class TestCommandsAreNotEncoded: + @pytest.fixture + async def r(self, create_redis): + async with create_redis(encoding="utf-16") as client: + yield client + await client.flushall() + + @pytest.mark.xfail + async def test_basic_command(self, r: redis.Redis): + await r.set("hello", "world") + + +class TestInvalidUserInput: + async def test_boolean_fails(self, r: redis.Redis): + with pytest.raises(DataError): + await r.set("a", True) # type: ignore + + async def test_none_fails(self, r: redis.Redis): + with pytest.raises(DataError): + await r.set("a", None) # type: ignore + + async def test_user_type_fails(self, r: redis.Redis): + class Foo: + def __str__(self): + return "Foo" + + with pytest.raises(DataError): + await r.set("a", Foo()) # type: ignore diff --git a/tests/test_anyio/test_hash.py b/tests/test_anyio/test_hash.py new file mode 100644 index 0000000000..c5d538eeca --- /dev/null +++ b/tests/test_anyio/test_hash.py @@ -0,0 +1,305 @@ +import math +from datetime import datetime, timedelta + +import anyio +import pytest + +from tests.conftest import skip_if_server_version_lt + +pytestmark = pytest.mark.anyio + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpire_basic(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + assert await r.hexpire("test:hash", 1, "field1") == [1] + await anyio.sleep(1.1) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpire_with_timedelta(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + assert await r.hexpire("test:hash", timedelta(seconds=1), "field1") == [1] + await anyio.sleep(1.1) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpire_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1"}) + assert await r.hexpire("test:hash", 2, "field1", xx=True) == [0] + assert await r.hexpire("test:hash", 2, "field1", nx=True) == [1] + assert await r.hexpire("test:hash", 1, "field1", xx=True) == [1] + assert await r.hexpire("test:hash", 2, "field1", nx=True) == [0] + await anyio.sleep(1.1) + assert await r.hexists("test:hash", "field1") is False + await r.hset("test:hash", "field1", "value1") + await r.hexpire("test:hash", 2, "field1") + assert await r.hexpire("test:hash", 1, "field1", gt=True) == [0] + assert await r.hexpire("test:hash", 1, "field1", lt=True) == [1] + await anyio.sleep(1.1) + assert await r.hexists("test:hash", "field1") is False + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpire_nonexistent_key_or_field(r): + await r.delete("test:hash") + assert await r.hexpire("test:hash", 1, "field1") == [-2] + await r.hset("test:hash", "field1", "value1") + assert await r.hexpire("test:hash", 1, "nonexistent_field") == [-2] + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpire_multiple_fields(r): + await r.delete("test:hash") + await r.hset( + "test:hash", + mapping={"field1": "value1", "field2": "value2", "field3": "value3"}, + ) + assert await r.hexpire("test:hash", 1, "field1", "field2") == [1, 1] + await anyio.sleep(1.1) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is False + assert await r.hexists("test:hash", "field3") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpire_basic(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + assert await r.hpexpire("test:hash", 500, "field1") == [1] + await anyio.sleep(0.6) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpire_with_timedelta(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + assert await r.hpexpire("test:hash", timedelta(milliseconds=500), "field1") == [1] + await anyio.sleep(0.6) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpire_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1"}) + assert await r.hpexpire("test:hash", 1500, "field1", xx=True) == [0] + assert await r.hpexpire("test:hash", 1500, "field1", nx=True) == [1] + assert await r.hpexpire("test:hash", 500, "field1", xx=True) == [1] + assert await r.hpexpire("test:hash", 1500, "field1", nx=True) == [0] + await anyio.sleep(0.6) + assert await r.hexists("test:hash", "field1") is False + await r.hset("test:hash", "field1", "value1") + await r.hpexpire("test:hash", 1000, "field1") + assert await r.hpexpire("test:hash", 500, "field1", gt=True) == [0] + assert await r.hpexpire("test:hash", 500, "field1", lt=True) == [1] + await anyio.sleep(0.6) + assert await r.hexists("test:hash", "field1") is False + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpire_nonexistent_key_or_field(r): + await r.delete("test:hash") + assert await r.hpexpire("test:hash", 500, "field1") == [-2] + await r.hset("test:hash", "field1", "value1") + assert await r.hpexpire("test:hash", 500, "nonexistent_field") == [-2] + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpire_multiple_fields(r): + await r.delete("test:hash") + await r.hset( + "test:hash", + mapping={"field1": "value1", "field2": "value2", "field3": "value3"}, + ) + assert await r.hpexpire("test:hash", 500, "field1", "field2") == [1, 1] + await anyio.sleep(0.6) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is False + assert await r.hexists("test:hash", "field3") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpireat_basic(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + exp_time = math.ceil((datetime.now() + timedelta(seconds=1)).timestamp()) + assert await r.hexpireat("test:hash", exp_time, "field1") == [1] + await anyio.sleep(2.1) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpireat_with_datetime(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + exp_time = (datetime.now() + timedelta(seconds=2)).replace(microsecond=0) + assert await r.hexpireat("test:hash", exp_time, "field1") == [1] + await anyio.sleep(2.1) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpireat_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1"}) + future_exp_time = int((datetime.now() + timedelta(seconds=2)).timestamp()) + past_exp_time = int((datetime.now() - timedelta(seconds=1)).timestamp()) + assert await r.hexpireat("test:hash", future_exp_time, "field1", xx=True) == [0] + assert await r.hexpireat("test:hash", future_exp_time, "field1", nx=True) == [1] + assert await r.hexpireat("test:hash", past_exp_time, "field1", gt=True) == [0] + assert await r.hexpireat("test:hash", past_exp_time, "field1", lt=True) == [2] + assert await r.hexists("test:hash", "field1") is False + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpireat_nonexistent_key_or_field(r): + await r.delete("test:hash") + future_exp_time = int((datetime.now() + timedelta(seconds=1)).timestamp()) + assert await r.hexpireat("test:hash", future_exp_time, "field1") == [-2] + await r.hset("test:hash", "field1", "value1") + assert await r.hexpireat("test:hash", future_exp_time, "nonexistent_field") == [-2] + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpireat_multiple_fields(r): + await r.delete("test:hash") + await r.hset( + "test:hash", + mapping={"field1": "value1", "field2": "value2", "field3": "value3"}, + ) + exp_time = math.ceil((datetime.now() + timedelta(seconds=1)).timestamp()) + assert await r.hexpireat("test:hash", exp_time, "field1", "field2") == [1, 1] + await anyio.sleep(2.1) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is False + assert await r.hexists("test:hash", "field3") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpireat_basic(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + exp_time = int((datetime.now() + timedelta(milliseconds=400)).timestamp() * 1000) + assert await r.hpexpireat("test:hash", exp_time, "field1") == [1] + await anyio.sleep(0.5) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpireat_with_datetime(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + exp_time = datetime.now() + timedelta(milliseconds=400) + assert await r.hpexpireat("test:hash", exp_time, "field1") == [1] + await anyio.sleep(0.5) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpireat_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1"}) + future_exp_time = int( + (datetime.now() + timedelta(milliseconds=500)).timestamp() * 1000 + ) + past_exp_time = int( + (datetime.now() - timedelta(milliseconds=500)).timestamp() * 1000 + ) + assert await r.hpexpireat("test:hash", future_exp_time, "field1", xx=True) == [0] + assert await r.hpexpireat("test:hash", future_exp_time, "field1", nx=True) == [1] + assert await r.hpexpireat("test:hash", past_exp_time, "field1", gt=True) == [0] + assert await r.hpexpireat("test:hash", past_exp_time, "field1", lt=True) == [2] + assert await r.hexists("test:hash", "field1") is False + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpireat_nonexistent_key_or_field(r): + await r.delete("test:hash") + future_exp_time = int( + (datetime.now() + timedelta(milliseconds=500)).timestamp() * 1000 + ) + assert await r.hpexpireat("test:hash", future_exp_time, "field1") == [-2] + await r.hset("test:hash", "field1", "value1") + assert await r.hpexpireat("test:hash", future_exp_time, "nonexistent_field") == [-2] + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpireat_multiple_fields(r): + await r.delete("test:hash") + await r.hset( + "test:hash", + mapping={"field1": "value1", "field2": "value2", "field3": "value3"}, + ) + exp_time = int((datetime.now() + timedelta(milliseconds=400)).timestamp() * 1000) + assert await r.hpexpireat("test:hash", exp_time, "field1", "field2") == [1, 1] + await anyio.sleep(0.5) + assert await r.hexists("test:hash", "field1") is False + assert await r.hexists("test:hash", "field2") is False + assert await r.hexists("test:hash", "field3") is True + + +@skip_if_server_version_lt("7.3.240") +async def test_hpersist_multiple_fields_mixed_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + await r.hexpire("test:hash", 5000, "field1") + assert await r.hpersist("test:hash", "field1", "field2", "field3") == [1, -1, -2] + + +@skip_if_server_version_lt("7.3.240") +async def test_hexpiretime_multiple_fields_mixed_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + future_time = int((datetime.now() + timedelta(minutes=30)).timestamp()) + await r.hexpireat("test:hash", future_time, "field1") + result = await r.hexpiretime("test:hash", "field1", "field2", "field3") + assert future_time - 10 < result[0] <= future_time + assert result[1:] == [-1, -2] + + +@skip_if_server_version_lt("7.3.240") +async def test_hpexpiretime_multiple_fields_mixed_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + future_time = int((datetime.now() + timedelta(minutes=30)).timestamp()) + await r.hexpireat("test:hash", future_time, "field1") + result = await r.hpexpiretime("test:hash", "field1", "field2", "field3") + assert future_time * 1000 - 10000 < result[0] <= future_time * 1000 + assert result[1:] == [-1, -2] + + +@skip_if_server_version_lt("7.3.240") +async def test_ttl_multiple_fields_mixed_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + future_time = int((datetime.now() + timedelta(minutes=30)).timestamp()) + await r.hexpireat("test:hash", future_time, "field1") + result = await r.httl("test:hash", "field1", "field2", "field3") + assert 30 * 60 - 10 < result[0] <= 30 * 60 + assert result[1:] == [-1, -2] + + +@skip_if_server_version_lt("7.3.240") +async def test_pttl_multiple_fields_mixed_conditions(r): + await r.delete("test:hash") + await r.hset("test:hash", mapping={"field1": "value1", "field2": "value2"}) + future_time = int((datetime.now() + timedelta(minutes=30)).timestamp()) + await r.hexpireat("test:hash", future_time, "field1") + result = await r.hpttl("test:hash", "field1", "field2", "field3") + assert 30 * 60000 - 10000 < result[0] <= 30 * 60000 + assert result[1:] == [-1, -2] diff --git a/tests/test_anyio/test_json.py b/tests/test_anyio/test_json.py new file mode 100644 index 0000000000..c1c8090573 --- /dev/null +++ b/tests/test_anyio/test_json.py @@ -0,0 +1,1031 @@ +import pytest +import redis.anyio as redis +from redis import exceptions +from redis.commands.json.path import Path +from tests.conftest import assert_resp_response, skip_ifmodversion_lt + +pytestmark = pytest.mark.anyio + + +@pytest.fixture +async def decoded_r(create_redis, stack_url): + async with create_redis(decode_responses=True, url=stack_url) as client: + yield client + + +@pytest.mark.redismod +async def test_json_setbinarykey(decoded_r: redis.Redis): + d = {"hello": "world", b"some": "value"} + with pytest.raises(TypeError): + decoded_r.json().set("somekey", Path.root_path(), d) + assert await decoded_r.json().set("somekey", Path.root_path(), d, decode_keys=True) + + +@pytest.mark.redismod +async def test_json_setgetdeleteforget(decoded_r: redis.Redis): + assert await decoded_r.json().set("foo", Path.root_path(), "bar") + assert await decoded_r.json().get("foo") == "bar" + assert await decoded_r.json().get("baz") is None + assert await decoded_r.json().delete("foo") == 1 + assert await decoded_r.json().forget("foo") == 0 # second delete + assert await decoded_r.exists("foo") == 0 + + +@pytest.mark.redismod +async def test_jsonget(decoded_r: redis.Redis): + await decoded_r.json().set("foo", Path.root_path(), "bar") + assert await decoded_r.json().get("foo") == "bar" + + +@pytest.mark.redismod +async def test_json_get_jset(decoded_r: redis.Redis): + assert await decoded_r.json().set("foo", Path.root_path(), "bar") + assert await decoded_r.json().get("foo") == "bar" + assert await decoded_r.json().get("baz") is None + assert 1 == await decoded_r.json().delete("foo") + assert await decoded_r.exists("foo") == 0 + + +@pytest.mark.redismod +async def test_nonascii_setgetdelete(decoded_r: redis.Redis): + assert await decoded_r.json().set("notascii", Path.root_path(), "hyvää-élève") + assert await decoded_r.json().get("notascii", no_escape=True) == "hyvää-élève" + assert 1 == await decoded_r.json().delete("notascii") + assert await decoded_r.exists("notascii") == 0 + + +@pytest.mark.redismod +@skip_ifmodversion_lt("2.6.0", "ReJSON") +async def test_json_merge(decoded_r: redis.Redis): + # Test with root path $ + assert await decoded_r.json().set( + "person_data", + "$", + {"person1": {"personal_data": {"name": "John"}}}, + ) + assert await decoded_r.json().merge( + "person_data", "$", {"person1": {"personal_data": {"hobbies": "reading"}}} + ) + assert await decoded_r.json().get("person_data") == { + "person1": {"personal_data": {"name": "John", "hobbies": "reading"}} + } + + # Test with root path path $.person1.personal_data + assert await decoded_r.json().merge( + "person_data", "$.person1.personal_data", {"country": "Israel"} + ) + assert await decoded_r.json().get("person_data") == { + "person1": { + "personal_data": {"name": "John", "hobbies": "reading", "country": "Israel"} + } + } + + # Test with null value to delete a value + assert await decoded_r.json().merge( + "person_data", "$.person1.personal_data", {"name": None} + ) + assert await decoded_r.json().get("person_data") == { + "person1": {"personal_data": {"country": "Israel", "hobbies": "reading"}} + } + + +@pytest.mark.redismod +async def test_jsonsetexistentialmodifiersshouldsucceed(decoded_r: redis.Redis): + obj = {"foo": "bar"} + assert await decoded_r.json().set("obj", Path.root_path(), obj) + + # Test that flags prevent updates when conditions are unmet + assert await decoded_r.json().set("obj", Path("foo"), "baz", nx=True) is None + assert await decoded_r.json().set("obj", Path("qaz"), "baz", xx=True) is None + + # Test that flags allow updates when conditions are met + assert await decoded_r.json().set("obj", Path("foo"), "baz", xx=True) + assert await decoded_r.json().set("obj", Path("qaz"), "baz", nx=True) + + # Test that flags are mutually exclusive + with pytest.raises(Exception): + await decoded_r.json().set("obj", Path("foo"), "baz", nx=True, xx=True) + + +@pytest.mark.redismod +async def test_mgetshouldsucceed(decoded_r: redis.Redis): + await decoded_r.json().set("1", Path.root_path(), 1) + await decoded_r.json().set("2", Path.root_path(), 2) + assert await decoded_r.json().mget(["1"], Path.root_path()) == [1] + + assert await decoded_r.json().mget([1, 2], Path.root_path()) == [1, 2] + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +@skip_ifmodversion_lt("2.6.0", "ReJSON") +async def test_mset(decoded_r: redis.Redis): + await decoded_r.json().mset( + [("1", Path.root_path(), 1), ("2", Path.root_path(), 2)] + ) + + assert await decoded_r.json().mget(["1"], Path.root_path()) == [1] + assert await decoded_r.json().mget(["1", "2"], Path.root_path()) == [1, 2] + + +@pytest.mark.redismod +@skip_ifmodversion_lt("99.99.99", "ReJSON") # todo: update after the release +async def test_clear(decoded_r: redis.Redis): + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 1 == await decoded_r.json().clear("arr", Path.root_path()) + assert_resp_response(decoded_r, await decoded_r.json().get("arr"), [], []) + + +@pytest.mark.redismod +async def test_type(decoded_r: redis.Redis): + await decoded_r.json().set("1", Path.root_path(), 1) + assert_resp_response( + decoded_r, + await decoded_r.json().type("1", Path.root_path()), + "integer", + ["integer"], + ) + assert_resp_response( + decoded_r, await decoded_r.json().type("1"), "integer", ["integer"] + ) + + +@pytest.mark.redismod +async def test_numincrby(decoded_r): + await decoded_r.json().set("num", Path.root_path(), 1) + assert_resp_response( + decoded_r, await decoded_r.json().numincrby("num", Path.root_path(), 1), 2, [2] + ) + res = await decoded_r.json().numincrby("num", Path.root_path(), 0.5) + assert_resp_response(decoded_r, res, 2.5, [2.5]) + res = await decoded_r.json().numincrby("num", Path.root_path(), -1.25) + assert_resp_response(decoded_r, res, 1.25, [1.25]) + + +@pytest.mark.redismod +async def test_nummultby(decoded_r: redis.Redis): + await decoded_r.json().set("num", Path.root_path(), 1) + + with pytest.deprecated_call(): + res = await decoded_r.json().nummultby("num", Path.root_path(), 2) + assert_resp_response(decoded_r, res, 2, [2]) + res = await decoded_r.json().nummultby("num", Path.root_path(), 2.5) + assert_resp_response(decoded_r, res, 5, [5]) + res = await decoded_r.json().nummultby("num", Path.root_path(), 0.5) + assert_resp_response(decoded_r, res, 2.5, [2.5]) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("99.99.99", "ReJSON") # todo: update after the release +async def test_toggle(decoded_r: redis.Redis): + await decoded_r.json().set("bool", Path.root_path(), False) + assert await decoded_r.json().toggle("bool", Path.root_path()) + assert await decoded_r.json().toggle("bool", Path.root_path()) is False + # check non-boolean value + await decoded_r.json().set("num", Path.root_path(), 1) + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().toggle("num", Path.root_path()) + + +@pytest.mark.redismod +async def test_strappend(decoded_r: redis.Redis): + await decoded_r.json().set("jsonkey", Path.root_path(), "foo") + assert 6 == await decoded_r.json().strappend("jsonkey", "bar") + assert "foobar" == await decoded_r.json().get("jsonkey", Path.root_path()) + + +@pytest.mark.redismod +async def test_strlen(decoded_r: redis.Redis): + await decoded_r.json().set("str", Path.root_path(), "foo") + assert 3 == await decoded_r.json().strlen("str", Path.root_path()) + await decoded_r.json().strappend("str", "bar", Path.root_path()) + assert 6 == await decoded_r.json().strlen("str", Path.root_path()) + assert 6 == await decoded_r.json().strlen("str") + + +@pytest.mark.redismod +async def test_arrappend(decoded_r: redis.Redis): + await decoded_r.json().set("arr", Path.root_path(), [1]) + assert 2 == await decoded_r.json().arrappend("arr", Path.root_path(), 2) + assert 4 == await decoded_r.json().arrappend("arr", Path.root_path(), 3, 4) + assert 7 == await decoded_r.json().arrappend("arr", Path.root_path(), *[5, 6, 7]) + + +@pytest.mark.redismod +async def test_arrindex(decoded_r: redis.Redis): + r_path = Path.root_path() + await decoded_r.json().set("arr", r_path, [0, 1, 2, 3, 4]) + assert 1 == await decoded_r.json().arrindex("arr", r_path, 1) + assert -1 == await decoded_r.json().arrindex("arr", r_path, 1, 2) + assert 4 == await decoded_r.json().arrindex("arr", r_path, 4) + assert 4 == await decoded_r.json().arrindex("arr", r_path, 4, start=0) + assert 4 == await decoded_r.json().arrindex("arr", r_path, 4, start=0, stop=5000) + assert -1 == await decoded_r.json().arrindex("arr", r_path, 4, start=0, stop=-1) + assert -1 == await decoded_r.json().arrindex("arr", r_path, 4, start=1, stop=3) + + +@pytest.mark.redismod +async def test_arrinsert(decoded_r: redis.Redis): + await decoded_r.json().set("arr", Path.root_path(), [0, 4]) + assert 5 == await decoded_r.json().arrinsert("arr", Path.root_path(), 1, *[1, 2, 3]) + assert await decoded_r.json().get("arr") == [0, 1, 2, 3, 4] + + # test prepends + await decoded_r.json().set("val2", Path.root_path(), [5, 6, 7, 8, 9]) + await decoded_r.json().arrinsert("val2", Path.root_path(), 0, ["some", "thing"]) + assert await decoded_r.json().get("val2") == [["some", "thing"], 5, 6, 7, 8, 9] + + +@pytest.mark.redismod +async def test_arrlen(decoded_r: redis.Redis): + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 5 == await decoded_r.json().arrlen("arr", Path.root_path()) + assert 5 == await decoded_r.json().arrlen("arr") + assert await decoded_r.json().arrlen("fakekey") is None + + +@pytest.mark.redismod +async def test_arrpop(decoded_r: redis.Redis): + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 4 == await decoded_r.json().arrpop("arr", Path.root_path(), 4) + assert 3 == await decoded_r.json().arrpop("arr", Path.root_path(), -1) + assert 2 == await decoded_r.json().arrpop("arr", Path.root_path()) + assert 0 == await decoded_r.json().arrpop("arr", Path.root_path(), 0) + assert [1] == await decoded_r.json().get("arr") + + # test out of bounds + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 4 == await decoded_r.json().arrpop("arr", Path.root_path(), 99) + + # none test + await decoded_r.json().set("arr", Path.root_path(), []) + assert await decoded_r.json().arrpop("arr") is None + + +@pytest.mark.redismod +async def test_arrtrim(decoded_r: redis.Redis): + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 3 == await decoded_r.json().arrtrim("arr", Path.root_path(), 1, 3) + assert [1, 2, 3] == await decoded_r.json().get("arr") + + # <0 test, should be 0 equivalent + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 0 == await decoded_r.json().arrtrim("arr", Path.root_path(), -1, 3) + + # testing stop > end + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 2 == await decoded_r.json().arrtrim("arr", Path.root_path(), 3, 99) + + # start > array size and stop + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 0 == await decoded_r.json().arrtrim("arr", Path.root_path(), 9, 1) + + # all larger + await decoded_r.json().set("arr", Path.root_path(), [0, 1, 2, 3, 4]) + assert 0 == await decoded_r.json().arrtrim("arr", Path.root_path(), 9, 11) + + +@pytest.mark.redismod +async def test_resp(decoded_r: redis.Redis): + obj = {"foo": "bar", "baz": 1, "qaz": True} + await decoded_r.json().set("obj", Path.root_path(), obj) + assert "bar" == await decoded_r.json().resp("obj", Path("foo")) + assert 1 == await decoded_r.json().resp("obj", Path("baz")) + assert await decoded_r.json().resp("obj", Path("qaz")) + assert isinstance(await decoded_r.json().resp("obj"), list) + + +@pytest.mark.redismod +async def test_objkeys(decoded_r: redis.Redis): + obj = {"foo": "bar", "baz": "qaz"} + await decoded_r.json().set("obj", Path.root_path(), obj) + keys = await decoded_r.json().objkeys("obj", Path.root_path()) + keys.sort() + exp = list(obj.keys()) + exp.sort() + assert exp == keys + + await decoded_r.json().set("obj", Path.root_path(), obj) + keys = await decoded_r.json().objkeys("obj") + assert keys == list(obj.keys()) + + assert await decoded_r.json().objkeys("fakekey") is None + + +@pytest.mark.redismod +async def test_objlen(decoded_r: redis.Redis): + obj = {"foo": "bar", "baz": "qaz"} + await decoded_r.json().set("obj", Path.root_path(), obj) + assert len(obj) == await decoded_r.json().objlen("obj", Path.root_path()) + + await decoded_r.json().set("obj", Path.root_path(), obj) + assert len(obj) == await decoded_r.json().objlen("obj") + + +# @pytest.mark.redismod +# async def test_json_commands_in_pipeline(decoded_r: redis.Redis): +# async with decoded_r.json().pipeline() as p: +# p.set("foo", Path.root_path(), "bar") +# p.get("foo") +# p.delete("foo") +# assert [True, "bar", 1] == await p.execute() +# assert await decoded_r.keys() == [] +# assert await decoded_r.get("foo") is None + +# # now with a true, json object +# await decoded_r.flushdb() +# p = await decoded_r.json().pipeline() +# d = {"hello": "world", "oh": "snap"} +# with pytest.deprecated_call(): +# p.jsonset("foo", Path.root_path(), d) +# p.jsonget("foo") +# p.exists("notarealkey") +# p.delete("foo") +# assert [True, d, 0, 1] == p.execute() +# assert await decoded_r.keys() == [] +# assert await decoded_r.get("foo") is None + + +@pytest.mark.redismod +async def test_json_delete_with_dollar(decoded_r: redis.Redis): + doc1 = {"a": 1, "nested": {"a": 2, "b": 3}} + assert await decoded_r.json().set("doc1", "$", doc1) + assert await decoded_r.json().delete("doc1", "$..a") == 2 + assert await decoded_r.json().get("doc1", "$") == [{"nested": {"b": 3}}] + + doc2 = {"a": {"a": 2, "b": 3}, "b": ["a", "b"], "nested": {"b": [True, "a", "b"]}} + assert await decoded_r.json().set("doc2", "$", doc2) + assert await decoded_r.json().delete("doc2", "$..a") == 1 + assert await decoded_r.json().get("doc2", "$") == [ + {"nested": {"b": [True, "a", "b"]}, "b": ["a", "b"]} + ] + + doc3 = [ + { + "ciao": ["non ancora"], + "nested": [ + {"ciao": [1, "a"]}, + {"ciao": [2, "a"]}, + {"ciaoc": [3, "non", "ciao"]}, + {"ciao": [4, "a"]}, + {"e": [5, "non", "ciao"]}, + ], + } + ] + assert await decoded_r.json().set("doc3", "$", doc3) + assert await decoded_r.json().delete("doc3", '$.[0]["nested"]..ciao') == 3 + + doc3val = [ + [ + { + "ciao": ["non ancora"], + "nested": [ + {}, + {}, + {"ciaoc": [3, "non", "ciao"]}, + {}, + {"e": [5, "non", "ciao"]}, + ], + } + ] + ] + assert await decoded_r.json().get("doc3", "$") == doc3val + + # Test async default path + assert await decoded_r.json().delete("doc3") == 1 + assert await decoded_r.json().get("doc3", "$") is None + + await decoded_r.json().delete("not_a_document", "..a") + + +@pytest.mark.redismod +async def test_json_forget_with_dollar(decoded_r: redis.Redis): + doc1 = {"a": 1, "nested": {"a": 2, "b": 3}} + assert await decoded_r.json().set("doc1", "$", doc1) + assert await decoded_r.json().forget("doc1", "$..a") == 2 + assert await decoded_r.json().get("doc1", "$") == [{"nested": {"b": 3}}] + + doc2 = {"a": {"a": 2, "b": 3}, "b": ["a", "b"], "nested": {"b": [True, "a", "b"]}} + assert await decoded_r.json().set("doc2", "$", doc2) + assert await decoded_r.json().forget("doc2", "$..a") == 1 + assert await decoded_r.json().get("doc2", "$") == [ + {"nested": {"b": [True, "a", "b"]}, "b": ["a", "b"]} + ] + + doc3 = [ + { + "ciao": ["non ancora"], + "nested": [ + {"ciao": [1, "a"]}, + {"ciao": [2, "a"]}, + {"ciaoc": [3, "non", "ciao"]}, + {"ciao": [4, "a"]}, + {"e": [5, "non", "ciao"]}, + ], + } + ] + assert await decoded_r.json().set("doc3", "$", doc3) + assert await decoded_r.json().forget("doc3", '$.[0]["nested"]..ciao') == 3 + + doc3val = [ + [ + { + "ciao": ["non ancora"], + "nested": [ + {}, + {}, + {"ciaoc": [3, "non", "ciao"]}, + {}, + {"e": [5, "non", "ciao"]}, + ], + } + ] + ] + assert await decoded_r.json().get("doc3", "$") == doc3val + + # Test async default path + assert await decoded_r.json().forget("doc3") == 1 + assert await decoded_r.json().get("doc3", "$") is None + + await decoded_r.json().forget("not_a_document", "..a") + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +async def test_json_mget_dollar(decoded_r: redis.Redis): + # Test mget with multi paths + await decoded_r.json().set( + "doc1", + "$", + {"a": 1, "b": 2, "nested": {"a": 3}, "c": None, "nested2": {"a": None}}, + ) + await decoded_r.json().set( + "doc2", + "$", + {"a": 4, "b": 5, "nested": {"a": 6}, "c": None, "nested2": {"a": [None]}}, + ) + # Compare also to single JSON.GET + assert await decoded_r.json().get("doc1", "$..a") == [1, 3, None] + assert await decoded_r.json().get("doc2", "$..a") == [4, 6, [None]] + + # Test mget with single path + assert await decoded_r.json().mget(["doc1"], "$..a") == [[1, 3, None]] + # Test mget with multi path + res = await decoded_r.json().mget(["doc1", "doc2"], "$..a") + assert res == [[1, 3, None], [4, 6, [None]]] + + # Test missing key + res = await decoded_r.json().mget(["doc1", "missing_doc"], "$..a") + assert res == [[1, 3, None], None] + res = await decoded_r.json().mget(["missing_doc1", "missing_doc2"], "$..a") + assert res == [None, None] + + +@pytest.mark.redismod +async def test_numby_commands_dollar(decoded_r: redis.Redis): + # Test NUMINCRBY + await decoded_r.json().set( + "doc1", "$", {"a": "b", "b": [{"a": 2}, {"a": 5.0}, {"a": "c"}]} + ) + # Test multi + assert await decoded_r.json().numincrby("doc1", "$..a", 2) == [None, 4, 7.0, None] + + res = await decoded_r.json().numincrby("doc1", "$..a", 2.5) + assert res == [None, 6.5, 9.5, None] + # Test single + assert await decoded_r.json().numincrby("doc1", "$.b[1].a", 2) == [11.5] + + assert await decoded_r.json().numincrby("doc1", "$.b[2].a", 2) == [None] + assert await decoded_r.json().numincrby("doc1", "$.b[1].a", 3.5) == [15.0] + + # Test NUMMULTBY + await decoded_r.json().set( + "doc1", "$", {"a": "b", "b": [{"a": 2}, {"a": 5.0}, {"a": "c"}]} + ) + + # test list + with pytest.deprecated_call(): + res = await decoded_r.json().nummultby("doc1", "$..a", 2) + assert res == [None, 4, 10, None] + res = await decoded_r.json().nummultby("doc1", "$..a", 2.5) + assert res == [None, 10.0, 25.0, None] + + # Test single + with pytest.deprecated_call(): + assert await decoded_r.json().nummultby("doc1", "$.b[1].a", 2) == [50.0] + assert await decoded_r.json().nummultby("doc1", "$.b[2].a", 2) == [None] + assert await decoded_r.json().nummultby("doc1", "$.b[1].a", 3) == [150.0] + + # test missing keys + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().numincrby("non_existing_doc", "$..a", 2) + await decoded_r.json().nummultby("non_existing_doc", "$..a", 2) + + # Test legacy NUMINCRBY + await decoded_r.json().set( + "doc1", "$", {"a": "b", "b": [{"a": 2}, {"a": 5.0}, {"a": "c"}]} + ) + assert_resp_response( + decoded_r, await decoded_r.json().numincrby("doc1", ".b[0].a", 3), 5, [5] + ) + + # Test legacy NUMMULTBY + await decoded_r.json().set( + "doc1", "$", {"a": "b", "b": [{"a": 2}, {"a": 5.0}, {"a": "c"}]} + ) + + with pytest.deprecated_call(): + assert_resp_response( + decoded_r, await decoded_r.json().nummultby("doc1", ".b[0].a", 3), 6, [6] + ) + + +@pytest.mark.redismod +async def test_strappend_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", "$", {"a": "foo", "nested1": {"a": "hello"}, "nested2": {"a": 31}} + ) + # Test multi + assert await decoded_r.json().strappend("doc1", "bar", "$..a") == [6, 8, None] + + res = [{"a": "foobar", "nested1": {"a": "hellobar"}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + # Test single + assert await decoded_r.json().strappend("doc1", "baz", "$.nested1.a") == [11] + + res = [{"a": "foobar", "nested1": {"a": "hellobarbaz"}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().strappend("non_existing_doc", "$..a", "err") + + # Test multi + assert await decoded_r.json().strappend("doc1", "bar", ".*.a") == 14 + res = [{"a": "foobar", "nested1": {"a": "hellobarbazbar"}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing path + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().strappend("doc1", "piu") + + +@pytest.mark.redismod +async def test_strlen_dollar(decoded_r: redis.Redis): + # Test multi + await decoded_r.json().set( + "doc1", "$", {"a": "foo", "nested1": {"a": "hello"}, "nested2": {"a": 31}} + ) + assert await decoded_r.json().strlen("doc1", "$..a") == [3, 5, None] + + res2 = await decoded_r.json().strappend("doc1", "bar", "$..a") + res1 = await decoded_r.json().strlen("doc1", "$..a") + assert res1 == res2 + + # Test single + assert await decoded_r.json().strlen("doc1", "$.nested1.a") == [8] + assert await decoded_r.json().strlen("doc1", "$.nested2.a") == [None] + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().strlen("non_existing_doc", "$..a") + + +@pytest.mark.redismod +async def test_arrappend_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + # Test multi + res = [3, 5, None] + assert await decoded_r.json().arrappend("doc1", "$..a", "bar", "racuda") == res + res = [ + { + "a": ["foo", "bar", "racuda"], + "nested1": {"a": ["hello", None, "world", "bar", "racuda"]}, + "nested2": {"a": 31}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + + # Test single + assert await decoded_r.json().arrappend("doc1", "$.nested1.a", "baz") == [6] + res = [ + { + "a": ["foo", "bar", "racuda"], + "nested1": {"a": ["hello", None, "world", "bar", "racuda", "baz"]}, + "nested2": {"a": 31}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrappend("non_existing_doc", "$..a") + + # Test legacy + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + # Test multi (all paths are updated, but return result of last path) + assert await decoded_r.json().arrappend("doc1", "..a", "bar", "racuda") == 5 + + res = [ + { + "a": ["foo", "bar", "racuda"], + "nested1": {"a": ["hello", None, "world", "bar", "racuda"]}, + "nested2": {"a": 31}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + # Test single + assert await decoded_r.json().arrappend("doc1", ".nested1.a", "baz") == 6 + res = [ + { + "a": ["foo", "bar", "racuda"], + "nested1": {"a": ["hello", None, "world", "bar", "racuda", "baz"]}, + "nested2": {"a": 31}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrappend("non_existing_doc", "$..a") + + +@pytest.mark.redismod +async def test_arrinsert_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + # Test multi + res = await decoded_r.json().arrinsert("doc1", "$..a", "1", "bar", "racuda") + assert res == [3, 5, None] + + res = [ + { + "a": ["foo", "bar", "racuda"], + "nested1": {"a": ["hello", "bar", "racuda", None, "world"]}, + "nested2": {"a": 31}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + # Test single + assert await decoded_r.json().arrinsert("doc1", "$.nested1.a", -2, "baz") == [6] + res = [ + { + "a": ["foo", "bar", "racuda"], + "nested1": {"a": ["hello", "bar", "racuda", "baz", None, "world"]}, + "nested2": {"a": 31}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrappend("non_existing_doc", "$..a") + + +@pytest.mark.redismod +async def test_arrlen_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + + # Test multi + assert await decoded_r.json().arrlen("doc1", "$..a") == [1, 3, None] + res = await decoded_r.json().arrappend("doc1", "$..a", "non", "abba", "stanza") + assert res == [4, 6, None] + + await decoded_r.json().clear("doc1", "$.a") + assert await decoded_r.json().arrlen("doc1", "$..a") == [0, 6, None] + # Test single + assert await decoded_r.json().arrlen("doc1", "$.nested1.a") == [6] + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrappend("non_existing_doc", "$..a") + + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + # Test multi (return result of last path) + assert await decoded_r.json().arrlen("doc1", "$..a") == [1, 3, None] + assert await decoded_r.json().arrappend("doc1", "..a", "non", "abba", "stanza") == 6 + + # Test single + assert await decoded_r.json().arrlen("doc1", ".nested1.a") == 6 + + # Test missing key + assert await decoded_r.json().arrlen("non_existing_doc", "..a") is None + + +@pytest.mark.redismod +async def test_arrpop_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + + # Test multi + assert await decoded_r.json().arrpop("doc1", "$..a", 1) == ['"foo"', None, None] + + res = [{"a": [], "nested1": {"a": ["hello", "world"]}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrpop("non_existing_doc", "..a") + + # # Test legacy + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + # Test multi (all paths are updated, but return result of last path) + assert await decoded_r.json().arrpop("doc1", "..a", "1") == "null" + res = [{"a": [], "nested1": {"a": ["hello", "world"]}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + # # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrpop("non_existing_doc", "..a") + + +@pytest.mark.redismod +async def test_arrtrim_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + # Test multi + assert await decoded_r.json().arrtrim("doc1", "$..a", "1", -1) == [0, 2, None] + res = [{"a": [], "nested1": {"a": [None, "world"]}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + assert await decoded_r.json().arrtrim("doc1", "$..a", "1", "1") == [0, 1, None] + res = [{"a": [], "nested1": {"a": ["world"]}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + # Test single + assert await decoded_r.json().arrtrim("doc1", "$.nested1.a", 1, 0) == [0] + res = [{"a": [], "nested1": {"a": []}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrtrim("non_existing_doc", "..a", "0", 1) + + # Test legacy + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": ["hello", None, "world"]}, + "nested2": {"a": 31}, + }, + ) + + # Test multi (all paths are updated, but return result of last path) + assert await decoded_r.json().arrtrim("doc1", "..a", "1", "-1") == 2 + + # Test single + assert await decoded_r.json().arrtrim("doc1", ".nested1.a", "1", "1") == 1 + res = [{"a": [], "nested1": {"a": ["world"]}, "nested2": {"a": 31}}] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().arrtrim("non_existing_doc", "..a", 1, 1) + + +@pytest.mark.redismod +async def test_objkeys_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "nested1": {"a": {"foo": 10, "bar": 20}}, + "a": ["foo"], + "nested2": {"a": {"baz": 50}}, + }, + ) + + # Test single + assert await decoded_r.json().objkeys("doc1", "$.nested1.a") == [["foo", "bar"]] + + # Test legacy + assert await decoded_r.json().objkeys("doc1", ".*.a") == ["foo", "bar"] + # Test single + assert await decoded_r.json().objkeys("doc1", ".nested2.a") == ["baz"] + + # Test missing key + assert await decoded_r.json().objkeys("non_existing_doc", "..a") is None + + # Test non existing doc + with pytest.raises(exceptions.ResponseError): + assert await decoded_r.json().objkeys("non_existing_doc", "$..a") == [] + + assert await decoded_r.json().objkeys("doc1", "$..nowhere") == [] + + +@pytest.mark.redismod +async def test_objlen_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "nested1": {"a": {"foo": 10, "bar": 20}}, + "a": ["foo"], + "nested2": {"a": {"baz": 50}}, + }, + ) + # Test multi + assert await decoded_r.json().objlen("doc1", "$..a") == [None, 2, 1] + # Test single + assert await decoded_r.json().objlen("doc1", "$.nested1.a") == [2] + + # Test missing key, and path + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().objlen("non_existing_doc", "$..a") + + assert await decoded_r.json().objlen("doc1", "$.nowhere") == [] + + # Test legacy + assert await decoded_r.json().objlen("doc1", ".*.a") == 2 + + # Test single + assert await decoded_r.json().objlen("doc1", ".nested2.a") == 1 + + # Test missing key + assert await decoded_r.json().objlen("non_existing_doc", "..a") is None + + # Test missing path + # with pytest.raises(exceptions.ResponseError): + await decoded_r.json().objlen("doc1", ".nowhere") + + +def load_types_data(nested_key_name): + td = { + "object": {}, + "array": [], + "string": "str", + "integer": 42, + "number": 1.2, + "boolean": False, + "null": None, + } + jdata = {} + types = [] + for i, (k, v) in zip(range(1, len(td) + 1), iter(td.items())): + jdata["nested" + str(i)] = {nested_key_name: v} + types.append(k) + + return jdata, types + + +@pytest.mark.redismod +async def test_type_dollar(decoded_r: redis.Redis): + jdata, jtypes = load_types_data("a") + await decoded_r.json().set("doc1", "$", jdata) + # Test multi + assert_resp_response( + decoded_r, await decoded_r.json().type("doc1", "$..a"), jtypes, [jtypes] + ) + + # Test single + res = await decoded_r.json().type("doc1", "$.nested2.a") + assert_resp_response(decoded_r, res, [jtypes[1]], [[jtypes[1]]]) + + # Test missing key + assert_resp_response( + decoded_r, await decoded_r.json().type("non_existing_doc", "..a"), None, [None] + ) + + +@pytest.mark.redismod +async def test_clear_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "nested1": {"a": {"foo": 10, "bar": 20}}, + "a": ["foo"], + "nested2": {"a": "claro"}, + "nested3": {"a": {"baz": 50}}, + }, + ) + + # Test multi + assert await decoded_r.json().clear("doc1", "$..a") == 3 + + res = [ + {"nested1": {"a": {}}, "a": [], "nested2": {"a": "claro"}, "nested3": {"a": {}}} + ] + assert await decoded_r.json().get("doc1", "$") == res + + # Test single + await decoded_r.json().set( + "doc1", + "$", + { + "nested1": {"a": {"foo": 10, "bar": 20}}, + "a": ["foo"], + "nested2": {"a": "claro"}, + "nested3": {"a": {"baz": 50}}, + }, + ) + assert await decoded_r.json().clear("doc1", "$.nested1.a") == 1 + res = [ + { + "nested1": {"a": {}}, + "a": ["foo"], + "nested2": {"a": "claro"}, + "nested3": {"a": {"baz": 50}}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing path (async defaults to root) + assert await decoded_r.json().clear("doc1") == 1 + assert await decoded_r.json().get("doc1", "$") == [{}] + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().clear("non_existing_doc", "$..a") + + +@pytest.mark.redismod +async def test_toggle_dollar(decoded_r: redis.Redis): + await decoded_r.json().set( + "doc1", + "$", + { + "a": ["foo"], + "nested1": {"a": False}, + "nested2": {"a": 31}, + "nested3": {"a": True}, + }, + ) + # Test multi + assert await decoded_r.json().toggle("doc1", "$..a") == [None, 1, None, 0] + res = [ + { + "a": ["foo"], + "nested1": {"a": True}, + "nested2": {"a": 31}, + "nested3": {"a": False}, + } + ] + assert await decoded_r.json().get("doc1", "$") == res + + # Test missing key + with pytest.raises(exceptions.ResponseError): + await decoded_r.json().toggle("non_existing_doc", "$..a") diff --git a/tests/test_anyio/test_lock.py b/tests/test_anyio/test_lock.py new file mode 100644 index 0000000000..c537b4c136 --- /dev/null +++ b/tests/test_anyio/test_lock.py @@ -0,0 +1,271 @@ +import anyio +import pytest +from redis.anyio.lock import Lock +from redis.exceptions import LockError, LockNotOwnedError + +pytestmark = pytest.mark.anyio + + +class TestLock: + @pytest.fixture + async def r_decoded(self, create_redis): + async with create_redis(decode_responses=True) as client: + yield client + await client.flushall() + + def get_lock(self, redis, *args, **kwargs): + kwargs["lock_class"] = Lock + return redis.lock(*args, **kwargs) + + async def test_lock(self, r): + lock = self.get_lock(r, "foo") + assert await lock.acquire(blocking=False) + assert await r.get("foo") == lock.local.token + assert await r.ttl("foo") == -1 + await lock.release() + assert await r.get("foo") is None + + async def test_lock_token(self, r): + lock = self.get_lock(r, "foo") + await self._test_lock_token(r, lock) + + async def test_lock_token_thread_local_false(self, r): + lock = self.get_lock(r, "foo", thread_local=False) + await self._test_lock_token(r, lock) + + async def _test_lock_token(self, r, lock): + assert await lock.acquire(blocking=False, token="test") + assert await r.get("foo") == b"test" + assert lock.local.token == b"test" + assert await r.ttl("foo") == -1 + await lock.release() + assert await r.get("foo") is None + assert lock.local.token is None + + async def test_locked(self, r): + lock = self.get_lock(r, "foo") + assert await lock.locked() is False + await lock.acquire(blocking=False) + assert await lock.locked() is True + await lock.release() + assert await lock.locked() is False + + async def _test_owned(self, client): + lock = self.get_lock(client, "foo") + assert await lock.owned() is False + await lock.acquire(blocking=False) + assert await lock.owned() is True + await lock.release() + assert await lock.owned() is False + + lock2 = self.get_lock(client, "foo") + assert await lock.owned() is False + assert await lock2.owned() is False + await lock2.acquire(blocking=False) + assert await lock.owned() is False + assert await lock2.owned() is True + await lock2.release() + assert await lock.owned() is False + assert await lock2.owned() is False + + async def test_owned(self, r): + await self._test_owned(r) + + async def test_owned_with_decoded_responses(self, r_decoded): + await self._test_owned(r_decoded) + + async def test_competing_locks(self, r): + lock1 = self.get_lock(r, "foo") + lock2 = self.get_lock(r, "foo") + assert await lock1.acquire(blocking=False) + assert not await lock2.acquire(blocking=False) + await lock1.release() + assert await lock2.acquire(blocking=False) + assert not await lock1.acquire(blocking=False) + await lock2.release() + + async def test_timeout(self, r): + lock = self.get_lock(r, "foo", timeout=10) + assert await lock.acquire(blocking=False) + assert 8 < (await r.ttl("foo")) <= 10 + await lock.release() + + async def test_float_timeout(self, r): + lock = self.get_lock(r, "foo", timeout=9.5) + assert await lock.acquire(blocking=False) + assert 8 < (await r.pttl("foo")) <= 9500 + await lock.release() + + async def test_blocking(self, r): + blocking = False + lock = self.get_lock(r, "foo", blocking=blocking) + assert not lock.blocking + + lock_2 = self.get_lock(r, "foo") + assert lock_2.blocking + + async def test_blocking_timeout(self, r): + lock1 = self.get_lock(r, "foo") + assert await lock1.acquire(blocking=False) + bt = 0.2 + sleep = 0.05 + lock2 = self.get_lock(r, "foo", sleep=sleep, blocking_timeout=bt) + start = anyio.current_time() + assert not await lock2.acquire() + # The elapsed duration should be less than the total blocking_timeout + assert bt >= (anyio.current_time() - start) > bt - sleep + await lock1.release() + + async def test_context_manager(self, r): + # blocking_timeout prevents a deadlock if the lock can't be acquired + # for some reason + async with self.get_lock(r, "foo", blocking_timeout=0.2) as lock: + assert await r.get("foo") == lock.local.token + assert await r.get("foo") is None + + async def test_context_manager_raises_when_locked_not_acquired(self, r): + await r.set("foo", "bar") + with pytest.raises(LockError): + async with self.get_lock(r, "foo", blocking_timeout=0.1): + pass + + async def test_context_manager_not_raise_on_release_lock_not_owned_error(self, r): + try: + async with self.get_lock( + r, "foo", timeout=0.1, raise_on_release_error=False + ): + await anyio.sleep(0.15) + except LockNotOwnedError: + pytest.fail("LockNotOwnedError should not have been raised") + + with pytest.raises(LockNotOwnedError): + async with self.get_lock( + r, "foo", timeout=0.1, raise_on_release_error=True + ): + await anyio.sleep(0.15) + + async def test_context_manager_not_raise_on_release_lock_error(self, r): + try: + async with self.get_lock( + r, "foo", timeout=0.1, raise_on_release_error=False + ) as lock: + await lock.release() + except LockError: + pytest.fail("LockError should not have been raised") + + with pytest.raises(LockError): + async with self.get_lock( + r, "foo", timeout=0.1, raise_on_release_error=True + ) as lock: + await lock.release() + + async def test_high_sleep_small_blocking_timeout(self, r): + lock1 = self.get_lock(r, "foo") + assert await lock1.acquire(blocking=False) + sleep = 60 + bt = 1 + lock2 = self.get_lock(r, "foo", sleep=sleep, blocking_timeout=bt) + start = anyio.current_time() + assert not await lock2.acquire() + # the elapsed timed is less than the blocking_timeout as the lock is + # unattainable given the sleep/blocking_timeout configuration + assert bt > (anyio.current_time() - start) + await lock1.release() + + async def test_releasing_unlocked_lock_raises_error(self, r): + lock = self.get_lock(r, "foo") + with pytest.raises(LockError): + await lock.release() + + async def test_releasing_lock_no_longer_owned_raises_error(self, r): + lock = self.get_lock(r, "foo") + await lock.acquire(blocking=False) + # manually change the token + await r.set("foo", "a") + with pytest.raises(LockNotOwnedError): + await lock.release() + # even though we errored, the token is still cleared + assert lock.local.token is None + + async def test_extend_lock(self, r): + lock = self.get_lock(r, "foo", timeout=10) + assert await lock.acquire(blocking=False) + assert 8000 < (await r.pttl("foo")) <= 10000 + assert await lock.extend(10) + assert 16000 < (await r.pttl("foo")) <= 20000 + await lock.release() + + async def test_extend_lock_replace_ttl(self, r): + lock = self.get_lock(r, "foo", timeout=10) + assert await lock.acquire(blocking=False) + assert 8000 < (await r.pttl("foo")) <= 10000 + assert await lock.extend(10, replace_ttl=True) + assert 8000 < (await r.pttl("foo")) <= 10000 + await lock.release() + + async def test_extend_lock_float(self, r): + lock = self.get_lock(r, "foo", timeout=10.5) + assert await lock.acquire(blocking=False) + assert 10400 < (await r.pttl("foo")) <= 10500 + old_ttl = await r.pttl("foo") + assert await lock.extend(10.5) + assert old_ttl + 10400 < (await r.pttl("foo")) <= old_ttl + 10500 + await lock.release() + + async def test_extending_unlocked_lock_raises_error(self, r): + lock = self.get_lock(r, "foo", timeout=10) + with pytest.raises(LockError): + await lock.extend(10) + + async def test_extending_lock_with_no_timeout_raises_error(self, r): + lock = self.get_lock(r, "foo") + assert await lock.acquire(blocking=False) + with pytest.raises(LockError): + await lock.extend(10) + await lock.release() + + async def test_extending_lock_no_longer_owned_raises_error(self, r): + lock = self.get_lock(r, "foo", timeout=10) + assert await lock.acquire(blocking=False) + await r.set("foo", "a") + with pytest.raises(LockNotOwnedError): + await lock.extend(10) + + async def test_reacquire_lock(self, r): + lock = self.get_lock(r, "foo", timeout=10) + assert await lock.acquire(blocking=False) + assert await r.pexpire("foo", 5000) + assert await r.pttl("foo") <= 5000 + assert await lock.reacquire() + assert 8000 < (await r.pttl("foo")) <= 10000 + await lock.release() + + async def test_reacquiring_unlocked_lock_raises_error(self, r): + lock = self.get_lock(r, "foo", timeout=10) + with pytest.raises(LockError): + await lock.reacquire() + + async def test_reacquiring_lock_with_no_timeout_raises_error(self, r): + lock = self.get_lock(r, "foo") + assert await lock.acquire(blocking=False) + with pytest.raises(LockError): + await lock.reacquire() + await lock.release() + + async def test_reacquiring_lock_no_longer_owned_raises_error(self, r): + lock = self.get_lock(r, "foo", timeout=10) + assert await lock.acquire(blocking=False) + await r.set("foo", "a") + with pytest.raises(LockNotOwnedError): + await lock.reacquire() + + +@pytest.mark.onlynoncluster +class TestLockClassSelection: + async def test_lock_class_argument(self, r): + class MyLock: + def __init__(self, *args, **kwargs): + pass + + lock = r.lock("foo", lock_class=MyLock) + assert isinstance(lock, MyLock) diff --git a/tests/test_anyio/test_monitor.py b/tests/test_anyio/test_monitor.py new file mode 100644 index 0000000000..60e45f34e8 --- /dev/null +++ b/tests/test_anyio/test_monitor.py @@ -0,0 +1,66 @@ +import pytest +from tests.conftest import skip_if_redis_enterprise, skip_ifnot_redis_enterprise + +from .conftest import wait_for_command + +pytestmark = pytest.mark.anyio + + +@pytest.mark.onlynoncluster +class TestMonitor: + async def test_wait_command_not_found(self, r): + """Make sure the wait_for_command func works when command is not found""" + async with r.monitor() as m: + response = await wait_for_command(r, m, "nothing") + assert response is None + + async def test_response_values(self, r): + db = r.connection_pool.connection_kwargs.get("db", 0) + async with r.monitor() as m: + await r.ping() + response = await wait_for_command(r, m, "PING") + assert isinstance(response["time"], float) + assert response["db"] == db + assert response["client_type"] in ("tcp", "unix") + assert isinstance(response["client_address"], str) + assert isinstance(response["client_port"], str) + assert response["command"] == "PING" + + async def test_command_with_quoted_key(self, r): + async with r.monitor() as m: + await r.get('foo"bar') + response = await wait_for_command(r, m, 'GET foo"bar') + assert response["command"] == 'GET foo"bar' + + async def test_command_with_binary_data(self, r): + async with r.monitor() as m: + byte_string = b"foo\x92" + await r.get(byte_string) + response = await wait_for_command(r, m, "GET foo\\x92") + assert response["command"] == "GET foo\\x92" + + async def test_command_with_escaped_data(self, r): + async with r.monitor() as m: + byte_string = b"foo\\x92" + await r.get(byte_string) + response = await wait_for_command(r, m, "GET foo\\\\x92") + assert response["command"] == "GET foo\\\\x92" + + @skip_if_redis_enterprise() + async def test_lua_script(self, r): + async with r.monitor() as m: + script = 'return redis.call("GET", "foo")' + assert await r.eval(script, 0) is None + response = await wait_for_command(r, m, "GET foo") + assert response["command"] == "GET foo" + assert response["client_type"] == "lua" + assert response["client_address"] == "lua" + assert response["client_port"] == "" + + @skip_ifnot_redis_enterprise() + async def test_lua_script_in_enterprise(self, r): + async with r.monitor() as m: + script = 'return redis.call("GET", "foo")' + assert await r.eval(script, 0) is None + response = await wait_for_command(r, m, "GET foo") + assert response is None diff --git a/tests/test_anyio/test_pipeline.py b/tests/test_anyio/test_pipeline.py new file mode 100644 index 0000000000..1bd932346f --- /dev/null +++ b/tests/test_anyio/test_pipeline.py @@ -0,0 +1,433 @@ +from unittest import mock + +import pytest +import redis +from redis.anyio.compat import aclosing +from tests.conftest import skip_if_server_version_lt + +from .conftest import wait_for_command + +pytestmark = pytest.mark.anyio + + +class TestPipeline: + async def test_pipeline_is_true(self, r): + """Ensure pipeline instances are not false-y""" + async with r.pipeline() as pipe: + assert pipe + + async def test_pipeline(self, r): + async with r.pipeline() as pipe: + ( + pipe.set("a", "a1") + .get("a") + .zadd("z", {"z1": 1}) + .zadd("z", {"z2": 4}) + .zincrby("z", 1, "z1") + ) + assert await pipe.execute() == [ + True, + b"a1", + True, + True, + 2.0, + ] + + async def test_pipeline_memoryview(self, r): + async with r.pipeline() as pipe: + (pipe.set("a", memoryview(b"a1")).get("a")) + assert await pipe.execute() == [True, b"a1"] + + async def test_pipeline_length(self, r): + async with r.pipeline() as pipe: + # Initially empty. + assert len(pipe) == 0 + + # Fill 'er up! + pipe.set("a", "a1").set("b", "b1").set("c", "c1") + assert len(pipe) == 3 + + # Execute calls reset(), so empty once again. + await pipe.execute() + assert len(pipe) == 0 + + async def test_pipeline_no_transaction(self, r): + async with r.pipeline(transaction=False) as pipe: + pipe.set("a", "a1").set("b", "b1").set("c", "c1") + assert await pipe.execute() == [True, True, True] + assert await r.get("a") == b"a1" + assert await r.get("b") == b"b1" + assert await r.get("c") == b"c1" + + @pytest.mark.onlynoncluster + async def test_pipeline_no_transaction_watch(self, r): + await r.set("a", 0) + + async with r.pipeline(transaction=False) as pipe: + await pipe.watch("a") + a = await pipe.get("a") + + pipe.multi() + pipe.set("a", int(a) + 1) + assert await pipe.execute() == [True] + + @pytest.mark.onlynoncluster + async def test_pipeline_no_transaction_watch_failure(self, r): + await r.set("a", 0) + + async with r.pipeline(transaction=False) as pipe: + await pipe.watch("a") + a = await pipe.get("a") + + await r.set("a", "bad") + + pipe.multi() + pipe.set("a", int(a) + 1) + + with pytest.raises(redis.WatchError): + await pipe.execute() + + assert await r.get("a") == b"bad" + + async def test_exec_error_in_response(self, r): + """ + an invalid pipeline command at exec time adds the exception instance + to the list of returned values + """ + await r.set("c", "a") + async with r.pipeline() as pipe: + pipe.set("a", 1).set("b", 2).lpush("c", 3).set("d", 4) + result = await pipe.execute(raise_on_error=False) + + assert result[0] + assert await r.get("a") == b"1" + assert result[1] + assert await r.get("b") == b"2" + + # we can't lpush to a key that's a string value, so this should + # be a ResponseError exception + assert isinstance(result[2], redis.ResponseError) + assert await r.get("c") == b"a" + + # since this isn't a transaction, the other commands after the + # error are still executed + assert result[3] + assert await r.get("d") == b"4" + + # make sure the pipe was restored to a working state + assert await pipe.set("z", "zzz").execute() == [True] + assert await r.get("z") == b"zzz" + + async def test_exec_error_raised(self, r): + await r.set("c", "a") + async with r.pipeline() as pipe: + pipe.set("a", 1).set("b", 2).lpush("c", 3).set("d", 4) + with pytest.raises(redis.ResponseError) as ex: + await pipe.execute() + assert str(ex.value).startswith( + "Command # 3 (LPUSH c 3) of pipeline caused error: " + ) + + # make sure the pipe was restored to a working state + assert await pipe.set("z", "zzz").execute() == [True] + assert await r.get("z") == b"zzz" + + @pytest.mark.onlynoncluster + async def test_transaction_with_empty_error_command(self, r): + """ + Commands with custom EMPTY_ERROR functionality return their default + values in the pipeline no matter the raise_on_error preference + """ + for error_switch in (True, False): + async with r.pipeline() as pipe: + pipe.set("a", 1).mget([]).set("c", 3) + result = await pipe.execute(raise_on_error=error_switch) + + assert result[0] + assert result[1] == [] + assert result[2] + + @pytest.mark.onlynoncluster + async def test_pipeline_with_empty_error_command(self, r): + """ + Commands with custom EMPTY_ERROR functionality return their default + values in the pipeline no matter the raise_on_error preference + """ + for error_switch in (True, False): + async with r.pipeline(transaction=False) as pipe: + pipe.set("a", 1).mget([]).set("c", 3) + result = await pipe.execute(raise_on_error=error_switch) + + assert result[0] + assert result[1] == [] + assert result[2] + + async def test_parse_error_raised(self, r): + async with r.pipeline() as pipe: + # the zrem is invalid because we don't pass any keys to it + pipe.set("a", 1).zrem("b").set("b", 2) + with pytest.raises(redis.ResponseError) as ex: + await pipe.execute() + + assert str(ex.value).startswith( + "Command # 2 (ZREM b) of pipeline caused error: " + ) + + # make sure the pipe was restored to a working state + assert await pipe.set("z", "zzz").execute() == [True] + assert await r.get("z") == b"zzz" + + @pytest.mark.onlynoncluster + async def test_parse_error_raised_transaction(self, r): + async with r.pipeline() as pipe: + pipe.multi() + # the zrem is invalid because we don't pass any keys to it + pipe.set("a", 1).zrem("b").set("b", 2) + with pytest.raises(redis.ResponseError) as ex: + await pipe.execute() + + assert str(ex.value).startswith( + "Command # 2 (ZREM b) of pipeline caused error: " + ) + + # make sure the pipe was restored to a working state + assert await pipe.set("z", "zzz").execute() == [True] + assert await r.get("z") == b"zzz" + + @pytest.mark.onlynoncluster + async def test_watch_succeed(self, r): + await r.set("a", 1) + await r.set("b", 2) + + async with r.pipeline() as pipe: + await pipe.watch("a", "b") + assert pipe.watching + a_value = await pipe.get("a") + b_value = await pipe.get("b") + assert a_value == b"1" + assert b_value == b"2" + pipe.multi() + + pipe.set("c", 3) + assert await pipe.execute() == [True] + assert not pipe.watching + + @pytest.mark.onlynoncluster + async def test_watch_failure(self, r): + await r.set("a", 1) + await r.set("b", 2) + + async with r.pipeline() as pipe: + await pipe.watch("a", "b") + await r.set("b", 3) + pipe.multi() + pipe.get("a") + with pytest.raises(redis.WatchError): + await pipe.execute() + + assert not pipe.watching + + @pytest.mark.onlynoncluster + async def test_watch_failure_in_empty_transaction(self, r): + await r.set("a", 1) + await r.set("b", 2) + + async with r.pipeline() as pipe: + await pipe.watch("a", "b") + await r.set("b", 3) + pipe.multi() + with pytest.raises(redis.WatchError): + await pipe.execute() + + assert not pipe.watching + + @pytest.mark.onlynoncluster + async def test_unwatch(self, r): + await r.set("a", 1) + await r.set("b", 2) + + async with r.pipeline() as pipe: + await pipe.watch("a", "b") + await r.set("b", 3) + await pipe.unwatch() + assert not pipe.watching + pipe.get("a") + assert await pipe.execute() == [b"1"] + + @pytest.mark.onlynoncluster + async def test_watch_exec_no_unwatch(self, r): + await r.set("a", 1) + await r.set("b", 2) + + async with r.monitor() as m: + async with r.pipeline() as pipe: + await pipe.watch("a", "b") + assert pipe.watching + a_value = await pipe.get("a") + b_value = await pipe.get("b") + assert a_value == b"1" + assert b_value == b"2" + pipe.multi() + pipe.set("c", 3) + assert await pipe.execute() == [True] + assert not pipe.watching + + unwatch_command = await wait_for_command(r, m, "UNWATCH") + assert unwatch_command is None, "should not send UNWATCH" + + @pytest.mark.onlynoncluster + async def test_watch_reset_unwatch(self, r): + await r.set("a", 1) + + async with r.monitor() as m: + async with r.pipeline() as pipe: + await pipe.watch("a") + assert pipe.watching + await pipe.reset() + assert not pipe.watching + + unwatch_command = await wait_for_command(r, m, "UNWATCH") + assert unwatch_command is not None + assert unwatch_command["command"] == "UNWATCH" + + @pytest.mark.onlynoncluster + async def test_aclose_is_reset(self, r): + async with r.pipeline() as pipe: + called = 0 + + async def mock_reset(): + nonlocal called + called += 1 + + with mock.patch.object(pipe, "reset", mock_reset): + await pipe.aclose() + assert called == 1 + + @pytest.mark.onlynoncluster + async def test_aclosing(self, r): + async with aclosing(r.pipeline()): + pass + + @pytest.mark.onlynoncluster + async def test_transaction_callable(self, r): + await r.set("a", 1) + await r.set("b", 2) + has_run = [] + + async def my_transaction(pipe): + a_value = await pipe.get("a") + assert a_value in (b"1", b"2") + b_value = await pipe.get("b") + assert b_value == b"2" + + # silly run-once code... incr's "a" so WatchError should be raised + # forcing this all to run again. this should incr "a" once to "2" + if not has_run: + await r.incr("a") + has_run.append("it has") + + pipe.multi() + pipe.set("c", int(a_value) + int(b_value)) + + result = await r.transaction(my_transaction, "a", "b") + assert result == [True] + assert await r.get("c") == b"4" + + @pytest.mark.onlynoncluster + async def test_transaction_callable_returns_value_from_callable(self, r): + async def callback(pipe): + # No need to do anything here since we only want the return value + return "a" + + res = await r.transaction(callback, "my-key", value_from_callable=True) + assert res == "a" + + async def test_exec_error_in_no_transaction_pipeline(self, r): + await r.set("a", 1) + async with r.pipeline(transaction=False) as pipe: + pipe.llen("a") + pipe.expire("a", 100) + + with pytest.raises(redis.ResponseError) as ex: + await pipe.execute() + + assert str(ex.value).startswith( + "Command # 1 (LLEN a) of pipeline caused error: " + ) + + assert await r.get("a") == b"1" + + async def test_exec_error_in_no_transaction_pipeline_unicode_command(self, r): + key = chr(3456) + "abcd" + chr(3421) + await r.set(key, 1) + async with r.pipeline(transaction=False) as pipe: + pipe.llen(key) + pipe.expire(key, 100) + + with pytest.raises(redis.ResponseError) as ex: + await pipe.execute() + + expected = f"Command # 1 (LLEN {key}) of pipeline caused error: " + assert str(ex.value).startswith(expected) + + assert await r.get(key) == b"1" + + async def test_pipeline_with_bitfield(self, r): + async with r.pipeline() as pipe: + pipe.set("a", "1") + bf = pipe.bitfield("b") + pipe2 = ( + bf.set("u8", 8, 255) + .get("u8", 0) + .get("u4", 8) # 1111 + .get("u4", 12) # 1111 + .get("u4", 13) # 1110 + .execute() + ) + pipe.get("a") + response = await pipe.execute() + + assert pipe == pipe2 + assert response == [True, [0, 0, 15, 15, 14], b"1"] + + async def test_pipeline_get(self, r): + await r.set("a", "a1") + async with r.pipeline() as pipe: + pipe.get("a") + assert await pipe.execute() == [b"a1"] + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("2.0.0") + async def test_pipeline_discard(self, r): + # empty pipeline should raise an error + async with r.pipeline() as pipe: + pipe.set("key", "someval") + await pipe.discard() + with pytest.raises(redis.exceptions.ResponseError): + await pipe.execute() + + # setting a pipeline and discarding should do the same + async with r.pipeline() as pipe: + pipe.set("key", "someval") + pipe.set("someotherkey", "val") + response = await pipe.execute() + pipe.set("key", "another value!") + await pipe.discard() + pipe.set("key", "another vae!") + with pytest.raises(redis.exceptions.ResponseError): + await pipe.execute() + + pipe.set("foo", "bar") + response = await pipe.execute() + assert response[0] + assert await r.get("foo") == b"bar" + + @pytest.mark.onlynoncluster + async def test_send_set_commands_over_async_pipeline(self, r: redis.anyio.Redis): + pipe = r.pipeline() + pipe.hset("hash:1", "foo", "bar") + pipe.hset("hash:1", "bar", "foo") + pipe.hset("hash:1", "baz", "bar") + pipe.hgetall("hash:1") + resp = await pipe.execute() + assert resp == [1, 1, 1, {b"bar": b"foo", b"baz": b"bar", b"foo": b"bar"}] diff --git a/tests/test_anyio/test_pubsub.py b/tests/test_anyio/test_pubsub.py new file mode 100644 index 0000000000..512162f334 --- /dev/null +++ b/tests/test_anyio/test_pubsub.py @@ -0,0 +1,1056 @@ +from __future__ import annotations + +import functools +import socket +from contextlib import asynccontextmanager +from typing import Optional +from unittest import mock +from unittest.mock import patch + +import anyio.lowlevel +import pytest +from anyio.abc import TaskStatus +from anyio.streams.memory import MemoryObjectReceiveStream, MemoryObjectSendStream + +import redis.anyio as redis +from redis.anyio.compat import aclosing +from redis.anyio.utils import wait_for_condition +from redis.exceptions import ConnectionError +from redis.typing import EncodableT +from redis.utils import HIREDIS_AVAILABLE +from tests.conftest import get_protocol_version, skip_if_server_version_lt + +pytestmark = pytest.mark.anyio + + +def with_timeout(t): + def wrapper(corofunc): + @functools.wraps(corofunc) + async def run(*args, **kwargs): + with anyio.fail_after(t): + return await corofunc(*args, **kwargs) + + return run + + return wrapper + + +async def wait_for_message(pubsub, timeout=0.2, ignore_subscribe_messages=False): + with anyio.move_on_after(timeout): + while True: + message = await pubsub.get_message( + ignore_subscribe_messages=ignore_subscribe_messages + ) + if message is not None: + return message + + return None + + +def make_message( + type, channel: Optional[str], data: EncodableT, pattern: Optional[str] = None +): + return { + "type": type, + "pattern": pattern and pattern.encode("utf-8") or None, + "channel": channel and channel.encode("utf-8") or None, + "data": data.encode("utf-8") if isinstance(data, str) else data, + } + + +def make_subscribe_test_data(pubsub, type): + if type == "channel": + return { + "p": pubsub, + "sub_type": "subscribe", + "unsub_type": "unsubscribe", + "sub_func": pubsub.subscribe, + "unsub_func": pubsub.unsubscribe, + "keys": ["foo", "bar", "uni" + chr(4456) + "code"], + } + elif type == "pattern": + return { + "p": pubsub, + "sub_type": "psubscribe", + "unsub_type": "punsubscribe", + "sub_func": pubsub.psubscribe, + "unsub_func": pubsub.punsubscribe, + "keys": ["f*", "b*", "uni" + chr(4456) + "*"], + } + assert False, f"invalid subscribe type: {type}" + + +@pytest.fixture +async def pubsub(r: redis.Redis): + async with r.pubsub() as p: + yield p + + +@pytest.mark.onlynoncluster +class TestPubSubSubscribeUnsubscribe: + async def _test_subscribe_unsubscribe( + self, p, sub_type, unsub_type, sub_func, unsub_func, keys + ): + for key in keys: + assert await sub_func(key) is None + + # should be a message for each channel/pattern we just subscribed to + for i, key in enumerate(keys): + assert await wait_for_message(p) == make_message(sub_type, key, i + 1) + + for key in keys: + assert await unsub_func(key) is None + + # should be a message for each channel/pattern we just unsubscribed + # from + for i, key in enumerate(keys): + i = len(keys) - 1 - i + assert await wait_for_message(p) == make_message(unsub_type, key, i) + + async def test_channel_subscribe_unsubscribe(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "channel") + await self._test_subscribe_unsubscribe(**kwargs) + + async def test_pattern_subscribe_unsubscribe(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "pattern") + await self._test_subscribe_unsubscribe(**kwargs) + + @pytest.mark.onlynoncluster + async def _test_resubscribe_on_reconnection( + self, p, sub_type, unsub_type, sub_func, unsub_func, keys + ): + for key in keys: + assert await sub_func(key) is None + + # should be a message for each channel/pattern we just subscribed to + for i, key in enumerate(keys): + assert await wait_for_message(p) == make_message(sub_type, key, i + 1) + + # manually disconnect + await p.connection.disconnect() + + # calling get_message again reconnects and resubscribes + # note, we may not re-subscribe to channels in exactly the same order + # so we have to do some extra checks to make sure we got them all + messages = [] + for i in range(len(keys)): + messages.append(await wait_for_message(p)) + + unique_channels = set() + assert len(messages) == len(keys) + for i, message in enumerate(messages): + assert message["type"] == sub_type + assert message["data"] == i + 1 + assert isinstance(message["channel"], bytes) + channel = message["channel"].decode("utf-8") + unique_channels.add(channel) + + assert len(unique_channels) == len(keys) + for channel in unique_channels: + assert channel in keys + + async def test_resubscribe_to_channels_on_reconnection(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "channel") + await self._test_resubscribe_on_reconnection(**kwargs) + + async def test_resubscribe_to_patterns_on_reconnection(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "pattern") + await self._test_resubscribe_on_reconnection(**kwargs) + + async def _test_subscribed_property( + self, p, sub_type, unsub_type, sub_func, unsub_func, keys + ): + assert p.subscribed is False + await sub_func(keys[0]) + # we're now subscribed even though we haven't processed the + # reply from the server just yet + assert p.subscribed is True + assert await wait_for_message(p) == make_message(sub_type, keys[0], 1) + # we're still subscribed + assert p.subscribed is True + + # unsubscribe from all channels + await unsub_func() + # we're still technically subscribed until we process the + # response messages from the server + assert p.subscribed is True + assert await wait_for_message(p) == make_message(unsub_type, keys[0], 0) + # now we're no longer subscribed as no more messages can be delivered + # to any channels we were listening to + assert p.subscribed is False + + # subscribing again flips the flag back + await sub_func(keys[0]) + assert p.subscribed is True + assert await wait_for_message(p) == make_message(sub_type, keys[0], 1) + + # unsubscribe again + await unsub_func() + assert p.subscribed is True + # subscribe to another channel before reading the unsubscribe response + await sub_func(keys[1]) + assert p.subscribed is True + # read the unsubscribe for key1 + assert await wait_for_message(p) == make_message(unsub_type, keys[0], 0) + # we're still subscribed to key2, so subscribed should still be True + assert p.subscribed is True + # read the key2 subscribe message + assert await wait_for_message(p) == make_message(sub_type, keys[1], 1) + await unsub_func() + # haven't read the message yet, so we're still subscribed + assert p.subscribed is True + assert await wait_for_message(p) == make_message(unsub_type, keys[1], 0) + # now we're finally unsubscribed + assert p.subscribed is False + + async def test_subscribe_property_with_channels(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "channel") + await self._test_subscribed_property(**kwargs) + + @pytest.mark.onlynoncluster + async def test_subscribe_property_with_patterns(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "pattern") + await self._test_subscribed_property(**kwargs) + + async def test_aclosing(self, r: redis.Redis): + p = r.pubsub() + async with aclosing(p): + assert p.subscribed is False + await p.subscribe("foo") + assert p.subscribed is True + assert p.subscribed is False + + async def test_context_manager(self, r: redis.Redis): + p = r.pubsub() + async with p: + assert p.subscribed is False + await p.subscribe("foo") + assert p.subscribed is True + assert p.subscribed is False + + async def test_close_is_aclose(self, r: redis.Redis): + """ + Test backwards compatible close method + """ + p = r.pubsub() + assert p.subscribed is False + await p.subscribe("foo") + assert p.subscribed is True + with pytest.deprecated_call(): + await p.close() + assert p.subscribed is False + + async def test_reset_is_aclose(self, r: redis.Redis): + """ + Test backwards compatible reset method + """ + p = r.pubsub() + assert p.subscribed is False + await p.subscribe("foo") + assert p.subscribed is True + with pytest.deprecated_call(): + await p.reset() + assert p.subscribed is False + + async def test_ignore_all_subscribe_messages(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + + checks = ( + (p.subscribe, "foo"), + (p.unsubscribe, "foo"), + (p.psubscribe, "f*"), + (p.punsubscribe, "f*"), + ) + + assert p.subscribed is False + for func, channel in checks: + assert await func(channel) is None + assert p.subscribed is True + assert await wait_for_message(p) is None + assert p.subscribed is False + await p.aclose() + + async def test_ignore_individual_subscribe_messages(self, pubsub): + p = pubsub + + checks = ( + (p.subscribe, "foo"), + (p.unsubscribe, "foo"), + (p.psubscribe, "f*"), + (p.punsubscribe, "f*"), + ) + + assert p.subscribed is False + for func, channel in checks: + assert await func(channel) is None + assert p.subscribed is True + message = await wait_for_message(p, ignore_subscribe_messages=True) + assert message is None + assert p.subscribed is False + + async def test_sub_unsub_resub_channels(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "channel") + await self._test_sub_unsub_resub(**kwargs) + + @pytest.mark.onlynoncluster + async def test_sub_unsub_resub_patterns(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "pattern") + await self._test_sub_unsub_resub(**kwargs) + + async def _test_sub_unsub_resub( + self, p, sub_type, unsub_type, sub_func, unsub_func, keys + ): + # https://github.com/andymccurdy/redis-py/issues/764 + key = keys[0] + await sub_func(key) + await unsub_func(key) + await sub_func(key) + assert p.subscribed is True + assert await wait_for_message(p) == make_message(sub_type, key, 1) + assert await wait_for_message(p) == make_message(unsub_type, key, 0) + assert await wait_for_message(p) == make_message(sub_type, key, 1) + assert p.subscribed is True + + async def test_sub_unsub_all_resub_channels(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "channel") + await self._test_sub_unsub_all_resub(**kwargs) + + async def test_sub_unsub_all_resub_patterns(self, pubsub): + kwargs = make_subscribe_test_data(pubsub, "pattern") + await self._test_sub_unsub_all_resub(**kwargs) + + async def _test_sub_unsub_all_resub( + self, p, sub_type, unsub_type, sub_func, unsub_func, keys + ): + # https://github.com/andymccurdy/redis-py/issues/764 + key = keys[0] + await sub_func(key) + await unsub_func() + await sub_func(key) + assert p.subscribed is True + assert await wait_for_message(p) == make_message(sub_type, key, 1) + assert await wait_for_message(p) == make_message(unsub_type, key, 0) + assert await wait_for_message(p) == make_message(sub_type, key, 1) + assert p.subscribed is True + + +@pytest.mark.onlynoncluster +class TestPubSubMessages: + def setup_method(self, method): + self.message = None + self.async_message = None + + def message_handler(self, message): + self.message = message + + async def async_message_handler(self, message): + self.async_message = message + + async def test_published_message_to_channel(self, r: redis.Redis, pubsub): + p = pubsub + await p.subscribe("foo") + assert await wait_for_message(p) == make_message("subscribe", "foo", 1) + assert await r.publish("foo", "test message") == 1 + + message = await wait_for_message(p) + assert isinstance(message, dict) + assert message == make_message("message", "foo", "test message") + + async def test_published_message_to_pattern(self, r: redis.Redis, pubsub): + p = pubsub + await p.subscribe("foo") + await p.psubscribe("f*") + assert await wait_for_message(p) == make_message("subscribe", "foo", 1) + assert await wait_for_message(p) == make_message("psubscribe", "f*", 2) + # 1 to pattern, 1 to channel + assert await r.publish("foo", "test message") == 2 + + message1 = await wait_for_message(p) + message2 = await wait_for_message(p) + assert isinstance(message1, dict) + assert isinstance(message2, dict) + + expected = [ + make_message("message", "foo", "test message"), + make_message("pmessage", "foo", "test message", pattern="f*"), + ] + + assert message1 in expected + assert message2 in expected + assert message1 != message2 + + async def test_channel_message_handler(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + await p.subscribe(foo=self.message_handler) + assert await wait_for_message(p) is None + assert await r.publish("foo", "test message") == 1 + assert await wait_for_message(p) is None + assert self.message == make_message("message", "foo", "test message") + await p.aclose() + + async def test_channel_async_message_handler(self, r): + p = r.pubsub(ignore_subscribe_messages=True) + await p.subscribe(foo=self.async_message_handler) + assert await wait_for_message(p) is None + assert await r.publish("foo", "test message") == 1 + assert await wait_for_message(p) is None + assert self.async_message == make_message("message", "foo", "test message") + await p.aclose() + + async def test_channel_sync_async_message_handler(self, r): + p = r.pubsub(ignore_subscribe_messages=True) + await p.subscribe(foo=self.message_handler) + await p.subscribe(bar=self.async_message_handler) + assert await wait_for_message(p) is None + assert await r.publish("foo", "test message") == 1 + assert await r.publish("bar", "test message 2") == 1 + assert await wait_for_message(p) is None + assert self.message == make_message("message", "foo", "test message") + assert self.async_message == make_message("message", "bar", "test message 2") + await p.aclose() + + @pytest.mark.onlynoncluster + async def test_pattern_message_handler(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + await p.psubscribe(**{"f*": self.message_handler}) + assert await wait_for_message(p) is None + assert await r.publish("foo", "test message") == 1 + assert await wait_for_message(p) is None + assert self.message == make_message( + "pmessage", "foo", "test message", pattern="f*" + ) + await p.aclose() + + async def test_unicode_channel_message_handler(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + channel = "uni" + chr(4456) + "code" + channels = {channel: self.message_handler} + await p.subscribe(**channels) + assert await wait_for_message(p) is None + assert await r.publish(channel, "test message") == 1 + assert await wait_for_message(p) is None + assert self.message == make_message("message", channel, "test message") + await p.aclose() + + @pytest.mark.onlynoncluster + # see: https://redis-py-cluster.readthedocs.io/en/stable/pubsub.html + # #known-limitations-with-pubsub + async def test_unicode_pattern_message_handler(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + pattern = "uni" + chr(4456) + "*" + channel = "uni" + chr(4456) + "code" + await p.psubscribe(**{pattern: self.message_handler}) + assert await wait_for_message(p) is None + assert await r.publish(channel, "test message") == 1 + assert await wait_for_message(p) is None + assert self.message == make_message( + "pmessage", channel, "test message", pattern=pattern + ) + await p.aclose() + + async def test_get_message_without_subscribe(self, r: redis.Redis, pubsub): + p = pubsub + with pytest.raises(RuntimeError) as info: + await p.get_message() + expect = ( + "connection not set: did you forget to call subscribe() or psubscribe()?" + ) + assert expect in info.exconly() + + +@pytest.mark.onlynoncluster +class TestPubSubRESP3Handler: + async def my_handler(self, message): + self.message = ["my handler", message] + + @pytest.mark.skipif(HIREDIS_AVAILABLE, reason="PythonParser only") + async def test_push_handler(self, r): + if get_protocol_version(r) in [2, "2", None]: + return + p = r.pubsub(push_handler_func=self.my_handler) + await p.subscribe("foo") + assert await wait_for_message(p) is None + assert self.message == ["my handler", [b"subscribe", b"foo", 1]] + assert await r.publish("foo", "test message") == 1 + assert await wait_for_message(p) is None + assert self.message == ["my handler", [b"message", b"foo", b"test message"]] + + +@pytest.mark.onlynoncluster +class TestPubSubAutoDecoding: + """These tests only validate that we get unicode values back""" + + channel = "uni" + chr(4456) + "code" + pattern = "uni" + chr(4456) + "*" + data = "abc" + chr(4458) + "123" + + def make_message(self, type, channel, data, pattern=None): + return {"type": type, "channel": channel, "pattern": pattern, "data": data} + + def setup_method(self, method): + self.message = None + + def message_handler(self, message): + self.message = message + + @pytest.fixture + async def r(self, create_redis): + async with create_redis(decode_responses=True) as client: + yield client + + async def test_channel_subscribe_unsubscribe(self, pubsub): + p = pubsub + await p.subscribe(self.channel) + assert await wait_for_message(p) == self.make_message( + "subscribe", self.channel, 1 + ) + + await p.unsubscribe(self.channel) + assert await wait_for_message(p) == self.make_message( + "unsubscribe", self.channel, 0 + ) + + async def test_pattern_subscribe_unsubscribe(self, pubsub): + p = pubsub + await p.psubscribe(self.pattern) + assert await wait_for_message(p) == self.make_message( + "psubscribe", self.pattern, 1 + ) + + await p.punsubscribe(self.pattern) + assert await wait_for_message(p) == self.make_message( + "punsubscribe", self.pattern, 0 + ) + + async def test_channel_publish(self, r: redis.Redis, pubsub): + p = pubsub + await p.subscribe(self.channel) + assert await wait_for_message(p) == self.make_message( + "subscribe", self.channel, 1 + ) + await r.publish(self.channel, self.data) + assert await wait_for_message(p) == self.make_message( + "message", self.channel, self.data + ) + + @pytest.mark.onlynoncluster + async def test_pattern_publish(self, r: redis.Redis, pubsub): + p = pubsub + await p.psubscribe(self.pattern) + assert await wait_for_message(p) == self.make_message( + "psubscribe", self.pattern, 1 + ) + await r.publish(self.channel, self.data) + assert await wait_for_message(p) == self.make_message( + "pmessage", self.channel, self.data, pattern=self.pattern + ) + + async def test_channel_message_handler(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + await p.subscribe(**{self.channel: self.message_handler}) + assert await wait_for_message(p) is None + await r.publish(self.channel, self.data) + assert await wait_for_message(p) is None + assert self.message == self.make_message("message", self.channel, self.data) + + # test that we reconnected to the correct channel + self.message = None + await p.connection.disconnect() + assert await wait_for_message(p) is None # should reconnect + new_data = self.data + "new data" + await r.publish(self.channel, new_data) + assert await wait_for_message(p) is None + assert self.message == self.make_message("message", self.channel, new_data) + await p.aclose() + + async def test_pattern_message_handler(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + await p.psubscribe(**{self.pattern: self.message_handler}) + assert await wait_for_message(p) is None + await r.publish(self.channel, self.data) + assert await wait_for_message(p) is None + assert self.message == self.make_message( + "pmessage", self.channel, self.data, pattern=self.pattern + ) + + # test that we reconnected to the correct pattern + self.message = None + await p.connection.disconnect() + assert await wait_for_message(p) is None # should reconnect + new_data = self.data + "new data" + await r.publish(self.channel, new_data) + assert await wait_for_message(p) is None + assert self.message == self.make_message( + "pmessage", self.channel, new_data, pattern=self.pattern + ) + await p.aclose() + + async def test_context_manager(self, r: redis.Redis): + async with r.pubsub() as pubsub: + await pubsub.subscribe("foo") + assert pubsub.connection is not None + + assert pubsub.connection is None + assert pubsub.channels == {} + assert pubsub.patterns == {} + await pubsub.aclose() + + +@pytest.mark.onlynoncluster +class TestPubSubRedisDown: + async def test_channel_subscribe(self, r: redis.Redis): + r = redis.Redis(host="localhost", port=6390) + p = r.pubsub() + with pytest.raises(ConnectionError): + await p.subscribe("foo") + + +@pytest.mark.onlynoncluster +class TestPubSubSubcommands: + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("2.8.0") + async def test_pubsub_channels(self, r: redis.Redis, pubsub): + p = pubsub + await p.subscribe("foo", "bar", "baz", "quux") + for i in range(4): + assert (await wait_for_message(p))["type"] == "subscribe" + expected = [b"bar", b"baz", b"foo", b"quux"] + assert all([channel in await r.pubsub_channels() for channel in expected]) + + @pytest.mark.onlynoncluster + @skip_if_server_version_lt("2.8.0") + async def test_pubsub_numsub(self, r: redis.Redis): + p1 = r.pubsub() + await p1.subscribe("foo", "bar", "baz") + for i in range(3): + assert (await wait_for_message(p1))["type"] == "subscribe" + p2 = r.pubsub() + await p2.subscribe("bar", "baz") + for i in range(2): + assert (await wait_for_message(p2))["type"] == "subscribe" + p3 = r.pubsub() + await p3.subscribe("baz") + assert (await wait_for_message(p3))["type"] == "subscribe" + + channels = [(b"foo", 1), (b"bar", 2), (b"baz", 3)] + assert await r.pubsub_numsub("foo", "bar", "baz") == channels + await p1.aclose() + await p2.aclose() + await p3.aclose() + + @skip_if_server_version_lt("2.8.0") + async def test_pubsub_numpat(self, r: redis.Redis): + p = r.pubsub() + await p.psubscribe("*oo", "*ar", "b*z") + for i in range(3): + assert (await wait_for_message(p))["type"] == "psubscribe" + assert await r.pubsub_numpat() == 3 + await p.aclose() + + +@pytest.mark.onlynoncluster +class TestPubSubPings: + @skip_if_server_version_lt("3.0.0") + async def test_send_pubsub_ping(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + await p.subscribe("foo") + await p.ping() + assert await wait_for_message(p) == make_message( + type="pong", channel=None, data="", pattern=None + ) + await p.aclose() + + @skip_if_server_version_lt("3.0.0") + async def test_send_pubsub_ping_message(self, r: redis.Redis): + p = r.pubsub(ignore_subscribe_messages=True) + await p.subscribe("foo") + await p.ping(message="hello world") + assert await wait_for_message(p) == make_message( + type="pong", channel=None, data="hello world", pattern=None + ) + await p.aclose() + + +@pytest.mark.onlynoncluster +class TestPubSubConnectionKilled: + @skip_if_server_version_lt("3.0.0") + async def test_connection_error_raised_when_connection_dies( + self, r: redis.Redis, pubsub + ): + p = pubsub + await p.subscribe("foo") + assert await wait_for_message(p) == make_message("subscribe", "foo", 1) + for client in await r.client_list(): + if client["cmd"] == "subscribe": + await r.client_kill_filter(_id=client["id"]) + with pytest.raises(ConnectionError): + await wait_for_message(p) + + +@pytest.mark.onlynoncluster +class TestPubSubTimeouts: + async def test_get_message_with_timeout_returns_none(self, pubsub): + p = pubsub + await p.subscribe("foo") + assert await wait_for_message(p) == make_message("subscribe", "foo", 1) + assert await p.get_message(timeout=0.01) is None + + +@pytest.mark.onlynoncluster +class TestPubSubReconnect: + @with_timeout(2) + async def test_reconnect_listen(self, r: redis.Redis, pubsub): + """ + Test that a loop processing PubSub messages can survive + a disconnect, by issuing a connect() call. + """ + + async def loop(send: MemoryObjectSendStream, task_status: TaskStatus[None]): + # must make sure the task exits + with send, anyio.fail_after(2): + await pubsub.subscribe("foo") + task_status.started() + while True: + try: + await pubsub.connect() + await loop_step(send) + except redis.ConnectionError: + await anyio.sleep(0.1) + + async def loop_step(send: MemoryObjectSendStream): + # get a single message via listen() + async with aclosing(pubsub.listen()) as messages: + async for message in messages: + await send.send(message) + break + + send, receive = anyio.create_memory_object_stream(1) + with receive: + async with anyio.create_task_group() as tg: + await tg.start(loop, send) + + # get the initial connect message + with anyio.fail_after(1): + message = await receive.receive() + assert message == { + "channel": b"foo", + "data": 1, + "pattern": None, + "type": "subscribe", + } + + # now, disconnect the connection. + await pubsub.connection.disconnect() + # await another auto-connect message + with anyio.fail_after(1, shield=True): + message = await receive.receive() + assert message == { + "channel": b"foo", + "data": 1, + "pattern": None, + "type": "subscribe", + } + + tg.cancel_scope.cancel() + + +@pytest.mark.onlynoncluster +class TestPubSubRun: + async def _subscribe(self, p, *args, **kwargs): + await p.subscribe(*args, **kwargs) + # Wait for the server to act on the subscription, to be sure that + # a subsequent publish on another connection will reach the pubsub. + while True: + message = await p.get_message(timeout=1) + if ( + message is not None + and message["type"] == "subscribe" + and message["channel"] == b"foo" + ): + return + + async def test_callbacks(self, r: redis.Redis, pubsub): + p = pubsub + send, receive = anyio.create_memory_object_stream(1) + + def callback(message): + send.send_nowait(message) + + with send, receive: + await self._subscribe(p, foo=callback) + async with anyio.create_task_group() as tg: + tg.start_soon(p.run) + await r.publish("foo", "bar") + message = await receive.receive() + tg.cancel_scope.cancel() + + assert message == { + "channel": b"foo", + "data": b"bar", + "pattern": None, + "type": "message", + } + + async def test_exception_handler(self, r: redis.Redis, pubsub): + p = pubsub + send, receive = anyio.create_memory_object_stream(1) + + def exception_handler_callback(e, pubsub) -> None: + assert pubsub == p + send.send_nowait(e) + + with send, receive: + await self._subscribe(p, foo=lambda x: None) + with mock.patch.object(p, "get_message", side_effect=Exception("error")): + async with anyio.create_task_group() as tg: + tg.start_soon( + lambda: p.run(exception_handler=exception_handler_callback) + ) + e = await receive.receive() + tg.cancel_scope.cancel() + + assert str(e) == "error" + + async def test_late_subscribe(self, r: redis.Redis, pubsub): + p = pubsub + send, receive = anyio.create_memory_object_stream(1) + + def callback(message): + send.send_nowait(message) + + with send, receive: + async with anyio.create_task_group() as tg: + tg.start_soon(p.run) + # wait until loop gets settled. Add a subscription + await anyio.sleep(0.1) + await p.subscribe(foo=callback) + + # wait tof the subscribe to finish. Cannot use _subscribe() because + # p.run() is already accepting messages + while True: + n = await r.publish("foo", "bar") + if n == 1: + break + await anyio.sleep(0.1) + + with anyio.fail_after(0.1): + message = await receive.receive() + + tg.cancel_scope.cancel() + assert message == { + "channel": b"foo", + "data": b"bar", + "pattern": None, + "type": "message", + } + + +# @pytest.mark.xfail +@pytest.mark.parametrize("method", ["get_message", "listen"]) +@pytest.mark.onlynoncluster +class TestPubSubAutoReconnect: + timeout = 2 + + @asynccontextmanager + async def mysetup(self, r, method): + send, receive = anyio.create_memory_object_stream(1) + # State: 0 = initial state , 1 = after disconnect, 2 = ConnectionError is seen, + # 3=successfully reconnected 4 = exit + self.state = 0 + self.cond = anyio.Condition() + if method == "get_message": + self.get_message = functools.partial(self.loop_step_get_message, send) + else: + self.get_message = functools.partial(self.loop_step_listen, send) + + with send: + async with r.pubsub() as self.pubsub, anyio.create_task_group() as tg: + await tg.start(self.loop, receive) + + # get the initial connect message + message = await receive.receive() + assert message == { + "channel": b"foo", + "data": 1, + "pattern": None, + "type": "subscribe", + } + + yield + + message = await receive.receive() + assert message == { + "channel": b"foo", + "data": 1, + "pattern": None, + "type": "subscribe", + } + + async with self.cond: + self.state = 4 # quit + self.cond.notify() + + async def test_reconnect_socket_error(self, r: redis.Redis, method): + """ + Test that a socket error will cause reconnect + """ + with anyio.fail_after(self.timeout): + async with self.mysetup(r, method): + # now, disconnect the connection, and wait for it to be re-established + async with self.cond: + assert self.state == 0 + self.state = 1 + with mock.patch.object(self.pubsub.connection, "_parser") as m: + m.read_response.side_effect = socket.error + m.can_read_destructive.side_effect = socket.error + + # wait until task noticies the disconnect until we + # undo the patch + await wait_for_condition(self.cond, lambda: self.state >= 2) + assert not self.pubsub.connection.is_connected + # it is in a disconnected state + + # wait for reconnect + await wait_for_condition( + self.cond, lambda: self.pubsub.connection.is_connected + ) + assert self.state == 3 + + async def test_reconnect_disconnect(self, r: redis.Redis, method): + """ + Test that a manual disconnect() will cause reconnect + """ + with anyio.fail_after(self.timeout): + async with self.mysetup(r, method): + # now, disconnect the connection, and wait for it to be re-established + async with self.cond: + self.state = 1 + await self.pubsub.connection.disconnect() + assert not self.pubsub.connection.is_connected + # wait for reconnect + await wait_for_condition( + self.cond, lambda: self.pubsub.connection.is_connected + ) + assert self.state == 3 + + async def loop( + self, receive: MemoryObjectReceiveStream, *, task_status: TaskStatus[None] + ): + # reader loop, performing state transitions as it + # discovers disconnects and reconnects + with receive: + task_status.started() + await self.pubsub.subscribe("foo") + while True: + await anyio.sleep(0.01) # give main thread chance to get lock + async with self.cond: + old_state = self.state + try: + if self.state == 4: + break + got_msg = await self.get_message() + assert got_msg + if self.state in (1, 2): + self.state = 3 # successful reconnect + except redis.ConnectionError: + assert self.state in (1, 2) + self.state = 2 # signal that we noticed the disconnect + finally: + self.cond.notify() + # make sure that we did notice the connection error + # or reconnected without any error + if old_state == 1: + assert self.state in (2, 3) + + async def loop_step_get_message(self, send: MemoryObjectSendStream) -> bool: + # get a single message via get_message + message = await self.pubsub.get_message(timeout=0.1) + if message is not None: + await send.send(message) + return True + return False + + async def loop_step_listen(self, send: MemoryObjectSendStream) -> bool: + # get a single message via listen() + with anyio.move_on_after(0.1): + async with aclosing(self.pubsub.listen()) as messages: + async for message in messages: + await send.send(message) + return True + + return False + + +@pytest.mark.onlynoncluster +class TestBaseException: + async def test_outer_timeout(self, r: redis.Redis): + """ + Using asyncio_timeout manually outside the inner method timeouts works. + This works on Python versions 3.8 and greater, at which time asyncio. + CancelledError became a BaseException instead of an Exception before. + """ + pubsub = r.pubsub() + await pubsub.subscribe("foo") + assert pubsub.connection.is_connected + + async def get_msg_or_timeout(timeout=0.1): + with anyio.fail_after(timeout): + # blocking method to return messages + while True: + response = await pubsub.parse_response(block=True) + message = await pubsub.handle_message( + response, ignore_subscribe_messages=False + ) + if message is not None: + return message + + # get subscribe message + msg = await get_msg_or_timeout(10) + assert msg is not None + # timeout waiting for another message which never arrives + assert pubsub.connection.is_connected + with pytest.raises(TimeoutError): + await get_msg_or_timeout() + # the timeout on the read should not cause disconnect + assert pubsub.connection.is_connected + + async def test_base_exception(self, r: redis.Redis): + """ + Manually trigger a BaseException inside the parser's .read_response method + and verify that it isn't caught + """ + pubsub = r.pubsub() + await pubsub.subscribe("foo") + assert pubsub.connection.is_connected + + async def get_msg(): + # blocking method to return messages + while True: + response = await pubsub.parse_response(block=True) + message = await pubsub.handle_message( + response, ignore_subscribe_messages=False + ) + if message is not None: + return message + + # get subscribe message + msg = await get_msg() + assert msg is not None + # timeout waiting for another message which never arrives + assert pubsub.connection.is_connected + with patch( + "redis.anyio._parsers._AnyIORESP2Parser.read_response" + ) as mock1, patch( + "redis.anyio._parsers._AnyIOHiredisParser.read_response" + ) as mock2, patch( + "redis.anyio._parsers._AnyIORESP3Parser.read_response" + ) as mock3: + mock1.side_effect = BaseException("boom") + mock2.side_effect = BaseException("boom") + mock3.side_effect = BaseException("boom") + + with pytest.raises(BaseException): + await get_msg() + + # the timeout on the read should not cause disconnect + assert pubsub.connection.is_connected diff --git a/tests/test_anyio/test_retry.py b/tests/test_anyio/test_retry.py new file mode 100644 index 0000000000..6a45911210 --- /dev/null +++ b/tests/test_anyio/test_retry.py @@ -0,0 +1,138 @@ +import pytest +from redis.anyio import Redis +from redis.anyio.connection import Connection, UnixDomainSocketConnection +from redis.anyio.retry import Retry +from redis.backoff import AbstractBackoff, ExponentialBackoff, NoBackoff +from redis.exceptions import ConnectionError, TimeoutError + +pytestmark = pytest.mark.anyio + + +class BackoffMock(AbstractBackoff): + def __init__(self): + self.reset_calls = 0 + self.calls = 0 + + def reset(self): + self.reset_calls += 1 + + def compute(self, failures): + self.calls += 1 + return 0 + + +class TestConnectionConstructorWithRetry: + "Test that the Connection constructors properly handles Retry objects" + + @pytest.mark.parametrize("Class", [Connection, UnixDomainSocketConnection]) + def test_retry_on_error_set(self, Class): + class CustomError(Exception): + pass + + retry_on_error = [ConnectionError, TimeoutError, CustomError] + c = Class(retry_on_error=retry_on_error) + assert c.retry_on_error == retry_on_error + assert isinstance(c.retry, Retry) + assert c.retry._retries == 1 + assert set(c.retry._supported_errors) == set(retry_on_error) + + @pytest.mark.parametrize("Class", [Connection, UnixDomainSocketConnection]) + def test_retry_on_error_not_set(self, Class): + c = Class() + assert c.retry_on_error == [] + assert isinstance(c.retry, Retry) + assert c.retry._retries == 0 + + @pytest.mark.parametrize("retry_on_timeout", [False, True]) + @pytest.mark.parametrize("Class", [Connection, UnixDomainSocketConnection]) + def test_retry_on_timeout(self, Class, retry_on_timeout): + c = Class(retry_on_timeout=retry_on_timeout) + assert c.retry_on_timeout == retry_on_timeout + assert isinstance(c.retry, Retry) + assert c.retry._retries == (1 if retry_on_timeout else 0) + + @pytest.mark.parametrize("retries", range(10)) + @pytest.mark.parametrize("Class", [Connection, UnixDomainSocketConnection]) + def test_retry_with_retry_on_timeout(self, Class, retries: int): + retry_on_timeout = retries > 0 + c = Class(retry_on_timeout=retry_on_timeout, retry=Retry(NoBackoff(), retries)) + assert c.retry_on_timeout == retry_on_timeout + assert isinstance(c.retry, Retry) + assert c.retry._retries == retries + + @pytest.mark.parametrize("retries", range(10)) + @pytest.mark.parametrize("Class", [Connection, UnixDomainSocketConnection]) + def test_retry_with_retry_on_error(self, Class, retries: int): + class CustomError(Exception): + pass + + retry_on_error = [ConnectionError, TimeoutError, CustomError] + c = Class(retry_on_error=retry_on_error, retry=Retry(NoBackoff(), retries)) + assert c.retry_on_error == retry_on_error + assert isinstance(c.retry, Retry) + assert c.retry._retries == retries + assert set(c.retry._supported_errors) == set(retry_on_error) + + +class TestRetry: + "Test that Retry calls backoff and retries the expected number of times" + + def setup_method(self, test_method): + self.actual_attempts = 0 + self.actual_failures = 0 + + async def _do(self): + self.actual_attempts += 1 + raise ConnectionError() + + async def _fail(self, error): + self.actual_failures += 1 + + async def _fail_inf(self, error): + self.actual_failures += 1 + if self.actual_failures == 5: + raise ConnectionError() + + @pytest.mark.parametrize("retries", range(10)) + async def test_retry(self, retries: int): + backoff = BackoffMock() + retry = Retry(backoff, retries) + with pytest.raises(ConnectionError): + await retry.call_with_retry(self._do, self._fail) + + assert self.actual_attempts == 1 + retries + assert self.actual_failures == 1 + retries + assert backoff.reset_calls == 1 + assert backoff.calls == retries + + async def test_infinite_retry(self): + backoff = BackoffMock() + # specify infinite retries, but give up after 5 + retry = Retry(backoff, -1) + with pytest.raises(ConnectionError): + await retry.call_with_retry(self._do, self._fail_inf) + + assert self.actual_attempts == 5 + assert self.actual_failures == 5 + + +class TestRedisClientRetry: + "Test the Redis client behavior with retries" + + async def test_get_set_retry_object(self, request): + retry = Retry(NoBackoff(), 2) + url = request.config.getoption("--redis-url") + r = await Redis.from_url(url, retry_on_timeout=True, retry=retry) + assert r.get_retry()._retries == retry._retries + assert isinstance(r.get_retry()._backoff, NoBackoff) + new_retry_policy = Retry(ExponentialBackoff(), 3) + exiting_conn = await r.connection_pool.get_connection() + r.set_retry(new_retry_policy) + assert r.get_retry()._retries == new_retry_policy._retries + assert isinstance(r.get_retry()._backoff, ExponentialBackoff) + assert exiting_conn.retry._retries == new_retry_policy._retries + await r.connection_pool.release(exiting_conn) + new_conn = await r.connection_pool.get_connection() + assert new_conn.retry._retries == new_retry_policy._retries + await r.connection_pool.release(new_conn) + await r.aclose() diff --git a/tests/test_anyio/test_scripting.py b/tests/test_anyio/test_scripting.py new file mode 100644 index 0000000000..0dc49d0251 --- /dev/null +++ b/tests/test_anyio/test_scripting.py @@ -0,0 +1,145 @@ +import pytest +from redis import exceptions +from tests.conftest import skip_if_server_version_lt + +multiply_script = """ +local value = redis.call('GET', KEYS[1]) +value = tonumber(value) +return value * ARGV[1]""" + +msgpack_hello_script = """ +local message = cmsgpack.unpack(ARGV[1]) +local name = message['name'] +return "hello " .. name +""" +msgpack_hello_script_broken = """ +local message = cmsgpack.unpack(ARGV[1]) +local names = message['name'] +return "hello " .. name +""" + +pytestmark = pytest.mark.anyio + + +@pytest.mark.onlynoncluster +class TestScripting: + @pytest.fixture + async def r(self, create_redis): + async with create_redis() as client: + yield client + await client.script_flush() + + async def test_eval(self, r): + await r.flushdb() + await r.set("a", 2) + # 2 * 3 == 6 + assert await r.eval(multiply_script, 1, "a", 3) == 6 + + @skip_if_server_version_lt("6.2.0") + async def test_script_flush(self, r): + await r.set("a", 2) + await r.script_load(multiply_script) + await r.script_flush("ASYNC") + + await r.set("a", 2) + await r.script_load(multiply_script) + await r.script_flush("SYNC") + + await r.set("a", 2) + await r.script_load(multiply_script) + await r.script_flush() + + with pytest.raises(exceptions.DataError): + await r.set("a", 2) + await r.script_load(multiply_script) + await r.script_flush("NOTREAL") + + async def test_evalsha(self, r): + await r.set("a", 2) + sha = await r.script_load(multiply_script) + # 2 * 3 == 6 + assert await r.evalsha(sha, 1, "a", 3) == 6 + + async def test_evalsha_script_not_loaded(self, r): + await r.set("a", 2) + sha = await r.script_load(multiply_script) + # remove the script from Redis's cache + await r.script_flush() + with pytest.raises(exceptions.NoScriptError): + await r.evalsha(sha, 1, "a", 3) + + async def test_script_loading(self, r): + # get the sha, then clear the cache + sha = await r.script_load(multiply_script) + await r.script_flush() + assert await r.script_exists(sha) == [False] + await r.script_load(multiply_script) + assert await r.script_exists(sha) == [True] + + async def test_script_object(self, r): + await r.script_flush() + await r.set("a", 2) + multiply = r.register_script(multiply_script) + precalculated_sha = multiply.sha + assert precalculated_sha + assert await r.script_exists(multiply.sha) == [False] + # Test second evalsha block (after NoScriptError) + assert await multiply(keys=["a"], args=[3]) == 6 + # At this point, the script should be loaded + assert await r.script_exists(multiply.sha) == [True] + # Test that the precalculated sha matches the one from redis + assert multiply.sha == precalculated_sha + # Test first evalsha block + assert await multiply(keys=["a"], args=[3]) == 6 + + async def test_script_object_in_pipeline(self, r): + await r.script_flush() + multiply = r.register_script(multiply_script) + precalculated_sha = multiply.sha + assert precalculated_sha + pipe = r.pipeline() + pipe.set("a", 2) + pipe.get("a") + await multiply(keys=["a"], args=[3], client=pipe) + assert await r.script_exists(multiply.sha) == [False] + # [SET worked, GET 'a', result of multiple script] + assert await pipe.execute() == [True, b"2", 6] + # The script should have been loaded by pipe.execute() + assert await r.script_exists(multiply.sha) == [True] + # The precalculated sha should have been the correct one + assert multiply.sha == precalculated_sha + + # purge the script from redis's cache and re-run the pipeline + # the multiply script should be reloaded by pipe.execute() + await r.script_flush() + pipe = r.pipeline() + pipe.set("a", 2) + pipe.get("a") + await multiply(keys=["a"], args=[3], client=pipe) + assert await r.script_exists(multiply.sha) == [False] + # [SET worked, GET 'a', result of multiple script] + assert await pipe.execute() == [True, b"2", 6] + assert await r.script_exists(multiply.sha) == [True] + + async def test_eval_msgpack_pipeline_error_in_lua(self, r): + msgpack_hello = r.register_script(msgpack_hello_script) + assert msgpack_hello.sha + + pipe = r.pipeline() + + # avoiding a dependency to msgpack, this is the output of + # msgpack.dumps({"name": "joe"}) + msgpack_message_1 = b"\x81\xa4name\xa3Joe" + + await msgpack_hello(args=[msgpack_message_1], client=pipe) + + assert await r.script_exists(msgpack_hello.sha) == [False] + assert (await pipe.execute())[0] == b"hello Joe" + assert await r.script_exists(msgpack_hello.sha) == [True] + + msgpack_hello_broken = r.register_script(msgpack_hello_script_broken) + + await msgpack_hello_broken(args=[msgpack_message_1], client=pipe) + with pytest.raises(exceptions.ResponseError) as excinfo: + await pipe.execute() + assert excinfo.type == exceptions.ResponseError diff --git a/tests/test_anyio/test_search.py b/tests/test_anyio/test_search.py new file mode 100644 index 0000000000..0748732fec --- /dev/null +++ b/tests/test_anyio/test_search.py @@ -0,0 +1,1815 @@ +import bz2 +import csv +import os +from io import TextIOWrapper + +import anyio +import numpy as np +import pytest +import redis.anyio as redis +import redis.commands.search +import redis.commands.search.aggregation as aggregations +import redis.commands.search.reducers as reducers +from redis.commands.search import AsyncSearch +from redis.commands.search.field import ( + GeoField, + NumericField, + TagField, + TextField, + VectorField, +) +from redis.commands.search.index_definition import IndexDefinition, IndexType +from redis.commands.search.query import GeoFilter, NumericFilter, Query +from redis.commands.search.result import Result +from redis.commands.search.suggestion import Suggestion +from tests.conftest import ( + is_resp2_connection, + skip_if_redis_enterprise, + skip_if_resp_version, + skip_if_server_version_gte, + skip_if_server_version_lt, + skip_ifmodversion_lt, +) + +pytestmark = pytest.mark.anyio + +WILL_PLAY_TEXT = os.path.abspath( + os.path.join(os.path.dirname(__file__), "testdata", "will_play_text.csv.bz2") +) + +TITLES_CSV = os.path.abspath( + os.path.join(os.path.dirname(__file__), "testdata", "titles.csv") +) + + +@pytest.fixture +async def decoded_r(create_redis, stack_url): + async with create_redis(decode_responses=True, url=stack_url) as client: + yield client + + +async def waitForIndex(env, idx, timeout=None): + delay = 0.1 + while True: + try: + res = await env.execute_command("FT.INFO", idx) + if int(res[res.index("indexing") + 1]) == 0: + break + except ValueError: + break + except AttributeError: + try: + if int(res["indexing"]) == 0: + break + except ValueError: + break + + await anyio.sleep(delay) + if timeout is not None: + timeout -= delay + if timeout <= 0: + break + + +def getClient(decoded_r: redis.Redis): + """ + Gets a client client attached to an index name which is ready to be + created + """ + return decoded_r + + +async def createIndex(decoded_r, num_docs=100, definition=None): + try: + await decoded_r.create_index( + (TextField("play", weight=5.0), TextField("txt"), NumericField("chapter")), + definition=definition, + ) + except redis.ResponseError: + await decoded_r.dropindex(delete_documents=True) + return createIndex(decoded_r, num_docs=num_docs, definition=definition) + + chapters = {} + bzfp = TextIOWrapper(bz2.BZ2File(WILL_PLAY_TEXT), encoding="utf8") + + r = csv.reader(bzfp, delimiter=";") + for n, line in enumerate(r): + play, chapter, _, text = line[1], line[2], line[4], line[5] + + key = f"{play}:{chapter}".lower() + d = chapters.setdefault(key, {}) + d["play"] = play + d["txt"] = d.get("txt", "") + " " + text + d["chapter"] = int(chapter or 0) + if len(chapters) == num_docs: + break + + indexer = decoded_r.batch_indexer(chunk_size=50) + assert isinstance(indexer, AsyncSearch.BatchIndexer) + assert 50 == indexer.chunk_size + + for key, doc in chapters.items(): + await indexer.client.client.hset(key, mapping=doc) + await indexer.commit() + + +@pytest.mark.redismod +async def test_client(decoded_r: redis.Redis): + num_docs = 500 + await createIndex(decoded_r.ft(), num_docs=num_docs) + await waitForIndex(decoded_r, "idx") + # verify info + info = await decoded_r.ft().info() + for k in [ + "index_name", + "index_options", + "attributes", + "num_docs", + "max_doc_id", + "num_terms", + "num_records", + "inverted_sz_mb", + "offset_vectors_sz_mb", + "doc_table_size_mb", + "key_table_size_mb", + "records_per_doc_avg", + "bytes_per_record_avg", + "offsets_per_term_avg", + "offset_bits_per_record_avg", + ]: + assert k in info + + assert decoded_r.ft().index_name == info["index_name"] + assert num_docs == int(info["num_docs"]) + + res = await decoded_r.ft().search("henry iv") + if is_resp2_connection(decoded_r): + assert isinstance(res, Result) + assert 225 == res.total + assert 10 == len(res.docs) + assert res.duration > 0 + + for doc in res.docs: + assert doc.id + assert doc.play == "Henry IV" + assert len(doc.txt) > 0 + + # test no content + res = await decoded_r.ft().search(Query("king").no_content()) + assert 194 == res.total + assert 10 == len(res.docs) + for doc in res.docs: + assert "txt" not in doc.__dict__ + assert "play" not in doc.__dict__ + + # test verbatim vs no verbatim + total = (await decoded_r.ft().search(Query("kings").no_content())).total + vtotal = ( + await decoded_r.ft().search(Query("kings").no_content().verbatim()) + ).total + assert total > vtotal + + # test in fields + txt_total = ( + await decoded_r.ft().search(Query("henry").no_content().limit_fields("txt")) + ).total + play_total = ( + await decoded_r.ft().search( + Query("henry").no_content().limit_fields("play") + ) + ).total + both_total = ( + await decoded_r.ft().search( + Query("henry").no_content().limit_fields("play", "txt") + ) + ).total + assert 129 == txt_total + assert 494 == play_total + assert 494 == both_total + + # test load_document + doc = await decoded_r.ft().load_document("henry vi part 3:62") + assert doc is not None + assert "henry vi part 3:62" == doc.id + assert doc.play == "Henry VI Part 3" + assert len(doc.txt) > 0 + + # test in-keys + ids = [x.id for x in (await decoded_r.ft().search(Query("henry"))).docs] + assert 10 == len(ids) + subset = ids[:5] + docs = await decoded_r.ft().search(Query("henry").limit_ids(*subset)) + assert len(subset) == docs.total + ids = [x.id for x in docs.docs] + assert set(ids) == set(subset) + + # test slop and in order + assert 193 == (await decoded_r.ft().search(Query("henry king"))).total + assert ( + 3 + == ( + await decoded_r.ft().search(Query("henry king").slop(0).in_order()) + ).total + ) + assert ( + 52 + == ( + await decoded_r.ft().search(Query("king henry").slop(0).in_order()) + ).total + ) + assert 53 == (await decoded_r.ft().search(Query("henry king").slop(0))).total + assert 167 == (await decoded_r.ft().search(Query("henry king").slop(100))).total + + # test delete document + await decoded_r.hset("doc-5ghs2", mapping={"play": "Death of a Salesman"}) + res = await decoded_r.ft().search(Query("death of a salesman")) + assert 1 == res.total + + assert 1 == await decoded_r.ft().delete_document("doc-5ghs2") + res = await decoded_r.ft().search(Query("death of a salesman")) + assert 0 == res.total + assert 0 == await decoded_r.ft().delete_document("doc-5ghs2") + + await decoded_r.hset("doc-5ghs2", mapping={"play": "Death of a Salesman"}) + res = await decoded_r.ft().search(Query("death of a salesman")) + assert 1 == res.total + await decoded_r.ft().delete_document("doc-5ghs2") + else: + assert isinstance(res, dict) + assert 225 == res["total_results"] + assert 10 == len(res["results"]) + + for doc in res["results"]: + assert doc["id"] + assert doc["extra_attributes"]["play"] == "Henry IV" + assert len(doc["extra_attributes"]["txt"]) > 0 + + # test no content + res = await decoded_r.ft().search(Query("king").no_content()) + assert 194 == res["total_results"] + assert 10 == len(res["results"]) + for doc in res["results"]: + assert "extra_attributes" not in doc.keys() + + # test verbatim vs no verbatim + total = (await decoded_r.ft().search(Query("kings").no_content()))[ + "total_results" + ] + vtotal = (await decoded_r.ft().search(Query("kings").no_content().verbatim()))[ + "total_results" + ] + assert total > vtotal + + # test in fields + txt_total = ( + await decoded_r.ft().search(Query("henry").no_content().limit_fields("txt")) + )["total_results"] + play_total = ( + await decoded_r.ft().search( + Query("henry").no_content().limit_fields("play") + ) + )["total_results"] + both_total = ( + await decoded_r.ft().search( + Query("henry").no_content().limit_fields("play", "txt") + ) + )["total_results"] + assert 129 == txt_total + assert 494 == play_total + assert 494 == both_total + + # test load_document + doc = await decoded_r.ft().load_document("henry vi part 3:62") + assert doc is not None + assert "henry vi part 3:62" == doc.id + assert doc.play == "Henry VI Part 3" + assert len(doc.txt) > 0 + + # test in-keys + ids = [ + x["id"] for x in (await decoded_r.ft().search(Query("henry")))["results"] + ] + assert 10 == len(ids) + subset = ids[:5] + docs = await decoded_r.ft().search(Query("henry").limit_ids(*subset)) + assert len(subset) == docs["total_results"] + ids = [x["id"] for x in docs["results"]] + assert set(ids) == set(subset) + + # test slop and in order + assert ( + 193 == (await decoded_r.ft().search(Query("henry king")))["total_results"] + ) + assert ( + 3 + == (await decoded_r.ft().search(Query("henry king").slop(0).in_order()))[ + "total_results" + ] + ) + assert ( + 52 + == (await decoded_r.ft().search(Query("king henry").slop(0).in_order()))[ + "total_results" + ] + ) + assert ( + 53 + == (await decoded_r.ft().search(Query("henry king").slop(0)))[ + "total_results" + ] + ) + assert ( + 167 + == (await decoded_r.ft().search(Query("henry king").slop(100)))[ + "total_results" + ] + ) + + # test delete document + await decoded_r.hset("doc-5ghs2", mapping={"play": "Death of a Salesman"}) + res = await decoded_r.ft().search(Query("death of a salesman")) + assert 1 == res["total_results"] + + assert 1 == await decoded_r.ft().delete_document("doc-5ghs2") + res = await decoded_r.ft().search(Query("death of a salesman")) + assert 0 == res["total_results"] + assert 0 == await decoded_r.ft().delete_document("doc-5ghs2") + + await decoded_r.hset("doc-5ghs2", mapping={"play": "Death of a Salesman"}) + res = await decoded_r.ft().search(Query("death of a salesman")) + assert 1 == res["total_results"] + await decoded_r.ft().delete_document("doc-5ghs2") + + +@pytest.mark.redismod +@pytest.mark.onlynoncluster +@skip_if_server_version_gte("7.9.0") +async def test_scores(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("txt"),)) + + await decoded_r.hset("doc1", mapping={"txt": "foo baz"}) + await decoded_r.hset("doc2", mapping={"txt": "foo bar"}) + + q = Query("foo ~bar").with_scores() + res = await decoded_r.ft().search(q) + if is_resp2_connection(decoded_r): + assert 2 == res.total + assert "doc2" == res.docs[0].id + assert 3.0 == res.docs[0].score + assert "doc1" == res.docs[1].id + else: + assert 2 == res["total_results"] + assert "doc2" == res["results"][0]["id"] + assert 3.0 == res["results"][0]["score"] + assert "doc1" == res["results"][1]["id"] + + +@pytest.mark.redismod +@pytest.mark.onlynoncluster +@skip_if_server_version_lt("7.9.0") +async def test_scores_with_new_default_scorer(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("txt"),)) + + await decoded_r.hset("doc1", mapping={"txt": "foo baz"}) + await decoded_r.hset("doc2", mapping={"txt": "foo bar"}) + + q = Query("foo ~bar").with_scores() + res = await decoded_r.ft().search(q) + if is_resp2_connection(decoded_r): + assert 2 == res.total + assert "doc2" == res.docs[0].id + assert 0.87 == pytest.approx(res.docs[0].score, 0.01) + assert "doc1" == res.docs[1].id + else: + assert 2 == res["total_results"] + assert "doc2" == res["results"][0]["id"] + assert 0.87 == pytest.approx(res["results"][0]["score"], 0.01) + assert "doc1" == res["results"][1]["id"] + + +@pytest.mark.redismod +async def test_stopwords(decoded_r: redis.Redis): + stopwords = ["foo", "bar", "baz"] + await decoded_r.ft().create_index((TextField("txt"),), stopwords=stopwords) + await decoded_r.hset("doc1", mapping={"txt": "foo bar"}) + await decoded_r.hset("doc2", mapping={"txt": "hello world"}) + await waitForIndex(decoded_r, "idx") + + q1 = Query("foo bar").no_content() + q2 = Query("foo bar hello world").no_content() + res1, res2 = await decoded_r.ft().search(q1), await decoded_r.ft().search(q2) + if is_resp2_connection(decoded_r): + assert 0 == res1.total + assert 1 == res2.total + else: + assert 0 == res1["total_results"] + assert 1 == res2["total_results"] + + +@pytest.mark.redismod +async def test_filters(decoded_r: redis.Redis): + await decoded_r.ft().create_index( + (TextField("txt"), NumericField("num"), GeoField("loc")) + ) + await decoded_r.hset( + "doc1", mapping={"txt": "foo bar", "num": 3.141, "loc": "-0.441,51.458"} + ) + await decoded_r.hset( + "doc2", mapping={"txt": "foo baz", "num": 2, "loc": "-0.1,51.2"} + ) + + await waitForIndex(decoded_r, "idx") + # Test numerical filter + q1 = Query("foo").add_filter(NumericFilter("num", 0, 2)).no_content() + q2 = ( + Query("foo") + .add_filter(NumericFilter("num", 2, NumericFilter.INF, minExclusive=True)) + .no_content() + ) + res1, res2 = await decoded_r.ft().search(q1), await decoded_r.ft().search(q2) + + if is_resp2_connection(decoded_r): + assert 1 == res1.total + assert 1 == res2.total + assert "doc2" == res1.docs[0].id + assert "doc1" == res2.docs[0].id + else: + assert 1 == res1["total_results"] + assert 1 == res2["total_results"] + assert "doc2" == res1["results"][0]["id"] + assert "doc1" == res2["results"][0]["id"] + + # Test geo filter + q1 = Query("foo").add_filter(GeoFilter("loc", -0.44, 51.45, 10)).no_content() + q2 = Query("foo").add_filter(GeoFilter("loc", -0.44, 51.45, 100)).no_content() + res1, res2 = await decoded_r.ft().search(q1), await decoded_r.ft().search(q2) + + if is_resp2_connection(decoded_r): + assert 1 == res1.total + assert 2 == res2.total + assert "doc1" == res1.docs[0].id + + # Sort results, after RDB reload order may change + res = [res2.docs[0].id, res2.docs[1].id] + res.sort() + assert ["doc1", "doc2"] == res + else: + assert 1 == res1["total_results"] + assert 2 == res2["total_results"] + assert "doc1" == res1["results"][0]["id"] + + # Sort results, after RDB reload order may change + res = [res2["results"][0]["id"], res2["results"][1]["id"]] + res.sort() + assert ["doc1", "doc2"] == res + + +@pytest.mark.redismod +async def test_sort_by(decoded_r: redis.Redis): + await decoded_r.ft().create_index( + (TextField("txt"), NumericField("num", sortable=True)) + ) + await decoded_r.hset("doc1", mapping={"txt": "foo bar", "num": 1}) + await decoded_r.hset("doc2", mapping={"txt": "foo baz", "num": 2}) + await decoded_r.hset("doc3", mapping={"txt": "foo qux", "num": 3}) + + # Test sort + q1 = Query("foo").sort_by("num", asc=True).no_content() + q2 = Query("foo").sort_by("num", asc=False).no_content() + res1, res2 = await decoded_r.ft().search(q1), await decoded_r.ft().search(q2) + + if is_resp2_connection(decoded_r): + assert 3 == res1.total + assert "doc1" == res1.docs[0].id + assert "doc2" == res1.docs[1].id + assert "doc3" == res1.docs[2].id + assert 3 == res2.total + assert "doc1" == res2.docs[2].id + assert "doc2" == res2.docs[1].id + assert "doc3" == res2.docs[0].id + else: + assert 3 == res1["total_results"] + assert "doc1" == res1["results"][0]["id"] + assert "doc2" == res1["results"][1]["id"] + assert "doc3" == res1["results"][2]["id"] + assert 3 == res2["total_results"] + assert "doc1" == res2["results"][2]["id"] + assert "doc2" == res2["results"][1]["id"] + assert "doc3" == res2["results"][0]["id"] + + +@pytest.mark.redismod +@skip_ifmodversion_lt("2.0.0", "search") +async def test_drop_index(decoded_r: redis.Redis): + """ + Ensure the index gets dropped by data remains by default + """ + for x in range(20): + for keep_docs in [[True, {}], [False, {"name": "haveit"}]]: + idx = "HaveIt" + index = getClient(decoded_r) + await index.hset("index:haveit", mapping={"name": "haveit"}) + idef = IndexDefinition(prefix=["index:"]) + await index.ft(idx).create_index((TextField("name"),), definition=idef) + await waitForIndex(index, idx) + await index.ft(idx).dropindex(delete_documents=keep_docs[0]) + i = await index.hgetall("index:haveit") + assert i == keep_docs[1] + + +@pytest.mark.redismod +async def test_example(decoded_r: redis.Redis): + # Creating the index definition and schema + await decoded_r.ft().create_index( + (TextField("title", weight=5.0), TextField("body")) + ) + + # Indexing a document + await decoded_r.hset( + "doc1", + mapping={ + "title": "RediSearch", + "body": "Redisearch impements a search engine on top of redis", + }, + ) + + # Searching with complex parameters: + q = Query("search engine").verbatim().no_content().paging(0, 5) + + res = await decoded_r.ft().search(q) + assert res is not None + + +@pytest.mark.redismod +async def test_auto_complete(decoded_r: redis.Redis): + n = 0 + with open(TITLES_CSV) as f: + cr = csv.reader(f) + + for row in cr: + n += 1 + term, score = row[0], float(row[1]) + assert n == await decoded_r.ft().sugadd("ac", Suggestion(term, score=score)) + + assert n == await decoded_r.ft().suglen("ac") + ret = await decoded_r.ft().sugget("ac", "bad", with_scores=True) + assert 2 == len(ret) + assert "badger" == ret[0].string + assert isinstance(ret[0].score, float) + assert 1.0 != ret[0].score + assert "badalte rishtey" == ret[1].string + assert isinstance(ret[1].score, float) + assert 1.0 != ret[1].score + + ret = await decoded_r.ft().sugget("ac", "bad", fuzzy=True, num=10) + assert 10 == len(ret) + assert 1.0 == ret[0].score + strs = {x.string for x in ret} + + for sug in strs: + assert 1 == await decoded_r.ft().sugdel("ac", sug) + # make sure a second delete returns 0 + for sug in strs: + assert 0 == await decoded_r.ft().sugdel("ac", sug) + + # make sure they were actually deleted + ret2 = await decoded_r.ft().sugget("ac", "bad", fuzzy=True, num=10) + for sug in ret2: + assert sug.string not in strs + + # Test with payload + await decoded_r.ft().sugadd("ac", Suggestion("pay1", payload="pl1")) + await decoded_r.ft().sugadd("ac", Suggestion("pay2", payload="pl2")) + await decoded_r.ft().sugadd("ac", Suggestion("pay3", payload="pl3")) + + sugs = await decoded_r.ft().sugget( + "ac", "pay", with_payloads=True, with_scores=True + ) + assert 3 == len(sugs) + for sug in sugs: + assert sug.payload + assert sug.payload.startswith("pl") + + +@pytest.mark.redismod +async def test_no_index(decoded_r: redis.Redis): + await decoded_r.ft().create_index( + ( + TextField("field"), + TextField("text", no_index=True, sortable=True), + NumericField("numeric", no_index=True, sortable=True), + GeoField("geo", no_index=True, sortable=True), + TagField("tag", no_index=True, sortable=True), + ) + ) + + await decoded_r.hset( + "doc1", + mapping={"field": "aaa", "text": "1", "numeric": "1", "geo": "1,1", "tag": "1"}, + ) + await decoded_r.hset( + "doc2", + mapping={"field": "aab", "text": "2", "numeric": "2", "geo": "2,2", "tag": "2"}, + ) + await waitForIndex(decoded_r, "idx") + + if is_resp2_connection(decoded_r): + res = await decoded_r.ft().search(Query("@text:aa*")) + assert 0 == res.total + + res = await decoded_r.ft().search(Query("@field:aa*")) + assert 2 == res.total + + res = await decoded_r.ft().search(Query("*").sort_by("text", asc=False)) + assert 2 == res.total + assert "doc2" == res.docs[0].id + + res = await decoded_r.ft().search(Query("*").sort_by("text", asc=True)) + assert "doc1" == res.docs[0].id + + res = await decoded_r.ft().search(Query("*").sort_by("numeric", asc=True)) + assert "doc1" == res.docs[0].id + + res = await decoded_r.ft().search(Query("*").sort_by("geo", asc=True)) + assert "doc1" == res.docs[0].id + + res = await decoded_r.ft().search(Query("*").sort_by("tag", asc=True)) + assert "doc1" == res.docs[0].id + else: + res = await decoded_r.ft().search(Query("@text:aa*")) + assert 0 == res["total_results"] + + res = await decoded_r.ft().search(Query("@field:aa*")) + assert 2 == res["total_results"] + + res = await decoded_r.ft().search(Query("*").sort_by("text", asc=False)) + assert 2 == res["total_results"] + assert "doc2" == res["results"][0]["id"] + + res = await decoded_r.ft().search(Query("*").sort_by("text", asc=True)) + assert "doc1" == res["results"][0]["id"] + + res = await decoded_r.ft().search(Query("*").sort_by("numeric", asc=True)) + assert "doc1" == res["results"][0]["id"] + + res = await decoded_r.ft().search(Query("*").sort_by("geo", asc=True)) + assert "doc1" == res["results"][0]["id"] + + res = await decoded_r.ft().search(Query("*").sort_by("tag", asc=True)) + assert "doc1" == res["results"][0]["id"] + + # Ensure exception is raised for non-indexable, non-sortable fields + with pytest.raises(Exception): + TextField("name", no_index=True, sortable=False) + with pytest.raises(Exception): + NumericField("name", no_index=True, sortable=False) + with pytest.raises(Exception): + GeoField("name", no_index=True, sortable=False) + with pytest.raises(Exception): + TagField("name", no_index=True, sortable=False) + + +@pytest.mark.redismod +async def test_explain(decoded_r: redis.Redis): + await decoded_r.ft().create_index( + (TextField("f1"), TextField("f2"), TextField("f3")) + ) + res = await decoded_r.ft().explain("@f3:f3_val @f2:f2_val @f1:f1_val") + assert res + + +@pytest.mark.redismod +async def test_explaincli(decoded_r: redis.Redis): + with pytest.raises(NotImplementedError): + await decoded_r.ft().explain_cli("foo") + + +@pytest.mark.redismod +async def test_summarize(decoded_r: redis.Redis): + await createIndex(decoded_r.ft()) + await waitForIndex(decoded_r, "idx") + + q = Query('"king henry"').paging(0, 1) + q.highlight(fields=("play", "txt"), tags=("", "")) + q.summarize("txt") + + if is_resp2_connection(decoded_r): + doc = sorted((await decoded_r.ft().search(q)).docs)[0] + assert "Henry IV" == doc.play + assert ( + "ACT I SCENE I. London. The palace. Enter KING HENRY, LORD JOHN OF LANCASTER, the EARL of WESTMORELAND, SIR... " # noqa + == doc.txt + ) + + q = Query('"king henry"').paging(0, 1).summarize().highlight() + + doc = sorted((await decoded_r.ft().search(q)).docs)[0] + assert "Henry ... " == doc.play + assert ( + "ACT I SCENE I. London. The palace. Enter KING HENRY, LORD JOHN OF LANCASTER, the EARL of WESTMORELAND, SIR... " # noqa + == doc.txt + ) + else: + doc = sorted((await decoded_r.ft().search(q))["results"])[0] + assert "Henry IV" == doc["extra_attributes"]["play"] + assert ( + "ACT I SCENE I. London. The palace. Enter KING HENRY, LORD JOHN OF LANCASTER, the EARL of WESTMORELAND, SIR... " # noqa + == doc["extra_attributes"]["txt"] + ) + + q = Query('"king henry"').paging(0, 1).summarize().highlight() + + doc = sorted((await decoded_r.ft().search(q))["results"])[0] + assert "Henry ... " == doc["extra_attributes"]["play"] + assert ( + "ACT I SCENE I. London. The palace. Enter KING HENRY, LORD JOHN OF LANCASTER, the EARL of WESTMORELAND, SIR... " # noqa + == doc["extra_attributes"]["txt"] + ) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("2.0.0", "search") +async def test_alias(decoded_r: redis.Redis): + index1 = getClient(decoded_r) + index2 = getClient(decoded_r) + + def1 = IndexDefinition(prefix=["index1:"]) + def2 = IndexDefinition(prefix=["index2:"]) + + ftindex1 = index1.ft("testAlias") + ftindex2 = index2.ft("testAlias2") + await ftindex1.create_index((TextField("name"),), definition=def1) + await ftindex2.create_index((TextField("name"),), definition=def2) + + await index1.hset("index1:lonestar", mapping={"name": "lonestar"}) + await index2.hset("index2:yogurt", mapping={"name": "yogurt"}) + + if is_resp2_connection(decoded_r): + res = (await ftindex1.search("*")).docs[0] + assert "index1:lonestar" == res.id + + # create alias and check for results + await ftindex1.aliasadd("spaceballs") + alias_client = getClient(decoded_r).ft("spaceballs") + res = (await alias_client.search("*")).docs[0] + assert "index1:lonestar" == res.id + + # Throw an exception when trying to add an alias that already exists + with pytest.raises(Exception): + await ftindex2.aliasadd("spaceballs") + + # update alias and ensure new results + await ftindex2.aliasupdate("spaceballs") + alias_client2 = getClient(decoded_r).ft("spaceballs") + + res = (await alias_client2.search("*")).docs[0] + assert "index2:yogurt" == res.id + else: + res = (await ftindex1.search("*"))["results"][0] + assert "index1:lonestar" == res["id"] + + # create alias and check for results + await ftindex1.aliasadd("spaceballs") + alias_client = getClient(await decoded_r).ft("spaceballs") + res = (await alias_client.search("*"))["results"][0] + assert "index1:lonestar" == res["id"] + + # Throw an exception when trying to add an alias that already exists + with pytest.raises(Exception): + await ftindex2.aliasadd("spaceballs") + + # update alias and ensure new results + await ftindex2.aliasupdate("spaceballs") + alias_client2 = getClient(await decoded_r).ft("spaceballs") + + res = (await alias_client2.search("*"))["results"][0] + assert "index2:yogurt" == res["id"] + + await ftindex2.aliasdel("spaceballs") + with pytest.raises(Exception): + (await alias_client2.search("*")).docs[0] + + +@pytest.mark.redismod +@pytest.mark.xfail(strict=False) +async def test_alias_basic(decoded_r: redis.Redis): + # Creating a client with one index + client = getClient(decoded_r) + await client.flushdb() + index1 = getClient(decoded_r).ft("testAlias") + + await index1.create_index((TextField("txt"),)) + await index1.client.hset("doc1", mapping={"txt": "text goes here"}) + + index2 = getClient(decoded_r).ft("testAlias2") + await index2.create_index((TextField("txt"),)) + await index2.client.hset("doc2", mapping={"txt": "text goes here"}) + + # add the actual alias and check + await index1.aliasadd("myalias") + alias_client = getClient(decoded_r).ft("myalias") + if is_resp2_connection(decoded_r): + res = sorted((await alias_client.search("*")).docs, key=lambda x: x.id) + assert "doc1" == res[0].id + + # Throw an exception when trying to add an alias that already exists + with pytest.raises(Exception): + await index2.aliasadd("myalias") + + # update the alias and ensure we get doc2 + await index2.aliasupdate("myalias") + alias_client2 = getClient(decoded_r).ft("myalias") + res = sorted((await alias_client2.search("*")).docs, key=lambda x: x.id) + assert "doc1" == res[0].id + else: + res = sorted((await alias_client.search("*"))["results"], key=lambda x: x["id"]) + assert "doc1" == res[0]["id"] + + # Throw an exception when trying to add an alias that already exists + with pytest.raises(Exception): + await index2.aliasadd("myalias") + + # update the alias and ensure we get doc2 + await index2.aliasupdate("myalias") + alias_client2 = getClient(client).ft("myalias") + res = sorted( + (await alias_client2.search("*"))["results"], key=lambda x: x["id"] + ) + assert "doc1" == res[0]["id"] + + # delete the alias and expect an error if we try to query again + await index2.aliasdel("myalias") + with pytest.raises(Exception): + _ = (await alias_client2.search("*")).docs[0] + + +@pytest.mark.redismod +async def test_tags(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("txt"), TagField("tags"))) + tags = "foo,foo bar,hello;world" + tags2 = "soba,ramen" + + await decoded_r.hset("doc1", mapping={"txt": "fooz barz", "tags": tags}) + await decoded_r.hset("doc2", mapping={"txt": "noodles", "tags": tags2}) + await waitForIndex(decoded_r, "idx") + + q = Query("@tags:{foo}") + if is_resp2_connection(decoded_r): + res = await decoded_r.ft().search(q) + assert 1 == res.total + + q = Query("@tags:{foo bar}") + res = await decoded_r.ft().search(q) + assert 1 == res.total + + q = Query("@tags:{foo\\ bar}") + res = await decoded_r.ft().search(q) + assert 1 == res.total + + q = Query("@tags:{hello\\;world}") + res = await decoded_r.ft().search(q) + assert 1 == res.total + + q2 = await decoded_r.ft().tagvals("tags") + assert (tags.split(",") + tags2.split(",")).sort() == q2.sort() + else: + res = await decoded_r.ft().search(q) + assert 1 == res["total_results"] + + q = Query("@tags:{foo bar}") + res = await decoded_r.ft().search(q) + assert 1 == res["total_results"] + + q = Query("@tags:{foo\\ bar}") + res = await decoded_r.ft().search(q) + assert 1 == res["total_results"] + + q = Query("@tags:{hello\\;world}") + res = await decoded_r.ft().search(q) + assert 1 == res["total_results"] + + q2 = await decoded_r.ft().tagvals("tags") + assert set(tags.split(",") + tags2.split(",")) == set(q2) + + +@pytest.mark.redismod +async def test_textfield_sortable_nostem(decoded_r: redis.Redis): + # Creating the index definition with sortable and no_stem + await decoded_r.ft().create_index((TextField("txt", sortable=True, no_stem=True),)) + + # Now get the index info to confirm its contents + response = await decoded_r.ft().info() + if is_resp2_connection(decoded_r): + assert "SORTABLE" in response["attributes"][0] + assert "NOSTEM" in response["attributes"][0] + else: + assert "SORTABLE" in response["attributes"][0]["flags"] + assert "NOSTEM" in response["attributes"][0]["flags"] + + +@pytest.mark.redismod +async def test_alter_schema_add(decoded_r: redis.Redis): + # Creating the index definition and schema + await decoded_r.ft().create_index(TextField("title")) + + # Using alter to add a field + await decoded_r.ft().alter_schema_add(TextField("body")) + + # Indexing a document + await decoded_r.hset( + "doc1", mapping={"title": "MyTitle", "body": "Some content only in the body"} + ) + + # Searching with parameter only in the body (the added field) + q = Query("only in the body") + + # Ensure we find the result searching on the added body field + res = await decoded_r.ft().search(q) + if is_resp2_connection(decoded_r): + assert 1 == res.total + else: + assert 1 == res["total_results"] + + +@pytest.mark.redismod +async def test_spell_check(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("f1"), TextField("f2"))) + + await decoded_r.hset( + "doc1", mapping={"f1": "some valid content", "f2": "this is sample text"} + ) + await decoded_r.hset("doc2", mapping={"f1": "very important", "f2": "lorem ipsum"}) + await waitForIndex(decoded_r, "idx") + + if is_resp2_connection(decoded_r): + # test spellcheck + res = await decoded_r.ft().spellcheck("impornant") + assert "important" == res["impornant"][0]["suggestion"] + + res = await decoded_r.ft().spellcheck("contnt") + assert "content" == res["contnt"][0]["suggestion"] + + # test spellcheck with Levenshtein distance + res = await decoded_r.ft().spellcheck("vlis") + assert res == {} + res = await decoded_r.ft().spellcheck("vlis", distance=2) + assert "valid" == res["vlis"][0]["suggestion"] + + # test spellcheck include + await decoded_r.ft().dict_add("dict", "lore", "lorem", "lorm") + res = await decoded_r.ft().spellcheck("lorm", include="dict") + assert len(res["lorm"]) == 3 + assert ( + res["lorm"][0]["suggestion"], + res["lorm"][1]["suggestion"], + res["lorm"][2]["suggestion"], + ) == ("lorem", "lore", "lorm") + assert (res["lorm"][0]["score"], res["lorm"][1]["score"]) == ("0.5", "0") + + # test spellcheck exclude + res = await decoded_r.ft().spellcheck("lorm", exclude="dict") + assert res == {} + else: + # test spellcheck + res = await decoded_r.ft().spellcheck("impornant") + assert "important" in res["results"]["impornant"][0].keys() + + res = await decoded_r.ft().spellcheck("contnt") + assert "content" in res["results"]["contnt"][0].keys() + + # test spellcheck with Levenshtein distance + res = await decoded_r.ft().spellcheck("vlis") + assert res == {"results": {"vlis": []}} + res = await decoded_r.ft().spellcheck("vlis", distance=2) + assert "valid" in res["results"]["vlis"][0].keys() + + # test spellcheck include + await decoded_r.ft().dict_add("dict", "lore", "lorem", "lorm") + res = await decoded_r.ft().spellcheck("lorm", include="dict") + assert len(res["results"]["lorm"]) == 3 + assert "lorem" in res["results"]["lorm"][0].keys() + assert "lore" in res["results"]["lorm"][1].keys() + assert "lorm" in res["results"]["lorm"][2].keys() + assert ( + res["results"]["lorm"][0]["lorem"], + res["results"]["lorm"][1]["lore"], + ) == (0.5, 0) + + # test spellcheck exclude + res = await decoded_r.ft().spellcheck("lorm", exclude="dict") + assert res == {"results": {}} + + +@pytest.mark.redismod +async def test_dict_operations(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("f1"), TextField("f2"))) + # Add three items + res = await decoded_r.ft().dict_add("custom_dict", "item1", "item2", "item3") + assert 3 == res + + # Remove one item + res = await decoded_r.ft().dict_del("custom_dict", "item2") + assert 1 == res + + # Dump dict and inspect content + res = await decoded_r.ft().dict_dump("custom_dict") + assert res == ["item1", "item3"] + + # Remove rest of the items before reload + await decoded_r.ft().dict_del("custom_dict", *res) + + +@pytest.mark.redismod +async def test_phonetic_matcher(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("name"),)) + await decoded_r.hset("doc1", mapping={"name": "Jon"}) + await decoded_r.hset("doc2", mapping={"name": "John"}) + + res = await decoded_r.ft().search(Query("Jon")) + if is_resp2_connection(decoded_r): + assert 1 == len(res.docs) + assert "Jon" == res.docs[0].name + else: + assert 1 == res["total_results"] + assert "Jon" == res["results"][0]["extra_attributes"]["name"] + + # Drop and create index with phonetic matcher + await decoded_r.flushdb() + + await decoded_r.ft().create_index((TextField("name", phonetic_matcher="dm:en"),)) + await decoded_r.hset("doc1", mapping={"name": "Jon"}) + await decoded_r.hset("doc2", mapping={"name": "John"}) + + res = await decoded_r.ft().search(Query("Jon")) + if is_resp2_connection(decoded_r): + assert 2 == len(res.docs) + assert ["John", "Jon"] == sorted(d.name for d in res.docs) + else: + assert 2 == res["total_results"] + assert ["John", "Jon"] == sorted( + d["extra_attributes"]["name"] for d in res["results"] + ) + + +@pytest.mark.redismod +@pytest.mark.onlynoncluster +# NOTE(imalinovskyi): This test contains hardcoded scores valid only for RediSearch 2.8+ +@skip_ifmodversion_lt("2.8.0", "search") +@skip_if_server_version_gte("7.9.0") +async def test_scorer(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("description"),)) + + await decoded_r.hset( + "doc1", mapping={"description": "The quick brown fox jumps over the lazy dog"} + ) + await decoded_r.hset( + "doc2", + mapping={ + "description": "Quick alice was beginning to get very tired of sitting by her quick sister on the bank, and of having nothing to do." # noqa + }, + ) + + if is_resp2_connection(decoded_r): + # default scorer is TFIDF + res = await decoded_r.ft().search(Query("quick").with_scores()) + assert 1.0 == res.docs[0].score + res = await decoded_r.ft().search(Query("quick").scorer("TFIDF").with_scores()) + assert 1.0 == res.docs[0].score + res = await decoded_r.ft().search( + Query("quick").scorer("TFIDF.DOCNORM").with_scores() + ) + assert 0.14285714285714285 == res.docs[0].score + res = await decoded_r.ft().search(Query("quick").scorer("BM25").with_scores()) + assert 0.22471909420069797 == res.docs[0].score + res = await decoded_r.ft().search(Query("quick").scorer("DISMAX").with_scores()) + assert 2.0 == res.docs[0].score + res = await decoded_r.ft().search( + Query("quick").scorer("DOCSCORE").with_scores() + ) + assert 1.0 == res.docs[0].score + res = await decoded_r.ft().search( + Query("quick").scorer("HAMMING").with_scores() + ) + assert 0.0 == res.docs[0].score + else: + res = await decoded_r.ft().search(Query("quick").with_scores()) + assert 1.0 == res["results"][0]["score"] + res = await decoded_r.ft().search(Query("quick").scorer("TFIDF").with_scores()) + assert 1.0 == res["results"][0]["score"] + res = await decoded_r.ft().search( + Query("quick").scorer("TFIDF.DOCNORM").with_scores() + ) + assert 0.14285714285714285 == res["results"][0]["score"] + res = await decoded_r.ft().search(Query("quick").scorer("BM25").with_scores()) + assert 0.22471909420069797 == res["results"][0]["score"] + res = await decoded_r.ft().search(Query("quick").scorer("DISMAX").with_scores()) + assert 2.0 == res["results"][0]["score"] + res = await decoded_r.ft().search( + Query("quick").scorer("DOCSCORE").with_scores() + ) + assert 1.0 == res["results"][0]["score"] + res = await decoded_r.ft().search( + Query("quick").scorer("HAMMING").with_scores() + ) + assert 0.0 == res["results"][0]["score"] + + +@pytest.mark.redismod +@pytest.mark.onlynoncluster +# NOTE(imalinovskyi): This test contains hardcoded scores valid only for RediSearch 2.8+ +@skip_ifmodversion_lt("2.8.0", "search") +@skip_if_server_version_lt("7.9.0") +async def test_scorer_with_new_default_scorer(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("description"),)) + + await decoded_r.hset( + "doc1", mapping={"description": "The quick brown fox jumps over the lazy dog"} + ) + await decoded_r.hset( + "doc2", + mapping={ + "description": "Quick alice was beginning to get very tired of sitting by her quick sister on the bank, and of having nothing to do." # noqa + }, + ) + + if is_resp2_connection(decoded_r): + # default scorer is BM25STD + res = await decoded_r.ft().search(Query("quick").with_scores()) + assert 0.23 == pytest.approx(res.docs[0].score, 0.05) + res = await decoded_r.ft().search(Query("quick").scorer("TFIDF").with_scores()) + assert 1.0 == res.docs[0].score + res = await decoded_r.ft().search( + Query("quick").scorer("TFIDF.DOCNORM").with_scores() + ) + assert 0.14285714285714285 == res.docs[0].score + res = await decoded_r.ft().search(Query("quick").scorer("BM25").with_scores()) + assert 0.22471909420069797 == res.docs[0].score + res = await decoded_r.ft().search(Query("quick").scorer("DISMAX").with_scores()) + assert 2.0 == res.docs[0].score + res = await decoded_r.ft().search( + Query("quick").scorer("DOCSCORE").with_scores() + ) + assert 1.0 == res.docs[0].score + res = await decoded_r.ft().search( + Query("quick").scorer("HAMMING").with_scores() + ) + assert 0.0 == res.docs[0].score + else: + res = await decoded_r.ft().search(Query("quick").with_scores()) + assert 0.23 == pytest.approx(res["results"][0]["score"], 0.05) + res = await decoded_r.ft().search(Query("quick").scorer("TFIDF").with_scores()) + assert 1.0 == res["results"][0]["score"] + res = await decoded_r.ft().search( + Query("quick").scorer("TFIDF.DOCNORM").with_scores() + ) + assert 0.14285714285714285 == res["results"][0]["score"] + res = await decoded_r.ft().search(Query("quick").scorer("BM25").with_scores()) + assert 0.22471909420069797 == res["results"][0]["score"] + res = await decoded_r.ft().search(Query("quick").scorer("DISMAX").with_scores()) + assert 2.0 == res["results"][0]["score"] + res = await decoded_r.ft().search( + Query("quick").scorer("DOCSCORE").with_scores() + ) + assert 1.0 == res["results"][0]["score"] + res = await decoded_r.ft().search( + Query("quick").scorer("HAMMING").with_scores() + ) + assert 0.0 == res["results"][0]["score"] + + +@pytest.mark.redismod +async def test_get(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("f1"), TextField("f2"))) + + assert [None] == await decoded_r.ft().get("doc1") + assert [None, None] == await decoded_r.ft().get("doc2", "doc1") + + await decoded_r.hset( + "doc1", mapping={"f1": "some valid content dd1", "f2": "this is sample text f1"} + ) + await decoded_r.hset( + "doc2", mapping={"f1": "some valid content dd2", "f2": "this is sample text f2"} + ) + + assert [ + ["f1", "some valid content dd2", "f2", "this is sample text f2"] + ] == await decoded_r.ft().get("doc2") + assert [ + ["f1", "some valid content dd1", "f2", "this is sample text f1"], + ["f1", "some valid content dd2", "f2", "this is sample text f2"], + ] == await decoded_r.ft().get("doc1", "doc2") + + +@pytest.mark.redismod +@pytest.mark.onlynoncluster +@skip_ifmodversion_lt("2.2.0", "search") +@skip_if_server_version_gte("7.9.0") +async def test_config(decoded_r: redis.Redis): + assert await decoded_r.ft().config_set("TIMEOUT", "100") + with pytest.raises(redis.ResponseError): + await decoded_r.ft().config_set("TIMEOUT", "null") + res = await decoded_r.ft().config_get("*") + assert "100" == res["TIMEOUT"] + res = await decoded_r.ft().config_get("TIMEOUT") + assert "100" == res["TIMEOUT"] + + +@pytest.mark.redismod +@pytest.mark.onlynoncluster +@skip_if_server_version_lt("7.9.0") +async def test_config_with_removed_ftconfig(decoded_r: redis.Redis): + assert await decoded_r.config_set("timeout", "100") + with pytest.raises(redis.ResponseError): + await decoded_r.config_set("timeout", "null") + res = await decoded_r.config_get("*") + assert "100" == res["timeout"] + res = await decoded_r.config_get("timeout") + assert "100" == res["timeout"] + + +@pytest.mark.redismod +@pytest.mark.onlynoncluster +async def test_aggregations_groupby(decoded_r: redis.Redis): + # Creating the index definition and schema + await decoded_r.ft().create_index( + ( + NumericField("random_num"), + TextField("title"), + TextField("body"), + TextField("parent"), + ) + ) + + # Indexing a document + await decoded_r.hset( + "search", + mapping={ + "title": "RediSearch", + "body": "Redisearch impements a search engine on top of redis", + "parent": "redis", + "random_num": 10, + }, + ) + await decoded_r.hset( + "ai", + mapping={ + "title": "RedisAI", + "body": "RedisAI executes Deep Learning/Machine Learning models and managing their data.", # noqa + "parent": "redis", + "random_num": 3, + }, + ) + await decoded_r.hset( + "json", + mapping={ + "title": "RedisJson", + "body": "RedisJSON implements ECMA-404 The JSON Data Interchange Standard as a native data type.", # noqa + "parent": "redis", + "random_num": 8, + }, + ) + + for dialect in [1, 2]: + if is_resp2_connection(decoded_r): + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.count()) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "3" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.count_distinct("@title")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "3" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.count_distinctish("@title")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "3" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.sum("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "21" # 10+8+3 + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.min("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "3" # min(10,8,3) + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.max("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "10" # max(10,8,3) + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.avg("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "7" # (10+3+8)/3 + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.stddev("random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "3.60555127546" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.quantile("@random_num", 0.5)) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[3] == "8" # median of 3,8,10 + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.tolist("@title")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert set(res[3]) == {"RediSearch", "RedisAI", "RedisJson"} + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.first_value("@title").alias("first")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res == ["parent", "redis", "first", "RediSearch"] + + req = ( + aggregations.AggregateRequest("redis") + .group_by( + "@parent", reducers.random_sample("@title", 2).alias("random") + ) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req)).rows[0] + assert res[1] == "redis" + assert res[2] == "random" + assert len(res[3]) == 2 + assert res[3][0] in ["RediSearch", "RedisAI", "RedisJson"] + else: + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.count()) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert res["extra_attributes"]["__generated_aliascount"] == "3" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.count_distinct("@title")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert ( + res["extra_attributes"]["__generated_aliascount_distincttitle"] == "3" + ) + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.count_distinctish("@title")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert ( + res["extra_attributes"]["__generated_aliascount_distinctishtitle"] + == "3" + ) + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.sum("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert res["extra_attributes"]["__generated_aliassumrandom_num"] == "21" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.min("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert res["extra_attributes"]["__generated_aliasminrandom_num"] == "3" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.max("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert res["extra_attributes"]["__generated_aliasmaxrandom_num"] == "10" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.avg("@random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert res["extra_attributes"]["__generated_aliasavgrandom_num"] == "7" + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.stddev("random_num")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert ( + res["extra_attributes"]["__generated_aliasstddevrandom_num"] + == "3.60555127546" + ) + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.quantile("@random_num", 0.5)) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert ( + res["extra_attributes"]["__generated_aliasquantilerandom_num,0.5"] + == "8" + ) + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.tolist("@title")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert set(res["extra_attributes"]["__generated_aliastolisttitle"]) == { + "RediSearch", + "RedisAI", + "RedisJson", + } + + req = ( + aggregations.AggregateRequest("redis") + .group_by("@parent", reducers.first_value("@title").alias("first")) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"] == {"parent": "redis", "first": "RediSearch"} + + req = ( + aggregations.AggregateRequest("redis") + .group_by( + "@parent", reducers.random_sample("@title", 2).alias("random") + ) + .dialect(dialect) + ) + + res = (await decoded_r.ft().aggregate(req))["results"][0] + assert res["extra_attributes"]["parent"] == "redis" + assert "random" in res["extra_attributes"].keys() + assert len(res["extra_attributes"]["random"]) == 2 + assert res["extra_attributes"]["random"][0] in [ + "RediSearch", + "RedisAI", + "RedisJson", + ] + + +@pytest.mark.redismod +async def test_aggregations_sort_by_and_limit(decoded_r: redis.Redis): + await decoded_r.ft().create_index((TextField("t1"), TextField("t2"))) + + await decoded_r.ft().client.hset("doc1", mapping={"t1": "a", "t2": "b"}) + await decoded_r.ft().client.hset("doc2", mapping={"t1": "b", "t2": "a"}) + + if is_resp2_connection(decoded_r): + # test sort_by using SortDirection + req = aggregations.AggregateRequest("*").sort_by( + aggregations.Asc("@t2"), aggregations.Desc("@t1") + ) + res = await decoded_r.ft().aggregate(req) + assert res.rows[0] == ["t2", "a", "t1", "b"] + assert res.rows[1] == ["t2", "b", "t1", "a"] + + # test sort_by without SortDirection + req = aggregations.AggregateRequest("*").sort_by("@t1") + res = await decoded_r.ft().aggregate(req) + assert res.rows[0] == ["t1", "a"] + assert res.rows[1] == ["t1", "b"] + + # test sort_by with max + req = aggregations.AggregateRequest("*").sort_by("@t1", max=1) + res = await decoded_r.ft().aggregate(req) + assert len(res.rows) == 1 + + # test limit + req = aggregations.AggregateRequest("*").sort_by("@t1").limit(1, 1) + res = await decoded_r.ft().aggregate(req) + assert len(res.rows) == 1 + assert res.rows[0] == ["t1", "b"] + else: + # test sort_by using SortDirection + req = aggregations.AggregateRequest("*").sort_by( + aggregations.Asc("@t2"), aggregations.Desc("@t1") + ) + res = (await decoded_r.ft().aggregate(req))["results"] + assert res[0]["extra_attributes"] == {"t2": "a", "t1": "b"} + assert res[1]["extra_attributes"] == {"t2": "b", "t1": "a"} + + # test sort_by without SortDirection + req = aggregations.AggregateRequest("*").sort_by("@t1") + res = (await decoded_r.ft().aggregate(req))["results"] + assert res[0]["extra_attributes"] == {"t1": "a"} + assert res[1]["extra_attributes"] == {"t1": "b"} + + # test sort_by with max + req = aggregations.AggregateRequest("*").sort_by("@t1", max=1) + res = await decoded_r.ft().aggregate(req) + assert len(res["results"]) == 1 + + # test limit + req = aggregations.AggregateRequest("*").sort_by("@t1").limit(1, 1) + res = await decoded_r.ft().aggregate(req) + assert len(res["results"]) == 1 + assert res["results"][0]["extra_attributes"] == {"t1": "b"} + + +@pytest.mark.redismod +@pytest.mark.experimental +async def test_withsuffixtrie(decoded_r: redis.Redis): + # create index + assert await decoded_r.ft().create_index((TextField("txt"),)) + await waitForIndex(decoded_r, getattr(decoded_r.ft(), "index_name", "idx")) + if is_resp2_connection(decoded_r): + info = await decoded_r.ft().info() + assert "WITHSUFFIXTRIE" not in info["attributes"][0] + assert await decoded_r.ft().dropindex() + + # create withsuffixtrie index (text field) + assert await decoded_r.ft().create_index(TextField("t", withsuffixtrie=True)) + await waitForIndex(decoded_r, getattr(decoded_r.ft(), "index_name", "idx")) + info = await decoded_r.ft().info() + assert "WITHSUFFIXTRIE" in info["attributes"][0] + assert await decoded_r.ft().dropindex() + + # create withsuffixtrie index (tag field) + assert await decoded_r.ft().create_index(TagField("t", withsuffixtrie=True)) + await waitForIndex(decoded_r, getattr(decoded_r.ft(), "index_name", "idx")) + info = await decoded_r.ft().info() + assert "WITHSUFFIXTRIE" in info["attributes"][0] + else: + info = await decoded_r.ft().info() + assert "WITHSUFFIXTRIE" not in info["attributes"][0]["flags"] + assert await decoded_r.ft().dropindex() + + # create withsuffixtrie index (text fields) + assert await decoded_r.ft().create_index(TextField("t", withsuffixtrie=True)) + await waitForIndex(decoded_r, getattr(decoded_r.ft(), "index_name", "idx")) + info = await decoded_r.ft().info() + assert "WITHSUFFIXTRIE" in info["attributes"][0]["flags"] + assert await decoded_r.ft().dropindex() + + # create withsuffixtrie index (tag field) + assert await decoded_r.ft().create_index(TagField("t", withsuffixtrie=True)) + await waitForIndex(decoded_r, getattr(decoded_r.ft(), "index_name", "idx")) + info = await decoded_r.ft().info() + assert "WITHSUFFIXTRIE" in info["attributes"][0]["flags"] + + +@pytest.mark.redismod +@skip_ifmodversion_lt("2.10.05", "search") +async def test_aggregations_add_scores(decoded_r: redis.Redis): + assert await decoded_r.ft().create_index( + ( + TextField("name", sortable=True, weight=5.0), + NumericField("age", sortable=True), + ) + ) + + assert await decoded_r.hset("doc1", mapping={"name": "bar", "age": "25"}) + assert await decoded_r.hset("doc2", mapping={"name": "foo", "age": "19"}) + + req = aggregations.AggregateRequest("*").add_scores() + res = await decoded_r.ft().aggregate(req) + + if isinstance(res, dict): + assert len(res["results"]) == 2 + assert res["results"][0]["extra_attributes"] == {"__score": "0.2"} + assert res["results"][1]["extra_attributes"] == {"__score": "0.2"} + else: + assert len(res.rows) == 2 + assert res.rows[0] == ["__score", "0.2"] + assert res.rows[1] == ["__score", "0.2"] + + +@pytest.mark.redismod +@skip_ifmodversion_lt("2.10.05", "search") +async def test_aggregations_hybrid_scoring(decoded_r: redis.Redis): + assert await decoded_r.ft().create_index( + ( + TextField("name", sortable=True, weight=5.0), + TextField("description", sortable=True, weight=5.0), + VectorField( + "vector", + "HNSW", + {"TYPE": "FLOAT32", "DIM": 2, "DISTANCE_METRIC": "COSINE"}, + ), + ) + ) + + assert await decoded_r.hset( + "doc1", + mapping={ + "name": "cat book", + "description": "an animal book about cats", + "vector": np.array([0.1, 0.2]).astype(np.float32).tobytes(), + }, + ) + assert await decoded_r.hset( + "doc2", + mapping={ + "name": "dog book", + "description": "an animal book about dogs", + "vector": np.array([0.2, 0.1]).astype(np.float32).tobytes(), + }, + ) + + query_string = "(@description:animal)=>[KNN 3 @vector $vec_param AS dist]" + req = ( + aggregations.AggregateRequest(query_string) + .scorer("BM25") + .add_scores() + .apply(hybrid_score="@__score + @dist") + .load("*") + .dialect(4) + ) + + res = await decoded_r.ft().aggregate( + req, + query_params={"vec_param": np.array([0.11, 0.22]).astype(np.float32).tobytes()}, + ) + + if isinstance(res, dict): + assert len(res["results"]) == 2 + else: + assert len(res.rows) == 2 + for row in res.rows: + len(row) == 6 + + +@pytest.mark.redismod +@skip_if_redis_enterprise() +async def test_search_commands_in_pipeline(decoded_r: redis.Redis): + p = await decoded_r.ft().pipeline() + p.create_index((TextField("txt"),)) + p.hset("doc1", mapping={"txt": "foo bar"}) + p.hset("doc2", mapping={"txt": "foo bar"}) + q = Query("foo bar").with_payloads() + await p.search(q) + res = await p.execute() + if is_resp2_connection(decoded_r): + assert res[:3] == ["OK", True, True] + assert 2 == res[3][0] + assert "doc1" == res[3][1] + assert "doc2" == res[3][4] + assert res[3][5] is None + assert res[3][3] == res[3][6] == ["txt", "foo bar"] + else: + assert res[:3] == ["OK", True, True] + assert 2 == res[3]["total_results"] + assert "doc1" == res[3]["results"][0]["id"] + assert "doc2" == res[3]["results"][1]["id"] + assert res[3]["results"][0]["payload"] is None + assert ( + res[3]["results"][0]["extra_attributes"] + == res[3]["results"][1]["extra_attributes"] + == {"txt": "foo bar"} + ) + + +@pytest.mark.redismod +async def test_query_timeout(decoded_r: redis.Redis): + q1 = Query("foo").timeout(5000) + assert q1.get_args() == ["foo", "TIMEOUT", 5000, "DIALECT", 2, "LIMIT", 0, 10] + q2 = Query("foo").timeout("not_a_number") + with pytest.raises(redis.ResponseError): + await decoded_r.ft().search(q2) + + +@pytest.mark.redismod +@skip_if_resp_version(3) +async def test_binary_and_text_fields(decoded_r: redis.Redis): + fake_vec = np.array([0.1, 0.2, 0.3, 0.4], dtype=np.float32) + + index_name = "mixed_index" + mixed_data = {"first_name": "🐍python", "vector_emb": fake_vec.tobytes()} + await decoded_r.hset(f"{index_name}:1", mapping=mixed_data) + + schema = [ + TagField("first_name"), + VectorField( + "embeddings_bio", + algorithm="HNSW", + attributes={ + "TYPE": "FLOAT32", + "DIM": 4, + "DISTANCE_METRIC": "COSINE", + }, + ), + ] + + await decoded_r.ft(index_name).create_index( + fields=schema, + definition=IndexDefinition( + prefix=[f"{index_name}:"], index_type=IndexType.HASH + ), + ) + await waitForIndex(decoded_r, index_name) + + query = ( + Query("*") + .return_field("vector_emb", decode_field=False) + .return_field("first_name") + ) + result = await decoded_r.ft(index_name).search(query=query, query_params={}) + docs = result.docs + + if len(docs) == 0: + hash_content = await decoded_r.hget(f"{index_name}:1", "first_name") + assert len(docs) > 0, ( + f"Returned search results are empty. Result: {result}; Hash: {hash_content}" + ) + + decoded_vec_from_search_results = np.frombuffer( + docs[0]["vector_emb"], dtype=np.float32 + ) + + assert np.array_equal(decoded_vec_from_search_results, fake_vec), ( + "The vectors are not equal" + ) + + assert docs[0]["first_name"] == mixed_data["first_name"], ( + "The text field is not decoded correctly" + ) diff --git a/tests/test_anyio/test_sentinel.py b/tests/test_anyio/test_sentinel.py new file mode 100644 index 0000000000..3b4c37f906 --- /dev/null +++ b/tests/test_anyio/test_sentinel.py @@ -0,0 +1,289 @@ +import socket +from unittest import mock + +import pytest +import redis.anyio.sentinel +from redis import exceptions +from redis.anyio.sentinel import ( + MasterNotFoundError, + Sentinel, + SentinelConnectionPool, + SlaveNotFoundError, +) + +pytestmark = pytest.mark.anyio + + +@pytest.fixture(scope="module") +def master_ip(master_host): + yield socket.gethostbyname(master_host[0]) + + +class SentinelTestClient: + def __init__(self, cluster, id): + self.cluster = cluster + self.id = id + + async def sentinel_masters(self): + self.cluster.connection_error_if_down(self) + self.cluster.timeout_if_down(self) + return {self.cluster.service_name: self.cluster.master} + + async def sentinel_slaves(self, master_name): + self.cluster.connection_error_if_down(self) + self.cluster.timeout_if_down(self) + if master_name != self.cluster.service_name: + return [] + return self.cluster.slaves + + async def execute_command(self, *args, **kwargs): + # wrapper purely to validate the calls don't explode + from redis.anyio.client import bool_ok + + return bool_ok + + +class SentinelTestCluster: + def __init__(self, service_name="mymaster", ip="127.0.0.1", port=6379): + self.clients = {} + self.master = { + "ip": ip, + "port": port, + "is_master": True, + "is_sdown": False, + "is_odown": False, + "num-other-sentinels": 0, + } + self.service_name = service_name + self.slaves = [] + self.nodes_down = set() + self.nodes_timeout = set() + + def connection_error_if_down(self, node): + if node.id in self.nodes_down: + raise exceptions.ConnectionError + + def timeout_if_down(self, node): + if node.id in self.nodes_timeout: + raise exceptions.TimeoutError + + def client(self, host, port, **kwargs): + return SentinelTestClient(self, (host, port)) + + +@pytest.fixture +async def cluster(master_ip): + cluster = SentinelTestCluster(ip=master_ip) + saved_Redis = redis.anyio.sentinel.Redis + redis.anyio.sentinel.Redis = cluster.client + yield cluster + redis.anyio.sentinel.Redis = saved_Redis + + +@pytest.fixture +def sentinel(request, cluster): + return Sentinel([("foo", 26379), ("bar", 26379)]) + + +@pytest.mark.onlynoncluster +async def test_discover_master(sentinel, master_ip): + address = await sentinel.discover_master("mymaster") + assert address == (master_ip, 6379) + + +@pytest.mark.onlynoncluster +async def test_discover_master_error(sentinel): + with pytest.raises(MasterNotFoundError): + await sentinel.discover_master("xxx") + + +@pytest.mark.onlynoncluster +async def test_discover_master_sentinel_down(cluster, sentinel, master_ip): + # Put first sentinel 'foo' down + cluster.nodes_down.add(("foo", 26379)) + address = await sentinel.discover_master("mymaster") + assert address == (master_ip, 6379) + # 'bar' is now first sentinel + assert sentinel.sentinels[0].id == ("bar", 26379) + + +@pytest.mark.onlynoncluster +async def test_discover_master_sentinel_timeout(cluster, sentinel, master_ip): + # Put first sentinel 'foo' down + cluster.nodes_timeout.add(("foo", 26379)) + address = await sentinel.discover_master("mymaster") + assert address == (master_ip, 6379) + # 'bar' is now first sentinel + assert sentinel.sentinels[0].id == ("bar", 26379) + + +@pytest.mark.onlynoncluster +async def test_master_min_other_sentinels(cluster, master_ip): + sentinel = Sentinel([("foo", 26379)], min_other_sentinels=1) + # min_other_sentinels + with pytest.raises(MasterNotFoundError): + await sentinel.discover_master("mymaster") + cluster.master["num-other-sentinels"] = 2 + address = await sentinel.discover_master("mymaster") + assert address == (master_ip, 6379) + + +@pytest.mark.onlynoncluster +async def test_master_odown(cluster, sentinel): + cluster.master["is_odown"] = True + with pytest.raises(MasterNotFoundError): + await sentinel.discover_master("mymaster") + + +@pytest.mark.onlynoncluster +async def test_master_sdown(cluster, sentinel): + cluster.master["is_sdown"] = True + with pytest.raises(MasterNotFoundError): + await sentinel.discover_master("mymaster") + + +@pytest.mark.onlynoncluster +async def test_discover_slaves(cluster, sentinel): + assert await sentinel.discover_slaves("mymaster") == [] + + cluster.slaves = [ + {"ip": "slave0", "port": 1234, "is_odown": False, "is_sdown": False}, + {"ip": "slave1", "port": 1234, "is_odown": False, "is_sdown": False}, + ] + assert await sentinel.discover_slaves("mymaster") == [ + ("slave0", 1234), + ("slave1", 1234), + ] + + # slave0 -> ODOWN + cluster.slaves[0]["is_odown"] = True + assert await sentinel.discover_slaves("mymaster") == [("slave1", 1234)] + + # slave1 -> SDOWN + cluster.slaves[1]["is_sdown"] = True + assert await sentinel.discover_slaves("mymaster") == [] + + cluster.slaves[0]["is_odown"] = False + cluster.slaves[1]["is_sdown"] = False + + # node0 -> DOWN + cluster.nodes_down.add(("foo", 26379)) + assert await sentinel.discover_slaves("mymaster") == [ + ("slave0", 1234), + ("slave1", 1234), + ] + cluster.nodes_down.clear() + + # node0 -> TIMEOUT + cluster.nodes_timeout.add(("foo", 26379)) + assert await sentinel.discover_slaves("mymaster") == [ + ("slave0", 1234), + ("slave1", 1234), + ] + + +@pytest.mark.onlynoncluster +async def test_master_for(cluster, sentinel, master_ip): + async with sentinel.master_for("mymaster", db=9) as master: + assert await master.ping() + assert master.connection_pool.master_address == (master_ip, 6379) + + # Use internal connection check + async with sentinel.master_for("mymaster", db=9, check_connection=True) as master: + assert await master.ping() + + +@pytest.mark.onlynoncluster +async def test_slave_for(cluster, sentinel): + cluster.slaves = [ + {"ip": "127.0.0.1", "port": 6379, "is_odown": False, "is_sdown": False} + ] + async with sentinel.slave_for("mymaster", db=9) as slave: + assert await slave.ping() + + +@pytest.mark.onlynoncluster +async def test_slave_for_slave_not_found_error(cluster, sentinel): + cluster.master["is_odown"] = True + async with sentinel.slave_for("mymaster", db=9) as slave: + with pytest.raises(SlaveNotFoundError): + await slave.ping() + + +@pytest.mark.onlynoncluster +async def test_slave_round_robin(cluster, sentinel, master_ip): + cluster.slaves = [ + {"ip": "slave0", "port": 6379, "is_odown": False, "is_sdown": False}, + {"ip": "slave1", "port": 6379, "is_odown": False, "is_sdown": False}, + ] + pool = SentinelConnectionPool("mymaster", sentinel) + rotator = pool.rotate_slaves() + assert await rotator.__anext__() in (("slave0", 6379), ("slave1", 6379)) + assert await rotator.__anext__() in (("slave0", 6379), ("slave1", 6379)) + # Fallback to master + assert await rotator.__anext__() == (master_ip, 6379) + with pytest.raises(SlaveNotFoundError): + await rotator.__anext__() + + +@pytest.mark.onlynoncluster +async def test_ckquorum(cluster, sentinel): + assert await sentinel.sentinel_ckquorum("mymaster") + + +@pytest.mark.onlynoncluster +async def test_flushconfig(cluster, sentinel): + assert await sentinel.sentinel_flushconfig() + + +@pytest.mark.onlynoncluster +async def test_reset(cluster, sentinel): + cluster.master["is_odown"] = True + assert await sentinel.sentinel_reset("mymaster") + + +@pytest.mark.onlynoncluster +@pytest.mark.parametrize("method_name", ["master_for", "slave_for"]) +async def test_auto_close_pool(cluster, sentinel, method_name): + """ + Check that the connection pool created by the sentinel client is + automatically closed + """ + + method = getattr(sentinel, method_name) + client = method("mymaster", db=9) + pool = client.connection_pool + assert client.auto_close_connection_pool is True + calls = 0 + + async def mock_disconnect(): + nonlocal calls + calls += 1 + + with mock.patch.object(pool, "disconnect", mock_disconnect): + await client.aclose() + + assert calls == 1 + await pool.disconnect() + + +@pytest.mark.onlynoncluster +async def test_repr_correctly_represents_connection_object(sentinel): + pool = SentinelConnectionPool("mymaster", sentinel) + connection = await pool.get_connection() + try: + assert ( + str(connection) + == "" # noqa: E501 + ) + assert connection.connection_pool == pool + await pool.release(connection) + + del pool + + assert ( + str(connection) + == "" # noqa: E501 + ) + finally: + await connection.disconnect() diff --git a/tests/test_anyio/test_sentinel_managed_connection.py b/tests/test_anyio/test_sentinel_managed_connection.py new file mode 100644 index 0000000000..fdd35de0c2 --- /dev/null +++ b/tests/test_anyio/test_sentinel_managed_connection.py @@ -0,0 +1,36 @@ +import socket +from unittest.mock import AsyncMock + +import pytest +from redis.anyio.retry import Retry +from redis.anyio.sentinel import SentinelManagedConnection +from redis.backoff import NoBackoff + +pytestmark = pytest.mark.anyio + + +async def test_connect_retry_on_timeout_error(connect_args): + """Test that the _connect function is retried in case of a timeout""" + connection_pool = AsyncMock() + connection_pool.get_master_address = AsyncMock( + return_value=(connect_args["host"], connect_args["port"]) + ) + conn = SentinelManagedConnection( + retry_on_timeout=True, + retry=Retry(NoBackoff(), 3), + connection_pool=connection_pool, + ) + origin_connect = conn._connect + conn._connect = AsyncMock() + + async def mock_connect(): + # connect only on the last retry + if conn._connect.call_count <= 2: + raise socket.timeout + else: + return await origin_connect() + + conn._connect.side_effect = mock_connect + await conn.connect() + assert conn._connect.call_count == 3 + await conn.disconnect() diff --git a/tests/test_anyio/test_ssl.py b/tests/test_anyio/test_ssl.py new file mode 100644 index 0000000000..699ed9b5b1 --- /dev/null +++ b/tests/test_anyio/test_ssl.py @@ -0,0 +1,57 @@ +from urllib.parse import urlparse +import pytest +import redis.anyio as redis + +# Skip test or not based on cryptography installation +try: + import cryptography # noqa + + skip_if_cryptography = pytest.mark.skipif(False, reason="") + skip_if_nocryptography = pytest.mark.skipif(False, reason="") +except ImportError: + skip_if_cryptography = pytest.mark.skipif(True, reason="cryptography not installed") + skip_if_nocryptography = pytest.mark.skipif( + True, reason="cryptography not installed" + ) + +pytestmark = pytest.mark.anyio + + +@pytest.mark.ssl +class TestSSL: + """Tests for SSL connections in asyncio.""" + + @pytest.fixture + async def _get_client(self, request): + ssl_url = request.config.option.redis_ssl_url + p = urlparse(ssl_url)[1].split(":") + client = redis.Redis(host=p[0], port=p[1], ssl=True) + yield client + await client.aclose() + + async def test_ssl_with_invalid_cert(self, _get_client): + """Test SSL connection with invalid certificate.""" + pass + + async def test_cert_reqs_none_with_check_hostname(self, request): + """Test that when ssl_cert_reqs=none is used with ssl_check_hostname=True, + the connection is created successfully with check_hostname internally set to False""" + ssl_url = request.config.option.redis_ssl_url + parsed_url = urlparse(ssl_url) + r = redis.Redis( + host=parsed_url.hostname, + port=parsed_url.port, + ssl=True, + ssl_cert_reqs="none", + # Check that ssl_check_hostname is ignored, when ssl_cert_reqs=none + ssl_check_hostname=True, + ) + try: + # Connection should be successful + assert await r.ping() + # check_hostname should have been automatically set to False + assert r.connection_pool.connection_class == redis.SSLConnection + conn = r.connection_pool.make_connection() + assert conn.check_hostname is False + finally: + await r.aclose() diff --git a/tests/test_anyio/test_timeseries.py b/tests/test_anyio/test_timeseries.py new file mode 100644 index 0000000000..576a95e3d5 --- /dev/null +++ b/tests/test_anyio/test_timeseries.py @@ -0,0 +1,919 @@ +import time +from time import sleep + +import pytest +import redis.anyio as redis +from tests.conftest import ( + assert_resp_response, + is_resp2_connection, + skip_if_server_version_gte, + skip_if_server_version_lt, + skip_ifmodversion_lt, +) + +pytestmark = pytest.mark.anyio + + +@pytest.fixture +async def decoded_r(create_redis, stack_url): + async with create_redis(decode_responses=True, url=stack_url) as client: + yield client + + +@pytest.mark.redismod +async def test_create(decoded_r: redis.Redis): + assert await decoded_r.ts().create(1) + assert await decoded_r.ts().create(2, retention_msecs=5) + assert await decoded_r.ts().create(3, labels={"Redis": "Labs"}) + assert await decoded_r.ts().create(4, retention_msecs=20, labels={"Time": "Series"}) + info = await decoded_r.ts().info(4) + assert_resp_response( + decoded_r, 20, info.get("retention_msecs"), info.get("retentionTime") + ) + assert "Series" == info["labels"]["Time"] + + # Test for a chunk size of 128 Bytes + assert await decoded_r.ts().create("time-serie-1", chunk_size=128) + info = await decoded_r.ts().info("time-serie-1") + assert_resp_response(decoded_r, 128, info.get("chunk_size"), info.get("chunkSize")) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.4.0", "timeseries") +async def test_create_duplicate_policy(decoded_r: redis.Redis): + # Test for duplicate policy + for duplicate_policy in ["block", "last", "first", "min", "max"]: + ts_name = f"time-serie-ooo-{duplicate_policy}" + assert await decoded_r.ts().create(ts_name, duplicate_policy=duplicate_policy) + info = await decoded_r.ts().info(ts_name) + assert_resp_response( + decoded_r, + duplicate_policy, + info.get("duplicate_policy"), + info.get("duplicatePolicy"), + ) + + +@pytest.mark.redismod +async def test_alter(decoded_r: redis.Redis): + assert await decoded_r.ts().create(1) + res = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, 0, res.get("retention_msecs"), res.get("retentionTime") + ) + assert await decoded_r.ts().alter(1, retention_msecs=10) + res = await decoded_r.ts().info(1) + assert {} == (await decoded_r.ts().info(1))["labels"] + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, 10, info.get("retention_msecs"), info.get("retentionTime") + ) + assert await decoded_r.ts().alter(1, labels={"Time": "Series"}) + res = await decoded_r.ts().info(1) + assert "Series" == (await decoded_r.ts().info(1))["labels"]["Time"] + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, 10, info.get("retention_msecs"), info.get("retentionTime") + ) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.4.0", "timeseries") +@skip_if_server_version_lt("7.9.0") +async def test_alter_duplicate_policy(decoded_r: redis.Redis): + assert await decoded_r.ts().create(1) + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, "block", info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + assert await decoded_r.ts().alter(1, duplicate_policy="min") + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, "min", info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.4.0", "timeseries") +@skip_if_server_version_gte("7.9.0") +async def test_alter_duplicate_policy_prior_redis_8(decoded_r: redis.Redis): + assert await decoded_r.ts().create(1) + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, None, info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + assert await decoded_r.ts().alter(1, duplicate_policy="min") + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, "min", info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + + +@pytest.mark.redismod +async def test_add(decoded_r: redis.Redis): + assert 1 == await decoded_r.ts().add(1, 1, 1) + assert 2 == await decoded_r.ts().add(2, 2, 3, retention_msecs=10) + assert 3 == await decoded_r.ts().add(3, 3, 2, labels={"Redis": "Labs"}) + assert 4 == await decoded_r.ts().add( + 4, 4, 2, retention_msecs=10, labels={"Redis": "Labs", "Time": "Series"} + ) + res = await decoded_r.ts().add(5, "*", 1) + assert abs(time.time() - round(float(res) / 1000)) < 1.0 + + info = await decoded_r.ts().info(4) + assert_resp_response( + decoded_r, 10, info.get("retention_msecs"), info.get("retentionTime") + ) + assert "Labs" == info["labels"]["Redis"] + + # Test for a chunk size of 128 Bytes on TS.ADD + assert await decoded_r.ts().add("time-serie-1", 1, 10.0, chunk_size=128) + info = await decoded_r.ts().info("time-serie-1") + assert_resp_response(decoded_r, 128, info.get("chunk_size"), info.get("chunkSize")) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.4.0", "timeseries") +async def test_add_duplicate_policy(decoded_r: redis.Redis): + # Test for duplicate policy BLOCK + assert 1 == await decoded_r.ts().add("time-serie-add-ooo-block", 1, 5.0) + with pytest.raises(Exception): + await decoded_r.ts().add( + "time-serie-add-ooo-block", 1, 5.0, on_duplicate="block" + ) + + # Test for duplicate policy LAST + assert 1 == await decoded_r.ts().add("time-serie-add-ooo-last", 1, 5.0) + assert 1 == await decoded_r.ts().add( + "time-serie-add-ooo-last", 1, 10.0, on_duplicate="last" + ) + res = await decoded_r.ts().get("time-serie-add-ooo-last") + assert 10.0 == res[1] + + # Test for duplicate policy FIRST + assert 1 == await decoded_r.ts().add("time-serie-add-ooo-first", 1, 5.0) + assert 1 == await decoded_r.ts().add( + "time-serie-add-ooo-first", 1, 10.0, on_duplicate="first" + ) + res = await decoded_r.ts().get("time-serie-add-ooo-first") + assert 5.0 == res[1] + + # Test for duplicate policy MAX + assert 1 == await decoded_r.ts().add("time-serie-add-ooo-max", 1, 5.0) + assert 1 == await decoded_r.ts().add( + "time-serie-add-ooo-max", 1, 10.0, on_duplicate="max" + ) + res = await decoded_r.ts().get("time-serie-add-ooo-max") + assert 10.0 == res[1] + + # Test for duplicate policy MIN + assert 1 == await decoded_r.ts().add("time-serie-add-ooo-min", 1, 5.0) + assert 1 == await decoded_r.ts().add( + "time-serie-add-ooo-min", 1, 10.0, on_duplicate="min" + ) + res = await decoded_r.ts().get("time-serie-add-ooo-min") + assert 5.0 == res[1] + + +@pytest.mark.redismod +async def test_madd(decoded_r: redis.Redis): + await decoded_r.ts().create("a") + assert [1, 2, 3] == await decoded_r.ts().madd( + [("a", 1, 5), ("a", 2, 10), ("a", 3, 15)] + ) + + +@pytest.mark.redismod +async def test_incrby_decrby(decoded_r: redis.Redis): + for _ in range(100): + assert await decoded_r.ts().incrby(1, 1) + sleep(0.001) + assert 100 == (await decoded_r.ts().get(1))[1] + for _ in range(100): + assert await decoded_r.ts().decrby(1, 1) + sleep(0.001) + assert 0 == (await decoded_r.ts().get(1))[1] + + assert await decoded_r.ts().incrby(2, 1.5, timestamp=5) + assert_resp_response(decoded_r, await decoded_r.ts().get(2), (5, 1.5), [5, 1.5]) + assert await decoded_r.ts().incrby(2, 2.25, timestamp=7) + assert_resp_response(decoded_r, await decoded_r.ts().get(2), (7, 3.75), [7, 3.75]) + assert await decoded_r.ts().decrby(2, 1.5, timestamp=15) + assert_resp_response(decoded_r, await decoded_r.ts().get(2), (15, 2.25), [15, 2.25]) + + # Test for a chunk size of 128 Bytes on TS.INCRBY + assert await decoded_r.ts().incrby("time-serie-1", 10, chunk_size=128) + info = await decoded_r.ts().info("time-serie-1") + assert_resp_response(decoded_r, 128, info.get("chunk_size"), info.get("chunkSize")) + + # Test for a chunk size of 128 Bytes on TS.DECRBY + assert await decoded_r.ts().decrby("time-serie-2", 10, chunk_size=128) + info = await decoded_r.ts().info("time-serie-2") + assert_resp_response(decoded_r, 128, info.get("chunk_size"), info.get("chunkSize")) + + +@pytest.mark.redismod +async def test_create_and_delete_rule(decoded_r: redis.Redis): + # test rule creation + time = 100 + await decoded_r.ts().create(1) + await decoded_r.ts().create(2) + await decoded_r.ts().createrule(1, 2, "avg", 100) + for i in range(50): + await decoded_r.ts().add(1, time + i * 2, 1) + await decoded_r.ts().add(1, time + i * 2 + 1, 2) + await decoded_r.ts().add(1, time * 2, 1.5) + assert round((await decoded_r.ts().get(2))[1], 5) == 1.5 + info = await decoded_r.ts().info(1) + if is_resp2_connection(decoded_r): + assert info.rules[0][1] == 100 + else: + assert info["rules"]["2"][0] == 100 + + # test rule deletion + await decoded_r.ts().deleterule(1, 2) + info = await decoded_r.ts().info(1) + assert not info["rules"] + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.10.0", "timeseries") +async def test_del_range(decoded_r: redis.Redis): + try: + await decoded_r.ts().delete("test", 0, 100) + except Exception as e: + assert e.__str__() != "" + + for i in range(100): + await decoded_r.ts().add(1, i, i % 7) + assert 22 == await decoded_r.ts().delete(1, 0, 21) + assert [] == await decoded_r.ts().range(1, 0, 21) + assert_resp_response( + decoded_r, await decoded_r.ts().range(1, 22, 22), [(22, 1.0)], [[22, 1.0]] + ) + + +@pytest.mark.redismod +async def test_range(decoded_r: redis.Redis): + for i in range(100): + await decoded_r.ts().add(1, i, i % 7) + assert 100 == len(await decoded_r.ts().range(1, 0, 200)) + for i in range(100): + await decoded_r.ts().add(1, i + 200, i % 7) + assert 200 == len(await decoded_r.ts().range(1, 0, 500)) + # last sample isn't returned + assert 20 == len( + await decoded_r.ts().range( + 1, 0, 500, aggregation_type="avg", bucket_size_msec=10 + ) + ) + assert 10 == len(await decoded_r.ts().range(1, 0, 500, count=10)) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.10.0", "timeseries") +async def test_range_advanced(decoded_r: redis.Redis): + for i in range(100): + await decoded_r.ts().add(1, i, i % 7) + await decoded_r.ts().add(1, i + 200, i % 7) + + assert 2 == len( + await decoded_r.ts().range( + 1, + 0, + 500, + filter_by_ts=[i for i in range(10, 20)], + filter_by_min_value=1, + filter_by_max_value=2, + ) + ) + res = await decoded_r.ts().range( + 1, 0, 10, aggregation_type="count", bucket_size_msec=10, align="+" + ) + assert_resp_response(decoded_r, res, [(0, 10.0), (10, 1.0)], [[0, 10.0], [10, 1.0]]) + res = await decoded_r.ts().range( + 1, 0, 10, aggregation_type="count", bucket_size_msec=10, align=5 + ) + assert_resp_response(decoded_r, res, [(0, 5.0), (5, 6.0)], [[0, 5.0], [5, 6.0]]) + res = await decoded_r.ts().range( + 1, 0, 10, aggregation_type="twa", bucket_size_msec=10 + ) + assert_resp_response(decoded_r, res, [(0, 2.55), (10, 3.0)], [[0, 2.55], [10, 3.0]]) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.10.0", "timeseries") +async def test_rev_range(decoded_r: redis.Redis): + for i in range(100): + await decoded_r.ts().add(1, i, i % 7) + assert 100 == len(await decoded_r.ts().range(1, 0, 200)) + for i in range(100): + await decoded_r.ts().add(1, i + 200, i % 7) + assert 200 == len(await decoded_r.ts().range(1, 0, 500)) + # first sample isn't returned + assert 20 == len( + await decoded_r.ts().revrange( + 1, 0, 500, aggregation_type="avg", bucket_size_msec=10 + ) + ) + assert 10 == len(await decoded_r.ts().revrange(1, 0, 500, count=10)) + assert 2 == len( + await decoded_r.ts().revrange( + 1, + 0, + 500, + filter_by_ts=[i for i in range(10, 20)], + filter_by_min_value=1, + filter_by_max_value=2, + ) + ) + assert_resp_response( + decoded_r, + await decoded_r.ts().revrange( + 1, 0, 10, aggregation_type="count", bucket_size_msec=10, align="+" + ), + [(10, 1.0), (0, 10.0)], + [[10, 1.0], [0, 10.0]], + ) + assert_resp_response( + decoded_r, + await decoded_r.ts().revrange( + 1, 0, 10, aggregation_type="count", bucket_size_msec=10, align=1 + ), + [(1, 10.0), (0, 1.0)], + [[1, 10.0], [0, 1.0]], + ) + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +async def test_multi_range(decoded_r: redis.Redis): + await decoded_r.ts().create(1, labels={"Test": "This", "team": "ny"}) + await decoded_r.ts().create( + 2, labels={"Test": "This", "Taste": "That", "team": "sf"} + ) + for i in range(100): + await decoded_r.ts().add(1, i, i % 7) + await decoded_r.ts().add(2, i, i % 11) + + res = await decoded_r.ts().mrange(0, 200, filters=["Test=This"]) + assert 2 == len(res) + if is_resp2_connection(decoded_r): + assert 100 == len(res[0]["1"][1]) + + res = await decoded_r.ts().mrange(0, 200, filters=["Test=This"], count=10) + assert 10 == len(res[0]["1"][1]) + + for i in range(100): + await decoded_r.ts().add(1, i + 200, i % 7) + res = await decoded_r.ts().mrange( + 0, 500, filters=["Test=This"], aggregation_type="avg", bucket_size_msec=10 + ) + assert 2 == len(res) + assert 20 == len(res[0]["1"][1]) + + # test withlabels + assert {} == res[0]["1"][0] + res = await decoded_r.ts().mrange( + 0, 200, filters=["Test=This"], with_labels=True + ) + assert {"Test": "This", "team": "ny"} == res[0]["1"][0] + else: + assert 100 == len(res["1"][2]) + + res = await decoded_r.ts().mrange(0, 200, filters=["Test=This"], count=10) + assert 10 == len(res["1"][2]) + + for i in range(100): + await decoded_r.ts().add(1, i + 200, i % 7) + res = await decoded_r.ts().mrange( + 0, 500, filters=["Test=This"], aggregation_type="avg", bucket_size_msec=10 + ) + assert 2 == len(res) + assert 20 == len(res["1"][2]) + + # test withlabels + assert {} == res["1"][0] + res = await decoded_r.ts().mrange( + 0, 200, filters=["Test=This"], with_labels=True + ) + assert {"Test": "This", "team": "ny"} == res["1"][0] + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +@skip_ifmodversion_lt("1.10.0", "timeseries") +async def test_multi_range_advanced(decoded_r: redis.Redis): + await decoded_r.ts().create(1, labels={"Test": "This", "team": "ny"}) + await decoded_r.ts().create( + 2, labels={"Test": "This", "Taste": "That", "team": "sf"} + ) + for i in range(100): + await decoded_r.ts().add(1, i, i % 7) + await decoded_r.ts().add(2, i, i % 11) + + # test with selected labels + res = await decoded_r.ts().mrange( + 0, 200, filters=["Test=This"], select_labels=["team"] + ) + if is_resp2_connection(decoded_r): + assert {"team": "ny"} == res[0]["1"][0] + assert {"team": "sf"} == res[1]["2"][0] + + # test with filterby + res = await decoded_r.ts().mrange( + 0, + 200, + filters=["Test=This"], + filter_by_ts=[i for i in range(10, 20)], + filter_by_min_value=1, + filter_by_max_value=2, + ) + assert [(15, 1.0), (16, 2.0)] == res[0]["1"][1] + + # test groupby + res = await decoded_r.ts().mrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="sum" + ) + assert [(0, 0.0), (1, 2.0), (2, 4.0), (3, 6.0)] == res[0]["Test=This"][1] + res = await decoded_r.ts().mrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="max" + ) + assert [(0, 0.0), (1, 1.0), (2, 2.0), (3, 3.0)] == res[0]["Test=This"][1] + res = await decoded_r.ts().mrange( + 0, 3, filters=["Test=This"], groupby="team", reduce="min" + ) + assert 2 == len(res) + assert [(0, 0.0), (1, 1.0), (2, 2.0), (3, 3.0)] == res[0]["team=ny"][1] + assert [(0, 0.0), (1, 1.0), (2, 2.0), (3, 3.0)] == res[1]["team=sf"][1] + + # test align + res = await decoded_r.ts().mrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align="-", + ) + assert [(0, 10.0), (10, 1.0)] == res[0]["1"][1] + res = await decoded_r.ts().mrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align=5, + ) + assert [(0, 5.0), (5, 6.0)] == res[0]["1"][1] + else: + assert {"team": "ny"} == res["1"][0] + assert {"team": "sf"} == res["2"][0] + + # test with filterby + res = await decoded_r.ts().mrange( + 0, + 200, + filters=["Test=This"], + filter_by_ts=[i for i in range(10, 20)], + filter_by_min_value=1, + filter_by_max_value=2, + ) + assert [[15, 1.0], [16, 2.0]] == res["1"][2] + + # test groupby + res = await decoded_r.ts().mrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="sum" + ) + assert [[0, 0.0], [1, 2.0], [2, 4.0], [3, 6.0]] == res["Test=This"][3] + res = await decoded_r.ts().mrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="max" + ) + assert [[0, 0.0], [1, 1.0], [2, 2.0], [3, 3.0]] == res["Test=This"][3] + res = await decoded_r.ts().mrange( + 0, 3, filters=["Test=This"], groupby="team", reduce="min" + ) + assert 2 == len(res) + assert [[0, 0.0], [1, 1.0], [2, 2.0], [3, 3.0]] == res["team=ny"][3] + assert [[0, 0.0], [1, 1.0], [2, 2.0], [3, 3.0]] == res["team=sf"][3] + + # test align + res = await decoded_r.ts().mrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align="-", + ) + assert [[0, 10.0], [10, 1.0]] == res["1"][2] + res = await decoded_r.ts().mrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align=5, + ) + assert [[0, 5.0], [5, 6.0]] == res["1"][2] + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +@skip_ifmodversion_lt("1.10.0", "timeseries") +async def test_multi_reverse_range(decoded_r: redis.Redis): + await decoded_r.ts().create(1, labels={"Test": "This", "team": "ny"}) + await decoded_r.ts().create( + 2, labels={"Test": "This", "Taste": "That", "team": "sf"} + ) + for i in range(100): + await decoded_r.ts().add(1, i, i % 7) + await decoded_r.ts().add(2, i, i % 11) + + res = await decoded_r.ts().mrange(0, 200, filters=["Test=This"]) + assert 2 == len(res) + if is_resp2_connection(decoded_r): + assert 100 == len(res[0]["1"][1]) + + res = await decoded_r.ts().mrange(0, 200, filters=["Test=This"], count=10) + assert 10 == len(res[0]["1"][1]) + + for i in range(100): + await decoded_r.ts().add(1, i + 200, i % 7) + res = await decoded_r.ts().mrevrange( + 0, 500, filters=["Test=This"], aggregation_type="avg", bucket_size_msec=10 + ) + assert 2 == len(res) + assert 20 == len(res[0]["1"][1]) + assert {} == res[0]["1"][0] + + # test withlabels + res = await decoded_r.ts().mrevrange( + 0, 200, filters=["Test=This"], with_labels=True + ) + assert {"Test": "This", "team": "ny"} == res[0]["1"][0] + + # test with selected labels + res = await decoded_r.ts().mrevrange( + 0, 200, filters=["Test=This"], select_labels=["team"] + ) + assert {"team": "ny"} == res[0]["1"][0] + assert {"team": "sf"} == res[1]["2"][0] + + # test filterby + res = await decoded_r.ts().mrevrange( + 0, + 200, + filters=["Test=This"], + filter_by_ts=[i for i in range(10, 20)], + filter_by_min_value=1, + filter_by_max_value=2, + ) + assert [(16, 2.0), (15, 1.0)] == res[0]["1"][1] + + # test groupby + res = await decoded_r.ts().mrevrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="sum" + ) + assert [(3, 6.0), (2, 4.0), (1, 2.0), (0, 0.0)] == res[0]["Test=This"][1] + res = await decoded_r.ts().mrevrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="max" + ) + assert [(3, 3.0), (2, 2.0), (1, 1.0), (0, 0.0)] == res[0]["Test=This"][1] + res = await decoded_r.ts().mrevrange( + 0, 3, filters=["Test=This"], groupby="team", reduce="min" + ) + assert 2 == len(res) + assert [(3, 3.0), (2, 2.0), (1, 1.0), (0, 0.0)] == res[0]["team=ny"][1] + assert [(3, 3.0), (2, 2.0), (1, 1.0), (0, 0.0)] == res[1]["team=sf"][1] + + # test align + res = await decoded_r.ts().mrevrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align="-", + ) + assert [(10, 1.0), (0, 10.0)] == res[0]["1"][1] + res = await decoded_r.ts().mrevrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align=1, + ) + assert [(1, 10.0), (0, 1.0)] == res[0]["1"][1] + else: + assert 100 == len(res["1"][2]) + + res = await decoded_r.ts().mrange(0, 200, filters=["Test=This"], count=10) + assert 10 == len(res["1"][2]) + + for i in range(100): + await decoded_r.ts().add(1, i + 200, i % 7) + res = await decoded_r.ts().mrevrange( + 0, 500, filters=["Test=This"], aggregation_type="avg", bucket_size_msec=10 + ) + assert 2 == len(res) + assert 20 == len(res["1"][2]) + assert {} == res["1"][0] + + # test withlabels + res = await decoded_r.ts().mrevrange( + 0, 200, filters=["Test=This"], with_labels=True + ) + assert {"Test": "This", "team": "ny"} == res["1"][0] + + # test with selected labels + res = await decoded_r.ts().mrevrange( + 0, 200, filters=["Test=This"], select_labels=["team"] + ) + assert {"team": "ny"} == res["1"][0] + assert {"team": "sf"} == res["2"][0] + + # test filterby + res = await decoded_r.ts().mrevrange( + 0, + 200, + filters=["Test=This"], + filter_by_ts=[i for i in range(10, 20)], + filter_by_min_value=1, + filter_by_max_value=2, + ) + assert [[16, 2.0], [15, 1.0]] == res["1"][2] + + # test groupby + res = await decoded_r.ts().mrevrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="sum" + ) + assert [[3, 6.0], [2, 4.0], [1, 2.0], [0, 0.0]] == res["Test=This"][3] + res = await decoded_r.ts().mrevrange( + 0, 3, filters=["Test=This"], groupby="Test", reduce="max" + ) + assert [[3, 3.0], [2, 2.0], [1, 1.0], [0, 0.0]] == res["Test=This"][3] + res = await decoded_r.ts().mrevrange( + 0, 3, filters=["Test=This"], groupby="team", reduce="min" + ) + assert 2 == len(res) + assert [[3, 3.0], [2, 2.0], [1, 1.0], [0, 0.0]] == res["team=ny"][3] + assert [[3, 3.0], [2, 2.0], [1, 1.0], [0, 0.0]] == res["team=sf"][3] + + # test align + res = await decoded_r.ts().mrevrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align="-", + ) + assert [[10, 1.0], [0, 10.0]] == res["1"][2] + res = await decoded_r.ts().mrevrange( + 0, + 10, + filters=["team=ny"], + aggregation_type="count", + bucket_size_msec=10, + align=1, + ) + assert [[1, 10.0], [0, 1.0]] == res["1"][2] + + +@pytest.mark.redismod +async def test_get(decoded_r: redis.Redis): + name = "test" + await decoded_r.ts().create(name) + assert not await decoded_r.ts().get(name) + await decoded_r.ts().add(name, 2, 3) + assert 2 == (await decoded_r.ts().get(name))[0] + await decoded_r.ts().add(name, 3, 4) + assert 4 == (await decoded_r.ts().get(name))[1] + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +async def test_mget(decoded_r: redis.Redis): + await decoded_r.ts().create(1, labels={"Test": "This"}) + await decoded_r.ts().create(2, labels={"Test": "This", "Taste": "That"}) + act_res = await decoded_r.ts().mget(["Test=This"]) + exp_res = [{"1": [{}, None, None]}, {"2": [{}, None, None]}] + exp_res_resp3 = {"1": [{}, []], "2": [{}, []]} + assert_resp_response(decoded_r, act_res, exp_res, exp_res_resp3) + await decoded_r.ts().add(1, "*", 15) + await decoded_r.ts().add(2, "*", 25) + res = await decoded_r.ts().mget(["Test=This"]) + if is_resp2_connection(decoded_r): + assert 15 == res[0]["1"][2] + assert 25 == res[1]["2"][2] + else: + assert 15 == res["1"][1][1] + assert 25 == res["2"][1][1] + res = await decoded_r.ts().mget(["Taste=That"]) + if is_resp2_connection(decoded_r): + assert 25 == res[0]["2"][2] + else: + assert 25 == res["2"][1][1] + + # test with_labels + if is_resp2_connection(decoded_r): + assert {} == res[0]["2"][0] + else: + assert {} == res["2"][0] + res = await decoded_r.ts().mget(["Taste=That"], with_labels=True) + if is_resp2_connection(decoded_r): + assert {"Taste": "That", "Test": "This"} == res[0]["2"][0] + else: + assert {"Taste": "That", "Test": "This"} == res["2"][0] + + +@pytest.mark.redismod +async def test_info(decoded_r: redis.Redis): + await decoded_r.ts().create( + 1, retention_msecs=5, labels={"currentLabel": "currentData"} + ) + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, 5, info.get("retention_msecs"), info.get("retentionTime") + ) + assert info["labels"]["currentLabel"] == "currentData" + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.4.0", "timeseries") +@skip_if_server_version_lt("7.9.0") +async def test_info_duplicate_policy(decoded_r: redis.Redis): + await decoded_r.ts().create( + 1, retention_msecs=5, labels={"currentLabel": "currentData"} + ) + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, "block", info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + + await decoded_r.ts().create("time-serie-2", duplicate_policy="min") + info = await decoded_r.ts().info("time-serie-2") + assert_resp_response( + decoded_r, "min", info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.4.0", "timeseries") +@skip_if_server_version_gte("7.9.0") +async def test_info_duplicate_policy_prior_redis_8(decoded_r: redis.Redis): + await decoded_r.ts().create( + 1, retention_msecs=5, labels={"currentLabel": "currentData"} + ) + info = await decoded_r.ts().info(1) + assert_resp_response( + decoded_r, None, info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + + await decoded_r.ts().create("time-serie-2", duplicate_policy="min") + info = await decoded_r.ts().info("time-serie-2") + assert_resp_response( + decoded_r, "min", info.get("duplicate_policy"), info.get("duplicatePolicy") + ) + + +@pytest.mark.onlynoncluster +@pytest.mark.redismod +async def test_query_index(decoded_r: redis.Redis): + await decoded_r.ts().create(1, labels={"Test": "This"}) + await decoded_r.ts().create(2, labels={"Test": "This", "Taste": "That"}) + assert 2 == len(await decoded_r.ts().queryindex(["Test=This"])) + assert 1 == len(await decoded_r.ts().queryindex(["Taste=That"])) + assert_resp_response( + decoded_r, await decoded_r.ts().queryindex(["Taste=That"]), [2], ["2"] + ) + + +@pytest.mark.redismod +async def test_uncompressed(decoded_r: redis.Redis): + await decoded_r.ts().create("compressed") + await decoded_r.ts().create("uncompressed", uncompressed=True) + for i in range(1000): + await decoded_r.ts().add("compressed", i, i) + await decoded_r.ts().add("uncompressed", i, i) + compressed_info = await decoded_r.ts().info("compressed") + uncompressed_info = await decoded_r.ts().info("uncompressed") + if is_resp2_connection(decoded_r): + assert compressed_info.memory_usage != uncompressed_info.memory_usage + else: + assert compressed_info["memoryUsage"] != uncompressed_info["memoryUsage"] + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.12.0", "timeseries") +async def test_create_with_insertion_filters(decoded_r: redis.Redis): + await decoded_r.ts().create( + "time-series-1", + duplicate_policy="last", + ignore_max_time_diff=5, + ignore_max_val_diff=10.0, + ) + assert 1000 == await decoded_r.ts().add("time-series-1", 1000, 1.0) + assert 1010 == await decoded_r.ts().add("time-series-1", 1010, 11.0) + assert 1010 == await decoded_r.ts().add("time-series-1", 1013, 10.0) + assert 1020 == await decoded_r.ts().add("time-series-1", 1020, 11.5) + assert 1021 == await decoded_r.ts().add("time-series-1", 1021, 22.0) + + data_points = await decoded_r.ts().range("time-series-1", "-", "+") + assert_resp_response( + decoded_r, + data_points, + [(1000, 1.0), (1010, 11.0), (1020, 11.5), (1021, 22.0)], + [[1000, 1.0], [1010, 11.0], [1020, 11.5], [1021, 22.0]], + ) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.12.0", "timeseries") +async def test_alter_with_insertion_filters(decoded_r: redis.Redis): + assert 1000 == await decoded_r.ts().add("time-series-1", 1000, 1.0) + assert 1010 == await decoded_r.ts().add("time-series-1", 1010, 11.0) + assert 1013 == await decoded_r.ts().add("time-series-1", 1013, 10.0) + + await decoded_r.ts().alter( + "time-series-1", + duplicate_policy="last", + ignore_max_time_diff=5, + ignore_max_val_diff=10.0, + ) + + assert 1013 == await decoded_r.ts().add("time-series-1", 1015, 11.5) + + data_points = await decoded_r.ts().range("time-series-1", "-", "+") + assert_resp_response( + decoded_r, + data_points, + [(1000, 1.0), (1010, 11.0), (1013, 10.0)], + [[1000, 1.0], [1010, 11.0], [1013, 10.0]], + ) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.12.0", "timeseries") +async def test_add_with_insertion_filters(decoded_r: redis.Redis): + assert 1000 == await decoded_r.ts().add( + "time-series-1", + 1000, + 1.0, + duplicate_policy="last", + ignore_max_time_diff=5, + ignore_max_val_diff=10.0, + ) + + assert 1000 == await decoded_r.ts().add("time-series-1", 1004, 3.0) + + data_points = await decoded_r.ts().range("time-series-1", "-", "+") + assert_resp_response(decoded_r, data_points, [(1000, 1.0)], [[1000, 1.0]]) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.12.0", "timeseries") +async def test_incrby_with_insertion_filters(decoded_r: redis.Redis): + assert 1000 == await decoded_r.ts().incrby( + "time-series-1", + 1.0, + timestamp=1000, + duplicate_policy="last", + ignore_max_time_diff=5, + ignore_max_val_diff=10.0, + ) + + assert 1000 == await decoded_r.ts().incrby("time-series-1", 3.0, timestamp=1000) + + data_points = await decoded_r.ts().range("time-series-1", "-", "+") + assert_resp_response(decoded_r, data_points, [(1000, 1.0)], [[1000, 1.0]]) + + assert 1000 == await decoded_r.ts().incrby("time-series-1", 10.1, timestamp=1000) + + data_points = await decoded_r.ts().range("time-series-1", "-", "+") + assert_resp_response(decoded_r, data_points, [(1000, 11.1)], [[1000, 11.1]]) + + +@pytest.mark.redismod +@skip_ifmodversion_lt("1.12.0", "timeseries") +async def test_decrby_with_insertion_filters(decoded_r: redis.Redis): + assert 1000 == await decoded_r.ts().decrby( + "time-series-1", + 1.0, + timestamp=1000, + duplicate_policy="last", + ignore_max_time_diff=5, + ignore_max_val_diff=10.0, + ) + + assert 1000 == await decoded_r.ts().decrby("time-series-1", 3.0, timestamp=1000) + + data_points = await decoded_r.ts().range("time-series-1", "-", "+") + assert_resp_response(decoded_r, data_points, [(1000, -1.0)], [[1000, -1.0]]) + + assert 1000 == await decoded_r.ts().decrby("time-series-1", 10.1, timestamp=1000) + + data_points = await decoded_r.ts().range("time-series-1", "-", "+") + assert_resp_response(decoded_r, data_points, [(1000, -11.1)], [[1000, -11.1]]) diff --git a/tests/test_anyio/test_vsets.py b/tests/test_anyio/test_vsets.py new file mode 100644 index 0000000000..3c1de6c846 --- /dev/null +++ b/tests/test_anyio/test_vsets.py @@ -0,0 +1,872 @@ +import json +import random +import numpy as np +import pytest +import redis +from redis.commands.vectorset.commands import QuantizationOptions + +from tests.conftest import ( + skip_if_server_version_lt, +) + +pytestmark = pytest.mark.anyio + + +@pytest.fixture +async def d_client(create_redis, redis_url): + async with create_redis(url=redis_url, decode_responses=True) as client: + yield client + + +@pytest.fixture +async def client(create_redis, redis_url): + async with create_redis(url=redis_url, decode_responses=False) as _client: + yield _client + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_with_values(d_client): + float_array = [1, 4.32, 0.11] + resp = await d_client.vset().vadd("myset", float_array, "elem1") + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem1") + assert _validate_quantization(float_array, emb, tolerance=0.1) + + with pytest.raises(redis.DataError): + await d_client.vset().vadd("myset_invalid_data", None, "elem1") + + with pytest.raises(redis.DataError): + await d_client.vset().vadd("myset_invalid_data", [12, 45], None, reduce_dim=3) + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_with_vector(d_client): + float_array = [1, 4.32, 0.11] + # Convert the list of floats to a byte array in fp32 format + byte_array = _to_fp32_blob_array(float_array) + resp = await d_client.vset().vadd("myset", byte_array, "elem1") + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem1") + assert _validate_quantization(float_array, emb, tolerance=0.1) + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_reduced_dim(d_client): + float_array = [1, 4.32, 0.11, 0.5, 0.9] + resp = await d_client.vset().vadd("myset", float_array, "elem1", reduce_dim=3) + assert resp == 1 + + dim = await d_client.vset().vdim("myset") + assert dim == 3 + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_cas(d_client): + float_array = [1, 4.32, 0.11, 0.5, 0.9] + resp = await d_client.vset().vadd( + "myset", vector=float_array, element="elem1", cas=True + ) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem1") + assert _validate_quantization(float_array, emb, tolerance=0.1) + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_no_quant(d_client): + float_array = [1, 4.32, 0.11, 0.5, 0.9] + resp = await d_client.vset().vadd( + "myset", + vector=float_array, + element="elem1", + quantization=QuantizationOptions.NOQUANT, + ) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem1") + assert _validate_quantization(float_array, emb, tolerance=0.0) + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_bin_quant(d_client): + float_array = [1, 4.32, 0.0, 0.05, -2.9] + resp = await d_client.vset().vadd( + "myset", + vector=float_array, + element="elem1", + quantization=QuantizationOptions.BIN, + ) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem1") + expected_array = [1, 1, -1, 1, -1] + assert _validate_quantization(expected_array, emb, tolerance=0.0) + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_q8_quant(d_client): + float_array = [1, 4.32, 10.0, -21, -2.9] + resp = await d_client.vset().vadd( + "myset", + vector=float_array, + element="elem1", + quantization=QuantizationOptions.BIN, + ) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem1") + expected_array = [1, 1, 1, -1, -1] + assert _validate_quantization(expected_array, emb, tolerance=0.0) + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_ef(d_client): + await d_client.vset().vadd("myset", vector=[5, 55, 65, -20, 30], element="elem1") + await d_client.vset().vadd( + "myset", vector=[-40, -40.32, 10.0, -4, 2.9], element="elem2" + ) + + float_array = [1, 4.32, 10.0, -21, -2.9] + resp = await d_client.vset().vadd("myset", float_array, "elem3", ef=1) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem3") + assert _validate_quantization(float_array, emb, tolerance=0.1) + + sim = await d_client.vset().vsim("myset", input="elem3", with_scores=True) + assert len(sim) == 3 + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_with_attr(d_client): + float_array = [1, 4.32, 10.0, -21, -2.9] + attrs_dict = {"key1": "value1", "key2": "value2"} + resp = await d_client.vset().vadd( + "myset", + vector=float_array, + element="elem3", + attributes=attrs_dict, + ) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem3") + assert _validate_quantization(float_array, emb, tolerance=0.1) + + attr_saved = await d_client.vset().vgetattr("myset", "elem3") + assert attr_saved == attrs_dict + + resp = await d_client.vset().vadd( + "myset", + vector=float_array, + element="elem4", + attributes={}, + ) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem4") + assert _validate_quantization(float_array, emb, tolerance=0.1) + + attr_saved = await d_client.vset().vgetattr("myset", "elem4") + assert attr_saved is None + + resp = await d_client.vset().vadd( + "myset", + vector=float_array, + element="elem5", + attributes=json.dumps(attrs_dict), + ) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem5") + assert _validate_quantization(float_array, emb, tolerance=0.1) + + attr_saved = await d_client.vset().vgetattr("myset", "elem5") + assert attr_saved == attrs_dict + + +@skip_if_server_version_lt("7.9.0") +async def test_add_elem_with_numlinks(d_client): + elements_count = 100 + vector_dim = 10 + for i in range(elements_count): + float_array = [random.randint(0, 10) for x in range(vector_dim)] + await d_client.vset().vadd( + "myset", + float_array, + f"elem{i}", + numlinks=8, + ) + + float_array = [1, 4.32, 0.11, 0.5, 0.9, 0.1, 0.2, 0.3, 0.4, 0.5] + resp = await d_client.vset().vadd("myset", float_array, "elem_numlinks", numlinks=8) + assert resp == 1 + + emb = await d_client.vset().vemb("myset", "elem_numlinks") + assert _validate_quantization(float_array, emb, tolerance=0.5) + + numlinks_all_layers = await d_client.vset().vlinks("myset", "elem_numlinks") + for neighbours_list_for_layer in numlinks_all_layers: + assert len(neighbours_list_for_layer) <= 8 + + +@skip_if_server_version_lt("7.9.0") +async def test_vsim_count(d_client): + elements_count = 30 + vector_dim = 800 + for i in range(elements_count): + float_array = [random.uniform(0, 10) for x in range(vector_dim)] + await d_client.vset().vadd( + "myset", + float_array, + f"elem{i}", + numlinks=64, + ) + + vsim = await d_client.vset().vsim("myset", input="elem1") + assert len(vsim) == 10 + assert isinstance(vsim, list) + assert isinstance(vsim[0], str) + + vsim = await d_client.vset().vsim("myset", input="elem1", count=5) + assert len(vsim) == 5 + assert isinstance(vsim, list) + assert isinstance(vsim[0], str) + + vsim = await d_client.vset().vsim("myset", input="elem1", count=50) + assert len(vsim) == 30 + assert isinstance(vsim, list) + assert isinstance(vsim[0], str) + + vsim = await d_client.vset().vsim("myset", input="elem1", count=15) + assert len(vsim) == 15 + assert isinstance(vsim, list) + assert isinstance(vsim[0], str) + + +@skip_if_server_version_lt("7.9.0") +async def test_vsim_with_scores(d_client): + elements_count = 20 + vector_dim = 50 + for i in range(elements_count): + float_array = [random.uniform(0, 10) for x in range(vector_dim)] + await d_client.vset().vadd( + "myset", + float_array, + f"elem{i}", + numlinks=64, + ) + + vsim = await d_client.vset().vsim("myset", input="elem1", with_scores=True) + assert len(vsim) == 10 + assert isinstance(vsim, dict) + assert isinstance(vsim["elem1"], float) + assert 0 <= vsim["elem1"] <= 1 + + +@skip_if_server_version_lt("7.9.0") +async def test_vsim_with_different_vector_input_types(d_client): + elements_count = 10 + vector_dim = 5 + for i in range(elements_count): + float_array = [random.uniform(0, 10) for x in range(vector_dim)] + attributes = {"index": i, "elem_name": f"elem_{i}"} + await d_client.vset().vadd( + "myset", + float_array, + f"elem_{i}", + numlinks=4, + attributes=attributes, + ) + sim = await d_client.vset().vsim("myset", input="elem_1") + assert len(sim) == 10 + assert isinstance(sim, list) + + float_array = [1, 4.32, 0.0, 0.05, -2.9] + sim_to_float_array = await d_client.vset().vsim("myset", input=float_array) + assert len(sim_to_float_array) == 10 + assert isinstance(sim_to_float_array, list) + + fp32_vector = _to_fp32_blob_array(float_array) + sim_to_fp32_vector = await d_client.vset().vsim("myset", input=fp32_vector) + assert len(sim_to_fp32_vector) == 10 + assert isinstance(sim_to_fp32_vector, list) + assert sim_to_float_array == sim_to_fp32_vector + + with pytest.raises(redis.DataError): + await d_client.vset().vsim("myset", input=None) + + +@skip_if_server_version_lt("7.9.0") +async def test_vsim_unexisting(d_client): + float_array = [1, 4.32, 0.11, 0.5, 0.9] + await d_client.vset().vadd("myset", vector=float_array, element="elem1", cas=True) + + with pytest.raises(redis.ResponseError): + await d_client.vset().vsim("myset", input="elem_not_existing") + + sim = await d_client.vset().vsim("myset_not_existing", input="elem1") + assert sim == [] + + +@skip_if_server_version_lt("7.9.0") +async def test_vsim_with_filter(d_client): + elements_count = 50 + vector_dim = 800 + for i in range(elements_count): + float_array = [random.uniform(0, 10) for x in range(vector_dim)] + attributes = {"index": i, "elem_name": f"elem_{i}"} + await d_client.vset().vadd( + "myset", + float_array, + f"elem_{i}", + numlinks=4, + attributes=attributes, + ) + float_array = [-random.uniform(10, 20) for x in range(vector_dim)] + attributes = {"index": elements_count, "elem_name": "elem_special"} + await d_client.vset().vadd( + "myset", + float_array, + "elem_special", + numlinks=4, + attributes=attributes, + ) + sim = await d_client.vset().vsim("myset", input="elem_1", filter=".index > 10") + assert len(sim) == 10 + assert isinstance(sim, list) + for elem in sim: + assert int(elem.split("_")[1]) > 10 + + sim = await d_client.vset().vsim( + "myset", + input="elem_1", + filter=".index > 10 and .index < 15 and .elem_name in ['elem_12', 'elem_17']", + ) + assert len(sim) == 1 + assert isinstance(sim, list) + assert sim[0] == "elem_12" + + sim = await d_client.vset().vsim( + "myset", + input="elem_1", + filter=".index > 25 and .elem_name in ['elem_12', 'elem_17', 'elem_19']", + ef=100, + ) + assert len(sim) == 0 + assert isinstance(sim, list) + + sim = await d_client.vset().vsim( + "myset", + input="elem_1", + filter=".index > 28 and .elem_name in ['elem_12', 'elem_17', 'elem_special']", + filter_ef=1, + ) + assert len(sim) == 0, ( + f"Expected 0 results, but got {len(sim)} with filter_ef=1, sim: {sim}" + ) + assert isinstance(sim, list) + + sim = await d_client.vset().vsim( + "myset", + input="elem_1", + filter=".index > 28 and .elem_name in ['elem_12', 'elem_17', 'elem_special']", + filter_ef=500, + ) + assert len(sim) == 1 + assert isinstance(sim, list) + + +@skip_if_server_version_lt("7.9.0") +async def test_vsim_truth_no_thread_enabled(d_client): + elements_count = 5000 + vector_dim = 50 + for i in range(1, elements_count + 1): + float_array = [random.uniform(10 * i, 1000 * i) for x in range(vector_dim)] + await d_client.vset().vadd("myset", float_array, f"elem_{i}") + + await d_client.vset().vadd("myset", [-22 for _ in range(vector_dim)], "elem_man_2") + + sim_without_truth = await d_client.vset().vsim( + "myset", input="elem_man_2", with_scores=True + ) + sim_truth = await d_client.vset().vsim( + "myset", input="elem_man_2", with_scores=True, truth=True + ) + + assert len(sim_without_truth) == 10 + assert len(sim_truth) == 10 + + assert isinstance(sim_without_truth, dict) + assert isinstance(sim_truth, dict) + + results_scores = list( + zip( + [v for _, v in sim_truth.items()], [v for _, v in sim_without_truth.items()] + ) + ) + + found_better_match = False + for score_with_truth, score_without_truth in results_scores: + if score_with_truth < score_without_truth: + assert False, ( + "Score with truth [{score_with_truth}] < score without truth [{score_without_truth}]" + ) + elif score_with_truth > score_without_truth: + found_better_match = True + + assert found_better_match + + sim_no_thread = await d_client.vset().vsim( + "myset", input="elem_man_2", with_scores=True, no_thread=True + ) + + assert len(sim_no_thread) == 10 + assert isinstance(sim_no_thread, dict) + + +@skip_if_server_version_lt("7.9.0") +async def test_vdim(d_client): + float_array = [1, 4.32, 0.11, 0.5, 0.9, 0.1, 0.2] + await d_client.vset().vadd("myset", float_array, "elem1") + + dim = await d_client.vset().vdim("myset") + assert dim == len(float_array) + + await d_client.vset().vadd("myset_reduced", float_array, "elem1", reduce_dim=4) + reduced_dim = await d_client.vset().vdim("myset_reduced") + assert reduced_dim == 4 + + with pytest.raises(redis.ResponseError): + await d_client.vset().vdim("myset_unexisting") + + +@skip_if_server_version_lt("7.9.0") +async def test_vcard(d_client): + n = 20 + for i in range(n): + float_array = [random.uniform(0, 10) for x in range(1, 8)] + await d_client.vset().vadd("myset", float_array, f"elem{i}") + + card = await d_client.vset().vcard("myset") + assert card == n + + with pytest.raises(redis.ResponseError): + await d_client.vset().vdim("myset_unexisting") + + +@skip_if_server_version_lt("7.9.0") +async def test_vrem(d_client): + n = 3 + for i in range(n): + float_array = [random.uniform(0, 10) for x in range(1, 8)] + await d_client.vset().vadd("myset", float_array, f"elem{i}") + + resp = await d_client.vset().vrem("myset", "elem2") + assert resp == 1 + + card = await d_client.vset().vcard("myset") + assert card == n - 1 + + resp = await d_client.vset().vrem("myset", "elem2") + assert resp == 0 + + card = await d_client.vset().vcard("myset") + assert card == n - 1 + + resp = await d_client.vset().vrem("myset_unexisting", "elem1") + assert resp == 0 + + +@skip_if_server_version_lt("7.9.0") +async def test_vemb_bin_quantization(d_client): + e = [1, 4.32, 0.0, 0.05, -2.9] + await d_client.vset().vadd( + "myset", + e, + "elem", + quantization=QuantizationOptions.BIN, + ) + emb_no_quant = await d_client.vset().vemb("myset", "elem") + assert emb_no_quant == [1, 1, -1, 1, -1] + + emb_no_quant_raw = await d_client.vset().vemb("myset", "elem", raw=True) + assert emb_no_quant_raw["quantization"] == "bin" + assert isinstance(emb_no_quant_raw["raw"], bytes) + assert isinstance(emb_no_quant_raw["l2"], float) + assert "range" not in emb_no_quant_raw + + +@skip_if_server_version_lt("7.9.0") +async def test_vemb_q8_quantization(d_client): + e = [1, 10.32, 0.0, 2.05, -12.5] + await d_client.vset().vadd("myset", e, "elem", quantization=QuantizationOptions.Q8) + + emb_q8_quant = await d_client.vset().vemb("myset", "elem") + assert _validate_quantization(e, emb_q8_quant, tolerance=0.1) + + emb_q8_quant_raw = await d_client.vset().vemb("myset", "elem", raw=True) + assert emb_q8_quant_raw["quantization"] == "int8" + assert isinstance(emb_q8_quant_raw["raw"], bytes) + assert isinstance(emb_q8_quant_raw["l2"], float) + assert isinstance(emb_q8_quant_raw["range"], float) + + +@skip_if_server_version_lt("7.9.0") +async def test_vemb_no_quantization(d_client): + e = [1, 10.32, 0.0, 2.05, -12.5] + await d_client.vset().vadd( + "myset", e, "elem", quantization=QuantizationOptions.NOQUANT + ) + + emb_no_quant = await d_client.vset().vemb("myset", "elem") + assert _validate_quantization(e, emb_no_quant, tolerance=0.1) + + emb_no_quant_raw = await d_client.vset().vemb("myset", "elem", raw=True) + assert emb_no_quant_raw["quantization"] == "f32" + assert isinstance(emb_no_quant_raw["raw"], bytes) + assert isinstance(emb_no_quant_raw["l2"], float) + assert "range" not in emb_no_quant_raw + + +@skip_if_server_version_lt("7.9.0") +async def test_vemb_default_quantization(d_client): + e = [1, 5.32, 0.0, 0.25, -5] + await d_client.vset().vadd("myset", vector=e, element="elem") + + emb_default_quant = await d_client.vset().vemb("myset", "elem") + assert _validate_quantization(e, emb_default_quant, tolerance=0.1) + + emb_default_quant_raw = await d_client.vset().vemb("myset", "elem", raw=True) + assert emb_default_quant_raw["quantization"] == "int8" + assert isinstance(emb_default_quant_raw["raw"], bytes) + assert isinstance(emb_default_quant_raw["l2"], float) + assert isinstance(emb_default_quant_raw["range"], float) + + +@skip_if_server_version_lt("7.9.0") +async def test_vemb_fp32_quantization(d_client): + float_array_fp32 = [1, 4.32, 0.11] + # Convert the list of floats to a byte array in fp32 format + byte_array = _to_fp32_blob_array(float_array_fp32) + await d_client.vset().vadd("myset", byte_array, "elem") + + emb_fp32_quant = await d_client.vset().vemb("myset", "elem") + assert _validate_quantization(float_array_fp32, emb_fp32_quant, tolerance=0.1) + + emb_fp32_quant_raw = await d_client.vset().vemb("myset", "elem", raw=True) + assert emb_fp32_quant_raw["quantization"] == "int8" + assert isinstance(emb_fp32_quant_raw["raw"], bytes) + assert isinstance(emb_fp32_quant_raw["l2"], float) + assert isinstance(emb_fp32_quant_raw["range"], float) + + +@skip_if_server_version_lt("7.9.0") +async def test_vemb_unexisting(d_client): + emb_not_existing = await d_client.vset().vemb("not_existing", "elem") + assert emb_not_existing is None + + e = [1, 5.32, 0.0, 0.25, -5] + await d_client.vset().vadd("myset", vector=e, element="elem") + emb_elem_not_existing = await d_client.vset().vemb("myset", "not_existing") + assert emb_elem_not_existing is None + + +@skip_if_server_version_lt("7.9.0") +async def test_vlinks(d_client): + elements_count = 100 + vector_dim = 800 + for i in range(elements_count): + float_array = [random.uniform(0, 10) for x in range(vector_dim)] + await d_client.vset().vadd( + "myset", + float_array, + f"elem{i}", + numlinks=8, + ) + + element_links_all_layers = await d_client.vset().vlinks("myset", "elem1") + assert len(element_links_all_layers) >= 1 + for neighbours_list_for_layer in element_links_all_layers: + assert isinstance(neighbours_list_for_layer, list) + for neighbour in neighbours_list_for_layer: + assert isinstance(neighbour, str) + + elem_links_all_layers_with_scores = await d_client.vset().vlinks( + "myset", "elem1", with_scores=True + ) + assert len(elem_links_all_layers_with_scores) >= 1 + for neighbours_dict_for_layer in elem_links_all_layers_with_scores: + assert isinstance(neighbours_dict_for_layer, dict) + for neighbour_key, score_value in neighbours_dict_for_layer.items(): + assert isinstance(neighbour_key, str) + assert isinstance(score_value, float) + + float_array = [0.75, 0.25, 0.5, 0.1, 0.9] + await d_client.vset().vadd("myset_one_elem_only", float_array, "elem1") + elem_no_neighbours_with_scores = await d_client.vset().vlinks( + "myset_one_elem_only", "elem1", with_scores=True + ) + assert len(elem_no_neighbours_with_scores) >= 1 + for neighbours_dict_for_layer in elem_no_neighbours_with_scores: + assert isinstance(neighbours_dict_for_layer, dict) + assert len(neighbours_dict_for_layer) == 0 + + elem_no_neighbours_no_scores = await d_client.vset().vlinks( + "myset_one_elem_only", "elem1" + ) + assert len(elem_no_neighbours_no_scores) >= 1 + for neighbours_list_for_layer in elem_no_neighbours_no_scores: + assert isinstance(neighbours_list_for_layer, list) + assert len(neighbours_list_for_layer) == 0 + + unexisting_element_links = await d_client.vset().vlinks("myset", "unexisting_elem") + assert unexisting_element_links is None + + unexisting_vset_links = await d_client.vset().vlinks("myset_unexisting", "elem1") + assert unexisting_vset_links is None + + unexisting_element_links = await d_client.vset().vlinks( + "myset", "unexisting_elem", with_scores=True + ) + assert unexisting_element_links is None + + unexisting_vset_links = await d_client.vset().vlinks( + "myset_unexisting", "elem1", with_scores=True + ) + assert unexisting_vset_links is None + + +@skip_if_server_version_lt("7.9.0") +async def test_vinfo(d_client): + elements_count = 100 + vector_dim = 800 + for i in range(elements_count): + float_array = [random.uniform(0, 10) for x in range(vector_dim)] + await d_client.vset().vadd( + "myset", + float_array, + f"elem{i}", + numlinks=8, + quantization=QuantizationOptions.BIN, + ) + + vset_info = await d_client.vset().vinfo("myset") + assert vset_info["quant-type"] == "bin" + assert vset_info["vector-dim"] == vector_dim + assert vset_info["size"] == elements_count + assert vset_info["max-level"] > 0 + assert vset_info["hnsw-max-node-uid"] == elements_count + + unexisting_vset_info = await d_client.vset().vinfo("myset_unexisting") + assert unexisting_vset_info is None + + +@skip_if_server_version_lt("7.9.0") +async def test_vset_vget_attributes(d_client): + float_array = [1, 4.32, 0.11] + attributes = {"key1": "value1", "key2": "value2"} + + # validate vgetattrs when no attributes are set with vadd + resp = await d_client.vset().vadd("myset", float_array, "elem1") + assert resp == 1 + + attrs = await d_client.vset().vgetattr("myset", "elem1") + assert attrs is None + + # validate vgetattrs when attributes are set with vadd + resp = await d_client.vset().vadd( + "myset_with_attrs", float_array, "elem1", attributes=attributes + ) + assert resp == 1 + + attrs = await d_client.vset().vgetattr("myset_with_attrs", "elem1") + assert attrs == attributes + + # Set attributes and get attributes + resp = await d_client.vset().vsetattr("myset", "elem1", attributes) + assert resp == 1 + attr_saved = await d_client.vset().vgetattr("myset", "elem1") + assert attr_saved == attributes + + # Set attributes to None + resp = await d_client.vset().vsetattr("myset", "elem1", None) + assert resp == 1 + attr_saved = await d_client.vset().vgetattr("myset", "elem1") + assert attr_saved is None + + # Set attributes to empty dict + resp = await d_client.vset().vsetattr("myset", "elem1", {}) + assert resp == 1 + attr_saved = await d_client.vset().vgetattr("myset", "elem1") + assert attr_saved is None + + # Set attributes provided as string + resp = await d_client.vset().vsetattr("myset", "elem1", json.dumps(attributes)) + assert resp == 1 + attr_saved = await d_client.vset().vgetattr("myset", "elem1") + assert attr_saved == attributes + + # Set attributes to unexisting element + resp = await d_client.vset().vsetattr("myset", "elem2", attributes) + assert resp == 0 + attr_saved = await d_client.vset().vgetattr("myset", "elem2") + assert attr_saved is None + + # Set attributes to unexisting vset + resp = await d_client.vset().vsetattr("myset_unexisting", "elem1", attributes) + assert resp == 0 + attr_saved = await d_client.vset().vgetattr("myset_unexisting", "elem1") + assert attr_saved is None + + +@skip_if_server_version_lt("7.9.0") +async def test_vrandmember(d_client): + elements = ["elem1", "elem2", "elem3"] + for elem in elements: + float_array = [random.uniform(0, 10) for x in range(1, 8)] + await d_client.vset().vadd("myset", float_array, element=elem) + + random_member = await d_client.vset().vrandmember("myset") + assert random_member in elements + + members_list = await d_client.vset().vrandmember("myset", count=2) + assert len(members_list) == 2 + assert all(member in elements for member in members_list) + + # Test with count greater than the number of elements + members_list = await d_client.vset().vrandmember("myset", count=10) + assert len(members_list) == len(elements) + assert all(member in elements for member in members_list) + + # Test with negative count + members_list = await d_client.vset().vrandmember("myset", count=-2) + assert len(members_list) == 2 + assert all(member in elements for member in members_list) + + # Test with count equal to the number of elements + members_list = await d_client.vset().vrandmember("myset", count=len(elements)) + assert len(members_list) == len(elements) + assert all(member in elements for member in members_list) + + # Test with count equal to 0 + members_list = await d_client.vset().vrandmember("myset", count=0) + assert members_list == [] + + # Test with count equal to 1 + members_list = await d_client.vset().vrandmember("myset", count=1) + assert len(members_list) == 1 + assert members_list[0] in elements + + # Test with count equal to -1 + members_list = await d_client.vset().vrandmember("myset", count=-1) + assert len(members_list) == 1 + assert members_list[0] in elements + + # Test with unexisting vset & without count + members_list = await d_client.vset().vrandmember("myset_unexisting") + assert members_list is None + + # Test with unexisting vset & count + members_list = await d_client.vset().vrandmember("myset_unexisting", count=5) + assert members_list == [] + + +@skip_if_server_version_lt("7.9.0") +async def test_vset_commands_without_decoding_responces(client): + # test vadd + elements = ["elem1", "elem2", "elem3"] + for elem in elements: + float_array = [random.uniform(0.5, 10) for x in range(0, 8)] + resp = await client.vset().vadd("myset", float_array, element=elem) + assert resp == 1 + + # test vemb + emb = await client.vset().vemb("myset", "elem1") + assert len(emb) == 8 + assert isinstance(emb, list) + assert all(isinstance(x, float) for x in emb) + + emb_raw = await client.vset().vemb("myset", "elem1", raw=True) + assert emb_raw["quantization"] == b"int8" + assert isinstance(emb_raw["raw"], bytes) + assert isinstance(emb_raw["l2"], float) + assert isinstance(emb_raw["range"], float) + + # test vsim + vsim = await client.vset().vsim("myset", input="elem1") + assert len(vsim) == 3 + assert isinstance(vsim, list) + assert isinstance(vsim[0], bytes) + + # test vsim with scores + vsim_with_scores = await client.vset().vsim( + "myset", input="elem1", with_scores=True + ) + assert len(vsim_with_scores) == 3 + assert isinstance(vsim_with_scores, dict) + assert isinstance(vsim_with_scores[b"elem1"], float) + + # test vlinks - no scores + element_links_all_layers = await client.vset().vlinks("myset", "elem1") + assert len(element_links_all_layers) >= 1 + for neighbours_list_for_layer in element_links_all_layers: + assert isinstance(neighbours_list_for_layer, list) + for neighbour in neighbours_list_for_layer: + assert isinstance(neighbour, bytes) + # test vlinks with scores + elem_links_all_layers_with_scores = await client.vset().vlinks( + "myset", "elem1", with_scores=True + ) + assert len(elem_links_all_layers_with_scores) >= 1 + for neighbours_dict_for_layer in elem_links_all_layers_with_scores: + assert isinstance(neighbours_dict_for_layer, dict) + for neighbour_key, score_value in neighbours_dict_for_layer.items(): + assert isinstance(neighbour_key, bytes) + assert isinstance(score_value, float) + + # test vinfo + vset_info = await client.vset().vinfo("myset") + assert vset_info[b"quant-type"] == b"int8" + assert vset_info[b"vector-dim"] == 8 + assert vset_info[b"size"] == len(elements) + assert vset_info[b"max-level"] >= 0 + assert vset_info[b"hnsw-max-node-uid"] == len(elements) + + # test vgetattr + attributes = {"key1": "value1", "key2": "value2"} + await client.vset().vsetattr("myset", "elem1", attributes) + attrs = await client.vset().vgetattr("myset", "elem1") + assert attrs == attributes + + # test vrandmember + random_member = await client.vset().vrandmember("myset") + assert isinstance(random_member, bytes) + assert random_member.decode("utf-8") in elements + + members_list = await client.vset().vrandmember("myset", count=2) + assert len(members_list) == 2 + assert all(member.decode("utf-8") in elements for member in members_list) + + +def _to_fp32_blob_array(float_array): + """ + Convert a list of floats to a byte array in fp32 format. + """ + # Convert the list of floats to a NumPy array with dtype np.float32 + arr = np.array(float_array, dtype=np.float32) + # Convert the NumPy array to a byte array + byte_array = arr.tobytes() + return byte_array + + +def _validate_quantization(original, quantized, tolerance=0.1): + original = np.array(original, dtype=np.float32) + quantized = np.array(quantized, dtype=np.float32) + + max_diff = np.max(np.abs(original - quantized)) + if max_diff > tolerance: + return False + else: + return True diff --git a/tests/test_anyio/testdata/jsontestdata.py b/tests/test_anyio/testdata/jsontestdata.py new file mode 100644 index 0000000000..0a920cc55b --- /dev/null +++ b/tests/test_anyio/testdata/jsontestdata.py @@ -0,0 +1,617 @@ +nested_large_key = r""" +{ + "jkra": [ + 154, + 4472, + [ + 8567, + false, + 363.84, + 5276, + "ha", + "rizkzs", + 93 + ], + false + ], + "hh": 20.77, + "mr": 973.217, + "ihbe": [ + 68, + [ + true, + { + "lqe": [ + 486.363, + [ + true, + { + "mp": { + "ory": "rj", + "qnl": "tyfrju", + "hf": None + }, + "uooc": 7418, + "xela": 20, + "bt": 7014, + "ia": 547, + "szec": 68.73 + }, + None + ], + 3622, + "iwk", + None + ], + "fepi": 19.954, + "ivu": { + "rmnd": 65.539, + "bk": 98, + "nc": "bdg", + "dlb": { + "hw": { + "upzz": [ + true, + { + "nwb": [ + 4259.47 + ], + "nbt": "yl" + }, + false, + false, + 65, + [ + [ + [], + 629.149, + "lvynqh", + "hsk", + [], + 2011.932, + true, + [] + ], + None, + "ymbc", + None + ], + "aj", + 97.425, + "hc", + 58 + ] + }, + "jq": true, + "bi": 3333, + "hmf": "pl", + "mrbj": [ + true, + false + ] + } + }, + "hfj": "lwk", + "utdl": "aku", + "alqb": [ + 74, + 534.389, + 7235, + [ + None, + false, + None + ] + ] + }, + None, + { + "lbrx": { + "vm": "ubdrbb" + }, + "tie": "iok", + "br": "ojro" + }, + 70.558, + [ + { + "mmo": None, + "dryu": None + } + ] + ], + true, + None, + false, + { + "jqun": 98, + "ivhq": [ + [ + [ + 675.936, + [ + 520.15, + 1587.4, + false + ], + "jt", + true, + { + "bn": None, + "ygn": "cve", + "zhh": true, + "aak": 9165, + "skx": true, + "qqsk": 662.28 + }, + { + "eio": 9933.6, + "agl": None, + "pf": false, + "kv": 5099.631, + "no": None, + "shly": 58 + }, + [ + None, + [ + "uiundu", + 726.652, + false, + 94.92, + 259.62, + { + "ntqu": None, + "frv": None, + "rvop": "upefj", + "jvdp": { + "nhx": [], + "bxnu": {}, + "gs": None, + "mqho": None, + "xp": 65, + "ujj": {} + }, + "ts": false, + "kyuk": [ + false, + 58, + {}, + "khqqif" + ] + }, + 167, + true, + "bhlej", + 53 + ], + 64, + { + "eans": "wgzfo", + "zfgb": 431.67, + "udy": [ + { + "gnt": [], + "zeve": {} + }, + { + "pg": {}, + "vsuc": {}, + "dw": 19, + "ffo": "uwsh", + "spk": "pjdyam", + "mc": [], + "wunb": {}, + "qcze": 2271.15, + "mcqx": None + }, + "qob" + ], + "wo": "zy" + }, + { + "dok": None, + "ygk": None, + "afdw": [ + 7848, + "ah", + None + ], + "foobar": 3.141592, + "wnuo": { + "zpvi": { + "stw": true, + "bq": {}, + "zord": true, + "omne": 3061.73, + "bnwm": "wuuyy", + "tuv": 7053, + "lepv": None, + "xap": 94.26 + }, + "nuv": false, + "hhza": 539.615, + "rqw": { + "dk": 2305, + "wibo": 7512.9, + "ytbc": 153, + "pokp": None, + "whzd": None, + "judg": [], + "zh": None + }, + "bcnu": "ji", + "yhqu": None, + "gwc": true, + "smp": { + "fxpl": 75, + "gc": [], + "vx": 9352.895, + "fbzf": 4138.27, + "tiaq": 354.306, + "kmfb": {}, + "fxhy": [], + "af": 94.46, + "wg": {}, + "fb": None + } + }, + "zvym": 2921, + "hhlh": [ + 45, + 214.345 + ], + "vv": "gqjoz" + }, + [ + "uxlu", + None, + "utl", + 64, + [ + 2695 + ], + [ + false, + None, + [ + "cfcrl", + [], + [], + 562, + 1654.9, + {}, + None, + "sqzud", + 934.6 + ], + { + "hk": true, + "ed": "lodube", + "ye": "ziwddj", + "ps": None, + "ir": {}, + "heh": false + }, + true, + 719, + 50.56, + [ + 99, + 6409, + None, + 4886, + "esdtkt", + {}, + None + ], + [ + false, + "bkzqw" + ] + ], + None, + 6357 + ], + { + "asvv": 22.873, + "vqm": { + "drmv": 68.12, + "tmf": 140.495, + "le": None, + "sanf": [ + true, + [], + "vyawd", + false, + 76.496, + [], + "sdfpr", + 33.16, + "nrxy", + "antje" + ], + "yrkh": 662.426, + "vxj": true, + "sn": 314.382, + "eorg": None + }, + "bavq": [ + 21.18, + 8742.66, + { + "eq": "urnd" + }, + 56.63, + "fw", + [ + {}, + "pjtr", + None, + "apyemk", + [], + [], + false, + {} + ], + { + "ho": None, + "ir": 124, + "oevp": 159, + "xdrv": 6705, + "ff": [], + "sx": false + }, + true, + None, + true + ], + "zw": "qjqaap", + "hr": { + "xz": 32, + "mj": 8235.32, + "yrtv": None, + "jcz": "vnemxe", + "ywai": [ + None, + 564, + false, + "vbr", + 54.741 + ], + "vw": 82, + "wn": true, + "pav": true + }, + "vxa": 881 + }, + "bgt", + "vuzk", + 857 + ] + ] + ], + None, + None, + { + "xyzl": "nvfff" + }, + true, + 13 + ], + "npd": None, + "ha": [ + [ + "du", + [ + 980, + { + "zdhd": [ + 129.986, + [ + "liehns", + 453, + { + "fuq": false, + "dxpn": {}, + "hmpx": 49, + "zb": "gbpt", + "vdqc": None, + "ysjg": false, + "gug": 7990.66 + }, + "evek", + [ + {} + ], + "dfywcu", + 9686, + None + ] + ], + "gpi": { + "gt": { + "qe": 7460, + "nh": "nrn", + "czj": 66.609, + "jwd": true, + "rb": "azwwe", + "fj": { + "csn": true, + "foobar": 1.61803398875, + "hm": "efsgw", + "zn": "vbpizt", + "tjo": 138.15, + "teo": {}, + "hecf": [], + "ls": false + } + }, + "xlc": 7916, + "jqst": 48.166, + "zj": "ivctu" + }, + "jl": 369.27, + "mxkx": None, + "sh": [ + true, + 373, + false, + "sdis", + 6217, + { + "ernm": None, + "srbo": 90.798, + "py": 677, + "jgrq": None, + "zujl": None, + "odsm": { + "pfrd": None, + "kwz": "kfvjzb", + "ptkp": false, + "pu": None, + "xty": None, + "ntx": [], + "nq": 48.19, + "lpyx": [] + }, + "ff": None, + "rvi": [ + "ych", + {}, + 72, + 9379, + 7897.383, + true, + {}, + 999.751, + false + ] + }, + true + ], + "ghe": [ + 24, + { + "lpr": true, + "qrs": true + }, + true, + false, + 7951.94, + true, + 2690.54, + [ + 93, + None, + None, + "rlz", + true, + "ky", + true + ] + ], + "vet": false, + "olle": None + }, + "jzm", + true + ], + None, + None, + 19.17, + 7145, + "ipsmk" + ], + false, + { + "du": 6550.959, + "sps": 8783.62, + "nblr": { + "dko": 9856.616, + "lz": { + "phng": "dj" + }, + "zeu": 766, + "tn": "dkr" + }, + "xa": "trdw", + "gn": 9875.687, + "dl": None, + "vuql": None + }, + { + "qpjo": None, + "das": { + "or": { + "xfy": None, + "xwvs": 4181.86, + "yj": 206.325, + "bsr": [ + "qrtsh" + ], + "wndm": { + "ve": 56, + "jyqa": true, + "ca": None + }, + "rpd": 9906, + "ea": "dvzcyt" + }, + "xwnn": 9272, + "rpx": "zpr", + "srzg": { + "beo": 325.6, + "sq": None, + "yf": None, + "nu": [ + 377, + "qda", + true + ], + "sfz": "zjk" + }, + "kh": "xnpj", + "rk": None, + "hzhn": [ + None + ], + "uio": 6249.12, + "nxrv": 1931.635, + "pd": None + }, + "pxlc": true, + "mjer": false, + "hdev": "msr", + "er": None + }, + "ug", + None, + "yrfoix", + 503.89, + 563 + ], + "tcy": 300, + "me": 459.17, + "tm": [ + 134.761, + "jcoels", + None + ], + "iig": 945.57, + "ad": "be" + }, + "ltpdm", + None, + 14.53 + ], + "xi": "gxzzs", + "zfpw": 1564.87, + "ow": None, + "tm": [ + 46, + 876.85 + ], + "xejv": None +} +""" # noqa diff --git a/tests/test_anyio/testdata/titles.csv b/tests/test_anyio/testdata/titles.csv new file mode 100644 index 0000000000..6428dd2aa3 --- /dev/null +++ b/tests/test_anyio/testdata/titles.csv @@ -0,0 +1,4861 @@ +bhoj shala,1 +radhika balakrishnan,1 +ltm,1 +sterlite energy,1 +troll doll,11 +sonnontio,1 +nickelodeon netherlands kids choice awards,1 +jamaica national basketball team,5 +clan mackenzie,1 +secure attention key,3 +template talk indo pakistani war of 1971,1 +hassan firouzabadi,2 +carter alan,1 +alan levy,1 +tim severin,2 +faux pas derived from chinese pronunciation,1 +jruby,3 +tobias nielsén,1 +avro 571 buffalo,1 +treasury stock,17 +שלום,10 +oxygen 19,1 +ntru,4 +tennis racquet,1 +place of birth,4 +council of canadians,1 +urshu,1 +american hotel,1 +dow corning corporation,3 +language based learning disability,3 +meri aashiqui tum se hi,30 +specificity,9 +edward l hedden,1 +pelli chesukundam,2 +of love and shadows,4 +fort san felipe,2 +american express gold card dress of lizzy gardiner,4 +jovian,5 +kitashinagawa station,1 +radhi jaidi,1 +cordelia scaife may,2 +minor earth major sky,1 +bunty lawless stakes,1 +high capacity color barcode,3 +lyla lerrol,1 +crawford roberts,1 +collin balester,1 +ugo crousillat,1 +om prakash chautala,3 +izzy hoyland,1 +the poet,2 +daryl sabara,6 +aromatic acid,2 +reina sofia,1 +swierczek masovian voivodeship,1 +housing segregation in the united states,2 +karen maser,1 +scaptia beyonceae,2 +kitakyushu city,1 +htc desire 610,4 +dostoevsky,3 +portal victorian era,1 +bose–einstein correlations,3 +ralph hodgson,1 +racquet club,2 +walter camp man of the year,1 +australian movies,1 +k04he,1 +australia–india relations,2 +john william howard thompson,1 +pro cathedral,1 +paddyfield pipit,2 +book finance,1 +ford maverick,10 +slurve,4 +mnozil brass,2 +fiesta 9 1/8 inch square luncheon plate sunflower,1 +korsi,1 +draft 140th operations group,2 +camp,29 +series acceleration,1 +aljouf,1 +democratic party of new mexico,2 +united kingdom general election debates 2010,2 +madura strait,2 +back examination,1 +borgata,2 +il ritorno di tobia,3 +ovaphagous,1 +motörhead,9 +hellmaster,1 +richard keynes,1 +cryogenic treatment,3 +monte porzio,1 +transliteration of arabic,1 +anti catholic,2 +a very merry pooh year,2 +suffixes in hebrew,3 +barr body,16 +alaska constitution,1 +juan garrido,1 +yi lijun,1 +wawa inc,2 +endre kelemen,1 +l brands,18 +lr44,1 +coat of arms of the nagorno karabakh republic,1 +antonino fernandez,1 +salisbury roller girls,1 +zayat,2 +ian meadows,2 +semigalia,1 +khloe and lamar,2 +holding,1 +larchmont edgewater,1 +dynamic parcel distribution,6 +seaworld,30 +assistant secretary of war,1 +digital currency,14 +mazomanie wisconsin,1 +sujatha rangarajan,8 +street child,1 +anna sheehan,1 +violence jack,2 +santi solari,1 +template talk texas in the civil war,1 +colorss foundation,1 +faucaria,1 +alfred gardyne de chastelain,2 +tramp,1 +cannington ontario,2 +penguinone,1 +cardiac arrest,2 +summan grouper,1 +cyndis list,1 +cbs,2 +salminus brasiliensis,2 +kodiak bear,26 +cinemascore,9 +phragmidium,1 +city of vultures,1 +lawrence g romo,1 +chandni chowk to china,1 +scarp retreat,1 +rosses point,1 +carretera de cádiz,1 +chamunda,8 +battle of stalingrad,1 +who came first,2 +salome,5 +portuguese historical museum,3 +westfield sarasota square,1 +muehrckes nails,3 +kennebec north carolina,1 +american classical league,1 +how do you like them apples,1 +mark halperin,20 +circo,1 +turner classic movies,2 +australian rules football in sweden,1 +household silver,3 +frank baird,1 +escape from east berlin,2 +a village romeo and juliet,1 +wally nesbitt,6 +joseph renzulli,2 +spalding gray,1 +dangaria kandha,1 +pms asterisk,2 +openal,1 +romy haag,1 +mh message handling system,4 +pioneer 4,4 +hmcs stettler,1 +gangsta,10 +major third,4 +joan osbourn,1 +mount columbia,2 +active galactic nucleus,14 +robert clary,8 +eva pracht,1 +ion implantation,5 +rydell poepon,4 +baller blockin,2 +enfield chase railway station,1 +serge aurier,13 +florin vlaicu,1 +van diemens land,9 +krishnapur bagalkot,1 +oleksandr zinchenko,96 +collaborations,2 +hecla,2 +amber marshall,7 +inácio henrique de gouveia,1 +bronze age korea,1 +slc punk,5 +ryan jack,2 +clathrus ruber,6 +angel of death,4 +valentines park,1 +extra pyramidal,1 +kiami davael,1 +oleg i shuplyak,1 +nidum,2 +friendship of salem,2 +bèze,3 +arnold weinstock,1 +able,1 +s d ugamchand,1 +the omega glory,2 +ami james,3 +denmark at the 1968 summer olympics,1 +kill me again,1 +richmond town square,1 +guy domville,1 +jessica simpson,1 +kinship care,1 +brugge railway station,2 +unobtainium,16 +carl johan bernadotte,3 +acacia concinna,5 +epinomis,1 +interlachen country club,1 +compromise tariff,1 +fairchild jk,1 +dog trainer,1 +brian dabul,1 +cai yong,1 +jezebel,7 +augarten porcelain,1 +summerslam 1992,1 +ion andoni goikoetxea,2 +dominican church vienna,1 +iffhs worlds best club coach,2 +uruguayan presidential election 2009,2 +saving the queen,1 +un cadavre,1 +history of the jews in france,4 +wbyg,1 +charles de brosses,2 +human weapon,2 +haunted castle,3 +austin maestro,1 +search for extra terrestrial intelligence,1 +suwon,9 +cost per impression,1 +osney lock,1 +markus eriksson,1 +cultural depictions of tony blair,2 +erich kempka,3 +pornogrind,5 +chekhov,1 +marilinda garcia,2 +hard drive,1 +small arms,9 +exploration of north america,8 +international korfball federation,1 +photographic lens design,4 +k hari prasad,1 +lebanese forces,3 +greece at the 2004 summer olympics,1 +lets trim our hair in accordance with the socialist lifestyle,2 +battle of cassinga,5 +donald and the wheel,1 +vti transmission,1 +gille chlerig earl of mar,1 +heart of atlanta motel inc v united states,6 +oh yeah,3 +carol decker,5 +prajakta shukre,4 +profiling,17 +thukima,1 +the great waldo search,1 +nick vincent,2 +the decision of the appeals jury is final and can only be overruled by a decision of the executive committee 2e,1 +civilization board game,1 +erasmus+,1 +eden phillpotts,1 +unleash the beast,1 +varoujan hakhbandian,1 +fermats last theorem,1 +conan the indomitable,1 +vagrant records,1 +house of villehardouin,1 +zoneyesha ulatha,1 +ashur bel nisheshu,1 +ten wijngaerde,2 +lgi homes,1 +american nietzsche a history of an icon and his ideas,1 +european magpie,3 +pablo soto,1 +terminiello v chicago,1 +vladimir cosma,2 +battle of yunnan burma road,1 +ophirodexia,1 +thudar,1 +northern irish,2 +bohemond of tarente,1 +anita moorjani,5 +serra do gerês,1 +fort horsted,1 +metre gauge,2 +stage show,3 +common flexor sheath of hand,2 +conall corc,1 +array slicing,6 +schüfftan process,1 +anmol malik,3 +out cold,2 +antiknock,2 +moss force,1 +paul medhurst,1 +somonauk illinois,1 +george crum,11 +baby talk,6 +daniel mann,4 +vacuum flask,10 +prostitution in the republic of ireland,5 +butch jones,7 +feminism in ukraine,1 +st marys church kilmore county wexford,1 +sonny emory,1 +satsuma han,1 +elben,1 +the best of the rippingtons,3 +m3p,1 +boat sharing,1 +iisco,1 +hoftoren,1 +cannabis in the united kingdom,6 +template talk germany districts saxony anhalt,1 +jean baptiste dutrou bornier,1 +teylers museum,1 +simons problem,2 +gerardus huysmans,1 +pupillary distance,5 +jane lowe,1 +palais de justice brussels,1 +hillsdale free will baptist college,1 +raf wattisham,2 +parnataara,1 +jensen beach campus of the florida institute of technology,1 +scottish gypsy and traveller groups,3 +cliffs shaft mine museum,3 +roaring forties,4 +where in time is carmen sandiego?,2 +perfect field,1 +rob schamberger,1 +lcd soundsystem,10 +alan rathbone,26 +setup,1 +gliding over all,4 +dastur,1 +flensburger brauerei,3 +berkeley global campus at richmond bay,1 +kanakapura,1 +mineworkers union of namibia,1 +tokneneng,3 +mapuche textiles,3 +peranakan beaded slippers,1 +goodra,2 +kanab ut,1 +the gold act 1968,4 +grey langur,1 +procol harum,5 +chris alexander,1 +ft walton beach metropolitan area,3 +dimensionless quantity,16 +the science of mind,1 +alfons schone,1 +euparthenos nubilis,1 +batrachotoxin,5 +fabric live 22,1 +mchenry boatwright,1 +langney sports club,1 +akela jones,1 +lookout,2 +matsuo tsurayaba,2 +general jackson,3 +hair removal,14 +african party for the independence of cape verde,4 +replica trick,1 +bromfenac,2 +make someone happy,1 +sam pancake,1 +denys finch hatton,10 +latin rhythm albums,1 +main bronchus,1 +campidoglio,4 +cathaoirleach,1 +emress justina,1 +sulzbach hesse,1 +noncicatricial alopecia,1 +sylvan place,4 +stalag i c,1 +league of extraordinary gentlemen,1 +sergey korolyov,2 +serbian presidential election 1997,1 +barnes lake millers lake michigan,1 +christmas island health centre,1 +dayton ballet,2 +gilles fauconnier,1 +harald svergja,1 +joanna newsom discography,2 +astro xi yue hd,1 +code sharing,3 +dreamcast vmu,1 +armand emmanuel du plessis duc de richelieu,1 +ecole supérieure des arts du cirque,2 +gerry mulligan,12 +kaaka kaaka,1 +mexico at the 2012 summer olympics,4 +bar wizards,2 +christmas is almost here again,2 +sterling heights michigan,4 +gaultheria procumbens,3 +eben etzebeth,8 +viktorija Čmilytė,1 +los angeles county california,39 +family entertainment,2 +quantum well,9 +elton,1 +allan frewin jones,1 +daniela ruah,32 +gkd legend,1 +coffman–graham algorithm,1 +santa clara durango,1 +brian protheroe,3 +crawler transporter,10 +lakshman,3 +fes el bali,2 +mary a krupsak,1 +irish rugby football union,5 +neuropsychiatry,2 +josé pirela,1 +bonaire status referendum 2015,1 +it,2 +playhouse in the park,1 +alexander yakovlev,7 +old bear,1 +graph tool,2 +merseyside west,1 +romanian armies in the battle of stalingrad,1 +dark they were and golden eyed,1 +aidan obrien,8 +town and davis,1 +suum cuique,3 +german american day,2 +northampton county pennsylvania,3 +candidates of the south australian state election 2010,1 +venator marginatus,2 +k60an,1 +template talk campaignbox seven years war european,1 +maravi,1 +flaithbertach ua néill,1 +junction ohio,1 +dave walter,1 +london transport board,1 +tuyuka,1 +the moodys,3 +noel,3 +eugen richter,1 +cowanshannock township armstrong county pennsylvania,1 +pre columbian gold museum,1 +lac demosson,1 +lincosamides,9 +the vegas connection,1 +stephen e harris,1 +alkali feldspar,2 +brant hansen,1 +draft carnatic music stub,4 +the chemicals between us,1 +blood and bravery,1 +san diego flash,3 +covert channel,5 +ernest w adams,1 +hills brothers coffee,1 +cosmic background explorer,4 +international union of pure and applied physics,2 +vladimir kramnik,21 +hinterland,2 +tinker bell and the legend of the neverbeast,5 +ophisops jerdonii,1 +fine gold,1 +net explosive quantity,3 +miss colorado teen usa,3 +royal philharmonic orchestra discography,1 +elyazid maddour,1 +matthew kelly,2 +templating language,1 +japan campaign,2 +barack obama on mass surveillance,2 +thomas r donahue,1 +old right,4 +spencer kimball,1 +golden kela awards,1 +blinn college,3 +w k simms,1 +quinto romano,1 +richard mulrooney,1 +mr backup z64,1 +monetization of us in kind food aid,1 +alex chilton,2 +propaganda in the peoples republic of china,4 +jiří skalák,8 +m5 stuart tank,1 +template talk ap defensive players of the year,1 +crisis,2 +azuchi momoyama period,1 +care and maintenance,2 +a$ap mob,3 +near field communication,111 +hips hips hooray,1 +promotional cd,1 +andean hairy armadillo,1 +trigueros del valle,1 +elmwood illinois,1 +cantonment florida,2 +margo t oge,1 +national park service,36 +monongalia county ballpark,3 +bakemonogatari,6 +felicia michaels,1 +institute of oriental studies of the russian academy of sciences,2 +economy of eritrea,2 +vincenzo chiarenza,1 +microelectronics,4 +fresno state bulldogs mens basketball,1 +maotou,1 +blokely,1 +duplicati,3 +goud,2 +niki reiser,1 +edward leonard ellington,1 +jaswant singh of marwar,1 +biharsharif,1 +dynasty /trackback/,1 +machrihanish,4 +jay steinberg,1 +peter luger steak house,3 +palookaville,1 +ferrari grand prix results,2 +bankruptcy discharge,2 +mike mccue,2 +nuestra belleza méxico 2013,2 +alex neal bullen,1 +gus macdonald baron macdonald of tradeston,2 +florida circuit court,1 +haarp,2 +v pudur block,1 +grocer,1 +shmuel hanavi,1 +isaqueena falls,2 +jean moulin university,1 +final fantasy collection,1 +template talk american frontier,1 +chex quest,4 +muslim students association,2 +marco pique,1 +jinja safari,1 +the collection,9 +urban districts of germany,5 +rajiv chilaka,1 +zion,2 +vf 32,1 +united states commission on civil rights,2 +zazam,1 +barnettas,4 +rebecca blasband,1 +lincoln village,1 +film soundtracks,1 +angus t jones,77 +snuppy,3 +w/indexphp,30 +file talk american world war ii senior military officials 1945jpeg,1 +worship leader,1 +ein qiniya,1 +buxton maine,1 +matt dewitt,1 +béla bollobás,3 +earlysville union church,1 +bae/mcdonnell douglas harrier ii gr9,1 +californian condor,2 +progressive enhancement,15 +its not my time,4 +ecw on tnn,2 +ihop,36 +aeronautical chart,1 +clique width,1 +fuengirola,8 +archicebus achilles,2 +comparison of alcopops,1 +carla anderson hills,1 +roanoke county virginia,2 +jaílson alves dos santos,1 +rameses revenge,1 +kaycee stroh,5 +les experts,1 +niels skousen,1 +apollo hoax theories,1 +mercedes w204,2 +enhanced mitigation experience toolkit,15 +bert barnes,1 +serializability,6 +ten plagues of egypt,1 +joe l brown,1 +category talk high importance chicago bears articles,1 +stephen caffrey,3 +european border surveillance system,2 +achytonix,1 +m2 machine gun,1 +gurieli,1 +kunefe,1 +m33 helmet,3 +little carmine,1 +smush,3 +josé horacio gómez,1 +product recall,1 +egger,1 +wisconsin highway 55,1 +harbledown,1 +low copy repeats,1 +curt gentry,1 +united colors of benetton,1 +adiabatic shear band,2 +pea galaxy,1 +where are you now,1 +dils,1 +surprise s1,1 +senate oceans caucus,2 +windsor new hampshire,1 +a hawk and a hacksaw,1 +i love it loud,2 +milbcom,1 +old world vulture,7 +camara v municipal court of city and county of san francisco,1 +ski dubai,1 +st cyprians school,2 +aibo,1 +ticker symbol,2 +hendrik houthakker,1 +shivering,5 +jacob arminius,1 +mowming,1 +panjiva,2 +namco libble rabble,5 +rudolph bing,1 +sindhi cap,2 +logician,1 +ford xa falcon,2 +the sunny side up show,1 +helen adams,2 +kharchin,1 +brittany maynard,13 +kim kyu jong,1 +messier 103,3 +leon boiler,1 +the rapeman,1 +twa flight 3,4 +leading ladies,1 +delta octantis,2 +qatari nationality law,1 +lionel cripps,1 +josé daniel carreño,1 +crypsotidia longicosta,1 +polish falcons,1 +highlands north gauteng,1 +the florida channel,1 +oreste barale,1 +ghazi of iraq,2 +charles grandison finney,4 +ahmet ali,1 +abbeytown,1 +caribou,3 +big two,2 +alien,14 +aslantaş dam,3 +theme of the traitor and the hero,1 +vladimir solovyov,1 +laguna ojo de liebre,1 +clive barton,1 +ebrahim daoud nonoo,1 +richard goodwin keats,2 +back to the who tour 51,1 +entertainmentwise,1 +ja preston,1 +john astin,19 +strict function,1 +cam ranh international airport,2 +gary pearson,1 +sven väth,8 +toad,6 +johnny pace,1 +hunt stockwell,1 +rolando schiavi,1 +claudia grassl,1 +oxford nova scotia,1 +maryland sheep and wool festival,1 +conquest of bread,1 +erevan,1 +comparison of islamic and jewish dietary laws,11 +sheila burnford,1 +estevan payan,1 +ocean butterflies international,7 +the royal winnipeg rifles,1 +green goblin in other media,2 +video gaming in japan,8 +church of the guanche people,4 +gustav hartlaub,2 +ian mcgeechan,4 +hammer and sickle,17 +konkiep river,1 +ceri richards,1 +decentralized,2 +depth psychology,3 +centennial parkway,1 +yugoslav monitor vardar,1 +battle of bobbili,2 +magnus iii of sweden,1 +england c national football team,2 +thuraakunu,1 +bab el ehr,1 +koi,1 +cully wilson,1 +money laundering,1 +stirling western australia,1 +jennifer dinoia,1 +eureka street,1 +message / call my name,1 +make in maharashtra,4 +huckleberry creek patrol cabin,1 +almost famous,5 +truck nuts,4 +vocus communications,1 +gikwik,1 +battle of bataan,4 +confluence pennsylvania,2 +islander 23,1 +mv skorpios ii,1 +single wire earth return,1 +politics of odisha,1 +crédit du nord,3 +piper methysticum,2 +coble,2 +kathleen a mattea,1 +coachella valley music and arts festival,50 +tooniverse,1 +spofforth castle,1 +arabian knight,2 +two airlines policy,1 +hinduja group,17 +swagg alabama,1 +portuguese profanity,1 +loomis gang,2 +nina veselova,2 +aegyrcitherium,1 +bees in paradise,1 +béládys anomaly,3 +badalte rishtey,1 +first bank fc,1 +cystoseira,1 +red book of endangered languages,1 +rose,6 +terry mcgurrin,3 +jason hawke,1 +peter chernin,1 +tu 204,1 +the man who walked alone,1 +tool grade steel,1 +wrist spin,1 +one step forward two steps back,1 +theodor boveri,1 +heunginjimun,1 +fama–french three factor model,34 +billy whitehurst,1 +rip it up,4 +red lorry yellow lorry,4 +nao tōyama,8 +general macarthur,1 +rabi oscillation,2 +devín,1 +olympus e 420,1 +hydra entertainment,1 +chris cheney,3 +rio all suite hotel and casino,3 +the death gate cycle,2 +fatima,1 +kamomioya shrine,1 +five nights at freddys 3,14 +the broom of the system,3 +robert blincoe,1 +history of wells fargo,9 +pinocytosis,4 +leaf phoenix,1 +wxmw,2 +tommy henriksen,13 +geri halliwell discography,2 +blade runneri have seen things you would not believe,1 +madhwa brahmins,1 +i/o ventures,1 +edorisi master ekhosuehi,2 +junior orange bowl,1 +khit,2 +sue jones,1 +immortalized,35 +city building series,4 +quran translation,1 +united states consulate,1 +dose response relationship,1 +caitriona,1 +colocolo,21 +medea class destroyer,1 +vaastav,1 +etc1,1 +john altoon,2 +thylacine,113 +cycling at the 1924 summer olympics,1 +margaret nagle,1 +superpower,57 +gülşen,1 +anthems to the welkin at dusk,4 +yerevan united fc,1 +the family fang,14 +domain,4 +high speed rail in india,14 +trifolium pratense,7 +florida mountains,2 +national city corp,5 +length of us participation in major wars,2 +acacia acanthoclada,1 +offas dyke path,2 +enduro,7 +howard center,1 +littlebits,4 +plácido domingo jr,1 +hookdale illinois,1 +the love language,1 +cupids arrows,1 +dc talk,7 +maesopsis eminii,1 +here comes goodbye,1 +freddie foreman,5 +marvel comics publishers,1 +consolidated city–county,5 +countess marianne bernadotte of wisborg,1 +los angeles baptist high school,1 +maglalatik,1 +deo,2 +meilichiu,1 +wade coleman,1 +monster soul,2 +julion alvarez,2 +platinum 166,1 +shark week,12 +hossbach memorandum,4 +jack c massey,3 +ardore,1 +philosopher king,5 +dynamic random access memory,5 +bronze age in southeastern europe,1 +tamil films of 2012,1 +nathalie cely,1 +italian capital,1 +optic tract,3 +shakti kumar,1 +who killed bruce lee,1 +parlement of brittany,3 +san juan national historic site,2 +livewell,2 +template talk om,1 +al bell,2 +pzl w 3 sokół,8 +durrës rail station,3 +david stubbs,1 +pharmacon,3 +railfan,7 +comics by country,2 +cullen baker,1 +maximum subarray problem,19 +outlaws and angels,1 +paradise falls,2 +mathias pogba,28 +donella meadows,4 +john leconte,2 +swaziland national football team,7 +gabriele detti,2 +if ever youre in my arms again,1 +christian basso,1 +helen shapiro,7 +taisha abelar,1 +fluid dynamics,1 +ernest wilberforce,1 +kocaeli university,2 +british m class submarine,1 +modern woodmen of america,1 +las posadas,3 +federal budget of germany,2 +liberation front of chad,1 +sandomierz,5 +ap italian language and culture,1 +manuel gonzález,1 +georgian military road,2 +clear creek county colorado,1 +matt clark,2 +test tube,18 +ak 47,1 +diège,1 +london school of economics+,1 +michael york,14 +half eagle,6 +strike force,1 +type 054 frigate,2 +sino indian relations,7 +fern,3 +louvencourt,1 +ghb receptor,2 +chondrolaryngoplasty,2 +andrew lewer,1 +ross king,1 +colpix records,1 +october 28,1 +tatsunori hara,1 +rossana lópez león,1 +haskell texas,3 +tower subway,2 +waspstrumental,1 +template talk nba anniversary teams,1 +george leo leech,1 +still nothing moves you,1 +blood cancer,3 +buffy lynne williams,1 +dpgc u know what im throwin up,1 +daniel nadler,1 +khalifa sankaré,2 +homo genus,1 +garðar thór cortes,3 +veyyil,1 +matt dodge,1 +hipponix subrufus,1 +anostraca,1 +hartshill park,1 +purple acid phosphatases,1 +austromyrtus dulcis,1 +shamirpet lake,1 +favila of asturias,2 +acute gastroenteritis,1 +dalton cache pleasant camp border crossing,1 +urobilinogen,13 +ss kawartha park,1 +professional chess association,1 +species extinction,1 +gapa hele bi sata,1 +phyllis lyon and del martin,1 +uk–us extradition treaty of 2003,1 +a woman killed with kindness,1 +how bizarre,1 +norm augustine,1 +geil,1 +volleyball at the 2015 southeast asian games,2 +jim ottaviani,1 +chekmagushevskiy district,1 +information search process,2 +queer,63 +william pidgeon,1 +amelia adamo,1 +nato ouvrage "g",1 +tamsin beaumont,1 +economy of syria,13 +douglas dc 8 20,1 +tama and friends,4 +pringles,22 +kannada grammar,7 +lotoja,1 +peony,1 +bmmi,1 +eurovision song contest 1992,11 +cerro blanco metro station,1 +sherlock the riddle of the crown jewels,4 +dorsa cato,1 +nkg2d,8 +specific heat,6 +nokia 6310i,2 +tergum,2 +bahai temple,1 +dal segno,5 +leigh chapman,2 +tupolev tu 144,60 +flight of ideas,1 +rita montaner,1 +vivien a schmidt,1 +battle of the treasury islands,2 +three kinds of evil destination,1 +richlite,1 +medinilla,2 +timeline of aids,1 +colin renfrew baron renfrew of kaimsthorn,2 +hélène rollès,1 +pedro winter,1 +sabine free state,1 +brzeg,1 +palisades park,1 +gas gangrene,11 +dotyk,2 +daniela kix,1 +canna,16 +property list,9 +john hamburg,1 +dunk island,5 +albreda,1 +scammed yankees,1 +wireball,3 +junior 4,1 +absolutely anything,15 +linux operating system,1 +solsbury hill,15 +notopholia,1 +scottish heraldry,2 +template talk paper data storage media,1 +category talk religion in ancient sparta,1 +category talk cancer deaths in puerto rico,1 +mid michigan community college,2 +tvb anniversary awards,1 +frederick taylor gates,1 +omoiyari yosan,3 +journal of the physical society of japan,1 +kings in the corner,2 +nungua,1 +amerika,4 +pacific marine environmental laboratory,1 +the thought exchange,1 +italian bee,5 +roma in spain,1 +sirinart,1 +crandon wisconsin,1 +shubnikov–de haas effect,6 +portrait of maria portinari,4 +colin mcmanus,1 +universal personal telecommunications,1 +royal docks,4 +brecon and radnorshire,3 +eilema caledonica,1 +chalon sur saône,8 +toyota grand hiace,1 +sophorose,1 +semirefined 2bwax,1 +mechanics institute chess club,1 +the culture high,2 +dont wake me up,1 +transcaucasian mole vole,1 +harry zvi tabor,1 +vhs assault rifle,1 +playing possum,2 +omar minaya,2 +private university,1 +yuki togashi,3 +ski free,2 +say no more,1 +diving at the 1999 summer universiade,1 +armando sosa peña,1 +timur tekkal,1 +jura elektroapparate,1 +pornographic magazine,1 +tukur yusuf buratai,1 +keep on moving,1 +laboulbeniomycetes,1 +chiropractor solve problems,1 +mark s allen,3 +committees of the european parliament,4 +blondie,7 +veblungsnes,1 +bank vault,10 +smiling irish eyes,1 +robert kalina,2 +polarization ellipse,2 +huntingdon priory,1 +energy in the united kingdom,34 +hamble,1 +raja sikander zaman,1 +perigea hippia,1 +college of liberal arts and sciences,1 +bootblock,1 +nato reporting names,2 +the serpentwar saga,1 +reformed churches in the netherlands,1 +collaborative document review,4 +combat mission beyond overlord,3 +vlra,2 +pat st john,1 +oceanid,5 +itapetinga,1 +insane championship wrestling,9 +nathaniel gorham,1 +estadio metropolitano de fútbol de lara,2 +william of saint amour,2 +new york drama critics circle award,1 +alliant rq 6 outrider,2 +ilsan,1 +top model po russki,1 +woolens,1 +rutledge minnesota,1 +joigny coach crash,2 +zhou enlai the last perfect revolutionary,1 +the theoretical minimum,1 +arrow security,1 +john shelton wilder,2 +jasdf,2 +katie may,2 +american jewish military history project,1 +business professionals of america,1 +questioned document examination,5 +motorola a760,1 +american steel & wire,1 +louis armstrong at the crescendo vol 1,1 +edward vernon,3 +maria taipaleenmäki,1 +margical history tour,2 +jar jar,1 +australian oxford dictionary,2 +revenue service,2 +odoardo farnese hereditary prince of parma,1 +weekend in new england,1 +laurence harbor new jersey,2 +aramark tower,1 +stealers wheel,1 +cephalon,1 +dawnguard,1 +saintsbury,2 +saint fuscien,1 +ryoko kuninaka,1 +farm to market road 1535,1 +alan kennedy,2 +esteban casagolda,1 +shin angyo onshi,1 +william gowland,1 +eastern religions,6 +kenny lala,1 +alphonso davies,1 +tadamasa hayashi,1 +meet the parents,2 +calvinist church,1 +ristorante paradiso,1 +jose joaquim champalimaud,1 +olis,1 +mill hill school,2 +lockroy,1 +battle of princeton,10 +cent,8 +brough superior ss80,1 +ras al khaima club,3 +washington international university,3 +bradley kasal,2 +miguel Ángel varvello,1 +oxygen permeability,1 +femoral circumflex artery,1 +golden sun dark dawn,4 +pusarla sindhu,1 +toyota winglet,1 +wind profiler,1 +montefiore medical center,2 +template talk guitar hero series,3 +little leaf linden,1 +ramana,4 +islam in the czech republic,2 +manuel vitorino,1 +joseph radetzky von radetz,3 +francois damiens,1 +parasite fighter,1 +friday night at st andrews,3 +hurbazum,1 +haidhausen,1 +petabox,2 +salmonella enteritidis,2 +matthew r denver,1 +de la salle,1 +anti terrorism act 2015,6 +brugsen,1 +mountain times,1 +columbia basin project,1 +common wallaroo,2 +clepsis brunneograpta,1 +red hot + dance,1 +mao fumei,1 +dark shrew,1 +coach,8 +come saturday morning,1 +aanmai thavarael,1 +hellenia,1 +donate life america,2 +plot of beauty and the beast toronto musical,1 +births in 1243,3 +main page/wiki/portal technology,8 +cambridgeshire archives and local studies,1 +big pines california,1 +pegasus in popular culture,4 +baron glendonbrook,1 +your face sounds familiar,5 +boom tube,2 +richard gough,8 +the new beginning in niigata,3 +american academy of health physics,1 +plain,9 +tushino airfield,1 +king george v coronation medal,1 +geologic overpressure,1 +seille,1 +calorimeter,25 +french civil service,1 +david l paterson,1 +chinese gunboat chung shan,2 +rhizobium inoculants,1 +wizard,4 +baghestan,1 +paustian house,2 +ellen pompeo,55 +damien williams,1 +tomoe tamiyasu,1 +acute epithelial keratitis,1 +casey abrams,8 +mendozite,1 +kantian ethics,2 +mcclure syndicate,1 +tokyo metro,6 +cuisine of guinea bissau,1 +mossberg 500,18 +mollie gillen,1 +above and beyond party,1 +joey carbone,1 +faulkner state community college,1 +tetsuya ishikawa,1 +electric flag,3 +meet the feebles,2 +kplm,1 +when we were twenty one,1 +horus bird,2 +youth in revolt,8 +spongebob squarepants revenge of the flying dutchman,3 +ehow,5 +nikos xydakis,2 +ziprasidone,19 +ulsan airport,1 +flechtingen,1 +dave christian,3 +delaware national guard,1 +skaria thomas,1 +iraca,1 +kkhi,2 +swimming at the 2015 world aquatics championships – mens 1500 metre freestyle,2 +crossing lines,37 +john du cane,1 +i8,1 +bauer pottery,1 +affinity sutton,4 +lotus 119,1 +uss arleigh burke,1 +palmar interossei,2 +nofx discography,4 +bwia west indies airways,3 +gopala ii,1 +north fork correctional facility,1 +szeged 2011,1 +milligram per cent,2 +halas and batchelor,1 +what the day owes the night,1 +sighișoara medieval festival,5 +scarning railway station,1 +cambridge hospital,1 +amnesia labyrinth,2 +cokie roberts,7 +savings identity,3 +pravia,1 +mcgrath,4 +pakistan boy scouts association,1 +dan carpenter,2 +marikina–infanta highway,2 +genetic analysis,2 +template talk ohio state university,1 +thomas chamberlain,4 +moe book,1 +coyote waits,1 +black protestant,1 +neetu singh,19 +mahmoud sarsak,1 +casa loma,28 +bedivere,8 +boundary park,2 +danger danger,14 +jennifer coolidge,49 +pop ya collar,1 +collaboration with the axis powers during world war ii,10 +greenskeepers,1 +the dukes children,1 +alaska off road warriors,1 +twenty five satang coin,1 +template talk private equity investors,2 +american red cross,24 +jason shepherd,1 +georgetown college,2 +ocean countess,1 +ammonium magnesium phosphate,1 +community supported agriculture,5 +philosophy of suicide,4 +yard ramp,2 +captain germany,1 +bob klapisch,1 +i will never let you down,2 +february 11,6 +ron dennis,13 +rancid,16 +the mall blackburn,1 +south high school,6 +charles allen culberson,1 +organizational behavior,66 +automatic route selection,1 +uss the sullivans,9 +yo no creo en los hombres,1 +janet,1 +serena armstrong jones viscountess linley,3 +louisiana–lafayette ragin cajuns mens basketball,1 +flower films,1 +michelle ellsworth,1 +norbertine rite,2 +spanish mump,1 +shah jahan,67 +fraser coast region,1 +matt cornwell,1 +nra,1 +crested butte mountain resort,1 +college football playoff national championship,2 +craig heaney,4 +devil weed,1 +satsuki sho,1 +jordaan brown,1 +little annie,4 +thiha htet aung,1 +the disreputable history of frankie landau banks,1 +mickey lewis,1 +eldar nizamutdinov,1 +m1825 forage cap,1 +antonina makarova,1 +mopani district municipality,2 +al jahra sc,1 +chaim topol,4 +tum saath ho jab apne,1 +piff the magic dragon,7 +imagining argentina,1 +ni 62,1 +phys rev lett,1 +the peoples political party,1 +casoto,1 +popular movement of the revolution,4 +huntingtown maryland,1 +la bohème,33 +khirbat al jawfa,1 +lycksele zoo,1 +deveti krug,2 +cuba at the 2000 summer olympics,2 +rose wilson,7 +sammy lee,2 +dave sheridan,10 +universal records,2 +antiquities trade,3 +shoveller,1 +tapered integration,1 +parker pen company,4 +mushahid hussain syed,1 +nynehead,1 +counter reformation,2 +nhl on nbc,11 +ronny rosenthal,2 +arsenie todiraş,3 +lobster random,1 +halliburton,37 +gordon county georgia,1 +belle isle florida,3 +molly stanton,3 +green crombec,1 +geodesist,2 +abd al rahman al sufi,4 +demography of japan,26 +live xxx tv,5 +naihanchi,1 +cofinite,1 +msnbot,5 +clausard,1 +mimidae,1 +wind direction,15 +irrational winding of a torus,1 +tursiops truncatus,1 +trustee,1 +lumacaftor/ivacaftor,2 +balancing lake,2 +shoe trees,1 +cycling at the 1928 summer olympics – mens team pursuit,1 +calponia harrisonfordi,1 +hindu rate of growth,1 +dee gordon,7 +passion white flag,2 +frog skin,1 +rudolf eucken,2 +bayantal govisümber,1 +christopher a iannella,1 +robert myers,1 +james simons,1 +meng xuenong,1 +abayomi olonisakin,1 +milton wynants,1 +cincinnatus powell,1 +atomic bomb band,1 +hopfield network,12 +jet pocket top must,1 +the state of the world,1 +welf i duke of bavaria,2 +american civil liberties union v national security agency,3 +elizabeth fedde,1 +librarything,2 +kim fletcher,1 +tracy island,2 +praise song for the day,1 +superstar,7 +ewen spencer,1 +back striped weasel,1 +cs concordia chiajna,1 +bruce curry,1 +malificent,1 +dr b r ambedkar university,2 +river plate,1 +desha county arkansas,1 +harare declaration,2 +patrick dehornoy,1 +paul alan cox,2 +auckland mounted rifles regiment,1 +mikoyan gurevich dis,3 +corn exchange manchester,2 +sharpshooter,1 +the new york times manga best sellers of 2013,1 +max perutz,2 +andrei makolov,1 +inazuma eleven saikyō gundan Ōga shūrai,2 +tatra 816,1 +ashwin sanghi,8 +pipestone township michigan,1 +craig shoemaker,1 +david bateson,1 +lew lehr,1 +crewe to manchester line,2 +samurai champloo,36 +tali ploskov,2 +janet sobel,3 +kabe station,1 +rippon,1 +alexander iii equestrian,1 +louban,2 +the twelfth night,1 +delaware state forest,1 +the amazing race china 3,1 +brillouins theorem,1 +extreme north,3 +super frelon,1 +george watsons,1 +mungo park,1 +workin together,3 +boy,12 +brownsville toros,1 +kim lim,1 +futsal,63 +motoring taxation in the united kingdom,1 +accelerator physics codes,1 +arytenoid cartilage,3 +the price of beauty,3 +life on the murder scene,2 +hydrophysa psyllalis,1 +jürgen brandt,2 +economic history association,2 +the sandwich girl,1 +heber macmahon,1 +volume 1 sound magic,2 +san francisco–oakland–hayward ca metropolitan statistical area,9 +harriet green,7 +tarnawa kolonia,1 +eur1 movement certificate,20 +anna nolan,2 +gulf of gökova,1 +havertown,2 +orlando scandrick,4 +doug owston correctional centre,1 +asterionella,4 +espostoa,1 +ranked voting system,10 +commercial law,39 +kirk,1 +mongolian cuisine,8 +turfanosuchus,1 +arthur anderson,4 +sven olof lindholm,1 +batherton,1 +dimetrodon,1 +pianos become the teeth,1 +united kingdom in the eurovision song contest 1976,1 +medieval,11 +it bites,1 +ion television,8 +seaboard system railroad,3 +sayan mountains,3 +musaffah,1 +charles de foucauld,3 +urgh a music war,1 +translit,1 +american revolutionary war/article from the 1911 encyclopedia part 1,1 +uss mauna kea,1 +powder burn,1 +bald faced hornet,9 +producer of the year,1 +the most wanted man,1 +clear history,8 +mikael lilius,1 +class invariant,4 +forever michael,3 +goofing off,3 +tower viewer,3 +claudiu marin,1 +nicolas cage,1 +waol,2 +s10 nbc respirator,2 +education outreach,1 +gyeongsan,2 +template talk saints2008draftpicks,1 +botaurus,1 +francis harper,1 +mauritanian general election 1971,1 +kirsty roper,2 +non steroidal anti inflammatory drug,17 +nearchus of elea,2 +resistance to antiviral drugs,1 +raghavendra rajkumar,5 +template talk cc sa/sandbox,1 +washington gubernatorial election 2012,2 +paul lovens,1 +express freighters australia,2 +bunny bleu,2 +osaka prefecture,2 +federal reserve bank of boston,4 +hacı ahmet,1 +underground chapter 1,10 +filippo simeoni,2 +the wonderful wizard of oz,3 +sailing away,1 +avelino gomez memorial award,1 +badger,65 +hongkou football stadium,3 +benjamin f cheatham,2 +fair isaac,2 +kwab,1 +al hank aaron award,3 +gender in dutch grammar,1 +idiom neutral,2 +da lata,1 +tuu languages,1 +derivations are used,1 +clete patterson,1 +danish folklore,4 +android app //orgwikipedia/http/enmwikipediaorg/wiki/westfield academy,1 +toto,8 +ea,1 +victory bond tour,1 +credai,2 +hérin,1 +st james louisiana,1 +necrolestes,2 +cable knit,1 +saunderstown,1 +us route 52 in ohio,1 +sailors rest tennessee,1 +adlai stevenson i,6 +miscibility,13 +help footnotes,13 +murrell belanger,1 +new holland pennsylvania,5 +haldanodon,1 +feminine psychology,2 +riot city wrestling,1 +mobile content management system,2 +zinio,1 +central differencing scheme,2 +enoch,2 +usp florence admax,1 +maester aemon,7 +norman "lechero" st john,1 +ice racing,1 +tiger cub economies,6 +klaipėda region,12 +wu qian,8 +malayalam films of 1987,1 +estadio nuevo la victoria,1 +nanotoxicology,2 +hot revolver,1 +nives ivankovic,1 +glen edward rogers,5 +epicene,3 +eochaid ailtlethan,1 +judiciary of finland,1 +en jersey,1 +statc,1 +atta kim,1 +mizi research,2 +acs applied materials & interfaces,1 +thank god youre here,9 +loneliness,8 +h e b plus,2 +corella bohol,1 +money in the bank,59 +golden circle air t bird,1 +flash forward,1 +category talk philippine television series by network,1 +dfmda,1 +the road to wellville,8 +ernst tüscher,1 +commission,14 +abdul rahman bin faisal,6 +oversea chinese banking corporation,7 +ray malavasi,1 +al qadisiyah fc,4 +anisfield wolf book award,1 +jacques van rees,1 +jakki tha motamouth,1 +scoop,1 +piti,2 +carlos reyes,1 +v o chidambaram pillai,6 +diamonds sparkle,1 +the great transformation,5 +cardston alberta temple,1 +la vendetta,1 +miyota nagano,1 +national shrine of st elizabeth ann seton,2 +chaotic,1 +breastfeeding and hiv,1 +friedemann schulz von thun,1 +mukhammas,2 +fishbowl worldwide media,1 +mohamed amin,3 +john densmore,10 +suryadevara nayaks,1 +metal gear solid peace walker,12 +ché café,2 +old growth,1 +lake view cemetery,1 +konigsberg class cruiser,1 +courts of law,1 +nova scotia peninsula,3 +jairam ramesh,4 +portal kerala/introduction,1 +edinburgh 50 000 – the final push,1 +ludachristmas,3 +motion blur,1 +deliberative process privilege,2 +bubblegram,1 +simon breach grenade,2 +tess henley,1 +gojinjo daiko,1 +common support aircraft,2 +zelda rubinstein,9 +yolanda kakabadse,1 +american studio woodturning movement,1 +richard carpenter,67 +vehicle door,3 +transmission system operator,9 +christa campbell,9 +marolles en brie,1 +korsholma castle,1 +murder of annie le,3 +kims,1 +zionist union,8 +portal current events/june 2004,2 +marination,8 +cap haïtien international airport,2 +fujima kansai,1 +vampire weekend discography,3 +moncton coliseum,2 +wing chair,1 +el laco,2 +castle fraser,1 +template talk greek political parties,1 +society finch,1 +chief executive officer,4 +battle of bloody run,3 +coat of arms of tunisia,2 +nishi kawaguchi station,1 +colonoscopy,30 +vic tayback,5 +lonnie mack discography,3 +yusuf salman yusuf,2 +marco simone,4 +saint just,1 +elizabeth taylor filmography,6 +haglöfs,2 +yunis al astal,1 +daymond john,36 +bedd y cawr hillfort,1 +durjoy datta,1 +wealtheow,1 +aaron mceneff,1 +culture in berlin,1 +temple of saturn,6 +nermin zolotić,1 +the darwin awards,1 +patricio pérez,1 +chris levine,1 +misanthropic,1 +dragster,2 +eldar,19 +chrzanowo gmina szelków,1 +zimmerberg base tunnel,6 +jakob schaffner,1 +california gubernatorial recall election 2003,1 +tommy moe,1 +bikrami calendar,1 +mama said,11 +hellenic armed forces,8 +candy box,3 +monstervision,3 +kachin independent army,1 +pro choice,1 +tshiluba language,1 +trucial states,9 +collana,1 +best music video short form,1 +pokémon +giratina+and+the+sky+warrior,1 +etteldorf,1 +academic grading in chile,2 +land and liberty,3 +australian bureau of meteorology,1 +cheoin gu,1 +william henry green,1 +ewsd,2 +gate of hell,1 +sioux falls regional airport,3 +nevelj zsenit,1 +bevo lebourveau,1 +ranjana ami ar asbona,1 +shaun fleming,1 +jean antoine siméon fort,1 +sports book,1 +vedran smailović,3 +simple harmonic motion,29 +wikipedia talk wikiproject film/archive 16,1 +princess jasmine,13 +great bustard,5 +allred unit,1 +cheng san,1 +mini paceman,1 +flavoprotein,2 +storage wars canada,3 +university rowing,2 +category talk wikiproject saskatchewan communities,1 +the washington sun,1 +rotary dial,6 +hailar district,1 +assistant secretary of the air force,2 +the décoration for the yellow house,5 +chris mclennan,1 +the cincinnati kid,4 +education in the republic of ireland,15 +steve brodie,2 +country club of detroit,1 +wazner,1 +portal spain,4 +senna,3 +william j bernd house,1 +balaji baji rao,8 +worth dying for,1 +cool ruler,1 +turn your lights down low,2 +mavroudis bougaidis,1 +national registry emergency medical technician,1 +james young,8 +eyewire,1 +dark matters twisted but true/,1 +josé pascual monzo,1 +german election 1928,2 +linton vassell,1 +convention on the participation of foreigners in public life at local level,1 +thorium fuel cycle,5 +honeybaby honeybaby,1 +golestan palace,3 +lombok international airport,11 +mainichi daily news,1 +k&p,1 +liberal network for latin america,1 +cádiz memorial,1 +grupo corripio,1 +elie and earlsferry,1 +isidore geoffroy saint hilaire,1 +al salmiya sc,2 +piano sonata hob xvi/33,1 +e f bleiler,1 +national register of historic places listings in york county virginia,3 +gupta empire,2 +german immigration to the united states,1 +through gates of splendor,2 +iap,1 +love takes wing,1 +tours de merle,1 +aleksey zelensky,1 +paul almond,2 +boston cambridge quincy ma nh metropolitan statistical area,1 +komiks presents dragonna,1 +princess victoire of france,1 +alan pownall,3 +tilak nagar,2 +lg life sciences co ltd,8 +before their eyes,1 +labor right,5 +michiko to hatchin,1 +susan p graber,1 +xii,1 +hanswulf,1 +symbol rate,17 +myo18b,2 +rowing at the 2010 asian games – mens coxed eight,1 +caspar weinberger jr,2 +bettle juice,1 +battle of the morannon,7 +darlington county south carolina,1 +mayfield pennsylvania,1 +ruwerrupt de mad,1 +luthfi assyaukanie,1 +fiat panda,30 +wickiup reservoir,1 +tanabe–sugano diagram,6 +alexander sacher masoch prize,1 +intracellular transport,1 +church of the val de grâce,1 +jebel ad dair,1 +rosalind e krauss,6 +cross origin resource sharing,97 +readiness to sacrifice,1 +creel terrazas family,1 +phase portrait,9 +subepithelial connective tissue graft,1 +lake malawi,18 +phillips & drew,1 +ernst vom rath,2 +infinitus,1 +geneva convention for the amelioration of the condition of the wounded and sick in armies in the field,2 +world heritage,1 +dole whip,8 +leveling effect,1 +bioship,3 +vanilloids,2 +superionic conductor,1 +basil bernstein,7 +armin b cremers,2 +szlichtyngowa,1 +beixinqiao station,1 +united states presidential election in utah 1980,1 +watson v united states,3 +willie mcgill,1 +melle belgium,1 +al majmaah,1 +mesolimbic dopamine pathway,1 +six flags new england,5 +acp,2 +geostrategy,2 +original folk blues,1 +wentworth military academy,1 +bromodichloromethane,3 +doublet,4 +tawfiq al rabiah,1 +sergej jakirović,1 +mako surgical corp,3 +empire of lies,1 +old southwest,1 +bay of arguin,1 +bringing up buddy,1 +mustapha hadji,7 +raymond kopa,7 +evil horde,1 +kettering england,1 +extravaganza,1 +christian labour party,2 +joice mujuru,6 +v,15 +le père,4 +my fathers dragon,2 +cumulus cloud,32 +fantasy on themes from mozarts figaro and don giovanni,1 +postpone indefinitely,1 +extreme point,1 +iraq–israel relations,1 +henry le scrope 3rd baron scrope of masham,1 +rating beer,1 +claude alvin villee jr,2 +clackamas town center,2 +roope latvala,4 +richard bethell 1st baron westbury,1 +ryan gosling,1 +yelina salas,1 +amicus,1 +cecilia bowes lyon countess of strathmore and kinghorne,6 +programming style,9 +now and then,9 +somethingawful,1 +nuka hiva campaign,1 +bostongurka,2 +jorge luis ochoa vázquez,1 +philip burton,1 +rainbow fish,7 +road kill,5 +christiane frenette,2 +as if,1 +paul ricard,1 +roberto dañino,1 +shoyu,1 +jakarta,96 +dean keith simonton,1 +mastocytosis,19 +hiroko yakushimaru,3 +problem of other minds,2 +jaunutis,1 +tfp deficiency,1 +access atlantech edutainment,1 +kristian thulesen dahl,1 +william wei,1 +andy san dimas,10 +kempten/allgäu,1 +augustus caesar,9 +conrad janis,1 +tugaya lanao del sur,1 +second generation antipsychotics,1 +anema e core,2 +sucking the 70s,1 +the czars,2 +vakulabharanam,1 +f double sharp,3 +prymnesin,1 +dick bavetta,2 +billy jones,3 +columbine,4 +file talk joseph bidenjpg,1 +mandelbrot set,79 +constant elasticity of variance model,2 +morris method,1 +al shamal stadium,5 +hes alright,1 +madurai massacre,1 +philip kwon,2 +christadelphians,7 +this man is dangerous,2 +kiowa creek community church,1 +pier paolo vergerio,1 +order of the most holy annunciation,2 +john plender,1 +vallée de joux,2 +graysby,1 +ludwig minkus,3 +potato aphid,1 +bánh bột chiên,1 +wilhelmstraße,1 +fee waybill,1 +designed to sell,1 +ironfall invasion,2 +lieutenant governor of the isle of man,1 +third reading,2 +eleanor roosevelt high school,1 +su zhe,1 +heat conductivity,1 +si satchanalai national park,1 +etale space,1 +faq,24 +low carbohydrate diet,1 +differentiation of integrals,1 +karl fogel,2 +tom chapman,3 +james gamble rogers,2 +jeff rector,1 +burkut,9 +joe robinson,1 +turtle flambeau flowage,1 +moves like jagger,3 +turbaco,1 +oghuz turk,2 +latent human error,5 +square number,17 +rugby football league championship third division,2 +altoona pennsylvania,23 +circus tent,1 +satirical novel,1 +claoxylon,1 +barbaros class frigate,4 +oyer and terminer,2 +telephone numbers in the bahamas,1 +thomas c krajeski,2 +mv glenachulish,1 +sports broadcasting contracts in australia,3 +car audio,1 +ted lewis,2 +eric bogosian/robotstxt,2 +furman university japanese garden,1 +jed clampett,2 +flintstone,2 +c of tranquility,2 +rutali,2 +berkhamsted place,1 +wissam ben yedder,13 +nt5e,1 +erol onaran,1 +allium amplectens,1 +the three musketeers,2 +north eastern alberta junior b hockey league,1 +doggie daddy,1 +lauma,1 +the love racket,1 +eta hoffman,1 +ryans four,3 +omerta – city of gangsters,1 +humberview secondary school,2 +parels,1 +the descent,1 +evgenia linetskaya,1 +manhunt international 1994,1 +american society of animal science,1 +american samoa national rugby union team,1 +faster faster,1 +all creatures great and small,1 +mama said knock you out,9 +rozhdestveno memorial estate,2 +wizard of odd,1 +lugalbanda,4 +beardsley minnesota,1 +the rogue prince,10 +uss escambia,1 +stormy weather,3 +couleurs sur paris,1 +madrigal,4 +colin tibbett,1 +lemelson–mit prize,2 +phonetical singing,1 +glucophage,3 +suetonius,10 +ungra,1 +black and white minstrel,1 +woolwich west by election 1975,1 +trolleybuses in wellington,2 +jason macdonald,3 +ussr state prize,2 +robert m anderson,1 +kichijōji,1 +apache kid wilderness,1 +sneaky pete,8 +edward knight,1 +fabiano santacroce,1 +hemendra kumar ray,1 +sweat therapy,1 +stewart onan,2 +israel–turkey relations,1 +natalie krill,5 +clinoporus biporosus,1 +kosmos 2470,2 +vladislav sendecki,1 +healthcare in madagascar,1 +template talk 2010 european ryder cup team,1 +richard lyons,1 +transfer of undertakings regs 2006,3 +image processor,3 +alvin wyckoff,1 +kōbō abe,1 +kettle valley rail trail,1 +my baby just cares for me,3 +u28,1 +western australia police,10 +scincidae,1 +partitionism,1 +glenmorangie distillery tour,1 +river cave,1 +szilárd tóth,1 +i dont want nobody to give me nothing,1 +city,67 +annabel dover,2 +placebo discography,8 +showbiz,8 +solio ranch,1 +loan,191 +morgan james,10 +international federation of film critics,3 +the frankenstones,2 +pastor bonus,1 +billy purvis,1 +the gunfighters,1 +sandefjord,2 +ohio wine,2 +for the love of a man,1 +drifters,10 +ilhéus,1 +bikini frankenstein,1 +subterranean homesick alien,1 +chemical nomenclature,17 +great wicomico river,1 +ingrid caven,1 +japanese destroyer takanami,1 +nosler partition,1 +wagaman northern territory,1 +slovak presidential election 2019,1 +fuggerei,12 +al hibah,1 +irish war of independence,2 +joan smallwood,1 +anthony j celebrezze jr,1 +mercedes benz m130 engine,2 +phineas and ferb,2 +belgium womens national football team,3 +reynevan,1 +joe,1 +alan wilson,1 +epha3,1 +belarus national handball team,1 +phaedra,14 +move,2 +amateur rocketry,3 +epizootic hemorrhagic disease,5 +prague derby,4 +basilica of st thérèse lisieux,1 +pompeianus,1 +solved game,3 +tramacet,19 +essar energy,3 +lumbar stenosis,1 +part,24 +hải vân tunnel,1 +vsm group,3 +walter hooper,2 +consumer needs,1 +bell helicopter,18 +launde abbey,2 +ramune,10 +declarations of war during world war ii,1 +saint laurent de la salanque,1 +balkenbrij,1 +balgheim,1 +out of the box,13 +cappella,1 +national pharmaceutical pricing authority,4 +friend and foe,1 +new democracy,1 +eastern phoebe,2 +isipum of geumgwan gaya,1 +tel quel,1 +traveler,12 +superbeast,1 +oddsac,1 +zamora spain,1 +declaration of state sovereignty of the russian soviet federative socialist republic,1 +chumash painted cave state historic park california,3 +zentiva,1 +british rail class 88,5 +west indies cricket board,3 +pauli jørgensen,1 +punisher kills the marvel universe,7 +william de percy,1 +vehicle production group,4 +uc irvine anteaters mens volleyball,2 +dong sik yoon,1 +hyæna,2 +canadian industries limited,1 +mr ii,1 +jim muhwezi,1 +citizen jane,2 +night and day concert,1 +double precision floating point format,2 +herbal liqueurs,1 +the fixed period,5 +pip/taz,1 +lesser caucasus,2 +uragasmanhandiya,2 +alternative words for british,2 +khuzaima qutbuddin,1 +helmut balderis,2 +wesley r edens,1 +scott sassa,4 +mutant mudds,3 +east krotz springs louisiana,1 +leonard frey,3 +counting sort,15 +leandro gonzález pírez,2 +shula marks,1 +sierville,1 +california commission on teacher credentialing,1 +raymond loewy,10 +beevor foundry,1 +dog snapper,2 +hitman contracts,5 +eduard herzog,1 +wittard nemesis of ragnarok,1 +cape may light,1 +al saunders,3 +distant earth,2 +beam of light,2 +arent we all?,1 +veridicality,1 +private enterprise,3 +rambhadracharya,3 +dps,5 +beckdorf,1 +rúaidhrí de valera,1 +vivian bang,3 +sugar pine,1 +vn parameswaran pillai,1 +henry ross perot sr,1 +the arcadian,1 +the record,6 +g turner howard iii,1 +oleksandr usyk,12 +mumbai suburban district,5 +vicente dutra,1 +paean,1 +scottish piping society of london,1 +ingot,11 +alex obrien,6 +autonomous counties of china,1 +kaleorid,1 +remix & repent,3 +gender performativity,7 +godheadsilo,1 +tonsilloliths,1 +la dawri,1 +kiran more,3 +billboard music award for woman of the year,1 +tahitian ukulele,1 +buick lacrosse,14 +draft helen milner jury sent home for the night,2 +history of japanese cuisine,6 +time tunnel,1 +albert odyssey 2,1 +oysters rockefeller,4 +jim mahon,1 +evolutionary invasion analysis,1 +sunk cost fallacy,3 +universidad de manila,1 +morgan crucible,1 +southern miss golden eagles football,2 +horatio alger,13 +biological psychopathology,1 +hollywood,115 +product manager,21 +thomas burgh 3rd baron burgh,1 +stan hack,1 +peloponesian war,1 +republic of china presidential election 2004,2 +sanitarium,4 +growthgate,1 +samuel e anderson,1 +bobo faulkner,1 +kaffebrenneriet,1 +monponsett pond seaplane base,1 +powers of horror,3 +viburnum burkwoodii,1 +new suez canal,5 +gerardo ortíz,2 +japhia life,1 +paul pastur,1 +fuller craft museum,1 +nomal valley,1 +inaugural address,1 +saint Étienne du vigan,1 +lip ribbon microphone,2 +mary cheney,2 +piebald,6 +kadambas,1 +transportation in omaha,7 +before the league,1 +feltham and heston by election 2011,1 +aboriginal music of canada,3 +dnssec,6 +sshtunnels,1 +robin benway,1 +swimming at the 1968 summer olympics – mens 4 x 200 metre freestyle relay,1 +commission internationale permanente pour lepreuve des armes à feu portatives,3 +death rock,1 +hugo junkers,6 +gmt,3 +keanu reeves,2 +beverly kansas,1 +charlotte blair parker,1 +kids,5 +weight bench,1 +kiasmos,8 +basque country autonomous basketball team,1 +gideon toury,2 +gugak/,1 +texass 32nd congressional district,2 +have you ever been lonely,1 +take the weather with you,1 +chukchi,1 +the magicians wife,1 +juan manuel bordeu,1 +port gaverne,1 +music for films iii,1 +northern edo masquerades,1 +hang gliding,15 +marine corps logistics base barstow,2 +century iii mall,1 +peter tarlow,1 +thermal hall effect,1 +david ogden stiers,18 +webmonkey,1 +five cereals,2 +osceola washington,1 +clover virginia,2 +sphinginae,2 +stuart brace,1 +al di meola discography,7 +sunflowers,1 +hasty generalization,4 +polish athletic association,1 +the purge 3,2 +bitetti combat mma 4,1 +hiroko nagata,2 +mona seilitz,1 +mixed member proportional representation,7 +rancho temecula,2 +sinai,1 +norrmalmstorg robbery,5 +silesian walls,1 +floyd stahl,1 +gary becker,1 +knowledge engineering,5 +port of mobile,1 +luckiest girl alive,2 +ilya rabinovich,1 +bridge,3 +el general,3 +cornerstone schools,1 +gozmo,1 +charles courtney curran,1 +broker,32 +us senate committee on banking housing and urban affairs,2 +retroversion of the sovereignty to the people,1 +giorgi baramidze,1 +lars grael,1 +abdul qadir,3 +pgrep,2 +category talk seasons in danish womens football,1 +malus sieversii,1 +god squad,4 +category of acts,1 +melkote,1 +linda langston,1 +sherry romanado,1 +montana sky,8 +history of burkina faso,1 +iso 639 kxu,1 +los angeles fire department museum and memorial,1 +recognize,1 +der bewegte mann,6 +davy pröpper,1 +outline of vehicles,2 +gesta francorum,1 +sidney w pink,1 +ronald pierce,1 +martin munkácsi,1 +nord noreg,1 +accounting rate of return,7 +urwerk,1 +albert gallo,1 +antennaria dioica,3 +transport in sudan,2 +fladry,1 +cumayeri,1 +bennington college,11 +pêro de alenquer,2 +sixth man,1 +william i of aquitaine,1 +radisson diamond,1 +belgian united nations command,1 +venus genetrix,1 +sayesha saigal,14 +inverse dynamics,2 +national constitutional assembly,1 +honey bear,4 +certosa di pavia,2 +selective breeding,31 +let your conscience be your guide,1 +han hyun jun,1 +closed loop,8 +template talk golf major championships master,1 +twin oaks community virginia,1 +red flag,3 +housing authority of new orleans,2 +joice heth,4 +toñito,1 +ivan pavlov,2 +madanapalle,4 +ptat,1 +renger van der zande,1 +anaerobic metabolism,2 +patrick osullivan,1 +shirakoya okuma,1 +permian high school,9 +thomas h ford,1 +southfield high school,1 +religion in kuwait,2 +nathrop colorado,1 +hefner hugh m,1 +whitney bashor,1 +pope shenouda iii of alexandria,7 +thomas henderson,1 +tokka and rahzar,13 +windows thumbnail cache,3 +consumer council for water,1 +sake bombs and happy endings,1 +lothlórien,1 +the space bar,4 +sakuma rail park,1 +oas albay,3 +dan frankel,1 +cliff hillegass,1 +iron sky,12 +pentile matrix family,1 +oregon system,1 +california sea lion,7 +jeanneau,2 +meadowhall interchange,1 +lille catholic university,1 +nuñomoral,1 +vending machine,30 +xarelto,1 +jonbenét ramsey,3 +progresso castelmaggiore,1 +tacticity,6 +wing arms,1 +gag,2 +hank greenberg,8 +garda síochána,14 +puggy,1 +p sainath,1 +the year of living dangerously,9 +army reserve components overseas training ribbon,1 +hmas nestor,1 +john beckwith,1 +florida constitution,2 +yonne,3 +benoît richaud,1 +mamilla pool,2 +gerald bull,14 +david halberstam,12 +my fair son,2 +ncaa division iii womens golf championships,1 +anniela,1 +king county,1 +kamil jankovský,1 +synaptic,3 +rab,6 +switched mode regulator,1 +history of biochemistry,1 +halaf,2 +henry colley,1 +co postcode area,3 +social finance uk,1 +cercospora,2 +the dao,1 +unité radicale,2 +shinji hashimoto,3 +tommy remengesau,3 +isobel gowdie,2 +mys prasad,9 +national palace museum of korea,1 +basílica del salvador,2 +no stone unturned,2 +walton group,1 +foramen ovale,1 +slavic neopaganism,1 +iowa county wisconsin,3 +melodi grand prix junior,1 +jarndyce and jarndyce,3 +talagunda,1 +nicholas of autrecourt,1 +substitution box,3 +the power of the daleks,1 +real gas,6 +edward w hincks,1 +kangxi dictionary,5 +natural world,1 +h h asquith,21 +francis steegmuller,1 +sasha roiz,3 +media manipulation,1 +looking for comedy in the muslim world,2 +bytown,4 +previsualization,1 +rita ora discography,11 +kiersey oklahoma,1 +henry greville 3rd earl of warwick,1 +draft,4 +phenolate,1 +i believe,1 +virologist,1 +relief in abstract,1 +eastern medical college,1 +purveyance,2 +ascending to infinity,2 +sportstime ohio,2 +church of wells,1 +ivory joe hunter,1 +wayne mcgregor,2 +luna 17,4 +viscount portman,2 +wikipedia talk wikipedia signpost/2009 07 27/technology report,1 +negramaro,1 +barking owl,2 +i need you,2 +brockway mountain drive,1 +template talk albatros aircraft,1 +future shock,11 +china national highway 317,1 +laurent gbagbo,7 +plum pudding model,18 +league of the rural people of finland,1 +dundees rising,1 +nikon f55,1 +olympic deaths,5 +gemma jones,19 +hafsa bint al hajj al rukuniyya,1 +personal child health record,1 +logic in computer science,11 +bhyve,3 +hothouse,1 +log house,6 +library of celsus,2 +the lizzie bennet diaries,1 +leave this town the b sides ep,1 +estimated time of arrival,8 +chariotry in ancient egypt,2 +american precision museum,1 +dimos moutsis,1 +scriptlet,1 +something in the wind,1 +sharka blue,1 +time on the cross the economics of american negro slavery,1 +tomislav kiš,1 +khalid islambouli,7 +bankruptcy abuse prevention and consumer protection act,7 +gračanica bosnia and herzegovina,2 +jungs theory of neurosis,5 +mgm animation,1 +soviet support for iran during the iran–iraq war,3 +native american,1 +template talk nigeria squad 1994 fifa world cup,1 +norwegian lutheran church,4 +adia barnes,1 +coatings,1 +mehdi hajizadeh,1 +the dead matter cemetery gates,1 +fuzzy little creatures,1 +waje,7 +anji,1 +heinz haber,1 +turkish albums chart,1 +sebastian steinberg,1 +price fixing cases,2 +bellator 48,1 +edgar r champlin,1 +otto hermann leopold heckmann,1 +bishops stortford fc,4 +stern–volmer relationship,6 +morgan quitno,2 +five star general,1 +iso 13406 2,1 +black prince,11 +leopard kung fu,1 +felix wong,5 +mary claire king,6 +alvar lidell,1 +playonline,1 +infantry branch,1 +andrew pattison,1 +john turmel,1 +kent,74 +edwin palmer hoyt,1 +captivity narratives,1 +jaguar xj220,1 +hms tanatside,2 +new faces,2 +edward levy lawson 1st baron burnham,1 +samuel woodfill,3 +jewish partisans,9 +abandonware,16 +early islamic philosophy,2 +sleeper cell,5 +media of africa,2 +san andreas,3 +luxuria,2 +egon hostovský,3 +pelagibacteraceae,1 +martin william currie,1 +borescope,21 +narratives of islamic origins the beginnings of islamic historical writing,1 +lecompton constitution,2 +axé bahia,2 +paul goodman,1 +template talk washington nationals roster navbox,1 +a saucerful of secrets,2 +david carol macdonnell mather,1 +portal buddhism,3 +florestópolis,1 +alecs+golf+ab,1 +bank alfalah,1 +frank pellegrino,3 +loutre,1 +erp4it,2 +monument to joe louis,2 +witch trial of nogaredo,1 +sabrina santiago,2 +no night so long,3 +helena carter,1 +renya mutaguchi,3 +yo yogi,4 +bolivarian alliance for the americas,3 +cooper boone,1 +uss iowa,24 +mitsuo iso,2 +cranberry,1 +batrachotomus,1 +richard lester,5 +bermudo pérez de traba,1 +rosser reeves ruby,1 +telecommunications in morocco,4 +i a richards,1 +nidhal guessoum,1 +lilliefors test,6 +the silenced,5 +mambilla plateau,1 +sociology of health and illness,3 +tereza chlebovská,2 +bismoll,3 +kim suna,1 +scream of the demon lover,1 +joan van ark,7 +intended nationally determined contributions,6 +dietary supplement,16 +last chance mining museum,1 +savoia marchetti s65,1 +if i can dream,1 +maharet and mekare,4 +nea anchialos national airport,2 +american journal of digestive diseases,1 +chance,2 +lockheed f 94c starfire,1 +the game game,1 +kuzey güney,3 +semmering base tunnel,1 +three mile island,1 +evaluation function,1 +robert mckee,4 +carmelo soria,1 +moneta nova,1 +pīnyīn,1 +international submarine band,3 +elections in the bahamas,5 +powell alabama,1 +kmgv,1 +charles stuart duke of kendal,2 +echo and narcissus,7 +trencrom hill,1 +ashwini dutt,1 +the herzegovina museum,1 +liverpool fc–manchester united fc rivalry,12 +kerber,1 +flakpanzer 38,8 +demographics of bihar,2 +rico reeds,1 +vandenberg afb space launch complex 3,1 +wiesendangen,1 +lamm,1 +allen doyle,2 +anusree,5 +broad spectrum,1 +bay middleton,2 +connect savannah,1 +history of immigration to canada,22 +waco fm,3 +nakano takeko,1 +murnau am staffelsee,2 +minarchy,1 +haymans dwarf epauletted fruit bat,1 +brachyglottis repanda,1 +associative,1 +mississippi aerial river transit,1 +stefano siragusa,2 +gregor the overlander,3 +marine raider,1 +pogorzans,1 +sportcity,2 +garancahua creek,1 +vincent dimartino,3 +ninja,2 +natural history museum of bern,1 +revolutionary catalonia,4 +chiayi,1 +alix strachey,3 +looe island,1 +college football usa 96,1 +off peak return,1 +minsk 1 airport,1 +evangelical lutheran church in burma,2 +riemann–roch theorem,1 +the comic strip,2 +vladimir istomin,1 +america again,2 +brown treecreeper,1 +american high school,1 +powerglide,2 +oolitic limestone,1 +daz1,1 +jarrow vikings,1 +pierre philippe thomire,1 +dorothy cadman,1 +gaston palewski,3 +twin river bridges,1 +im yours,1 +ambrose dudley 3rd earl of warwick,3 +ssim,2 +original hits,1 +cosmonaut,9 +special educational needs and disability act 2001,4 +will you speak this word,1 +history of wolverhampton wanderers fc,1 +don lawrence,1 +tokyo metropolitan museum of photography,1 +orduspor,1 +john lukacs,3 +patrice collazo,1 +lords resistance army insurgency,5 +ronald "slim" williams,5 +drivin for linemen 200,1 +nicolò da ponte,1 +bucky pope,1 +ewing miles brown,2 +ugly kid joe,28 +american flight 11,1 +louzouer,1 +district hospital agra,1 +jessica jane applegate,1 +sexuality educators,1 +serie a scandal of 2006,1 +at war with reality,1 +stephen wiltshire,13 +vechigen switzerland,1 +rikki clarke,3 +rayakottai,1 +permanent magnet electric motor,1 +qazi imdadul haq,1 +plywood,49 +ntr telugu desam party,1 +skin lightening,1 +royal natal national park,1 +uss mcdougal,2 +queen of the sun,1 +karanjachromene,1 +on 90,1 +enrique márquez,1 +siegfried and roy,1 +city manager,6 +wrdg,1 +why i am not a christian,3 +protein coding region,1 +royal bank of queensland gympie,1 +british invasions of the river plate,2 +yasufumi nakanoue,1 +magnetic man,1 +kickback,3 +tillandsia subg allardtia,1 +north american nr 349,1 +edict of amboise,1 +st andrew square edinburgh,2 +flag of washington,2 +timeless,2 +new york state route 125,3 +fudge,3 +single entry bookkeeping system,5 +refractive surgery,8 +bi monthly,1 +park high school stanmore,1 +norton anthology of english literature,1 +michael wines,1 +gaff rig,1 +kosmos 1793,1 +major facilitator superfamily,2 +talpur dynasty,1 +byron bradfute,1 +quercitello,1 +rcmp national protective security program,1 +ann kobayashi,1 +recurring saturday night live characters and sketches,3 +abraham hill,1 +nagapattinam district,4 +pidgeon,3 +mycalessos,1 +technical university of hamburg,1 +electric shock&ei=ahp0tbk0emvo gbe v2bbw&sa=x&oi=translate&ct=result&resnum=2&ved=0ceaq7gewaq&prev=/search?q=electric+shock&hl=da&biw=1024&bih=618&prmd=ivns,2 +aim 54 phoenix,18 +undercut,5 +gokhale memorial girls college,1 +digital penetration,19 +centre for peace studies tromsø,1 +richie williams,1 +walloon region,1 +albany city hall,2 +maxine carr,4 +anglosphere,18 +effect of world war i on children in the united states,1 +josh bell,1 +german thaya,1 +brian murphy,3 +marguerite countess of blessington,1 +leak,1 +bubble point,5 +international federation of human rights,1 +clubcorp,2 +greater philadelphia,1 +daniel albright,1 +macas,1 +roses,4 +woleu ntem,1 +shades of blue,1 +say aah,2 +curtiss sbc,1 +ion andone,1 +firstborn,1 +marringarr language,2 +ann e todd,1 +native american day,4 +stand my ground,1 +bavington,1 +classification of indigenous peoples of the americas,2 +always,6 +leola south dakota,1 +psycilicibin,2 +roy rogers,1 +marmalade,1 +national prize of the gdr,1 +shilp guru,1 +m2 e 50,1 +jorge majfud,2 +cutter and bone,1 +william steeves,1 +lisa swerling,2 +grace quigley,5 +telecommunications in yemen,1 +rarotonga international airport,7 +cycling at the 2010 central american and caribbean games,2 +mazda b3000,1 +hanwencun,1 +adurfrazgird,1 +ivan ivanov vano,1 +yhwh,1 +qarshi,4 +oshibori,2 +uppada,1 +iain clough,1 +painted desert,7 +tugzip,1 +my little pony fighting is magic,143 +pantheon,2 +chinese people in zambia,1 +yves saint laurent,3 +texas helicopter m79t jet wasp ii,1 +forever reign,1 +charlotte crosby,32 +ealdormen,9 +copper phosphate,2 +mean absolute difference,5 +hôtel de soubise,5 +josh rees,2 +non commissioned officer,70 +gb jones,1 +im feeling you,2 +book of shadows,9 +brain trauma,1 +sulpitius verulanus,1 +vikranth,5 +space adaptation syndrome,6 +united states presidential election in hawaii 1988,1 +joe garner,4 +river suir bridge,2 +the beach boys medley,1 +joyce castle,1 +christophe wargnier,1 +ik people,2 +sketch show,1 +buena vista police department,1 +file talk layzie bone clevelandjpg,1 +gillian osullivan,3 +prince albert of saxe coburg and gotha,2 +berean academy,1 +motorcraft quality parts 500,1 +frederick law olmsted,21 +born this way,9 +sterling virginia,4 +if wishes were horses beggars would ride,1 +section mark,1 +tapi,1 +navy cross,1 +housekeeper,1 +gian battista marino,1 +planá,1 +chiromantes haematocheir,1 +colonial life & accident insurance company,4 +aduana building,2 +kim johnston ulrich,1 +berkelium 254,1 +m&t bank corp,2 +sit up,1 +sheknows,1 +phantom lady,1 +bruce kamsinky,1 +commercial drive,1 +chinese people in the netherlands,1 +sylvia young theatre school,4 +influenza a virus subtype h2n3,1 +dracut,2 +nate webster,1 +vila velebita,1 +uaz patriot,4 +democratic unification party,1 +alexander slidell mackenzie,1 +portland mulino airport,1 +first person shooter,2 +the temporary widow,1 +terry austin,1 +the foremans treachery,1 +hms blenheim,1 +sodium dichloro s triazinetrione,1 +kurt becher,1 +cumberland gap tn,1 +newton cotes,1 +daphne guinness,6 +internal tide,1 +god and gender in hinduism,2 +howlin for you,1 +stellarator,14 +cavea,3 +faye ginsburg,1 +lady cop,3 +template talk yugoslavia squad 1986 fiba world championship,1 +solidarity economy,1 +second presidency of carlos andrés pérez,1 +bora bora,71 +xfs,1 +christina bonde,1 +agriculture in australia,20 +scenic drive,1 +richard mantell,1 +motordrome,1 +broadview hawks,1 +misty,2 +international bank of commerce,2 +istanbul sapphire,5 +changkat keruing,1 +the hotel inspector unseen,1 +tharwa australian capital territory,2 +strauss,2 +shock film,1 +ulick burke 1st marquess of clanricarde,2 +valencia cathedral,5 +kay bojesen,1 +palogneux,1 +texas beltway 8,1 +jackie walorski,7 +capital punishment in montana,1 +byte pair encoding,2 +upper deerfield township new jersey,2 +lucca comics & games,1 +lee chae young,1 +czar alexander ii,1 +kool ad,6 +leopold van limburg stirum,1 +john dunn,1 +policeman,2 +what dreams may come,3 +grant ginder,1 +chieverfueil,2 +long island express,1 +malmö sweden,2 +song for my father,1 +see saw,2 +jean jacques françois le barbier,5 +do rag,11 +dsb bank,2 +davical,6 +cervical cap,1 +gershon yankelewitz,1 +the last hurrah,4 +category talk educational institutions established in 1906,1 +tour pleyel,1 +león klimovsky,1 +phyoe phyoe aung,1 +phil sawyer,2 +android app //orgwikipedia/http/enmwikipediaorg/wiki/swiftkey,1 +deontological,3 +juan dixon,12 +robert pine,4 +alexander tilloch galt,2 +common tailorbird,12 +derailed,7 +mike campbell,3 +terminator 2 3 d battle across time,3 +technische universität münchen,4 +baloana,1 +echis leucogaster,1 +lahore pigeon,1 +william de beauchamp 9th earl of warwick,2 +erin go bragh,14 +economics u$a,1 +villafranca montes de oca,1 +pope eusebius,2 +martin kruskal,1 +félix de blochausen,1 +jeff jacoby,1 +mark krein,2 +travis wester,2 +fort louis de la louisiane,1 +weddingwire,2 +ping,54 +don swayze,8 +steve hamilton,3 +rhenish,1 +winrar,3 +births in 1561,4 +copyright law of the netherlands,2 +floodland,9 +tamil nadu tourism development corporation,1 +dolls house,1 +chkrootkit,1 +search for the hero,1 +avenal,1 +tini,2 +patamona,1 +aspendos international opera and ballet festival,2 +felix cora jr,5 +yellow cardinal,2 +antony jay,1 +conda,1 +a tramp shining,1 +william miller,1 +holomictic lake,2 +growler,2 +the violence of summer,1 +meerschaum,3 +cd138,1 +karl friedrich may,1 +history of iraq,2 +henry ford,139 +rumwold,1 +beatrice di tenda,1 +blaze,1 +nick corfield,1 +walt longmire,5 +eleazar maccabeus,1 +business edition,1 +karl oyston,4 +gypsy beats and balkan bangers,1 +fa premier league 2004 05,1 +agawan radar bomb scoring site,1 +the hall of the dead,1 +combat training centre,1 +moroccan portuguese conflicts,2 +pokipsy,1 +minor characters in csi crime scene investigation,1 +miguel molina,1 +buckypaper,2 +magazine,4 +forget about it,2 +marco schällibaum,1 +r d smith,1 +nfl playoff results,2 +four score,1 +centenary bank,2 +london borough of camden,12 +bhumij,1 +counter reformation/trackback/,1 +billy volek,1 +cover song,1 +awang bay,1 +douglas fitzgerald dowd,3 +architecture of ancient greece,5 +ny1,2 +academy award for best visual effects,3 +history of the mbta,2 +triangle group,1 +charles r fenwick,1 +berenice i of egypt,1 +window detector,1 +corruption perception index,1 +leffrinckoucke,1 +lee anna clark,1 +burndy,2 +inset day,2 +american association of motor vehicle administrators,1 +ckm matrix,1 +angiopoietin 1,1 +steven marsh,1 +open reading frame,27 +telesystems,1 +pastoral poetry,1 +west wycombe park,2 +lithium,7 +nogales international airport,1 +wajków,1 +sls 1,1 +trillo,2 +max s,1 +verndale,1 +yes sir i can boogie,1 +blog spam,10 +daniel veyt,1 +william brown,3 +takami yoshimoto,1 +josh greenberg,4 +geoffrey heyworth 1st baron heyworth,1 +medeina,3 +anja steinlechner,1 +riviera beach florida,2 +gerris wilkinson,1 +north american lutheran church,1 +paul dillett,11 +proto euphratean language,1 +best selling books,2 +pumpellyite,1 +business objects,1 +fodor,2 +xanadu,3 +london river,1 +draft juan de orduña,2 +barriemore barlow,3 +jew harp,1 +birmingham,1 +titus davis,1 +march 2012 gaza–israel clashes,1 +energy demand management,2 +aquarium of the americas,3 +tto,1 +l h c tippett,1 +optical fiber,88 +onești,2 +stanley ntagali,1 +prussian blue,1 +bill kovach,2 +hip pointer,3 +alessandra amoroso,4 +fleet racing,1 +navy maryland rivalry,1 +cornering force,1 +the mighty quest for epic loot,5 +katalyst,2 +the beef seeds,1 +shack out on 101,1 +aircraft carrier operations,1 +overseas province,2 +institute of state and law,1 +light truck,5 +plastics in the construction industry,2 +little zizou,2 +congenic,2 +adriaen van utrecht,1 +brian mcgrath,3 +parvati,1 +jason gwynne,1 +kphp,1 +miryusif mirbabayev,1 +kōriyama castle,3 +the making of a legend gone with the wind,2 +shot traps,1 +awa tag team championship,1 +littlebourne,2 +franchot tone,4 +john dudley 2nd earl of warwick,2 +mass spec,1 +final fantasy vi,44 +gerry ellis,1 +adon olam,3 +man 24310,1 +p n okeke ojiudu,1 +unqi,1 +snom,1 +bruce bagemihl,1 +category talk animals described in 1932,1 +metalist oblast sports complex,1 +colley harman scotland,1 +suka,1 +anita sarkeesian,81 +kazakhstan national under 17 football team,1 +ym,2 +matt barnes,1 +tour phare,1 +bellus–claisen rearrangement,2 +turkey at the 2012 summer olympics,1 +irréversible,32 +umbilical nonseverance,1 +wood stave,1 +indian pentecostal church of god,1 +camponotus nearcticus,3 +john tesh,13 +syncline,4 +skins,50 +kelsey manitoba,1 +alkayida,2 +polyglotism,17 +forensic statistics,2 +ram vilas sharma,8 +pearl jam,71 +dj max fever,1 +islamic view of miracles,5 +kds,1 +alabama cavefish,1 +johanna drucker,1 +tom wolk,4 +rottenburg,2 +goshen connecticut,2 +maker media,1 +morphett street adelaide,1 +keystone hotel,1 +baseball hall of fame balloting 2005,1 +gongzhuling south railway station,1 +ss charles bulfinch,1 +sig mkmo,1 +cartman finds love,2 +embassy of syria in washington dc,1 +charles prince of wales,175 +teachings of the prophet joseph smith,1 +charles iv,1 +alethea steven,1 +type i rifle,2 +a peter bailey,1 +brain cancer,1 +eric l clay,2 +jett bandy,1 +moro rebellion,9 +eustachów,1 +avianca el salvador,2 +dont stop the party,4 +reciprocal function,1 +dagmar damková,1 +hautmont,1 +penguin english dictionary,2 +waddie mitchell,1 +technician fourth grade,3 +hot girls in love,1 +critérium du dauphiné,59 +love song,2 +roger ii,2 +whitbread book award,1 +thomas colepeper 2nd baron colepeper,2 +a king and no king,1 +big fish & begonia,5 +mayville new york,2 +molecularity,1 +ed romero,1 +one watt initiative,3 +jeremy hellickson,2 +william morgan,1 +giammario piscitella,1 +eastern lesser bamboo lemur,1 +padre abad district,1 +don brodie,1 +facts on the ground,1 +undeniable evolution and the science of creation,1 +john of giscala,1 +bryce harper,45 +gabriela irimia,1 +empire earth mobile,1 +the queen vic,1 +helen rowland,1 +mixed nuts,5 +malacosteus niger,2 +george r r martin/a song of ice and fire,1 +brock osweiler,11 +tough,1 +outline of agriculture,4 +sea wolf,1 +mo vaughn,4 +the brood of erys,1 +composite unit training exercise,1 +isabella acres,4 +the jersey,5 +coal creek bridge,1 +habana libre,1 +nicole pulliam,1 +john shortland,1 +daniel pollen,1 +magic kit,1 +baruch adonai l&,1 +a daughters a daughter,2 +laughlin nevada,11 +tubercule,1 +louis laurie,1 +internet boom,3 +conversion of paul,1 +comparison of software calculators,1 +choctaw freedmen,2 +josh eady,1 +hôpital charles lemoyne,2 +u mobile,2 +john tomlinson,1 +baré esporte clube,2 +tuğçe güder,2 +highams park railway station,4 +newport east,1 +clothing industry,6 +scott rosenberg,6 +my 5 wives,2 +matt godfrey,1 +port ellen,2 +winecoff hotel fire,1 +fide world chess championship 2005,2 +lara piper,1 +the little mermaid,1 +foxmail,6 +penn lyon homes,1 +stockholm opera,1 +american journal of theology,1 +bernard gorcey,3 +rodger collins,1 +clarkeulia sepiaria,1 +korean era name,3 +melide ticino,1 +unknown to no one,1 +asilinae,1 +scânteia train accident,1 +parti de la liberté et de la justice sociale,1 +falkland islands sovereignty dispute,13 +castile,10 +french battleship flandre,1 +nils taube,1 +anisa haghdadi,1 +william tell told again,2 +magister,3 +zgc 7,1 +national agricultural cooperative marketing federation of india,3 +les bingaman,1 +chebfun,1 +portal current events/august 2014,2 +eparchy of oradea mare,1 +tempo and mode in evolution,2 +seili,1 +boniface,3 +supportersvereniging ajax,1 +support team,1 +lactometer,1 +twice as sweet,1 +spruce pine mining district,2 +banknotes of the east african shilling,1 +cerebral cortex,3 +tagalogs,1 +german diaspora,8 +grammelot,1 +max a,1 +category talk vienna culture,1 +cheung kong graduate school of business,1 +three certainties,1 +multani,3 +barry callebaut,15 +joanne mcneil,1 +z grill,4 +commonwealth of australia constitution act 1900,1 +ganzorigiin mandakhnaran,1 +peter h schultz,1 +ea pga tour,3 +scars & memories,1 +exodus from lydda,1 +states reorganisation act 1956,4 +guy brown,1 +horsebridge,1 +arthur mafokate,1 +aldus manutius,5 +american daylight,3 +jean chaufourier,2 +edmond de caillou,1 +hms iron duke,9 +displeased records,1 +quantum turing machine,3 +ncert textbook controversies,2 +dracs,1 +beyrouth governorate,1 +staphylococcus caprae,1 +tankard,2 +surfaid international,1 +hohenthurn,2 +mission x 41,1 +professional wrestling hall of fame,2 +george mountbatten 4th marquess of milford haven,2 +athletics at the 2012 summer paralympics womens club throw f31 32/51,1 +knots and crosses,1 +edge vector,1 +philippe arthuys,1 +baron raglan,1 +odell beckham jr,3 +elfriede geiringer,1 +hyflux,1 +author level metrics,2 +ieee fellow,1 +pori brigade,3 +polyphenol antioxidant,1 +the brothers,8 +kakaji Ōita,1 +shyam srinivasan,2 +shahid kapoor,88 +chuckie williams,1 +colonial,4 +roman spain,1 +convolvulus pluricaulis,1 +william j burns international detective agency,1 +accessibility for ontarians with disabilities act 2005,1 +linguist,1 +agonist,2 +xiaozi,1 +holker hall,1 +novatium,1 +alois jirásek,1 +lesser crested tern,1 +names of european cities in different languages z,1 +hydrogen cooled turbogenerator,2 +indian airlines flight 257,1 +united states attorney for the northern district of indiana,1 +this is us,11 +transaction capabilities application part,1 +culiacán,6 +hash based message authentication code,65 +heinz murach,1 +dual citizen,2 +zhizn’ za tsarya,1 +gabriel taborin technical school foundation inc,1 +deaths in july 1999,1 +aponi vi arizona,1 +amish in the city,2 +goodbye cruel world,1 +st augustine grass,10 +moesi,1 +violette leduc,3 +methyl formate,9 +you walk away,1 +the traveler,1 +bond,89 +moa cuba,3 +hebrew medicine,1 +women in the russian and soviet military,2 +help log,2 +cuillin,5 +back fire,14 +salesrepresentativesbiz,1 +hogsnort rupert,1 +dwarf minke whale,1 +embassy of albania ottawa,1 +cotai water jet,1 +st lucie county florida,8 +wesselman,1 +american indian art,1 +richard arkless,1 +trolleybuses in bergen,1 +vama buzăului,1 +far east movement,9 +threes a crowd,1 +insane,3 +linux technology center,4 +patty duke,24 +smuckers,1 +kapalua,1 +amf futsal world cup,5 +umes chandra college,1 +jnanappana,2 +bar bar bar,1 +beretta m951,2 +libertarian anarchism,1 +fart proudly,4 +peyton place,5 +phase detection autofocus,1 +cavalry in the american civil war,9 +class stratification,1 +battle of cockpit point,1 +regiment van heutsz,2 +ana rivas logan,1 +nenya,1 +westland wah 64 apache,1 +roslyn harbor new york,3 +august wilhelm von hofmann,1 +professional baseball,2 +douglas feith,1 +pogrom,21 +aušra kėdainiai,1 +pseudopeptidoglycan,4 +arquà petrarca,1 +wayampi,1 +conservative government 1866 1868,1 +world naked bike ride,28 +fruitvale oil field,2 +shuttle buran,1 +robert c pruyn,1 +totem,1 +megalotheca,1 +nkechi egbe,1 +james p comeford,1 +heavens memo pad,7 +cauca valley,1 +jungfraujoch railway station,2 +seo in guk,24 +bold for delphi,1 +multiple frames interface,1 +zhenli ye gon,6 +kyabram victoria,1 +two stars for peace solution,1 +couette flow,9 +new formalism,2 +template talk 1930s comedy film stub,1 +template talk scream,1 +joona toivio,4 +iaaf silver label road race,1 +super bowl xxviii,5 +i aint never,1 +paul little racing,1 +jacobite rising of 1715,3 +katherine archuleta,1 +programmable logic device,12 +footsteps of our fathers,2 +once upon a tour,1 +tauck,1 +budapest memorandum on security assurances,5 +prostitution in chad,2 +bebedouro,2 +vice,2 +madredeus,1 +p diddy,1 +princess alice of the united kingdom,20 +jerry hairston jr,1 +neo noir,3 +self evaluation motives,1 +relativity the special and the general theory,2 +the sign of four,3 +kevin deyoung,1 +robin long,1 +mokshaa helsa,1 +nagaon,1 +aniceto esquivel sáenz,1 +sda,2 +german battlecruiser gneisenau,1 +assisted reproductive technology,12 +cmmg,1 +vision of you,1 +keshia chanté discography,1 +biofuel in the united kingdom,1 +katinka ingabogovinanana,1 +hutt valley,1 +garwol dong,1 +tunceli province,3 +edwin bickerstaff,1 +halloween 3 awesomeland,1 +canadian records in track and field,1 +ubisoft são paulo,1 +midstream,16 +jethro tull,4 +childhoods end,55 +ss rohilla,1 +lagranges four square theorem,6 +bucky pizzarelli,3 +jannik bandowski,80 +guðni Ágústsson,1 +multidimensional probability distribution,1 +brno–tuřany airport,2 +broughtonia,5 +cold hands warm heart,1 +simone biles,32 +bf homes parañaque,2 +akaflieg köln ls11,3 +street fighter legacy,2 +beautiful kisses,1 +first modern olympics,1 +macbook air,1 +dublab,1 +silent night deadly night,6 +earth defense force 2025,2 +grant township carroll county iowa,1 +gary williams,1 +malmö aviation,1 +geographical pricing,2 +anaheim memorial medical center,1 +mary+mallon,1 +henry a byroade,1 +wawasan 2020,4 +eurovision dance contest,6 +lydia polgreen,1 +pilsen kansas,1 +colin sampson,1 +neelamegha perumal temple,1 +james bye,2 +canadian federation of agriculture,1 +f w de klerk,34 +bob casey jr,3 +northport east,1 +elian gonzalez affair,1 +aleksei bibik,1 +anthony dias blue,1 +pyaar ke side effects,4 +fusako kitashirakawa,1 +cal robertson,4 +shandong national cultural heritage list,1 +police story 3 super cop,5 +the third ingredient,3 +dean horrix,1 +pico el león,1 +cesar chavez street,1 +prospered,1 +children in cocoa production,5 +gervase helwys,1 +binary digit,1 +kovai sarala,4 +mathematics and music,1 +macroglossum,1 +f gary gray,21 +broadsoft,2 +cachan,4 +bukkake,21 +church of st margaret of scotland,1 +christopher cockerell,3 +amsterdam oud zuid,1 +county of bogong,1 +intel mobile communications,1 +the legend of white fang,1 +millwright,19 +will buckley,1 +bill jelen,2 +template talk san francisco 49ers coach navbox,1 +amalia garcía,1 +because he lives,1 +air charts,1 +stade edmond machtens,1 +henry stommel,1 +dxgi,1 +misr el makasa sc,1 +chad price,2 +carl henning wijkmark,1 +acanthogorgiidae,1 +diqduq,1 +prelog strain,2 +crispin the cross of lead,4 +avraham adan,2 +barbershop arranging,1 +free x tv,1 +eric guillot,1 +kht,1 +never a dull moment,1 +lwów school of mathematics,1 +sears centre,3 +chin state,6 +van halen 2007 2008 tour,1 +robert weinberg,3 +fierté montréal,2 +vince jack,1 +heikki kuula,1 +architecture of the republic of macedonia,1 +glossary of education terms,1 +aleksandra szwed,1 +military history of europe,3 +exeter central railway station,1 +staroselye,1 +lee thomas,7 +saint peters square,2 +romanization of hispania,2 +file talk dodecahedrongif,1 +signed and sealed in blood,8 +colleges of worcester consortium,1 +district electoral divisions,1 +galkot,1 +king África,3 +monetary policy,57 +brp ang pangulo,2 +battle of mạo khê,1 +air tube,1 +ruth ashton taylor,2 +keith jensen,1 +headland alabama,1 +willie loomis,1 +interactive data extraction and analysis,2 +georgetown city hall,2 +chuck es in love,2 +weeksville brooklyn,1 +anatoly sagalevich,2 +browett lindley & co,1 +barnawartha victoria,1 +pop,2 +black balance,2 +aceratorchis,1 +emmeline pethick lawrence baroness pethick lawrence,1 +osso buco,1 +herminie cadolle,2 +telegram & gazette,2 +le van hieu,1 +pine honey,2 +nexvax2,1 +leicester north railway station,1 +jacqueline foster,1 +bill handel,3 +nizami street,1 +radke,1 +bob mulder,1 +ambroise thomas,4 +carles puigdemont i casamajó,1 +callable bond,6 +tesco metro,2 +mohan dharia,1 +great hammerhead,12 +vinko coce,3 +john mayne,1 +cobb cloverleaf,1 +uhlan,10 +giulio migliaccio,1 +belmont university,6 +rinucumab,1 +kearny high school,1 +chūgen,1 +stages,2 +boar%27s head carol,1 +knight of the bath,1 +ayres thrush,7 +sing hallelujah,1 +the tender land,2 +wholesale banking,1 +jean jacques perrey,5 +maxime bossis,2 +sherman records,1 +alan osório da costa silva,1 +fannie willis johnson house,1 +blacks equation,2 +levinthals paradox,2 +thomas scully,2 +necron,3 +university of alberta school of business,5 +lake shetek,1 +toby maduot,1 +gavriil golovkin,1 +sweetwater,3 +atlantic revolutions,2 +jaime reyes (comics,1 +kajang by election 2014,1 +mycotoxigenic,1 +san marco altarpiece,2 +line impedance stabilization network,2 +santiago hernández,1 +jazzland,3 +host–guest chemistry,4 +giovanni florio,2 +st marylebone school,1 +acqua fragile,1 +the horse whisperer,10 +don francis,1 +mike molesevich,1 +brad wright,1 +north melbourne football club,3 +brady dragmire,1 +margaret snowling,2 +wing chun terms,4 +mckey sullivan,1 +derek ford,1 +cache bus,1 +bernie grant arts centre,2 +amata francisca,1 +sinha,2 +larissa loukianenko,1 +oceans apart&sa=u&ved=0ahukewjw4n6eqdblahun7gmkhxxebd8qfgg4mag&usg=afqjcnhhjagrbamjgaxc7rpsso4i9z jgw,1 +anemone heart,2 +alison mcinnes,1 +juan lindo,1 +mahesh bhupati,1 +baháí faith in taiwan,5 +cinema impero,1 +template talk rob thomas,1 +likin,1 +science & faith,1 +fort saint elmo,3 +delhi kumar,6 +juha lallukka,1 +situational sexual behavior,2 +milligan indiana,1 +william em lands,1 +karl anselm duke of urach,2 +hérold goulon,1 +vedic mathematics,20 +move to this,1 +koussan,1 +floored,1 +raghu nandan mandal,1 +angels gods secret agents,1 +orthogonal,2 +the little house on the prairie,1 +chilean pintail,1 +guardian angel,2 +st leonard maryland,1 +green parties in the united kingdom,1 +time to say goodbye,1 +alba michigan,2 +harbourfront centre,1 +corner tube boiler,1 +consensus government,1 +ppru 1,1 +corporate anniversary,4 +sazerac company,5 +kyle friend,1 +bmw k1100lt,1 +pergola marche,1 +commonwealth of kentucky,2 +taiwan passport,2 +clare quilty,1 +domenico caprioli,1 +frank m hull,1 +cheng sui,2 +nazi board games,3 +spark bridge,1 +derrick thomas,6 +wunnumin 1,1 +emotion remixed +,4 +brian howard dix,2 +brigalow queensland,2 +burgi dynasty,1 +apolonia supermercados,1 +brandon lafell,2 +one day,24 +nara period,9 +template talk the land before time,1 +assyrians in iraq,1 +trade union reform and employment rights act 1993,2 +template talk evansville crimson giants seasons,1 +boys be smile / 目覚めた朝にはきみが隣に,2 +kapuloan sundha kecil,1 +human impact of internet use,1 +kolkata metro line 2,3 +saint pardoux morterolles,1 +carfin grotto,2 +samuel johnson prize,3 +french royal family,1 +android app //orgwikipedia/http/enmwikipediaorg/wiki/victoria park,1 +mazda xedos 9,1 +măiestrit,1 +petroleum economist,2 +penetration,2 +adrian rawlins,8 +plutonium 239,11 +culture of montreal,1 +british germans,2 +warszawa wesoła railway station,1 +lorenzo di bonaventura,6 +military ranks of estonia,1 +uss flint,8 +arthur f defranzo,1 +sadeh,1 +jammu and kashmir,3 +igor budan,2 +charmila,2 +choi,1 +mohammed ali khan walajah,1 +sourabh varma,1 +after here through midland,1 +martyn day,1 +justin larouche,1 +illinoiss 6th congressional district,4 +jackson wy,1 +tyson apostol,4 +mitch morse,1 +robert davila,1 +canons regular of saint john cantius,1 +giant girdled lizard,2 +cascade volcanoes,5 +fools day,1 +cordyline indivisa,1 +pueraria,2 +swiss folklore,4 +meretz,3 +united states senate elections 1836 and 1837,1 +baby i need your love/ easy come easy go,1 +butrus al bustani,2 +the lion the lamb the man,1 +rushikulya,1 +brickworks,3 +alliance party of kenya,1 +ludlow college,1 +internationalism,11 +ernest halliwell,1 +constantine phipps 1st marquess of normanby,1 +kari ye bozorg,1 +signal flow,4 +i beam,1 +devils lake,1 +union of artists of the ussr,2 +index of saint kitts and nevis related articles,1 +ethernet physical layer,18 +dimensional analysis,16 +anatomical directions,2 +supreme court of guam,1 +sentul kuala lumpur,2 +ducefixion,1 +red breasted merganser,4 +reservation,3 +in the land of blood and honey,9 +kate spade,2 +albina airstrip,1 +kankakee,1 +servicelink,2 +castilleja levisecta,1 +tonmeister,2 +chanda sahib,1 +lists of patriarchs archbishops and bishops,1 +mach zehnder modulator,1 +giants causeway,79 +literal,7 +uss gerald r ford,1 +monster hunter portable 3rd,3 +bayern munich v norwich city,1 +banking industry,1 +prankton united,1 +st elmo w acosta,1 +speech disorder,9 +welcome to my dna,1 +nouriel roubini,6 +arthur kill,2 +bill grundy,7 +jake gyllenhaal,1 +world bowl 2000,1 +wnt7a,1 +pink flamingo,2 +tridentine calendar,1 +ray ratto,1 +f 88 voodoo,1 +super star,4 +ondřej havelka,1 +sophia dorothea of celle,12 +clavulina tepurumenga,1 +vampire bats,4 +ihsan,1 +ocotea foetens,1 +gannett inc,1 +kemira,4 +gre–nal,2 +farm bureau mutual,1 +pete fox,1 +let him have it,3 +backwoods home magazine,6 +te reo maori remixes,1 +hussain andaryas,1 +bagun sumbrai,1 +the westin paris – vendôme,4 +xochiquetzal,4 +players tour championship 2013/2014,1 +picnic,7 +josh elliott,5 +ernak,3 +gracias,1 +k280ff,1 +bandaranaike–chelvanayakam pact,1 +patrick baert,1 +nausicaä of the valley of the wind,33 +al jurisich,1 +twitter,230 +window,38 +the power hour,1 +duplex worm,1 +sonam bajwa,16 +baljit singh deo,1 +indian jews,1 +outline of madagascar,1 +outback 8,1 +dye fig,1 +british columbia recall and initiative referendum 1991,1 +felipe suau,1 +north perry ohio,1 +gilbeys gin,1 +philippe cavoret,1 +luděk pachman,1 +the it girl,1 +dragonnades,1 +rick debruhl,2 +xpath 20,2 +sean mcnulty,1 +william moser,1 +international centre for the settlement of investment disputes,1 +mendes napoli,2 +canadian rugby championship,1 +battle of maidstone,2 +boulevard theatre,2 +snow sheep,3 +penalty corner,1 +michael ricketts,5 +crocodile,2 +job safety analysis,5 +duffy antigen,1 +counties of virginia,1 +a place to bury strangers,5 +socialist workers’ party of iran,1 +wlw t,1 +core autosport,1 +west francia,10 +karen kilgariff,2 +pacific tsunami museum,1 +first avenue,1 +troubadour,1 +great podil fire,1 +chilean presidential referendum 1988,1 +pavol schmidt,1 +handguard,1 +crime without passion,1 +dio at donington uk live 1983 & 1987,1 +optic nerves,1 +wake forest school of medicine,1 +new jersey jewish news,2 +luke boden,2 +chris hicky,1 +beforu,2 +verch,1 +st roch,3 +civitas,1 +tmrevolution,3 +jamie spencer,1 +bond beam,1 +megan fox,4 +battle of bayan,1 +japan airlines flight 472,1 +yuen kay san,1 +the friendly ghost,1 +rice,14 +jack dellal,16 +lee ranaldo,9 +the overlanders,1 +earl castle stewart,5 +first down,1 +rheum maximowiczii,1 +washington state republican party,2 +ostwald bas rhin,1 +tennessee open,1 +kenneth kister,1 +ted kennedy,72 +preben elkjaer,1 +india reynolds,2 +santagata de goti,1 +henrietta churchill 2nd duchess of marlborough,1 +creteil,1 +ntt data,3 +zoot allures,4 +theatre of ancient greece,29 +bujinkan,6 +clube ferroviário da huíla,2 +nhn,4 +hp series 80,2 +interstate 15,4 +moszczanka,1 +lawnside school district,1 +virunga mountains,5 +hallway,1 +serb peoples radical party,1 +free dance,1 +mishawaka amphitheatre,1 +deerhead kansas,1 +utopiayile rajavu,1 +john w olver transit center,1 +futa tooro,1 +digoxigenin,5 +thomas schirrmacher,1 +twipra kingdom,1 +pulpwood,6 +think blue linux,1 +raho city taxi,1 +frederic remington art museum,1 +wajdi mouawad,1 +semi automatic firearm,12 +phyllis chase,1 +malden new york,1 +the aetiology of hysteria,2 +my maserati does 185,1 +friedrich wilhelm von jagow,1 +apne rang hazaar,1 +bór greater poland voivodeship,1 +india rubber,2 +bring your daughter to the slaughter,4 +yasser radwan,1 +kuala ketil,1 +notre dame de paris,1 +yuanjiang,1 +fengjuan,1 +tockenham,1 +transnistrian presidential election 1991,1 +gautami,28 +providenciales airport,1 +donald chumley,1 +middle finger,8 +calke abbey,4 +thou shalt not kill,1 +trail,7 +battle of dunkirk,43 +eyre yorke block,3 +mactan,3 +american ninja warrior,2 +nevel papperman,1 +ninja storm power rangers,1 +uss castle rock,1 +turcos,1 +philippine sea frontier,1 +irom chanu sharmila,7 +for the first time,2 +stian ringstad,1 +tréon,1 +hiro fujikake,1 +renewable energy in norway,4 +dedh ishqiya,18 +leucothoe,2 +ecmo,2 +knfm,1 +gangnam gu,1 +oadby town fc,1 +clamperl,2 +mummy cave,2 +kenneth d bailey,2 +peter freuchen,2 +dayanand bandodkar,2 +shawn crahan,16 +barbara trentham,2 +university of virginia school of nursing,1 +vöckla,1 +intuitive surgical inc,1 +cyncoed,4 +john l stevens,1 +daniel farabello,1 +trent harmon,5 +feroze gandhi unchahar thermal power station,1 +samuel powell,1 +pan slavic,1 +swimming at the 1992 summer olympics – womens 4 × 100 metre freestyle relay,1 +human behaviour,2 +siege of port royal,3 +eridug,1 +lafee,1 +north bethesda trail,1 +scheveningen system,1 +special penn thing,1 +pserimos,1 +pravda vítězí,1 +wiki dankowska,1 +transcript,13 +second inauguration of grover cleveland,1 +spent fuel,1 +ertms regional,2 +frederick scherger,1 +nivis,1 +herbert hugo menges,1 +kapitan sino,1 +samson,34 +minae mizumura,2 +gro kvinlog,1 +chasing shadows,2 +d j fontana,1 +massively multiplayer online game,27 +capture of new orleans,8 +meat puppet,1 +american pet products manufacturers association,3 +villardonnel,1 +sessile serrated adenoma,3 +patch products,1 +lodovico altieri,1 +portal,2 +jake maskall,4 +the shops at la cantera,8 +stage struck,5 +elizabeth m tamposi,2 +taylor swift,22 +forum spam,9 +barry cowdrill,3 +patagopteryx,2 +korg ms 2000,1 +hmas dubbo,2 +ss khaplang,2 +kevin kelly,1 +punk goes pop volume 5,3 +spurt,2 +bristol pound,5 +military history of finland during world war ii,10 +laguardia,1 +josé marcó del pont,1 +conditional expectation,18 +the beat goes on,1 +patricia buckley ebrey,1 +ali ibn yusuf,2 +caristii,1 +william l brandon,1 +fomite,5 +barcelona el prat airport,7 +mattequartier,4 +invading the sacred,1 +jefferson station,3 +chibalo,1 +phil voyles,1 +ramen,41 +archbishopric of athens,1 +robert arnot,1 +diethylhydroxylamine,2 +christian vazquez,1 +servage hosting,1 +ufo alien invasion,1 +blackburn railway station,3 +performance metric,19 +pencilings,1 +phosphoenolpyruvate,1 +under lights,2 +diego de la hoya,1 +felipe caicedo,5 +jimmy arguello,1 +cielo dalcamo,1 +jan navrátil,1 +linear pottery culture,9 +wbga,1 +k36dd,1 +die hard 2,22 +companding,8 +this is the modern world,10 +cosmology,26 +craig borten,1 +red pelicans,1 +ac gilbert,2 +fougasse,1 +leonardos robot,4 +john of whithorn,2 +david prescott barrows,2 +http cookie,168 +emilia telese,6 +herăstrău park,2 +lauro villar,1 +earl of lincoln,1 +born again,2 +milan rufus,1 +weper,2 +levitt bernstein,1 +jean de thevenot,1 +jill paton walsh,2 +leudal,1 +kyle mccafferty,1 +pluralistic walkthrough,2 +greetings to the new brunette,3 +angus maccoll,1 +loco live,2 +palm i705,1 +saila laakkonen,1 +ssta,1 +buch,1 +eduardo cunha,7 +marie bouliard,1 +mystic society,2 +chu jus house,1 +boob tube,8 +il mestiere della vita,1 +hadley fraser,7 +marek larwood,2 +imperial knight,2 +adbc,1 +houdini,8 +patrice talon,3 +iodamoeba,1 +long march,26 +nyinba,1 +maurice dunkley,1 +new south wales state election 1874–75,1 +john lee carroll,1 +poya bridge,1 +category talk military units and formations established in 2004,1 +the family values tour 1999,2 +brødrene hartmann,1 +miomelon,1 +john moran bailey,1 +san juan archipelago,1 +come as you are,7 +hypo niederösterreich,1 +saturn vi,2 +cherokee county kansas,1 +maher abu remeleh,1 +file talk jb grace singlejpg,1 +count paris,8 +template talk anime and manga,1 +kntv,4 +ganges river dolphin,4 +jerry pacht,1 +rapid response,1 +crunch bandicoot,1 +big gay love,2 +john mckay,1 +bareq,1 +nikon d2x,1 +intercontinental paris le grand hotel,1 +oakland alternative high school,1 +ekow eshun,1 +jimmy fortune,1 +american gladiator,2 +ella sophia armitage,1 +united we stand what more can i give,5 +maruti suzuki celerio,1 +geraldo rivera/trackback/,1 +dogs tobramycin contain a primary amine,1 +hot coffee mod,11 +shriners,25 +mora missouri,1 +seattle wa,1 +all star baseball 2003,1 +comparison of android e book reader software,7 +calling out loud,2 +initiative 912,1 +charles batchelor,2 +terry spraggan,2 +wallace thurman,2 +stefan smith,2 +george holding,22 +institute of business administration sukkar,1 +staten island new york,4 +valency,1 +chintamani taluk,1 +mahatma gandhi,1 +co orbital,1 +epex spot,1 +theodoric the great,3 +fk novi pazar,1 +zappas olympics,2 +gustav krupp von bohlen und halbach,1 +yasmany tomás,4 +notre temps,1 +cats %,1 +intramolecular vibrational energy redistribution,1 +graduate management admission test,49 +robin fleming,1 +daniel gadzhev,1 +achaean league,7 +the four books,1 +tunica people,1 +murray hurst,1 +hajipur,7 +wolfgang fischer,1 +bethel minnesota,2 +wincdemu,1 +aleksandar luković,5 +zilog,6 +will to live,1 +pgc,1 +captain sky,1 +eprobemide,1 +gunther plüschow,1 +jackson laboratory,3 +ss orontes,2 +bishop morlino,1 +eldorado air force station,2 +tin oxide,1 +john bell,2 +ajay banga,2 +nail polish remover induced contact dermatitis,1 +quinctia,1 +a/n urm 25d signal generator,1 +the art company,3 +seawind 300c,1 +half and half,7 +constantia czirenberg,1 +halifax county north carolina,4 +tunica vaginalis,9 +life & times of michael k,2 +methyl propionate,1 +carla bley band,1 +us secret service,2 +maría elena moyano,2 +lory meagher cup,9 +malay sultanate,1 +third lanark,1 +olivier dacourt,10 +angri,2 +ukrainian catholic eparchy of saints peter and paul,1 +phosphinooxazolines,1 +allied health professions,24 +hydroxybenzoic acid,1 +srinatha,3 +zone melting,5 +miko,1 +robert b downs,1 +resource management,3 +new year tree,1 +agraw imazighen,1 +catmando,8 +python ide,5 +rocky mount wilson roanoke rapids nc combined statistical area,1 +spanish crown,3 +ianis zicu,1 +william c hubbard,2 +islamic marital jurisprudence,5 +the school of night,1 +krdc,4 +el centro imperials,1 +atiq uz zaman,1 +sliba zkha,1 +file no mosquesvg,8 +herzegovinians,1 +paradise lost,1 +the fairly oddparents,6 +civic alliance,1 +anbu,3 +broadcaster,2 +le bon,1 +columbus nebraska,4 +inuit people,1 +the menace,6 +ilya ilyich mechnikov,1 +algonquin college,4 +seat córdoba wrc,1 +european route e30,6 +three lakes florida,1 +k10de,1 +glyphonyx rhopalacanthus,1 +ask rhod gilbert,1 +bolas criollas,1 +county borough of southport,1 +roll on mississippi,1 +pulitzer prize for photography,7 +mark fisher,1 +oakley g kelly,1 +tajikistani presidential election 1999,1 +the relapse,4 +nabil bentaleb,8 +apprentice,1 +dale brown,3 +studebaker packard hawk series,1 +yu gi oh trading card game,14 +paralimni,2 +institut national polytechnique de toulouse,1 +to catch a spy,1 +hammer,4 +mount judi,2 +thomas posey,1 +maxime baca,1 +arthur susskind,1 +elkins constructors,2 +siege of gaeta,1 +pemex,1 +henry o flipper award,1 +mccordsville indiana,1 +carife,1 +prima donna,1 +proton,1 +henry farrell,1 +randall davidson,1 +history of georgia,11 +beef tongue,4 +ted spread,4 +douglas xt 30,3 +heavenly mother,1 +monte santangelo,1 +lothar matthaus,1 +american party,2 +tire kingdom,1 +bastrop state park,3 +james maurice gavin,1 +blue bird all american,4 +time and a word,10 +runny babbit,1 +nordic regional airlines,6 +advanced scientifics,2 +the space traders,2 +mongol invasion of anatolia,1 +abu hayyan al gharnati,1 +lisa geoghan,3 +valentia harbour railway station,1 +silo,10 +jimmy zhingchak,1 +glamma kid,1 +bonneville high school,1 +secant line,5 +the longshots,2 +costa rican general election 1917,1 +an emotion away,1 +rawlins high school,1 +cold inflation pressure,4 +receptionthe,2 +tom payne,8 +tb treatment,1 +hatikvah,8 +ol yellow eyes is back,1 +vincent mroz,1 +travis bickle,1 +qatar stars league 1985–86,1 +electronic document management,1 +orliska,1 +gáspár orbán,1 +sunabeda,1 +donatus magnus,1 +lawrence e spivak,2 +cavalieri,1 +aw kuchler,1 +coat of arms of kuwait,1 +wallis–zieff–goldblatt syndrome,1 +doug heffernan,3 +g3 battlecruiser,3 +imran abbas,1 +plymouth,1 +gould colorado,1 +in japan,1 +delmar watson,1 +skygusty west virginia,1 +vesque sisters,1 +rushton triangular lodge,1 +italic font,3 +warner w hodgdon carolina 500,1 +blackamoors,5 +magna cum laude,14 +follow that horse,1 +jean snella,1 +chris frith,1 +soul power,2 +spare me the details,1 +ymer xhaferi,1 +murano glass,5 +michel magras,1 +rashard and wallace go to white castle,1 +venus figurines of malta,1 +didnt we almost have it all,1 +ew,1 +david h koch institute for integrative cancer research,2 +black coyote,1 +priob,2 +piera coppola,1 +budhism,4 +south african class h1 4 8 2t,1 +dimitris papamichael+dimitris+papamixail,3 +system sensor,1 +farragut class destroyer,1 +no down payment,1 +william rogers,1 +desperate choices to save my child,1 +joe launchbury,7 +queen seondeok of silla,11 +adams county wisconsin,1 +bandhan bank,1 +x ray tubes,1 +sporadic group,1 +lozovaya,1 +mairead maguire,3 +royal challengers bangalore in 2016,1 +janko of czarnków,1 +marosormenyes,1 +the deadly reclaim,1 +rick doblin,1 +gwen jorgensen,6 +shire of halls creek,1 +carlton house,6 +urad bean,1 +baton rouge louisiana,39 +kiel institute for the world economy,3 +the satuc cup,1 +harlem division,1 +argonaut,2 +choi jeongrye,2 +optical disc image,2 +groesbeek canadian war cemetery,2 +rangpur india,1 +android n,72 +tjeld class patrol boat,1 +together for yes,2 +tender dracula,1 +shane nelson,1 +palazzo ducale urbino,1 +angels,4 +double centralizer theorem,1 +homme,4 +world heart federation,1 +patricia ja lee,4 +a date with elvis,1 +saints row,1 +lanzhou lamian,1 +subcompact car,1 +jojo discography,5 +gary,18 +global returnable asset identifier,1 +aloysia weber,2 +emperor nero,2 +heavyweights,6 +hush records,1 +mewa textil service,2 +michigan gubernatorial election 1986,1 +solanine,9 +andré moritz,3 +foreign relations of china,12 +william t anderson,3 +lindquist field,1 +biggersdale hole,1 +manayunk/norristown line,1 +aliti,1 +budhivanta,3 +tm forum,4 +off plan property,1 +wu xin the monster killer,4 +aharon leib shteinman,1 +mark catano,1 +llanfihangel,1 +atp–adp translocase,4 +tótkomlós,1 +nikita magaloff,1 +xo telescope,1 +pseudomonas rhizosphaerae,1 +pccooler,1 +arcion therapeutics inc,8 +oklahoma gubernatorial election 2010,1 +seed treatment,3 +connecticut education network,1 +company85,1 +bryan molloy,1 +roupeiro,1 +wendt beach park,2 +entick v carrington,3 +firemens auxiliary,1 +shotcrete,14 +sepharial,1 +poet laureate of virginia,1 +musth,6 +dragon run state forest,3 +focal point,10 +pacific drilling,1 +intro,2 +priscus,1 +rokurō mochizuki,1 +bofur,2 +tiffany mount,1 +thanasis papazoglou,12 +life is grand,1 +ergersheim bas rhin,1 +medical reserve corps,3 +anthony ashley cooper 2nd earl of shaftesbury,1 +uefa euro 2012 group a,32 +america movil sab de cv,1 +christopher cook,1 +vladimir makanin,1 +file talk first battle of saratogausmaeduhistorygif,1 +dean foods,4 +logical thinking,1 +tychonic system,1 +hand washing,17 +bioresonance therapy,4 +günther burstyn,4 +religion in the united kingdom,35 +bancroft ontario,2 +alberta enterprise group,1 +belizean spanish,1 +minuscule 22,1 +hmga2,3 +sidama people,1 +shigeaki mori,2 +moonstars,1 +hazard,24 +chilis,6 +rango,3 +kenichi itō,1 +isle of rum,1 +shortwood united fc,1 +bronx gangs,1 +heterometaboly,2 +beagling,4 +jurgen pommerenke,1 +rockin,1 +st maria maggiore,1 +philipp reis,1 +timeboxing,12 +template talk tallahassee radio,1 +aarti puri,2 +john paul verree,2 +adam tomkins,1 +knoppers,1 +sven olov eriksson,1 +ruth bowyer,1 +höfðatorg tower 1,1 +citywire,3 +helen bosanquet,1 +ulex europaeus,4 +richard martyn,1 +hana sugisaki,2 +its all over now baby blue,6 +the myths and legends of king arthur and the knights of the round table,2 +dooce,1 +german submarine u 9,1 +george shearing,4 +bishop of winchester,3 +maximilian karl lamoral odonnell,2 +hec edmundson,1 +morgawr,3 +sovereign state,67 +avignon—la mitis—matane—matapédia,1 +duramax v8 engine,12 +villa rustica,2 +carl dorsey,1 +clairol,6 +abruzzo,22 +momsen lung,10 +m23 rebellion,2 +kira oreilly,1 +constitutive relation,2 +bifrontal craniotomy,1 +basilica of st nicholas amsterdam,2 +marinus kraus,1 +moog prodigy,2 +lucy hale,49 +lingiya,1 +idiopathic orbital inflammatory disease,3 +shaanxi youser group,1 +apeirohedron,1 +program of all inclusive care for the elderly,2 +tv3 ghana,3 +arnold schwarzenegger,338 +raquel carriedo tomás,1 +cincinnati playhouse in the park,2 +colobomata,2 +star craft 2,1 +yaaf,1 +fc santa clarita,1 +release me,3 +notts county supporters trust,1 +westchester airport,1 +slowhand at 70 – live at the royal albert hall,1 +bruce gray,2 +only the good die young,1 +sewell thomas stadium,1 +kyle cook,1 +northwest passage,1 +eurex airlines,1 +uss pierre,1 +feitsui dam,1 +sales force,1 +obrien class destroyer,5 +sant longowal institute of engineering and technology,3 +united states presidential election in oklahoma 1952,1 +edyta bartosiewicz,1 +marquess of dorset,1 +whiting wyoming,1 +akanda,1 +jim brewster,1 +mozdok republic of north ossetia alania,1 +maritime gendarmerie,2 +paresh patel,1 +communication art,1 +santa anita handicap,2 +dahlia,44 +qikpad,1 +pudhaiyal,3 +oroshi,1 +ioda,3 +willis j gertsch,1 +scurvy grass,1 +bombing of rotterdam,2 +gagarin russia,1 +dynamic apnea without fins,1 +loess,14 +hans adolf krebs,4 +poręby stare,1 +kismat ki baazi,1 +malcolm slesser,1 +blue crane route local municipality,1 +jean michel basquiat,104 +customs trade partnership against terrorism,3 +lower cove newfoundland and labrador,1 +aashiqui 2,6 +elliott lee,1 +edison electric light company,2 +i rigoberta menchú,1 +battle of tennōji,2 +transport workers union of america,1 +physical review b,1 +way too far,1 +breguet 941,1 +manuel hegen,1 +the blacklist,12 +john dorahy,4 +cinderella sanyu,1 +luis castañeda lossio,1 +headquarters of a military area,1 +jbala people,2 +petrofac emirates,1 +ins garuda,3 +australia national rugby league team,2 +state of emergency 2,3 +mexican sex comedy,2 +baby anikha,1 +notions,1 +android app //orgwikipedia/http/enmwikipediaorg/wiki/elasticity,1 +kissing you,2 +montearagón,1 +grzegorz proksa,3 +shook,1 +may hegglin anomaly,1 +chrysler rb engine,2 +gmcsf,2 +blacksburg,1 +chris hollod,1 +the new guy,1 +thulimbah queensland,1 +sust,1 +knight kadosh,2 +details,4 +nickel mining in new caledonia,3 +easter hotspot,1 +surinamese interior war,1 +field corn,2 +bolesław iii wrymouth,6 +lutwyche queensland,1 +michael campbell,1 +military ranks of turkey,3 +mícheal martin,1 +the architects dream,2 +joel robert,1 +thomas smith,1 +inclusion probability,1 +fucked company,1 +genderfluid,5 +lewisham by election 1891,1 +net promoter,98 +donald stewart,1 +xml base,2 +bhikhu parekh,4 +anthocharis cardamines,1 +vuosaari,1 +demographics of burundi,1 +dst,1 +david ensor,2 +mount pavlof,1 +vince young,5 +st beunos ignatian spirituality centre,4 +ezekiel 48,1 +lewis elliott chaze,1 +template talk croatia squad 2012 mens european water polo championship,1 +the voice of the philippines,4 +whites ferry,1 +cananga odorata,9 +man of steel,2 +john michael talbot,2 +superior oblique myokymia,2 +anisochilus,2 +e421,1 +midnight rider,14 +matrícula consular,1 +first nehru ministry,2 +christopher mcculloch,2 +ems chemie,12 +dominique martin,1 +university club of washington dc,1 +nurse education,5 +theyre coming to take me away ha haaa,1 +bill dauterive,4 +belhar,1 +heel and toe,4 +university of the arctic members,2 +mitava,1 +wjmx fm,1 +father callahan,4 +divine word academy of dagupan,1 +bogs,1 +denny heck,2 +church of st james valletta,1 +field cathedral of the polish army,1 +indian skimmer,1 +history of british airways,3 +international mobile subscriber identity,38 +suzel roche,1 +steven watt,1 +duke ellineton,1 +kirbys avalanche,4 diff --git a/tests/test_anyio/testdata/will_play_text.csv.bz2 b/tests/test_anyio/testdata/will_play_text.csv.bz2 new file mode 100755 index 0000000000..e3bec9de82 Binary files /dev/null and b/tests/test_anyio/testdata/will_play_text.csv.bz2 differ diff --git a/tests/test_asyncio/test_bloom.py b/tests/test_asyncio/test_bloom.py index d67858570f..9122d60ccc 100644 --- a/tests/test_asyncio/test_bloom.py +++ b/tests/test_asyncio/test_bloom.py @@ -10,6 +10,8 @@ skip_ifmodversion_lt, ) +pytestmark = pytest.mark.asyncio + @pytest_asyncio.fixture() async def decoded_r(create_redis, stack_url): diff --git a/tests/test_asyncio/test_cluster.py b/tests/test_asyncio/test_cluster.py index 1b3fbd5526..45be271eb8 100644 --- a/tests/test_asyncio/test_cluster.py +++ b/tests/test_asyncio/test_cluster.py @@ -50,7 +50,7 @@ from ..ssl_utils import get_tls_certificates from .compat import aclosing, mock -pytestmark = pytest.mark.onlycluster +pytestmark = [pytest.mark.onlycluster, pytest.mark.asyncio] default_host = "127.0.0.1" diff --git a/tests/test_asyncio/test_commands.py b/tests/test_asyncio/test_commands.py index bfb6855a0f..e3b19cb04f 100644 --- a/tests/test_asyncio/test_commands.py +++ b/tests/test_asyncio/test_commands.py @@ -38,6 +38,8 @@ else: from async_timeout import timeout as async_timeout +pytestmark = pytest.mark.asyncio + REDIS_6_VERSION = "5.9.0" diff --git a/tests/test_asyncio/test_connect.py b/tests/test_asyncio/test_connect.py index 62e8665d1f..d8d983966d 100644 --- a/tests/test_asyncio/test_connect.py +++ b/tests/test_asyncio/test_connect.py @@ -13,6 +13,8 @@ from ..ssl_utils import CertificateType, get_tls_certificates +pytestmark = pytest.mark.asyncio + _CLIENT_NAME = "test-suite-client" _CMD_SEP = b"\r\n" _SUCCESS_RESP = b"+OK" + _CMD_SEP diff --git a/tests/test_asyncio/test_connection.py b/tests/test_asyncio/test_connection.py index 38764d30cd..4aa0419922 100644 --- a/tests/test_asyncio/test_connection.py +++ b/tests/test_asyncio/test_connection.py @@ -30,6 +30,7 @@ @pytest.mark.onlynoncluster +@pytest.mark.asyncio async def test_invalid_response(create_redis): r = await create_redis(single_connection_client=True) @@ -51,6 +52,7 @@ async def test_invalid_response(create_redis): @pytest.mark.onlynoncluster +@pytest.mark.asyncio async def test_single_connection(): """Test that concurrent requests on a single client are synchronised.""" r = Redis(single_connection_client=True) @@ -99,6 +101,7 @@ async def get_conn(): @skip_if_server_version_lt("4.0.0") @pytest.mark.redismod @pytest.mark.onlynoncluster +@pytest.mark.asyncio async def test_loading_external_modules(r): def inner(): pass @@ -119,12 +122,14 @@ def inner(): # assert mod.get('fookey') == d +@pytest.mark.asyncio async def test_socket_param_regression(r): """A regression test for issue #1060""" conn = UnixDomainSocketConnection() _ = await conn.disconnect() is True +@pytest.mark.asyncio async def test_can_run_concurrent_commands(r): if getattr(r, "connection", None) is not None: # Concurrent commands are only supported on pooled or cluster connections @@ -134,6 +139,7 @@ async def test_can_run_concurrent_commands(r): assert all(await asyncio.gather(*(r.ping() for _ in range(10)))) +@pytest.mark.asyncio async def test_connect_retry_on_timeout_error(connect_args): """Test that the _connect function is retried in case of a timeout""" conn = Connection( @@ -155,6 +161,7 @@ async def mock_connect(): await conn.disconnect() +@pytest.mark.asyncio async def test_connect_without_retry_on_os_error(): """Test that the _connect function is not being retried in case of a OSError""" with patch.object(Connection, "_connect") as _connect: @@ -165,6 +172,7 @@ async def test_connect_without_retry_on_os_error(): assert _connect.call_count == 1 +@pytest.mark.asyncio async def test_connect_timeout_error_without_retry(): """Test that the _connect function is not being retried if retry_on_timeout is set to False""" @@ -178,6 +186,7 @@ async def test_connect_timeout_error_without_retry(): @pytest.mark.onlynoncluster +@pytest.mark.asyncio async def test_connection_parse_response_resume(r: redis.Redis): """ This test verifies that the Connection parser, @@ -212,6 +221,7 @@ async def test_connection_parse_response_resume(r: redis.Redis): [_AsyncRESP2Parser, _AsyncRESP3Parser, _AsyncHiredisParser], ids=["AsyncRESP2Parser", "AsyncRESP3Parser", "AsyncHiredisParser"], ) +@pytest.mark.asyncio async def test_connection_disconect_race(parser_class, connect_args): """ This test reproduces the case in issue #2349 @@ -293,12 +303,14 @@ async def dummy_method(*args, **kwargs): @pytest.mark.onlynoncluster -def test_create_single_connection_client_from_url(): +@pytest.mark.asyncio +async def test_create_single_connection_client_from_url(): client = Redis.from_url("redis://localhost:6379/0?", single_connection_client=True) assert client.single_connection_client is True @pytest.mark.parametrize("from_url", (True, False), ids=("from_url", "from_args")) +@pytest.mark.asyncio async def test_pool_auto_close(request, from_url): """Verify that basic Redis instances have auto_close_connection_pool set to True""" @@ -315,6 +327,7 @@ async def get_redis_connection(): await r1.aclose() +@pytest.mark.asyncio async def test_close_is_aclose(request): """Verify close() calls aclose()""" calls = 0 @@ -334,6 +347,7 @@ async def mock_aclose(self): await r1.close() +@pytest.mark.asyncio async def test_pool_from_url_deprecation(request): url: str = request.config.getoption("--redis-url") @@ -341,6 +355,7 @@ async def test_pool_from_url_deprecation(request): return Redis.from_url(url, auto_close_connection_pool=False) +@pytest.mark.asyncio async def test_pool_auto_close_disable(request): """Verify that auto_close_connection_pool can be disabled (deprecated)""" @@ -359,6 +374,7 @@ async def get_redis_connection(): @pytest.mark.parametrize("from_url", (True, False), ids=("from_url", "from_args")) +@pytest.mark.asyncio async def test_redis_connection_pool(request, from_url): """Verify that basic Redis instances using `connection_pool` have auto_close_connection_pool set to False""" @@ -391,6 +407,7 @@ async def mock_disconnect(_): @pytest.mark.parametrize("from_url", (True, False), ids=("from_url", "from_args")) +@pytest.mark.asyncio async def test_redis_from_pool(request, from_url): """Verify that basic Redis instances created using `from_pool()` have auto_close_connection_pool set to True""" @@ -423,6 +440,7 @@ async def mock_disconnect(_): @pytest.mark.parametrize("auto_close", (True, False)) +@pytest.mark.asyncio async def test_redis_pool_auto_close_arg(request, auto_close): """test that redis instance where pool is provided have auto_close_connection_pool set to False, regardless of arg""" @@ -449,6 +467,7 @@ async def mock_disconnect(_): await pool.disconnect() +@pytest.mark.asyncio async def test_client_garbage_collection(request): """ Test that a Redis client will call _close() on any @@ -472,6 +491,7 @@ async def test_client_garbage_collection(request): await pool.aclose() +@pytest.mark.asyncio async def test_connection_garbage_collection(request): """ Test that a Connection object will call close() on the @@ -525,12 +545,14 @@ async def test_connection_garbage_collection(request): ), ], ) +@pytest.mark.asyncio async def test_format_error_message(conn, error, expected_message): """Test that the _error_message function formats errors correctly""" error_message = conn._error_message(error) assert error_message == expected_message +@pytest.mark.asyncio async def test_network_connection_failure(): exp_err = rf"^Error {ECONNREFUSED} connecting to 127.0.0.1:9999.(.+)$" with pytest.raises(ConnectionError, match=exp_err): @@ -538,6 +560,7 @@ async def test_network_connection_failure(): await redis.set("a", "b") +@pytest.mark.asyncio async def test_unix_socket_connection_failure(): exp_err = "Error 2 connecting to unix:///tmp/a.sock. No such file or directory." with pytest.raises(ConnectionError, match=exp_err): diff --git a/tests/test_asyncio/test_connection_pool.py b/tests/test_asyncio/test_connection_pool.py index 09409e04a8..8ab36e408d 100644 --- a/tests/test_asyncio/test_connection_pool.py +++ b/tests/test_asyncio/test_connection_pool.py @@ -39,6 +39,7 @@ def has_no_connected_connections(pool: redis.ConnectionPool): for x in pool._available_connections + list(pool._in_use_connections) ) + @pytest.mark.asyncio async def test_auto_disconnect_redis_created_pool(self, r: redis.Redis): new_conn = await self.create_two_conn(r) assert new_conn != r.connection @@ -46,6 +47,7 @@ async def test_auto_disconnect_redis_created_pool(self, r: redis.Redis): await r.aclose() assert self.has_no_connected_connections(r.connection_pool) + @pytest.mark.asyncio async def test_do_not_auto_disconnect_redis_created_pool(self, r2: redis.Redis): assert r2.auto_close_connection_pool is False, ( "The connection pool should not be disconnected as a manually created " @@ -59,6 +61,7 @@ async def test_do_not_auto_disconnect_redis_created_pool(self, r2: redis.Redis): assert len(r2.connection_pool._available_connections) == 1 assert r2.connection_pool._available_connections[0].is_connected + @pytest.mark.asyncio async def test_auto_release_override_true_manual_created_pool(self, r: redis.Redis): assert r.auto_close_connection_pool is True, "This is from the class fixture" await self.create_two_conn(r) @@ -70,6 +73,7 @@ async def test_auto_release_override_true_manual_created_pool(self, r: redis.Red assert self.has_no_connected_connections(r.connection_pool) @pytest.mark.parametrize("auto_close_conn_pool", [True, False]) + @pytest.mark.asyncio async def test_close_override(self, r: redis.Redis, auto_close_conn_pool): r.auto_close_connection_pool = auto_close_conn_pool await self.create_two_conn(r) @@ -77,6 +81,7 @@ async def test_close_override(self, r: redis.Redis, auto_close_conn_pool): assert self.has_no_connected_connections(r.connection_pool) @pytest.mark.parametrize("auto_close_conn_pool", [True, False]) + @pytest.mark.asyncio async def test_negate_auto_close_client_pool( self, r: redis.Redis, auto_close_conn_pool ): @@ -133,6 +138,7 @@ async def get_pool( finally: await pool.disconnect(inuse_connections=True) + @pytest.mark.asyncio async def test_connection_creation(self): connection_kwargs = {"foo": "bar", "biz": "baz"} async with self.get_pool( @@ -142,6 +148,7 @@ async def test_connection_creation(self): assert isinstance(connection, DummyConnection) assert connection.kwargs == connection_kwargs + @pytest.mark.asyncio async def test_aclosing(self): connection_kwargs = {"foo": "bar", "biz": "baz"} pool = redis.ConnectionPool( @@ -152,6 +159,7 @@ async def test_aclosing(self): async with aclosing(pool): pass + @pytest.mark.asyncio async def test_multiple_connections(self, master_host): connection_kwargs = {"host": master_host[0]} async with self.get_pool(connection_kwargs=connection_kwargs) as pool: @@ -159,6 +167,7 @@ async def test_multiple_connections(self, master_host): c2 = await pool.get_connection() assert c1 != c2 + @pytest.mark.asyncio async def test_max_connections(self, master_host): connection_kwargs = {"host": master_host[0]} async with self.get_pool( @@ -169,6 +178,7 @@ async def test_max_connections(self, master_host): with pytest.raises(redis.ConnectionError): await pool.get_connection() + @pytest.mark.asyncio async def test_reuse_previously_released_connection(self, master_host): connection_kwargs = {"host": master_host[0]} async with self.get_pool(connection_kwargs=connection_kwargs) as pool: @@ -177,6 +187,7 @@ async def test_reuse_previously_released_connection(self, master_host): c2 = await pool.get_connection() assert c1 == c2 + @pytest.mark.asyncio async def test_repr_contains_db_info_tcp(self): connection_kwargs = { "host": "localhost", @@ -190,6 +201,7 @@ async def test_repr_contains_db_info_tcp(self): expected = "host=localhost,port=6379,db=1,client_name=test-client" assert expected in repr(pool) + @pytest.mark.asyncio async def test_repr_contains_db_info_unix(self): connection_kwargs = {"path": "/abc", "db": 1, "client_name": "test-client"} async with self.get_pool( @@ -215,6 +227,7 @@ async def get_pool(self, connection_kwargs=None, max_connections=10, timeout=20) finally: await pool.disconnect(inuse_connections=True) + @pytest.mark.asyncio async def test_connection_creation(self, master_host): connection_kwargs = { "foo": "bar", @@ -227,6 +240,7 @@ async def test_connection_creation(self, master_host): assert isinstance(connection, DummyConnection) assert connection.kwargs == connection_kwargs + @pytest.mark.asyncio async def test_disconnect(self, master_host): """A regression test for #1047""" connection_kwargs = { @@ -239,6 +253,7 @@ async def test_disconnect(self, master_host): await pool.get_connection() await pool.disconnect() + @pytest.mark.asyncio async def test_multiple_connections(self, master_host): connection_kwargs = {"host": master_host[0], "port": master_host[1]} async with self.get_pool(connection_kwargs=connection_kwargs) as pool: @@ -246,6 +261,7 @@ async def test_multiple_connections(self, master_host): c2 = await pool.get_connection() assert c1 != c2 + @pytest.mark.asyncio async def test_connection_pool_blocks_until_timeout(self, master_host): """When out of connections, block for timeout seconds, then raise""" connection_kwargs = {"host": master_host[0]} @@ -262,6 +278,7 @@ async def test_connection_pool_blocks_until_timeout(self, master_host): assert asyncio.get_running_loop().time() - start >= 0.05 await c1.disconnect() + @pytest.mark.asyncio async def test_connection_pool_blocks_until_conn_available(self, master_host): """ When out of connections, block until another connection is released @@ -282,6 +299,7 @@ async def target(): stop = asyncio.get_running_loop().time() assert (stop - start) <= 0.2 + @pytest.mark.asyncio async def test_reuse_previously_released_connection(self, master_host): connection_kwargs = {"host": master_host[0]} async with self.get_pool(connection_kwargs=connection_kwargs) as pool: @@ -572,6 +590,7 @@ def get_connection(self): class TestConnection: + @pytest.mark.asyncio async def test_on_connect_error(self): """ An error in Connection.on_connect should disconnect from the server @@ -590,6 +609,7 @@ async def test_on_connect_error(self): @pytest.mark.onlynoncluster @skip_if_server_version_lt("2.8.8") @skip_if_redis_enterprise() + @pytest.mark.asyncio async def test_busy_loading_disconnects_socket(self, r): """ If Redis raises a LOADING error, the connection should be @@ -603,6 +623,7 @@ async def test_busy_loading_disconnects_socket(self, r): @pytest.mark.onlynoncluster @skip_if_server_version_lt("2.8.8") @skip_if_redis_enterprise() + @pytest.mark.asyncio async def test_busy_loading_from_pipeline_immediate_command(self, r): """ BusyLoadingErrors should raise from Pipelines that execute a @@ -621,6 +642,7 @@ async def test_busy_loading_from_pipeline_immediate_command(self, r): @pytest.mark.onlynoncluster @skip_if_server_version_lt("2.8.8") @skip_if_redis_enterprise() + @pytest.mark.asyncio async def test_busy_loading_from_pipeline(self, r): """ BusyLoadingErrors should be raised from a pipeline execution @@ -637,12 +659,14 @@ async def test_busy_loading_from_pipeline(self, r): @skip_if_server_version_lt("2.8.8") @skip_if_redis_enterprise() + @pytest.mark.asyncio async def test_read_only_error(self, r): """READONLY errors get turned into ReadOnlyError exceptions""" with pytest.raises(redis.ReadOnlyError): await r.execute_command("DEBUG", "ERROR", "READONLY blah blah") @skip_if_redis_enterprise() + @pytest.mark.asyncio async def test_oom_error(self, r): """OOM errors get turned into OutOfMemoryError exceptions""" with pytest.raises(redis.OutOfMemoryError): @@ -675,6 +699,7 @@ def test_connect_from_url_unix(self): ) @skip_if_redis_enterprise() + @pytest.mark.asyncio async def test_connect_no_auth_supplied_when_required(self, r): """ AuthenticationError should be raised when the server requires a @@ -686,6 +711,7 @@ async def test_connect_no_auth_supplied_when_required(self, r): ) @skip_if_redis_enterprise() + @pytest.mark.asyncio async def test_connect_invalid_password_supplied(self, r): """AuthenticationError should be raised when sending the wrong password""" with pytest.raises(redis.AuthenticationError): @@ -716,12 +742,14 @@ def assert_interval_advanced(self, connection): diff = connection.next_health_check - asyncio.get_running_loop().time() assert self.interval >= diff > (self.interval - 1) + @pytest.mark.asyncio async def test_health_check_runs(self, r): if r.connection: r.connection.next_health_check = asyncio.get_running_loop().time() - 1 await r.connection.check_health() self.assert_interval_advanced(r.connection) + @pytest.mark.asyncio async def test_arbitrary_command_invokes_health_check(self, r): # invoke a command to make sure the connection is entirely setup if r.connection: @@ -735,6 +763,7 @@ async def test_arbitrary_command_invokes_health_check(self, r): self.assert_interval_advanced(r.connection) + @pytest.mark.asyncio async def test_arbitrary_command_advances_next_health_check(self, r): if r.connection: await r.get("foo") @@ -744,6 +773,7 @@ async def test_arbitrary_command_advances_next_health_check(self, r): await r.get("foo") assert next_health_check < r.connection.next_health_check + @pytest.mark.asyncio async def test_health_check_not_invoked_within_interval(self, r): if r.connection: await r.get("foo") @@ -754,6 +784,7 @@ async def test_health_check_not_invoked_within_interval(self, r): ping_call_spec = (("PING",), {"check_health": False}) assert ping_call_spec not in m.call_args_list + @pytest.mark.asyncio async def test_health_check_in_pipeline(self, r): async with r.pipeline(transaction=False) as pipe: pipe.connection = await pipe.connection_pool.get_connection() @@ -765,6 +796,7 @@ async def test_health_check_in_pipeline(self, r): m.assert_any_call("PING", check_health=False) assert responses == [True, b"bar"] + @pytest.mark.asyncio async def test_health_check_in_transaction(self, r): async with r.pipeline(transaction=True) as pipe: pipe.connection = await pipe.connection_pool.get_connection() @@ -776,6 +808,7 @@ async def test_health_check_in_transaction(self, r): m.assert_any_call("PING", check_health=False) assert responses == [True, b"bar"] + @pytest.mark.asyncio async def test_health_check_in_watched_pipeline(self, r): await r.set("foo", "bar") async with r.pipeline(transaction=False) as pipe: @@ -800,6 +833,7 @@ async def test_health_check_in_watched_pipeline(self, r): assert responses == [True, b"not-bar"] m.assert_any_call("PING", check_health=False) + @pytest.mark.asyncio async def test_health_check_in_pubsub_before_subscribe(self, r): """A health check happens before the first [p]subscribe""" p = r.pubsub() @@ -819,6 +853,7 @@ async def test_health_check_in_pubsub_before_subscribe(self, r): subscribe_message = await wait_for_message(p) assert subscribe_message["type"] == "subscribe" + @pytest.mark.asyncio async def test_health_check_in_pubsub_after_subscribed(self, r): """ Pubsub can handle a new subscribe when it's time to check the @@ -859,6 +894,7 @@ async def test_health_check_in_pubsub_after_subscribed(self, r): m.assert_any_call("PING", p.HEALTH_CHECK_MESSAGE, check_health=False) self.assert_interval_advanced(p.connection) + @pytest.mark.asyncio async def test_health_check_in_pubsub_poll(self, r): """ Polling a pubsub connection that's subscribed will regularly diff --git a/tests/test_asyncio/test_cwe_404.py b/tests/test_asyncio/test_cwe_404.py index 677e165fc6..03e90aef1c 100644 --- a/tests/test_asyncio/test_cwe_404.py +++ b/tests/test_asyncio/test_cwe_404.py @@ -6,6 +6,8 @@ from redis.asyncio.cluster import RedisCluster from redis.asyncio.connection import async_timeout +pytestmark = pytest.mark.asyncio + class DelayProxy: def __init__(self, addr, redis_addr, delay: float = 0.0): diff --git a/tests/test_asyncio/test_encoding.py b/tests/test_asyncio/test_encoding.py index 74a9f28b2d..00bf0966cb 100644 --- a/tests/test_asyncio/test_encoding.py +++ b/tests/test_asyncio/test_encoding.py @@ -3,6 +3,8 @@ import redis.asyncio as redis from redis.exceptions import DataError +pytestmark = pytest.mark.asyncio + @pytest.mark.onlynoncluster class TestEncoding: diff --git a/tests/test_asyncio/test_hash.py b/tests/test_asyncio/test_hash.py index 4fbc02c5fe..12994eae1b 100644 --- a/tests/test_asyncio/test_hash.py +++ b/tests/test_asyncio/test_hash.py @@ -9,6 +9,8 @@ from tests.conftest import skip_if_server_version_lt from tests.test_asyncio.test_utils import redis_server_time +pytestmark = pytest.mark.asyncio + @skip_if_server_version_lt("7.3.240") async def test_hexpire_basic(r): diff --git a/tests/test_asyncio/test_json.py b/tests/test_asyncio/test_json.py index 852fd4aaa6..13a838062a 100644 --- a/tests/test_asyncio/test_json.py +++ b/tests/test_asyncio/test_json.py @@ -5,6 +5,8 @@ from redis.commands.json.path import Path from tests.conftest import assert_resp_response, skip_ifmodversion_lt +pytestmark = pytest.mark.asyncio + @pytest_asyncio.fixture() async def decoded_r(create_redis, stack_url): diff --git a/tests/test_asyncio/test_lock.py b/tests/test_asyncio/test_lock.py index fff045a7f4..8f28a250df 100644 --- a/tests/test_asyncio/test_lock.py +++ b/tests/test_asyncio/test_lock.py @@ -6,6 +6,7 @@ from redis.exceptions import LockError, LockNotOwnedError +@pytest.mark.asyncio class TestLock: @pytest_asyncio.fixture() async def r_decoded(self, create_redis): diff --git a/tests/test_asyncio/test_monitor.py b/tests/test_asyncio/test_monitor.py index 73ee3cf811..cd629973c6 100644 --- a/tests/test_asyncio/test_monitor.py +++ b/tests/test_asyncio/test_monitor.py @@ -3,6 +3,8 @@ from .conftest import wait_for_command +pytestmark = pytest.mark.asyncio + @pytest.mark.onlynoncluster class TestMonitor: diff --git a/tests/test_asyncio/test_pipeline.py b/tests/test_asyncio/test_pipeline.py index 19e11dc792..0ca31995c2 100644 --- a/tests/test_asyncio/test_pipeline.py +++ b/tests/test_asyncio/test_pipeline.py @@ -5,6 +5,8 @@ from .compat import aclosing, mock from .conftest import wait_for_command +pytestmark = pytest.mark.asyncio + class TestPipeline: async def test_pipeline_is_true(self, r): diff --git a/tests/test_asyncio/test_pubsub.py b/tests/test_asyncio/test_pubsub.py index 13a6158b40..c6289e39a7 100644 --- a/tests/test_asyncio/test_pubsub.py +++ b/tests/test_asyncio/test_pubsub.py @@ -22,6 +22,8 @@ from .compat import aclosing, create_task, mock +pytestmark = pytest.mark.asyncio + def with_timeout(t): def wrapper(corofunc): diff --git a/tests/test_asyncio/test_retry.py b/tests/test_asyncio/test_retry.py index cd251a986f..6796e53a7d 100644 --- a/tests/test_asyncio/test_retry.py +++ b/tests/test_asyncio/test_retry.py @@ -119,6 +119,7 @@ async def test_infinite_retry(self): class TestRedisClientRetry: "Test the Redis client behavior with retries" + @pytest.mark.asyncio async def test_get_set_retry_object(self, request): retry = Retry(NoBackoff(), 2) url = request.config.getoption("--redis-url") diff --git a/tests/test_asyncio/test_scripting.py b/tests/test_asyncio/test_scripting.py index b8e100c04a..c492b68b4b 100644 --- a/tests/test_asyncio/test_scripting.py +++ b/tests/test_asyncio/test_scripting.py @@ -3,6 +3,8 @@ from redis import exceptions from tests.conftest import skip_if_server_version_lt +pytestmark = pytest.mark.asyncio + multiply_script = """ local value = redis.call('GET', KEYS[1]) value = tonumber(value) diff --git a/tests/test_asyncio/test_search.py b/tests/test_asyncio/test_search.py index 2ee74f710f..2e64d8f496 100644 --- a/tests/test_asyncio/test_search.py +++ b/tests/test_asyncio/test_search.py @@ -31,6 +31,8 @@ skip_ifmodversion_lt, ) +pytestmark = pytest.mark.asyncio + WILL_PLAY_TEXT = os.path.abspath( os.path.join(os.path.dirname(__file__), "testdata", "will_play_text.csv.bz2") ) diff --git a/tests/test_asyncio/test_sentinel.py b/tests/test_asyncio/test_sentinel.py index a27ba92bb8..16387a45a9 100644 --- a/tests/test_asyncio/test_sentinel.py +++ b/tests/test_asyncio/test_sentinel.py @@ -12,6 +12,8 @@ SlaveNotFoundError, ) +pytestmark = pytest.mark.asyncio + @pytest_asyncio.fixture(scope="module") def master_ip(master_host): diff --git a/tests/test_asyncio/test_ssl.py b/tests/test_asyncio/test_ssl.py index 75800f22de..8f58928409 100644 --- a/tests/test_asyncio/test_ssl.py +++ b/tests/test_asyncio/test_ssl.py @@ -28,10 +28,12 @@ async def _get_client(self, request): yield client await client.aclose() + @pytest.mark.asyncio async def test_ssl_with_invalid_cert(self, _get_client): """Test SSL connection with invalid certificate.""" pass + @pytest.mark.asyncio async def test_cert_reqs_none_with_check_hostname(self, request): """Test that when ssl_cert_reqs=none is used with ssl_check_hostname=True, the connection is created successfully with check_hostname internally set to False""" diff --git a/tests/test_asyncio/test_timeseries.py b/tests/test_asyncio/test_timeseries.py index 24c36c9ca2..f542e29429 100644 --- a/tests/test_asyncio/test_timeseries.py +++ b/tests/test_asyncio/test_timeseries.py @@ -12,6 +12,8 @@ skip_ifmodversion_lt, ) +pytestmark = pytest.mark.asyncio + @pytest_asyncio.fixture() async def decoded_r(create_redis, stack_url): diff --git a/tests/test_asyncio/test_vsets.py b/tests/test_asyncio/test_vsets.py index 4ae336acf8..dc26a185cb 100644 --- a/tests/test_asyncio/test_vsets.py +++ b/tests/test_asyncio/test_vsets.py @@ -10,6 +10,8 @@ skip_if_server_version_lt, ) +pytestmark = pytest.mark.asyncio + @pytest_asyncio.fixture() async def d_client(create_redis, redis_url): diff --git a/tests/test_search.py b/tests/test_search.py index 7e4f59eb79..2804ef0073 100644 --- a/tests/test_search.py +++ b/tests/test_search.py @@ -1560,7 +1560,7 @@ def test_aggregations_add_scores(client): @pytest.mark.redismod @skip_ifmodversion_lt("2.10.05", "search") -async def test_aggregations_hybrid_scoring(client): +def test_aggregations_hybrid_scoring(client): client.ft().create_index( ( TextField("name", sortable=True, weight=5.0),