view: Turn calculate_view_update_throttling_delay into node_update_backlog member

The free function calculate_view_update_throttling_delay() took the
view_flow_control_delay_limit_in_ms as a parameter, which forced its
two callers (storage_proxy and view_update_generator) to fish the
option out of db::config via database::get_config(). Now that the
option lives on node_update_backlog, make the throttling calculation a
member of node_update_backlog and have the callers invoke it on their
node_update_backlog reference.

This removes two database::get_config() call sites.

Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>
This commit is contained in:
Pavel Emelyanov
2026-04-24 13:52:12 +03:00
parent 855372db3c
commit 111165d9de
5 changed files with 24 additions and 24 deletions

View File

@@ -63,6 +63,9 @@ public:
update_backlog fetch_shard(unsigned shard);
seastar::future<std::optional<update_backlog>> fetch_if_changed();
std::chrono::microseconds calculate_throttling_delay(update_backlog backlog,
db::timeout_clock::time_point timeout) const;
// Exposed for testing only.
update_backlog load() const {
return _max.load(std::memory_order_relaxed);

View File

@@ -45,6 +45,7 @@
#include "db/view/view_builder.hh"
#include "db/view/view_updating_consumer.hh"
#include "db/view/view_update_generator.hh"
#include "db/view/node_view_update_backlog.hh"
#include "db/view/regular_column_transformation.hh"
#include "db/system_keyspace_view_types.hh"
#include "db/system_keyspace.hh"
@@ -3492,18 +3493,27 @@ future<> delete_ghost_rows_visitor::do_accept_new_row(partition_key pk, clusteri
}
}
std::chrono::microseconds calculate_view_update_throttling_delay(db::view::update_backlog backlog,
db::timeout_clock::time_point timeout,
uint32_t view_flow_control_delay_limit_in_ms) {
// View updates are asynchronous, and because of this limiting their concurrency requires
// a special approach. The current algorithm places all of the pending view updates in the backlog
// and artificially slows down new responses to coordinator requests based on how full the backlog is.
// This function calculates how much a request should be slowed down based on the backlog's fullness.
// The equation is basically: delay(in seconds) = view_fullness_ratio^3
// The more full the backlog gets the more aggressively the requests are slowed down.
// The delay is limited to the amount of time left until timeout.
// After the timeout the request fails, so there's no point in waiting longer than that.
// The second argument defines this timeout point - we can't delay the request more than this time point.
// See: https://www.scylladb.com/2018/12/04/worry-free-ingestion-flow-control/
std::chrono::microseconds node_update_backlog::calculate_throttling_delay(update_backlog backlog,
db::timeout_clock::time_point timeout) const {
auto adjust = [] (float x) { return x * x * x; };
auto budget = std::max(service::storage_proxy::clock_type::duration(0),
timeout - service::storage_proxy::clock_type::now());
std::chrono::microseconds ret(uint32_t(adjust(backlog.relative_size()) * view_flow_control_delay_limit_in_ms * 1000));
auto budget = std::max(db::timeout_clock::duration(0),
timeout - db::timeout_clock::now());
std::chrono::microseconds ret(uint32_t(adjust(backlog.relative_size()) * _view_flow_control_delay_limit_in_ms() * 1000));
// "budget" has millisecond resolution and can potentially be long
// in the future so converting it to microseconds may overflow.
// So to compare buget and ret we need to convert both to the lower
// resolution.
if (std::chrono::duration_cast<service::storage_proxy::clock_type::duration>(ret) < budget) {
if (std::chrono::duration_cast<db::timeout_clock::duration>(ret) < budget) {
return ret;
} else {
// budget is small (< ret) so can be converted to microseconds

View File

@@ -43,7 +43,7 @@ public:
// Returns the number of bytes in the backlog divided by the maximum number of bytes
// that the backlog can hold before employing admission control. While the backlog
// is below the threshold, the coordinator will slow down the view updates up to
// calculate_view_update_throttling_delay()::delay_limit_us. Above the threshold,
// node_update_backlog::calculate_throttling_delay()::delay_limit_us. Above the threshold,
// the coordinator will reject the writes that would increase the backlog. On the
// replica, the writes will start failing only after reaching the hard limit '_max'.
float relative_size() const {
@@ -70,18 +70,4 @@ public:
}
};
// View updates are asynchronous, and because of this limiting their concurrency requires
// a special approach. The current algorithm places all of the pending view updates in the backlog
// and artificially slows down new responses to coordinator requests based on how full the backlog is.
// This function calculates how much a request should be slowed down based on the backlog's fullness.
// The equation is basically: delay(in seconds) = view_fullness_ratio^3
// The more full the backlog gets the more aggressively the requests are slowed down.
// The delay is limited to the amount of time left until timeout.
// After the timeout the request fails, so there's no point in waiting longer than that.
// The second argument defines this timeout point - we can't delay the request more than this time point.
// See: https://www.scylladb.com/2018/12/04/worry-free-ingestion-flow-control/
std::chrono::microseconds calculate_view_update_throttling_delay(
update_backlog backlog,
db::timeout_clock::time_point timeout,
uint32_t view_flow_control_delay_limit_in_ms);
}

View File

@@ -7,6 +7,7 @@
*/
#include "db/view/view_update_backlog.hh"
#include "db/view/node_view_update_backlog.hh"
#include <seastar/core/timed_out_error.hh>
#include "gms/inet_address.hh"
#include <seastar/util/defer.hh>
@@ -499,7 +500,7 @@ future<> view_update_generator::generate_and_propagate_view_updates(const replic
// the one which limits the number of incoming client requests by delaying the response to the client.
if (batch_num > 0) {
update_backlog local_backlog = _db.get_view_update_backlog();
std::chrono::microseconds throttle_delay = calculate_view_update_throttling_delay(local_backlog, timeout, _db.get_config().view_flow_control_delay_limit_in_ms());
std::chrono::microseconds throttle_delay = _node_update_backlog.calculate_throttling_delay(local_backlog, timeout);
co_await seastar::sleep(throttle_delay);

View File

@@ -1940,7 +1940,7 @@ public:
// Calculates how much to delay completing the request. The delay adds to the request's inherent latency.
template<typename Func>
void delay(tracing::trace_state_ptr trace, Func&& on_resume) {
auto delay = db::view::calculate_view_update_throttling_delay(_view_backlog, _expire_timer.get_timeout(), _proxy->data_dictionary().get_config().view_flow_control_delay_limit_in_ms());
auto delay = _proxy->_max_view_update_backlog.calculate_throttling_delay(_view_backlog, _expire_timer.get_timeout());
stats().last_mv_flow_control_delay = delay;
stats().mv_flow_control_delay += delay.count();
if (delay.count() == 0) {