Skip to content

Commit

Permalink
GH-1784 Rename cpu-effort-percent to produce-block-offset-ms and chan…
Browse files Browse the repository at this point in the history
…ge meaning to be over complete round
  • Loading branch information
heifner committed Oct 19, 2023
1 parent f65b06c commit 0c5ff7e
Show file tree
Hide file tree
Showing 9 changed files with 67 additions and 72 deletions.
17 changes: 3 additions & 14 deletions docs/01_nodeos/03_plugins/producer_plugin/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -72,20 +72,9 @@ Config Options for eosio::producer_plugin:
can extend during low usage (only
enforced subjectively; use 1000 to not
enforce any limit)
--produce-time-offset-us arg (=0) Offset of non last block producing time
in microseconds. Valid range 0 ..
-block_time_interval.
--last-block-time-offset-us arg (=-200000)
Offset of last block producing time in
microseconds. Valid range 0 ..
-block_time_interval.
--cpu-effort-percent arg (=80) Percentage of cpu block production time
used to produce block. Whole number
percentages, e.g. 80 for 80%
--last-block-cpu-effort-percent arg (=80)
Percentage of cpu block production time
used to produce last block. Whole
number percentages, e.g. 80 for 80%
--produce-block-offset-ms arg (=450) The number of milliseconds early the
last block of a production round should
be produced.
--max-block-cpu-usage-threshold-us arg (=5000)
Threshold of CPU block production to
consider block full; when within
Expand Down
2 changes: 1 addition & 1 deletion libraries/chain/include/eosio/chain/config.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ const static uint32_t default_max_inline_action_size = 512 * 102
const static uint16_t default_max_inline_action_depth = 4;
const static uint16_t default_max_auth_depth = 6;
const static uint32_t default_sig_cpu_bill_pct = 50 * percent_1; // billable percentage of signature recovery
const static uint32_t default_block_cpu_effort_pct = 90 * percent_1; // percentage of block time used for producing block
const static uint32_t default_produce_block_offset_ms = 450;
const static uint16_t default_controller_thread_pool_size = 2;
const static uint32_t default_max_variable_signature_length = 16384u;
const static uint32_t default_max_action_return_value_size = 256;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,28 +42,28 @@ namespace block_timing_util {
// In the past, a producer would always start a block `config::block_interval_us` ahead of its block time. However,
// it causes the last block in a block production round being released too late for the next producer to have
// received it and start producing on schedule. To mitigate the problem, we leave no time gap in block producing. For
// example, given block_interval=500 ms and cpu effort=400 ms, assuming the our round start at time point 0; in the
// example, given block_interval=500 ms and cpu effort=400 ms, assuming our round starts at time point 0; in the
// past, the block start time points would be at time point -500, 0, 500, 1000, 1500, 2000 .... With this new
// approach, the block time points would become -500, -100, 300, 700, 1100 ...
inline fc::time_point production_round_block_start_time(uint32_t cpu_effort_us, chain::block_timestamp_type block_time) {
inline fc::time_point production_round_block_start_time(fc::microseconds cpu_effort_us, chain::block_timestamp_type block_time) {
uint32_t block_slot = block_time.slot;
uint32_t production_round_start_block_slot =
(block_slot / chain::config::producer_repetitions) * chain::config::producer_repetitions;
uint32_t production_round_index = block_slot % chain::config::producer_repetitions;
return chain::block_timestamp_type(production_round_start_block_slot - 1).to_time_point() +
fc::microseconds(cpu_effort_us * production_round_index);
fc::microseconds(cpu_effort_us.count() * production_round_index);
}

inline fc::time_point calculate_producing_block_deadline(uint32_t cpu_effort_us, chain::block_timestamp_type block_time) {
inline fc::time_point calculate_producing_block_deadline(fc::microseconds cpu_effort_us, chain::block_timestamp_type block_time) {
auto estimated_deadline = production_round_block_start_time(cpu_effort_us, block_time) + fc::microseconds(cpu_effort_us);
auto now = fc::time_point::now();
if (estimated_deadline > now) {
return estimated_deadline;
} else {
// This could only happen when the producer stop producing and then comes back alive in the middle of its own
// production round. In this case, we just use the hard deadline.
const auto hard_deadline = block_time.to_time_point() - fc::microseconds(chain::config::block_interval_us - cpu_effort_us);
return std::min(hard_deadline, now + fc::microseconds(cpu_effort_us));
const auto hard_deadline = block_time.to_time_point() - fc::microseconds(chain::config::block_interval_us - cpu_effort_us.count());
return std::min(hard_deadline, now + cpu_effort_us);
}
}

Expand Down Expand Up @@ -118,7 +118,7 @@ namespace block_timing_util {
// Return the *next* block start time according to its block time slot.
// Returns empty optional if no producers are in the active_schedule.
// block_num is only used for watermark minimum offset.
inline std::optional<fc::time_point> calculate_producer_wake_up_time(uint32_t cpu_effort_us, uint32_t block_num,
inline std::optional<fc::time_point> calculate_producer_wake_up_time(fc::microseconds cpu_effort_us, uint32_t block_num,
const chain::block_timestamp_type& ref_block_time,
const std::set<chain::account_name>& producers,
const std::vector<chain::producer_authority>& active_schedule,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ class producer_plugin : public appbase::plugin<producer_plugin> {
struct runtime_options {
std::optional<int32_t> max_transaction_time;
std::optional<int32_t> max_irreversible_block_age;
std::optional<int32_t> cpu_effort_us;
std::optional<int32_t> produce_block_offset_ms;
std::optional<int32_t> subjective_cpu_leeway_us;
std::optional<uint32_t> greylist_limit;
};
Expand Down Expand Up @@ -196,7 +196,7 @@ class producer_plugin : public appbase::plugin<producer_plugin> {

} //eosio

FC_REFLECT(eosio::producer_plugin::runtime_options, (max_transaction_time)(max_irreversible_block_age)(cpu_effort_us)(subjective_cpu_leeway_us)(greylist_limit));
FC_REFLECT(eosio::producer_plugin::runtime_options, (max_transaction_time)(max_irreversible_block_age)(produce_block_offset_ms)(subjective_cpu_leeway_us)(greylist_limit));
FC_REFLECT(eosio::producer_plugin::greylist_params, (accounts));
FC_REFLECT(eosio::producer_plugin::whitelist_blacklist, (actor_whitelist)(actor_blacklist)(contract_whitelist)(contract_blacklist)(action_blacklist)(key_blacklist) )
FC_REFLECT(eosio::producer_plugin::integrity_hash_information, (head_block_id)(integrity_hash))
Expand Down
47 changes: 26 additions & 21 deletions plugins/producer_plugin/producer_plugin.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,6 @@ class producer_plugin_impl : public std::enable_shared_from_this<producer_plugin
bool maybe_produce_block();
bool block_is_exhausted() const;
bool remove_expired_trxs(const fc::time_point& deadline);
bool remove_expired_blacklisted_trxs(const fc::time_point& deadline);
bool process_unapplied_trxs(const fc::time_point& deadline);
bool retire_deferred_trxs(const fc::time_point& deadline);
bool process_incoming_trxs(const fc::time_point& deadline, unapplied_transaction_queue::iterator& itr);
Expand Down Expand Up @@ -473,7 +472,7 @@ class producer_plugin_impl : public std::enable_shared_from_this<producer_plugin
producer_plugin::runtime_options get_runtime_options() const {
return {_max_transaction_time_ms,
_max_irreversible_block_age_us.count() < 0 ? -1 : _max_irreversible_block_age_us.count() / 1'000'000,
_cpu_effort_us,
get_produce_block_offset().count() / 1'000,
chain_plug->chain().get_subjective_cpu_leeway() ? chain_plug->chain().get_subjective_cpu_leeway()->count()
: std::optional<int32_t>(),
chain_plug->chain().get_greylist_limit()};
Expand Down Expand Up @@ -501,14 +500,14 @@ class producer_plugin_impl : public std::enable_shared_from_this<producer_plugin
std::atomic<int32_t> _max_transaction_time_ms; // modified by app thread, read by net_plugin thread pool
std::atomic<uint32_t> _received_block{0}; // modified by net_plugin thread pool
fc::microseconds _max_irreversible_block_age_us;
int32_t _cpu_effort_us = 0;
// produce-block-offset is in terms of the complete round, internally use calculated value for each block of round
fc::microseconds _produce_block_cpu_effort_us;
fc::time_point _pending_block_deadline;
uint32_t _max_block_cpu_usage_threshold_us = 0;
uint32_t _max_block_net_usage_threshold_bytes = 0;
bool _disable_subjective_p2p_billing = true;
bool _disable_subjective_api_billing = true;
fc::time_point _irreversible_block_time;
fc::time_point _idle_trx_time{fc::time_point::now()};

std::vector<chain::digest_type> _protocol_features_to_activate;
bool _protocol_features_signaled = false; // to mark whether it has been signaled in start_block
Expand Down Expand Up @@ -613,6 +612,17 @@ class producer_plugin_impl : public std::enable_shared_from_this<producer_plugin
void repost_exhausted_transactions(const fc::time_point& deadline);
bool push_read_only_transaction(transaction_metadata_ptr trx, next_function<transaction_trace_ptr> next);

void set_produce_block_offset(uint32_t produce_block_offset_ms) {
EOS_ASSERT(produce_block_offset_ms < (config::producer_repetitions * config::block_interval_ms), plugin_config_exception,
"produce-block-offset-ms ${p} must be [0 - ${max})", ("p", produce_block_offset_ms)("max", config::producer_repetitions * config::block_interval_ms));
_produce_block_cpu_effort_us = fc::milliseconds( config::block_interval_ms - produce_block_offset_ms / config::producer_repetitions );
}

fc::microseconds get_produce_block_offset() const {
return fc::milliseconds( (config::block_interval_ms * config::producer_repetitions) -
((_produce_block_cpu_effort_us.count()/1000) * config::producer_repetitions) );
}

void on_block(const block_state_ptr& bsp) {
auto& chain = chain_plug->chain();
auto before = _unapplied_transactions.size();
Expand Down Expand Up @@ -1033,8 +1043,8 @@ void producer_plugin::set_program_options(
"account that can not access to extended CPU/NET virtual resources")
("greylist-limit", boost::program_options::value<uint32_t>()->default_value(1000),
"Limit (between 1 and 1000) on the multiple that CPU/NET virtual resources can extend during low usage (only enforced subjectively; use 1000 to not enforce any limit)")
("cpu-effort-percent", bpo::value<uint32_t>()->default_value(config::default_block_cpu_effort_pct / config::percent_1),
"Percentage of cpu block production time used to produce block. Whole number percentages, e.g. 80 for 80%")
("produce-block-offset-ms", bpo::value<uint32_t>()->default_value(config::default_produce_block_offset_ms),
"The number of milliseconds early the last block of a production round should be produced.")
("max-block-cpu-usage-threshold-us", bpo::value<uint32_t>()->default_value( 5000 ),
"Threshold of CPU block production to consider block full; when within threshold of max-block-cpu-usage block can be produced immediately")
("max-block-net-usage-threshold-bytes", bpo::value<uint32_t>()->default_value( 1024 ),
Expand Down Expand Up @@ -1126,12 +1136,7 @@ void producer_plugin_impl::plugin_initialize(const boost::program_options::varia
_account_fails.set_max_failures_per_account(options.at("subjective-account-max-failures").as<uint32_t>(),
subjective_account_max_failures_window_size);

uint32_t cpu_effort_pct = options.at("cpu-effort-percent").as<uint32_t>();
EOS_ASSERT(cpu_effort_pct >= 0 && cpu_effort_pct <= 100, plugin_config_exception,
"cpu-effort-percent ${pct} must be 0 - 100", ("pct", cpu_effort_pct));
cpu_effort_pct *= config::percent_1;

_cpu_effort_us = EOS_PERCENT(config::block_interval_us, cpu_effort_pct);
set_produce_block_offset(options.at("produce-block-offset-ms").as<uint32_t>());

_max_block_cpu_usage_threshold_us = options.at("max-block-cpu-usage-threshold-us").as<uint32_t>();
EOS_ASSERT(_max_block_cpu_usage_threshold_us < config::block_interval_us,
Expand Down Expand Up @@ -1441,8 +1446,8 @@ void producer_plugin_impl::update_runtime_options(const producer_plugin::runtime
check_speculating = true;
}

if (options.cpu_effort_us) {
_cpu_effort_us = *options.cpu_effort_us;
if (options.produce_block_offset_ms) {
set_produce_block_offset(*options.produce_block_offset_ms);
}

if (check_speculating && in_speculating_mode()) {
Expand Down Expand Up @@ -1838,10 +1843,10 @@ producer_plugin_impl::start_block_result producer_plugin_impl::start_block() {
}
}

_pending_block_deadline = block_timing_util::calculate_producing_block_deadline(_cpu_effort_us, block_time);
_pending_block_deadline = block_timing_util::calculate_producing_block_deadline(_produce_block_cpu_effort_us, block_time);
} else if (!_producers.empty()) {
// cpu effort percent doesn't matter for the first block of the round, use max (block_interval_us) for cpu effort
auto wake_time = block_timing_util::calculate_producer_wake_up_time(config::block_interval_us, chain.head_block_num(), chain.head_block_time(),
auto wake_time = block_timing_util::calculate_producer_wake_up_time(fc::microseconds(config::block_interval_us), chain.head_block_num(), chain.head_block_time(),
_producers, chain.head_block_state()->active_schedule.producers,
_producer_watermarks);
if (wake_time)
Expand Down Expand Up @@ -2004,8 +2009,8 @@ bool producer_plugin_impl::remove_expired_trxs(const fc::time_point& deadline) {
});

if (exhausted && in_producing_mode()) {
fc_wlog(_log, "Unable to process all expired transactions of the ${n} transactions in the unapplied queue before deadline, "
"Expired ${expired}", ("n", orig_count)("expired", num_expired));
fc_wlog(_log, "Unable to process all expired transactions of the ${n} transactions in the unapplied queue before deadline ${d}, "
"Expired ${expired}", ("n", orig_count)("d", deadline)("expired", num_expired));
} else {
fc_dlog(_log, "Processed ${ex} expired transactions of the ${n} transactions in the unapplied queue.", ("n", orig_count)("ex", num_expired));
}
Expand Down Expand Up @@ -2473,7 +2478,7 @@ void producer_plugin_impl::schedule_production_loop() {
if (!_producers.empty() && !production_disabled_by_policy()) {
chain::controller& chain = chain_plug->chain();
fc_dlog(_log, "Waiting till another block is received and scheduling Speculative/Production Change");
auto wake_time = block_timing_util::calculate_producer_wake_up_time(_cpu_effort_us, chain.head_block_num(), calculate_pending_block_time(),
auto wake_time = block_timing_util::calculate_producer_wake_up_time(_produce_block_cpu_effort_us, chain.head_block_num(), calculate_pending_block_time(),
_producers, chain.head_block_state()->active_schedule.producers,
_producer_watermarks);
schedule_delayed_production_loop(weak_from_this(), wake_time);
Expand All @@ -2492,7 +2497,7 @@ void producer_plugin_impl::schedule_production_loop() {
chain::controller& chain = chain_plug->chain();
fc_dlog(_log, "Speculative Block Created; Scheduling Speculative/Production Change");
EOS_ASSERT(chain.is_building_block(), missing_pending_block_state, "speculating without pending_block_state");
auto wake_time = block_timing_util::calculate_producer_wake_up_time(_cpu_effort_us, chain.pending_block_num(), chain.pending_block_timestamp(),
auto wake_time = block_timing_util::calculate_producer_wake_up_time(_produce_block_cpu_effort_us, chain.pending_block_num(), chain.pending_block_timestamp(),
_producers, chain.head_block_state()->active_schedule.producers,
_producer_watermarks);
schedule_delayed_production_loop(weak_from_this(), wake_time);
Expand All @@ -2509,7 +2514,7 @@ void producer_plugin_impl::schedule_maybe_produce_block(bool exhausted) {
assert(in_producing_mode());
// we succeeded but block may be exhausted
static const boost::posix_time::ptime epoch(boost::gregorian::date(1970, 1, 1));
auto deadline = block_timing_util::calculate_producing_block_deadline(_cpu_effort_us, chain.pending_block_time());
auto deadline = block_timing_util::calculate_producing_block_deadline(_produce_block_cpu_effort_us, chain.pending_block_time());

if (!exhausted && deadline > fc::time_point::now()) {
// ship this block off no later than its deadline
Expand Down
Loading

0 comments on commit 0c5ff7e

Please sign in to comment.