Commit cc218aac authored by Dominik Charousset's avatar Dominik Charousset

Add new http::serve function

parent 2b7b17b8
...@@ -139,6 +139,7 @@ else() ...@@ -139,6 +139,7 @@ else()
endfunction() endfunction()
endif() endif()
add_net_example(http time-server)
add_net_example(web_socket echo) add_net_example(web_socket echo)
add_net_example(web_socket hello-client) add_net_example(web_socket hello-client)
add_net_example(web_socket quote-server) add_net_example(web_socket quote-server)
// Simple WebSocket server that sends everything it receives back to the sender.
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/caf_main.hpp"
#include "caf/deep_to_string.hpp"
#include "caf/event_based_actor.hpp"
#include "caf/net/http/serve.hpp"
#include "caf/net/middleman.hpp"
#include "caf/net/stream_transport.hpp"
#include "caf/net/tcp_accept_socket.hpp"
#include "caf/net/tcp_stream_socket.hpp"
#include "caf/scheduled_actor/flow.hpp"
#include <iostream>
#include <utility>
static constexpr uint16_t default_port = 8080;
struct config : caf::actor_system_config {
config() {
opt_group{custom_options_, "global"} //
.add<uint16_t>("port,p", "port to listen for incoming connections");
}
};
int caf_main(caf::actor_system& sys, const config& cfg) {
using namespace caf;
// Open up a TCP port for incoming connections.
auto port = caf::get_or(cfg, "port", default_port);
net::tcp_accept_socket fd;
if (auto maybe_fd = net::make_tcp_accept_socket({ipv4_address{}, port},
true)) {
std::cout << "*** started listening for incoming connections on port "
<< port << '\n';
fd = std::move(*maybe_fd);
} else {
std::cerr << "*** unable to open port " << port << ": "
<< to_string(maybe_fd.error()) << '\n';
return EXIT_FAILURE;
}
// Create buffers to signal events from the WebSocket server to the worker.
auto [worker_res, server_res] = net::http::make_request_resource();
// Spin up the HTTP server.
net::http::serve(sys, fd, std::move(server_res));
// Spin up a worker to handle the HTTP requests.
auto worker = sys.spawn([wres = worker_res](event_based_actor* self) {
// For each incoming request ...
wres
.observe_on(self) //
.for_each([](const net::http::request& req) {
// ... we simply return the current time as string.
// Note: we may not respond more than once to a single request.
auto str = caf::deep_to_string(make_timestamp());
req.respond(net::http::status::ok, "text/plain", str);
});
});
sys.await_all_actors_done();
return EXIT_SUCCESS;
}
CAF_MAIN(caf::net::middleman)
...@@ -131,6 +131,12 @@ public: ...@@ -131,6 +131,12 @@ public:
return pimpl_.compare(other.pimpl_); return pimpl_.compare(other.pimpl_);
} }
// -- utility ----------------------------------------------------------------
/// Erases each `x` from `xs` where `x.disposed()`.
/// @returns The number of erased elements.
static size_t erase_disposed(std::vector<disposable>& xs);
private: private:
intrusive_ptr<impl> pimpl_; intrusive_ptr<impl> pimpl_;
}; };
......
...@@ -10,6 +10,18 @@ ...@@ -10,6 +10,18 @@
namespace caf { namespace caf {
// -- member types -------------------------------------------------------------
disposable::impl::~impl() {
// nop
}
disposable disposable::impl::as_disposable() noexcept {
return disposable{intrusive_ptr<impl>{this}};
}
// -- factories ----------------------------------------------------------------
namespace { namespace {
class composite_impl : public ref_counted, public disposable::impl { class composite_impl : public ref_counted, public disposable::impl {
...@@ -52,14 +64,6 @@ private: ...@@ -52,14 +64,6 @@ private:
} // namespace } // namespace
disposable::impl::~impl() {
// nop
}
disposable disposable::impl::as_disposable() noexcept {
return disposable{intrusive_ptr<impl>{this}};
}
disposable disposable::make_composite(std::vector<disposable> entries) { disposable disposable::make_composite(std::vector<disposable> entries) {
if (entries.empty()) if (entries.empty())
return {}; return {};
...@@ -67,4 +71,18 @@ disposable disposable::make_composite(std::vector<disposable> entries) { ...@@ -67,4 +71,18 @@ disposable disposable::make_composite(std::vector<disposable> entries) {
return disposable{make_counted<composite_impl>(std::move(entries))}; return disposable{make_counted<composite_impl>(std::move(entries))};
} }
// -- utility ------------------------------------------------------------------
size_t disposable::erase_disposed(std::vector<disposable>& xs) {
auto is_disposed = [](auto& hdl) { return hdl.disposed(); };
auto xs_end = xs.end();
if (auto e = std::remove_if(xs.begin(), xs_end, is_disposed); e != xs_end) {
auto res = std::distance(e, xs_end);
xs.erase(e, xs_end);
return static_cast<size_t>(res);
} else {
return 0;
}
}
} // namespace caf } // namespace caf
...@@ -752,12 +752,9 @@ void scheduled_actor::run_actions() { ...@@ -752,12 +752,9 @@ void scheduled_actor::run_actions() {
void scheduled_actor::update_watched_disposables() { void scheduled_actor::update_watched_disposables() {
CAF_LOG_TRACE(""); CAF_LOG_TRACE("");
auto disposed = [](auto& hdl) { return hdl.disposed(); }; [[maybe_unused]] auto n = disposable::erase_disposed(watched_disposables_);
auto& xs = watched_disposables_; CAF_LOG_DEBUG_IF(n > 0, "now watching" << watched_disposables_.size()
if (auto e = std::remove_if(xs.begin(), xs.end(), disposed); e != xs.end()) { << "disposables");
xs.erase(e, xs.end());
CAF_LOG_DEBUG("now watching" << xs.size() << "disposables");
}
} }
} // namespace caf } // namespace caf
...@@ -39,6 +39,9 @@ caf_add_component( ...@@ -39,6 +39,9 @@ caf_add_component(
src/net/http/header.cpp src/net/http/header.cpp
src/net/http/lower_layer.cpp src/net/http/lower_layer.cpp
src/net/http/method.cpp src/net/http/method.cpp
src/net/http/request.cpp
src/net/http/response.cpp
src/net/http/serve.cpp
src/net/http/server.cpp src/net/http/server.cpp
src/net/http/status.cpp src/net/http/status.cpp
src/net/http/upper_layer.cpp src/net/http/upper_layer.cpp
......
...@@ -48,9 +48,8 @@ struct udp_datagram_socket; ...@@ -48,9 +48,8 @@ struct udp_datagram_socket;
// -- smart pointer aliases ---------------------------------------------------- // -- smart pointer aliases ----------------------------------------------------
using multiplexer_ptr = std::shared_ptr<multiplexer>; using multiplexer_ptr = intrusive_ptr<multiplexer>;
using socket_manager_ptr = intrusive_ptr<socket_manager>; using socket_manager_ptr = intrusive_ptr<socket_manager>;
using weak_multiplexer_ptr = std::weak_ptr<multiplexer>;
// -- miscellaneous aliases ---------------------------------------------------- // -- miscellaneous aliases ----------------------------------------------------
......
...@@ -11,6 +11,8 @@ ...@@ -11,6 +11,8 @@
#include "caf/net/generic_lower_layer.hpp" #include "caf/net/generic_lower_layer.hpp"
#include "caf/net/http/fwd.hpp" #include "caf/net/http/fwd.hpp"
#include <string_view>
namespace caf::net::http { namespace caf::net::http {
/// Parses HTTP requests and passes them to the upper layer. /// Parses HTTP requests and passes them to the upper layer.
...@@ -24,8 +26,15 @@ public: ...@@ -24,8 +26,15 @@ public:
/// Stops reading messages until calling `request_messages`. /// Stops reading messages until calling `request_messages`.
virtual void suspend_reading() = 0; virtual void suspend_reading() = 0;
/// Sends the next header to the client. /// Starts writing an HTTP header.
virtual bool send_header(status code, const header_fields_map& fields) = 0; virtual void begin_header(status code) = 0;
/// Adds a header field. Users may only call this function between
/// `begin_header` and `end_header`.
virtual void add_header_field(std::string_view key, std::string_view val) = 0;
/// Seals the header and transports it to the client.
virtual bool end_header() = 0;
/// Sends the payload after the header. /// Sends the payload after the header.
virtual bool send_payload(const_byte_span bytes) = 0; virtual bool send_payload(const_byte_span bytes) = 0;
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/async/execution_context.hpp"
#include "caf/async/promise.hpp"
#include "caf/byte_span.hpp"
#include "caf/detail/net_export.hpp"
#include "caf/net/http/fwd.hpp"
#include "caf/net/http/header.hpp"
#include "caf/net/http/response.hpp"
#include <cstdint>
#include <memory>
#include <string>
#include <string_view>
#include <type_traits>
namespace caf::net::http {
/// Handle type (implicitly shared) that represents an HTTP client request.
class CAF_NET_EXPORT request {
public:
struct impl {
header hdr;
std::vector<std::byte> body;
async::promise<response> prom;
};
request() = default;
request(request&&) = default;
request(const request&) = default;
request& operator=(request&&) = default;
request& operator=(const request&) = default;
/// @private
explicit request(std::shared_ptr<impl> pimpl) : pimpl_(std::move(pimpl)) {
// nop
}
/// Returns the HTTP header.
/// @pre `valid()`
const header& hdr() const {
return pimpl_->hdr;
}
/// Returns the HTTP body (payload).
/// @pre `valid()`
const_byte_span body() const {
return make_span(pimpl_->body);
}
/// Sends an HTTP response message to the client. Automatically sets the
/// `Content-Type` and `Content-Length` header fields.
/// @pre `valid()`
void respond(status code, std::string_view content_type,
const_byte_span content) const;
/// Sends an HTTP response message to the client. Automatically sets the
/// `Content-Type` and `Content-Length` header fields.
/// @pre `valid()`
void respond(status code, std::string_view content_type,
std::string_view content) const {
return respond(code, content_type, as_bytes(make_span(content)));
}
private:
std::shared_ptr<impl> pimpl_;
};
} // namespace caf::net::http
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/async/promise.hpp"
#include "caf/byte_buffer.hpp"
#include "caf/default_enum_inspect.hpp"
#include "caf/detail/net_export.hpp"
#include "caf/net/http/fwd.hpp"
#include "caf/span.hpp"
#include "caf/unordered_flat_map.hpp"
#include <cstdint>
#include <memory>
#include <string>
#include <type_traits>
namespace caf::net::http {
/// Handle type (implicitly shared) that represents an HTTP server response.
class CAF_NET_EXPORT response {
public:
using fields_map = unordered_flat_map<std::string, std::string>;
struct impl {
status code;
fields_map fields;
byte_buffer body;
};
response(status code, fields_map fields, byte_buffer body);
/// Returns the HTTP status code.
status code() const {
return pimpl_->code;
}
/// Returns the HTTP header fields.
span<const std::pair<std::string, std::string>> header_fields() const {
return pimpl_->fields.container();
}
/// Returns the HTTP body (payload).
const_byte_span body() const {
return make_span(pimpl_->body);
}
private:
std::shared_ptr<impl> pimpl_;
};
} // namespace caf::net::http
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/actor_system.hpp"
#include "caf/async/blocking_producer.hpp"
#include "caf/detail/atomic_ref_counted.hpp"
#include "caf/detail/net_export.hpp"
#include "caf/net/http/request.hpp"
#include "caf/net/http/server.hpp"
#include "caf/net/middleman.hpp"
#include <memory>
#include <type_traits>
namespace caf::detail {
class CAF_NET_EXPORT http_request_producer : public atomic_ref_counted,
public async::producer {
public:
using buffer_ptr = async::spsc_buffer_ptr<net::http::request>;
http_request_producer(buffer_ptr buf) : buf_(std::move(buf)) {
// nop
}
static auto make(buffer_ptr buf) {
auto ptr = make_counted<http_request_producer>(buf);
buf->set_producer(ptr);
return ptr;
}
void on_consumer_ready() override;
void on_consumer_cancel() override;
void on_consumer_demand(size_t) override;
void ref_producer() const noexcept override;
void deref_producer() const noexcept override;
bool push(const net::http::request& item);
private:
buffer_ptr buf_;
};
using http_request_producer_ptr = intrusive_ptr<http_request_producer>;
class CAF_NET_EXPORT http_flow_adapter : public net::http::upper_layer {
public:
explicit http_flow_adapter(http_request_producer_ptr ptr)
: producer_(std::move(ptr)) {
// nop
}
bool prepare_send() override;
bool done_sending() override;
void abort(const error& reason) override;
error init(net::socket_manager* owner, net::http::lower_layer* down,
const settings& config) override;
ptrdiff_t consume(const net::http::header& hdr,
const_byte_span payload) override;
static auto make(http_request_producer_ptr ptr) {
return std::make_unique<http_flow_adapter>(ptr);
}
private:
async::execution_context* parent_ = nullptr;
net::http::lower_layer* down_ = nullptr;
std::vector<disposable> pending_;
http_request_producer_ptr producer_;
};
template <class Transport>
class http_acceptor_factory {
public:
explicit http_acceptor_factory(http_request_producer_ptr producer)
: producer_(std::move(producer)) {
// nop
}
error init(net::socket_manager*, const settings&) {
return none;
}
template <class Socket>
net::socket_manager_ptr make(net::multiplexer* mpx, Socket fd) {
auto app = http_flow_adapter::make(producer_);
auto serv = net::http::server::make(std::move(app));
auto transport = Transport::make(fd, std::move(serv));
auto res = net::socket_manager::make(mpx, fd, std::move(transport));
mpx->watch(res->as_disposable());
return res;
}
void abort(const error&) {
// nop
}
private:
http_request_producer_ptr producer_;
};
} // namespace caf::detail
namespace caf::net::http {
/// Convenience function for creating async resources for connecting the HTTP
/// server to a worker.
inline auto make_request_resource() {
return async::make_spsc_buffer_resource<request>();
}
/// Listens for incoming HTTP requests on @p fd.
/// @param sys The host system.
/// @param fd An accept socket in listening mode. For a TCP socket, this socket
/// must already listen to a port.
/// @param out A buffer resource that connects the server to a listener that
/// processes the requests.
/// @param limit The maximum amount of connections before closing @p fd. Passing
/// 0 means "no limit".
template <class Transport = stream_transport, class Socket>
disposable serve(actor_system& sys, Socket fd,
async::producer_resource<request> out, size_t limit = 0) {
using factory_t = detail::http_acceptor_factory<Transport>;
using impl_t = connection_acceptor<Socket, factory_t>;
if (auto buf = out.try_open()) {
auto& mpx = sys.network_manager().mpx();
auto producer = detail::http_request_producer::make(std::move(buf));
auto factory = factory_t{std::move(producer)};
auto impl = impl_t::make(fd, limit, std::move(factory));
auto ptr = socket_manager::make(&mpx, fd, std::move(impl));
mpx.init(ptr);
return ptr->as_disposable();
} else {
return disposable{};
}
// TODO: serve() should return a disposable to stop the HTTP server.
}
} // namespace caf::net::http
...@@ -6,6 +6,7 @@ ...@@ -6,6 +6,7 @@
#include "caf/byte_span.hpp" #include "caf/byte_span.hpp"
#include "caf/detail/append_hex.hpp" #include "caf/detail/append_hex.hpp"
#include "caf/detail/net_export.hpp"
#include "caf/error.hpp" #include "caf/error.hpp"
#include "caf/logger.hpp" #include "caf/logger.hpp"
#include "caf/net/connection_acceptor.hpp" #include "caf/net/connection_acceptor.hpp"
...@@ -30,7 +31,8 @@ ...@@ -30,7 +31,8 @@
namespace caf::net::http { namespace caf::net::http {
/// Implements the server part for the HTTP Protocol as defined in RFC 7231. /// Implements the server part for the HTTP Protocol as defined in RFC 7231.
class server : public stream_oriented::upper_layer, public http::lower_layer { class CAF_NET_EXPORT server : public stream_oriented::upper_layer,
public http::lower_layer {
public: public:
// -- member types ----------------------------------------------------------- // -- member types -----------------------------------------------------------
...@@ -85,7 +87,11 @@ public: ...@@ -85,7 +87,11 @@ public:
void suspend_reading() override; void suspend_reading() override;
bool send_header(status code, const header_fields_map& fields) override; void begin_header(status code) override;
void add_header_field(std::string_view key, std::string_view val) override;
bool end_header() override;
bool send_payload(const_byte_span bytes) override; bool send_payload(const_byte_span bytes) override;
......
...@@ -20,18 +20,27 @@ namespace caf::net::http::v1 { ...@@ -20,18 +20,27 @@ namespace caf::net::http::v1 {
CAF_NET_EXPORT std::pair<std::string_view, byte_span> CAF_NET_EXPORT std::pair<std::string_view, byte_span>
split_header(byte_span bytes); split_header(byte_span bytes);
/// Writes an HTTP header to the buffer. /// Writes an HTTP header to @p buf.
CAF_NET_EXPORT void write_header(status code, const header_fields_map& fields, CAF_NET_EXPORT void write_header(status code, const header_fields_map& fields,
byte_buffer& buf); byte_buffer& buf);
/// Writes a complete HTTP response to the buffer. Automatically sets /// Write the status code for an HTTP header to @p buf.
/// Content-Type and Content-Length header fields. CAF_NET_EXPORT void begin_header(status code, byte_buffer& buf);
/// Write a header field to @p buf.
CAF_NET_EXPORT void add_header_field(std::string_view key, std::string_view val,
byte_buffer& buf);
/// Write the status code for an HTTP header to @buf.
CAF_NET_EXPORT bool end_header(byte_buffer& buf);
/// Writes a complete HTTP response to @p buf. Automatically sets Content-Type
/// and Content-Length header fields.
CAF_NET_EXPORT void write_response(status code, std::string_view content_type, CAF_NET_EXPORT void write_response(status code, std::string_view content_type,
std::string_view content, byte_buffer& buf); std::string_view content, byte_buffer& buf);
/// Writes a complete HTTP response to the buffer. Automatically sets /// Writes a complete HTTP response to @p buf. Automatically sets Content-Type
/// Content-Type and Content-Length header fields followed by the user-defined /// and Content-Length header fields followed by the user-defined @p fields.
/// @p fields.
CAF_NET_EXPORT void write_response(status code, std::string_view content_type, CAF_NET_EXPORT void write_response(status code, std::string_view content_type,
std::string_view content, std::string_view content,
const header_fields_map& fields, const header_fields_map& fields,
......
...@@ -69,19 +69,15 @@ public: ...@@ -69,19 +69,15 @@ public:
} }
multiplexer& mpx() noexcept { multiplexer& mpx() noexcept {
return mpx_; return *mpx_;
} }
const multiplexer& mpx() const noexcept { const multiplexer& mpx() const noexcept {
return mpx_; return *mpx_;
} }
multiplexer* mpx_ptr() noexcept { multiplexer* mpx_ptr() const noexcept {
return &mpx_; return mpx_.get();
}
const multiplexer* mpx_ptr() const noexcept {
return &mpx_;
} }
private: private:
...@@ -91,7 +87,7 @@ private: ...@@ -91,7 +87,7 @@ private:
actor_system& sys_; actor_system& sys_;
/// Stores the global socket I/O multiplexer. /// Stores the global socket I/O multiplexer.
multiplexer mpx_; multiplexer_ptr mpx_;
/// Runs the multiplexer's event loop /// Runs the multiplexer's event loop
std::thread mpx_thread_; std::thread mpx_thread_;
......
...@@ -9,6 +9,8 @@ ...@@ -9,6 +9,8 @@
#include <thread> #include <thread>
#include "caf/action.hpp" #include "caf/action.hpp"
#include "caf/async/execution_context.hpp"
#include "caf/detail/atomic_ref_counted.hpp"
#include "caf/detail/net_export.hpp" #include "caf/detail/net_export.hpp"
#include "caf/net/fwd.hpp" #include "caf/net/fwd.hpp"
#include "caf/net/operation.hpp" #include "caf/net/operation.hpp"
...@@ -28,7 +30,8 @@ namespace caf::net { ...@@ -28,7 +30,8 @@ namespace caf::net {
class pollset_updater; class pollset_updater;
/// Multiplexes any number of ::socket_manager objects with a ::socket. /// Multiplexes any number of ::socket_manager objects with a ::socket.
class CAF_NET_EXPORT multiplexer { class CAF_NET_EXPORT multiplexer : public detail::atomic_ref_counted,
public async::execution_context {
public: public:
// -- member types ----------------------------------------------------------- // -- member types -----------------------------------------------------------
...@@ -55,13 +58,18 @@ public: ...@@ -55,13 +58,18 @@ public:
// -- constructors, destructors, and assignment operators -------------------- // -- constructors, destructors, and assignment operators --------------------
~multiplexer();
// -- factories --------------------------------------------------------------
/// @param parent Points to the owning middleman instance. May be `nullptr` /// @param parent Points to the owning middleman instance. May be `nullptr`
/// only for the purpose of unit testing if no @ref /// only for the purpose of unit testing if no @ref
/// socket_manager requires access to the @ref middleman or the /// socket_manager requires access to the @ref middleman or the
/// @ref actor_system. /// @ref actor_system.
explicit multiplexer(middleman* parent); static multiplexer_ptr make(middleman* parent) {
auto ptr = new multiplexer(parent);
~multiplexer(); return multiplexer_ptr{ptr, false};
}
// -- initialization --------------------------------------------------------- // -- initialization ---------------------------------------------------------
...@@ -87,34 +95,33 @@ public: ...@@ -87,34 +95,33 @@ public:
/// Computes the current mask for the manager. Mostly useful for testing. /// Computes the current mask for the manager. Mostly useful for testing.
operation mask_of(const socket_manager_ptr& mgr); operation mask_of(const socket_manager_ptr& mgr);
// -- implementation of execution_context ------------------------------------
void ref_execution_context() const noexcept override;
void deref_execution_context() const noexcept override;
void schedule(action what) override;
void watch(disposable what) override;
// -- thread-safe signaling -------------------------------------------------- // -- thread-safe signaling --------------------------------------------------
/// Schedules a call to `mgr->handle_error(sec::disposed)`. /// Schedules a call to `mgr->handle_error(sec::disposed)`.
/// @thread-safe /// @thread-safe
void dispose(const socket_manager_ptr& mgr); void dispose(socket_manager_ptr mgr);
/// Stops further reading by `mgr`. /// Stops further reading by `mgr`.
/// @thread-safe /// @thread-safe
void shutdown_reading(const socket_manager_ptr& mgr); void shutdown_reading(socket_manager_ptr mgr);
/// Stops further writing by `mgr`. /// Stops further writing by `mgr`.
/// @thread-safe /// @thread-safe
void shutdown_writing(const socket_manager_ptr& mgr); void shutdown_writing(socket_manager_ptr mgr);
/// Schedules an action for execution on this multiplexer.
/// @thread-safe
void schedule(const action& what);
/// Schedules an action for execution on this multiplexer.
/// @thread-safe
template <class F>
void schedule_fn(F f) {
schedule(make_action(std::move(f)));
}
/// Registers `mgr` for initialization in the multiplexer's thread. /// Registers `mgr` for initialization in the multiplexer's thread.
/// @thread-safe /// @thread-safe
void init(const socket_manager_ptr& mgr); void init(socket_manager_ptr mgr);
/// Signals the multiplexer to initiate shutdown. /// Signals the multiplexer to initiate shutdown.
/// @thread-safe /// @thread-safe
...@@ -176,6 +183,7 @@ protected: ...@@ -176,6 +183,7 @@ protected:
/// Writes `opcode` and pointer to `mgr` the the pipe for handling an event /// Writes `opcode` and pointer to `mgr` the the pipe for handling an event
/// later via the pollset updater. /// later via the pollset updater.
/// @warning assumes ownership of @p ptr.
template <class T> template <class T>
void write_to_pipe(uint8_t opcode, T* ptr); void write_to_pipe(uint8_t opcode, T* ptr);
...@@ -217,7 +225,14 @@ protected: ...@@ -217,7 +225,14 @@ protected:
/// Signals whether shutdown has been requested. /// Signals whether shutdown has been requested.
bool shutting_down_ = false; bool shutting_down_ = false;
/// Keeps track of watched disposables.
std::vector<disposable> watched_;
private: private:
// -- constructors, destructors, and assignment operators --------------------
explicit multiplexer(middleman* parent);
// -- internal callbacks the pollset updater --------------------------------- // -- internal callbacks the pollset updater ---------------------------------
void do_shutdown(); void do_shutdown();
......
...@@ -60,7 +60,7 @@ disposable serve(actor_system& sys, Socket fd) { ...@@ -60,7 +60,7 @@ disposable serve(actor_system& sys, Socket fd) {
auto impl = impl_t::make(fd, 0, std::move(factory)); auto impl = impl_t::make(fd, 0, std::move(factory));
auto mgr = socket_manager::make(mpx, fd, std::move(impl)); auto mgr = socket_manager::make(mpx, fd, std::move(impl));
mpx->init(mgr); mpx->init(mgr);
return mgr->make_disposer(); return mgr->as_disposable();
} }
} // namespace caf::net::prometheus } // namespace caf::net::prometheus
...@@ -8,8 +8,10 @@ ...@@ -8,8 +8,10 @@
#include "caf/actor.hpp" #include "caf/actor.hpp"
#include "caf/actor_system.hpp" #include "caf/actor_system.hpp"
#include "caf/callback.hpp" #include "caf/callback.hpp"
#include "caf/detail/atomic_ref_counted.hpp"
#include "caf/detail/infer_actor_shell_ptr_type.hpp" #include "caf/detail/infer_actor_shell_ptr_type.hpp"
#include "caf/detail/net_export.hpp" #include "caf/detail/net_export.hpp"
#include "caf/disposable.hpp"
#include "caf/error.hpp" #include "caf/error.hpp"
#include "caf/fwd.hpp" #include "caf/fwd.hpp"
#include "caf/net/actor_shell.hpp" #include "caf/net/actor_shell.hpp"
...@@ -17,7 +19,6 @@ ...@@ -17,7 +19,6 @@
#include "caf/net/socket.hpp" #include "caf/net/socket.hpp"
#include "caf/net/socket_event_layer.hpp" #include "caf/net/socket_event_layer.hpp"
#include "caf/net/typed_actor_shell.hpp" #include "caf/net/typed_actor_shell.hpp"
#include "caf/ref_counted.hpp"
#include "caf/sec.hpp" #include "caf/sec.hpp"
#include <type_traits> #include <type_traits>
...@@ -25,7 +26,8 @@ ...@@ -25,7 +26,8 @@
namespace caf::net { namespace caf::net {
/// Manages the lifetime of a single socket and handles any I/O events on it. /// Manages the lifetime of a single socket and handles any I/O events on it.
class CAF_NET_EXPORT socket_manager : public ref_counted { class CAF_NET_EXPORT socket_manager : public detail::atomic_ref_counted,
public disposable_impl {
public: public:
// -- member types ----------------------------------------------------------- // -- member types -----------------------------------------------------------
...@@ -74,10 +76,6 @@ public: ...@@ -74,10 +76,6 @@ public:
return make_actor_shell<Handle>(std::move(f)); return make_actor_shell<Handle>(std::move(f));
} }
/// Returns a thread-safe disposer for stopping the socket manager from an
/// outside context.
disposable make_disposer();
// -- properties ------------------------------------------------------------- // -- properties -------------------------------------------------------------
/// Returns the handle for the managed socket. /// Returns the handle for the managed socket.
...@@ -193,6 +191,16 @@ public: ...@@ -193,6 +191,16 @@ public:
/// @param code The error code as reported by the operating system. /// @param code The error code as reported by the operating system.
void handle_error(sec code); void handle_error(sec code);
// -- implementation of disposable_impl --------------------------------------
void dispose() override;
bool disposed() const noexcept override;
void ref_disposable() const noexcept override;
void deref_disposable() const noexcept override;
private: private:
// -- utility functions ------------------------------------------------------ // -- utility functions ------------------------------------------------------
...@@ -204,7 +212,11 @@ private: ...@@ -204,7 +212,11 @@ private:
/// the socket in its destructor. /// the socket in its destructor.
socket fd_; socket fd_;
/// Points to the multiplexer that owns this manager. /// Points to the multiplexer that executes this manager. Note: we do not need
/// to increase the reference count for the multiplexer, because the
/// multiplexer owns all managers in the sense that calling any member
/// function on a socket manager may not occur if the actor system has shut
/// down (and the multiplexer is part of the actor system).
multiplexer* mpx_; multiplexer* mpx_;
/// Stores flags for the socket file descriptor. /// Stores flags for the socket file descriptor.
...@@ -212,6 +224,8 @@ private: ...@@ -212,6 +224,8 @@ private:
/// Stores the event handler that operators on the socket file descriptor. /// Stores the event handler that operators on the socket file descriptor.
event_handler_ptr handler_; event_handler_ptr handler_;
std::atomic<bool> disposed_;
}; };
/// @relates socket_manager /// @relates socket_manager
......
...@@ -21,10 +21,10 @@ lower_layer::~lower_layer() { ...@@ -21,10 +21,10 @@ lower_layer::~lower_layer() {
bool lower_layer::send_response(status code, std::string_view content_type, bool lower_layer::send_response(status code, std::string_view content_type,
const_byte_span content) { const_byte_span content) {
auto content_size = std::to_string(content.size()); auto content_size = std::to_string(content.size());
header_fields_map fields; begin_header(code);
fields.emplace("Content-Type"sv, content_type); add_header_field("Content-Type"sv, content_type);
fields.emplace("Content-Length"sv, content_size); add_header_field("Content-Length"sv, content_size);
return send_header(code, fields) && send_payload(content); return end_header();
} }
bool lower_layer::send_response(status code, std::string_view content_type, bool lower_layer::send_response(status code, std::string_view content_type,
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#include "caf/net/http/request.hpp"
using namespace std::literals;
// TODO: reduce number of memory allocations for producing the response.
namespace caf::net::http {
void request::respond(status code, std::string_view content_type,
const_byte_span content) const {
auto content_size = std::to_string(content.size());
unordered_flat_map<std::string, std::string> fields;
fields.emplace("Content-Type"sv, content_type);
fields.emplace("Content-Length"sv, content_size);
auto body = std::vector<std::byte>{content.begin(), content.end()};
pimpl_->prom.set_value(response{code, std::move(fields), std::move(body)});
}
} // namespace caf::net::http
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#include "caf/net/http/response.hpp"
namespace caf::net::http {
response::response(status code, fields_map fm, std::vector<std::byte> body) {
pimpl_ = std::make_shared<impl>(impl{code, std::move(fm), std::move(body)});
}
} // namespace caf::net::http
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#include "caf/net/http/serve.hpp"
namespace caf::detail {
// TODO: there is currently no back-pressure from the worker to the server.
// -- http_request_producer ----------------------------------------------------
void http_request_producer::on_consumer_ready() {
// nop
}
void http_request_producer::on_consumer_cancel() {
}
void http_request_producer::on_consumer_demand(size_t) {
// nop
}
void http_request_producer::ref_producer() const noexcept {
ref();
}
void http_request_producer::deref_producer() const noexcept {
deref();
}
bool http_request_producer::push(const net::http::request& item) {
return buf_->push(item);
}
// -- http_flow_adapter --------------------------------------------------------
bool http_flow_adapter::prepare_send() {
return true;
}
bool http_flow_adapter::done_sending() {
return true;
}
void http_flow_adapter::abort(const error&) {
for (auto& pending : pending_)
pending.dispose();
}
error http_flow_adapter::init(net::socket_manager* owner,
net::http::lower_layer* down, const settings&) {
parent_ = owner->mpx_ptr();
down_ = down;
down_->request_messages();
return none;
}
ptrdiff_t http_flow_adapter::consume(const net::http::header& hdr,
const_byte_span payload) {
using namespace net::http;
auto prom = async::promise<response>();
auto fut = prom.get_future();
auto buf = std::vector<std::byte>{payload.begin(), payload.end()};
auto impl = request::impl{hdr, std::move(buf), std::move(prom)};
producer_->push(request{std::make_shared<request::impl>(std::move(impl))});
auto hdl = fut.bind_to(parent_).then(
[this](const response& res) {
down_->begin_header(res.code());
for (auto& [key, val] : res.header_fields())
down_->add_header_field(key, val);
std::ignore = down_->end_header();
down_->send_payload(res.body());
// TODO: we should close the connection unless indicated otherwise
// (keepalive flag?). Also, we should clean up pending_.
},
[this](const error& err) {
auto description = to_string(err);
down_->send_response(status::internal_server_error, "text/plain",
description);
// TODO: see above.
});
pending_.emplace_back(std::move(hdl));
return static_cast<ptrdiff_t>(payload.size());
}
} // namespace caf::detail
...@@ -31,11 +31,17 @@ void server::suspend_reading() { ...@@ -31,11 +31,17 @@ void server::suspend_reading() {
down_->configure_read(receive_policy::stop()); down_->configure_read(receive_policy::stop());
} }
bool server::send_header(status code, const header_fields_map& fields) { void server::begin_header(status code) {
down_->begin_output(); down_->begin_output();
v1::write_header(code, fields, down_->output_buffer()); v1::begin_header(code, down_->output_buffer());
down_->end_output(); }
return true;
void server::add_header_field(std::string_view key, std::string_view val) {
v1::add_header_field(key, val, down_->output_buffer());
}
bool server::end_header() {
return v1::end_header(down_->output_buffer()) && down_->end_output();
} }
bool server::send_payload(const_byte_span bytes) { bool server::send_payload(const_byte_span bytes) {
......
...@@ -61,6 +61,24 @@ void write_header(status code, const header_fields_map& fields, ...@@ -61,6 +61,24 @@ void write_header(status code, const header_fields_map& fields,
out << "\r\n"sv; out << "\r\n"sv;
} }
void begin_header(status code, byte_buffer& buf) {
writer out{&buf};
out << "HTTP/1.1 "sv << std::to_string(static_cast<int>(code)) << ' '
<< phrase(code) << "\r\n"sv;
}
void add_header_field(std::string_view key, std::string_view val,
byte_buffer& buf) {
writer out{&buf};
out << key << ": "sv << val << "\r\n"sv;
}
bool end_header(byte_buffer& buf) {
writer out{&buf};
out << "\r\n"sv;
return true;
}
void write_response(status code, std::string_view content_type, void write_response(status code, std::string_view content_type,
std::string_view content, byte_buffer& buf) { std::string_view content, byte_buffer& buf) {
header_fields_map fields; header_fields_map fields;
......
...@@ -55,7 +55,8 @@ void middleman::init_global_meta_objects() { ...@@ -55,7 +55,8 @@ void middleman::init_global_meta_objects() {
// nop // nop
} }
middleman::middleman(actor_system& sys) : sys_(sys), mpx_(this) { middleman::middleman(actor_system& sys)
: sys_(sys), mpx_(multiplexer::make(this)) {
// nop // nop
} }
...@@ -69,28 +70,28 @@ void middleman::start() { ...@@ -69,28 +70,28 @@ void middleman::start() {
CAF_SET_LOGGER_SYS(&sys_); CAF_SET_LOGGER_SYS(&sys_);
detail::set_thread_name("caf.net.mpx"); detail::set_thread_name("caf.net.mpx");
sys_.thread_started(); sys_.thread_started();
mpx_.set_thread_id(); mpx_->set_thread_id();
launch_background_tasks(sys_); launch_background_tasks(sys_);
mpx_.run(); mpx_->run();
sys_.thread_terminates(); sys_.thread_terminates();
}}; }};
} else { } else {
mpx_.set_thread_id(); mpx_->set_thread_id();
} }
} }
void middleman::stop() { void middleman::stop() {
mpx_.shutdown(); mpx_->shutdown();
if (mpx_thread_.joinable()) if (mpx_thread_.joinable())
mpx_thread_.join(); mpx_thread_.join();
else else
mpx_.run(); mpx_->run();
} }
void middleman::init(actor_system_config& cfg) { void middleman::init(actor_system_config& cfg) {
if (auto err = mpx_.init()) { if (auto err = mpx_->init()) {
CAF_LOG_ERROR("mpx_.init() failed: " << err); CAF_LOG_ERROR("mpx_->init() failed: " << err);
CAF_RAISE_ERROR("mpx_.init() failed"); CAF_RAISE_ERROR("mpx_->init() failed");
} }
if (auto node_uri = get_if<uri>(&cfg, "caf.middleman.this-node")) { if (auto node_uri = get_if<uri>(&cfg, "caf.middleman.this-node")) {
auto this_node = make_node_id(std::move(*node_uri)); auto this_node = make_node_id(std::move(*node_uri));
......
...@@ -165,46 +165,61 @@ operation multiplexer::mask_of(const socket_manager_ptr& mgr) { ...@@ -165,46 +165,61 @@ operation multiplexer::mask_of(const socket_manager_ptr& mgr) {
return to_operation(mgr, std::nullopt); return to_operation(mgr, std::nullopt);
} }
// -- implementation of execution_context --------------------------------------
void multiplexer::ref_execution_context() const noexcept {
ref();
}
void multiplexer::deref_execution_context() const noexcept {
deref();
}
void multiplexer::schedule(action what) {
CAF_LOG_TRACE("");
auto ptr = std::move(what).as_intrusive_ptr().release();
write_to_pipe(pollset_updater::code::run_action, ptr);
}
void multiplexer::watch(disposable what) {
watched_.emplace_back(what);
}
// -- thread-safe signaling ---------------------------------------------------- // -- thread-safe signaling ----------------------------------------------------
void multiplexer::dispose(const socket_manager_ptr& mgr) { void multiplexer::dispose(socket_manager_ptr mgr) {
CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id)); CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id));
if (std::this_thread::get_id() == tid_) { if (std::this_thread::get_id() == tid_) {
do_dispose(mgr); do_dispose(mgr);
} else { } else {
write_to_pipe(pollset_updater::code::dispose_manager, mgr.get()); write_to_pipe(pollset_updater::code::dispose_manager, mgr.release());
} }
} }
void multiplexer::shutdown_reading(const socket_manager_ptr& mgr) { void multiplexer::shutdown_reading(socket_manager_ptr mgr) {
CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id)); CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id));
if (std::this_thread::get_id() == tid_) { if (std::this_thread::get_id() == tid_) {
do_shutdown_reading(mgr); do_shutdown_reading(mgr);
} else { } else {
write_to_pipe(pollset_updater::code::shutdown_reading, mgr.get()); write_to_pipe(pollset_updater::code::shutdown_reading, mgr.release());
} }
} }
void multiplexer::shutdown_writing(const socket_manager_ptr& mgr) { void multiplexer::shutdown_writing(socket_manager_ptr mgr) {
CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id)); CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id));
if (std::this_thread::get_id() == tid_) { if (std::this_thread::get_id() == tid_) {
do_shutdown_writing(mgr); do_shutdown_writing(mgr);
} else { } else {
write_to_pipe(pollset_updater::code::shutdown_writing, mgr.get()); write_to_pipe(pollset_updater::code::shutdown_writing, mgr.release());
} }
} }
void multiplexer::schedule(const action& what) { void multiplexer::init(socket_manager_ptr mgr) {
CAF_LOG_TRACE("");
write_to_pipe(pollset_updater::code::run_action, what.ptr());
}
void multiplexer::init(const socket_manager_ptr& mgr) {
CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id)); CAF_LOG_TRACE(CAF_ARG2("socket", mgr->handle().id));
if (std::this_thread::get_id() == tid_) { if (std::this_thread::get_id() == tid_) {
do_init(mgr); do_init(mgr);
} else { } else {
write_to_pipe(pollset_updater::code::init_manager, mgr.get()); write_to_pipe(pollset_updater::code::init_manager, mgr.release());
} }
} }
...@@ -358,8 +373,10 @@ void multiplexer::run() { ...@@ -358,8 +373,10 @@ void multiplexer::run() {
// need to block the signal at thread level since some APIs (such as OpenSSL) // need to block the signal at thread level since some APIs (such as OpenSSL)
// are unsafe to call otherwise. // are unsafe to call otherwise.
block_sigpipe(); block_sigpipe();
while (!shutting_down_ || pollset_.size() > 1) while (!shutting_down_ || pollset_.size() > 1 || !watched_.empty()) {
poll_once(true); poll_once(true);
disposable::erase_disposed(watched_);
}
// Close the pipe to block any future event. // Close the pipe to block any future event.
std::lock_guard<std::mutex> guard{write_lock_}; std::lock_guard<std::mutex> guard{write_lock_};
if (write_handle_ != invalid_socket) { if (write_handle_ != invalid_socket) {
...@@ -427,9 +444,7 @@ multiplexer::poll_update& multiplexer::update_for(socket_manager* mgr) { ...@@ -427,9 +444,7 @@ multiplexer::poll_update& multiplexer::update_for(socket_manager* mgr) {
template <class T> template <class T>
void multiplexer::write_to_pipe(uint8_t opcode, T* ptr) { void multiplexer::write_to_pipe(uint8_t opcode, T* ptr) {
pollset_updater::msg_buf buf; pollset_updater::msg_buf buf;
if (ptr) { // Note: no intrusive_ptr_add_ref(ptr) since we take ownership of `ptr`.
intrusive_ptr_add_ref(ptr);
}
buf[0] = static_cast<std::byte>(opcode); buf[0] = static_cast<std::byte>(opcode);
auto value = reinterpret_cast<intptr_t>(ptr); auto value = reinterpret_cast<intptr_t>(ptr);
memcpy(buf.data() + 1, &value, sizeof(intptr_t)); memcpy(buf.data() + 1, &value, sizeof(intptr_t));
......
...@@ -15,7 +15,7 @@ namespace caf::net { ...@@ -15,7 +15,7 @@ namespace caf::net {
socket_manager::socket_manager(multiplexer* mpx, socket fd, socket_manager::socket_manager(multiplexer* mpx, socket fd,
event_handler_ptr handler) event_handler_ptr handler)
: fd_(fd), mpx_(mpx), handler_(std::move(handler)) { : fd_(fd), mpx_(mpx), handler_(std::move(handler)), disposed_(false) {
CAF_ASSERT(fd_ != invalid_socket); CAF_ASSERT(fd_ != invalid_socket);
CAF_ASSERT(mpx_ != nullptr); CAF_ASSERT(mpx_ != nullptr);
CAF_ASSERT(handler_ != nullptr); CAF_ASSERT(handler_ != nullptr);
...@@ -31,50 +31,8 @@ socket_manager::~socket_manager() { ...@@ -31,50 +31,8 @@ socket_manager::~socket_manager() {
socket_manager_ptr socket_manager::make(multiplexer* mpx, socket handle, socket_manager_ptr socket_manager::make(multiplexer* mpx, socket handle,
event_handler_ptr handler) { event_handler_ptr handler) {
CAF_ASSERT(mpx != nullptr); CAF_ASSERT(mpx != nullptr);
return make_counted<socket_manager>(mpx, handle, std::move(handler)); return make_counted<socket_manager>(std::move(mpx), handle,
} std::move(handler));
namespace {
class disposer : public detail::atomic_ref_counted, public disposable_impl {
public:
disposer(multiplexer* mpx, socket_manager_ptr mgr)
: mpx_(mpx), mgr_(std::move(mgr)) {
// nop
}
void dispose() {
std::unique_lock guard{mtx_};
if (mpx_) {
mpx_->dispose(mgr_);
mpx_ = nullptr;
mgr_ = nullptr;
}
}
bool disposed() const noexcept {
std::unique_lock guard{mtx_};
return mpx_ == nullptr;
}
void ref_disposable() const noexcept {
ref();
}
void deref_disposable() const noexcept {
deref();
}
private:
mutable std::mutex mtx_;
multiplexer* mpx_;
socket_manager_ptr mgr_;
};
} // namespace
disposable socket_manager::make_disposer() {
return disposable{make_counted<disposer>(mpx_, this)};
} }
// -- properties --------------------------------------------------------------- // -- properties ---------------------------------------------------------------
...@@ -129,6 +87,7 @@ void socket_manager::shutdown_write() { ...@@ -129,6 +87,7 @@ void socket_manager::shutdown_write() {
void socket_manager::shutdown() { void socket_manager::shutdown() {
flags_.read_closed = true; flags_.read_closed = true;
flags_.write_closed = true; flags_.write_closed = true;
disposed_ = true;
deregister(); deregister();
} }
...@@ -158,11 +117,15 @@ void socket_manager::schedule(action what) { ...@@ -158,11 +117,15 @@ void socket_manager::schedule(action what) {
void socket_manager::close_read() noexcept { void socket_manager::close_read() noexcept {
// TODO: extend transport API for closing read operations. // TODO: extend transport API for closing read operations.
flags_.read_closed = true; flags_.read_closed = true;
if (flags_.write_closed)
disposed_ = true;
} }
void socket_manager::close_write() noexcept { void socket_manager::close_write() noexcept {
// TODO: extend transport API for closing write operations. // TODO: extend transport API for closing write operations.
flags_.write_closed = true; flags_.write_closed = true;
if (flags_.read_closed)
disposed_ = true;
} }
error socket_manager::init(const settings& cfg) { error socket_manager::init(const settings& cfg) {
...@@ -186,9 +149,29 @@ void socket_manager::handle_error(sec code) { ...@@ -186,9 +149,29 @@ void socket_manager::handle_error(sec code) {
if (handler_) { if (handler_) {
handler_->abort(make_error(code)); handler_->abort(make_error(code));
handler_ = nullptr; handler_ = nullptr;
disposed_ = true;
} }
} }
// -- implementation of disposable_impl ----------------------------------------
void socket_manager::dispose() {
if (!disposed())
mpx_->dispose(this);
}
bool socket_manager::disposed() const noexcept {
return disposed_.load();
}
void socket_manager::ref_disposable() const noexcept {
ref();
}
void socket_manager::deref_disposable() const noexcept {
deref();
}
// -- utility functions -------------------------------------------------------- // -- utility functions --------------------------------------------------------
socket_manager_ptr socket_manager::strong_this() { socket_manager_ptr socket_manager::strong_this() {
......
...@@ -144,10 +144,10 @@ public: ...@@ -144,10 +144,10 @@ public:
}; };
struct fixture : test_coordinator_fixture<> { struct fixture : test_coordinator_fixture<> {
fixture() : mm(sys), mpx(&mm) { fixture() : mm(sys), mpx(net::multiplexer::make(&mm)) {
mpx.set_thread_id(); mpx->set_thread_id();
if (auto err = mpx.init()) if (auto err = mpx->init())
CAF_FAIL("mpx.init() failed: " << err); CAF_FAIL("mpx->init() failed: " << err);
auto sockets = unbox(net::make_stream_socket_pair()); auto sockets = unbox(net::make_stream_socket_pair());
self_socket_guard.reset(sockets.first); self_socket_guard.reset(sockets.first);
testee_socket_guard.reset(sockets.second); testee_socket_guard.reset(sockets.second);
...@@ -162,8 +162,8 @@ struct fixture : test_coordinator_fixture<> { ...@@ -162,8 +162,8 @@ struct fixture : test_coordinator_fixture<> {
if (!predicate()) if (!predicate())
return; return;
for (size_t i = 0; i < 1000; ++i) { for (size_t i = 0; i < 1000; ++i) {
mpx.apply_updates(); mpx->apply_updates();
mpx.poll_once(false); mpx->poll_once(false);
std::byte tmp[1024]; std::byte tmp[1024];
auto bytes = read(self_socket_guard.socket(), make_span(tmp, 1024)); auto bytes = read(self_socket_guard.socket(), make_span(tmp, 1024));
if (bytes > 0) if (bytes > 0)
...@@ -182,7 +182,7 @@ struct fixture : test_coordinator_fixture<> { ...@@ -182,7 +182,7 @@ struct fixture : test_coordinator_fixture<> {
} }
net::middleman mm; net::middleman mm;
net::multiplexer mpx; net::multiplexer_ptr mpx;
net::socket_guard<net::stream_socket> self_socket_guard; net::socket_guard<net::stream_socket> self_socket_guard;
net::socket_guard<net::stream_socket> testee_socket_guard; net::socket_guard<net::stream_socket> testee_socket_guard;
byte_buffer recv_buf; byte_buffer recv_buf;
...@@ -201,7 +201,7 @@ CAF_TEST(actor shells expose their mailbox to their owners) { ...@@ -201,7 +201,7 @@ CAF_TEST(actor shells expose their mailbox to their owners) {
auto app_uptr = app_t::make(); auto app_uptr = app_t::make();
auto app = app_uptr.get(); auto app = app_uptr.get();
auto transport = net::stream_transport::make(fd, std::move(app_uptr)); auto transport = net::stream_transport::make(fd, std::move(app_uptr));
auto mgr = net::socket_manager::make(&mpx, fd, std::move(transport)); auto mgr = net::socket_manager::make(mpx.get(), fd, std::move(transport));
if (auto err = mgr->init(content(cfg))) if (auto err = mgr->init(content(cfg)))
CAF_FAIL("mgr->init() failed: " << err); CAF_FAIL("mgr->init() failed: " << err);
auto hdl = app->self.as_actor(); auto hdl = app->self.as_actor();
...@@ -222,7 +222,7 @@ CAF_TEST(actor shells can send requests and receive responses) { ...@@ -222,7 +222,7 @@ CAF_TEST(actor shells can send requests and receive responses) {
auto app_uptr = app_t::make(worker); auto app_uptr = app_t::make(worker);
auto app = app_uptr.get(); auto app = app_uptr.get();
auto transport = net::stream_transport::make(fd, std::move(app_uptr)); auto transport = net::stream_transport::make(fd, std::move(app_uptr));
auto mgr = net::socket_manager::make(&mpx, fd, std::move(transport)); auto mgr = net::socket_manager::make(mpx.get(), fd, std::move(transport));
if (auto err = mgr->init(content(cfg))) if (auto err = mgr->init(content(cfg)))
CAF_FAIL("mgr->init() failed: " << err); CAF_FAIL("mgr->init() failed: " << err);
send(input); send(input);
......
...@@ -177,12 +177,12 @@ SCENARIO("calling suspend_reading temporarily halts receiving of messages") { ...@@ -177,12 +177,12 @@ SCENARIO("calling suspend_reading temporarily halts receiving of messages") {
read(fd1, rd_buf); read(fd1, rd_buf);
} }
}}; }};
net::multiplexer mpx{nullptr}; auto mpx = net::multiplexer::make(nullptr);
mpx.set_thread_id(); mpx->set_thread_id();
if (auto err = mpx.init()) if (auto err = mpx->init())
FAIL("mpx.init failed: " << err); FAIL("mpx->init failed: " << err);
mpx.apply_updates(); mpx->apply_updates();
REQUIRE_EQ(mpx.num_socket_managers(), 1u); REQUIRE_EQ(mpx->num_socket_managers(), 1u);
if (auto err = net::nonblocking(fd2, true)) if (auto err = net::nonblocking(fd2, true))
CAF_FAIL("nonblocking returned an error: " << err); CAF_FAIL("nonblocking returned an error: " << err);
auto buf = std::make_shared<string_list>(); auto buf = std::make_shared<string_list>();
...@@ -190,15 +190,15 @@ SCENARIO("calling suspend_reading temporarily halts receiving of messages") { ...@@ -190,15 +190,15 @@ SCENARIO("calling suspend_reading temporarily halts receiving of messages") {
auto app_ptr = app.get(); auto app_ptr = app.get();
auto framing = net::length_prefix_framing::make(std::move(app)); auto framing = net::length_prefix_framing::make(std::move(app));
auto transport = net::stream_transport::make(fd2, std::move(framing)); auto transport = net::stream_transport::make(fd2, std::move(framing));
auto mgr = net::socket_manager::make(&mpx, fd2, std::move(transport)); auto mgr = net::socket_manager::make(mpx.get(), fd2, std::move(transport));
CHECK_EQ(mgr->init(settings{}), none); CHECK_EQ(mgr->init(settings{}), none);
mpx.apply_updates(); mpx->apply_updates();
REQUIRE_EQ(mpx.num_socket_managers(), 2u); REQUIRE_EQ(mpx->num_socket_managers(), 2u);
CHECK_EQ(mpx.mask_of(mgr), net::operation::read); CHECK_EQ(mpx->mask_of(mgr), net::operation::read);
WHEN("the app calls suspend_reading") { WHEN("the app calls suspend_reading") {
while (mpx.num_socket_managers() > 1u) while (mpx->num_socket_managers() > 1u)
mpx.poll_once(true); mpx->poll_once(true);
CHECK_EQ(mpx.mask_of(mgr), net::operation::none); CHECK_EQ(mpx->mask_of(mgr), net::operation::none);
if (CHECK_EQ(buf->size(), 3u)) { if (CHECK_EQ(buf->size(), 3u)) {
CHECK_EQ(buf->at(0), "first"); CHECK_EQ(buf->at(0), "first");
CHECK_EQ(buf->at(1), "second"); CHECK_EQ(buf->at(1), "second");
...@@ -206,11 +206,11 @@ SCENARIO("calling suspend_reading temporarily halts receiving of messages") { ...@@ -206,11 +206,11 @@ SCENARIO("calling suspend_reading temporarily halts receiving of messages") {
} }
THEN("users can resume it manually") { THEN("users can resume it manually") {
app_ptr->continue_reading(); app_ptr->continue_reading();
mpx.apply_updates(); mpx->apply_updates();
mpx.poll_once(true); mpx->poll_once(true);
CHECK_EQ(mpx.mask_of(mgr), net::operation::read); CHECK_EQ(mpx->mask_of(mgr), net::operation::read);
while (mpx.num_socket_managers() > 1u) while (mpx->num_socket_managers() > 1u)
mpx.poll_once(true); mpx->poll_once(true);
if (CHECK_EQ(buf->size(), 5u)) { if (CHECK_EQ(buf->size(), 5u)) {
CHECK_EQ(buf->at(0), "first"); CHECK_EQ(buf->at(0), "first");
CHECK_EQ(buf->at(1), "second"); CHECK_EQ(buf->at(1), "second");
......
...@@ -124,45 +124,45 @@ private: ...@@ -124,45 +124,45 @@ private:
}; };
struct fixture { struct fixture {
fixture() : mpx(nullptr) { fixture() : mpx(net::multiplexer::make(nullptr)) {
manager_count = std::make_shared<std::atomic<size_t>>(0); manager_count = std::make_shared<std::atomic<size_t>>(0);
mpx.set_thread_id(); mpx->set_thread_id();
} }
~fixture() { ~fixture() {
mpx.shutdown(); mpx->shutdown();
exhaust(); exhaust();
REQUIRE_EQ(*manager_count, 0u); REQUIRE_EQ(*manager_count, 0u);
} }
void exhaust() { void exhaust() {
mpx.apply_updates(); mpx->apply_updates();
while (mpx.poll_once(false)) while (mpx->poll_once(false))
; // Repeat. ; // Repeat.
} }
void apply_updates() { void apply_updates() {
mpx.apply_updates(); mpx->apply_updates();
} }
std::pair<mock_event_layer*, net::socket_manager_ptr> std::pair<mock_event_layer*, net::socket_manager_ptr>
make_manager(net::stream_socket fd, std::string name) { make_manager(net::stream_socket fd, std::string name) {
auto mock = mock_event_layer::make(fd, std::move(name), manager_count); auto mock = mock_event_layer::make(fd, std::move(name), manager_count);
auto mock_ptr = mock.get(); auto mock_ptr = mock.get();
auto mgr = net::socket_manager::make(&mpx, fd, std::move(mock)); auto mgr = net::socket_manager::make(mpx.get(), fd, std::move(mock));
std::ignore = mgr->init(settings{}); std::ignore = mgr->init(settings{});
return {mock_ptr, std::move(mgr)}; return {mock_ptr, std::move(mgr)};
} }
void init() { void init() {
if (auto err = mpx.init()) if (auto err = mpx->init())
FAIL("mpx.init failed: " << err); FAIL("mpx->init failed: " << err);
exhaust(); exhaust();
} }
shared_count manager_count; shared_count manager_count;
net::multiplexer mpx; net::multiplexer_ptr mpx;
}; };
} // namespace } // namespace
...@@ -173,7 +173,7 @@ SCENARIO("the multiplexer has no socket managers after default construction") { ...@@ -173,7 +173,7 @@ SCENARIO("the multiplexer has no socket managers after default construction") {
GIVEN("a default constructed multiplexer") { GIVEN("a default constructed multiplexer") {
WHEN("querying the number of socket managers") { WHEN("querying the number of socket managers") {
THEN("the result is 0") { THEN("the result is 0") {
CHECK_EQ(mpx.num_socket_managers(), 0u); CHECK_EQ(mpx->num_socket_managers(), 0u);
} }
} }
} }
...@@ -183,10 +183,10 @@ SCENARIO("the multiplexer constructs the pollset updater while initializing") { ...@@ -183,10 +183,10 @@ SCENARIO("the multiplexer constructs the pollset updater while initializing") {
GIVEN("an initialized multiplexer") { GIVEN("an initialized multiplexer") {
WHEN("querying the number of socket managers") { WHEN("querying the number of socket managers") {
THEN("the result is 1") { THEN("the result is 1") {
CHECK_EQ(mpx.num_socket_managers(), 0u); CHECK_EQ(mpx->num_socket_managers(), 0u);
CHECK_EQ(mpx.init(), none); CHECK_EQ(mpx->init(), none);
exhaust(); exhaust();
CHECK_EQ(mpx.num_socket_managers(), 1u); CHECK_EQ(mpx->num_socket_managers(), 1u);
} }
} }
} }
...@@ -202,7 +202,7 @@ SCENARIO("socket managers can register for read and write operations") { ...@@ -202,7 +202,7 @@ SCENARIO("socket managers can register for read and write operations") {
alice_mgr->register_reading(); alice_mgr->register_reading();
bob_mgr->register_reading(); bob_mgr->register_reading();
apply_updates(); apply_updates();
CHECK_EQ(mpx.num_socket_managers(), 3u); CHECK_EQ(mpx->num_socket_managers(), 3u);
THEN("the multiplexer runs callbacks on socket activity") { THEN("the multiplexer runs callbacks on socket activity") {
alice->send("Hello Bob!"); alice->send("Hello Bob!");
alice_mgr->register_writing(); alice_mgr->register_writing();
...@@ -218,9 +218,9 @@ SCENARIO("a multiplexer terminates its thread after shutting down") { ...@@ -218,9 +218,9 @@ SCENARIO("a multiplexer terminates its thread after shutting down") {
init(); init();
auto go_time = std::make_shared<barrier>(2); auto go_time = std::make_shared<barrier>(2);
auto mpx_thread = std::thread{[this, go_time] { auto mpx_thread = std::thread{[this, go_time] {
mpx.set_thread_id(); mpx->set_thread_id();
go_time->arrive_and_wait(); go_time->arrive_and_wait();
mpx.run(); mpx->run();
}}; }};
go_time->arrive_and_wait(); go_time->arrive_and_wait();
auto [alice_fd, bob_fd] = unbox(net::make_stream_socket_pair()); auto [alice_fd, bob_fd] = unbox(net::make_stream_socket_pair());
...@@ -229,7 +229,7 @@ SCENARIO("a multiplexer terminates its thread after shutting down") { ...@@ -229,7 +229,7 @@ SCENARIO("a multiplexer terminates its thread after shutting down") {
alice_mgr->register_reading(); alice_mgr->register_reading();
bob_mgr->register_reading(); bob_mgr->register_reading();
WHEN("calling shutdown on the multiplexer") { WHEN("calling shutdown on the multiplexer") {
mpx.shutdown(); mpx->shutdown();
THEN("the thread terminates and all socket managers get shut down") { THEN("the thread terminates and all socket managers get shut down") {
mpx_thread.join(); mpx_thread.join();
CHECK(alice_mgr->read_closed()); CHECK(alice_mgr->read_closed());
...@@ -251,7 +251,7 @@ SCENARIO("a multiplexer terminates its thread after shutting down") { ...@@ -251,7 +251,7 @@ SCENARIO("a multiplexer terminates its thread after shutting down") {
// alice->register_reading(); // alice->register_reading();
// bob->register_reading(); // bob->register_reading();
// apply_updates(); // apply_updates();
// CHECK_EQ(mpx.num_socket_managers(), 3u); // CHECK_EQ(mpx->num_socket_managers(), 3u);
// THEN("the multiplexer swaps out the socket managers for the socket") { // THEN("the multiplexer swaps out the socket managers for the socket") {
// alice->send("Hello Bob!"); // alice->send("Hello Bob!");
// alice->register_writing(); // alice->register_writing();
......
...@@ -187,9 +187,9 @@ SCENARIO("ssl::transport::make_client performs the client handshake") { ...@@ -187,9 +187,9 @@ SCENARIO("ssl::transport::make_client performs the client handshake") {
key_1_pem_path}; key_1_pem_path};
WHEN("connecting as a client to an SSL server") { WHEN("connecting as a client to an SSL server") {
THEN("CAF transparently calls SSL_connect") { THEN("CAF transparently calls SSL_connect") {
net::multiplexer mpx{nullptr}; auto mpx = net::multiplexer::make(nullptr);
mpx.set_thread_id(); mpx->set_thread_id();
std::ignore = mpx.init(); std::ignore = mpx->init();
auto ctx = unbox(ssl::context::make_client(ssl::tls::any)); auto ctx = unbox(ssl::context::make_client(ssl::tls::any));
auto conn = unbox(ctx.new_connection(client_fd)); auto conn = unbox(ctx.new_connection(client_fd));
auto done = std::make_shared<bool>(false); auto done = std::make_shared<bool>(false);
...@@ -197,12 +197,12 @@ SCENARIO("ssl::transport::make_client performs the client handshake") { ...@@ -197,12 +197,12 @@ SCENARIO("ssl::transport::make_client performs the client handshake") {
auto mock = mock_application::make(done, buf); auto mock = mock_application::make(done, buf);
auto transport = ssl::transport::make_client(std::move(conn), auto transport = ssl::transport::make_client(std::move(conn),
std::move(mock)); std::move(mock));
auto mgr = net::socket_manager::make(&mpx, client_fd, auto mgr = net::socket_manager::make(mpx.get(), client_fd,
std::move(transport)); std::move(transport));
mpx.init(mgr); mpx->init(mgr);
mpx.apply_updates(); mpx->apply_updates();
while (!*done) while (!*done)
mpx.poll_once(true); mpx->poll_once(true);
if (CHECK_EQ(buf->size(), 16u)) { // 4x 32-bit integers if (CHECK_EQ(buf->size(), 16u)) { // 4x 32-bit integers
caf::binary_deserializer src{nullptr, *buf}; caf::binary_deserializer src{nullptr, *buf};
for (int i = 0; i < 4; ++i) { for (int i = 0; i < 4; ++i) {
...@@ -225,9 +225,9 @@ SCENARIO("ssl::transport::make_server performs the server handshake") { ...@@ -225,9 +225,9 @@ SCENARIO("ssl::transport::make_server performs the server handshake") {
std::thread client{dummy_tls_client, client_fd}; std::thread client{dummy_tls_client, client_fd};
WHEN("acting as the SSL server") { WHEN("acting as the SSL server") {
THEN("CAF transparently calls SSL_accept") { THEN("CAF transparently calls SSL_accept") {
net::multiplexer mpx{nullptr}; auto mpx = net::multiplexer::make(nullptr);
mpx.set_thread_id(); mpx->set_thread_id();
std::ignore = mpx.init(); std::ignore = mpx->init();
auto ctx = unbox(ssl::context::make_server(ssl::tls::any)); auto ctx = unbox(ssl::context::make_server(ssl::tls::any));
REQUIRE(ctx.use_certificate_from_file(cert_1_pem_path, // REQUIRE(ctx.use_certificate_from_file(cert_1_pem_path, //
ssl::format::pem)); ssl::format::pem));
...@@ -239,12 +239,12 @@ SCENARIO("ssl::transport::make_server performs the server handshake") { ...@@ -239,12 +239,12 @@ SCENARIO("ssl::transport::make_server performs the server handshake") {
auto mock = mock_application::make(done, buf); auto mock = mock_application::make(done, buf);
auto transport = ssl::transport::make_server(std::move(conn), auto transport = ssl::transport::make_server(std::move(conn),
std::move(mock)); std::move(mock));
auto mgr = net::socket_manager::make(&mpx, server_fd, auto mgr = net::socket_manager::make(mpx.get(), server_fd,
std::move(transport)); std::move(transport));
mpx.init(mgr); mpx->init(mgr);
mpx.apply_updates(); mpx->apply_updates();
while (!*done) while (!*done)
mpx.poll_once(true); mpx->poll_once(true);
if (CHECK_EQ(buf->size(), 16u)) { // 4x 32-bit integers if (CHECK_EQ(buf->size(), 16u)) { // 4x 32-bit integers
caf::binary_deserializer src{nullptr, *buf}; caf::binary_deserializer src{nullptr, *buf};
for (int i = 0; i < 4; ++i) { for (int i = 0; i < 4; ++i) {
......
...@@ -30,15 +30,15 @@ struct fixture : test_coordinator_fixture<> { ...@@ -30,15 +30,15 @@ struct fixture : test_coordinator_fixture<> {
using byte_buffer_ptr = std::shared_ptr<byte_buffer>; using byte_buffer_ptr = std::shared_ptr<byte_buffer>;
fixture() fixture()
: mpx(nullptr), : mpx(net::multiplexer::make(nullptr)),
recv_buf(1024), recv_buf(1024),
shared_recv_buf{std::make_shared<byte_buffer>()}, shared_recv_buf{std::make_shared<byte_buffer>()},
shared_send_buf{std::make_shared<byte_buffer>()} { shared_send_buf{std::make_shared<byte_buffer>()} {
mpx.set_thread_id(); mpx->set_thread_id();
mpx.apply_updates(); mpx->apply_updates();
if (auto err = mpx.init()) if (auto err = mpx->init())
FAIL("mpx.init failed: " << err); FAIL("mpx->init failed: " << err);
REQUIRE_EQ(mpx.num_socket_managers(), 1u); REQUIRE_EQ(mpx->num_socket_managers(), 1u);
auto sockets = unbox(net::make_stream_socket_pair()); auto sockets = unbox(net::make_stream_socket_pair());
send_socket_guard.reset(sockets.first); send_socket_guard.reset(sockets.first);
recv_socket_guard.reset(sockets.second); recv_socket_guard.reset(sockets.second);
...@@ -47,11 +47,11 @@ struct fixture : test_coordinator_fixture<> { ...@@ -47,11 +47,11 @@ struct fixture : test_coordinator_fixture<> {
} }
bool handle_io_event() override { bool handle_io_event() override {
return mpx.poll_once(false); return mpx->poll_once(false);
} }
settings config; settings config;
net::multiplexer mpx; net::multiplexer_ptr mpx;
byte_buffer recv_buf; byte_buffer recv_buf;
net::socket_guard<net::stream_socket> send_socket_guard; net::socket_guard<net::stream_socket> send_socket_guard;
net::socket_guard<net::stream_socket> recv_socket_guard; net::socket_guard<net::stream_socket> recv_socket_guard;
...@@ -119,11 +119,11 @@ CAF_TEST(receive) { ...@@ -119,11 +119,11 @@ CAF_TEST(receive) {
auto mock = mock_application::make(shared_recv_buf, shared_send_buf); auto mock = mock_application::make(shared_recv_buf, shared_send_buf);
auto transport = net::stream_transport::make(recv_socket_guard.get(), auto transport = net::stream_transport::make(recv_socket_guard.get(),
std::move(mock)); std::move(mock));
auto mgr = net::socket_manager::make(&mpx, recv_socket_guard.release(), auto mgr = net::socket_manager::make(mpx.get(), recv_socket_guard.release(),
std::move(transport)); std::move(transport));
CHECK_EQ(mgr->init(config), none); CHECK_EQ(mgr->init(config), none);
mpx.apply_updates(); mpx->apply_updates();
CHECK_EQ(mpx.num_socket_managers(), 2u); CHECK_EQ(mpx->num_socket_managers(), 2u);
CHECK_EQ(static_cast<size_t>(write(send_socket_guard.socket(), CHECK_EQ(static_cast<size_t>(write(send_socket_guard.socket(),
as_bytes(make_span(hello_manager)))), as_bytes(make_span(hello_manager)))),
hello_manager.size()); hello_manager.size());
...@@ -138,13 +138,13 @@ CAF_TEST(send) { ...@@ -138,13 +138,13 @@ CAF_TEST(send) {
auto mock = mock_application::make(shared_recv_buf, shared_send_buf); auto mock = mock_application::make(shared_recv_buf, shared_send_buf);
auto transport = net::stream_transport::make(recv_socket_guard.get(), auto transport = net::stream_transport::make(recv_socket_guard.get(),
std::move(mock)); std::move(mock));
auto mgr = net::socket_manager::make(&mpx, recv_socket_guard.release(), auto mgr = net::socket_manager::make(mpx.get(), recv_socket_guard.release(),
std::move(transport)); std::move(transport));
CHECK_EQ(mgr->init(config), none); CHECK_EQ(mgr->init(config), none);
mpx.apply_updates(); mpx->apply_updates();
CHECK_EQ(mpx.num_socket_managers(), 2u); CHECK_EQ(mpx->num_socket_managers(), 2u);
mgr->register_writing(); mgr->register_writing();
mpx.apply_updates(); mpx->apply_updates();
while (handle_io_event()) while (handle_io_event())
; ;
recv_buf.resize(hello_manager.size()); recv_buf.resize(hello_manager.size());
......
...@@ -144,10 +144,10 @@ public: ...@@ -144,10 +144,10 @@ public:
}; };
struct fixture : test_coordinator_fixture<> { struct fixture : test_coordinator_fixture<> {
fixture() : mm(sys), mpx(&mm) { fixture() : mm(sys), mpx(net::multiplexer::make(&mm)) {
mpx.set_thread_id(); mpx->set_thread_id();
if (auto err = mpx.init()) if (auto err = mpx->init())
CAF_FAIL("mpx.init() failed: " << err); CAF_FAIL("mpx->init() failed: " << err);
auto sockets = unbox(net::make_stream_socket_pair()); auto sockets = unbox(net::make_stream_socket_pair());
self_socket_guard.reset(sockets.first); self_socket_guard.reset(sockets.first);
testee_socket_guard.reset(sockets.second); testee_socket_guard.reset(sockets.second);
...@@ -162,8 +162,8 @@ struct fixture : test_coordinator_fixture<> { ...@@ -162,8 +162,8 @@ struct fixture : test_coordinator_fixture<> {
if (!predicate()) if (!predicate())
return; return;
for (size_t i = 0; i < 1000; ++i) { for (size_t i = 0; i < 1000; ++i) {
mpx.apply_updates(); mpx->apply_updates();
mpx.poll_once(false); mpx->poll_once(false);
std::byte tmp[1024]; std::byte tmp[1024];
auto bytes = read(self_socket_guard.socket(), make_span(tmp, 1024)); auto bytes = read(self_socket_guard.socket(), make_span(tmp, 1024));
if (bytes > 0) if (bytes > 0)
...@@ -182,7 +182,7 @@ struct fixture : test_coordinator_fixture<> { ...@@ -182,7 +182,7 @@ struct fixture : test_coordinator_fixture<> {
} }
net::middleman mm; net::middleman mm;
net::multiplexer mpx; net::multiplexer_ptr mpx;
net::socket_guard<net::stream_socket> self_socket_guard; net::socket_guard<net::stream_socket> self_socket_guard;
net::socket_guard<net::stream_socket> testee_socket_guard; net::socket_guard<net::stream_socket> testee_socket_guard;
byte_buffer recv_buf; byte_buffer recv_buf;
...@@ -201,7 +201,7 @@ CAF_TEST(actor shells expose their mailbox to their owners) { ...@@ -201,7 +201,7 @@ CAF_TEST(actor shells expose their mailbox to their owners) {
auto app_uptr = app_t::make(); auto app_uptr = app_t::make();
auto app = app_uptr.get(); auto app = app_uptr.get();
auto transport = net::stream_transport::make(fd, std::move(app_uptr)); auto transport = net::stream_transport::make(fd, std::move(app_uptr));
auto mgr = net::socket_manager::make(&mpx, fd, std::move(transport)); auto mgr = net::socket_manager::make(mpx.get(), fd, std::move(transport));
if (auto err = mgr->init(content(cfg))) if (auto err = mgr->init(content(cfg)))
CAF_FAIL("mgr->init() failed: " << err); CAF_FAIL("mgr->init() failed: " << err);
auto hdl = app->self.as_actor(); auto hdl = app->self.as_actor();
...@@ -222,7 +222,7 @@ CAF_TEST(actor shells can send requests and receive responses) { ...@@ -222,7 +222,7 @@ CAF_TEST(actor shells can send requests and receive responses) {
auto app_uptr = app_t::make(worker); auto app_uptr = app_t::make(worker);
auto app = app_uptr.get(); auto app = app_uptr.get();
auto transport = net::stream_transport::make(fd, std::move(app_uptr)); auto transport = net::stream_transport::make(fd, std::move(app_uptr));
auto mgr = net::socket_manager::make(&mpx, fd, std::move(transport)); auto mgr = net::socket_manager::make(mpx.get(), fd, std::move(transport));
if (auto err = mgr->init(content(cfg))) if (auto err = mgr->init(content(cfg)))
CAF_FAIL("mgr->init() failed: " << err); CAF_FAIL("mgr->init() failed: " << err);
send(input); send(input);
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment