Unverified Commit 71254c55 authored by Dominik Charousset's avatar Dominik Charousset Committed by GitHub

Merge pull request #33

Port io::basp::worker
parents 89dddbcf b24edf52
......@@ -41,6 +41,9 @@ set(LIBCAF_NET_SRCS
src/tcp_accept_socket.cpp
src/tcp_stream_socket.cpp
src/udp_datagram_socket.cpp
src/defaults.cpp
src/message_queue.cpp
src/worker.cpp
)
add_custom_target(libcaf_net)
......
......@@ -26,13 +26,19 @@
#include <vector>
#include "caf/actor_addr.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/byte.hpp"
#include "caf/callback.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/worker_hub.hpp"
#include "caf/error.hpp"
#include "caf/net/basp/connection_state.hpp"
#include "caf/net/basp/constants.hpp"
#include "caf/net/basp/header.hpp"
#include "caf/net/basp/message_queue.hpp"
#include "caf/net/basp/message_type.hpp"
#include "caf/net/basp/worker.hpp"
#include "caf/net/endpoint_manager.hpp"
#include "caf/net/packet_writer.hpp"
#include "caf/net/receive_policy.hpp"
......@@ -57,6 +63,8 @@ public:
using byte_span = span<const byte>;
using hub_type = detail::worker_hub<worker>;
struct test_tag {};
// -- constructors, destructors, and assignment operators --------------------
......@@ -75,6 +83,10 @@ public:
// Allow unit tests to run the application without endpoint manager.
if (!std::is_base_of<test_tag, Parent>::value)
manager_ = &parent.manager();
auto workers = get_or(system_->config(), "middleman.workers",
defaults::middleman::workers);
for (size_t i = 0; i < workers; ++i)
hub_->add_new_worker(*queue_, proxies_);
// Write handshake.
auto hdr = parent.next_header_buffer();
auto payload = parent.next_payload_buffer();
......@@ -193,6 +205,10 @@ private:
/// Provides pointers to the actor system as well as the registry,
/// serializers and deserializer.
scoped_execution_unit executor_;
std::unique_ptr<message_queue> queue_;
std::unique_ptr<hub_type> hub_;
};
} // namespace basp
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <cstdint>
#include <mutex>
#include <vector>
#include "caf/actor_control_block.hpp"
#include "caf/fwd.hpp"
#include "caf/mailbox_element.hpp"
namespace caf {
namespace net {
namespace basp {
/// Enforces strict order of message delivery, i.e., deliver messages in the
/// same order as if they were deserialized by a single thread.
class message_queue {
public:
// -- member types -----------------------------------------------------------
/// Request for sending a message to an actor at a later time.
struct actor_msg {
uint64_t id;
strong_actor_ptr receiver;
mailbox_element_ptr content;
};
// -- constructors, destructors, and assignment operators --------------------
message_queue();
// -- mutators ---------------------------------------------------------------
/// Adds a new message to the queue or deliver it immediately if possible.
void push(execution_unit* ctx, uint64_t id, strong_actor_ptr receiver,
mailbox_element_ptr content);
/// Marks given ID as dropped, effectively skipping it without effect.
void drop(execution_unit* ctx, uint64_t id);
/// Returns the next ascending ID.
uint64_t new_id();
// -- member variables -------------------------------------------------------
/// Protects all other properties.
std::mutex lock;
/// The next available ascending ID. The counter is large enough to overflow
/// after roughly 600 years if we dispatch a message every microsecond.
uint64_t next_id;
/// The next ID that we can ship.
uint64_t next_undelivered;
/// Keeps messages in sorted order in case a message other than
/// `next_undelivered` gets ready first.
std::vector<actor_msg> pending;
};
} // namespace basp
} // namespace net
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <vector>
#include "caf/actor_control_block.hpp"
#include "caf/actor_proxy.hpp"
#include "caf/binary_deserializer.hpp"
#include "caf/config.hpp"
#include "caf/detail/scope_guard.hpp"
#include "caf/detail/sync_request_bouncer.hpp"
#include "caf/execution_unit.hpp"
#include "caf/logger.hpp"
#include "caf/message.hpp"
#include "caf/message_id.hpp"
#include "caf/net/basp/header.hpp"
#include "caf/node_id.hpp"
namespace caf {
namespace net {
namespace basp {
template <class Subtype>
class remote_message_handler {
public:
void handle_remote_message(execution_unit* ctx) {
// Local variables.
auto& dref = static_cast<Subtype&>(*this);
auto& payload = dref.payload_;
auto& hdr = dref.hdr_;
auto& registry = dref.system_->registry();
auto& proxies = *dref.proxies_;
CAF_LOG_TRACE(CAF_ARG(hdr) << CAF_ARG2("payload.size", payload.size()));
// Deserialize payload.
actor_id src_id = 0;
node_id src_node;
actor_id dst_id = 0;
std::vector<strong_actor_ptr> fwd_stack;
message content;
binary_deserializer source{ctx, payload};
if (auto err = source(src_node, src_id, dst_id, fwd_stack, content)) {
CAF_LOG_ERROR("could not deserialize payload: " << CAF_ARG(err));
return;
}
// Sanity checks.
if (dst_id == 0)
return;
// Try to fetch the receiver.
auto dst_hdl = registry.get(dst_id);
if (dst_hdl == nullptr) {
CAF_LOG_DEBUG("no actor found for given ID, drop message");
return;
}
// Try to fetch the sender.
strong_actor_ptr src_hdl;
if (src_node != none && src_id != 0)
src_hdl = proxies.get_or_put(src_node, src_id);
// Ship the message.
auto ptr = make_mailbox_element(std::move(src_hdl),
make_message_id(hdr.operation_data),
std::move(fwd_stack), std::move(content));
dst_hdl->get()->enqueue(std::move(ptr), nullptr);
}
};
} // namespace basp
} // namespace net
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <atomic>
#include <cstdint>
#include <vector>
#include "caf/config.hpp"
#include "caf/detail/abstract_worker.hpp"
#include "caf/detail/worker_hub.hpp"
#include "caf/net/basp/header.hpp"
#include "caf/net/basp/message_queue.hpp"
#include "caf/net/basp/remote_message_handler.hpp"
#include "caf/net/fwd.hpp"
#include "caf/node_id.hpp"
#include "caf/resumable.hpp"
namespace caf {
namespace net {
namespace basp {
/// Deserializes payloads for BASP messages asynchronously.
class worker : public detail::abstract_worker,
public remote_message_handler<worker> {
public:
// -- friends ----------------------------------------------------------------
friend remote_message_handler<worker>;
// -- member types -----------------------------------------------------------
using super = detail::abstract_worker;
using scheduler_type = scheduler::abstract_coordinator;
using buffer_type = std::vector<byte>;
using hub_type = detail::worker_hub<worker>;
// -- constructors, destructors, and assignment operators --------------------
/// Only the ::worker_hub has access to the construtor.
worker(hub_type& hub, message_queue& queue, proxy_registry& proxies);
~worker() override;
// -- management -------------------------------------------------------------
void launch(const node_id& last_hop, const basp::header& hdr,
span<const byte> payload);
// -- implementation of resumable --------------------------------------------
resume_result resume(execution_unit* ctx, size_t) override;
private:
// -- constants and assertions -----------------------------------------------
/// Stores how many bytes the "first half" of this object requires.
static constexpr size_t pointer_members_size = sizeof(hub_type*)
+ sizeof(message_queue*)
+ sizeof(proxy_registry*)
+ sizeof(actor_system*);
static_assert(CAF_CACHE_LINE_SIZE > pointer_members_size,
"invalid cache line size");
// -- member variables -------------------------------------------------------
/// Points to our home hub.
hub_type* hub_;
/// Points to the queue for establishing strict ordering.
message_queue* queue_;
/// Points to our proxy registry / factory.
proxy_registry* proxies_;
/// Points to the parent system.
actor_system* system_;
/// Prevents false sharing when writing to `next`.
char pad_[CAF_CACHE_LINE_SIZE - pointer_members_size];
/// ID for local ordering.
uint64_t msg_id_;
/// Identifies the node that sent us `hdr_` and `payload_`.
node_id last_hop_;
/// The header for the next message. Either a direct_message or a
/// routed_message.
header hdr_;
/// Contains whatever this worker deserializes next.
buffer_type payload_;
};
} // namespace basp
} // namespace net
} // namespace caf
......@@ -44,7 +44,8 @@ namespace caf {
namespace net {
namespace basp {
application::application(proxy_registry& proxies) : proxies_(proxies) {
application::application(proxy_registry& proxies)
: proxies_(proxies), queue_{new message_queue}, hub_{new hub_type} {
// nop
}
......@@ -248,34 +249,38 @@ error application::handle_handshake(packet_writer&, header hdr,
error application::handle_actor_message(packet_writer&, header hdr,
byte_span payload) {
CAF_LOG_TRACE(CAF_ARG(hdr) << CAF_ARG2("payload.size", payload.size()));
// Deserialize payload.
actor_id src_id = 0;
node_id src_node;
actor_id dst_id = 0;
std::vector<strong_actor_ptr> fwd_stack;
message content;
binary_deserializer source{&executor_, payload};
if (auto err = source(src_node, src_id, dst_id, fwd_stack, content))
return err;
// Sanity checks.
if (dst_id == 0)
return ec::invalid_payload;
// Try to fetch the receiver.
auto dst_hdl = system().registry().get(dst_id);
if (dst_hdl == nullptr) {
CAF_LOG_DEBUG("no actor found for given ID, drop message");
return caf::none;
auto worker = hub_->pop();
if (worker != nullptr) {
CAF_LOG_DEBUG("launch BASP worker for deserializing an actor_message");
worker->launch(node_id{}, hdr, payload);
} else {
CAF_LOG_DEBUG(
"out of BASP workers, continue deserializing an actor_message");
// If no worker is available then we have no other choice than to take
// the performance hit and deserialize in this thread.
struct handler : remote_message_handler<handler> {
handler(message_queue* queue, proxy_registry* proxies,
actor_system* system, node_id last_hop, basp::header& hdr,
byte_span payload)
: queue_(queue),
proxies_(proxies),
system_(system),
last_hop_(std::move(last_hop)),
hdr_(hdr),
payload_(payload) {
msg_id_ = queue_->new_id();
}
message_queue* queue_;
proxy_registry* proxies_;
actor_system* system_;
node_id last_hop_;
basp::header& hdr_;
byte_span payload_;
uint64_t msg_id_;
};
handler f{queue_.get(), &proxies_, system_, node_id{}, hdr, payload};
f.handle_remote_message(&executor_);
}
// Try to fetch the sender.
strong_actor_ptr src_hdl;
if (src_node != none && src_id != 0)
src_hdl = proxies_.get_or_put(src_node, src_id);
// Ship the message.
auto ptr = make_mailbox_element(std::move(src_hdl),
make_message_id(hdr.operation_data),
std::move(fwd_stack), std::move(content));
dst_hdl->get()->enqueue(std::move(ptr), nullptr);
return none;
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/net/basp/message_queue.hpp"
#include <iterator>
namespace caf {
namespace net {
namespace basp {
message_queue::message_queue() : next_id(0), next_undelivered(0) {
// nop
}
void message_queue::push(execution_unit* ctx, uint64_t id,
strong_actor_ptr receiver,
mailbox_element_ptr content) {
std::unique_lock<std::mutex> guard{lock};
CAF_ASSERT(id >= next_undelivered);
CAF_ASSERT(id < next_id);
auto first = pending.begin();
auto last = pending.end();
if (id == next_undelivered) {
// Dispatch current head.
if (receiver != nullptr)
receiver->enqueue(std::move(content), ctx);
auto next = id + 1;
// Check whether we can deliver more.
if (first == last || first->id != next) {
next_undelivered = next;
CAF_ASSERT(next_undelivered <= next_id);
return;
}
// Deliver everything until reaching a non-consecutive ID or the end.
auto i = first;
for (; i != last && i->id == next; ++i, ++next)
if (i->receiver != nullptr)
i->receiver->enqueue(std::move(i->content), ctx);
next_undelivered = next;
pending.erase(first, i);
CAF_ASSERT(next_undelivered <= next_id);
return;
}
// Get the insertion point.
auto pred = [&](const actor_msg& x) { return x.id >= id; };
pending.emplace(std::find_if(first, last, pred),
actor_msg{id, std::move(receiver), std::move(content)});
}
void message_queue::drop(execution_unit* ctx, uint64_t id) {
push(ctx, id, nullptr, nullptr);
}
uint64_t message_queue::new_id() {
std::unique_lock<std::mutex> guard{lock};
return next_id++;
}
} // namespace basp
} // namespace net
} // namespace caf
......@@ -75,9 +75,9 @@ test::peer_entry& test::emplace(const node_id& peer_id, stream_socket first,
using transport_type = stream_transport<basp::application>;
nonblocking(second, true);
auto mpx = mm_.mpx();
basp::application app{proxies_};
auto mgr = make_endpoint_manager(mpx, mm_.system(),
transport_type{second,
basp::application{proxies_}});
transport_type{second, std::move(app)});
if (auto err = mgr->init()) {
CAF_LOG_ERROR("mgr->init() failed: " << mm_.system().render(err));
CAF_RAISE_ERROR("mgr->init() failed");
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/net/basp/worker.hpp"
#include "caf/actor_system.hpp"
#include "caf/byte.hpp"
#include "caf/net/basp/message_queue.hpp"
#include "caf/proxy_registry.hpp"
#include "caf/scheduler/abstract_coordinator.hpp"
namespace caf {
namespace net {
namespace basp {
// -- constructors, destructors, and assignment operators ----------------------
worker::worker(hub_type& hub, message_queue& queue, proxy_registry& proxies)
: hub_(&hub), queue_(&queue), proxies_(&proxies), system_(&proxies.system()) {
CAF_IGNORE_UNUSED(pad_);
}
worker::~worker() {
// nop
}
// -- management ---------------------------------------------------------------
void worker::launch(const node_id& last_hop, const basp::header& hdr,
span<const byte> payload) {
msg_id_ = queue_->new_id();
last_hop_ = last_hop;
memcpy(&hdr_, &hdr, sizeof(basp::header));
payload_.assign(payload.begin(), payload.end());
ref();
system_->scheduler().enqueue(this);
}
// -- implementation of resumable ----------------------------------------------
resumable::resume_result worker::resume(execution_unit* ctx, size_t) {
ctx->proxy_registry_ptr(proxies_);
handle_remote_message(ctx);
hub_->push(this);
return resumable::awaiting_message;
}
} // namespace basp
} // namespace net
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#define CAF_SUITE net.basp.message_queue
#include "caf/net/basp/message_queue.hpp"
#include "caf/test/dsl.hpp"
#include "caf/actor_cast.hpp"
#include "caf/actor_system.hpp"
#include "caf/behavior.hpp"
using namespace caf;
namespace {
behavior testee_impl() {
return {
[](ok_atom, int) {
// nop
},
};
}
struct fixture : test_coordinator_fixture<> {
net::basp::message_queue queue;
strong_actor_ptr testee;
fixture() {
auto hdl = sys.spawn<lazy_init>(testee_impl);
testee = actor_cast<strong_actor_ptr>(hdl);
}
void acquire_ids(size_t num) {
for (size_t i = 0; i < num; ++i)
queue.new_id();
}
void push(int msg_id) {
queue.push(nullptr, static_cast<uint64_t>(msg_id), testee,
make_mailbox_element(self->ctrl(), make_message_id(), {},
ok_atom::value, msg_id));
}
};
} // namespace
CAF_TEST_FIXTURE_SCOPE(message_queue_tests, fixture)
CAF_TEST(default construction) {
CAF_CHECK_EQUAL(queue.next_id, 0u);
CAF_CHECK_EQUAL(queue.next_undelivered, 0u);
CAF_CHECK_EQUAL(queue.pending.size(), 0u);
}
CAF_TEST(ascending IDs) {
CAF_CHECK_EQUAL(queue.new_id(), 0u);
CAF_CHECK_EQUAL(queue.new_id(), 1u);
CAF_CHECK_EQUAL(queue.new_id(), 2u);
CAF_CHECK_EQUAL(queue.next_undelivered, 0u);
}
CAF_TEST(push order 0 - 1 - 2) {
acquire_ids(3);
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
push(1);
expect((ok_atom, int), from(self).to(testee).with(_, 1));
push(2);
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 0 - 2 - 1) {
acquire_ids(3);
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(1);
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 1 - 0 - 2) {
acquire_ids(3);
push(1);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 1));
push(2);
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 1 - 2 - 0) {
acquire_ids(3);
push(1);
disallow((ok_atom, int), from(self).to(testee));
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 2 - 0 - 1) {
acquire_ids(3);
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
push(1);
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 2 - 1 - 0) {
acquire_ids(3);
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(1);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(dropping) {
acquire_ids(3);
push(2);
disallow((ok_atom, int), from(self).to(testee));
queue.drop(nullptr, 1);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST_FIXTURE_SCOPE_END()
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#define CAF_SUITE net.basp.worker
#include "caf/net/basp/worker.hpp"
#include "caf/test/dsl.hpp"
#include "caf/actor_cast.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/actor_system.hpp"
#include "caf/make_actor.hpp"
#include "caf/net/basp/message_queue.hpp"
#include "caf/proxy_registry.hpp"
#include "caf/serializer_impl.hpp"
using namespace caf;
namespace {
behavior testee_impl() {
return {
[](ok_atom) {
// nop
},
};
}
class mock_actor_proxy : public actor_proxy {
public:
explicit mock_actor_proxy(actor_config& cfg) : actor_proxy(cfg) {
// nop
}
void enqueue(mailbox_element_ptr, execution_unit*) override {
CAF_FAIL("mock_actor_proxy::enqueue called");
}
void kill_proxy(execution_unit*, error) override {
// nop
}
};
class mock_proxy_registry_backend : public proxy_registry::backend {
public:
mock_proxy_registry_backend(actor_system& sys) : sys_(sys) {
// nop
}
strong_actor_ptr make_proxy(node_id nid, actor_id aid) override {
actor_config cfg;
return make_actor<mock_actor_proxy, strong_actor_ptr>(aid, nid, &sys_, cfg);
}
void set_last_hop(node_id*) override {
// nop
}
private:
actor_system& sys_;
};
struct fixture : test_coordinator_fixture<> {
detail::worker_hub<net::basp::worker> hub;
net::basp::message_queue queue;
mock_proxy_registry_backend proxies_backend;
proxy_registry proxies;
node_id last_hop;
actor testee;
fixture() : proxies_backend(sys), proxies(sys, proxies_backend) {
auto tmp = make_node_id(123, "0011223344556677889900112233445566778899");
last_hop = unbox(std::move(tmp));
testee = sys.spawn<lazy_init>(testee_impl);
sys.registry().put(testee.id(), testee);
}
~fixture() {
sys.registry().erase(testee.id());
}
};
} // namespace
CAF_TEST_FIXTURE_SCOPE(worker_tests, fixture)
CAF_TEST(deliver serialized message) {
CAF_MESSAGE("create the BASP worker");
CAF_REQUIRE_EQUAL(hub.peek(), nullptr);
hub.add_new_worker(queue, proxies);
CAF_REQUIRE_NOT_EQUAL(hub.peek(), nullptr);
auto w = hub.pop();
CAF_MESSAGE("create a fake message + BASP header");
std::vector<byte> payload;
std::vector<strong_actor_ptr> stages;
serializer_impl<std::vector<byte>> sink{sys, payload};
if (auto err = sink(node_id{}, self->id(), testee.id(), stages,
make_message(ok_atom::value)))
CAF_FAIL("unable to serialize message: " << sys.render(err));
net::basp::header hdr{net::basp::message_type::actor_message,
static_cast<uint32_t>(payload.size()),
make_message_id().integer_value()};
CAF_MESSAGE("launch worker");
w->launch(last_hop, hdr, payload);
sched.run_once();
expect((ok_atom), from(_).to(testee));
}
CAF_TEST_FIXTURE_SCOPE_END()
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