mirror of
https://github.com/oxen-io/oxen-mq.git
synced 2023-12-13 21:00:31 +01:00
All sends use dontwait; add send failure callbacks
We really don't *ever* want send to block, no matter how it is called, since the send is always in the proxy thread. This makes the actual send call always non-blocking, and adds callbacks that we can invoke on send failures: either on queue full errors (which might be recoverable), or both full queue and hard failures (which are generally not recoverable). These callbacks are both optional: they have to be passed in using `send_option::queue_full` (if you just want queue full notifies) or `send_option::queue_failure` (if you want queue full notifies *and* other send exceptions).
This commit is contained in:
parent
8e1b2dffa5
commit
716d73d196
8 changed files with 203 additions and 37 deletions
|
@ -61,9 +61,9 @@ bool LokiMQ::proxy_check_auth(size_t conn_index, bool outgoing, const peer_info&
|
|||
|
||||
try {
|
||||
if (outgoing)
|
||||
send_direct_message(connections[conn_index], std::move(reply), command, zmq::send_flags::dontwait);
|
||||
send_direct_message(connections[conn_index], std::move(reply), command);
|
||||
else
|
||||
send_routed_message(connections[conn_index], peer.route, std::move(reply), command, zmq::send_flags::dontwait);
|
||||
send_routed_message(connections[conn_index], peer.route, std::move(reply), command);
|
||||
} catch (const zmq::error_t&) { /* can't send: possibly already disconnected. Ignore. */ }
|
||||
|
||||
return false;
|
||||
|
|
|
@ -8,7 +8,7 @@ struct ConnectionID;
|
|||
|
||||
namespace detail {
|
||||
template <typename... T>
|
||||
bt_dict build_send(ConnectionID to, string_view cmd, const T&... opts);
|
||||
bt_dict build_send(ConnectionID to, string_view cmd, T&&... opts);
|
||||
}
|
||||
|
||||
/// Opaque data structure representing a connection which supports ==, !=, < and std::hash. For
|
||||
|
@ -67,7 +67,7 @@ private:
|
|||
friend class LokiMQ;
|
||||
friend struct std::hash<ConnectionID>;
|
||||
template <typename... T>
|
||||
friend bt_dict detail::build_send(ConnectionID to, string_view cmd, const T&... opts);
|
||||
friend bt_dict detail::build_send(ConnectionID to, string_view cmd, T&&... opts);
|
||||
friend std::ostream& operator<<(std::ostream& o, const ConnectionID& conn);
|
||||
};
|
||||
|
||||
|
|
|
@ -94,9 +94,8 @@ void LokiMQ::_queue_timer_job(int timer_id) {
|
|||
|
||||
void LokiMQ::add_timer(std::function<void()> job, std::chrono::milliseconds interval, bool squelch) {
|
||||
if (proxy_thread.joinable()) {
|
||||
auto *jobptr = new std::function<void()>{std::move(job)};
|
||||
detail::send_control(get_control_socket(), "TIMER", bt_serialize(bt_list{{
|
||||
reinterpret_cast<uintptr_t>(jobptr),
|
||||
detail::serialize_object(std::move(job)),
|
||||
interval.count(),
|
||||
squelch}}));
|
||||
} else {
|
||||
|
|
|
@ -38,41 +38,41 @@ inline zmq::message_t create_message(string_view data) {
|
|||
}
|
||||
|
||||
template <typename It>
|
||||
void send_message_parts(zmq::socket_t &sock, It begin, It end, const zmq::send_flags flags = zmq::send_flags::none) {
|
||||
bool send_message_parts(zmq::socket_t &sock, It begin, It end) {
|
||||
while (begin != end) {
|
||||
zmq::message_t &msg = *begin++;
|
||||
sock.send(msg, begin == end ? flags : flags | zmq::send_flags::sndmore);
|
||||
if (!sock.send(msg, begin == end ? zmq::send_flags::dontwait : zmq::send_flags::dontwait | zmq::send_flags::sndmore))
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename Container>
|
||||
void send_message_parts(zmq::socket_t &sock, Container &&c, zmq::send_flags flags = zmq::send_flags::none) {
|
||||
send_message_parts(sock, c.begin(), c.end(), flags);
|
||||
bool send_message_parts(zmq::socket_t &sock, Container &&c) {
|
||||
return send_message_parts(sock, c.begin(), c.end());
|
||||
}
|
||||
|
||||
/// Sends a message with an initial route. `msg` and `data` can be empty: if `msg` is empty then
|
||||
/// the msg frame will be an empty message; if `data` is empty then the data frame will be omitted.
|
||||
/// `flags` is passed through to zmq: typically given `zmq::send_flags::dontwait` to throw rather
|
||||
/// than block if a message can't be queued.
|
||||
inline void send_routed_message(zmq::socket_t &socket, std::string route, std::string msg = {}, std::string data = {},
|
||||
zmq::send_flags flags = zmq::send_flags::none) {
|
||||
inline bool send_routed_message(zmq::socket_t &socket, std::string route, std::string msg = {}, std::string data = {}) {
|
||||
assert(!route.empty());
|
||||
std::array<zmq::message_t, 3> msgs{{create_message(std::move(route))}};
|
||||
if (!msg.empty())
|
||||
msgs[1] = create_message(std::move(msg));
|
||||
if (!data.empty())
|
||||
msgs[2] = create_message(std::move(data));
|
||||
send_message_parts(socket, msgs.begin(), data.empty() ? std::prev(msgs.end()) : msgs.end(), flags);
|
||||
return send_message_parts(socket, msgs.begin(), data.empty() ? std::prev(msgs.end()) : msgs.end());
|
||||
}
|
||||
|
||||
// Sends some stuff to a socket directly. If dontwait is true then we throw instead of blocking if
|
||||
// the message cannot be accepted by zmq (i.e. because the outgoing buffer is full).
|
||||
inline void send_direct_message(zmq::socket_t &socket, std::string msg, std::string data = {},
|
||||
zmq::send_flags flags = zmq::send_flags::none) {
|
||||
inline bool send_direct_message(zmq::socket_t &socket, std::string msg, std::string data = {}) {
|
||||
std::array<zmq::message_t, 2> msgs{{create_message(std::move(msg))}};
|
||||
if (!data.empty())
|
||||
msgs[1] = create_message(std::move(data));
|
||||
send_message_parts(socket, msgs.begin(), data.empty() ? std::prev(msgs.end()) : msgs.end(), flags);
|
||||
return send_message_parts(socket, msgs.begin(), data.empty() ? std::prev(msgs.end()) : msgs.end());
|
||||
}
|
||||
|
||||
// Receive all the parts of a single message from the given socket. Returns true if a message was
|
||||
|
|
|
@ -419,9 +419,10 @@ private:
|
|||
// either accepting curve connections, or not accepting curve).
|
||||
void setup_outgoing_socket(zmq::socket_t& socket, string_view remote_pubkey = {});
|
||||
|
||||
/// Common connection implementation used by proxy_connect/proxy_send. Returns the socket
|
||||
/// and, if a routing prefix is needed, the required prefix (or an empty string if not needed).
|
||||
/// For an optional connect that fail, returns nullptr for the socket.
|
||||
/// Common connection implementation used by proxy_connect/proxy_send. Returns the socket and,
|
||||
/// if a routing prefix is needed, the required prefix (or an empty string if not needed). For
|
||||
/// an optional connect that fails (or some other connection failure), returns nullptr for the
|
||||
/// socket.
|
||||
///
|
||||
/// @param pubkey the pubkey to connect to
|
||||
/// @param connect_hint if we need a new connection and this is non-empty then we *may* use it
|
||||
|
@ -1060,10 +1061,60 @@ struct request_timeout {
|
|||
explicit request_timeout(std::chrono::milliseconds time) : time{std::move(time)} {}
|
||||
};
|
||||
|
||||
/// Specifies a callback to invoke if the message couldn't be queued for delivery. There are
|
||||
/// generally two failure modes here: a full queue, and a send exception. This callback is invoked
|
||||
/// for both; to only catch full queues see `queue_full` instead.
|
||||
///
|
||||
/// A full queue means there are too many messages queued for delivery already that haven't been
|
||||
/// delivered yet (i.e. because the remote is slow); this error is potentially recoverable if the
|
||||
/// remote end wakes up and receives/acknoledges its messages.
|
||||
///
|
||||
/// A send exception is not recoverable: it indicates some failure such as the remote having
|
||||
/// disconnected or an internal send error.
|
||||
///
|
||||
/// This callback can be used by a caller to log, attempt to resend, or take other appropriate
|
||||
/// action.
|
||||
///
|
||||
/// Note that this callback is *not* exhaustive for all possible send failures: there are failure
|
||||
/// cases (such as when a message is queued but the connection fails before delivery) that do not
|
||||
/// trigger this failure at all; rather this callback only signals an immediate queuing failure.
|
||||
struct queue_failure {
|
||||
using callback_t = std::function<void(const zmq::error_t* exc)>;
|
||||
/// Callback; invoked with nullptr for a queue full failure, otherwise will be set to a copy of
|
||||
/// the raised exception.
|
||||
callback_t callback;
|
||||
};
|
||||
|
||||
/// This is similar to queue_failure_callback, but is only invoked on a (potentially recoverable)
|
||||
/// full queue failure. Send failures are simply dropped.
|
||||
struct queue_full {
|
||||
using callback_t = std::function<void()>;
|
||||
callback_t callback;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
namespace detail {
|
||||
|
||||
/// Takes an rvalue reference, moves it into a new instance then returns a uintptr_t value
|
||||
/// containing the pointer to be serialized to pass (via lokimq queues) from one thread to another.
|
||||
/// Must be matched with a deserializer_pointer on the other side to reconstitute the object and
|
||||
/// destroy the intermediate pointer.
|
||||
template <typename T> uintptr_t serialize_object(T&& obj) {
|
||||
auto* ptr = new T{std::forward<T>(obj)};
|
||||
return reinterpret_cast<uintptr_t>(ptr);
|
||||
}
|
||||
|
||||
/// Takes a uintptr_t as produced by serialize_pointer and the type, converts the serialized value
|
||||
/// back into a pointer, moves it into a new instance (to be returned) and destroys the
|
||||
/// intermediate.
|
||||
template <typename T> T deserialize_object(uintptr_t ptrval) {
|
||||
auto* ptr = reinterpret_cast<T*>(ptrval);
|
||||
T ret{std::move(*ptr)};
|
||||
delete ptr;
|
||||
return ret;
|
||||
}
|
||||
|
||||
// Sends a control message to the given socket consisting of the command plus optional dict
|
||||
// data (only sent if the data is non-empty).
|
||||
void send_control(zmq::socket_t& sock, string_view cmd, std::string data = {});
|
||||
|
@ -1105,20 +1156,28 @@ inline void apply_send_option(bt_list&, bt_dict& control_data, const send_option
|
|||
control_data["request_timeout"] = timeout.time.count();
|
||||
}
|
||||
|
||||
/// `queue_failure` specialization
|
||||
inline void apply_send_option(bt_list&, bt_dict& control_data, send_option::queue_failure f) {
|
||||
control_data["send_fail"] = serialize_object(std::move(f.callback));
|
||||
}
|
||||
|
||||
/// `queue_full` specialization
|
||||
inline void apply_send_option(bt_list&, bt_dict& control_data, send_option::queue_full f) {
|
||||
control_data["send_full_q"] = serialize_object(std::move(f.callback));
|
||||
}
|
||||
|
||||
/// Extracts a pubkey, SN status, and auth level from a zmq message received on a *listening*
|
||||
/// socket.
|
||||
std::tuple<std::string, bool, AuthLevel> extract_metadata(zmq::message_t& msg);
|
||||
|
||||
template <typename... T>
|
||||
bt_dict build_send(ConnectionID to, string_view cmd, const T&... opts) {
|
||||
bt_dict build_send(ConnectionID to, string_view cmd, T&&... opts) {
|
||||
bt_dict control_data;
|
||||
bt_list parts{{cmd}};
|
||||
#ifdef __cpp_fold_expressions
|
||||
(detail::apply_send_option(parts, control_data, opts),...);
|
||||
(detail::apply_send_option(parts, control_data, std::forward<T>(opts)),...);
|
||||
#else
|
||||
(void) std::initializer_list<int>{(detail::apply_send_option(parts, control_data, opts), 0)...};
|
||||
(void) std::initializer_list<int>{(detail::apply_send_option(parts, control_data, std::forward<T>(opts)), 0)...};
|
||||
#endif
|
||||
|
||||
if (to.sn())
|
||||
|
@ -1148,7 +1207,7 @@ void LokiMQ::request(ConnectionID to, string_view cmd, ReplyCallback callback, c
|
|||
const auto reply_tag = make_random_string(15); // 15 random bytes is lots and should keep us in most stl implementations' small string optimization
|
||||
bt_dict control_data = detail::build_send(std::move(to), cmd, reply_tag, opts...);
|
||||
control_data["request"] = true;
|
||||
control_data["request_callback"] = reinterpret_cast<uintptr_t>(new ReplyCallback{std::move(callback)});
|
||||
control_data["request_callback"] = detail::serialize_object(std::move(callback));
|
||||
control_data["request_tag"] = string_view{reply_tag};
|
||||
detail::send_control(get_control_socket(), "SEND", bt_serialize(std::move(control_data)));
|
||||
}
|
||||
|
|
|
@ -70,13 +70,7 @@ void LokiMQ::proxy_send(bt_dict_consumer data) {
|
|||
if (!data.skip_until("request_callback"))
|
||||
throw std::runtime_error("Internal error: received request without request_callback");
|
||||
|
||||
// The initiator gives up ownership of the callback to us (serializing it through a
|
||||
// uintptr_t), so we take the pointer, move the value out of it, then destroy the pointer we
|
||||
// were given. Further down, if we are able to send the request successfully, we set up the
|
||||
// pending request.
|
||||
auto* cbptr = reinterpret_cast<ReplyCallback*>(data.consume_integer<uintptr_t>());
|
||||
request_callback = std::move(*cbptr);
|
||||
delete cbptr;
|
||||
request_callback = detail::deserialize_object<ReplyCallback>(data.consume_integer<uintptr_t>());
|
||||
|
||||
if (!data.skip_until("request_tag"))
|
||||
throw std::runtime_error("Internal error: received request without request_name");
|
||||
|
@ -88,11 +82,20 @@ void LokiMQ::proxy_send(bt_dict_consumer data) {
|
|||
throw std::runtime_error("Internal error: Invalid proxy send command; send parts missing");
|
||||
bt_list_consumer send = data.consume_list_consumer();
|
||||
|
||||
send_option::queue_failure::callback_t callback_nosend;
|
||||
if (data.skip_until("send_fail"))
|
||||
callback_nosend = detail::deserialize_object<decltype(callback_nosend)>(data.consume_integer<uintptr_t>());
|
||||
|
||||
send_option::queue_full::callback_t callback_noqueue;
|
||||
if (data.skip_until("send_full_q"))
|
||||
callback_noqueue = detail::deserialize_object<decltype(callback_noqueue)>(data.consume_integer<uintptr_t>());
|
||||
|
||||
// Now figure out which socket to send to and do the actual sending. We can repeat this loop
|
||||
// multiple times, if we're sending to a SN, because it's possible that we have multiple
|
||||
// connections open to that SN (e.g. one out + one in) so if one fails we can clean up that
|
||||
// connection and try the next one.
|
||||
bool retry = true, sent = false;
|
||||
bool retry = true, sent = false, warned = false;
|
||||
std::unique_ptr<zmq::error_t> send_error;
|
||||
while (retry) {
|
||||
retry = false;
|
||||
zmq::socket_t *send_to;
|
||||
|
@ -103,7 +106,7 @@ void LokiMQ::proxy_send(bt_dict_consumer data) {
|
|||
LMQ_LOG(debug, "Not sending: send is optional and no connection to ",
|
||||
to_hex(conn_id.pk), " is currently established");
|
||||
else
|
||||
LMQ_LOG(error, "Unable to send to ", to_hex(conn_id.pk), ": no connection address found");
|
||||
LMQ_LOG(error, "Unable to send to ", to_hex(conn_id.pk), ": no valid connection address found");
|
||||
break;
|
||||
}
|
||||
send_to = sock_route.first;
|
||||
|
@ -126,8 +129,7 @@ void LokiMQ::proxy_send(bt_dict_consumer data) {
|
|||
}
|
||||
|
||||
try {
|
||||
send_message_parts(*send_to, build_send_parts(send, conn_id.route));
|
||||
sent = true;
|
||||
sent = send_message_parts(*send_to, build_send_parts(send, conn_id.route));
|
||||
} catch (const zmq::error_t &e) {
|
||||
if (e.num() == EHOSTUNREACH && !conn_id.route.empty() /*= incoming conn*/) {
|
||||
|
||||
|
@ -158,6 +160,11 @@ void LokiMQ::proxy_send(bt_dict_consumer data) {
|
|||
}
|
||||
if (!retry) {
|
||||
LMQ_LOG(warn, "Unable to send message to ", conn_id, ": ", e.what());
|
||||
warned = true;
|
||||
if (callback_nosend) {
|
||||
job([callback = std::move(callback_nosend), error = e] { callback(&error); });
|
||||
callback_nosend = nullptr;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -171,6 +178,14 @@ void LokiMQ::proxy_send(bt_dict_consumer data) {
|
|||
job([callback = std::move(request_callback)] { callback(false, {}); });
|
||||
}
|
||||
}
|
||||
if (!sent) {
|
||||
if (callback_nosend)
|
||||
job([callback = std::move(callback_nosend)] { callback(nullptr); });
|
||||
else if (callback_noqueue)
|
||||
job(std::move(callback_noqueue));
|
||||
else if (!warned)
|
||||
LMQ_LOG(warn, "Unable to send message to ", conn_id, ": sending would block");
|
||||
}
|
||||
}
|
||||
|
||||
void LokiMQ::proxy_reply(bt_dict_consumer data) {
|
||||
|
@ -456,7 +471,7 @@ bool LokiMQ::proxy_handle_builtin(size_t conn_index, std::vector<zmq::message_t>
|
|||
route = view(parts[0]);
|
||||
cmd = view(parts[1]);
|
||||
}
|
||||
LMQ_TRACE("Checking for builtins: ", cmd, " from ", peer_address(parts.back()));
|
||||
LMQ_TRACE("Checking for builtins: '", cmd, "' from ", peer_address(parts.back()));
|
||||
|
||||
if (cmd == "REPLY") {
|
||||
size_t tag_pos = (outgoing ? 1 : 2);
|
||||
|
|
|
@ -234,9 +234,9 @@ void LokiMQ::proxy_to_worker(size_t conn_index, std::vector<zmq::message_t>& par
|
|||
LMQ_LOG(warn, "Invalid command '", command, "' sent by remote [", to_hex(peer->pubkey), "]/", peer_address(parts.back()));
|
||||
try {
|
||||
if (outgoing)
|
||||
send_direct_message(connections[conn_index], "UNKNOWNCOMMAND", command, zmq::send_flags::dontwait);
|
||||
send_direct_message(connections[conn_index], "UNKNOWNCOMMAND");
|
||||
else
|
||||
send_routed_message(connections[conn_index], peer->route, "UNKNOWNCOMMAND", command, zmq::send_flags::dontwait);
|
||||
send_routed_message(connections[conn_index], peer->route, "UNKNOWNCOMMAND");
|
||||
} catch (const zmq::error_t&) { /* can't send: possibly already disconnected. Ignore. */ }
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -279,3 +279,96 @@ TEST_CASE("deferred replies on incoming connections", "[commands][hey google]")
|
|||
REQUIRE( google_knows == personal_details );
|
||||
}
|
||||
}
|
||||
|
||||
TEST_CASE("send failure callbacks", "[commands][queue_full]") {
|
||||
std::string listen = "tcp://127.0.0.1:4567";
|
||||
LokiMQ server{
|
||||
"", "", // generate ephemeral keys
|
||||
false, // not a service node
|
||||
[](auto) { return ""; },
|
||||
get_logger("S» ")
|
||||
};
|
||||
server.log_level(LogLevel::debug);
|
||||
server.listen_plain(listen, [](auto /*ip*/, auto /*pk*/) { return Allow{AuthLevel::none, false}; });
|
||||
|
||||
std::atomic<int> send_attempts{0};
|
||||
std::atomic<int> send_failures{0};
|
||||
// ZMQ TCP sockets' HWM is complicated and OS dependent; sender and receiver (probably) each
|
||||
// have 1000 message queues, but there is also the TCP queue to worry about which means we can
|
||||
// have more queued before we fill up, so we send 4kiB of null with each message so that we
|
||||
// don't get too much TCP queuing.
|
||||
std::string junk(4096, '0');
|
||||
server.add_category("x", Access{AuthLevel::none})
|
||||
.add_command("x", [&](Message& m) {
|
||||
for (int x = 0; x < 500; x++) {
|
||||
++send_attempts;
|
||||
m.send_back("y.y", junk, send_option::queue_full{[&]() { ++send_failures; }});
|
||||
}
|
||||
});
|
||||
|
||||
server.start();
|
||||
|
||||
// Use a raw socket here because I want to stall it by not reading from it at all, and that is
|
||||
// hard with LokiMQ.
|
||||
zmq::context_t client_ctx;
|
||||
zmq::socket_t client{client_ctx, zmq::socket_type::dealer};
|
||||
client.connect(listen);
|
||||
// Handshake: we send HI, they reply HELLO.
|
||||
client.send(zmq::message_t{"HI", 2}, zmq::send_flags::none);
|
||||
zmq::message_t hello;
|
||||
client.recv(hello);
|
||||
string_view hello_sv{hello.data<char>(), hello.size()};
|
||||
{
|
||||
auto lock = catch_lock();
|
||||
REQUIRE( hello_sv == "HELLO" );
|
||||
REQUIRE_FALSE( hello.more() );
|
||||
}
|
||||
|
||||
// Tell the remote to queue up a batch of messages
|
||||
client.send(zmq::message_t{"x.x", 3}, zmq::send_flags::none);
|
||||
|
||||
int i;
|
||||
for (i = 0; i < 20; i++) {
|
||||
if (send_attempts.load() >= 500)
|
||||
break;
|
||||
std::this_thread::sleep_for(25ms);
|
||||
}
|
||||
{
|
||||
auto lock = catch_lock();
|
||||
REQUIRE( i <= 4 ); // should be not too slow
|
||||
// We have two buffers here: 1000 on the receiver, and 1000 on the client, which means we
|
||||
// should be able to get 2000 out before we hit HWM. We should only have been sent 501 so
|
||||
// far (the "HELLO" handshake + 500 "y.y" messages).
|
||||
REQUIRE( send_attempts.load() == 500 );
|
||||
REQUIRE( send_failures.load() == 0 );
|
||||
}
|
||||
|
||||
// Now we want to tell the server to send enough to fill the outgoing queue and start stalling.
|
||||
// This is complicated as it depends on ZMQ internals *and* OS-level TCP buffers, so we really
|
||||
// don't know precisely where this will start failing.
|
||||
//
|
||||
// In practice, I seem to reach HWM (for this test, with this amount of data being sent, on my
|
||||
// Debian desktop) after 2499 messages (that is, queuing 2500 gives 1 failure).
|
||||
int expected_attempts = 500;
|
||||
for (int i = 0; i < 10; i++) {
|
||||
client.send(zmq::message_t{"x.x", 3}, zmq::send_flags::none);
|
||||
expected_attempts += 500;
|
||||
if (i >= 4) {
|
||||
std::this_thread::sleep_for(25ms);
|
||||
if (send_failures.load() > 0)
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
for (i = 0; i < 10; i++) {
|
||||
if (send_attempts.load() >= expected_attempts)
|
||||
break;
|
||||
std::this_thread::sleep_for(25ms);
|
||||
}
|
||||
{
|
||||
auto lock = catch_lock();
|
||||
REQUIRE( i <= 8 );
|
||||
REQUIRE( send_attempts.load() == expected_attempts );
|
||||
REQUIRE( send_failures.load() > 0 );
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in a new issue