Unverified Commit 27a0009d authored by Joseph Noir's avatar Joseph Noir Committed by GitHub

Merge pull request #990

Make credit computation configurable
parents d8fe071c 6f071ab1
......@@ -46,10 +46,12 @@ set(LIBCAF_CORE_SRCS
src/binary_deserializer.cpp
src/binary_serializer.cpp
src/blocking_actor.cpp
src/complexity_based_credit_controller.cpp
src/config_option.cpp
src/config_option_adder.cpp
src/config_option_set.cpp
src/config_value.cpp
src/credit_controller.cpp
src/decorator/sequencer.cpp
src/default_attachable.cpp
src/defaults.cpp
......@@ -138,12 +140,14 @@ set(LIBCAF_CORE_SRCS
src/sec.cpp
src/serializer.cpp
src/settings.cpp
src/size_based_credit_controller.cpp
src/skip.cpp
src/stream_aborter.cpp
src/stream_manager.cpp
src/string_algorithms.cpp
src/string_view.cpp
src/term.cpp
src/test_credit_controller.cpp
src/thread_hook.cpp
src/timestamp.cpp
src/tracing_data.cpp
......
......@@ -18,6 +18,8 @@
#pragma once
#include <vector>
#include "caf/default_downstream_manager.hpp"
#include "caf/detail/stream_stage_driver_impl.hpp"
#include "caf/detail/stream_stage_impl.hpp"
......@@ -70,17 +72,22 @@ attach_stream_stage(scheduled_actor* self, const stream<In>& in,
CAF_IGNORE_UNUSED(token);
using output_type = typename stream_stage_trait_t<Fun>::output;
using state_type = typename stream_stage_trait_t<Fun>::state;
static_assert(std::is_same<
void(state_type&),
typename detail::get_callable_trait<Init>::fun_sig>::value,
"Expected signature `void (State&)` for init function");
static_assert(std::is_same<
void(state_type&, downstream<output_type>&, In),
typename detail::get_callable_trait<Fun>::fun_sig>::value,
static_assert(
std::is_same<void(state_type&),
typename detail::get_callable_trait<Init>::fun_sig>::value,
"Expected signature `void (State&)` for init function");
using consume_one = void(state_type&, downstream<output_type>&, In);
using consume_all
= void(state_type&, downstream<output_type>&, std::vector<In>&);
using fun_sig = typename detail::get_callable_trait<Fun>::fun_sig;
static_assert(std::is_same<fun_sig, consume_one>::value
|| std::is_same<fun_sig, consume_all>::value,
"Expected signature `void (State&, downstream<Out>&, In)` "
"or `void (State&, downstream<Out>&, std::vector<In>&)` "
"for consume function");
using driver = detail::stream_stage_driver_impl<
typename Trait::input, DownstreamManager, Fun, Finalize>;
using driver
= detail::stream_stage_driver_impl<typename Trait::input, DownstreamManager,
Fun, Finalize>;
return attach_stream_stage<driver>(self, in, std::move(xs), std::move(init),
std::move(fun), std::move(fin));
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <cstdint>
#include "caf/downstream_msg.hpp"
#include "caf/fwd.hpp"
namespace caf {
/// Computes credit for an attached source.
class credit_controller {
public:
// -- member types -----------------------------------------------------------
/// Wraps an assignment of the controller to its source.
struct assignment {
/// Stores how much credit we assign to the source.
int32_t credit;
/// Stores how many elements we demand per batch.
int32_t batch_size;
};
// -- constructors, destructors, and assignment operators --------------------
explicit credit_controller(scheduled_actor* self);
virtual ~credit_controller();
// -- properties -------------------------------------------------------------
scheduled_actor* self() noexcept {
return self_;
}
// -- pure virtual functions -------------------------------------------------
/// Called before processing the batch `x` in order to allow the controller
/// to keep statistics on incoming batches.
virtual void before_processing(downstream_msg::batch& x) = 0;
/// Called after processing the batch `x` in order to allow the controller to
/// keep statistics on incoming batches.
/// @note The consumer may alter the batch while processing it, for example
/// by moving each element of the batch downstream.
virtual void after_processing(downstream_msg::batch& x) = 0;
/// Assigns initial credit during the stream handshake.
/// @returns The initial credit for the new sources.
virtual assignment compute_initial() = 0;
/// Assigs new credit to the source after a cycle ends.
/// @param cycle Duration of a cycle.
/// @param max_downstream_credit Maximum downstream capacity as reported by
/// the downstream manager. Controllers may use
/// this capacity as an upper bound.
virtual assignment compute(timespan cycle, int32_t max_downstream_credit) = 0;
// -- virtual functions ------------------------------------------------------
/// Computes a credit assignment to the source after crossing the
/// low-threshold. May assign zero credit.
virtual assignment compute_bridge();
/// Returns the threshold for when we may give extra credit to a source
/// during a cycle.
/// @returns Zero or a negative value if the controller never grants bridge
/// credit, otherwise the threshold for calling `compute_bridge` to
/// generate additional credit.
virtual int32_t threshold() const noexcept;
private:
// -- member variables -------------------------------------------------------
/// Points to the parent system.
scheduled_actor* self_;
};
} // namespace caf
......@@ -36,6 +36,14 @@ namespace stream {
extern const timespan desired_batch_complexity;
extern const timespan max_batch_delay;
extern const timespan credit_round_interval;
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
......
......@@ -5,7 +5,7 @@
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* 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 *
......@@ -16,75 +16,61 @@
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/config.hpp"
#define CAF_SUITE inbound_path
#include "caf/test/unit_test.hpp"
#include <string>
#include "caf/inbound_path.hpp"
using namespace std;
using namespace caf;
namespace {
#define PRINT(format, ...) \
{ \
char buf[200]; \
snprintf(buf, 200, format, __VA_ARGS__); \
CAF_MESSAGE(buf); \
}
struct fixture {
inbound_path::stats_t x;
void calculate(int32_t total_items, int32_t total_time) {
int32_t c = 1000;
int32_t d = 100;
int32_t n = total_items;
int32_t t = total_time;
int32_t m = t > 0 ? std::max((c * n) / t, 1) : 1;
int32_t b = t > 0 ? std::max((d * n) / t, 1) : 1;
PRINT("with a cycle C = %dns, desied complexity D = %d,", c, d);
PRINT("number of items N = %d, and time delta t = %d:", n, t);
PRINT("- throughput M = max(C * N / t, 1) = max(%d * %d / %d, 1) = %d",
c, n, t, m);
PRINT("- items/batch B = max(D * N / t, 1) = max(%d * %d / %d, 1) = %d",
d, n, t, b);
auto cr = x.calculate(timespan(c), timespan(d));
CAF_CHECK_EQUAL(cr.items_per_batch, b);
CAF_CHECK_EQUAL(cr.max_throughput, m);
}
void store(int32_t batch_size, int32_t calculation_time_ns) {
inbound_path::stats_t::measurement m{batch_size,
timespan{calculation_time_ns}};
x.store(m);
}
};
#pragma once
#include "caf/credit_controller.hpp"
namespace caf {
namespace detail {
/// Computes credit for an attached source based on measuring the complexity of
/// incoming batches.
class complexity_based_credit_controller : public credit_controller {
public:
// -- member types -----------------------------------------------------------
using super = credit_controller;
// -- constants --------------------------------------------------------------
/// Stores how many elements we buffer at most after the handshake.
int32_t initial_buffer_size = 50;
/// Stores how many elements we allow per batch after the handshake.
int32_t initial_batch_size = 10;
// -- constructors, destructors, and assignment operators --------------------
} // namespace
explicit complexity_based_credit_controller(scheduled_actor* self);
CAF_TEST_FIXTURE_SCOPE(inbound_path_tests, fixture)
~complexity_based_credit_controller() override;
CAF_TEST(default_constructed) {
calculate(0, 0);
}
// -- overrides --------------------------------------------------------------
CAF_TEST(one_store) {
CAF_MESSAGE("store a measurement for 500ns with batch size of 50");
store(50, 500);
calculate(50, 500);
}
void before_processing(downstream_msg::batch& x) override;
CAF_TEST(multiple_stores) {
CAF_MESSAGE("store measurements: (50, 500ns), (60, 400ns), (40, 600ns)");
store(50, 500);
store(40, 600);
store(60, 400);
calculate(150, 1500);
}
void after_processing(downstream_msg::batch& x) override;
assignment compute_initial() override;
assignment compute(timespan cycle, int32_t) override;
private:
// -- member variables -------------------------------------------------------
/// Total number of elements in all processed batches in the current cycle.
int64_t num_elements_ = 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`.
timestamp processing_begin_;
/// Stores the desired per-batch complexity.
timespan complexity_;
};
CAF_TEST_FIXTURE_SCOPE_END()
} // namespace detail
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include "caf/credit_controller.hpp"
namespace caf::detail {
/// A credit controller that estimates the bytes required to store incoming
/// batches and constrains credit based on upper bounds for memory usage.
class size_based_credit_controller : public credit_controller {
public:
// -- member types -----------------------------------------------------------
using super = credit_controller;
// -- constants --------------------------------------------------------------
/// Configures at what buffer level we grant bridge credit (0 to 1).
static constexpr float buffer_threshold = 0.75f;
/// Configures how many samples we require for recalculating buffer sizes.
static constexpr int32_t min_samples = 10;
/// Stores how many elements we buffer at most after the handshake.
int32_t initial_buffer_size = 10;
/// Stores how many elements we allow per batch after the handshake.
int32_t initial_batch_size = 2;
// -- constructors, destructors, and assignment operators --------------------
explicit size_based_credit_controller(scheduled_actor* self);
~size_based_credit_controller() override;
// -- overrides --------------------------------------------------------------
void before_processing(downstream_msg::batch& x) override;
void after_processing(downstream_msg::batch& x) override;
assignment compute_initial() override;
assignment compute(timespan cycle, int32_t) override;
assignment compute_bridge() override;
int32_t threshold() const noexcept override;
private:
// -- member variables -------------------------------------------------------
/// Total number of elements in all processed batches in the current cycle.
int64_t num_batches_ = 0;
/// Stores how many elements the buffer should hold at most.
int32_t buffer_size_ = initial_buffer_size;
/// Stores how many elements each batch should contain.
int32_t batch_size_ = initial_batch_size;
/// 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 how many elements we have sampled during the current cycle.
int32_t sampled_elements_ = 0;
/// Stores approximately how many bytes the sampled elements require when
/// serialized.
int32_t sampled_total_size_ = 0;
/// Counter for keeping track of when to sample a batch.
int32_t sample_counter_ = 0;
/// Configured how many batches we skip for the size sampling.
int32_t sample_rate_ = 1;
};
} // namespace caf::detail
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| 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"
namespace caf {
namespace detail {
/// Computes predictable credit in unit tests.
class test_credit_controller : public credit_controller {
public:
// -- member types -----------------------------------------------------------
using super = credit_controller;
// -- constructors, destructors, and assignment operators --------------------
using super::super;
~test_credit_controller() override;
// -- overrides --------------------------------------------------------------
void before_processing(downstream_msg::batch& x) override;
void after_processing(downstream_msg::batch& x) override;
assignment compute_initial() override;
assignment compute(timespan cycle, int32_t) override;
private:
/// Total number of elements in all processed batches in the current cycle.
int64_t num_elements_ = 0;
};
} // namespace detail
} // namespace caf
......@@ -23,6 +23,7 @@
#include "caf/actor_clock.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/credit_controller.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/meta/type_name.hpp"
#include "caf/rtti_pair.hpp"
......@@ -54,63 +55,22 @@ public:
stream_slots slots;
/// 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.
int32_t assigned_credit;
int32_t assigned_credit = 0;
/// Priority of incoming batches from this source.
stream_priority prio;
stream_priority prio = stream_priority::normal;
/// 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.
int64_t last_batch_id;
int64_t last_batch_id = 0;
/// Amount of credit we assign sources after receiving `open`.
static constexpr int initial_credit = 50;
/// Stores statistics for measuring complexity of incoming batches.
struct stats_t {
/// Wraps a time measurement for a single processed batch.
struct measurement {
/// Number of items in the batch.
int32_t batch_size;
/// Elapsed time for processing all elements of the batch.
timespan calculation_time;
};
/// Wraps the resulf of `stats_t::calculate()`.
struct calculation_result {
/// Number of items per credit cycle.
int32_t max_throughput;
/// Number of items per batch to reach the desired batch complexity.
int32_t items_per_batch;
};
/// Total number of elements in all processed batches.
int64_t num_elements;
/// Elapsed time for processing all elements of all batches.
timespan processing_time;
stats_t();
/// Returns the maximum number of items this actor could handle for given
/// cycle length with a minimum of 1.
calculation_result calculate(timespan cycle, timespan desired_complexity);
/// Adds a measurement to this statistic.
void store(measurement x);
/// Resets this statistic.
void reset();
};
/// Summarizes how many elements we processed during the last cycle and how
/// much time we spent processing those elements.
stats_t stats;
/// Controller for assigning credit to the source.
std::unique_ptr<credit_controller> controller_;
/// Stores the time point of the last credit decision for this source.
actor_clock::time_point last_credit_decision;
......@@ -146,10 +106,8 @@ public:
/// waiting in the mailbox.
/// @param now Current timestamp.
/// @param cycle Time between credit rounds.
/// @param desired_batch_complexity Desired processing time per batch.
void emit_ack_batch(local_actor* self, int32_t queued_items,
actor_clock::time_point now, timespan cycle,
timespan desired_batch_complexity);
actor_clock::time_point now, timespan cycle);
/// Returns whether the path received no input since last emitting
/// `ack_batch`, i.e., `last_acked_batch_id == last_batch_id`.
......@@ -166,8 +124,8 @@ public:
const strong_actor_ptr& hdl,
error reason);
private:
actor_clock& clock();
/// Returns a pointer to the parent actor.
scheduled_actor* self();
};
/// @relates inbound_path
......@@ -178,4 +136,3 @@ typename Inspector::return_type inspect(Inspector& f, inbound_path& x) {
}
} // namespace caf
......@@ -54,7 +54,7 @@ public:
auto this_worker = this;
this_thread_ = std::thread{[this_worker] {
CAF_SET_LOGGER_SYS(&this_worker->system());
detail::set_thread_name("caf.multiplexer");
detail::set_thread_name("caf.worker");
this_worker->system().thread_started();
this_worker->run();
this_worker->system().thread_terminates();
......
......@@ -82,7 +82,9 @@ actor_system_config::actor_system_config()
.add<timespan>(stream_max_batch_delay, "max-batch-delay",
"maximum delay for partial batches")
.add<timespan>(stream_credit_round_interval, "credit-round-interval",
"time between emitting credit");
"time between emitting credit")
.add<atom_value>("credit-policy",
"selects an algorithm for credit computation");
opt_group{custom_options_, "scheduler"}
.add<atom_value>("policy", "'stealing' (default) or 'sharing'")
.add<size_t>("max-threads", "maximum number of worker threads")
......@@ -158,6 +160,11 @@ settings actor_system_config::dump_content() const {
defaults::stream::max_batch_delay);
put_missing(stream_group, "credit-round-interval",
defaults::stream::credit_round_interval);
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
auto& scheduler_group = result["scheduler"].as_dictionary();
put_missing(scheduler_group, "policy", defaults::scheduler::policy);
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| 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/complexity_based_credit_controller.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/scheduled_actor.hpp"
// Safe us some typing and very ugly formatting.
#define impl complexity_based_credit_controller
namespace caf::detail {
impl::impl(scheduled_actor* self) : super(self) {
auto& cfg = self->system().config();
complexity_ = cfg.stream_desired_batch_complexity;
}
impl::~impl() {
// nop
}
void impl::before_processing(downstream_msg::batch& x) {
num_elements_ += x.xs_size;
processing_begin_ = make_timestamp();
}
void impl::after_processing(downstream_msg::batch&) {
processing_time_ += make_timestamp() - processing_begin_;
}
credit_controller::assignment impl::compute_initial() {
return {initial_buffer_size, initial_batch_size};
}
credit_controller::assignment
impl::compute(timespan cycle, int32_t downstream_capacity) {
// Max throughput = C * (N / t), where C = cycle length, N = measured items,
// and t = measured time. Desired batch size is the same formula with D
// (desired complexity) instead of C. We compute our values in 64-bit for
// more precision before truncating to a 32-bit integer type at the end.
int64_t total_ns = processing_time_.count();
if (total_ns == 0)
return {1, 1};
// Helper for truncating a 64-bit integer to a 32-bit integer with a minimum
// value of 1.
auto clamp = [](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);
};
// Instead of C * (N / t) we calculate (C * N) / t to avoid double conversion
// and rounding errors.
assignment result;
// Give enough credit to last 2 cycles, but don't exceed downstream capacity.
result.credit = 2 * clamp((cycle.count() * num_elements_) / total_ns);
result.credit = std::min(result.credit, downstream_capacity);
result.batch_size = clamp((complexity_.count() * num_elements_) / total_ns);
// Reset state and return.
num_elements_ = 0;
processing_time_ = timespan{0};
return result;
}
} // namespace caf::detail
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| 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/credit_controller.hpp"
namespace caf {
credit_controller::credit_controller(scheduled_actor* self) : self_(self) {
// nop
}
credit_controller::~credit_controller() {
// nop
}
credit_controller::assignment credit_controller::compute_bridge() {
return {0, 0};
}
int32_t credit_controller::threshold() const noexcept {
return -1;
}
} // namespace caf
......@@ -49,6 +49,14 @@ namespace stream {
const timespan desired_batch_complexity = us(50);
const timespan max_batch_delay = ms(5);
const timespan credit_round_interval = ms(10);
const atom_value credit_policy = atom("complexity");
namespace size_policy {
const int32_t bytes_per_batch = 2048; // 2 KB
const int32_t buffer_capacity = 64 * 1024; // 64 KB
} // namespace size_policy
} // namespace stream
......
This diff is collapsed.
......@@ -1158,12 +1158,13 @@ scheduled_actor::advance_streams(actor_clock::time_point now) {
CAF_LOG_DEBUG("new credit round");
auto cycle = stream_ticks_.interval();
cycle *= static_cast<decltype(cycle)::rep>(credit_round_ticks_);
auto bc = home_system().config().stream_desired_batch_complexity;
auto& qs = get_downstream_queue().queues();
for (auto& kvp : qs) {
auto inptr = kvp.second.policy().handler.get();
auto bs = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(this, bs, now, cycle, bc);
if (inptr != nullptr) {
auto tts = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(this, tts, now, cycle);
}
}
}
return stream_ticks_.next_timeout(now, {max_batch_delay_ticks_,
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| 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::detail {
impl::impl(scheduled_actor* self) : super(self) {
auto& cfg = self->system().config();
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) {
if (++sample_counter_ == sample_rate_) {
sampled_elements_ += x.xs_size;
sampled_total_size_ += serialized_size(self()->system(), x.xs);
sample_counter_ = 0;
}
++num_batches_;
}
void impl::after_processing(downstream_msg::batch&) {
// nop
}
credit_controller::assignment impl::compute_initial() {
return {buffer_size_, batch_size_};
}
credit_controller::assignment impl::compute(timespan, int32_t) {
if (sampled_elements_ >= min_samples) {
// 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 size.
auto bytes_per_element = clamp_i32(sampled_total_size_ / sampled_elements_);
batch_size_ = clamp_i32(bytes_per_batch_ / bytes_per_element);
buffer_size_ = clamp_i32(buffer_capacity_ / bytes_per_element);
// Reset bookkeeping state.
sampled_elements_ = 0;
sampled_total_size_ = 0;
// Adjust the sample rate to reach min_samples in the next cycle.
sample_rate_ = clamp_i32(num_batches_ / min_samples);
if (sample_counter_ >= sample_rate_)
sample_counter_ = 0;
num_batches_ = 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::threshold() const noexcept {
return static_cast<int32_t>(buffer_size_ * buffer_threshold);
}
} // namespace caf::detail
......@@ -155,7 +155,6 @@ void stream_manager::advance() {
if (!inbound_paths_.empty()) {
auto now = self_->clock().now();
auto& cfg = self_->system().config();
auto bc = cfg.stream_desired_batch_complexity;
auto interval = cfg.stream_credit_round_interval;
auto& qs = self_->get_downstream_queue().queues();
// Iterate all queues for inbound traffic.
......@@ -163,8 +162,8 @@ void stream_manager::advance() {
auto inptr = kvp.second.policy().handler.get();
// Ignore inbound paths of other managers.
if (inptr->mgr.get() == this) {
auto bs = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(self_, bs, now, interval, bc);
auto tts = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(self_, tts, now, interval);
}
}
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| 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/test_credit_controller.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/scheduled_actor.hpp"
namespace caf {
namespace detail {
test_credit_controller::~test_credit_controller() {
// nop
}
void test_credit_controller::before_processing(downstream_msg::batch& x) {
num_elements_ += x.xs_size;
}
void test_credit_controller::after_processing(downstream_msg::batch&) {
// nop
}
credit_controller::assignment test_credit_controller::compute_initial() {
return {50, 50};
}
credit_controller::assignment
test_credit_controller::compute(timespan cycle, int32_t) {
auto& cfg = self()->system().config();
auto complexity = cfg.stream_desired_batch_complexity;
// Max throughput = C * (N / t), where C = cycle length, N = measured items,
// and t = measured time. Desired batch size is the same formula with D
// (desired complexity) instead of C. We compute our values in 64-bit for
// more precision before truncating to a 32-bit integer type at the end.
int64_t total_ns = num_elements_ * 1000; // calculate with 1us per element
if (total_ns == 0)
return {1, 1};
// Helper for truncating a 64-bit integer to a 32-bit integer with a minimum
// value of 1.
auto clamp = [](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);
};
// Instead of C * (N / t) we calculate (C * N) / t to avoid double conversion
// and rounding errors.
assignment result;
result.credit = clamp((cycle.count() * num_elements_) / total_ns);
result.batch_size = clamp((complexity.count() * num_elements_) / total_ns);
// Reset state and return.
num_elements_ = 0;
return result;
}
} // namespace detail
} // namespace caf
......@@ -329,7 +329,6 @@ public:
void advance_time() {
auto cycle = std::chrono::milliseconds(100);
auto desired_batch_complexity = std::chrono::microseconds(50);
auto f = [&](tick_type x) {
if (x % ticks_per_force_batches_interval == 0) {
// Force batches on all output paths.
......@@ -341,9 +340,8 @@ public:
auto& qs = get<dmsg_id::value>(mbox.queues()).queues();
for (auto& kvp : qs) {
auto inptr = kvp.second.policy().handler.get();
auto bs = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(this, bs, now(), cycle,
desired_batch_complexity);
auto tts = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(this, tts, now(), cycle);
}
}
};
......
......@@ -693,6 +693,7 @@ public:
cfg.set("middleman.network-backend", caf::atom("testing"));
cfg.set("middleman.manual-multiplexing", true);
cfg.set("middleman.workers", size_t{0});
cfg.set("stream.credit-policy", caf::atom("testing"));
return cfg;
}
......
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