diff --git a/doc/advanced.rst b/doc/advanced.rst index fe6ac3bdf..b048f2b29 100644 --- a/doc/advanced.rst +++ b/doc/advanced.rst @@ -5,4 +5,5 @@ Advanced features :maxdepth: 2 recv-chunk + recv-chunk-group recv-stats diff --git a/doc/changelog.rst b/doc/changelog.rst index ddb9dafeb..1b863e6ff 100644 --- a/doc/changelog.rst +++ b/doc/changelog.rst @@ -3,8 +3,21 @@ Changelog .. rubric:: Development version +- Add support for :doc:`recv-chunk-group` to assemble chunks in parallel. +- Simplify the way receive streams shut down. Users should not notice any + change, but custom reader implementations will need to be updated. - Update :meth:`!test_async_flush` and :meth:`!test_async_flush_fail` to keep handles to async tasks, to prevent them being garbage collected too early. +- Fix a bug where copying a :cpp:class:`spead2::recv::stream_config` would not + deep copy the names of custom statistics, and so any statistics added to the + copy would also affect the original, and there were also potential race + conditions if a stream config was modified while holding stream statistics. +- Fix a bug (caused by the bug above) where passing a + :cpp:class:`spead2::recv::stream_config` to construct a + :cpp:class:`spead2::recv::chunk_stream` would modify the config. Passing + the same config to construct two chunk streams would fail with an error. +- Fix the type annotation for the :py:class:`~.ChunkRingStream` constructor: + the parameter name for `chunk_stream_config` was incorrect. .. rubric:: 3.11.1 diff --git a/doc/cpp-recv-chunk-group.rst b/doc/cpp-recv-chunk-group.rst new file mode 100644 index 000000000..4de6c171c --- /dev/null +++ b/doc/cpp-recv-chunk-group.rst @@ -0,0 +1,19 @@ +Chunking stream groups +====================== + +For an overview, refer to :doc:`recv-chunk-group`. This page is a reference for the +C++ API. + +.. doxygenclass:: spead2::recv::chunk_stream_group_config + :members: + +.. doxygenclass:: spead2::recv::chunk_stream_group + :members: + +.. doxygenclass:: spead2::recv::chunk_stream_group_member + :members: + +Ringbuffer convenience API +-------------------------- +.. doxygenclass:: spead2::recv::chunk_stream_ring_group + :members: diff --git a/doc/cpp.rst b/doc/cpp.rst index 976c6cd06..236ed96cd 100644 --- a/doc/cpp.rst +++ b/doc/cpp.rst @@ -40,3 +40,4 @@ search path, and you need to set :envvar:`PKG_CONFIG_PATH` to cpp-logging cpp-ibverbs cpp-recv-chunk + cpp-recv-chunk-group diff --git a/doc/dev-recv-chunk-group.rst b/doc/dev-recv-chunk-group.rst new file mode 100644 index 000000000..4e1b0c398 --- /dev/null +++ b/doc/dev-recv-chunk-group.rst @@ -0,0 +1,58 @@ +Synchronisation in chunk stream groups +====================================== +.. cpp:namespace-push:: spead2::recv + +For chunk stream groups to achieve the goal of allowing multi-core scaling, it +is necessary to minimise locking. The implementation achieves this by avoiding +any packet- or heap-granularity locking, and performing locking only at chunk +granularity. Chunks are assumed to be large enough that this minimises total +overhead, although it should be noted that these locks are expected to be +highly contended and there may be further work possible to reduce the +overheads. + +To avoid the need for heap-level locking, each member stream has its own +sliding window with pointers to the chunks, so that heaps which fall inside an +existing chunk can be serviced without locking. However, this causes a problem +when flushing chunks from the group's window: a stream might still be writing +to the chunk at the time. Additionally, it might not be possible to allocate a +new chunk until an old chunk is flushed e.g., if there is a fixed pool of +chunks rather than dynamic allocation. + +The group keeps its own copy of the head positions (oldest chunk) from the +individual streams, protected by the group mutex rather than the stream +mutexes. The group then maintains its head chunk position to match the oldest +head position of any of the member streams. When the group wishes to +evict a chunk, it simply needs to wait for all streams to make enough progress +that the group's head moves past that chunk. + +The wait is achieved using a condition variable that is notified whenever the +head position increases. This allows the group mutex to be dropped while +waiting, which prevents the deadlocks that might otherwise occur if the mutex +was held while waiting and another stream was attemping to lock the group mutex +to make forward progress. + +In lossless eviction mode, this is all that is needed, although it is +non-trivial to see that this won't deadlock with all the streams sitting in +the wait loop waiting for other streams to make forward progress. That this +cannot happen is due to the requirement that the stream's window cannot be +larger than the group's. Consider the active call to +:cpp:func:`chunk_stream_group::get_chunk` with the smallest chunk ID. That +stream is guaranteed to have already readied any chunk due to be evicted from +the group, and the same is true of any other stream that is waiting in +:cpp:func:`~chunk_stream_group::get_chunk`, and so forward progress depends +only on streams that are not blocked in +:cpp:func:`~chunk_stream_group::get_chunk`. + +In lossy eviction mode, we need to make sure that such streams make forward +progress even if no new packets arrive on them. This is achieved by posting an +asynchronous callback to all streams requesting them to flush out chunks that +are now too old. The callback will never reach streams that have already +stopped; we handle this at the time the stream stops, by treating it as having +a head of ``INT64_MAX``. + +While lossless mode is normally allowed to block indefinitely, we do need to +interrupt things in :cpp:func:`chunk_stream_group::stop`. This is handled +similarly to lossy eviction mode, where all streams are requested to flush up +to ``INT64_MAX``. + +.. cpp:namespace-pop:: diff --git a/doc/dev-recv-destruction.rst b/doc/dev-recv-destruction.rst new file mode 100644 index 000000000..3510ac1b4 --- /dev/null +++ b/doc/dev-recv-destruction.rst @@ -0,0 +1,54 @@ +Destruction of receive streams +============================== +The asynchronous and parallel nature of spead2 makes destroying a receive +stream a tricky operation: there may be pending asio completion handlers that +will try to push packets into the stream, leading to a race condition. While +asio guarantees that closing a socket will cancel any pending asynchronous +operations on that socket, this doesn't account for cases where the operation +has already completed but the completion handler is either pending or is +currently running. + +Up to version 3.11, this was handled by a shutdown protocol +between :cpp:class:`spead2::recv::stream` and +:cpp:class:`spead2::recv::reader`. The reader was required to notify the +stream when it had completely shut down, and +:cpp:func:`spead2::recv::stream::stop` would block until all readers had +performed this notification (via a semaphore). This protocol was complicated, +and it relied on the reader being able to make forward progress while the +thread calling :cpp:func:`~spead2::recv::stream::stop` was blocked. + +Newer versions take a different approach based on shared pointers. The ideal +case would be to have the whole stream always managed by a shared pointer, so +that a completion handler that interfaces with the stream could keep a copy of +the shared pointer and thus keep it alive as long as needed. However, that is +not possible to do in a backwards-compatible way. Instead, a minimal set of +fields is placed inside a shared pointer, namely: + +- The ``queue_mutex`` +- A flag indicating whether the stream has stopped. + +For convenience, the flag is encoded as a pointer, which holds either a +pointer to the stream (if not stopped) or a null pointer (if stopped). Each +completion handler holds a shared reference to this structure. When it wishes +to access the stream, it should: + +1. Lock the mutex. +2. Get the pointer back to the stream from the shared structure, aborting if + it gets a null pointer. +3. Manipulate the stream. +4. Drop the mutex. + +This prevents use-after-free errors because the stream cannot be destroyed +without first stopping, and stopping locks the mutex. Hence, the stream cannot +disappear asynchronously during step 3. Note that it can, however, stop +during step 3 if the completion handler causes it to stop. + +Using shared pointers in this way can add overhead because atomically +incrementing and decrementing reference counts can be expensive, particularly +if it causes cache line migrations between processor cores. To minimise +reference count manipulation, the :cpp:class:`~spead2::recv::reader` class +encapsulates this workflow in its +:cpp:class:`~spead2::recv::reader::bind_handler` member function, which +provides the facilities to move the shared pointer along a linear chain of +completion handlers so that the reference count does not need to be +adjusted. diff --git a/doc/developer.rst b/doc/developer.rst new file mode 100644 index 000000000..c2ecbb60a --- /dev/null +++ b/doc/developer.rst @@ -0,0 +1,12 @@ +Developer documentation +======================= + +This section documents internal design decisions that users will generally not +need to be aware of, although some of it may be useful if you plan to subclass +the C++ classes to extend functionality. + +.. toctree:: + :maxdepth: 2 + + dev-recv-destruction + dev-recv-chunk-group diff --git a/doc/index.rst b/doc/index.rst index a91affd5d..611117781 100644 --- a/doc/index.rst +++ b/doc/index.rst @@ -18,6 +18,7 @@ Contents: perf tools migrate-3 + developer changelog license diff --git a/doc/py-recv-chunk-group.rst b/doc/py-recv-chunk-group.rst new file mode 100644 index 000000000..6b57fa68d --- /dev/null +++ b/doc/py-recv-chunk-group.rst @@ -0,0 +1,89 @@ +Chunking stream groups +====================== + +For an overview, refer to :doc:`recv-chunk-group`. This page is a reference for the +Python API. It extends the API for :doc:`chunks `. + +.. py:class:: spead2.recv.ChunkStreamGroupConfig(**kwargs) + + Parameters for a chunk stream group. The configuration options + can either be passed to the constructor (as keyword arguments) or set as + properties after construction. + + :param int max_chunks: + The maximum number of chunks that can be live at the same time. + :param EvictionMode eviction_mode: + The chunk eviction mode. + + .. py:class:: EvictionMode + + Eviction mode when it is necessary to advance the group window. See + the :doc:`overview ` for more details. + + .. py:attribute:: LOSSY + + force streams to release incomplete chunks + + .. py:attribute:: LOSSLESS + + a chunk will only be marked ready when all streams have marked it + ready + +.. py:class:: spead2.recv.ChunkStreamRingGroup(config, data_ringbuffer, free_ringbuffer) + + Stream group that uses ringbuffers to manage chunks. + + When a fresh chunk is needed, it is retrieved from a ringbuffer of free + chunks (the "free ring"). When a chunk is flushed, it is pushed to a "data + ring". These may be shared between groups, but both will be stopped as soon + as any of the members streams are stopped. The intended use case is + parallel groups that are started and stopped together. + + It behaves like a :py:class:`~collections.abc.Sequence` of the contained + streams. + + :param config: Group configuration + :type config: :py:class:`spead2.recv.ChunkStreamGroupConfig` + :param data_ringbuffer: Ringbuffer onto which the completed chunks are placed. + :type data_ringbuffer: :py:class:`spead2.recv.ChunkRingbuffer` + :param free_ringbuffer: Ringbuffer from which new chunks are obtained. + :type free_ringbuffer: :py:class:`spead2.recv.ChunkRingbuffer` + + .. py:attribute:: data_ringbuffer + + The data ringbuffer given to the constructor. + + .. py:attribute:: free_ringbuffer + + The free ringbuffer given to the constructor. + + .. py:method:: add_free_chunk(chunk) + + Add a chunk to the free ringbuffer. This takes care of zeroing out the + :py:attr:`.Chunk.present` array, and it will suppress the + :exc:`spead2.Stopped` exception if the free ringbuffer has been stopped. + + If the free ring is full, it will raise :exc:`spead2.Full` rather than + blocking. The free ringbuffer should be constructed with enough slots that + this does not happen. + + .. py:method:: emplace_back(thread_pool, config, chunk_stream_config) + + Add a new stream. + + :param thread_pool: Thread pool handling the I/O + :type thread_pool: :py:class:`spead2.ThreadPool` + :param config: Stream configuration + :type config: :py:class:`spead2.recv.StreamConfig` + :param chunk_config: Chunking configuration + :type chunk_config: :py:class:`spead2.recv.ChunkStreamConfig` + :rtype: :py:class:`spead2.recv.ChunkStreamGroupMember` + +.. py:class:: spead2.recv.ChunkStreamGroupMember + + A component stream in a :py:class:`~spead2.recv.ChunkStreamRingGroup`. + This class cannot be instantiated directly. Use + :py:meth:`.ChunkStreamRingGroup.emplace_back` instead. + + It provides the same methods for adding readers as + :py:class:`spead2.recv.Stream`. diff --git a/doc/py-recv-chunk.rst b/doc/py-recv-chunk.rst index da3ecf0a2..587d3b65c 100644 --- a/doc/py-recv-chunk.rst +++ b/doc/py-recv-chunk.rst @@ -225,18 +225,18 @@ Reference .. py:attribute:: data_ringbuffer - The data ringbuffer given to the constructor. + The data ringbuffer given to the constructor. .. py:attribute:: free_ringbuffer - The free ringbuffer given to the constructor. + The free ringbuffer given to the constructor. .. py:method:: add_free_chunk(chunk) - Add a chunk to the free ringbuffer. This takes care of zeroing out the - :py:attr:`.Chunk.present` array, and it will suppress the - :exc:`spead2.Stopped` exception if the free ringbuffer has been stopped. + Add a chunk to the free ringbuffer. This takes care of zeroing out the + :py:attr:`.Chunk.present` array, and it will suppress the + :exc:`spead2.Stopped` exception if the free ringbuffer has been stopped. - If the free ring is full, it will raise :exc:`spead2.Full` rather than - blocking. The free ringbuffer should be constructed with enough slots that - this does not happen. + If the free ring is full, it will raise :exc:`spead2.Full` rather than + blocking. The free ringbuffer should be constructed with enough slots that + this does not happen. diff --git a/doc/py.rst b/doc/py.rst index c53a3bd31..fa55c1366 100644 --- a/doc/py.rst +++ b/doc/py.rst @@ -21,3 +21,4 @@ with the C++ backend. py-logging py-ibverbs py-recv-chunk + py-recv-chunk-group diff --git a/doc/recv-chunk-group.rst b/doc/recv-chunk-group.rst new file mode 100644 index 000000000..176f8e5ed --- /dev/null +++ b/doc/recv-chunk-group.rst @@ -0,0 +1,82 @@ +Chunking stream groups +====================== +.. cpp:namespace-push:: spead2::recv + +While the :doc:`recv-chunk` allows for high-bandwidth streams to be received +with low overhead, it still has a fundamental scaling limitation: each chunk +can only be constructed from a single thread. :dfn:`Chunk stream groups` allow +this overhead to be overcome, although not without caveats. + +Each stream is still limited to a single thread. However, a :dfn:`group` of +streams can share the same sequence of chunks, with each stream contributing +a subset of the data in each chunk. Making use of this feature requires +that load balancing is implemented at the network level, using different +destination addresses or ports so that the incoming heaps can be multiplexed +into multiple streams. + +As with a single chunk stream, the group keeps a sliding window of chunks and +obtains new ones from an allocation callback. When the window slides forward, +chunks that fall out the back of the window are provided to a ready callback. +Each member stream also has its own sliding window, which can be smaller (but not +larger) than the group's window. When the group's window slides forward, the +streams' windows are adjusted to ensure they still fit within the group's +window. In other words, a stream's window determines how much reordering is +tolerated within a stream, while the group's window determines how out-of-sync +the streams are allowed to become. + +When desynchronisation does occur, there is a choice of strategies. The default +strategy is eager but potentially lossy: when the group's window moves forward, +the trailing chunk is marked ready as soon as possible, even if this causes +some stream windows to shrink below their normal size. An alternative strategy +is lossless: when the group's window needs to move forward, it is blocked +until all the member streams have caught up. This latter mode is intended for +use with lossless transports such as TCP. However, if one of the component streams +stops functioning (for example, because it is routed on a network path that is +down) it prevents the entire group from making forward progress. + +The general flow (in C++) is + +1. Create a :cpp:class:`chunk_stream_group_config`. +2. Create a :cpp:class:`chunk_stream_group`. +3. Use :cpp:func:`chunk_stream_group::emplace_back` to + create the streams. +4. Add readers to the streams. +5. Process the data. +6. Optionally, call :cpp:func:`chunk_stream_group::stop()` + (otherwise it will be called on destruction). +7. Destroy the group. + +In Python the process is similar, although garbage collection replaces +explicit destruction. + +Ringbuffer convenience API +-------------------------- +As for standalone chunk streams, there is a simplified API using ringbuffers, +which is also the only API available for Python. A +:cpp:class:`chunk_stream_ring_group` is a group that allocates +data from one ringbuffer and send ready data to another. The description of +:ref:`that api ` largely applies here too. The +ringbuffers can be shared between groups. + +Caveats +------- +This is an advanced API that sacrifices some user-friendliness for +performance, and thus some care is needed to use it safely. + +- It is vital that all the streams can make forward progress independently, + as otherwise deadlocks can occur. For example, if they share a thread pool, + the pool must have at least as many threads as streams. It's recommended + that each stream has its own single-threaded thread pool. +- The streams must all be added to the group before adding any readers to + the streams. Once a group has received some data, an exception will be thrown + if one attempts to add a new stream. +- The stream ID associated with each chunk will be the stream ID of one of the + component streams, but it is undefined which one. +- When the allocate and ready callbacks are invoked, it's not specified which + stream's batch statistics pointer will be passed. +- Two streams must not write to the same bytes of a chunk (in the payload, + present array or extra data), as this is undefined behaviour in C++. +- Calling :cpp:func:`~stream::stop` on a member stream will stop the whole + group. + +.. cpp:namespace-pop:: diff --git a/doc/recv-chunk.rst b/doc/recv-chunk.rst index 654691ebb..6188946a7 100644 --- a/doc/recv-chunk.rst +++ b/doc/recv-chunk.rst @@ -105,6 +105,8 @@ At present it is only possible to write a contiguous piece of data per heap. The data is transferred to the chunk even if the heap is incomplete (and hence not marked in the ``present`` array). +.. _recv-chunk-ringbuffer: + Ringbuffer convenience API -------------------------- A subclass is provided that takes care of the allocation and ready callbacks diff --git a/examples/Makefile.am b/examples/Makefile.am index 22ce1962a..2b00376b9 100644 --- a/examples/Makefile.am +++ b/examples/Makefile.am @@ -1,4 +1,4 @@ -# Copyright 2016 National Research Foundation (SARAO) +# Copyright 2016, 2020-2021, 2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -18,6 +18,7 @@ include $(top_srcdir)/src/Makefile.inc.am noinst_PROGRAMS = \ recv_chunk_example \ recv_chunk_ring_example \ + recv_chunk_group_example \ test_recv \ test_send \ test_ringbuffer @@ -33,6 +34,8 @@ recv_chunk_example_SOURCES = recv_chunk_example.cpp recv_chunk_ring_example_SOURCES = recv_chunk_ring_example.cpp +recv_chunk_group_example_SOURCES = recv_chunk_group_example.cpp + if SPEAD2_USE_CUDA V_NVCC = $(v_NVCC_@AM_V@) diff --git a/examples/recv_chunk_group_example.cpp b/examples/recv_chunk_group_example.cpp new file mode 100644 index 000000000..60565b30c --- /dev/null +++ b/examples/recv_chunk_group_example.cpp @@ -0,0 +1,113 @@ +/* Copyright 2023 National Research Foundation (SARAO) + * + * This program is free software: you can redistribute it and/or modify it under + * the terms of the GNU Lesser General Public License as published by the Free + * Software Foundation, either version 3 of the License, or (at your option) any + * later version. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS + * FOR A PARTICULAR PURPOSE. See the GNU Lesser General Public License for more + * details. + * + * You should have received a copy of the GNU Lesser General Public License + * along with this program. If not, see . + */ + +/** + * @file + * + * This is an example of using the chunk stream group API with ringbuffers. + * To test it, run + * spead2_send localhost:8888 localhost:8889 --heaps 1000 --heap-size 65536 --rate 10. + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +static constexpr std::size_t heap_payload_size = 65536; +static constexpr std::size_t heaps_per_chunk = 64; +static constexpr std::size_t chunk_payload_size = heaps_per_chunk * heap_payload_size; + +static void chunk_place(spead2::recv::chunk_place_data *data, std::size_t data_size) +{ + // We requested only the heap ID and size + auto heap_cnt = data->items[0]; + auto payload_size = data->items[1]; + // If the payload size doesn't match, discard the heap (could be descriptors etc). + if (payload_size == heap_payload_size) + { + data->chunk_id = heap_cnt / heaps_per_chunk; + data->heap_index = heap_cnt % heaps_per_chunk; + data->heap_offset = data->heap_index * heap_payload_size; + } +} + +int main() +{ + constexpr std::size_t num_streams = 2; + constexpr int max_chunks = 4; + auto chunk_config = spead2::recv::chunk_stream_config() + .set_items({spead2::HEAP_CNT_ID, spead2::HEAP_LENGTH_ID}) + .set_max_chunks(max_chunks) + .set_place(chunk_place); + auto stream_config = spead2::recv::stream_config(); + auto group_config = spead2::recv::chunk_stream_group_config() + .set_max_chunks(max_chunks); + using chunk_ringbuffer = spead2::ringbuffer>; + auto data_ring = std::make_shared(max_chunks); + auto free_ring = std::make_shared(max_chunks); + auto allocator = std::make_shared(); + + spead2::recv::chunk_stream_ring_group<> group(group_config, data_ring, free_ring); + spead2::thread_pool workers[num_streams]; + for (std::size_t i = 0; i < num_streams; i++) + { + group.emplace_back(workers[i], stream_config, chunk_config); + } + for (int i = 0; i < max_chunks; i++) + { + std::unique_ptr chunk{new spead2::recv::chunk}; + chunk->present = allocator->allocate(heaps_per_chunk, nullptr); + chunk->present_size = heaps_per_chunk; + chunk->data = allocator->allocate(chunk_payload_size, nullptr); + group.add_free_chunk(std::move(chunk)); + } + + for (std::size_t i = 0; i < num_streams; i++) + { + boost::asio::ip::udp::endpoint endpoint(boost::asio::ip::address_v4::any(), 8888 + i); + group[i].emplace_reader( + endpoint, spead2::recv::udp_reader::default_max_size, 1024 * 1024); + } + while (true) + { + try + { + auto chunk = data_ring->pop(); + auto n_present = std::accumulate( + chunk->present.get(), + chunk->present.get() + chunk->present_size, std::size_t(0)); + std::cout << "Received chunk " << chunk->chunk_id << " with " + << n_present << " / " << heaps_per_chunk << " heaps\n"; + group.add_free_chunk(std::move(chunk)); + } + catch (spead2::ringbuffer_stopped &) + { + break; + } + } + + return 0; +} diff --git a/examples/recv_chunk_group_example.py b/examples/recv_chunk_group_example.py new file mode 100755 index 000000000..27fb4b598 --- /dev/null +++ b/examples/recv_chunk_group_example.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python3 + +# Copyright 2023 National Research Foundation (SARAO) +# +# This program is free software: you can redistribute it and/or modify it under +# the terms of the GNU Lesser General Public License as published by the Free +# Software Foundation, either version 3 of the License, or (at your option) any +# later version. +# +# This program is distributed in the hope that it will be useful, but WITHOUT +# ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS +# FOR A PARTICULAR PURPOSE. See the GNU Lesser General Public License for more +# details. +# +# You should have received a copy of the GNU Lesser General Public License +# along with this program. If not, see . + +# This is an example of using the chunk stream group receive API with +# ringbuffers. To test it, run +# spead2_send localhost:8888 localhost:8889 --heaps 1000 --heap-size 65536 --rate 10 + +from spead2.numba import intp_to_voidptr +import spead2.recv +from spead2.recv.numba import chunk_place_data + +import numba +from numba import types +import numpy as np +import scipy + +HEAP_PAYLOAD_SIZE = 65536 +HEAPS_PER_CHUNK = 64 +CHUNK_PAYLOAD_SIZE = HEAPS_PER_CHUNK * HEAP_PAYLOAD_SIZE + + +@numba.cfunc(types.void(types.CPointer(chunk_place_data), types.uintp), nopython=True) +def chunk_place(data_ptr, data_size): + data = numba.carray(data_ptr, 1) + items = numba.carray(intp_to_voidptr(data[0].items), 2, dtype=np.int64) + heap_cnt = items[0] + payload_size = items[1] + # If the payload size doesn't match, discard the heap (could be descriptors etc). + if payload_size == HEAP_PAYLOAD_SIZE: + data[0].chunk_id = heap_cnt // HEAPS_PER_CHUNK + data[0].heap_index = heap_cnt % HEAPS_PER_CHUNK + data[0].heap_offset = data[0].heap_index * HEAP_PAYLOAD_SIZE + + +def main(): + NUM_STREAMS = 2 + MAX_CHUNKS = 4 + place_callback = scipy.LowLevelCallable( + chunk_place.ctypes, + signature='void (void *, size_t)' + ) + chunk_config = spead2.recv.ChunkStreamConfig( + items=[spead2.HEAP_CNT_ID, spead2.HEAP_LENGTH_ID], + max_chunks=MAX_CHUNKS, + place=place_callback) + group_config = spead2.recv.ChunkStreamGroupConfig(max_chunks=MAX_CHUNKS) + data_ring = spead2.recv.ChunkRingbuffer(MAX_CHUNKS) + free_ring = spead2.recv.ChunkRingbuffer(MAX_CHUNKS) + group = spead2.recv.ChunkStreamRingGroup(group_config, data_ring, free_ring) + for _ in range(NUM_STREAMS): + group.emplace_back( + spead2.ThreadPool(), + spead2.recv.StreamConfig(), + chunk_config + ) + for _ in range(MAX_CHUNKS): + chunk = spead2.recv.Chunk( + present=np.empty(HEAPS_PER_CHUNK, np.uint8), + data=np.empty(CHUNK_PAYLOAD_SIZE, np.uint8) + ) + group.add_free_chunk(chunk) + for i in range(NUM_STREAMS): + group[i].add_udp_reader(8888 + i, buffer_size=1024 * 1024, bind_hostname='127.0.0.1') + for chunk in data_ring: + n_present = np.sum(chunk.present) + print( + f"Received chunk {chunk.chunk_id} with " + f"{n_present} / {HEAPS_PER_CHUNK} heaps") + group.add_free_chunk(chunk) + + +if __name__ == '__main__': + main() diff --git a/examples/recv_chunk_ring_example.cpp b/examples/recv_chunk_ring_example.cpp index bcd7afcc6..a44b9f80b 100644 --- a/examples/recv_chunk_ring_example.cpp +++ b/examples/recv_chunk_ring_example.cpp @@ -40,8 +40,6 @@ static constexpr std::size_t heap_payload_size = 65536; static constexpr std::size_t heaps_per_chunk = 64; static constexpr std::size_t chunk_payload_size = heaps_per_chunk * heap_payload_size; -static std::shared_ptr allocator; - static void chunk_place(spead2::recv::chunk_place_data *data, std::size_t data_size) { // We requested only the heap ID and size diff --git a/include/Makefile.am b/include/Makefile.am index b4b3134c6..c29387e98 100644 --- a/include/Makefile.am +++ b/include/Makefile.am @@ -1,4 +1,4 @@ -# Copyright 2016-2021 National Research Foundation (SARAO) +# Copyright 2016-2021, 2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -37,6 +37,7 @@ nobase_include_HEADERS = \ spead2/common_unbounded_queue.h \ spead2/portable_endian.h \ spead2/recv_chunk_stream.h \ + spead2/recv_chunk_stream_group.h \ spead2/recv_heap.h \ spead2/recv_inproc.h \ spead2/recv_live_heap.h \ diff --git a/include/spead2/common_thread_pool.h b/include/spead2/common_thread_pool.h index be6938e6d..1a03974c8 100644 --- a/include/spead2/common_thread_pool.h +++ b/include/spead2/common_thread_pool.h @@ -28,7 +28,6 @@ #include #include #include -#include namespace spead2 { diff --git a/include/spead2/recv_chunk_stream.h b/include/spead2/recv_chunk_stream.h index 0983c2dfd..daf610cb0 100644 --- a/include/spead2/recv_chunk_stream.h +++ b/include/spead2/recv_chunk_stream.h @@ -1,4 +1,4 @@ -/* Copyright 2021-2022 National Research Foundation (SARAO) +/* Copyright 2021-2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -21,15 +21,19 @@ #ifndef SPEAD2_RECV_CHUNK_STREAM #define SPEAD2_RECV_CHUNK_STREAM +#include #include #include #include #include #include #include +#include +#include #include #include #include +#include #include #include @@ -38,9 +42,22 @@ namespace spead2 namespace recv { +namespace detail +{ + +template class chunk_ring_pair; + +} // namespace detail + /// Storage for a chunk with metadata class chunk { + friend class chunk_stream_group; + template friend class detail::chunk_ring_pair; +private: + /// Linked list of chunks to dispose of at shutdown + std::unique_ptr graveyard_next; + public: /// Chunk ID std::int64_t chunk_id = -1; @@ -195,36 +212,183 @@ class chunk_stream_config namespace detail { -class chunk_stream_allocator; - /** - * Base class that holds the internal state of @ref - * spead2::recv::chunk_stream. + * Sliding window of chunk pointers. * - * This is split into a separate class to avoid some initialisation ordering - * problems: it is constructed before the @ref spead2::recv::stream base class, - * allowing the latter to use function objects that reference this class. + * @internal The chunk IDs are kept as unsigned values, so that the tail can + * be larger than any actual chunk ID. */ -class chunk_stream_state +class chunk_window { private: + /// Circular buffer of chunks under construction. + std::vector chunks; + std::uint64_t head_chunk = 0, tail_chunk = 0; ///< chunk IDs of valid chunk range + std::size_t head_pos = 0, tail_pos = 0; ///< Positions corresponding to @ref head and @ref tail in @ref chunks + +public: + /// Send the oldest chunk to the ready callback + template + void flush_head(const F &ready_chunk) + { + assert(head_chunk < tail_chunk); + if (chunks[head_pos]) + { + ready_chunk(chunks[head_pos]); + chunks[head_pos] = nullptr; + } + head_chunk++; + head_pos++; + if (head_pos == chunks.size()) + head_pos = 0; // wrap around the circular buffer + } + + /// Send the oldest chunk to the ready callback + template + void flush_head(const F1 &ready_chunk, const F2 &head_updated) + { + flush_head(ready_chunk); + head_updated(head_chunk); + } + + /** + * Send all the chunks to the ready callback. Afterwards, + * the head and tail are both advanced to @a next_chunk. + */ + template + void flush_all(std::uint64_t next_chunk, const F1 &ready_chunk, const F2 &head_updated) + { + std::uint64_t orig_head = head_chunk; + while (!empty()) + flush_head(ready_chunk); + head_chunk = tail_chunk = next_chunk; + if (head_chunk != orig_head) + head_updated(head_chunk); + } + + /// Flush until the head is at least @a target + template + void flush_until(std::uint64_t target, const F1 &ready_chunk, const F2 &head_updated) + { + if (head_chunk < target) + { + while (head_chunk != tail_chunk && head_chunk < target) + flush_head(ready_chunk); + if (head_chunk < target) + head_chunk = tail_chunk = target; + head_updated(target); + } + } + + explicit chunk_window(std::size_t max_chunks); + + /** + * Obtain a pointer to a chunk with ID @a chunk_id. + * + * If @a chunk_id falls outside the window, returns nullptr. + */ + chunk *get_chunk(std::uint64_t chunk_id) const + { + if (chunk_id >= head_chunk && chunk_id < tail_chunk) + { + std::size_t pos = chunk_id - head_chunk + head_pos; + const std::size_t max_chunks = chunks.size(); + if (pos >= max_chunks) + pos -= max_chunks; // wrap around the circular storage + return chunks[pos]; + } + else + return nullptr; + } + + /** + * Obtain a pointer to a chunk with ID @a chunk_id. + * + * If @a chunk_id is behind the window, returns nullptr. If it is ahead of + * the window, the window is advanced using @a allocate_chunk and + * @a ready_chunk. If the head_chunk is updated, the new value is passed to + * @a head_updated. + */ + template + chunk *get_chunk( + std::uint64_t chunk_id, std::uintptr_t stream_id, + const F1 &allocate_chunk, const F2 &ready_chunk, const F3 &head_updated) + { + // chunk_id must be a valid int64_t + assert(chunk_id <= std::uint64_t(std::numeric_limits::max())); + const std::size_t max_chunks = chunks.size(); + if (chunk_id >= head_chunk) + { + // We've moved beyond the end of our current window, and need to + // allocate fresh chunks. + if (chunk_id >= tail_chunk && chunk_id - tail_chunk >= max_chunks) + { + /* We've jumped ahead so far that the entire current window + * is stale. Flush it all and fast-forward to the new window. + * We leave it to the while loop below to actually allocate + * the chunks. + */ + flush_all(chunk_id - (max_chunks - 1), ready_chunk, head_updated); + } + while (chunk_id >= tail_chunk) + { + if (tail_chunk - head_chunk == max_chunks) + flush_head(ready_chunk, head_updated); + chunks[tail_pos] = allocate_chunk(tail_chunk); + if (chunks[tail_pos]) + { + chunks[tail_pos]->chunk_id = tail_chunk; + chunks[tail_pos]->stream_id = stream_id; + } + tail_chunk++; + tail_pos++; + if (tail_pos == max_chunks) + tail_pos = 0; // wrap around circular buffer + } + // Find position of chunk within the storage + std::size_t pos = chunk_id - head_chunk + head_pos; + if (pos >= max_chunks) + pos -= max_chunks; // wrap around the circular storage + return chunks[pos]; + } + else + return nullptr; + } + + std::uint64_t get_head_chunk() const { return head_chunk; } + std::uint64_t get_tail_chunk() const { return tail_chunk; } + bool empty() const { return head_chunk == tail_chunk; } +}; + +template class chunk_stream_allocator; + +/// Parts of @ref chunk_stream_state that don't depend on the chunk manager +class chunk_stream_state_base +{ +protected: struct free_place_data { - void operator()(unsigned char *ptr); + void operator()(unsigned char *ptr) const; }; const packet_memcpy_function orig_memcpy; ///< Packet memcpy provided by the user const chunk_stream_config chunk_config; const std::uintptr_t stream_id; const std::size_t base_stat_index; ///< Index of first custom stat - /// Circular buffer of chunks under construction - std::vector> chunks; - std::int64_t head_chunk = 0, tail_chunk = 0; ///< chunk IDs of valid chunk range - std::size_t head_pos = 0, tail_pos = 0; ///< Positions corresponding to @ref head and @ref tail in @ref chunks + + /** + * Circular buffer of chunks under construction. + * + * This class might or might not have exclusive ownership of the chunks, + * depending on the template parameter. + */ + chunk_window chunks; + /** - * Scratch area for use by @ref allocate. This contains not just the @ref - * chunk_place_data, but also the various arrays it points to. They're - * allocated contiguously to minimise the number of cache lines accessed. + * Scratch area for use by @ref chunk_place_function. This contains not + * just the @ref chunk_place_data, but also the various arrays it points + * to. They're allocated contiguously to minimise the number of cache lines + * accessed. */ std::unique_ptr place_data_storage; chunk_place_data *place_data; @@ -232,14 +396,24 @@ class chunk_stream_state void packet_memcpy(const spead2::memory_allocator::pointer &allocation, const packet_header &packet) const; - /// Send the oldest chunk to the ready callback - void flush_head(); + /// Implementation of @ref stream::heap_ready + void do_heap_ready(live_heap &&lh); protected: - std::int64_t get_head_chunk() const { return head_chunk; } - std::int64_t get_tail_chunk() const { return tail_chunk; } + std::uint64_t get_head_chunk() const { return chunks.get_head_chunk(); } + std::uint64_t get_tail_chunk() const { return chunks.get_tail_chunk(); } + bool chunk_too_old(std::int64_t chunk_id) const + { + // Need to check against 0 explicitly to avoid signed/unsigned mixup + return chunk_id < 0 || std::uint64_t(chunk_id) < chunks.get_head_chunk(); + } public: + /// Constructor + chunk_stream_state_base( + const stream_config &config, + const chunk_stream_config &chunk_config); + /** * Structure associated with each heap, as the deleter of the * allocated pointer. @@ -255,12 +429,43 @@ class chunk_stream_state void operator()(std::uint8_t *) const {} }; - /// Constructor - chunk_stream_state(const stream_config &config, const chunk_stream_config &chunk_config); - /// Get the stream's chunk configuration const chunk_stream_config &get_chunk_config() const { return chunk_config; } + /** + * Get the metadata associated with a heap payload pointer. If the pointer + * was not allocated by a chunk stream, returns @c nullptr. + */ + static const heap_metadata *get_heap_metadata(const memory_allocator::pointer &ptr); +}; + +/** + * Base class that holds the internal state of @ref + * spead2::recv::chunk_stream. + * + * This is split into a separate class to avoid some initialisation ordering + * problems: it is constructed before the @ref spead2::recv::stream base class, + * allowing the latter to use function objects that reference this class. + * + * The template parameter allows the policy for allocating and releasing + * chunks to be customised. See @ref chunk_manager_simple for the API. + */ +template +class chunk_stream_state : public chunk_stream_state_base +{ +private: + using chunk_manager_t = CM; + friend chunk_manager_t; + + chunk_manager_t chunk_manager; + +public: + /// Constructor + chunk_stream_state( + const stream_config &config, + const chunk_stream_config &chunk_config, + chunk_manager_t chunk_manager); + /// Compute the config to pass down to @ref spead2::recv::stream. stream_config adjust_config(const stream_config &config); @@ -273,15 +478,19 @@ class chunk_stream_state std::pair allocate( std::size_t size, const packet_header &packet); - /// Send all in-flight chunks to the ready callback + /// Send all in-flight chunks to the ready callback (not thread-safe) void flush_chunks(); +}; - /** - * Get the @ref heap_metadata associated with a heap payload pointer. - * If the pointer was not allocated by a chunk stream, returns @c - * nullptr. - */ - static const heap_metadata *get_heap_metadata(const memory_allocator::pointer &ptr); +class chunk_manager_simple +{ +public: + explicit chunk_manager_simple(const chunk_stream_config &chunk_config); + + std::uint64_t *get_batch_stats(chunk_stream_state &state) const; + chunk *allocate_chunk(chunk_stream_state &state, std::int64_t chunk_id); + void ready_chunk(chunk_stream_state &state, chunk *c); + void head_updated(chunk_stream_state &state, std::uint64_t head_chunk) {} }; /** @@ -289,30 +498,54 @@ class chunk_stream_state * * It forwards allocation requests to @ref chunk_stream_state. */ +template class chunk_stream_allocator final : public memory_allocator { private: - chunk_stream_state &stream; + chunk_stream_state &stream; public: - explicit chunk_stream_allocator(chunk_stream_state &stream); + explicit chunk_stream_allocator(chunk_stream_state &stream); virtual pointer allocate(std::size_t size, void *hint) override; }; +template +chunk_stream_allocator::chunk_stream_allocator(chunk_stream_state &stream) + : stream(stream) +{ +} + +template +memory_allocator::pointer chunk_stream_allocator::allocate(std::size_t size, void *hint) +{ + if (hint) + { + auto alloc = stream.allocate(size, *reinterpret_cast(hint)); + // Use the heap_metadata as the deleter + return pointer(alloc.first, std::move(alloc.second)); + } + // Probably unreachable, but provides a safety net + return memory_allocator::allocate(size, hint); +} + +extern template class chunk_stream_state; +extern template class chunk_stream_allocator; + } // namespace detail /** * Stream that writes incoming heaps into chunks. */ -class chunk_stream : private detail::chunk_stream_state, public stream +class chunk_stream : private detail::chunk_stream_state, public stream { - friend class chunk_stream_state; + friend class detail::chunk_stream_state; + friend class detail::chunk_manager_simple; virtual void heap_ready(live_heap &&) override; public: - using heap_metadata = detail::chunk_stream_state::heap_metadata; + using heap_metadata = detail::chunk_stream_state_base::heap_metadata; /** * Constructor. @@ -338,7 +571,7 @@ class chunk_stream : private detail::chunk_stream_state, public stream * @param config Basic stream configuration * @param chunk_config Configuration for chunking * - * @throw invalid_value if any of the function pointers in @a chunk_config + * @throw invalid_argument if any of the function pointers in @a chunk_config * have not been set. */ chunk_stream( @@ -346,14 +579,60 @@ class chunk_stream : private detail::chunk_stream_state, public stream const stream_config &config, const chunk_stream_config &chunk_config); - using detail::chunk_stream_state::get_chunk_config; - using detail::chunk_stream_state::get_heap_metadata; + using detail::chunk_stream_state_base::get_chunk_config; + using detail::chunk_stream_state_base::get_heap_metadata; virtual void stop_received() override; virtual void stop() override; virtual ~chunk_stream() override; }; +namespace detail +{ + +/// Common functionality between @ref chunk_ring_stream and @ref chunk_stream_ring_group +template>, + typename FreeRingbuffer = ringbuffer>> +class chunk_ring_pair +{ +protected: + const std::shared_ptr data_ring; + const std::shared_ptr free_ring; + /// Temporary stroage for linked list of in-flight chunks while stopping + std::unique_ptr graveyard; + + chunk_ring_pair(std::shared_ptr data_ring, std::shared_ptr free_ring); + +public: + /// Create an allocate function that obtains chunks from the free ring + chunk_allocate_function make_allocate(); + /** + * Create a ready function that pushes chunks to the data ring. + * + * The orig_ready function is called first. + */ + chunk_ready_function make_ready(const chunk_ready_function &orig_ready); + + /** + * Add a chunk to the free ringbuffer. This takes care of zeroing out + * the @ref spead2::recv::chunk::present array, and it will suppress the + * @ref spead2::ringbuffer_stopped error if the free ringbuffer has been + * stopped (in which case the argument will not have been moved from). + * + * If the free ring is full, it will throw @ref spead2::ringbuffer_full + * rather than blocking. The free ringbuffer should be constructed with + * enough slots that this does not happen. + */ + void add_free_chunk(std::unique_ptr &&c); + + /// Retrieve the data ringbuffer passed to the constructor + std::shared_ptr get_data_ringbuffer() const { return data_ring; } + /// Retrieve the free ringbuffer passed to the constructor + std::shared_ptr get_free_ringbuffer() const { return free_ring; } +}; + +} // namespace detail + /** * Wrapper around @ref chunk_stream that uses ringbuffers to manage chunks. * @@ -365,28 +644,16 @@ class chunk_stream : private detail::chunk_stream_state, public stream * * When @ref stop is called, any in-flight chunks (that are not in either * of the ringbuffers) will be freed from the thread that called @ref stop. - * - * It's important to note that the free ring is also stopped if the stream - * is stopped by a stream control item. The user must thus be prepared to - * deal gracefully with a @ref ringbuffer_stopped exception when - * pushing to the free ring. */ template>, typename FreeRingbuffer = ringbuffer>> -class chunk_ring_stream : public chunk_stream +class chunk_ring_stream : public detail::chunk_ring_pair, public chunk_stream { private: - std::shared_ptr data_ring; - std::shared_ptr free_ring; - /// Temporary storage for in-flight chunks during @ref stop - std::vector> graveyard; - /// Create a new @ref spead2::recv::chunk_stream_config that uses the ringbuffers static chunk_stream_config adjust_chunk_config( const chunk_stream_config &chunk_config, - DataRingbuffer &data_ring, - FreeRingbuffer &free_ring, - std::vector> &graveyard); + detail::chunk_ring_pair &ring_pair); public: /** @@ -410,57 +677,175 @@ class chunk_ring_stream : public chunk_stream std::shared_ptr data_ring, std::shared_ptr free_ring); - /** - * Add a chunk to the free ringbuffer. This takes care of zeroing out - * the @ref spead2::recv::chunk::present array, and it will suppress the - * @ref spead2::ringbuffer_stopped error if the free ringbuffer has been - * stopped (in which case the argument will not have been moved from). - * - * If the free ring is full, it will throw @ref spead2::ringbuffer_full - * rather than blocking. The free ringbuffer should be constructed with - * enough slots that this does not happen. - */ - void add_free_chunk(std::unique_ptr &&c); - - /// Retrieve the data ringbuffer passed to the constructor - std::shared_ptr get_data_ringbuffer() const { return data_ring; } - /// Retrieve the free ringbuffer passed to the constructor - std::shared_ptr get_free_ringbuffer() const { return free_ring; } - virtual void stop_received() override; virtual void stop() override; virtual ~chunk_ring_stream(); }; +namespace detail +{ + +template +chunk_stream_state::chunk_stream_state( + const stream_config &config, + const chunk_stream_config &chunk_config, + chunk_manager_t chunk_manager) + : chunk_stream_state_base(config, chunk_config), + chunk_manager(std::move(chunk_manager)) +{ +} + +template +stream_config chunk_stream_state::adjust_config(const stream_config &config) +{ + using namespace std::placeholders; + stream_config new_config = config; + // Unsized heaps won't work with the custom allocator + new_config.set_allow_unsized_heaps(false); + new_config.set_memory_allocator(std::make_shared>(*this)); + // Override the original memcpy with our custom version + new_config.set_memcpy(std::bind(&chunk_stream_state::packet_memcpy, this, _1, _2)); + // Add custom statistics + new_config.add_stat("too_old_heaps"); + new_config.add_stat("rejected_heaps"); + return new_config; +} + +template +void chunk_stream_state::flush_chunks() +{ + chunks.flush_all( + std::numeric_limits::max(), + [this](chunk *c) { chunk_manager.ready_chunk(*this, c); }, + [this](std::uint64_t head_chunk) { chunk_manager.head_updated(*this, head_chunk); } + ); +} + +template +std::pair +chunk_stream_state::allocate(std::size_t size, const packet_header &packet) +{ + // Used to get a non-null pointer + static std::uint8_t dummy_uint8; + + // Keep these in sync with stats added in adjust_config + static constexpr std::size_t too_old_heaps_offset = 0; + static constexpr std::size_t rejected_heaps_offset = 1; + + /* Extract the user's requested items. + * TODO: this could possibly be optimised with a hash table (with a + * perfect hash function chosen in advance), but for the expected + * sizes the overheads will probably outweight the benefits. + */ + const auto &item_ids = get_chunk_config().get_items(); + std::fill(place_data->items, place_data->items + item_ids.size(), -1); + pointer_decoder decoder(packet.heap_address_bits); + /* packet.pointers and packet.n_items skips initial "special" item + * pointers. To allow them to be matched as well, we start from the + * original packet and skip over the 8-byte header. + */ + for (const std::uint8_t *p = packet.packet + 8; p != packet.payload; p += sizeof(item_pointer_t)) + { + item_pointer_t pointer = load_be(p); + if (decoder.is_immediate(pointer)) + { + item_pointer_t id = decoder.get_id(pointer); + for (std::size_t j = 0; j < item_ids.size(); j++) + if (item_ids[j] == id) + place_data->items[j] = decoder.get_immediate(pointer); + } + } + + /* TODO: see if the storage can be in the class with the deleter + * just referencing it. That will avoid the implied memory allocation + * in constructing the std::function underlying the deleter. + */ + std::pair out; + out.first = &dummy_uint8; // Use a non-null value to avoid confusion with empty pointers + heap_metadata &metadata = out.second; + + place_data->packet = packet.packet; + place_data->packet_size = packet.payload + packet.payload_length - packet.packet; + place_data->chunk_id = -1; + place_data->heap_index = 0; + place_data->heap_offset = 0; + place_data->batch_stats = chunk_manager.get_batch_stats(*this); + place_data->extra_offset = 0; + place_data->extra_size = 0; + chunk_config.get_place()(place_data, sizeof(*place_data)); + std::int64_t chunk_id = place_data->chunk_id; + if (chunk_too_old(chunk_id)) + { + // We don't want this heap. + metadata.chunk_id = -1; + metadata.chunk_ptr = nullptr; + std::size_t stat_offset = (chunk_id >= 0) ? too_old_heaps_offset : rejected_heaps_offset; + place_data->batch_stats[base_stat_index + stat_offset]++; + return out; + } + else + { + chunk *chunk_ptr = chunks.get_chunk( + chunk_id, + stream_id, + [this](std::int64_t chunk_id) { return chunk_manager.allocate_chunk(*this, chunk_id); }, + [this](chunk *c) { chunk_manager.ready_chunk(*this, c); }, + [this](std::uint64_t head_chunk) { chunk_manager.head_updated(*this, head_chunk); } + ); + if (chunk_ptr) + { + chunk &c = *chunk_ptr; + out.first = c.data.get() + place_data->heap_offset; + metadata.chunk_id = chunk_id; + metadata.heap_index = place_data->heap_index; + metadata.heap_offset = place_data->heap_offset; + metadata.chunk_ptr = &c; + if (place_data->extra_size > 0) + { + assert(place_data->extra_size <= chunk_config.get_max_heap_extra()); + assert(c.extra); + std::memcpy(c.extra.get() + place_data->extra_offset, place_data->extra, place_data->extra_size); + } + return out; + } + else + { + // the allocator didn't allocate a chunk for this slot. + metadata.chunk_id = -1; + metadata.chunk_ptr = nullptr; + return out; + } + } +} + template -chunk_ring_stream::chunk_ring_stream( - io_service_ref io_service, - const stream_config &config, - const chunk_stream_config &chunk_config, - std::shared_ptr data_ring, - std::shared_ptr free_ring) - : chunk_stream( - io_service, - config, - adjust_chunk_config(chunk_config, *data_ring, *free_ring, graveyard)), - data_ring(std::move(data_ring)), - free_ring(std::move(free_ring)) +chunk_ring_pair::chunk_ring_pair( + std::shared_ptr data_ring, + std::shared_ptr free_ring) + : data_ring(std::move(data_ring)), free_ring(std::move(free_ring)) { - this->data_ring->add_producer(); - // Ensure that we don't run out of memory during shutdown - graveyard.reserve(get_chunk_config().get_max_chunks()); } template -chunk_stream_config chunk_ring_stream::adjust_chunk_config( - const chunk_stream_config &chunk_config, - DataRingbuffer &data_ring, - FreeRingbuffer &free_ring, - std::vector> &graveyard) +void chunk_ring_pair::add_free_chunk(std::unique_ptr &&c) { - chunk_stream_config new_config = chunk_config; - // Set the allocate callback to get a chunk from the free ringbuffer - new_config.set_allocate([&free_ring] (std::int64_t, std::uint64_t *) -> std::unique_ptr { + // Mark all heaps as not yet present + std::memset(c->present.get(), 0, c->present_size); + try + { + free_ring->try_push(std::move(c)); + } + catch (spead2::ringbuffer_stopped &) + { + // Suppress the error + } +} + +template +chunk_allocate_function chunk_ring_pair::make_allocate() +{ + FreeRingbuffer &free_ring = *this->free_ring; + return [&free_ring] (std::int64_t, std::uint64_t *) -> std::unique_ptr { try { return free_ring.pop(); @@ -471,12 +856,17 @@ chunk_stream_config chunk_ring_stream::adjust_ch // ignore this chunk return nullptr; } - }); - // Set the ready callback to push chunks to the data ringbuffer - auto orig_ready = chunk_config.get_ready(); - new_config.set_ready( - [&data_ring, &graveyard, orig_ready] (std::unique_ptr &&c, - std::uint64_t *batch_stats) { + }; +} + +template +chunk_ready_function chunk_ring_pair::make_ready( + const chunk_ready_function &orig_ready) +{ + DataRingbuffer &data_ring = *this->data_ring; + std::unique_ptr &graveyard = this->graveyard; + return [&data_ring, &graveyard, orig_ready] (std::unique_ptr &&c, + std::uint64_t *batch_stats) { try { if (orig_ready) @@ -488,32 +878,50 @@ chunk_stream_config chunk_ring_stream::adjust_ch { // Suppress the error, move the chunk to the graveyard log_info("dropped chunk %d due to external stop", c->chunk_id); - graveyard.push_back(std::move(c)); + assert(!c->graveyard_next); // chunk should not already be in a linked list + c->graveyard_next = std::move(graveyard); + graveyard = std::move(c); } - }); - return new_config; + }; } +} // namespace detail + template -void chunk_ring_stream::add_free_chunk(std::unique_ptr &&c) +chunk_ring_stream::chunk_ring_stream( + io_service_ref io_service, + const stream_config &config, + const chunk_stream_config &chunk_config, + std::shared_ptr data_ring, + std::shared_ptr free_ring) + : detail::chunk_ring_pair(std::move(data_ring), std::move(free_ring)), + chunk_stream( + io_service, + config, + adjust_chunk_config(chunk_config, *this)) { - // Mark all heaps as not yet present - std::memset(c->present.get(), 0, c->present_size); - try - { - free_ring->try_push(std::move(c)); - } - catch (spead2::ringbuffer_stopped &) - { - // Suppress the error - } + this->data_ring->add_producer(); +} + +template +chunk_stream_config chunk_ring_stream::adjust_chunk_config( + const chunk_stream_config &chunk_config, + detail::chunk_ring_pair &ring_pair) +{ + chunk_stream_config new_config = chunk_config; + // Set the allocate callback to get a chunk from the free ringbuffer + new_config.set_allocate(ring_pair.make_allocate()); + // Set the ready callback to push chunks to the data ringbuffer + auto orig_ready = chunk_config.get_ready(); + new_config.set_ready(ring_pair.make_ready(chunk_config.get_ready())); + return new_config; } template void chunk_ring_stream::stop_received() { chunk_stream::stop_received(); - data_ring->remove_producer(); + this->data_ring->remove_producer(); } template @@ -521,12 +929,14 @@ void chunk_ring_stream::stop() { // Stop the ringbuffers first, so that if the calling code is no longer // servicing them it will not lead to a deadlock as we flush. - free_ring->stop(); - data_ring->stop(); // NB: NOT remove_producer as that might not break a deadlock + this->free_ring->stop(); + this->data_ring->stop(); // NB: NOT remove_producer as that might not break a deadlock chunk_stream::stop(); { - std::lock_guard lock(queue_mutex); - graveyard.clear(); // free chunks that didn't make it into data_ring + // Locking is probably not needed, as all readers are terminated by + // chunk_stream::stop(). But it should be safe. + std::lock_guard lock(get_queue_mutex()); + this->graveyard.reset(); // free chunks that didn't make it into data_ring } } diff --git a/include/spead2/recv_chunk_stream_group.h b/include/spead2/recv_chunk_stream_group.h new file mode 100644 index 000000000..50e8db9ee --- /dev/null +++ b/include/spead2/recv_chunk_stream_group.h @@ -0,0 +1,472 @@ +/* Copyright 2023 National Research Foundation (SARAO) + * + * This program is free software: you can redistribute it and/or modify it under + * the terms of the GNU Lesser General Public License as published by the Free + * Software Foundation, either version 3 of the License, or (at your option) any + * later version. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS + * FOR A PARTICULAR PURPOSE. See the GNU Lesser General Public License for more + * details. + * + * You should have received a copy of the GNU Lesser General Public License + * along with this program. If not, see . + */ + +/** + * @file + */ + +#ifndef SPEAD2_RECV_CHUNK_STREAM_GROUP +#define SPEAD2_RECV_CHUNK_STREAM_GROUP + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace spead2 +{ +namespace recv +{ + +/// Configuration for chunk_stream_group +class chunk_stream_group_config +{ +public: + /// Default value for @ref set_max_chunks + static constexpr std::size_t default_max_chunks = chunk_stream_config::default_max_chunks; + + /** + * Eviction mode when it is necessary to advance the group window. See the + * @verbatim embed:rst:inline :doc:`overview ` @endverbatim + * for more details. + */ + enum class eviction_mode + { + LOSSY, ///< force streams to release incomplete chunks + LOSSLESS ///< a chunk will only be marked ready when all streams have marked it ready + }; + +private: + std::size_t max_chunks = default_max_chunks; + eviction_mode eviction_mode_ = eviction_mode::LOSSY; + chunk_allocate_function allocate; + chunk_ready_function ready; + +public: + /** + * Set the maximum number of chunks that can be live at the same time. + * A value of 1 means that heaps must be received in order: once a + * chunk is started, no heaps from a previous chunk will be accepted. + * + * @throw std::invalid_argument if @a max_chunks is 0. + */ + chunk_stream_group_config &set_max_chunks(std::size_t max_chunks); + /// Return the maximum number of chunks that can be live at the same time. + std::size_t get_max_chunks() const { return max_chunks; } + + /// Set chunk eviction mode. See @ref eviction_mode. + chunk_stream_group_config &set_eviction_mode(eviction_mode eviction_mode_); + /// Return the current eviction mode + eviction_mode get_eviction_mode() const { return eviction_mode_; } + + /// Set the function used to allocate a chunk. + chunk_stream_group_config &set_allocate(chunk_allocate_function allocate); + /// Get the function used to allocate a chunk. + const chunk_allocate_function &get_allocate() const { return allocate; } + + /// Set the function that is provided with completed chunks. + chunk_stream_group_config &set_ready(chunk_ready_function ready); + /// Get the function that is provided with completed chunks. + const chunk_ready_function &get_ready() const { return ready; } +}; + +class chunk_stream_group; + +namespace detail +{ + +class chunk_manager_group +{ +private: + chunk_stream_group &group; + +public: + explicit chunk_manager_group(chunk_stream_group &group); + + std::uint64_t *get_batch_stats(chunk_stream_state &state) const; + chunk *allocate_chunk(chunk_stream_state &state, std::int64_t chunk_id); + void ready_chunk(chunk_stream_state &state, chunk *c) {} + void head_updated(chunk_stream_state &state, std::uint64_t head_chunk); +}; + +} // namespace detail + +class chunk_stream_group_member; + +/** + * A holder for a collection of streams that share chunks. The group owns the + * component streams, and takes care of stopping and destroying them when the + * group is stopped or destroyed. + * + * It presents an interface similar to @c std::vector for observing the set + * of attached streams. + * + * The public interface must only be called from one thread at a time, and + * all streams must be added before any readers are attached to them. + */ +class chunk_stream_group +{ +private: + friend class detail::chunk_manager_group; + friend class chunk_stream_group_member; + + const chunk_stream_group_config config; + + std::mutex mutex; ///< Protects all the mutable state + /// Notified when the reference count of a chunk reaches zero + std::condition_variable ready_condition; + + /** + * Circular buffer of chunks under construction. + * + * Ownership of the chunks is shared between the group and the member + * streams, but reference counting is manual (rather than using + * std::shared_ptr) so that the reference count can be embedded in the + * object, and to facilitate code sharing with @ref chunk_stream. + */ + detail::chunk_window chunks; + + /** + * The component streams. + * + * This is protected by the mutex, except that read-only access is always + * permitted in methods called by the user. This is safe because writes + * only happen in methods called by the user (@ref emplace_back), and the + * user is required to serialise their calls. + */ + std::vector> streams; + + /** + * Copy of the head chunk ID from each stream. This copy is protected by + * the group's mutex rather than the streams'. + * + * The minimum element must always be equal to @c chunks.get_head_chunk(). + */ + std::vector head_chunks; + + /** + * Last value passed to all streams' async_flush_until. + */ + std::uint64_t last_flush_until = 0; + + /** + * Obtain the chunk with a given ID. + * + * This will shift the window if the chunk_id is beyond the tail. If the + * chunk is too old, it will return @c nullptr. The reference count of the + * returned chunk will be incremented. + * + * This function is thread-safe. + */ + chunk *get_chunk(std::uint64_t chunk_id, std::uintptr_t stream_id, std::uint64_t *batch_stats); + + /** + * Called by a stream to report movement in its head pointer. This function + * takes the group mutex. + */ + void stream_head_updated(chunk_stream_group_member &s, std::uint64_t head_chunk); + + /** + * Pass a chunk to the user-provided ready function. The caller is + * responsible for ensuring that the chunk is no longer in use. + * + * The caller must hold the group mutex. + */ + void ready_chunk(chunk *c, std::uint64_t *batch_stats); + + // Helper classes for implementing iterators + template + class dereference + { + public: + decltype(*std::declval()) operator()(const T &ptr) const { return *ptr; } + }; + + template + class dereference_const + { + public: + const decltype(*std::declval()) operator()(const T &ptr) const { return *ptr; } + }; + +protected: + /** + * Called by @ref emplace_back for newly-constructed streams. The group's + * mutex is held when this is called. + */ + virtual void stream_added(chunk_stream_group_member &s) {} + /** + * Called when a stream stops (whether from the network or the user). + * + * The stream's @c queue_mutex is locked when this is called. + */ + virtual void stream_stop_received(chunk_stream_group_member &s) {} + +public: + using iterator = boost::transform_iterator< + dereference>, + std::vector>::iterator + >; + using const_iterator = boost::transform_iterator< + dereference_const>, + std::vector>::const_iterator + >; + + explicit chunk_stream_group(const chunk_stream_group_config &config); + virtual ~chunk_stream_group(); + + const chunk_stream_group_config &get_config() const { return config; } + + /// Add a new stream + chunk_stream_group_member &emplace_back( + io_service_ref io_service, + const stream_config &config, + const chunk_stream_config &chunk_config); + + /// Add a new stream, possibly of a subclass + template + T &emplace_back(Args&&... args); + + /** + * @name Vector-like access to the streams. + * Iterator invalidation rules are the same as for @c std::vector i.e., + * modifying the set of streams invalidates iterators. + * @{ + */ + /// Number of streams + std::size_t size() const { return streams.size(); } + /// Whether there are any streams + bool empty() const { return streams.empty(); } + /// Get the stream at a given index + chunk_stream_group_member &operator[](std::size_t index) { return *streams[index]; } + /// Get the stream at a given index + const chunk_stream_group_member &operator[](std::size_t index) const { return *streams[index]; } + /// Get an iterator to the first stream + iterator begin() noexcept; + /// Get an iterator past the last stream + iterator end() noexcept; + /// Get an iterator to the first stream + const_iterator begin() const noexcept; + /// Get a const iterator past the last stream + const_iterator end() const noexcept; + /// Get an iterator to the first stream + const_iterator cbegin() const noexcept; + /// Get a const iterator past the last stream + const_iterator cend() const noexcept; + /** + * @} + */ + + /// Stop all streams and release all chunks. + virtual void stop(); +}; + +/** + * Single single within a group managed by @ref chunk_stream_group. + */ +class chunk_stream_group_member : private detail::chunk_stream_state, public stream +{ + friend class detail::chunk_manager_group; + friend class chunk_stream_group; + +private: + chunk_stream_group &group; // TODO: redundant - also stored inside the manager + const std::size_t group_index; ///< Position of the chunk within the group + + virtual void heap_ready(live_heap &&) override; + + /** + * Flush all chunks with an ID strictly less than @a chunk_id. + * + * This function returns immediately, and the work is done later on the + * io_service. It is safe to call from any thread. + */ + void async_flush_until(std::uint64_t chunk_id); + +protected: + /** + * Constructor. + * + * This class passes a modified @a config to the base class constructor. + * See @ref chunk_stream for more information. + * + * The @link chunk_stream_config::set_allocate allocate@endlink and + * @link chunk_stream_config::set_ready ready@endlink callbacks are + * ignored, and the group's callbacks are used instead. + * + * @param group Group to which this stream belongs + * @param group_index Position of this stream within the group + * @param io_service I/O service (also used by the readers). + * @param config Basic stream configuration + * @param chunk_config Configuration for chunking + * + * @throw invalid_argument if the place function pointer in @a chunk_config + * has not been set. + */ + chunk_stream_group_member( + chunk_stream_group &group, + std::size_t group_index, + io_service_ref io_service, + const stream_config &config, + const chunk_stream_config &chunk_config); + + /** + * Stop just this stream. This does the real work of stopping the stream, + * whereas the public @ref stop function stops the entire group. + * + * This should only be called from @ref chunk_stream_group::stop. + */ + virtual void stop1(); + +public: + using heap_metadata = detail::chunk_stream_state_base::heap_metadata; + + using detail::chunk_stream_state_base::get_chunk_config; + using detail::chunk_stream_state_base::get_heap_metadata; + + virtual void stop_received() override; + virtual void stop() override; + /* Note: most stream classes have a destructor that calls stop(), + * but that's not required nor safe for this class: stop() calls + * group.stop(), but the stream is only destroyed as part of destroying + * the group. Instead, the group's destructor ensures that stop1 is + * called. + */ +}; + +/** + * Wrapper around @ref chunk_stream_group that uses ringbuffers to manage + * chunks. + * + * When a fresh chunk is needed, it is retrieved from a ringbuffer of free + * chunks (the "free ring"). When a chunk is flushed, it is pushed to a + * "data ring". These may be shared between groups, but both will be + * stopped as soon as any of the members streams are stopped. The intended use + * case is parallel groups that are started and stopped together. + * + * When the group is stopped, the ringbuffers are both stopped, and readied + * chunks are diverted into a graveyard. The graveyard is then emptied from + * the thread calling @ref stop. This makes it safe to use chunks that can only + * safely be freed from the caller's thread (e.g. a Python thread holding the + * GIL). + */ +template>, + typename FreeRingbuffer = ringbuffer>> +class chunk_stream_ring_group +: public detail::chunk_ring_pair, public chunk_stream_group +{ +private: + /// Create a new @ref chunk_stream_group_config that uses the ringbuffers + static chunk_stream_group_config adjust_group_config( + const chunk_stream_group_config &config, + detail::chunk_ring_pair &ring_pair); + +protected: + virtual void stream_added(chunk_stream_group_member &s) override; + virtual void stream_stop_received(chunk_stream_group_member &s) override; + +public: + chunk_stream_ring_group( + const chunk_stream_group_config &group_config, + std::shared_ptr data_ring, + std::shared_ptr free_ring); + virtual void stop() override; + + ~chunk_stream_ring_group(); +}; + + +template +T &chunk_stream_group::emplace_back(Args&&... args) +{ + std::lock_guard lock(mutex); + if (chunks.get_tail_chunk() != 0 || last_flush_until != 0) + { + throw std::runtime_error("Cannot add a stream after group has started receiving data"); + } + std::unique_ptr stream(new T( + *this, streams.size(), std::forward(args)...)); + chunk_stream_group_member &ret = *stream; + streams.push_back(std::move(stream)); + head_chunks.push_back(0); + stream_added(ret); + return ret; +} + +template +chunk_stream_ring_group::chunk_stream_ring_group( + const chunk_stream_group_config &group_config, + std::shared_ptr data_ring, + std::shared_ptr free_ring) + : detail::chunk_ring_pair(std::move(data_ring), std::move(free_ring)), + chunk_stream_group(adjust_group_config(group_config, *this)) +{ +} + +template +chunk_stream_group_config chunk_stream_ring_group::adjust_group_config( + const chunk_stream_group_config &config, + detail::chunk_ring_pair &ring_pair) +{ + chunk_stream_group_config new_config = config; + new_config.set_allocate(ring_pair.make_allocate()); + new_config.set_ready(ring_pair.make_ready(config.get_ready())); + return new_config; +} + +template +void chunk_stream_ring_group::stream_added( + chunk_stream_group_member &s) +{ + chunk_stream_group::stream_added(s); + this->data_ring->add_producer(); +} + +template +void chunk_stream_ring_group::stream_stop_received( + chunk_stream_group_member &s) +{ + chunk_stream_group::stream_stop_received(s); + this->data_ring->remove_producer(); +} + +template +void chunk_stream_ring_group::stop() +{ + // Shut down the rings so that if the caller is no longer servicing them, it will + // not lead to a deadlock during shutdown. + this->data_ring->stop(); + this->free_ring->stop(); + chunk_stream_group::stop(); + this->graveyard.reset(); // Release chunks from the graveyard +} + +template +chunk_stream_ring_group::~chunk_stream_ring_group() +{ + stop(); +} + +} // namespace recv +} // namespace spead2 + +#endif // SPEAD2_RECV_CHUNK_STREAM_GROUP diff --git a/include/spead2/recv_inproc.h b/include/spead2/recv_inproc.h index b98d2e1f6..a44af7ae1 100644 --- a/include/spead2/recv_inproc.h +++ b/include/spead2/recv_inproc.h @@ -1,4 +1,4 @@ -/* Copyright 2018 National Research Foundation (SARAO) +/* Copyright 2018, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -24,7 +24,6 @@ #include #include #include -#include #include namespace spead2 @@ -43,8 +42,12 @@ class inproc_reader : public reader void process_one_packet(stream_base::add_packet_state &state, const inproc_queue::packet &packet); - void packet_handler(const boost::system::error_code &error, std::size_t bytes_received); - void enqueue(); + void packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, + const boost::system::error_code &error, + std::size_t bytes_received); + void enqueue(handler_context ctx); public: /// Constructor. @@ -52,7 +55,6 @@ class inproc_reader : public reader stream &owner, std::shared_ptr queue); - virtual void stop() override; virtual bool lossy() const override; }; diff --git a/include/spead2/recv_mem.h b/include/spead2/recv_mem.h index d3bb1f287..03c84d633 100644 --- a/include/spead2/recv_mem.h +++ b/include/spead2/recv_mem.h @@ -1,4 +1,4 @@ -/* Copyright 2015 National Research Foundation (SARAO) +/* Copyright 2015, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -22,7 +22,7 @@ #define SPEAD2_RECV_MEM_H #include -#include +#include namespace spead2 { @@ -51,7 +51,6 @@ class mem_reader : public reader mem_reader(stream &owner, const std::uint8_t *ptr, std::size_t length); - virtual void stop() override {} virtual bool lossy() const override; }; diff --git a/include/spead2/recv_reader.h b/include/spead2/recv_reader.h index a03dd7608..22259d0a8 100644 --- a/include/spead2/recv_reader.h +++ b/include/spead2/recv_reader.h @@ -1,4 +1,4 @@ -/* Copyright 2015, 2019 National Research Foundation (SARAO) +/* Copyright 2015, 2019, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -16,96 +16,14 @@ /** * @file + * + * This file exists purely for backwards compatibility. The code has moved to + * recv_stream.h. */ #ifndef SPEAD2_RECV_READER_H #define SPEAD2_RECV_READER_H -#include -#include -#include - -namespace spead2 -{ -namespace recv -{ - -class stream; -class stream_base; - -/** - * Abstract base class for asynchronously reading data and passing it into - * a stream. Subclasses will usually override @ref stop. - * - * The lifecycle of a reader is: - * - construction - * - @ref stop (called with @ref stream_base::queue_mutex held) - * - destruction - * - * All of the above occur with @ref stream::reader_mutex held. - * - * Once the reader has completed its work (whether because @ref stop was called or - * because of network input), it must call @ref stopped to indicate that - * it can be safely destroyed. - */ -class reader -{ -private: - stream &owner; ///< Owning stream - -protected: - /// Called by last completion handler - void stopped(); - -public: - explicit reader(stream &owner) : owner(owner) {} - virtual ~reader() = default; - - /// Retrieve the wrapped stream - stream &get_stream() const { return owner; } - - /** - * Retrieve the wrapped stream's base class. This is normally only used - * to construct a @ref stream_base::add_packet_state. - */ - stream_base &get_stream_base() const; - - /// Retrieve the @c io_service corresponding to the owner - boost::asio::io_service &get_io_service(); - - /** - * Cancel any pending asynchronous operations. This is called with the - * owner's queue_mutex and reader_mutex held. This function does not need - * to wait for completion handlers to run, but it must schedule a call to - * @ref stopped. - */ - virtual void stop() = 0; - - /** - * Whether the reader risks losing data if it is not given a chance to - * run (true by default). This is used to control whether a warning - * should be given when the consumer is applying back-pressure. - */ - virtual bool lossy() const; -}; - -/** - * Factory for creating a new reader. This is used by @ref - * stream::emplace_reader to create the reader. The default implementation - * simply chains to the constructor, but it can be overloaded in cases where - * it is desirable to select the class dynamically. - */ -template -struct reader_factory -{ - template - static std::unique_ptr make_reader(Args&&... args) - { - return std::unique_ptr(new Reader(std::forward(args)...)); - } -}; - -} // namespace recv -} // namespace spead2 +#include #endif // SPEAD2_RECV_READER_H diff --git a/include/spead2/recv_stream.h b/include/spead2/recv_stream.h index e8dd4cbc5..554778ef1 100644 --- a/include/spead2/recv_stream.h +++ b/include/spead2/recv_stream.h @@ -1,4 +1,4 @@ -/* Copyright 2015, 2017-2021 National Research Foundation (SARAO) +/* Copyright 2015, 2017-2021, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -38,7 +38,6 @@ #include #include #include -#include #include #include #include @@ -53,6 +52,7 @@ namespace recv { struct packet_header; +class stream; /// Registration information about a statistic counter. class stream_stat_config @@ -184,7 +184,7 @@ class stream_stats_iterator : public boost::iterator_facade< class stream_stats { private: - std::shared_ptr> config; + std::shared_ptr> config; std::vector values; public: @@ -206,9 +206,9 @@ class stream_stats /// Construct with the default set of statistics, and all zero values stream_stats(); /// Construct with all zero values - explicit stream_stats(std::shared_ptr> config); + explicit stream_stats(std::shared_ptr> config); /// Construct with provided values - stream_stats(std::shared_ptr> config, + stream_stats(std::shared_ptr> config, std::vector values); /* Copy constructor and copy assignment need to be implemented manually @@ -357,8 +357,15 @@ class stream_config bool allow_out_of_order = false; /// A user-defined identifier for a stream std::uintptr_t stream_id = 0; - /// Statistics (includes the built-in ones) - std::shared_ptr> stats; + /** Statistics (includes the built-in ones) + * + * This is a shared_ptr so that instances of @ref stream_stats can share + * it. Every modification creates a new vector (copy-on-write). This is + * potentially very inefficient, since it creates a copy even when there + * are no sharers, but there are not expected to be huge numbers of + * statistics. + */ + std::shared_ptr> stats; public: stream_config(); @@ -456,6 +463,8 @@ class stream_config std::size_t next_stat_index() const { return stats->size(); } }; +class stream_base; + /** * Encapsulation of a SPEAD stream. Packets are fed in through @ref add_packet. * The base class does nothing with heaps; subclasses will typically override @@ -501,8 +510,8 @@ class stream_config * Avoiding deadlocks requires a careful design with several mutexes. It's * governed by the requirement that @ref heap_ready may block indefinitely, and * this must not block other functions. Thus, several mutexes are involved: - * - @ref queue_mutex: protects values only used by @ref add_packet. This - * may be locked for long periods. + * - @ref shared_state::queue_mutex: protects values only used + * by @ref add_packet. This may be locked for long periods. * - @ref stats_mutex: protects stream statistics, and is mostly locked for * writes (assuming the user is only occasionally checking the stats). * @@ -513,6 +522,7 @@ class stream_config */ class stream_base { + friend class reader; public: struct add_packet_state; @@ -560,18 +570,36 @@ class stream_base /// Stream configuration const stream_config config; -protected: +private: + struct shared_state + { + /** + * Mutex protecting the state of the queue. This includes + * - @ref queue_storage + * - @ref buckets + * - @ref head + * - @ref stopped + * + * Subclasses may use it to protect additional state. It is guaranteed to + * be locked when @ref heap_ready is called. + */ + mutable std::mutex queue_mutex; + + /** + * Pointer back to the owning stream. This is set to @c nullptr + * when the stream is stopped. + */ + stream_base *self; + + explicit shared_state(stream_base *self) : self(self) {} + }; + /** - * Mutex protecting the state of the queue. This includes - * - @ref queue_storage - * - @ref buckets - * - @ref head - * - @ref stopped - * - * Subclasses may use it to protect additional state. It is guaranteed to - * be locked when @ref heap_ready is called. + * State that is indirectly held via @c std::shared_ptr. The indirection + * allows readers to have access to the mutex in a way that won't + * unexpectedly vanish from under them. */ - mutable std::mutex queue_mutex; + std::shared_ptr shared; private: /// @ref stop_received has been called, either externally or by stream control @@ -596,15 +624,16 @@ class stream_base /** * Callback called when a heap is being ejected from the live list. - * The heap might or might not be complete. The @ref queue_mutex will be + * The heap might or might not be complete. The + * @ref spead2::recv::stream_base::shared_state::queue_mutex will be * locked during this call, which will block @ref stop and @ref flush. */ virtual void heap_ready(live_heap &&) {} - /// Implementation of @ref flush that assumes the caller has locked @ref queue_mutex + /// Implementation of @ref flush that assumes the caller has locked @ref shared_state::queue_mutex void flush_unlocked(); - /// Implementation of @ref stop that assumes the caller has locked @ref queue_mutex + /// Implementation of @ref stop that assumes the caller has locked @ref shared_state::queue_mutex void stop_unlocked(); /// Implementation of @ref add_packet_state::add_packet @@ -632,20 +661,73 @@ class stream_base * It is undefined what happens if @ref add_packet is called after a stream * is stopped. * - * This is called with @ref queue_mutex locked. Users must not call this - * function themselves; instead, call @ref stop. + * This is called with @ref spead2::recv::stream_base::shared_state::queue_mutex + * locked. Users must not call this function themselves; instead, call @ref + * stop. */ virtual void stop_received(); + std::mutex &get_queue_mutex() const { return shared->queue_mutex; } + + /** + * Schedule a function to be called on an executor, with the lock held. + * This is a fire-and-forget operation. If the stream is stopped before the + * callback fires, the callback is silently ignored. + */ + template + void post(ExecutionContext &ex, F &&func) + { + class wrapper + { + private: + std::shared_ptr shared; + typename std::decay::type func; + + public: + wrapper(std::shared_ptr shared, F&& func) + : shared(std::move(shared)), func(std::forward(func)) + { + } + + /* Prevent copying, while allowing moving (copying is safe but inefficient) + * Move assignment is not implemented because it fails to compile if + * F is not move-assignable. This can probably be solved with + * std::enable_if, but it doesn't seem worth the effort. + */ + wrapper(const wrapper &) = delete; + wrapper &operator=(const wrapper &) = delete; + wrapper(wrapper &&) = default; + + void operator()() const + { + std::lock_guard lock(shared->queue_mutex); + stream_base *self = shared->self; + if (self) + func(*self); + } + }; + + // TODO: can do this with a lambda (with perfect forwarding) in C++14 + boost::asio::post(ex, wrapper(shared, std::forward(func))); + } + public: /** * State for a batch of calls to @ref add_packet. Constructing this object - * locks the stream's @ref queue_mutex. + * locks the stream's @ref shared_state::queue_mutex. + * + * After constructing this object, one *must* check whether @ref owner is + * null. If so, do not call any methods except for @ref stop and + * @ref is_stopped. + * + * While this object is alive, one must also keep alive a + * @c std::shared_ptr to the @ref shared_state. */ struct add_packet_state { - stream_base &owner; - std::lock_guard lock; ///< Holds a lock on the owner's @ref queue_mutex + /// Holds a lock on the owner's @ref shared_state::queue_mutex + std::lock_guard lock; + stream_base *owner; // Updates to the statistics std::uint64_t packets = 0; @@ -654,12 +736,13 @@ class stream_base std::uint64_t single_packet_heaps = 0; std::uint64_t search_dist = 0; - explicit add_packet_state(stream_base &owner); + explicit add_packet_state(shared_state &owner); + explicit add_packet_state(stream_base &s) : add_packet_state(*s.shared) {} ~add_packet_state(); - bool is_stopped() const { return owner.stopped; } + bool is_stopped() const { return owner == nullptr || owner->stopped; } /// Indicate that the stream has stopped (e.g. because the remote peer disconnected) - void stop() { owner.stop_unlocked(); } + void stop() { if (owner) owner->stop_unlocked(); } /** * Add a packet that was received, and which has been examined by @ref * decode_packet, and returns @c true if it is consumed. Even though @ref @@ -667,8 +750,15 @@ class stream_base * by @ref live_heap::add_packet e.g., because it is a duplicate. * * It is an error to call this after the stream has been stopped. + * + * Calling this function may cause the readers to be destroyed, + * including the reader that is calling this function. */ - bool add_packet(const packet_header &packet) { return owner.add_packet(*this, packet); } + bool add_packet(const packet_header &packet) + { + assert(!is_stopped()); + return owner->add_packet(*this, packet); + } }; /** @@ -696,6 +786,155 @@ class stream_base stream_stats get_stats() const; }; +/** + * Abstract base class for asynchronously reading data and passing it into + * a stream. Subclasses will usually override @ref stop. + * + * The lifecycle of a reader is: + * - The reader mutex is taken + * - construction + * - The queue mutex is taken + * - the stream stops + * - the reader mutex is tken + * - destruction + * - the stream is destroyed + * + * Destruction must ensure that any pending asynchronous operations are + * handled. Since destruction may happen on a separate thread to the one + * running in-flight handlers, care must be taken not to access the stream or + * the reader after the stream is stopped. In many cases this can be + * facilitated using @ref bind_handler, although it is still important to + * re-check whether the stream has stopped after calling + * @ref stream_base::add_packet_state::add_packet. + */ +class reader +{ +private: + boost::asio::io_service &io_service; + std::shared_ptr owner; ///< Access to owning stream + +protected: + class handler_context + { + friend class reader; + private: + std::shared_ptr owner; + + public: + explicit handler_context(std::shared_ptr owner) + : owner(std::move(owner)) + { + assert(this->owner); + } + + // Whether the context is still valid + explicit operator bool() const noexcept { return bool(owner); } + bool operator!() const noexcept { return !owner; } + + /* Prevent copy construction and assignment. They're perfectly safe, + * but potentially slow (atomic reference count manipulation) so + * they're disabled to prevent them being used by accident. + */ + handler_context(handler_context &) = delete; + handler_context &operator=(handler_context &) = delete; + handler_context(handler_context &&) = default; + handler_context &operator=(handler_context &&) = default; + }; + + template + class bound_handler + { + private: + handler_context ctx; + T orig; + + public: + template + bound_handler(handler_context ctx, U &&orig) + : ctx(std::move(ctx)), orig(std::forward(orig)) + { + } + + template + void operator()(Args&&... args) + { + // Note: because we give away our shared pointer, this can only be + // called once. Fortunately, asio makes that guarantee. + assert(ctx); + stream_base::add_packet_state state(*ctx.owner); + if (!state.is_stopped()) + orig(std::move(ctx), state, std::forward(args)...); + } + }; + + handler_context make_handler_context() const + { + return handler_context(owner); + } + + /** + * Wrap a function object to manage locking and lifetime. This is intended + * to be used to bind a completion handler. The wrapper handler is called + * with extra arguments prefixed, so it should have the signature + * void handler(handler_context ctx, stream_base::add_packet_state &state, ...); + * + * The @ref reader::handler_context can be passed (by rvalue + * reference) to a single call to @ref bind_handler, which is cheaper + * than the overload that doesn't take it (it avoids manipulating reference + * counts on a @c std::shared_ptr). + * + * At the time the wrapped handler is invoked, the stream is guaranteed to still + * exist and not yet have been stopped. After calling + * @ref stream_base::add_packet_state::add_packet one must again check whether + * the stream has been stopped, as this can cause the reader to be destroyed. + */ + template + bound_handler::type> bind_handler(T &&handler) const + { + return bind_handler(make_handler_context(), std::forward(handler)); + } + + /** + * Overload that takes an existing @ref reader::handler_context. + */ + template + bound_handler::type> bind_handler(handler_context ctx, T &&handler) const + { + assert(ctx); // make sure it hasn't already been used + return bound_handler::type>(std::move(ctx), std::forward(handler)); + } + +public: + explicit reader(stream &owner); + virtual ~reader() = default; + + /// Retrieve the @c io_service corresponding to the owner + boost::asio::io_service &get_io_service() { return io_service; } + + /** + * Whether the reader risks losing data if it is not given a chance to + * run (true by default). This is used to control whether a warning + * should be given when the consumer is applying back-pressure. + */ + virtual bool lossy() const; +}; + +/** + * Factory for creating a new reader. This is used by @ref + * stream::emplace_reader to create the reader. The default implementation + * simply chains to the constructor, but it can be overloaded in cases where + * it is desirable to select the class dynamically. + */ +template +struct reader_factory +{ + template + static std::unique_ptr make_reader(Args&&... args) + { + return std::unique_ptr(new Reader(std::forward(args)...)); + } +}; + /** * Stream that is fed by subclasses of @ref reader. * @@ -745,6 +984,19 @@ class stream : protected stream_base /// Actual implementation of @ref stop void stop_impl(); + using stream_base::post; // Make base class version visible, despite being overloaded + + /** + * Schedule a function to be called on the stream's io_service, with the + * lock held. This is a fire-and-forget operation. If the stream is stopped + * before the callback fires, the callback is silently dropped. + */ + template + void post(F &&func) + { + post(get_io_service(), std::forward(func)); + } + public: using stream_base::get_config; using stream_base::get_stats; @@ -756,13 +1008,13 @@ class stream : protected stream_base /** * Add a new reader by passing its constructor arguments, excluding - * the initial @a stream argument. + * the initial @a io_service and @a owner arguments. */ template void emplace_reader(Args&&... args) { std::lock_guard lock(reader_mutex); - // See comments in stop_impl for why we do this check + // See comments in stop_received for why we do this check if (!stop_readers) { // Guarantee space before constructing the reader @@ -776,9 +1028,9 @@ class stream : protected stream_base } /** - * Stop the stream and block until all the readers have wound up. After - * calling this there should be no more outstanding completion handlers - * in the thread pool. + * Stop the stream. After this returns, the io_service may still have + * outstanding completion handlers, but they should be no-ops when they're + * called. * * In most cases subclasses should override @ref stop_received rather than * this function. However, if @ref heap_ready can block indefinitely, this @@ -790,6 +1042,16 @@ class stream : protected stream_base bool is_lossy() const; }; +/** + * Push packets found in a block of memory to a stream. Returns a pointer to + * after the last packet found in the stream. Processing stops as soon as + * after @ref decode_packet fails (because there is no way to find the next + * packet after a corrupt one), but packets may still be rejected by the stream. + * + * The stream is @em not stopped. + */ +const std::uint8_t *mem_to_stream(stream_base::add_packet_state &state, const std::uint8_t *ptr, std::size_t length); + /** * Push packets found in a block of memory to a stream. Returns a pointer to * after the last packet found in the stream. Processing stops as soon as diff --git a/include/spead2/recv_tcp.h b/include/spead2/recv_tcp.h index 38eb685b6..b1167504c 100644 --- a/include/spead2/recv_tcp.h +++ b/include/spead2/recv_tcp.h @@ -29,7 +29,6 @@ #include #include #include -#include #include #include @@ -44,10 +43,13 @@ namespace recv class tcp_reader : public reader { private: - /// The acceptor object - boost::asio::ip::tcp::acceptor acceptor; - /// TCP peer socket (i.e., the one connected to the remote end) - boost::asio::ip::tcp::socket peer; + /* The definition order is important here: the buffer must outlive the peer + * socket, so that the destructor cancels an asynchronous buffer read + * before the buffer is destroyed. + * + * Similarly, the accepter must be destroyed before the peer. + */ + /// Maximum packet size we will accept. Needed mostly for the underlying packet deserialization logic std::size_t max_size; /// Buffer for packet data reception @@ -63,15 +65,24 @@ class tcp_reader : public reader /// Number of packets to hold on each buffer for asynchronous receive static constexpr std::size_t pkts_per_buffer = 64; + /// TCP peer socket (i.e., the one connected to the remote end) + boost::asio::ip::tcp::socket peer; + /// The acceptor object + boost::asio::ip::tcp::acceptor acceptor; + /// Start an asynchronous receive - void enqueue_receive(); + void enqueue_receive(handler_context ctx); /// Callback on completion of asynchronous accept void accept_handler( + handler_context ctx, + stream_base::add_packet_state &state, const boost::system::error_code &error); /// Callback on completion of asynchronous receive void packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, const boost::system::error_code &error, std::size_t bytes_transferred); @@ -140,8 +151,6 @@ class tcp_reader : public reader boost::asio::ip::tcp::acceptor &&acceptor, std::size_t max_size = default_max_size); - virtual void stop() override; - virtual bool lossy() const override; }; diff --git a/include/spead2/recv_udp.h b/include/spead2/recv_udp.h index a33dc82f9..c9164bee1 100644 --- a/include/spead2/recv_udp.h +++ b/include/spead2/recv_udp.h @@ -1,4 +1,4 @@ -/* Copyright 2015, 2020 National Research Foundation (SARAO) +/* Copyright 2015, 2020, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -31,7 +31,6 @@ #endif #include #include -#include #include #include @@ -46,8 +45,10 @@ namespace recv class udp_reader : public udp_reader_base { private: - /// UDP socket we are listening on - boost::asio::ip::udp::socket socket; + /* Note: declaration order is import for correct destruction + * (the stream must be closed before we start destroying buffers). + */ + /// Unused, but need to provide the memory for asio to write to boost::asio::ip::udp::endpoint endpoint; /// Maximum packet size we will accept @@ -63,12 +64,16 @@ class udp_reader : public udp_reader_base /// Buffer for asynchronous receive, of size @a max_size + 1. std::unique_ptr buffer; #endif + /// UDP socket we are listening on + boost::asio::ip::udp::socket socket; /// Start an asynchronous receive - void enqueue_receive(); + void enqueue_receive(handler_context ctx); /// Callback on completion of asynchronous receive void packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, const boost::system::error_code &error, std::size_t bytes_transferred); @@ -166,8 +171,6 @@ class udp_reader : public udp_reader_base stream &owner, boost::asio::ip::udp::socket &&socket, std::size_t max_size = default_max_size); - - virtual void stop() override; }; /** diff --git a/include/spead2/recv_udp_base.h b/include/spead2/recv_udp_base.h index d5b0ef417..9c6c8716a 100644 --- a/include/spead2/recv_udp_base.h +++ b/include/spead2/recv_udp_base.h @@ -1,4 +1,4 @@ -/* Copyright 2016 National Research Foundation (SARAO) +/* Copyright 2016, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -23,7 +23,6 @@ #include #include -#include #include namespace spead2 diff --git a/include/spead2/recv_udp_ibv.h b/include/spead2/recv_udp_ibv.h index d7213c48f..91b5b88c7 100644 --- a/include/spead2/recv_udp_ibv.h +++ b/include/spead2/recv_udp_ibv.h @@ -1,4 +1,4 @@ -/* Copyright 2016, 2019-2020 National Research Foundation (SARAO) +/* Copyright 2016, 2019-2020, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -39,7 +39,6 @@ #include #include #include -#include #include #include @@ -113,8 +112,6 @@ class udp_ibv_reader_core : public udp_reader_base const std::size_t max_size; ///< Number of times to poll before waiting const int max_poll; - /// Signals poll-mode to stop - std::atomic stop_poll; void join_groups(const std::vector &endpoints, const boost::asio::ip::address &interface_address); @@ -128,8 +125,6 @@ class udp_ibv_reader_core : public udp_reader_base udp_ibv_reader_core( stream &owner, const udp_ibv_config &config); - - virtual void stop() override; }; /** @@ -151,24 +146,28 @@ class udp_ibv_reader_base : public udp_ibv_reader_core * If @a consume_event is true, an event should be removed and consumed * from the completion channel. */ - void packet_handler(const boost::system::error_code &error, - bool consume_event); + void packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, + const boost::system::error_code &error, + bool consume_event); /** * Request a callback when there is data (or as soon as possible, in * polling mode or when @a need_poll is true). */ - void enqueue_receive(bool needs_poll); + void enqueue_receive(handler_context ctx, bool needs_poll); using udp_ibv_reader_core::udp_ibv_reader_core; }; template -void udp_ibv_reader_base::packet_handler(const boost::system::error_code &error, - bool consume_event) +void udp_ibv_reader_base::packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, + const boost::system::error_code &error, + bool consume_event) { - stream_base::add_packet_state state(get_stream_base()); - bool need_poll = true; if (!error) { @@ -184,43 +183,34 @@ void udp_ibv_reader_base::packet_handler(const boost::system::error_cod static_cast(this)->recv_cq.ack_events(1); } } - if (state.is_stopped()) + for (int i = 0; i < max_poll; i++) { - log_info("UDP reader: discarding packet received after stream stopped"); - } - else - { - for (int i = 0; i < max_poll; i++) + if (comp_channel) { - if (comp_channel) + if (i == max_poll - 1) { - if (i == max_poll - 1) - { - /* We need to call req_notify_cq *before* the last - * poll_once, because notifications are edge-triggered. - * If we did it the other way around, there is a race - * where a new packet can arrive after poll_once but - * before req_notify_cq, failing to trigger a - * notification. - */ - static_cast(this)->recv_cq.req_notify(false); - need_poll = false; - } - } - else if (stop_poll.load()) - break; - poll_result result = static_cast(this)->poll_once(state); - if (result == poll_result::stopped) - break; - else if (result == poll_result::partial) - { - /* If we armed req_notify_cq but then didn't drain the CQ, and - * we get no more packets, then we won't get woken up again, so - * we need to poll again next time we go around the event loop. + /* We need to call req_notify_cq *before* the last + * poll_once, because notifications are edge-triggered. + * If we did it the other way around, there is a race + * where a new packet can arrive after poll_once but + * before req_notify_cq, failing to trigger a + * notification. */ - need_poll = true; + static_cast(this)->recv_cq.req_notify(false); + need_poll = false; } } + poll_result result = static_cast(this)->poll_once(state); + if (result == poll_result::stopped) + break; + else if (result == poll_result::partial) + { + /* If we armed req_notify_cq but then didn't drain the CQ, and + * we get no more packets, then we won't get woken up again, so + * we need to poll again next time we go around the event loop. + */ + need_poll = true; + } } } else if (error != boost::asio::error::operation_aborted) @@ -228,14 +218,12 @@ void udp_ibv_reader_base::packet_handler(const boost::system::error_cod if (!state.is_stopped()) { - enqueue_receive(need_poll); + enqueue_receive(std::move(ctx), need_poll); } - else - stopped(); } template -void udp_ibv_reader_base::enqueue_receive(bool need_poll) +void udp_ibv_reader_base::enqueue_receive(handler_context ctx, bool need_poll) { using namespace std::placeholders; if (comp_channel && !need_poll) @@ -243,14 +231,21 @@ void udp_ibv_reader_base::enqueue_receive(bool need_poll) // Asynchronous mode comp_channel_wrapper.async_read_some( boost::asio::null_buffers(), - std::bind(&udp_ibv_reader_base::packet_handler, this, _1, true)); + bind_handler( + std::move(ctx), + std::bind(&udp_ibv_reader_base::packet_handler, this, _1, _2, _3, true))); } else { // Polling mode - get_io_service().post( - std::bind(&udp_ibv_reader_base::packet_handler, this, - boost::system::error_code(), false)); + boost::asio::post( + get_io_service(), + bind_handler( + std::move(ctx), + std::bind(&udp_ibv_reader_base::packet_handler, this, _1, _2, + boost::system::error_code(), false) + ) + ); } } diff --git a/include/spead2/recv_udp_ibv_mprq.h b/include/spead2/recv_udp_ibv_mprq.h index 82de26cdc..74a97da18 100644 --- a/include/spead2/recv_udp_ibv_mprq.h +++ b/include/spead2/recv_udp_ibv_mprq.h @@ -1,4 +1,4 @@ -/* Copyright 2019-2020 National Research Foundation (SARAO) +/* Copyright 2019-2020, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -35,7 +35,6 @@ #include #include #include -#include #include #include #include diff --git a/include/spead2/recv_udp_pcap.h b/include/spead2/recv_udp_pcap.h index 1f273e1b6..1d7071af8 100644 --- a/include/spead2/recv_udp_pcap.h +++ b/include/spead2/recv_udp_pcap.h @@ -1,4 +1,4 @@ -/* Copyright 2016-2017 National Research Foundation (SARAO) +/* Copyright 2016-2017, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -28,7 +28,6 @@ #include #include #include -#include #include #include @@ -49,7 +48,7 @@ class udp_pcap_file_reader : public udp_reader_base pcap_t *handle; udp_unpacker udp_from_frame; - void run(); + void run(handler_context ctx, stream_base::add_packet_state &state); public: /** @@ -64,7 +63,6 @@ class udp_pcap_file_reader : public udp_reader_base udp_pcap_file_reader(stream &owner, const std::string &filename, const std::string &filter = ""); virtual ~udp_pcap_file_reader(); - virtual void stop() override; virtual bool lossy() const override; }; diff --git a/src/Makefile.am b/src/Makefile.am index 64b0d0512..637238f9b 100644 --- a/src/Makefile.am +++ b/src/Makefile.am @@ -1,4 +1,4 @@ -# Copyright 2016, 2020, 2022 National Research Foundation (SARAO) +# Copyright 2016, 2020-2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -76,12 +76,12 @@ libspead2_a_SOURCES = \ common_socket.cpp \ common_thread_pool.cpp \ recv_chunk_stream.cpp \ + recv_chunk_stream_group.cpp \ recv_heap.cpp \ recv_inproc.cpp \ recv_live_heap.cpp \ recv_mem.cpp \ recv_packet.cpp \ - recv_reader.cpp \ recv_ring_stream.cpp \ recv_stream.cpp \ recv_tcp.cpp \ diff --git a/src/common_thread_pool.cpp b/src/common_thread_pool.cpp index 215ce0aa5..fb24a1b51 100644 --- a/src/common_thread_pool.cpp +++ b/src/common_thread_pool.cpp @@ -124,7 +124,7 @@ void thread_pool::stop() } catch (std::exception &e) { - log_warning("worker thread throw an exception: %s", e.what()); + log_warning("worker thread threw an exception: %s", e.what()); } } workers.clear(); diff --git a/src/py_recv.cpp b/src/py_recv.cpp index a4f92e1cb..26b20a900 100644 --- a/src/py_recv.cpp +++ b/src/py_recv.cpp @@ -1,4 +1,4 @@ -/* Copyright 2015, 2017, 2020-2022 National Research Foundation (SARAO) +/* Copyright 2015, 2017, 2020-2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -465,28 +466,38 @@ static void push_chunk(T func, chunk &c) typedef ringbuffer, semaphore_fd, semaphore_fd> chunk_ringbuffer; -class chunk_ring_stream_wrapper : public chunk_ring_stream -{ -private: - exit_stopper stopper{[this] { stop(); }}; +/* Note: ring_stream_wrapper drops the GIL while stopping. We + * can't do that here because stop() can free chunks that were + * in flight, which involves interaction with the Python API. + * I think the only reason ring_stream_wrapper drops the GIL is + * that logging used to directly acquire the GIL, and so if stop() + * did any logging it would deadlock. Now that logging is pushed + * off to a separate thread that should no longer be an issue. + */ +#define EXIT_STOPPER_WRAPPER(cls, base) \ + class cls : public base \ + { \ + private: \ + exit_stopper stopper{[this] { stop(); }}; \ + public: \ + using base::base; \ + virtual void stop() override \ + { \ + stopper.reset(); \ + base::stop(); \ + } \ + } -public: - using chunk_ring_stream::chunk_ring_stream; +// These aliases are needed because a type passed to a macro cannot contain a comma +using chunk_ring_stream_orig = chunk_ring_stream; +using chunk_stream_ring_group_orig = chunk_stream_ring_group; - virtual void stop() override - { - stopper.reset(); - /* Note: ring_stream_wrapper drops the GIL while stopping. We - * can't do that here because stop() can free chunks that were - * in flight, which involves interaction with the Python API. - * I think the only reason ring_stream_wrapper drops the GIL is - * that logging used to directly acquire the GIL, and so if stop() - * did any logging it would deadlock. Now that logging is pushed - * off to a separate thread that should no longer be an issue. - */ - chunk_ring_stream::stop(); - } -}; +EXIT_STOPPER_WRAPPER(chunk_ring_stream_wrapper, chunk_ring_stream_orig); +EXIT_STOPPER_WRAPPER(chunk_stream_ring_group_wrapper, chunk_stream_ring_group_orig); +// We don't need to wrap chunk_stream_group_member, because we've wrapped +// chunk_stream_ring_group and its stop will stop the member streams. + +#undef EXIT_STOPPER_WRAPPER /// Register the receiver module with Python py::module register_module(py::module &parent) @@ -882,7 +893,29 @@ py::module register_module(py::module &parent) "extra", [](const chunk &c) -> const memory_allocator::pointer & { return c.extra; }, [](chunk &c, memory_allocator::pointer &&value) { c.extra = std::move(value); }); - py::class_(m, "ChunkRingStream") + // Don't allow ChunkRingPair to be constructed from Python. It exists + // purely to be a base class. + using chunk_ring_pair = detail::chunk_ring_pair; + py::class_(m, "ChunkRingPair") + .def( + "add_free_chunk", + [](chunk_ring_pair &self, chunk &c) + { + push_chunk( + [&self](std::unique_ptr &&wrapper) + { + self.add_free_chunk(std::move(wrapper)); + }, + c + ); + }, + "chunk"_a) + .def_property_readonly("data_ringbuffer", SPEAD2_PTMF(chunk_ring_pair, get_data_ringbuffer)) + .def_property_readonly("free_ringbuffer", SPEAD2_PTMF(chunk_ring_pair, get_free_ringbuffer)); + + py::class_, + stream>(m, "ChunkRingStream") .def(py::init, const stream_config &, const chunk_stream_config &, @@ -897,22 +930,7 @@ py::module register_module(py::module &parent) // This allows Python subclasses to be passed then later retrieved // from properties. py::keep_alive<1, 5>(), - py::keep_alive<1, 6>()) - .def( - "add_free_chunk", - [](chunk_ring_stream_wrapper &stream, chunk &c) - { - push_chunk( - [&stream](std::unique_ptr &&wrapper) - { - stream.add_free_chunk(std::move(wrapper)); - }, - c - ); - }, - "chunk"_a) - .def_property_readonly("data_ringbuffer", SPEAD2_PTMF(chunk_ring_stream_wrapper, get_data_ringbuffer)) - .def_property_readonly("free_ringbuffer", SPEAD2_PTMF(chunk_ring_stream_wrapper, get_free_ringbuffer)); + py::keep_alive<1, 6>()); py::class_>(m, "ChunkRingbuffer") .def(py::init(), "maxsize"_a) .def("qsize", SPEAD2_PTMF(chunk_ringbuffer, size)) @@ -967,6 +985,78 @@ py::module register_module(py::module &parent) } }); + py::class_ chunk_stream_group_config_cls(m, "ChunkStreamGroupConfig"); + chunk_stream_group_config_cls + .def(py::init(&data_class_constructor)) + .def_property("max_chunks", + SPEAD2_PTMF(chunk_stream_group_config, get_max_chunks), + SPEAD2_PTMF(chunk_stream_group_config, set_max_chunks)) + .def_property("eviction_mode", + SPEAD2_PTMF(chunk_stream_group_config, get_eviction_mode), + SPEAD2_PTMF(chunk_stream_group_config, set_eviction_mode)) + .def_readonly_static("DEFAULT_MAX_CHUNKS", &chunk_stream_group_config::default_max_chunks); + py::enum_(chunk_stream_group_config_cls, "EvictionMode") + .value("LOSSY", chunk_stream_group_config::eviction_mode::LOSSY) + .value("LOSSLESS", chunk_stream_group_config::eviction_mode::LOSSLESS); + + py::class_(m, "ChunkStreamGroupMember"); + + py::class_>(m, "ChunkStreamRingGroup") + .def(py::init, + std::shared_ptr>(), + "config"_a, + "data_ringbuffer"_a.none(false), + "free_ringbuffer"_a.none(false), + // Keep the Python ringbuffer objects alive, not just the C++ side. + // This allows Python subclasses to be passed then later retrieved + // from properties. + py::keep_alive<1, 3>(), + py::keep_alive<1, 4>()) + .def_property_readonly( + "config", SPEAD2_PTMF(chunk_stream_ring_group_wrapper, get_config)) + .def( + "emplace_back", + [](chunk_stream_ring_group_wrapper &group, + std::shared_ptr thread_pool, + const stream_config &config, + const chunk_stream_config &chunk_stream_config) -> chunk_stream_group_member & { + return group.emplace_back(std::move(thread_pool), config, chunk_stream_config); + }, + "thread_pool"_a, "config"_a, "chunk_stream_config"_a, + py::return_value_policy::reference_internal + ) + .def("__len__", SPEAD2_PTMF(chunk_stream_ring_group_wrapper, size)) + .def( + "__getitem__", + [](chunk_stream_ring_group_wrapper &group, std::ptrdiff_t index) -> chunk_stream_group_member & { + if (index < 0) + index += group.size(); + if (index >= 0 && std::size_t(index) < group.size()) + return group[index]; + else + throw py::index_error(); + }, + py::return_value_policy::reference_internal + ) + .def( + "__getitem__", + [](chunk_stream_ring_group_wrapper &group, const py::slice &slice) { + py::list out; + std::size_t start, stop, step, length; + if (!slice.compute(group.size(), &start, &stop, &step, &length)) + throw py::error_already_set(); + py::object self = py::cast(group); + for (std::size_t i = 0; i < length; i++) { + out.append(py::cast(group[start], py::return_value_policy::reference_internal, self)); + start += step; + } + return out; + } + ) + .def("stop", SPEAD2_PTMF(chunk_stream_ring_group_wrapper, stop)); + return m; } diff --git a/src/recv_chunk_stream.cpp b/src/recv_chunk_stream.cpp index 88d9662bf..aa4fa7dd7 100644 --- a/src/recv_chunk_stream.cpp +++ b/src/recv_chunk_stream.cpp @@ -1,4 +1,4 @@ -/* Copyright 2021-2022 National Research Foundation (SARAO) +/* Copyright 2021-2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -27,7 +27,6 @@ #include #include #include -#include #include #include #include @@ -105,7 +104,9 @@ static std::size_t round_up(std::size_t size, std::size_t align) return (size + align - 1) / align * align; } -chunk_stream_state::chunk_stream_state( +chunk_window::chunk_window(std::size_t max_chunks) : chunks(max_chunks) {} + +chunk_stream_state_base::chunk_stream_state_base( const stream_config &config, const chunk_stream_config &chunk_config) : orig_memcpy(config.get_memcpy()), chunk_config(chunk_config), @@ -115,10 +116,6 @@ chunk_stream_state::chunk_stream_state( { if (!this->chunk_config.get_place()) throw std::invalid_argument("chunk_config.place is not set"); - if (!this->chunk_config.get_allocate()) - throw std::invalid_argument("chunk_config.allocate is not set"); - if (!this->chunk_config.get_ready()) - throw std::invalid_argument("chunk_config.ready is not set"); /* Compute the memory required for place_data_storage. The layout is * - chunk_place_data @@ -166,20 +163,20 @@ chunk_stream_state::chunk_stream_state( place_data_storage.reset(ptr); } -void chunk_stream_state::free_place_data::operator()(unsigned char *ptr) +void chunk_stream_state_base::free_place_data::operator()(unsigned char *ptr) const { // TODO: should this use std::launder in C++17? auto *place_data = reinterpret_cast(ptr); place_data->~chunk_place_data(); - operator delete[](ptr); + operator delete(ptr); } -void chunk_stream_state::packet_memcpy( +void chunk_stream_state_base::packet_memcpy( const memory_allocator::pointer &allocation, const packet_header &packet) const { const heap_metadata &metadata = *get_heap_metadata(allocation); - if (metadata.chunk_id < head_chunk) + if (chunk_too_old(metadata.chunk_id)) { // The packet corresponds to a chunk that has already been aged out // TODO: increment a counter / log a warning @@ -196,218 +193,73 @@ void chunk_stream_state::packet_memcpy( } } -stream_config chunk_stream_state::adjust_config(const stream_config &config) -{ - using namespace std::placeholders; - stream_config new_config = config; - // Unsized heaps won't work with the custom allocator - new_config.set_allow_unsized_heaps(false); - new_config.set_memory_allocator(std::make_shared(*this)); - // Override the original memcpy with our custom version - new_config.set_memcpy(std::bind(&chunk_stream_state::packet_memcpy, this, _1, _2)); - // Add custom statistics - new_config.add_stat("too_old_heaps"); - new_config.add_stat("rejected_heaps"); - return new_config; -} - -void chunk_stream_state::flush_head() +void chunk_stream_state_base::do_heap_ready(live_heap &&lh) { - assert(head_chunk < tail_chunk); - if (chunks[head_pos]) + if (lh.is_complete()) { - std::uint64_t *batch_stats = static_cast(this)->batch_stats.data(); - chunk_config.get_ready()(std::move(chunks[head_pos]), batch_stats); - // If the ready callback didn't take over ownership, free it. - chunks[head_pos].reset(); + heap h(std::move(lh)); + auto metadata = get_heap_metadata(h.get_payload()); + // We need to check the chunk_id because the chunk might have been aged + // out while the heap was incomplete. + if (metadata && metadata->chunk_ptr + && !chunk_too_old(metadata->chunk_id) + && !get_chunk_config().get_packet_presence_payload_size()) + { + assert(metadata->heap_index < metadata->chunk_ptr->present_size); + metadata->chunk_ptr->present[metadata->heap_index] = true; + } } - head_chunk++; - head_pos++; - if (head_pos == chunks.size()) - head_pos = 0; // wrap around the circular buffer -} - -void chunk_stream_state::flush_chunks() -{ - while (head_chunk != tail_chunk) - flush_head(); } -const chunk_stream_state::heap_metadata *chunk_stream_state::get_heap_metadata( +const chunk_stream_state_base::heap_metadata *chunk_stream_state_base::get_heap_metadata( const memory_allocator::pointer &ptr) { return ptr.get_deleter().target(); } -// Used to get a non-null pointer -static std::uint8_t dummy_uint8; - -// Keep these in sync with stats added in adjust_config -static constexpr std::size_t too_old_heaps_offset = 0; -static constexpr std::size_t rejected_heaps_offset = 1; - -std::pair -chunk_stream_state::allocate(std::size_t size, const packet_header &packet) +chunk_manager_simple::chunk_manager_simple(const chunk_stream_config &chunk_config) { - /* Extract the user's requested items. - * TODO: this could possibly be optimised with a hash table (with a - * perfect hash function chosen in advance), but for the expected - * sizes the overheads will probably outweight the benefits. - */ - const auto &item_ids = get_chunk_config().get_items(); - std::fill(place_data->items, place_data->items + item_ids.size(), -1); - pointer_decoder decoder(packet.heap_address_bits); - /* packet.pointers and packet.n_items skips initial "special" item - * pointers. To allow them to be matched as well, we start from the - * original packet and skip over the 8-byte header. - */ - for (const std::uint8_t *p = packet.packet + 8; p != packet.payload; p += sizeof(item_pointer_t)) - { - item_pointer_t pointer = load_be(p); - if (decoder.is_immediate(pointer)) - { - item_pointer_t id = decoder.get_id(pointer); - for (std::size_t j = 0; j < item_ids.size(); j++) - if (item_ids[j] == id) - place_data->items[j] = decoder.get_immediate(pointer); - } - } + if (!chunk_config.get_allocate()) + throw std::invalid_argument("chunk_config.allocate is not set"); + if (!chunk_config.get_ready()) + throw std::invalid_argument("chunk_config.ready is not set"); +} - /* TODO: see if the storage can be in the class with the deleter - * just referencing it. That will avoid the implied memory allocation - * in constructing the std::function underlying the deleter. - */ - std::pair out; - out.first = &dummy_uint8; // Use a non-null value to avoid confusion with empty pointers - heap_metadata &metadata = out.second; - - place_data->packet = packet.packet; - place_data->packet_size = packet.payload + packet.payload_length - packet.packet; - place_data->chunk_id = -1; - place_data->heap_index = 0; - place_data->heap_offset = 0; - place_data->batch_stats = static_cast(this)->batch_stats.data(); - place_data->extra_offset = 0; - place_data->extra_size = 0; - chunk_config.get_place()(place_data, sizeof(*place_data)); - auto chunk_id = place_data->chunk_id; - if (chunk_id < head_chunk) - { - // We don't want this heap. - metadata.chunk_id = -1; - metadata.chunk_ptr = nullptr; - std::size_t stat_offset = (chunk_id >= 0) ? too_old_heaps_offset : rejected_heaps_offset; - place_data->batch_stats[base_stat_index + stat_offset]++; - return out; - } - else - { - std::size_t max_chunks = chunk_config.get_max_chunks(); - if (chunk_id >= tail_chunk) - { - // We've moved beyond the end of our current window, and need to - // allocate fresh chunks. - const auto &allocate = chunk_config.get_allocate(); - if (chunk_id >= tail_chunk + std::int64_t(max_chunks)) - { - /* We've jumped ahead so far that the entire current window - * is stale. Flush it all and fast-forward to the new window. - * We leave it to the while loop below to actually allocate - * the chunks. - */ - flush_chunks(); - head_chunk = tail_chunk = chunk_id - (max_chunks - 1); - head_pos = tail_pos = 0; - } - while (chunk_id >= tail_chunk) - { - if (std::size_t(tail_chunk - head_chunk) == max_chunks) - flush_head(); - chunks[tail_pos] = allocate(tail_chunk, place_data->batch_stats); - if (chunks[tail_pos]) - { - chunks[tail_pos]->chunk_id = tail_chunk; - chunks[tail_pos]->stream_id = stream_id; - } - tail_chunk++; - tail_pos++; - if (tail_pos == max_chunks) - tail_pos = 0; // wrap around circular buffer - } - } - // Find position of chunk within the storage - std::size_t pos = chunk_id - head_chunk + head_pos; - if (pos >= max_chunks) - pos -= max_chunks; // wrap around the circular storage - if (chunks[pos]) - { - chunk &c = *chunks[pos]; - out.first = c.data.get() + place_data->heap_offset; - metadata.chunk_id = chunk_id; - metadata.heap_index = place_data->heap_index; - metadata.heap_offset = place_data->heap_offset; - metadata.chunk_ptr = &c; - if (place_data->extra_size > 0) - { - assert(place_data->extra_size <= chunk_config.get_max_heap_extra()); - assert(c.extra); - std::memcpy(c.extra.get() + place_data->extra_offset, place_data->extra, place_data->extra_size); - } - return out; - } - else - { - // the allocator didn't allocate a chunk for this slot. - metadata.chunk_id = -1; - metadata.chunk_ptr = nullptr; - return out; - } - } +std::uint64_t *chunk_manager_simple::get_batch_stats(chunk_stream_state &state) const +{ + return static_cast(&state)->batch_stats.data(); } -chunk_stream_allocator::chunk_stream_allocator(chunk_stream_state &stream) - : stream(stream) +chunk *chunk_manager_simple::allocate_chunk(chunk_stream_state &state, std::int64_t chunk_id) { + const auto &allocate = state.chunk_config.get_allocate(); + std::unique_ptr owned = allocate(chunk_id, state.place_data->batch_stats); + return owned.release(); // ready_chunk will re-take ownership } -memory_allocator::pointer chunk_stream_allocator::allocate(std::size_t size, void *hint) +void chunk_manager_simple::ready_chunk(chunk_stream_state &state, chunk *c) { - if (hint) - { - auto alloc = stream.allocate(size, *reinterpret_cast(hint)); - // Use the heap_metadata as the deleter - return pointer(alloc.first, std::move(alloc.second)); - } - // Probably unreachable, but provides a safety net - return memory_allocator::allocate(size, hint); + std::unique_ptr owned(c); + state.chunk_config.get_ready()(std::move(owned), get_batch_stats(state)); } +template class chunk_stream_state; +template class chunk_stream_allocator; + } // namespace detail chunk_stream::chunk_stream( io_service_ref io_service, const stream_config &config, const chunk_stream_config &chunk_config) - : chunk_stream_state(config, chunk_config), + : chunk_stream_state(config, chunk_config, detail::chunk_manager_simple(chunk_config)), stream(std::move(io_service), adjust_config(config)) { } void chunk_stream::heap_ready(live_heap &&lh) { - if (lh.is_complete()) - { - heap h(std::move(lh)); - auto metadata = get_heap_metadata(h.get_payload()); - // We need to check the chunk_id because the chunk might have been aged - // out while the heap was incomplete. - if (metadata && metadata->chunk_ptr && metadata->chunk_id >= get_head_chunk() - && !get_chunk_config().get_packet_presence_payload_size()) - { - assert(metadata->heap_index < metadata->chunk_ptr->present_size); - metadata->chunk_ptr->present[metadata->heap_index] = true; - } - } + do_heap_ready(std::move(lh)); } void chunk_stream::stop_received() @@ -419,7 +271,7 @@ void chunk_stream::stop_received() void chunk_stream::stop() { { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(get_queue_mutex()); flush_chunks(); } stream::stop(); diff --git a/src/recv_chunk_stream_group.cpp b/src/recv_chunk_stream_group.cpp new file mode 100644 index 000000000..efe2bc131 --- /dev/null +++ b/src/recv_chunk_stream_group.cpp @@ -0,0 +1,291 @@ +/* Copyright 2023 National Research Foundation (SARAO) + * + * This program is free software: you can redistribute it and/or modify it under + * the terms of the GNU Lesser General Public License as published by the Free + * Software Foundation, either version 3 of the License, or (at your option) any + * later version. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS + * FOR A PARTICULAR PURPOSE. See the GNU Lesser General Public License for more + * details. + * + * You should have received a copy of the GNU Lesser General Public License + * along with this program. If not, see . + */ + +/** + * @file + */ + +#include +#include +#include +#include +#include +#include +#include + +namespace spead2 +{ +namespace recv +{ + +constexpr std::size_t chunk_stream_group_config::default_max_chunks; + +chunk_stream_group_config &chunk_stream_group_config::set_max_chunks(std::size_t max_chunks) +{ + if (max_chunks == 0) + throw std::invalid_argument("max_chunks cannot be 0"); + this->max_chunks = max_chunks; + return *this; +} + +chunk_stream_group_config &chunk_stream_group_config::set_eviction_mode(eviction_mode eviction_mode_) +{ + this->eviction_mode_ = eviction_mode_; + return *this; +} + +chunk_stream_group_config &chunk_stream_group_config::set_allocate(chunk_allocate_function allocate) +{ + this->allocate = std::move(allocate); + return *this; +} + +chunk_stream_group_config &chunk_stream_group_config::set_ready(chunk_ready_function ready) +{ + this->ready = std::move(ready); + return *this; +} + +namespace detail +{ + +chunk_manager_group::chunk_manager_group(chunk_stream_group &group) + : group(group) +{ +} + +std::uint64_t *chunk_manager_group::get_batch_stats(chunk_stream_state &state) const +{ + return static_cast(&state)->batch_stats.data(); +} + +chunk *chunk_manager_group::allocate_chunk( + chunk_stream_state &state, std::int64_t chunk_id) +{ + return group.get_chunk(chunk_id, state.stream_id, state.place_data->batch_stats); +} + +void chunk_manager_group::head_updated( + chunk_stream_state &state, std::uint64_t head_chunk) +{ + group.stream_head_updated(static_cast(state), head_chunk); +} + +} // namespace detail + +chunk_stream_group::chunk_stream_group(const chunk_stream_group_config &config) + : config(config), chunks(config.get_max_chunks()) +{ +} + +chunk_stream_group::~chunk_stream_group() +{ + stop(); +} + +chunk_stream_group::iterator chunk_stream_group::begin() noexcept +{ + return iterator(streams.begin()); +} + +chunk_stream_group::iterator chunk_stream_group::end() noexcept +{ + return iterator(streams.end()); +} + +chunk_stream_group::const_iterator chunk_stream_group::begin() const noexcept +{ + return const_iterator(streams.begin()); +} + +chunk_stream_group::const_iterator chunk_stream_group::end() const noexcept +{ + return const_iterator(streams.end()); +} + +chunk_stream_group::const_iterator chunk_stream_group::cbegin() const noexcept +{ + return const_iterator(streams.begin()); +} + +chunk_stream_group::const_iterator chunk_stream_group::cend() const noexcept +{ + return const_iterator(streams.end()); +} + +chunk_stream_group_member &chunk_stream_group::emplace_back( + io_service_ref io_service, + const stream_config &config, + const chunk_stream_config &chunk_config) +{ + return emplace_back(std::move(io_service), config, chunk_config); +} + +void chunk_stream_group::stop() +{ + /* The mutex is not held while stopping streams, so that callbacks + * triggered by stopping the streams can take the lock if necessary. + * + * It's safe to iterate streams without the mutex because this function + * is called by the user, so a simultaneous call to emplace_back would + * violate the requirement that the user doesn't call the API from more + * than one thread at a time. + */ + if (config.get_eviction_mode() == chunk_stream_group_config::eviction_mode::LOSSLESS) + { + /* Stopping a stream that is currently waiting in get_chunk could + * deadlock. In lossy mode, there are already provisions to guarantee + * forward progress, but in lossless mode we need some help. + */ + for (const auto &stream : streams) + { + stream->async_flush_until(std::numeric_limits::max()); + } + } + for (const auto &stream : streams) + stream->stop1(); +} + +chunk *chunk_stream_group::get_chunk(std::uint64_t chunk_id, std::uintptr_t stream_id, std::uint64_t *batch_stats) +{ + std::unique_lock lock(mutex); + /* Streams should not be requesting chunks older than their heads, and the group + * head is at least as old as any stream head. + */ + assert(chunk_id >= chunks.get_head_chunk()); + /* Any chunk old enough be made ready needs to first be released by the + * member streams. To do that, we request all the streams to flush, then + * wait until it is safe, using the condition variable to wake up + * whenever there is forward progress. + * + * Another thread may call get_chunk in the meantime and advance the + * window, so we must be careful not to assume anything about the + * state after a wait. + */ + const std::size_t max_chunks = config.get_max_chunks(); + if (chunk_id - chunks.get_head_chunk() >= max_chunks) + { + std::uint64_t target = chunk_id - (max_chunks - 1); // first chunk we don't need to flush + if (config.get_eviction_mode() == chunk_stream_group_config::eviction_mode::LOSSY + && target > last_flush_until) + { + for (const auto &s : streams) + s->async_flush_until(target); + last_flush_until = target; + } + while (chunks.get_head_chunk() < target) + { + ready_condition.wait(lock); + } + } + + chunk *c = chunks.get_chunk( + chunk_id, + stream_id, + [this, batch_stats](std::int64_t id) { + return config.get_allocate()(id, batch_stats).release(); + }, + [](chunk *) { + // Should be unreachable, as we've ensured this by waiting above + assert(false); + }, + [](std::uint64_t) {} // Don't need notification for head moving + ); + return c; +} + +void chunk_stream_group::ready_chunk(chunk *c, std::uint64_t *batch_stats) +{ + std::unique_ptr owned(c); + config.get_ready()(std::move(owned), batch_stats); +} + +void chunk_stream_group::stream_head_updated(chunk_stream_group_member &s, std::uint64_t head_chunk) +{ + std::lock_guard lock(mutex); + std::size_t stream_index = s.group_index; + std::uint64_t old = head_chunks[stream_index]; + assert(head_chunk > old); // head_updated should only be called on forward progress + head_chunks[stream_index] = head_chunk; + // Update so that our head chunk is min(head_chunks). We can skip the work + // if we weren't previously the oldest. + if (chunks.get_head_chunk() == old) + { + auto min_head_chunk = *std::min_element(head_chunks.begin(), head_chunks.end()); + chunks.flush_until( + min_head_chunk, + [this, &s](chunk *c) { ready_chunk(c, s.batch_stats.data()); }, + [this](std::uint64_t) { ready_condition.notify_all(); } + ); + } +} + +chunk_stream_group_member::chunk_stream_group_member( + chunk_stream_group &group, + std::size_t group_index, + io_service_ref io_service, + const stream_config &config, + const chunk_stream_config &chunk_config) + : chunk_stream_state(config, chunk_config, detail::chunk_manager_group(group)), + stream(std::move(io_service), adjust_config(config)), + group(group), group_index(group_index) +{ + if (chunk_config.get_max_chunks() > group.config.get_max_chunks()) + throw std::invalid_argument("stream max_chunks must not be larger than group max_chunks"); +} + +void chunk_stream_group_member::heap_ready(live_heap &&lh) +{ + do_heap_ready(std::move(lh)); +} + +void chunk_stream_group_member::async_flush_until(std::uint64_t chunk_id) +{ + post([chunk_id](stream_base &s) { + chunk_stream_group_member &self = static_cast(s); + self.chunks.flush_until( + chunk_id, + [](chunk *) {}, + [&self](std::uint64_t head_chunk) { + self.group.stream_head_updated(self, head_chunk); + } + ); + }); +} + +void chunk_stream_group_member::stop1() +{ + { + std::lock_guard lock(get_queue_mutex()); + flush_chunks(); + } + stream::stop(); +} + +void chunk_stream_group_member::stop_received() +{ + stream::stop_received(); + flush_chunks(); + group.stream_stop_received(*this); +} + +void chunk_stream_group_member::stop() +{ + group.stop(); +} + +} // namespace recv +} // namespace spead2 diff --git a/src/recv_inproc.cpp b/src/recv_inproc.cpp index 3a9a3ddee..6a64733e7 100644 --- a/src/recv_inproc.cpp +++ b/src/recv_inproc.cpp @@ -1,4 +1,4 @@ -/* Copyright 2018-2019 National Research Foundation (SARAO) +/* Copyright 2018-2019, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -24,7 +24,7 @@ #include #include #include -#include +#include namespace spead2 { @@ -39,7 +39,7 @@ inproc_reader::inproc_reader( data_sem_wrapper(wrap_fd(owner.get_io_service(), this->queue->buffer.get_data_sem().get_fd())) { - enqueue(); + enqueue(make_handler_context()); } void inproc_reader::process_one_packet(stream_base::add_packet_state &state, @@ -58,57 +58,41 @@ void inproc_reader::process_one_packet(stream_base::add_packet_state &state, } void inproc_reader::packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, const boost::system::error_code &error, std::size_t bytes_transferred) { - stream_base::add_packet_state state(get_stream_base()); if (!error) { - if (state.is_stopped()) + try { - log_info("inproc reader: discarding packet received after stream stopped"); + inproc_queue::packet packet = queue->buffer.try_pop(); + process_one_packet(state, packet); + /* TODO: could grab a batch of packets to amortise costs */ } - else + catch (ringbuffer_stopped &) { - try - { - inproc_queue::packet packet = queue->buffer.try_pop(); - process_one_packet(state, packet); - /* TODO: could grab a batch of packets to amortise costs */ - } - catch (ringbuffer_stopped &) - { - state.stop(); - } - catch (ringbuffer_empty &) - { - // spurious wakeup - no action needed - } + state.stop(); + } + catch (ringbuffer_empty &) + { + // spurious wakeup - no action needed } } else if (error != boost::asio::error::operation_aborted) log_warning("Error in inproc receiver: %1%", error.message()); if (!state.is_stopped()) - enqueue(); - else - { - data_sem_wrapper.close(); - stopped(); - } + enqueue(std::move(ctx)); } -void inproc_reader::enqueue() +void inproc_reader::enqueue(handler_context ctx) { using namespace std::placeholders; data_sem_wrapper.async_read_some( boost::asio::null_buffers(), - std::bind(&inproc_reader::packet_handler, this, _1, _2)); -} - -void inproc_reader::stop() -{ - data_sem_wrapper.close(); + bind_handler(std::move(ctx), std::bind(&inproc_reader::packet_handler, this, _1, _2, _3, _4))); } bool inproc_reader::lossy() const diff --git a/src/recv_mem.cpp b/src/recv_mem.cpp index ccc90c1e0..3b69c4252 100644 --- a/src/recv_mem.cpp +++ b/src/recv_mem.cpp @@ -1,4 +1,4 @@ -/* Copyright 2015, 2019 National Research Foundation (SARAO) +/* Copyright 2015, 2019, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -20,7 +20,6 @@ #include #include -#include #include #include @@ -35,13 +34,14 @@ mem_reader::mem_reader( : reader(owner), ptr(ptr), length(length) { assert(ptr != nullptr); - get_io_service().post([this] { - mem_to_stream(get_stream_base(), this->ptr, this->length); - // There will be no more data, so we can stop the stream immediately. - stream_base::add_packet_state state(get_stream_base()); - state.stop(); - stopped(); - }); + boost::asio::post( + get_io_service(), + bind_handler([this] (handler_context ctx, stream_base::add_packet_state &state) { + mem_to_stream(state, this->ptr, this->length); + // There will be no more data, so we can stop the stream immediately. + state.stop(); + }) + ); } bool mem_reader::lossy() const diff --git a/src/recv_reader.cpp b/src/recv_reader.cpp deleted file mode 100644 index b2f8e9da3..000000000 --- a/src/recv_reader.cpp +++ /dev/null @@ -1,53 +0,0 @@ -/* Copyright 2015, 2019 National Research Foundation (SARAO) - * - * This program is free software: you can redistribute it and/or modify it under - * the terms of the GNU Lesser General Public License as published by the Free - * Software Foundation, either version 3 of the License, or (at your option) any - * later version. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS - * FOR A PARTICULAR PURPOSE. See the GNU Lesser General Public License for more - * details. - * - * You should have received a copy of the GNU Lesser General Public License - * along with this program. If not, see . - */ - -/** - * @file - */ - -#include -#include - -namespace spead2 -{ -namespace recv -{ - -void reader::stopped() -{ - // Schedule it to run later so that at the time it occurs there are no - // further references to *this. - stream *owner_ptr = &owner; - get_io_service().post([owner_ptr] { owner_ptr->readers_stopped.put(); }); -} - -bool reader::lossy() const -{ - return true; -} - -boost::asio::io_service &reader::get_io_service() -{ - return owner.get_io_service(); -} - -stream_base &reader::get_stream_base() const -{ - return owner; -} - -} // namespace recv -} // namespace spead2 diff --git a/src/recv_stream.cpp b/src/recv_stream.cpp index 416f19aee..fa3fcd847 100644 --- a/src/recv_stream.cpp +++ b/src/recv_stream.cpp @@ -1,4 +1,4 @@ -/* Copyright 2015, 2017-2021 National Research Foundation (SARAO) +/* Copyright 2015, 2017-2021, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -98,7 +98,7 @@ static std::size_t get_stat_index( } -static std::shared_ptr> make_default_stats() +static std::shared_ptr> make_default_stats() { auto stats = std::make_shared>(); // Keep this in sync with the stream_stat_* constexprs in the header @@ -121,21 +121,21 @@ static std::shared_ptr> make_default_stats() * Sharing this means the compatibility check for operator+ requires only a * pointer comparison rather than comparing arrays. */ -static std::shared_ptr> default_stats = make_default_stats(); +static std::shared_ptr> default_stats = make_default_stats(); stream_stats::stream_stats() : stream_stats(default_stats) { } -stream_stats::stream_stats(std::shared_ptr> config) +stream_stats::stream_stats(std::shared_ptr> config) : stream_stats(config, std::vector(config->size())) { // Note: annoyingly, can't use std::move(config) above, because we access // config to get the size to use for the vector. } -stream_stats::stream_stats(std::shared_ptr> config, +stream_stats::stream_stats(std::shared_ptr> config, std::vector values) : config(std::move(config)), values(std::move(values)), @@ -358,12 +358,11 @@ std::size_t stream_config::add_stat(std::string name, stream_stat_config::mode m { if (spead2::recv::get_stat_index_nothrow(*stats, name) != stats->size()) throw std::invalid_argument("A statistic called " + name + " already exists"); - // If we're pointing at the default, make a copy so that we don't modify - // the default. - if (stats == default_stats) - stats = std::make_shared>(*default_stats); - std::size_t index = stats->size(); - stats->emplace_back(std::move(name), mode); + // Make a copy so that we don't modify any shared copies + auto new_stats = std::make_shared>(*stats); + std::size_t index = new_stats->size(); + new_stats->emplace_back(std::move(name), mode); + stats = std::move(new_stats); return index; } @@ -381,6 +380,7 @@ stream_base::stream_base(const stream_config &config) substreams(new substream[config.get_substreams() + 1]), substream_div(config.get_substreams()), config(config), + shared(std::make_shared(this)), stats(config.get_stats().size()), batch_stats(config.get_stats().size()) { @@ -439,35 +439,36 @@ void stream_base::unlink_entry(queue_entry *entry) entry->next = INVALID_ENTRY; } -stream_base::add_packet_state::add_packet_state(stream_base &owner) - : owner(owner), lock(owner.queue_mutex) +stream_base::add_packet_state::add_packet_state(shared_state &owner) + : lock(owner.queue_mutex), owner(owner.self) { - std::fill(owner.batch_stats.begin(), owner.batch_stats.end(), 0); + if (this->owner) + std::fill(this->owner->batch_stats.begin(), this->owner->batch_stats.end(), 0); } stream_base::add_packet_state::~add_packet_state() { - if (!packets && is_stopped()) + if (!owner || (!packets && is_stopped())) return; // Stream was stopped before we could do anything - don't count as a batch - std::lock_guard stats_lock(owner.stats_mutex); + std::lock_guard stats_lock(owner->stats_mutex); // The built-in stats are updated directly; batch_stats is not used - owner.stats[stream_stat_indices::packets] += packets; - owner.stats[stream_stat_indices::batches]++; - owner.stats[stream_stat_indices::heaps] += complete_heaps + incomplete_heaps_evicted; - owner.stats[stream_stat_indices::incomplete_heaps_evicted] += incomplete_heaps_evicted; - owner.stats[stream_stat_indices::single_packet_heaps] += single_packet_heaps; - owner.stats[stream_stat_indices::search_dist] += search_dist; - auto &owner_max_batch = owner.stats[stream_stat_indices::max_batch]; + owner->stats[stream_stat_indices::packets] += packets; + owner->stats[stream_stat_indices::batches]++; + owner->stats[stream_stat_indices::heaps] += complete_heaps + incomplete_heaps_evicted; + owner->stats[stream_stat_indices::incomplete_heaps_evicted] += incomplete_heaps_evicted; + owner->stats[stream_stat_indices::single_packet_heaps] += single_packet_heaps; + owner->stats[stream_stat_indices::search_dist] += search_dist; + auto &owner_max_batch = owner->stats[stream_stat_indices::max_batch]; owner_max_batch = std::max(owner_max_batch, packets); // Update custom statistics - const auto &stats_config = owner.get_config().get_stats(); + const auto &stats_config = owner->get_config().get_stats(); for (std::size_t i = stream_stat_indices::custom; i < stats_config.size(); i++) - owner.stats[i] = stats_config[i].combine(owner.stats[i], owner.batch_stats[i]); + owner->stats[i] = stats_config[i].combine(owner->stats[i], owner->batch_stats[i]); } bool stream_base::add_packet(add_packet_state &state, const packet_header &packet) { - const stream_config &config = state.owner.get_config(); + const stream_config &config = state.owner->get_config(); assert(!stopped); state.packets++; if (packet.heap_length < 0 && !config.get_allow_unsized_heaps()) @@ -585,7 +586,7 @@ void stream_base::flush_unlocked() void stream_base::flush() { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(shared->queue_mutex); flush_unlocked(); } @@ -597,7 +598,7 @@ void stream_base::stop_unlocked() void stream_base::stop() { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(shared->queue_mutex); stop_unlocked(); } @@ -605,6 +606,7 @@ void stream_base::stop_received() { assert(!stopped); stopped = true; + shared->self = nullptr; flush_unlocked(); } @@ -616,6 +618,17 @@ stream_stats stream_base::get_stats() const } +reader::reader(stream &owner) + : io_service(owner.get_io_service()), owner(owner.shared) +{ +} + +bool reader::lossy() const +{ + return true; +} + + stream::stream(io_service_ref io_service, const stream_config &config) : stream_base(config), thread_pool_holder(std::move(io_service).get_shared_thread_pool()), @@ -627,38 +640,18 @@ void stream::stop_received() { stream_base::stop_received(); std::lock_guard lock(reader_mutex); - for (const auto &reader : readers) - reader->stop(); + readers.clear(); + /* This ensures that once we clear out the readers, any future call to + * emplace_reader will silently be ignored. This avoids issues if there + * is a race between the user calling emplace_reader and a stop packet + * in the stream. + */ + stop_readers = true; } void stream::stop_impl() { stream_base::stop(); - - std::size_t n_readers; - { - std::lock_guard lock(reader_mutex); - /* Prevent any further calls to emplace_reader from doing anything, so - * that n_readers will remain accurate. - */ - stop_readers = true; - n_readers = readers.size(); - } - - // Wait until all readers have wound up all their completion handlers - while (n_readers > 0) - { - semaphore_get(readers_stopped); - n_readers--; - } - - { - /* This lock is not strictly needed since no other thread can touch - * readers any more, but is harmless. - */ - std::lock_guard lock(reader_mutex); - readers.clear(); - } } void stream::stop() @@ -678,9 +671,8 @@ stream::~stream() } -const std::uint8_t *mem_to_stream(stream_base &s, const std::uint8_t *ptr, std::size_t length) +const std::uint8_t *mem_to_stream(stream_base::add_packet_state &state, const std::uint8_t *ptr, std::size_t length) { - stream_base::add_packet_state state(s); while (length > 0 && !state.is_stopped()) { packet_header packet; @@ -697,5 +689,11 @@ const std::uint8_t *mem_to_stream(stream_base &s, const std::uint8_t *ptr, std:: return ptr; } +const std::uint8_t *mem_to_stream(stream_base &s, const std::uint8_t *ptr, std::size_t length) +{ + stream_base::add_packet_state state(s); + return mem_to_stream(state, ptr, length); +} + } // namespace recv } // namespace spead2 diff --git a/src/recv_tcp.cpp b/src/recv_tcp.cpp index 50fe64d0c..869304f0d 100644 --- a/src/recv_tcp.cpp +++ b/src/recv_tcp.cpp @@ -30,7 +30,7 @@ #include #include #include -#include +#include #include #include #include @@ -50,17 +50,25 @@ tcp_reader::tcp_reader( boost::asio::ip::tcp::acceptor &&acceptor, std::size_t max_size, std::size_t buffer_size) - : reader(owner), acceptor(std::move(acceptor)), - peer(get_socket_io_service(this->acceptor)), + : reader(owner), max_size(max_size), buffer(new std::uint8_t[max_size * pkts_per_buffer]), head(buffer.get()), - tail(buffer.get()) + tail(buffer.get()), + peer(get_socket_io_service(acceptor)), + acceptor(std::move(acceptor)) { assert(socket_uses_io_service(this->acceptor, get_io_service())); set_socket_recv_buffer_size(this->acceptor, buffer_size); - this->acceptor.async_accept(peer, - std::bind(&tcp_reader::accept_handler, this, std::placeholders::_1)); + /* We need to hold the stream's queue_mutex, because that guards access + * to the sockets. This is a heavy-weight way to do it, but since it + * only happens once per connection it is probably not worth trying to + * add a lighter-weight interface to stream. + */ + using namespace std::placeholders; + this->acceptor.async_accept( + peer, + bind_handler(std::bind(&tcp_reader::accept_handler, this, _1, _2, _3))); } tcp_reader::tcp_reader( @@ -84,11 +92,11 @@ tcp_reader::tcp_reader( } void tcp_reader::packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, const boost::system::error_code &error, std::size_t bytes_transferred) { - stream_base::add_packet_state state(get_stream_base()); - bool read_more = false; if (!error) { @@ -106,12 +114,7 @@ void tcp_reader::packet_handler( log_warning("Error in TCP receiver: %1%", error.message()); if (read_more) - enqueue_receive(); - else - { - peer.close(); - stopped(); - } + enqueue_receive(std::move(ctx)); } bool tcp_reader::parse_packet(stream_base::add_packet_state &state) @@ -214,27 +217,19 @@ bool tcp_reader::skip_bytes() return to_skip > 0; } -void tcp_reader::accept_handler(const boost::system::error_code &error) +void tcp_reader::accept_handler(handler_context ctx, stream_base::add_packet_state &state, const boost::system::error_code &error) { - /* We need to hold the stream's queue_mutex, because that guards access - * to the sockets. This is a heavy-weight way to do it, but since it - * only happens once per connection it is probably not worth trying to - * add a lighter-weight interface to @c stream. - */ - stream_base::add_packet_state state(get_stream_base()); - acceptor.close(); if (!error) - enqueue_receive(); + enqueue_receive(std::move(ctx)); else { if (error != boost::asio::error::operation_aborted) log_warning("Error in TCP accept: %1%", error.message()); - stopped(); } } -void tcp_reader::enqueue_receive() +void tcp_reader::enqueue_receive(handler_context ctx) { using namespace std::placeholders; @@ -254,20 +249,7 @@ void tcp_reader::enqueue_receive() peer.async_receive( boost::asio::buffer(tail, bufsize - (tail - buf)), - std::bind(&tcp_reader::packet_handler, this, _1, _2)); -} - -void tcp_reader::stop() -{ - /* asio guarantees that closing a socket will cancel any pending - * operations on it. - * Don't put any logging here: it could be running in a shutdown - * path where it is no longer safe to do so. - */ - if (peer.is_open()) - peer.close(); - if (acceptor.is_open()) - acceptor.close(); + bind_handler(std::move(ctx), std::bind(&tcp_reader::packet_handler, this, _1, _2, _3, _4))); } bool tcp_reader::lossy() const diff --git a/src/recv_udp.cpp b/src/recv_udp.cpp index 473058fb1..1b18ff70d 100644 --- a/src/recv_udp.cpp +++ b/src/recv_udp.cpp @@ -1,4 +1,4 @@ -/* Copyright 2015, 2019-2020 National Research Foundation (SARAO) +/* Copyright 2015, 2019-2020, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -35,7 +35,7 @@ #include #include #include -#include +#include #include #include #include @@ -63,12 +63,13 @@ udp_reader::udp_reader( stream &owner, boost::asio::ip::udp::socket &&socket, std::size_t max_size) - : udp_reader_base(owner), socket(std::move(socket)), max_size(max_size), + : udp_reader_base(owner), max_size(max_size), #if SPEAD2_USE_RECVMMSG - buffer(mmsg_count), iov(mmsg_count), msgvec(mmsg_count) + buffer(mmsg_count), iov(mmsg_count), msgvec(mmsg_count), #else - buffer(new std::uint8_t[max_size + 1]) + buffer(new std::uint8_t[max_size + 1]), #endif + socket(std::move(socket)) { assert(socket_uses_io_service(this->socket, get_io_service())); #if SPEAD2_USE_RECVMMSG @@ -84,7 +85,7 @@ udp_reader::udp_reader( } #endif - enqueue_receive(); + enqueue_receive(make_handler_context()); } static boost::asio::ip::udp::socket make_bound_v4_socket( @@ -182,53 +183,43 @@ udp_reader::udp_reader( } void udp_reader::packet_handler( + handler_context ctx, + stream_base::add_packet_state &state, const boost::system::error_code &error, std::size_t bytes_transferred) { - stream_base::add_packet_state state(get_stream_base()); if (!error) { - if (state.is_stopped()) +#if SPEAD2_USE_RECVMMSG + int received = recvmmsg(socket.native_handle(), msgvec.data(), msgvec.size(), + MSG_DONTWAIT, nullptr); + log_debug("recvmmsg returned %1%", received); + if (received == -1 && errno != EAGAIN && errno != EWOULDBLOCK) { - log_info("UDP reader: discarding packet received after stream stopped"); + std::error_code code(errno, std::system_category()); + log_warning("recvmmsg failed: %1% (%2%)", code.value(), code.message()); } - else + for (int i = 0; i < received; i++) { -#if SPEAD2_USE_RECVMMSG - int received = recvmmsg(socket.native_handle(), msgvec.data(), msgvec.size(), - MSG_DONTWAIT, nullptr); - log_debug("recvmmsg returned %1%", received); - if (received == -1 && errno != EAGAIN && errno != EWOULDBLOCK) - { - std::error_code code(errno, std::system_category()); - log_warning("recvmmsg failed: %1% (%2%)", code.value(), code.message()); - } - for (int i = 0; i < received; i++) - { - bool stopped = process_one_packet(state, - buffer[i].get(), msgvec[i].msg_len, max_size); - if (stopped) - break; - } + bool stopped = process_one_packet(state, + buffer[i].get(), msgvec[i].msg_len, max_size); + if (stopped) + break; + } #else - process_one_packet(state, buffer.get(), bytes_transferred, max_size); + process_one_packet(state, buffer.get(), bytes_transferred, max_size); #endif - } } else if (error != boost::asio::error::operation_aborted) log_warning("Error in UDP receiver: %1%", error.message()); if (!state.is_stopped()) { - enqueue_receive(); - } - else - { - stopped(); + enqueue_receive(std::move(ctx)); } } -void udp_reader::enqueue_receive() +void udp_reader::enqueue_receive(handler_context ctx) { using namespace std::placeholders; socket.async_receive_from( @@ -238,17 +229,7 @@ void udp_reader::enqueue_receive() boost::asio::buffer(buffer.get(), max_size + 1), #endif endpoint, - std::bind(&udp_reader::packet_handler, this, _1, _2)); -} - -void udp_reader::stop() -{ - /* asio guarantees that closing a socket will cancel any pending - * operations on it. - * Don't put any logging here: it could be running in a shutdown - * path where it is no longer safe to do so. - */ - socket.close(); + bind_handler(std::move(ctx), std::bind(&udp_reader::packet_handler, this, _1, _2, _3, _4))); } ///////////////////////////////////////////////////////////////////////////// diff --git a/src/recv_udp_base.cpp b/src/recv_udp_base.cpp index f848a0296..913d3424a 100644 --- a/src/recv_udp_base.cpp +++ b/src/recv_udp_base.cpp @@ -1,4 +1,4 @@ -/* Copyright 2016, 2019 National Research Foundation (SARAO) +/* Copyright 2016, 2019, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -20,7 +20,6 @@ #include #include -#include #include #include #include diff --git a/src/recv_udp_ibv.cpp b/src/recv_udp_ibv.cpp index 58e9e08f0..1c3134dae 100644 --- a/src/recv_udp_ibv.cpp +++ b/src/recv_udp_ibv.cpp @@ -1,4 +1,4 @@ -/* Copyright 2016-2020 National Research Foundation (SARAO) +/* Copyright 2016-2020, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -35,7 +35,6 @@ #include #include #include -#include #include #include #include @@ -89,8 +88,7 @@ udp_ibv_reader_core::udp_ibv_reader_core( event_channel(nullptr), comp_channel_wrapper(owner.get_io_service()), max_size(config.get_max_size()), - max_poll(config.get_max_poll()), - stop_poll(false) + max_poll(config.get_max_poll()) { if (config.get_endpoints().empty()) throw std::invalid_argument("endpoints is empty"); @@ -120,14 +118,6 @@ void udp_ibv_reader_core::join_groups( } } -void udp_ibv_reader_core::stop() -{ - if (comp_channel) - comp_channel_wrapper.close(); - else - stop_poll = true; -} - } // namespace detail static std::size_t compute_n_slots(const rdma_cm_id_t &cm_id, std::size_t buffer_size, @@ -275,7 +265,7 @@ udp_ibv_reader::udp_ibv_reader( qp.post_recv(&slots[i].wr); } - enqueue_receive(true); + enqueue_receive(make_handler_context(), true); qp.modify(IBV_QPS_RTR); join_groups(config.get_endpoints(), config.get_interface_address()); } diff --git a/src/recv_udp_ibv_mprq.cpp b/src/recv_udp_ibv_mprq.cpp index 45da47479..356384c36 100644 --- a/src/recv_udp_ibv_mprq.cpp +++ b/src/recv_udp_ibv_mprq.cpp @@ -226,7 +226,7 @@ udp_ibv_mprq_reader::udp_ibv_mprq_reader( post_wr(i * wqe_size); flows = create_flows(qp, config.get_endpoints(), cm_id->port_num); - enqueue_receive(true); + enqueue_receive(make_handler_context(), true); join_groups(config.get_endpoints(), config.get_interface_address()); } diff --git a/src/recv_udp_pcap.cpp b/src/recv_udp_pcap.cpp index 4b292e187..5f484ef01 100644 --- a/src/recv_udp_pcap.cpp +++ b/src/recv_udp_pcap.cpp @@ -1,4 +1,4 @@ -/* Copyright 2016-2017, 2019 National Research Foundation (SARAO) +/* Copyright 2016-2017, 2019, 2023 National Research Foundation (SARAO) * * This program is free software: you can redistribute it and/or modify it under * the terms of the GNU Lesser General Public License as published by the Free @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include #include @@ -44,11 +44,10 @@ namespace spead2 namespace recv { -void udp_pcap_file_reader::run() +void udp_pcap_file_reader::run(handler_context ctx, stream_base::add_packet_state &state) { const int BATCH = 64; // maximum number of packets to process in one go - spead2::recv::stream_base::add_packet_state state(get_stream_base()); for (int pass = 0; pass < BATCH; pass++) { if (state.is_stopped()) @@ -93,9 +92,10 @@ void udp_pcap_file_reader::run() } // Run ourselves again if (!state.is_stopped()) - get_io_service().post([this] { run(); }); - else - stopped(); + { + using namespace std::placeholders; + boost::asio::post(get_io_service(), bind_handler(std::move(ctx), std::bind(&udp_pcap_file_reader::run, this, _1, _2))); + } } udp_pcap_file_reader::udp_pcap_file_reader(stream &owner, const std::string &filename, const std::string &user_filter) @@ -130,7 +130,8 @@ udp_pcap_file_reader::udp_pcap_file_reader(stream &owner, const std::string &fil udp_from_frame = (linktype == DLT_EN10MB) ? udp_from_ethernet : udp_from_linux_sll; // Process the file - get_io_service().post([this] { run(); }); + using namespace std::placeholders; + boost::asio::post(get_io_service(), bind_handler(std::bind(&udp_pcap_file_reader::run, this, _1, _2))); } udp_pcap_file_reader::~udp_pcap_file_reader() @@ -139,10 +140,6 @@ udp_pcap_file_reader::~udp_pcap_file_reader() pcap_close(handle); } -void udp_pcap_file_reader::stop() -{ -} - bool udp_pcap_file_reader::lossy() const { return false; diff --git a/src/spead2/recv/__init__.py b/src/spead2/recv/__init__.py index e785bd868..c72ecd1e8 100644 --- a/src/spead2/recv/__init__.py +++ b/src/spead2/recv/__init__.py @@ -1,4 +1,4 @@ -# Copyright 2015, 2020-2021 National Research Foundation (SARAO) +# Copyright 2015, 2020-2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -54,13 +54,40 @@ bytes, in the order they appeared in the original packet. """ +from collections.abc import Sequence as _Sequence + from spead2._spead2.recv import ( # noqa: F401 - StreamConfig, RingStreamConfig, Stream, Heap, IncompleteHeap, - Chunk, ChunkStreamConfig, ChunkRingStream, ChunkRingbuffer, - StreamStats, StreamStatConfig + Chunk, + ChunkRingPair, + ChunkRingStream, + ChunkRingbuffer, + ChunkStreamConfig, + ChunkStreamGroupConfig, + ChunkStreamGroupMember, + ChunkStreamRingGroup as _ChunkStreamRingGroup, + Heap, + IncompleteHeap, + RingStreamConfig, + Stream, + StreamConfig, + StreamStatConfig, + StreamStats, ) from . import stream_stat_indices # noqa: F401 try: from spead2._spead2.recv import UdpIbvConfig # noqa: F401 except ImportError: pass + + +# Ideally we'd inherit from _Sequence, but that gives errors about +# mismatched metaclasses. So instead we copy the mixin methods. +class ChunkStreamRingGroup(_ChunkStreamRingGroup): + count = _Sequence.count + index = _Sequence.index + __iter__ = _Sequence.__iter__ + __contains__ = _Sequence.__contains__ + __reversed__ = _Sequence.__reversed__ + + +_Sequence.register(ChunkStreamRingGroup) diff --git a/src/spead2/recv/__init__.pyi b/src/spead2/recv/__init__.pyi index 9f30fccc2..2de59e3d0 100644 --- a/src/spead2/recv/__init__.pyi +++ b/src/spead2/recv/__init__.pyi @@ -1,4 +1,4 @@ -# Copyright 2019-2022 National Research Foundation (SARAO) +# Copyright 2019-2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -13,6 +13,7 @@ # You should have received a copy of the GNU Lesser General Public License # along with this program. If not, see . +import collections.abc import enum import socket from typing import Iterator, Iterable, Any, List, Tuple, Sequence, Union, Text, Optional, ClassVar, overload @@ -255,14 +256,48 @@ class ChunkRingbuffer(_ChunkRingbuffer): def put(self, chunk: Chunk) -> None: ... def __iter__(self) -> Iterator[Chunk]: ... -class ChunkRingStream(_Stream): +class ChunkRingPair: + def add_free_chunk(self, chunk: Chunk) -> None: ... + @property + def data_ringbuffer(self) -> _ChunkRingbuffer: ... + @property + def free_ringbuffer(self) -> _ChunkRingbuffer: ... + +class ChunkRingStream(_Stream, ChunkRingPair): def __init__( self, thread_pool: spead2.ThreadPool, config: StreamConfig, - chunk_config: ChunkStreamConfig, + chunk_stream_config: ChunkStreamConfig, data_ringbuffer: _ChunkRingbuffer, free_ringbuffer: _ChunkRingbuffer) -> None: ... - def add_free_chunk(self, chunk: Chunk) -> None: ... +class ChunkStreamGroupConfig: + class EvictionMode(enum.Enum): + LOSSY = ... + LOSSLESS = ... + + DEFAULT_MAX_CHUNKS: ClassVar[int] @property - def data_ringbuffer(self) -> _ChunkRingbuffer: ... + def max_chunks(self) -> int: ... @property - def free_ringbuffer(self) -> _ChunkRingbuffer: ... + def eviction_mode(self) -> ChunkStreamGroupConfig.EvictionMode: ... + + def __init__(self, *, max_chunks=..., eviction_mode=...) -> None: ... + +class ChunkStreamRingGroup(ChunkRingPair, collections.abc.Sequence[ChunkStreamGroupMember]): + def __init__( + self, config: ChunkStreamGroupConfig, data_ringbuffer: _ChunkRingbuffer, + free_ringbuffer: _ChunkRingbuffer) -> None: ... + @property + def config(self) -> ChunkStreamGroupConfig: ... + def emplace_back( + self, thread_pool: spead2.ThreadPool, config: spead2.StreamConfig, + chunk_stream_config: spead2.ChunkStreamConfig) -> ChunkStreamGroupMember: ... + def stop(self) -> None: ... + # These are marked abstract in Sequence, so need to be implemented here + @overload + def __getitem__(self, index: int) -> ChunkStreamGroupMember: ... + @overload + def __getitem__(self, index: slice) -> Sequence[ChunkStreamGroupMember]: ... + def __len__(self) -> int: ... + +class ChunkStreamGroupMember(_Stream): + pass diff --git a/tests/test_passthrough.py b/tests/test_passthrough.py index 687965ea9..a8a6217b2 100644 --- a/tests/test_passthrough.py +++ b/tests/test_passthrough.py @@ -1,4 +1,4 @@ -# Copyright 2015, 2019-2022 National Research Foundation (SARAO) +# Copyright 2015, 2019-2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -508,7 +508,7 @@ def _interface_address(self): pytest.skip('Envar SPEAD2_TEST_IBV_INTERFACE_ADDRESS not set') return ifaddr - def setup(self): + def setup_method(self): # mlx5 drivers only enable multicast loopback if there are multiple # device contexts. The sender and receiver end up sharing one, so we # need to explicitly create another. @@ -516,7 +516,7 @@ def setup(self): pytest.skip('IBV support not compiled in') self._extra_context = spead2.IbvContext(self._interface_address()) - def teardown(self): + def teardown_method(self): self._extra_context.reset() def prepare_receivers(self, receivers): diff --git a/tests/test_recv.py b/tests/test_recv.py index caa2d7a7b..05214a70c 100644 --- a/tests/test_recv.py +++ b/tests/test_recv.py @@ -1,4 +1,4 @@ -# Copyright 2015, 2017, 2019-2022 National Research Foundation (SARAO) +# Copyright 2015, 2017, 2019-2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -209,7 +209,7 @@ def make_numpy_descriptor_from(self, id, name, description, array): class TestDecode: """Various types of descriptors must be correctly interpreted to decode data""" - def setup(self): + def setup_method(self): self.flavour = FLAVOUR def data_to_heaps(self, data, **kwargs): @@ -955,7 +955,7 @@ def test_deprecated_constants(self): class TestStream: """Tests for the stream API.""" - def setup(self): + def setup_method(self): self.flavour = FLAVOUR def test_full_stop(self): @@ -1167,7 +1167,7 @@ def test_illegal_udp_port(self): class TestTcpReader: - def setup(self): + def setup_method(self): self.receiver = recv.Stream(spead2.ThreadPool()) recv_sock = socket.socket() recv_sock.bind(("127.0.0.1", 0)) @@ -1179,7 +1179,7 @@ def setup(self): self.send_sock = socket.socket() self.send_sock.connect(("127.0.0.1", port)) - def teardown(self): + def teardown_method(self): self.close() def close(self): diff --git a/tests/test_recv_chunk_stream_group.py b/tests/test_recv_chunk_stream_group.py new file mode 100644 index 000000000..39a852d00 --- /dev/null +++ b/tests/test_recv_chunk_stream_group.py @@ -0,0 +1,396 @@ +# Copyright 2023 National Research Foundation (SARAO) +# +# This program is free software: you can redistribute it and/or modify it under +# the terms of the GNU Lesser General Public License as published by the Free +# Software Foundation, either version 3 of the License, or (at your option) any +# later version. +# +# This program is distributed in the hope that it will be useful, but WITHOUT +# ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS +# FOR A PARTICULAR PURPOSE. See the GNU Lesser General Public License for more +# details. +# +# You should have received a copy of the GNU Lesser General Public License +# along with this program. If not, see . + +import collections.abc +import ctypes +import gc +import threading +import time +import weakref + +import numba +from numba import types +import numpy as np +import pytest +import scipy + +import spead2 +import spead2.recv as recv +import spead2.send as send +from spead2.numba import intp_to_voidptr +from spead2.recv.numba import chunk_place_data + +from tests.test_recv_chunk_stream import ( + CHUNK_PAYLOAD_SIZE, HEAP_PAYLOAD_SIZE, HEAPS_PER_CHUNK, place_plain_llc +) + +STREAMS = 4 +LOSSY_PARAM = pytest.param(recv.ChunkStreamGroupConfig.EvictionMode.LOSSY, id="lossy") +LOSSLESS_PARAM = pytest.param(recv.ChunkStreamGroupConfig.EvictionMode.LOSSLESS, id="lossless") + + +@numba.cfunc( + types.void(types.CPointer(chunk_place_data), types.uintp, types.CPointer(types.int64)), + nopython=True) +def place_bias(data_ptr, data_size, user_data_ptr): + # Biases the chunk_id by the user parameter + data = numba.carray(data_ptr, 1) + items = numba.carray(intp_to_voidptr(data[0].items), 2, dtype=np.int64) + heap_cnt = items[0] + payload_size = items[1] + user_data = numba.carray(user_data_ptr, 1) + if payload_size == HEAP_PAYLOAD_SIZE: + data[0].chunk_id = heap_cnt // HEAPS_PER_CHUNK + user_data[0] + data[0].heap_index = heap_cnt % HEAPS_PER_CHUNK + data[0].heap_offset = data[0].heap_index * HEAP_PAYLOAD_SIZE + + +place_bias_llc = scipy.LowLevelCallable( + place_bias.ctypes, signature='void (void *, size_t, void *)') + + +class TestChunkStreamGroupConfig: + def test_default_construct(self): + config = recv.ChunkStreamGroupConfig() + assert config.max_chunks == config.DEFAULT_MAX_CHUNKS + assert config.eviction_mode == recv.ChunkStreamGroupConfig.EvictionMode.LOSSY + + def test_zero_max_chunks(self): + with pytest.raises(ValueError): + recv.ChunkStreamGroupConfig(max_chunks=0) + + def test_max_chunks(self): + config = recv.ChunkStreamGroupConfig(max_chunks=3) + assert config.max_chunks == 3 + config.max_chunks = 4 + assert config.max_chunks == 4 + + def test_eviction_mode(self): + EvictionMode = recv.ChunkStreamGroupConfig.EvictionMode + config = recv.ChunkStreamGroupConfig(eviction_mode=EvictionMode.LOSSLESS) + assert config.eviction_mode == EvictionMode.LOSSLESS + config.eviction_mode = EvictionMode.LOSSY + assert config.eviction_mode == EvictionMode.LOSSY + + +class TestChunkStreamRingGroupSequence: + """Test that ChunkStreamRingGroup behaves like a sequence.""" + + def make_group(self, n_streams): + group = spead2.recv.ChunkStreamRingGroup( + spead2.recv.ChunkStreamGroupConfig(), + spead2.recv.ChunkRingbuffer(4), + spead2.recv.ChunkRingbuffer(4) + ) + streams = [] + for _ in range(n_streams): + streams.append( + group.emplace_back( + spead2.ThreadPool(), + spead2.recv.StreamConfig(), + spead2.recv.ChunkStreamConfig(place=place_plain_llc) + ) + ) + return group, streams + + def test_len(self): + group, _ = self.make_group(5) + assert len(group) == 5 + + def test_getitem_simple(self): + group, streams = self.make_group(3) + assert group[0] is streams[0] + assert group[1] is streams[1] + assert group[2] is streams[2] + + def test_getitem_wrap(self): + group, streams = self.make_group(3) + assert group[-1] is streams[-1] + assert group[-2] is streams[-2] + assert group[-3] is streams[-3] + + def test_getitem_bad(self): + group, streams = self.make_group(3) + with pytest.raises(IndexError): + group[3] + with pytest.raises(IndexError): + group[-4] + + def test_getitem_slice(self): + group, streams = self.make_group(5) + assert group[1:3] == streams[1:3] + assert group[4:0:-2] == streams[4:0:-2] + assert group[1:-1:2] == streams[1:-1:2] + + def test_getitem_slice_gc(self): + """Test that the streams returned by getitem keep the group alive.""" + group = self.make_group(5)[0] + group_weak = weakref.ref(group) + streams = group[1:3] + del group + for i in range(5): # Try extra hard to GC on pypy + gc.collect() + assert group_weak() is not None + + # Now delete the things that are keeping it alive + streams.clear() + for i in range(5): + gc.collect() + assert group_weak() is None + + def test_iter(self): + group, streams = self.make_group(5) + assert list(group) == streams + + def test_reversed(self): + group, streams = self.make_group(5) + assert list(reversed(group)) == list(reversed(streams)) + + def test_contains(self): + group, streams = self.make_group(2) + assert streams[0] in group + assert streams[1] in group + assert None not in group + + def test_count(self): + group, streams = self.make_group(2) + assert group.count(streams[0]) == 1 + assert group.count(streams[1]) == 1 + assert group.count(group) == 0 + + def test_index(self): + group, streams = self.make_group(2) + assert group.index(streams[0]) == 0 + assert group.index(streams[1]) == 1 + assert group.index(streams[1], 1, 2) == 1 + with pytest.raises(ValueError): + group.index(None) + with pytest.raises(ValueError): + group.index(streams[0], 1) + + def test_registered(self): + assert issubclass(spead2.recv.ChunkStreamRingGroup, collections.abc.Sequence) + + +class TestChunkStreamRingGroup: + @pytest.fixture + def data_ring(self): + return spead2.recv.ChunkRingbuffer(4) + + @pytest.fixture + def free_ring(self): + ring = spead2.recv.ChunkRingbuffer(4) + while not ring.full(): + ring.put( + recv.Chunk( + present=np.zeros(HEAPS_PER_CHUNK, np.uint8), + data=np.zeros(CHUNK_PAYLOAD_SIZE, np.uint8) + ) + ) + return ring + + @pytest.fixture + def queues(self): + return [spead2.InprocQueue() for _ in range(STREAMS)] + + @pytest.fixture(params=[LOSSY_PARAM, LOSSLESS_PARAM]) + def eviction_mode(self, request): + return request.param + + @pytest.fixture + def chunk_id_bias(self): + return np.array([0], np.int64) + + @pytest.fixture + def group(self, eviction_mode, data_ring, free_ring, queues, chunk_id_bias): + group_config = recv.ChunkStreamGroupConfig(max_chunks=4, eviction_mode=eviction_mode) + group = recv.ChunkStreamRingGroup(group_config, data_ring, free_ring) + # max_heaps is artificially high to make test_packet_too_old work + config = spead2.recv.StreamConfig(max_heaps=128) + place_llc = scipy.LowLevelCallable( + place_bias.ctypes, + user_data=chunk_id_bias.ctypes.data_as(ctypes.c_void_p), + signature='void (void *, size_t, void *)') + chunk_stream_config = spead2.recv.ChunkStreamConfig( + items=[0x1000, spead2.HEAP_LENGTH_ID], + max_chunks=4, + place=place_llc, + ) + for queue in queues: + group.emplace_back( + spead2.ThreadPool(), + config=config, + chunk_stream_config=chunk_stream_config + ) + for stream, queue in zip(group, queues): + stream.add_inproc_reader(queue) + yield group + group.stop() + + @pytest.fixture + def send_stream(self, queues): + return send.InprocStream(spead2.ThreadPool(), queues, send.StreamConfig()) + + def _send_data(self, send_stream, data, eviction_mode, heaps=None): + """Send the data. + + To send only a subset of heaps (or to send out of order), pass the + indices to skip in `heaps`. + """ + lossy = (eviction_mode == recv.ChunkStreamGroupConfig.EvictionMode.LOSSY) + data_by_heap = data.reshape(-1, HEAP_PAYLOAD_SIZE) + ig = spead2.send.ItemGroup() + ig.add_item(0x1000, 'position', 'position in stream', (), format=[('u', 32)]) + ig.add_item(0x1001, 'payload', 'payload data', (HEAP_PAYLOAD_SIZE,), dtype=np.uint8) + # In lossy mode the behaviour is inherently non-deterministic. + # We just feed the data in slowly enough that we expect heaps provided + # before a sleep to be processed before those after the sleep. + for i in heaps: + ig['position'].value = i + ig['payload'].value = data_by_heap[i] + heap = ig.get_heap(data='all', descriptors='none') + send_stream.send_heap(heap, substream_index=i % STREAMS) + if lossy: + time.sleep(0.001) + + def _verify(self, group, data, expected_present, chunk_id_bias=0): + expected_present = expected_present.reshape(-1, HEAPS_PER_CHUNK) + expected_chunk_ids = np.nonzero(np.any(expected_present, axis=1))[0] + chunks = len(expected_present) + data_by_heap = data.reshape(chunks, HEAPS_PER_CHUNK, -1) + + def next_real_chunk(): + # Skip padding chunks + while True: + chunk = group.data_ringbuffer.get() + if any(chunk.present): + return chunk + else: + group.add_free_chunk(chunk) + + for i in expected_chunk_ids: + chunk = next_real_chunk() + assert chunk.chunk_id == i + chunk_id_bias + np.testing.assert_equal(chunk.present, expected_present[i]) + actual_data = chunk.data.reshape(HEAPS_PER_CHUNK, -1) + for j in range(HEAPS_PER_CHUNK): + if expected_present[i, j]: + np.testing.assert_equal(actual_data[j], data_by_heap[i, j]) + group.add_free_chunk(chunk) + + # Stopping all the queues should shut down the data ringbuffer + with pytest.raises(spead2.Stopped): + group.data_ringbuffer.get() + + def _test_simple(self, group, send_stream, chunks, heaps, chunk_id_bias=0): + """Send a given set of heaps (in order) and check that they arrive correctly.""" + rng = np.random.default_rng(seed=1) + data = rng.integers(0, 256, chunks * CHUNK_PAYLOAD_SIZE, np.uint8) + + def send(): + self._send_data(send_stream, data, group.config.eviction_mode, heaps) + # Stop all the queues, which should flush everything and stop the + # data ring. + for queue in send_stream.queues: + queue.stop() + + send_thread = threading.Thread(target=send) + send_thread.start() + + expected_present = np.zeros(chunks * HEAPS_PER_CHUNK, np.uint8) + expected_present[heaps] = True + self._verify(group, data, expected_present, chunk_id_bias) + + send_thread.join() + + def test_full_in_order(self, group, send_stream): + """Send all the data, in order.""" + chunks = 20 + heaps = list(range(chunks * HEAPS_PER_CHUNK)) + self._test_simple(group, send_stream, chunks, heaps) + + def test_missing_stream(self, group, send_stream): + """Skip sending data to one of the streams.""" + chunks = 20 + heaps = [i for i in range(chunks * HEAPS_PER_CHUNK) if i % STREAMS != 2] + self._test_simple(group, send_stream, chunks, heaps) + + def test_half_missing_stream(self, group, send_stream): + """Skip sending data to one of the streams after a certain point.""" + chunks = 20 + heaps = [ + i for i in range(chunks * HEAPS_PER_CHUNK) + if i < 7 * HEAPS_PER_CHUNK or i % STREAMS != 2 + ] + self._test_simple(group, send_stream, chunks, heaps) + + def test_missing_chunks(self, group, send_stream): + """Skip sending some whole chunks.""" + chunks = 20 + skip = [1, 6, 7, 13, 14, 15, 16, 17, 18] + heaps = [i for i in range(chunks * HEAPS_PER_CHUNK) if i // HEAPS_PER_CHUNK not in skip] + self._test_simple(group, send_stream, chunks, heaps) + + @pytest.mark.parametrize("eviction_mode", [LOSSLESS_PARAM]) + def test_lossless_late_stream(self, group, send_stream): + """Send one stream later than the others, to make sure lossless mode really works.""" + chunks = 20 + rng = np.random.default_rng(seed=1) + data = rng.integers(0, 256, chunks * CHUNK_PAYLOAD_SIZE, np.uint8) + heaps1 = [i for i in range(chunks * HEAPS_PER_CHUNK) if i % STREAMS != 2] + heaps2 = [i for i in range(chunks * HEAPS_PER_CHUNK) if i % STREAMS == 2] + + def send(): + self._send_data(send_stream, data, group.config.eviction_mode, heaps1) + time.sleep(0.01) + self._send_data(send_stream, data, group.config.eviction_mode, heaps2) + # Stop all the queues, which should flush everything and stop the + # data ring. + for queue in send_stream.queues: + queue.stop() + + send_thread = threading.Thread(target=send) + send_thread.start() + + expected_present = np.ones(chunks * HEAPS_PER_CHUNK, np.uint8) + self._verify(group, data, expected_present) + + send_thread.join() + + def test_large_chunk_ids(self, group, send_stream, chunk_id_bias): + chunks = 20 + heaps = list(range(chunks * HEAPS_PER_CHUNK)) + # Ensure that the last chunk will have the maximum possible chunk ID (2**63-1) + chunk_id_bias[0] = 2**63 - chunks + self._test_simple(group, send_stream, chunks, heaps, chunk_id_bias=chunk_id_bias[0]) + + def test_unblock_stop(self, group, send_stream): + """Stop the group without stopping the queues.""" + chunks = 20 + # Leave one stream half-missing, to really jam things up + n_heaps = chunks * HEAPS_PER_CHUNK + heaps = [i for i in range(n_heaps) if i < n_heaps // 2 or i % STREAMS != 2] + rng = np.random.default_rng(seed=1) + data = rng.integers(0, 256, chunks * CHUNK_PAYLOAD_SIZE, np.uint8) + + self._send_data(send_stream, data, group.config.eviction_mode, heaps) + time.sleep(0.01) # Give it time to consume some of the data + group.stop() + + # We don't care how many chunks we get, as long as the loop + # terminates. + for i, chunk in enumerate(group.data_ringbuffer): + assert chunk.chunk_id == i + group.add_free_chunk(chunk) diff --git a/tests/test_send.py b/tests/test_send.py index 5a011f820..8742dcdf2 100644 --- a/tests/test_send.py +++ b/tests/test_send.py @@ -1,4 +1,4 @@ -# Copyright 2015, 2019-2021 National Research Foundation (SARAO) +# Copyright 2015, 2019-2021, 2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -111,7 +111,7 @@ def offset_generator(fields): class TestEncode: """Test heap encoding of various data""" - def setup(self): + def setup_method(self): self.flavour = Flavour(4, 64, 48, 0) def test_empty(self): @@ -525,7 +525,7 @@ def test_bad_max_heaps(self): class TestStream: - def setup(self): + def setup_method(self): # A slow stream, so that we can test overflowing the queue self.flavour = Flavour(4, 64, 48, 0) self.stream = send.BytesStream( @@ -538,7 +538,7 @@ def setup(self): self.heap = ig.get_heap() self.threads = [] - def teardown(self): + def teardown_method(self): for thread in self.threads: thread.join() @@ -773,7 +773,7 @@ def test_failed_connect(self): class TestInprocStream: - def setup(self): + def setup_method(self): self.flavour = Flavour(4, 64, 48, 0) self.queue = spead2.InprocQueue() self.stream = send.InprocStream(spead2.ThreadPool(), [self.queue]) diff --git a/tests/test_send_asyncio.py b/tests/test_send_asyncio.py index 303e7180b..d8de79928 100644 --- a/tests/test_send_asyncio.py +++ b/tests/test_send_asyncio.py @@ -1,4 +1,4 @@ -# Copyright 2015, 2019-2022 National Research Foundation (SARAO) +# Copyright 2015, 2019-2023 National Research Foundation (SARAO) # # This program is free software: you can redistribute it and/or modify it under # the terms of the GNU Lesser General Public License as published by the Free @@ -29,7 +29,7 @@ @pytest.mark.asyncio class TestUdpStream: - def setup(self): + def setup_method(self): # Make a stream slow enough that we can test async interactions config = spead2.send.StreamConfig(rate=5e6) self.stream = UdpStream(spead2.ThreadPool(), [('localhost', 8888)], config)