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 block-generation

This commit is contained in:
SpyCheese 2023-03-28 16:52:33 +03:00
commit d082ac36b0
261 changed files with 24449 additions and 1791 deletions

View file

@ -624,6 +624,7 @@ void BlockDbImpl::get_block_by_id(ton::BlockId blk_id, bool need_data, td::Promi
}
}
promise(it->second);
return;
}
promise(td::Status::Error(-666, "block not found in database"));
}
@ -642,6 +643,7 @@ void BlockDbImpl::get_state_by_id(ton::BlockId blk_id, bool need_data, td::Promi
}
}
promise(it->second);
return;
}
if (zerostate.not_null() && blk_id == zerostate->blk.id) {
LOG(DEBUG) << "get_state_by_id(): zerostate requested";
@ -666,6 +668,7 @@ void BlockDbImpl::get_out_queue_info_by_id(ton::BlockId blk_id, td::Promise<td::
if (it == state_info.end()) {
promise(td::Status::Error(
-666, std::string{"cannot obtain output queue info for block "} + blk_id.to_str() + " : cannot load state"));
return;
}
if (it->second->data.is_null()) {
LOG(DEBUG) << "loading data for state " << blk_id.to_str();
@ -679,6 +682,7 @@ void BlockDbImpl::get_out_queue_info_by_id(ton::BlockId blk_id, td::Promise<td::
if (it2 == block_info.end()) {
promise(td::Status::Error(-666, std::string{"cannot obtain output queue info for block "} + blk_id.to_str() +
" : cannot load block description"));
return;
}
vm::StaticBagOfCellsDbLazy::Options options;
auto res = vm::StaticBagOfCellsDbLazy::create(it->second->data.clone(), options);
@ -707,10 +711,12 @@ void BlockDbImpl::get_out_queue_info_by_id(ton::BlockId blk_id, td::Promise<td::
if (it->second->blk.root_hash != state_root->get_hash().bits()) {
promise(td::Status::Error(
-668, std::string{"state for block "} + blk_id.to_str() + " is invalid : state root hash mismatch"));
return;
}
vm::CellSlice cs = vm::load_cell_slice(state_root);
if (!cs.have(64, 1) || cs.prefetch_ulong(32) != 0x9023afde) {
promise(td::Status::Error(-668, std::string{"state for block "} + blk_id.to_str() + " is invalid"));
return;
}
auto out_queue_info = cs.prefetch_ref();
promise(Ref<OutputQueueInfoDescr>{true, blk_id, it2->second->blk.root_hash.cbits(), state_root->get_hash().bits(),
@ -758,6 +764,7 @@ void BlockDbImpl::save_new_block(ton::BlockIdExt id, td::BufferSlice data, int a
auto save_res = save_db_file(id.file_hash, data, FMode::chk_if_exists | FMode::overwrite | FMode::chk_file_hash);
if (save_res.is_error()) {
promise(std::move(save_res));
return;
}
auto sz = data.size();
auto lev = bb.alloc<log::NewBlock>(id.id, id.root_hash, id.file_hash, data.size(), authority & 0xff);
@ -780,6 +787,7 @@ void BlockDbImpl::save_new_state(ton::BlockIdExt id, td::BufferSlice data, int a
auto save_res = save_db_file(id.file_hash, data, FMode::chk_if_exists | FMode::overwrite | FMode::chk_file_hash);
if (save_res.is_error()) {
promise(std::move(save_res));
return;
}
auto sz = data.size();
auto lev = bb.alloc<log::NewState>(id.id, id.root_hash, id.file_hash, data.size(), authority & 0xff);

View file

@ -1000,7 +1000,7 @@ bool Account::skip_copy_depth_balance(vm::CellBuilder& cb, vm::CellSlice& cs) co
}
const Account t_Account, t_AccountE{true};
const RefTo<Account> t_Ref_Account;
const RefTo<Account> t_Ref_AccountE{true};
bool ShardAccount::extract_account_state(Ref<vm::CellSlice> cs_ref, Ref<vm::Cell>& acc_state) {
if (cs_ref.is_null()) {

View file

@ -536,7 +536,7 @@ struct Account final : TLB_Complex {
};
extern const Account t_Account, t_AccountE;
extern const RefTo<Account> t_Ref_Account;
extern const RefTo<Account> t_Ref_AccountE;
struct AccountStatus final : TLB {
enum { acc_state_uninit, acc_state_frozen, acc_state_active, acc_state_nonexist };
@ -572,7 +572,7 @@ struct ShardAccount final : TLB_Complex {
return cs.advance_ext(0x140, 1);
}
bool validate_skip(int* ops, vm::CellSlice& cs, bool weak = false) const override {
return cs.advance(0x140) && t_Ref_Account.validate_skip(ops, cs, weak);
return cs.advance(0x140) && t_Ref_AccountE.validate_skip(ops, cs, weak);
}
static bool unpack(vm::CellSlice& cs, Record& info) {
return info.unpack(cs);

View file

@ -366,7 +366,7 @@ trans_merge_install$0111 split_info:SplitMergeInfo
smc_info#076ef1ea actions:uint16 msgs_sent:uint16
unixtime:uint32 block_lt:uint64 trans_lt:uint64
rand_seed:bits256 balance_remaining:CurrencyCollection
myself:MsgAddressInt = SmartContractInfo;
myself:MsgAddressInt global_config:(Maybe Cell) = SmartContractInfo;
//
//
out_list_empty$_ = OutList 0;

View file

@ -315,6 +315,113 @@ td::Result<TransactionList::Info> TransactionList::validate() const {
return std::move(res);
}
td::Result<BlockTransaction::Info> BlockTransaction::validate(bool check_proof) const {
if (root.is_null()) {
return td::Status::Error("transactions are expected to be non-empty");
}
if (check_proof && proof->get_hash().bits().compare(root->get_hash().bits(), 256)) {
return td::Status::Error(PSLICE() << "transaction hash mismatch: Merkle proof expects "
<< proof->get_hash().bits().to_hex(256)
<< " but received data has " << root->get_hash().bits().to_hex(256));
}
block::gen::Transaction::Record trans;
if (!tlb::unpack_cell(root, trans)) {
return td::Status::Error("cannot unpack transaction cell");
}
Info res;
res.blkid = blkid;
res.now = trans.now;
res.lt = trans.lt;
res.hash = root->get_hash().bits();
res.transaction = root;
return std::move(res);
}
td::Result<BlockTransactionList::Info> BlockTransactionList::validate(bool check_proof) const {
constexpr int max_answer_transactions = 256;
TRY_RESULT_PREFIX(list, vm::std_boc_deserialize_multi(std::move(transactions_boc)), "cannot deserialize transactions boc: ");
std::vector<td::Ref<vm::Cell>> tx_proofs(list.size());
if (check_proof) {
try {
TRY_RESULT(proof_cell, vm::std_boc_deserialize(std::move(proof_boc)));
auto virt_root = vm::MerkleProof::virtualize(proof_cell, 1);
if (blkid.root_hash != virt_root->get_hash().bits()) {
return td::Status::Error("Invalid block proof root hash");
}
block::gen::Block::Record blk;
block::gen::BlockExtra::Record extra;
if (!(tlb::unpack_cell(virt_root, blk) && tlb::unpack_cell(std::move(blk.extra), extra))) {
return td::Status::Error("Error unpacking proof cell");
}
vm::AugmentedDictionary acc_dict{vm::load_cell_slice_ref(extra.account_blocks), 256,
block::tlb::aug_ShardAccountBlocks};
bool eof = false;
ton::LogicalTime reverse = reverse_mode ? ~0ULL : 0;
ton::LogicalTime trans_lt = static_cast<ton::LogicalTime>(start_lt);
td::Bits256 cur_addr = start_addr;
bool allow_same = true;
int count = 0;
while (!eof && count < req_count && count < max_answer_transactions) {
auto value = acc_dict.extract_value(
acc_dict.vm::DictionaryFixed::lookup_nearest_key(cur_addr.bits(), 256, !reverse, allow_same));
if (value.is_null()) {
eof = true;
break;
}
allow_same = false;
if (cur_addr != start_addr) {
trans_lt = reverse;
}
block::gen::AccountBlock::Record acc_blk;
if (!tlb::csr_unpack(std::move(value), acc_blk) || acc_blk.account_addr != cur_addr) {
return td::Status::Error("Error unpacking proof account block");
}
vm::AugmentedDictionary trans_dict{vm::DictNonEmpty(), std::move(acc_blk.transactions), 64,
block::tlb::aug_AccountTransactions};
td::BitArray<64> cur_trans{(long long)trans_lt};
while (count < req_count && count < max_answer_transactions) {
auto tvalue = trans_dict.extract_value_ref(
trans_dict.vm::DictionaryFixed::lookup_nearest_key(cur_trans.bits(), 64, !reverse));
if (tvalue.is_null()) {
trans_lt = reverse;
break;
}
if (static_cast<size_t>(count) < tx_proofs.size()) {
tx_proofs[count] = std::move(tvalue);
}
count++;
}
}
if (static_cast<size_t>(count) != list.size()) {
return td::Status::Error(PSLICE() << "Txs count mismatch in proof (" << count << ") and response (" << list.size() << ")");
}
} catch (vm::VmError& err) {
return err.as_status("Couldn't verify proof: ");
} catch (vm::VmVirtError& err) {
return err.as_status("Couldn't verify proof: ");
} catch (...) {
return td::Status::Error("Unknown exception raised while verifying proof");
}
}
Info res;
for (int i = 0; i < static_cast<int>(list.size()); i++) {
auto& root = list[i];
BlockTransaction transaction;
transaction.root = root;
transaction.blkid = blkid;
transaction.proof = tx_proofs[i];
TRY_RESULT(info, transaction.validate(check_proof));
res.transactions.push_back(std::move(info));
}
return std::move(res);
}
td::Status BlockProofLink::validate(td::uint32* save_utime) const {
if (save_utime) {
*save_utime = 0;
@ -362,7 +469,7 @@ td::Status BlockProofLink::validate(td::uint32* save_utime) const {
if (to.seqno()) {
TRY_STATUS(check_block_header(vd_root, to));
if (!(tlb::unpack_cell(vd_root, blk) && tlb::unpack_cell(blk.info, info))) {
return td::Status::Error("cannot unpack header for block "s + from.to_str());
return td::Status::Error("cannot unpack header for block "s + to.to_str());
}
if (info.key_block != is_key) {
return td::Status::Error(PSTRING() << "incorrect is_key_block value " << is_key << " for destination block "

View file

@ -88,4 +88,36 @@ struct TransactionList {
td::Result<Info> validate() const;
};
struct BlockTransaction {
ton::BlockIdExt blkid;
td::Ref<vm::Cell> root;
td::Ref<vm::Cell> proof;
struct Info {
ton::BlockIdExt blkid;
td::uint32 now;
ton::LogicalTime lt;
ton::Bits256 hash;
td::Ref<vm::Cell> transaction;
};
td::Result<Info> validate(bool check_proof) const;
};
struct BlockTransactionList {
ton::BlockIdExt blkid;
td::BufferSlice transactions_boc;
td::BufferSlice proof_boc;
ton::LogicalTime start_lt;
td::Bits256 start_addr;
bool reverse_mode;
int req_count;
struct Info {
ton::BlockIdExt blkid;
std::vector<BlockTransaction::Info> transactions;
};
td::Result<Info> validate(bool check_proof) const;
};
} // namespace block

View file

@ -47,6 +47,7 @@
#include "fift/Fift.h"
#include "fift/Dictionary.h"
#include "fift/SourceLookup.h"
#include "fift/IntCtx.h"
#include "fift/words.h"
#include "td/utils/logging.h"
@ -308,7 +309,7 @@ td::RefInt256 create_smartcontract(td::RefInt256 smc_addr, Ref<vm::Cell> code, R
THRERR("cannot create smart-contract AccountStorage");
Ref<vm::DataCell> storage = cb.finalize();
vm::CellStorageStat stats;
PDO(stats.compute_used_storage(Ref<vm::Cell>(storage)));
PDO(stats.compute_used_storage(Ref<vm::Cell>(storage)).is_ok());
if (verbosity > 2) {
std::cerr << "storage is:\n";
vm::load_cell_slice(storage).print_rec(std::cerr);
@ -866,8 +867,9 @@ int main(int argc, char* const argv[]) {
case 'v':
new_verbosity_level = VERBOSITY_NAME(FATAL) + (verbosity = td::to_integer<int>(td::Slice(optarg)));
break;
case 'V':
std::cout << "create-state build information: [ Commit: " << GitMetadata::CommitSHA1() << ", Date: " << GitMetadata::CommitDate() << "]\n";
case 'V':
std::cout << "create-state build information: [ Commit: " << GitMetadata::CommitSHA1()
<< ", Date: " << GitMetadata::CommitDate() << "]\n";
std::exit(0);
break;
case 'h':

View file

@ -644,7 +644,6 @@ class Config {
static td::Result<std::vector<int>> unpack_param_dict(vm::Dictionary& dict);
static td::Result<std::vector<int>> unpack_param_dict(Ref<vm::Cell> dict_root);
protected:
Config(int _mode) : mode(_mode) {
config_addr.set_zero();
}

View file

@ -20,6 +20,7 @@
#include "block/block.h"
#include "block/block-parse.h"
#include "block/block-auto.h"
#include "crypto/openssl/rand.hpp"
#include "td/utils/bits.h"
#include "td/utils/uint128.h"
#include "ton/ton-shard.h"
@ -513,6 +514,7 @@ td::RefInt256 Account::compute_storage_fees(ton::UnixTime now, const std::vector
return StoragePrices::compute_storage_fees(now, pricing, storage_stat, last_paid, is_special, is_masterchain());
}
namespace transaction {
Transaction::Transaction(const Account& _account, int ttype, ton::LogicalTime req_start_lt, ton::UnixTime _now,
Ref<vm::Cell> _inmsg)
: trans_type(ttype)
@ -588,15 +590,19 @@ bool Transaction::unpack_input_msg(bool ihr_delivered, const ActionPhaseConfig*
in_msg_type = 2;
in_msg_extern = true;
// compute forwarding fees for this external message
vm::CellStorageStat sstat; // for message size
sstat.compute_used_storage(cs); // message body
sstat.bits -= cs.size(); // bits in the root cells are free
sstat.cells--; // the root cell itself is not counted as a cell
vm::CellStorageStat sstat; // for message size
auto cell_info = sstat.compute_used_storage(cs).move_as_ok(); // message body
sstat.bits -= cs.size(); // bits in the root cells are free
sstat.cells--; // the root cell itself is not counted as a cell
LOG(DEBUG) << "storage paid for a message: " << sstat.cells << " cells, " << sstat.bits << " bits";
if (sstat.bits > cfg->size_limits.max_msg_bits || sstat.cells > cfg->size_limits.max_msg_cells) {
LOG(DEBUG) << "inbound external message too large, invalid";
return false;
}
if (cell_info.max_merkle_depth > max_allowed_merkle_depth) {
LOG(DEBUG) << "inbound external message has too big merkle depth, invalid";
return false;
}
// fetch message pricing info
CHECK(cfg);
const MsgPrices& msg_prices = cfg->fetch_msg_prices(account.is_masterchain());
@ -745,6 +751,7 @@ bool Transaction::prepare_credit_phase() {
total_fees += std::move(collected);
return true;
}
} // namespace transaction
bool ComputePhaseConfig::parse_GasLimitsPrices(Ref<vm::Cell> cell, td::RefInt256& freeze_due_limit,
td::RefInt256& delete_due_limit) {
@ -837,6 +844,7 @@ td::RefInt256 ComputePhaseConfig::compute_gas_price(td::uint64 gas_used) const {
: td::rshift(gas_price256 * (gas_used - flat_gas_limit), 16, 1) + flat_gas_price;
}
namespace transaction {
bool Transaction::compute_gas_limits(ComputePhase& cp, const ComputePhaseConfig& cfg) {
// Compute gas limits
if (account.is_special) {
@ -1057,13 +1065,21 @@ bool Transaction::prepare_compute_phase(const ComputePhaseConfig& cfg) {
std::unique_ptr<StringLoggerTail> logger;
auto vm_log = vm::VmLog();
if (cfg.with_vm_log) {
logger = std::make_unique<StringLoggerTail>();
size_t log_max_size = cfg.vm_log_verbosity > 0 ? 1024 * 1024 : 256;
logger = std::make_unique<StringLoggerTail>(log_max_size);
vm_log.log_interface = logger.get();
vm_log.log_options = td::LogOptions(VERBOSITY_NAME(DEBUG), true, false);
if (cfg.vm_log_verbosity > 1) {
vm_log.log_mask |= vm::VmLog::ExecLocation;
if (cfg.vm_log_verbosity > 2) {
vm_log.log_mask |= vm::VmLog::DumpStack | vm::VmLog::GasRemaining;
}
}
}
vm::VmState vm{new_code, std::move(stack), gas, 1, new_data, vm_log, compute_vm_libraries(cfg)};
vm.set_max_data_depth(cfg.max_vm_data_depth);
vm.set_c7(prepare_vm_c7(cfg)); // tuple with SmartContractInfo
vm.set_chksig_always_succeed(cfg.ignore_chksig);
// vm.incr_stack_trace(1); // enable stack dump after each step
LOG(DEBUG) << "starting VM";
@ -1145,19 +1161,20 @@ bool Transaction::prepare_action_phase(const ActionPhaseConfig& cfg) {
ap.reserved_balance.set_zero();
td::Ref<vm::Cell> old_code = new_code, old_data = new_data, old_library = new_library;
auto enforce_state_size_limits = [&]() {
auto enforce_state_limits = [&]() {
if (account.is_special) {
return true;
}
if (!check_state_size_limit(cfg)) {
auto S = check_state_limits(cfg);
if (S.is_error()) {
// Rollback changes to state, fail action phase
LOG(INFO) << "Account state size exceeded limits";
LOG(INFO) << "Account state size exceeded limits: " << S.move_as_error();
new_storage_stat.clear();
new_code = old_code;
new_data = old_data;
new_library = old_library;
ap.result_code = 50;
ap.state_size_too_big = true;
ap.state_exceeds_limits = true;
return false;
}
return true;
@ -1238,8 +1255,8 @@ bool Transaction::prepare_action_phase(const ActionPhaseConfig& cfg) {
ap.no_funds = true;
}
LOG(DEBUG) << "invalid action " << ap.result_arg << " in action list: error code " << ap.result_code;
// This is reuqired here because changes to libraries are applied even if actipn phase fails
enforce_state_size_limits();
// This is reuqired here because changes to libraries are applied even if action phase fails
enforce_state_limits();
return true;
}
}
@ -1249,7 +1266,7 @@ bool Transaction::prepare_action_phase(const ActionPhaseConfig& cfg) {
new_code = ap.new_code;
}
new_data = compute_phase->new_data; // tentative persistent data update applied
if (!enforce_state_size_limits()) {
if (!enforce_state_limits()) {
return true;
}
@ -1322,8 +1339,8 @@ int Transaction::try_action_change_library(vm::CellSlice& cs, ActionPhase& ap, c
return 41;
}
vm::CellStorageStat sstat;
sstat.compute_used_storage(lib_ref);
if (sstat.cells > cfg.size_limits.max_library_cells) {
auto cell_info = sstat.compute_used_storage(lib_ref).move_as_ok();
if (sstat.cells > cfg.size_limits.max_library_cells || cell_info.max_merkle_depth > max_allowed_merkle_depth) {
return 43;
}
vm::CellBuilder cb;
@ -1338,6 +1355,7 @@ int Transaction::try_action_change_library(vm::CellSlice& cs, ActionPhase& ap, c
ap.spec_actions++;
return 0;
}
} // namespace transaction
// msg_fwd_fees = (lump_price + ceil((bit_price * msg.bits + cell_price * msg.cells)/2^16)) nanograms
// ihr_fwd_fees = ceil((msg_fwd_fees * ihr_price_factor)/2^16) nanograms
@ -1372,6 +1390,7 @@ td::RefInt256 MsgPrices::get_next_part(td::RefInt256 total) const {
return (std::move(total) * next_frac) >> 16;
}
namespace transaction {
bool Transaction::check_replace_src_addr(Ref<vm::CellSlice>& src_addr) const {
int t = (int)src_addr->prefetch_ulong(2);
if (!t && src_addr->size_ext() == 2) {
@ -1594,16 +1613,27 @@ int Transaction::try_action_send_msg(const vm::CellSlice& cs0, ActionPhase& ap,
// compute size of message
vm::CellStorageStat sstat; // for message size
// preliminary storage estimation of the resulting message
sstat.add_used_storage(msg.init, true, 3); // message init
sstat.add_used_storage(msg.body, true, 3); // message body (the root cell itself is not counted)
unsigned max_merkle_depth = 0;
auto add_used_storage = [&](const auto& x, unsigned skip_root_count) {
if (x.not_null()) {
auto res = sstat.add_used_storage(x, true, skip_root_count).move_as_ok();
max_merkle_depth = std::max(max_merkle_depth, res.max_merkle_depth);
}
};
add_used_storage(msg.init, 3); // message init
add_used_storage(msg.body, 3); // message body (the root cell itself is not counted)
if (!ext_msg) {
sstat.add_used_storage(info.value->prefetch_ref());
add_used_storage(info.value->prefetch_ref(), 0);
}
LOG(DEBUG) << "storage paid for a message: " << sstat.cells << " cells, " << sstat.bits << " bits";
if (sstat.bits > cfg.size_limits.max_msg_bits || sstat.cells > cfg.size_limits.max_msg_cells) {
LOG(DEBUG) << "message too large, invalid";
return skip_invalid ? 0 : 40;
}
if (max_merkle_depth > max_allowed_merkle_depth) {
LOG(DEBUG) << "message has too big merkle depth, invalid";
return skip_invalid ? 0 : 40;
}
// compute forwarding fees
auto fees_c = msg_prices.compute_fwd_ihr_fees(sstat.cells, sstat.bits, info.ihr_disabled);
@ -1855,7 +1885,7 @@ int Transaction::try_action_reserve_currency(vm::CellSlice& cs, ActionPhase& ap,
return 0;
}
bool Transaction::check_state_size_limit(const ActionPhaseConfig& cfg) {
td::Status Transaction::check_state_limits(const ActionPhaseConfig& cfg) {
auto cell_equal = [](const td::Ref<vm::Cell>& a, const td::Ref<vm::Cell>& b) -> bool {
if (a.is_null()) {
return b.is_null();
@ -1867,21 +1897,36 @@ bool Transaction::check_state_size_limit(const ActionPhaseConfig& cfg) {
};
if (cell_equal(account.code, new_code) && cell_equal(account.data, new_data) &&
cell_equal(account.library, new_library)) {
return true;
return td::Status::OK();
}
// new_storage_stat is used here beause these stats will be reused in compute_state()
new_storage_stat.limit_cells = cfg.size_limits.max_acc_state_cells;
new_storage_stat.limit_bits = cfg.size_limits.max_acc_state_bits;
new_storage_stat.add_used_storage(new_code);
new_storage_stat.add_used_storage(new_data);
new_storage_stat.add_used_storage(new_library);
td::Timer timer;
auto add_used_storage = [&](const td::Ref<vm::Cell>& cell) -> td::Status {
if (cell.not_null()) {
TRY_RESULT(res, new_storage_stat.add_used_storage(cell));
if (res.max_merkle_depth > max_allowed_merkle_depth) {
return td::Status::Error("too big merkle depth");
}
}
return td::Status::OK();
};
TRY_STATUS(add_used_storage(new_code));
TRY_STATUS(add_used_storage(new_data));
TRY_STATUS(add_used_storage(new_library));
if (timer.elapsed() > 0.1) {
LOG(INFO) << "Compute used storage took " << timer.elapsed() << "s";
}
if (acc_status == Account::acc_active) {
new_storage_stat.clear_limit();
} else {
new_storage_stat.clear();
}
return new_storage_stat.cells <= cfg.size_limits.max_acc_state_cells &&
new_storage_stat.bits <= cfg.size_limits.max_acc_state_bits;
new_storage_stat.bits <= cfg.size_limits.max_acc_state_bits
? td::Status::OK()
: td::Status::Error("state too big");
}
bool Transaction::prepare_bounce_phase(const ActionPhaseConfig& cfg) {
@ -1978,6 +2023,7 @@ bool Transaction::prepare_bounce_phase(const ActionPhaseConfig& cfg) {
bp.ok = true;
return true;
}
} // namespace transaction
/*
*
@ -2033,6 +2079,7 @@ static td::optional<vm::CellStorageStat> try_update_storage_stat(const vm::CellS
return new_stat;
}
namespace transaction {
bool Transaction::compute_state() {
if (new_total_state.not_null()) {
return true;
@ -2108,7 +2155,7 @@ bool Transaction::compute_state() {
stats = new_stats.unwrap();
} else {
td::Timer timer;
CHECK(stats.add_used_storage(Ref<vm::Cell>(storage)));
stats.add_used_storage(Ref<vm::Cell>(storage)).ensure();
if (timer.elapsed() > 0.1) {
LOG(INFO) << "Compute used storage took " << timer.elapsed() << "s";
}
@ -2460,6 +2507,7 @@ void Transaction::extract_out_msgs(std::vector<LtCellRef>& list) {
list.emplace_back(start_lt + i + 1, std::move(out_msgs[i]));
}
}
} // namespace transaction
void Account::push_transaction(Ref<vm::Cell> trans_root, ton::LogicalTime trans_lt) {
transactions.emplace_back(trans_lt, std::move(trans_root));
@ -2503,4 +2551,82 @@ bool Account::libraries_changed() const {
}
}
td::Status FetchConfigParams::fetch_config_params(const block::Config& config,
Ref<vm::Cell>* old_mparams,
std::vector<block::StoragePrices>* storage_prices,
block::StoragePhaseConfig* storage_phase_cfg,
td::BitArray<256>* rand_seed,
block::ComputePhaseConfig* compute_phase_cfg,
block::ActionPhaseConfig* action_phase_cfg,
td::RefInt256* masterchain_create_fee,
td::RefInt256* basechain_create_fee,
ton::WorkchainId wc,
ton::UnixTime now) {
*old_mparams = config.get_config_param(9);
{
auto res = config.get_storage_prices();
if (res.is_error()) {
return res.move_as_error();
}
*storage_prices = res.move_as_ok();
}
if (rand_seed->is_zero()) {
// generate rand seed
prng::rand_gen().strong_rand_bytes(rand_seed->data(), 32);
LOG(DEBUG) << "block random seed set to " << rand_seed->to_hex();
}
TRY_RESULT(size_limits, config.get_size_limits_config());
{
// compute compute_phase_cfg / storage_phase_cfg
auto cell = config.get_config_param(wc == ton::masterchainId ? 20 : 21);
if (cell.is_null()) {
return td::Status::Error(-668, "cannot fetch current gas prices and limits from masterchain configuration");
}
if (!compute_phase_cfg->parse_GasLimitsPrices(std::move(cell), storage_phase_cfg->freeze_due_limit,
storage_phase_cfg->delete_due_limit)) {
return td::Status::Error(-668, "cannot unpack current gas prices and limits from masterchain configuration");
}
compute_phase_cfg->block_rand_seed = *rand_seed;
compute_phase_cfg->max_vm_data_depth = size_limits.max_vm_data_depth;
compute_phase_cfg->global_config = config.get_root_cell();
compute_phase_cfg->suspended_addresses = config.get_suspended_addresses(now);
}
{
// compute action_phase_cfg
block::gen::MsgForwardPrices::Record rec;
auto cell = config.get_config_param(24);
if (cell.is_null() || !tlb::unpack_cell(std::move(cell), rec)) {
return td::Status::Error(-668, "cannot fetch masterchain message transfer prices from masterchain configuration");
}
action_phase_cfg->fwd_mc =
block::MsgPrices{rec.lump_price, rec.bit_price, rec.cell_price, rec.ihr_price_factor,
(unsigned)rec.first_frac, (unsigned)rec.next_frac};
cell = config.get_config_param(25);
if (cell.is_null() || !tlb::unpack_cell(std::move(cell), rec)) {
return td::Status::Error(-668, "cannot fetch standard message transfer prices from masterchain configuration");
}
action_phase_cfg->fwd_std =
block::MsgPrices{rec.lump_price, rec.bit_price, rec.cell_price, rec.ihr_price_factor,
(unsigned)rec.first_frac, (unsigned)rec.next_frac};
action_phase_cfg->workchains = &config.get_workchain_list();
action_phase_cfg->bounce_msg_body = (config.has_capability(ton::capBounceMsgBody) ? 256 : 0);
action_phase_cfg->size_limits = size_limits;
}
{
// fetch block_grams_created
auto cell = config.get_config_param(14);
if (cell.is_null()) {
*basechain_create_fee = *masterchain_create_fee = td::zero_refint();
} else {
block::gen::BlockCreateFees::Record create_fees;
if (!(tlb::unpack_cell(cell, create_fees) &&
block::tlb::t_Grams.as_integer_to(create_fees.masterchain_block_fee, *masterchain_create_fee) &&
block::tlb::t_Grams.as_integer_to(create_fees.basechain_block_fee, *basechain_create_fee))) {
return td::Status::Error(-668, "cannot unpack BlockCreateFees from configuration parameter #14");
}
}
}
return td::Status::OK();
}
} // namespace block

View file

@ -35,7 +35,10 @@ using td::Ref;
using LtCellRef = std::pair<ton::LogicalTime, Ref<vm::Cell>>;
struct Account;
namespace transaction {
struct Transaction;
} // namespace transaction
struct CollatorError {
std::string msg;
@ -106,9 +109,11 @@ struct ComputePhaseConfig {
std::unique_ptr<vm::Dictionary> libraries;
Ref<vm::Cell> global_config;
td::BitArray<256> block_rand_seed;
bool ignore_chksig{false};
bool with_vm_log{false};
td::uint16 max_vm_data_depth = 512;
std::unique_ptr<vm::Dictionary> suspended_addresses;
int vm_log_verbosity = 0;
ComputePhaseConfig(td::uint64 _gas_price = 0, td::uint64 _gas_limit = 0, td::uint64 _gas_credit = 0)
: gas_price(_gas_price), gas_limit(_gas_limit), special_gas_limit(_gas_limit), gas_credit(_gas_credit) {
compute_threshold();
@ -186,7 +191,7 @@ struct ActionPhase {
bool code_changed{false};
bool action_list_invalid{false};
bool acc_delete_req{false};
bool state_size_too_big{false};
bool state_exceeds_limits{false};
enum { acst_unchanged = 0, acst_frozen = 2, acst_deleted = 3 };
int acc_status_change{acst_unchanged};
td::RefInt256 total_fwd_fees; // all fees debited from the account
@ -273,7 +278,7 @@ struct Account {
bool create_account_block(vm::CellBuilder& cb); // stores an AccountBlock with all transactions
protected:
friend struct Transaction;
friend struct transaction::Transaction;
bool set_split_depth(int split_depth);
bool check_split_depth(int split_depth) const;
bool forget_split_depth();
@ -288,7 +293,9 @@ struct Account {
bool compute_my_addr(bool force = false);
};
namespace transaction {
struct Transaction {
static constexpr unsigned max_allowed_merkle_depth = 2;
enum {
tr_none,
tr_ord,
@ -354,7 +361,7 @@ struct Transaction {
std::vector<Ref<vm::Cell>> compute_vm_libraries(const ComputePhaseConfig& cfg);
bool prepare_compute_phase(const ComputePhaseConfig& cfg);
bool prepare_action_phase(const ActionPhaseConfig& cfg);
bool check_state_size_limit(const ActionPhaseConfig& cfg);
td::Status check_state_limits(const ActionPhaseConfig& cfg);
bool prepare_bounce_phase(const ActionPhaseConfig& cfg);
bool compute_state();
bool serialize();
@ -390,5 +397,20 @@ struct Transaction {
bool serialize_bounce_phase(vm::CellBuilder& cb);
bool unpack_msg_state(bool lib_only = false);
};
} // namespace transaction
struct FetchConfigParams {
static td::Status fetch_config_params(const block::Config& config,
Ref<vm::Cell>* old_mparams,
std::vector<block::StoragePrices>* storage_prices,
StoragePhaseConfig* storage_phase_cfg,
td::BitArray<256>* rand_seed,
ComputePhaseConfig* compute_phase_cfg,
ActionPhaseConfig* action_phase_cfg,
td::RefInt256* masterchain_create_fee,
td::RefInt256* basechain_create_fee,
ton::WorkchainId wc,
ton::UnixTime now);
};
} // namespace block