Commit 40dd17a2 authored by Dominik Charousset's avatar Dominik Charousset

Add size-based credit policy

parent e3d02e56
...@@ -140,6 +140,7 @@ set(LIBCAF_CORE_SRCS ...@@ -140,6 +140,7 @@ set(LIBCAF_CORE_SRCS
src/sec.cpp src/sec.cpp
src/serializer.cpp src/serializer.cpp
src/settings.cpp src/settings.cpp
src/size_based_credit_controller.cpp
src/skip.cpp src/skip.cpp
src/stream_aborter.cpp src/stream_aborter.cpp
src/stream_manager.cpp src/stream_manager.cpp
......
...@@ -38,6 +38,13 @@ extern const timespan max_batch_delay; ...@@ -38,6 +38,13 @@ extern const timespan max_batch_delay;
extern const timespan credit_round_interval; extern const timespan credit_round_interval;
extern const atom_value credit_policy; extern const atom_value credit_policy;
namespace size_policy {
extern const int32_t bytes_per_batch;
extern const int32_t buffer_capacity;
} // namespace size_policy
} // namespace streaming } // namespace streaming
namespace scheduler { namespace scheduler {
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include "caf/credit_controller.hpp"
#include <chrono>
namespace caf {
namespace detail {
/// .
class size_based_credit_controller : public credit_controller {
public:
// -- member types -----------------------------------------------------------
using super = credit_controller;
using clock_type = std::chrono::steady_clock;
using time_point = clock_type::time_point;
// -- constructors, destructors, and assignment operators --------------------
explicit size_based_credit_controller(scheduled_actor* self);
~size_based_credit_controller() override;
// -- implementation of virtual functions ------------------------------------
void before_processing(downstream_msg::batch& x) override;
void after_processing(downstream_msg::batch& x) override;
assignment compute_initial() override;
assignment compute(timespan cycle) override;
assignment compute_bridge() override;
int32_t low_threshold() const noexcept override;
private:
// -- member variables -------------------------------------------------------
/// Total number of elements in all processed batches in the current cycle.
int64_t num_elements_ = 0;
/// Measured size of all sampled elements.
int64_t total_size_ = 0;
/// Elapsed time for processing all elements of all batches in the current
/// cycle.
timespan processing_time_;
/// Timestamp of the last call to `before_processing`.
time_point processing_begin_;
/// Stores the desired per-batch complexity.
timespan complexity_;
/// Stores how many elements the buffer should hold at most.
int32_t buffer_size_ = 1;
/// Stores how many elements each batch should contain.
int32_t batch_size_ = 1;
/// Configures how many bytes we store in total.
int32_t buffer_capacity_;
/// Configures how many bytes we transfer per batch.
int32_t bytes_per_batch_;
/// Stores the current write position in the ring buffer.
size_t ringbuf_pos_ = 0;
/// Stores how many valid entries the ring buffer holds.
size_t ringbuf_size_ = 0;
/// Records recent calculations for buffer and batch sizes.
std::array<std::pair<int32_t, int32_t>, 32> ringbuf_;
};
} // namespace detail
} // namespace caf
...@@ -55,19 +55,22 @@ public: ...@@ -55,19 +55,22 @@ public:
stream_slots slots; stream_slots slots;
/// Stores the last computed desired batch size. /// Stores the last computed desired batch size.
int32_t desired_batch_size; int32_t desired_batch_size = 0;
/// Amount of credit we have signaled upstream. /// Amount of credit we have signaled upstream.
int32_t assigned_credit; int32_t assigned_credit = 0;
/// Upstream capacity for limiting the amount of credit we can give.
int32_t downstream_capacity = 0;
/// Priority of incoming batches from this source. /// Priority of incoming batches from this source.
stream_priority prio; stream_priority prio = stream_priority::normal;
/// ID of the last acknowledged batch ID. /// ID of the last acknowledged batch ID.
int64_t last_acked_batch_id; int64_t last_acked_batch_id = 0;
/// ID of the last received batch. /// ID of the last received batch.
int64_t last_batch_id; int64_t last_batch_id = 0;
/// Controller for assigning credit to the source. /// Controller for assigning credit to the source.
std::unique_ptr<credit_controller> controller_; std::unique_ptr<credit_controller> controller_;
...@@ -124,12 +127,9 @@ public: ...@@ -124,12 +127,9 @@ public:
const strong_actor_ptr& hdl, const strong_actor_ptr& hdl,
error reason); error reason);
private: /// Returns a pointer to the parent actor.
scheduled_actor* self(); scheduled_actor* self();
actor_system& system();
actor_clock& clock();
}; };
/// @relates inbound_path /// @relates inbound_path
......
...@@ -161,6 +161,10 @@ settings actor_system_config::dump_content() const { ...@@ -161,6 +161,10 @@ settings actor_system_config::dump_content() const {
put_missing(stream_group, "credit-round-interval", put_missing(stream_group, "credit-round-interval",
defaults::stream::credit_round_interval); defaults::stream::credit_round_interval);
put_missing(stream_group, "credit-policy", defaults::stream::credit_policy); put_missing(stream_group, "credit-policy", defaults::stream::credit_policy);
put_missing(stream_group, "size-policy.buffer-capacity",
defaults::stream::size_policy::buffer_capacity);
put_missing(stream_group, "size-policy.bytes-per-batch",
defaults::stream::size_policy::bytes_per_batch);
// -- scheduler parameters // -- scheduler parameters
auto& scheduler_group = result["scheduler"].as_dictionary(); auto& scheduler_group = result["scheduler"].as_dictionary();
put_missing(scheduler_group, "policy", defaults::scheduler::policy); put_missing(scheduler_group, "policy", defaults::scheduler::policy);
......
...@@ -71,7 +71,8 @@ credit_controller::assignment impl::compute(timespan cycle) { ...@@ -71,7 +71,8 @@ credit_controller::assignment impl::compute(timespan cycle) {
// Instead of C * (N / t) we calculate (C * N) / t to avoid double conversion // Instead of C * (N / t) we calculate (C * N) / t to avoid double conversion
// and rounding errors. // and rounding errors.
assignment result; assignment result;
result.credit = clamp((cycle.count() * num_elements_) / total_ns); // Give enough credit to last 2 cycles.
result.credit = 2 * clamp((cycle.count() * num_elements_) / total_ns);
result.batch_size = clamp((complexity_.count() * num_elements_) / total_ns); result.batch_size = clamp((complexity_.count() * num_elements_) / total_ns);
// Reset state and return. // Reset state and return.
num_elements_ = 0; num_elements_ = 0;
......
...@@ -51,6 +51,13 @@ const timespan max_batch_delay = ms(5); ...@@ -51,6 +51,13 @@ const timespan max_batch_delay = ms(5);
const timespan credit_round_interval = ms(10); const timespan credit_round_interval = ms(10);
const atom_value credit_policy = atom("complexity"); const atom_value credit_policy = atom("complexity");
namespace size_policy {
const int32_t bytes_per_batch = 4 * 1024; // 4 kB
const int32_t buffer_capacity = 64 * 1024; // 64 kB
} // namespace size_policy
} // namespace stream } // namespace stream
namespace scheduler { namespace scheduler {
......
...@@ -21,6 +21,7 @@ ...@@ -21,6 +21,7 @@
#include "caf/actor_system_config.hpp" #include "caf/actor_system_config.hpp"
#include "caf/defaults.hpp" #include "caf/defaults.hpp"
#include "caf/detail/complexity_based_credit_controller.hpp" #include "caf/detail/complexity_based_credit_controller.hpp"
#include "caf/detail/size_based_credit_controller.hpp"
#include "caf/detail/test_credit_controller.hpp" #include "caf/detail/test_credit_controller.hpp"
#include "caf/logger.hpp" #include "caf/logger.hpp"
#include "caf/no_stages.hpp" #include "caf/no_stages.hpp"
...@@ -30,25 +31,62 @@ ...@@ -30,25 +31,62 @@
namespace caf { namespace caf {
namespace {
constexpr bool force_ack = true;
void emit_ack_batch(inbound_path& path, credit_controller::assignment x,
bool force_ack_msg = false) {
CAF_ASSERT(x.batch_size > 0);
path.desired_batch_size = x.batch_size;
auto guard = detail::make_scope_guard([&] {
if (!force_ack_msg || path.up_to_date())
return;
unsafe_send_as(path.self(), path.hdl,
make<upstream_msg::ack_batch>(path.slots.invert(),
path.self()->address(), 0,
x.batch_size,
path.last_batch_id,
path.downstream_capacity));
path.last_acked_batch_id = path.last_batch_id;
});
auto credit = std::min(x.credit, path.downstream_capacity);
if (credit <= path.assigned_credit)
return;
auto new_credit = path.mgr->acquire_credit(&path,
credit - path.assigned_credit);
if (new_credit < 1)
return;
guard.disable();
unsafe_send_as(path.self(), path.hdl,
make<upstream_msg::ack_batch>(path.slots.invert(),
path.self()->address(),
new_credit, x.batch_size,
path.last_batch_id,
path.downstream_capacity));
path.last_acked_batch_id = path.last_batch_id;
path.assigned_credit += new_credit;
}
} // namespace
inbound_path::inbound_path(stream_manager_ptr mgr_ptr, stream_slots id, inbound_path::inbound_path(stream_manager_ptr mgr_ptr, stream_slots id,
strong_actor_ptr ptr, rtti_pair in_type) strong_actor_ptr ptr, rtti_pair in_type)
: mgr(std::move(mgr_ptr)), : mgr(std::move(mgr_ptr)), hdl(std::move(ptr)), slots(id) {
hdl(std::move(ptr)),
slots(id),
assigned_credit(0),
prio(stream_priority::normal),
last_acked_batch_id(0),
last_batch_id(0) {
CAF_IGNORE_UNUSED(in_type); CAF_IGNORE_UNUSED(in_type);
mgr->register_input_path(this); mgr->register_input_path(this);
CAF_STREAM_LOG_DEBUG(mgr->self()->name() CAF_STREAM_LOG_DEBUG(mgr->self()->name()
<< "opens input stream with element type" << "opens input stream with element type"
<< mgr->self()->system().types().portable_name(in_type) << mgr->self()->system().types().portable_name(in_type)
<< "at slot" << id.receiver << "from" << hdl); << "at slot" << id.receiver << "from" << hdl);
switch (atom_uint(get_or(system().config(), "stream.credit-policy", switch (atom_uint(get_or(self()->system().config(), "stream.credit-policy",
defaults::stream::credit_policy))) { defaults::stream::credit_policy))) {
case atom_uint("testing"): case atom_uint("testing"):
controller_.reset(new detail::test_credit_controller(self())); controller_.reset(new detail::test_credit_controller(self()));
break;
case atom_uint("size"):
controller_.reset(new detail::size_based_credit_controller(self()));
break;
default: default:
controller_.reset(new detail::complexity_based_credit_controller(self())); controller_.reset(new detail::complexity_based_credit_controller(self()));
} }
...@@ -70,14 +108,15 @@ void inbound_path::handle(downstream_msg::batch& x) { ...@@ -70,14 +108,15 @@ void inbound_path::handle(downstream_msg::batch& x) {
// Do not log a message when "running out of credit" for the first batch // Do not log a message when "running out of credit" for the first batch
// that can easily consume the initial credit in one shot. // that can easily consume the initial credit in one shot.
CAF_STREAM_LOG_DEBUG_IF(next_credit_decision.time_since_epoch().count() > 0, CAF_STREAM_LOG_DEBUG_IF(next_credit_decision.time_since_epoch().count() > 0,
mgr->self()->name() << "ran out of credit at slot" mgr->self()->name()
<< slots.receiver << "with approx." << "ran out of credit at slot" << slots.receiver);
<< (next_credit_decision - t0)
<< "until next cycle");
} else { } else {
assigned_credit -= batch_size; assigned_credit -= batch_size;
CAF_ASSERT(assigned_credit >= 0); CAF_ASSERT(assigned_credit >= 0);
} }
auto threshold = controller_->low_threshold();
if (threshold >= 0 && assigned_credit <= threshold)
caf::emit_ack_batch(*this, controller_->compute_bridge());
controller_->before_processing(x); controller_->before_processing(x);
mgr->handle(this, x); mgr->handle(this, x);
controller_->after_processing(x); controller_->after_processing(x);
...@@ -89,6 +128,7 @@ void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from) { ...@@ -89,6 +128,7 @@ void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from) {
// Update state. // Update state.
auto initial = controller_->compute_initial(); auto initial = controller_->compute_initial();
assigned_credit = mgr->acquire_credit(this, initial.credit); assigned_credit = mgr->acquire_credit(this, initial.credit);
downstream_capacity = assigned_credit;
CAF_ASSERT(assigned_credit >= 0); CAF_ASSERT(assigned_credit >= 0);
desired_batch_size = std::min(initial.batch_size, assigned_credit); desired_batch_size = std::min(initial.batch_size, assigned_credit);
// Make sure we receive errors from this point on. // Make sure we receive errors from this point on.
...@@ -100,57 +140,16 @@ void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from) { ...@@ -100,57 +140,16 @@ void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from) {
std::move(rebind_from), std::move(rebind_from),
self->ctrl(), assigned_credit, self->ctrl(), assigned_credit,
desired_batch_size)); desired_batch_size));
last_credit_decision = clock().now(); last_credit_decision = self->clock().now();
} }
void inbound_path::emit_ack_batch(local_actor* self, int32_t queued_items, void inbound_path::emit_ack_batch(local_actor*, int32_t,
actor_clock::time_point now, timespan cycle) { actor_clock::time_point now, timespan cycle) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(queued_items) << CAF_ARG(cycle) CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(cycle));
<< CAF_ARG(complexity)); downstream_capacity = mgr->out().max_capacity();
CAF_IGNORE_UNUSED(queued_items);
// Update timestamps.
last_credit_decision = now; last_credit_decision = now;
next_credit_decision = now + cycle; next_credit_decision = now + cycle;
// Hand out enough credit to fill our queue for 2 cycles but never exceed caf::emit_ack_batch(*this, controller_->compute(cycle), force_ack);
// the downstream capacity.
auto& out = mgr->out();
auto x = controller_->compute(cycle);
auto max_capacity = std::min(x.credit * 2, out.max_capacity());
CAF_ASSERT(max_capacity > 0);
// Protect against overflow on `assigned_credit`.
auto max_new_credit = std::numeric_limits<int32_t>::max() - assigned_credit;
// Compute the amount of credit we grant in this round.
auto credit = std::min(std::max(max_capacity
- static_cast<int32_t>(out.buffered())
- assigned_credit,
0),
max_new_credit);
CAF_ASSERT(credit >= 0);
// The manager can restrict or adjust the amount of credit.
credit = std::min(mgr->acquire_credit(this, credit), max_new_credit);
CAF_STREAM_LOG_DEBUG(mgr->self()->name()
<< "grants" << credit << "new credit at slot"
<< slots.receiver << "after receiving"
<< stats.num_elements << "elements that took"
<< stats.processing_time
<< CAF_ARG2("max_throughput", x.max_throughput)
<< CAF_ARG2("max_downstream_capacity",
out.max_capacity())
<< CAF_ARG(assigned_credit));
if (credit == 0 && up_to_date())
return;
CAF_LOG_DEBUG(CAF_ARG(assigned_credit)
<< CAF_ARG(max_capacity) << CAF_ARG(queued_items)
<< CAF_ARG(credit) << CAF_ARG(x.batch_size));
assigned_credit += credit;
CAF_ASSERT(assigned_credit >= 0);
desired_batch_size = x.batch_size;
unsafe_send_as(self, hdl,
make<upstream_msg::ack_batch>(slots.invert(), self->address(),
static_cast<int32_t>(credit),
desired_batch_size,
last_batch_id, max_capacity));
last_acked_batch_id = last_batch_id;
} }
bool inbound_path::up_to_date() { bool inbound_path::up_to_date() {
...@@ -189,12 +188,4 @@ scheduled_actor* inbound_path::self() { ...@@ -189,12 +188,4 @@ scheduled_actor* inbound_path::self() {
return mgr->self(); return mgr->self();
} }
actor_system& inbound_path::system() {
return mgr->self()->system();
}
actor_clock& inbound_path::clock() {
return mgr->self()->clock();
}
} // namespace caf } // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/size_based_credit_controller.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/serialized_size.hpp"
#include "caf/scheduled_actor.hpp"
// Safe us some typing and very ugly formatting.
#define impl size_based_credit_controller
namespace caf {
namespace detail {
impl::impl(scheduled_actor* self) : super(self) {
auto& cfg = self->system().config();
complexity_ = cfg.stream_desired_batch_complexity;
buffer_capacity_ = get_or(cfg, "stream.size-policy.buffer-capacity",
defaults::stream::size_policy::buffer_capacity);
bytes_per_batch_ = get_or(cfg, "stream.size-policy.bytes-per-batch",
defaults::stream::size_policy::bytes_per_batch);
}
impl::~impl() {
// nop
}
void impl::before_processing(downstream_msg::batch& x) {
num_elements_ += x.xs_size;
total_size_ = serialized_size(self()->system(), x.xs);
processing_begin_ = clock_type::now();
}
void impl::after_processing(downstream_msg::batch&) {
processing_time_ += clock_type::now() - processing_begin_;
}
credit_controller::assignment impl::compute_initial() {
return {buffer_size_, batch_size_};
}
credit_controller::assignment impl::compute(timespan) {
// Update batch and buffer size if we have at least 10 new data points to
// work with.
if (num_elements_ > 9) {
// Helper for truncating a 64-bit integer to a 32-bit integer with a
// minimum value of 1.
auto clamp_i32 = [](int64_t x) -> int32_t {
static constexpr auto upper_bound = std::numeric_limits<int32_t>::max();
if (x > upper_bound)
return upper_bound;
if (x <= 0)
return 1;
return static_cast<int32_t>(x);
};
// Calculate ideal batch size by complexity.
auto total_ns = processing_time_.count();
CAF_ASSERT(total_ns > 0);
auto size_by_complexity = clamp_i32((complexity_.count() * num_elements_)
/ total_ns);
// Calculate ideal batch size by size.
auto bytes_per_element = clamp_i32(total_size_ / num_elements_);
auto size_by_bytes = clamp_i32(bytes_per_batch_ / bytes_per_element);
// Always pick the smaller of the two in order to achieve a good tradeoff
// between size and computational complexity
auto batch_size = clamp_i32(std::min(size_by_bytes, size_by_complexity));
auto buffer_size = std::min(clamp_i32(buffer_capacity_ / bytes_per_element),
4 * batch_size_);
// Add a new entry to our ringbuffer and calculate batch and buffer sizes
// based on the average recorded sizes.
auto& kvp = ringbuf_[ringbuf_pos_];
kvp.first = buffer_size;
kvp.second = batch_size;
ringbuf_pos_ = (ringbuf_pos_ + 1) % ringbuf_.size();
if (ringbuf_size_ < ringbuf_.size())
++ringbuf_size_;
using int32_pair = std::pair<int32_t, int32_t>;
auto plus = [](int32_pair x, int32_pair y) {
return int32_pair{x.first + y.first, x.second + y.second};
};
auto psum = std::accumulate(ringbuf_.begin(),
ringbuf_.begin() + ringbuf_size_,
int32_pair{0, 0}, plus);
buffer_size_ = psum.first / ringbuf_size_;
batch_size_ = psum.second / ringbuf_size_;
// Reset bookkeeping state.
num_elements_ = 0;
total_size_ = 0;
processing_time_ = timespan{0};
}
return {buffer_size_, batch_size_};
}
credit_controller::assignment impl::compute_bridge() {
CAF_ASSERT(batch_size_ > 0);
CAF_ASSERT(buffer_size_ > batch_size_);
return {buffer_size_, batch_size_};
}
int32_t impl::low_threshold() const noexcept {
return buffer_size_ / 4;
}
} // namespace detail
} // namespace caf
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment