diff --git a/.gitignore b/.gitignore index dbf217c8..52801c40 100644 --- a/.gitignore +++ b/.gitignore @@ -21,4 +21,16 @@ venv*/ dask-worker-space/* .pre-commit-config.yaml +*.o +scaler/io/cpp/cpp.cpp +scaler/io/cpp/CMakeFiles +scaler/io/cpp/example +scaler/io/cpp/Makefile +benchmark.py +benches/** +*.so +*.out .DS_Store +CMakeFiles/ +CMakeCache.txt +cmake_install.cmake diff --git a/documentation.md b/documentation.md new file mode 100644 index 00000000..8b250e22 --- /dev/null +++ b/documentation.md @@ -0,0 +1,84 @@ +# Replacing ZMQ in Scaler + +Scaler relies on ZeroMQ (ZMQ) heavily for its networking. +We aim to replace Scaler's usage of ZMQ with a custom solution built in C++. + +## Requirements + +ZMQ sockets are a powerful abstraction and if we are to replace their usage in the Scaler, then we must implement some of its behaviours. In particular we have the following requirements: + +- **Multi-protocol**: ZMQ sockets abstract over many kinds of transports: TCP, Unix, intraprocess, etc. +- **Guaranteed message delivery**: ZMQ guarantees that messages will be delivered despite network issues. +- **Reconnects**: ZMQ sockets are durable and reconnect if the underlying connection fails. + - **Flexible connection order**: ZMQ sockets support issuing a `connect()` _before_ the remote socket has called `bind()`. +- **Multiple peers**: Unlike standard network sockets ZMQ sockets can represent a connection between multiple peers. + - e.g., a TCP socket represents a connection between two endpoints, but a ZMQ socket can be connected to multiple endpoints at the same time. +- **Socket patterns**: ZMQ sockets come in multiple types that impact their routing behaviour, such as dealer, pub, router, etc. + +In addition to implementing those ZMQ features, we also have the following requirements: + +- **Async _and_ sync**: We need to support both async and sync interfaces. +- **Thread-safety**: Our ZMQ socket replacement needs to be thread-safe. +- **Multiple backends**: We need to support implementing our interface with different kinds of backends e.g. epoll, io_uring, etc. +- **Multiple consumer languages**: The library needs to be usable from multiple languages e.g. Python, C/C++, and possibly more. + +## Implementation + +### Structures + +```c++ +// rationale: +// - we need a context to manage the io threads +// - intraprocess connectors need a context to discover and communicate with each other +// +// lifetime: +// - the context is the longest lived object in the library +// - you should normally only have one context per program +// - the lifetime begins when you call `io_context_init()` and ends when you call `io_context_destroy()` +// +// usage: +// - the main usage of io context is to be passed to connector initialization +// - use `io_context_*()` functions to operate on the context +// +// assumptions: +// - the user is not going to access the internal state of the io context directly +struct IoContext { + // the io threads + std::vector threads; + std::atomic_uint8_t thread_rr; + + std::vector inprocs; + std::shared_mutex intra_process_mutex; +}; + +// rationale: +// - the interface for intraprocess and socket-based connectors needs to be unified +// +// lifetime: +// - the connector lives within the scope of the io context and thread context +// - all connectors MUST be destroyed before the io context is destroyed +// - its lifetime begins when you call `connector_init()` and ends when you call `connector_destroy()` +// - its lifetime is flexible and determined by the library user +// +// usage: +// - the connector is the primary interface of the library +// - the connector is used to send and receive messages +// - use `connector_*()` functions to operate on the connector +// +// assumptions: +// - the user is not going to access the internal state of the connector directly +struct Connector +{ + enum Type + { + Socket, + IntraProcess + } type; + + union + { + IntraProcessConnector *intra_process; + NetworkConnector *network; + }; +}; +``` diff --git a/scaler/CMakeLists.txt b/scaler/CMakeLists.txt index ca45022f..6b944b57 100644 --- a/scaler/CMakeLists.txt +++ b/scaler/CMakeLists.txt @@ -1,2 +1,2 @@ add_subdirectory(object_storage) - +add_subdirectory(io/cpp) diff --git a/scaler/client/agent/client_agent.py b/scaler/client/agent/client_agent.py index 5360f709..ccf7ad0b 100644 --- a/scaler/client/agent/client_agent.py +++ b/scaler/client/agent/client_agent.py @@ -3,8 +3,6 @@ import threading from typing import Optional -import zmq.asyncio - from scaler.client.agent.disconnect_manager import ClientDisconnectManager from scaler.client.agent.future_manager import ClientFutureManager from scaler.client.agent.heartbeat_manager import ClientHeartbeatManager @@ -28,16 +26,16 @@ from scaler.protocol.python.mixins import Message from scaler.utility.event_loop import create_async_loop_routine from scaler.utility.exceptions import ClientCancelledException, ClientQuitException, ClientShutdownException -from scaler.utility.zmq_config import ZMQConfig +from scaler.io.model import IoContext, ConnectorType, Address class ClientAgent(threading.Thread): def __init__( self, identity: bytes, - client_agent_address: ZMQConfig, - scheduler_address: ZMQConfig, - context: zmq.Context, + client_agent_address: Address, + scheduler_address: Address, + session: IoContext, future_manager: ClientFutureManager, stop_event: threading.Event, timeout_seconds: int, @@ -54,25 +52,26 @@ def __init__( self._identity = identity self._client_agent_address = client_agent_address self._scheduler_address = scheduler_address - self._context = context + self._session = session self._future_manager = future_manager self._connector_internal = AsyncConnector( - context=zmq.asyncio.Context.shadow(self._context), + session=self._session, name="client_agent_internal", - socket_type=zmq.PAIR, + type_=ConnectorType.Pair, bind_or_connect="bind", address=self._client_agent_address, callback=self.__on_receive_from_client, identity=None, ) + self._connector_external = AsyncConnector( - context=zmq.asyncio.Context.shadow(self._context), + session=self._session, name="client_agent_external", - socket_type=zmq.DEALER, - address=self._scheduler_address, + type_=ConnectorType.Dealer, bind_or_connect="connect", + address=self._scheduler_address, callback=self.__on_receive_from_scheduler, identity=self._identity, ) @@ -191,7 +190,7 @@ async def __get_loops(self): logging.info("ClientAgent: client quitting") self._future_manager.set_all_futures_with_exception(exception) elif isinstance(exception, TimeoutError): - logging.error(f"ClientAgent: client timeout when connecting to {self._scheduler_address.to_address()}") + logging.error(f"ClientAgent: client timeout when connecting to {self._scheduler_address}") self._future_manager.set_all_futures_with_exception(exception) else: raise exception diff --git a/scaler/client/client.py b/scaler/client/client.py index fd850feb..d9fed627 100644 --- a/scaler/client/client.py +++ b/scaler/client/client.py @@ -8,9 +8,6 @@ from inspect import signature from typing import Any, Callable, Dict, Iterable, List, Optional, Tuple, Union -import zmq -import zmq.asyncio - from scaler.client.agent.client_agent import ClientAgent from scaler.client.agent.future_manager import ClientFutureManager from scaler.client.future import ScalerFuture @@ -26,9 +23,10 @@ from scaler.utility.graph.topological_sorter import TopologicalSorter from scaler.utility.metadata.profile_result import ProfileResult from scaler.utility.metadata.task_flags import TaskFlags, retrieve_task_flags_from_task -from scaler.utility.zmq_config import ZMQConfig, ZMQType from scaler.worker.agent.processor.processor import Processor +from scaler.io.model import IoContext, ConnectorType, Address, IntraProcessAddress + @dataclasses.dataclass class _CallNode: @@ -83,23 +81,23 @@ def __initialize__( self._profiling = profiling self._identity = f"{os.getpid()}|Client|{uuid.uuid4().bytes.hex()}".encode() - self._client_agent_address = ZMQConfig(ZMQType.inproc, host=f"scaler_client_{uuid.uuid4().hex}") - self._scheduler_address = ZMQConfig.from_string(address) + self._client_agent_address = IntraProcessAddress(f"scaler_client_{uuid.uuid4().hex}") + self._scheduler_address = Address.from_str(address) self._timeout_seconds = timeout_seconds self._heartbeat_interval_seconds = heartbeat_interval_seconds self._stop_event = threading.Event() - self._context = zmq.Context() + self._session = IoContext(io_threads=1) self._connector = SyncConnector( - context=self._context, socket_type=zmq.PAIR, address=self._client_agent_address, identity=self._identity + session=self._session, type_=ConnectorType.Pair, address=self._client_agent_address, identity=self._identity ) self._future_manager = ClientFutureManager(self._serializer) self._agent = ClientAgent( identity=self._identity, client_agent_address=self._client_agent_address, - scheduler_address=ZMQConfig.from_string(address), - context=self._context, + scheduler_address=self._scheduler_address, + session=self._session, future_manager=self._future_manager, stop_event=self._stop_event, timeout_seconds=self._timeout_seconds, @@ -108,7 +106,7 @@ def __initialize__( ) self._agent.start() - logging.info(f"ScalerClient: connect to {self._scheduler_address.to_address()}") + logging.info(f"ScalerClient: connect to {self._scheduler_address}") self._object_buffer = ObjectBuffer(self._identity, self._serializer, self._connector) self._future_factory = functools.partial(ScalerFuture, connector=self._connector) @@ -158,7 +156,7 @@ def fibonacci(client: Client, n: int): """ return { - "address": self._scheduler_address.to_address(), + "address": str(self._scheduler_address), "profiling": self._profiling, "timeout_seconds": self._timeout_seconds, "heartbeat_interval_seconds": self._heartbeat_interval_seconds, @@ -326,7 +324,7 @@ def disconnect(self): self.__destroy() return - logging.info(f"ScalerClient: disconnect from {self._scheduler_address.to_address()}") + logging.info(f"ScalerClient: disconnect from {self._scheduler_address}") self._future_manager.cancel_all_futures() @@ -353,7 +351,7 @@ def shutdown(self): self.__destroy() return - logging.info(f"ScalerClient: request shutdown for {self._scheduler_address.to_address()}") + logging.info(f"ScalerClient: request shutdown for {self._scheduler_address}") self._future_manager.cancel_all_futures() @@ -545,7 +543,7 @@ def __assert_client_not_stopped(self): def __destroy(self): self._agent.join() - self._context.destroy(linger=1) + self._session.destroy() @staticmethod def __get_parent_task_priority() -> Optional[int]: diff --git a/scaler/cluster/cluster.py b/scaler/cluster/cluster.py index 1993dc40..e6b4dcd6 100644 --- a/scaler/cluster/cluster.py +++ b/scaler/cluster/cluster.py @@ -5,14 +5,15 @@ from typing import List, Optional, Tuple from scaler.utility.logging.utility import setup_logger -from scaler.utility.zmq_config import ZMQConfig from scaler.worker.worker import Worker +from scaler.io.model import TCPAddress + class Cluster(multiprocessing.get_context("spawn").Process): # type: ignore[misc] def __init__( self, - address: ZMQConfig, + address: TCPAddress, worker_io_threads: int, worker_names: List[str], heartbeat_interval_seconds: int, diff --git a/scaler/cluster/combo.py b/scaler/cluster/combo.py index 6c54bde4..f54b4960 100644 --- a/scaler/cluster/combo.py +++ b/scaler/cluster/combo.py @@ -23,6 +23,7 @@ from scaler.utility.network_util import get_available_tcp_port from scaler.utility.zmq_config import ZMQConfig +from scaler.io.model import Address, TCPAddress class SchedulerClusterCombo: def __init__( @@ -51,9 +52,9 @@ def __init__( logging_config_file: Optional[str] = None, ): if address is None: - self._address = ZMQConfig.from_string(f"tcp://127.0.0.1:{get_available_tcp_port()}") + self._address = TCPAddress.localhost(get_available_tcp_port()) else: - self._address = ZMQConfig.from_string(address) + self._address = Address.from_string(address) self._cluster = Cluster( address=self._address, @@ -101,7 +102,7 @@ def shutdown(self): self._scheduler.join() def get_address(self) -> str: - return self._address.to_address() + return f"{self._address}" def __get_prefix(self): return f"{self.__class__.__name__}:" diff --git a/scaler/cluster/scheduler.py b/scaler/cluster/scheduler.py index b770160e..f5982d5a 100644 --- a/scaler/cluster/scheduler.py +++ b/scaler/cluster/scheduler.py @@ -7,13 +7,14 @@ from scaler.scheduler.scheduler import Scheduler, scheduler_main from scaler.utility.event_loop import register_event_loop from scaler.utility.logging.utility import setup_logger -from scaler.utility.zmq_config import ZMQConfig + +from scaler.io.model import TCPAddress class SchedulerProcess(multiprocessing.get_context("spawn").Process): # type: ignore[misc] def __init__( self, - address: ZMQConfig, + address: TCPAddress, io_threads: int, max_number_of_tasks_waiting: int, per_worker_queue_size: int, diff --git a/scaler/entry_points/cluster.py b/scaler/entry_points/cluster.py index 0965802c..fbf0c35c 100644 --- a/scaler/entry_points/cluster.py +++ b/scaler/entry_points/cluster.py @@ -13,7 +13,7 @@ DEFAULT_WORKER_DEATH_TIMEOUT, ) from scaler.utility.event_loop import EventLoopType, register_event_loop -from scaler.utility.zmq_config import ZMQConfig +from scaler.io.model import Address def get_args(): @@ -76,10 +76,10 @@ def get_args(): "When set, suspends worker processors using the SIGTSTP signal instead of a synchronization event, " "fully halting computation on suspended tasks. Note that this may cause some tasks to fail if they " "do not support being paused at the OS level (e.g. tasks requiring active network connections)." - ), + ) ) parser.add_argument( - "--log-hub-address", "-la", default=None, type=ZMQConfig.from_string, help="address for Worker send logs" + "--log-hub-address", "-la", default=None, type=Address.from_str, help="address for Worker send logs" ) parser.add_argument( "--logging-paths", @@ -105,7 +105,7 @@ def get_args(): help="use standard python the .conf file the specify python logging file configuration format, this will " "bypass --logging-paths and --logging-level at the same time, and this will not work on per worker logging", ) - parser.add_argument("address", type=ZMQConfig.from_string, help="scheduler address to connect to") + parser.add_argument("address", type=Address.from_str, help="scheduler address to connect to") return parser.parse_args() diff --git a/scaler/entry_points/scheduler.py b/scaler/entry_points/scheduler.py index 3809c8af..f90c5a86 100644 --- a/scaler/entry_points/scheduler.py +++ b/scaler/entry_points/scheduler.py @@ -17,12 +17,13 @@ from scaler.scheduler.scheduler import scheduler_main from scaler.utility.event_loop import EventLoopType, register_event_loop from scaler.utility.logging.utility import setup_logger -from scaler.utility.zmq_config import ZMQConfig + +from scaler.io.model import Address def get_args(): parser = argparse.ArgumentParser("scaler scheduler", formatter_class=argparse.ArgumentDefaultsHelpFormatter) - parser.add_argument("--io-threads", type=int, default=DEFAULT_IO_THREADS, help="number of io threads for zmq") + parser.add_argument("--io-threads", type=int, default=DEFAULT_IO_THREADS, help="number of io threads") parser.add_argument( "--max-number-of-tasks-waiting", "-mt", @@ -102,7 +103,7 @@ def get_args(): help="use standard python the .conf file the specify python logging file configuration format, this will " "bypass --logging-path", ) - parser.add_argument("address", type=ZMQConfig.from_string, help="scheduler address to connect to") + parser.add_argument("address", type=Address.from_str, help="scheduler address to connect to") return parser.parse_args() diff --git a/scaler/entry_points/top.py b/scaler/entry_points/top.py index 6a5f6533..6cc32947 100644 --- a/scaler/entry_points/top.py +++ b/scaler/entry_points/top.py @@ -13,7 +13,6 @@ format_percentage, format_seconds, ) -from scaler.utility.zmq_config import ZMQConfig SORT_BY_OPTIONS = { ord("n"): "worker", diff --git a/scaler/io/async_binder.py b/scaler/io/async_binder.py index bb2e1923..c66fc8e0 100644 --- a/scaler/io/async_binder.py +++ b/scaler/io/async_binder.py @@ -2,79 +2,65 @@ import os import uuid from collections import defaultdict -from typing import Awaitable, Callable, Dict, List, Optional - -import zmq.asyncio -from zmq import Frame from scaler.io.utility import deserialize, serialize from scaler.protocol.python.mixins import Message from scaler.protocol.python.status import BinderStatus from scaler.utility.mixins import Looper, Reporter -from scaler.utility.zmq_config import ZMQConfig +from scaler.io.model import ConnectorType, IoContext, Address, Connector + +from typing import TypeAlias, Callable, Awaitable + +BinderCallback: TypeAlias = Callable[[bytes, Message], Awaitable[None]] class AsyncBinder(Looper, Reporter): - def __init__(self, context: zmq.asyncio.Context, name: str, address: ZMQConfig, identity: Optional[bytes] = None): - self._address = address + _connector: Connector + _identity: bytes + _callback: BinderCallback | None + _received: dict[str, int] + _sent: dict[str, int] + def __init__(self, session: IoContext, name: str, address: Address, identity: bytes | None = None) -> None: if identity is None: identity = f"{os.getpid()}|{name}|{uuid.uuid4()}".encode() self._identity = identity - self._context = context - self._socket = self._context.socket(zmq.ROUTER) - self.__set_socket_options() - self._socket.bind(self._address.to_address()) + self._callback = None + self._received = defaultdict(lambda: 0) + self._sent = defaultdict(lambda: 0) - self._callback: Optional[Callable[[bytes, Message], Awaitable[None]]] = None - - self._received: Dict[str, int] = defaultdict(lambda: 0) - self._sent: Dict[str, int] = defaultdict(lambda: 0) + self._connector = Connector(session, self._identity, ConnectorType.Router, address.protocol) + self._connector.bind(address) def destroy(self): - self._context.destroy(linger=0) + self._connector.destroy() - def register(self, callback: Callable[[bytes, Message], Awaitable[None]]): + def register(self, callback: BinderCallback) -> None: self._callback = callback - async def routine(self): - frames: List[Frame] = await self._socket.recv_multipart(copy=False) - if not self.__is_valid_message(frames): - return + async def send(self, to: bytes, message: Message) -> None: + self.__count_sent(message.__class__.__name__) + await self._connector.send(to=to, data=serialize(message)) + + async def routine(self) -> None: + client_msg = await self._connector.recv() + message = deserialize(client_msg.payload) - source, payload = frames - message: Optional[Message] = deserialize(payload.bytes) if message is None: - logging.error(f"received unknown message from {source.bytes!r}: {payload!r}") + logging.error(f"received unknown message from {client_msg.address!r}: {client_msg.payload!r}") return self.__count_received(message.__class__.__name__) - await self._callback(source.bytes, message) - - async def send(self, to: bytes, message: Message): - self.__count_sent(message.__class__.__name__) - await self._socket.send_multipart([to, serialize(message)], copy=False) + await self._callback(client_msg.address, message) def get_status(self) -> BinderStatus: return BinderStatus.new_msg(received=self._received, sent=self._sent) - def __set_socket_options(self): - self._socket.setsockopt(zmq.IDENTITY, self._identity) - self._socket.setsockopt(zmq.SNDHWM, 0) - self._socket.setsockopt(zmq.RCVHWM, 0) - - def __is_valid_message(self, frames: List[Frame]) -> bool: - if len(frames) < 2: - logging.error(f"{self.__get_prefix()} received unexpected frames {frames}") - return False - - return True - - def __count_received(self, message_type: str): + def __count_received(self, message_type: str) -> None: self._received[message_type] += 1 - def __count_sent(self, message_type: str): + def __count_sent(self, message_type: str) -> None: self._sent[message_type] += 1 def __get_prefix(self): diff --git a/scaler/io/async_connector.py b/scaler/io/async_connector.py index a9a02c4c..af51fabd 100644 --- a/scaler/io/async_connector.py +++ b/scaler/io/async_connector.py @@ -1,104 +1,80 @@ import logging import os import uuid -from typing import Awaitable, Callable, List, Literal, Optional +from typing import Literal -import zmq.asyncio from scaler.io.utility import deserialize, serialize from scaler.protocol.python.mixins import Message -from scaler.utility.zmq_config import ZMQConfig + +from scaler.io.model import Connector, ConnectorType, IoContext, Address + + + +from typing import TypeAlias, Callable, Awaitable + +ConnectorCallback: TypeAlias = Callable[[Message], Awaitable[None]] class AsyncConnector: + _connector: Connector + _address: Address + _identity: bytes + _callback: ConnectorCallback | None + def __init__( self, - context: zmq.asyncio.Context, + session: IoContext, name: str, - socket_type: int, - address: ZMQConfig, + type_: ConnectorType, + address: Address, bind_or_connect: Literal["bind", "connect"], - callback: Optional[Callable[[Message], Awaitable[None]]], - identity: Optional[bytes], + callback: ConnectorCallback | None, + identity: bytes | None, ): - self._address = address - - self._context = context - self._socket = self._context.socket(socket_type) - if identity is None: identity = f"{os.getpid()}|{name}|{uuid.uuid4().bytes.hex()}".encode() self._identity = identity - # set socket option - self._socket.setsockopt(zmq.IDENTITY, self._identity) - self._socket.setsockopt(zmq.SNDHWM, 0) - self._socket.setsockopt(zmq.RCVHWM, 0) - - if bind_or_connect == "bind": - self._socket.bind(self._address.to_address()) - elif bind_or_connect == "connect": - self._socket.connect(self._address.to_address()) - else: - raise TypeError("bind_or_connect has to be 'bind' or 'connect'") - - self._callback: Optional[Callable[[Message], Awaitable[None]]] = callback + self._address = address + self._callback = callback + self._connector = Connector(session, self._identity, type_, self._address.protocol) - def __del__(self): - self.destroy() + match bind_or_connect: + case "bind": + self._connector.bind(self._address) + case "connect": + self._connector.connect(self._address) + case _: + raise TypeError("bind_or_connect must be 'bind' or 'connect'") def destroy(self): - if self._socket.closed: - return + self._connector.destroy() - self._socket.close(linger=1) + @property + def address(self) -> str: + return str(self._address) @property def identity(self) -> bytes: return self._identity - @property - def socket(self) -> zmq.asyncio.Socket: - return self._socket - - @property - def address(self) -> str: - return self._address.to_address() - - async def routine(self): + async def routine(self) -> None: if self._callback is None: return - message: Optional[Message] = await self.receive() + client_msg = await self._connector.recv() + message = deserialize(client_msg.payload) + if message is None: + logging.error(f"received unknown message: {client_msg.payload!r}") return await self._callback(message) - async def receive(self) -> Optional[Message]: - if self._context.closed: - return None - - if self._socket.closed: - return None - - payload = await self._socket.recv(copy=False) - result: Optional[Message] = deserialize(payload.bytes) - if result is None: - logging.error(f"received unknown message: {payload.bytes!r}") - return None - - return result - - async def send(self, message: Message): - await self._socket.send(serialize(message), copy=False) - - def __is_valid_message(self, frames: List[bytes]) -> bool: - if len(frames) > 1: - logging.error(f"{self.__get_prefix()} received unexpected frames {frames}") - return False - return True + async def send(self, message: Message) -> None: + await self._connector.send(data=serialize(message)) def __get_prefix(self): return f"{self.__class__.__name__}[{self._identity.decode()}]:" diff --git a/scaler/io/config.py b/scaler/io/config.py index a20f5fed..53423306 100644 --- a/scaler/io/config.py +++ b/scaler/io/config.py @@ -21,7 +21,7 @@ # ========================== # SCHEDULER SPECIFIC OPTIONS -# number of threads for zmq socket to handle +# number of io threads DEFAULT_IO_THREADS = 1 # if all workers are full and busy working, this option determine how many additional tasks scheduler can receive and diff --git a/scaler/io/cpp/CMakeLists.txt b/scaler/io/cpp/CMakeLists.txt new file mode 100644 index 00000000..ba7671e4 --- /dev/null +++ b/scaler/io/cpp/CMakeLists.txt @@ -0,0 +1,14 @@ +cmake_minimum_required(VERSION 4.0) + +set(CMAKE_CXX_STANDARD 23) +set(CMAKE_CXX_STANDARD_REQUIRED ON) + +add_executable(example) + +target_sources(example PRIVATE src/example.cpp) + +# Add sanitizers in debug mode +if(CMAKE_BUILD_TYPE STREQUAL "Debug") + target_compile_options(example PRIVATE -fsanitize=address -fsanitize=leak -fsanitize=undefined) + target_link_options(example PRIVATE -fsanitize=address -fsanitize=leak -fsanitize=undefined) +endif() diff --git a/scaler/io/cpp/__init__.py b/scaler/io/cpp/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/scaler/io/cpp/build.py b/scaler/io/cpp/build.py new file mode 100644 index 00000000..786bd0e5 --- /dev/null +++ b/scaler/io/cpp/build.py @@ -0,0 +1,26 @@ +from cffi import FFI + +builder = FFI() + +with open("src/defs.h") as f: + defs = f.read() + +builder.cdef(defs) + +import os + +builder.set_source( + "cpp", + '#include "src/main.hpp"', + source_extension=".cpp", + # runs gcc like it's g++ and links the C++ standard library + extra_compile_args=( + ["-std=c++23", "-Wall", "-Wextra", "-pedantic"] + if os.getenv("CC") == "clang" + else ["-xc++", "-lstdc++", "-shared-libgcc", "-std=gnu++23", "-Wall", "-Wextra", "-Werror", "-O0"] + ), + language="c++", +) + +if __name__ == "__main__": + builder.compile(verbose=True) diff --git a/scaler/io/cpp/errors.py b/scaler/io/cpp/errors.py new file mode 100644 index 00000000..910721f2 --- /dev/null +++ b/scaler/io/cpp/errors.py @@ -0,0 +1,87 @@ +from scaler.io.cpp.ffi import ffi, lib, FFITypes + +from enum import IntEnum, unique +import os +import signal + +def check_status(value: "FFITypes.CData") -> None: + exception = CppException.from_status(value) + if exception is not None: + raise exception + +@unique +class Code(IntEnum): + AlreadyBound = lib.AlreadyBound + InvalidAddress = lib.InvalidAddress + UnsupportedOperation = lib.UnsupportedOperation + NoThreads = lib.NoThreads + +class CppException(Exception): + message: str | None + + def __init__(self, message: str | None): + self.message = message + + @staticmethod + def from_status(status: "FFITypes.CData") -> "CppException | None": + if status.message == ffi.NULL: + message = None + else: + message = ffi.string(status.message).decode("utf-8") + + match status.type: + case lib.Ok: + return None + case lib.Errno: + return ErrnoException(status.no, message) + case lib.Logical: + return LogicalException(Code(status.code), message) + case lib.Signal: + return SignalException(status.signal, message) + +class LogicalException(CppException): + code: Code + + def __init__(self, code: Code, message: str | None): + super().__init__(message) + self.code = code + + def __str__(self): + if self.message is None: + return f"Logical error: Code: {self.code.name} ({self.code.value}" + return f"Logical error: Code: {self.code.name} ({self.code.value}): {self.message}" + +class ErrnoException(CppException): + errno: int + + def __init__(self, errno: int, message: str | None): + super().__init__(message) + self.errno = errno + + + def __str__(self): + if self.message is None: + return os.strerror(self.errno) + return f"{self.message}: {os.strerror(self.errno)}" + +class SignalException(CppException): + signal: int + + def __init__(self, signal: int, message: str | None): + super().__init__(message) + self.signal = signal + + def __str__(self): + if self.message is None: + return f"Signal error: Signal: {self.strsignal} ({self.signame})" + return f"Signal error: Signal: {self.strsignal} ({self.signame}): {self.message}" + + @property + def signame(self) -> str: + """e.g. SIGINT""" + return signal.Signals(self.signal).name + + @property + def strsignal(self) -> str: + """e.g. Interrupted""" + return signal.strsignal(self.signal) diff --git a/scaler/io/cpp/ffi.py b/scaler/io/cpp/ffi.py new file mode 100644 index 00000000..cfa41036 --- /dev/null +++ b/scaler/io/cpp/ffi.py @@ -0,0 +1,135 @@ +__ALL__ = ["FFITypes", "ffi", "lib", "c_async", "c_async_wrap"] + +from .cpp import ffi, lib +from cffi import FFI as FFITypes + +from .errors import CppException + + +class LibType: + Pair: int + Pub: int + Sub: int + Dealer: int + Router: int + + TCP: int + IntraProcess: int + InterProcess: int + + AlreadyBound: int + InvalidAddress: int + UnsupportedOperation: int + NoThreads: int + PeerShutdown: int + + Ok: int + Errno: int + Logical: int + Signal: int + + def io_context_init(ioctx: "FFITypes.CData", num_threads: int) -> None: + (ioctx, num_threads) + + def io_context_destroy(ioctx: "FFITypes.CData") -> None: + (ioctx,) + + def message_destroy(recv: "FFITypes.CData") -> None: + (recv,) + + def connector_init(ioctx: "FFITypes.CData", connector: "FFITypes.CData", transport: int, type_: int, identity: bytes, len: int) -> None: + (ioctx, connector, transport, type_, identity, len) + + def connector_bind(connector: "FFITypes.CData", host: bytes, port: int) -> "FFITypes.CData": + (connector, host, port) + + def connector_connect(connector: "FFITypes.CData", host: bytes, port: int) -> None: + (connector, host, port) + + def connector_send_async( + future: "FFITypes.CData", connector: "FFITypes.CData", to: bytes, to_len: int, data: bytes, data_len: int + ) -> None: + (future, connector, to, to_len, data, data_len) + + def connector_send_sync(connector: "FFITypes.CData", to: bytes, to_len: int, data: bytes, data_len: int) -> None: + (connector, to, to_len, data, data_len) + + def connector_recv_async(future: "FFITypes.CData", connector: "FFITypes.CData") -> None: + (future, connector) + + def connector_recv_sync(connector: "FFITypes.CData", msg: "FFITypes.CData") -> None: + (connector, msg) + + def connector_destroy(connector: "FFITypes.CData") -> None: + (connector,) + + +# type hints for FFI and Lib +ffi: FFITypes +lib: LibType + +import asyncio +from typing import Callable, ParamSpec, TypeVar, Concatenate + + +class Message: + __match_args__ = ("address", "payload") + + address: bytes + payload: bytes + + def __init__(self, obj: "FFITypes.CData"): # Message * + # copy the address + self.address = bytes(ffi.buffer(obj.address.data, obj.address.len)) + # copy the payload + self.payload = bytes(ffi.buffer(obj.payload.data, obj.payload.len)) + + +# this is called from C to inform the asyncio runtime that a future was completed +@ffi.def_extern() +def future_set_result(future_handle: "FFITypes.CData", result: "FFITypes.CData") -> None: + if result == ffi.NULL: + result = None + else: + msg = ffi.cast("struct Message *", result) + result = Message(msg) + + future: asyncio.Future = ffi.from_handle(future_handle) + + if future.done(): + return + + # using `call_soon_threadsafe()` is very important: + # - https://docs.python.org/3/library/asyncio-eventloop.html#asyncio.loop.call_soon_threadsafe + future.get_loop().call_soon_threadsafe(future.set_result, result) + +@ffi.def_extern() +def future_set_status(future_handle: "FFITypes.CData", status: "FFITypes.CData") -> None: + status = ffi.cast("struct Status *", status) + future: asyncio.Future = ffi.from_handle(future_handle) + + if future.done(): + return + + exc = CppException.from_status(status) + + # status was "ok" + if exc is None: + future.get_loop().call_soon_threadsafe(future.set_result, None) + else: + future.get_loop().call_soon_threadsafe(future.set_exception, exc) + +# these type variables make type hints work +# in Python 3.12+ we can use the new syntax instead of defining these: +# async def c_async[**P, R](...): ... +P = ParamSpec("P") +R = TypeVar("R") + + +# c_async is a helper function to call async C functions +# example: c_async(lib.async_binder_recv, binder) +async def c_async(fn: Callable[Concatenate["FFITypes.CData", P], R], *args: P.args, **kwargs: P.kwargs) -> R: + future = asyncio.get_running_loop().create_future() + handle = ffi.new_handle(future) + fn(handle, *args, **kwargs) + return await future diff --git a/scaler/io/cpp/src/common.hpp b/scaler/io/cpp/src/common.hpp new file mode 100644 index 00000000..ebb22bd8 --- /dev/null +++ b/scaler/io/cpp/src/common.hpp @@ -0,0 +1,508 @@ +#pragma once + +// cffi generates C code, which doesn't like C++'s enum class +// so we use plain enums during compilation +// you can set the TYPECK define in your IDE for safer enum typing +#ifdef TYPECK +#define ENUM enum class +#else +#define ENUM enum +#endif + +#include + +// C +#include +#include +#include +// #include "backtrace.h" + +// C++ +#include +#include +#include + +// System +#include +#include +#include +#include +#include +#include +#include + +// #define mydebug() std::cout << __FILE__ << ":" << __LINE__ << ":" << __PRETTY_FUNCTION__ << std::endl + +#define mydebug() ; + +// Python callback +void future_set_result(void* future, void* data); +void future_set_status(void* future, void* status); + +#define PROPAGATE(expr) \ + if (auto status = (expr); status.type != StatusType::Ok) \ + return status; + +void print_trace(void) { + void* array[10]; + char** strings; + int size, i; + + size = backtrace(array, 10); + strings = backtrace_symbols(array, size); + if (strings != NULL) { + printf("Obtained %d stack frames.\n", size); + for (i = 0; i < size; i++) + printf("%s\n", strings[i]); + } + + free(strings); +} + +// this is an unrecoverable error that exits the program +// prints a message plus the source location +[[noreturn]] void panic(std::string message, const std::source_location& location = std::source_location::current()) { + auto file_name = std::string(location.file_name()); + file_name = file_name.substr(file_name.find_last_of("/") + 1); + + std::cout << "panic at " << file_name << ":" << location.line() << ":" << location.column() << " in function [" + << location.function_name() << "]: " << message << std::endl; + + print_trace(); + + std::abort(); +} + +[[noreturn]] void unreachable(const std::source_location& location = std::source_location::current()) { + panic("unreachable", location); +} + +ENUM Code {AlreadyBound, InvalidAddress, UnsupportedOperation, NoThreads, PeerShutdown}; + +ENUM StatusType {Ok, Logical, Errno, Signal}; + +struct Status { + StatusType type; + const char* message; + union { + Code code; + int no; + int signal; + }; + + bool is_ok() const { return type == StatusType::Ok; } + + static Status ok() { return {.type = StatusType::Ok, .message = NULL, .code = (Code)0}; } + + static Status from_code(const char* message, Code code) { + return { + .type = StatusType::Logical, + .message = message, + .code = code, + }; + } + + static Status from_errno(const char* message, int err = errno) { + return { + .type = StatusType::Errno, + .message = message, + .no = err, + }; + } + + static Status from_signal(const char* message, int signal) { + return { + .type = StatusType::Signal, + .message = message, + .signal = signal, + }; + } +}; + +// how to control flow +// continue is truthy +// break is falsy +struct ControlFlow { + enum Value { Continue, Break } value; + + constexpr ControlFlow(Value value): value(value) {} + constexpr operator Value() const { return value; } + + operator bool() const { + switch (this->value) { + case Continue: return true; + case Break: return false; + } + + unreachable(); + } +}; + +uint8_t* datadup(const uint8_t* data, size_t len) { + uint8_t* dup = new uint8_t[len]; + std::memcpy(dup, data, len); + return dup; +} + +struct Bytes { + uint8_t* data; + size_t len; + + enum { Owned, Borrowed } tag; + + void free() { + if (tag != Owned) + return; + + if (is_empty()) + return; + + delete[] data; + this->data = NULL; + } + + bool operator==(const Bytes& other) const { + if (len != other.len) + return false; + + if (data == other.data) + return true; + + return std::memcmp(data, other.data, len) == 0; + } + + bool operator!() const { return is_empty(); } + + bool is_empty() const { return this->data == NULL; } + + // debugging utility + std::string as_string() const { + if (is_empty()) + return "[EMPTY]"; + + return std::string((char*)data, len); + } + + Bytes ref() { return {.data = this->data, .len = this->len, .tag = Borrowed}; } + + static Bytes alloc(size_t len) { return {.data = new uint8_t[len], .len = len, .tag = Owned}; } + + static Bytes empty() { + return { + .data = NULL, + .len = 0, + .tag = Owned, + }; + } + + static Bytes copy(const uint8_t* data, size_t len) { + return { + .data = datadup(data, len), + .len = len, + .tag = Owned, + }; + } + + static Bytes clone(const Bytes& bytes) { + if (bytes.is_empty()) + panic("tried to clone empty bytes"); + + return { + .data = datadup(bytes.data, bytes.len), + .len = bytes.len, + .tag = Owned, + }; + } +}; + +struct Message { + // the address the message was received from, or to send to + // + // for received messages, the address data is + // owned by the peer it was received from + Bytes address; + + // the payload of the message + // + // for received messages, the payload data is owned + // and must be freed when the message is destroyed + Bytes payload; +}; + +// free a message's resources +void message_destroy(Message* msg) { + mydebug(); + msg->payload.free(); + msg->address.free(); +} + +void serialize_u32(uint32_t x, uint8_t buffer[4]) { + buffer[0] = x & 0xFF; + buffer[1] = (x >> 8) & 0xFF; + buffer[2] = (x >> 16) & 0xFF; + buffer[3] = (x >> 24) & 0xFF; +} + +void deserialize_u32(const uint8_t buffer[4], uint32_t* x) { + *x = buffer[0] | buffer[1] << 8 | buffer[2] << 16 | buffer[3] << 24; +} + +typedef uint64_t timerfd_t; + +// timerfd analogue of eventfd_read() +// 0 -> ok, read the value from the buffer +// -1 -> error, check errno +int timerfd_read(int fd, timerfd_t* value) { + auto n = read(fd, (uint8_t*)value, sizeof(timerfd_t)); + + if (n > 0) { + if (n != sizeof(timerfd_t)) + panic("failed to read timerfd: " + std::to_string(errno)); + + return 0; + } + + return -1; +} + +// read a timerfd value and discard it +// return value is the same as timerfd_read() +int timerfd_read2(int fd) { + timerfd_t value; + return timerfd_read(fd, &value); +} + +// reset the timerfd to 0 +void timerfd_reset(int fd) { + itimerspec spec { + .it_interval = {.tv_sec = 0, .tv_nsec = 0}, + .it_value = {.tv_sec = 0, .tv_nsec = 0}, + }; + + if (timerfd_settime(fd, 0, &spec, NULL) < 0) + panic("failed to reset timerfd: " + std::to_string(errno)); +} + +int eventfd_wait(int fd) { + eventfd_t value; + return eventfd_read(fd, &value); +} + +int eventfd_signal(int fd) { + return eventfd_write(fd, 1); +} + +int eventfd_reset(int fd) { + for (;;) { + if (eventfd_wait(fd) < 0) { + if (errno == EAGAIN) + return 0; + + return -1; + } + } +} + +enum FdWait : int8_t { + Ready = 0, + Timeout = -1, + Other = -2, +}; + +// wait for an event on a file descriptor, or for a signal to arrive, possibly with a timeout +// +// fd: the file descriptor to wait on +// timeout: te number of milliseconds to wait, or -1 to wait indefinitely +// events: the events to wait for, e.g. POLLIN, POLLOUT -- passed directly to poll() +// +// return value: +// - Fdwait::Ready (0): the file descriptor is ready +// - Fdwait::Timeout (-1): the timeout expired +// - Fdwait::Other (-2): fd_wait failed for some other reason; check errno +// - (>0): a signal was received, the value is the signal number +int8_t fd_wait(int fd, int timeout, short int events) { + pollfd fds[2]; + + fds[0] = { + .fd = fd, + .events = events, + .revents = 0, + }; + + sigset_t sigs; + sigemptyset(&sigs); + sigaddset(&sigs, SIGINT); + sigaddset(&sigs, SIGQUIT); + sigaddset(&sigs, SIGTERM); + + auto signal_fd = signalfd(-1, &sigs, 0); + + fds[1] = { + .fd = signal_fd, + .events = POLLIN, + .revents = 0, + }; + + auto n = poll(fds, 2, timeout); + + if (n == 0) { + close(signal_fd); + return (int8_t)FdWait::Timeout; + } + + if (n < 0) { + close(signal_fd); + return (int8_t)FdWait::Other; + } + + if (fds[1].revents & POLLIN) { + signalfd_siginfo info; + + if (read(signal_fd, &info, sizeof(info)) != sizeof(info)) + return FdWait::Other; + + close(signal_fd); + return info.ssi_signo; + } + + close(signal_fd); + return 0; +} + +// not thread safe, but can be copied safely +struct Completer { + ENUM Type {None, Future, Semaphore} type; + + // owned by the caller + union { + void* future_ptr; + sem_t* sem; + }; + + // must be allocated with `new` + // this is why the completer is not thread safe + // but can be copied and shared within the same thread + uint8_t* counter; + + bool completed() const { return counter == NULL; } + + void set_counter(uint8_t counter) { + if (this->completed()) + panic("counter already completed"); + + if (counter <= 0) + panic("counter must be > 0"); + + *this->counter = counter; + } + + // complete with a result + // may be NULL + // not thread safe + void complete(void* result = NULL) { + if (this->completed()) + panic("counter already completed"); + + (*this->counter)--; + + if (*this->counter > 0) + return; + + delete this->counter; + this->counter = NULL; + + switch (this->type) { + case Completer::Type::None: break; + case Completer::Type::Future: future_set_result(this->future_ptr, result); break; + case Completer::Type::Semaphore: + if (sem_post(this->sem) < 0) + panic("failed to post semaphore: " + std::to_string(errno)); + break; + } + } + + void complete_status(Status* status) { + if (this->completed()) + panic("counter already completed"); + + (*this->counter)--; + + if (*this->counter > 0) + return; + + delete this->counter; + this->counter = NULL; + + switch (this->type) { + case Completer::Type::None: break; + case Completer::Type::Future: future_set_status(this->future_ptr, status); break; + case Completer::Type::Semaphore: + if (sem_post(this->sem) < 0) + panic("failed to post semaphore: " + std::to_string(errno)); + break; + } + } + + void complete_ok() { + auto status = Status::ok(); + this->complete_status(&status); + } + + static constexpr Completer none(uint8_t counter = 1) { + return { + .type = Type::None, + .future_ptr = nullptr, + .counter = new uint8_t(counter), + }; + } + + static Completer future(void* future, uint8_t counter = 1) { + return { + .type = Type::Future, + .future_ptr = future, + .counter = new uint8_t(counter), + }; + } + + static Completer semaphore(sem_t* sem, uint8_t counter = 1) { + return { + .type = Type::Semaphore, + .sem = sem, + .counter = new uint8_t(counter), + }; + } +}; + +ENUM IoProgress: uint8_t {Header, Payload}; + +// an in-progress io operation +struct IoOperation { + IoProgress progress; + Completer completer; + size_t cursor; + + uint8_t buffer[4]; + Bytes payload; + + bool completed() const { return progress == IoProgress::Payload && cursor == payload.len; } + + static IoOperation read(Completer completer = Completer::none()) { + return { + .progress = IoProgress::Header, + .completer = completer, + .cursor = 0, + .buffer = {0}, + .payload = Bytes::empty()}; + } + + // the payload must live at least as long as the operation does + static IoOperation write(Bytes payload, Completer completer = Completer::none()) { + return { + .progress = IoProgress::Header, + .completer = completer, + .cursor = 0, + .buffer = {0}, + .payload = payload, + }; + } +}; diff --git a/scaler/io/cpp/src/connector.cpp b/scaler/io/cpp/src/connector.cpp new file mode 100644 index 00000000..8bb1fc68 --- /dev/null +++ b/scaler/io/cpp/src/connector.cpp @@ -0,0 +1,89 @@ +#include "connector.hpp" + +Status connector_init( + IoContext* ioctx, Connector* connector, Transport transport, ConnectorType type, uint8_t* identity, size_t len) { + switch (transport) { + case Transport::TCP: + case Transport::InterProcess: { + new (connector) Connector {.type = Connector::Socket, .network = nullptr}; + + return network_connector_init(ioctx, &connector->network, transport, type, identity, len); + } + case Transport::IntraProcess: { + new (connector) Connector {.type = Connector::IntraProcess, .intra_process = nullptr}; + + return intra_process_init(ioctx, &connector->intra_process, identity, len); + } + default: unreachable(); + } +} + +Status connector_destroy(Connector* connector) { + switch (connector->type) { + case Connector::Socket: { + auto status = network_connector_destroy(connector->network); + delete connector->network; + return status; + } + case Connector::IntraProcess: { + auto status = intra_process_destroy(connector->intra_process); + delete connector->intra_process; + return status; + } + default: unreachable(); + } +} + +Status connector_connect(Connector* connector, const char* host, uint16_t port) { + switch (connector->type) { + case Connector::Socket: return network_connector_connect(connector->network, host, port); + case Connector::IntraProcess: return intra_process_connect(connector->intra_process, host); + default: unreachable(); + } +} + +Status connector_bind(Connector* connector, const char* host, uint16_t port) { + switch (connector->type) { + case Connector::Socket: return network_connector_bind(connector->network, host, port); + case Connector::IntraProcess: return intra_process_bind(connector->intra_process, host); + default: unreachable(); + } +} + +void connector_send_async( + void* future, Connector* connector, uint8_t* to, size_t to_len, uint8_t* data, size_t data_len) { + switch (connector->type) { + case Connector::Socket: + return network_connector_send_async(future, connector->network, to, to_len, data, data_len); + case Connector::IntraProcess: { + // intraprocess only support sync send, so perform the send synchronously and complete the future + auto status = intra_process_send_sync(connector->intra_process, data, data_len); + return future_set_status(future, &status); + } + default: unreachable(); + } +} + +Status connector_send_sync(Connector* connector, uint8_t* to, size_t to_len, uint8_t* data, size_t data_len) { + switch (connector->type) { + case Connector::Socket: return network_connector_send_sync(connector->network, to, to_len, data, data_len); + case Connector::IntraProcess: return intra_process_send_sync(connector->intra_process, data, data_len); + default: unreachable(); + } +} + +void connector_recv_async(void* future, Connector* connector) { + switch (connector->type) { + case Connector::Socket: return network_connector_recv_async(future, connector->network); + case Connector::IntraProcess: return intra_process_recv_async(future, connector->intra_process); + default: unreachable(); + } +} + +Status connector_recv_sync(Connector* connector, struct Message* msg) { + switch (connector->type) { + case Connector::Socket: return network_connector_recv_sync(connector->network, msg); + case Connector::IntraProcess: return intra_process_recv_sync(connector->intra_process, msg); + default: unreachable(); + } +} diff --git a/scaler/io/cpp/src/connector.hpp b/scaler/io/cpp/src/connector.hpp new file mode 100644 index 00000000..53bccf8c --- /dev/null +++ b/scaler/io/cpp/src/connector.hpp @@ -0,0 +1,46 @@ +#pragma once + +#include "intra_process_connector.hpp" +#include "network_connector.hpp" + +// rationale: +// - the interface for intraprocess and socket-based connectors needs to be unified +// +// lifetime: +// - the connector lives within the scope of the io context and thread context +// - all connectors MUST be destroyed before the io context is destroyed +// - its lifetime begins when you call `connector_init()` and ends when you call `connector_destroy()` +// - its lifetime is flexible and determined by the library user +// +// usage: +// - the connector is the primary interface of the library +// - the connector is used to send and receive messages +// - use `connector_*()` functions to operate on the connector +// +// assumptions: +// - the user is not going to access the internal state of the connector directly +struct Connector +{ + enum Type + { + Socket, + IntraProcess + } type; + + union + { + IntraProcessConnector *intra_process; + NetworkConnector *network; + }; +}; + +extern "C" { +Status connector_init(IoContext* ioctx, Connector *connector, Transport transport, ConnectorType type, uint8_t *identity, size_t len); +Status connector_destroy(Connector *connector); +Status connector_bind(Connector *connector, const char *host, uint16_t port); +Status connector_connect(Connector *connector, const char *host, uint16_t port); +void connector_send_async(void *future, Connector *connector, uint8_t *to, size_t to_len, uint8_t *data, size_t data_len); +Status connector_send_sync(Connector *connector, uint8_t *to, size_t to_len, uint8_t *data, size_t data_len); +void connector_recv_async(void *future, Connector *connector); +Status connector_recv_sync(Connector *connector, Message *msg); +} diff --git a/scaler/io/cpp/src/defs.h b/scaler/io/cpp/src/defs.h new file mode 100644 index 00000000..6d7e1fbc --- /dev/null +++ b/scaler/io/cpp/src/defs.h @@ -0,0 +1,90 @@ +// this file contains C-compatible definitions for the C++ code in the other files +// this is the interface exposed to Python + +struct Bytes +{ + uint8_t *data; + size_t len; + ...; +}; + +struct Message +{ + struct Bytes address; + struct Bytes payload; +}; + +enum ConnectorType +{ + Pair, + Sub, + Pub, + Dealer, + Router +}; + +enum Transport +{ + TCP, + IntraProcess, + InterProcess +}; + +struct IoContext +{ + // this means that Python doesn't know about the internals of the struct + // the compiler will figure out the size of the struct for us + // based upon the full definition in the C++ code + ...; +}; + +struct Connector +{ + ...; +}; + +enum Code +{ + AlreadyBound, + InvalidAddress, + UnsupportedOperation, + NoThreads, + PeerShutdown +}; + +enum StatusType +{ + Ok, + Logical, + Errno, + Signal +}; + +struct Status +{ + enum StatusType type; + const char *message; + union + { + enum Code code; + int no; + int signal; + }; +}; + +// Python callback +extern "Python+C" void future_set_result(void *future, void *data); +extern "Python+C" void future_set_status(void *future, void *status); + +struct Status io_context_init(struct IoContext *ioctx, size_t num_threads); +struct Status io_context_destroy(struct IoContext *ioctx, bool destruct); +void message_destroy(struct Message *message); + +struct Status connector_init(struct IoContext *ioctx, struct Connector *connector, enum Transport transport, enum ConnectorType type, uint8_t *identity, size_t len); +struct Status connector_destroy(struct Connector *connector); +struct Status connector_bind(struct Connector *connector, const char *host, uint16_t port); +struct Status connector_connect(struct Connector *connector, const char *host, uint16_t port); +void connector_send_async(void *future, struct Connector *connector, uint8_t *to, size_t to_len, uint8_t *data, size_t data_len); +struct Status connector_send_sync(struct Connector *connector, uint8_t *to, size_t to_len, uint8_t *data, size_t data_len); +void connector_recv_async(void *future, struct Connector *connector); +struct Status connector_recv_sync(struct Connector *connector, struct Message *msg); diff --git a/scaler/io/cpp/src/example.cpp b/scaler/io/cpp/src/example.cpp new file mode 100644 index 00000000..d79d6774 --- /dev/null +++ b/scaler/io/cpp/src/example.cpp @@ -0,0 +1,183 @@ +#include "main.hpp" + +#define CHECK(expr) \ + if (auto status = (expr); status.type != StatusType::Ok) \ + panic(status.message, std::source_location::current()) + +struct Future { + sem_t* sem; + enum { Msg, Status_, Initial } tag; + + union { + Message* msg; + Status status; + }; + + void complete_msg(Message* msg) { + tag = Msg; + + this->msg = new Message { + .address = Bytes::clone(msg->address), + .payload = Bytes::clone(msg->payload), + }; + + if (sem_post(sem) < 0) + panic("failed to post semaphore"); + } + + void complete_status(Status* status) { + tag = Status_; + this->status = *status; + + if (sem_post(sem) < 0) + panic("failed to post semaphore"); + } + + void wait() { + if (sem_wait(sem) < 0) + panic("failed to wait on semaphore"); + } + + Future(): sem(new sem_t), tag(Initial) { + if (sem_init(sem, 0, 0) < 0) + panic("failed to initialize semaphore"); + }; + + ~Future() { + if (tag == Msg) { + delete msg; + } + + if (sem_destroy(sem) < 0) + panic("failed to destroy semaphore"); + delete sem; + } +}; + +void future_set_result(void* future, void* data) { + auto fut = static_cast(future); + auto msg = static_cast(data); + + fut->complete_msg(msg); +} + +void future_set_status(void* future, void* status) { + auto fut = static_cast(future); + auto stat = static_cast(status); + + fut->complete_status(stat); +} + +void example_one() { + IoContext ioctx; + CHECK(io_context_init(&ioctx, 0)); + + Connector conn_a, conn_b; + CHECK(connector_init(&ioctx, &conn_a, Transport::IntraProcess, ConnectorType::Pair, (uint8_t*)"conn_a", 6)); + CHECK(connector_init(&ioctx, &conn_b, Transport::IntraProcess, ConnectorType::Pair, (uint8_t*)"conn_b", 6)); + + CHECK(connector_bind(&conn_a, "conn_a", 0)); + CHECK(connector_connect(&conn_b, "conn_a", 0)); + + std::string content = "Hello from conn_a; example one"; + + CHECK(connector_send_sync(&conn_a, NULL, 0, (uint8_t*)content.c_str(), content.size())); + + Message msg; + CHECK(connector_recv_sync(&conn_b, &msg)); + + message_destroy(&msg); + + CHECK(connector_destroy(&conn_a)); + CHECK(connector_destroy(&conn_b)); + CHECK(io_context_destroy(&ioctx, false)); +} + +void example_two() { + IoContext ioctx; + CHECK(io_context_init(&ioctx, 1)); + + Connector conn_a, conn_b; + CHECK(connector_init(&ioctx, &conn_a, Transport::IntraProcess, ConnectorType::Pair, (uint8_t*)"conn_a", 6)); + CHECK(connector_init(&ioctx, &conn_b, Transport::IntraProcess, ConnectorType::Pair, (uint8_t*)"conn_b", 6)); + + std::string addr = "/tmp/conn_a"; + + CHECK(connector_bind(&conn_a, addr.c_str(), 0)); + CHECK(connector_connect(&conn_b, addr.c_str(), 0)); + + Future send, recv; + connector_recv_async(&recv, &conn_b); + + std::string content = "Hello from conn_a; example two"; + connector_send_async(&send, &conn_a, NULL, 0, (uint8_t*)content.c_str(), content.size()); + + send.wait(); + + if (send.tag != Future::Status_) { + panic("send failed: " + std::string(send.status.message)); + } + + if (send.status.type != StatusType::Ok) { + panic("send failed: " + std::string(send.status.message)); + } + + recv.wait(); + + if (recv.tag != Future::Msg) { + panic("recv failed: " + std::string(recv.status.message)); + } + + CHECK(connector_destroy(&conn_a)); + CHECK(connector_destroy(&conn_b)); + CHECK(io_context_destroy(&ioctx, false)); +} + +void example_three() { + IoContext ioctx; + CHECK(io_context_init(&ioctx, 1)); + + Connector conn_a, conn_b; + CHECK(connector_init(&ioctx, &conn_a, Transport::TCP, ConnectorType::Pair, (uint8_t*)"conn_a", 6)); + CHECK(connector_init(&ioctx, &conn_b, Transport::TCP, ConnectorType::Pair, (uint8_t*)"conn_b", 6)); + + std::string addr = "127.0.0.1"; + uint16_t port = 12345; + + CHECK(connector_bind(&conn_a, addr.c_str(), port)); + CHECK(connector_connect(&conn_b, addr.c_str(), port)); + + Future send, recv; + connector_recv_async(&recv, &conn_b); + + std::string content = "Hello from conn_a; example three"; + connector_send_async(&send, &conn_a, NULL, 0, (uint8_t*)content.c_str(), content.size()); + + send.wait(); + + if (send.tag != Future::Status_) { + panic("send failed: " + std::string(send.status.message)); + } + + if (send.status.type != StatusType::Ok) { + panic("send failed: " + std::string(send.status.message)); + } + + recv.wait(); + + if (recv.tag != Future::Msg) { + panic("recv failed: " + std::string(recv.status.message)); + } + + CHECK(connector_destroy(&conn_a)); + CHECK(connector_destroy(&conn_b)); + CHECK(io_context_destroy(&ioctx, false)); +} + +int main() { + example_one(); + example_two(); + example_three(); + + return 0; +} diff --git a/scaler/io/cpp/src/intra_process_connector.cpp b/scaler/io/cpp/src/intra_process_connector.cpp new file mode 100644 index 00000000..ebebcbd0 --- /dev/null +++ b/scaler/io/cpp/src/intra_process_connector.cpp @@ -0,0 +1,219 @@ +#include "intra_process_connector.hpp" + +Status IntraProcessConnector::ensure_epoll() { + if (this->thread == nullptr) + return Status::from_code("async operations require a session with threads", Code::NoThreads); + + if (this->epoll.exchange(true)) + return Status::ok(); + + this->thread->add_epoll(this->recv_buffer_event_fd, EPOLLIN | EPOLLET, EpollType::IntraProcessConnectorRecv, this); + this->thread->add_epoll(this->recv_event_fd, EPOLLIN | EPOLLET, EpollType::IntraProcessConnectorRecv, this); + + return Status::ok(); +} + +void IntraProcessConnector::remove_from_epoll() { + if (!this->epoll) + return; + + this->thread->remove_epoll(this->recv_buffer_event_fd); + this->thread->remove_epoll(this->recv_event_fd); +} + +Status intra_process_init(IoContext* ioctx, IntraProcessConnector** connector, uint8_t* identity, size_t len) { + *connector = new IntraProcessConnector { + .ioctx = ioctx, + .thread = ioctx->next_thread(), + .queue = ConcurrentQueue(), + .recv = ConcurrentQueue(), + .recv_buffer_event_fd = eventfd(0, EFD_NONBLOCK | EFD_SEMAPHORE), + .recv_event_fd = eventfd(0, EFD_NONBLOCK | EFD_SEMAPHORE), + .unmuted_event_fd = eventfd(0, EFD_NONBLOCK), + .identity = Bytes::copy(identity, len), + .bind = std::nullopt, + .connecting = std::nullopt, + .peer = std::nullopt, + .epoll = false, + }; + + // take exclusive lock on the session to add the client + ioctx->intra_process_mutex.lock(); + ioctx->inprocs.push_back(*connector); + ioctx->intra_process_mutex.unlock(); + + return Status::ok(); +} + +Status intra_process_bind(IntraProcessConnector* connector, const char* addr) { + if (connector->bind) + return Status::from_code("interprocess connector already bound", Code::AlreadyBound); + + std::string bind(addr); + + connector->ioctx->intra_process_mutex.lock(); + + // check for conflicts + for (auto other: connector->ioctx->inprocs) { + if (other == connector) + continue; + + if (other->bind == bind) + return Status::from_errno("intraprocess address already in use", EADDRINUSE); + } + + // set the bind address + connector->bind = bind; + + // check for any pending connections + for (auto other: connector->ioctx->inprocs) { + if (other == connector) + continue; + + if (other->connecting == bind) { + other->connecting = std::nullopt; + other->peer = connector; + connector->peer = other; + + if (eventfd_signal(other->unmuted_event_fd) < 0) + return Status::from_errno("intraprocess failed to signal unmuted_event_fd"); + } + } + + connector->ioctx->intra_process_mutex.unlock(); + + return Status::ok(); +} + +Status intra_process_connect(IntraProcessConnector* connector, const char* addr) { + std::string connecting(addr); + + connector->ioctx->intra_process_mutex.lock(); + + for (auto other: connector->ioctx->inprocs) { + if (other == connector) + continue; + + // we found a matching bind + if (other->bind == connecting) { + other->peer = connector; + connector->peer = other; + + if (eventfd_signal(other->unmuted_event_fd) < 0) + return Status::from_errno("failed to signal unmuted_event_fd"); + + connector->ioctx->intra_process_mutex.unlock(); + return Status::ok(); + } + } + + // the connection is pending + connector->connecting = connecting; + connector->ioctx->intra_process_mutex.unlock(); + + return Status::ok(); +} + +Status intra_process_send_sync(IntraProcessConnector* connector, uint8_t* data, size_t len) { + for (;;) { + connector->ioctx->intra_process_mutex.lock_shared(); + + if (connector->peer) { + Message msg { + // we need to clone the identity because the sending client + // has an independent lifetime from the message / receiving client + .address = Bytes::clone(connector->identity), + + // the caller (Python) owns the data, so we need to copy it + .payload = Bytes::copy(data, len), + }; + + auto peer = *connector->peer; + peer->queue.enqueue(msg); + + // signal the receiving client (semaphore) + if (eventfd_signal(peer->recv_buffer_event_fd) < 0) + return Status::from_errno("failed to signal recv_buffer_event_fd"); + + connector->ioctx->intra_process_mutex.unlock_shared(); + return Status::ok(); + } + + connector->ioctx->intra_process_mutex.unlock_shared(); + + // wait for a connection + wait: + if (auto code = fd_wait(connector->unmuted_event_fd, -1, POLLIN)) { + if (code > 0) + return Status::from_signal("fdwait: unmuted_event_fd", code); + + return Status::from_errno("failed to wait for unmuted_event_fd"); + } + + if (eventfd_wait(connector->unmuted_event_fd) < 0) { + // pre-empted, go back to waiting + if (errno == EAGAIN) + goto wait; + + return Status::from_errno("failed to wait on unmuted_event_fd"); + } + } + + return Status::ok(); +} + +Status intra_process_recv_sync(IntraProcessConnector* connector, Message* msg) { +wait: + if (auto code = fd_wait(connector->recv_buffer_event_fd, -1, POLLIN)) { + if (code > 0) + return Status::from_signal("fdwait: recv_buffer_event_fd", code); + + return Status::from_errno("failed to wait for recv_buffer_event_fd"); + } + + if (eventfd_wait(connector->recv_buffer_event_fd) < 0) { + if (errno == EAGAIN) + goto wait; // pre-empted, try again + + return Status::from_errno("failed to read eventfd"); + } + + // after decrementing the semaphore, we have claimed the message from the queue + // this also guarantees that the message is in the queue + while (!connector->queue.try_dequeue(*msg)) + ; // wait + + return Status::ok(); +} + +void intra_process_recv_async(void* future, IntraProcessConnector* connector) { + // ensure that the client is in the epoll + // this allows sync-only clients to avoid epoll overhead + if (auto status = connector->ensure_epoll(); status.type != StatusType::Ok) + return future_set_status(future, &status); + + connector->recv.enqueue(future); + + if (eventfd_signal(connector->recv_event_fd) < 0) { + auto status = Status::from_errno("failed to signal recv_event_fd"); + return future_set_status(future, &status); + } +} + +Status intra_process_destroy(IntraProcessConnector* connector) { + connector->remove_from_epoll(); + connector->ioctx->intra_process_mutex.lock(); + std::erase(connector->ioctx->inprocs, connector); + + if (connector->peer) { + auto peer = *connector->peer; + peer->peer = std::nullopt; + } + + connector->ioctx->intra_process_mutex.unlock(); + + mydebug(); + connector->identity.free(); + + return Status::ok(); +} diff --git a/scaler/io/cpp/src/intra_process_connector.hpp b/scaler/io/cpp/src/intra_process_connector.hpp new file mode 100644 index 00000000..8d809404 --- /dev/null +++ b/scaler/io/cpp/src/intra_process_connector.hpp @@ -0,0 +1,64 @@ +#pragma once + +// C +#include +#include + +// C++ +#include +#include +#include +#include +#include + +// Third-party +#include "third_party/concurrentqueue.h" + +// Common +#include "common.hpp" + +using moodycamel::ConcurrentQueue; + +// --- declarations --- +struct IntraProcessConnector; + +// First-party +#include "common.hpp" +#include "io_context.hpp" + +Status intra_process_init(IoContext* ioctx, IntraProcessConnector** connector, uint8_t* identity, size_t len); +Status intra_process_bind(struct IntraProcessConnector* connector, const char* addr); +Status intra_process_connect(struct IntraProcessConnector* connector, const char* addr); +Status intra_process_send_sync(struct IntraProcessConnector* connector, uint8_t* data, size_t len); +Status intra_process_recv_sync(struct IntraProcessConnector* connector, struct Message* msg); +void intra_process_recv_async(void* future, struct IntraProcessConnector* connector); +Status intra_process_destroy(struct IntraProcessConnector* connector); + +// -- structs -- + +// inproc sockets are always pair sockets +struct IntraProcessConnector { + IoContext* ioctx; + ThreadContext* thread; + + // at any time at least one of these will be empty + ConcurrentQueue queue; + ConcurrentQueue recv; + + // semaphore: counts number of messages in .queue + int recv_buffer_event_fd; + // semaphore: counts number of pending receive futures in .recv + int recv_event_fd; + // signals that the connector is unmuted + int unmuted_event_fd; + + Bytes identity; + std::optional bind; + std::optional connecting; + std::optional peer; + + std::atomic_bool epoll; + + Status ensure_epoll(); + void remove_from_epoll(); +}; diff --git a/scaler/io/cpp/src/io_context.cpp b/scaler/io/cpp/src/io_context.cpp new file mode 100644 index 00000000..8918bfb2 --- /dev/null +++ b/scaler/io/cpp/src/io_context.cpp @@ -0,0 +1,576 @@ +#include "io_context.hpp" + +std::string EpollType::as_string() const { + switch (value) { + case EpollType::ConnectorSend: return "ConnectorSend"; + case EpollType::ConnectorRecv: return "ConnectorRecv"; + case EpollType::ConnectorListener: return "ConnectorListener"; + case EpollType::ConnectorPeer: return "ConnectorPeer"; + case EpollType::IntraProcessConnectorRecv: return "IntraProcessConnectorRecv"; + case EpollType::ConnectTimer: return "ConnectTimer"; + case EpollType::Control: return "Control"; + case EpollType::Closed: return "Closed"; + } + + unreachable(); +} + +void ThreadContext::control(ControlRequest request) { + this->control_queue.enqueue(request); + + if (eventfd_signal(this->control_efd) < 0) + panic("failed to write to eventfd: " + std::to_string(errno)); +} + +// arm the reconnect timer if it is not already armed +void ThreadContext::ensure_timer_armed() { + if (this->timer_armed) + return; + + itimerspec spec { + .it_interval = {.tv_sec = 0, .tv_nsec = 0}, + .it_value = {.tv_sec = 3, .tv_nsec = 0}, + }; + + if (timerfd_settime(this->connect_timer_tfd, 0, &spec, NULL) < 0) + panic("failed to arm timer"); + + this->timer_armed = true; +} + +void ThreadContext::add_connector(NetworkConnector* connector) { + this->add_epoll(connector->send_event_fd, EPOLLIN | EPOLLET, EpollType::ConnectorSend, connector); + this->add_epoll(connector->recv_event_fd, EPOLLIN | EPOLLET, EpollType::ConnectorRecv, connector); +} + +void ThreadContext::add_peer(RawPeer* peer) { + this->add_epoll(peer->fd, EPOLLIN | EPOLLOUT | EPOLLET, EpollType::ConnectorPeer, peer); +} + +void ThreadContext::remove_connector(NetworkConnector* connector) { + this->remove_epoll(connector->send_event_fd); + this->remove_epoll(connector->recv_event_fd); +} + +// this free's the peer's EpollData +void ThreadContext::remove_peer(RawPeer* peer) { + this->remove_epoll(peer->fd); + + std::erase_if(this->connecting, [peer](RawPeer* p) { return p == peer; }); +} + +// must be called on io-thread +void ThreadContext::add_epoll(int fd, uint32_t flags, EpollType type, void* data) { + auto edata = new EpollData { + .fd = fd, + .type = type, + .ptr = data, + }; + + epoll_event event {.events = flags, .data = {.ptr = edata}}; + + epoll_ctl(this->epoll_fd, EPOLL_CTL_ADD, fd, &event); + + this->io_cache.push_back(edata); +} + +// must be called on io-thread +void ThreadContext::remove_epoll(int fd) { + if (epoll_ctl(this->epoll_fd, EPOLL_CTL_DEL, fd, NULL) < 0) { + // we ignore enoent because it means the fd was already removed + if (errno != ENOENT) + panic("failed to remove epoll fd: " + std::to_string(fd) + "; " + strerror(errno)); + } + + auto edata = std::find_if(this->io_cache.begin(), this->io_cache.end(), [fd](EpollData* d) { return d->fd == fd; }); + + if (edata != this->io_cache.end()) { + delete *edata; + this->io_cache.erase(edata); + } +} + +EpollData* ThreadContext::epoll_by_fd(int fd) { + auto edata = std::find_if(this->io_cache.begin(), this->io_cache.end(), [fd](EpollData* d) { return d->fd == fd; }); + + if (edata != this->io_cache.end()) + return *edata; + + return nullptr; +} + +void ThreadContext::start() { + this->thread = std::thread(io_thread_main, this); +} + +void set_sock_opts(int fd) { + timeval tv {.tv_sec = 1, .tv_usec = 0}; + + setsockopt(fd, SOL_SOCKET, SO_RCVTIMEO, &tv, sizeof(tv)); + setsockopt(fd, SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv)); + + int on = 1; + setsockopt(fd, SOL_SOCKET, SO_REUSEADDR, &on, sizeof(on)); +} + +// complete acceptance of peer +void complete_peer_connect(RawPeer* peer) { + auto was_muted = peer->connector->muted(); + + peer->state = PeerState::Connected; + peer->connector->peers.push_back(peer); + + // it's possible that this is a reconnect + // so we check if the identity matches any existing peers + RawPeer* other; + if (peer->connector->peer_by_id(peer->identity, &other) && other != peer) { + if (other->state != PeerState::Disconnected) { + panic("peer attempting to reconnect while already connected: " + peer->identity.as_string()); + } + + // steal the messages from the other peer + while (!other->queue.empty()) { + auto send = other->queue.front(); + peer->queue.push_back(send); + other->queue.pop_front(); + } + + // we can delete the other peer now + mydebug(); + other->identity.free(); + other->connector->remove_peer(other); + delete other; + } + + if (was_muted) + peer->connector->unmute(); +} + +void network_connector_connect_peer_inner(RawPeer* peer, int domain, socklen_t len) { + peer->fd = socket(domain, SOCK_STREAM | SOCK_NONBLOCK, 0); + + if (peer->fd < 0) + panic("failed to create socket: " + std::to_string(errno)); + + auto res = connect(peer->fd, (sockaddr*)&peer->addr, len); + + if (res < 0 && errno != EINPROGRESS) + panic("failed to connect to peer: " + std::to_string(errno)); + + peer->state = PeerState::Connecting; // set peer state + peer->write_op = IoOperation::write(peer->connector->identity); // write our identity + peer->read_op = IoOperation::read(); // read the peer's identity + + peer->connector->thread->add_peer(peer); +} + +// begin the connection process for a peer +// the peer's fd will be overwritten with the new socket +void network_connector_connect_peer(RawPeer* peer) { + switch (peer->connector->transport) { + case Transport::TCP: return network_connector_connect_peer_inner(peer, AF_INET, sizeof(sockaddr_in)); + case Transport::InterProcess: return network_connector_connect_peer_inner(peer, AF_UNIX, sizeof(sockaddr_un)); + case Transport::IntraProcess: panic("unsupported"); + } +} + +// epoll handlers +void network_connector_send_event(NetworkConnector* connector) { + for (;;) { + if (connector->muted()) + return; + + if (eventfd_wait(connector->send_event_fd) < 0) { + // semaphore is zero, we can epoll_wait() again + if (errno == EAGAIN) + break; + + panic("handle eventfd read error: " + std::to_string(errno)); + } + + // invariant: if we decremented the semaphore the queue must have a message + // we loop because thread synchronization may be delayed + SendMessage send; + while (!connector->send_queue.try_dequeue(send)) + ; // wait + + connector->send(send); + } +} + +void network_connector_recv_event(NetworkConnector* connector) { + // in this event, the recv_event_fd has proc'd + // but, read the recv_buffer_event_fd first + // to ensure that a message is available to complete the future + + for (;;) { + if (eventfd_wait(connector->recv_buffer_event_fd) < 0) { + if (errno == EAGAIN) + break; + + panic("handle eventfd read error: " + std::to_string(errno)); + } + + // decrement the semaphore + if (eventfd_wait(connector->recv_event_fd) < 0) { + // there are no more recvs + if (errno == EAGAIN) { + // we decremented the buffer efd but aren't processing a message + // so we need to re-increment the semaphore + if (eventfd_signal(connector->recv_buffer_event_fd) < 0) + panic("failed to signal eventfd: " + std::to_string(errno)); + + break; // back to epoll_wait() + } + + panic("handle eventfd read error:" + std::to_string(errno)); + } + + // invariant: if we decrement the semaphore the queue must have a future + void* future; + while (!connector->recv_queue.try_dequeue(future)) + ; // wait + + Message message; + while (!connector->recv_buffer.try_dequeue(message)) + ; // wait + + // this is the address of a stack variable + // ok because the called code copies the message immediately + future_set_result(future, &message); + + // we're done with the message + message_destroy(&message); + } +} + +// may call reconnect_peer() +ControlFlow write_identity(RawPeer* peer) { + auto result = write_message(peer->fd, &*peer->write_op); + + switch (result) { + // identity writes have a none completer, so we don't need to complete it + case IoState::Done: + peer->write_op = std::nullopt; + return ControlFlow::Continue; + case IoState::Blocked: return ControlFlow::Continue; + case IoState::Reset: { + peer->write_op = std::nullopt; + reconnect_peer(peer); + } + return ControlFlow::Break; + case IoState::Closed: unreachable(); // never returned by write_message() + } + + unreachable(); +} + +// may call reconnect_peer() +ControlFlow read_identity(RawPeer* peer) { + auto result = read_message(peer->fd, &*peer->read_op); + + switch (result) { + case IoState::Done: + peer->identity = peer->read_op->payload; // set identity + peer->read_op = std::nullopt; // reset + return ControlFlow::Continue; + case IoState::Blocked: return ControlFlow::Continue; + case IoState::Reset: reconnect_peer(peer); return ControlFlow::Break; + case IoState::Closed: remove_peer(peer); return ControlFlow::Break; + } + + unreachable(); +} + +void network_connector_listener_event(NetworkConnector* connector) { + for (;;) { + sockaddr_storage addr; + socklen_t addr_len = sizeof(addr); + auto fd = accept4(connector->fd, (sockaddr*)&addr, &addr_len, SOCK_NONBLOCK); + + if (fd < 0) { + // no more connections to accept; back to epoll_wait() + if (errno == EAGAIN || errno == EWOULDBLOCK) + break; + + panic("accept(): " + std::to_string(errno)); + } + + set_sock_opts(fd); + + // create the peer and add it to the epoll + // when it becomes writable the common logic + // will take care of exchanging the identity + // and all the other stuff + auto peer = new RawPeer { + .connector = connector, + .identity = Bytes::empty(), + .addr = addr, + .type = PeerType::Connectee, + .fd = fd, + .queue = std::deque(), + .state = PeerState::Connecting, + // read the peer's identity + .read_op = IoOperation::read(), + // write our identity + .write_op = IoOperation::write(connector->identity), + }; + + connector->thread->add_peer(peer); + } +} + +void network_connector_peer_event_connecting(epoll_event* event) { + auto data = (EpollData*)event->data.ptr; + auto peer = data->peer; + + if (event->events & EPOLLIN && peer->read_op) + if (read_identity(peer) == ControlFlow::Break) + return; + + if (event->events & EPOLLOUT && peer->write_op) + if (write_identity(peer) == ControlFlow::Break) + return; + + // check if we're done + if (!peer->read_op && !peer->write_op) { + complete_peer_connect(peer); + + // we're edge triggered so it's important that we check this + network_connector_peer_event_connected(event); + } +} + +void network_connector_peer_event_connected(epoll_event* event) { + auto edata = (EpollData*)event->data.ptr; + auto peer = edata->peer; + + if (event->events & EPOLLOUT) + if (epollout_peer(peer) == ControlFlow::Break) + return; + + if (event->events & EPOLLIN) + if (epollin_peer(peer) == ControlFlow::Break) + return; +} + +// may call reconnect_peer() +void network_connector_peer_event(epoll_event* event) { + auto edata = (EpollData*)event->data.ptr; + auto peer = edata->peer; + + assert(peer != nullptr); + + // if (event->events & EPOLLHUP) + // { + // reconnect_peer(peer); + // return; + // } + + if (event->events & EPOLLERR) { + // this is how we would check the errno if we needed it + // a lot of time this will be 111 (ECONNREFUSED) + // int result; + // socklen_t result_len = sizeof(result); + // if (getsockopt(peer->fd, SOL_SOCKET, SO_ERROR, &result, &result_len) < 0) + // panic("failed to getsockopt: " + std::to_string(errno)); + + if (peer->state == PeerState::Connecting) { + // this is a connection error + peer->write_op = std::nullopt; + peer->read_op = std::nullopt; + } + + return reconnect_peer(peer); + } + + if (peer->state == PeerState::Disconnected) + panic("network_connector_peer_event(): peer is disconnected"); + + if (peer->state == PeerState::Connecting) + network_connector_peer_event_connecting(event); + else if (peer->state == PeerState::Connected) + network_connector_peer_event_connected(event); +} + +void connect_timer_event(ThreadContext* ctx) { + if (timerfd_read2(ctx->connect_timer_tfd) < 0) { + panic("failed to read from connect timer: " + std::to_string(errno)); + } + + if (!ctx->connecting.empty()) { + auto peer = ctx->connecting.front(); + network_connector_connect_peer(peer); + ctx->connecting.pop_front(); + } + + ctx->timer_armed = false; + + if (!ctx->connecting.empty()) + ctx->ensure_timer_armed(); +} + +void control_event(ThreadContext* ctx) { + for (;;) { + if (eventfd_wait(ctx->control_efd) < 0) { + if (errno == EAGAIN) + break; + + panic("failed to read from control eventfd: " + std::to_string(errno)); + } + + // a control request has been received + ControlRequest request; + while (!ctx->control_queue.try_dequeue(request)) + ; // wait + + switch (request.op) { + case ControlOperation::AddConnector: ctx->add_connector(request.connector); break; + case ControlOperation::DestroyConnector: { + auto connector = request.connector; + + // make a copy because `connector->peers` will be modified + for (RawPeer* peer: std::vector(connector->peers)) + remove_peer(peer); + + ctx->remove_connector(connector); + + if (connector->fd > 0) + close(connector->fd); + + mydebug(); + connector->identity.free(); + + close(connector->send_event_fd); + close(connector->recv_event_fd); + close(connector->recv_buffer_event_fd); + } break; + case ControlOperation::Connect: network_connector_connect_peer(request.peer); break; + } + + request.complete(); + } +} + +// either recv() was called or a message was buffered -- same handler +void intra_process_recv_event(IntraProcessConnector* connector) { + if (eventfd_wait(connector->recv_buffer_event_fd) < 0) { + if (errno == EAGAIN) + return; + + panic("failed to read from eventfd: " + std::to_string(errno)); + } + + if (eventfd_wait(connector->recv_event_fd) < 0) { + if (errno == EAGAIN) { + // we need to re-increment the semaphore because we didn't process the message + if (eventfd_signal(connector->recv_buffer_event_fd) < 0) + panic("failed to write to eventfd: " + std::to_string(errno)); + + return; + } + + panic("failed to read from eventfd: " + std::to_string(errno)); + } + + Message message; + while (!connector->queue.try_dequeue(message)) + ; // wait + + void* future; + while (connector->recv.try_dequeue(future)) + ; + + // this is the address of a stack variable + // ok because the called code copies the message immediately + future_set_result(future, &message); + message_destroy(&message); +} + +void io_thread_main(ThreadContext* ctx) { + epoll_event event; + for (;;) { + auto n_events = epoll_wait(ctx->epoll_fd, &event, 1, -1); + + // spurrious wakeup + if (n_events == 0) + continue; + + EpollData* data = (EpollData*)event.data.ptr; + + // clang-format off + switch (data->type) + { + case EpollType::ConnectorSend: network_connector_send_event(data->connector); break; // client send() ET + case EpollType::ConnectorRecv: network_connector_recv_event(data->connector); break; // client recv() ET + case EpollType::ConnectorListener: network_connector_listener_event(data->connector); break; // new connection ET + case EpollType::ConnectorPeer: network_connector_peer_event(&event); break; // peer has data ET + case EpollType::IntraProcessConnectorRecv: intra_process_recv_event(data->inproc); break; // intraprocess recv() ET + case EpollType::ConnectTimer: connect_timer_event(ctx); break; // connect timer LT + case EpollType::Control: control_event(ctx); break; // control event LT + case EpollType::Closed: return; // exit LT + + default: + panic("unknown epoll event type: " + std::to_string(data->type)); + } + // clang-format on + } +} + +// --- public api --- + +Status io_context_init(IoContext* ioctx, size_t num_threads) { + new (ioctx) IoContext { + .threads = std::vector(), + .inprocs = std::vector(), + .intra_process_mutex = std::shared_mutex(), + .thread_rr = 0}; + + // exactly size the vector to avoid reallocation + ioctx->threads.reserve(num_threads); + + for (size_t i = 0; i < num_threads; i++) { + ioctx->threads.emplace_back( + ThreadContext { + // note: this does not start the thread + .id = i, + .ioctx = ioctx, + .thread = std::thread(), + .io_cache = std::vector(), + .connecting = std::deque(), + .control_queue = ConcurrentQueue(), + .control_efd = eventfd(0, EFD_NONBLOCK | EFD_SEMAPHORE), + .epoll_fd = epoll_create1(0), + .epoll_close_efd = eventfd(0, 0), + + .connect_timer_tfd = timerfd_create(CLOCK_MONOTONIC, TFD_NONBLOCK), + .timer_armed = false, + }); + + auto ctx = &ioctx->threads.back(); + + ctx->add_epoll(ctx->epoll_close_efd, EPOLLIN, EpollType::Closed, nullptr); + ctx->add_epoll(ctx->control_efd, EPOLLIN, EpollType::Control, nullptr); + ctx->add_epoll(ctx->connect_timer_tfd, EPOLLIN, EpollType::ConnectTimer, nullptr); + + ctx->start(); + } + + return Status::ok(); +} + +// this must be called after all registered clients have been destroyed +Status io_context_destroy(IoContext* ioctx, bool destruct) { + for (auto& ctx: ioctx->threads) { + if (eventfd_signal(ctx.epoll_close_efd) < 0) + return Status::from_errno("failed to write to epoll_close_efd"); + + ctx.thread.join(); + } + + // run destructor in-place without freeing memory (it's owned by Python) + if (destruct) + ioctx->~IoContext(); + + return Status::ok(); +} diff --git a/scaler/io/cpp/src/io_context.hpp b/scaler/io/cpp/src/io_context.hpp new file mode 100644 index 00000000..1e073727 --- /dev/null +++ b/scaler/io/cpp/src/io_context.hpp @@ -0,0 +1,170 @@ +#pragma once + +// C++ +#include +#include +#include +#include + +// System +#include +#include + +// Common +#include "common.hpp" + +// --- declarations --- + +struct EpollType; +struct EpollData; +ENUM ControlOperation: uint8_t; +struct ControlRequest; +struct ThreadContext; +struct IoContext; + +// First-party +#include "intra_process_connector.hpp" +#include "network_connector.hpp" + +void set_sock_opts(int fd); +void complete_peer_connect(RawPeer* peer); +void network_connector_connect_peer_inner(RawPeer* peer, int address_family, socklen_t len); +void network_connector_connect_peer(RawPeer* peer); + +ControlFlow read_identity(RawPeer* peer); +ControlFlow write_identity(RawPeer* peer); + +// epoll handlers +void network_connector_send_event(NetworkConnector* connector); +void network_connector_recv_event(NetworkConnector* connector); +void network_connector_listener_event(NetworkConnector* connector); +void network_connector_peer_event_connecting(epoll_event* event); +void network_connector_peer_event_connected(epoll_event* event); +void intra_process_recv_event(IntraProcessConnector* connector); + +void io_thread_main(ThreadContext* ctx); + +extern "C" { +Status io_context_init(IoContext* ioctx, size_t num_threads); +Status io_context_destroy(IoContext* ioctx, bool destruct); +} + +// --- structs --- + +struct EpollType { + enum Value { + ConnectorSend, + ConnectorRecv, + ConnectorListener, + ConnectorPeer, + IntraProcessConnectorRecv, + + ConnectTimer, + Control, + Closed, + }; + + constexpr EpollType(Value value): value(value) {} + constexpr operator Value() const { return value; } + + std::string as_string() const; + +private: + Value value; +}; + +struct EpollData { + int fd; + EpollType type; + + union { + void* ptr; + NetworkConnector* connector; + IntraProcessConnector* inproc; + RawPeer* peer; + }; +}; + +ENUM ControlOperation: uint8_t { + AddConnector, + DestroyConnector, + Connect, + }; + +struct ControlRequest { + ControlOperation op; + Completer completer; + + union { + void* data; + NetworkConnector* connector; + RawPeer* peer; + }; + + void complete(void* result = NULL) { completer.complete(result); } +}; + +struct ThreadContext { + size_t id; + IoContext* ioctx; + std::thread thread; + std::vector io_cache; + std::deque connecting; + ConcurrentQueue control_queue; + int control_efd; + int epoll_fd; + int epoll_close_efd; + int connect_timer_tfd; + bool timer_armed; + + void ensure_timer_armed(); + void add_connector(NetworkConnector* connector); + void remove_connector(NetworkConnector* connector); + void add_peer(RawPeer* peer); + void remove_peer(RawPeer* peer); + + // must be called on io-thread + void add_epoll(int fd, uint32_t flags, EpollType type, void* data); + // must be called on io-thread + void remove_epoll(int fd); + EpollData* epoll_by_fd(int fd); + void control(ControlRequest request); + void start(); +}; + +// rationale: +// - we need a context to manage the io threads +// - intraprocess connectors need a context to discover and communicate with each other +// +// lifetime: +// - the context is the longest lived object in the library +// - you should normally only have one context per program +// - the lifetime begins when you call `io_context_init()` and ends when you call `io_context_destroy()` +// +// usage: +// - the main usage of io context is to be passed to connector initialization +// - use `io_context_*()` functions to operate on the context +// +// assumptions: +// - the user is not going to access the internal state of the io context directly +struct IoContext { + // the io threads + std::vector threads; + std::vector inprocs; + std::shared_mutex intra_process_mutex; + std::atomic_uint8_t thread_rr; + + inline size_t num_threads() { return threads.size(); }; + + // round-robin the threads + // returns nullptr if there are no threads + ThreadContext* next_thread() { + if (threads.empty()) + return nullptr; + + auto rr = thread_rr++; + return &threads[rr % num_threads()]; + } + + inline bool is_single_threaded() { return num_threads() == 1; }; +}; diff --git a/scaler/io/cpp/src/main.hpp b/scaler/io/cpp/src/main.hpp new file mode 100644 index 00000000..9c42e99a --- /dev/null +++ b/scaler/io/cpp/src/main.hpp @@ -0,0 +1,10 @@ +#include "common.hpp" +#include "io_context.hpp" +#include "network_connector.hpp" +#include "intra_process_connector.hpp" +#include "connector.hpp" + +#include "io_context.cpp" +#include "network_connector.cpp" +#include "intra_process_connector.cpp" +#include "connector.cpp" diff --git a/scaler/io/cpp/src/network_connector.cpp b/scaler/io/cpp/src/network_connector.cpp new file mode 100644 index 00000000..57bbe666 --- /dev/null +++ b/scaler/io/cpp/src/network_connector.cpp @@ -0,0 +1,710 @@ +#include "network_connector.hpp" + +#include "io_context.hpp" + +bool NetworkConnector::peer_by_id(Bytes id, RawPeer** peer) { + auto it = std::find_if(this->peers.begin(), this->peers.end(), [id](RawPeer* p) { return p->identity == id; }); + + if (it != this->peers.end()) { + *peer = *it; + return true; + } + + return false; +} + +void NetworkConnector::remove_peer(RawPeer* peer) { + std::erase(this->peers, peer); +} + +bool NetworkConnector::muted() { + // these types mute when they have no peers + if (this->type == ConnectorType::Pair || this->type == ConnectorType::Dealer) + return this->peers.empty(); + + // other types drop messages when they have no peers + return false; +} + +size_t NetworkConnector::peer_rr() { + // why modulo twice? the number of peers might have changed + auto rr = this->rr; + this->rr = (this->rr + 1) % this->peers.size(); + + return rr % this->peers.size(); +} + +// receive a message +// this will either complete a waiting recv request or buffer the message +void NetworkConnector::recv_msg(Message message) { + // if there's a waiting recv, complete it immediately + if (eventfd_wait(this->recv_event_fd) == 0) { + void* future; + while (!this->recv_queue.try_dequeue(future)) + ; // wait + + future_set_result(future, &message); + message_destroy(&message); + } else if (errno == EAGAIN) // o.w. res < 0 + { + // buffer the message + this->recv_buffer.enqueue(message); + + if (eventfd_signal(this->recv_buffer_event_fd) < 0) + panic("failed to write to eventfd: " + std::to_string(errno)); + } else + panic("failed to read eventfd: " + std::to_string(errno)); +}; + +void NetworkConnector::unmute() { + // these types do not mute + if (this->type == ConnectorType::Pub || this->type == ConnectorType::Router) + return; + + network_connector_send_event(this); +} + +// panics if the client is muted +void NetworkConnector::send(SendMessage send) { + switch (this->type) { + case ConnectorType::Pair: { + if (this->peers.empty()) + panic("client: muted"); + + auto peer = this->peers[0]; + write_enqueue(peer, send); + } break; + case ConnectorType::Router: { + RawPeer* peer; + if (!this->peer_by_id(send.msg.address, &peer)) + break; // routers drop messages + + write_enqueue(peer, send); + } break; + case ConnectorType::Pub: { + auto n_peers = this->peers.size(); + + // pub sockets don't mute + // we only do this check so .set_counter() doesn't panic + if (n_peers == 0) { + // completer needs to be completed + send.completer.complete_ok(); + return; + } + + // the completer needs to be completed once for each peer + // note: completer.complete*() is not thread safe + // however, all of the peers run on the same thread + send.completer.set_counter(n_peers); + + // if the socket has no peers, the message is dropped + // we need to copy the peers because the vector may be modified + for (auto peer: std::vector(this->peers)) + write_enqueue(peer, send); + } break; + case ConnectorType::Dealer: { + if (this->peers.empty()) + panic("client: muted"); + + // dealers round-robin their peers + auto peer = this->peers[this->peer_rr()]; + + write_enqueue(peer, send); + } break; + default: unreachable(); + } +} + +// takes ownership of the `payload` +void RawPeer::recv_msg(Bytes payload) { + Message message { + // the lifetime of the identity and this message are decoupled + // so it's important that we clone the data + .address = Bytes::clone(this->identity), + .payload = payload, + }; + + this->connector->recv_msg(message); +} + +// try to write `len` bytes of `data` to `fd` +// this is a nonblocking operation and may only write some of the bytes +// +// never returns IoState::Closed +[[nodiscard]] IoResult writeall(int fd, uint8_t* data, size_t len) { + size_t total = 0; + + while (total < len) { + auto n = write(fd, data + total, len - total); + + if (n < 0) { + if (errno == EAGAIN || errno == EWOULDBLOCK) + return { + .tag = IoState::Blocked, + .n_bytes = total, + }; + + if (errno == EPIPE || errno == ECONNRESET) + return { + .tag = IoState::Reset, + .n_bytes = total, + }; + + panic("write error: " + std::to_string(errno)); + } + + total += n; + } + + return { + .tag = IoState::Done, + .n_bytes = total, + }; +} + +// write a message +// nonblocking, resumable +// never returns IoState::Closed +[[nodiscard]] IoState write_message(int fd, IoOperation* op) { + switch (op->progress) { + case IoProgress::Header: { + // serialize the header + // this may happen multiple times if we get blocked + uint8_t header[4]; + serialize_u32(htonl((uint32_t)op->payload.len), header); + + auto result = writeall(fd, header + op->cursor, 4 - op->cursor); + op->cursor += result.n_bytes; + + if (result.tag != IoState::Done) + return result.tag; + + op->progress = IoProgress::Payload; + op->cursor = 0; + } + [[fallthrough]]; + case IoProgress::Payload: { + auto result = writeall(fd, op->payload.data + op->cursor, op->payload.len - op->cursor); + op->cursor += result.n_bytes; + + return result.tag; + } + } + + unreachable(); +} + +ControlFlow epollin_peer(RawPeer* peer) { + for (;;) { + if (!peer->read_op) + // note: no completer + peer->read_op = IoOperation::read(); + + auto result = read_message(peer->fd, &*peer->read_op); + + switch (result) { + case IoState::Done: { + peer->recv_msg(peer->read_op->payload); + peer->read_op = std::nullopt; + } break; + case IoState::Blocked: return ControlFlow::Continue; + case IoState::Reset: reconnect_peer(peer); return ControlFlow::Break; + case IoState::Closed: remove_peer(peer); return ControlFlow::Break; + } + } +} + +// process the send queue until the socket blocks, the queue is exhausted, or the peer disconnects +ControlFlow epollout_peer(RawPeer* peer) { + for (;;) { + if (!peer->write_op) { + if (peer->queue.empty()) + return ControlFlow::Continue; // queue exhausted + + auto send = peer->queue.front(); + peer->write_op = IoOperation::write(send.payload, send.completer); + peer->queue.pop_front(); + } + + auto result = write_message(peer->fd, &*peer->write_op); + + switch (result) { + case IoState::Done: { + peer->write_op->completer.complete_ok(); + peer->write_op = std::nullopt; + } break; + case IoState::Blocked: return ControlFlow::Continue; + case IoState::Reset: + reconnect_peer(peer); + return ControlFlow::Break; // we need to go back to epoll_wait() after calling reconnect_peer() + case IoState::Closed: + unreachable(); // this is never returned by write_message(); write() cannot detect a graceful + // disconnect + } + } +} + +// note: peer may be in reconnecting state after calling this +// the peer's EpollData may have been freed +void write_enqueue(RawPeer* peer, SendMessage send) { + // the address is not needed after the message is queued on the peer + send.msg.address.free(); + + peer->queue.push_back({.completer = send.completer, .payload = send.msg.payload}); + + // if there's a write op, our send will be picked up when the fd becomes writable + // otherwise we can write immdiately + if (peer->state == PeerState::Connected && !peer->write_op) + epollout_peer(peer); +} + +// try to read `len` bytes out of `fd` into `buf` +// this is a nonblocking operation and may only read some of the bytes +[[nodiscard]] IoResult readexact(int fd, uint8_t* buf, size_t len) { + size_t total = 0; + + while (total < len) { + auto n = read(fd, buf + total, len - total); + + if (n < 0) { + if (errno == EAGAIN || errno == EWOULDBLOCK) + return { + .tag = IoState::Blocked, + .n_bytes = total, + }; + + if (errno == ECONNRESET || errno == EPIPE) + return { + .tag = IoState::Reset, + .n_bytes = total, + }; + + // todo: handle other errors? + panic("read error: " + std::to_string(errno) + " ; fd: " + std::to_string(fd)); + } + + // graceful disconnect + if (n == 0) + return { + .tag = IoState::Closed, + .n_bytes = total, + }; + + total += n; + } + + return { + .tag = IoState::Done, + .n_bytes = total, + }; +} + +// read a message +// nonblocking, resumable +[[nodiscard]] IoState read_message(int fd, IoOperation* op) { + switch (op->progress) { + case IoProgress::Header: { + auto result = readexact(fd, op->buffer + op->cursor, 4 - op->cursor); + op->cursor += result.n_bytes; + + if (result.tag != IoState::Done) + return result.tag; + + uint32_t len; + deserialize_u32(op->buffer, &len); + len = ntohl(len); + + op->progress = IoProgress::Payload; + op->cursor = 0; + op->payload = Bytes::alloc(len); + } + [[fallthrough]]; + case IoProgress::Payload: { + auto result = readexact(fd, op->payload.data + op->cursor, op->payload.len - op->cursor); + op->cursor += result.n_bytes; + + return result.tag; + } + } + + unreachable(); +} + +// disconnect peer, remove peer from connector, and delete the peer +// the peer is freed and the fd is closed after this call +void remove_peer(RawPeer* peer) { + disconnect_peer(peer); + + // remove from the connector's peer list + peer->connector->remove_peer(peer); + + // we must complete all outstanding sends so that + // the caller does not hang + while (!peer->queue.empty()) { + auto send = peer->queue.front(); + + auto status = Status::from_code("peer shutdown", Code::PeerShutdown); + send.completer.complete_status(&status); + + mydebug(); + send.payload.free(); + + peer->queue.pop_front(); + } + + // note: read_op was handled by disconnect_peer() + // note: write_op was also handled by disconnect_peer() + // and the send was put back in the queue + + mydebug(); + peer->identity.free(); + + delete peer; +} + +// leaves the peer in the vector for a later reconnect +void disconnect_peer(RawPeer* peer) { + peer->connector->thread->remove_peer(peer); + + if (peer->write_op) { + // we guarantee writes, so we put the message back in the queue + // at the front to maintain ordering + peer->queue.push_front({.completer = peer->write_op->completer, .payload = peer->write_op->payload}); + peer->write_op = std::nullopt; + } + + // reset the read op + // if there was a read in progress, the sender will resend the message from the beginning after reconnect + if (peer->read_op) { + mydebug(); + peer->read_op->payload.free(); + peer->read_op = std::nullopt; + } + + if (peer->fd > 0) + close(peer->fd); + peer->fd = -1; + + peer->state = PeerState::Disconnected; +} + +// this disconnects a peer and prepares it for a reconnect (server side) +void reconnect_peer(RawPeer* peer) { + disconnect_peer(peer); + + // retry the connection if we're the connector + // otherwise, leave the peer in the vector and wait for reconnect + if (peer->type == PeerType::Connector) { + auto thread = peer->connector->thread; + thread->connecting.push_back(peer); + thread->ensure_timer_armed(); + } +} + +// --- public api --- + +Status network_connector_init( + IoContext* ioctx, + NetworkConnector** connector, + Transport transport, + ConnectorType type, + uint8_t* identity, + size_t len) { + if (ioctx->threads.empty()) + return Status::from_code("network connectors require a session with threads", Code::NoThreads); + + *connector = new NetworkConnector { + .type = type, + .transport = transport, + .thread = ioctx->next_thread(), + .ioctx = ioctx, + .identity = Bytes::copy(identity, len), + .rr = 0, + .fd = -1, + .addr = std::nullopt, + .peers = std::vector(), + .send_event_fd = eventfd(0, EFD_NONBLOCK | EFD_SEMAPHORE), + .send_queue = ConcurrentQueue(), + .recv_event_fd = eventfd(0, EFD_NONBLOCK | EFD_SEMAPHORE), + .recv_queue = ConcurrentQueue(), + .recv_buffer_event_fd = eventfd(0, EFD_NONBLOCK | EFD_SEMAPHORE), + .recv_buffer = ConcurrentQueue(), + }; + + (*connector)->thread->add_connector(*connector); + + return Status::ok(); +} + +Status network_connector_bind_tcp(NetworkConnector* connector, const char* host, uint16_t port) { + sockaddr_storage address; + std::memset(&address, 0, sizeof(address)); + + connector->fd = socket(AF_INET, SOCK_STREAM | SOCK_NONBLOCK, 0); + + if (connector->fd < 0) + return Status::from_errno("failed to create tcp socket"); + + set_sock_opts(connector->fd); + + in_addr_t in_addr = strcmp(host, "*") ? inet_addr(host) : INADDR_ANY; + + if (in_addr == INADDR_NONE) + return Status::from_code("tcp address could not be parsed", Code::InvalidAddress); + + *(sockaddr_in*)&address = { + .sin_family = AF_INET, + .sin_port = htons(port), + .sin_addr = {.s_addr = in_addr}, + .sin_zero = {0}, + }; + + connector->addr = address; + + if (bind(connector->fd, (sockaddr*)&address, sizeof(sockaddr_in)) < 0) + return Status::from_errno("failed to bind tcp socket"); + + return Status::ok(); +} + +Status network_connector_bind_unix(NetworkConnector* connector, const char* path) { + sockaddr_storage address; + std::memset(&address, 0, sizeof(address)); + + connector->fd = socket(AF_UNIX, SOCK_STREAM | SOCK_NONBLOCK, 0); + + if (connector->fd < 0) + return Status::from_errno("failed to create unix socket"); + + // remove the previous lock from the fs + if (unlink(path) < 0 && errno != ENOENT) + return Status::from_errno("failed to unlink previous unix socket"); + + auto addr_un = (sockaddr_un*)&address; + + *addr_un = {.sun_family = AF_UNIX, .sun_path = {0}}; + + std::strncpy(addr_un->sun_path, path, sizeof(addr_un->sun_path) - 1); + + if (bind(connector->fd, (sockaddr*)addr_un, sizeof(sockaddr_un)) < 0) + return Status::from_errno("failed to bind unix socket"); + + return Status::ok(); +} + +Status network_connector_bind(NetworkConnector* connector, const char* host, uint16_t port) { + if (connector->fd > 0) + return Status::from_code("network connector already bound", Code::AlreadyBound); + + sockaddr_storage address; + std::memset(&address, 0, sizeof(address)); + + switch (connector->transport) { + case Transport::TCP: PROPAGATE(network_connector_bind_tcp(connector, host, port)); break; + case Transport::InterProcess: PROPAGATE(network_connector_bind_unix(connector, host)); break; + case Transport::IntraProcess: + // panic: it should be impossible for this code to be reached no matter what Python does + panic("Client does not support IntraProcess transport"); + } + + if (listen(connector->fd, 16) < 0) + return Status::from_errno("failed to listen on socket"); + + connector->thread->add_epoll(connector->fd, EPOLLIN | EPOLLET, EpollType::ConnectorListener, connector); + + return Status::ok(); +} + +Status network_connector_connect(NetworkConnector* connector, const char* addr, uint16_t port) { + sockaddr_storage address; + std::memset(&address, 0, sizeof(address)); + + switch (connector->transport) { + case Transport::InterProcess: { + auto addr_un = (sockaddr_un*)&address; + + *addr_un = {.sun_family = AF_UNIX, .sun_path = {0}}; + + std::strncpy(addr_un->sun_path, addr, sizeof(addr_un->sun_path) - 1); + } break; + case Transport::TCP: { + *(sockaddr_in*)&address = { + .sin_family = AF_INET, + .sin_port = htons(port), + .sin_addr = {.s_addr = inet_addr(addr)}, + .sin_zero = {0}, + }; + } break; + case Transport::IntraProcess: panic("Client does not support IntraProcess transport"); + } + + auto peer = new RawPeer { + .connector = connector, + .identity = Bytes::empty(), + .addr = address, + .type = PeerType::Connector, + .fd = -1, // a real fd will be assigned later + .queue = std::deque(), + .state = PeerState::Disconnected, + .read_op = std::nullopt, + .write_op = std::nullopt, + }; + + ControlRequest request { + .op = ControlOperation::Connect, + .completer = Completer::none(), + .peer = peer, + }; + + connector->thread->control(request); + + return Status::ok(); +} + +void network_connector_send_async( + void* future, NetworkConnector* connector, uint8_t* to, size_t to_len, uint8_t* data, size_t data_len) { + if (connector->type == ConnectorType::Sub) { + auto status = + Status::from_code("clients of type 'sub' do not support sending messages", Code::UnsupportedOperation); + + return future_set_status(future, &status); + } + + // this data is owned by the caller, + // but it's kept alive until the future is resolved + SendMessage send { + .completer = Completer::future(future), + .msg = + { + .address = + { + .data = to, + .len = to_len, + .tag = Bytes::Borrowed, + }, + .payload = + { + .data = data, + .len = data_len, + .tag = Bytes::Borrowed, + }, + }, + }; + + connector->send_queue.enqueue(send); + + if (eventfd_signal(connector->send_event_fd) < 0) { + auto status = Status::from_errno("failed to write to eventfd"); + return future_set_status(future, &status); + } +} + +Status network_connector_send_sync( + NetworkConnector* connector, uint8_t* to, size_t to_len, uint8_t* data, size_t data_len) { + if (connector->type == ConnectorType::Sub) + return Status::from_code("clients of type 'sub' do not support sending messages", Code::UnsupportedOperation); + + // sync operations on network connector are implemented using semaphores + // this greatly simplifies the implementation + sem_t* sem = (sem_t*)std::malloc(sizeof(sem_t)); + + if (sem_init(sem, 0, 0) < 0) + return Status::from_errno("failed to initialize semaphore"); + + SendMessage send { + .completer = Completer::semaphore(sem), + .msg = + {.address = + { + .data = to, + .len = to_len, + .tag = Bytes::Borrowed, + }, + .payload = + { + .data = data, + .len = data_len, + .tag = Bytes::Borrowed, + }}, + }; + + connector->send_queue.enqueue(send); + + if (eventfd_signal(connector->send_event_fd) < 0) + return Status::from_errno("failed to write to eventfd"); + + if (sem_wait(sem) < 0) { + // if this fails, the original error is lost + if (sem_destroy(sem) < 0) + return Status::from_errno("failed to destroy semaphore"); + std::free(sem); + + return Status::from_errno("failed to await semaphore"); + } + + if (sem_destroy(sem) < 0) + return Status::from_errno("failed to destroy semaphore"); + std::free(sem); + + return Status::ok(); +} + +void network_connector_recv_async(void* future, NetworkConnector* connector) { + connector->recv_queue.enqueue(future); + + if (eventfd_signal(connector->recv_event_fd) < 0) { + auto status = Status::from_errno("failed to write to eventfd"); + return future_set_status(future, &status); + } +} + +Status network_connector_recv_sync(NetworkConnector* connector, Message* msg) { +wait: + if (auto code = fd_wait(connector->recv_buffer_event_fd, -1, POLLIN)) { + if (code > 0) + return Status::from_signal("fdwait: recv_buffer_event_fd", code); + + return Status::from_errno("failed to wait for fd in sync recv"); + } + + if (eventfd_wait(connector->recv_buffer_event_fd) < 0) { + if (errno == EAGAIN) + goto wait; // pre-empted + + return Status::from_errno("failed to read eventfd"); + } + + while (!connector->recv_buffer.try_dequeue(*msg)) + ; // wait + + return Status::ok(); +} + +Status network_connector_destroy([[maybe_unused]] NetworkConnector* connector) { + sem_t* sem = (sem_t*)std::malloc(sizeof(sem_t)); + + if (sem_init(sem, 0, 0) < 0) + return Status::from_errno("failed to initialize semaphore"); + + ControlRequest request { + .op = ControlOperation::DestroyConnector, + .completer = Completer::semaphore(sem), + .connector = connector, + }; + + connector->thread->control(request); + +wait: + if (sem_wait(sem) < 0) { + if (errno == EINTR) + goto wait; // just wait again + + return Status::from_errno("failed to await semaphore"); + } + + if (sem_destroy(sem) < 0) + return Status::from_errno("failed to destroy semaphore"); + std::free(sem); + + return Status::ok(); +} diff --git a/scaler/io/cpp/src/network_connector.hpp b/scaler/io/cpp/src/network_connector.hpp new file mode 100644 index 00000000..c356dac5 --- /dev/null +++ b/scaler/io/cpp/src/network_connector.hpp @@ -0,0 +1,173 @@ +#pragma once + +// C +#include + +// C++ +#include +#include +#include + +// System +#include +#include +#include +#include + +// Third-party +#include "third_party/concurrentqueue.h" + +// First-party +#include "common.hpp" +#include "io_context.hpp" + +using moodycamel::ConcurrentQueue; + +// --- declarations --- + +struct NetworkConnector; +struct RawPeer; +struct IoResult; +struct SendMessage; +enum class PeerType; +enum class PeerState; +enum class IoState; + +ENUM ConnectorType: uint8_t; +ENUM Transport: uint8_t; + +[[nodiscard]] IoResult writeall(int fd, uint8_t* data, size_t len); +[[nodiscard]] IoState write_message(int fd, IoOperation* op); +[[nodiscard]] IoResult readexact(int fd, uint8_t* buf, size_t len); +[[nodiscard]] IoState read_message(int fd, IoOperation* op); + +void write_enqueue(RawPeer* peer, SendMessage send); +void reconnect_peer(RawPeer* peer); +void disconnect_peer(RawPeer* peer); +void remove_peer(RawPeer* peer); +ControlFlow epollin_peer(RawPeer* peer); +ControlFlow epollout_peer(RawPeer* peer); + +Status network_connector_bind_tcp(NetworkConnector* connector, const char* host, uint16_t port); +Status network_connector_bind_unix(NetworkConnector* connector, const char* path); + +// -- interface -- + +Status network_connector_init( + IoContext* ioctx, + NetworkConnector* connector, + Transport transport, + ConnectorType type, + uint8_t* identity, + size_t len); +Status network_connector_bind(NetworkConnector* connector, const char* host, uint16_t port); +Status network_connector_connect(NetworkConnector* connector, const char* addr, uint16_t port); +void network_connector_send_async( + void* future, NetworkConnector* connector, uint8_t* to, size_t to_len, uint8_t* data, size_t data_len); +Status network_connector_send_sync( + NetworkConnector* connector, uint8_t* to, size_t to_len, uint8_t* data, size_t data_len); +void network_connector_recv_async(void* future, NetworkConnector* connector); +Status network_connector_recv_sync(NetworkConnector* connector, Message* msg); +Status network_connector_destroy(NetworkConnector* connector); + +// --- structs --- + +struct SendMessage { + // resolved when the message is send + Completer completer; + + // the payload + Message msg; +}; + +// like SendMessage, but for the send queue +struct SendPayload { + Completer completer; + Bytes payload; +}; + +ENUM ConnectorType: uint8_t {Pair, Pub, Sub, Dealer, Router}; + +// Clients are tcp or unix domain sockets (uds, ipc) +// no variant for in-process because they're handled separately +ENUM Transport: uint8_t {TCP, IntraProcess, InterProcess}; + +struct NetworkConnector { + ConnectorType type; + Transport transport; + + ThreadContext* thread; // the thread that this connector is bound to + IoContext* ioctx; // backreference to session + Bytes identity; // the identity of this connector + + size_t rr; // round robin for dealer + + int fd; // the bound socket, <0 when not bound + std::optional addr; // addr for when we're bound + std::vector peers; + + int send_event_fd; // event fd for send queue + ConcurrentQueue send_queue; // the send queue for Python thread -> io thread communication + int recv_event_fd; // event fd for recv queue + ConcurrentQueue recv_queue; // the recv queue for io thread -> Python thread communication + int recv_buffer_event_fd; // event fd for recv buffer, only needed for sync connectors + ConcurrentQueue recv_buffer; // these are messages that have been received + + // must hold mutex + bool peer_by_id(Bytes id, RawPeer** peer); + void remove_peer(RawPeer* peer); + bool muted(); + size_t peer_rr(); + void recv_msg(Message message); + void unmute(); + + // send a message to a peer according to the client type's rules + // - must have exclusive access to the client + void send(SendMessage send); +}; + +enum class PeerType { + // we connected to the remote + Connector, + + // the remote connected to us + Connectee +}; + +enum class PeerState { + Connecting, + Connected, + Disconnected, +}; + +// created in two scenarios: +// - connecting to a remote +// - accepting a connection from a remote +struct RawPeer { + NetworkConnector* connector; // the binder that this peer belongs to + Bytes identity; // the peer's address, i.e. identity + sockaddr_storage addr; // the peer's address + PeerType type; // the type of peer + int fd; // the socket fd of this peer + + std::deque queue; // messages to be sent by this peer + + PeerState state; // the state of the peer + + std::optional read_op; // the current read operation + std::optional write_op; // the current write operation + + void recv_msg(Bytes payload); +}; + +enum class IoState { + Done, // the read or write is complete + Blocked, // the operation blocked, but some progress may have been made + Closed, // the peer has gracefully closed the connection + Reset, // the connection was reset +}; + +struct IoResult { + IoState tag; + size_t n_bytes; +}; diff --git a/scaler/io/cpp/src/third_party/concurrentqueue.h b/scaler/io/cpp/src/third_party/concurrentqueue.h new file mode 100644 index 00000000..2fc77540 --- /dev/null +++ b/scaler/io/cpp/src/third_party/concurrentqueue.h @@ -0,0 +1,3747 @@ +// Provides a C++11 implementation of a multi-producer, multi-consumer lock-free queue. +// An overview, including benchmark results, is provided here: +// http://moodycamel.com/blog/2014/a-fast-general-purpose-lock-free-queue-for-c++ +// The full design is also described in excruciating detail at: +// http://moodycamel.com/blog/2014/detailed-design-of-a-lock-free-queue + +// Simplified BSD license: +// Copyright (c) 2013-2020, Cameron Desrochers. +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, +// are permitted provided that the following conditions are met: +// +// - Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// - Redistributions in binary form must reproduce the above copyright notice, this list of +// conditions and the following disclaimer in the documentation and/or other materials +// provided with the distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT +// OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +// HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR +// TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +// EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// Also dual-licensed under the Boost Software License (see LICENSE.md) + +#pragma once + +#if defined(__GNUC__) && !defined(__INTEL_COMPILER) +// Disable -Wconversion warnings (spuriously triggered when Traits::size_t and +// Traits::index_t are set to < 32 bits, causing integer promotion, causing warnings +// upon assigning any computed values) +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wconversion" + +#ifdef MCDBGQ_USE_RELACY +#pragma GCC diagnostic ignored "-Wint-to-pointer-cast" +#endif +#endif + +#if defined(_MSC_VER) && (!defined(_HAS_CXX17) || !_HAS_CXX17) +// VS2019 with /W4 warns about constant conditional expressions but unless /std=c++17 or higher +// does not support `if constexpr`, so we have no choice but to simply disable the warning +#pragma warning(push) +#pragma warning(disable: 4127) // conditional expression is constant +#endif + +#if defined(__APPLE__) +#include "TargetConditionals.h" +#endif + +#ifdef MCDBGQ_USE_RELACY +#include "relacy/relacy_std.hpp" +#include "relacy_shims.h" +// We only use malloc/free anyway, and the delete macro messes up `= delete` method declarations. +// We'll override the default trait malloc ourselves without a macro. +#undef new +#undef delete +#undef malloc +#undef free +#else +#include // Requires C++11. Sorry VS2010. +#include +#endif +#include // for max_align_t +#include +#include +#include +#include +#include +#include +#include // for CHAR_BIT +#include +#include // partly for __WINPTHREADS_VERSION if on MinGW-w64 w/ POSIX threading +#include // used for thread exit synchronization + +// Platform-specific definitions of a numeric thread ID type and an invalid value +namespace moodycamel { namespace details { + template struct thread_id_converter { + typedef thread_id_t thread_id_numeric_size_t; + typedef thread_id_t thread_id_hash_t; + static thread_id_hash_t prehash(thread_id_t const& x) { return x; } + }; +} } +#if defined(MCDBGQ_USE_RELACY) +namespace moodycamel { namespace details { + typedef std::uint32_t thread_id_t; + static const thread_id_t invalid_thread_id = 0xFFFFFFFFU; + static const thread_id_t invalid_thread_id2 = 0xFFFFFFFEU; + static inline thread_id_t thread_id() { return rl::thread_index(); } +} } +#elif defined(_WIN32) || defined(__WINDOWS__) || defined(__WIN32__) +// No sense pulling in windows.h in a header, we'll manually declare the function +// we use and rely on backwards-compatibility for this not to break +extern "C" __declspec(dllimport) unsigned long __stdcall GetCurrentThreadId(void); +namespace moodycamel { namespace details { + static_assert(sizeof(unsigned long) == sizeof(std::uint32_t), "Expected size of unsigned long to be 32 bits on Windows"); + typedef std::uint32_t thread_id_t; + static const thread_id_t invalid_thread_id = 0; // See http://blogs.msdn.com/b/oldnewthing/archive/2004/02/23/78395.aspx + static const thread_id_t invalid_thread_id2 = 0xFFFFFFFFU; // Not technically guaranteed to be invalid, but is never used in practice. Note that all Win32 thread IDs are presently multiples of 4. + static inline thread_id_t thread_id() { return static_cast(::GetCurrentThreadId()); } +} } +#elif defined(__arm__) || defined(_M_ARM) || defined(__aarch64__) || (defined(__APPLE__) && TARGET_OS_IPHONE) || defined(__MVS__) || defined(MOODYCAMEL_NO_THREAD_LOCAL) +namespace moodycamel { namespace details { + static_assert(sizeof(std::thread::id) == 4 || sizeof(std::thread::id) == 8, "std::thread::id is expected to be either 4 or 8 bytes"); + + typedef std::thread::id thread_id_t; + static const thread_id_t invalid_thread_id; // Default ctor creates invalid ID + + // Note we don't define a invalid_thread_id2 since std::thread::id doesn't have one; it's + // only used if MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED is defined anyway, which it won't + // be. + static inline thread_id_t thread_id() { return std::this_thread::get_id(); } + + template struct thread_id_size { }; + template<> struct thread_id_size<4> { typedef std::uint32_t numeric_t; }; + template<> struct thread_id_size<8> { typedef std::uint64_t numeric_t; }; + + template<> struct thread_id_converter { + typedef thread_id_size::numeric_t thread_id_numeric_size_t; +#ifndef __APPLE__ + typedef std::size_t thread_id_hash_t; +#else + typedef thread_id_numeric_size_t thread_id_hash_t; +#endif + + static thread_id_hash_t prehash(thread_id_t const& x) + { +#ifndef __APPLE__ + return std::hash()(x); +#else + return *reinterpret_cast(&x); +#endif + } + }; +} } +#else +// Use a nice trick from this answer: http://stackoverflow.com/a/8438730/21475 +// In order to get a numeric thread ID in a platform-independent way, we use a thread-local +// static variable's address as a thread identifier :-) +#if defined(__GNUC__) || defined(__INTEL_COMPILER) +#define MOODYCAMEL_THREADLOCAL __thread +#elif defined(_MSC_VER) +#define MOODYCAMEL_THREADLOCAL __declspec(thread) +#else +// Assume C++11 compliant compiler +#define MOODYCAMEL_THREADLOCAL thread_local +#endif +namespace moodycamel { namespace details { + typedef std::uintptr_t thread_id_t; + static const thread_id_t invalid_thread_id = 0; // Address can't be nullptr + static const thread_id_t invalid_thread_id2 = 1; // Member accesses off a null pointer are also generally invalid. Plus it's not aligned. + inline thread_id_t thread_id() { static MOODYCAMEL_THREADLOCAL int x; return reinterpret_cast(&x); } +} } +#endif + +// Constexpr if +#ifndef MOODYCAMEL_CONSTEXPR_IF +#if (defined(_MSC_VER) && defined(_HAS_CXX17) && _HAS_CXX17) || __cplusplus > 201402L +#define MOODYCAMEL_CONSTEXPR_IF if constexpr +#define MOODYCAMEL_MAYBE_UNUSED [[maybe_unused]] +#else +#define MOODYCAMEL_CONSTEXPR_IF if +#define MOODYCAMEL_MAYBE_UNUSED +#endif +#endif + +// Exceptions +#ifndef MOODYCAMEL_EXCEPTIONS_ENABLED +#if (defined(_MSC_VER) && defined(_CPPUNWIND)) || (defined(__GNUC__) && defined(__EXCEPTIONS)) || (!defined(_MSC_VER) && !defined(__GNUC__)) +#define MOODYCAMEL_EXCEPTIONS_ENABLED +#endif +#endif +#ifdef MOODYCAMEL_EXCEPTIONS_ENABLED +#define MOODYCAMEL_TRY try +#define MOODYCAMEL_CATCH(...) catch(__VA_ARGS__) +#define MOODYCAMEL_RETHROW throw +#define MOODYCAMEL_THROW(expr) throw (expr) +#else +#define MOODYCAMEL_TRY MOODYCAMEL_CONSTEXPR_IF (true) +#define MOODYCAMEL_CATCH(...) else MOODYCAMEL_CONSTEXPR_IF (false) +#define MOODYCAMEL_RETHROW +#define MOODYCAMEL_THROW(expr) +#endif + +#ifndef MOODYCAMEL_NOEXCEPT +#if !defined(MOODYCAMEL_EXCEPTIONS_ENABLED) +#define MOODYCAMEL_NOEXCEPT +#define MOODYCAMEL_NOEXCEPT_CTOR(type, valueType, expr) true +#define MOODYCAMEL_NOEXCEPT_ASSIGN(type, valueType, expr) true +#elif defined(_MSC_VER) && defined(_NOEXCEPT) && _MSC_VER < 1800 +// VS2012's std::is_nothrow_[move_]constructible is broken and returns true when it shouldn't :-( +// We have to assume *all* non-trivial constructors may throw on VS2012! +#define MOODYCAMEL_NOEXCEPT _NOEXCEPT +#define MOODYCAMEL_NOEXCEPT_CTOR(type, valueType, expr) (std::is_rvalue_reference::value && std::is_move_constructible::value ? std::is_trivially_move_constructible::value : std::is_trivially_copy_constructible::value) +#define MOODYCAMEL_NOEXCEPT_ASSIGN(type, valueType, expr) ((std::is_rvalue_reference::value && std::is_move_assignable::value ? std::is_trivially_move_assignable::value || std::is_nothrow_move_assignable::value : std::is_trivially_copy_assignable::value || std::is_nothrow_copy_assignable::value) && MOODYCAMEL_NOEXCEPT_CTOR(type, valueType, expr)) +#elif defined(_MSC_VER) && defined(_NOEXCEPT) && _MSC_VER < 1900 +#define MOODYCAMEL_NOEXCEPT _NOEXCEPT +#define MOODYCAMEL_NOEXCEPT_CTOR(type, valueType, expr) (std::is_rvalue_reference::value && std::is_move_constructible::value ? std::is_trivially_move_constructible::value || std::is_nothrow_move_constructible::value : std::is_trivially_copy_constructible::value || std::is_nothrow_copy_constructible::value) +#define MOODYCAMEL_NOEXCEPT_ASSIGN(type, valueType, expr) ((std::is_rvalue_reference::value && std::is_move_assignable::value ? std::is_trivially_move_assignable::value || std::is_nothrow_move_assignable::value : std::is_trivially_copy_assignable::value || std::is_nothrow_copy_assignable::value) && MOODYCAMEL_NOEXCEPT_CTOR(type, valueType, expr)) +#else +#define MOODYCAMEL_NOEXCEPT noexcept +#define MOODYCAMEL_NOEXCEPT_CTOR(type, valueType, expr) noexcept(expr) +#define MOODYCAMEL_NOEXCEPT_ASSIGN(type, valueType, expr) noexcept(expr) +#endif +#endif + +#ifndef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED +#ifdef MCDBGQ_USE_RELACY +#define MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED +#else +// VS2013 doesn't support `thread_local`, and MinGW-w64 w/ POSIX threading has a crippling bug: http://sourceforge.net/p/mingw-w64/bugs/445 +// g++ <=4.7 doesn't support thread_local either. +// Finally, iOS/ARM doesn't have support for it either, and g++/ARM allows it to compile but it's unconfirmed to actually work +#if (!defined(_MSC_VER) || _MSC_VER >= 1900) && (!defined(__MINGW32__) && !defined(__MINGW64__) || !defined(__WINPTHREADS_VERSION)) && (!defined(__GNUC__) || __GNUC__ > 4 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 8)) && (!defined(__APPLE__) || !TARGET_OS_IPHONE) && !defined(__arm__) && !defined(_M_ARM) && !defined(__aarch64__) && !defined(__MVS__) +// Assume `thread_local` is fully supported in all other C++11 compilers/platforms +#define MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED // tentatively enabled for now; years ago several users report having problems with it on +#endif +#endif +#endif + +// VS2012 doesn't support deleted functions. +// In this case, we declare the function normally but don't define it. A link error will be generated if the function is called. +#ifndef MOODYCAMEL_DELETE_FUNCTION +#if defined(_MSC_VER) && _MSC_VER < 1800 +#define MOODYCAMEL_DELETE_FUNCTION +#else +#define MOODYCAMEL_DELETE_FUNCTION = delete +#endif +#endif + +namespace moodycamel { namespace details { +#ifndef MOODYCAMEL_ALIGNAS +// VS2013 doesn't support alignas or alignof, and align() requires a constant literal +#if defined(_MSC_VER) && _MSC_VER <= 1800 +#define MOODYCAMEL_ALIGNAS(alignment) __declspec(align(alignment)) +#define MOODYCAMEL_ALIGNOF(obj) __alignof(obj) +#define MOODYCAMEL_ALIGNED_TYPE_LIKE(T, obj) typename details::Vs2013Aligned::value, T>::type + template struct Vs2013Aligned { }; // default, unsupported alignment + template struct Vs2013Aligned<1, T> { typedef __declspec(align(1)) T type; }; + template struct Vs2013Aligned<2, T> { typedef __declspec(align(2)) T type; }; + template struct Vs2013Aligned<4, T> { typedef __declspec(align(4)) T type; }; + template struct Vs2013Aligned<8, T> { typedef __declspec(align(8)) T type; }; + template struct Vs2013Aligned<16, T> { typedef __declspec(align(16)) T type; }; + template struct Vs2013Aligned<32, T> { typedef __declspec(align(32)) T type; }; + template struct Vs2013Aligned<64, T> { typedef __declspec(align(64)) T type; }; + template struct Vs2013Aligned<128, T> { typedef __declspec(align(128)) T type; }; + template struct Vs2013Aligned<256, T> { typedef __declspec(align(256)) T type; }; +#else + template struct identity { typedef T type; }; +#define MOODYCAMEL_ALIGNAS(alignment) alignas(alignment) +#define MOODYCAMEL_ALIGNOF(obj) alignof(obj) +#define MOODYCAMEL_ALIGNED_TYPE_LIKE(T, obj) alignas(alignof(obj)) typename details::identity::type +#endif +#endif +} } + + +// TSAN can false report races in lock-free code. To enable TSAN to be used from projects that use this one, +// we can apply per-function compile-time suppression. +// See https://clang.llvm.org/docs/ThreadSanitizer.html#has-feature-thread-sanitizer +#define MOODYCAMEL_NO_TSAN +#if defined(__has_feature) + #if __has_feature(thread_sanitizer) + #undef MOODYCAMEL_NO_TSAN + #define MOODYCAMEL_NO_TSAN __attribute__((no_sanitize("thread"))) + #endif // TSAN +#endif // TSAN + +// Compiler-specific likely/unlikely hints +namespace moodycamel { namespace details { +#if defined(__GNUC__) + static inline bool (likely)(bool x) { return __builtin_expect((x), true); } + static inline bool (unlikely)(bool x) { return __builtin_expect((x), false); } +#else + static inline bool (likely)(bool x) { return x; } + static inline bool (unlikely)(bool x) { return x; } +#endif +} } + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG +#include "internal/concurrentqueue_internal_debug.h" +#endif + +namespace moodycamel { +namespace details { + template + struct const_numeric_max { + static_assert(std::is_integral::value, "const_numeric_max can only be used with integers"); + static const T value = std::numeric_limits::is_signed + ? (static_cast(1) << (sizeof(T) * CHAR_BIT - 1)) - static_cast(1) + : static_cast(-1); + }; + +#if defined(__GLIBCXX__) + typedef ::max_align_t std_max_align_t; // libstdc++ forgot to add it to std:: for a while +#else + typedef std::max_align_t std_max_align_t; // Others (e.g. MSVC) insist it can *only* be accessed via std:: +#endif + + // Some platforms have incorrectly set max_align_t to a type with <8 bytes alignment even while supporting + // 8-byte aligned scalar values (*cough* 32-bit iOS). Work around this with our own union. See issue #64. + typedef union { + std_max_align_t x; + long long y; + void* z; + } max_align_t; +} + +// Default traits for the ConcurrentQueue. To change some of the +// traits without re-implementing all of them, inherit from this +// struct and shadow the declarations you wish to be different; +// since the traits are used as a template type parameter, the +// shadowed declarations will be used where defined, and the defaults +// otherwise. +struct ConcurrentQueueDefaultTraits +{ + // General-purpose size type. std::size_t is strongly recommended. + typedef std::size_t size_t; + + // The type used for the enqueue and dequeue indices. Must be at least as + // large as size_t. Should be significantly larger than the number of elements + // you expect to hold at once, especially if you have a high turnover rate; + // for example, on 32-bit x86, if you expect to have over a hundred million + // elements or pump several million elements through your queue in a very + // short space of time, using a 32-bit type *may* trigger a race condition. + // A 64-bit int type is recommended in that case, and in practice will + // prevent a race condition no matter the usage of the queue. Note that + // whether the queue is lock-free with a 64-int type depends on the whether + // std::atomic is lock-free, which is platform-specific. + typedef std::size_t index_t; + + // Internally, all elements are enqueued and dequeued from multi-element + // blocks; this is the smallest controllable unit. If you expect few elements + // but many producers, a smaller block size should be favoured. For few producers + // and/or many elements, a larger block size is preferred. A sane default + // is provided. Must be a power of 2. + static const size_t BLOCK_SIZE = 32; + + // For explicit producers (i.e. when using a producer token), the block is + // checked for being empty by iterating through a list of flags, one per element. + // For large block sizes, this is too inefficient, and switching to an atomic + // counter-based approach is faster. The switch is made for block sizes strictly + // larger than this threshold. + static const size_t EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD = 32; + + // How many full blocks can be expected for a single explicit producer? This should + // reflect that number's maximum for optimal performance. Must be a power of 2. + static const size_t EXPLICIT_INITIAL_INDEX_SIZE = 32; + + // How many full blocks can be expected for a single implicit producer? This should + // reflect that number's maximum for optimal performance. Must be a power of 2. + static const size_t IMPLICIT_INITIAL_INDEX_SIZE = 32; + + // The initial size of the hash table mapping thread IDs to implicit producers. + // Note that the hash is resized every time it becomes half full. + // Must be a power of two, and either 0 or at least 1. If 0, implicit production + // (using the enqueue methods without an explicit producer token) is disabled. + static const size_t INITIAL_IMPLICIT_PRODUCER_HASH_SIZE = 32; + + // Controls the number of items that an explicit consumer (i.e. one with a token) + // must consume before it causes all consumers to rotate and move on to the next + // internal queue. + static const std::uint32_t EXPLICIT_CONSUMER_CONSUMPTION_QUOTA_BEFORE_ROTATE = 256; + + // The maximum number of elements (inclusive) that can be enqueued to a sub-queue. + // Enqueue operations that would cause this limit to be surpassed will fail. Note + // that this limit is enforced at the block level (for performance reasons), i.e. + // it's rounded up to the nearest block size. + static const size_t MAX_SUBQUEUE_SIZE = details::const_numeric_max::value; + + // The number of times to spin before sleeping when waiting on a semaphore. + // Recommended values are on the order of 1000-10000 unless the number of + // consumer threads exceeds the number of idle cores (in which case try 0-100). + // Only affects instances of the BlockingConcurrentQueue. + static const int MAX_SEMA_SPINS = 10000; + + // Whether to recycle dynamically-allocated blocks into an internal free list or + // not. If false, only pre-allocated blocks (controlled by the constructor + // arguments) will be recycled, and all others will be `free`d back to the heap. + // Note that blocks consumed by explicit producers are only freed on destruction + // of the queue (not following destruction of the token) regardless of this trait. + static const bool RECYCLE_ALLOCATED_BLOCKS = false; + + +#ifndef MCDBGQ_USE_RELACY + // Memory allocation can be customized if needed. + // malloc should return nullptr on failure, and handle alignment like std::malloc. +#if defined(malloc) || defined(free) + // Gah, this is 2015, stop defining macros that break standard code already! + // Work around malloc/free being special macros: + static inline void* WORKAROUND_malloc(size_t size) { return malloc(size); } + static inline void WORKAROUND_free(void* ptr) { return free(ptr); } + static inline void* (malloc)(size_t size) { return WORKAROUND_malloc(size); } + static inline void (free)(void* ptr) { return WORKAROUND_free(ptr); } +#else + static inline void* malloc(size_t size) { return std::malloc(size); } + static inline void free(void* ptr) { return std::free(ptr); } +#endif +#else + // Debug versions when running under the Relacy race detector (ignore + // these in user code) + static inline void* malloc(size_t size) { return rl::rl_malloc(size, $); } + static inline void free(void* ptr) { return rl::rl_free(ptr, $); } +#endif +}; + + +// When producing or consuming many elements, the most efficient way is to: +// 1) Use one of the bulk-operation methods of the queue with a token +// 2) Failing that, use the bulk-operation methods without a token +// 3) Failing that, create a token and use that with the single-item methods +// 4) Failing that, use the single-parameter methods of the queue +// Having said that, don't create tokens willy-nilly -- ideally there should be +// a maximum of one token per thread (of each kind). +struct ProducerToken; +struct ConsumerToken; + +template class ConcurrentQueue; +template class BlockingConcurrentQueue; +class ConcurrentQueueTests; + + +namespace details +{ + struct ConcurrentQueueProducerTypelessBase + { + ConcurrentQueueProducerTypelessBase* next; + std::atomic inactive; + ProducerToken* token; + + ConcurrentQueueProducerTypelessBase() + : next(nullptr), inactive(false), token(nullptr) + { + } + }; + + template struct _hash_32_or_64 { + static inline std::uint32_t hash(std::uint32_t h) + { + // MurmurHash3 finalizer -- see https://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp + // Since the thread ID is already unique, all we really want to do is propagate that + // uniqueness evenly across all the bits, so that we can use a subset of the bits while + // reducing collisions significantly + h ^= h >> 16; + h *= 0x85ebca6b; + h ^= h >> 13; + h *= 0xc2b2ae35; + return h ^ (h >> 16); + } + }; + template<> struct _hash_32_or_64<1> { + static inline std::uint64_t hash(std::uint64_t h) + { + h ^= h >> 33; + h *= 0xff51afd7ed558ccd; + h ^= h >> 33; + h *= 0xc4ceb9fe1a85ec53; + return h ^ (h >> 33); + } + }; + template struct hash_32_or_64 : public _hash_32_or_64<(size > 4)> { }; + + static inline size_t hash_thread_id(thread_id_t id) + { + static_assert(sizeof(thread_id_t) <= 8, "Expected a platform where thread IDs are at most 64-bit values"); + return static_cast(hash_32_or_64::thread_id_hash_t)>::hash( + thread_id_converter::prehash(id))); + } + + template + static inline bool circular_less_than(T a, T b) + { + static_assert(std::is_integral::value && !std::numeric_limits::is_signed, "circular_less_than is intended to be used only with unsigned integer types"); + return static_cast(a - b) > static_cast(static_cast(1) << (static_cast(sizeof(T) * CHAR_BIT - 1))); + // Note: extra parens around rhs of operator<< is MSVC bug: https://developercommunity2.visualstudio.com/t/C4554-triggers-when-both-lhs-and-rhs-is/10034931 + // silencing the bug requires #pragma warning(disable: 4554) around the calling code and has no effect when done here. + } + + template + static inline char* align_for(char* ptr) + { + const std::size_t alignment = std::alignment_of::value; + return ptr + (alignment - (reinterpret_cast(ptr) % alignment)) % alignment; + } + + template + static inline T ceil_to_pow_2(T x) + { + static_assert(std::is_integral::value && !std::numeric_limits::is_signed, "ceil_to_pow_2 is intended to be used only with unsigned integer types"); + + // Adapted from http://graphics.stanford.edu/~seander/bithacks.html#RoundUpPowerOf2 + --x; + x |= x >> 1; + x |= x >> 2; + x |= x >> 4; + for (std::size_t i = 1; i < sizeof(T); i <<= 1) { + x |= x >> (i << 3); + } + ++x; + return x; + } + + template + static inline void swap_relaxed(std::atomic& left, std::atomic& right) + { + T temp = left.load(std::memory_order_relaxed); + left.store(right.load(std::memory_order_relaxed), std::memory_order_relaxed); + right.store(temp, std::memory_order_relaxed); + } + + template + static inline T const& nomove(T const& x) + { + return x; + } + + template + struct nomove_if + { + template + static inline T const& eval(T const& x) + { + return x; + } + }; + + template<> + struct nomove_if + { + template + static inline auto eval(U&& x) + -> decltype(std::forward(x)) + { + return std::forward(x); + } + }; + + template + static inline auto deref_noexcept(It& it) MOODYCAMEL_NOEXCEPT -> decltype(*it) + { + return *it; + } + +#if defined(__clang__) || !defined(__GNUC__) || __GNUC__ > 4 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 8) + template struct is_trivially_destructible : std::is_trivially_destructible { }; +#else + template struct is_trivially_destructible : std::has_trivial_destructor { }; +#endif + +#ifdef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED +#ifdef MCDBGQ_USE_RELACY + typedef RelacyThreadExitListener ThreadExitListener; + typedef RelacyThreadExitNotifier ThreadExitNotifier; +#else + class ThreadExitNotifier; + + struct ThreadExitListener + { + typedef void (*callback_t)(void*); + callback_t callback; + void* userData; + + ThreadExitListener* next; // reserved for use by the ThreadExitNotifier + ThreadExitNotifier* chain; // reserved for use by the ThreadExitNotifier + }; + + class ThreadExitNotifier + { + public: + static void subscribe(ThreadExitListener* listener) + { + auto& tlsInst = instance(); + std::lock_guard guard(mutex()); + listener->next = tlsInst.tail; + listener->chain = &tlsInst; + tlsInst.tail = listener; + } + + static void unsubscribe(ThreadExitListener* listener) + { + std::lock_guard guard(mutex()); + if (!listener->chain) { + return; // race with ~ThreadExitNotifier + } + auto& tlsInst = *listener->chain; + listener->chain = nullptr; + ThreadExitListener** prev = &tlsInst.tail; + for (auto ptr = tlsInst.tail; ptr != nullptr; ptr = ptr->next) { + if (ptr == listener) { + *prev = ptr->next; + break; + } + prev = &ptr->next; + } + } + + private: + ThreadExitNotifier() : tail(nullptr) { } + ThreadExitNotifier(ThreadExitNotifier const&) MOODYCAMEL_DELETE_FUNCTION; + ThreadExitNotifier& operator=(ThreadExitNotifier const&) MOODYCAMEL_DELETE_FUNCTION; + + ~ThreadExitNotifier() + { + // This thread is about to exit, let everyone know! + assert(this == &instance() && "If this assert fails, you likely have a buggy compiler! Change the preprocessor conditions such that MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED is no longer defined."); + std::lock_guard guard(mutex()); + for (auto ptr = tail; ptr != nullptr; ptr = ptr->next) { + ptr->chain = nullptr; + ptr->callback(ptr->userData); + } + } + + // Thread-local + static inline ThreadExitNotifier& instance() + { + static thread_local ThreadExitNotifier notifier; + return notifier; + } + + static inline std::mutex& mutex() + { + // Must be static because the ThreadExitNotifier could be destroyed while unsubscribe is called + static std::mutex mutex; + return mutex; + } + + private: + ThreadExitListener* tail; + }; +#endif +#endif + + template struct static_is_lock_free_num { enum { value = 0 }; }; + template<> struct static_is_lock_free_num { enum { value = ATOMIC_CHAR_LOCK_FREE }; }; + template<> struct static_is_lock_free_num { enum { value = ATOMIC_SHORT_LOCK_FREE }; }; + template<> struct static_is_lock_free_num { enum { value = ATOMIC_INT_LOCK_FREE }; }; + template<> struct static_is_lock_free_num { enum { value = ATOMIC_LONG_LOCK_FREE }; }; + template<> struct static_is_lock_free_num { enum { value = ATOMIC_LLONG_LOCK_FREE }; }; + template struct static_is_lock_free : static_is_lock_free_num::type> { }; + template<> struct static_is_lock_free { enum { value = ATOMIC_BOOL_LOCK_FREE }; }; + template struct static_is_lock_free { enum { value = ATOMIC_POINTER_LOCK_FREE }; }; +} + + +struct ProducerToken +{ + template + explicit ProducerToken(ConcurrentQueue& queue); + + template + explicit ProducerToken(BlockingConcurrentQueue& queue); + + ProducerToken(ProducerToken&& other) MOODYCAMEL_NOEXCEPT + : producer(other.producer) + { + other.producer = nullptr; + if (producer != nullptr) { + producer->token = this; + } + } + + inline ProducerToken& operator=(ProducerToken&& other) MOODYCAMEL_NOEXCEPT + { + swap(other); + return *this; + } + + void swap(ProducerToken& other) MOODYCAMEL_NOEXCEPT + { + std::swap(producer, other.producer); + if (producer != nullptr) { + producer->token = this; + } + if (other.producer != nullptr) { + other.producer->token = &other; + } + } + + // A token is always valid unless: + // 1) Memory allocation failed during construction + // 2) It was moved via the move constructor + // (Note: assignment does a swap, leaving both potentially valid) + // 3) The associated queue was destroyed + // Note that if valid() returns true, that only indicates + // that the token is valid for use with a specific queue, + // but not which one; that's up to the user to track. + inline bool valid() const { return producer != nullptr; } + + ~ProducerToken() + { + if (producer != nullptr) { + producer->token = nullptr; + producer->inactive.store(true, std::memory_order_release); + } + } + + // Disable copying and assignment + ProducerToken(ProducerToken const&) MOODYCAMEL_DELETE_FUNCTION; + ProducerToken& operator=(ProducerToken const&) MOODYCAMEL_DELETE_FUNCTION; + +private: + template friend class ConcurrentQueue; + friend class ConcurrentQueueTests; + +protected: + details::ConcurrentQueueProducerTypelessBase* producer; +}; + + +struct ConsumerToken +{ + template + explicit ConsumerToken(ConcurrentQueue& q); + + template + explicit ConsumerToken(BlockingConcurrentQueue& q); + + ConsumerToken(ConsumerToken&& other) MOODYCAMEL_NOEXCEPT + : initialOffset(other.initialOffset), lastKnownGlobalOffset(other.lastKnownGlobalOffset), itemsConsumedFromCurrent(other.itemsConsumedFromCurrent), currentProducer(other.currentProducer), desiredProducer(other.desiredProducer) + { + } + + inline ConsumerToken& operator=(ConsumerToken&& other) MOODYCAMEL_NOEXCEPT + { + swap(other); + return *this; + } + + void swap(ConsumerToken& other) MOODYCAMEL_NOEXCEPT + { + std::swap(initialOffset, other.initialOffset); + std::swap(lastKnownGlobalOffset, other.lastKnownGlobalOffset); + std::swap(itemsConsumedFromCurrent, other.itemsConsumedFromCurrent); + std::swap(currentProducer, other.currentProducer); + std::swap(desiredProducer, other.desiredProducer); + } + + // Disable copying and assignment + ConsumerToken(ConsumerToken const&) MOODYCAMEL_DELETE_FUNCTION; + ConsumerToken& operator=(ConsumerToken const&) MOODYCAMEL_DELETE_FUNCTION; + +private: + template friend class ConcurrentQueue; + friend class ConcurrentQueueTests; + +private: // but shared with ConcurrentQueue + std::uint32_t initialOffset; + std::uint32_t lastKnownGlobalOffset; + std::uint32_t itemsConsumedFromCurrent; + details::ConcurrentQueueProducerTypelessBase* currentProducer; + details::ConcurrentQueueProducerTypelessBase* desiredProducer; +}; + +// Need to forward-declare this swap because it's in a namespace. +// See http://stackoverflow.com/questions/4492062/why-does-a-c-friend-class-need-a-forward-declaration-only-in-other-namespaces +template +inline void swap(typename ConcurrentQueue::ImplicitProducerKVP& a, typename ConcurrentQueue::ImplicitProducerKVP& b) MOODYCAMEL_NOEXCEPT; + + +template +class ConcurrentQueue +{ +public: + typedef ::moodycamel::ProducerToken producer_token_t; + typedef ::moodycamel::ConsumerToken consumer_token_t; + + typedef typename Traits::index_t index_t; + typedef typename Traits::size_t size_t; + + static const size_t BLOCK_SIZE = static_cast(Traits::BLOCK_SIZE); + static const size_t EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD = static_cast(Traits::EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD); + static const size_t EXPLICIT_INITIAL_INDEX_SIZE = static_cast(Traits::EXPLICIT_INITIAL_INDEX_SIZE); + static const size_t IMPLICIT_INITIAL_INDEX_SIZE = static_cast(Traits::IMPLICIT_INITIAL_INDEX_SIZE); + static const size_t INITIAL_IMPLICIT_PRODUCER_HASH_SIZE = static_cast(Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE); + static const std::uint32_t EXPLICIT_CONSUMER_CONSUMPTION_QUOTA_BEFORE_ROTATE = static_cast(Traits::EXPLICIT_CONSUMER_CONSUMPTION_QUOTA_BEFORE_ROTATE); +#ifdef _MSC_VER +#pragma warning(push) +#pragma warning(disable: 4307) // + integral constant overflow (that's what the ternary expression is for!) +#pragma warning(disable: 4309) // static_cast: Truncation of constant value +#endif + static const size_t MAX_SUBQUEUE_SIZE = (details::const_numeric_max::value - static_cast(Traits::MAX_SUBQUEUE_SIZE) < BLOCK_SIZE) ? details::const_numeric_max::value : ((static_cast(Traits::MAX_SUBQUEUE_SIZE) + (BLOCK_SIZE - 1)) / BLOCK_SIZE * BLOCK_SIZE); +#ifdef _MSC_VER +#pragma warning(pop) +#endif + + static_assert(!std::numeric_limits::is_signed && std::is_integral::value, "Traits::size_t must be an unsigned integral type"); + static_assert(!std::numeric_limits::is_signed && std::is_integral::value, "Traits::index_t must be an unsigned integral type"); + static_assert(sizeof(index_t) >= sizeof(size_t), "Traits::index_t must be at least as wide as Traits::size_t"); + static_assert((BLOCK_SIZE > 1) && !(BLOCK_SIZE & (BLOCK_SIZE - 1)), "Traits::BLOCK_SIZE must be a power of 2 (and at least 2)"); + static_assert((EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD > 1) && !(EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD & (EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD - 1)), "Traits::EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD must be a power of 2 (and greater than 1)"); + static_assert((EXPLICIT_INITIAL_INDEX_SIZE > 1) && !(EXPLICIT_INITIAL_INDEX_SIZE & (EXPLICIT_INITIAL_INDEX_SIZE - 1)), "Traits::EXPLICIT_INITIAL_INDEX_SIZE must be a power of 2 (and greater than 1)"); + static_assert((IMPLICIT_INITIAL_INDEX_SIZE > 1) && !(IMPLICIT_INITIAL_INDEX_SIZE & (IMPLICIT_INITIAL_INDEX_SIZE - 1)), "Traits::IMPLICIT_INITIAL_INDEX_SIZE must be a power of 2 (and greater than 1)"); + static_assert((INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) || !(INITIAL_IMPLICIT_PRODUCER_HASH_SIZE & (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE - 1)), "Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE must be a power of 2"); + static_assert(INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0 || INITIAL_IMPLICIT_PRODUCER_HASH_SIZE >= 1, "Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE must be at least 1 (or 0 to disable implicit enqueueing)"); + +public: + // Creates a queue with at least `capacity` element slots; note that the + // actual number of elements that can be inserted without additional memory + // allocation depends on the number of producers and the block size (e.g. if + // the block size is equal to `capacity`, only a single block will be allocated + // up-front, which means only a single producer will be able to enqueue elements + // without an extra allocation -- blocks aren't shared between producers). + // This method is not thread safe -- it is up to the user to ensure that the + // queue is fully constructed before it starts being used by other threads (this + // includes making the memory effects of construction visible, possibly with a + // memory barrier). + explicit ConcurrentQueue(size_t capacity = 32 * BLOCK_SIZE) + : producerListTail(nullptr), + producerCount(0), + initialBlockPoolIndex(0), + nextExplicitConsumerId(0), + globalExplicitConsumerOffset(0) + { + implicitProducerHashResizeInProgress.clear(std::memory_order_relaxed); + populate_initial_implicit_producer_hash(); + populate_initial_block_list(capacity / BLOCK_SIZE + ((capacity & (BLOCK_SIZE - 1)) == 0 ? 0 : 1)); + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + // Track all the producers using a fully-resolved typed list for + // each kind; this makes it possible to debug them starting from + // the root queue object (otherwise wacky casts are needed that + // don't compile in the debugger's expression evaluator). + explicitProducers.store(nullptr, std::memory_order_relaxed); + implicitProducers.store(nullptr, std::memory_order_relaxed); +#endif + } + + // Computes the correct amount of pre-allocated blocks for you based + // on the minimum number of elements you want available at any given + // time, and the maximum concurrent number of each type of producer. + ConcurrentQueue(size_t minCapacity, size_t maxExplicitProducers, size_t maxImplicitProducers) + : producerListTail(nullptr), + producerCount(0), + initialBlockPoolIndex(0), + nextExplicitConsumerId(0), + globalExplicitConsumerOffset(0) + { + implicitProducerHashResizeInProgress.clear(std::memory_order_relaxed); + populate_initial_implicit_producer_hash(); + size_t blocks = (((minCapacity + BLOCK_SIZE - 1) / BLOCK_SIZE) - 1) * (maxExplicitProducers + 1) + 2 * (maxExplicitProducers + maxImplicitProducers); + populate_initial_block_list(blocks); + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + explicitProducers.store(nullptr, std::memory_order_relaxed); + implicitProducers.store(nullptr, std::memory_order_relaxed); +#endif + } + + // Note: The queue should not be accessed concurrently while it's + // being deleted. It's up to the user to synchronize this. + // This method is not thread safe. + ~ConcurrentQueue() + { + // Destroy producers + auto ptr = producerListTail.load(std::memory_order_relaxed); + while (ptr != nullptr) { + auto next = ptr->next_prod(); + if (ptr->token != nullptr) { + ptr->token->producer = nullptr; + } + destroy(ptr); + ptr = next; + } + + // Destroy implicit producer hash tables + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE != 0) { + auto hash = implicitProducerHash.load(std::memory_order_relaxed); + while (hash != nullptr) { + auto prev = hash->prev; + if (prev != nullptr) { // The last hash is part of this object and was not allocated dynamically + for (size_t i = 0; i != hash->capacity; ++i) { + hash->entries[i].~ImplicitProducerKVP(); + } + hash->~ImplicitProducerHash(); + (Traits::free)(hash); + } + hash = prev; + } + } + + // Destroy global free list + auto block = freeList.head_unsafe(); + while (block != nullptr) { + auto next = block->freeListNext.load(std::memory_order_relaxed); + if (block->dynamicallyAllocated) { + destroy(block); + } + block = next; + } + + // Destroy initial free list + destroy_array(initialBlockPool, initialBlockPoolSize); + } + + // Disable copying and copy assignment + ConcurrentQueue(ConcurrentQueue const&) MOODYCAMEL_DELETE_FUNCTION; + ConcurrentQueue& operator=(ConcurrentQueue const&) MOODYCAMEL_DELETE_FUNCTION; + + // Moving is supported, but note that it is *not* a thread-safe operation. + // Nobody can use the queue while it's being moved, and the memory effects + // of that move must be propagated to other threads before they can use it. + // Note: When a queue is moved, its tokens are still valid but can only be + // used with the destination queue (i.e. semantically they are moved along + // with the queue itself). + ConcurrentQueue(ConcurrentQueue&& other) MOODYCAMEL_NOEXCEPT + : producerListTail(other.producerListTail.load(std::memory_order_relaxed)), + producerCount(other.producerCount.load(std::memory_order_relaxed)), + initialBlockPoolIndex(other.initialBlockPoolIndex.load(std::memory_order_relaxed)), + initialBlockPool(other.initialBlockPool), + initialBlockPoolSize(other.initialBlockPoolSize), + freeList(std::move(other.freeList)), + nextExplicitConsumerId(other.nextExplicitConsumerId.load(std::memory_order_relaxed)), + globalExplicitConsumerOffset(other.globalExplicitConsumerOffset.load(std::memory_order_relaxed)) + { + // Move the other one into this, and leave the other one as an empty queue + implicitProducerHashResizeInProgress.clear(std::memory_order_relaxed); + populate_initial_implicit_producer_hash(); + swap_implicit_producer_hashes(other); + + other.producerListTail.store(nullptr, std::memory_order_relaxed); + other.producerCount.store(0, std::memory_order_relaxed); + other.nextExplicitConsumerId.store(0, std::memory_order_relaxed); + other.globalExplicitConsumerOffset.store(0, std::memory_order_relaxed); + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + explicitProducers.store(other.explicitProducers.load(std::memory_order_relaxed), std::memory_order_relaxed); + other.explicitProducers.store(nullptr, std::memory_order_relaxed); + implicitProducers.store(other.implicitProducers.load(std::memory_order_relaxed), std::memory_order_relaxed); + other.implicitProducers.store(nullptr, std::memory_order_relaxed); +#endif + + other.initialBlockPoolIndex.store(0, std::memory_order_relaxed); + other.initialBlockPoolSize = 0; + other.initialBlockPool = nullptr; + + reown_producers(); + } + + inline ConcurrentQueue& operator=(ConcurrentQueue&& other) MOODYCAMEL_NOEXCEPT + { + return swap_internal(other); + } + + // Swaps this queue's state with the other's. Not thread-safe. + // Swapping two queues does not invalidate their tokens, however + // the tokens that were created for one queue must be used with + // only the swapped queue (i.e. the tokens are tied to the + // queue's movable state, not the object itself). + inline void swap(ConcurrentQueue& other) MOODYCAMEL_NOEXCEPT + { + swap_internal(other); + } + +private: + ConcurrentQueue& swap_internal(ConcurrentQueue& other) + { + if (this == &other) { + return *this; + } + + details::swap_relaxed(producerListTail, other.producerListTail); + details::swap_relaxed(producerCount, other.producerCount); + details::swap_relaxed(initialBlockPoolIndex, other.initialBlockPoolIndex); + std::swap(initialBlockPool, other.initialBlockPool); + std::swap(initialBlockPoolSize, other.initialBlockPoolSize); + freeList.swap(other.freeList); + details::swap_relaxed(nextExplicitConsumerId, other.nextExplicitConsumerId); + details::swap_relaxed(globalExplicitConsumerOffset, other.globalExplicitConsumerOffset); + + swap_implicit_producer_hashes(other); + + reown_producers(); + other.reown_producers(); + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + details::swap_relaxed(explicitProducers, other.explicitProducers); + details::swap_relaxed(implicitProducers, other.implicitProducers); +#endif + + return *this; + } + +public: + // Enqueues a single item (by copying it). + // Allocates memory if required. Only fails if memory allocation fails (or implicit + // production is disabled because Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE is 0, + // or Traits::MAX_SUBQUEUE_SIZE has been defined and would be surpassed). + // Thread-safe. + inline bool enqueue(T const& item) + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) return false; + else return inner_enqueue(item); + } + + // Enqueues a single item (by moving it, if possible). + // Allocates memory if required. Only fails if memory allocation fails (or implicit + // production is disabled because Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE is 0, + // or Traits::MAX_SUBQUEUE_SIZE has been defined and would be surpassed). + // Thread-safe. + inline bool enqueue(T&& item) + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) return false; + else return inner_enqueue(std::move(item)); + } + + // Enqueues a single item (by copying it) using an explicit producer token. + // Allocates memory if required. Only fails if memory allocation fails (or + // Traits::MAX_SUBQUEUE_SIZE has been defined and would be surpassed). + // Thread-safe. + inline bool enqueue(producer_token_t const& token, T const& item) + { + return inner_enqueue(token, item); + } + + // Enqueues a single item (by moving it, if possible) using an explicit producer token. + // Allocates memory if required. Only fails if memory allocation fails (or + // Traits::MAX_SUBQUEUE_SIZE has been defined and would be surpassed). + // Thread-safe. + inline bool enqueue(producer_token_t const& token, T&& item) + { + return inner_enqueue(token, std::move(item)); + } + + // Enqueues several items. + // Allocates memory if required. Only fails if memory allocation fails (or + // implicit production is disabled because Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE + // is 0, or Traits::MAX_SUBQUEUE_SIZE has been defined and would be surpassed). + // Note: Use std::make_move_iterator if the elements should be moved instead of copied. + // Thread-safe. + template + bool enqueue_bulk(It itemFirst, size_t count) + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) return false; + else return inner_enqueue_bulk(itemFirst, count); + } + + // Enqueues several items using an explicit producer token. + // Allocates memory if required. Only fails if memory allocation fails + // (or Traits::MAX_SUBQUEUE_SIZE has been defined and would be surpassed). + // Note: Use std::make_move_iterator if the elements should be moved + // instead of copied. + // Thread-safe. + template + bool enqueue_bulk(producer_token_t const& token, It itemFirst, size_t count) + { + return inner_enqueue_bulk(token, itemFirst, count); + } + + // Enqueues a single item (by copying it). + // Does not allocate memory. Fails if not enough room to enqueue (or implicit + // production is disabled because Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE + // is 0). + // Thread-safe. + inline bool try_enqueue(T const& item) + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) return false; + else return inner_enqueue(item); + } + + // Enqueues a single item (by moving it, if possible). + // Does not allocate memory (except for one-time implicit producer). + // Fails if not enough room to enqueue (or implicit production is + // disabled because Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE is 0). + // Thread-safe. + inline bool try_enqueue(T&& item) + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) return false; + else return inner_enqueue(std::move(item)); + } + + // Enqueues a single item (by copying it) using an explicit producer token. + // Does not allocate memory. Fails if not enough room to enqueue. + // Thread-safe. + inline bool try_enqueue(producer_token_t const& token, T const& item) + { + return inner_enqueue(token, item); + } + + // Enqueues a single item (by moving it, if possible) using an explicit producer token. + // Does not allocate memory. Fails if not enough room to enqueue. + // Thread-safe. + inline bool try_enqueue(producer_token_t const& token, T&& item) + { + return inner_enqueue(token, std::move(item)); + } + + // Enqueues several items. + // Does not allocate memory (except for one-time implicit producer). + // Fails if not enough room to enqueue (or implicit production is + // disabled because Traits::INITIAL_IMPLICIT_PRODUCER_HASH_SIZE is 0). + // Note: Use std::make_move_iterator if the elements should be moved + // instead of copied. + // Thread-safe. + template + bool try_enqueue_bulk(It itemFirst, size_t count) + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) return false; + else return inner_enqueue_bulk(itemFirst, count); + } + + // Enqueues several items using an explicit producer token. + // Does not allocate memory. Fails if not enough room to enqueue. + // Note: Use std::make_move_iterator if the elements should be moved + // instead of copied. + // Thread-safe. + template + bool try_enqueue_bulk(producer_token_t const& token, It itemFirst, size_t count) + { + return inner_enqueue_bulk(token, itemFirst, count); + } + + + + // Attempts to dequeue from the queue. + // Returns false if all producer streams appeared empty at the time they + // were checked (so, the queue is likely but not guaranteed to be empty). + // Never allocates. Thread-safe. + template + bool try_dequeue(U& item) + { + // Instead of simply trying each producer in turn (which could cause needless contention on the first + // producer), we score them heuristically. + size_t nonEmptyCount = 0; + ProducerBase* best = nullptr; + size_t bestSize = 0; + for (auto ptr = producerListTail.load(std::memory_order_acquire); nonEmptyCount < 3 && ptr != nullptr; ptr = ptr->next_prod()) { + auto size = ptr->size_approx(); + if (size > 0) { + if (size > bestSize) { + bestSize = size; + best = ptr; + } + ++nonEmptyCount; + } + } + + // If there was at least one non-empty queue but it appears empty at the time + // we try to dequeue from it, we need to make sure every queue's been tried + if (nonEmptyCount > 0) { + if ((details::likely)(best->dequeue(item))) { + return true; + } + for (auto ptr = producerListTail.load(std::memory_order_acquire); ptr != nullptr; ptr = ptr->next_prod()) { + if (ptr != best && ptr->dequeue(item)) { + return true; + } + } + } + return false; + } + + // Attempts to dequeue from the queue. + // Returns false if all producer streams appeared empty at the time they + // were checked (so, the queue is likely but not guaranteed to be empty). + // This differs from the try_dequeue(item) method in that this one does + // not attempt to reduce contention by interleaving the order that producer + // streams are dequeued from. So, using this method can reduce overall throughput + // under contention, but will give more predictable results in single-threaded + // consumer scenarios. This is mostly only useful for internal unit tests. + // Never allocates. Thread-safe. + template + bool try_dequeue_non_interleaved(U& item) + { + for (auto ptr = producerListTail.load(std::memory_order_acquire); ptr != nullptr; ptr = ptr->next_prod()) { + if (ptr->dequeue(item)) { + return true; + } + } + return false; + } + + // Attempts to dequeue from the queue using an explicit consumer token. + // Returns false if all producer streams appeared empty at the time they + // were checked (so, the queue is likely but not guaranteed to be empty). + // Never allocates. Thread-safe. + template + bool try_dequeue(consumer_token_t& token, U& item) + { + // The idea is roughly as follows: + // Every 256 items from one producer, make everyone rotate (increase the global offset) -> this means the highest efficiency consumer dictates the rotation speed of everyone else, more or less + // If you see that the global offset has changed, you must reset your consumption counter and move to your designated place + // If there's no items where you're supposed to be, keep moving until you find a producer with some items + // If the global offset has not changed but you've run out of items to consume, move over from your current position until you find an producer with something in it + + if (token.desiredProducer == nullptr || token.lastKnownGlobalOffset != globalExplicitConsumerOffset.load(std::memory_order_relaxed)) { + if (!update_current_producer_after_rotation(token)) { + return false; + } + } + + // If there was at least one non-empty queue but it appears empty at the time + // we try to dequeue from it, we need to make sure every queue's been tried + if (static_cast(token.currentProducer)->dequeue(item)) { + if (++token.itemsConsumedFromCurrent == EXPLICIT_CONSUMER_CONSUMPTION_QUOTA_BEFORE_ROTATE) { + globalExplicitConsumerOffset.fetch_add(1, std::memory_order_relaxed); + } + return true; + } + + auto tail = producerListTail.load(std::memory_order_acquire); + auto ptr = static_cast(token.currentProducer)->next_prod(); + if (ptr == nullptr) { + ptr = tail; + } + while (ptr != static_cast(token.currentProducer)) { + if (ptr->dequeue(item)) { + token.currentProducer = ptr; + token.itemsConsumedFromCurrent = 1; + return true; + } + ptr = ptr->next_prod(); + if (ptr == nullptr) { + ptr = tail; + } + } + return false; + } + + // Attempts to dequeue several elements from the queue. + // Returns the number of items actually dequeued. + // Returns 0 if all producer streams appeared empty at the time they + // were checked (so, the queue is likely but not guaranteed to be empty). + // Never allocates. Thread-safe. + template + size_t try_dequeue_bulk(It itemFirst, size_t max) + { + size_t count = 0; + for (auto ptr = producerListTail.load(std::memory_order_acquire); ptr != nullptr; ptr = ptr->next_prod()) { + count += ptr->dequeue_bulk(itemFirst, max - count); + if (count == max) { + break; + } + } + return count; + } + + // Attempts to dequeue several elements from the queue using an explicit consumer token. + // Returns the number of items actually dequeued. + // Returns 0 if all producer streams appeared empty at the time they + // were checked (so, the queue is likely but not guaranteed to be empty). + // Never allocates. Thread-safe. + template + size_t try_dequeue_bulk(consumer_token_t& token, It itemFirst, size_t max) + { + if (token.desiredProducer == nullptr || token.lastKnownGlobalOffset != globalExplicitConsumerOffset.load(std::memory_order_relaxed)) { + if (!update_current_producer_after_rotation(token)) { + return 0; + } + } + + size_t count = static_cast(token.currentProducer)->dequeue_bulk(itemFirst, max); + if (count == max) { + if ((token.itemsConsumedFromCurrent += static_cast(max)) >= EXPLICIT_CONSUMER_CONSUMPTION_QUOTA_BEFORE_ROTATE) { + globalExplicitConsumerOffset.fetch_add(1, std::memory_order_relaxed); + } + return max; + } + token.itemsConsumedFromCurrent += static_cast(count); + max -= count; + + auto tail = producerListTail.load(std::memory_order_acquire); + auto ptr = static_cast(token.currentProducer)->next_prod(); + if (ptr == nullptr) { + ptr = tail; + } + while (ptr != static_cast(token.currentProducer)) { + auto dequeued = ptr->dequeue_bulk(itemFirst, max); + count += dequeued; + if (dequeued != 0) { + token.currentProducer = ptr; + token.itemsConsumedFromCurrent = static_cast(dequeued); + } + if (dequeued == max) { + break; + } + max -= dequeued; + ptr = ptr->next_prod(); + if (ptr == nullptr) { + ptr = tail; + } + } + return count; + } + + + + // Attempts to dequeue from a specific producer's inner queue. + // If you happen to know which producer you want to dequeue from, this + // is significantly faster than using the general-case try_dequeue methods. + // Returns false if the producer's queue appeared empty at the time it + // was checked (so, the queue is likely but not guaranteed to be empty). + // Never allocates. Thread-safe. + template + inline bool try_dequeue_from_producer(producer_token_t const& producer, U& item) + { + return static_cast(producer.producer)->dequeue(item); + } + + // Attempts to dequeue several elements from a specific producer's inner queue. + // Returns the number of items actually dequeued. + // If you happen to know which producer you want to dequeue from, this + // is significantly faster than using the general-case try_dequeue methods. + // Returns 0 if the producer's queue appeared empty at the time it + // was checked (so, the queue is likely but not guaranteed to be empty). + // Never allocates. Thread-safe. + template + inline size_t try_dequeue_bulk_from_producer(producer_token_t const& producer, It itemFirst, size_t max) + { + return static_cast(producer.producer)->dequeue_bulk(itemFirst, max); + } + + + // Returns an estimate of the total number of elements currently in the queue. This + // estimate is only accurate if the queue has completely stabilized before it is called + // (i.e. all enqueue and dequeue operations have completed and their memory effects are + // visible on the calling thread, and no further operations start while this method is + // being called). + // Thread-safe. + size_t size_approx() const + { + size_t size = 0; + for (auto ptr = producerListTail.load(std::memory_order_acquire); ptr != nullptr; ptr = ptr->next_prod()) { + size += ptr->size_approx(); + } + return size; + } + + + // Returns true if the underlying atomic variables used by + // the queue are lock-free (they should be on most platforms). + // Thread-safe. + static constexpr bool is_lock_free() + { + return + details::static_is_lock_free::value == 2 && + details::static_is_lock_free::value == 2 && + details::static_is_lock_free::value == 2 && + details::static_is_lock_free::value == 2 && + details::static_is_lock_free::value == 2 && + details::static_is_lock_free::thread_id_numeric_size_t>::value == 2; + } + + +private: + friend struct ProducerToken; + friend struct ConsumerToken; + struct ExplicitProducer; + friend struct ExplicitProducer; + struct ImplicitProducer; + friend struct ImplicitProducer; + friend class ConcurrentQueueTests; + + enum AllocationMode { CanAlloc, CannotAlloc }; + + + /////////////////////////////// + // Queue methods + /////////////////////////////// + + template + inline bool inner_enqueue(producer_token_t const& token, U&& element) + { + return static_cast(token.producer)->ConcurrentQueue::ExplicitProducer::template enqueue(std::forward(element)); + } + + template + inline bool inner_enqueue(U&& element) + { + auto producer = get_or_add_implicit_producer(); + return producer == nullptr ? false : producer->ConcurrentQueue::ImplicitProducer::template enqueue(std::forward(element)); + } + + template + inline bool inner_enqueue_bulk(producer_token_t const& token, It itemFirst, size_t count) + { + return static_cast(token.producer)->ConcurrentQueue::ExplicitProducer::template enqueue_bulk(itemFirst, count); + } + + template + inline bool inner_enqueue_bulk(It itemFirst, size_t count) + { + auto producer = get_or_add_implicit_producer(); + return producer == nullptr ? false : producer->ConcurrentQueue::ImplicitProducer::template enqueue_bulk(itemFirst, count); + } + + inline bool update_current_producer_after_rotation(consumer_token_t& token) + { + // Ah, there's been a rotation, figure out where we should be! + auto tail = producerListTail.load(std::memory_order_acquire); + if (token.desiredProducer == nullptr && tail == nullptr) { + return false; + } + auto prodCount = producerCount.load(std::memory_order_relaxed); + auto globalOffset = globalExplicitConsumerOffset.load(std::memory_order_relaxed); + if ((details::unlikely)(token.desiredProducer == nullptr)) { + // Aha, first time we're dequeueing anything. + // Figure out our local position + // Note: offset is from start, not end, but we're traversing from end -- subtract from count first + std::uint32_t offset = prodCount - 1 - (token.initialOffset % prodCount); + token.desiredProducer = tail; + for (std::uint32_t i = 0; i != offset; ++i) { + token.desiredProducer = static_cast(token.desiredProducer)->next_prod(); + if (token.desiredProducer == nullptr) { + token.desiredProducer = tail; + } + } + } + + std::uint32_t delta = globalOffset - token.lastKnownGlobalOffset; + if (delta >= prodCount) { + delta = delta % prodCount; + } + for (std::uint32_t i = 0; i != delta; ++i) { + token.desiredProducer = static_cast(token.desiredProducer)->next_prod(); + if (token.desiredProducer == nullptr) { + token.desiredProducer = tail; + } + } + + token.lastKnownGlobalOffset = globalOffset; + token.currentProducer = token.desiredProducer; + token.itemsConsumedFromCurrent = 0; + return true; + } + + + /////////////////////////// + // Free list + /////////////////////////// + + template + struct FreeListNode + { + FreeListNode() : freeListRefs(0), freeListNext(nullptr) { } + + std::atomic freeListRefs; + std::atomic freeListNext; + }; + + // A simple CAS-based lock-free free list. Not the fastest thing in the world under heavy contention, but + // simple and correct (assuming nodes are never freed until after the free list is destroyed), and fairly + // speedy under low contention. + template // N must inherit FreeListNode or have the same fields (and initialization of them) + struct FreeList + { + FreeList() : freeListHead(nullptr) { } + FreeList(FreeList&& other) : freeListHead(other.freeListHead.load(std::memory_order_relaxed)) { other.freeListHead.store(nullptr, std::memory_order_relaxed); } + void swap(FreeList& other) { details::swap_relaxed(freeListHead, other.freeListHead); } + + FreeList(FreeList const&) MOODYCAMEL_DELETE_FUNCTION; + FreeList& operator=(FreeList const&) MOODYCAMEL_DELETE_FUNCTION; + + inline void add(N* node) + { +#ifdef MCDBGQ_NOLOCKFREE_FREELIST + debug::DebugLock lock(mutex); +#endif + // We know that the should-be-on-freelist bit is 0 at this point, so it's safe to + // set it using a fetch_add + if (node->freeListRefs.fetch_add(SHOULD_BE_ON_FREELIST, std::memory_order_acq_rel) == 0) { + // Oh look! We were the last ones referencing this node, and we know + // we want to add it to the free list, so let's do it! + add_knowing_refcount_is_zero(node); + } + } + + inline N* try_get() + { +#ifdef MCDBGQ_NOLOCKFREE_FREELIST + debug::DebugLock lock(mutex); +#endif + auto head = freeListHead.load(std::memory_order_acquire); + while (head != nullptr) { + auto prevHead = head; + auto refs = head->freeListRefs.load(std::memory_order_relaxed); + if ((refs & REFS_MASK) == 0 || !head->freeListRefs.compare_exchange_strong(refs, refs + 1, std::memory_order_acquire)) { + head = freeListHead.load(std::memory_order_acquire); + continue; + } + + // Good, reference count has been incremented (it wasn't at zero), which means we can read the + // next and not worry about it changing between now and the time we do the CAS + auto next = head->freeListNext.load(std::memory_order_relaxed); + if (freeListHead.compare_exchange_strong(head, next, std::memory_order_acquire, std::memory_order_relaxed)) { + // Yay, got the node. This means it was on the list, which means shouldBeOnFreeList must be false no + // matter the refcount (because nobody else knows it's been taken off yet, it can't have been put back on). + assert((head->freeListRefs.load(std::memory_order_relaxed) & SHOULD_BE_ON_FREELIST) == 0); + + // Decrease refcount twice, once for our ref, and once for the list's ref + head->freeListRefs.fetch_sub(2, std::memory_order_release); + return head; + } + + // OK, the head must have changed on us, but we still need to decrease the refcount we increased. + // Note that we don't need to release any memory effects, but we do need to ensure that the reference + // count decrement happens-after the CAS on the head. + refs = prevHead->freeListRefs.fetch_sub(1, std::memory_order_acq_rel); + if (refs == SHOULD_BE_ON_FREELIST + 1) { + add_knowing_refcount_is_zero(prevHead); + } + } + + return nullptr; + } + + // Useful for traversing the list when there's no contention (e.g. to destroy remaining nodes) + N* head_unsafe() const { return freeListHead.load(std::memory_order_relaxed); } + + private: + inline void add_knowing_refcount_is_zero(N* node) + { + // Since the refcount is zero, and nobody can increase it once it's zero (except us, and we run + // only one copy of this method per node at a time, i.e. the single thread case), then we know + // we can safely change the next pointer of the node; however, once the refcount is back above + // zero, then other threads could increase it (happens under heavy contention, when the refcount + // goes to zero in between a load and a refcount increment of a node in try_get, then back up to + // something non-zero, then the refcount increment is done by the other thread) -- so, if the CAS + // to add the node to the actual list fails, decrease the refcount and leave the add operation to + // the next thread who puts the refcount back at zero (which could be us, hence the loop). + auto head = freeListHead.load(std::memory_order_relaxed); + while (true) { + node->freeListNext.store(head, std::memory_order_relaxed); + node->freeListRefs.store(1, std::memory_order_release); + if (!freeListHead.compare_exchange_strong(head, node, std::memory_order_release, std::memory_order_relaxed)) { + // Hmm, the add failed, but we can only try again when the refcount goes back to zero + if (node->freeListRefs.fetch_add(SHOULD_BE_ON_FREELIST - 1, std::memory_order_acq_rel) == 1) { + continue; + } + } + return; + } + } + + private: + // Implemented like a stack, but where node order doesn't matter (nodes are inserted out of order under contention) + std::atomic freeListHead; + + static const std::uint32_t REFS_MASK = 0x7FFFFFFF; + static const std::uint32_t SHOULD_BE_ON_FREELIST = 0x80000000; + +#ifdef MCDBGQ_NOLOCKFREE_FREELIST + debug::DebugMutex mutex; +#endif + }; + + + /////////////////////////// + // Block + /////////////////////////// + + enum InnerQueueContext { implicit_context = 0, explicit_context = 1 }; + + struct Block + { + Block() + : next(nullptr), elementsCompletelyDequeued(0), freeListRefs(0), freeListNext(nullptr), dynamicallyAllocated(true) + { +#ifdef MCDBGQ_TRACKMEM + owner = nullptr; +#endif + } + + template + inline bool is_empty() const + { + MOODYCAMEL_CONSTEXPR_IF (context == explicit_context && BLOCK_SIZE <= EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD) { + // Check flags + for (size_t i = 0; i < BLOCK_SIZE; ++i) { + if (!emptyFlags[i].load(std::memory_order_relaxed)) { + return false; + } + } + + // Aha, empty; make sure we have all other memory effects that happened before the empty flags were set + std::atomic_thread_fence(std::memory_order_acquire); + return true; + } + else { + // Check counter + if (elementsCompletelyDequeued.load(std::memory_order_relaxed) == BLOCK_SIZE) { + std::atomic_thread_fence(std::memory_order_acquire); + return true; + } + assert(elementsCompletelyDequeued.load(std::memory_order_relaxed) <= BLOCK_SIZE); + return false; + } + } + + // Returns true if the block is now empty (does not apply in explicit context) + template + inline bool set_empty(MOODYCAMEL_MAYBE_UNUSED index_t i) + { + MOODYCAMEL_CONSTEXPR_IF (context == explicit_context && BLOCK_SIZE <= EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD) { + // Set flag + assert(!emptyFlags[BLOCK_SIZE - 1 - static_cast(i & static_cast(BLOCK_SIZE - 1))].load(std::memory_order_relaxed)); + emptyFlags[BLOCK_SIZE - 1 - static_cast(i & static_cast(BLOCK_SIZE - 1))].store(true, std::memory_order_release); + return false; + } + else { + // Increment counter + auto prevVal = elementsCompletelyDequeued.fetch_add(1, std::memory_order_acq_rel); + assert(prevVal < BLOCK_SIZE); + return prevVal == BLOCK_SIZE - 1; + } + } + + // Sets multiple contiguous item statuses to 'empty' (assumes no wrapping and count > 0). + // Returns true if the block is now empty (does not apply in explicit context). + template + inline bool set_many_empty(MOODYCAMEL_MAYBE_UNUSED index_t i, size_t count) + { + MOODYCAMEL_CONSTEXPR_IF (context == explicit_context && BLOCK_SIZE <= EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD) { + // Set flags + std::atomic_thread_fence(std::memory_order_release); + i = BLOCK_SIZE - 1 - static_cast(i & static_cast(BLOCK_SIZE - 1)) - count + 1; + for (size_t j = 0; j != count; ++j) { + assert(!emptyFlags[i + j].load(std::memory_order_relaxed)); + emptyFlags[i + j].store(true, std::memory_order_relaxed); + } + return false; + } + else { + // Increment counter + auto prevVal = elementsCompletelyDequeued.fetch_add(count, std::memory_order_acq_rel); + assert(prevVal + count <= BLOCK_SIZE); + return prevVal + count == BLOCK_SIZE; + } + } + + template + inline void set_all_empty() + { + MOODYCAMEL_CONSTEXPR_IF (context == explicit_context && BLOCK_SIZE <= EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD) { + // Set all flags + for (size_t i = 0; i != BLOCK_SIZE; ++i) { + emptyFlags[i].store(true, std::memory_order_relaxed); + } + } + else { + // Reset counter + elementsCompletelyDequeued.store(BLOCK_SIZE, std::memory_order_relaxed); + } + } + + template + inline void reset_empty() + { + MOODYCAMEL_CONSTEXPR_IF (context == explicit_context && BLOCK_SIZE <= EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD) { + // Reset flags + for (size_t i = 0; i != BLOCK_SIZE; ++i) { + emptyFlags[i].store(false, std::memory_order_relaxed); + } + } + else { + // Reset counter + elementsCompletelyDequeued.store(0, std::memory_order_relaxed); + } + } + + inline T* operator[](index_t idx) MOODYCAMEL_NOEXCEPT { return static_cast(static_cast(elements)) + static_cast(idx & static_cast(BLOCK_SIZE - 1)); } + inline T const* operator[](index_t idx) const MOODYCAMEL_NOEXCEPT { return static_cast(static_cast(elements)) + static_cast(idx & static_cast(BLOCK_SIZE - 1)); } + + private: + static_assert(std::alignment_of::value <= sizeof(T), "The queue does not support types with an alignment greater than their size at this time"); + MOODYCAMEL_ALIGNED_TYPE_LIKE(char[sizeof(T) * BLOCK_SIZE], T) elements; + public: + Block* next; + std::atomic elementsCompletelyDequeued; + std::atomic emptyFlags[BLOCK_SIZE <= EXPLICIT_BLOCK_EMPTY_COUNTER_THRESHOLD ? BLOCK_SIZE : 1]; + public: + std::atomic freeListRefs; + std::atomic freeListNext; + bool dynamicallyAllocated; // Perhaps a better name for this would be 'isNotPartOfInitialBlockPool' + +#ifdef MCDBGQ_TRACKMEM + void* owner; +#endif + }; + static_assert(std::alignment_of::value >= std::alignment_of::value, "Internal error: Blocks must be at least as aligned as the type they are wrapping"); + + +#ifdef MCDBGQ_TRACKMEM +public: + struct MemStats; +private: +#endif + + /////////////////////////// + // Producer base + /////////////////////////// + + struct ProducerBase : public details::ConcurrentQueueProducerTypelessBase + { + ProducerBase(ConcurrentQueue* parent_, bool isExplicit_) : + tailIndex(0), + headIndex(0), + dequeueOptimisticCount(0), + dequeueOvercommit(0), + tailBlock(nullptr), + isExplicit(isExplicit_), + parent(parent_) + { + } + + virtual ~ProducerBase() { } + + template + inline bool dequeue(U& element) + { + if (isExplicit) { + return static_cast(this)->dequeue(element); + } + else { + return static_cast(this)->dequeue(element); + } + } + + template + inline size_t dequeue_bulk(It& itemFirst, size_t max) + { + if (isExplicit) { + return static_cast(this)->dequeue_bulk(itemFirst, max); + } + else { + return static_cast(this)->dequeue_bulk(itemFirst, max); + } + } + + inline ProducerBase* next_prod() const { return static_cast(next); } + + inline size_t size_approx() const + { + auto tail = tailIndex.load(std::memory_order_relaxed); + auto head = headIndex.load(std::memory_order_relaxed); + return details::circular_less_than(head, tail) ? static_cast(tail - head) : 0; + } + + inline index_t getTail() const { return tailIndex.load(std::memory_order_relaxed); } + protected: + std::atomic tailIndex; // Where to enqueue to next + std::atomic headIndex; // Where to dequeue from next + + std::atomic dequeueOptimisticCount; + std::atomic dequeueOvercommit; + + Block* tailBlock; + + public: + bool isExplicit; + ConcurrentQueue* parent; + + protected: +#ifdef MCDBGQ_TRACKMEM + friend struct MemStats; +#endif + }; + + + /////////////////////////// + // Explicit queue + /////////////////////////// + + struct ExplicitProducer : public ProducerBase + { + explicit ExplicitProducer(ConcurrentQueue* parent_) : + ProducerBase(parent_, true), + blockIndex(nullptr), + pr_blockIndexSlotsUsed(0), + pr_blockIndexSize(EXPLICIT_INITIAL_INDEX_SIZE >> 1), + pr_blockIndexFront(0), + pr_blockIndexEntries(nullptr), + pr_blockIndexRaw(nullptr) + { + size_t poolBasedIndexSize = details::ceil_to_pow_2(parent_->initialBlockPoolSize) >> 1; + if (poolBasedIndexSize > pr_blockIndexSize) { + pr_blockIndexSize = poolBasedIndexSize; + } + + new_block_index(0); // This creates an index with double the number of current entries, i.e. EXPLICIT_INITIAL_INDEX_SIZE + } + + ~ExplicitProducer() + { + // Destruct any elements not yet dequeued. + // Since we're in the destructor, we can assume all elements + // are either completely dequeued or completely not (no halfways). + if (this->tailBlock != nullptr) { // Note this means there must be a block index too + // First find the block that's partially dequeued, if any + Block* halfDequeuedBlock = nullptr; + if ((this->headIndex.load(std::memory_order_relaxed) & static_cast(BLOCK_SIZE - 1)) != 0) { + // The head's not on a block boundary, meaning a block somewhere is partially dequeued + // (or the head block is the tail block and was fully dequeued, but the head/tail are still not on a boundary) + size_t i = (pr_blockIndexFront - pr_blockIndexSlotsUsed) & (pr_blockIndexSize - 1); + while (details::circular_less_than(pr_blockIndexEntries[i].base + BLOCK_SIZE, this->headIndex.load(std::memory_order_relaxed))) { + i = (i + 1) & (pr_blockIndexSize - 1); + } + assert(details::circular_less_than(pr_blockIndexEntries[i].base, this->headIndex.load(std::memory_order_relaxed))); + halfDequeuedBlock = pr_blockIndexEntries[i].block; + } + + // Start at the head block (note the first line in the loop gives us the head from the tail on the first iteration) + auto block = this->tailBlock; + do { + block = block->next; + if (block->ConcurrentQueue::Block::template is_empty()) { + continue; + } + + size_t i = 0; // Offset into block + if (block == halfDequeuedBlock) { + i = static_cast(this->headIndex.load(std::memory_order_relaxed) & static_cast(BLOCK_SIZE - 1)); + } + + // Walk through all the items in the block; if this is the tail block, we need to stop when we reach the tail index + auto lastValidIndex = (this->tailIndex.load(std::memory_order_relaxed) & static_cast(BLOCK_SIZE - 1)) == 0 ? BLOCK_SIZE : static_cast(this->tailIndex.load(std::memory_order_relaxed) & static_cast(BLOCK_SIZE - 1)); + while (i != BLOCK_SIZE && (block != this->tailBlock || i != lastValidIndex)) { + (*block)[i++]->~T(); + } + } while (block != this->tailBlock); + } + + // Destroy all blocks that we own + if (this->tailBlock != nullptr) { + auto block = this->tailBlock; + do { + auto nextBlock = block->next; + this->parent->add_block_to_free_list(block); + block = nextBlock; + } while (block != this->tailBlock); + } + + // Destroy the block indices + auto header = static_cast(pr_blockIndexRaw); + while (header != nullptr) { + auto prev = static_cast(header->prev); + header->~BlockIndexHeader(); + (Traits::free)(header); + header = prev; + } + } + + template + inline bool enqueue(U&& element) + { + index_t currentTailIndex = this->tailIndex.load(std::memory_order_relaxed); + index_t newTailIndex = 1 + currentTailIndex; + if ((currentTailIndex & static_cast(BLOCK_SIZE - 1)) == 0) { + // We reached the end of a block, start a new one + auto startBlock = this->tailBlock; + auto originalBlockIndexSlotsUsed = pr_blockIndexSlotsUsed; + if (this->tailBlock != nullptr && this->tailBlock->next->ConcurrentQueue::Block::template is_empty()) { + // We can re-use the block ahead of us, it's empty! + this->tailBlock = this->tailBlock->next; + this->tailBlock->ConcurrentQueue::Block::template reset_empty(); + + // We'll put the block on the block index (guaranteed to be room since we're conceptually removing the + // last block from it first -- except instead of removing then adding, we can just overwrite). + // Note that there must be a valid block index here, since even if allocation failed in the ctor, + // it would have been re-attempted when adding the first block to the queue; since there is such + // a block, a block index must have been successfully allocated. + } + else { + // Whatever head value we see here is >= the last value we saw here (relatively), + // and <= its current value. Since we have the most recent tail, the head must be + // <= to it. + auto head = this->headIndex.load(std::memory_order_relaxed); + assert(!details::circular_less_than(currentTailIndex, head)); + if (!details::circular_less_than(head, currentTailIndex + BLOCK_SIZE) + || (MAX_SUBQUEUE_SIZE != details::const_numeric_max::value && (MAX_SUBQUEUE_SIZE == 0 || MAX_SUBQUEUE_SIZE - BLOCK_SIZE < currentTailIndex - head))) { + // We can't enqueue in another block because there's not enough leeway -- the + // tail could surpass the head by the time the block fills up! (Or we'll exceed + // the size limit, if the second part of the condition was true.) + return false; + } + // We're going to need a new block; check that the block index has room + if (pr_blockIndexRaw == nullptr || pr_blockIndexSlotsUsed == pr_blockIndexSize) { + // Hmm, the circular block index is already full -- we'll need + // to allocate a new index. Note pr_blockIndexRaw can only be nullptr if + // the initial allocation failed in the constructor. + + MOODYCAMEL_CONSTEXPR_IF (allocMode == CannotAlloc) { + return false; + } + else if (!new_block_index(pr_blockIndexSlotsUsed)) { + return false; + } + } + + // Insert a new block in the circular linked list + auto newBlock = this->parent->ConcurrentQueue::template requisition_block(); + if (newBlock == nullptr) { + return false; + } +#ifdef MCDBGQ_TRACKMEM + newBlock->owner = this; +#endif + newBlock->ConcurrentQueue::Block::template reset_empty(); + if (this->tailBlock == nullptr) { + newBlock->next = newBlock; + } + else { + newBlock->next = this->tailBlock->next; + this->tailBlock->next = newBlock; + } + this->tailBlock = newBlock; + ++pr_blockIndexSlotsUsed; + } + + MOODYCAMEL_CONSTEXPR_IF (!MOODYCAMEL_NOEXCEPT_CTOR(T, U, new (static_cast(nullptr)) T(std::forward(element)))) { + // The constructor may throw. We want the element not to appear in the queue in + // that case (without corrupting the queue): + MOODYCAMEL_TRY { + new ((*this->tailBlock)[currentTailIndex]) T(std::forward(element)); + } + MOODYCAMEL_CATCH (...) { + // Revert change to the current block, but leave the new block available + // for next time + pr_blockIndexSlotsUsed = originalBlockIndexSlotsUsed; + this->tailBlock = startBlock == nullptr ? this->tailBlock : startBlock; + MOODYCAMEL_RETHROW; + } + } + else { + (void)startBlock; + (void)originalBlockIndexSlotsUsed; + } + + // Add block to block index + auto& entry = blockIndex.load(std::memory_order_relaxed)->entries[pr_blockIndexFront]; + entry.base = currentTailIndex; + entry.block = this->tailBlock; + blockIndex.load(std::memory_order_relaxed)->front.store(pr_blockIndexFront, std::memory_order_release); + pr_blockIndexFront = (pr_blockIndexFront + 1) & (pr_blockIndexSize - 1); + + MOODYCAMEL_CONSTEXPR_IF (!MOODYCAMEL_NOEXCEPT_CTOR(T, U, new (static_cast(nullptr)) T(std::forward(element)))) { + this->tailIndex.store(newTailIndex, std::memory_order_release); + return true; + } + } + + // Enqueue + new ((*this->tailBlock)[currentTailIndex]) T(std::forward(element)); + + this->tailIndex.store(newTailIndex, std::memory_order_release); + return true; + } + + template + bool dequeue(U& element) + { + auto tail = this->tailIndex.load(std::memory_order_relaxed); + auto overcommit = this->dequeueOvercommit.load(std::memory_order_relaxed); + if (details::circular_less_than(this->dequeueOptimisticCount.load(std::memory_order_relaxed) - overcommit, tail)) { + // Might be something to dequeue, let's give it a try + + // Note that this if is purely for performance purposes in the common case when the queue is + // empty and the values are eventually consistent -- we may enter here spuriously. + + // Note that whatever the values of overcommit and tail are, they are not going to change (unless we + // change them) and must be the same value at this point (inside the if) as when the if condition was + // evaluated. + + // We insert an acquire fence here to synchronize-with the release upon incrementing dequeueOvercommit below. + // This ensures that whatever the value we got loaded into overcommit, the load of dequeueOptisticCount in + // the fetch_add below will result in a value at least as recent as that (and therefore at least as large). + // Note that I believe a compiler (signal) fence here would be sufficient due to the nature of fetch_add (all + // read-modify-write operations are guaranteed to work on the latest value in the modification order), but + // unfortunately that can't be shown to be correct using only the C++11 standard. + // See http://stackoverflow.com/questions/18223161/what-are-the-c11-memory-ordering-guarantees-in-this-corner-case + std::atomic_thread_fence(std::memory_order_acquire); + + // Increment optimistic counter, then check if it went over the boundary + auto myDequeueCount = this->dequeueOptimisticCount.fetch_add(1, std::memory_order_relaxed); + + // Note that since dequeueOvercommit must be <= dequeueOptimisticCount (because dequeueOvercommit is only ever + // incremented after dequeueOptimisticCount -- this is enforced in the `else` block below), and since we now + // have a version of dequeueOptimisticCount that is at least as recent as overcommit (due to the release upon + // incrementing dequeueOvercommit and the acquire above that synchronizes with it), overcommit <= myDequeueCount. + // However, we can't assert this since both dequeueOptimisticCount and dequeueOvercommit may (independently) + // overflow; in such a case, though, the logic still holds since the difference between the two is maintained. + + // Note that we reload tail here in case it changed; it will be the same value as before or greater, since + // this load is sequenced after (happens after) the earlier load above. This is supported by read-read + // coherency (as defined in the standard), explained here: http://en.cppreference.com/w/cpp/atomic/memory_order + tail = this->tailIndex.load(std::memory_order_acquire); + if ((details::likely)(details::circular_less_than(myDequeueCount - overcommit, tail))) { + // Guaranteed to be at least one element to dequeue! + + // Get the index. Note that since there's guaranteed to be at least one element, this + // will never exceed tail. We need to do an acquire-release fence here since it's possible + // that whatever condition got us to this point was for an earlier enqueued element (that + // we already see the memory effects for), but that by the time we increment somebody else + // has incremented it, and we need to see the memory effects for *that* element, which is + // in such a case is necessarily visible on the thread that incremented it in the first + // place with the more current condition (they must have acquired a tail that is at least + // as recent). + auto index = this->headIndex.fetch_add(1, std::memory_order_acq_rel); + + + // Determine which block the element is in + + auto localBlockIndex = blockIndex.load(std::memory_order_acquire); + auto localBlockIndexHead = localBlockIndex->front.load(std::memory_order_acquire); + + // We need to be careful here about subtracting and dividing because of index wrap-around. + // When an index wraps, we need to preserve the sign of the offset when dividing it by the + // block size (in order to get a correct signed block count offset in all cases): + auto headBase = localBlockIndex->entries[localBlockIndexHead].base; + auto blockBaseIndex = index & ~static_cast(BLOCK_SIZE - 1); + auto offset = static_cast(static_cast::type>(blockBaseIndex - headBase) / static_cast::type>(BLOCK_SIZE)); + auto block = localBlockIndex->entries[(localBlockIndexHead + offset) & (localBlockIndex->size - 1)].block; + + // Dequeue + auto& el = *((*block)[index]); + if (!MOODYCAMEL_NOEXCEPT_ASSIGN(T, T&&, element = std::move(el))) { + // Make sure the element is still fully dequeued and destroyed even if the assignment + // throws + struct Guard { + Block* block; + index_t index; + + ~Guard() + { + (*block)[index]->~T(); + block->ConcurrentQueue::Block::template set_empty(index); + } + } guard = { block, index }; + + element = std::move(el); // NOLINT + } + else { + element = std::move(el); // NOLINT + el.~T(); // NOLINT + block->ConcurrentQueue::Block::template set_empty(index); + } + + return true; + } + else { + // Wasn't anything to dequeue after all; make the effective dequeue count eventually consistent + this->dequeueOvercommit.fetch_add(1, std::memory_order_release); // Release so that the fetch_add on dequeueOptimisticCount is guaranteed to happen before this write + } + } + + return false; + } + + template + bool MOODYCAMEL_NO_TSAN enqueue_bulk(It itemFirst, size_t count) + { + // First, we need to make sure we have enough room to enqueue all of the elements; + // this means pre-allocating blocks and putting them in the block index (but only if + // all the allocations succeeded). + index_t startTailIndex = this->tailIndex.load(std::memory_order_relaxed); + auto startBlock = this->tailBlock; + auto originalBlockIndexFront = pr_blockIndexFront; + auto originalBlockIndexSlotsUsed = pr_blockIndexSlotsUsed; + + Block* firstAllocatedBlock = nullptr; + + // Figure out how many blocks we'll need to allocate, and do so + size_t blockBaseDiff = ((startTailIndex + count - 1) & ~static_cast(BLOCK_SIZE - 1)) - ((startTailIndex - 1) & ~static_cast(BLOCK_SIZE - 1)); + index_t currentTailIndex = (startTailIndex - 1) & ~static_cast(BLOCK_SIZE - 1); + if (blockBaseDiff > 0) { + // Allocate as many blocks as possible from ahead + while (blockBaseDiff > 0 && this->tailBlock != nullptr && this->tailBlock->next != firstAllocatedBlock && this->tailBlock->next->ConcurrentQueue::Block::template is_empty()) { + blockBaseDiff -= static_cast(BLOCK_SIZE); + currentTailIndex += static_cast(BLOCK_SIZE); + + this->tailBlock = this->tailBlock->next; + firstAllocatedBlock = firstAllocatedBlock == nullptr ? this->tailBlock : firstAllocatedBlock; + + auto& entry = blockIndex.load(std::memory_order_relaxed)->entries[pr_blockIndexFront]; + entry.base = currentTailIndex; + entry.block = this->tailBlock; + pr_blockIndexFront = (pr_blockIndexFront + 1) & (pr_blockIndexSize - 1); + } + + // Now allocate as many blocks as necessary from the block pool + while (blockBaseDiff > 0) { + blockBaseDiff -= static_cast(BLOCK_SIZE); + currentTailIndex += static_cast(BLOCK_SIZE); + + auto head = this->headIndex.load(std::memory_order_relaxed); + assert(!details::circular_less_than(currentTailIndex, head)); + bool full = !details::circular_less_than(head, currentTailIndex + BLOCK_SIZE) || (MAX_SUBQUEUE_SIZE != details::const_numeric_max::value && (MAX_SUBQUEUE_SIZE == 0 || MAX_SUBQUEUE_SIZE - BLOCK_SIZE < currentTailIndex - head)); + if (pr_blockIndexRaw == nullptr || pr_blockIndexSlotsUsed == pr_blockIndexSize || full) { + MOODYCAMEL_CONSTEXPR_IF (allocMode == CannotAlloc) { + // Failed to allocate, undo changes (but keep injected blocks) + pr_blockIndexFront = originalBlockIndexFront; + pr_blockIndexSlotsUsed = originalBlockIndexSlotsUsed; + this->tailBlock = startBlock == nullptr ? firstAllocatedBlock : startBlock; + return false; + } + else if (full || !new_block_index(originalBlockIndexSlotsUsed)) { + // Failed to allocate, undo changes (but keep injected blocks) + pr_blockIndexFront = originalBlockIndexFront; + pr_blockIndexSlotsUsed = originalBlockIndexSlotsUsed; + this->tailBlock = startBlock == nullptr ? firstAllocatedBlock : startBlock; + return false; + } + + // pr_blockIndexFront is updated inside new_block_index, so we need to + // update our fallback value too (since we keep the new index even if we + // later fail) + originalBlockIndexFront = originalBlockIndexSlotsUsed; + } + + // Insert a new block in the circular linked list + auto newBlock = this->parent->ConcurrentQueue::template requisition_block(); + if (newBlock == nullptr) { + pr_blockIndexFront = originalBlockIndexFront; + pr_blockIndexSlotsUsed = originalBlockIndexSlotsUsed; + this->tailBlock = startBlock == nullptr ? firstAllocatedBlock : startBlock; + return false; + } + +#ifdef MCDBGQ_TRACKMEM + newBlock->owner = this; +#endif + newBlock->ConcurrentQueue::Block::template set_all_empty(); + if (this->tailBlock == nullptr) { + newBlock->next = newBlock; + } + else { + newBlock->next = this->tailBlock->next; + this->tailBlock->next = newBlock; + } + this->tailBlock = newBlock; + firstAllocatedBlock = firstAllocatedBlock == nullptr ? this->tailBlock : firstAllocatedBlock; + + ++pr_blockIndexSlotsUsed; + + auto& entry = blockIndex.load(std::memory_order_relaxed)->entries[pr_blockIndexFront]; + entry.base = currentTailIndex; + entry.block = this->tailBlock; + pr_blockIndexFront = (pr_blockIndexFront + 1) & (pr_blockIndexSize - 1); + } + + // Excellent, all allocations succeeded. Reset each block's emptiness before we fill them up, and + // publish the new block index front + auto block = firstAllocatedBlock; + while (true) { + block->ConcurrentQueue::Block::template reset_empty(); + if (block == this->tailBlock) { + break; + } + block = block->next; + } + + MOODYCAMEL_CONSTEXPR_IF (MOODYCAMEL_NOEXCEPT_CTOR(T, decltype(*itemFirst), new (static_cast(nullptr)) T(details::deref_noexcept(itemFirst)))) { + blockIndex.load(std::memory_order_relaxed)->front.store((pr_blockIndexFront - 1) & (pr_blockIndexSize - 1), std::memory_order_release); + } + } + + // Enqueue, one block at a time + index_t newTailIndex = startTailIndex + static_cast(count); + currentTailIndex = startTailIndex; + auto endBlock = this->tailBlock; + this->tailBlock = startBlock; + assert((startTailIndex & static_cast(BLOCK_SIZE - 1)) != 0 || firstAllocatedBlock != nullptr || count == 0); + if ((startTailIndex & static_cast(BLOCK_SIZE - 1)) == 0 && firstAllocatedBlock != nullptr) { + this->tailBlock = firstAllocatedBlock; + } + while (true) { + index_t stopIndex = (currentTailIndex & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + if (details::circular_less_than(newTailIndex, stopIndex)) { + stopIndex = newTailIndex; + } + MOODYCAMEL_CONSTEXPR_IF (MOODYCAMEL_NOEXCEPT_CTOR(T, decltype(*itemFirst), new (static_cast(nullptr)) T(details::deref_noexcept(itemFirst)))) { + while (currentTailIndex != stopIndex) { + new ((*this->tailBlock)[currentTailIndex++]) T(*itemFirst++); + } + } + else { + MOODYCAMEL_TRY { + while (currentTailIndex != stopIndex) { + // Must use copy constructor even if move constructor is available + // because we may have to revert if there's an exception. + // Sorry about the horrible templated next line, but it was the only way + // to disable moving *at compile time*, which is important because a type + // may only define a (noexcept) move constructor, and so calls to the + // cctor will not compile, even if they are in an if branch that will never + // be executed + new ((*this->tailBlock)[currentTailIndex]) T(details::nomove_if(nullptr)) T(details::deref_noexcept(itemFirst)))>::eval(*itemFirst)); + ++currentTailIndex; + ++itemFirst; + } + } + MOODYCAMEL_CATCH (...) { + // Oh dear, an exception's been thrown -- destroy the elements that + // were enqueued so far and revert the entire bulk operation (we'll keep + // any allocated blocks in our linked list for later, though). + auto constructedStopIndex = currentTailIndex; + auto lastBlockEnqueued = this->tailBlock; + + pr_blockIndexFront = originalBlockIndexFront; + pr_blockIndexSlotsUsed = originalBlockIndexSlotsUsed; + this->tailBlock = startBlock == nullptr ? firstAllocatedBlock : startBlock; + + if (!details::is_trivially_destructible::value) { + auto block = startBlock; + if ((startTailIndex & static_cast(BLOCK_SIZE - 1)) == 0) { + block = firstAllocatedBlock; + } + currentTailIndex = startTailIndex; + while (true) { + stopIndex = (currentTailIndex & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + if (details::circular_less_than(constructedStopIndex, stopIndex)) { + stopIndex = constructedStopIndex; + } + while (currentTailIndex != stopIndex) { + (*block)[currentTailIndex++]->~T(); + } + if (block == lastBlockEnqueued) { + break; + } + block = block->next; + } + } + MOODYCAMEL_RETHROW; + } + } + + if (this->tailBlock == endBlock) { + assert(currentTailIndex == newTailIndex); + break; + } + this->tailBlock = this->tailBlock->next; + } + + MOODYCAMEL_CONSTEXPR_IF (!MOODYCAMEL_NOEXCEPT_CTOR(T, decltype(*itemFirst), new (static_cast(nullptr)) T(details::deref_noexcept(itemFirst)))) { + if (firstAllocatedBlock != nullptr) + blockIndex.load(std::memory_order_relaxed)->front.store((pr_blockIndexFront - 1) & (pr_blockIndexSize - 1), std::memory_order_release); + } + + this->tailIndex.store(newTailIndex, std::memory_order_release); + return true; + } + + template + size_t dequeue_bulk(It& itemFirst, size_t max) + { + auto tail = this->tailIndex.load(std::memory_order_relaxed); + auto overcommit = this->dequeueOvercommit.load(std::memory_order_relaxed); + auto desiredCount = static_cast(tail - (this->dequeueOptimisticCount.load(std::memory_order_relaxed) - overcommit)); + if (details::circular_less_than(0, desiredCount)) { + desiredCount = desiredCount < max ? desiredCount : max; + std::atomic_thread_fence(std::memory_order_acquire); + + auto myDequeueCount = this->dequeueOptimisticCount.fetch_add(desiredCount, std::memory_order_relaxed); + + tail = this->tailIndex.load(std::memory_order_acquire); + auto actualCount = static_cast(tail - (myDequeueCount - overcommit)); + if (details::circular_less_than(0, actualCount)) { + actualCount = desiredCount < actualCount ? desiredCount : actualCount; + if (actualCount < desiredCount) { + this->dequeueOvercommit.fetch_add(desiredCount - actualCount, std::memory_order_release); + } + + // Get the first index. Note that since there's guaranteed to be at least actualCount elements, this + // will never exceed tail. + auto firstIndex = this->headIndex.fetch_add(actualCount, std::memory_order_acq_rel); + + // Determine which block the first element is in + auto localBlockIndex = blockIndex.load(std::memory_order_acquire); + auto localBlockIndexHead = localBlockIndex->front.load(std::memory_order_acquire); + + auto headBase = localBlockIndex->entries[localBlockIndexHead].base; + auto firstBlockBaseIndex = firstIndex & ~static_cast(BLOCK_SIZE - 1); + auto offset = static_cast(static_cast::type>(firstBlockBaseIndex - headBase) / static_cast::type>(BLOCK_SIZE)); + auto indexIndex = (localBlockIndexHead + offset) & (localBlockIndex->size - 1); + + // Iterate the blocks and dequeue + auto index = firstIndex; + do { + auto firstIndexInBlock = index; + index_t endIndex = (index & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + endIndex = details::circular_less_than(firstIndex + static_cast(actualCount), endIndex) ? firstIndex + static_cast(actualCount) : endIndex; + auto block = localBlockIndex->entries[indexIndex].block; + if (MOODYCAMEL_NOEXCEPT_ASSIGN(T, T&&, details::deref_noexcept(itemFirst) = std::move((*(*block)[index])))) { + while (index != endIndex) { + auto& el = *((*block)[index]); + *itemFirst++ = std::move(el); + el.~T(); + ++index; + } + } + else { + MOODYCAMEL_TRY { + while (index != endIndex) { + auto& el = *((*block)[index]); + *itemFirst = std::move(el); + ++itemFirst; + el.~T(); + ++index; + } + } + MOODYCAMEL_CATCH (...) { + // It's too late to revert the dequeue, but we can make sure that all + // the dequeued objects are properly destroyed and the block index + // (and empty count) are properly updated before we propagate the exception + do { + block = localBlockIndex->entries[indexIndex].block; + while (index != endIndex) { + (*block)[index++]->~T(); + } + block->ConcurrentQueue::Block::template set_many_empty(firstIndexInBlock, static_cast(endIndex - firstIndexInBlock)); + indexIndex = (indexIndex + 1) & (localBlockIndex->size - 1); + + firstIndexInBlock = index; + endIndex = (index & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + endIndex = details::circular_less_than(firstIndex + static_cast(actualCount), endIndex) ? firstIndex + static_cast(actualCount) : endIndex; + } while (index != firstIndex + actualCount); + + MOODYCAMEL_RETHROW; + } + } + block->ConcurrentQueue::Block::template set_many_empty(firstIndexInBlock, static_cast(endIndex - firstIndexInBlock)); + indexIndex = (indexIndex + 1) & (localBlockIndex->size - 1); + } while (index != firstIndex + actualCount); + + return actualCount; + } + else { + // Wasn't anything to dequeue after all; make the effective dequeue count eventually consistent + this->dequeueOvercommit.fetch_add(desiredCount, std::memory_order_release); + } + } + + return 0; + } + + private: + struct BlockIndexEntry + { + index_t base; + Block* block; + }; + + struct BlockIndexHeader + { + size_t size; + std::atomic front; // Current slot (not next, like pr_blockIndexFront) + BlockIndexEntry* entries; + void* prev; + }; + + + bool new_block_index(size_t numberOfFilledSlotsToExpose) + { + auto prevBlockSizeMask = pr_blockIndexSize - 1; + + // Create the new block + pr_blockIndexSize <<= 1; + auto newRawPtr = static_cast((Traits::malloc)(sizeof(BlockIndexHeader) + std::alignment_of::value - 1 + sizeof(BlockIndexEntry) * pr_blockIndexSize)); + if (newRawPtr == nullptr) { + pr_blockIndexSize >>= 1; // Reset to allow graceful retry + return false; + } + + auto newBlockIndexEntries = reinterpret_cast(details::align_for(newRawPtr + sizeof(BlockIndexHeader))); + + // Copy in all the old indices, if any + size_t j = 0; + if (pr_blockIndexSlotsUsed != 0) { + auto i = (pr_blockIndexFront - pr_blockIndexSlotsUsed) & prevBlockSizeMask; + do { + newBlockIndexEntries[j++] = pr_blockIndexEntries[i]; + i = (i + 1) & prevBlockSizeMask; + } while (i != pr_blockIndexFront); + } + + // Update everything + auto header = new (newRawPtr) BlockIndexHeader; + header->size = pr_blockIndexSize; + header->front.store(numberOfFilledSlotsToExpose - 1, std::memory_order_relaxed); + header->entries = newBlockIndexEntries; + header->prev = pr_blockIndexRaw; // we link the new block to the old one so we can free it later + + pr_blockIndexFront = j; + pr_blockIndexEntries = newBlockIndexEntries; + pr_blockIndexRaw = newRawPtr; + blockIndex.store(header, std::memory_order_release); + + return true; + } + + private: + std::atomic blockIndex; + + // To be used by producer only -- consumer must use the ones in referenced by blockIndex + size_t pr_blockIndexSlotsUsed; + size_t pr_blockIndexSize; + size_t pr_blockIndexFront; // Next slot (not current) + BlockIndexEntry* pr_blockIndexEntries; + void* pr_blockIndexRaw; + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + public: + ExplicitProducer* nextExplicitProducer; + private: +#endif + +#ifdef MCDBGQ_TRACKMEM + friend struct MemStats; +#endif + }; + + + ////////////////////////////////// + // Implicit queue + ////////////////////////////////// + + struct ImplicitProducer : public ProducerBase + { + ImplicitProducer(ConcurrentQueue* parent_) : + ProducerBase(parent_, false), + nextBlockIndexCapacity(IMPLICIT_INITIAL_INDEX_SIZE), + blockIndex(nullptr) + { + new_block_index(); + } + + ~ImplicitProducer() + { + // Note that since we're in the destructor we can assume that all enqueue/dequeue operations + // completed already; this means that all undequeued elements are placed contiguously across + // contiguous blocks, and that only the first and last remaining blocks can be only partially + // empty (all other remaining blocks must be completely full). + +#ifdef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED + // Unregister ourselves for thread termination notification + if (!this->inactive.load(std::memory_order_relaxed)) { + details::ThreadExitNotifier::unsubscribe(&threadExitListener); + } +#endif + + // Destroy all remaining elements! + auto tail = this->tailIndex.load(std::memory_order_relaxed); + auto index = this->headIndex.load(std::memory_order_relaxed); + Block* block = nullptr; + assert(index == tail || details::circular_less_than(index, tail)); + bool forceFreeLastBlock = index != tail; // If we enter the loop, then the last (tail) block will not be freed + while (index != tail) { + if ((index & static_cast(BLOCK_SIZE - 1)) == 0 || block == nullptr) { + if (block != nullptr) { + // Free the old block + this->parent->add_block_to_free_list(block); + } + + block = get_block_index_entry_for_index(index)->value.load(std::memory_order_relaxed); + } + + ((*block)[index])->~T(); + ++index; + } + // Even if the queue is empty, there's still one block that's not on the free list + // (unless the head index reached the end of it, in which case the tail will be poised + // to create a new block). + if (this->tailBlock != nullptr && (forceFreeLastBlock || (tail & static_cast(BLOCK_SIZE - 1)) != 0)) { + this->parent->add_block_to_free_list(this->tailBlock); + } + + // Destroy block index + auto localBlockIndex = blockIndex.load(std::memory_order_relaxed); + if (localBlockIndex != nullptr) { + for (size_t i = 0; i != localBlockIndex->capacity; ++i) { + localBlockIndex->index[i]->~BlockIndexEntry(); + } + do { + auto prev = localBlockIndex->prev; + localBlockIndex->~BlockIndexHeader(); + (Traits::free)(localBlockIndex); + localBlockIndex = prev; + } while (localBlockIndex != nullptr); + } + } + + template + inline bool enqueue(U&& element) + { + index_t currentTailIndex = this->tailIndex.load(std::memory_order_relaxed); + index_t newTailIndex = 1 + currentTailIndex; + if ((currentTailIndex & static_cast(BLOCK_SIZE - 1)) == 0) { + // We reached the end of a block, start a new one + auto head = this->headIndex.load(std::memory_order_relaxed); + assert(!details::circular_less_than(currentTailIndex, head)); + if (!details::circular_less_than(head, currentTailIndex + BLOCK_SIZE) || (MAX_SUBQUEUE_SIZE != details::const_numeric_max::value && (MAX_SUBQUEUE_SIZE == 0 || MAX_SUBQUEUE_SIZE - BLOCK_SIZE < currentTailIndex - head))) { + return false; + } +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + debug::DebugLock lock(mutex); +#endif + // Find out where we'll be inserting this block in the block index + BlockIndexEntry* idxEntry; + if (!insert_block_index_entry(idxEntry, currentTailIndex)) { + return false; + } + + // Get ahold of a new block + auto newBlock = this->parent->ConcurrentQueue::template requisition_block(); + if (newBlock == nullptr) { + rewind_block_index_tail(); + idxEntry->value.store(nullptr, std::memory_order_relaxed); + return false; + } +#ifdef MCDBGQ_TRACKMEM + newBlock->owner = this; +#endif + newBlock->ConcurrentQueue::Block::template reset_empty(); + + MOODYCAMEL_CONSTEXPR_IF (!MOODYCAMEL_NOEXCEPT_CTOR(T, U, new (static_cast(nullptr)) T(std::forward(element)))) { + // May throw, try to insert now before we publish the fact that we have this new block + MOODYCAMEL_TRY { + new ((*newBlock)[currentTailIndex]) T(std::forward(element)); + } + MOODYCAMEL_CATCH (...) { + rewind_block_index_tail(); + idxEntry->value.store(nullptr, std::memory_order_relaxed); + this->parent->add_block_to_free_list(newBlock); + MOODYCAMEL_RETHROW; + } + } + + // Insert the new block into the index + idxEntry->value.store(newBlock, std::memory_order_relaxed); + + this->tailBlock = newBlock; + + MOODYCAMEL_CONSTEXPR_IF (!MOODYCAMEL_NOEXCEPT_CTOR(T, U, new (static_cast(nullptr)) T(std::forward(element)))) { + this->tailIndex.store(newTailIndex, std::memory_order_release); + return true; + } + } + + // Enqueue + new ((*this->tailBlock)[currentTailIndex]) T(std::forward(element)); + + this->tailIndex.store(newTailIndex, std::memory_order_release); + return true; + } + + template + bool dequeue(U& element) + { + // See ExplicitProducer::dequeue for rationale and explanation + index_t tail = this->tailIndex.load(std::memory_order_relaxed); + index_t overcommit = this->dequeueOvercommit.load(std::memory_order_relaxed); + if (details::circular_less_than(this->dequeueOptimisticCount.load(std::memory_order_relaxed) - overcommit, tail)) { + std::atomic_thread_fence(std::memory_order_acquire); + + index_t myDequeueCount = this->dequeueOptimisticCount.fetch_add(1, std::memory_order_relaxed); + tail = this->tailIndex.load(std::memory_order_acquire); + if ((details::likely)(details::circular_less_than(myDequeueCount - overcommit, tail))) { + index_t index = this->headIndex.fetch_add(1, std::memory_order_acq_rel); + + // Determine which block the element is in + auto entry = get_block_index_entry_for_index(index); + + // Dequeue + auto block = entry->value.load(std::memory_order_relaxed); + auto& el = *((*block)[index]); + + if (!MOODYCAMEL_NOEXCEPT_ASSIGN(T, T&&, element = std::move(el))) { +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + // Note: Acquiring the mutex with every dequeue instead of only when a block + // is released is very sub-optimal, but it is, after all, purely debug code. + debug::DebugLock lock(producer->mutex); +#endif + struct Guard { + Block* block; + index_t index; + BlockIndexEntry* entry; + ConcurrentQueue* parent; + + ~Guard() + { + (*block)[index]->~T(); + if (block->ConcurrentQueue::Block::template set_empty(index)) { + entry->value.store(nullptr, std::memory_order_relaxed); + parent->add_block_to_free_list(block); + } + } + } guard = { block, index, entry, this->parent }; + + element = std::move(el); // NOLINT + } + else { + element = std::move(el); // NOLINT + el.~T(); // NOLINT + + if (block->ConcurrentQueue::Block::template set_empty(index)) { + { +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + debug::DebugLock lock(mutex); +#endif + // Add the block back into the global free pool (and remove from block index) + entry->value.store(nullptr, std::memory_order_relaxed); + } + this->parent->add_block_to_free_list(block); // releases the above store + } + } + + return true; + } + else { + this->dequeueOvercommit.fetch_add(1, std::memory_order_release); + } + } + + return false; + } + +#ifdef _MSC_VER +#pragma warning(push) +#pragma warning(disable: 4706) // assignment within conditional expression +#endif + template + bool enqueue_bulk(It itemFirst, size_t count) + { + // First, we need to make sure we have enough room to enqueue all of the elements; + // this means pre-allocating blocks and putting them in the block index (but only if + // all the allocations succeeded). + + // Note that the tailBlock we start off with may not be owned by us any more; + // this happens if it was filled up exactly to the top (setting tailIndex to + // the first index of the next block which is not yet allocated), then dequeued + // completely (putting it on the free list) before we enqueue again. + + index_t startTailIndex = this->tailIndex.load(std::memory_order_relaxed); + auto startBlock = this->tailBlock; + Block* firstAllocatedBlock = nullptr; + auto endBlock = this->tailBlock; + + // Figure out how many blocks we'll need to allocate, and do so + size_t blockBaseDiff = ((startTailIndex + count - 1) & ~static_cast(BLOCK_SIZE - 1)) - ((startTailIndex - 1) & ~static_cast(BLOCK_SIZE - 1)); + index_t currentTailIndex = (startTailIndex - 1) & ~static_cast(BLOCK_SIZE - 1); + if (blockBaseDiff > 0) { +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + debug::DebugLock lock(mutex); +#endif + do { + blockBaseDiff -= static_cast(BLOCK_SIZE); + currentTailIndex += static_cast(BLOCK_SIZE); + + // Find out where we'll be inserting this block in the block index + BlockIndexEntry* idxEntry = nullptr; // initialization here unnecessary but compiler can't always tell + Block* newBlock; + bool indexInserted = false; + auto head = this->headIndex.load(std::memory_order_relaxed); + assert(!details::circular_less_than(currentTailIndex, head)); + bool full = !details::circular_less_than(head, currentTailIndex + BLOCK_SIZE) || (MAX_SUBQUEUE_SIZE != details::const_numeric_max::value && (MAX_SUBQUEUE_SIZE == 0 || MAX_SUBQUEUE_SIZE - BLOCK_SIZE < currentTailIndex - head)); + + if (full || !(indexInserted = insert_block_index_entry(idxEntry, currentTailIndex)) || (newBlock = this->parent->ConcurrentQueue::template requisition_block()) == nullptr) { + // Index allocation or block allocation failed; revert any other allocations + // and index insertions done so far for this operation + if (indexInserted) { + rewind_block_index_tail(); + idxEntry->value.store(nullptr, std::memory_order_relaxed); + } + currentTailIndex = (startTailIndex - 1) & ~static_cast(BLOCK_SIZE - 1); + for (auto block = firstAllocatedBlock; block != nullptr; block = block->next) { + currentTailIndex += static_cast(BLOCK_SIZE); + idxEntry = get_block_index_entry_for_index(currentTailIndex); + idxEntry->value.store(nullptr, std::memory_order_relaxed); + rewind_block_index_tail(); + } + this->parent->add_blocks_to_free_list(firstAllocatedBlock); + this->tailBlock = startBlock; + + return false; + } + +#ifdef MCDBGQ_TRACKMEM + newBlock->owner = this; +#endif + newBlock->ConcurrentQueue::Block::template reset_empty(); + newBlock->next = nullptr; + + // Insert the new block into the index + idxEntry->value.store(newBlock, std::memory_order_relaxed); + + // Store the chain of blocks so that we can undo if later allocations fail, + // and so that we can find the blocks when we do the actual enqueueing + if ((startTailIndex & static_cast(BLOCK_SIZE - 1)) != 0 || firstAllocatedBlock != nullptr) { + assert(this->tailBlock != nullptr); + this->tailBlock->next = newBlock; + } + this->tailBlock = newBlock; + endBlock = newBlock; + firstAllocatedBlock = firstAllocatedBlock == nullptr ? newBlock : firstAllocatedBlock; + } while (blockBaseDiff > 0); + } + + // Enqueue, one block at a time + index_t newTailIndex = startTailIndex + static_cast(count); + currentTailIndex = startTailIndex; + this->tailBlock = startBlock; + assert((startTailIndex & static_cast(BLOCK_SIZE - 1)) != 0 || firstAllocatedBlock != nullptr || count == 0); + if ((startTailIndex & static_cast(BLOCK_SIZE - 1)) == 0 && firstAllocatedBlock != nullptr) { + this->tailBlock = firstAllocatedBlock; + } + while (true) { + index_t stopIndex = (currentTailIndex & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + if (details::circular_less_than(newTailIndex, stopIndex)) { + stopIndex = newTailIndex; + } + MOODYCAMEL_CONSTEXPR_IF (MOODYCAMEL_NOEXCEPT_CTOR(T, decltype(*itemFirst), new (static_cast(nullptr)) T(details::deref_noexcept(itemFirst)))) { + while (currentTailIndex != stopIndex) { + new ((*this->tailBlock)[currentTailIndex++]) T(*itemFirst++); + } + } + else { + MOODYCAMEL_TRY { + while (currentTailIndex != stopIndex) { + new ((*this->tailBlock)[currentTailIndex]) T(details::nomove_if(nullptr)) T(details::deref_noexcept(itemFirst)))>::eval(*itemFirst)); + ++currentTailIndex; + ++itemFirst; + } + } + MOODYCAMEL_CATCH (...) { + auto constructedStopIndex = currentTailIndex; + auto lastBlockEnqueued = this->tailBlock; + + if (!details::is_trivially_destructible::value) { + auto block = startBlock; + if ((startTailIndex & static_cast(BLOCK_SIZE - 1)) == 0) { + block = firstAllocatedBlock; + } + currentTailIndex = startTailIndex; + while (true) { + stopIndex = (currentTailIndex & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + if (details::circular_less_than(constructedStopIndex, stopIndex)) { + stopIndex = constructedStopIndex; + } + while (currentTailIndex != stopIndex) { + (*block)[currentTailIndex++]->~T(); + } + if (block == lastBlockEnqueued) { + break; + } + block = block->next; + } + } + + currentTailIndex = (startTailIndex - 1) & ~static_cast(BLOCK_SIZE - 1); + for (auto block = firstAllocatedBlock; block != nullptr; block = block->next) { + currentTailIndex += static_cast(BLOCK_SIZE); + auto idxEntry = get_block_index_entry_for_index(currentTailIndex); + idxEntry->value.store(nullptr, std::memory_order_relaxed); + rewind_block_index_tail(); + } + this->parent->add_blocks_to_free_list(firstAllocatedBlock); + this->tailBlock = startBlock; + MOODYCAMEL_RETHROW; + } + } + + if (this->tailBlock == endBlock) { + assert(currentTailIndex == newTailIndex); + break; + } + this->tailBlock = this->tailBlock->next; + } + this->tailIndex.store(newTailIndex, std::memory_order_release); + return true; + } +#ifdef _MSC_VER +#pragma warning(pop) +#endif + + template + size_t dequeue_bulk(It& itemFirst, size_t max) + { + auto tail = this->tailIndex.load(std::memory_order_relaxed); + auto overcommit = this->dequeueOvercommit.load(std::memory_order_relaxed); + auto desiredCount = static_cast(tail - (this->dequeueOptimisticCount.load(std::memory_order_relaxed) - overcommit)); + if (details::circular_less_than(0, desiredCount)) { + desiredCount = desiredCount < max ? desiredCount : max; + std::atomic_thread_fence(std::memory_order_acquire); + + auto myDequeueCount = this->dequeueOptimisticCount.fetch_add(desiredCount, std::memory_order_relaxed); + + tail = this->tailIndex.load(std::memory_order_acquire); + auto actualCount = static_cast(tail - (myDequeueCount - overcommit)); + if (details::circular_less_than(0, actualCount)) { + actualCount = desiredCount < actualCount ? desiredCount : actualCount; + if (actualCount < desiredCount) { + this->dequeueOvercommit.fetch_add(desiredCount - actualCount, std::memory_order_release); + } + + // Get the first index. Note that since there's guaranteed to be at least actualCount elements, this + // will never exceed tail. + auto firstIndex = this->headIndex.fetch_add(actualCount, std::memory_order_acq_rel); + + // Iterate the blocks and dequeue + auto index = firstIndex; + BlockIndexHeader* localBlockIndex; + auto indexIndex = get_block_index_index_for_index(index, localBlockIndex); + do { + auto blockStartIndex = index; + index_t endIndex = (index & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + endIndex = details::circular_less_than(firstIndex + static_cast(actualCount), endIndex) ? firstIndex + static_cast(actualCount) : endIndex; + + auto entry = localBlockIndex->index[indexIndex]; + auto block = entry->value.load(std::memory_order_relaxed); + if (MOODYCAMEL_NOEXCEPT_ASSIGN(T, T&&, details::deref_noexcept(itemFirst) = std::move((*(*block)[index])))) { + while (index != endIndex) { + auto& el = *((*block)[index]); + *itemFirst++ = std::move(el); + el.~T(); + ++index; + } + } + else { + MOODYCAMEL_TRY { + while (index != endIndex) { + auto& el = *((*block)[index]); + *itemFirst = std::move(el); + ++itemFirst; + el.~T(); + ++index; + } + } + MOODYCAMEL_CATCH (...) { + do { + entry = localBlockIndex->index[indexIndex]; + block = entry->value.load(std::memory_order_relaxed); + while (index != endIndex) { + (*block)[index++]->~T(); + } + + if (block->ConcurrentQueue::Block::template set_many_empty(blockStartIndex, static_cast(endIndex - blockStartIndex))) { +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + debug::DebugLock lock(mutex); +#endif + entry->value.store(nullptr, std::memory_order_relaxed); + this->parent->add_block_to_free_list(block); + } + indexIndex = (indexIndex + 1) & (localBlockIndex->capacity - 1); + + blockStartIndex = index; + endIndex = (index & ~static_cast(BLOCK_SIZE - 1)) + static_cast(BLOCK_SIZE); + endIndex = details::circular_less_than(firstIndex + static_cast(actualCount), endIndex) ? firstIndex + static_cast(actualCount) : endIndex; + } while (index != firstIndex + actualCount); + + MOODYCAMEL_RETHROW; + } + } + if (block->ConcurrentQueue::Block::template set_many_empty(blockStartIndex, static_cast(endIndex - blockStartIndex))) { + { +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + debug::DebugLock lock(mutex); +#endif + // Note that the set_many_empty above did a release, meaning that anybody who acquires the block + // we're about to free can use it safely since our writes (and reads!) will have happened-before then. + entry->value.store(nullptr, std::memory_order_relaxed); + } + this->parent->add_block_to_free_list(block); // releases the above store + } + indexIndex = (indexIndex + 1) & (localBlockIndex->capacity - 1); + } while (index != firstIndex + actualCount); + + return actualCount; + } + else { + this->dequeueOvercommit.fetch_add(desiredCount, std::memory_order_release); + } + } + + return 0; + } + + private: + // The block size must be > 1, so any number with the low bit set is an invalid block base index + static const index_t INVALID_BLOCK_BASE = 1; + + struct BlockIndexEntry + { + std::atomic key; + std::atomic value; + }; + + struct BlockIndexHeader + { + size_t capacity; + std::atomic tail; + BlockIndexEntry* entries; + BlockIndexEntry** index; + BlockIndexHeader* prev; + }; + + template + inline bool insert_block_index_entry(BlockIndexEntry*& idxEntry, index_t blockStartIndex) + { + auto localBlockIndex = blockIndex.load(std::memory_order_relaxed); // We're the only writer thread, relaxed is OK + if (localBlockIndex == nullptr) { + return false; // this can happen if new_block_index failed in the constructor + } + size_t newTail = (localBlockIndex->tail.load(std::memory_order_relaxed) + 1) & (localBlockIndex->capacity - 1); + idxEntry = localBlockIndex->index[newTail]; + if (idxEntry->key.load(std::memory_order_relaxed) == INVALID_BLOCK_BASE || + idxEntry->value.load(std::memory_order_relaxed) == nullptr) { + + idxEntry->key.store(blockStartIndex, std::memory_order_relaxed); + localBlockIndex->tail.store(newTail, std::memory_order_release); + return true; + } + + // No room in the old block index, try to allocate another one! + MOODYCAMEL_CONSTEXPR_IF (allocMode == CannotAlloc) { + return false; + } + else if (!new_block_index()) { + return false; + } + else { + localBlockIndex = blockIndex.load(std::memory_order_relaxed); + newTail = (localBlockIndex->tail.load(std::memory_order_relaxed) + 1) & (localBlockIndex->capacity - 1); + idxEntry = localBlockIndex->index[newTail]; + assert(idxEntry->key.load(std::memory_order_relaxed) == INVALID_BLOCK_BASE); + idxEntry->key.store(blockStartIndex, std::memory_order_relaxed); + localBlockIndex->tail.store(newTail, std::memory_order_release); + return true; + } + } + + inline void rewind_block_index_tail() + { + auto localBlockIndex = blockIndex.load(std::memory_order_relaxed); + localBlockIndex->tail.store((localBlockIndex->tail.load(std::memory_order_relaxed) - 1) & (localBlockIndex->capacity - 1), std::memory_order_relaxed); + } + + inline BlockIndexEntry* get_block_index_entry_for_index(index_t index) const + { + BlockIndexHeader* localBlockIndex; + auto idx = get_block_index_index_for_index(index, localBlockIndex); + return localBlockIndex->index[idx]; + } + + inline size_t get_block_index_index_for_index(index_t index, BlockIndexHeader*& localBlockIndex) const + { +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + debug::DebugLock lock(mutex); +#endif + index &= ~static_cast(BLOCK_SIZE - 1); + localBlockIndex = blockIndex.load(std::memory_order_acquire); + auto tail = localBlockIndex->tail.load(std::memory_order_acquire); + auto tailBase = localBlockIndex->index[tail]->key.load(std::memory_order_relaxed); + assert(tailBase != INVALID_BLOCK_BASE); + // Note: Must use division instead of shift because the index may wrap around, causing a negative + // offset, whose negativity we want to preserve + auto offset = static_cast(static_cast::type>(index - tailBase) / static_cast::type>(BLOCK_SIZE)); + size_t idx = (tail + offset) & (localBlockIndex->capacity - 1); + assert(localBlockIndex->index[idx]->key.load(std::memory_order_relaxed) == index && localBlockIndex->index[idx]->value.load(std::memory_order_relaxed) != nullptr); + return idx; + } + + bool new_block_index() + { + auto prev = blockIndex.load(std::memory_order_relaxed); + size_t prevCapacity = prev == nullptr ? 0 : prev->capacity; + auto entryCount = prev == nullptr ? nextBlockIndexCapacity : prevCapacity; + auto raw = static_cast((Traits::malloc)( + sizeof(BlockIndexHeader) + + std::alignment_of::value - 1 + sizeof(BlockIndexEntry) * entryCount + + std::alignment_of::value - 1 + sizeof(BlockIndexEntry*) * nextBlockIndexCapacity)); + if (raw == nullptr) { + return false; + } + + auto header = new (raw) BlockIndexHeader; + auto entries = reinterpret_cast(details::align_for(raw + sizeof(BlockIndexHeader))); + auto index = reinterpret_cast(details::align_for(reinterpret_cast(entries) + sizeof(BlockIndexEntry) * entryCount)); + if (prev != nullptr) { + auto prevTail = prev->tail.load(std::memory_order_relaxed); + auto prevPos = prevTail; + size_t i = 0; + do { + prevPos = (prevPos + 1) & (prev->capacity - 1); + index[i++] = prev->index[prevPos]; + } while (prevPos != prevTail); + assert(i == prevCapacity); + } + for (size_t i = 0; i != entryCount; ++i) { + new (entries + i) BlockIndexEntry; + entries[i].key.store(INVALID_BLOCK_BASE, std::memory_order_relaxed); + index[prevCapacity + i] = entries + i; + } + header->prev = prev; + header->entries = entries; + header->index = index; + header->capacity = nextBlockIndexCapacity; + header->tail.store((prevCapacity - 1) & (nextBlockIndexCapacity - 1), std::memory_order_relaxed); + + blockIndex.store(header, std::memory_order_release); + + nextBlockIndexCapacity <<= 1; + + return true; + } + + private: + size_t nextBlockIndexCapacity; + std::atomic blockIndex; + +#ifdef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED + public: + details::ThreadExitListener threadExitListener; + private: +#endif + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + public: + ImplicitProducer* nextImplicitProducer; + private: +#endif + +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODBLOCKINDEX + mutable debug::DebugMutex mutex; +#endif +#ifdef MCDBGQ_TRACKMEM + friend struct MemStats; +#endif + }; + + + ////////////////////////////////// + // Block pool manipulation + ////////////////////////////////// + + void populate_initial_block_list(size_t blockCount) + { + initialBlockPoolSize = blockCount; + if (initialBlockPoolSize == 0) { + initialBlockPool = nullptr; + return; + } + + initialBlockPool = create_array(blockCount); + if (initialBlockPool == nullptr) { + initialBlockPoolSize = 0; + } + for (size_t i = 0; i < initialBlockPoolSize; ++i) { + initialBlockPool[i].dynamicallyAllocated = false; + } + } + + inline Block* try_get_block_from_initial_pool() + { + if (initialBlockPoolIndex.load(std::memory_order_relaxed) >= initialBlockPoolSize) { + return nullptr; + } + + auto index = initialBlockPoolIndex.fetch_add(1, std::memory_order_relaxed); + + return index < initialBlockPoolSize ? (initialBlockPool + index) : nullptr; + } + + inline void add_block_to_free_list(Block* block) + { +#ifdef MCDBGQ_TRACKMEM + block->owner = nullptr; +#endif + if (!Traits::RECYCLE_ALLOCATED_BLOCKS && block->dynamicallyAllocated) { + destroy(block); + } + else { + freeList.add(block); + } + } + + inline void add_blocks_to_free_list(Block* block) + { + while (block != nullptr) { + auto next = block->next; + add_block_to_free_list(block); + block = next; + } + } + + inline Block* try_get_block_from_free_list() + { + return freeList.try_get(); + } + + // Gets a free block from one of the memory pools, or allocates a new one (if applicable) + template + Block* requisition_block() + { + auto block = try_get_block_from_initial_pool(); + if (block != nullptr) { + return block; + } + + block = try_get_block_from_free_list(); + if (block != nullptr) { + return block; + } + + MOODYCAMEL_CONSTEXPR_IF (canAlloc == CanAlloc) { + return create(); + } + else { + return nullptr; + } + } + + +#ifdef MCDBGQ_TRACKMEM + public: + struct MemStats { + size_t allocatedBlocks; + size_t usedBlocks; + size_t freeBlocks; + size_t ownedBlocksExplicit; + size_t ownedBlocksImplicit; + size_t implicitProducers; + size_t explicitProducers; + size_t elementsEnqueued; + size_t blockClassBytes; + size_t queueClassBytes; + size_t implicitBlockIndexBytes; + size_t explicitBlockIndexBytes; + + friend class ConcurrentQueue; + + private: + static MemStats getFor(ConcurrentQueue* q) + { + MemStats stats = { 0 }; + + stats.elementsEnqueued = q->size_approx(); + + auto block = q->freeList.head_unsafe(); + while (block != nullptr) { + ++stats.allocatedBlocks; + ++stats.freeBlocks; + block = block->freeListNext.load(std::memory_order_relaxed); + } + + for (auto ptr = q->producerListTail.load(std::memory_order_acquire); ptr != nullptr; ptr = ptr->next_prod()) { + bool implicit = dynamic_cast(ptr) != nullptr; + stats.implicitProducers += implicit ? 1 : 0; + stats.explicitProducers += implicit ? 0 : 1; + + if (implicit) { + auto prod = static_cast(ptr); + stats.queueClassBytes += sizeof(ImplicitProducer); + auto head = prod->headIndex.load(std::memory_order_relaxed); + auto tail = prod->tailIndex.load(std::memory_order_relaxed); + auto hash = prod->blockIndex.load(std::memory_order_relaxed); + if (hash != nullptr) { + for (size_t i = 0; i != hash->capacity; ++i) { + if (hash->index[i]->key.load(std::memory_order_relaxed) != ImplicitProducer::INVALID_BLOCK_BASE && hash->index[i]->value.load(std::memory_order_relaxed) != nullptr) { + ++stats.allocatedBlocks; + ++stats.ownedBlocksImplicit; + } + } + stats.implicitBlockIndexBytes += hash->capacity * sizeof(typename ImplicitProducer::BlockIndexEntry); + for (; hash != nullptr; hash = hash->prev) { + stats.implicitBlockIndexBytes += sizeof(typename ImplicitProducer::BlockIndexHeader) + hash->capacity * sizeof(typename ImplicitProducer::BlockIndexEntry*); + } + } + for (; details::circular_less_than(head, tail); head += BLOCK_SIZE) { + //auto block = prod->get_block_index_entry_for_index(head); + ++stats.usedBlocks; + } + } + else { + auto prod = static_cast(ptr); + stats.queueClassBytes += sizeof(ExplicitProducer); + auto tailBlock = prod->tailBlock; + bool wasNonEmpty = false; + if (tailBlock != nullptr) { + auto block = tailBlock; + do { + ++stats.allocatedBlocks; + if (!block->ConcurrentQueue::Block::template is_empty() || wasNonEmpty) { + ++stats.usedBlocks; + wasNonEmpty = wasNonEmpty || block != tailBlock; + } + ++stats.ownedBlocksExplicit; + block = block->next; + } while (block != tailBlock); + } + auto index = prod->blockIndex.load(std::memory_order_relaxed); + while (index != nullptr) { + stats.explicitBlockIndexBytes += sizeof(typename ExplicitProducer::BlockIndexHeader) + index->size * sizeof(typename ExplicitProducer::BlockIndexEntry); + index = static_cast(index->prev); + } + } + } + + auto freeOnInitialPool = q->initialBlockPoolIndex.load(std::memory_order_relaxed) >= q->initialBlockPoolSize ? 0 : q->initialBlockPoolSize - q->initialBlockPoolIndex.load(std::memory_order_relaxed); + stats.allocatedBlocks += freeOnInitialPool; + stats.freeBlocks += freeOnInitialPool; + + stats.blockClassBytes = sizeof(Block) * stats.allocatedBlocks; + stats.queueClassBytes += sizeof(ConcurrentQueue); + + return stats; + } + }; + + // For debugging only. Not thread-safe. + MemStats getMemStats() + { + return MemStats::getFor(this); + } + private: + friend struct MemStats; +#endif + + + ////////////////////////////////// + // Producer list manipulation + ////////////////////////////////// + + ProducerBase* recycle_or_create_producer(bool isExplicit) + { +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODHASH + debug::DebugLock lock(implicitProdMutex); +#endif + // Try to re-use one first + for (auto ptr = producerListTail.load(std::memory_order_acquire); ptr != nullptr; ptr = ptr->next_prod()) { + if (ptr->inactive.load(std::memory_order_relaxed) && ptr->isExplicit == isExplicit) { + bool expected = true; + if (ptr->inactive.compare_exchange_strong(expected, /* desired */ false, std::memory_order_acquire, std::memory_order_relaxed)) { + // We caught one! It's been marked as activated, the caller can have it + return ptr; + } + } + } + + return add_producer(isExplicit ? static_cast(create(this)) : create(this)); + } + + ProducerBase* add_producer(ProducerBase* producer) + { + // Handle failed memory allocation + if (producer == nullptr) { + return nullptr; + } + + producerCount.fetch_add(1, std::memory_order_relaxed); + + // Add it to the lock-free list + auto prevTail = producerListTail.load(std::memory_order_relaxed); + do { + producer->next = prevTail; + } while (!producerListTail.compare_exchange_weak(prevTail, producer, std::memory_order_release, std::memory_order_relaxed)); + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + if (producer->isExplicit) { + auto prevTailExplicit = explicitProducers.load(std::memory_order_relaxed); + do { + static_cast(producer)->nextExplicitProducer = prevTailExplicit; + } while (!explicitProducers.compare_exchange_weak(prevTailExplicit, static_cast(producer), std::memory_order_release, std::memory_order_relaxed)); + } + else { + auto prevTailImplicit = implicitProducers.load(std::memory_order_relaxed); + do { + static_cast(producer)->nextImplicitProducer = prevTailImplicit; + } while (!implicitProducers.compare_exchange_weak(prevTailImplicit, static_cast(producer), std::memory_order_release, std::memory_order_relaxed)); + } +#endif + + return producer; + } + + void reown_producers() + { + // After another instance is moved-into/swapped-with this one, all the + // producers we stole still think their parents are the other queue. + // So fix them up! + for (auto ptr = producerListTail.load(std::memory_order_relaxed); ptr != nullptr; ptr = ptr->next_prod()) { + ptr->parent = this; + } + } + + + ////////////////////////////////// + // Implicit producer hash + ////////////////////////////////// + + struct ImplicitProducerKVP + { + std::atomic key; + ImplicitProducer* value; // No need for atomicity since it's only read by the thread that sets it in the first place + + ImplicitProducerKVP() : value(nullptr) { } + + ImplicitProducerKVP(ImplicitProducerKVP&& other) MOODYCAMEL_NOEXCEPT + { + key.store(other.key.load(std::memory_order_relaxed), std::memory_order_relaxed); + value = other.value; + } + + inline ImplicitProducerKVP& operator=(ImplicitProducerKVP&& other) MOODYCAMEL_NOEXCEPT + { + swap(other); + return *this; + } + + inline void swap(ImplicitProducerKVP& other) MOODYCAMEL_NOEXCEPT + { + if (this != &other) { + details::swap_relaxed(key, other.key); + std::swap(value, other.value); + } + } + }; + + template + friend void moodycamel::swap(typename ConcurrentQueue::ImplicitProducerKVP&, typename ConcurrentQueue::ImplicitProducerKVP&) MOODYCAMEL_NOEXCEPT; + + struct ImplicitProducerHash + { + size_t capacity; + ImplicitProducerKVP* entries; + ImplicitProducerHash* prev; + }; + + inline void populate_initial_implicit_producer_hash() + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) { + return; + } + else { + implicitProducerHashCount.store(0, std::memory_order_relaxed); + auto hash = &initialImplicitProducerHash; + hash->capacity = INITIAL_IMPLICIT_PRODUCER_HASH_SIZE; + hash->entries = &initialImplicitProducerHashEntries[0]; + for (size_t i = 0; i != INITIAL_IMPLICIT_PRODUCER_HASH_SIZE; ++i) { + initialImplicitProducerHashEntries[i].key.store(details::invalid_thread_id, std::memory_order_relaxed); + } + hash->prev = nullptr; + implicitProducerHash.store(hash, std::memory_order_relaxed); + } + } + + void swap_implicit_producer_hashes(ConcurrentQueue& other) + { + MOODYCAMEL_CONSTEXPR_IF (INITIAL_IMPLICIT_PRODUCER_HASH_SIZE == 0) { + return; + } + else { + // Swap (assumes our implicit producer hash is initialized) + initialImplicitProducerHashEntries.swap(other.initialImplicitProducerHashEntries); + initialImplicitProducerHash.entries = &initialImplicitProducerHashEntries[0]; + other.initialImplicitProducerHash.entries = &other.initialImplicitProducerHashEntries[0]; + + details::swap_relaxed(implicitProducerHashCount, other.implicitProducerHashCount); + + details::swap_relaxed(implicitProducerHash, other.implicitProducerHash); + if (implicitProducerHash.load(std::memory_order_relaxed) == &other.initialImplicitProducerHash) { + implicitProducerHash.store(&initialImplicitProducerHash, std::memory_order_relaxed); + } + else { + ImplicitProducerHash* hash; + for (hash = implicitProducerHash.load(std::memory_order_relaxed); hash->prev != &other.initialImplicitProducerHash; hash = hash->prev) { + continue; + } + hash->prev = &initialImplicitProducerHash; + } + if (other.implicitProducerHash.load(std::memory_order_relaxed) == &initialImplicitProducerHash) { + other.implicitProducerHash.store(&other.initialImplicitProducerHash, std::memory_order_relaxed); + } + else { + ImplicitProducerHash* hash; + for (hash = other.implicitProducerHash.load(std::memory_order_relaxed); hash->prev != &initialImplicitProducerHash; hash = hash->prev) { + continue; + } + hash->prev = &other.initialImplicitProducerHash; + } + } + } + + // Only fails (returns nullptr) if memory allocation fails + ImplicitProducer* get_or_add_implicit_producer() + { + // Note that since the data is essentially thread-local (key is thread ID), + // there's a reduced need for fences (memory ordering is already consistent + // for any individual thread), except for the current table itself. + + // Start by looking for the thread ID in the current and all previous hash tables. + // If it's not found, it must not be in there yet, since this same thread would + // have added it previously to one of the tables that we traversed. + + // Code and algorithm adapted from http://preshing.com/20130605/the-worlds-simplest-lock-free-hash-table + +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODHASH + debug::DebugLock lock(implicitProdMutex); +#endif + + auto id = details::thread_id(); + auto hashedId = details::hash_thread_id(id); + + auto mainHash = implicitProducerHash.load(std::memory_order_acquire); + assert(mainHash != nullptr); // silence clang-tidy and MSVC warnings (hash cannot be null) + for (auto hash = mainHash; hash != nullptr; hash = hash->prev) { + // Look for the id in this hash + auto index = hashedId; + while (true) { // Not an infinite loop because at least one slot is free in the hash table + index &= hash->capacity - 1u; + + auto probedKey = hash->entries[index].key.load(std::memory_order_relaxed); + if (probedKey == id) { + // Found it! If we had to search several hashes deep, though, we should lazily add it + // to the current main hash table to avoid the extended search next time. + // Note there's guaranteed to be room in the current hash table since every subsequent + // table implicitly reserves space for all previous tables (there's only one + // implicitProducerHashCount). + auto value = hash->entries[index].value; + if (hash != mainHash) { + index = hashedId; + while (true) { + index &= mainHash->capacity - 1u; + auto empty = details::invalid_thread_id; +#ifdef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED + auto reusable = details::invalid_thread_id2; + if (mainHash->entries[index].key.compare_exchange_strong(empty, id, std::memory_order_seq_cst, std::memory_order_relaxed) || + mainHash->entries[index].key.compare_exchange_strong(reusable, id, std::memory_order_seq_cst, std::memory_order_relaxed)) { +#else + if (mainHash->entries[index].key.compare_exchange_strong(empty, id, std::memory_order_seq_cst, std::memory_order_relaxed)) { +#endif + mainHash->entries[index].value = value; + break; + } + ++index; + } + } + + return value; + } + if (probedKey == details::invalid_thread_id) { + break; // Not in this hash table + } + ++index; + } + } + + // Insert! + auto newCount = 1 + implicitProducerHashCount.fetch_add(1, std::memory_order_relaxed); + while (true) { + // NOLINTNEXTLINE(clang-analyzer-core.NullDereference) + if (newCount >= (mainHash->capacity >> 1) && !implicitProducerHashResizeInProgress.test_and_set(std::memory_order_acquire)) { + // We've acquired the resize lock, try to allocate a bigger hash table. + // Note the acquire fence synchronizes with the release fence at the end of this block, and hence when + // we reload implicitProducerHash it must be the most recent version (it only gets changed within this + // locked block). + mainHash = implicitProducerHash.load(std::memory_order_acquire); + if (newCount >= (mainHash->capacity >> 1)) { + size_t newCapacity = mainHash->capacity << 1; + while (newCount >= (newCapacity >> 1)) { + newCapacity <<= 1; + } + auto raw = static_cast((Traits::malloc)(sizeof(ImplicitProducerHash) + std::alignment_of::value - 1 + sizeof(ImplicitProducerKVP) * newCapacity)); + if (raw == nullptr) { + // Allocation failed + implicitProducerHashCount.fetch_sub(1, std::memory_order_relaxed); + implicitProducerHashResizeInProgress.clear(std::memory_order_relaxed); + return nullptr; + } + + auto newHash = new (raw) ImplicitProducerHash; + newHash->capacity = static_cast(newCapacity); + newHash->entries = reinterpret_cast(details::align_for(raw + sizeof(ImplicitProducerHash))); + for (size_t i = 0; i != newCapacity; ++i) { + new (newHash->entries + i) ImplicitProducerKVP; + newHash->entries[i].key.store(details::invalid_thread_id, std::memory_order_relaxed); + } + newHash->prev = mainHash; + implicitProducerHash.store(newHash, std::memory_order_release); + implicitProducerHashResizeInProgress.clear(std::memory_order_release); + mainHash = newHash; + } + else { + implicitProducerHashResizeInProgress.clear(std::memory_order_release); + } + } + + // If it's < three-quarters full, add to the old one anyway so that we don't have to wait for the next table + // to finish being allocated by another thread (and if we just finished allocating above, the condition will + // always be true) + if (newCount < (mainHash->capacity >> 1) + (mainHash->capacity >> 2)) { + auto producer = static_cast(recycle_or_create_producer(false)); + if (producer == nullptr) { + implicitProducerHashCount.fetch_sub(1, std::memory_order_relaxed); + return nullptr; + } + +#ifdef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED + producer->threadExitListener.callback = &ConcurrentQueue::implicit_producer_thread_exited_callback; + producer->threadExitListener.userData = producer; + details::ThreadExitNotifier::subscribe(&producer->threadExitListener); +#endif + + auto index = hashedId; + while (true) { + index &= mainHash->capacity - 1u; + auto empty = details::invalid_thread_id; +#ifdef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED + auto reusable = details::invalid_thread_id2; + if (mainHash->entries[index].key.compare_exchange_strong(reusable, id, std::memory_order_seq_cst, std::memory_order_relaxed)) { + implicitProducerHashCount.fetch_sub(1, std::memory_order_relaxed); // already counted as a used slot + mainHash->entries[index].value = producer; + break; + } +#endif + if (mainHash->entries[index].key.compare_exchange_strong(empty, id, std::memory_order_seq_cst, std::memory_order_relaxed)) { + mainHash->entries[index].value = producer; + break; + } + ++index; + } + return producer; + } + + // Hmm, the old hash is quite full and somebody else is busy allocating a new one. + // We need to wait for the allocating thread to finish (if it succeeds, we add, if not, + // we try to allocate ourselves). + mainHash = implicitProducerHash.load(std::memory_order_acquire); + } + } + +#ifdef MOODYCAMEL_CPP11_THREAD_LOCAL_SUPPORTED + void implicit_producer_thread_exited(ImplicitProducer* producer) + { + // Remove from hash +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODHASH + debug::DebugLock lock(implicitProdMutex); +#endif + auto hash = implicitProducerHash.load(std::memory_order_acquire); + assert(hash != nullptr); // The thread exit listener is only registered if we were added to a hash in the first place + auto id = details::thread_id(); + auto hashedId = details::hash_thread_id(id); + details::thread_id_t probedKey; + + // We need to traverse all the hashes just in case other threads aren't on the current one yet and are + // trying to add an entry thinking there's a free slot (because they reused a producer) + for (; hash != nullptr; hash = hash->prev) { + auto index = hashedId; + do { + index &= hash->capacity - 1u; + probedKey = id; + if (hash->entries[index].key.compare_exchange_strong(probedKey, details::invalid_thread_id2, std::memory_order_seq_cst, std::memory_order_relaxed)) { + break; + } + ++index; + } while (probedKey != details::invalid_thread_id); // Can happen if the hash has changed but we weren't put back in it yet, or if we weren't added to this hash in the first place + } + + // Mark the queue as being recyclable + producer->inactive.store(true, std::memory_order_release); + } + + static void implicit_producer_thread_exited_callback(void* userData) + { + auto producer = static_cast(userData); + auto queue = producer->parent; + queue->implicit_producer_thread_exited(producer); + } +#endif + + ////////////////////////////////// + // Utility functions + ////////////////////////////////// + + template + static inline void* aligned_malloc(size_t size) + { + MOODYCAMEL_CONSTEXPR_IF (std::alignment_of::value <= std::alignment_of::value) + return (Traits::malloc)(size); + else { + size_t alignment = std::alignment_of::value; + void* raw = (Traits::malloc)(size + alignment - 1 + sizeof(void*)); + if (!raw) + return nullptr; + char* ptr = details::align_for(reinterpret_cast(raw) + sizeof(void*)); + *(reinterpret_cast(ptr) - 1) = raw; + return ptr; + } + } + + template + static inline void aligned_free(void* ptr) + { + MOODYCAMEL_CONSTEXPR_IF (std::alignment_of::value <= std::alignment_of::value) + return (Traits::free)(ptr); + else + (Traits::free)(ptr ? *(reinterpret_cast(ptr) - 1) : nullptr); + } + + template + static inline U* create_array(size_t count) + { + assert(count > 0); + U* p = static_cast(aligned_malloc(sizeof(U) * count)); + if (p == nullptr) + return nullptr; + + for (size_t i = 0; i != count; ++i) + new (p + i) U(); + return p; + } + + template + static inline void destroy_array(U* p, size_t count) + { + if (p != nullptr) { + assert(count > 0); + for (size_t i = count; i != 0; ) + (p + --i)->~U(); + } + aligned_free(p); + } + + template + static inline U* create() + { + void* p = aligned_malloc(sizeof(U)); + return p != nullptr ? new (p) U : nullptr; + } + + template + static inline U* create(A1&& a1) + { + void* p = aligned_malloc(sizeof(U)); + return p != nullptr ? new (p) U(std::forward(a1)) : nullptr; + } + + template + static inline void destroy(U* p) + { + if (p != nullptr) + p->~U(); + aligned_free(p); + } + +private: + std::atomic producerListTail; + std::atomic producerCount; + + std::atomic initialBlockPoolIndex; + Block* initialBlockPool; + size_t initialBlockPoolSize; + +#ifndef MCDBGQ_USEDEBUGFREELIST + FreeList freeList; +#else + debug::DebugFreeList freeList; +#endif + + std::atomic implicitProducerHash; + std::atomic implicitProducerHashCount; // Number of slots logically used + ImplicitProducerHash initialImplicitProducerHash; + std::array initialImplicitProducerHashEntries; + std::atomic_flag implicitProducerHashResizeInProgress; + + std::atomic nextExplicitConsumerId; + std::atomic globalExplicitConsumerOffset; + +#ifdef MCDBGQ_NOLOCKFREE_IMPLICITPRODHASH + debug::DebugMutex implicitProdMutex; +#endif + +#ifdef MOODYCAMEL_QUEUE_INTERNAL_DEBUG + std::atomic explicitProducers; + std::atomic implicitProducers; +#endif +}; + + +template +ProducerToken::ProducerToken(ConcurrentQueue& queue) + : producer(queue.recycle_or_create_producer(true)) +{ + if (producer != nullptr) { + producer->token = this; + } +} + +template +ProducerToken::ProducerToken(BlockingConcurrentQueue& queue) + : producer(reinterpret_cast*>(&queue)->recycle_or_create_producer(true)) +{ + if (producer != nullptr) { + producer->token = this; + } +} + +template +ConsumerToken::ConsumerToken(ConcurrentQueue& queue) + : itemsConsumedFromCurrent(0), currentProducer(nullptr), desiredProducer(nullptr) +{ + initialOffset = queue.nextExplicitConsumerId.fetch_add(1, std::memory_order_release); + lastKnownGlobalOffset = static_cast(-1); +} + +template +ConsumerToken::ConsumerToken(BlockingConcurrentQueue& queue) + : itemsConsumedFromCurrent(0), currentProducer(nullptr), desiredProducer(nullptr) +{ + initialOffset = reinterpret_cast*>(&queue)->nextExplicitConsumerId.fetch_add(1, std::memory_order_release); + lastKnownGlobalOffset = static_cast(-1); +} + +template +inline void swap(ConcurrentQueue& a, ConcurrentQueue& b) MOODYCAMEL_NOEXCEPT +{ + a.swap(b); +} + +inline void swap(ProducerToken& a, ProducerToken& b) MOODYCAMEL_NOEXCEPT +{ + a.swap(b); +} + +inline void swap(ConsumerToken& a, ConsumerToken& b) MOODYCAMEL_NOEXCEPT +{ + a.swap(b); +} + +template +inline void swap(typename ConcurrentQueue::ImplicitProducerKVP& a, typename ConcurrentQueue::ImplicitProducerKVP& b) MOODYCAMEL_NOEXCEPT +{ + a.swap(b); +} + +} + +#if defined(_MSC_VER) && (!defined(_HAS_CXX17) || !_HAS_CXX17) +#pragma warning(pop) +#endif + +#if defined(__GNUC__) && !defined(__INTEL_COMPILER) +#pragma GCC diagnostic pop +#endif diff --git a/scaler/io/model.py b/scaler/io/model.py new file mode 100644 index 00000000..77f4a3c8 --- /dev/null +++ b/scaler/io/model.py @@ -0,0 +1,266 @@ +__ALL__ = [ + "Session", + "Message", + "Connector", + "ConnectorType", + "TcpAddr", + "InprocAddr", + "Addr", + "Protocol", +] + +from scaler.io.cpp.ffi import FFITypes, ffi, lib as C, c_async, Message +from scaler.io.cpp.errors import check_status + +from enum import IntEnum, unique +from abc import ABC, abstractmethod + + +class IoContext: + _obj: "FFITypes.CData" + _clients: list + _destroyed: bool = False + + def __init__(self, io_threads: int) -> None: + self._obj = ffi.new("struct IoContext *") + check_status( + C.io_context_init(self._obj, io_threads) + ) + + self._clients = [] + + def destroy(self) -> None: + if self._destroyed: + return + self._destroyed = True + + for client in self._clients: + client.destroy() + + check_status( + C.io_context_destroy(self._obj, True) + ) + + @property + def destroyed(self) -> bool: + return self._destroyed + + def register_client(self, client) -> None: + self._clients.append(client) + + def __enter__(self) -> "IoContext": + return self + + def __exit__(self, _exc_type, _exc_value, _traceback) -> None: + return + +@unique +class ConnectorType(IntEnum): + Pair = C.Pair + Pub = C.Pub + Sub = C.Sub + Dealer = C.Dealer + Router = C.Router + + +@unique +class Protocol(IntEnum): + TCP = C.TCP + IntraProcess = C.IntraProcess + InterProcess = C.InterProcess + + +class Address(ABC): + @property + @abstractmethod + def protocol(self) -> Protocol: ... + + @staticmethod + def from_str(addr: str) -> "Address": + protocol, addr = addr.split("://") + + match protocol: + case "tcp": + addr, port = addr.split(":") + return TCPAddress(host=addr, port=int(port)) + case "intraprocess": + return IntraProcessAddress(name=addr) + case "interprocess": + return InterProcessAddress(path=addr) + case _: + raise ValueError(f"unknown protocol: {protocol}") + + +class TCPAddress(Address): + __match_args__ = ("host", "port") + + host: str + port: int + + def __init__(self, host: str, port: int): + if not isinstance(host, str): + raise TypeError(f"host must be a string; is {type(host)}") + + if not isinstance(port, int): + raise TypeError(f"port must be an integer; is {type(port)}") + + self.host = host + self.port = port + + def __str__(self) -> str: + return f"tcp://{self.host}:{self.port}" + + def copywith(self, host: str | None = None, port: int | None = None) -> "TCPAddress": + return TCPAddress(host=host or self.host, port=port or self.port) + + @property + def protocol(self) -> Protocol: + return Protocol.TCP + + @staticmethod + def bindall(port: int) -> "TCPAddress": + if not isinstance(port, int): + raise TypeError(f"port must be an integer; is {type(port)}") + + return TCPAddress(host="*", port=port) + + @staticmethod + def localhost(port: int) -> "TCPAddress": + if not isinstance(port, int): + raise TypeError(f"port must be an integer; is {type(port)}") + + return TCPAddress(host="127.0.0.1", port=port) + + +class IntraProcessAddress(Address): + __match_args__ = ("name",) + + name: str + + def __init__(self, name: str): + self.name = name + + def __str__(self) -> str: + return f"intraprocess://{self.name}" + + @property + def protocol(self) -> Protocol: + return Protocol.IntraProcess + + +class InterProcessAddress(Address): + __match_args__ = ("path",) + + path: str + + def __init__(self, path: str): + self.path = path + + def __str__(self) -> str: + return f"interprocess://{self.path}" + + @property + def protocol(self) -> Protocol: + return Protocol.InterProcess + +class Connector: + _obj: "FFITypes.CData" + _destroyed: bool = False + _session: IoContext + + def __init__(self, ioctx: IoContext, identity: bytes, type_: ConnectorType, protocol: Protocol): + if ioctx.destroyed: + raise RuntimeError("io context is destroyed") + + self._obj = ffi.new("struct Connector *") + check_status( + C.connector_init(ioctx._obj, self._obj, protocol.value, type_.value, identity, len(identity)) + ) + + self._session = ioctx + self._session.register_client(self) + + def destroy(self) -> None: + if self._destroyed: + return + self._destroyed = True + + check_status( + C.connector_destroy(self._obj) + ) + + def __check_destroyed(self) -> None: + if self._destroyed: + raise RuntimeError("client is destroyed") + + def bind(self, addr: Address) -> None: + self.__check_destroyed() + + match addr: + case TCPAddress(): + host, port = addr.host, addr.port + case InterProcessAddress(): + host, port = addr.path, 0 + case IntraProcessAddress(): + host, port = addr.name, 0 + + check_status( + C.connector_bind(self._obj, host.encode(), port) + ) + + def connect(self, addr: Address) -> None: + self.__check_destroyed() + + match addr: + case TCPAddress(): + host, port = addr.host, addr.port + case InterProcessAddress(): + host, port = addr.path, 0 + case IntraProcessAddress(): + host, port = addr.name, 0 + + check_status( + C.connector_connect(self._obj, host.encode(), port) + ) + + async def send(self, to: bytes | None = None, data: bytes | None = None) -> None: + self.__check_destroyed() + + if to is None: + to, to_len = ffi.NULL, 0 + else: + to_len = len(to) + + await c_async(C.connector_send_async, self._obj, to, to_len, data, len(data)) + + def send_sync(self, to: bytes | None = None, data: bytes | None = None) -> None: + self.__check_destroyed() + + if to is None: + to, to_len = ffi.NULL, 0 + else: + to_len = len(to) + + check_status( + C.connector_send_sync(self._obj, to, to_len, data, len(data)) + ) + + async def recv(self) -> Message: + self.__check_destroyed() + + return await c_async(C.connector_recv_async, self._obj) + + def recv_sync(self) -> Message: + self.__check_destroyed() + + msg = ffi.new("struct Message *") + check_status( + C.connector_recv_sync(self._obj, msg) + ) + + # copy the message + msg_ = Message(msg) + + # free data + C.message_destroy(msg) + return msg_ diff --git a/scaler/io/sync_connector.py b/scaler/io/sync_connector.py index b3c9cb85..70a4b7fc 100644 --- a/scaler/io/sync_connector.py +++ b/scaler/io/sync_connector.py @@ -5,40 +5,45 @@ import uuid from typing import Optional -import zmq - from scaler.io.utility import deserialize, serialize from scaler.protocol.python.mixins import Message -from scaler.utility.zmq_config import ZMQConfig + +from scaler.io.model import ConnectorType, IoContext, Address, TCPAddress, IntraProcessAddress, InterProcessAddress, Connector, TCPAddress, IntraProcessAddress class SyncConnector: - def __init__(self, context: zmq.Context, socket_type: int, address: ZMQConfig, identity: Optional[bytes]): + _connector: Connector + + def __init__(self, + session: IoContext, + type_: ConnectorType, + address: Address, + identity: bytes | None): self._address = address - self._context = context - self._socket = self._context.socket(socket_type) + match address: + case TCPAddress(): + host = address.host + case IntraProcessAddress(): + host = address.name + case InterProcessAddress(): + host = address.path self._identity: bytes = ( - f"{os.getpid()}|{socket.gethostname().split('.')[0]}|{uuid.uuid4()}".encode() + f"{os.getpid()}|{host}|{uuid.uuid4()}".encode() if identity is None else identity ) - # set socket option - self._socket.setsockopt(zmq.IDENTITY, self._identity) - self._socket.setsockopt(zmq.SNDHWM, 0) - self._socket.setsockopt(zmq.RCVHWM, 0) - - self._socket.connect(self._address.to_address()) - + self._connector = Connector(session, self._identity, type_, address.protocol) + self._connector.connect(self._address) self._lock = threading.Lock() def close(self): - self._socket.close() + self._connector.destroy() @property - def address(self) -> ZMQConfig: + def address(self) -> Address: return self._address @property @@ -47,13 +52,13 @@ def identity(self) -> bytes: def send(self, message: Message): with self._lock: - self._socket.send(serialize(message), copy=False) + self._connector.send_sync(data=serialize(message)) def receive(self) -> Optional[Message]: with self._lock: - payload = self._socket.recv(copy=False) + msg = self._connector.recv_sync() - return self.__compose_message(payload.bytes) + return self.__compose_message(msg.payload) def __compose_message(self, payload: bytes) -> Optional[Message]: result: Optional[Message] = deserialize(payload) diff --git a/scaler/io/sync_subscriber.py b/scaler/io/sync_subscriber.py index 17bc5b77..1f45ed23 100644 --- a/scaler/io/sync_subscriber.py +++ b/scaler/io/sync_subscriber.py @@ -2,17 +2,16 @@ import threading from typing import Callable, Optional -import zmq from scaler.io.utility import deserialize from scaler.protocol.python.mixins import Message -from scaler.utility.zmq_config import ZMQConfig +from scaler.io.model import Connector, Address, IoContext, ConnectorType class SyncSubscriber(threading.Thread): def __init__( self, - address: ZMQConfig, + address: Address, callback: Callable[[Message], None], topic: bytes, exit_callback: Optional[Callable[[], None]] = None, @@ -30,11 +29,11 @@ def __init__( self.daemon = bool(daemonic) self._timeout_seconds = timeout_seconds - self._context: Optional[zmq.Context] = None - self._socket: Optional[zmq.Socket] = None + self._session: IoContext | None = None + self._client: Connector | None = None def __close(self): - self._socket.close() + self._connector.destroy() def __stop_polling(self): self._stop_event.set() @@ -54,24 +53,13 @@ def run(self) -> None: self.__close() def __initialize(self): - self._context = zmq.Context.instance() - self._socket = self._context.socket(zmq.SUB) - self._socket.setsockopt(zmq.RCVHWM, 0) - - if self._timeout_seconds == -1: - self._socket.setsockopt(zmq.RCVTIMEO, self._timeout_seconds) - else: - self._socket.setsockopt(zmq.RCVTIMEO, self._timeout_seconds * 1000) - - self._socket.subscribe(self._topic) - self._socket.connect(self._address.to_address()) - self._socket.connect(self._address.to_address()) + self._session = IoContext(io_threads=1) + self._connector = Connector(self._session, "sync_subscriber".encode(), ConnectorType.Sub, self._address.protocol) + self._connector.connect(self._address) def __routine_polling(self): - try: - self.__routine_receive(self._socket.recv(copy=False).bytes) - except zmq.Again: - raise TimeoutError(f"Cannot connect to {self._address.to_address()} in {self._timeout_seconds} seconds") + msg_ = self._connector.recv_sync() + self.__routine_receive(msg_.payload) def __routine_receive(self, payload: bytes): result: Optional[Message] = deserialize(payload) diff --git a/scaler/scheduler/config.py b/scaler/scheduler/config.py index 576f222a..7a2cee38 100644 --- a/scaler/scheduler/config.py +++ b/scaler/scheduler/config.py @@ -1,12 +1,12 @@ import dataclasses -from scaler.utility.zmq_config import ZMQConfig +from scaler.io.model import TCPAddress @dataclasses.dataclass class SchedulerConfig: event_loop: str - address: ZMQConfig + address: TCPAddress io_threads: int max_number_of_tasks_waiting: int per_worker_queue_size: int diff --git a/scaler/scheduler/scheduler.py b/scaler/scheduler/scheduler.py index 2d378d18..761ec373 100644 --- a/scaler/scheduler/scheduler.py +++ b/scaler/scheduler/scheduler.py @@ -2,8 +2,6 @@ import functools import logging -import zmq.asyncio - from scaler.io.async_binder import AsyncBinder from scaler.io.async_connector import AsyncConnector from scaler.io.config import CLEANUP_INTERVAL_SECONDS, STATUS_REPORT_INTERVAL_SECONDS @@ -30,27 +28,26 @@ from scaler.scheduler.worker_manager import VanillaWorkerManager from scaler.utility.event_loop import create_async_loop_routine from scaler.utility.exceptions import ClientShutdownException -from scaler.utility.zmq_config import ZMQConfig, ZMQType + +from scaler.io.model import IoContext, ConnectorType, TCPAddress class Scheduler: def __init__(self, config: SchedulerConfig): - if config.address.type != ZMQType.tcp: + if not isinstance(config.address, TCPAddress): raise TypeError( - f"{self.__class__.__name__}: scheduler address must be tcp type: {config.address.to_address()}" + f"{self.__class__.__name__}: scheduler address must be tcp type: {config.address}" ) - self._address_monitor = ZMQConfig( - type=ZMQType.ipc, host=f"/tmp/{config.address.host}_{config.address.port}_monitor" - ) + self._address_monitor = config.address.copywith(port=config.address.port + 2) - logging.info(f"{self.__class__.__name__}: monitor address is {self._address_monitor.to_address()}") - self._context = zmq.asyncio.Context(io_threads=config.io_threads) - self._binder = AsyncBinder(context=self._context, name="scheduler", address=config.address) + logging.info(f"{self.__class__.__name__}: monitor address is {self._address_monitor}") + self._session = IoContext(config.io_threads) + self._binder = AsyncBinder(session=self._session, name="scheduler", address=config.address) self._binder_monitor = AsyncConnector( - context=self._context, + session=self._session, name="scheduler_monitor", - socket_type=zmq.PUB, + type_=ConnectorType.Pub, address=self._address_monitor, bind_or_connect="bind", callback=None, @@ -175,6 +172,7 @@ async def get_loops(self): self._binder.destroy() self._binder_monitor.destroy() + self._session.destroy() @functools.wraps(Scheduler) diff --git a/scaler/worker/agent/processor/processor.py b/scaler/worker/agent/processor/processor.py index 45d57f04..c5515360 100644 --- a/scaler/worker/agent/processor/processor.py +++ b/scaler/worker/agent/processor/processor.py @@ -9,7 +9,6 @@ from typing import Callable, List, Optional, Tuple import tblib.pickling_support -import zmq from scaler.io.config import DUMMY_CLIENT from scaler.io.sync_connector import SyncConnector @@ -27,9 +26,12 @@ from scaler.utility.exceptions import MissingObjects from scaler.utility.logging.utility import setup_logger from scaler.utility.object_utility import generate_object_id, generate_serializer_object_id, serialize_failure -from scaler.utility.zmq_config import ZMQConfig from scaler.worker.agent.processor.object_cache import ObjectCache + +from scaler.io.model import IoContext, ConnectorType, TCPAddress + + SUSPEND_SIGNAL = "SIGUSR1" # use str instead of a signal.Signal to not trigger an import error on unsupported systems. _current_processor: ContextVar[Optional["Processor"]] = ContextVar("_current_processor", default=None) @@ -39,7 +41,7 @@ class Processor(multiprocessing.get_context("spawn").Process): # type: ignore def __init__( self, event_loop: str, - address: ZMQConfig, + address: TCPAddress, resume_event: Optional[EventType], resumed_event: Optional[EventType], garbage_collect_interval_seconds: int, @@ -85,11 +87,12 @@ def __initialize(self): logging_paths.append("/dev/stdout") setup_logger(log_paths=tuple(logging_paths), logging_level=self._logging_level) + tblib.pickling_support.install() + self._session = IoContext(1) self._connector = SyncConnector( - context=zmq.Context(), socket_type=zmq.DEALER, address=self._address, identity=None - ) + session=self._session, type_=ConnectorType.Dealer, address=self._address, identity=None) self._object_cache = ObjectCache( garbage_collect_interval_seconds=self._garbage_collect_interval_seconds, @@ -128,10 +131,6 @@ def __run_forever(self): self.__on_connector_receive(message) - except zmq.error.ZMQError as e: - if e.errno != zmq.ENOTSOCK: # ignore if socket got closed - raise - except (KeyboardInterrupt, InterruptedError): pass diff --git a/scaler/worker/agent/processor_holder.py b/scaler/worker/agent/processor_holder.py index 05604518..fd12cd4c 100644 --- a/scaler/worker/agent/processor_holder.py +++ b/scaler/worker/agent/processor_holder.py @@ -8,15 +8,15 @@ from scaler.io.config import DEFAULT_PROCESSOR_KILL_DELAY_SECONDS from scaler.protocol.python.message import Task -from scaler.utility.zmq_config import ZMQConfig from scaler.worker.agent.processor.processor import SUSPEND_SIGNAL, Processor +from scaler.io.model import TCPAddress class ProcessorHolder: def __init__( self, event_loop: str, - address: ZMQConfig, + address: TCPAddress, garbage_collect_interval_seconds: int, trim_memory_threshold_bytes: int, hard_suspend: bool, diff --git a/scaler/worker/agent/processor_manager.py b/scaler/worker/agent/processor_manager.py index 0f63b936..fba4785b 100644 --- a/scaler/worker/agent/processor_manager.py +++ b/scaler/worker/agent/processor_manager.py @@ -6,7 +6,6 @@ from typing import Dict, List, Optional, Tuple import tblib.pickling_support -import zmq.asyncio # from scaler.utility.logging.utility import setup_logger from scaler.io.async_binder import AsyncBinder @@ -26,15 +25,15 @@ from scaler.utility.metadata.profile_result import ProfileResult from scaler.utility.mixins import Looper from scaler.utility.object_utility import generate_object_id, serialize_failure -from scaler.utility.zmq_config import ZMQConfig, ZMQType from scaler.worker.agent.mixins import HeartbeatManager, ObjectTracker, ProcessorManager, ProfilingManager, TaskManager from scaler.worker.agent.processor_holder import ProcessorHolder +from scaler.io.model import IoContext, InterProcessAddress class VanillaProcessorManager(Looper, ProcessorManager): def __init__( self, - context: zmq.asyncio.Context, + session: IoContext, event_loop: str, garbage_collect_interval_seconds: int, trim_memory_threshold_bytes: int, @@ -53,7 +52,7 @@ def __init__( self._logging_level = logging_level self._address_path = os.path.join(tempfile.gettempdir(), f"scaler_worker_{uuid.uuid4().hex}") - self._address = ZMQConfig(ZMQType.ipc, host=self._address_path) + self._address = InterProcessAddress(self._address_path) self._heartbeat: Optional[HeartbeatManager] = None self._task_manager: Optional[TaskManager] = None @@ -67,9 +66,12 @@ def __init__( self._can_accept_task_lock: asyncio.Lock = asyncio.Lock() - self._binder_internal: AsyncBinder = AsyncBinder( - context=context, name="processor_manager", address=self._address, identity=None + self._session = session + + self._binder_internal = AsyncBinder( + session=self._session, name="processor_manager", address=self._address ) + self._binder_internal.register(self.__on_receive_internal) def register( diff --git a/scaler/worker/worker.py b/scaler/worker/worker.py index dd249973..37d613bc 100644 --- a/scaler/worker/worker.py +++ b/scaler/worker/worker.py @@ -4,8 +4,6 @@ import signal from typing import Optional, Tuple -import zmq.asyncio - from scaler.io.async_connector import AsyncConnector from scaler.io.config import PROFILING_INTERVAL_SECONDS from scaler.protocol.python.message import ( @@ -21,7 +19,6 @@ from scaler.utility.event_loop import create_async_loop_routine, register_event_loop from scaler.utility.exceptions import ClientShutdownException from scaler.utility.logging.utility import setup_logger -from scaler.utility.zmq_config import ZMQConfig from scaler.worker.agent.heartbeat_manager import VanillaHeartbeatManager from scaler.worker.agent.object_tracker import VanillaObjectTracker from scaler.worker.agent.processor_manager import VanillaProcessorManager @@ -29,13 +26,15 @@ from scaler.worker.agent.task_manager import VanillaTaskManager from scaler.worker.agent.timeout_manager import VanillaTimeoutManager +from scaler.io.model import IoContext, TCPAddress, ConnectorType + class Worker(multiprocessing.get_context("spawn").Process): # type: ignore def __init__( self, event_loop: str, name: str, - address: ZMQConfig, + address: TCPAddress, io_threads: int, heartbeat_interval_seconds: int, garbage_collect_interval_seconds: int, @@ -63,7 +62,7 @@ def __init__( self._logging_paths = logging_paths self._logging_level = logging_level - self._context: Optional[zmq.asyncio.Context] = None + self._session: Optional[IoContext] = None self._connector_external: Optional[AsyncConnector] = None self._task_manager: Optional[VanillaTaskManager] = None self._heartbeat_manager: Optional[VanillaHeartbeatManager] = None @@ -82,11 +81,11 @@ def __initialize(self): setup_logger() register_event_loop(self._event_loop) - self._context = zmq.asyncio.Context() + self._session = IoContext(self._io_threads) self._connector_external = AsyncConnector( - context=self._context, + session=self._session, name=self.name, - socket_type=zmq.DEALER, + type_=ConnectorType.Dealer, address=self._address, bind_or_connect="connect", callback=self.__on_receive_external, @@ -98,7 +97,7 @@ def __initialize(self): self._task_manager = VanillaTaskManager(task_timeout_seconds=self._task_timeout_seconds) self._timeout_manager = VanillaTimeoutManager(death_timeout_seconds=self._death_timeout_seconds) self._processor_manager = VanillaProcessorManager( - context=self._context, + session=self._session, event_loop=self._event_loop, garbage_collect_interval_seconds=self._garbage_collect_interval_seconds, trim_memory_threshold_bytes=self._trim_memory_threshold_bytes, @@ -172,15 +171,15 @@ async def __get_loops(self): except asyncio.CancelledError: pass except (ClientShutdownException, TimeoutError) as e: - logging.info(f"Worker[{self.pid}]: {str(e)}") + logging.info(f"Worker[{self.pid}]: {e}") except Exception as e: logging.exception(f"Worker[{self.pid}]: failed with unhandled exception:\n{(e)}") await self._connector_external.send(DisconnectRequest.new_msg(self._connector_external.identity)) - self._connector_external.destroy() - self._processor_manager.destroy("quited") - logging.info(f"Worker[{self.pid}]: quited") + self._processor_manager.destroy("quitted") + self._session.destroy() + logging.info(f"Worker[{self.pid}]: quitted") def __run_forever(self): self._loop.run_until_complete(self._task) diff --git a/test.py b/test.py new file mode 100644 index 00000000..1f1aba59 --- /dev/null +++ b/test.py @@ -0,0 +1,18 @@ +import sys + +port = int(sys.argv[1]) + +import math +from scaler import Client + +with Client(address=f"tcp://127.0.0.1:{port}") as client: + # Submits 100 tasks + futures = [ + client.submit(math.sqrt, i) + for i in range(0, 100) + ] + + # Collects the results and sums them + result = sum(future.result() for future in futures) + + print("!!!! THIS IS THE RESULT !!!!;;", result) # 661.46