mirror of
https://github.com/ton-blockchain/ton
synced 2025-03-09 15:40:10 +00:00
Merge branch 'testnet' into block-generation
# Conflicts: # tonlib/tonlib/TonlibClient.cpp # validator-engine/validator-engine.hpp # validator/validator-options.hpp # validator/validator.h
This commit is contained in:
commit
2ee7cbc106
47 changed files with 1331 additions and 965 deletions
|
@ -601,7 +601,7 @@ void ArchiveManager::load_package(PackageId id) {
|
|||
}
|
||||
|
||||
desc.file =
|
||||
td::actor::create_actor<ArchiveSlice>("slice", id.id, id.key, id.temp, false, db_root_, archive_lru_.get());
|
||||
td::actor::create_actor<ArchiveSlice>("slice", id.id, id.key, id.temp, false, db_root_, archive_lru_.get(), statistics_);
|
||||
|
||||
m.emplace(id, std::move(desc));
|
||||
update_permanent_slices();
|
||||
|
@ -636,7 +636,7 @@ const ArchiveManager::FileDescription *ArchiveManager::add_file_desc(ShardIdFull
|
|||
td::mkdir(db_root_ + id.path()).ensure();
|
||||
std::string prefix = PSTRING() << db_root_ << id.path() << id.name();
|
||||
new_desc.file =
|
||||
td::actor::create_actor<ArchiveSlice>("slice", id.id, id.key, id.temp, false, db_root_, archive_lru_.get());
|
||||
td::actor::create_actor<ArchiveSlice>("slice", id.id, id.key, id.temp, false, db_root_, archive_lru_.get(), statistics_);
|
||||
const FileDescription &desc = f.emplace(id, std::move(new_desc));
|
||||
if (!id.temp) {
|
||||
update_desc(f, desc, shard, seqno, ts, lt);
|
||||
|
@ -829,7 +829,10 @@ void ArchiveManager::start_up() {
|
|||
if (opts_->get_max_open_archive_files() > 0) {
|
||||
archive_lru_ = td::actor::create_actor<ArchiveLru>("archive_lru", opts_->get_max_open_archive_files());
|
||||
}
|
||||
index_ = std::make_shared<td::RocksDb>(td::RocksDb::open(db_root_ + "/files/globalindex").move_as_ok());
|
||||
if (!opts_->get_disable_rocksdb_stats()) {
|
||||
statistics_ = td::RocksDb::create_statistics();
|
||||
}
|
||||
index_ = std::make_shared<td::RocksDb>(td::RocksDb::open(db_root_ + "/files/globalindex", statistics_).move_as_ok());
|
||||
std::string value;
|
||||
auto v = index_->get(create_serialize_tl_object<ton_api::db_files_index_key>().as_slice(), value);
|
||||
v.ensure();
|
||||
|
@ -903,6 +906,28 @@ void ArchiveManager::start_up() {
|
|||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!opts_->get_disable_rocksdb_stats()) {
|
||||
alarm_timestamp() = td::Timestamp::in(60.0);
|
||||
}
|
||||
}
|
||||
|
||||
void ArchiveManager::alarm() {
|
||||
alarm_timestamp() = td::Timestamp::in(60.0);
|
||||
auto stats = td::RocksDb::statistics_to_string(statistics_);
|
||||
auto to_file_r = td::FileFd::open(db_root_ + "/db_stats.txt", td::FileFd::Truncate | td::FileFd::Create | td::FileFd::Write, 0644);
|
||||
if (to_file_r.is_error()) {
|
||||
LOG(ERROR) << "Failed to open db_stats.txt: " << to_file_r.move_as_error();
|
||||
return;
|
||||
}
|
||||
auto to_file = to_file_r.move_as_ok();
|
||||
auto res = to_file.write(stats);
|
||||
to_file.close();
|
||||
if (res.is_error()) {
|
||||
LOG(ERROR) << "Failed to write to db_stats.txt: " << res.move_as_error();
|
||||
return;
|
||||
}
|
||||
td::RocksDb::reset_statistics(statistics_);
|
||||
}
|
||||
|
||||
void ArchiveManager::run_gc(UnixTime mc_ts, UnixTime gc_ts, UnixTime archive_ttl) {
|
||||
|
|
|
@ -70,6 +70,7 @@ class ArchiveManager : public td::actor::Actor {
|
|||
td::Promise<td::BufferSlice> promise);
|
||||
|
||||
void start_up() override;
|
||||
void alarm() override;
|
||||
|
||||
void commit_transaction();
|
||||
void set_async_mode(bool mode, td::Promise<td::Unit> promise);
|
||||
|
@ -173,6 +174,8 @@ class ArchiveManager : public td::actor::Actor {
|
|||
bool huge_transaction_started_ = false;
|
||||
td::uint32 huge_transaction_size_ = 0;
|
||||
|
||||
std::shared_ptr<rocksdb::Statistics> statistics_;
|
||||
|
||||
FileMap &get_file_map(const PackageId &p) {
|
||||
return p.key ? key_files_ : p.temp ? temp_files_ : files_;
|
||||
}
|
||||
|
|
|
@ -465,7 +465,7 @@ void ArchiveSlice::get_archive_id(BlockSeqno masterchain_seqno, td::Promise<td::
|
|||
void ArchiveSlice::before_query() {
|
||||
if (status_ == st_closed) {
|
||||
LOG(DEBUG) << "Opening archive slice " << db_path_;
|
||||
kv_ = std::make_unique<td::RocksDb>(td::RocksDb::open(db_path_).move_as_ok());
|
||||
kv_ = std::make_unique<td::RocksDb>(td::RocksDb::open(db_path_, statistics_).move_as_ok());
|
||||
std::string value;
|
||||
auto R2 = kv_->get("status", value);
|
||||
R2.ensure();
|
||||
|
@ -604,14 +604,15 @@ void ArchiveSlice::set_async_mode(bool mode, td::Promise<td::Unit> promise) {
|
|||
}
|
||||
|
||||
ArchiveSlice::ArchiveSlice(td::uint32 archive_id, bool key_blocks_only, bool temp, bool finalized, std::string db_root,
|
||||
td::actor::ActorId<ArchiveLru> archive_lru)
|
||||
td::actor::ActorId<ArchiveLru> archive_lru, std::shared_ptr<rocksdb::Statistics> statistics)
|
||||
: archive_id_(archive_id)
|
||||
, key_blocks_only_(key_blocks_only)
|
||||
, temp_(temp)
|
||||
, finalized_(finalized)
|
||||
, p_id_(archive_id_, key_blocks_only_, temp_)
|
||||
, db_root_(std::move(db_root))
|
||||
, archive_lru_(std::move(archive_lru)) {
|
||||
, archive_lru_(std::move(archive_lru))
|
||||
, statistics_(statistics) {
|
||||
db_path_ = PSTRING() << db_root_ << p_id_.path() << p_id_.name() << ".index";
|
||||
}
|
||||
|
||||
|
|
|
@ -23,6 +23,10 @@
|
|||
#include "fileref.hpp"
|
||||
#include <map>
|
||||
|
||||
namespace rocksdb {
|
||||
class Statistics;
|
||||
}
|
||||
|
||||
namespace ton {
|
||||
|
||||
namespace validator {
|
||||
|
@ -81,7 +85,7 @@ class ArchiveLru;
|
|||
class ArchiveSlice : public td::actor::Actor {
|
||||
public:
|
||||
ArchiveSlice(td::uint32 archive_id, bool key_blocks_only, bool temp, bool finalized, std::string db_root,
|
||||
td::actor::ActorId<ArchiveLru> archive_lru);
|
||||
td::actor::ActorId<ArchiveLru> archive_lru, std::shared_ptr<rocksdb::Statistics> statistics = nullptr);
|
||||
|
||||
void get_archive_id(BlockSeqno masterchain_seqno, td::Promise<td::uint64> promise);
|
||||
|
||||
|
@ -151,6 +155,7 @@ class ArchiveSlice : public td::actor::Actor {
|
|||
|
||||
std::string db_root_;
|
||||
td::actor::ActorId<ArchiveLru> archive_lru_;
|
||||
std::shared_ptr<rocksdb::Statistics> statistics_;
|
||||
std::unique_ptr<td::KeyValue> kv_;
|
||||
|
||||
struct PackageInfo {
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
#include "rootdb.hpp"
|
||||
|
||||
#include "td/db/RocksDb.h"
|
||||
#include "td/utils/filesystem.h"
|
||||
|
||||
#include "ton/ton-tl.hpp"
|
||||
#include "ton/ton-io.hpp"
|
||||
|
@ -83,7 +84,12 @@ void CellDbIn::start_up() {
|
|||
};
|
||||
|
||||
CellDbBase::start_up();
|
||||
cell_db_ = std::make_shared<td::RocksDb>(td::RocksDb::open(path_).move_as_ok());
|
||||
if (!opts_->get_disable_rocksdb_stats()) {
|
||||
statistics_ = td::RocksDb::create_statistics();
|
||||
statistics_flush_at_ = td::Timestamp::in(60.0);
|
||||
}
|
||||
cell_db_ = std::make_shared<td::RocksDb>(td::RocksDb::open(path_, statistics_).move_as_ok());
|
||||
|
||||
|
||||
boc_ = vm::DynamicBagOfCellsDb::create();
|
||||
boc_->set_celldb_compress_depth(opts_->get_celldb_compress_depth());
|
||||
|
@ -155,7 +161,29 @@ void CellDbIn::get_cell_db_reader(td::Promise<std::shared_ptr<vm::CellDbReader>>
|
|||
promise.set_result(boc_->get_cell_db_reader());
|
||||
}
|
||||
|
||||
void CellDbIn::flush_db_stats() {
|
||||
auto stats = td::RocksDb::statistics_to_string(statistics_);
|
||||
auto to_file_r = td::FileFd::open(path_ + "/db_stats.txt", td::FileFd::Truncate | td::FileFd::Create | td::FileFd::Write, 0644);
|
||||
if (to_file_r.is_error()) {
|
||||
LOG(ERROR) << "Failed to open db_stats.txt: " << to_file_r.move_as_error();
|
||||
return;
|
||||
}
|
||||
auto to_file = to_file_r.move_as_ok();
|
||||
auto res = to_file.write(stats);
|
||||
to_file.close();
|
||||
if (res.is_error()) {
|
||||
LOG(ERROR) << "Failed to write to db_stats.txt: " << res.move_as_error();
|
||||
return;
|
||||
}
|
||||
td::RocksDb::reset_statistics(statistics_);
|
||||
}
|
||||
|
||||
void CellDbIn::alarm() {
|
||||
if (statistics_flush_at_ && statistics_flush_at_.is_in_past()) {
|
||||
statistics_flush_at_ = td::Timestamp::in(60.0);
|
||||
flush_db_stats();
|
||||
}
|
||||
|
||||
if (migrate_after_ && migrate_after_.is_in_past()) {
|
||||
migrate_cells();
|
||||
}
|
||||
|
|
|
@ -27,6 +27,10 @@
|
|||
#include "auto/tl/ton_api.h"
|
||||
#include "validator.h"
|
||||
|
||||
namespace rocksdb {
|
||||
class Statistics;
|
||||
}
|
||||
|
||||
namespace ton {
|
||||
|
||||
namespace validator {
|
||||
|
@ -56,6 +60,8 @@ class CellDbIn : public CellDbBase {
|
|||
|
||||
void migrate_cell(td::Bits256 hash);
|
||||
|
||||
void flush_db_stats();
|
||||
|
||||
CellDbIn(td::actor::ActorId<RootDb> root_db, td::actor::ActorId<CellDb> parent, std::string path,
|
||||
td::Ref<ValidatorManagerOptions> opts);
|
||||
|
||||
|
@ -103,6 +109,8 @@ class CellDbIn : public CellDbBase {
|
|||
|
||||
std::unique_ptr<vm::DynamicBagOfCellsDb> boc_;
|
||||
std::shared_ptr<vm::KeyValue> cell_db_;
|
||||
std::shared_ptr<rocksdb::Statistics> statistics_;
|
||||
td::Timestamp statistics_flush_at_ = td::Timestamp::never();
|
||||
|
||||
std::function<void(const vm::CellLoader::LoadResult&)> on_load_callback_;
|
||||
std::set<td::Bits256> cells_to_migrate_;
|
||||
|
|
|
@ -110,7 +110,7 @@ class LiteServerCacheImpl : public LiteServerCache {
|
|||
std::set<td::Bits256> send_message_cache_;
|
||||
size_t send_message_error_cnt_ = 0;
|
||||
|
||||
static const size_t MAX_CACHE_SIZE = 64 << 20;
|
||||
static constexpr size_t MAX_CACHE_SIZE = 64 << 20;
|
||||
};
|
||||
|
||||
} // namespace ton::validator
|
||||
|
|
|
@ -233,6 +233,9 @@ void LiteQuery::perform() {
|
|||
[&](lite_api::liteServer_lookupBlock& q) {
|
||||
this->perform_lookupBlock(ton::create_block_id_simple(q.id_), q.mode_, q.lt_, q.utime_);
|
||||
},
|
||||
[&](lite_api::liteServer_lookupBlockWithProof& q) {
|
||||
this->perform_lookupBlockWithProof(ton::create_block_id_simple(q.id_), ton::create_block_id(q.mc_block_id_), q.mode_, q.lt_, q.utime_);
|
||||
},
|
||||
[&](lite_api::liteServer_listBlockTransactions& q) {
|
||||
this->perform_listBlockTransactions(ton::create_block_id(q.id_), q.mode_, q.count_,
|
||||
(q.mode_ & 128) ? q.after_->account_ : td::Bits256::zero(),
|
||||
|
@ -266,6 +269,9 @@ void LiteQuery::perform() {
|
|||
[&](lite_api::liteServer_getLibraries& q) {
|
||||
this->perform_getLibraries(q.library_list_);
|
||||
},
|
||||
[&](lite_api::liteServer_getLibrariesWithProof& q) {
|
||||
this->perform_getLibrariesWithProof(ton::create_block_id(q.id_), q.mode_, q.library_list_);
|
||||
},
|
||||
[&](lite_api::liteServer_getShardBlockProof& q) {
|
||||
this->perform_getShardBlockProof(create_block_id(q.id_));
|
||||
},
|
||||
|
@ -964,6 +970,100 @@ void LiteQuery::continue_getLibraries(Ref<ton::validator::MasterchainState> mc_s
|
|||
finish_query(std::move(b));
|
||||
}
|
||||
|
||||
void LiteQuery::perform_getLibrariesWithProof(BlockIdExt blkid, int mode, std::vector<td::Bits256> library_list) {
|
||||
LOG(INFO) << "started a getLibrariesWithProof(<list of " << library_list.size() << " parameters>) liteserver query";
|
||||
if (library_list.size() > 16) {
|
||||
LOG(INFO) << "too many libraries requested, returning only first 16";
|
||||
library_list.resize(16);
|
||||
}
|
||||
sort( library_list.begin(), library_list.end() );
|
||||
library_list.erase( unique( library_list.begin(), library_list.end() ), library_list.end() );
|
||||
|
||||
set_continuation([this, library_list, mode]() -> void { continue_getLibrariesWithProof(library_list, mode); });
|
||||
request_mc_block_data_state(blkid);
|
||||
}
|
||||
|
||||
void LiteQuery::continue_getLibrariesWithProof(std::vector<td::Bits256> library_list, int mode) {
|
||||
LOG(INFO) << "obtained masterchain block = " << base_blk_id_.to_str();
|
||||
CHECK(mc_state_.not_null());
|
||||
|
||||
Ref<vm::Cell> state_proof, data_proof;
|
||||
if (!make_mc_state_root_proof(state_proof)) {
|
||||
return;
|
||||
}
|
||||
|
||||
vm::MerkleProofBuilder pb{mc_state_->root_cell()};
|
||||
block::gen::ShardStateUnsplit::Record state;
|
||||
if (!tlb::unpack_cell(pb.root(), state)) {
|
||||
fatal_error("cannot unpack header of shardchain state "s + base_blk_id_.to_str());
|
||||
}
|
||||
auto libraries_dict = vm::Dictionary(state.r1.libraries->prefetch_ref(), 256);
|
||||
|
||||
std::vector<ton::tl_object_ptr<ton::lite_api::liteServer_libraryEntry>> result;
|
||||
std::vector<td::Bits256> result_hashes;
|
||||
for (const auto& hash : library_list) {
|
||||
LOG(INFO) << "looking for library " << hash.to_hex();
|
||||
|
||||
auto csr = libraries_dict.lookup(hash.bits(), 256);
|
||||
if (csr.is_null() || csr->prefetch_ulong(2) != 0 || !csr->have_refs()) { // shared_lib_descr$00 lib:^Cell
|
||||
continue;
|
||||
}
|
||||
block::gen::LibDescr::Record libdescr;
|
||||
if (!tlb::csr_unpack(csr, libdescr)) {
|
||||
fatal_error("cannot unpack LibDescr record "s + hash.to_hex());
|
||||
return;
|
||||
}
|
||||
if (mode & 1) {
|
||||
// include first 16 publishers in the proof
|
||||
auto publishers_dict = vm::Dictionary{vm::DictNonEmpty(), libdescr.publishers, 256};
|
||||
auto iter = publishers_dict.begin();
|
||||
constexpr int max_publishers = 15; // set to 15 because publishers_dict.begin() counts as the first visit
|
||||
for (int i = 0; i < max_publishers && iter != publishers_dict.end(); ++i, ++iter) {}
|
||||
}
|
||||
|
||||
result_hashes.push_back(hash);
|
||||
}
|
||||
|
||||
auto data_proof_boc = pb.extract_proof_boc();
|
||||
if (data_proof_boc.is_error()) {
|
||||
fatal_error(data_proof_boc.move_as_error());
|
||||
return;
|
||||
}
|
||||
auto state_proof_boc = vm::std_boc_serialize(std::move(state_proof));
|
||||
if (state_proof_boc.is_error()) {
|
||||
fatal_error(state_proof_boc.move_as_error());
|
||||
return;
|
||||
}
|
||||
|
||||
for (const auto& hash : result_hashes) {
|
||||
auto csr = libraries_dict.lookup(hash.bits(), 256);
|
||||
block::gen::LibDescr::Record libdescr;
|
||||
if (!tlb::csr_unpack(csr, libdescr)) {
|
||||
fatal_error("cannot unpack LibDescr record "s + hash.to_hex());
|
||||
return;
|
||||
}
|
||||
if (!libdescr.lib->get_hash().bits().equals(hash.bits(), 256)) {
|
||||
LOG(ERROR) << "public library hash mismatch: expected " << hash.to_hex() << " , found "
|
||||
<< libdescr.lib->get_hash().to_hex();
|
||||
continue;
|
||||
}
|
||||
td::BufferSlice libdata;
|
||||
if (!(mode & 2)) {
|
||||
auto data = vm::std_boc_serialize(libdescr.lib);
|
||||
if (data.is_error()) {
|
||||
LOG(WARNING) << "library serialization failed: " << data.move_as_error().to_string();
|
||||
continue;
|
||||
}
|
||||
libdata = data.move_as_ok();
|
||||
}
|
||||
result.push_back(ton::create_tl_object<ton::lite_api::liteServer_libraryEntry>(hash, std::move(libdata)));
|
||||
}
|
||||
|
||||
auto b = ton::create_serialize_tl_object<ton::lite_api::liteServer_libraryResultWithProof>(ton::create_tl_lite_block_id(base_blk_id_), mode, std::move(result),
|
||||
state_proof_boc.move_as_ok(), data_proof_boc.move_as_ok());
|
||||
finish_query(std::move(b));
|
||||
}
|
||||
|
||||
void LiteQuery::perform_getOneTransaction(BlockIdExt blkid, WorkchainId workchain, StdSmcAddress addr, LogicalTime lt) {
|
||||
LOG(INFO) << "started a getOneTransaction(" << blkid.to_str() << ", " << workchain << ", " << addr.to_hex() << ","
|
||||
<< lt << ") liteserver query";
|
||||
|
@ -1333,8 +1433,8 @@ static td::Ref<vm::Tuple> prepare_vm_c7(ton::UnixTime now, ton::LogicalTime lt,
|
|||
td::make_refint(lt), // trans_lt:Integer
|
||||
std::move(rand_seed_int), // rand_seed:Integer
|
||||
balance.as_vm_tuple(), // balance_remaining:[Integer (Maybe Cell)]
|
||||
my_addr, // myself:MsgAddressInt
|
||||
config ? config->get_root_cell() : vm::StackEntry() // global_config:(Maybe Cell) ] = SmartContractInfo;
|
||||
my_addr, // myself:MsgAddressInt
|
||||
config ? config->get_root_cell() : vm::StackEntry() // global_config:(Maybe Cell) ] = SmartContractInfo;
|
||||
};
|
||||
if (config && config->get_global_version() >= 4) {
|
||||
tuple.push_back(my_code); // code:Cell
|
||||
|
@ -1350,6 +1450,9 @@ static td::Ref<vm::Tuple> prepare_vm_c7(ton::UnixTime now, ton::LogicalTime lt,
|
|||
if (config && config->get_global_version() >= 6) {
|
||||
tuple.push_back(config->get_unpacked_config_tuple(now)); // unpacked_config_tuple:[...]
|
||||
tuple.push_back(due_payment); // due_payment:Integer
|
||||
// precomiled_gas_usage:(Maybe Integer)
|
||||
auto precompiled = config->get_precompiled_contracts_config().get_contract(my_code->get_hash().bits());
|
||||
tuple.push_back(precompiled ? td::make_refint(precompiled.value().gas_usage) : vm::StackEntry());
|
||||
}
|
||||
auto tuple_ref = td::make_cnt_ref<std::vector<vm::StackEntry>>(std::move(tuple));
|
||||
LOG(DEBUG) << "SmartContractInfo initialized with " << vm::StackEntry(tuple_ref).to_string();
|
||||
|
@ -1430,6 +1533,7 @@ void LiteQuery::finish_runSmcMethod(td::BufferSlice shard_proof, td::BufferSlice
|
|||
auto c7 = prepare_vm_c7(gen_utime, gen_lt, td::make_ref<vm::CellSlice>(acc.addr->clone()), balance, config.get(),
|
||||
std::move(code), due_payment);
|
||||
vm.set_c7(c7); // tuple with SmartContractInfo
|
||||
vm.set_global_version(config->get_global_version());
|
||||
// vm.incr_stack_trace(1); // enable stack dump after each step
|
||||
LOG(INFO) << "starting VM to run GET-method of smart contract " << acc_workchain_ << ":" << acc_addr_.to_hex();
|
||||
// **** RUN VM ****
|
||||
|
@ -1849,7 +1953,7 @@ void LiteQuery::continue_getConfigParams(int mode, std::vector<int> param_list)
|
|||
void LiteQuery::perform_getAllShardsInfo(BlockIdExt blkid) {
|
||||
LOG(INFO) << "started a getAllShardsInfo(" << blkid.to_str() << ") liteserver query";
|
||||
set_continuation([&]() -> void { continue_getAllShardsInfo(); });
|
||||
request_mc_block_data_state(blkid);
|
||||
request_mc_block_data(blkid);
|
||||
}
|
||||
|
||||
void LiteQuery::continue_getShardInfo(ShardIdFull shard, bool exact) {
|
||||
|
@ -1896,30 +2000,30 @@ void LiteQuery::continue_getShardInfo(ShardIdFull shard, bool exact) {
|
|||
|
||||
void LiteQuery::continue_getAllShardsInfo() {
|
||||
LOG(INFO) << "completing getAllShardsInfo() query";
|
||||
Ref<vm::Cell> proof1, proof2;
|
||||
if (!make_mc_state_root_proof(proof1)) {
|
||||
vm::MerkleProofBuilder mpb{mc_block_->root_cell()};
|
||||
block::gen::Block::Record blk;
|
||||
block::gen::BlockExtra::Record extra;
|
||||
block::gen::McBlockExtra::Record mc_extra;
|
||||
if (!tlb::unpack_cell(mpb.root(), blk) || !tlb::unpack_cell(blk.extra, extra) || !extra.custom->have_refs() ||
|
||||
!tlb::unpack_cell(extra.custom->prefetch_ref(), mc_extra)) {
|
||||
fatal_error("cannot unpack header of block "s + mc_block_->block_id().to_str());
|
||||
return;
|
||||
}
|
||||
vm::MerkleProofBuilder mpb{mc_state_->root_cell()};
|
||||
auto shards_dict = block::ShardConfig::extract_shard_hashes_dict(mpb.root());
|
||||
if (!shards_dict) {
|
||||
fatal_error("cannot extract ShardHashes from last mc state");
|
||||
return;
|
||||
}
|
||||
if (!mpb.extract_proof_to(proof2)) {
|
||||
vm::Dictionary shards_dict(std::move(mc_extra.shard_hashes), 32);
|
||||
Ref<vm::Cell> proof;
|
||||
if (!mpb.extract_proof_to(proof)) {
|
||||
fatal_error("cannot construct Merkle proof for all shards dictionary");
|
||||
return;
|
||||
}
|
||||
shards_dict = block::ShardConfig::extract_shard_hashes_dict(mc_state_->root_cell());
|
||||
vm::CellBuilder cb;
|
||||
Ref<vm::Cell> cell;
|
||||
if (!(std::move(shards_dict)->append_dict_to_bool(cb) && cb.finalize_to(cell))) {
|
||||
fatal_error("cannot store ShardHashes from last mc state into a new cell");
|
||||
auto proof_boc = vm::std_boc_serialize(std::move(proof));
|
||||
if (proof_boc.is_error()) {
|
||||
fatal_error(proof_boc.move_as_error());
|
||||
return;
|
||||
}
|
||||
auto proof = vm::std_boc_serialize_multi({std::move(proof1), std::move(proof2)});
|
||||
if (proof.is_error()) {
|
||||
fatal_error(proof.move_as_error());
|
||||
vm::CellBuilder cb;
|
||||
Ref<vm::Cell> cell;
|
||||
if (!(shards_dict.append_dict_to_bool(cb) && cb.finalize_to(cell))) {
|
||||
fatal_error("cannot store ShardHashes from last mc block into a new cell");
|
||||
return;
|
||||
}
|
||||
auto data = vm::std_boc_serialize(std::move(cell));
|
||||
|
@ -1929,10 +2033,307 @@ void LiteQuery::continue_getAllShardsInfo() {
|
|||
}
|
||||
LOG(INFO) << "getAllShardInfo() query completed";
|
||||
auto b = ton::create_serialize_tl_object<ton::lite_api::liteServer_allShardsInfo>(
|
||||
ton::create_tl_lite_block_id(base_blk_id_), proof.move_as_ok(), data.move_as_ok());
|
||||
ton::create_tl_lite_block_id(base_blk_id_), proof_boc.move_as_ok(), data.move_as_ok());
|
||||
finish_query(std::move(b));
|
||||
}
|
||||
|
||||
void LiteQuery::perform_lookupBlockWithProof(BlockId blkid, BlockIdExt mc_blkid, int mode, LogicalTime lt, UnixTime utime) {
|
||||
if (!((1 << (mode & 7)) & 0x16)) {
|
||||
fatal_error("exactly one of mode.0, mode.1 and mode.2 bits must be set");
|
||||
return;
|
||||
}
|
||||
if (!mc_blkid.is_masterchain_ext()) {
|
||||
fatal_error("masterchain block id must be specified");
|
||||
return;
|
||||
}
|
||||
if (!(mode & 1)) {
|
||||
blkid.seqno = 0;
|
||||
}
|
||||
if (!(mode & 2)) {
|
||||
lt = 0;
|
||||
}
|
||||
if (!(mode & 4)) {
|
||||
utime = 0;
|
||||
}
|
||||
mode_ = mode;
|
||||
base_blk_id_ = mc_blkid;
|
||||
LOG(INFO) << "started a lookupBlockWithProof(" << blkid.to_str() << ", " << mc_blkid.to_str() << ", " << mode << ", "
|
||||
<< lt << ", " << utime << ") liteserver query";
|
||||
|
||||
ton::AccountIdPrefixFull pfx{blkid.workchain, blkid.shard};
|
||||
auto P = td::PromiseCreator::lambda(
|
||||
[Self = actor_id(this), mc_blkid, manager = manager_, mode, pfx](td::Result<ConstBlockHandle> res) {
|
||||
if (res.is_error()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, res.move_as_error());
|
||||
return;
|
||||
}
|
||||
auto handle = res.move_as_ok();
|
||||
if (!handle->inited_masterchain_ref_block()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, td::Status::Error("block doesn't have masterchain ref"));
|
||||
return;
|
||||
}
|
||||
if (handle->masterchain_ref_block() > mc_blkid.seqno()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, td::Status::Error("specified mc block is older than block's masterchain ref"));
|
||||
return;
|
||||
}
|
||||
LOG(DEBUG) << "requesting data for block " << handle->id().to_str();
|
||||
td::actor::send_closure_later(manager, &ValidatorManager::get_block_data_from_db, handle,
|
||||
[Self, mc_ref_blkid = handle->masterchain_ref_block(), mc_blkid, pfx, mode](td::Result<Ref<BlockData>> res) {
|
||||
if (res.is_error()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, res.move_as_error());
|
||||
} else {
|
||||
td::actor::send_closure_later(Self, &LiteQuery::continue_lookupBlockWithProof_getHeaderProof, res.move_as_ok(), pfx, mc_ref_blkid);
|
||||
}
|
||||
});
|
||||
});
|
||||
|
||||
if (mode & 2) {
|
||||
td::actor::send_closure_later(manager_, &ValidatorManager::get_block_by_lt_from_db_for_litequery, pfx, lt, std::move(P));
|
||||
} else if (mode & 4) {
|
||||
td::actor::send_closure_later(manager_, &ValidatorManager::get_block_by_unix_time_from_db_for_litequery, pfx, utime, std::move(P));
|
||||
} else {
|
||||
td::actor::send_closure_later(manager_, &ValidatorManager::get_block_by_seqno_from_db_for_litequery, pfx, blkid.seqno, std::move(P));
|
||||
}
|
||||
}
|
||||
|
||||
void LiteQuery::continue_lookupBlockWithProof_getHeaderProof(Ref<ton::validator::BlockData> block, AccountIdPrefixFull req_prefix, BlockSeqno masterchain_ref_seqno) {
|
||||
blk_id_ = block->block_id();
|
||||
LOG(INFO) << "obtained data for getBlockHeader(" << blk_id_.to_str() << ", " << mode_ << ")";
|
||||
CHECK(block.not_null());
|
||||
auto block_root = block->root_cell();
|
||||
if (block_root.is_null()) {
|
||||
fatal_error("block has no valid root cell");
|
||||
return;
|
||||
}
|
||||
|
||||
vm::MerkleProofBuilder mpb{block_root};
|
||||
std::vector<BlockIdExt> prev;
|
||||
BlockIdExt mc_blkid;
|
||||
bool after_split;
|
||||
td::Status S = block::unpack_block_prev_blk_try(mpb.root(), blk_id_, prev, mc_blkid, after_split);
|
||||
if (S.is_error()) {
|
||||
fatal_error(std::move(S));
|
||||
return;
|
||||
}
|
||||
auto proof_data = mpb.extract_proof_boc();
|
||||
if (proof_data.is_error()) {
|
||||
fatal_error(proof_data.move_as_error());
|
||||
return;
|
||||
}
|
||||
lookup_header_proof_ = proof_data.move_as_ok();
|
||||
|
||||
bool include_prev = mode_ & 6;
|
||||
if (include_prev) {
|
||||
BlockIdExt prev_blkid;
|
||||
for (auto& p : prev) {
|
||||
if (ton::shard_contains(p.shard_full(), req_prefix)) {
|
||||
prev_blkid = p;
|
||||
}
|
||||
}
|
||||
CHECK(prev_blkid.is_valid());
|
||||
get_block_handle_checked(prev_blkid, [Self = actor_id(this), masterchain_ref_seqno, manager = manager_](td::Result<ConstBlockHandle> R) mutable {
|
||||
if (R.is_error()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, R.move_as_error());
|
||||
return;
|
||||
}
|
||||
td::actor::send_closure(manager, &ValidatorManager::get_block_data_from_db, R.move_as_ok(),
|
||||
[Self, masterchain_ref_seqno](td::Result<Ref<BlockData>> res) mutable {
|
||||
if (res.is_error()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, res.move_as_error());
|
||||
return;
|
||||
}
|
||||
td::actor::send_closure(Self, &LiteQuery::continue_lookupBlockWithProof_gotPrevBlockData, res.move_as_ok(), masterchain_ref_seqno);
|
||||
});
|
||||
});
|
||||
} else {
|
||||
continue_lookupBlockWithProof_gotPrevBlockData(Ref<BlockData>(), masterchain_ref_seqno);
|
||||
}
|
||||
}
|
||||
|
||||
void LiteQuery::continue_lookupBlockWithProof_gotPrevBlockData(Ref<BlockData> prev_block, BlockSeqno masterchain_ref_seqno) {
|
||||
if (prev_block.not_null()) {
|
||||
CHECK(prev_block.not_null());
|
||||
if (prev_block->root_cell().is_null()) {
|
||||
fatal_error("block has no valid root cell");
|
||||
return;
|
||||
}
|
||||
vm::MerkleProofBuilder mpb{prev_block->root_cell()};
|
||||
block::gen::Block::Record blk;
|
||||
block::gen::BlockInfo::Record info;
|
||||
if (!(tlb::unpack_cell(mpb.root(), blk) && tlb::unpack_cell(blk.info, info))) {
|
||||
fatal_error(td::Status::Error("cannot unpack prev block header"));
|
||||
return;
|
||||
}
|
||||
auto proof_data = mpb.extract_proof_boc();
|
||||
if (proof_data.is_error()) {
|
||||
fatal_error(proof_data.move_as_error());
|
||||
return;
|
||||
}
|
||||
lookup_prev_header_proof_ = proof_data.move_as_ok();
|
||||
}
|
||||
|
||||
if (!blk_id_.is_masterchain()) {
|
||||
ton::AccountIdPrefixFull pfx{ton::masterchainId, ton::shardIdAll};
|
||||
td::actor::send_closure_later(manager_, &ValidatorManager::get_block_by_seqno_from_db, pfx, masterchain_ref_seqno,
|
||||
[manager = manager_, Self = actor_id(this)](td::Result<ConstBlockHandle> R) mutable {
|
||||
if (R.is_error()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, R.move_as_error());
|
||||
return;
|
||||
}
|
||||
td::actor::send_closure(manager, &ValidatorManager::get_block_data_from_db, R.move_as_ok(),
|
||||
[Self](td::Result<Ref<BlockData>> res) mutable {
|
||||
if (res.is_error()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, res.move_as_error());
|
||||
return;
|
||||
}
|
||||
td::actor::send_closure(Self, &LiteQuery::continue_lookupBlockWithProof_buildProofLinks, res.move_as_ok(), std::vector<std::pair<BlockIdExt, td::Ref<vm::Cell>>>());
|
||||
});
|
||||
});
|
||||
} else {
|
||||
base_blk_id_alt_ = blk_id_;
|
||||
td::actor::send_closure(actor_id(this), &LiteQuery::continue_lookupBlockWithProof_getClientMcBlockDataState, std::vector<std::pair<BlockIdExt, Ref<vm::Cell>>>());
|
||||
}
|
||||
}
|
||||
|
||||
void LiteQuery::continue_lookupBlockWithProof_buildProofLinks(td::Ref<BlockData> cur_block,
|
||||
std::vector<std::pair<BlockIdExt, td::Ref<vm::Cell>>> result) {
|
||||
BlockIdExt cur_id = cur_block->block_id();
|
||||
BlockIdExt prev_id;
|
||||
vm::MerkleProofBuilder mpb{cur_block->root_cell()};
|
||||
if (cur_id.is_masterchain()) {
|
||||
base_blk_id_alt_ = cur_id;
|
||||
block::gen::Block::Record blk;
|
||||
block::gen::BlockExtra::Record extra;
|
||||
block::gen::McBlockExtra::Record mc_extra;
|
||||
if (!tlb::unpack_cell(mpb.root(), blk) || !tlb::unpack_cell(blk.extra, extra) || !extra.custom->have_refs() ||
|
||||
!tlb::unpack_cell(extra.custom->prefetch_ref(), mc_extra)) {
|
||||
fatal_error("cannot unpack header of block "s + cur_id.to_str());
|
||||
return;
|
||||
}
|
||||
block::ShardConfig shards(mc_extra.shard_hashes->prefetch_ref());
|
||||
ShardIdFull shard_id = blk_id_.shard_full();
|
||||
shard_id.shard = (shard_id.shard & ~(1 << (63 - shard_id.pfx_len()))) | 1;
|
||||
Ref<block::McShardHash> shard_hash = shards.get_shard_hash(shard_id, false);
|
||||
if (shard_hash.is_null()) {
|
||||
fatal_error("shard not found");
|
||||
return;
|
||||
}
|
||||
prev_id = shard_hash->top_block_id();
|
||||
} else {
|
||||
std::vector<BlockIdExt> prev;
|
||||
BlockIdExt mc_blkid;
|
||||
bool after_split;
|
||||
td::Status S = block::unpack_block_prev_blk_try(mpb.root(), cur_id, prev, mc_blkid, after_split);
|
||||
if (S.is_error()) {
|
||||
fatal_error(std::move(S));
|
||||
return;
|
||||
}
|
||||
bool found = false;
|
||||
for (const BlockIdExt& id : prev) {
|
||||
if (shard_intersects(id.shard_full(), blk_id_.shard_full())) {
|
||||
found = true;
|
||||
prev_id = id;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!found) {
|
||||
fatal_error("failed to find block chain");
|
||||
return;
|
||||
}
|
||||
}
|
||||
auto proof = mpb.extract_proof();
|
||||
if (proof.is_error()) {
|
||||
fatal_error(proof.move_as_error_prefix("cannot serialize Merkle proof : "));
|
||||
return;
|
||||
}
|
||||
result.emplace_back(prev_id, proof.move_as_ok());
|
||||
|
||||
if (prev_id == blk_id_) {
|
||||
CHECK(base_blk_id_alt_.is_masterchain());
|
||||
if (base_blk_id_alt_ != base_blk_id_) {
|
||||
continue_lookupBlockWithProof_getClientMcBlockDataState(std::move(result));
|
||||
} else {
|
||||
continue_lookupBlockWithProof_getMcBlockPrev(std::move(result));
|
||||
}
|
||||
return;
|
||||
}
|
||||
if (result.size() == 8) {
|
||||
// Chains of shardblocks between masterchain blocks can't be longer than 8 (see collator.cpp:991)
|
||||
fatal_error("proof chain is too long");
|
||||
return;
|
||||
}
|
||||
|
||||
td::actor::send_closure_later(
|
||||
manager_, &ValidatorManager::get_block_data_from_db_short, prev_id,
|
||||
[Self = actor_id(this), result = std::move(result)](td::Result<Ref<BlockData>> R) mutable {
|
||||
if (R.is_error()) {
|
||||
td::actor::send_closure(Self, &LiteQuery::abort_query, R.move_as_error());
|
||||
} else {
|
||||
td::actor::send_closure_later(Self, &LiteQuery::continue_lookupBlockWithProof_buildProofLinks, R.move_as_ok(),
|
||||
std::move(result));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void LiteQuery::continue_lookupBlockWithProof_getClientMcBlockDataState(std::vector<std::pair<BlockIdExt, td::Ref<vm::Cell>>> links) {
|
||||
set_continuation([this, links = std::move(links)]() -> void {
|
||||
continue_lookupBlockWithProof_getMcBlockPrev(std::move(links));
|
||||
});
|
||||
request_mc_block_data_state(base_blk_id_);
|
||||
}
|
||||
|
||||
void LiteQuery::continue_lookupBlockWithProof_getMcBlockPrev(std::vector<std::pair<BlockIdExt, td::Ref<vm::Cell>>> links) {
|
||||
td::BufferSlice mc_state_proof_buf, client_mc_blk_proof_buf;
|
||||
|
||||
if (base_blk_id_alt_ != base_blk_id_) {
|
||||
vm::MerkleProofBuilder mpb{mc_state_->root_cell()};
|
||||
auto prev_blocks_dict = block::get_prev_blocks_dict(mpb.root());
|
||||
if (!prev_blocks_dict) {
|
||||
fatal_error(td::Status::Error("cannot extract prev_blocks from mc state"));
|
||||
return;
|
||||
}
|
||||
if (!block::check_old_mc_block_id(*prev_blocks_dict, base_blk_id_alt_)) {
|
||||
fatal_error(td::Status::Error("client mc blkid is not in prev_blocks"));
|
||||
return;
|
||||
}
|
||||
auto client_mc_blk_proof = mpb.extract_proof_boc();
|
||||
if (client_mc_blk_proof.is_error()) {
|
||||
fatal_error(client_mc_blk_proof.move_as_error());
|
||||
return;
|
||||
}
|
||||
client_mc_blk_proof_buf = client_mc_blk_proof.move_as_ok();
|
||||
|
||||
Ref<vm::Cell> mc_state_proof;
|
||||
if (!make_mc_state_root_proof(mc_state_proof)) {
|
||||
fatal_error(td::Status::Error("cannot create Merkle proof for mc state"));
|
||||
return;
|
||||
}
|
||||
auto mc_state_proof_boc = vm::std_boc_serialize(std::move(mc_state_proof));
|
||||
if (mc_state_proof_boc.is_error()) {
|
||||
fatal_error(mc_state_proof_boc.move_as_error());
|
||||
return;
|
||||
}
|
||||
mc_state_proof_buf = mc_state_proof_boc.move_as_ok();
|
||||
}
|
||||
|
||||
std::vector<tl_object_ptr<lite_api::liteServer_shardBlockLink>> links_res;
|
||||
for (auto& p : links) {
|
||||
auto prev_block_proof = vm::std_boc_serialize(std::move(p.second));
|
||||
if (prev_block_proof.is_error()) {
|
||||
fatal_error(prev_block_proof.move_as_error());
|
||||
return;
|
||||
}
|
||||
links_res.push_back(
|
||||
create_tl_object<lite_api::liteServer_shardBlockLink>(create_tl_lite_block_id(p.first), prev_block_proof.move_as_ok()));
|
||||
}
|
||||
|
||||
auto b = ton::create_serialize_tl_object<ton::lite_api::liteServer_lookupBlockResult>(ton::create_tl_lite_block_id(blk_id_),
|
||||
mode_, ton::create_tl_lite_block_id(base_blk_id_alt_), std::move(mc_state_proof_buf), std::move(client_mc_blk_proof_buf),
|
||||
std::move(links_res), std::move(lookup_header_proof_), std::move(lookup_prev_header_proof_));
|
||||
finish_query(std::move(b));
|
||||
}
|
||||
|
||||
|
||||
void LiteQuery::perform_lookupBlock(BlockId blkid, int mode, LogicalTime lt, UnixTime utime) {
|
||||
if (!((1 << (mode & 7)) & 0x16)) {
|
||||
fatal_error("exactly one of mode.0, mode.1 and mode.2 bits must be set");
|
||||
|
|
|
@ -69,6 +69,9 @@ class LiteQuery : public td::actor::Actor {
|
|||
std::unique_ptr<block::BlockProofChain> chain_;
|
||||
Ref<vm::Stack> stack_;
|
||||
|
||||
td::BufferSlice lookup_header_proof_;
|
||||
td::BufferSlice lookup_prev_header_proof_;
|
||||
|
||||
public:
|
||||
enum {
|
||||
default_timeout_msec = 4500, // 4.5 seconds
|
||||
|
@ -124,6 +127,8 @@ class LiteQuery : public td::actor::Actor {
|
|||
UnixTime gen_utime, LogicalTime gen_lt);
|
||||
void perform_getLibraries(std::vector<td::Bits256> library_list);
|
||||
void continue_getLibraries(Ref<MasterchainState> mc_state, BlockIdExt blkid, std::vector<td::Bits256> library_list);
|
||||
void perform_getLibrariesWithProof(BlockIdExt blkid, int mode, std::vector<td::Bits256> library_list);
|
||||
void continue_getLibrariesWithProof(std::vector<td::Bits256> library_list, int mode);
|
||||
void perform_getOneTransaction(BlockIdExt blkid, WorkchainId workchain, StdSmcAddress addr, LogicalTime lt);
|
||||
void continue_getOneTransaction();
|
||||
void perform_getTransactions(WorkchainId workchain, StdSmcAddress addr, LogicalTime lt, Bits256 hash, unsigned count);
|
||||
|
@ -138,6 +143,12 @@ class LiteQuery : public td::actor::Actor {
|
|||
void perform_getConfigParams(BlockIdExt blkid, int mode, std::vector<int> param_list = {});
|
||||
void continue_getConfigParams(int mode, std::vector<int> param_list);
|
||||
void perform_lookupBlock(BlockId blkid, int mode, LogicalTime lt, UnixTime utime);
|
||||
void perform_lookupBlockWithProof(BlockId blkid, BlockIdExt client_mc_blkid, int mode, LogicalTime lt, UnixTime utime);
|
||||
void continue_lookupBlockWithProof_getHeaderProof(Ref<ton::validator::BlockData> block, AccountIdPrefixFull req_prefix, BlockSeqno masterchain_ref_seqno);
|
||||
void continue_lookupBlockWithProof_gotPrevBlockData(Ref<BlockData> prev_block, BlockSeqno masterchain_ref_seqno);
|
||||
void continue_lookupBlockWithProof_buildProofLinks(td::Ref<BlockData> cur_block, std::vector<std::pair<BlockIdExt, td::Ref<vm::Cell>>> result);
|
||||
void continue_lookupBlockWithProof_getClientMcBlockDataState(std::vector<std::pair<BlockIdExt, td::Ref<vm::Cell>>> links);
|
||||
void continue_lookupBlockWithProof_getMcBlockPrev(std::vector<std::pair<BlockIdExt, td::Ref<vm::Cell>>> links);
|
||||
void perform_listBlockTransactions(BlockIdExt blkid, int mode, int count, Bits256 account, LogicalTime lt);
|
||||
void finish_listBlockTransactions(int mode, int count);
|
||||
void perform_listBlockTransactionsExt(BlockIdExt blkid, int mode, int count, Bits256 account, LogicalTime lt);
|
||||
|
|
|
@ -314,7 +314,8 @@ bool AsyncStateSerializer::need_serialize(BlockHandle handle) {
|
|||
if (handle->id().id.seqno == 0 || !handle->is_key_block()) {
|
||||
return false;
|
||||
}
|
||||
return ValidatorManager::is_persistent_state(handle->unix_time(), last_key_block_ts_);
|
||||
return ValidatorManager::is_persistent_state(handle->unix_time(), last_key_block_ts_) &&
|
||||
ValidatorManager::persistent_state_ttl(handle->unix_time()) > (UnixTime)td::Clocks::system();
|
||||
}
|
||||
|
||||
} // namespace validator
|
||||
|
|
|
@ -121,6 +121,9 @@ struct ValidatorManagerOptionsImpl : public ValidatorManagerOptions {
|
|||
double get_archive_preload_period() const override {
|
||||
return archive_preload_period_;
|
||||
}
|
||||
bool get_disable_rocksdb_stats() const override {
|
||||
return disable_rocksdb_stats_;
|
||||
}
|
||||
ValidatorMode validator_mode() const override {
|
||||
return validator_mode_;
|
||||
}
|
||||
|
@ -186,6 +189,9 @@ struct ValidatorManagerOptionsImpl : public ValidatorManagerOptions {
|
|||
void set_archive_preload_period(double value) override {
|
||||
archive_preload_period_ = value;
|
||||
}
|
||||
void set_disable_rocksdb_stats(bool value) override {
|
||||
disable_rocksdb_stats_ = value;
|
||||
}
|
||||
void set_validator_mode(ValidatorMode value) override {
|
||||
validator_mode_ = value;
|
||||
}
|
||||
|
@ -232,6 +238,7 @@ struct ValidatorManagerOptionsImpl : public ValidatorManagerOptions {
|
|||
td::uint32 celldb_compress_depth_{0};
|
||||
size_t max_open_archive_files_ = 0;
|
||||
double archive_preload_period_ = 0.0;
|
||||
bool disable_rocksdb_stats_;
|
||||
ValidatorMode validator_mode_ = validator_normal;
|
||||
};
|
||||
|
||||
|
|
|
@ -84,6 +84,7 @@ struct ValidatorManagerOptions : public td::CntObject {
|
|||
virtual td::uint32 get_celldb_compress_depth() const = 0;
|
||||
virtual size_t get_max_open_archive_files() const = 0;
|
||||
virtual double get_archive_preload_period() const = 0;
|
||||
virtual bool get_disable_rocksdb_stats() const = 0;
|
||||
virtual ValidatorMode validator_mode() const = 0;
|
||||
|
||||
virtual void set_zero_block_id(BlockIdExt block_id) = 0;
|
||||
|
@ -106,6 +107,7 @@ struct ValidatorManagerOptions : public td::CntObject {
|
|||
virtual void set_celldb_compress_depth(td::uint32 value) = 0;
|
||||
virtual void set_max_open_archive_files(size_t value) = 0;
|
||||
virtual void set_archive_preload_period(double value) = 0;
|
||||
virtual void set_disable_rocksdb_stats(bool value) = 0;
|
||||
virtual void set_validator_mode(ValidatorMode value) = 0;
|
||||
|
||||
static td::Ref<ValidatorManagerOptions> create(
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue