1
0
Fork 0
mirror of https://github.com/ton-blockchain/ton synced 2025-03-09 15:40:10 +00:00

Merge branch 'testnet' into accelerator

This commit is contained in:
SpyCheese 2024-07-10 15:35:54 +03:00
commit d74d3f1fc2
45 changed files with 1770 additions and 126 deletions

View file

@ -26,9 +26,14 @@
namespace ton::validator {
CollatorNode::CollatorNode(adnl::AdnlNodeIdShort local_id, td::actor::ActorId<ValidatorManager> manager,
td::actor::ActorId<adnl::Adnl> adnl, td::actor::ActorId<rldp::Rldp> rldp)
: local_id_(local_id), manager_(std::move(manager)), adnl_(std::move(adnl)), rldp_(std::move(rldp)) {
CollatorNode::CollatorNode(adnl::AdnlNodeIdShort local_id, td::Ref<ValidatorManagerOptions> opts,
td::actor::ActorId<ValidatorManager> manager, td::actor::ActorId<adnl::Adnl> adnl,
td::actor::ActorId<rldp::Rldp> rldp)
: local_id_(local_id)
, opts_(std::move(opts))
, manager_(std::move(manager))
, adnl_(std::move(adnl))
, rldp_(std::move(rldp)) {
}
void CollatorNode::start_up() {
@ -383,7 +388,7 @@ void CollatorNode::generate_block(ShardIdFull shard, CatchainSeqno cc_seqno, std
cache_entry->block_seqno = block_seqno;
run_collate_query(
shard, last_masterchain_state_->get_block_id(), std::move(prev_blocks), Ed25519_PublicKey{td::Bits256::zero()},
last_masterchain_state_->get_validator_set(shard), manager_, timeout,
last_masterchain_state_->get_validator_set(shard), opts_->get_collator_options(), manager_, timeout,
[=, SelfId = actor_id(this), timer = td::Timer{}](td::Result<BlockCandidate> R) {
LOG(INFO) << "generate block result"
<< ": shard=" << shard.to_str() << ", cc_seqno=" << cc_seqno << ", next_block_seqno=" << block_seqno

View file

@ -26,8 +26,9 @@ class ValidatorManager;
class CollatorNode : public td::actor::Actor {
public:
CollatorNode(adnl::AdnlNodeIdShort local_id, td::actor::ActorId<ValidatorManager> manager,
td::actor::ActorId<adnl::Adnl> adnl, td::actor::ActorId<rldp::Rldp> rldp);
CollatorNode(adnl::AdnlNodeIdShort local_id, td::Ref<ValidatorManagerOptions> opts,
td::actor::ActorId<ValidatorManager> manager, td::actor::ActorId<adnl::Adnl> adnl,
td::actor::ActorId<rldp::Rldp> rldp);
void start_up() override;
void tear_down() override;
void add_shard(ShardIdFull shard);
@ -36,12 +37,17 @@ class CollatorNode : public td::actor::Actor {
void new_masterchain_block_notification(td::Ref<MasterchainState> state);
void update_validator_group_info(ShardIdFull shard, std::vector<BlockIdExt> prev, CatchainSeqno cc_seqno);
void update_options(td::Ref<ValidatorManagerOptions> opts) {
opts_ = std::move(opts);
}
private:
void receive_query(adnl::AdnlNodeIdShort src, td::BufferSlice data, td::Promise<td::BufferSlice> promise);
bool can_collate_shard(ShardIdFull shard) const;
adnl::AdnlNodeIdShort local_id_;
td::Ref<ValidatorManagerOptions> opts_;
td::actor::ActorId<ValidatorManager> manager_;
td::actor::ActorId<adnl::Adnl> adnl_;
td::actor::ActorId<rldp::Rldp> rldp_;

View file

@ -87,9 +87,9 @@ void run_validate_query(ShardIdFull shard, BlockIdExt min_masterchain_block_id,
td::Promise<ValidateCandidateResult> promise, unsigned mode = 0);
void run_collate_query(ShardIdFull shard, const BlockIdExt& min_masterchain_block_id, std::vector<BlockIdExt> prev,
Ed25519_PublicKey creator, td::Ref<ValidatorSet> validator_set,
td::actor::ActorId<ValidatorManager> manager, td::Timestamp timeout,
td::Promise<BlockCandidate> promise, td::CancellationToken cancellation_token = {},
unsigned mode = 0);
td::Ref<CollatorOptions> collator_opts, td::actor::ActorId<ValidatorManager> manager,
td::Timestamp timeout, td::Promise<BlockCandidate> promise,
td::CancellationToken cancellation_token = {}, unsigned mode = 0);
void run_collate_hardfork(ShardIdFull shard, const BlockIdExt& min_masterchain_block_id, std::vector<BlockIdExt> prev,
td::actor::ActorId<ValidatorManager> manager, td::Timestamp timeout,
td::Promise<BlockCandidate> promise);

View file

@ -70,6 +70,7 @@ class Collator final : public td::actor::Actor {
std::vector<Ref<ShardState>> prev_states;
std::vector<Ref<BlockData>> prev_block_data;
Ed25519_PublicKey created_by_;
Ref<CollatorOptions> collator_opts_;
Ref<ValidatorSet> validator_set_;
td::actor::ActorId<ValidatorManager> manager;
td::Timestamp timeout;
@ -89,9 +90,9 @@ class Collator final : public td::actor::Actor {
public:
Collator(ShardIdFull shard, bool is_hardfork, BlockIdExt min_masterchain_block_id, std::vector<BlockIdExt> prev,
Ref<ValidatorSet> validator_set, Ed25519_PublicKey collator_id, td::actor::ActorId<ValidatorManager> manager,
td::Timestamp timeout, td::Promise<BlockCandidate> promise, td::CancellationToken cancellation_token,
unsigned mode);
Ref<ValidatorSet> validator_set, Ed25519_PublicKey collator_id, Ref<CollatorOptions> collator_opts,
td::actor::ActorId<ValidatorManager> manager, td::Timestamp timeout, td::Promise<BlockCandidate> promise,
td::CancellationToken cancellation_token, unsigned mode);
~Collator() override = default;
bool is_busy() const {
return busy_;
@ -199,6 +200,7 @@ class Collator final : public td::actor::Actor {
sibling_out_msg_queue_;
std::map<StdSmcAddress, size_t> unprocessed_deferred_messages_; // number of messages from dispatch queue in new_msgs
td::uint64 out_msg_queue_size_ = 0;
td::uint64 old_out_msg_queue_size_ = 0;
bool have_out_msg_queue_size_in_state_ = false;
std::unique_ptr<vm::Dictionary> ihr_pending;
std::shared_ptr<block::MsgProcessedUptoCollection> processed_upto_, sibling_processed_upto_;
@ -219,6 +221,8 @@ class Collator final : public td::actor::Actor {
unsigned dispatch_queue_ops_{0};
std::map<StdSmcAddress, LogicalTime> last_dispatch_queue_emitted_lt_;
bool have_unprocessed_account_dispatch_queue_ = true;
td::uint64 defer_out_queue_size_limit_;
td::uint64 hard_defer_out_queue_size_limit_;
bool msg_metadata_enabled_ = false;
bool deferring_messages_enabled_ = false;

View file

@ -50,7 +50,6 @@ static const td::uint32 SPLIT_MAX_QUEUE_SIZE = 100000;
static const td::uint32 MERGE_MAX_QUEUE_SIZE = 2047;
static const td::uint32 SKIP_EXTERNALS_QUEUE_SIZE = 8000;
static const int HIGH_PRIORITY_EXTERNAL = 10; // don't skip high priority externals when queue is big
static const int DEFER_MESSAGES_AFTER = 10; // 10'th and later messages from address will be deferred
#define DBG(__n) dbg(__n)&&
#define DSTART int __dcnt = 0;
@ -71,19 +70,22 @@ static inline bool dbg(int c) {
* @param prev A vector of BlockIdExt representing the previous blocks.
* @param validator_set A reference to the ValidatorSet.
* @param collator_id The public key of the block creator.
* @param collator_opts A reference to CollatorOptions.
* @param manager The ActorId of the ValidatorManager.
* @param timeout The timeout for the collator.
* @param promise The promise to return the result.
*/
Collator::Collator(ShardIdFull shard, bool is_hardfork, BlockIdExt min_masterchain_block_id,
std::vector<BlockIdExt> prev, td::Ref<ValidatorSet> validator_set, Ed25519_PublicKey collator_id,
td::actor::ActorId<ValidatorManager> manager, td::Timestamp timeout,
td::Promise<BlockCandidate> promise, td::CancellationToken cancellation_token, unsigned mode)
Ref<CollatorOptions> collator_opts, td::actor::ActorId<ValidatorManager> manager,
td::Timestamp timeout, td::Promise<BlockCandidate> promise, td::CancellationToken cancellation_token,
unsigned mode)
: shard_(shard)
, is_hardfork_(is_hardfork)
, min_mc_block_id{min_masterchain_block_id}
, prev_blocks(std::move(prev))
, created_by_(collator_id)
, collator_opts_(collator_opts)
, validator_set_(std::move(validator_set))
, manager(manager)
, timeout(timeout)
@ -1868,6 +1870,7 @@ bool Collator::try_collate() {
last_proc_int_msg_.second.set_zero();
first_unproc_int_msg_.first = ~0ULL;
first_unproc_int_msg_.second.set_ones();
old_out_msg_queue_size_ = out_msg_queue_size_;
if (is_masterchain()) {
LOG(DEBUG) << "getting the list of special smart contracts";
auto res = config_->get_special_smartcontracts();
@ -2052,6 +2055,10 @@ bool Collator::fetch_config_params() {
return fatal_error(res.move_as_error());
}
compute_phase_cfg_.libraries = std::make_unique<vm::Dictionary>(config_->get_libraries_root(), 256);
defer_out_queue_size_limit_ = std::max<td::uint64>(collator_opts_->defer_out_queue_size_limit,
compute_phase_cfg_.size_limits.defer_out_queue_size_limit);
// This one is checked in validate-query
hard_defer_out_queue_size_limit_ = compute_phase_cfg_.size_limits.defer_out_queue_size_limit;
return true;
}
@ -3176,8 +3183,10 @@ int Collator::process_one_new_message(block::NewOutMsg msg, bool enqueue_only, R
bool is_special_account = is_masterchain() && config_->is_special_smartcontract(src_addr);
bool defer = false;
if (!from_dispatch_queue) {
if (deferring_messages_enabled_ && !is_special && !is_special_account && msg.msg_idx != 0) {
if (++sender_generated_messages_count_[src_addr] >= DEFER_MESSAGES_AFTER) {
if (deferring_messages_enabled_ && collator_opts_->deferring_enabled && !is_special && !is_special_account &&
msg.msg_idx != 0) {
if (++sender_generated_messages_count_[src_addr] >= collator_opts_->defer_messages_after ||
out_msg_queue_size_ > defer_out_queue_size_limit_) {
defer = true;
}
}
@ -3776,18 +3785,24 @@ int Collator::process_external_message(Ref<vm::Cell> msg) {
* @returns True if the processing was successful, false otherwise.
*/
bool Collator::process_dispatch_queue() {
if (out_msg_queue_size_ > defer_out_queue_size_limit_ && old_out_msg_queue_size_ > hard_defer_out_queue_size_limit_) {
return true;
}
have_unprocessed_account_dispatch_queue_ = true;
size_t max_total_count[3] = {1 << 30, 150, 150};
size_t max_per_initiator[3] = {1 << 30, 20, 0};
if (out_msg_queue_size_ <= 256) {
size_t max_total_count[3] = {1 << 30, collator_opts_->dispatch_phase_2_max_total,
collator_opts_->dispatch_phase_3_max_total};
size_t max_per_initiator[3] = {1 << 30, collator_opts_->dispatch_phase_2_max_per_initiator, 0};
if (collator_opts_->dispatch_phase_3_max_per_initiator) {
max_per_initiator[2] = collator_opts_->dispatch_phase_3_max_per_initiator.value();
} else if (out_msg_queue_size_ <= 256) {
max_per_initiator[2] = 10;
} else if (out_msg_queue_size_ <= 512) {
max_per_initiator[2] = 2;
} else if (out_msg_queue_size_ <= 2048) {
} else if (out_msg_queue_size_ <= 1500) {
max_per_initiator[2] = 1;
}
for (int iter = 0; iter < 3; ++iter) {
if (max_per_initiator[iter] == 0) {
if (max_per_initiator[iter] == 0 || max_total_count[iter] == 0) {
continue;
}
vm::AugmentedDictionary cur_dispatch_queue{dispatch_queue_->get_root(), 256, block::tlb::aug_DispatchQueue};
@ -3826,7 +3841,8 @@ bool Collator::process_dispatch_queue() {
// Remove message from DispatchQueue
bool ok;
if (iter == 0 || (iter == 1 && sender_generated_messages_count_[src_addr] >= DEFER_MESSAGES_AFTER)) {
if (iter == 0 ||
(iter == 1 && sender_generated_messages_count_[src_addr] >= collator_opts_->defer_messages_after)) {
ok = cur_dispatch_queue.lookup_delete(src_addr).not_null();
} else {
dict.lookup_delete(key);

View file

@ -214,8 +214,9 @@ void run_validate_query(ShardIdFull shard, BlockIdExt min_masterchain_block_id,
void run_collate_query(ShardIdFull shard, const BlockIdExt& min_masterchain_block_id, std::vector<BlockIdExt> prev,
Ed25519_PublicKey creator, td::Ref<ValidatorSet> validator_set,
td::actor::ActorId<ValidatorManager> manager, td::Timestamp timeout,
td::Promise<BlockCandidate> promise, td::CancellationToken cancellation_token, unsigned mode) {
td::Ref<CollatorOptions> collator_opts, td::actor::ActorId<ValidatorManager> manager,
td::Timestamp timeout, td::Promise<BlockCandidate> promise,
td::CancellationToken cancellation_token, unsigned mode) {
BlockSeqno seqno = 0;
for (auto& p : prev) {
if (p.seqno() > seqno) {
@ -224,8 +225,8 @@ void run_collate_query(ShardIdFull shard, const BlockIdExt& min_masterchain_bloc
}
td::actor::create_actor<Collator>(PSTRING() << "collate" << shard.to_str() << ":" << (seqno + 1), shard, false,
min_masterchain_block_id, std::move(prev), std::move(validator_set), creator,
std::move(manager), timeout, std::move(promise), std::move(cancellation_token),
mode)
std::move(collator_opts), std::move(manager), timeout, std::move(promise),
std::move(cancellation_token), mode)
.release();
}
@ -240,8 +241,8 @@ void run_collate_hardfork(ShardIdFull shard, const BlockIdExt& min_masterchain_b
}
td::actor::create_actor<Collator>(PSTRING() << "collate" << shard.to_str() << ":" << (seqno + 1), shard, true,
min_masterchain_block_id, std::move(prev), td::Ref<ValidatorSet>{},
Ed25519_PublicKey{Bits256::zero()}, std::move(manager), timeout, std::move(promise),
td::CancellationToken{}, 0)
Ed25519_PublicKey{Bits256::zero()}, td::Ref<CollatorOptions>{true},
std::move(manager), timeout, std::move(promise), td::CancellationToken{}, 0)
.release();
}

View file

@ -1004,6 +1004,7 @@ bool ValidateQuery::fetch_config_params() {
action_phase_cfg_.action_fine_enabled = config_->get_global_version() >= 4;
action_phase_cfg_.bounce_on_fail_enabled = config_->get_global_version() >= 4;
action_phase_cfg_.message_skip_enabled = config_->get_global_version() >= 8;
action_phase_cfg_.disable_custom_fess = config_->get_global_version() >= 8;
action_phase_cfg_.mc_blackhole_addr = config_->get_burning_config().blackhole_addr;
}
{
@ -2298,6 +2299,7 @@ bool ValidateQuery::prepare_out_msg_queue_size() {
// if after_split then out_msg_queue_size is always present, since it is calculated during split
old_out_msg_queue_size_ = ps_.out_msg_queue_size_.value();
out_msg_queue_size_known_ = true;
have_out_msg_queue_size_in_state_ = true;
return true;
}
if (!store_out_msg_queue_size_) { // Don't need it
@ -3499,7 +3501,7 @@ bool ValidateQuery::check_account_dispatch_queue_update(td::Bits256 addr, Ref<vm
}
}
if (old_dict_size > 0 && max_removed_lt == 0) {
have_unprocessed_account_dispatch_queue_ = true;
++processed_account_dispatch_queues_;
}
return true;
}
@ -3526,6 +3528,28 @@ bool ValidateQuery::unpack_dispatch_queue_update() {
if (!res) {
return reject_query("invalid DispatchQueue dictionary in the new state");
}
if (have_out_msg_queue_size_in_state_ &&
old_out_msg_queue_size_ <= compute_phase_cfg_.size_limits.defer_out_queue_size_limit) {
// Check that at least one message was taken from each AccountDispatchQueue
try {
have_unprocessed_account_dispatch_queue_ = false;
td::uint64 total_account_dispatch_queues = 0;
ps_.dispatch_queue_->check_for_each([&](Ref<vm::CellSlice>, td::ConstBitPtr, int n) -> bool {
assert(n == 352);
++total_account_dispatch_queues;
if (total_account_dispatch_queues > processed_account_dispatch_queues_) {
return false;
}
return true;
});
have_unprocessed_account_dispatch_queue_ =
(total_account_dispatch_queues != processed_account_dispatch_queues_);
} catch (vm::VmVirtError&) {
// VmVirtError can happen if we have only a proof of ShardState
have_unprocessed_account_dispatch_queue_ = true;
}
}
} catch (vm::VmError& err) {
return reject_query("invalid DispatchQueue dictionary difference between the old and the new state: "s +
err.get_msg());
@ -3788,7 +3812,8 @@ bool ValidateQuery::check_in_msg(td::ConstBitPtr key, Ref<vm::CellSlice> in_msg)
}
if (have_unprocessed_account_dispatch_queue_ && tag != block::gen::InMsg::msg_import_ext &&
tag != block::gen::InMsg::msg_import_deferred_tr && tag != block::gen::InMsg::msg_import_deferred_fin) {
// Collator is requeired to take at least one message from each AccountDispatchQueue (unless the block is full)
// Collator is requeired to take at least one message from each AccountDispatchQueue
// (unless the block is full or unless out_msg_queue_size is big)
// If some AccountDispatchQueue is unporcessed then it's not allowed to import other messages except for externals
return reject_query("required DispatchQueue processing is not done, but some other internal messages are imported");
}

View file

@ -237,11 +237,13 @@ class ValidateQuery : public td::actor::Actor {
std::set<StdSmcAddress> account_expected_defer_all_messages_;
td::uint64 old_out_msg_queue_size_ = 0, new_out_msg_queue_size_ = 0;
bool out_msg_queue_size_known_ = false;
bool have_out_msg_queue_size_in_state_ = false;
bool msg_metadata_enabled_ = false;
bool deferring_messages_enabled_ = false;
bool store_out_msg_queue_size_ = false;
td::uint64 processed_account_dispatch_queues_ = 0;
bool have_unprocessed_account_dispatch_queue_ = false;
td::PerfWarningTimer perf_timer_;

View file

@ -128,8 +128,8 @@ void ValidatorManagerImpl::sync_complete(td::Promise<td::Unit> promise) {
}
Ed25519_PublicKey created_by{td::Bits256::zero()};
td::as<td::uint32>(created_by.as_bits256().data() + 32 - 4) = ((unsigned)std::time(nullptr) >> 8);
run_collate_query(shard_id, last_masterchain_block_id_, prev, created_by, val_set, actor_id(this),
td::Timestamp::in(10.0), std::move(P));
run_collate_query(shard_id, last_masterchain_block_id_, prev, created_by, val_set, td::Ref<CollatorOptions>{true},
actor_id(this), td::Timestamp::in(10.0), std::move(P));
}
void ValidatorManagerImpl::validate_fake(BlockCandidate candidate, std::vector<BlockIdExt> prev, BlockIdExt last,

View file

@ -3365,6 +3365,9 @@ void ValidatorManagerImpl::update_options(td::Ref<ValidatorManagerOptions> opts)
td::actor::send_closure(group.second.actor, &ValidatorGroup::update_options, opts,
opts->need_monitor(group.second.shard, last_masterchain_state_));
}
for (auto &collator : collator_nodes_) {
td::actor::send_closure(collator.second.actor, &CollatorNode::update_options, opts);
}
opts_ = std::move(opts);
}
@ -3409,7 +3412,7 @@ void ValidatorManagerImpl::add_collator(adnl::AdnlNodeIdShort id, ShardIdFull sh
auto it = collator_nodes_.find(id);
if (it == collator_nodes_.end()) {
it = collator_nodes_.emplace(id, Collator()).first;
it->second.actor = td::actor::create_actor<CollatorNode>("collatornode", id, actor_id(this), adnl_, rldp_);
it->second.actor = td::actor::create_actor<CollatorNode>("collatornode", id, opts_, actor_id(this), adnl_, rldp_);
}
if (!it->second.shards.insert(shard).second) {
return;

View file

@ -514,8 +514,8 @@ void ValidatorGroup::collate_block(td::uint32 round_id, td::Timestamp timeout, t
}
if (self_collate) {
run_collate_query(shard_, min_masterchain_block_id_, prev_block_ids_,
Ed25519_PublicKey{local_id_full_.ed25519_value().raw()}, validator_set_, manager_,
td::Timestamp::in(10.0), std::move(promise));
Ed25519_PublicKey{local_id_full_.ed25519_value().raw()}, validator_set_,
opts_->get_collator_options(), manager_, td::Timestamp::in(10.0), std::move(promise));
return;
}
if (collator_adnl_id.is_zero()) {

View file

@ -142,6 +142,9 @@ struct ValidatorManagerOptionsImpl : public ValidatorManagerOptions {
bool get_state_serializer_enabled() const override {
return state_serializer_enabled_;
}
td::Ref<CollatorOptions> get_collator_options() const override {
return collator_options_;
}
td::Ref<CollatorsList> get_collators_list() const override {
return collators_list_;
}
@ -228,6 +231,9 @@ struct ValidatorManagerOptionsImpl : public ValidatorManagerOptions {
void set_state_serializer_enabled(bool value) override {
state_serializer_enabled_ = value;
}
void set_collator_options(td::Ref<CollatorOptions> value) override {
collator_options_ = std::move(value);
}
void set_collators_list(td::Ref<CollatorsList> list) override {
collators_list_ = std::move(list);
}
@ -281,6 +287,7 @@ struct ValidatorManagerOptionsImpl : public ValidatorManagerOptions {
bool celldb_preload_all_ = false;
td::optional<double> catchain_max_block_delay_;
bool state_serializer_enabled_ = true;
td::Ref<CollatorOptions> collator_options_{true};
td::Ref<CollatorsList> collators_list_{true, CollatorsList{}};
};

View file

@ -52,6 +52,21 @@ struct PerfTimerStats {
std::deque<std::pair<double, double>> stats; // <Time::now(), duration>
};
struct CollatorOptions : public td::CntObject {
bool deferring_enabled = true;
// Defer messages from account after Xth message in block (excluding first messages from transactions)
td::uint32 defer_messages_after = 10;
// Defer all messages if out msg queue size is greater than X (excluding first messages from transactions)
td::uint64 defer_out_queue_size_limit = 2048;
// See Collator::process_dispatch_queue
td::uint32 dispatch_phase_2_max_total = 150;
td::uint32 dispatch_phase_3_max_total = 150;
td::uint32 dispatch_phase_2_max_per_initiator = 20;
td::optional<td::uint32> dispatch_phase_3_max_per_initiator; // Default - depends on out msg queue size
};
struct CollatorsList : public td::CntObject {
struct Collator {
adnl::AdnlNodeIdShort adnl_id;
@ -105,6 +120,7 @@ struct ValidatorManagerOptions : public td::CntObject {
virtual bool get_celldb_preload_all() const = 0;
virtual td::optional<double> get_catchain_max_block_delay() const = 0;
virtual bool get_state_serializer_enabled() const = 0;
virtual td::Ref<CollatorOptions> get_collator_options() const = 0;
virtual td::Ref<CollatorsList> get_collators_list() const = 0;
virtual void set_zero_block_id(BlockIdExt block_id) = 0;
@ -134,6 +150,7 @@ struct ValidatorManagerOptions : public td::CntObject {
virtual void set_celldb_preload_all(bool value) = 0;
virtual void set_catchain_max_block_delay(double value) = 0;
virtual void set_state_serializer_enabled(bool value) = 0;
virtual void set_collator_options(td::Ref<CollatorOptions> value) = 0;
virtual void set_collators_list(td::Ref<CollatorsList> list) = 0;
static td::Ref<ValidatorManagerOptions> create(