From c86b115d51e59940ac043a90de55dc33884a6ce4 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 10 Apr 2024 15:22:36 +0800 Subject: [PATCH 01/22] [coro_io] fix client pool slow connect bug --- include/ylt/coro_io/client_pool.hpp | 271 ++++++++---------- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 46 +-- .../standalone/cinatra/coro_http_client.hpp | 2 + src/coro_io/tests/test_client_pool.cpp | 2 +- .../examples/base_examples/channel.cpp | 8 +- .../examples/base_examples/client_pool.cpp | 62 +++- .../examples/base_examples/client_pools.cpp | 6 +- .../base_examples/concurrent_clients.cpp | 4 +- 8 files changed, 205 insertions(+), 196 deletions(-) diff --git a/include/ylt/coro_io/client_pool.hpp b/include/ylt/coro_io/client_pool.hpp index 062755890..636dfe315 100644 --- a/include/ylt/coro_io/client_pool.hpp +++ b/include/ylt/coro_io/client_pool.hpp @@ -42,6 +42,7 @@ #include #include +#include "async_simple/Common.h" #include "async_simple/coro/Collect.h" #include "coro_io.hpp" #include "detail/client_queue.hpp" @@ -105,46 +106,14 @@ class client_pool : public std::enable_shared_from_this< co_return; } - struct client_connect_helper { - std::unique_ptr client; - std::weak_ptr pool_watcher; - std::weak_ptr spinlock_watcher; - client_connect_helper(std::unique_ptr&& client, - std::weak_ptr&& pool_watcher, - std::weak_ptr&& spinlock_watcher) - : client(std::move(client)), - pool_watcher(std::move(pool_watcher)), - spinlock_watcher(std::move(spinlock_watcher)) {} - client_connect_helper(client_connect_helper&& o) - : client(std::move(o.client)), - pool_watcher(std::move(o.pool_watcher)), - spinlock_watcher(std::move(o.spinlock_watcher)) {} - client_connect_helper& operator=(client_connect_helper&& o) { - client = std::move(o.client); - pool_watcher = std::move(o.pool_watcher); - spinlock_watcher = std::move(o.spinlock_watcher); - return *this; - } - ~client_connect_helper() { - if (client) { - if (auto pool = pool_watcher.lock(); pool) { - int cnt = 0; - while (spinlock_watcher.lock()) { - std::this_thread::yield(); - ++cnt; - if (cnt % 10000 == 0) { - ELOG_WARN << "spinlock of client{" << client.get() << "},host:{" - << client->get_host() << ":" << client->get_port() - << "}cost too much time, spin count: " << cnt; - } - } - pool->collect_free_client(std::move(client)); - } - } - } - }; + static auto rand_time(std::chrono::milliseconds ms) { + static thread_local std::default_random_engine r; + std::uniform_real_distribution e(0.7f, 1.3f); + return std::chrono::milliseconds{static_cast(e(r) * ms.count())}; + } async_simple::coro::Lazy reconnect(std::unique_ptr& client) { + using namespace std::chrono_literals; for (unsigned int i = 0; i < pool_config_.connect_retry_count; ++i) { ELOG_DEBUG << "try to reconnect client{" << client.get() << "},host:{" << client->get_host() << ":" << client->get_port() @@ -164,9 +133,10 @@ class client_pool : public std::enable_shared_from_this< ELOG_DEBUG << "reconnect client{" << client.get() << "} failed. If client close:{" << client->has_closed() << "}"; - auto wait_time = pool_config_.reconnect_wait_time - cost_time; + auto wait_time = rand_time( + (pool_config_.reconnect_wait_time * (i + 1) - cost_time) / 1ms * 1ms); if (wait_time.count() > 0) - co_await coro_io::sleep_for(wait_time); + co_await coro_io::sleep_for(wait_time, &client->get_executor()); } ELOG_WARN << "reconnect client{" << client.get() << "},host:{" << client->get_host() << ":" << client->get_port() @@ -174,137 +144,115 @@ class client_pool : public std::enable_shared_from_this< client = nullptr; } - async_simple::coro::Lazy connect_client( - client_connect_helper helper) { - ELOG_DEBUG << "try to connect client{" << helper.client.get() + struct promise_handler { + std::atomic flag_ = false; + async_simple::Promise> promise_; + }; + + async_simple::coro::Lazy connect_client( + std::unique_ptr client, std::weak_ptr watcher, + std::shared_ptr handler) { + ELOG_DEBUG << "try to connect client{" << client.get() << "} to host:" << host_name_; - auto result = co_await helper.client->connect(host_name_); + auto result = co_await client->connect(host_name_); + std::shared_ptr self = watcher.lock(); if (!client_t::is_ok(result)) { - ELOG_DEBUG << "connect client{" << helper.client.get() << "} to failed. "; - co_await reconnect(helper.client); + ELOG_DEBUG << "connect client{" << client.get() << "} to failed. "; + if (self) { + co_await reconnect(client); + } } - if (helper.client) { - ELOG_DEBUG << "connect client{" << helper.client.get() << "} successful!"; + if (client) { + ELOG_DEBUG << "connect client{" << client.get() << "} successful!"; + } + auto has_response = handler->flag_.exchange(true); + if (!has_response) { + handler->promise_.setValue(std::move(client)); + } + else { + auto conn_lim = std::min(10u, pool_config_.max_connection); + if (self && free_clients_.size() < conn_lim && client) { + enqueue(free_clients_, std::move(client), pool_config_.idle_timeout); + } } - - co_return std::move(helper); - } - - auto rand_time() { - static thread_local std::default_random_engine r; - std::uniform_int_distribution e(-25, 25); - return std::chrono::milliseconds{100 + e(r)}; } async_simple::coro::Lazy> get_client( const typename client_t::config& client_config) { std::unique_ptr client; - free_clients_.try_dequeue(client); if (!client) { short_connect_clients_.try_dequeue(client); } - assert(client == nullptr || !client->has_closed()); if (client == nullptr) { - client = std::make_unique(*io_context_pool_.get_executor()); - if (!client->init_config(client_config)) { - ELOG_ERROR << "init client config{" << client.get() << "} failed."; - co_return nullptr; - } - auto spinlock = std::make_shared(false); - auto client_ptr = client.get(); - auto result = co_await async_simple::coro::collectAny( - connect_client(client_connect_helper{ - std::move(client), this->shared_from_this(), spinlock}), - coro_io::sleep_for(rand_time())); - if (result.index() == 0) { // connect finish in 100ms - co_return std::move(std::get<0>(result).value().client); - } - else if (result.index() == 1) { // connect time cost more than 100ms - ELOG_DEBUG << "slow connection of client{" << client_ptr - << "}, try to get free client from pool."; - std::unique_ptr cli; - if (short_connect_clients_.try_dequeue(cli) || - free_clients_.try_dequeue(cli)) { - spinlock = nullptr; - ELOG_DEBUG << "get free client{" << cli.get() - << "} from pool. skip wait client{" << client_ptr - << "} connect"; - co_return std::move(cli); + std::unique_ptr cli; + auto executor = io_context_pool_.get_executor(); + client = std::make_unique(*executor); + if (!client->init_config(client_config)) + AS_UNLIKELY { + ELOG_ERROR << "init client config failed."; + co_return nullptr; } - else { - auto promise = std::make_unique< - async_simple::Promise>>(); - auto* promise_address = promise.get(); - promise_queue.enqueue(promise_address); - spinlock = nullptr; - if (short_connect_clients_.try_dequeue(cli) || - free_clients_.try_dequeue(cli)) { - collect_free_client(std::move(cli)); - } - ELOG_DEBUG << "wait for free client waiter promise{" - << promise_address << "} response because slow client{" - << client_ptr << "}"; - - auto res = co_await collectAny( - [](auto promise) - -> async_simple::coro::Lazy> { - co_return co_await promise->getFuture(); - }(std::move(promise)), - coro_io::sleep_for(this->pool_config_.max_connection_time)); - if (res.index() == 0) { - auto& res0 = std::get<0>(res); - if (!res0.hasError()) { - auto& cli = res0.value(); - ELOG_DEBUG << "get free client{" << cli.get() << "} from promise{" - << promise_address << "}. skip wait client{" - << client_ptr << "} connect"; - co_return std::move(cli); - } - else { - ELOG_ERROR << "Unexcepted branch"; - co_return nullptr; - } - } - else { - ELOG_ERROR << "Unexcepted branch. Out of max limitation of connect " - "time, connect " - "failed. skip wait client{" - << client_ptr << "} connect. " - << "skip wait promise {" << promise_address - << "} response"; - co_return nullptr; + auto client_ptr = client.get(); + auto handler = std::make_shared(); + connect_client(std::move(client), this->weak_from_this(), handler) + .start([](auto&&) { + }); + auto timer = std::make_shared( + executor->get_asio_executor()); + timer->expires_after(std::chrono::milliseconds{20}); + timer->async_await().start([watcher = this->weak_from_this(), handler, + client_ptr, timer](auto&& res) { + if (/*REMOVE?*/ res.value() && !handler->flag_) { + if (auto self = watcher.lock(); self) { + ++self->promise_cnt_; + self->promise_queue_.enqueue(handler); + timer->expires_after( + std::max(std::chrono::milliseconds{0}, + self->pool_config_.max_connection_time - + std::chrono::milliseconds{20})); + timer->async_await().start([handler = std::move(handler), + client_ptr = client_ptr](auto&& res) { + auto has_response = handler->flag_.exchange(true); + if (!has_response) { + ELOG_ERROR << "Out of max limitation of connect " + "time, connect " + "failed. skip wait client{" + << client_ptr << "} connect. "; + handler->promise_.setValue(std::unique_ptr{nullptr}); + } + }); } } - } - else { - ELOG_ERROR << "unknown collectAny index while wait client{" - << client_ptr << "} connect"; - co_return nullptr; + }); + ELOG_DEBUG << "wait client by promise {" << &handler->promise_ << "}"; + client = co_await handler->promise_.getFuture(); + ; + // REMOVE? + if (client) { + executor->schedule([timer] { + timer->cancel(); + }); } } else { ELOG_DEBUG << "get free client{" << client.get() << "}. from queue"; - co_return std::move(client); } + co_return std::move(client); } void enqueue( coro_io::detail::client_queue>& clients, - std::unique_ptr client, bool is_short_client) { + std::unique_ptr client, + std::chrono::milliseconds collect_time) { if (clients.enqueue(std::move(client)) == 1) { std::size_t expected = 0; if (clients.collecter_cnt_.compare_exchange_strong(expected, 1)) { ELOG_DEBUG << "start timeout client collecter of client_pool{" << host_name_ << "}"; collect_idle_timeout_client( - this->shared_from_this(), clients, - (std::max)( - (is_short_client - ? (std::min)(pool_config_.idle_timeout, - pool_config_.short_connect_idle_timeout) - : pool_config_.idle_timeout), - std::chrono::milliseconds{50}), + this->weak_from_this(), clients, + (std::max)(collect_time, std::chrono::milliseconds{50}), pool_config_.idle_queue_per_max_clear_count) .via(coro_io::get_global_executor()) .start([](auto&&) { @@ -314,28 +262,41 @@ class client_pool : public std::enable_shared_from_this< } void collect_free_client(std::unique_ptr client) { - ELOG_DEBUG << "collect free client{" << client.get() << "}"; - if (client && !client->has_closed()) { - async_simple::Promise>* promise = nullptr; - if (promise_queue.try_dequeue(promise)) { - promise->setValue(std::move(client)); - ELOG_DEBUG << "collect free client{" << client.get() - << "} wake up promise{" << promise << "}"; + if (!client || !client->has_closed()) { + std::shared_ptr handler; + if (promise_cnt_) { + int cnt = 0; + while (promise_queue_.try_dequeue(handler)) { + ++cnt; + auto is_time_out = handler->flag_.exchange(true); + if (!is_time_out) { + handler->promise_.setValue(std::move(client)); + promise_cnt_ -= cnt; + ELOG_DEBUG << "collect free client{" << client.get() + << "} and wake up promise{" << &handler->promise_ << "}"; + return; + } + } + promise_cnt_ -= cnt; } - else if (free_clients_.size() < pool_config_.max_connection) { - ELOG_DEBUG << "collect free client{" << client.get() << "} enqueue"; - enqueue(free_clients_, std::move(client), false); + + if (free_clients_.size() < pool_config_.max_connection) { + if (client) { + ELOG_DEBUG << "collect free client{" << client.get() << "} enqueue"; + enqueue(free_clients_, std::move(client), pool_config_.idle_timeout); + } } else { ELOG_DEBUG << "out of max connection limit <<" << pool_config_.max_connection << ", collect free client{" << client.get() << "} enqueue short connect queue"; - enqueue(short_connect_clients_, std::move(client), true); + enqueue(short_connect_clients_, std::move(client), + pool_config_.short_connect_idle_timeout); } } else { ELOG_DEBUG << "client{" << client.get() - << "} is nullptr or is closed. we won't collect it"; + << "} is closed. we won't collect it"; } return; @@ -489,8 +450,8 @@ class client_pool : public std::enable_shared_from_this< coro_io::detail::client_queue> short_connect_clients_; client_pools_t* pools_manager_ = nullptr; - moodycamel::ConcurrentQueue>*> - promise_queue; + std::atomic promise_cnt_ = 0; + moodycamel::ConcurrentQueue> promise_queue_; async_simple::Promise idle_timeout_waiter; std::string host_name_; pool_config pool_config_; @@ -559,8 +520,8 @@ class client_pools { } } if (has_inserted) { - ELOG_DEBUG << "add new client pool of {" << host_name - << "} to hash table"; + // ELOG_DEBUG << "add new client pool of {" << host_name + // << "} to hash table"; } else { ELOG_DEBUG << "add new client pool of {" << host_name diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index b3d924f7a..4271b962a 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -129,7 +129,7 @@ class coro_rpc_client { */ coro_rpc_client(asio::io_context::executor_type executor, uint32_t client_id = 0) - : executor(executor), + : executor_(executor), timer_(executor), socket_(std::make_shared(executor)) { config_.client_id = client_id; @@ -142,7 +142,7 @@ class coro_rpc_client { coro_rpc_client( coro_io::ExecutorWrapper<> &executor = *coro_io::get_global_executor(), uint32_t client_id = 0) - : executor(executor.get_asio_executor()), + : executor_(executor.get_asio_executor()), timer_(executor.get_asio_executor()), socket_(std::make_shared( executor.get_asio_executor())) { @@ -325,7 +325,7 @@ class coro_rpc_client { std::error_code err_code; timer_.cancel(err_code); - if (is_timeout_) { + if (*is_timeout_) { ret = rpc_result{ unexpect_t{}, rpc_error{errc::timed_out, "rpc call timed out"}}; } @@ -340,7 +340,7 @@ class coro_rpc_client { /*! * Get inner executor */ - auto &get_executor() { return executor; } + auto &get_executor() { return executor_; } uint32_t get_client_id() const { return config_.client_id; } @@ -381,8 +381,8 @@ class coro_rpc_client { void reset() { close_socket(socket_); socket_ = - std::make_shared(executor.get_asio_executor()); - is_timeout_ = false; + std::make_shared(executor_.get_asio_executor()); + *is_timeout_ = false; has_closed_ = false; } static bool is_ok(coro_rpc::err_code ec) noexcept { return !ec; } @@ -411,18 +411,18 @@ class coro_rpc_client { }); std::error_code ec = co_await coro_io::async_connect( - &executor, *socket_, config_.host, config_.port); + &executor_, *socket_, config_.host, config_.port); std::error_code err_code; timer_.cancel(err_code); if (ec) { - if (is_timeout_) { + if (*is_timeout_) { co_return errc::timed_out; } co_return errc::not_connected; } - if (is_timeout_) { + if (*is_timeout_) { ELOGV(WARN, "client_id %d connect timeout", config_.client_id); co_return errc::timed_out; } @@ -475,13 +475,14 @@ class coro_rpc_client { #endif async_simple::coro::Lazy timeout(auto duration, std::string err_msg) { timer_.expires_after(duration); + auto socker_watcher = socket_; + auto timeout_watcher = is_timeout_; bool is_timeout = co_await timer_.async_await(); if (!is_timeout) { co_return false; } - - is_timeout_ = is_timeout; - close_socket(socket_); + *timeout_watcher = is_timeout; + close_socket(socker_watcher); co_return true; } @@ -656,10 +657,10 @@ class coro_rpc_client { } #ifdef UNIT_TEST_INJECT if (g_action == inject_action::force_inject_client_write_data_timeout) { - is_timeout_ = true; + *is_timeout_ = true; } #endif - if (is_timeout_) { + if (*is_timeout_) { r = rpc_result{ unexpect_t{}, rpc_error{.code = errc::timed_out, .msg = {}}}; } @@ -789,13 +790,12 @@ class coro_rpc_client { offset, std::forward(args)...); } - void close_socket(std::shared_ptr socket) { - asio::dispatch( - executor.get_asio_executor(), [socket = std::move(socket)]() { - asio::error_code ignored_ec; - socket->shutdown(asio::ip::tcp::socket::shutdown_both, ignored_ec); - socket->close(ignored_ec); - }); + static void close_socket(std::shared_ptr socket) { + asio::dispatch(socket->get_executor(), [socket = std::move(socket)]() { + asio::error_code ignored_ec; + socket->shutdown(asio::ip::tcp::socket::shutdown_both, ignored_ec); + socket->close(ignored_ec); + }); } #ifdef UNIT_TEST_INJECT @@ -813,7 +813,7 @@ class coro_rpc_client { } #endif private: - coro_io::ExecutorWrapper<> executor; + coro_io::ExecutorWrapper<> executor_; coro_io::period_timer timer_; std::shared_ptr socket_; std::string read_buf_, resp_attachment_buf_; @@ -825,7 +825,7 @@ class coro_rpc_client { std::unique_ptr> ssl_stream_; bool ssl_init_ret_ = true; #endif - bool is_timeout_ = false; + std::shared_ptr is_timeout_ = std::make_shared(false); std::atomic has_closed_ = false; }; } // namespace coro_rpc diff --git a/include/ylt/standalone/cinatra/coro_http_client.hpp b/include/ylt/standalone/cinatra/coro_http_client.hpp index ad8a3bc3f..dd1c8c266 100644 --- a/include/ylt/standalone/cinatra/coro_http_client.hpp +++ b/include/ylt/standalone/cinatra/coro_http_client.hpp @@ -197,6 +197,8 @@ class coro_http_client : public std::enable_shared_from_this { }); } + coro_io::ExecutorWrapper<> &get_executor() { return executor_wrapper_; } + #ifdef CINATRA_ENABLE_SSL bool init_ssl(int verify_mode, const std::string &base_path, const std::string &cert_file, const std::string &sni_hostname) { diff --git a/src/coro_io/tests/test_client_pool.cpp b/src/coro_io/tests/test_client_pool.cpp index 684fc5dca..3a9eff4ac 100644 --- a/src/coro_io/tests/test_client_pool.cpp +++ b/src/coro_io/tests/test_client_pool.cpp @@ -184,7 +184,7 @@ TEST_CASE("test reconnect retry wait time exclude reconnect cost time") { CHECK(pool->free_client_count() == 100); auto dur = std::chrono::steady_clock::now() - tp; std::cout << dur.count() << std::endl; - CHECK((dur >= 500ms && dur <= 800ms)); + CHECK((dur >= 400ms && dur <= 800ms)); server.stop(); co_return; }()); diff --git a/src/coro_rpc/examples/base_examples/channel.cpp b/src/coro_rpc/examples/base_examples/channel.cpp index e57b274cd..99f23c833 100644 --- a/src/coro_rpc/examples/base_examples/channel.cpp +++ b/src/coro_rpc/examples/base_examples/channel.cpp @@ -29,6 +29,8 @@ #include #include #include + +#include "ylt/easylog.hpp" using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; @@ -51,18 +53,18 @@ Lazy call_echo(std::shared_ptr> channel, [](coro_rpc_client &client, std::string_view hostname) -> Lazy { auto res = co_await client.call("Hello world!"); if (!res.has_value()) { - std::cout << "coro_rpc err: \n" << res.error().msg; + ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; co_return; } if (res.value() != "Hello world!"sv) { - std::cout << "err echo resp: \n" << res.value(); + ELOG_ERROR << "err echo resp: \n" << res.value(); co_return; } ++qps; co_return; }); if (!res) { - std::cout << "client pool err: connect failed.\n"; + ELOG_ERROR << "client pool err: connect failed.\n"; } } --working_echo; diff --git a/src/coro_rpc/examples/base_examples/client_pool.cpp b/src/coro_rpc/examples/base_examples/client_pool.cpp index a086016d2..23ba2b628 100644 --- a/src/coro_rpc/examples/base_examples/client_pool.cpp +++ b/src/coro_rpc/examples/base_examples/client_pool.cpp @@ -18,12 +18,20 @@ #include #include #include +#include #include #include #include #include #include + +#include "async_simple/coro/Collect.h" +#include "async_simple/coro/Lazy.h" +#include "async_simple/coro/SyncAwait.h" +#include "ylt/coro_io/io_context_pool.hpp" +#include "ylt/easylog.hpp" std::string echo(std::string_view sv); + using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; @@ -31,34 +39,52 @@ using namespace std::string_view_literals; std::atomic qps = 0; std::atomic working_echo = 0; +std::atomic busy_echo = 0; +struct guard { + guard(std::atomic &ref) : ref(ref) { ++ref; } + ~guard() { --ref; } + std::atomic &ref; +}; /*! * \example helloworld/concurrency_clients.main.cpp * \brief demo for run concurrency clients */ -Lazy call_echo(coro_io::client_pool &client_pool, - int cnt) { +Lazy> call_echo( + coro_io::client_pool &client_pool, int cnt) { + std::vector result; + result.reserve(cnt); ++working_echo; for (int i = 0; i < cnt; ++i) { + auto tp = std::chrono::steady_clock::now(); auto res = co_await client_pool.send_request( [=](coro_rpc_client &client) -> Lazy { + guard g{busy_echo}; + if (client.has_closed()) { + co_return; + } auto res = co_await client.call("Hello world!"); if (!res.has_value()) { - std::cout << "coro_rpc err: \n" << res.error().msg; + ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; + client.close(); co_return; } if (res.value() != "Hello world!"sv) { - std::cout << "err echo resp: \n" << res.value(); + ELOG_ERROR << "err echo resp: \n" << res.value(); co_return; } ++qps; co_return; }); if (!res) { - std::cout << "client pool err: connect failed.\n"; + ELOG_ERROR << "client pool err: connect failed.\n"; + } + else { + result.push_back(std::chrono::duration_cast( + std::chrono::steady_clock::now() - tp)); } } - --working_echo; + co_return std::move(result); } Lazy qps_watcher(coro_io::client_pool &clients) { @@ -68,11 +94,23 @@ Lazy qps_watcher(coro_io::client_pool &clients) { uint64_t cnt = qps.exchange(0); std::cout << "QPS:" << cnt << " free connection: " << clients.free_client_count() - << " working echo:" << working_echo << std::endl; + << " working echo:" << working_echo << " busy echo:" << busy_echo + << std::endl; cnt = 0; } } - +std::vector result; +void latency_watcher() { + std::sort(result.begin(), result.end()); + auto arr = {0.1, 0.3, 0.5, 0.7, 0.9, 0.95, 0.99, 0.999, 0.9999, 0.99999, 1.0}; + for (auto e : arr) { + std::cout + << (e * 100) << "% request finished in:" + << result[std::max(0, result.size() * e - 1)].count() / + 1000.0 + << "ms" << std::endl; + } +} int main() { auto thread_cnt = std::thread::hardware_concurrency(); auto client_pool = coro_io::client_pool::create( @@ -81,11 +119,17 @@ int main() { .max_connection = thread_cnt * 20, .client_config = {.timeout_duration = std::chrono::seconds{50}}}); + auto finish_executor = coro_io::get_global_block_executor(); for (int i = 0, lim = thread_cnt * 20; i < lim; ++i) { - call_echo(*client_pool, 10000).start([](auto &&) { + call_echo(*client_pool, 10000).start([finish_executor](auto &&res) { + finish_executor->schedule([res = std::move(res.value())] { + result.insert(result.end(), res.begin(), res.end()); + --working_echo; + }); }); } syncAwait(qps_watcher(*client_pool)); + latency_watcher(); std::cout << "Done!" << std::endl; return 0; } \ No newline at end of file diff --git a/src/coro_rpc/examples/base_examples/client_pools.cpp b/src/coro_rpc/examples/base_examples/client_pools.cpp index b06303a98..8bf414d60 100644 --- a/src/coro_rpc/examples/base_examples/client_pools.cpp +++ b/src/coro_rpc/examples/base_examples/client_pools.cpp @@ -48,18 +48,18 @@ Lazy call_echo(coro_io::client_pools &client_pools, [=](coro_rpc_client &client) -> Lazy { auto res = co_await client.call("Hello world!"); if (!res.has_value()) { - std::cout << "coro_rpc err: \n" << res.error().msg; + ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; co_return; } if (res.value() != "Hello world!"sv) { - std::cout << "err echo resp: \n" << res.value(); + ELOG_ERROR << "err echo resp: \n" << res.value(); co_return; } ++qps; co_return; }); if (!res) { - std::cout << "client pool err: connect failed.\n"; + ELOG_ERROR << "client pool err: connect failed.\n"; } } --working_echo; diff --git a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp index fdbf6847d..c8e8e7a20 100644 --- a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp +++ b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp @@ -54,11 +54,11 @@ Lazy call_echo(int cnt) { for (int i = 0; i < cnt; ++i) { auto res = co_await client.call("Hello world!"); if (!res.has_value()) { - std::cout << "coro_rpc err: \n" << res.error().msg; + ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; co_return; } if (res.value() != "Hello world!"sv) { - std::cout << "err echo resp: \n" << res.value(); + ELOG_ERROR << "err echo resp: \n" << res.value(); co_return; } ++qps; From 8094d3b98c12a4da30000ceef9dfac214fc3ed77 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 10 Apr 2024 16:35:50 +0800 Subject: [PATCH 02/22] fix gcc crash --- include/ylt/coro_io/client_pool.hpp | 6 +- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 79 +++++++++++-------- src/coro_rpc/tests/test_coro_rpc_client.cpp | 2 +- 3 files changed, 48 insertions(+), 39 deletions(-) diff --git a/include/ylt/coro_io/client_pool.hpp b/include/ylt/coro_io/client_pool.hpp index 636dfe315..fe62f34b7 100644 --- a/include/ylt/coro_io/client_pool.hpp +++ b/include/ylt/coro_io/client_pool.hpp @@ -208,9 +208,9 @@ class client_pool : public std::enable_shared_from_this< ++self->promise_cnt_; self->promise_queue_.enqueue(handler); timer->expires_after( - std::max(std::chrono::milliseconds{0}, - self->pool_config_.max_connection_time - - std::chrono::milliseconds{20})); + (std::max)(std::chrono::milliseconds{0}, + self->pool_config_.max_connection_time - + std::chrono::milliseconds{20})); timer->async_await().start([handler = std::move(handler), client_ptr = client_ptr](auto&& res) { auto has_response = handler->flag_.exchange(true); diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index 4271b962a..55d03c019 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -129,9 +129,8 @@ class coro_rpc_client { */ coro_rpc_client(asio::io_context::executor_type executor, uint32_t client_id = 0) - : executor_(executor), - timer_(executor), - socket_(std::make_shared(executor)) { + : control_(std::make_shared(executor, false)), + timer_(executor) { config_.client_id = client_id; } @@ -142,10 +141,9 @@ class coro_rpc_client { coro_rpc_client( coro_io::ExecutorWrapper<> &executor = *coro_io::get_global_executor(), uint32_t client_id = 0) - : executor_(executor.get_asio_executor()), - timer_(executor.get_asio_executor()), - socket_(std::make_shared( - executor.get_asio_executor())) { + : control_( + std::make_shared(executor.get_asio_executor(), false)), + timer_(executor.get_asio_executor()) { config_.client_id = client_id; } @@ -317,7 +315,7 @@ class coro_rpc_client { } else { #endif - ret = co_await call_impl(*socket_, std::move(args)...); + ret = co_await call_impl(control_->socket_, std::move(args)...); #ifdef YLT_ENABLE_SSL } #endif @@ -325,7 +323,7 @@ class coro_rpc_client { std::error_code err_code; timer_.cancel(err_code); - if (*is_timeout_) { + if (control_->is_timeout_) { ret = rpc_result{ unexpect_t{}, rpc_error{errc::timed_out, "rpc call timed out"}}; } @@ -340,7 +338,7 @@ class coro_rpc_client { /*! * Get inner executor */ - auto &get_executor() { return executor_; } + auto &get_executor() { return control_->executor_; } uint32_t get_client_id() const { return config_.client_id; } @@ -350,7 +348,7 @@ class coro_rpc_client { } has_closed_ = true; ELOGV(INFO, "client_id %d close", config_.client_id); - close_socket(socket_); + close_socket(control_); } bool set_req_attachment(std::string_view attachment) { @@ -379,10 +377,10 @@ class coro_rpc_client { }; void reset() { - close_socket(socket_); - socket_ = - std::make_shared(executor_.get_asio_executor()); - *is_timeout_ = false; + close_socket(control_); + control_->socket_ = + asio::ip::tcp::socket(control_->executor_.get_asio_executor()); + control_->is_timeout_ = false; has_closed_ = false; } static bool is_ok(coro_rpc::err_code ec) noexcept { return !ec; } @@ -411,23 +409,23 @@ class coro_rpc_client { }); std::error_code ec = co_await coro_io::async_connect( - &executor_, *socket_, config_.host, config_.port); + &control_->executor_, control_->socket_, config_.host, config_.port); std::error_code err_code; timer_.cancel(err_code); if (ec) { - if (*is_timeout_) { + if (control_->is_timeout_) { co_return errc::timed_out; } co_return errc::not_connected; } - if (*is_timeout_) { + if (control_->is_timeout_) { ELOGV(WARN, "client_id %d connect timeout", config_.client_id); co_return errc::timed_out; } - socket_->set_option(asio::ip::tcp::no_delay(true), ec); + control_->socket_.set_option(asio::ip::tcp::no_delay(true), ec); #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { @@ -465,7 +463,7 @@ class coro_rpc_client { asio::ssl::host_name_verification(config_.ssl_domain)); ssl_stream_ = std::make_unique>( - *socket_, ssl_ctx_); + control_->socket_, ssl_ctx_); ssl_init_ret_ = true; } catch (std::exception &e) { ELOGV(ERROR, "init ssl failed: %s", e.what()); @@ -475,15 +473,17 @@ class coro_rpc_client { #endif async_simple::coro::Lazy timeout(auto duration, std::string err_msg) { timer_.expires_after(duration); - auto socker_watcher = socket_; - auto timeout_watcher = is_timeout_; + std::weak_ptr socket_watcher = control_; bool is_timeout = co_await timer_.async_await(); if (!is_timeout) { co_return false; } - *timeout_watcher = is_timeout; - close_socket(socker_watcher); - co_return true; + if (auto self = socket_watcher.lock()) { + self->is_timeout_ = is_timeout; + close_socket(self); + co_return true; + } + co_return false; } template @@ -582,7 +582,7 @@ class coro_rpc_client { ret = co_await coro_io::async_write( socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); ELOGV(INFO, "client_id %d shutdown", config_.client_id); - socket_->shutdown(asio::ip::tcp::socket::shutdown_send); + control_->socket_.shutdown(asio::ip::tcp::socket::shutdown_send); r = rpc_result{ unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; co_return r; @@ -657,10 +657,10 @@ class coro_rpc_client { } #ifdef UNIT_TEST_INJECT if (g_action == inject_action::force_inject_client_write_data_timeout) { - *is_timeout_ = true; + control_->is_timeout_ = true; } #endif - if (*is_timeout_) { + if (control_->is_timeout_) { r = rpc_result{ unexpect_t{}, rpc_error{.code = errc::timed_out, .msg = {}}}; } @@ -790,11 +790,21 @@ class coro_rpc_client { offset, std::forward(args)...); } - static void close_socket(std::shared_ptr socket) { - asio::dispatch(socket->get_executor(), [socket = std::move(socket)]() { + struct control_t { + asio::ip::tcp::socket socket_; + bool is_timeout_; + coro_io::ExecutorWrapper<> executor_; + control_t(asio::io_context::executor_type executor, bool is_timeout) + : socket_(executor), is_timeout_(is_timeout), executor_(executor) {} + }; + + static void close_socket( + std::shared_ptr control) { + control->executor_.schedule([control = std::move(control)]() { asio::error_code ignored_ec; - socket->shutdown(asio::ip::tcp::socket::shutdown_both, ignored_ec); - socket->close(ignored_ec); + control->socket_.shutdown(asio::ip::tcp::socket::shutdown_both, + ignored_ec); + control->socket_.close(ignored_ec); }); } @@ -812,10 +822,10 @@ class coro_rpc_client { call(std::forward(args)...)); } #endif + private: - coro_io::ExecutorWrapper<> executor_; coro_io::period_timer timer_; - std::shared_ptr socket_; + std::shared_ptr control_; std::string read_buf_, resp_attachment_buf_; std::string_view req_attachment_; config config_; @@ -825,7 +835,6 @@ class coro_rpc_client { std::unique_ptr> ssl_stream_; bool ssl_init_ret_ = true; #endif - std::shared_ptr is_timeout_ = std::make_shared(false); std::atomic has_closed_ = false; }; } // namespace coro_rpc diff --git a/src/coro_rpc/tests/test_coro_rpc_client.cpp b/src/coro_rpc/tests/test_coro_rpc_client.cpp index a820f8e73..f2bc1eaec 100644 --- a/src/coro_rpc/tests/test_coro_rpc_client.cpp +++ b/src/coro_rpc/tests/test_coro_rpc_client.cpp @@ -116,7 +116,7 @@ TEST_CASE("testing client") { g_action = {}; auto f = [&io_context, &port]() -> Lazy { auto client = co_await create_client(io_context, port); - auto ret = co_await client->template call_for(20ms); + auto ret = co_await client->template call_for(10ms); CHECK_MESSAGE(ret.error().code == coro_rpc::errc::timed_out, ret.error().msg); co_return; From 6799b0f103fe026e2a44de9c1dbdfaf51c576188 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 10 Apr 2024 17:26:18 +0800 Subject: [PATCH 03/22] fix --- include/ylt/coro_io/client_pool.hpp | 30 +++++++++++------------------ 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/include/ylt/coro_io/client_pool.hpp b/include/ylt/coro_io/client_pool.hpp index fe62f34b7..2f44949f2 100644 --- a/include/ylt/coro_io/client_pool.hpp +++ b/include/ylt/coro_io/client_pool.hpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -165,8 +166,8 @@ class client_pool : public std::enable_shared_from_this< if (client) { ELOG_DEBUG << "connect client{" << client.get() << "} successful!"; } - auto has_response = handler->flag_.exchange(true); - if (!has_response) { + auto has_get_connect = handler->flag_.exchange(true); + if (!has_get_connect) { handler->promise_.setValue(std::move(client)); } else { @@ -203,7 +204,7 @@ class client_pool : public std::enable_shared_from_this< timer->expires_after(std::chrono::milliseconds{20}); timer->async_await().start([watcher = this->weak_from_this(), handler, client_ptr, timer](auto&& res) { - if (/*REMOVE?*/ res.value() && !handler->flag_) { + if (res.value() && !handler->flag_) { if (auto self = watcher.lock(); self) { ++self->promise_cnt_; self->promise_queue_.enqueue(handler); @@ -213,8 +214,8 @@ class client_pool : public std::enable_shared_from_this< std::chrono::milliseconds{20})); timer->async_await().start([handler = std::move(handler), client_ptr = client_ptr](auto&& res) { - auto has_response = handler->flag_.exchange(true); - if (!has_response) { + auto has_get_connect = handler->flag_.exchange(true); + if (!has_get_connect) { ELOG_ERROR << "Out of max limitation of connect " "time, connect " "failed. skip wait client{" @@ -227,11 +228,10 @@ class client_pool : public std::enable_shared_from_this< }); ELOG_DEBUG << "wait client by promise {" << &handler->promise_ << "}"; client = co_await handler->promise_.getFuture(); - ; - // REMOVE? if (client) { executor->schedule([timer] { - timer->cancel(); + std::error_code ignore_ec; + timer->cancel(ignore_ec); }); } } @@ -262,14 +262,14 @@ class client_pool : public std::enable_shared_from_this< } void collect_free_client(std::unique_ptr client) { - if (!client || !client->has_closed()) { + if (!client->has_closed()) { std::shared_ptr handler; if (promise_cnt_) { int cnt = 0; while (promise_queue_.try_dequeue(handler)) { ++cnt; - auto is_time_out = handler->flag_.exchange(true); - if (!is_time_out) { + auto has_get_connect = handler->flag_.exchange(true); + if (!has_get_connect) { handler->promise_.setValue(std::move(client)); promise_cnt_ -= cnt; ELOG_DEBUG << "collect free client{" << client.get() @@ -519,14 +519,6 @@ class client_pools { iter->second = pool; } } - if (has_inserted) { - // ELOG_DEBUG << "add new client pool of {" << host_name - // << "} to hash table"; - } - else { - ELOG_DEBUG << "add new client pool of {" << host_name - << "} failed, element existed."; - } } return iter->second; } From 9ce4f7fae22e091768ac3e9ab76bfa64da535e39 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 10 Apr 2024 17:50:18 +0800 Subject: [PATCH 04/22] fix fake mem leak --- src/coro_rpc/tests/test_coro_rpc_client.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/coro_rpc/tests/test_coro_rpc_client.cpp b/src/coro_rpc/tests/test_coro_rpc_client.cpp index f2bc1eaec..71b69eab4 100644 --- a/src/coro_rpc/tests/test_coro_rpc_client.cpp +++ b/src/coro_rpc/tests/test_coro_rpc_client.cpp @@ -72,9 +72,9 @@ TEST_CASE("testing client") { std::string port = std::to_string(coro_rpc_server_port); asio::io_context io_context; std::promise promise; + auto worker = std::make_unique(io_context); auto future = promise.get_future(); std::thread thd([&io_context, &promise] { - asio::io_context::work work(io_context); promise.set_value(); io_context.run(); }); @@ -154,7 +154,7 @@ TEST_CASE("testing client") { } server.stop(); - io_context.stop(); + worker = nullptr; thd.join(); } @@ -163,8 +163,8 @@ TEST_CASE("testing client with inject server") { std::string port = std::to_string(coro_rpc_server_port); ELOGV(INFO, "inject server port: %d", port.data()); asio::io_context io_context; + auto worker = std::make_unique(io_context); std::thread thd([&io_context] { - asio::io_context::work work(io_context); io_context.run(); }); coro_rpc_server server(2, coro_rpc_server_port); @@ -212,7 +212,7 @@ TEST_CASE("testing client with inject server") { } server.stop(); - io_context.stop(); + worker = nullptr; thd.join(); g_action = inject_action::nothing; } @@ -245,15 +245,15 @@ class SSLClientTester { std::promise promise; auto future = promise.get_future(); + worker = std::make_unique(io_context); thd = std::thread([this, &promise] { - asio::io_context::work work(io_context); promise.set_value(); io_context.run(); }); future.wait(); } ~SSLClientTester() { - io_context.stop(); + worker = nullptr; thd.join(); } void inject(std::string msg, std::string& path, ssl_type type) { @@ -342,6 +342,7 @@ class SSLClientTester { ssl_type dh; asio::io_context io_context; std::thread thd; + std::unique_ptr worker; }; TEST_CASE("testing client with ssl server") { From 7b79cbdeed57482096be999a12a74f6a0989b556 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Fri, 26 Apr 2024 17:40:52 +0800 Subject: [PATCH 05/22] coro_rpc_client allow async_all --- include/ylt/coro_io/io_context_pool.hpp | 2 +- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 654 ++++++++++++------ include/ylt/coro_rpc/impl/errno.h | 7 + include/ylt/coro_rpc/impl/expected.hpp | 2 +- src/coro_rpc/tests/test_coro_rpc_client.cpp | 6 + 5 files changed, 440 insertions(+), 231 deletions(-) diff --git a/include/ylt/coro_io/io_context_pool.hpp b/include/ylt/coro_io/io_context_pool.hpp index 07eb8e522..b45c63ff9 100644 --- a/include/ylt/coro_io/io_context_pool.hpp +++ b/include/ylt/coro_io/io_context_pool.hpp @@ -75,7 +75,7 @@ class ExecutorWrapper : public async_simple::Executor { context_t &context() { return executor_.context(); } - auto get_asio_executor() { return executor_; } + auto get_asio_executor() const { return executor_; } operator ExecutorImpl() { return executor_; } diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index 55d03c019..c42954d8a 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -22,10 +22,12 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include @@ -33,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -40,6 +43,8 @@ #include "asio/buffer.hpp" #include "asio/dispatch.hpp" #include "asio/registered_buffer.hpp" +#include "async_simple/Executor.h" +#include "async_simple/Promise.h" #include "common_service.hpp" #include "context.hpp" #include "expected.hpp" @@ -80,6 +85,23 @@ template <> struct rpc_return_type { using type = std::monostate; }; + +struct rpc_resp_buffer { + std::string read_buf_; + std::string resp_attachment_buf_; +}; + +template +struct async_rpc_result { + T result_; + rpc_resp_buffer buffer_; +}; + +template <> +struct async_rpc_result { + rpc_resp_buffer buffer_; +}; + template using rpc_return_type_t = typename rpc_return_type::type; /*! @@ -130,7 +152,7 @@ class coro_rpc_client { coro_rpc_client(asio::io_context::executor_type executor, uint32_t client_id = 0) : control_(std::make_shared(executor, false)), - timer_(executor) { + timer_(std::make_unique(executor)) { config_.client_id = client_id; } @@ -143,7 +165,7 @@ class coro_rpc_client { uint32_t client_id = 0) : control_( std::make_shared(executor.get_asio_executor(), false)), - timer_(executor.get_asio_executor()) { + timer_(std::make_unique(executor.get_asio_executor())) { config_.client_id = client_id; } @@ -166,7 +188,7 @@ class coro_rpc_client { * * @return true if client closed, otherwise false. */ - [[nodiscard]] bool has_closed() { return has_closed_; } + [[nodiscard]] bool has_closed() { return control_->has_closed_; } /*! * Reconnect server @@ -261,8 +283,8 @@ class coro_rpc_client { template async_simple::coro::Lazy< rpc_result()), coro_rpc_protocol>> - call(Args... args) { - return call_for(std::chrono::seconds(5), std::move(args)...); + call(Args&&... args) { + return call_for(std::chrono::seconds(5), std::forward(args)...); } /*! @@ -279,60 +301,27 @@ class coro_rpc_client { template async_simple::coro::Lazy< rpc_result()), coro_rpc_protocol>> - call_for(auto duration, Args... args) { - using R = decltype(get_return_type()); - - if (has_closed_) - AS_UNLIKELY { - ELOGV(ERROR, "client has been closed, please re-connect"); - auto ret = rpc_result{ - unexpect_t{}, - rpc_error{errc::io_error, - "client has been closed, please re-connect"}}; - co_return ret; + call_for(auto duration, Args&&... args) { + is_waiting_for_response_=true; + using return_type=decltype(get_return_type()); + auto result = co_await send_request_for(duration, std::forward(args)...); + if (result) { + auto async_result = co_await result.value(); + if (async_result) { + if constexpr (std::is_same_v) { + co_return expected{}; + } + else { + co_return expected{std::move(async_result.value().result_)}; + } + } + else { + co_return expected{unexpect_t{},std::move(async_result.error())}; } - - rpc_result ret; -#ifdef YLT_ENABLE_SSL - if (!ssl_init_ret_) { - ret = rpc_result{ - unexpect_t{}, - rpc_error{errc::not_connected, - std::string{make_error_message(errc::not_connected)}}}; - co_return ret; - } -#endif - - static_check(); - - timeout(duration, "rpc call timer canceled").start([](auto &&) { - }); - -#ifdef YLT_ENABLE_SSL - if (!config_.ssl_cert_path.empty()) { - assert(ssl_stream_); - ret = co_await call_impl(*ssl_stream_, std::move(args)...); } else { -#endif - ret = co_await call_impl(control_->socket_, std::move(args)...); -#ifdef YLT_ENABLE_SSL - } -#endif - - std::error_code err_code; - timer_.cancel(err_code); - - if (control_->is_timeout_) { - ret = rpc_result{ - unexpect_t{}, rpc_error{errc::timed_out, "rpc call timed out"}}; + co_return expected{unexpect_t{},std::move(result.error())}; } - -#ifdef UNIT_TEST_INJECT - ELOGV(INFO, "client_id %d call %s %s", config_.client_id, - get_func_name().data(), ret ? "ok" : "failed"); -#endif - co_return ret; } /*! @@ -343,10 +332,6 @@ class coro_rpc_client { uint32_t get_client_id() const { return config_.client_id; } void close() { - if (has_closed_) { - return; - } - has_closed_ = true; ELOGV(INFO, "client_id %d close", config_.client_id); close_socket(control_); } @@ -360,10 +345,10 @@ class coro_rpc_client { return true; } - std::string_view get_resp_attachment() const { return resp_attachment_buf_; } + std::string_view get_resp_attachment() const { return control_->resp_buffer_.resp_attachment_buf_; } std::string release_resp_attachment() { - return std::move(resp_attachment_buf_); + return std::move(control_->resp_buffer_.resp_attachment_buf_); } template @@ -381,7 +366,7 @@ class coro_rpc_client { control_->socket_ = asio::ip::tcp::socket(control_->executor_.get_asio_executor()); control_->is_timeout_ = false; - has_closed_ = false; + control_->has_closed_ = false; } static bool is_ok(coro_rpc::err_code ec) noexcept { return !ec; } [[nodiscard]] async_simple::coro::Lazy connect( @@ -392,7 +377,7 @@ class coro_rpc_client { co_return errc::not_connected; } #endif - if (!is_reconnect.value && has_closed_) + if (!is_reconnect.value && control_->has_closed_) AS_UNLIKELY { ELOGV(ERROR, "a closed client is not allowed connect again, please use " @@ -400,18 +385,18 @@ class coro_rpc_client { "client"); co_return errc::io_error; } - has_closed_ = false; + control_->has_closed_ = false; ELOGV(INFO, "client_id %d begin to connect %s", config_.client_id, config_.port.data()); - timeout(config_.timeout_duration, "connect timer canceled") + timeout(*this->timer_,config_.timeout_duration, "connect timer canceled") .start([](auto &&) { }); std::error_code ec = co_await coro_io::async_connect( &control_->executor_, control_->socket_, config_.host, config_.port); std::error_code err_code; - timer_.cancel(err_code); + timer_->cancel(err_code); if (ec) { if (control_->is_timeout_) { @@ -429,9 +414,9 @@ class coro_rpc_client { #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { - assert(ssl_stream_); + assert(control_->ssl_stream_); auto shake_ec = co_await coro_io::async_handshake( - ssl_stream_, asio::ssl::stream_base::client); + control_->ssl_stream_, asio::ssl::stream_base::client); if (shake_ec) { ELOGV(WARN, "client_id %d handshake failed: %s", config_.client_id, shake_ec.message().data()); @@ -461,7 +446,7 @@ class coro_rpc_client { ssl_ctx_.set_verify_mode(asio::ssl::verify_peer); ssl_ctx_.set_verify_callback( asio::ssl::host_name_verification(config_.ssl_domain)); - ssl_stream_ = + control_->ssl_stream_ = std::make_unique>( control_->socket_, ssl_ctx_); ssl_init_ret_ = true; @@ -471,10 +456,12 @@ class coro_rpc_client { return ssl_init_ret_; } #endif - async_simple::coro::Lazy timeout(auto duration, std::string err_msg) { - timer_.expires_after(duration); + + async_simple::coro::Lazy timeout(coro_io::period_timer &timer, + auto duration, std::string err_msg) { + timer.expires_after(duration); std::weak_ptr socket_watcher = control_; - bool is_timeout = co_await timer_.async_await(); + bool is_timeout = co_await timer.async_await(); if (!is_timeout) { co_return false; } @@ -530,148 +517,6 @@ class coro_rpc_client { } } - template - async_simple::coro::Lazy< - rpc_result()), coro_rpc_protocol>> - call_impl(Socket &socket, Args... args) { - using R = decltype(get_return_type()); - - auto buffer = prepare_buffer(std::move(args)...); - - rpc_result r{}; - if (buffer.empty()) { - r = rpc_result{ - unexpect_t{}, rpc_error{errc::message_too_large, - "rpc body serialize size too big"}}; - co_return r; - } -#ifdef GENERATE_BENCHMARK_DATA - std::ofstream file( - benchmark_file_path + std::string{get_func_name()} + ".in", - std::ofstream::binary | std::ofstream::out); - file << std::string_view{(char *)buffer.data(), buffer.size()}; - file.close(); -#endif - std::pair ret; -#ifdef UNIT_TEST_INJECT - if (g_action == inject_action::client_send_bad_header) { - buffer[0] = (std::byte)(uint8_t(buffer[0]) + 1); - } - if (g_action == inject_action::client_close_socket_after_send_header) { - ret = co_await coro_io::async_write( - socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); - ELOGV(INFO, "client_id %d close socket", config_.client_id); - close(); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - co_return r; - } - else if (g_action == - inject_action::client_close_socket_after_send_partial_header) { - ret = co_await coro_io::async_write( - socket, - asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN - 1)); - ELOGV(INFO, "client_id %d close socket", config_.client_id); - close(); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - co_return r; - } - else if (g_action == - inject_action::client_shutdown_socket_after_send_header) { - ret = co_await coro_io::async_write( - socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); - ELOGV(INFO, "client_id %d shutdown", config_.client_id); - control_->socket_.shutdown(asio::ip::tcp::socket::shutdown_send); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - co_return r; - } - else { -#endif - if (req_attachment_.empty()) { - ret = co_await coro_io::async_write( - socket, asio::buffer(buffer.data(), buffer.size())); - } - else { - std::array iov{ - asio::const_buffer{buffer.data(), buffer.size()}, - asio::const_buffer{req_attachment_.data(), req_attachment_.size()}}; - ret = co_await coro_io::async_write(socket, iov); - req_attachment_ = {}; - } -#ifdef UNIT_TEST_INJECT - } -#endif - if (!ret.first) { -#ifdef UNIT_TEST_INJECT - if (g_action == inject_action::client_close_socket_after_send_payload) { - ELOGV(INFO, "client_id %d client_close_socket_after_send_payload", - config_.client_id); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - close(); - co_return r; - } -#endif - coro_rpc_protocol::resp_header header; - ret = co_await coro_io::async_read( - socket, - asio::buffer((char *)&header, coro_rpc_protocol::RESP_HEAD_LEN)); - if (!ret.first) { - uint32_t body_len = header.length; - struct_pack::detail::resize(read_buf_, body_len); - if (header.attach_length == 0) { - ret = co_await coro_io::async_read( - socket, asio::buffer(read_buf_.data(), body_len)); - resp_attachment_buf_.clear(); - } - else { - struct_pack::detail::resize(resp_attachment_buf_, - header.attach_length); - std::array iov{ - asio::mutable_buffer{read_buf_.data(), body_len}, - asio::mutable_buffer{resp_attachment_buf_.data(), - resp_attachment_buf_.size()}}; - ret = co_await coro_io::async_read(socket, iov); - } - if (!ret.first) { -#ifdef GENERATE_BENCHMARK_DATA - std::ofstream file( - benchmark_file_path + std::string{get_func_name()} + ".out", - std::ofstream::binary | std::ofstream::out); - file << std::string_view{(char *)&header, - coro_rpc_protocol::RESP_HEAD_LEN}; - file << read_buf_; - file << resp_attachment_buf_; - file.close(); -#endif - bool ec = false; - r = handle_response_buffer(read_buf_, header.err_code, ec); - if (ec) { - close(); - } - co_return r; - } - } - } -#ifdef UNIT_TEST_INJECT - if (g_action == inject_action::force_inject_client_write_data_timeout) { - control_->is_timeout_ = true; - } -#endif - if (control_->is_timeout_) { - r = rpc_result{ - unexpect_t{}, rpc_error{.code = errc::timed_out, .msg = {}}}; - } - else { - r = rpc_result{ - unexpect_t{}, - rpc_error{.code = errc::io_error, .msg = ret.first.message()}}; - } - close(); - co_return r; - } /* * buffer layout * ┌────────────────┬────────────────┐ @@ -681,7 +526,7 @@ class coro_rpc_client { * └────────────────┴────────────────┘ */ template - std::vector prepare_buffer(Args &&...args) { + std::vector prepare_buffer(uint32_t& id, Args &&...args) { std::vector buffer; std::size_t offset = coro_rpc_protocol::REQ_HEAD_LEN; if constexpr (sizeof...(Args) > 0) { @@ -697,8 +542,11 @@ class coro_rpc_client { header.magic = coro_rpc_protocol::magic_number; header.function_id = func_id(); header.attach_length = req_attachment_.size(); + id = request_id_++; + ELOG_TRACE<<"send request ID:"< - rpc_result handle_response_buffer(std::string &buffer, - uint8_t rpc_errc, - bool &error_happen) { + static rpc_result handle_response_buffer(std::string_view buffer, + uint8_t rpc_errc,bool& should_close) { rpc_return_type_t ret; struct_pack::err_code ec; rpc_error err; @@ -743,21 +590,20 @@ class coro_rpc_client { err.val() = rpc_errc; ec = struct_pack::deserialize_to(err.msg, buffer); if SP_LIKELY (!ec) { - ELOGV(WARNING, "deserilaize rpc result failed"); - error_happen = true; + should_close = true; return rpc_result{unexpect_t{}, std::move(err)}; } } else { ec = struct_pack::deserialize_to(err, buffer); if SP_LIKELY (!ec) { - ELOGV(WARNING, "deserilaize rpc result failed"); return rpc_result{unexpect_t{}, std::move(err)}; } } } - error_happen = true; + should_close = true; // deserialize failed. + ELOGV(WARNING, "deserilaize rpc result failed"); err = {errc::invalid_rpc_result, "failed to deserialize rpc return value"}; return rpc_result{unexpect_t{}, std::move(err)}; } @@ -790,16 +636,58 @@ class coro_rpc_client { offset, std::forward(args)...); } + struct async_rpc_raw_result_value_type { + std::variant buffer_; + uint8_t errc_; + }; + + using async_rpc_raw_result=std::variant; + + struct control_t; + + struct handler_t { + std::unique_ptr timer_; + control_t* control_; + async_simple::Promise promise_; + handler_t(std::unique_ptr&& timer,control_t* control,async_simple::Promise &&promise):timer_(std::move(timer)), control_(control),promise_(std::move(promise)) {} + void operator()(rpc_resp_buffer&& buffer,uint8_t rpc_errc) { + timer_->cancel(); + if (control_) /*is waiting for response*/ { + promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{std::string_view{control_->resp_buffer_.read_buf_},rpc_errc}}); + } + else { + promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{std::move(buffer),rpc_errc}}); + } + } + void local_error (std::error_code& ec) { + timer_->cancel(); + promise_.setValue(async_rpc_raw_result{ec}); + } + }; struct control_t { - asio::ip::tcp::socket socket_; +#ifdef YLT_ENABLE_SSL + std::unique_ptr> ssl_stream_; +#endif +#ifdef GENERATE_BENCHMARK_DATA + std::string func_name_; +#endif bool is_timeout_; + std::atomic has_closed_ = false; coro_io::ExecutorWrapper<> executor_; + std::unordered_map response_handler_table_; + rpc_resp_buffer resp_buffer_; + asio::ip::tcp::socket socket_; + std::atomic is_recving_=false; control_t(asio::io_context::executor_type executor, bool is_timeout) - : socket_(executor), is_timeout_(is_timeout), executor_(executor) {} + : socket_(executor), is_timeout_(is_timeout), has_closed_(false), executor_(executor) {} }; static void close_socket( std::shared_ptr control) { + if (control->has_closed_) { + return; + } + control->has_closed_ = true; control->executor_.schedule([control = std::move(control)]() { asio::error_code ignored_ec; control->socket_.shutdown(asio::ip::tcp::socket::shutdown_both, @@ -823,18 +711,326 @@ class coro_rpc_client { } #endif + + + template + async_simple::coro::Lazy + send_request_for_impl(auto duration, uint32_t& id, Args &&...args) { + using R = decltype(get_return_type()); + + if (control_->has_closed_) + AS_UNLIKELY { + ELOGV(ERROR, "client has been closed, please re-connect"); + co_return rpc_error{errc::io_error, + "client has been closed, please re-connect"}; + } + +#ifdef YLT_ENABLE_SSL + if (!ssl_init_ret_) { + co_return rpc_error{errc::not_connected}}; + } +#endif + + static_check(); + + if (duration.count() > 0) { + if (timer_ == nullptr) + timer_ = std::make_unique(control_->executor_.get_asio_executor()); + timeout(*timer_, duration, "rpc call timer canceled").start([](auto &&) { + }); + } + +#ifdef YLT_ENABLE_SSL + if (!config_.ssl_cert_path.empty()) { + assert(control_->ssl_stream_); + co_return co_await send_impl(*control_->ssl_stream_, id, std::forward(args)...); + } + else { +#endif + co_return co_await send_impl(control_->socket_, id, std::forward(args)...); +#ifdef YLT_ENABLE_SSL + } +#endif + } + + + static void send_err_response(control_t* controller, std::error_code& errc) { + rpc_error ec; + for (auto &e:controller->response_handler_table_) { + e.second.local_error(errc); + } + controller->response_handler_table_.clear(); + } + template + static async_simple::coro::Lazy recv(std::shared_ptr controller, Socket& socket) { + std::pair ret; + do { + coro_rpc_protocol::resp_header header; + ret = co_await coro_io::async_read( + socket, + asio::buffer((char *)&header, coro_rpc_protocol::RESP_HEAD_LEN)); + if (ret.first) { + ELOG_ERROR<<"read rpc head failed, error msg:"<func_name_ + ".out", + std::ofstream::binary | std::ofstream::out); + file << std::string_view{(char *)&header, + coro_rpc_protocol::RESP_HEAD_LEN}; + file << controller->resp_buffer_.read_buf_; + file << controller->resp_buffer_.resp_attachment_buf_; + file.close(); +#endif + if (auto iter=controller->response_handler_table_.find(header.seq_num);iter!=controller->response_handler_table_.end()) { + ELOG_TRACE<<"find request ID:"<second(std::move(controller->resp_buffer_), header.err_code); + controller->response_handler_table_.erase(iter); + } + else { + ELOG_ERROR<<"unexists request ID:"<response_handler_table_.size() == 0) { + controller->is_recving_= false; + co_return; + } + } while (true); + controller->is_recving_ = false; + close_socket(controller); + send_err_response(controller.get(),ret.first); + co_return; + } + + template + static async_simple::coro::Lazy,rpc_error>> get_deserializer(async_simple::Future future,std::weak_ptr watcher) { + auto executor = co_await async_simple::CurrentExecutor(); + auto executorFuture = std::move(future).via(executor); + auto ret_ = co_await std::move(executorFuture); + + if (ret_.index() ==1) [[unlikely]] { // local error + auto& ret=std::get<1>(ret_); + if (ret.value()==static_cast(std::errc::operation_canceled) || ret.value()==static_cast(std::errc::timed_out)) { + co_return coro_rpc::unexpected{rpc_error{errc::timed_out,ret.message()}}; + } + else { + co_return coro_rpc::unexpected{rpc_error{errc::io_error,ret.message()}}; + } + } + + bool should_close=false; + std::string_view buffer_view; + auto &ret=std::get<0>(ret_); + if (ret.buffer_.index()==0) { + buffer_view = std::get<0>(ret.buffer_).read_buf_; + } + else { + buffer_view = std::get<1>(ret.buffer_); + } + auto result = handle_response_buffer(buffer_view,ret.errc_,should_close); + if (should_close) { + if (auto w=watcher.lock();w) { + close_socket(std::move(w)); + } + } + if (result) { + if constexpr (std::is_same_v) { + if (ret.buffer_.index()==0) { + co_return async_rpc_result{std::move(std::get<0>(ret.buffer_))}; + } + else { + co_return async_rpc_result{}; + } + } + else { + if (ret.buffer_.index()==0) { + co_return async_rpc_result{result.value(),std::move(std::get<0>(ret.buffer_))}; + } + else { + co_return async_rpc_result{result.value()}; + } + } + } else { + co_return coro_rpc::unexpected{result.error()}; + } + } +public: + + template + async_simple::coro::Lazy())>, rpc_error>>, + rpc_error>> + send_request(Args &&...args) { + return send_request_for(std::chrono::seconds{5}, std::forward(args)...); + } + + template + async_simple::coro::Lazy())>, rpc_error>>, + rpc_error>> + send_request_for(auto duration, Args &&...args) { + uint32_t id; + auto result = co_await send_request_for_impl(duration, id, std::forward(args)...); + auto &control = *control_; + if (!result) { + async_simple::Promise promise; + auto future = promise.getFuture(); + bool is_waiting_for_response=is_waiting_for_response_; + is_waiting_for_response_=false; + auto &&[_, is_ok] = control.response_handler_table_.try_emplace( + id, std::move(timer_), is_waiting_for_response?control_.get():nullptr,std::move(promise)); + if (!is_ok) [[unlikely]] { + close(); + err_code ec=errc::serial_number_conflict; + co_return coro_rpc::unexpected{ec}; + } + else { + if (!control.is_recving_) { + control.is_recving_ = true; +#ifdef YLT_ENABLE_SSL + if (!config_.ssl_cert_path.empty()) { + assert(control.ssl_stream_); + recv(control_,*control.ssl_stream_).start([](auto&&){}); + } + else { +#endif + recv(control_,control.socket_).start([](auto&&){}); +#ifdef YLT_ENABLE_SSL + } +#endif + } + co_return get_deserializer())>(std::move(future),std::weak_ptr{control_}); + } + } + else { + co_return coro_rpc::unexpected{std::move(result)}; + } + } +private: + + + template + async_simple::coro::Lazy + send_impl(Socket &socket, uint32_t& id, Args&&... args) { + auto buffer = prepare_buffer(id, std::forward(args)...); + if (buffer.empty()) { + co_return rpc_error{errc::message_too_large}; + } +#ifdef GENERATE_BENCHMARK_DATA + control_->func_name_ = get_func_name(); + std::ofstream file( + benchmark_file_path + control_->func_name_ + ".in", + std::ofstream::binary | std::ofstream::out); + file << std::string_view{(char *)buffer.data(), buffer.size()}; + file.close(); +#endif + std::pair ret; +#ifdef UNIT_TEST_INJECT + if (g_action == inject_action::client_send_bad_header) { + buffer[0] = (std::byte)(uint8_t(buffer[0]) + 1); + } + if (g_action == inject_action::client_close_socket_after_send_header) { + ret = co_await coro_io::async_write( + socket, + asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); + ELOGV(INFO, "client_id %d close socket", config_.client_id); + close(); + co_return rpc_error{errc::io_error, ret.first.message()}; + } + else if (g_action == + inject_action::client_close_socket_after_send_partial_header) { + ret = co_await coro_io::async_write( + socket, + asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN - 1)); + ELOGV(INFO, "client_id %d close socket", config_.client_id); + close(); + co_return rpc_error{errc::io_error, ret.first.message()}; + } + else if (g_action == + inject_action::client_shutdown_socket_after_send_header) { + ret = co_await coro_io::async_write( + socket, + asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); + ELOGV(INFO, "client_id %d shutdown", config_.client_id); + control_->socket_.shutdown(asio::ip::tcp::socket::shutdown_send); + co_return rpc_error{errc::io_error, ret.first.message()}; + } + else { +#endif + if (req_attachment_.empty()) { + ret = co_await coro_io::async_write( + socket, asio::buffer(buffer.data(), buffer.size())); + } + else { + std::array iov{ + asio::const_buffer{buffer.data(), buffer.size()}, + asio::const_buffer{req_attachment_.data(), + req_attachment_.size()}}; + ret = co_await coro_io::async_write(socket, iov); + req_attachment_ = {}; + } +#ifdef UNIT_TEST_INJECT + } +#endif +#ifdef UNIT_TEST_INJECT + if (g_action == inject_action::force_inject_client_write_data_timeout) { + control_->is_timeout_ = true; + } +#endif +#ifdef UNIT_TEST_INJECT + if (g_action == inject_action::client_close_socket_after_send_payload) { + ELOGV(INFO, "client_id %d client_close_socket_after_send_payload", + config_.client_id); + close(); + co_return rpc_error{errc::io_error, ret.first.message()}; + } +#endif + if (ret.first) { + close(); + if (control_->is_timeout_) { + co_return rpc_error{errc::timed_out}; + } + else { + co_return rpc_error{errc::io_error, ret.first.message()}; + } + } + co_return rpc_error{}; + } + private: - coro_io::period_timer timer_; + std::atomic is_waiting_for_response_=false; + std::atomic request_id_{0}; + std::unique_ptr timer_; std::shared_ptr control_; - std::string read_buf_, resp_attachment_buf_; std::string_view req_attachment_; config config_; constexpr static std::size_t default_read_buf_size_ = 256; #ifdef YLT_ENABLE_SSL asio::ssl::context ssl_ctx_{asio::ssl::context::sslv23}; - std::unique_ptr> ssl_stream_; bool ssl_init_ret_ = true; #endif - std::atomic has_closed_ = false; }; } // namespace coro_rpc diff --git a/include/ylt/coro_rpc/impl/errno.h b/include/ylt/coro_rpc/impl/errno.h index 0d7c1b6bd..76543396e 100644 --- a/include/ylt/coro_rpc/impl/errno.h +++ b/include/ylt/coro_rpc/impl/errno.h @@ -35,6 +35,7 @@ enum class errc : uint16_t { message_too_large, server_has_ran, invalid_rpc_result, + serial_number_conflict, }; inline constexpr std::string_view make_error_message(errc ec) noexcept { switch (ec) { @@ -70,6 +71,8 @@ inline constexpr std::string_view make_error_message(errc ec) noexcept { return "server has ran"; case errc::invalid_rpc_result: return "invalid rpc result"; + case errc::serial_number_conflict: + return "serial number conflict"; default: return "unknown user-defined error"; } @@ -103,8 +106,12 @@ inline bool operator!(errc ec) noexcept { return ec == errc::ok; } struct rpc_error { coro_rpc::err_code code; //!< error code std::string msg; //!< error message + rpc_error(){} + rpc_error(coro_rpc::err_code code, std::string_view msg):code(code),msg(std::string{msg}){} + rpc_error(coro_rpc::err_code code):code(code),msg(std::string{make_error_message(code)}){} uint16_t& val() { return *(uint16_t*)&(code.ec); } const uint16_t& val() const { return *(uint16_t*)&(code.ec); } + constexpr operator bool() const noexcept { return code; } }; STRUCT_PACK_REFL(rpc_error, val(), msg); diff --git a/include/ylt/coro_rpc/impl/expected.hpp b/include/ylt/coro_rpc/impl/expected.hpp index 4a0a7e91e..bddbe2ff8 100644 --- a/include/ylt/coro_rpc/impl/expected.hpp +++ b/include/ylt/coro_rpc/impl/expected.hpp @@ -50,6 +50,6 @@ struct coro_rpc_protocol; template -using rpc_result = expected; +using rpc_result = expected; } // namespace coro_rpc \ No newline at end of file diff --git a/src/coro_rpc/tests/test_coro_rpc_client.cpp b/src/coro_rpc/tests/test_coro_rpc_client.cpp index 71b69eab4..2d927c23e 100644 --- a/src/coro_rpc/tests/test_coro_rpc_client.cpp +++ b/src/coro_rpc/tests/test_coro_rpc_client.cpp @@ -56,6 +56,10 @@ Lazy> create_client( co_return client; } + void show(auto &s) { + return; + } + TEST_CASE("testing client") { { coro_rpc::coro_rpc_client client; @@ -146,6 +150,7 @@ TEST_CASE("testing client") { std::string arg; arg.resize(2048); auto ret = co_await client->template call(arg); + show(ret); CHECK(ret.value() == arg); co_return; }; @@ -205,6 +210,7 @@ TEST_CASE("testing client with inject server") { auto client = co_await create_client(io_context, port); g_action = inject_action::close_socket_after_send_length; auto ret = co_await client->template call(); + show(ret); REQUIRE_MESSAGE(ret.error().code == coro_rpc::errc::io_error, ret.error().msg); }; From 7b5a520d57a15496ff4558e324858ce8d17c7c75 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Tue, 7 May 2024 18:01:58 +0800 Subject: [PATCH 06/22] fix client --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 19 ++- .../examples/base_examples/client_pool.cpp | 146 +++++++++++------- .../base_examples/concurrent_clients.cpp | 114 +++++++++----- .../examples/base_examples/rpc_service.cpp | 4 +- 4 files changed, 182 insertions(+), 101 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index c42954d8a..b94eaeece 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -139,6 +139,7 @@ class coro_rpc_client { std::chrono::milliseconds{5000}; std::string host; std::string port; + bool enable_tcp_no_delay_ = true; #ifdef YLT_ENABLE_SSL std::filesystem::path ssl_cert_path; std::string ssl_domain; @@ -332,7 +333,7 @@ class coro_rpc_client { uint32_t get_client_id() const { return config_.client_id; } void close() { - ELOGV(INFO, "client_id %d close", config_.client_id); + //ELOGV(INFO, "client_id %d close", config_.client_id); close_socket(control_); } @@ -409,8 +410,9 @@ class coro_rpc_client { ELOGV(WARN, "client_id %d connect timeout", config_.client_id); co_return errc::timed_out; } - - control_->socket_.set_option(asio::ip::tcp::no_delay(true), ec); + if (config_.enable_tcp_no_delay_==true) { + control_->socket_.set_option(asio::ip::tcp::no_delay(true), ec); + } #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { @@ -614,11 +616,20 @@ class coro_rpc_client { constexpr bool has_conn_v = requires { typename First::return_type; }; return util::get_args(); } + template + static decltype(auto) add_const(U&& u) { + if constexpr (std::is_const_v>) { + return struct_pack::detail::declval(); + } + else { + return struct_pack::detail::declval(); + } + } template void pack_to_impl(Buffer &buffer, std::size_t offset, Args &&...args) { struct_pack::serialize_to_with_offset( - buffer, offset, std::forward(std::forward(args))...); + buffer, offset, std::forward(args))>((std::forward(args)))...); } template diff --git a/src/coro_rpc/examples/base_examples/client_pool.cpp b/src/coro_rpc/examples/base_examples/client_pool.cpp index 23ba2b628..437144227 100644 --- a/src/coro_rpc/examples/base_examples/client_pool.cpp +++ b/src/coro_rpc/examples/base_examples/client_pool.cpp @@ -17,9 +17,12 @@ #include #include #include +#include #include #include #include +#include +#include #include #include #include @@ -27,19 +30,24 @@ #include "async_simple/coro/Collect.h" #include "async_simple/coro/Lazy.h" +#include "async_simple/coro/Mutex.h" #include "async_simple/coro/SyncAwait.h" #include "ylt/coro_io/io_context_pool.hpp" +#include "ylt/coro_rpc/impl/coro_rpc_client.hpp" +#include "ylt/coro_rpc/impl/errno.h" +#include "ylt/coro_rpc/impl/expected.hpp" #include "ylt/easylog.hpp" std::string echo(std::string_view sv); +constexpr auto thread_cnt = 96*2; +constexpr auto request_cnt= 200000; using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; -std::atomic qps = 0; - +auto finish_executor = coro_io::get_global_block_executor(); std::atomic working_echo = 0; -std::atomic busy_echo = 0; +std::atomic stop=false; struct guard { guard(std::atomic &ref) : ref(ref) { ++ref; } ~guard() { --ref; } @@ -50,57 +58,92 @@ struct guard { * \brief demo for run concurrency clients */ -Lazy> call_echo( - coro_io::client_pool &client_pool, int cnt) { - std::vector result; - result.reserve(cnt); +std::vector> clients; + +std::atomic& get_qps(int id) { + static std::atomic ar[thread_cnt*8]; + return ar[id*8]; +} + +int& get_cnt(int id) { + static int ar[thread_cnt*16]; + return ar[id*16]; +} +int& get_flag(int id) { + static int ar[thread_cnt*16]; + return ar[id*16]; +} +std::vector& get_result(int id) { + static std::vector ar[thread_cnt*3]; + return ar[id*3]; +} +int cnt_max=10; +Lazy send(int id) { + auto &cli=*clients[id]; + auto& qps=get_qps(id); + auto &cnt=get_cnt(id); + auto &result=get_result(id); ++working_echo; - for (int i = 0; i < cnt; ++i) { + for (;result.size() Lazy { - guard g{busy_echo}; - if (client.has_closed()) { - co_return; - } - auto res = co_await client.call("Hello world!"); - if (!res.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; - client.close(); - co_return; - } - if (res.value() != "Hello world!"sv) { - ELOG_ERROR << "err echo resp: \n" << res.value(); - co_return; - } - ++qps; - co_return; - }); - if (!res) { - ELOG_ERROR << "client pool err: connect failed.\n"; + auto res_ = co_await cli.send_request("Hello world!"); + if (!res_.has_value()) { + ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; + continue; } - else { - result.push_back(std::chrono::duration_cast( - std::chrono::steady_clock::now() - tp)); + res_.value().start([id,&qps,&cnt,&result,old_tp=tp](auto&&res) { + auto&res1=res.value(); + if (!res1.has_value()) { + ELOG_ERROR << "coro_rpc err: \n" << res1.error().msg; + } + else { + ++qps; + result.push_back(std::chrono::duration_cast( + std::chrono::steady_clock::now()-old_tp)); + auto tmp=cnt--; + if (tmp==cnt_max) { + get_flag(id)=true; + } + else if (tmp==cnt_max/2 && get_flag(id)) { + get_flag(id)=false; + send(id).start([](auto&& res){ + }); + } + } + }); + auto cnt_tmp=++cnt; + if (cnt_tmp==cnt_max) { + break; } } - co_return std::move(result); + --working_echo; + co_return; } -Lazy qps_watcher(coro_io::client_pool &clients) { +Lazy qps_watcher() { using namespace std::chrono_literals; - while (working_echo > 0) { + do { co_await coro_io::sleep_for(1s); - uint64_t cnt = qps.exchange(0); + uint64_t cnt = 0; + for (int i=0;i0); } std::vector result; void latency_watcher() { + result.reserve(request_cnt*thread_cnt); + for (int i=0;i::create( - "localhost:8801", - coro_io::client_pool::pool_config{ - .max_connection = thread_cnt * 20, - .client_config = {.timeout_duration = std::chrono::seconds{50}}}); - - auto finish_executor = coro_io::get_global_block_executor(); - for (int i = 0, lim = thread_cnt * 20; i < lim; ++i) { - call_echo(*client_pool, 10000).start([finish_executor](auto &&res) { - finish_executor->schedule([res = std::move(res.value())] { - result.insert(result.end(), res.begin(), res.end()); - --working_echo; - }); + for (int i=0;i(coro_io::get_global_executor()->get_asio_executor())); + syncAwait(clients.back()->connect("localhost:8801")); + get_result(i).reserve(request_cnt); + } + for (int i = 0, lim = thread_cnt; i < lim; ++i) { + send(i).via(&clients[i]->get_executor()).start([](auto &&res) { }); } - syncAwait(qps_watcher(*client_pool)); + syncAwait(qps_watcher()); latency_watcher(); std::cout << "Done!" << std::endl; return 0; diff --git a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp index c8e8e7a20..65d7fa4ab 100644 --- a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp +++ b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp @@ -13,80 +13,114 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include -#include -#include #include #include #include +#include #include -#include +#include +#include +#include +#include #include #include #include #include +#include "async_simple/coro/Collect.h" +#include "async_simple/coro/Lazy.h" +#include "async_simple/coro/Mutex.h" +#include "async_simple/coro/SyncAwait.h" +#include "ylt/coro_io/io_context_pool.hpp" #include "ylt/coro_rpc/impl/coro_rpc_client.hpp" +#include "ylt/coro_rpc/impl/errno.h" +#include "ylt/coro_rpc/impl/expected.hpp" +#include "ylt/easylog.hpp" std::string echo(std::string_view sv); + +constexpr auto thread_cnt = 96*20; using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; -using namespace std::chrono_literals; -std::atomic qps = 0; +auto finish_executor = coro_io::get_global_block_executor(); std::atomic working_echo = 0; -/*! - * \example helloworld/concurrency_clients.main.cpp - * \brief demo for run concurrency clients - */ -Lazy call_echo(int cnt) { +std::vector> clients; + +std::atomic& get_qps(int id) { + static std::atomic ar[thread_cnt*8]; + return ar[id*8]; +} + +Lazy> send(int id,int cnt) { + std::vector result; + auto &cli=*clients[id]; + auto& qps=get_qps(id); + result.reserve(cnt); ++working_echo; - coro_rpc_client client; - auto ec = co_await client.connect("localhost:8801"); - for (int i = 0; i < 3 && !ec; ++i) { - co_await coro_io::sleep_for(rand() % 10000 * 1ms); - ec = co_await client.reconnect("localhost:8801"); - } - if (!ec) { - for (int i = 0; i < cnt; ++i) { - auto res = co_await client.call("Hello world!"); - if (!res.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; - co_return; - } - if (res.value() != "Hello world!"sv) { - ELOG_ERROR << "err echo resp: \n" << res.value(); - co_return; - } - ++qps; + auto tp = std::chrono::steady_clock::now(); + for (int i=0;i("Hello world!"); + if (!res_.has_value()) { + ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; + continue; } - } - else { - std::cout << "connect failed \n"; + ++qps; + auto old_tp=tp; + tp = std::chrono::steady_clock::now(); + result.push_back(std::chrono::duration_cast(tp-old_tp)); } --working_echo; + co_return std::move(result); } Lazy qps_watcher() { using namespace std::chrono_literals; - while (working_echo > 0) { + do { co_await coro_io::sleep_for(1s); - uint64_t cnt = qps.exchange(0); - std::cout << "QPS:" << cnt << " working echo:" << working_echo << std::endl; + uint64_t cnt = 0; + for (int i=0;i0); +} +std::vector result; +void latency_watcher() { + std::sort(result.begin(), result.end()); + auto arr = {0.1, 0.3, 0.5, 0.7, 0.9, 0.95, 0.99, 0.999, 0.9999, 0.99999, 1.0}; + for (auto e : arr) { + std::cout + << (e * 100) << "% request finished in:" + << result[std::max(0, result.size() * e - 1)].count() / + 1000.0 + << "ms" << std::endl; } } - int main() { - auto thread_cnt = std::thread::hardware_concurrency(); - for (int i = 0, lim = thread_cnt * 20; i < lim; ++i) { - call_echo(100000).start([](auto &&) { + for (int i=0;i(coro_io::get_global_executor()->get_asio_executor())); + syncAwait(clients.back()->connect("localhost:8801")); + } + for (int i = 0, lim = thread_cnt; i < lim; ++i) { + send(i,20000).via(&clients[i]->get_executor()).start([](auto &&res) { + finish_executor->schedule([res = std::move(res.value())] { + result.insert(result.end(), res.begin(), res.end()); + }); }); } syncAwait(qps_watcher()); + latency_watcher(); std::cout << "Done!" << std::endl; return 0; } \ No newline at end of file diff --git a/src/coro_rpc/examples/base_examples/rpc_service.cpp b/src/coro_rpc/examples/base_examples/rpc_service.cpp index 27f04db59..e661e6728 100644 --- a/src/coro_rpc/examples/base_examples/rpc_service.cpp +++ b/src/coro_rpc/examples/base_examples/rpc_service.cpp @@ -102,8 +102,8 @@ Lazy nested_echo(std::string_view sv) { coro_io::g_clients_pool().at("127.0.0.1:8802"); assert(client != nullptr); ELOGV(INFO, "connect another server"); - auto ret = co_await client->send_request([sv](coro_rpc_client &client) { - return client.call(sv); + auto ret = co_await client->send_request([sv](coro_rpc_client &client) ->Lazy> { + co_return co_await client.call(sv); }); co_return ret.value().value(); } From 88a650d59d598ef229005551cfa4b6ab1ce759ab Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 8 May 2024 11:24:08 +0800 Subject: [PATCH 07/22] fix ssl --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index b94eaeece..40e1486f4 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -738,7 +738,7 @@ class coro_rpc_client { #ifdef YLT_ENABLE_SSL if (!ssl_init_ret_) { - co_return rpc_error{errc::not_connected}}; + co_return rpc_error{errc::not_connected}; } #endif From ef963f09058ee8f5d8d860b72c63115a5d070a3c Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Fri, 26 Apr 2024 17:40:52 +0800 Subject: [PATCH 08/22] coro_rpc_client allow async_all --- include/ylt/coro_io/io_context_pool.hpp | 2 +- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 654 ++++++++++++------ include/ylt/coro_rpc/impl/errno.h | 7 + include/ylt/coro_rpc/impl/expected.hpp | 2 +- src/coro_rpc/tests/test_coro_rpc_client.cpp | 6 + 5 files changed, 440 insertions(+), 231 deletions(-) diff --git a/include/ylt/coro_io/io_context_pool.hpp b/include/ylt/coro_io/io_context_pool.hpp index 07eb8e522..b45c63ff9 100644 --- a/include/ylt/coro_io/io_context_pool.hpp +++ b/include/ylt/coro_io/io_context_pool.hpp @@ -75,7 +75,7 @@ class ExecutorWrapper : public async_simple::Executor { context_t &context() { return executor_.context(); } - auto get_asio_executor() { return executor_; } + auto get_asio_executor() const { return executor_; } operator ExecutorImpl() { return executor_; } diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index 55d03c019..c42954d8a 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -22,10 +22,12 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include @@ -33,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -40,6 +43,8 @@ #include "asio/buffer.hpp" #include "asio/dispatch.hpp" #include "asio/registered_buffer.hpp" +#include "async_simple/Executor.h" +#include "async_simple/Promise.h" #include "common_service.hpp" #include "context.hpp" #include "expected.hpp" @@ -80,6 +85,23 @@ template <> struct rpc_return_type { using type = std::monostate; }; + +struct rpc_resp_buffer { + std::string read_buf_; + std::string resp_attachment_buf_; +}; + +template +struct async_rpc_result { + T result_; + rpc_resp_buffer buffer_; +}; + +template <> +struct async_rpc_result { + rpc_resp_buffer buffer_; +}; + template using rpc_return_type_t = typename rpc_return_type::type; /*! @@ -130,7 +152,7 @@ class coro_rpc_client { coro_rpc_client(asio::io_context::executor_type executor, uint32_t client_id = 0) : control_(std::make_shared(executor, false)), - timer_(executor) { + timer_(std::make_unique(executor)) { config_.client_id = client_id; } @@ -143,7 +165,7 @@ class coro_rpc_client { uint32_t client_id = 0) : control_( std::make_shared(executor.get_asio_executor(), false)), - timer_(executor.get_asio_executor()) { + timer_(std::make_unique(executor.get_asio_executor())) { config_.client_id = client_id; } @@ -166,7 +188,7 @@ class coro_rpc_client { * * @return true if client closed, otherwise false. */ - [[nodiscard]] bool has_closed() { return has_closed_; } + [[nodiscard]] bool has_closed() { return control_->has_closed_; } /*! * Reconnect server @@ -261,8 +283,8 @@ class coro_rpc_client { template async_simple::coro::Lazy< rpc_result()), coro_rpc_protocol>> - call(Args... args) { - return call_for(std::chrono::seconds(5), std::move(args)...); + call(Args&&... args) { + return call_for(std::chrono::seconds(5), std::forward(args)...); } /*! @@ -279,60 +301,27 @@ class coro_rpc_client { template async_simple::coro::Lazy< rpc_result()), coro_rpc_protocol>> - call_for(auto duration, Args... args) { - using R = decltype(get_return_type()); - - if (has_closed_) - AS_UNLIKELY { - ELOGV(ERROR, "client has been closed, please re-connect"); - auto ret = rpc_result{ - unexpect_t{}, - rpc_error{errc::io_error, - "client has been closed, please re-connect"}}; - co_return ret; + call_for(auto duration, Args&&... args) { + is_waiting_for_response_=true; + using return_type=decltype(get_return_type()); + auto result = co_await send_request_for(duration, std::forward(args)...); + if (result) { + auto async_result = co_await result.value(); + if (async_result) { + if constexpr (std::is_same_v) { + co_return expected{}; + } + else { + co_return expected{std::move(async_result.value().result_)}; + } + } + else { + co_return expected{unexpect_t{},std::move(async_result.error())}; } - - rpc_result ret; -#ifdef YLT_ENABLE_SSL - if (!ssl_init_ret_) { - ret = rpc_result{ - unexpect_t{}, - rpc_error{errc::not_connected, - std::string{make_error_message(errc::not_connected)}}}; - co_return ret; - } -#endif - - static_check(); - - timeout(duration, "rpc call timer canceled").start([](auto &&) { - }); - -#ifdef YLT_ENABLE_SSL - if (!config_.ssl_cert_path.empty()) { - assert(ssl_stream_); - ret = co_await call_impl(*ssl_stream_, std::move(args)...); } else { -#endif - ret = co_await call_impl(control_->socket_, std::move(args)...); -#ifdef YLT_ENABLE_SSL - } -#endif - - std::error_code err_code; - timer_.cancel(err_code); - - if (control_->is_timeout_) { - ret = rpc_result{ - unexpect_t{}, rpc_error{errc::timed_out, "rpc call timed out"}}; + co_return expected{unexpect_t{},std::move(result.error())}; } - -#ifdef UNIT_TEST_INJECT - ELOGV(INFO, "client_id %d call %s %s", config_.client_id, - get_func_name().data(), ret ? "ok" : "failed"); -#endif - co_return ret; } /*! @@ -343,10 +332,6 @@ class coro_rpc_client { uint32_t get_client_id() const { return config_.client_id; } void close() { - if (has_closed_) { - return; - } - has_closed_ = true; ELOGV(INFO, "client_id %d close", config_.client_id); close_socket(control_); } @@ -360,10 +345,10 @@ class coro_rpc_client { return true; } - std::string_view get_resp_attachment() const { return resp_attachment_buf_; } + std::string_view get_resp_attachment() const { return control_->resp_buffer_.resp_attachment_buf_; } std::string release_resp_attachment() { - return std::move(resp_attachment_buf_); + return std::move(control_->resp_buffer_.resp_attachment_buf_); } template @@ -381,7 +366,7 @@ class coro_rpc_client { control_->socket_ = asio::ip::tcp::socket(control_->executor_.get_asio_executor()); control_->is_timeout_ = false; - has_closed_ = false; + control_->has_closed_ = false; } static bool is_ok(coro_rpc::err_code ec) noexcept { return !ec; } [[nodiscard]] async_simple::coro::Lazy connect( @@ -392,7 +377,7 @@ class coro_rpc_client { co_return errc::not_connected; } #endif - if (!is_reconnect.value && has_closed_) + if (!is_reconnect.value && control_->has_closed_) AS_UNLIKELY { ELOGV(ERROR, "a closed client is not allowed connect again, please use " @@ -400,18 +385,18 @@ class coro_rpc_client { "client"); co_return errc::io_error; } - has_closed_ = false; + control_->has_closed_ = false; ELOGV(INFO, "client_id %d begin to connect %s", config_.client_id, config_.port.data()); - timeout(config_.timeout_duration, "connect timer canceled") + timeout(*this->timer_,config_.timeout_duration, "connect timer canceled") .start([](auto &&) { }); std::error_code ec = co_await coro_io::async_connect( &control_->executor_, control_->socket_, config_.host, config_.port); std::error_code err_code; - timer_.cancel(err_code); + timer_->cancel(err_code); if (ec) { if (control_->is_timeout_) { @@ -429,9 +414,9 @@ class coro_rpc_client { #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { - assert(ssl_stream_); + assert(control_->ssl_stream_); auto shake_ec = co_await coro_io::async_handshake( - ssl_stream_, asio::ssl::stream_base::client); + control_->ssl_stream_, asio::ssl::stream_base::client); if (shake_ec) { ELOGV(WARN, "client_id %d handshake failed: %s", config_.client_id, shake_ec.message().data()); @@ -461,7 +446,7 @@ class coro_rpc_client { ssl_ctx_.set_verify_mode(asio::ssl::verify_peer); ssl_ctx_.set_verify_callback( asio::ssl::host_name_verification(config_.ssl_domain)); - ssl_stream_ = + control_->ssl_stream_ = std::make_unique>( control_->socket_, ssl_ctx_); ssl_init_ret_ = true; @@ -471,10 +456,12 @@ class coro_rpc_client { return ssl_init_ret_; } #endif - async_simple::coro::Lazy timeout(auto duration, std::string err_msg) { - timer_.expires_after(duration); + + async_simple::coro::Lazy timeout(coro_io::period_timer &timer, + auto duration, std::string err_msg) { + timer.expires_after(duration); std::weak_ptr socket_watcher = control_; - bool is_timeout = co_await timer_.async_await(); + bool is_timeout = co_await timer.async_await(); if (!is_timeout) { co_return false; } @@ -530,148 +517,6 @@ class coro_rpc_client { } } - template - async_simple::coro::Lazy< - rpc_result()), coro_rpc_protocol>> - call_impl(Socket &socket, Args... args) { - using R = decltype(get_return_type()); - - auto buffer = prepare_buffer(std::move(args)...); - - rpc_result r{}; - if (buffer.empty()) { - r = rpc_result{ - unexpect_t{}, rpc_error{errc::message_too_large, - "rpc body serialize size too big"}}; - co_return r; - } -#ifdef GENERATE_BENCHMARK_DATA - std::ofstream file( - benchmark_file_path + std::string{get_func_name()} + ".in", - std::ofstream::binary | std::ofstream::out); - file << std::string_view{(char *)buffer.data(), buffer.size()}; - file.close(); -#endif - std::pair ret; -#ifdef UNIT_TEST_INJECT - if (g_action == inject_action::client_send_bad_header) { - buffer[0] = (std::byte)(uint8_t(buffer[0]) + 1); - } - if (g_action == inject_action::client_close_socket_after_send_header) { - ret = co_await coro_io::async_write( - socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); - ELOGV(INFO, "client_id %d close socket", config_.client_id); - close(); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - co_return r; - } - else if (g_action == - inject_action::client_close_socket_after_send_partial_header) { - ret = co_await coro_io::async_write( - socket, - asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN - 1)); - ELOGV(INFO, "client_id %d close socket", config_.client_id); - close(); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - co_return r; - } - else if (g_action == - inject_action::client_shutdown_socket_after_send_header) { - ret = co_await coro_io::async_write( - socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); - ELOGV(INFO, "client_id %d shutdown", config_.client_id); - control_->socket_.shutdown(asio::ip::tcp::socket::shutdown_send); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - co_return r; - } - else { -#endif - if (req_attachment_.empty()) { - ret = co_await coro_io::async_write( - socket, asio::buffer(buffer.data(), buffer.size())); - } - else { - std::array iov{ - asio::const_buffer{buffer.data(), buffer.size()}, - asio::const_buffer{req_attachment_.data(), req_attachment_.size()}}; - ret = co_await coro_io::async_write(socket, iov); - req_attachment_ = {}; - } -#ifdef UNIT_TEST_INJECT - } -#endif - if (!ret.first) { -#ifdef UNIT_TEST_INJECT - if (g_action == inject_action::client_close_socket_after_send_payload) { - ELOGV(INFO, "client_id %d client_close_socket_after_send_payload", - config_.client_id); - r = rpc_result{ - unexpect_t{}, rpc_error{errc::io_error, ret.first.message()}}; - close(); - co_return r; - } -#endif - coro_rpc_protocol::resp_header header; - ret = co_await coro_io::async_read( - socket, - asio::buffer((char *)&header, coro_rpc_protocol::RESP_HEAD_LEN)); - if (!ret.first) { - uint32_t body_len = header.length; - struct_pack::detail::resize(read_buf_, body_len); - if (header.attach_length == 0) { - ret = co_await coro_io::async_read( - socket, asio::buffer(read_buf_.data(), body_len)); - resp_attachment_buf_.clear(); - } - else { - struct_pack::detail::resize(resp_attachment_buf_, - header.attach_length); - std::array iov{ - asio::mutable_buffer{read_buf_.data(), body_len}, - asio::mutable_buffer{resp_attachment_buf_.data(), - resp_attachment_buf_.size()}}; - ret = co_await coro_io::async_read(socket, iov); - } - if (!ret.first) { -#ifdef GENERATE_BENCHMARK_DATA - std::ofstream file( - benchmark_file_path + std::string{get_func_name()} + ".out", - std::ofstream::binary | std::ofstream::out); - file << std::string_view{(char *)&header, - coro_rpc_protocol::RESP_HEAD_LEN}; - file << read_buf_; - file << resp_attachment_buf_; - file.close(); -#endif - bool ec = false; - r = handle_response_buffer(read_buf_, header.err_code, ec); - if (ec) { - close(); - } - co_return r; - } - } - } -#ifdef UNIT_TEST_INJECT - if (g_action == inject_action::force_inject_client_write_data_timeout) { - control_->is_timeout_ = true; - } -#endif - if (control_->is_timeout_) { - r = rpc_result{ - unexpect_t{}, rpc_error{.code = errc::timed_out, .msg = {}}}; - } - else { - r = rpc_result{ - unexpect_t{}, - rpc_error{.code = errc::io_error, .msg = ret.first.message()}}; - } - close(); - co_return r; - } /* * buffer layout * ┌────────────────┬────────────────┐ @@ -681,7 +526,7 @@ class coro_rpc_client { * └────────────────┴────────────────┘ */ template - std::vector prepare_buffer(Args &&...args) { + std::vector prepare_buffer(uint32_t& id, Args &&...args) { std::vector buffer; std::size_t offset = coro_rpc_protocol::REQ_HEAD_LEN; if constexpr (sizeof...(Args) > 0) { @@ -697,8 +542,11 @@ class coro_rpc_client { header.magic = coro_rpc_protocol::magic_number; header.function_id = func_id(); header.attach_length = req_attachment_.size(); + id = request_id_++; + ELOG_TRACE<<"send request ID:"< - rpc_result handle_response_buffer(std::string &buffer, - uint8_t rpc_errc, - bool &error_happen) { + static rpc_result handle_response_buffer(std::string_view buffer, + uint8_t rpc_errc,bool& should_close) { rpc_return_type_t ret; struct_pack::err_code ec; rpc_error err; @@ -743,21 +590,20 @@ class coro_rpc_client { err.val() = rpc_errc; ec = struct_pack::deserialize_to(err.msg, buffer); if SP_LIKELY (!ec) { - ELOGV(WARNING, "deserilaize rpc result failed"); - error_happen = true; + should_close = true; return rpc_result{unexpect_t{}, std::move(err)}; } } else { ec = struct_pack::deserialize_to(err, buffer); if SP_LIKELY (!ec) { - ELOGV(WARNING, "deserilaize rpc result failed"); return rpc_result{unexpect_t{}, std::move(err)}; } } } - error_happen = true; + should_close = true; // deserialize failed. + ELOGV(WARNING, "deserilaize rpc result failed"); err = {errc::invalid_rpc_result, "failed to deserialize rpc return value"}; return rpc_result{unexpect_t{}, std::move(err)}; } @@ -790,16 +636,58 @@ class coro_rpc_client { offset, std::forward(args)...); } + struct async_rpc_raw_result_value_type { + std::variant buffer_; + uint8_t errc_; + }; + + using async_rpc_raw_result=std::variant; + + struct control_t; + + struct handler_t { + std::unique_ptr timer_; + control_t* control_; + async_simple::Promise promise_; + handler_t(std::unique_ptr&& timer,control_t* control,async_simple::Promise &&promise):timer_(std::move(timer)), control_(control),promise_(std::move(promise)) {} + void operator()(rpc_resp_buffer&& buffer,uint8_t rpc_errc) { + timer_->cancel(); + if (control_) /*is waiting for response*/ { + promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{std::string_view{control_->resp_buffer_.read_buf_},rpc_errc}}); + } + else { + promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{std::move(buffer),rpc_errc}}); + } + } + void local_error (std::error_code& ec) { + timer_->cancel(); + promise_.setValue(async_rpc_raw_result{ec}); + } + }; struct control_t { - asio::ip::tcp::socket socket_; +#ifdef YLT_ENABLE_SSL + std::unique_ptr> ssl_stream_; +#endif +#ifdef GENERATE_BENCHMARK_DATA + std::string func_name_; +#endif bool is_timeout_; + std::atomic has_closed_ = false; coro_io::ExecutorWrapper<> executor_; + std::unordered_map response_handler_table_; + rpc_resp_buffer resp_buffer_; + asio::ip::tcp::socket socket_; + std::atomic is_recving_=false; control_t(asio::io_context::executor_type executor, bool is_timeout) - : socket_(executor), is_timeout_(is_timeout), executor_(executor) {} + : socket_(executor), is_timeout_(is_timeout), has_closed_(false), executor_(executor) {} }; static void close_socket( std::shared_ptr control) { + if (control->has_closed_) { + return; + } + control->has_closed_ = true; control->executor_.schedule([control = std::move(control)]() { asio::error_code ignored_ec; control->socket_.shutdown(asio::ip::tcp::socket::shutdown_both, @@ -823,18 +711,326 @@ class coro_rpc_client { } #endif + + + template + async_simple::coro::Lazy + send_request_for_impl(auto duration, uint32_t& id, Args &&...args) { + using R = decltype(get_return_type()); + + if (control_->has_closed_) + AS_UNLIKELY { + ELOGV(ERROR, "client has been closed, please re-connect"); + co_return rpc_error{errc::io_error, + "client has been closed, please re-connect"}; + } + +#ifdef YLT_ENABLE_SSL + if (!ssl_init_ret_) { + co_return rpc_error{errc::not_connected}}; + } +#endif + + static_check(); + + if (duration.count() > 0) { + if (timer_ == nullptr) + timer_ = std::make_unique(control_->executor_.get_asio_executor()); + timeout(*timer_, duration, "rpc call timer canceled").start([](auto &&) { + }); + } + +#ifdef YLT_ENABLE_SSL + if (!config_.ssl_cert_path.empty()) { + assert(control_->ssl_stream_); + co_return co_await send_impl(*control_->ssl_stream_, id, std::forward(args)...); + } + else { +#endif + co_return co_await send_impl(control_->socket_, id, std::forward(args)...); +#ifdef YLT_ENABLE_SSL + } +#endif + } + + + static void send_err_response(control_t* controller, std::error_code& errc) { + rpc_error ec; + for (auto &e:controller->response_handler_table_) { + e.second.local_error(errc); + } + controller->response_handler_table_.clear(); + } + template + static async_simple::coro::Lazy recv(std::shared_ptr controller, Socket& socket) { + std::pair ret; + do { + coro_rpc_protocol::resp_header header; + ret = co_await coro_io::async_read( + socket, + asio::buffer((char *)&header, coro_rpc_protocol::RESP_HEAD_LEN)); + if (ret.first) { + ELOG_ERROR<<"read rpc head failed, error msg:"<func_name_ + ".out", + std::ofstream::binary | std::ofstream::out); + file << std::string_view{(char *)&header, + coro_rpc_protocol::RESP_HEAD_LEN}; + file << controller->resp_buffer_.read_buf_; + file << controller->resp_buffer_.resp_attachment_buf_; + file.close(); +#endif + if (auto iter=controller->response_handler_table_.find(header.seq_num);iter!=controller->response_handler_table_.end()) { + ELOG_TRACE<<"find request ID:"<second(std::move(controller->resp_buffer_), header.err_code); + controller->response_handler_table_.erase(iter); + } + else { + ELOG_ERROR<<"unexists request ID:"<response_handler_table_.size() == 0) { + controller->is_recving_= false; + co_return; + } + } while (true); + controller->is_recving_ = false; + close_socket(controller); + send_err_response(controller.get(),ret.first); + co_return; + } + + template + static async_simple::coro::Lazy,rpc_error>> get_deserializer(async_simple::Future future,std::weak_ptr watcher) { + auto executor = co_await async_simple::CurrentExecutor(); + auto executorFuture = std::move(future).via(executor); + auto ret_ = co_await std::move(executorFuture); + + if (ret_.index() ==1) [[unlikely]] { // local error + auto& ret=std::get<1>(ret_); + if (ret.value()==static_cast(std::errc::operation_canceled) || ret.value()==static_cast(std::errc::timed_out)) { + co_return coro_rpc::unexpected{rpc_error{errc::timed_out,ret.message()}}; + } + else { + co_return coro_rpc::unexpected{rpc_error{errc::io_error,ret.message()}}; + } + } + + bool should_close=false; + std::string_view buffer_view; + auto &ret=std::get<0>(ret_); + if (ret.buffer_.index()==0) { + buffer_view = std::get<0>(ret.buffer_).read_buf_; + } + else { + buffer_view = std::get<1>(ret.buffer_); + } + auto result = handle_response_buffer(buffer_view,ret.errc_,should_close); + if (should_close) { + if (auto w=watcher.lock();w) { + close_socket(std::move(w)); + } + } + if (result) { + if constexpr (std::is_same_v) { + if (ret.buffer_.index()==0) { + co_return async_rpc_result{std::move(std::get<0>(ret.buffer_))}; + } + else { + co_return async_rpc_result{}; + } + } + else { + if (ret.buffer_.index()==0) { + co_return async_rpc_result{result.value(),std::move(std::get<0>(ret.buffer_))}; + } + else { + co_return async_rpc_result{result.value()}; + } + } + } else { + co_return coro_rpc::unexpected{result.error()}; + } + } +public: + + template + async_simple::coro::Lazy())>, rpc_error>>, + rpc_error>> + send_request(Args &&...args) { + return send_request_for(std::chrono::seconds{5}, std::forward(args)...); + } + + template + async_simple::coro::Lazy())>, rpc_error>>, + rpc_error>> + send_request_for(auto duration, Args &&...args) { + uint32_t id; + auto result = co_await send_request_for_impl(duration, id, std::forward(args)...); + auto &control = *control_; + if (!result) { + async_simple::Promise promise; + auto future = promise.getFuture(); + bool is_waiting_for_response=is_waiting_for_response_; + is_waiting_for_response_=false; + auto &&[_, is_ok] = control.response_handler_table_.try_emplace( + id, std::move(timer_), is_waiting_for_response?control_.get():nullptr,std::move(promise)); + if (!is_ok) [[unlikely]] { + close(); + err_code ec=errc::serial_number_conflict; + co_return coro_rpc::unexpected{ec}; + } + else { + if (!control.is_recving_) { + control.is_recving_ = true; +#ifdef YLT_ENABLE_SSL + if (!config_.ssl_cert_path.empty()) { + assert(control.ssl_stream_); + recv(control_,*control.ssl_stream_).start([](auto&&){}); + } + else { +#endif + recv(control_,control.socket_).start([](auto&&){}); +#ifdef YLT_ENABLE_SSL + } +#endif + } + co_return get_deserializer())>(std::move(future),std::weak_ptr{control_}); + } + } + else { + co_return coro_rpc::unexpected{std::move(result)}; + } + } +private: + + + template + async_simple::coro::Lazy + send_impl(Socket &socket, uint32_t& id, Args&&... args) { + auto buffer = prepare_buffer(id, std::forward(args)...); + if (buffer.empty()) { + co_return rpc_error{errc::message_too_large}; + } +#ifdef GENERATE_BENCHMARK_DATA + control_->func_name_ = get_func_name(); + std::ofstream file( + benchmark_file_path + control_->func_name_ + ".in", + std::ofstream::binary | std::ofstream::out); + file << std::string_view{(char *)buffer.data(), buffer.size()}; + file.close(); +#endif + std::pair ret; +#ifdef UNIT_TEST_INJECT + if (g_action == inject_action::client_send_bad_header) { + buffer[0] = (std::byte)(uint8_t(buffer[0]) + 1); + } + if (g_action == inject_action::client_close_socket_after_send_header) { + ret = co_await coro_io::async_write( + socket, + asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); + ELOGV(INFO, "client_id %d close socket", config_.client_id); + close(); + co_return rpc_error{errc::io_error, ret.first.message()}; + } + else if (g_action == + inject_action::client_close_socket_after_send_partial_header) { + ret = co_await coro_io::async_write( + socket, + asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN - 1)); + ELOGV(INFO, "client_id %d close socket", config_.client_id); + close(); + co_return rpc_error{errc::io_error, ret.first.message()}; + } + else if (g_action == + inject_action::client_shutdown_socket_after_send_header) { + ret = co_await coro_io::async_write( + socket, + asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); + ELOGV(INFO, "client_id %d shutdown", config_.client_id); + control_->socket_.shutdown(asio::ip::tcp::socket::shutdown_send); + co_return rpc_error{errc::io_error, ret.first.message()}; + } + else { +#endif + if (req_attachment_.empty()) { + ret = co_await coro_io::async_write( + socket, asio::buffer(buffer.data(), buffer.size())); + } + else { + std::array iov{ + asio::const_buffer{buffer.data(), buffer.size()}, + asio::const_buffer{req_attachment_.data(), + req_attachment_.size()}}; + ret = co_await coro_io::async_write(socket, iov); + req_attachment_ = {}; + } +#ifdef UNIT_TEST_INJECT + } +#endif +#ifdef UNIT_TEST_INJECT + if (g_action == inject_action::force_inject_client_write_data_timeout) { + control_->is_timeout_ = true; + } +#endif +#ifdef UNIT_TEST_INJECT + if (g_action == inject_action::client_close_socket_after_send_payload) { + ELOGV(INFO, "client_id %d client_close_socket_after_send_payload", + config_.client_id); + close(); + co_return rpc_error{errc::io_error, ret.first.message()}; + } +#endif + if (ret.first) { + close(); + if (control_->is_timeout_) { + co_return rpc_error{errc::timed_out}; + } + else { + co_return rpc_error{errc::io_error, ret.first.message()}; + } + } + co_return rpc_error{}; + } + private: - coro_io::period_timer timer_; + std::atomic is_waiting_for_response_=false; + std::atomic request_id_{0}; + std::unique_ptr timer_; std::shared_ptr control_; - std::string read_buf_, resp_attachment_buf_; std::string_view req_attachment_; config config_; constexpr static std::size_t default_read_buf_size_ = 256; #ifdef YLT_ENABLE_SSL asio::ssl::context ssl_ctx_{asio::ssl::context::sslv23}; - std::unique_ptr> ssl_stream_; bool ssl_init_ret_ = true; #endif - std::atomic has_closed_ = false; }; } // namespace coro_rpc diff --git a/include/ylt/coro_rpc/impl/errno.h b/include/ylt/coro_rpc/impl/errno.h index 0d7c1b6bd..76543396e 100644 --- a/include/ylt/coro_rpc/impl/errno.h +++ b/include/ylt/coro_rpc/impl/errno.h @@ -35,6 +35,7 @@ enum class errc : uint16_t { message_too_large, server_has_ran, invalid_rpc_result, + serial_number_conflict, }; inline constexpr std::string_view make_error_message(errc ec) noexcept { switch (ec) { @@ -70,6 +71,8 @@ inline constexpr std::string_view make_error_message(errc ec) noexcept { return "server has ran"; case errc::invalid_rpc_result: return "invalid rpc result"; + case errc::serial_number_conflict: + return "serial number conflict"; default: return "unknown user-defined error"; } @@ -103,8 +106,12 @@ inline bool operator!(errc ec) noexcept { return ec == errc::ok; } struct rpc_error { coro_rpc::err_code code; //!< error code std::string msg; //!< error message + rpc_error(){} + rpc_error(coro_rpc::err_code code, std::string_view msg):code(code),msg(std::string{msg}){} + rpc_error(coro_rpc::err_code code):code(code),msg(std::string{make_error_message(code)}){} uint16_t& val() { return *(uint16_t*)&(code.ec); } const uint16_t& val() const { return *(uint16_t*)&(code.ec); } + constexpr operator bool() const noexcept { return code; } }; STRUCT_PACK_REFL(rpc_error, val(), msg); diff --git a/include/ylt/coro_rpc/impl/expected.hpp b/include/ylt/coro_rpc/impl/expected.hpp index 4a0a7e91e..bddbe2ff8 100644 --- a/include/ylt/coro_rpc/impl/expected.hpp +++ b/include/ylt/coro_rpc/impl/expected.hpp @@ -50,6 +50,6 @@ struct coro_rpc_protocol; template -using rpc_result = expected; +using rpc_result = expected; } // namespace coro_rpc \ No newline at end of file diff --git a/src/coro_rpc/tests/test_coro_rpc_client.cpp b/src/coro_rpc/tests/test_coro_rpc_client.cpp index 71b69eab4..2d927c23e 100644 --- a/src/coro_rpc/tests/test_coro_rpc_client.cpp +++ b/src/coro_rpc/tests/test_coro_rpc_client.cpp @@ -56,6 +56,10 @@ Lazy> create_client( co_return client; } + void show(auto &s) { + return; + } + TEST_CASE("testing client") { { coro_rpc::coro_rpc_client client; @@ -146,6 +150,7 @@ TEST_CASE("testing client") { std::string arg; arg.resize(2048); auto ret = co_await client->template call(arg); + show(ret); CHECK(ret.value() == arg); co_return; }; @@ -205,6 +210,7 @@ TEST_CASE("testing client with inject server") { auto client = co_await create_client(io_context, port); g_action = inject_action::close_socket_after_send_length; auto ret = co_await client->template call(); + show(ret); REQUIRE_MESSAGE(ret.error().code == coro_rpc::errc::io_error, ret.error().msg); }; From 04f23c37a97b4abc7e0cba4b6639498dbd4e3ea9 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Tue, 7 May 2024 18:01:58 +0800 Subject: [PATCH 09/22] fix client --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 19 ++- .../examples/base_examples/client_pool.cpp | 146 +++++++++++------- .../base_examples/concurrent_clients.cpp | 114 +++++++++----- .../examples/base_examples/rpc_service.cpp | 4 +- 4 files changed, 182 insertions(+), 101 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index c42954d8a..b94eaeece 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -139,6 +139,7 @@ class coro_rpc_client { std::chrono::milliseconds{5000}; std::string host; std::string port; + bool enable_tcp_no_delay_ = true; #ifdef YLT_ENABLE_SSL std::filesystem::path ssl_cert_path; std::string ssl_domain; @@ -332,7 +333,7 @@ class coro_rpc_client { uint32_t get_client_id() const { return config_.client_id; } void close() { - ELOGV(INFO, "client_id %d close", config_.client_id); + //ELOGV(INFO, "client_id %d close", config_.client_id); close_socket(control_); } @@ -409,8 +410,9 @@ class coro_rpc_client { ELOGV(WARN, "client_id %d connect timeout", config_.client_id); co_return errc::timed_out; } - - control_->socket_.set_option(asio::ip::tcp::no_delay(true), ec); + if (config_.enable_tcp_no_delay_==true) { + control_->socket_.set_option(asio::ip::tcp::no_delay(true), ec); + } #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { @@ -614,11 +616,20 @@ class coro_rpc_client { constexpr bool has_conn_v = requires { typename First::return_type; }; return util::get_args(); } + template + static decltype(auto) add_const(U&& u) { + if constexpr (std::is_const_v>) { + return struct_pack::detail::declval(); + } + else { + return struct_pack::detail::declval(); + } + } template void pack_to_impl(Buffer &buffer, std::size_t offset, Args &&...args) { struct_pack::serialize_to_with_offset( - buffer, offset, std::forward(std::forward(args))...); + buffer, offset, std::forward(args))>((std::forward(args)))...); } template diff --git a/src/coro_rpc/examples/base_examples/client_pool.cpp b/src/coro_rpc/examples/base_examples/client_pool.cpp index 23ba2b628..437144227 100644 --- a/src/coro_rpc/examples/base_examples/client_pool.cpp +++ b/src/coro_rpc/examples/base_examples/client_pool.cpp @@ -17,9 +17,12 @@ #include #include #include +#include #include #include #include +#include +#include #include #include #include @@ -27,19 +30,24 @@ #include "async_simple/coro/Collect.h" #include "async_simple/coro/Lazy.h" +#include "async_simple/coro/Mutex.h" #include "async_simple/coro/SyncAwait.h" #include "ylt/coro_io/io_context_pool.hpp" +#include "ylt/coro_rpc/impl/coro_rpc_client.hpp" +#include "ylt/coro_rpc/impl/errno.h" +#include "ylt/coro_rpc/impl/expected.hpp" #include "ylt/easylog.hpp" std::string echo(std::string_view sv); +constexpr auto thread_cnt = 96*2; +constexpr auto request_cnt= 200000; using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; -std::atomic qps = 0; - +auto finish_executor = coro_io::get_global_block_executor(); std::atomic working_echo = 0; -std::atomic busy_echo = 0; +std::atomic stop=false; struct guard { guard(std::atomic &ref) : ref(ref) { ++ref; } ~guard() { --ref; } @@ -50,57 +58,92 @@ struct guard { * \brief demo for run concurrency clients */ -Lazy> call_echo( - coro_io::client_pool &client_pool, int cnt) { - std::vector result; - result.reserve(cnt); +std::vector> clients; + +std::atomic& get_qps(int id) { + static std::atomic ar[thread_cnt*8]; + return ar[id*8]; +} + +int& get_cnt(int id) { + static int ar[thread_cnt*16]; + return ar[id*16]; +} +int& get_flag(int id) { + static int ar[thread_cnt*16]; + return ar[id*16]; +} +std::vector& get_result(int id) { + static std::vector ar[thread_cnt*3]; + return ar[id*3]; +} +int cnt_max=10; +Lazy send(int id) { + auto &cli=*clients[id]; + auto& qps=get_qps(id); + auto &cnt=get_cnt(id); + auto &result=get_result(id); ++working_echo; - for (int i = 0; i < cnt; ++i) { + for (;result.size() Lazy { - guard g{busy_echo}; - if (client.has_closed()) { - co_return; - } - auto res = co_await client.call("Hello world!"); - if (!res.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; - client.close(); - co_return; - } - if (res.value() != "Hello world!"sv) { - ELOG_ERROR << "err echo resp: \n" << res.value(); - co_return; - } - ++qps; - co_return; - }); - if (!res) { - ELOG_ERROR << "client pool err: connect failed.\n"; + auto res_ = co_await cli.send_request("Hello world!"); + if (!res_.has_value()) { + ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; + continue; } - else { - result.push_back(std::chrono::duration_cast( - std::chrono::steady_clock::now() - tp)); + res_.value().start([id,&qps,&cnt,&result,old_tp=tp](auto&&res) { + auto&res1=res.value(); + if (!res1.has_value()) { + ELOG_ERROR << "coro_rpc err: \n" << res1.error().msg; + } + else { + ++qps; + result.push_back(std::chrono::duration_cast( + std::chrono::steady_clock::now()-old_tp)); + auto tmp=cnt--; + if (tmp==cnt_max) { + get_flag(id)=true; + } + else if (tmp==cnt_max/2 && get_flag(id)) { + get_flag(id)=false; + send(id).start([](auto&& res){ + }); + } + } + }); + auto cnt_tmp=++cnt; + if (cnt_tmp==cnt_max) { + break; } } - co_return std::move(result); + --working_echo; + co_return; } -Lazy qps_watcher(coro_io::client_pool &clients) { +Lazy qps_watcher() { using namespace std::chrono_literals; - while (working_echo > 0) { + do { co_await coro_io::sleep_for(1s); - uint64_t cnt = qps.exchange(0); + uint64_t cnt = 0; + for (int i=0;i0); } std::vector result; void latency_watcher() { + result.reserve(request_cnt*thread_cnt); + for (int i=0;i::create( - "localhost:8801", - coro_io::client_pool::pool_config{ - .max_connection = thread_cnt * 20, - .client_config = {.timeout_duration = std::chrono::seconds{50}}}); - - auto finish_executor = coro_io::get_global_block_executor(); - for (int i = 0, lim = thread_cnt * 20; i < lim; ++i) { - call_echo(*client_pool, 10000).start([finish_executor](auto &&res) { - finish_executor->schedule([res = std::move(res.value())] { - result.insert(result.end(), res.begin(), res.end()); - --working_echo; - }); + for (int i=0;i(coro_io::get_global_executor()->get_asio_executor())); + syncAwait(clients.back()->connect("localhost:8801")); + get_result(i).reserve(request_cnt); + } + for (int i = 0, lim = thread_cnt; i < lim; ++i) { + send(i).via(&clients[i]->get_executor()).start([](auto &&res) { }); } - syncAwait(qps_watcher(*client_pool)); + syncAwait(qps_watcher()); latency_watcher(); std::cout << "Done!" << std::endl; return 0; diff --git a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp index c8e8e7a20..65d7fa4ab 100644 --- a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp +++ b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp @@ -13,80 +13,114 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include -#include -#include #include #include #include +#include #include -#include +#include +#include +#include +#include #include #include #include #include +#include "async_simple/coro/Collect.h" +#include "async_simple/coro/Lazy.h" +#include "async_simple/coro/Mutex.h" +#include "async_simple/coro/SyncAwait.h" +#include "ylt/coro_io/io_context_pool.hpp" #include "ylt/coro_rpc/impl/coro_rpc_client.hpp" +#include "ylt/coro_rpc/impl/errno.h" +#include "ylt/coro_rpc/impl/expected.hpp" +#include "ylt/easylog.hpp" std::string echo(std::string_view sv); + +constexpr auto thread_cnt = 96*20; using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; -using namespace std::chrono_literals; -std::atomic qps = 0; +auto finish_executor = coro_io::get_global_block_executor(); std::atomic working_echo = 0; -/*! - * \example helloworld/concurrency_clients.main.cpp - * \brief demo for run concurrency clients - */ -Lazy call_echo(int cnt) { +std::vector> clients; + +std::atomic& get_qps(int id) { + static std::atomic ar[thread_cnt*8]; + return ar[id*8]; +} + +Lazy> send(int id,int cnt) { + std::vector result; + auto &cli=*clients[id]; + auto& qps=get_qps(id); + result.reserve(cnt); ++working_echo; - coro_rpc_client client; - auto ec = co_await client.connect("localhost:8801"); - for (int i = 0; i < 3 && !ec; ++i) { - co_await coro_io::sleep_for(rand() % 10000 * 1ms); - ec = co_await client.reconnect("localhost:8801"); - } - if (!ec) { - for (int i = 0; i < cnt; ++i) { - auto res = co_await client.call("Hello world!"); - if (!res.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; - co_return; - } - if (res.value() != "Hello world!"sv) { - ELOG_ERROR << "err echo resp: \n" << res.value(); - co_return; - } - ++qps; + auto tp = std::chrono::steady_clock::now(); + for (int i=0;i("Hello world!"); + if (!res_.has_value()) { + ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; + continue; } - } - else { - std::cout << "connect failed \n"; + ++qps; + auto old_tp=tp; + tp = std::chrono::steady_clock::now(); + result.push_back(std::chrono::duration_cast(tp-old_tp)); } --working_echo; + co_return std::move(result); } Lazy qps_watcher() { using namespace std::chrono_literals; - while (working_echo > 0) { + do { co_await coro_io::sleep_for(1s); - uint64_t cnt = qps.exchange(0); - std::cout << "QPS:" << cnt << " working echo:" << working_echo << std::endl; + uint64_t cnt = 0; + for (int i=0;i0); +} +std::vector result; +void latency_watcher() { + std::sort(result.begin(), result.end()); + auto arr = {0.1, 0.3, 0.5, 0.7, 0.9, 0.95, 0.99, 0.999, 0.9999, 0.99999, 1.0}; + for (auto e : arr) { + std::cout + << (e * 100) << "% request finished in:" + << result[std::max(0, result.size() * e - 1)].count() / + 1000.0 + << "ms" << std::endl; } } - int main() { - auto thread_cnt = std::thread::hardware_concurrency(); - for (int i = 0, lim = thread_cnt * 20; i < lim; ++i) { - call_echo(100000).start([](auto &&) { + for (int i=0;i(coro_io::get_global_executor()->get_asio_executor())); + syncAwait(clients.back()->connect("localhost:8801")); + } + for (int i = 0, lim = thread_cnt; i < lim; ++i) { + send(i,20000).via(&clients[i]->get_executor()).start([](auto &&res) { + finish_executor->schedule([res = std::move(res.value())] { + result.insert(result.end(), res.begin(), res.end()); + }); }); } syncAwait(qps_watcher()); + latency_watcher(); std::cout << "Done!" << std::endl; return 0; } \ No newline at end of file diff --git a/src/coro_rpc/examples/base_examples/rpc_service.cpp b/src/coro_rpc/examples/base_examples/rpc_service.cpp index 27f04db59..e661e6728 100644 --- a/src/coro_rpc/examples/base_examples/rpc_service.cpp +++ b/src/coro_rpc/examples/base_examples/rpc_service.cpp @@ -102,8 +102,8 @@ Lazy nested_echo(std::string_view sv) { coro_io::g_clients_pool().at("127.0.0.1:8802"); assert(client != nullptr); ELOGV(INFO, "connect another server"); - auto ret = co_await client->send_request([sv](coro_rpc_client &client) { - return client.call(sv); + auto ret = co_await client->send_request([sv](coro_rpc_client &client) ->Lazy> { + co_return co_await client.call(sv); }); co_return ret.value().value(); } From db02794753556cd8282e8da99053df68c53701d2 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 8 May 2024 11:24:08 +0800 Subject: [PATCH 10/22] fix ssl --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index b94eaeece..40e1486f4 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -738,7 +738,7 @@ class coro_rpc_client { #ifdef YLT_ENABLE_SSL if (!ssl_init_ret_) { - co_return rpc_error{errc::not_connected}}; + co_return rpc_error{errc::not_connected}; } #endif From 9c1f100ee4075dd1b45f3f987ba685520fcff7d3 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 8 May 2024 11:48:22 +0800 Subject: [PATCH 11/22] fix timeout --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index 40e1486f4..7228d9244 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -766,7 +766,9 @@ class coro_rpc_client { static void send_err_response(control_t* controller, std::error_code& errc) { - rpc_error ec; + if (controller->is_timeout_) { + errc = std::make_error_code(std::errc::timed_out); + } for (auto &e:controller->response_handler_table_) { e.second.local_error(errc); } From 9254d8712081fcaa3759c39dbb0cd8de53e1060d Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 8 May 2024 11:52:40 +0800 Subject: [PATCH 12/22] fix format --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 268 ++++++++++-------- include/ylt/coro_rpc/impl/errno.h | 8 +- .../examples/base_examples/client_pool.cpp | 92 +++--- .../base_examples/concurrent_clients.cpp | 46 +-- .../examples/base_examples/rpc_service.cpp | 8 +- src/coro_rpc/tests/test_coro_rpc_client.cpp | 4 +- 6 files changed, 234 insertions(+), 192 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index 7228d9244..7c1f3be8d 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -166,7 +166,8 @@ class coro_rpc_client { uint32_t client_id = 0) : control_( std::make_shared(executor.get_asio_executor(), false)), - timer_(std::make_unique(executor.get_asio_executor())) { + timer_(std::make_unique( + executor.get_asio_executor())) { config_.client_id = client_id; } @@ -284,7 +285,7 @@ class coro_rpc_client { template async_simple::coro::Lazy< rpc_result()), coro_rpc_protocol>> - call(Args&&... args) { + call(Args &&...args) { return call_for(std::chrono::seconds(5), std::forward(args)...); } @@ -302,26 +303,30 @@ class coro_rpc_client { template async_simple::coro::Lazy< rpc_result()), coro_rpc_protocol>> - call_for(auto duration, Args&&... args) { - is_waiting_for_response_=true; - using return_type=decltype(get_return_type()); - auto result = co_await send_request_for(duration, std::forward(args)...); + call_for(auto duration, Args &&...args) { + is_waiting_for_response_ = true; + using return_type = decltype(get_return_type()); + auto result = co_await send_request_for( + duration, std::forward(args)...); if (result) { auto async_result = co_await result.value(); if (async_result) { if constexpr (std::is_same_v) { - co_return expected{}; + co_return expected{}; } else { - co_return expected{std::move(async_result.value().result_)}; + co_return expected{ + std::move(async_result.value().result_)}; } } else { - co_return expected{unexpect_t{},std::move(async_result.error())}; + co_return expected{ + unexpect_t{}, std::move(async_result.error())}; } } else { - co_return expected{unexpect_t{},std::move(result.error())}; + co_return expected{unexpect_t{}, + std::move(result.error())}; } } @@ -333,7 +338,7 @@ class coro_rpc_client { uint32_t get_client_id() const { return config_.client_id; } void close() { - //ELOGV(INFO, "client_id %d close", config_.client_id); + // ELOGV(INFO, "client_id %d close", config_.client_id); close_socket(control_); } @@ -346,7 +351,9 @@ class coro_rpc_client { return true; } - std::string_view get_resp_attachment() const { return control_->resp_buffer_.resp_attachment_buf_; } + std::string_view get_resp_attachment() const { + return control_->resp_buffer_.resp_attachment_buf_; + } std::string release_resp_attachment() { return std::move(control_->resp_buffer_.resp_attachment_buf_); @@ -390,7 +397,7 @@ class coro_rpc_client { ELOGV(INFO, "client_id %d begin to connect %s", config_.client_id, config_.port.data()); - timeout(*this->timer_,config_.timeout_duration, "connect timer canceled") + timeout(*this->timer_, config_.timeout_duration, "connect timer canceled") .start([](auto &&) { }); @@ -410,7 +417,7 @@ class coro_rpc_client { ELOGV(WARN, "client_id %d connect timeout", config_.client_id); co_return errc::timed_out; } - if (config_.enable_tcp_no_delay_==true) { + if (config_.enable_tcp_no_delay_ == true) { control_->socket_.set_option(asio::ip::tcp::no_delay(true), ec); } @@ -528,7 +535,7 @@ class coro_rpc_client { * └────────────────┴────────────────┘ */ template - std::vector prepare_buffer(uint32_t& id, Args &&...args) { + std::vector prepare_buffer(uint32_t &id, Args &&...args) { std::vector buffer; std::size_t offset = coro_rpc_protocol::REQ_HEAD_LEN; if constexpr (sizeof...(Args) > 0) { @@ -545,9 +552,9 @@ class coro_rpc_client { header.function_id = func_id(); header.attach_length = req_attachment_.size(); id = request_id_++; - ELOG_TRACE<<"send request ID:"< - static rpc_result handle_response_buffer(std::string_view buffer, - uint8_t rpc_errc,bool& should_close) { + static rpc_result handle_response_buffer( + std::string_view buffer, uint8_t rpc_errc, bool &should_close) { rpc_return_type_t ret; struct_pack::err_code ec; rpc_error err; @@ -616,9 +623,9 @@ class coro_rpc_client { constexpr bool has_conn_v = requires { typename First::return_type; }; return util::get_args(); } - template - static decltype(auto) add_const(U&& u) { - if constexpr (std::is_const_v>) { + template + static decltype(auto) add_const(U &&u) { + if constexpr (std::is_const_v>) { return struct_pack::detail::declval(); } else { @@ -629,7 +636,9 @@ class coro_rpc_client { template void pack_to_impl(Buffer &buffer, std::size_t offset, Args &&...args) { struct_pack::serialize_to_with_offset( - buffer, offset, std::forward(args))>((std::forward(args)))...); + buffer, offset, + std::forward(args))>( + (std::forward(args)))...); } template @@ -648,39 +657,47 @@ class coro_rpc_client { } struct async_rpc_raw_result_value_type { - std::variant buffer_; + std::variant buffer_; uint8_t errc_; }; - using async_rpc_raw_result=std::variant; + using async_rpc_raw_result = + std::variant; struct control_t; struct handler_t { std::unique_ptr timer_; - control_t* control_; + control_t *control_; async_simple::Promise promise_; - handler_t(std::unique_ptr&& timer,control_t* control,async_simple::Promise &&promise):timer_(std::move(timer)), control_(control),promise_(std::move(promise)) {} - void operator()(rpc_resp_buffer&& buffer,uint8_t rpc_errc) { + handler_t(std::unique_ptr &&timer, + control_t *control, + async_simple::Promise &&promise) + : timer_(std::move(timer)), + control_(control), + promise_(std::move(promise)) {} + void operator()(rpc_resp_buffer &&buffer, uint8_t rpc_errc) { timer_->cancel(); if (control_) /*is waiting for response*/ { - promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{std::string_view{control_->resp_buffer_.read_buf_},rpc_errc}}); + promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{ + std::string_view{control_->resp_buffer_.read_buf_}, rpc_errc}}); } else { - promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{std::move(buffer),rpc_errc}}); + promise_.setValue(async_rpc_raw_result{ + async_rpc_raw_result_value_type{std::move(buffer), rpc_errc}}); } } - void local_error (std::error_code& ec) { + void local_error(std::error_code &ec) { timer_->cancel(); promise_.setValue(async_rpc_raw_result{ec}); - } + } }; struct control_t { #ifdef YLT_ENABLE_SSL std::unique_ptr> ssl_stream_; #endif #ifdef GENERATE_BENCHMARK_DATA - std::string func_name_; + std::string func_name_; #endif bool is_timeout_; std::atomic has_closed_ = false; @@ -688,9 +705,12 @@ class coro_rpc_client { std::unordered_map response_handler_table_; rpc_resp_buffer resp_buffer_; asio::ip::tcp::socket socket_; - std::atomic is_recving_=false; + std::atomic is_recving_ = false; control_t(asio::io_context::executor_type executor, bool is_timeout) - : socket_(executor), is_timeout_(is_timeout), has_closed_(false), executor_(executor) {} + : socket_(executor), + is_timeout_(is_timeout), + has_closed_(false), + executor_(executor) {} }; static void close_socket( @@ -722,18 +742,17 @@ class coro_rpc_client { } #endif - - template - async_simple::coro::Lazy - send_request_for_impl(auto duration, uint32_t& id, Args &&...args) { + async_simple::coro::Lazy send_request_for_impl(auto duration, + uint32_t &id, + Args &&...args) { using R = decltype(get_return_type()); if (control_->has_closed_) AS_UNLIKELY { ELOGV(ERROR, "client has been closed, please re-connect"); co_return rpc_error{errc::io_error, - "client has been closed, please re-connect"}; + "client has been closed, please re-connect"}; } #ifdef YLT_ENABLE_SSL @@ -746,7 +765,8 @@ class coro_rpc_client { if (duration.count() > 0) { if (timer_ == nullptr) - timer_ = std::make_unique(control_->executor_.get_asio_executor()); + timer_ = std::make_unique( + control_->executor_.get_asio_executor()); timeout(*timer_, duration, "rpc call timer canceled").start([](auto &&) { }); } @@ -754,122 +774,138 @@ class coro_rpc_client { #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { assert(control_->ssl_stream_); - co_return co_await send_impl(*control_->ssl_stream_, id, std::forward(args)...); + co_return co_await send_impl(*control_->ssl_stream_, id, + std::forward(args)...); } else { #endif - co_return co_await send_impl(control_->socket_, id, std::forward(args)...); + co_return co_await send_impl(control_->socket_, id, + std::forward(args)...); #ifdef YLT_ENABLE_SSL } #endif } - - static void send_err_response(control_t* controller, std::error_code& errc) { + static void send_err_response(control_t *controller, std::error_code &errc) { if (controller->is_timeout_) { errc = std::make_error_code(std::errc::timed_out); } - for (auto &e:controller->response_handler_table_) { + for (auto &e : controller->response_handler_table_) { e.second.local_error(errc); } controller->response_handler_table_.clear(); } - template - static async_simple::coro::Lazy recv(std::shared_ptr controller, Socket& socket) { - std::pair ret; + template + static async_simple::coro::Lazy recv( + std::shared_ptr controller, Socket &socket) { + std::pair ret; do { coro_rpc_protocol::resp_header header; ret = co_await coro_io::async_read( socket, asio::buffer((char *)&header, coro_rpc_protocol::RESP_HEAD_LEN)); if (ret.first) { - ELOG_ERROR<<"read rpc head failed, error msg:"<func_name_ + ".out", - std::ofstream::binary | std::ofstream::out); + std::ofstream file(benchmark_file_path + controller->func_name_ + ".out", + std::ofstream::binary | std::ofstream::out); file << std::string_view{(char *)&header, - coro_rpc_protocol::RESP_HEAD_LEN}; + coro_rpc_protocol::RESP_HEAD_LEN}; file << controller->resp_buffer_.read_buf_; file << controller->resp_buffer_.resp_attachment_buf_; file.close(); #endif - if (auto iter=controller->response_handler_table_.find(header.seq_num);iter!=controller->response_handler_table_.end()) { - ELOG_TRACE<<"find request ID:"<response_handler_table_.find(header.seq_num); + iter != controller->response_handler_table_.end()) { + ELOG_TRACE << "find request ID:" << header.seq_num + << ". start notify response handler"; iter->second(std::move(controller->resp_buffer_), header.err_code); controller->response_handler_table_.erase(iter); } else { - ELOG_ERROR<<"unexists request ID:"<response_handler_table_.size() == 0) { - controller->is_recving_= false; + controller->is_recving_ = false; co_return; } } while (true); controller->is_recving_ = false; - close_socket(controller); - send_err_response(controller.get(),ret.first); + close_socket(controller); + send_err_response(controller.get(), ret.first); co_return; } template - static async_simple::coro::Lazy,rpc_error>> get_deserializer(async_simple::Future future,std::weak_ptr watcher) { + static async_simple::coro::Lazy, rpc_error>> + get_deserializer(async_simple::Future future, + std::weak_ptr watcher) { auto executor = co_await async_simple::CurrentExecutor(); auto executorFuture = std::move(future).via(executor); auto ret_ = co_await std::move(executorFuture); - - if (ret_.index() ==1) [[unlikely]] { // local error - auto& ret=std::get<1>(ret_); - if (ret.value()==static_cast(std::errc::operation_canceled) || ret.value()==static_cast(std::errc::timed_out)) { - co_return coro_rpc::unexpected{rpc_error{errc::timed_out,ret.message()}}; + + if (ret_.index() == 1) [[unlikely]] { // local error + auto &ret = std::get<1>(ret_); + if (ret.value() == static_cast(std::errc::operation_canceled) || + ret.value() == static_cast(std::errc::timed_out)) { + co_return coro_rpc::unexpected{ + rpc_error{errc::timed_out, ret.message()}}; } else { - co_return coro_rpc::unexpected{rpc_error{errc::io_error,ret.message()}}; + co_return coro_rpc::unexpected{ + rpc_error{errc::io_error, ret.message()}}; } } - - bool should_close=false; + + bool should_close = false; std::string_view buffer_view; - auto &ret=std::get<0>(ret_); - if (ret.buffer_.index()==0) { + auto &ret = std::get<0>(ret_); + if (ret.buffer_.index() == 0) { buffer_view = std::get<0>(ret.buffer_).read_buf_; } else { buffer_view = std::get<1>(ret.buffer_); } - auto result = handle_response_buffer(buffer_view,ret.errc_,should_close); + auto result = + handle_response_buffer(buffer_view, ret.errc_, should_close); if (should_close) { - if (auto w=watcher.lock();w) { + if (auto w = watcher.lock(); w) { close_socket(std::move(w)); } } if (result) { if constexpr (std::is_same_v) { - if (ret.buffer_.index()==0) { + if (ret.buffer_.index() == 0) { co_return async_rpc_result{std::move(std::get<0>(ret.buffer_))}; } else { @@ -877,26 +913,29 @@ class coro_rpc_client { } } else { - if (ret.buffer_.index()==0) { - co_return async_rpc_result{result.value(),std::move(std::get<0>(ret.buffer_))}; + if (ret.buffer_.index() == 0) { + co_return async_rpc_result{result.value(), + std::move(std::get<0>(ret.buffer_))}; } else { co_return async_rpc_result{result.value()}; } } - } else { + } + else { co_return coro_rpc::unexpected{result.error()}; } } -public: - template + public: + template async_simple::coro::Lazy())>, rpc_error>>, rpc_error>> send_request(Args &&...args) { - return send_request_for(std::chrono::seconds{5}, std::forward(args)...); + return send_request_for(std::chrono::seconds{5}, + std::forward(args)...); } template @@ -906,57 +945,61 @@ class coro_rpc_client { rpc_error>> send_request_for(auto duration, Args &&...args) { uint32_t id; - auto result = co_await send_request_for_impl(duration, id, std::forward(args)...); + auto result = co_await send_request_for_impl( + duration, id, std::forward(args)...); auto &control = *control_; if (!result) { async_simple::Promise promise; auto future = promise.getFuture(); - bool is_waiting_for_response=is_waiting_for_response_; - is_waiting_for_response_=false; + bool is_waiting_for_response = is_waiting_for_response_; + is_waiting_for_response_ = false; auto &&[_, is_ok] = control.response_handler_table_.try_emplace( - id, std::move(timer_), is_waiting_for_response?control_.get():nullptr,std::move(promise)); + id, std::move(timer_), + is_waiting_for_response ? control_.get() : nullptr, + std::move(promise)); if (!is_ok) [[unlikely]] { close(); - err_code ec=errc::serial_number_conflict; + err_code ec = errc::serial_number_conflict; co_return coro_rpc::unexpected{ec}; } else { if (!control.is_recving_) { - control.is_recving_ = true; + control.is_recving_ = true; #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { assert(control.ssl_stream_); - recv(control_,*control.ssl_stream_).start([](auto&&){}); + recv(control_, *control.ssl_stream_).start([](auto &&) { + }); } else { #endif - recv(control_,control.socket_).start([](auto&&){}); + recv(control_, control.socket_).start([](auto &&) { + }); #ifdef YLT_ENABLE_SSL } -#endif +#endif } - co_return get_deserializer())>(std::move(future),std::weak_ptr{control_}); + co_return get_deserializer())>( + std::move(future), std::weak_ptr{control_}); } } else { co_return coro_rpc::unexpected{std::move(result)}; } } -private: - + private: template - async_simple::coro::Lazy - send_impl(Socket &socket, uint32_t& id, Args&&... args) { + async_simple::coro::Lazy send_impl(Socket &socket, uint32_t &id, + Args &&...args) { auto buffer = prepare_buffer(id, std::forward(args)...); if (buffer.empty()) { co_return rpc_error{errc::message_too_large}; - } + } #ifdef GENERATE_BENCHMARK_DATA control_->func_name_ = get_func_name(); - std::ofstream file( - benchmark_file_path + control_->func_name_ + ".in", - std::ofstream::binary | std::ofstream::out); + std::ofstream file(benchmark_file_path + control_->func_name_ + ".in", + std::ofstream::binary | std::ofstream::out); file << std::string_view{(char *)buffer.data(), buffer.size()}; file.close(); #endif @@ -967,14 +1010,13 @@ class coro_rpc_client { } if (g_action == inject_action::client_close_socket_after_send_header) { ret = co_await coro_io::async_write( - socket, - asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); + socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); ELOGV(INFO, "client_id %d close socket", config_.client_id); close(); co_return rpc_error{errc::io_error, ret.first.message()}; } else if (g_action == - inject_action::client_close_socket_after_send_partial_header) { + inject_action::client_close_socket_after_send_partial_header) { ret = co_await coro_io::async_write( socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN - 1)); @@ -983,10 +1025,9 @@ class coro_rpc_client { co_return rpc_error{errc::io_error, ret.first.message()}; } else if (g_action == - inject_action::client_shutdown_socket_after_send_header) { + inject_action::client_shutdown_socket_after_send_header) { ret = co_await coro_io::async_write( - socket, - asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); + socket, asio::buffer(buffer.data(), coro_rpc_protocol::REQ_HEAD_LEN)); ELOGV(INFO, "client_id %d shutdown", config_.client_id); control_->socket_.shutdown(asio::ip::tcp::socket::shutdown_send); co_return rpc_error{errc::io_error, ret.first.message()}; @@ -1000,8 +1041,7 @@ class coro_rpc_client { else { std::array iov{ asio::const_buffer{buffer.data(), buffer.size()}, - asio::const_buffer{req_attachment_.data(), - req_attachment_.size()}}; + asio::const_buffer{req_attachment_.data(), req_attachment_.size()}}; ret = co_await coro_io::async_write(socket, iov); req_attachment_ = {}; } @@ -1034,7 +1074,7 @@ class coro_rpc_client { } private: - std::atomic is_waiting_for_response_=false; + std::atomic is_waiting_for_response_ = false; std::atomic request_id_{0}; std::unique_ptr timer_; std::shared_ptr control_; diff --git a/include/ylt/coro_rpc/impl/errno.h b/include/ylt/coro_rpc/impl/errno.h index 76543396e..2fd4dabac 100644 --- a/include/ylt/coro_rpc/impl/errno.h +++ b/include/ylt/coro_rpc/impl/errno.h @@ -106,9 +106,11 @@ inline bool operator!(errc ec) noexcept { return ec == errc::ok; } struct rpc_error { coro_rpc::err_code code; //!< error code std::string msg; //!< error message - rpc_error(){} - rpc_error(coro_rpc::err_code code, std::string_view msg):code(code),msg(std::string{msg}){} - rpc_error(coro_rpc::err_code code):code(code),msg(std::string{make_error_message(code)}){} + rpc_error() {} + rpc_error(coro_rpc::err_code code, std::string_view msg) + : code(code), msg(std::string{msg}) {} + rpc_error(coro_rpc::err_code code) + : code(code), msg(std::string{make_error_message(code)}) {} uint16_t& val() { return *(uint16_t*)&(code.ec); } const uint16_t& val() const { return *(uint16_t*)&(code.ec); } constexpr operator bool() const noexcept { return code; } diff --git a/src/coro_rpc/examples/base_examples/client_pool.cpp b/src/coro_rpc/examples/base_examples/client_pool.cpp index 437144227..761fb9fdd 100644 --- a/src/coro_rpc/examples/base_examples/client_pool.cpp +++ b/src/coro_rpc/examples/base_examples/client_pool.cpp @@ -39,19 +39,19 @@ #include "ylt/easylog.hpp" std::string echo(std::string_view sv); -constexpr auto thread_cnt = 96*2; -constexpr auto request_cnt= 200000; +constexpr auto thread_cnt = 96 * 2; +constexpr auto request_cnt = 200000; using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; auto finish_executor = coro_io::get_global_block_executor(); std::atomic working_echo = 0; -std::atomic stop=false; +std::atomic stop = false; struct guard { - guard(std::atomic &ref) : ref(ref) { ++ref; } + guard(std::atomic& ref) : ref(ref) { ++ref; } ~guard() { --ref; } - std::atomic &ref; + std::atomic& ref; }; /*! * \example helloworld/concurrency_clients.main.cpp @@ -61,58 +61,58 @@ struct guard { std::vector> clients; std::atomic& get_qps(int id) { - static std::atomic ar[thread_cnt*8]; - return ar[id*8]; + static std::atomic ar[thread_cnt * 8]; + return ar[id * 8]; } int& get_cnt(int id) { - static int ar[thread_cnt*16]; - return ar[id*16]; + static int ar[thread_cnt * 16]; + return ar[id * 16]; } int& get_flag(int id) { - static int ar[thread_cnt*16]; - return ar[id*16]; + static int ar[thread_cnt * 16]; + return ar[id * 16]; } std::vector& get_result(int id) { - static std::vector ar[thread_cnt*3]; - return ar[id*3]; + static std::vector ar[thread_cnt * 3]; + return ar[id * 3]; } -int cnt_max=10; +int cnt_max = 10; Lazy send(int id) { - auto &cli=*clients[id]; - auto& qps=get_qps(id); - auto &cnt=get_cnt(id); - auto &result=get_result(id); + auto& cli = *clients[id]; + auto& qps = get_qps(id); + auto& cnt = get_cnt(id); + auto& result = get_result(id); ++working_echo; - for (;result.size()("Hello world!"); if (!res_.has_value()) { ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; continue; } - res_.value().start([id,&qps,&cnt,&result,old_tp=tp](auto&&res) { - auto&res1=res.value(); + res_.value().start([id, &qps, &cnt, &result, old_tp = tp](auto&& res) { + auto& res1 = res.value(); if (!res1.has_value()) { ELOG_ERROR << "coro_rpc err: \n" << res1.error().msg; } else { ++qps; result.push_back(std::chrono::duration_cast( - std::chrono::steady_clock::now()-old_tp)); - auto tmp=cnt--; - if (tmp==cnt_max) { - get_flag(id)=true; + std::chrono::steady_clock::now() - old_tp)); + auto tmp = cnt--; + if (tmp == cnt_max) { + get_flag(id) = true; } - else if (tmp==cnt_max/2 && get_flag(id)) { - get_flag(id)=false; - send(id).start([](auto&& res){ + else if (tmp == cnt_max / 2 && get_flag(id)) { + get_flag(id) = false; + send(id).start([](auto&& res) { }); } } }); - auto cnt_tmp=++cnt; - if (cnt_tmp==cnt_max) { + auto cnt_tmp = ++cnt; + if (cnt_tmp == cnt_max) { break; } } @@ -125,23 +125,22 @@ Lazy qps_watcher() { do { co_await coro_io::sleep_for(1s); uint64_t cnt = 0; - for (int i=0;i0); + std::cout << "QPS:" + << cnt + // << " free connection: " << clients.free_client_count() + << " working echo:" << working_echo << std::endl; + } while (working_echo > 0); } std::vector result; void latency_watcher() { - result.reserve(request_cnt*thread_cnt); - for (int i=0;i(coro_io::get_global_executor()->get_asio_executor())); + for (int i = 0; i < thread_cnt; ++i) { + clients.emplace_back(std::make_unique( + coro_io::get_global_executor()->get_asio_executor())); syncAwait(clients.back()->connect("localhost:8801")); get_result(i).reserve(request_cnt); } for (int i = 0, lim = thread_cnt; i < lim; ++i) { - send(i).via(&clients[i]->get_executor()).start([](auto &&res) { + send(i).via(&clients[i]->get_executor()).start([](auto&& res) { }); } syncAwait(qps_watcher()); diff --git a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp index 65d7fa4ab..bf087a659 100644 --- a/src/coro_rpc/examples/base_examples/concurrent_clients.cpp +++ b/src/coro_rpc/examples/base_examples/concurrent_clients.cpp @@ -39,7 +39,7 @@ #include "ylt/easylog.hpp" std::string echo(std::string_view sv); -constexpr auto thread_cnt = 96*20; +constexpr auto thread_cnt = 96 * 20; using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; @@ -50,27 +50,28 @@ std::atomic working_echo = 0; std::vector> clients; std::atomic& get_qps(int id) { - static std::atomic ar[thread_cnt*8]; - return ar[id*8]; + static std::atomic ar[thread_cnt * 8]; + return ar[id * 8]; } -Lazy> send(int id,int cnt) { +Lazy> send(int id, int cnt) { std::vector result; - auto &cli=*clients[id]; - auto& qps=get_qps(id); + auto& cli = *clients[id]; + auto& qps = get_qps(id); result.reserve(cnt); ++working_echo; auto tp = std::chrono::steady_clock::now(); - for (int i=0;i("Hello world!"); if (!res_.has_value()) { ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; continue; } ++qps; - auto old_tp=tp; + auto old_tp = tp; tp = std::chrono::steady_clock::now(); - result.push_back(std::chrono::duration_cast(tp-old_tp)); + result.push_back( + std::chrono::duration_cast(tp - old_tp)); } --working_echo; co_return std::move(result); @@ -81,19 +82,17 @@ Lazy qps_watcher() { do { co_await coro_io::sleep_for(1s); uint64_t cnt = 0; - for (int i=0;i0); + } while (working_echo > 0); } std::vector result; void latency_watcher() { @@ -108,12 +107,13 @@ void latency_watcher() { } } int main() { - for (int i=0;i(coro_io::get_global_executor()->get_asio_executor())); + for (int i = 0; i < thread_cnt; ++i) { + clients.emplace_back(std::make_unique( + coro_io::get_global_executor()->get_asio_executor())); syncAwait(clients.back()->connect("localhost:8801")); } for (int i = 0, lim = thread_cnt; i < lim; ++i) { - send(i,20000).via(&clients[i]->get_executor()).start([](auto &&res) { + send(i, 20000).via(&clients[i]->get_executor()).start([](auto&& res) { finish_executor->schedule([res = std::move(res.value())] { result.insert(result.end(), res.begin(), res.end()); }); diff --git a/src/coro_rpc/examples/base_examples/rpc_service.cpp b/src/coro_rpc/examples/base_examples/rpc_service.cpp index e661e6728..da6eb6db3 100644 --- a/src/coro_rpc/examples/base_examples/rpc_service.cpp +++ b/src/coro_rpc/examples/base_examples/rpc_service.cpp @@ -102,9 +102,11 @@ Lazy nested_echo(std::string_view sv) { coro_io::g_clients_pool().at("127.0.0.1:8802"); assert(client != nullptr); ELOGV(INFO, "connect another server"); - auto ret = co_await client->send_request([sv](coro_rpc_client &client) ->Lazy> { - co_return co_await client.call(sv); - }); + auto ret = co_await client->send_request( + [sv](coro_rpc_client &client) + -> Lazy> { + co_return co_await client.call(sv); + }); co_return ret.value().value(); } diff --git a/src/coro_rpc/tests/test_coro_rpc_client.cpp b/src/coro_rpc/tests/test_coro_rpc_client.cpp index 2d927c23e..cd5314001 100644 --- a/src/coro_rpc/tests/test_coro_rpc_client.cpp +++ b/src/coro_rpc/tests/test_coro_rpc_client.cpp @@ -56,9 +56,7 @@ Lazy> create_client( co_return client; } - void show(auto &s) { - return; - } +void show(auto& s) { return; } TEST_CASE("testing client") { { From 676dd5e95ee3c27e9b56387120bcc17d9e568873 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 8 May 2024 17:09:42 +0800 Subject: [PATCH 13/22] dont reuse timer to make thread-safe --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index 7c1f3be8d..9ddff161c 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -744,7 +744,7 @@ class coro_rpc_client { template async_simple::coro::Lazy send_request_for_impl(auto duration, - uint32_t &id, + uint32_t &id,coro_io::period_timer& timer, Args &&...args) { using R = decltype(get_return_type()); @@ -764,10 +764,7 @@ class coro_rpc_client { static_check(); if (duration.count() > 0) { - if (timer_ == nullptr) - timer_ = std::make_unique( - control_->executor_.get_asio_executor()); - timeout(*timer_, duration, "rpc call timer canceled").start([](auto &&) { + timeout(timer, duration, "rpc call timer canceled").start([](auto &&) { }); } @@ -945,8 +942,9 @@ class coro_rpc_client { rpc_error>> send_request_for(auto duration, Args &&...args) { uint32_t id; + auto timer = std::make_unique(control_->executor_.get_asio_executor()); auto result = co_await send_request_for_impl( - duration, id, std::forward(args)...); + duration, id, *timer, std::forward(args)...); auto &control = *control_; if (!result) { async_simple::Promise promise; @@ -954,7 +952,7 @@ class coro_rpc_client { bool is_waiting_for_response = is_waiting_for_response_; is_waiting_for_response_ = false; auto &&[_, is_ok] = control.response_handler_table_.try_emplace( - id, std::move(timer_), + id, std::move(timer), is_waiting_for_response ? control_.get() : nullptr, std::move(promise)); if (!is_ok) [[unlikely]] { From 1693cba2fa555bbac0eb4c0be95c31f3fe9d2a51 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 8 May 2024 17:10:44 +0800 Subject: [PATCH 14/22] fix attachment --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 43 +++++++++++++------ 1 file changed, 31 insertions(+), 12 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index 9ddff161c..dc04fbdde 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -306,8 +306,9 @@ class coro_rpc_client { call_for(auto duration, Args &&...args) { is_waiting_for_response_ = true; using return_type = decltype(get_return_type()); - auto result = co_await send_request_for( - duration, std::forward(args)...); + auto result = co_await send_request_for_with_attachment( + duration, req_attachment_, std::forward(args)...); + req_attachment_ = {}; if (result) { auto async_result = co_await result.value(); if (async_result) { @@ -744,7 +745,7 @@ class coro_rpc_client { template async_simple::coro::Lazy send_request_for_impl(auto duration, - uint32_t &id,coro_io::period_timer& timer, + uint32_t &id,coro_io::period_timer& timer, std::string_view attachment, Args &&...args) { using R = decltype(get_return_type()); @@ -771,12 +772,12 @@ class coro_rpc_client { #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { assert(control_->ssl_stream_); - co_return co_await send_impl(*control_->ssl_stream_, id, + co_return co_await send_impl(*control_->ssl_stream_, id,attachment, std::forward(args)...); } else { #endif - co_return co_await send_impl(control_->socket_, id, + co_return co_await send_impl(control_->socket_, id,attachment, std::forward(args)...); #ifdef YLT_ENABLE_SSL } @@ -931,7 +932,7 @@ class coro_rpc_client { async_rpc_result())>, rpc_error>>, rpc_error>> send_request(Args &&...args) { - return send_request_for(std::chrono::seconds{5}, + return send_request_for_with_attachment(std::chrono::seconds{5},{}, std::forward(args)...); } @@ -940,11 +941,30 @@ class coro_rpc_client { async_simple::coro::Lazy())>, rpc_error>>, rpc_error>> - send_request_for(auto duration, Args &&...args) { + send_request_with_attachment(std::string_view request_attachment,Args &&...args) { + return send_request_for_with_attachment(std::chrono::seconds{5},request_attachment, + std::forward(args)...); + } + + template + async_simple::coro::Lazy())>, rpc_error>>, + rpc_error>> + send_request_for(Args &&...args) { + return send_request_for_with_attachment(std::chrono::seconds{5},std::string_view{},std::forward(args)...); + } + + template + async_simple::coro::Lazy())>, rpc_error>>, + rpc_error>> + send_request_for_with_attachment(auto time_out_duration, std::string_view request_attachment, Args &&...args) { uint32_t id; auto timer = std::make_unique(control_->executor_.get_asio_executor()); auto result = co_await send_request_for_impl( - duration, id, *timer, std::forward(args)...); + time_out_duration, id, *timer, request_attachment, std::forward(args)...); auto &control = *control_; if (!result) { async_simple::Promise promise; @@ -988,7 +1008,7 @@ class coro_rpc_client { private: template - async_simple::coro::Lazy send_impl(Socket &socket, uint32_t &id, + async_simple::coro::Lazy send_impl(Socket &socket, uint32_t &id, std::string_view req_attachment, Args &&...args) { auto buffer = prepare_buffer(id, std::forward(args)...); if (buffer.empty()) { @@ -1032,16 +1052,15 @@ class coro_rpc_client { } else { #endif - if (req_attachment_.empty()) { + if (req_attachment.empty()) { ret = co_await coro_io::async_write( socket, asio::buffer(buffer.data(), buffer.size())); } else { std::array iov{ asio::const_buffer{buffer.data(), buffer.size()}, - asio::const_buffer{req_attachment_.data(), req_attachment_.size()}}; + asio::const_buffer{req_attachment.data(), req_attachment.size()}}; ret = co_await coro_io::async_write(socket, iov); - req_attachment_ = {}; } #ifdef UNIT_TEST_INJECT } From 541664b806909adba786b0435965b52667e39f70 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 8 May 2024 17:18:41 +0800 Subject: [PATCH 15/22] add mutex lock for thread-safe --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index dc04fbdde..c3159fe25 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -45,6 +45,7 @@ #include "asio/registered_buffer.hpp" #include "async_simple/Executor.h" #include "async_simple/Promise.h" +#include "async_simple/coro/Mutex.h" #include "common_service.hpp" #include "context.hpp" #include "expected.hpp" @@ -1053,6 +1054,7 @@ class coro_rpc_client { else { #endif if (req_attachment.empty()) { + auto loc = co_await write_mutex_.coScopedLock(); ret = co_await coro_io::async_write( socket, asio::buffer(buffer.data(), buffer.size())); } @@ -1060,6 +1062,7 @@ class coro_rpc_client { std::array iov{ asio::const_buffer{buffer.data(), buffer.size()}, asio::const_buffer{req_attachment.data(), req_attachment.size()}}; + auto loc = co_await write_mutex_.coScopedLock(); ret = co_await coro_io::async_write(socket, iov); } #ifdef UNIT_TEST_INJECT @@ -1091,6 +1094,7 @@ class coro_rpc_client { } private: + async_simple::coro::Mutex write_mutex_; std::atomic is_waiting_for_response_ = false; std::atomic request_id_{0}; std::unique_ptr timer_; From 9dd572c36eb8324e190a4f292b0d01778b734a89 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Wed, 15 May 2024 18:04:07 +0800 Subject: [PATCH 16/22] use atomic lock instead of mutex --- include/ylt/coro_io/coro_io.hpp | 2 +- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 68 +++++++-- .../examples/base_examples/channel.cpp | 115 ++++++++++++---- .../examples/base_examples/client_pool.cpp | 130 +++++++++--------- .../examples/base_examples/client_pools.cpp | 39 +++++- 5 files changed, 241 insertions(+), 113 deletions(-) diff --git a/include/ylt/coro_io/coro_io.hpp b/include/ylt/coro_io/coro_io.hpp index d4d114ed6..eab7727b0 100644 --- a/include/ylt/coro_io/coro_io.hpp +++ b/include/ylt/coro_io/coro_io.hpp @@ -317,7 +317,7 @@ inline async_simple::coro::Lazy sleep_for(Duration d) { template struct post_helper { void operator()(auto handler) { - asio::dispatch(e, [this, handler]() { + asio::post(e, [this, handler]() { try { if constexpr (std::is_same_v>) { func(); diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index c3159fe25..e59d95d53 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -46,6 +46,7 @@ #include "async_simple/Executor.h" #include "async_simple/Promise.h" #include "async_simple/coro/Mutex.h" +#include "async_simple/coro/SpinLock.h" #include "common_service.hpp" #include "context.hpp" #include "expected.hpp" @@ -707,7 +708,7 @@ class coro_rpc_client { std::unordered_map response_handler_table_; rpc_resp_buffer resp_buffer_; asio::ip::tcp::socket socket_; - std::atomic is_recving_ = false; + std::atomic recving_cnt_ = 0; control_t(asio::io_context::executor_type executor, bool is_timeout) : socket_(executor), is_timeout_(is_timeout), @@ -842,24 +843,23 @@ class coro_rpc_client { file << controller->resp_buffer_.resp_attachment_buf_; file.close(); #endif + --controller->recving_cnt_; if (auto iter = controller->response_handler_table_.find(header.seq_num); iter != controller->response_handler_table_.end()) { ELOG_TRACE << "find request ID:" << header.seq_num << ". start notify response handler"; iter->second(std::move(controller->resp_buffer_), header.err_code); controller->response_handler_table_.erase(iter); + if (controller->response_handler_table_.empty()) { + co_return; + } } else { ELOG_ERROR << "unexists request ID:" << header.seq_num << ". close the socket."; break; } - if (controller->response_handler_table_.size() == 0) { - controller->is_recving_ = false; - co_return; - } } while (true); - controller->is_recving_ = false; close_socket(controller); send_err_response(controller.get(), ret.first); co_return; @@ -869,9 +869,7 @@ class coro_rpc_client { static async_simple::coro::Lazy, rpc_error>> get_deserializer(async_simple::Future future, std::weak_ptr watcher) { - auto executor = co_await async_simple::CurrentExecutor(); - auto executorFuture = std::move(future).via(executor); - auto ret_ = co_await std::move(executorFuture); + auto ret_ = co_await std::move(future); if (ret_.index() == 1) [[unlikely]] { // local error auto &ret = std::get<1>(ret_); @@ -956,12 +954,31 @@ class coro_rpc_client { return send_request_for_with_attachment(std::chrono::seconds{5},std::string_view{},std::forward(args)...); } + struct recving_guard { + recving_guard(control_t* ctrl):ctrl_(ctrl){ + ctrl_->recving_cnt_++; + } + void release() { + ctrl_=nullptr; + } + ~recving_guard() { + if (ctrl_) { + --ctrl_->recving_cnt_; + } + } + control_t* ctrl_; + }; + template async_simple::coro::Lazy())>, rpc_error>>, rpc_error>> send_request_for_with_attachment(auto time_out_duration, std::string_view request_attachment, Args &&...args) { + recving_guard guard(control_.get()); + // if (control_->recving_cnt_ > 1) { + // ELOG_ERROR<<"SHIT2:"<recving_cnt_<<"addr:"<<(void*)this; + // } uint32_t id; auto timer = std::make_unique(control_->executor_.get_asio_executor()); auto result = co_await send_request_for_impl( @@ -972,6 +989,7 @@ class coro_rpc_client { auto future = promise.getFuture(); bool is_waiting_for_response = is_waiting_for_response_; is_waiting_for_response_ = false; + bool is_empty = control.response_handler_table_.empty(); auto &&[_, is_ok] = control.response_handler_table_.try_emplace( id, std::move(timer), is_waiting_for_response ? control_.get() : nullptr, @@ -982,8 +1000,7 @@ class coro_rpc_client { co_return coro_rpc::unexpected{ec}; } else { - if (!control.is_recving_) { - control.is_recving_ = true; + if (is_empty) { #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { assert(control.ssl_stream_); @@ -998,6 +1015,7 @@ class coro_rpc_client { } #endif } + guard.release(); co_return get_deserializer())>( std::move(future), std::weak_ptr{control_}); } @@ -1007,6 +1025,10 @@ class coro_rpc_client { } } + uint32_t get_pipeline_size() { + return control_->recving_cnt_; + } + private: template async_simple::coro::Lazy send_impl(Socket &socket, uint32_t &id, std::string_view req_attachment, @@ -1054,16 +1076,34 @@ class coro_rpc_client { else { #endif if (req_attachment.empty()) { - auto loc = co_await write_mutex_.coScopedLock(); + // auto loc = co_await write_mutex_.coScopedLock(); + while (true) { + bool expected = false; + if (write_mutex_.compare_exchange_weak(expected,true)) { + break; + } + //ELOG_ERROR<<"SHIT"; + co_await coro_io::post([](){},&control_->executor_); + } ret = co_await coro_io::async_write( socket, asio::buffer(buffer.data(), buffer.size())); + write_mutex_ = false; } else { std::array iov{ asio::const_buffer{buffer.data(), buffer.size()}, asio::const_buffer{req_attachment.data(), req_attachment.size()}}; - auto loc = co_await write_mutex_.coScopedLock(); + // auto loc = co_await write_mutex_.coScopedLock(); + while (true) { + bool expected = false; + if (write_mutex_.compare_exchange_weak(expected,true)) { + break; + } + //ELOG_ERROR<<"SHIT"; + co_await coro_io::post([](){},&control_->executor_); + } ret = co_await coro_io::async_write(socket, iov); + write_mutex_ = false; } #ifdef UNIT_TEST_INJECT } @@ -1094,7 +1134,7 @@ class coro_rpc_client { } private: - async_simple::coro::Mutex write_mutex_; + std::atomic write_mutex_=false; std::atomic is_waiting_for_response_ = false; std::atomic request_id_{0}; std::unique_ptr timer_; diff --git a/src/coro_rpc/examples/base_examples/channel.cpp b/src/coro_rpc/examples/base_examples/channel.cpp index 99f23c833..dad17ac3f 100644 --- a/src/coro_rpc/examples/base_examples/channel.cpp +++ b/src/coro_rpc/examples/base_examples/channel.cpp @@ -30,6 +30,9 @@ #include #include +#include "ylt/coro_io/io_context_pool.hpp" +#include "ylt/coro_rpc/impl/coro_rpc_client.hpp" +#include "ylt/coro_rpc/impl/expected.hpp" #include "ylt/easylog.hpp" using namespace coro_rpc; using namespace async_simple::coro; @@ -39,37 +42,72 @@ std::string echo(std::string_view sv); std::atomic qps = 0; std::atomic working_echo = 0; -/*! - * \example helloworld/concurrency_clients.main.cpp - * \brief demo for run concurrency clients - */ -Lazy call_echo(std::shared_ptr> channel, - int cnt) { - while (true) { - ++working_echo; - for (int i = 0; i < cnt; ++i) { - auto res = co_await channel->send_request( - [](coro_rpc_client &client, std::string_view hostname) -> Lazy { - auto res = co_await client.call("Hello world!"); - if (!res.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; - co_return; - } - if (res.value() != "Hello world!"sv) { - ELOG_ERROR << "err echo resp: \n" << res.value(); - co_return; - } - ++qps; - co_return; - }); - if (!res) { - ELOG_ERROR << "client pool err: connect failed.\n"; - } +int request_cnt = 10000; + +// Lazy> call_echo(std::shared_ptr> channel) { +// std::vector result; +// result.reserve(request_cnt); +// auto tp = std::chrono::steady_clock::now(); +// ++working_echo; +// for (int i = 0; i < request_cnt; ++i) { +// auto res = co_await channel->send_request( +// [](coro_rpc_client &client, std::string_view hostname) -> Lazy { +// auto res = co_await client.call("Hello world!"); +// if (!res.has_value()) { +// ELOG_ERROR << "coro_rpc err: \n" << res.error().msg; +// co_return; +// } +// if (res.value() != "Hello world!"sv) { +// ELOG_ERROR << "err echo resp: \n" << res.value(); +// co_return; +// } +// co_return; +// }); +// if (!res) { +// ELOG_ERROR << "client pool err: connect failed.\n"; +// break; +// } +// ++qps; +// auto old_tp=tp; +// tp= std::chrono::steady_clock::now(); +// result.push_back(std::chrono::duration_cast( +// tp - old_tp)); +// } +// co_return std::move(result); +// } + +Lazy> call_echo(std::shared_ptr> channel) { + std::vector result; + result.reserve(request_cnt); + auto tp = std::chrono::steady_clock::now(); + ++working_echo; + for (int i = 0; i < request_cnt; ++i) { + auto res = co_await channel->send_request( + [](coro_rpc_client &client, std::string_view hostname) -> Lazy, coro_rpc::rpc_error>>, coro_rpc::rpc_error>> { + co_return co_await client.send_request("Hello world!"); + }); + if (!res) { + ELOG_ERROR << "client pool err: connect failed.\n"; + break; + } + auto &res1=res.value(); + if (!res1) { + ELOG_ERROR << "send request failed.\n"; + break; } - --working_echo; - co_await coro_io::sleep_for(30s); + auto res2 = co_await res1.value(); + if (!res2) { + ELOG_ERROR << "recv response failed\n"; + break; + } + ++qps; + auto old_tp=tp; + tp= std::chrono::steady_clock::now(); + result.push_back(std::chrono::duration_cast( + tp - old_tp)); } + co_return std::move(result); } Lazy qps_watcher() { @@ -87,6 +125,19 @@ Lazy qps_watcher() { } } +std::vector result; +void latency_watcher() { + std::sort(result.begin(), result.end()); + auto arr = {0.1, 0.3, 0.5, 0.7, 0.9, 0.95, 0.99, 0.999, 0.9999, 0.99999, 1.0}; + for (auto e : arr) { + std::cout + << (e * 100) << "% request finished in:" + << result[std::max(0, result.size() * e - 1)].count() / + 1000.0 + << "ms" << std::endl; + } +} + int main() { auto hosts = std::vector{"127.0.0.1:8801", "localhost:8801"}; @@ -95,11 +146,17 @@ int main() { hosts, coro_io::channel::channel_config{ .pool_config{.max_connection = worker_cnt}}); auto chan_ptr = std::make_shared(std::move(chan)); + auto executor = coro_io::get_global_block_executor(); for (int i = 0; i < worker_cnt; ++i) { - call_echo(chan_ptr, 10000).start([](auto &&) { + call_echo(chan_ptr).start([=](auto&& res) { + executor->schedule([res = std::move(res.value())]() mutable{ + result.insert(result.end(), res.begin(), res.end()); + --working_echo; + }); }); } syncAwait(qps_watcher()); + latency_watcher(); std::cout << "Done!" << std::endl; return 0; } \ No newline at end of file diff --git a/src/coro_rpc/examples/base_examples/client_pool.cpp b/src/coro_rpc/examples/base_examples/client_pool.cpp index 761fb9fdd..2a63b8442 100644 --- a/src/coro_rpc/examples/base_examples/client_pool.cpp +++ b/src/coro_rpc/examples/base_examples/client_pool.cpp @@ -37,10 +37,11 @@ #include "ylt/coro_rpc/impl/errno.h" #include "ylt/coro_rpc/impl/expected.hpp" #include "ylt/easylog.hpp" +#include "ylt/easylog/record.hpp" std::string echo(std::string_view sv); -constexpr auto thread_cnt = 96 * 2; -constexpr auto request_cnt = 200000; +constexpr unsigned thread_cnt = 1920; +constexpr auto request_cnt = 1000; using namespace coro_rpc; using namespace async_simple::coro; using namespace std::string_view_literals; @@ -60,76 +61,74 @@ struct guard { std::vector> clients; -std::atomic& get_qps(int id) { - static std::atomic ar[thread_cnt * 8]; - return ar[id * 8]; +std::atomic rr=0; +auto st=std::min(thread_cnt,std::thread::hardware_concurrency()); +std::atomic tid=0; +uint64_t& get_tid() { + thread_local static uint64_t mytid=tid++; + return mytid; } -int& get_cnt(int id) { - static int ar[thread_cnt * 16]; - return ar[id * 16]; -} -int& get_flag(int id) { - static int ar[thread_cnt * 16]; - return ar[id * 16]; -} -std::vector& get_result(int id) { - static std::vector ar[thread_cnt * 3]; - return ar[id * 3]; +// coro_rpc_client& get_client() { +// thread_local static uint32_t rr=get_tid(); +// coro_rpc_client* cli; +// cli = clients[rr%thread_cnt].get(); +// rr+=st; +// if (rr>thread_cnt) { +// rr%=st; +// } +// return *cli; +// } + +coro_rpc_client& get_client() { + coro_rpc_client* cli; + auto v=rr.fetch_add(101); + unsigned min_sz=UINT32_MAX; + v%=thread_cnt; + for (int i=0;min_sz>0 && i<=0;++i) { + auto sz=clients[v]->get_pipeline_size(); + if (min_sz > sz) { + cli = clients[v].get(); + min_sz = sz; + } + v+=1; + v%=thread_cnt; + } + return *cli; } -int cnt_max = 10; -Lazy send(int id) { - auto& cli = *clients[id]; - auto& qps = get_qps(id); - auto& cnt = get_cnt(id); - auto& result = get_result(id); +std::atomic qps=0; +Lazy> send() { + std::vector result; + result.reserve(request_cnt); + auto tp = std::chrono::steady_clock::now(); ++working_echo; - for (; result.size() < request_cnt;) { - auto tp = std::chrono::steady_clock::now(); + int id=0; + for (int i=0;i("Hello world!"); if (!res_.has_value()) { ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; continue; } - res_.value().start([id, &qps, &cnt, &result, old_tp = tp](auto&& res) { - auto& res1 = res.value(); - if (!res1.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res1.error().msg; - } - else { - ++qps; - result.push_back(std::chrono::duration_cast( - std::chrono::steady_clock::now() - old_tp)); - auto tmp = cnt--; - if (tmp == cnt_max) { - get_flag(id) = true; - } - else if (tmp == cnt_max / 2 && get_flag(id)) { - get_flag(id) = false; - send(id).start([](auto&& res) { - }); - } - } - }); - auto cnt_tmp = ++cnt; - if (cnt_tmp == cnt_max) { + auto res1 = co_await res_.value(); + if (!res1.has_value()) { + ELOG_ERROR << "coro_rpc err: \n" << res1.error().msg; break; } + qps.fetch_add(1,std::memory_order::release); + auto old_tp=tp; + tp= std::chrono::steady_clock::now(); + result.push_back(std::chrono::duration_cast( + tp - old_tp)); } - --working_echo; - co_return; + co_return std::move(result); } Lazy qps_watcher() { using namespace std::chrono_literals; do { co_await coro_io::sleep_for(1s); - uint64_t cnt = 0; - for (int i = 0; i < thread_cnt; ++i) { - auto& qps = get_qps(i); - uint64_t tmp = qps.exchange(0); - cnt += tmp; - } + uint64_t cnt = qps.exchange(0); std::cout << "QPS:" << cnt // << " free connection: " << clients.free_client_count() @@ -138,11 +137,6 @@ Lazy qps_watcher() { } std::vector result; void latency_watcher() { - result.reserve(request_cnt * thread_cnt); - for (int i = 0; i < thread_cnt; ++i) { - auto& res = get_result(i); - result.insert(result.end(), res.begin(), res.end()); - } std::sort(result.begin(), result.end()); auto arr = {0.1, 0.3, 0.5, 0.7, 0.9, 0.95, 0.99, 0.999, 0.9999, 0.99999, 1.0}; for (auto e : arr) { @@ -154,15 +148,25 @@ void latency_watcher() { } } int main() { + for (int i = 0; i < std::thread::hardware_concurrency(); ++i) { + coro_io::get_global_executor()->schedule([i](){ + get_tid()=i; + }); + } for (int i = 0; i < thread_cnt; ++i) { clients.emplace_back(std::make_unique( coro_io::get_global_executor()->get_asio_executor())); syncAwait(clients.back()->connect("localhost:8801")); - get_result(i).reserve(request_cnt); } - for (int i = 0, lim = thread_cnt; i < lim; ++i) { - send(i).via(&clients[i]->get_executor()).start([](auto&& res) { - }); + auto executor = coro_io::get_global_block_executor(); + // auto e = coro_io::get_global_executor(); + for (int i = 0, lim = thread_cnt*10; i < lim; ++i) { + coro_io::get_global_executor()->schedule([=](){send().start([executor](auto&& res) { + executor->schedule([res = std::move(res.value())]() mutable{ + result.insert(result.end(), res.begin(), res.end()); + --working_echo; + }); + });}); } syncAwait(qps_watcher()); latency_watcher(); diff --git a/src/coro_rpc/examples/base_examples/client_pools.cpp b/src/coro_rpc/examples/base_examples/client_pools.cpp index 8bf414d60..f568d0e2d 100644 --- a/src/coro_rpc/examples/base_examples/client_pools.cpp +++ b/src/coro_rpc/examples/base_examples/client_pools.cpp @@ -26,6 +26,7 @@ #include #include #include +#include "ylt/coro_io/io_context_pool.hpp" std::string echo(std::string_view sv); using namespace coro_rpc; using namespace async_simple::coro; @@ -39,10 +40,14 @@ std::atomic working_echo = 0; * \brief demo for run concurrency clients */ -Lazy call_echo(coro_io::client_pools &client_pools, - int cnt) { +int request_cnt =10000; + +Lazy> call_echo(coro_io::client_pools &client_pools) { ++working_echo; - for (int i = 0; i < cnt; ++i) { + std::vector result; + result.reserve(request_cnt); + auto tp = std::chrono::steady_clock::now(); + for (int i = 0; i < request_cnt; ++i) { auto res = co_await client_pools.send_request( i % 2 ? "localhost:8801" : "127.0.0.1:8801", [=](coro_rpc_client &client) -> Lazy { @@ -60,9 +65,14 @@ Lazy call_echo(coro_io::client_pools &client_pools, }); if (!res) { ELOG_ERROR << "client pool err: connect failed.\n"; + break; } + auto old_tp=tp; + tp= std::chrono::steady_clock::now(); + result.push_back(std::chrono::duration_cast( + tp - old_tp)); } - --working_echo; + co_return std::move(result); } Lazy qps_watcher(coro_io::client_pools &clients) { @@ -78,15 +88,32 @@ Lazy qps_watcher(coro_io::client_pools &clients) { cnt = 0; } } - +std::vector result; +void latency_watcher() { + std::sort(result.begin(), result.end()); + auto arr = {0.1, 0.3, 0.5, 0.7, 0.9, 0.95, 0.99, 0.999, 0.9999, 0.99999, 1.0}; + for (auto e : arr) { + std::cout + << (e * 100) << "% request finished in:" + << result[std::max(0, result.size() * e - 1)].count() / + 1000.0 + << "ms" << std::endl; + } +} int main() { auto thread_cnt = std::thread::hardware_concurrency(); auto &clients = coro_io::g_clients_pool(); + auto executor=coro_io::get_global_block_executor(); for (int i = 0, lim = thread_cnt * 20; i < lim; ++i) { - call_echo(clients, 10000).start([](auto &&) { + call_echo(clients).start([=](auto && res) { + executor->schedule([res = std::move(res.value())]() mutable { + result.insert(result.end(), res.begin(), res.end()); + --working_echo; + }); }); } syncAwait(qps_watcher(clients)); + latency_watcher(); std::cout << "Done!" << std::endl; return 0; } \ No newline at end of file From a0b0cc83d1092ddb2f918ef9dedf8b825fb6e32a Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Thu, 16 May 2024 16:10:12 +0800 Subject: [PATCH 17/22] update client_pool code --- .../examples/base_examples/client_pool.cpp | 80 ++++--------------- 1 file changed, 15 insertions(+), 65 deletions(-) diff --git a/src/coro_rpc/examples/base_examples/client_pool.cpp b/src/coro_rpc/examples/base_examples/client_pool.cpp index 2a63b8442..79f7162ee 100644 --- a/src/coro_rpc/examples/base_examples/client_pool.cpp +++ b/src/coro_rpc/examples/base_examples/client_pool.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -48,80 +49,40 @@ using namespace std::string_view_literals; auto finish_executor = coro_io::get_global_block_executor(); std::atomic working_echo = 0; -std::atomic stop = false; -struct guard { - guard(std::atomic& ref) : ref(ref) { ++ref; } - ~guard() { --ref; } - std::atomic& ref; -}; /*! * \example helloworld/concurrency_clients.main.cpp * \brief demo for run concurrency clients */ -std::vector> clients; +auto client_pool=coro_io::client_pool::create("127.0.0.1:8801"); -std::atomic rr=0; -auto st=std::min(thread_cnt,std::thread::hardware_concurrency()); -std::atomic tid=0; -uint64_t& get_tid() { - thread_local static uint64_t mytid=tid++; - return mytid; -} - -// coro_rpc_client& get_client() { -// thread_local static uint32_t rr=get_tid(); -// coro_rpc_client* cli; -// cli = clients[rr%thread_cnt].get(); -// rr+=st; -// if (rr>thread_cnt) { -// rr%=st; -// } -// return *cli; -// } - -coro_rpc_client& get_client() { - coro_rpc_client* cli; - auto v=rr.fetch_add(101); - unsigned min_sz=UINT32_MAX; - v%=thread_cnt; - for (int i=0;min_sz>0 && i<=0;++i) { - auto sz=clients[v]->get_pipeline_size(); - if (min_sz > sz) { - cli = clients[v].get(); - min_sz = sz; - } - v+=1; - v%=thread_cnt; - } - return *cli; -} std::atomic qps=0; Lazy> send() { - std::vector result; - result.reserve(request_cnt); + std::vector latencys; + latencys.reserve(request_cnt); auto tp = std::chrono::steady_clock::now(); ++working_echo; int id=0; for (int i=0;i("Hello world!"); - if (!res_.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res_.error().msg; + auto result = co_await client_pool->send_request([](coro_rpc_client& client) ->Lazy> { + co_return co_await client.call("Hello world!"); + }); + if (!result) { + ELOG_ERROR << "get client form client pool failed: \n" << std::make_error_code(result.error()).message(); continue; } - auto res1 = co_await res_.value(); - if (!res1.has_value()) { - ELOG_ERROR << "coro_rpc err: \n" << res1.error().msg; + auto &call_result = result.value(); + if (!call_result) { + ELOG_ERROR << "call err: \n" << call_result.error().msg; break; } qps.fetch_add(1,std::memory_order::release); auto old_tp=tp; tp= std::chrono::steady_clock::now(); - result.push_back(std::chrono::duration_cast( + latencys.push_back(std::chrono::duration_cast( tp - old_tp)); } - co_return std::move(result); + co_return std::move(latencys); } Lazy qps_watcher() { @@ -131,7 +92,7 @@ Lazy qps_watcher() { uint64_t cnt = qps.exchange(0); std::cout << "QPS:" << cnt - // << " free connection: " << clients.free_client_count() + << " free connection: " << client_pool->free_client_count() << " working echo:" << working_echo << std::endl; } while (working_echo > 0); } @@ -148,18 +109,7 @@ void latency_watcher() { } } int main() { - for (int i = 0; i < std::thread::hardware_concurrency(); ++i) { - coro_io::get_global_executor()->schedule([i](){ - get_tid()=i; - }); - } - for (int i = 0; i < thread_cnt; ++i) { - clients.emplace_back(std::make_unique( - coro_io::get_global_executor()->get_asio_executor())); - syncAwait(clients.back()->connect("localhost:8801")); - } auto executor = coro_io::get_global_block_executor(); - // auto e = coro_io::get_global_executor(); for (int i = 0, lim = thread_cnt*10; i < lim; ++i) { coro_io::get_global_executor()->schedule([=](){send().start([executor](auto&& res) { executor->schedule([res = std::move(res.value())]() mutable{ From ca230e8bcf6d4d689964c040fd62d9bf4dcdc708 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Thu, 16 May 2024 17:49:23 +0800 Subject: [PATCH 18/22] simply code --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 154 ++++++++---------- include/ylt/coro_rpc/impl/expected.hpp | 3 +- .../examples/base_examples/channel.cpp | 20 +-- src/coro_rpc/tests/test_router.cpp | 5 +- 4 files changed, 77 insertions(+), 105 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index e59d95d53..b7d5e5541 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -94,16 +94,30 @@ struct rpc_resp_buffer { }; template -struct async_rpc_result { +struct async_rpc_result_value_t { +private: T result_; rpc_resp_buffer buffer_; +public: + async_rpc_result_value_t(T&& result,rpc_resp_buffer&& buffer): result_(std::move(result)),buffer_(std::move(buffer)){} + async_rpc_result_value_t(T&& result): result_(std::move(result)){} + T& result() noexcept {return result_;} + const T& result() const noexcept {return result_;} + std::string_view attachment() const noexcept {return buffer_.resp_attachment_buf_;} + rpc_resp_buffer release_buffer() {return std::move(buffer_);} }; + template <> -struct async_rpc_result { +struct async_rpc_result_value_t { rpc_resp_buffer buffer_; + std::string_view attachment() const noexcept {return buffer_.resp_attachment_buf_;} + rpc_resp_buffer release_buffer() {return std::move(buffer_);} }; +template +using async_rpc_result = expected, rpc_error>; + template using rpc_return_type_t = typename rpc_return_type::type; /*! @@ -286,7 +300,7 @@ class coro_rpc_client { */ template async_simple::coro::Lazy< - rpc_result()), coro_rpc_protocol>> + rpc_result())>> call(Args &&...args) { return call_for(std::chrono::seconds(5), std::forward(args)...); } @@ -304,32 +318,25 @@ class coro_rpc_client { */ template async_simple::coro::Lazy< - rpc_result()), coro_rpc_protocol>> + rpc_result())>> call_for(auto duration, Args &&...args) { - is_waiting_for_response_ = true; using return_type = decltype(get_return_type()); - auto result = co_await send_request_for_with_attachment( + auto async_result = co_await co_await send_request_for_with_attachment( duration, req_attachment_, std::forward(args)...); req_attachment_ = {}; - if (result) { - auto async_result = co_await result.value(); - if (async_result) { - if constexpr (std::is_same_v) { - co_return expected{}; - } - else { - co_return expected{ - std::move(async_result.value().result_)}; - } + if (async_result) { + control_->resp_buffer_ = async_result->release_buffer(); + if constexpr (std::is_same_v) { + co_return expected{}; } else { co_return expected{ - unexpect_t{}, std::move(async_result.error())}; + std::move(async_result->result())}; } } else { - co_return expected{unexpect_t{}, - std::move(result.error())}; + co_return expected{ + unexpect_t{}, std::move(async_result.error())}; } } @@ -580,7 +587,7 @@ class coro_rpc_client { } template - static rpc_result handle_response_buffer( + static rpc_result handle_response_buffer( std::string_view buffer, uint8_t rpc_errc, bool &should_close) { rpc_return_type_t ret; struct_pack::err_code ec; @@ -603,13 +610,13 @@ class coro_rpc_client { ec = struct_pack::deserialize_to(err.msg, buffer); if SP_LIKELY (!ec) { should_close = true; - return rpc_result{unexpect_t{}, std::move(err)}; + return rpc_result{unexpect_t{}, std::move(err)}; } } else { ec = struct_pack::deserialize_to(err, buffer); if SP_LIKELY (!ec) { - return rpc_result{unexpect_t{}, std::move(err)}; + return rpc_result{unexpect_t{}, std::move(err)}; } } } @@ -617,7 +624,7 @@ class coro_rpc_client { // deserialize failed. ELOGV(WARNING, "deserilaize rpc result failed"); err = {errc::invalid_rpc_result, "failed to deserialize rpc return value"}; - return rpc_result{unexpect_t{}, std::move(err)}; + return rpc_result{unexpect_t{}, std::move(err)}; } template @@ -660,7 +667,7 @@ class coro_rpc_client { } struct async_rpc_raw_result_value_type { - std::variant buffer_; + rpc_resp_buffer buffer_; uint8_t errc_; }; @@ -671,24 +678,15 @@ class coro_rpc_client { struct handler_t { std::unique_ptr timer_; - control_t *control_; async_simple::Promise promise_; handler_t(std::unique_ptr &&timer, - control_t *control, async_simple::Promise &&promise) : timer_(std::move(timer)), - control_(control), promise_(std::move(promise)) {} void operator()(rpc_resp_buffer &&buffer, uint8_t rpc_errc) { timer_->cancel(); - if (control_) /*is waiting for response*/ { - promise_.setValue(async_rpc_raw_result{async_rpc_raw_result_value_type{ - std::string_view{control_->resp_buffer_.read_buf_}, rpc_errc}}); - } - else { - promise_.setValue(async_rpc_raw_result{ - async_rpc_raw_result_value_type{std::move(buffer), rpc_errc}}); - } + promise_.setValue(async_rpc_raw_result{ + async_rpc_raw_result_value_type{std::move(buffer), rpc_errc}}); } void local_error(std::error_code &ec) { timer_->cancel(); @@ -738,7 +736,7 @@ class coro_rpc_client { } template - rpc_result()), coro_rpc_protocol> sync_call( + rpc_result())> sync_call( Args &&...args) { return async_simple::coro::syncAwait( call(std::forward(args)...)); @@ -866,7 +864,7 @@ class coro_rpc_client { } template - static async_simple::coro::Lazy, rpc_error>> + static async_simple::coro::Lazy> get_deserializer(async_simple::Future future, std::weak_ptr watcher) { auto ret_ = co_await std::move(future); @@ -885,16 +883,9 @@ class coro_rpc_client { } bool should_close = false; - std::string_view buffer_view; auto &ret = std::get<0>(ret_); - if (ret.buffer_.index() == 0) { - buffer_view = std::get<0>(ret.buffer_).read_buf_; - } - else { - buffer_view = std::get<1>(ret.buffer_); - } auto result = - handle_response_buffer(buffer_view, ret.errc_, should_close); + handle_response_buffer(ret.buffer_.read_buf_, ret.errc_, should_close); if (should_close) { if (auto w = watcher.lock(); w) { close_socket(std::move(w)); @@ -902,21 +893,11 @@ class coro_rpc_client { } if (result) { if constexpr (std::is_same_v) { - if (ret.buffer_.index() == 0) { - co_return async_rpc_result{std::move(std::get<0>(ret.buffer_))}; - } - else { - co_return async_rpc_result{}; - } + co_return async_rpc_result{async_rpc_result_value_t{std::move(ret.buffer_)}}; } else { - if (ret.buffer_.index() == 0) { - co_return async_rpc_result{result.value(), - std::move(std::get<0>(ret.buffer_))}; - } - else { - co_return async_rpc_result{result.value()}; - } + co_return async_rpc_result{async_rpc_result_value_t{std::move(result.value()), + std::move(ret.buffer_)}}; } } else { @@ -926,30 +907,28 @@ class coro_rpc_client { public: template - async_simple::coro::Lazy())>, rpc_error>>, - rpc_error>> + async_simple::coro::Lazy< + async_simple::coro::Lazy< + async_rpc_result())>>> send_request(Args &&...args) { + return send_request_for_with_attachment(std::chrono::seconds{5},{}, std::forward(args)...); } template - async_simple::coro::Lazy())>, rpc_error>>, - rpc_error>> + async_simple::coro::Lazy< + async_simple::coro::Lazy< + async_rpc_result())>>> send_request_with_attachment(std::string_view request_attachment,Args &&...args) { return send_request_for_with_attachment(std::chrono::seconds{5},request_attachment, std::forward(args)...); } template - async_simple::coro::Lazy())>, rpc_error>>, - rpc_error>> + async_simple::coro::Lazy< + async_simple::coro::Lazy< + async_rpc_result())>>> send_request_for(Args &&...args) { return send_request_for_with_attachment(std::chrono::seconds{5},std::string_view{},std::forward(args)...); } @@ -968,17 +947,20 @@ class coro_rpc_client { } control_t* ctrl_; }; - +private: + template + static async_simple::coro::Lazy> + build_failed_rpc_result(rpc_error err) { + co_return unexpected{err}; + } +public: template - async_simple::coro::Lazy())>, rpc_error>>, - rpc_error>> + async_simple::coro::Lazy< + async_simple::coro::Lazy< + async_rpc_result())>>> send_request_for_with_attachment(auto time_out_duration, std::string_view request_attachment, Args &&...args) { + using rpc_return_t = decltype(get_return_type()); recving_guard guard(control_.get()); - // if (control_->recving_cnt_ > 1) { - // ELOG_ERROR<<"SHIT2:"<recving_cnt_<<"addr:"<<(void*)this; - // } uint32_t id; auto timer = std::make_unique(control_->executor_.get_asio_executor()); auto result = co_await send_request_for_impl( @@ -987,17 +969,13 @@ class coro_rpc_client { if (!result) { async_simple::Promise promise; auto future = promise.getFuture(); - bool is_waiting_for_response = is_waiting_for_response_; - is_waiting_for_response_ = false; bool is_empty = control.response_handler_table_.empty(); auto &&[_, is_ok] = control.response_handler_table_.try_emplace( id, std::move(timer), - is_waiting_for_response ? control_.get() : nullptr, std::move(promise)); if (!is_ok) [[unlikely]] { close(); - err_code ec = errc::serial_number_conflict; - co_return coro_rpc::unexpected{ec}; + co_return build_failed_rpc_result(rpc_error{coro_rpc::errc::serial_number_conflict}); } else { if (is_empty) { @@ -1016,12 +994,13 @@ class coro_rpc_client { #endif } guard.release(); - co_return get_deserializer())>( + co_return get_deserializer( std::move(future), std::weak_ptr{control_}); } } else { - co_return coro_rpc::unexpected{std::move(result)}; + auto i=build_failed_rpc_result(std::move(result)); + co_return build_failed_rpc_result(std::move(result)); } } @@ -1076,13 +1055,11 @@ class coro_rpc_client { else { #endif if (req_attachment.empty()) { - // auto loc = co_await write_mutex_.coScopedLock(); while (true) { bool expected = false; if (write_mutex_.compare_exchange_weak(expected,true)) { break; } - //ELOG_ERROR<<"SHIT"; co_await coro_io::post([](){},&control_->executor_); } ret = co_await coro_io::async_write( @@ -1093,13 +1070,11 @@ class coro_rpc_client { std::array iov{ asio::const_buffer{buffer.data(), buffer.size()}, asio::const_buffer{req_attachment.data(), req_attachment.size()}}; - // auto loc = co_await write_mutex_.coScopedLock(); while (true) { bool expected = false; if (write_mutex_.compare_exchange_weak(expected,true)) { break; } - //ELOG_ERROR<<"SHIT"; co_await coro_io::post([](){},&control_->executor_); } ret = co_await coro_io::async_write(socket, iov); @@ -1135,7 +1110,6 @@ class coro_rpc_client { private: std::atomic write_mutex_=false; - std::atomic is_waiting_for_response_ = false; std::atomic request_id_{0}; std::unique_ptr timer_; std::shared_ptr control_; diff --git a/include/ylt/coro_rpc/impl/expected.hpp b/include/ylt/coro_rpc/impl/expected.hpp index bddbe2ff8..dc786f780 100644 --- a/include/ylt/coro_rpc/impl/expected.hpp +++ b/include/ylt/coro_rpc/impl/expected.hpp @@ -48,8 +48,7 @@ namespace protocol { struct coro_rpc_protocol; } -template +template using rpc_result = expected; } // namespace coro_rpc \ No newline at end of file diff --git a/src/coro_rpc/examples/base_examples/channel.cpp b/src/coro_rpc/examples/base_examples/channel.cpp index dad17ac3f..8ae855f4b 100644 --- a/src/coro_rpc/examples/base_examples/channel.cpp +++ b/src/coro_rpc/examples/base_examples/channel.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -32,6 +33,7 @@ #include "ylt/coro_io/io_context_pool.hpp" #include "ylt/coro_rpc/impl/coro_rpc_client.hpp" +#include "ylt/coro_rpc/impl/errno.h" #include "ylt/coro_rpc/impl/expected.hpp" #include "ylt/easylog.hpp" using namespace coro_rpc; @@ -84,22 +86,20 @@ Lazy> call_echo(std::shared_ptrsend_request( - [](coro_rpc_client &client, std::string_view hostname) -> Lazy, coro_rpc::rpc_error>>, coro_rpc::rpc_error>> { - co_return co_await client.send_request("Hello world!"); + [](coro_rpc_client &client, std::string_view hostname) { + return client.send_request("Hello world!"); }); if (!res) { - ELOG_ERROR << "client pool err: connect failed.\n"; + ELOG_ERROR << "client pool err: connect failed.\n"<result()!="Hello world!") { + ELOG_ERROR << "error rpc reponse\n"<result(); } ++qps; auto old_tp=tp; diff --git a/src/coro_rpc/tests/test_router.cpp b/src/coro_rpc/tests/test_router.cpp index 9c7aaac0d..d50cb5e2e 100644 --- a/src/coro_rpc/tests/test_router.cpp +++ b/src/coro_rpc/tests/test_router.cpp @@ -67,12 +67,11 @@ struct RPC_trait { }; using coro_rpc_protocol = coro_rpc::protocol::coro_rpc_protocol; template -rpc_result, coro_rpc_protocol> +rpc_result> get_result(const auto &pair) { auto &&[rpc_errc, buffer] = pair; using T = util::function_return_type_t; - using return_type = rpc_result, - coro_rpc_protocol>; + using return_type = rpc_result>; rpc_return_type_t ret; struct_pack::err_code ec; rpc_error err; From 62d649c34e149b2707ae1612e5c0a83a9fce056b Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Fri, 17 May 2024 11:24:47 +0800 Subject: [PATCH 19/22] use connect() instead of reconnect() --- include/ylt/coro_io/client_pool.hpp | 2 +- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 59 ++++--------------- .../standalone/cinatra/coro_http_client.hpp | 12 ++-- src/coro_io/tests/test_client_pool.cpp | 2 +- 4 files changed, 20 insertions(+), 55 deletions(-) diff --git a/include/ylt/coro_io/client_pool.hpp b/include/ylt/coro_io/client_pool.hpp index 2f44949f2..b8760ab52 100644 --- a/include/ylt/coro_io/client_pool.hpp +++ b/include/ylt/coro_io/client_pool.hpp @@ -121,7 +121,7 @@ class client_pool : public std::enable_shared_from_this< << "}, try count:" << i << "max retry limit:" << pool_config_.connect_retry_count; auto pre_time_point = std::chrono::steady_clock::now(); - bool ok = client_t::is_ok(co_await client->reconnect(host_name_)); + bool ok = client_t::is_ok(co_await client->connect(host_name_)); auto post_time_point = std::chrono::steady_clock::now(); auto cost_time = post_time_point - pre_time_point; ELOG_DEBUG << "reconnect client{" << client.get() diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index b7d5e5541..b902c87d6 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -178,12 +178,12 @@ class coro_rpc_client { * @param io_context asio io_context, async event handler */ coro_rpc_client( - coro_io::ExecutorWrapper<> &executor = *coro_io::get_global_executor(), + coro_io::ExecutorWrapper<> *executor = coro_io::get_global_executor(), uint32_t client_id = 0) : control_( - std::make_shared(executor.get_asio_executor(), false)), + std::make_shared(executor->get_asio_executor(), false)), timer_(std::make_unique( - executor.get_asio_executor())) { + executor->get_asio_executor())) { config_.client_id = client_id; } @@ -208,41 +208,6 @@ class coro_rpc_client { */ [[nodiscard]] bool has_closed() { return control_->has_closed_; } - /*! - * Reconnect server - * - * If connect hasn't been closed, it will be closed first then connect to - * server, else the client will connect to server directly - * - * @param host server address - * @param port server port - * @param timeout_duration RPC call timeout - * @return error code - */ - [[nodiscard]] async_simple::coro::Lazy reconnect( - std::string host, std::string port, - std::chrono::steady_clock::duration timeout_duration = - std::chrono::seconds(5)) { - config_.host = std::move(host); - config_.port = std::move(port); - config_.timeout_duration = - std::chrono::duration_cast(timeout_duration); - reset(); - return connect(is_reconnect_t{true}); - } - - [[nodiscard]] async_simple::coro::Lazy reconnect( - std::string endpoint, - std::chrono::steady_clock::duration timeout_duration = - std::chrono::seconds(5)) { - auto pos = endpoint.find(':'); - config_.host = endpoint.substr(0, pos); - config_.port = endpoint.substr(pos + 1); - config_.timeout_duration = - std::chrono::duration_cast(timeout_duration); - reset(); - return connect(is_reconnect_t{true}); - } /*! * Connect server * @@ -387,22 +352,19 @@ class coro_rpc_client { control_->has_closed_ = false; } static bool is_ok(coro_rpc::err_code ec) noexcept { return !ec; } - [[nodiscard]] async_simple::coro::Lazy connect( - is_reconnect_t is_reconnect = is_reconnect_t{false}) { + [[nodiscard]] async_simple::coro::Lazy connect() { + if (should_reset_) { + reset(); + } + else { + should_reset_ = true; + } #ifdef YLT_ENABLE_SSL if (!ssl_init_ret_) { std::cout << "ssl_init_ret_: " << ssl_init_ret_ << std::endl; co_return errc::not_connected; } #endif - if (!is_reconnect.value && control_->has_closed_) - AS_UNLIKELY { - ELOGV(ERROR, - "a closed client is not allowed connect again, please use " - "reconnect function or create a new " - "client"); - co_return errc::io_error; - } control_->has_closed_ = false; ELOGV(INFO, "client_id %d begin to connect %s", config_.client_id, @@ -1109,6 +1071,7 @@ class coro_rpc_client { } private: + bool should_reset_ = false; std::atomic write_mutex_=false; std::atomic request_id_{0}; std::unique_ptr timer_; diff --git a/include/ylt/standalone/cinatra/coro_http_client.hpp b/include/ylt/standalone/cinatra/coro_http_client.hpp index 62df486c1..a12e7fab8 100644 --- a/include/ylt/standalone/cinatra/coro_http_client.hpp +++ b/include/ylt/standalone/cinatra/coro_http_client.hpp @@ -284,6 +284,12 @@ class coro_http_client : public std::enable_shared_from_this { // only make socket connet(or handshake) to the host async_simple::coro::Lazy connect(std::string uri) { + if (should_reset_) { + reset(); + } + else { + should_reset_ = false; + } resp_data data{}; bool no_schema = !has_schema(uri); std::string append_uri; @@ -896,11 +902,6 @@ class coro_http_client : public std::enable_shared_from_this { resp_chunk_str_.clear(); } - async_simple::coro::Lazy reconnect(std::string uri) { - reset(); - co_return co_await connect(std::move(uri)); - } - std::string_view get_host() { return host_; } std::string_view get_port() { return port_; } @@ -2119,6 +2120,7 @@ class coro_http_client : public std::enable_shared_from_this { std::string redirect_uri_; bool enable_follow_redirect_ = false; bool enable_timeout_ = false; + bool should_reset_ = false; std::chrono::steady_clock::duration conn_timeout_duration_ = std::chrono::seconds(8); std::chrono::steady_clock::duration req_timeout_duration_ = diff --git a/src/coro_io/tests/test_client_pool.cpp b/src/coro_io/tests/test_client_pool.cpp index 3a9eff4ac..4da5b6884 100644 --- a/src/coro_io/tests/test_client_pool.cpp +++ b/src/coro_io/tests/test_client_pool.cpp @@ -158,7 +158,7 @@ struct mock_client : public coro_rpc::coro_rpc_client { using coro_rpc::coro_rpc_client::coro_rpc_client; async_simple::coro::Lazy reconnect( const std::string &hostname) { - auto ec = co_await this->coro_rpc::coro_rpc_client::reconnect(hostname); + auto ec = co_await this->coro_rpc::coro_rpc_client::connect(hostname); if (ec) { co_await coro_io::sleep_for(300ms); } From 634de16adfa2fb7c19b3c6dff54259b67dbfddc7 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Fri, 17 May 2024 15:51:07 +0800 Subject: [PATCH 20/22] fix reconnect --- include/ylt/coro_io/client_pool.hpp | 73 ++++++++++----------- src/coro_io/tests/test_client_pool.cpp | 4 +- src/coro_rpc/tests/test_coro_rpc_server.cpp | 7 -- 3 files changed, 37 insertions(+), 47 deletions(-) diff --git a/include/ylt/coro_io/client_pool.hpp b/include/ylt/coro_io/client_pool.hpp index b8760ab52..7444fd673 100644 --- a/include/ylt/coro_io/client_pool.hpp +++ b/include/ylt/coro_io/client_pool.hpp @@ -78,11 +78,11 @@ class client_pool : public std::enable_shared_from_this< break; } while (true) { - ELOG_DEBUG << "start collect timeout client of pool{" + ELOG_TRACE << "start collect timeout client of pool{" << self->host_name_ << "}, now client count: " << clients.size(); std::size_t is_all_cleared = clients.clear_old(clear_cnt); - ELOG_DEBUG << "finish collect timeout client of pool{" + ELOG_TRACE << "finish collect timeout client of pool{" << self->host_name_ << "}, now client cnt: " << clients.size(); if (is_all_cleared != 0) [[unlikely]] { @@ -109,36 +109,42 @@ class client_pool : public std::enable_shared_from_this< static auto rand_time(std::chrono::milliseconds ms) { static thread_local std::default_random_engine r; - std::uniform_real_distribution e(0.7f, 1.3f); + std::uniform_real_distribution e(1.0f, 1.2f); return std::chrono::milliseconds{static_cast(e(r) * ms.count())}; } - async_simple::coro::Lazy reconnect(std::unique_ptr& client) { + static async_simple::coro::Lazy reconnect(std::unique_ptr& client, std::weak_ptr watcher) { using namespace std::chrono_literals; - for (unsigned int i = 0; i < pool_config_.connect_retry_count; ++i) { - ELOG_DEBUG << "try to reconnect client{" << client.get() << "},host:{" + std::shared_ptr self = watcher.lock(); + uint32_t i = UINT32_MAX; // (at least connect once) + do { + ELOG_TRACE << "try to reconnect client{" << client.get() << "},host:{" << client->get_host() << ":" << client->get_port() - << "}, try count:" << i - << "max retry limit:" << pool_config_.connect_retry_count; + << "}, try count:" << i << "max retry limit:" + << self->pool_config_.connect_retry_count; auto pre_time_point = std::chrono::steady_clock::now(); - bool ok = client_t::is_ok(co_await client->connect(host_name_)); + bool ok = client_t::is_ok(co_await client->connect(self->host_name_)); auto post_time_point = std::chrono::steady_clock::now(); auto cost_time = post_time_point - pre_time_point; - ELOG_DEBUG << "reconnect client{" << client.get() + ELOG_TRACE << "reconnect client{" << client.get() << "} cost time: " << cost_time / std::chrono::milliseconds{1} << "ms"; if (ok) { - ELOG_DEBUG << "reconnect client{" << client.get() << "} success"; + ELOG_TRACE << "reconnect client{" << client.get() << "} success"; co_return; } - ELOG_DEBUG << "reconnect client{" << client.get() + ELOG_TRACE << "reconnect client{" << client.get() << "} failed. If client close:{" << client->has_closed() << "}"; auto wait_time = rand_time( - (pool_config_.reconnect_wait_time * (i + 1) - cost_time) / 1ms * 1ms); + (self->pool_config_.reconnect_wait_time - cost_time) / 1ms * + 1ms); + self = nullptr; if (wait_time.count() > 0) co_await coro_io::sleep_for(wait_time, &client->get_executor()); - } + self = watcher.lock(); + ++i; + } while (i < self->pool_config_.connect_retry_count); ELOG_WARN << "reconnect client{" << client.get() << "},host:{" << client->get_host() << ":" << client->get_port() << "} out of max limit, stop retry. connect failed"; @@ -150,30 +156,21 @@ class client_pool : public std::enable_shared_from_this< async_simple::Promise> promise_; }; - async_simple::coro::Lazy connect_client( + static async_simple::coro::Lazy connect_client( std::unique_ptr client, std::weak_ptr watcher, std::shared_ptr handler) { - ELOG_DEBUG << "try to connect client{" << client.get() - << "} to host:" << host_name_; - auto result = co_await client->connect(host_name_); - std::shared_ptr self = watcher.lock(); - if (!client_t::is_ok(result)) { - ELOG_DEBUG << "connect client{" << client.get() << "} to failed. "; - if (self) { - co_await reconnect(client); - } - } - if (client) { - ELOG_DEBUG << "connect client{" << client.get() << "} successful!"; - } + co_await reconnect(client,watcher); auto has_get_connect = handler->flag_.exchange(true); if (!has_get_connect) { handler->promise_.setValue(std::move(client)); } else { - auto conn_lim = std::min(10u, pool_config_.max_connection); - if (self && free_clients_.size() < conn_lim && client) { - enqueue(free_clients_, std::move(client), pool_config_.idle_timeout); + if (client) { + auto self = watcher.lock(); + auto conn_lim = std::min(10u, self->pool_config_.max_connection); + if (self && self->free_clients_.size() < conn_lim ) { + self->enqueue(self->free_clients_, std::move(client), self->pool_config_.idle_timeout); + } } } } @@ -226,7 +223,7 @@ class client_pool : public std::enable_shared_from_this< } } }); - ELOG_DEBUG << "wait client by promise {" << &handler->promise_ << "}"; + ELOG_TRACE << "wait client by promise {" << &handler->promise_ << "}"; client = co_await handler->promise_.getFuture(); if (client) { executor->schedule([timer] { @@ -236,7 +233,7 @@ class client_pool : public std::enable_shared_from_this< } } else { - ELOG_DEBUG << "get free client{" << client.get() << "}. from queue"; + ELOG_TRACE << "get free client{" << client.get() << "}. from queue"; } co_return std::move(client); } @@ -248,7 +245,7 @@ class client_pool : public std::enable_shared_from_this< if (clients.enqueue(std::move(client)) == 1) { std::size_t expected = 0; if (clients.collecter_cnt_.compare_exchange_strong(expected, 1)) { - ELOG_DEBUG << "start timeout client collecter of client_pool{" + ELOG_TRACE << "start timeout client collecter of client_pool{" << host_name_ << "}"; collect_idle_timeout_client( this->weak_from_this(), clients, @@ -272,7 +269,7 @@ class client_pool : public std::enable_shared_from_this< if (!has_get_connect) { handler->promise_.setValue(std::move(client)); promise_cnt_ -= cnt; - ELOG_DEBUG << "collect free client{" << client.get() + ELOG_TRACE << "collect free client{" << client.get() << "} and wake up promise{" << &handler->promise_ << "}"; return; } @@ -282,12 +279,12 @@ class client_pool : public std::enable_shared_from_this< if (free_clients_.size() < pool_config_.max_connection) { if (client) { - ELOG_DEBUG << "collect free client{" << client.get() << "} enqueue"; + ELOG_TRACE << "collect free client{" << client.get() << "} enqueue"; enqueue(free_clients_, std::move(client), pool_config_.idle_timeout); } } else { - ELOG_DEBUG << "out of max connection limit <<" + ELOG_TRACE << "out of max connection limit <<" << pool_config_.max_connection << ", collect free client{" << client.get() << "} enqueue short connect queue"; enqueue(short_connect_clients_, std::move(client), @@ -295,7 +292,7 @@ class client_pool : public std::enable_shared_from_this< } } else { - ELOG_DEBUG << "client{" << client.get() + ELOG_TRACE << "client{" << client.get() << "} is closed. we won't collect it"; } diff --git a/src/coro_io/tests/test_client_pool.cpp b/src/coro_io/tests/test_client_pool.cpp index 4da5b6884..18d00ee5d 100644 --- a/src/coro_io/tests/test_client_pool.cpp +++ b/src/coro_io/tests/test_client_pool.cpp @@ -156,7 +156,7 @@ TEST_CASE("test reconnect") { struct mock_client : public coro_rpc::coro_rpc_client { using coro_rpc::coro_rpc_client::coro_rpc_client; - async_simple::coro::Lazy reconnect( + async_simple::coro::Lazy connect( const std::string &hostname) { auto ec = co_await this->coro_rpc::coro_rpc_client::connect(hostname); if (ec) { @@ -184,7 +184,7 @@ TEST_CASE("test reconnect retry wait time exclude reconnect cost time") { CHECK(pool->free_client_count() == 100); auto dur = std::chrono::steady_clock::now() - tp; std::cout << dur.count() << std::endl; - CHECK((dur >= 400ms && dur <= 800ms)); + CHECK((dur >= 500ms && dur <= 799ms)); server.stop(); co_return; }()); diff --git a/src/coro_rpc/tests/test_coro_rpc_server.cpp b/src/coro_rpc/tests/test_coro_rpc_server.cpp index 6cf512043..23f522204 100644 --- a/src/coro_rpc/tests/test_coro_rpc_server.cpp +++ b/src/coro_rpc/tests/test_coro_rpc_server.cpp @@ -346,13 +346,6 @@ TEST_CASE("test server accept error") { REQUIRE_MESSAGE(ret.error().code == coro_rpc::errc::io_error, ret.error().msg); REQUIRE(client.has_closed() == true); - - ec = syncAwait(client.connect("127.0.0.1", "8810")); - REQUIRE_MESSAGE(ec == coro_rpc::errc::io_error, - std::to_string(client.get_client_id()).append(ec.message())); - ret = syncAwait(client.call()); - CHECK(!ret); - REQUIRE(client.has_closed() == true); g_action = {}; } From 8387a820a19d533fd0b354ef8e616dbfe14aefce Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Fri, 17 May 2024 16:53:14 +0800 Subject: [PATCH 21/22] format --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 222 +++++++++++------- 1 file changed, 132 insertions(+), 90 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index b902c87d6..ddcf0f5d5 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -95,27 +95,32 @@ struct rpc_resp_buffer { template struct async_rpc_result_value_t { -private: + private: T result_; rpc_resp_buffer buffer_; -public: - async_rpc_result_value_t(T&& result,rpc_resp_buffer&& buffer): result_(std::move(result)),buffer_(std::move(buffer)){} - async_rpc_result_value_t(T&& result): result_(std::move(result)){} - T& result() noexcept {return result_;} - const T& result() const noexcept {return result_;} - std::string_view attachment() const noexcept {return buffer_.resp_attachment_buf_;} - rpc_resp_buffer release_buffer() {return std::move(buffer_);} -}; + public: + async_rpc_result_value_t(T &&result, rpc_resp_buffer &&buffer) + : result_(std::move(result)), buffer_(std::move(buffer)) {} + async_rpc_result_value_t(T &&result) : result_(std::move(result)) {} + T &result() noexcept { return result_; } + const T &result() const noexcept { return result_; } + std::string_view attachment() const noexcept { + return buffer_.resp_attachment_buf_; + } + rpc_resp_buffer release_buffer() { return std::move(buffer_); } +}; template <> struct async_rpc_result_value_t { rpc_resp_buffer buffer_; - std::string_view attachment() const noexcept {return buffer_.resp_attachment_buf_;} - rpc_resp_buffer release_buffer() {return std::move(buffer_);} + std::string_view attachment() const noexcept { + return buffer_.resp_attachment_buf_; + } + rpc_resp_buffer release_buffer() { return std::move(buffer_); } }; -template +template using async_rpc_result = expected, rpc_error>; template @@ -191,6 +196,8 @@ class coro_rpc_client { std::string_view get_port() const { return config_.port; } + const config &get_config() const { return config_; } + [[nodiscard]] bool init_config(const config &conf) { config_ = conf; #ifdef YLT_ENABLE_SSL @@ -223,22 +230,50 @@ class coro_rpc_client { std::string host, std::string port, std::chrono::steady_clock::duration timeout_duration = std::chrono::seconds(5)) { + auto lock_ok = connect_mutex_.tryLock(); + if (!lock_ok) { + co_await connect_mutex_.coScopedLock(); + co_return err_code{}; + // do nothing, someone has reconnect the client + } config_.host = std::move(host); config_.port = std::move(port); config_.timeout_duration = std::chrono::duration_cast(timeout_duration); - return connect(); + auto ret = co_await connect_impl(); + connect_mutex_.unlock(); + co_return std::move(ret); } [[nodiscard]] async_simple::coro::Lazy connect( std::string_view endpoint, std::chrono::steady_clock::duration timeout_duration = std::chrono::seconds(5)) { auto pos = endpoint.find(':'); + auto lock_ok = connect_mutex_.tryLock(); + if (!lock_ok) { + co_await connect_mutex_.coScopedLock(); + co_return err_code{}; + // do nothing, someone has reconnect the client + } config_.host = endpoint.substr(0, pos); config_.port = endpoint.substr(pos + 1); config_.timeout_duration = std::chrono::duration_cast(timeout_duration); - return connect(); + auto ret = co_await connect_impl(); + connect_mutex_.unlock(); + co_return std::move(ret); + } + + [[nodiscard]] async_simple::coro::Lazy connect() { + auto lock_ok = connect_mutex_.tryLock(); + if (!lock_ok) { + co_await connect_mutex_.coScopedLock(); + co_return err_code{}; + // do nothing, someone has reconnect the client + } + auto ret = co_await connect_impl(); + connect_mutex_.unlock(); + co_return std::move(ret); } #ifdef YLT_ENABLE_SSL @@ -264,9 +299,8 @@ class coro_rpc_client { * @return RPC call result */ template - async_simple::coro::Lazy< - rpc_result())>> - call(Args &&...args) { + async_simple::coro::Lazy())>> call( + Args &&...args) { return call_for(std::chrono::seconds(5), std::forward(args)...); } @@ -282,12 +316,12 @@ class coro_rpc_client { * @return RPC call result */ template - async_simple::coro::Lazy< - rpc_result())>> + async_simple::coro::Lazy())>> call_for(auto duration, Args &&...args) { using return_type = decltype(get_return_type()); - auto async_result = co_await co_await send_request_for_with_attachment( - duration, req_attachment_, std::forward(args)...); + auto async_result = + co_await co_await send_request_for_with_attachment( + duration, req_attachment_, std::forward(args)...); req_attachment_ = {}; if (async_result) { control_->resp_buffer_ = async_result->release_buffer(); @@ -352,7 +386,8 @@ class coro_rpc_client { control_->has_closed_ = false; } static bool is_ok(coro_rpc::err_code ec) noexcept { return !ec; } - [[nodiscard]] async_simple::coro::Lazy connect() { + + [[nodiscard]] async_simple::coro::Lazy connect_impl() { if (should_reset_) { reset(); } @@ -549,8 +584,9 @@ class coro_rpc_client { } template - static rpc_result handle_response_buffer( - std::string_view buffer, uint8_t rpc_errc, bool &should_close) { + static rpc_result handle_response_buffer(std::string_view buffer, + uint8_t rpc_errc, + bool &should_close) { rpc_return_type_t ret; struct_pack::err_code ec; rpc_error err; @@ -643,8 +679,7 @@ class coro_rpc_client { async_simple::Promise promise_; handler_t(std::unique_ptr &&timer, async_simple::Promise &&promise) - : timer_(std::move(timer)), - promise_(std::move(promise)) {} + : timer_(std::move(timer)), promise_(std::move(promise)) {} void operator()(rpc_resp_buffer &&buffer, uint8_t rpc_errc) { timer_->cancel(); promise_.setValue(async_rpc_raw_result{ @@ -678,10 +713,10 @@ class coro_rpc_client { static void close_socket( std::shared_ptr control) { - if (control->has_closed_) { + bool expected = false; + if (!control->has_closed_.compare_exchange_strong(expected, true)) { return; } - control->has_closed_ = true; control->executor_.schedule([control = std::move(control)]() { asio::error_code ignored_ec; control->socket_.shutdown(asio::ip::tcp::socket::shutdown_both, @@ -698,17 +733,16 @@ class coro_rpc_client { } template - rpc_result())> sync_call( - Args &&...args) { + rpc_result())> sync_call(Args &&...args) { return async_simple::coro::syncAwait( call(std::forward(args)...)); } #endif template - async_simple::coro::Lazy send_request_for_impl(auto duration, - uint32_t &id,coro_io::period_timer& timer, std::string_view attachment, - Args &&...args) { + async_simple::coro::Lazy send_request_for_impl( + auto duration, uint32_t &id, coro_io::period_timer &timer, + std::string_view attachment, Args &&...args) { using R = decltype(get_return_type()); if (control_->has_closed_) @@ -734,12 +768,12 @@ class coro_rpc_client { #ifdef YLT_ENABLE_SSL if (!config_.ssl_cert_path.empty()) { assert(control_->ssl_stream_); - co_return co_await send_impl(*control_->ssl_stream_, id,attachment, + co_return co_await send_impl(*control_->ssl_stream_, id, attachment, std::forward(args)...); } else { #endif - co_return co_await send_impl(control_->socket_, id,attachment, + co_return co_await send_impl(control_->socket_, id, attachment, std::forward(args)...); #ifdef YLT_ENABLE_SSL } @@ -826,9 +860,9 @@ class coro_rpc_client { } template - static async_simple::coro::Lazy> - get_deserializer(async_simple::Future future, - std::weak_ptr watcher) { + static async_simple::coro::Lazy> get_deserializer( + async_simple::Future future, + std::weak_ptr watcher) { auto ret_ = co_await std::move(future); if (ret_.index() == 1) [[unlikely]] { // local error @@ -846,8 +880,8 @@ class coro_rpc_client { bool should_close = false; auto &ret = std::get<0>(ret_); - auto result = - handle_response_buffer(ret.buffer_.read_buf_, ret.errc_, should_close); + auto result = handle_response_buffer(ret.buffer_.read_buf_, ret.errc_, + should_close); if (should_close) { if (auto w = watcher.lock(); w) { close_socket(std::move(w)); @@ -855,11 +889,12 @@ class coro_rpc_client { } if (result) { if constexpr (std::is_same_v) { - co_return async_rpc_result{async_rpc_result_value_t{std::move(ret.buffer_)}}; + co_return async_rpc_result{ + async_rpc_result_value_t{std::move(ret.buffer_)}}; } else { - co_return async_rpc_result{async_rpc_result_value_t{std::move(result.value()), - std::move(ret.buffer_)}}; + co_return async_rpc_result{async_rpc_result_value_t{ + std::move(result.value()), std::move(ret.buffer_)}}; } } else { @@ -869,75 +904,76 @@ class coro_rpc_client { public: template - async_simple::coro::Lazy< - async_simple::coro::Lazy< - async_rpc_result())>>> + async_simple::coro::Lazy())>>> send_request(Args &&...args) { - - return send_request_for_with_attachment(std::chrono::seconds{5},{}, - std::forward(args)...); + return send_request_for_with_attachment(std::chrono::seconds{5}, {}, + std::forward(args)...); } template - async_simple::coro::Lazy< - async_simple::coro::Lazy< - async_rpc_result())>>> - send_request_with_attachment(std::string_view request_attachment,Args &&...args) { - return send_request_for_with_attachment(std::chrono::seconds{5},request_attachment, - std::forward(args)...); + async_simple::coro::Lazy())>>> + send_request_with_attachment(std::string_view request_attachment, + Args &&...args) { + return send_request_for_with_attachment(std::chrono::seconds{5}, + request_attachment, + std::forward(args)...); } template - async_simple::coro::Lazy< - async_simple::coro::Lazy< - async_rpc_result())>>> + async_simple::coro::Lazy())>>> send_request_for(Args &&...args) { - return send_request_for_with_attachment(std::chrono::seconds{5},std::string_view{},std::forward(args)...); + return send_request_for_with_attachment(std::chrono::seconds{5}, + std::string_view{}, + std::forward(args)...); } struct recving_guard { - recving_guard(control_t* ctrl):ctrl_(ctrl){ - ctrl_->recving_cnt_++; - } - void release() { - ctrl_=nullptr; - } + recving_guard(control_t *ctrl) : ctrl_(ctrl) { ctrl_->recving_cnt_++; } + void release() { ctrl_ = nullptr; } ~recving_guard() { if (ctrl_) { --ctrl_->recving_cnt_; } } - control_t* ctrl_; + control_t *ctrl_; }; -private: + + private: template - static async_simple::coro::Lazy> - build_failed_rpc_result(rpc_error err) { + static async_simple::coro::Lazy> build_failed_rpc_result( + rpc_error err) { co_return unexpected{err}; } -public: + + public: template - async_simple::coro::Lazy< - async_simple::coro::Lazy< - async_rpc_result())>>> - send_request_for_with_attachment(auto time_out_duration, std::string_view request_attachment, Args &&...args) { - using rpc_return_t = decltype(get_return_type()); + async_simple::coro::Lazy())>>> + send_request_for_with_attachment(auto time_out_duration, + std::string_view request_attachment, + Args &&...args) { + using rpc_return_t = decltype(get_return_type()); recving_guard guard(control_.get()); uint32_t id; - auto timer = std::make_unique(control_->executor_.get_asio_executor()); + auto timer = std::make_unique( + control_->executor_.get_asio_executor()); auto result = co_await send_request_for_impl( - time_out_duration, id, *timer, request_attachment, std::forward(args)...); + time_out_duration, id, *timer, request_attachment, + std::forward(args)...); auto &control = *control_; if (!result) { async_simple::Promise promise; auto future = promise.getFuture(); bool is_empty = control.response_handler_table_.empty(); auto &&[_, is_ok] = control.response_handler_table_.try_emplace( - id, std::move(timer), - std::move(promise)); + id, std::move(timer), std::move(promise)); if (!is_ok) [[unlikely]] { close(); - co_return build_failed_rpc_result(rpc_error{coro_rpc::errc::serial_number_conflict}); + co_return build_failed_rpc_result( + rpc_error{coro_rpc::errc::serial_number_conflict}); } else { if (is_empty) { @@ -961,18 +997,17 @@ class coro_rpc_client { } } else { - auto i=build_failed_rpc_result(std::move(result)); + auto i = build_failed_rpc_result(std::move(result)); co_return build_failed_rpc_result(std::move(result)); } } - uint32_t get_pipeline_size() { - return control_->recving_cnt_; - } + uint32_t get_pipeline_size() { return control_->recving_cnt_; } private: template - async_simple::coro::Lazy send_impl(Socket &socket, uint32_t &id, std::string_view req_attachment, + async_simple::coro::Lazy send_impl(Socket &socket, uint32_t &id, + std::string_view req_attachment, Args &&...args) { auto buffer = prepare_buffer(id, std::forward(args)...); if (buffer.empty()) { @@ -1019,10 +1054,13 @@ class coro_rpc_client { if (req_attachment.empty()) { while (true) { bool expected = false; - if (write_mutex_.compare_exchange_weak(expected,true)) { + if (write_mutex_.compare_exchange_weak(expected, true)) { break; } - co_await coro_io::post([](){},&control_->executor_); + co_await coro_io::post( + []() { + }, + &control_->executor_); } ret = co_await coro_io::async_write( socket, asio::buffer(buffer.data(), buffer.size())); @@ -1034,10 +1072,13 @@ class coro_rpc_client { asio::const_buffer{req_attachment.data(), req_attachment.size()}}; while (true) { bool expected = false; - if (write_mutex_.compare_exchange_weak(expected,true)) { + if (write_mutex_.compare_exchange_weak(expected, true)) { break; } - co_await coro_io::post([](){},&control_->executor_); + co_await coro_io::post( + []() { + }, + &control_->executor_); } ret = co_await coro_io::async_write(socket, iov); write_mutex_ = false; @@ -1072,7 +1113,8 @@ class coro_rpc_client { private: bool should_reset_ = false; - std::atomic write_mutex_=false; + async_simple::coro::Mutex connect_mutex_; + std::atomic write_mutex_ = false; std::atomic request_id_{0}; std::unique_ptr timer_; std::shared_ptr control_; From 0f7f35a9c438a8f8788cdfbeb46d00f0f63c8dc9 Mon Sep 17 00:00:00 2001 From: "Zezheng.Li" Date: Fri, 17 May 2024 17:09:42 +0800 Subject: [PATCH 22/22] fix conversation --- include/ylt/coro_rpc/impl/coro_rpc_client.hpp | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp index ddcf0f5d5..1ad48a14b 100644 --- a/include/ylt/coro_rpc/impl/coro_rpc_client.hpp +++ b/include/ylt/coro_rpc/impl/coro_rpc_client.hpp @@ -88,7 +88,7 @@ struct rpc_return_type { using type = std::monostate; }; -struct rpc_resp_buffer { +struct resp_body { std::string read_buf_; std::string resp_attachment_buf_; }; @@ -97,10 +97,10 @@ template struct async_rpc_result_value_t { private: T result_; - rpc_resp_buffer buffer_; + resp_body buffer_; public: - async_rpc_result_value_t(T &&result, rpc_resp_buffer &&buffer) + async_rpc_result_value_t(T &&result, resp_body &&buffer) : result_(std::move(result)), buffer_(std::move(buffer)) {} async_rpc_result_value_t(T &&result) : result_(std::move(result)) {} T &result() noexcept { return result_; } @@ -108,16 +108,16 @@ struct async_rpc_result_value_t { std::string_view attachment() const noexcept { return buffer_.resp_attachment_buf_; } - rpc_resp_buffer release_buffer() { return std::move(buffer_); } + resp_body release_buffer() { return std::move(buffer_); } }; template <> struct async_rpc_result_value_t { - rpc_resp_buffer buffer_; + resp_body buffer_; std::string_view attachment() const noexcept { return buffer_.resp_attachment_buf_; } - rpc_resp_buffer release_buffer() { return std::move(buffer_); } + resp_body release_buffer() { return std::move(buffer_); } }; template @@ -586,7 +586,7 @@ class coro_rpc_client { template static rpc_result handle_response_buffer(std::string_view buffer, uint8_t rpc_errc, - bool &should_close) { + bool &has_error) { rpc_return_type_t ret; struct_pack::err_code ec; rpc_error err; @@ -607,7 +607,7 @@ class coro_rpc_client { err.val() = rpc_errc; ec = struct_pack::deserialize_to(err.msg, buffer); if SP_LIKELY (!ec) { - should_close = true; + has_error = true; return rpc_result{unexpect_t{}, std::move(err)}; } } @@ -618,7 +618,7 @@ class coro_rpc_client { } } } - should_close = true; + has_error = true; // deserialize failed. ELOGV(WARNING, "deserilaize rpc result failed"); err = {errc::invalid_rpc_result, "failed to deserialize rpc return value"}; @@ -665,7 +665,7 @@ class coro_rpc_client { } struct async_rpc_raw_result_value_type { - rpc_resp_buffer buffer_; + resp_body buffer_; uint8_t errc_; }; @@ -680,7 +680,7 @@ class coro_rpc_client { handler_t(std::unique_ptr &&timer, async_simple::Promise &&promise) : timer_(std::move(timer)), promise_(std::move(promise)) {} - void operator()(rpc_resp_buffer &&buffer, uint8_t rpc_errc) { + void operator()(resp_body &&buffer, uint8_t rpc_errc) { timer_->cancel(); promise_.setValue(async_rpc_raw_result{ async_rpc_raw_result_value_type{std::move(buffer), rpc_errc}}); @@ -701,7 +701,7 @@ class coro_rpc_client { std::atomic has_closed_ = false; coro_io::ExecutorWrapper<> executor_; std::unordered_map response_handler_table_; - rpc_resp_buffer resp_buffer_; + resp_body resp_buffer_; asio::ip::tcp::socket socket_; std::atomic recving_cnt_ = 0; control_t(asio::io_context::executor_type executor, bool is_timeout) @@ -860,7 +860,7 @@ class coro_rpc_client { } template - static async_simple::coro::Lazy> get_deserializer( + static async_simple::coro::Lazy> deserialize_rpc_result( async_simple::Future future, std::weak_ptr watcher) { auto ret_ = co_await std::move(future); @@ -878,11 +878,11 @@ class coro_rpc_client { } } - bool should_close = false; + bool has_error = false; auto &ret = std::get<0>(ret_); auto result = handle_response_buffer(ret.buffer_.read_buf_, ret.errc_, - should_close); - if (should_close) { + has_error); + if (has_error) { if (auto w = watcher.lock(); w) { close_socket(std::move(w)); } @@ -992,7 +992,7 @@ class coro_rpc_client { #endif } guard.release(); - co_return get_deserializer( + co_return deserialize_rpc_result( std::move(future), std::weak_ptr{control_}); } }