From 213f1407e342cdaac00d8b551c1c695823173f00 Mon Sep 17 00:00:00 2001 From: battlmonstr Date: Fri, 25 Oct 2024 20:49:31 +0200 Subject: [PATCH] db: provide SnapshotRepository to DataModel (#2440) * no more DataModel::set_snapshot_repository singleton * BodyFindByBlockNumMultiQuery: useful without RoTx * DataStoreRef: source of creating DataModel queries for kv::api, sync and daemon * Buffer works with BufferDataModel * ExecutionPipeline accepts StageContainerFactory * stages and forks use DataModelFactory * SnapshotRepository: init with path instead of settings --- cmd/capi/execute.cpp | 6 +- cmd/dev/check_changes.cpp | 7 +- cmd/dev/db_toolbox.cpp | 49 +++++++--- cmd/dev/scan_txs.cpp | 2 +- cmd/dev/snapshots.cpp | 25 ++++- silkworm/capi/fork_validator.cpp | 32 ++++++- silkworm/capi/rpcdaemon.cpp | 7 +- silkworm/capi/silkworm.cpp | 42 ++++++--- silkworm/db/access_layer.cpp | 94 +++++++------------ silkworm/db/access_layer.hpp | 41 ++++---- silkworm/db/access_layer_test.cpp | 2 - silkworm/db/blocks/bodies/body_queries.hpp | 17 ++++ silkworm/db/buffer.cpp | 4 +- silkworm/db/buffer.hpp | 45 ++++++++- silkworm/db/buffer_test.cpp | 12 +-- silkworm/db/chain/local_chain_storage.cpp | 2 - silkworm/db/chain/local_chain_storage.hpp | 4 +- silkworm/db/data_store.hpp | 29 ++++++ .../snapshots/snapshot_repository.cpp | 18 ++-- .../snapshots/snapshot_repository.hpp | 12 +-- silkworm/db/freezer.cpp | 9 +- silkworm/db/kv/api/direct_service.cpp | 11 ++- silkworm/db/kv/api/direct_service.hpp | 11 ++- silkworm/db/kv/api/direct_service_test.cpp | 7 +- silkworm/db/kv/api/local_transaction.cpp | 4 +- silkworm/db/kv/api/local_transaction.hpp | 12 ++- silkworm/db/kv/state_changes_stream_test.cpp | 7 +- silkworm/db/snapshot_benchmark.cpp | 12 +-- silkworm/db/snapshot_repository_test.cpp | 17 ++-- silkworm/db/snapshot_sync.cpp | 14 +-- silkworm/db/snapshot_sync.hpp | 5 +- silkworm/db/snapshot_sync_test.cpp | 4 +- silkworm/db/state/local_state.hpp | 6 +- silkworm/db/test_util/make_repository.cpp | 10 +- silkworm/db/test_util/make_repository.hpp | 4 +- .../db/test_util/test_database_context.cpp | 2 +- .../execution/header_chain_plus_exec_test.cpp | 14 +-- silkworm/node/node.cpp | 43 ++++++++- silkworm/node/stagedsync/execution_engine.cpp | 8 +- silkworm/node/stagedsync/execution_engine.hpp | 5 +- .../node/stagedsync/execution_engine_test.cpp | 29 +++--- .../node/stagedsync/execution_pipeline.cpp | 65 +++---------- .../node/stagedsync/execution_pipeline.hpp | 16 ++-- .../node/stagedsync/forks/canonical_chain.cpp | 23 +++-- .../node/stagedsync/forks/canonical_chain.hpp | 13 ++- .../node/stagedsync/forks/extending_fork.cpp | 5 +- silkworm/node/stagedsync/forks/fork.cpp | 23 ++--- silkworm/node/stagedsync/forks/fork.hpp | 12 ++- silkworm/node/stagedsync/forks/fork_test.cpp | 19 ++-- silkworm/node/stagedsync/forks/main_chain.cpp | 33 +++---- silkworm/node/stagedsync/forks/main_chain.hpp | 14 ++- .../node/stagedsync/forks/main_chain_test.cpp | 25 +++-- .../node/stagedsync/stages/stage_bodies.cpp | 21 +++-- .../node/stagedsync/stages/stage_bodies.hpp | 10 +- .../stagedsync/stages/stage_bodies_test.cpp | 20 +++- .../stagedsync/stages/stage_execution.cpp | 7 +- .../stagedsync/stages/stage_execution.hpp | 4 + .../node/stagedsync/stages/stage_headers.cpp | 22 ++++- .../node/stagedsync/stages/stage_headers.hpp | 12 ++- .../stagedsync/stages/stage_headers_test.cpp | 13 ++- .../stages/stage_history_index_test.cpp | 4 +- .../stagedsync/stages/stage_interhashes.cpp | 4 +- .../stagedsync/stages/stage_interhashes.hpp | 9 +- .../node/stagedsync/stages/stage_senders.cpp | 4 +- .../node/stagedsync/stages/stage_senders.hpp | 3 + .../stagedsync/stages/stage_tx_lookup.cpp | 8 +- .../stagedsync/stages/stage_tx_lookup.hpp | 4 + .../stages/stage_tx_lookup_test.cpp | 9 +- .../node/stagedsync/stages_factory_impl.cpp | 63 +++++++++++++ .../node/stagedsync/stages_factory_impl.hpp | 47 ++++++++++ silkworm/node/stagedsync/stages_test.cpp | 25 +++-- .../node/test_util/make_stages_factory.cpp | 43 +++++++++ .../node/test_util/make_stages_factory.hpp | 31 ++++++ .../node/test_util/mock_execution_engine.hpp | 25 ++++- silkworm/rpc/daemon.cpp | 42 +++++---- silkworm/rpc/daemon.hpp | 9 +- silkworm/rpc/ethdb/file/local_database.cpp | 9 +- silkworm/rpc/ethdb/file/local_database.hpp | 7 +- silkworm/rpc/test_util/api_test_database.hpp | 30 ++++-- silkworm/sentry/eth/status_data_provider.cpp | 4 +- silkworm/sentry/eth/status_data_provider.hpp | 2 +- silkworm/sync/block_exchange.cpp | 14 +-- silkworm/sync/block_exchange.hpp | 8 +- silkworm/sync/internals/body_retrieval.cpp | 2 - silkworm/sync/internals/body_retrieval.hpp | 5 +- silkworm/sync/internals/header_retrieval.cpp | 6 +- silkworm/sync/internals/header_retrieval.hpp | 4 +- .../sync/internals/header_retrieval_test.cpp | 9 +- .../sync/messages/inbound_block_bodies.cpp | 2 +- .../sync/messages/inbound_block_bodies.hpp | 2 +- .../sync/messages/inbound_block_headers.cpp | 2 +- .../sync/messages/inbound_block_headers.hpp | 2 +- .../messages/inbound_get_block_bodies.cpp | 5 +- .../messages/inbound_get_block_bodies.hpp | 2 +- .../messages/inbound_get_block_headers.cpp | 6 +- .../messages/inbound_get_block_headers.hpp | 2 +- silkworm/sync/messages/inbound_new_block.cpp | 2 +- silkworm/sync/messages/inbound_new_block.hpp | 2 +- .../messages/inbound_new_block_hashes.cpp | 2 +- .../messages/inbound_new_block_hashes.hpp | 2 +- silkworm/sync/messages/internal_message.hpp | 6 +- .../sync/messages/internal_message_test.cpp | 7 +- silkworm/sync/messages/message.hpp | 4 +- .../sync/messages/outbound_block_bodies.cpp | 2 +- .../sync/messages/outbound_block_bodies.hpp | 2 +- .../sync/messages/outbound_block_headers.cpp | 2 +- .../sync/messages/outbound_block_headers.hpp | 2 +- .../messages/outbound_get_block_bodies.cpp | 2 +- .../messages/outbound_get_block_bodies.hpp | 2 +- .../messages/outbound_get_block_headers.cpp | 2 +- .../messages/outbound_get_block_headers.hpp | 2 +- silkworm/sync/messages/outbound_message.hpp | 2 +- silkworm/sync/messages/outbound_new_block.cpp | 2 +- silkworm/sync/messages/outbound_new_block.hpp | 2 +- .../messages/outbound_new_block_hashes.cpp | 2 +- .../messages/outbound_new_block_hashes.hpp | 2 +- silkworm/sync/sync.cpp | 10 +- silkworm/sync/sync.hpp | 9 +- silkworm/sync/sync_pos_test.cpp | 11 ++- .../sync/test_util/mock_block_exchange.hpp | 5 +- 120 files changed, 1046 insertions(+), 534 deletions(-) create mode 100644 silkworm/db/data_store.hpp create mode 100644 silkworm/node/stagedsync/stages_factory_impl.cpp create mode 100644 silkworm/node/stagedsync/stages_factory_impl.hpp create mode 100644 silkworm/node/test_util/make_stages_factory.cpp create mode 100644 silkworm/node/test_util/make_stages_factory.hpp diff --git a/cmd/capi/execute.cpp b/cmd/capi/execute.cpp index 649a3a7196..9c8512bf34 100644 --- a/cmd/capi/execute.cpp +++ b/cmd/capi/execute.cpp @@ -430,15 +430,11 @@ int main(int argc, char* argv[]) { return init_status_code; } - // Add snapshots to Silkworm API library - SnapshotSettings snapshot_settings{}; - snapshot_settings.repository_dir = data_dir.snapshots().path(); - int status_code = -1; if (settings.execute_blocks_settings) { // Execute specified block range using Silkworm API library SnapshotRepository repository{ - snapshot_settings, + data_dir.snapshots().path(), std::make_unique(), std::make_unique(), }; diff --git a/cmd/dev/check_changes.cpp b/cmd/dev/check_changes.cpp index e9a8e5a920..219023a2df 100644 --- a/cmd/dev/check_changes.cpp +++ b/cmd/dev/check_changes.cpp @@ -111,13 +111,12 @@ int main(int argc, char* argv[]) { } snapshots::SnapshotRepository repository{ - snapshots::SnapshotSettings{}, + data_dir.snapshots().path(), std::make_unique(), std::make_unique(), }; repository.reopen_folder(); - db::DataModel::set_snapshot_repository(&repository); - db::DataModel access_layer{txn}; + db::DataModel access_layer{txn, repository}; AnalysisCache analysis_cache{/*max_size=*/5'000}; std::vector receipts; @@ -130,7 +129,7 @@ int main(int argc, char* argv[]) { break; } - db::Buffer buffer{txn}; + db::Buffer buffer{txn, std::make_unique(access_layer)}; buffer.set_historical_block(block_num); ExecutionProcessor processor{block, *rule_set, buffer, *chain_config}; diff --git a/cmd/dev/db_toolbox.cpp b/cmd/dev/db_toolbox.cpp index 2529f6d5ae..80a5a92850 100644 --- a/cmd/dev/db_toolbox.cpp +++ b/cmd/dev/db_toolbox.cpp @@ -62,7 +62,9 @@ #include #include #include +#include #include +#include #include "../common/common.hpp" @@ -562,6 +564,29 @@ void do_stage_set(EnvConfig& config, const std::string& stage_name, uint32_t new std::cout << "\n Stage " << stage_name << " touched from " << old_height << " to " << new_height << "\n\n"; } +static silkworm::stagedsync::BodiesStageFactory make_bodies_stage_factory( + const silkworm::ChainConfig& chain_config, + silkworm::db::DataModelFactory data_model_factory) { + return [chain_config, data_model_factory = std::move(data_model_factory)](silkworm::stagedsync::SyncContext* sync_context) { + return std::make_unique( + sync_context, + chain_config, + data_model_factory, + [] { return 0; }); + }; +}; + +static silkworm::stagedsync::StageContainerFactory make_stages_factory( + const silkworm::NodeSettings& node_settings, + silkworm::db::DataModelFactory data_model_factory) { + auto bodies_stage_factory = make_bodies_stage_factory(*node_settings.chain_config, data_model_factory); + return silkworm::stagedsync::StagesFactoryImpl::to_factory({ + node_settings, + std::move(data_model_factory), + std::move(bodies_stage_factory), + }); +} + void unwind(EnvConfig& config, BlockNum unwind_point, bool remove_blocks) { ensure(config.exclusive, "Function requires exclusive access to database"); @@ -572,25 +597,29 @@ void unwind(EnvConfig& config, BlockNum unwind_point, bool remove_blocks) { auto chain_config{read_chain_config(txn)}; ensure(chain_config.has_value(), "Not an initialized Silkworm db or unknown/custom chain"); + auto data_directory = std::make_unique(); + snapshots::SnapshotRepository repository{ + data_directory->snapshots().path(), + std::make_unique(), + std::make_unique(), + }; + db::DataModelFactory data_model_factory = [&](db::ROTxn& tx) { return db::DataModel{tx, repository}; }; + boost::asio::io_context io_context; NodeSettings settings{ - .data_directory = std::make_unique(), + .data_directory = std::move(data_directory), .chaindata_env_config = config, .chain_config = chain_config}; - stagedsync::BodiesStageFactory bodies_stage_factory = [&](stagedsync::SyncContext* sync_context) { - return std::make_unique(sync_context, *settings.chain_config, [] { return 0; }); - }; - stagedsync::TimerFactory log_timer_factory = [&](std::function callback) { return std::make_shared(io_context.get_executor(), settings.sync_loop_log_interval_seconds * 1000, std::move(callback)); }; stagedsync::ExecutionPipeline stage_pipeline{ - &settings, + data_model_factory, std::move(log_timer_factory), - std::move(bodies_stage_factory), + make_stages_factory(settings, data_model_factory), }; const auto unwind_result{stage_pipeline.unwind(txn, unwind_point)}; @@ -2237,16 +2266,12 @@ void do_freeze(EnvConfig& config, const DataDirectory& data_dir, bool keep_block auto env = open_env(config); StageSchedulerAdapter stage_scheduler{RWAccess{env}}; - snapshots::SnapshotSettings settings; - settings.repository_dir = data_dir.snapshots().path(); - settings.no_downloader = true; snapshots::SnapshotRepository repository{ - std::move(settings), + data_dir.snapshots().path(), std::make_unique(), std::make_unique(), }; repository.reopen_folder(); - DataModel::set_snapshot_repository(&repository); Freezer freezer{ROAccess{env}, repository, stage_scheduler, data_dir.temp().path(), keep_blocks}; diff --git a/cmd/dev/scan_txs.cpp b/cmd/dev/scan_txs.cpp index a04505e86c..e71f41fc10 100644 --- a/cmd/dev/scan_txs.cpp +++ b/cmd/dev/scan_txs.cpp @@ -88,7 +88,7 @@ int main(int argc, char* argv[]) { break; } - db::Buffer buffer{txn}; + db::Buffer buffer{txn, std::make_unique(txn)}; buffer.set_historical_block(block_num); ExecutionProcessor processor{block, *rule_set, buffer, *chain_config}; diff --git a/cmd/dev/snapshots.cpp b/cmd/dev/snapshots.cpp index 18650f89ab..e2687be7f7 100644 --- a/cmd/dev/snapshots.cpp +++ b/cmd/dev/snapshots.cpp @@ -295,9 +295,9 @@ void decode_segment(const SnapshotSubcommandSettings& settings, int repetitions) SILK_INFO << "Decode snapshot elapsed: " << as_milliseconds(elapsed) << " msec"; } -static SnapshotRepository make_repository(SnapshotSettings settings) { +static SnapshotRepository make_repository(const SnapshotSettings& settings) { return SnapshotRepository{ - std::move(settings), + settings.repository_dir, std::make_unique(), std::make_unique(), }; @@ -1056,10 +1056,27 @@ void sync(const SnapshotSettings& settings) { TemporaryDirectory tmp_dir; db::EnvConfig chaindata_env_config{tmp_dir.path()}; auto chaindata_env = db::open_env(chaindata_env_config); + + SnapshotRepository repository{ + settings.repository_dir, + std::make_unique(), + std::make_unique(), + }; + + db::DataStoreRef data_store{ + chaindata_env, // NOLINT(cppcoreguidelines-slicing) + repository, + }; + test_util::TaskRunner runner; NoopStageSchedulerAdapter stage_scheduler; - // NOLINTNEXTLINE(cppcoreguidelines-slicing) - db::SnapshotSync snapshot_sync{settings, kMainnetConfig.chain_id, chaindata_env, tmp_dir.path(), stage_scheduler}; + db::SnapshotSync snapshot_sync{ + settings, + kMainnetConfig.chain_id, + data_store, + tmp_dir.path(), + stage_scheduler, + }; runner.run(snapshot_sync.download_snapshots()); std::chrono::duration elapsed{std::chrono::steady_clock::now() - start}; diff --git a/silkworm/capi/fork_validator.cpp b/silkworm/capi/fork_validator.cpp index a1bf9e0522..209fcd8fde 100644 --- a/silkworm/capi/fork_validator.cpp +++ b/silkworm/capi/fork_validator.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include "common.hpp" #include "instance.hpp" @@ -80,12 +80,29 @@ static void set_node_settings(SilkwormHandle handle, const struct SilkwormForkVa handle->node_settings.keep_db_txn_open = false; // Ensure that the transaction is closed after each request, Erigon manages transactions differently } -static silkworm::stagedsync::BodiesStageFactory make_bodies_stage_factory(const silkworm::ChainConfig& chain_config) { - return [chain_config](silkworm::stagedsync::SyncContext* sync_context) { - return std::make_unique(sync_context, chain_config, [] { return 0; }); +static silkworm::stagedsync::BodiesStageFactory make_bodies_stage_factory( + const silkworm::ChainConfig& chain_config, + silkworm::db::DataModelFactory data_model_factory) { + return [chain_config, data_model_factory = std::move(data_model_factory)](silkworm::stagedsync::SyncContext* sync_context) { + return std::make_unique( + sync_context, + chain_config, + data_model_factory, + [] { return 0; }); }; }; +static silkworm::stagedsync::StageContainerFactory make_stages_factory( + const silkworm::NodeSettings& node_settings, + silkworm::db::DataModelFactory data_model_factory) { + auto bodies_stage_factory = make_bodies_stage_factory(*node_settings.chain_config, data_model_factory); + return silkworm::stagedsync::StagesFactoryImpl::to_factory({ + node_settings, + std::move(data_model_factory), + std::move(bodies_stage_factory), + }); +} + SILKWORM_EXPORT int silkworm_start_fork_validator(SilkwormHandle handle, MDBX_env* mdbx_env, const struct SilkwormForkValidatorSettings* settings) SILKWORM_NOEXCEPT { if (!handle) { return SILKWORM_INVALID_HANDLE; @@ -111,11 +128,16 @@ SILKWORM_EXPORT int silkworm_start_fork_validator(SilkwormHandle handle, MDBX_en silkworm::db::EnvUnmanaged unmanaged_env{mdbx_env}; silkworm::db::RWAccess rw_access{unmanaged_env}; + silkworm::db::DataModelFactory data_model_factory = [handle](silkworm::db::ROTxn& tx) { + return silkworm::db::DataModel{tx, *handle->snapshot_repository}; + }; + handle->execution_engine = std::make_unique( /* executor = */ std::nullopt, // ExecutionEngine manages an internal io_context handle->node_settings, + data_model_factory, /* log_timer_factory = */ std::nullopt, - make_bodies_stage_factory(*handle->node_settings.chain_config), + make_stages_factory(handle->node_settings, data_model_factory), rw_access); silkworm::log::Info("Execution engine created"); diff --git a/silkworm/capi/rpcdaemon.cpp b/silkworm/capi/rpcdaemon.cpp index b57a6a8ffd..00cc092658 100644 --- a/silkworm/capi/rpcdaemon.cpp +++ b/silkworm/capi/rpcdaemon.cpp @@ -101,10 +101,13 @@ SILKWORM_EXPORT int silkworm_start_rpcdaemon(SilkwormHandle handle, MDBX_env* en } auto daemon_settings = make_daemon_settings(handle, *settings); - db::EnvUnmanaged unmanaged_env{env}; + db::DataStoreRef data_store{ + db::EnvUnmanaged{env}, + *handle->snapshot_repository, + }; // Create the one-and-only Silkrpc daemon - handle->rpcdaemon = std::make_unique(daemon_settings, std::make_optional(unmanaged_env)); + handle->rpcdaemon = std::make_unique(daemon_settings, data_store); // Check protocol version compatibility with Core Services if (!daemon_settings.skip_protocol_check) { diff --git a/silkworm/capi/silkworm.cpp b/silkworm/capi/silkworm.cpp index 2c1be6a544..1440f8bccd 100644 --- a/silkworm/capi/silkworm.cpp +++ b/silkworm/capi/silkworm.cpp @@ -223,11 +223,11 @@ SILKWORM_EXPORT int silkworm_init(SilkwormHandle* handle, const struct SilkwormS log::init(log_settings); log::Info{"Silkworm build info", log_args_for_version()}; // NOLINT(*-unused-raii) + auto data_dir_path = parse_path(settings->data_dir_path); auto snapshot_repository = std::make_unique( - snapshots::SnapshotSettings{}, + DataDirectory{data_dir_path}.snapshots().path(), std::make_unique(), std::make_unique()); - db::DataModel::set_snapshot_repository(snapshot_repository.get()); // NOLINTNEXTLINE(bugprone-unhandled-exception-at-new) *handle = new SilkwormInstance{ @@ -235,7 +235,7 @@ SILKWORM_EXPORT int silkworm_init(SilkwormHandle* handle, const struct SilkwormS .context_pool_settings = { .num_contexts = settings->num_contexts > 0 ? settings->num_contexts : silkworm::concurrency::kDefaultNumContexts, }, - .data_dir_path = parse_path(settings->data_dir_path), + .data_dir_path = std::move(data_dir_path), .node_settings = {}, .snapshot_repository = std::move(snapshot_repository), .rpcdaemon = {}, @@ -406,16 +406,18 @@ class BlockProvider { public: BlockProvider(BoundedBuffer>* block_buffer, - mdbx::env env, + db::ROAccess db_access, + db::DataModelFactory data_model_factory, BlockNum start_block, BlockNum max_block) : block_buffer_{block_buffer}, - env_{std::move(env)}, + db_access_{std::move(db_access)}, + data_model_factory_{std::move(data_model_factory)}, start_block_{start_block}, max_block_{max_block} {} void operator()() { - db::ROTxnManaged txn{env_}; - db::DataModel access_layer{txn}; + db::ROTxnManaged txn = db_access_.start_ro_tx(); + db::DataModel access_layer = data_model_factory_(txn); BlockNum current_block{start_block_}; size_t refresh_counter{kTxnRefreshThreshold}; @@ -433,11 +435,11 @@ class BlockProvider { if (--refresh_counter == 0) { txn.abort(); - txn = db::ROTxnManaged{env_}; + txn = db_access_.start_ro_tx(); refresh_counter = kTxnRefreshThreshold; } } - } catch (const boost::thread_interrupted& ti) { + } catch (const boost::thread_interrupted&) { SILK_TRACE << "thread_interrupted in block provider thread"; } catch (const std::exception& ex) { SILK_WARN << "unexpected exception in block provider thread: what=" << ex.what(); @@ -448,7 +450,8 @@ class BlockProvider { private: BoundedBuffer>* block_buffer_; - mdbx::env env_; + db::ROAccess db_access_; + db::DataModelFactory data_model_factory_; BlockNum start_block_; BlockNum max_block_; }; @@ -488,7 +491,7 @@ int silkworm_execute_blocks_ephemeral(SilkwormHandle handle, MDBX_txn* mdbx_txn, try { auto txn = db::RWTxnUnmanaged{mdbx_txn}; - db::Buffer state_buffer{txn}; + db::Buffer state_buffer{txn, std::make_unique(db::DataModel{txn, *handle->snapshot_repository})}; state_buffer.set_memory_limit(batch_size); const size_t max_batch_size{batch_size}; @@ -497,7 +500,7 @@ int silkworm_execute_blocks_ephemeral(SilkwormHandle handle, MDBX_txn* mdbx_txn, BlockNum block_number{start_block}; BlockNum batch_start_block_number{start_block}; BlockNum last_block_number = 0; - db::DataModel da_layer{txn}; + db::DataModel da_layer{txn, *handle->snapshot_repository}; AnalysisCache analysis_cache{execution::block::BlockExecutor::kDefaultAnalysisCacheSize}; execution::block::BlockExecutor block_executor{*chain_info, write_receipts, write_call_traces, write_change_sets}; @@ -607,12 +610,23 @@ int silkworm_execute_blocks_perpetual(SilkwormHandle handle, MDBX_env* mdbx_env, auto txn = db::RWTxnManaged{unmanaged_env}; const auto env_path = unmanaged_env.get_path(); - db::Buffer state_buffer{txn}; + db::Buffer state_buffer{txn, std::make_unique(db::DataModel{txn, *handle->snapshot_repository})}; state_buffer.set_memory_limit(batch_size); BoundedBuffer> block_buffer{kMaxBlockBufferSize}; [[maybe_unused]] auto _ = gsl::finally([&block_buffer] { block_buffer.terminate_and_release_all(); }); - BlockProvider block_provider{&block_buffer, unmanaged_env, start_block, max_block}; + + db::DataModelFactory data_model_factory = [handle](db::ROTxn& tx) { + return db::DataModel{tx, *handle->snapshot_repository}; + }; + + BlockProvider block_provider{ + &block_buffer, + db::ROAccess{unmanaged_env}, + std::move(data_model_factory), + start_block, + max_block, + }; boost::strict_scoped_thread block_provider_thread(block_provider); const size_t max_batch_size{batch_size}; diff --git a/silkworm/db/access_layer.cpp b/silkworm/db/access_layer.cpp index 33c90db6e3..baf011549c 100644 --- a/silkworm/db/access_layer.cpp +++ b/silkworm/db/access_layer.cpp @@ -1011,13 +1011,6 @@ void write_last_finalized_block(RWTxn& txn, const evmc::bytes32& hash) { write_last_fcu_field(txn, kFinalizedBlockHash, hash); } -void DataModel::set_snapshot_repository(snapshots::SnapshotRepository* repository) { - ensure(repository, "DataModel::set_snapshot_repository: repository is null"); - repository_ = repository; -} - -DataModel::DataModel(ROTxn& txn) : txn_{txn} {} - std::optional DataModel::read_chain_config() const { return db::read_chain_config(txn_); } @@ -1045,12 +1038,12 @@ BlockNum DataModel::highest_block_number() const { } // If none is found on db, then ask the snapshot repository (if any) for highest block - return repository_ ? repository_->max_block_available() : 0; + return repository_.max_block_available(); } -BlockNum DataModel::highest_frozen_block_number() { +BlockNum DataModel::highest_frozen_block_number() const { // Ask the snapshot repository (if any) for highest block - return repository_ ? repository_->max_block_available() : 0; + return repository_.max_block_available(); } std::optional DataModel::read_header(BlockNum block_number, HashAsArray block_hash) const { @@ -1058,7 +1051,8 @@ std::optional DataModel::read_header(BlockNum block_number, HashAsA } std::optional DataModel::read_header(BlockNum block_number, const Hash& block_hash) const { - if (repository_ && repository_->max_block_available() && block_number <= repository_->max_block_available()) { + BlockNum repository_max_block_num = repository_.max_block_available(); + if ((repository_max_block_num > 0) && (block_number <= repository_max_block_num)) { auto header = read_header_from_snapshot(block_number); if (header && header->hash() == block_hash) { // reading using hash avoid this heavy hash calculation return header; @@ -1069,7 +1063,8 @@ std::optional DataModel::read_header(BlockNum block_number, const H } std::optional DataModel::read_header(BlockNum block_number) const { - if (repository_ && repository_->max_block_available() && block_number <= repository_->max_block_available()) { + BlockNum repository_max_block_num = repository_.max_block_available(); + if ((repository_max_block_num > 0) && (block_number <= repository_max_block_num)) { return read_header_from_snapshot(block_number); } auto hash = db::read_canonical_header_hash(txn_, block_number); @@ -1085,6 +1080,14 @@ std::optional DataModel::read_header(const Hash& block_hash) const return read_header_from_snapshot(block_hash); } +std::pair, std::optional> DataModel::read_head_header_and_hash() const { + auto hash_opt = read_head_header_hash(txn_); + if (!hash_opt) return {std::nullopt, std::nullopt}; + Hash hash{*hash_opt}; + auto header = read_header(hash); + return {std::move(header), hash}; +} + std::optional DataModel::read_block_number(const Hash& block_hash) const { // Assume recent blocks are more probable: first lookup the block in the db auto block_number{db::read_block_number(txn_, block_hash)}; @@ -1215,10 +1218,10 @@ void DataModel::for_last_n_headers(size_t n, absl::FunctionRefmax_block_available() : 0; + auto block_number_in_snapshots = repository_.max_block_available(); // We've reached the first header in db but still need to read more from snapshots - if (repository_ && last_read_number_from_db > 0) { + if (last_read_number_from_db > 0) { ensure(*last_read_number_from_db == block_number_in_snapshots + 1, "db and snapshot block numbers are not contiguous"); } @@ -1241,11 +1244,7 @@ bool DataModel::read_block(BlockNum number, bool read_senders, Block& block) con return read_block(hash->bytes, number, read_senders, block); } -bool DataModel::read_block_from_snapshot(BlockNum height, Block& block) { - if (!repository_) { - return false; - } - +bool DataModel::read_block_from_snapshot(BlockNum height, Block& block) const { auto block_header{read_header_from_snapshot(height)}; if (!block_header) return false; @@ -1254,28 +1253,20 @@ bool DataModel::read_block_from_snapshot(BlockNum height, Block& block) { return read_body_from_snapshot(height, block); } -std::optional DataModel::read_header_from_snapshot(BlockNum height) { - if (!repository_) { - return {}; - } - +std::optional DataModel::read_header_from_snapshot(BlockNum height) const { std::optional block_header; // We know the header snapshot in advance: find it based on target block number - const auto [segment_and_index, _] = repository_->find_segment(SnapshotType::headers, height); + const auto [segment_and_index, _] = repository_.find_segment(SnapshotType::headers, height); if (segment_and_index) { block_header = HeaderFindByBlockNumQuery{*segment_and_index}.exec(height); } return block_header; } -std::optional DataModel::read_header_from_snapshot(const Hash& hash) { - if (!repository_) { - return {}; - } - +std::optional DataModel::read_header_from_snapshot(const Hash& hash) const { std::optional block_header; // We don't know the header snapshot in advance: search for block hash in each header snapshot in reverse order - for (const auto& bundle_ptr : repository_->view_bundles_reverse()) { + for (const auto& bundle_ptr : repository_.view_bundles_reverse()) { const auto& bundle = *bundle_ptr; auto segment_and_index = bundle.segment_and_index(SnapshotType::headers); block_header = HeaderFindByHashQuery{segment_and_index}.exec(hash); @@ -1284,20 +1275,11 @@ std::optional DataModel::read_header_from_snapshot(const Hash& hash return block_header; } -std::optional DataModel::read_body_for_storage_from_snapshot(BlockNum height) { - if (!repository_) { - return std::nullopt; - } - - // We know the body snapshot in advance: find it based on target block number - const auto [segment_and_index, _] = repository_->find_segment(SnapshotType::bodies, height); - if (!segment_and_index) return std::nullopt; - - auto stored_body = BodyFindByBlockNumQuery{*segment_and_index}.exec(height); - return stored_body; +std::optional DataModel::read_body_for_storage_from_snapshot(BlockNum height) const { + return BodyFindByBlockNumMultiQuery{repository_}.exec(height); } -bool DataModel::read_body_from_snapshot(BlockNum height, BlockBody& body) { +bool DataModel::read_body_from_snapshot(BlockNum height, BlockBody& body) const { auto stored_body = read_body_for_storage_from_snapshot(height); if (!stored_body) return false; @@ -1315,13 +1297,9 @@ bool DataModel::read_body_from_snapshot(BlockNum height, BlockBody& body) { return true; } -bool DataModel::is_body_in_snapshot(BlockNum height) { - if (!repository_) { - return false; - } - +bool DataModel::is_body_in_snapshot(BlockNum height) const { // We know the body snapshot in advance: find it based on target block number - const auto [segment_and_index, _] = repository_->find_segment(SnapshotType::bodies, height); + const auto [segment_and_index, _] = repository_.find_segment(SnapshotType::bodies, height); if (segment_and_index) { const auto stored_body = BodyFindByBlockNumQuery{*segment_and_index}.exec(height); return stored_body.has_value(); @@ -1330,12 +1308,12 @@ bool DataModel::is_body_in_snapshot(BlockNum height) { return false; } -bool DataModel::read_transactions_from_snapshot(BlockNum height, uint64_t base_txn_id, uint64_t txn_count, std::vector& txs) { +bool DataModel::read_transactions_from_snapshot(BlockNum height, uint64_t base_txn_id, uint64_t txn_count, std::vector& txs) const { if (txn_count == 0) { return true; } - const auto [segment_and_index, _] = repository_->find_segment(SnapshotType::transactions, height); + const auto [segment_and_index, _] = repository_.find_segment(SnapshotType::transactions, height); if (!segment_and_index) return false; txs = TransactionRangeFromIdQuery{*segment_and_index}.exec_into_vector(base_txn_id, txn_count); @@ -1343,8 +1321,8 @@ bool DataModel::read_transactions_from_snapshot(BlockNum height, uint64_t base_t return true; } -bool DataModel::read_rlp_transactions_from_snapshot(BlockNum height, std::vector& rlp_txs) { - const auto [body_segment_and_index, _] = repository_->find_segment(SnapshotType::bodies, height); +bool DataModel::read_rlp_transactions_from_snapshot(BlockNum height, std::vector& rlp_txs) const { + const auto [body_segment_and_index, _] = repository_.find_segment(SnapshotType::bodies, height); if (body_segment_and_index) { auto stored_body = BodyFindByBlockNumQuery{*body_segment_and_index}.exec(height); if (!stored_body) return false; @@ -1355,7 +1333,7 @@ bool DataModel::read_rlp_transactions_from_snapshot(BlockNum height, std::vector if (txn_count == 0) return true; - const auto [tx_segment_and_index, _2] = repository_->find_segment(SnapshotType::transactions, height); + const auto [tx_segment_and_index, _2] = repository_.find_segment(SnapshotType::transactions, height); if (!tx_segment_and_index) return false; rlp_txs = TransactionPayloadRlpRangeFromIdQuery{*tx_segment_and_index}.exec_into_vector(base_txn_id, txn_count); @@ -1391,12 +1369,8 @@ std::optional DataModel::read_tx_lookup_from_db(const evmc::bytes32& t return std::stoul(silkworm::to_hex(from_slice(data.value)), nullptr, 16); } -std::optional DataModel::read_tx_lookup_from_snapshot(const evmc::bytes32& tx_hash) { - if (!repository_) { - return {}; - } - - TransactionBlockNumByTxnHashRepoQuery query{repository_->view_bundles_reverse()}; +std::optional DataModel::read_tx_lookup_from_snapshot(const evmc::bytes32& tx_hash) const { + TransactionBlockNumByTxnHashRepoQuery query{repository_.view_bundles_reverse()}; return query.exec(tx_hash); } diff --git a/silkworm/db/access_layer.hpp b/silkworm/db/access_layer.hpp index dad2a13e46..1937c93e4d 100644 --- a/silkworm/db/access_layer.hpp +++ b/silkworm/db/access_layer.hpp @@ -261,14 +261,11 @@ void write_last_finalized_block(RWTxn& txn, const evmc::bytes32& hash); class DataModel { public: - static void set_snapshot_repository(snapshots::SnapshotRepository* repository); - - explicit DataModel(ROTxn& txn); - ~DataModel() = default; - - // Not copyable nor movable - DataModel(const DataModel&) = delete; - DataModel& operator=(const DataModel&) = delete; + DataModel( + ROTxn& txn, + snapshots::SnapshotRepository& repository) + : txn_{txn}, + repository_{repository} {} //! Retrieve the chain configuration for which database is populated std::optional read_chain_config() const; @@ -280,7 +277,7 @@ class DataModel { BlockNum highest_block_number() const; //! Get the highest block number frozen into snapshots - static BlockNum highest_frozen_block_number(); + BlockNum highest_frozen_block_number() const; //! Read block header with the specified key (block number, hash) std::optional read_header(BlockNum block_number, HashAsArray hash) const; @@ -294,6 +291,9 @@ class DataModel { //! Read block header with the specified block number std::optional read_header(BlockNum block_number) const; + //! Reads the highest header hash from table::kHeadHeader and a corresponding header + std::pair, std::optional> read_head_header_and_hash() const; + //! Read block number from hash std::optional read_block_number(const Hash& block_hash) const; @@ -306,7 +306,7 @@ class DataModel { [[nodiscard]] bool read_body(const Hash& hash, BlockBody& body) const; //! Read block body for storage from the snapshot repository - static std::optional read_body_for_storage_from_snapshot(BlockNum height); + std::optional read_body_for_storage_from_snapshot(BlockNum height) const; //! Read the canonical block header at specified height std::optional read_canonical_header_hash(BlockNum height) const; @@ -343,19 +343,20 @@ class DataModel { void for_last_n_headers(size_t n, absl::FunctionRef callback) const; private: - static bool read_block_from_snapshot(BlockNum height, Block& block); - static std::optional read_header_from_snapshot(BlockNum height); - static std::optional read_header_from_snapshot(const Hash& hash); - static bool read_body_from_snapshot(BlockNum height, BlockBody& body); - static bool is_body_in_snapshot(BlockNum height); - static bool read_rlp_transactions_from_snapshot(BlockNum height, std::vector& rlp_txs); - static bool read_transactions_from_snapshot(BlockNum height, uint64_t base_txn_id, uint64_t txn_count, std::vector& txs); + bool read_block_from_snapshot(BlockNum height, Block& block) const; + std::optional read_header_from_snapshot(BlockNum height) const; + std::optional read_header_from_snapshot(const Hash& hash) const; + bool read_body_from_snapshot(BlockNum height, BlockBody& body) const; + bool is_body_in_snapshot(BlockNum height) const; + bool read_rlp_transactions_from_snapshot(BlockNum height, std::vector& rlp_txs) const; + bool read_transactions_from_snapshot(BlockNum height, uint64_t base_txn_id, uint64_t txn_count, std::vector& txs) const; std::optional read_tx_lookup_from_db(const evmc::bytes32& tx_hash) const; - static std::optional read_tx_lookup_from_snapshot(const evmc::bytes32& tx_hash); - - static inline snapshots::SnapshotRepository* repository_{nullptr}; + std::optional read_tx_lookup_from_snapshot(const evmc::bytes32& tx_hash) const; ROTxn& txn_; + snapshots::SnapshotRepository& repository_; }; +using DataModelFactory = std::function; + } // namespace silkworm::db diff --git a/silkworm/db/access_layer_test.cpp b/silkworm/db/access_layer_test.cpp index d09f4c67d8..39b19de6fb 100644 --- a/silkworm/db/access_layer_test.cpp +++ b/silkworm/db/access_layer_test.cpp @@ -452,8 +452,6 @@ TEST_CASE("Difficulty", "[db][access_layer]") { write_total_difficulty(txn, block_num, hash, difficulty); CHECK(read_total_difficulty(txn, block_num, hash) == difficulty); - DataModel model{txn}; - CHECK(model.read_total_difficulty(block_num, hash) == difficulty); } TEST_CASE("Headers and bodies", "[db][access_layer]") { diff --git a/silkworm/db/blocks/bodies/body_queries.hpp b/silkworm/db/blocks/bodies/body_queries.hpp index 66bc13c9c4..d594b92a85 100644 --- a/silkworm/db/blocks/bodies/body_queries.hpp +++ b/silkworm/db/blocks/bodies/body_queries.hpp @@ -17,6 +17,7 @@ #pragma once #include +#include #include "body_segment.hpp" @@ -24,4 +25,20 @@ namespace silkworm::snapshots { using BodyFindByBlockNumQuery = FindByIdQuery; +class BodyFindByBlockNumMultiQuery { + public: + // TODO: use a sub-interface of SnapshotRepository + explicit BodyFindByBlockNumMultiQuery(SnapshotRepository& repository) + : repository_{repository} {} + + std::optional exec(BlockNum block_num) { + const auto [segment_and_index, _] = repository_.find_segment(SnapshotType::bodies, block_num); + if (!segment_and_index) return std::nullopt; + return BodyFindByBlockNumQuery{*segment_and_index}.exec(block_num); + } + + private: + SnapshotRepository& repository_; +}; + } // namespace silkworm::snapshots diff --git a/silkworm/db/buffer.cpp b/silkworm/db/buffer.cpp index aae00fd6cb..c75420b548 100644 --- a/silkworm/db/buffer.cpp +++ b/silkworm/db/buffer.cpp @@ -504,7 +504,7 @@ std::optional Buffer::read_header(uint64_t block_number, const evmc if (auto it{headers_.find(key)}; it != headers_.end()) { return it->second; } - return access_layer_.read_header(block_number, block_hash.bytes); + return data_model_->read_header(block_number, Hash{block_hash.bytes}); } bool Buffer::read_body(uint64_t block_number, const evmc::bytes32& block_hash, BlockBody& out) const noexcept { @@ -513,7 +513,7 @@ bool Buffer::read_body(uint64_t block_number, const evmc::bytes32& block_hash, B out = it->second; return true; } - return access_layer_.read_body(block_number, block_hash.bytes, /*read_senders=*/false, out); + return data_model_->read_body(block_number, block_hash.bytes, /*read_senders=*/false, out); } std::optional Buffer::read_account(const evmc::address& address) const noexcept { diff --git a/silkworm/db/buffer.hpp b/silkworm/db/buffer.hpp index 97a2c0402d..823612e75a 100644 --- a/silkworm/db/buffer.hpp +++ b/silkworm/db/buffer.hpp @@ -17,6 +17,7 @@ #pragma once #include +#include #include #include #include @@ -37,11 +38,49 @@ namespace silkworm::db { +struct BufferDataModel { + virtual ~BufferDataModel() = default; + virtual std::optional read_header(BlockNum block_num, const Hash& block_hash) const = 0; + [[nodiscard]] virtual bool read_body(BlockNum block_num, HashAsArray hash, bool read_senders, BlockBody& body) const = 0; +}; + +class BufferROTxDataModel : public BufferDataModel { + public: + explicit BufferROTxDataModel(ROTxn& tx) : tx_{tx} {} + ~BufferROTxDataModel() override = default; + std::optional read_header(BlockNum block_num, const Hash& block_hash) const override { + return db::read_header(tx_, block_num, block_hash); + } + [[nodiscard]] bool read_body(BlockNum block_num, HashAsArray hash, bool read_senders, BlockBody& body) const override { + return db::read_body(tx_, block_num, hash, read_senders, body); + } + + private: + ROTxn& tx_; +}; + +class BufferFullDataModel : public BufferDataModel { + public: + explicit BufferFullDataModel(DataModel data_model) : data_model_{data_model} {} + ~BufferFullDataModel() override = default; + std::optional read_header(BlockNum block_num, const Hash& block_hash) const override { + return data_model_.read_header(block_num, block_hash); + } + [[nodiscard]] bool read_body(BlockNum block_num, HashAsArray hash, bool read_senders, BlockBody& body) const override { + return data_model_.read_body(block_num, hash, read_senders, body); + } + + private: + DataModel data_model_; +}; + class Buffer : public State { public: - explicit Buffer(RWTxn& txn) + explicit Buffer( + RWTxn& txn, + std::unique_ptr data_model) : txn_{txn}, - access_layer_{txn_} {} + data_model_{std::move(data_model)} {} /** @name Settings */ //!@{ @@ -161,7 +200,7 @@ class Buffer : public State { private: RWTxn& txn_; - DataModel access_layer_; + std::unique_ptr data_model_; // Settings diff --git a/silkworm/db/buffer_test.cpp b/silkworm/db/buffer_test.cpp index 4708aad6c7..dce0af9087 100644 --- a/silkworm/db/buffer_test.cpp +++ b/silkworm/db/buffer_test.cpp @@ -54,7 +54,7 @@ TEST_CASE("Buffer storage", "[silkworm][db][buffer]") { upsert_storage_value(*state, key, location_a.bytes, value_a1.bytes); upsert_storage_value(*state, key, location_b.bytes, value_b.bytes); - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; SECTION("Reads storage by address and location") { CHECK(buffer.read_storage(address, kDefaultIncarnation, location_a) == value_a1); @@ -251,7 +251,7 @@ TEST_CASE("Buffer account", "[silkworm][db][buffer]") { Account current_account; current_account.balance = kEther; - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; buffer.begin_block(1, 1); buffer.update_account(address, /*initial=*/std::nullopt, current_account); REQUIRE(!buffer.account_changes().empty()); @@ -284,7 +284,7 @@ TEST_CASE("Buffer account", "[silkworm][db][buffer]") { current_account.nonce = 2; current_account.balance = kEther; - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; buffer.begin_block(1, 1); buffer.update_account(address, /*initial=*/initial_account, current_account); REQUIRE(!buffer.account_changes().empty()); @@ -316,7 +316,7 @@ TEST_CASE("Buffer account", "[silkworm][db][buffer]") { account.incarnation = kDefaultIncarnation; account.code_hash = to_bytes32(keccak256(address.bytes).bytes); // Just a fake hash - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; buffer.begin_block(1, 1); buffer.update_account(address, /*initial=*/account, /*current=*/std::nullopt); REQUIRE(!buffer.account_changes().empty()); @@ -338,7 +338,7 @@ TEST_CASE("Buffer account", "[silkworm][db][buffer]") { account.code_hash = to_bytes32(keccak256(address.bytes).bytes); // Just a fake hash // Block 1: create contract account - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; buffer.begin_block(1, 1); buffer.update_account(address, /*initial=*/std::nullopt, /*current=*/account); REQUIRE(!buffer.account_changes().empty()); @@ -369,7 +369,7 @@ TEST_CASE("Buffer account", "[silkworm][db][buffer]") { current_account.nonce = 2; current_account.balance = kEther; - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; buffer.begin_block(1, 1); buffer.update_account(address, /*initial=*/initial_account, current_account); REQUIRE(buffer.account_changes().empty()); diff --git a/silkworm/db/chain/local_chain_storage.cpp b/silkworm/db/chain/local_chain_storage.cpp index 11a26d5a8e..8def0116e5 100644 --- a/silkworm/db/chain/local_chain_storage.cpp +++ b/silkworm/db/chain/local_chain_storage.cpp @@ -22,8 +22,6 @@ namespace silkworm::db::chain { -LocalChainStorage::LocalChainStorage(ROTxn& txn) : data_model_{txn} {} - Task LocalChainStorage::read_chain_config() const { const auto chain_config{data_model_.read_chain_config()}; if (!chain_config) { diff --git a/silkworm/db/chain/local_chain_storage.hpp b/silkworm/db/chain/local_chain_storage.hpp index 19257db166..5b4e7530f3 100644 --- a/silkworm/db/chain/local_chain_storage.hpp +++ b/silkworm/db/chain/local_chain_storage.hpp @@ -17,7 +17,6 @@ #pragma once #include -#include #include "chain_storage.hpp" @@ -27,7 +26,8 @@ namespace silkworm::db::chain { //! in local database (accessed via MDBX API) or local snapshot files (accessed via custom snapshot API) class LocalChainStorage : public ChainStorage { public: - explicit LocalChainStorage(db::ROTxn& txn); + explicit LocalChainStorage(db::DataModel data_model) + : data_model_{data_model} {} ~LocalChainStorage() override = default; Task read_chain_config() const override; diff --git a/silkworm/db/data_store.hpp b/silkworm/db/data_store.hpp new file mode 100644 index 0000000000..c1a8525610 --- /dev/null +++ b/silkworm/db/data_store.hpp @@ -0,0 +1,29 @@ +/* + Copyright 2024 The Silkworm Authors + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#pragma once + +#include +#include + +namespace silkworm::db { + +struct DataStoreRef { + mdbx::env chaindata_env; + snapshots::SnapshotRepository& repository; +}; + +} // namespace silkworm::db diff --git a/silkworm/db/datastore/snapshots/snapshot_repository.cpp b/silkworm/db/datastore/snapshots/snapshot_repository.cpp index 8004065643..ea0bad1305 100644 --- a/silkworm/db/datastore/snapshots/snapshot_repository.cpp +++ b/silkworm/db/datastore/snapshots/snapshot_repository.cpp @@ -28,10 +28,10 @@ namespace silkworm::snapshots { namespace fs = std::filesystem; SnapshotRepository::SnapshotRepository( - SnapshotSettings settings, + std::filesystem::path dir_path, std::unique_ptr step_converter, std::unique_ptr bundle_factory) - : settings_(std::move(settings)), + : dir_path_(std::move(dir_path)), step_converter_(std::move(step_converter)), bundle_factory_(std::move(bundle_factory)), bundles_(std::make_shared()) {} @@ -68,7 +68,7 @@ size_t SnapshotRepository::bundles_count() const { } void SnapshotRepository::close() { - SILK_TRACE << "Close snapshot repository folder: " << settings_.repository_dir.string(); + SILK_TRACE << "Close snapshot repository folder: " << dir_path_.string(); std::scoped_lock lock(bundles_mutex_); bundles_ = std::make_shared(); } @@ -114,7 +114,7 @@ std::vector> SnapshotRepository::missing_indexes() } void SnapshotRepository::reopen_folder() { - SILK_INFO << "Reopen snapshot repository folder: " << settings_.repository_dir.string(); + SILK_INFO << "Reopen snapshot repository folder: " << dir_path_.string(); SnapshotPathList all_snapshot_paths = get_segment_files(); SnapshotPathList all_index_paths = get_idx_files(); @@ -198,14 +198,14 @@ std::vector> SnapshotRepository::bundles_in_rang } SnapshotPathList SnapshotRepository::get_files(const std::string& ext) const { - ensure(fs::exists(settings_.repository_dir), - [&]() { return "SnapshotRepository: " + settings_.repository_dir.string() + " does not exist"; }); - ensure(fs::is_directory(settings_.repository_dir), - [&]() { return "SnapshotRepository: " + settings_.repository_dir.string() + " is a not folder"; }); + ensure(fs::exists(dir_path_), + [&]() { return "SnapshotRepository: " + dir_path_.string() + " does not exist"; }); + ensure(fs::is_directory(dir_path_), + [&]() { return "SnapshotRepository: " + dir_path_.string() + " is a not folder"; }); // Load the resulting files w/ desired extension ensuring they are snapshots SnapshotPathList snapshot_files; - for (const auto& file : fs::directory_iterator{settings_.repository_dir}) { + for (const auto& file : fs::directory_iterator{dir_path_}) { if (!fs::is_regular_file(file.path()) || file.path().extension().string() != ext) { continue; } diff --git a/silkworm/db/datastore/snapshots/snapshot_repository.hpp b/silkworm/db/datastore/snapshots/snapshot_repository.hpp index 7dba14ff7f..e95a2a1703 100644 --- a/silkworm/db/datastore/snapshots/snapshot_repository.hpp +++ b/silkworm/db/datastore/snapshots/snapshot_repository.hpp @@ -33,7 +33,6 @@ #include "segment_and_index.hpp" #include "snapshot_bundle.hpp" #include "snapshot_bundle_factory.hpp" -#include "snapshot_settings.hpp" namespace silkworm::snapshots { @@ -46,14 +45,13 @@ struct IndexBuilder; //! - segments have [from:to) semantic class SnapshotRepository { public: - explicit SnapshotRepository( - SnapshotSettings settings, + SnapshotRepository( + std::filesystem::path dir_path, std::unique_ptr step_converter, std::unique_ptr bundle_factory); ~SnapshotRepository(); - const SnapshotSettings& settings() const { return settings_; } - std::filesystem::path path() const { return settings_.repository_dir; } + const std::filesystem::path& path() const { return dir_path_; } const SnapshotBundleFactory& bundle_factory() const { return *bundle_factory_; } void reopen_folder(); @@ -125,8 +123,8 @@ class SnapshotRepository { SnapshotPathList stale_index_paths() const; - //! The configuration settings for snapshots - SnapshotSettings settings_; + //! Path to the snapshots directory + std::filesystem::path dir_path_; //! Converts timestamp units to steps std::unique_ptr step_converter_; diff --git a/silkworm/db/freezer.cpp b/silkworm/db/freezer.cpp index efbdf1961e..7f0acf59dd 100644 --- a/silkworm/db/freezer.cpp +++ b/silkworm/db/freezer.cpp @@ -26,6 +26,7 @@ #include #include "access_layer.hpp" +#include "blocks/bodies/body_queries.hpp" #include "blocks/bodies/body_segment_collation.hpp" #include "blocks/headers/header_segment_collation.hpp" #include "datastore/segment_collation.hpp" @@ -57,8 +58,8 @@ static BlockNum get_first_stored_header_num(ROTxn& txn) { return num_opt.value_or(0); } -static std::optional get_next_base_txn_id(BlockNum number) { - auto body = DataModel::read_body_for_storage_from_snapshot(number); +static std::optional get_next_base_txn_id(SnapshotRepository& repository, BlockNum number) { + auto body = BodyFindByBlockNumMultiQuery{repository}.exec(number); if (!body) return std::nullopt; return body->base_txn_id + body->txn_count; } @@ -73,9 +74,9 @@ std::unique_ptr Freezer::next_command() { return get_tip_num(db_tx); }(); - uint64_t base_txn_id = [last_frozen]() -> uint64_t { + uint64_t base_txn_id = [this, last_frozen]() -> uint64_t { if (last_frozen == 0) return 0; - auto id = get_next_base_txn_id(last_frozen); + auto id = get_next_base_txn_id(snapshots_, last_frozen); SILKWORM_ASSERT(id.has_value()); return *id; }(); diff --git a/silkworm/db/kv/api/direct_service.cpp b/silkworm/db/kv/api/direct_service.cpp index 11519a9d09..5c195e498a 100644 --- a/silkworm/db/kv/api/direct_service.cpp +++ b/silkworm/db/kv/api/direct_service.cpp @@ -23,8 +23,13 @@ namespace silkworm::db::kv::api { -DirectService::DirectService(ServiceRouter router, ::mdbx::env chaindata_env, StateCache* state_cache) - : router_{router}, chaindata_env_{std::move(chaindata_env)}, state_cache_{state_cache} {} +DirectService::DirectService( + ServiceRouter router, + DataStoreRef data_store, + StateCache* state_cache) + : router_{router}, + data_store_{std::move(data_store)}, + state_cache_{state_cache} {} // rpc Version(google.protobuf.Empty) returns (types.VersionReply); Task DirectService::version() { @@ -33,7 +38,7 @@ Task DirectService::version() { // rpc Tx(stream Cursor) returns (stream Pair); Task> DirectService::begin_transaction() { - co_return std::make_unique(chaindata_env_, state_cache_); + co_return std::make_unique(data_store_, state_cache_); } // rpc StateChanges(StateChangeRequest) returns (stream StateChangeBatch); diff --git a/silkworm/db/kv/api/direct_service.hpp b/silkworm/db/kv/api/direct_service.hpp index 5c55dbab0e..602ebfa889 100644 --- a/silkworm/db/kv/api/direct_service.hpp +++ b/silkworm/db/kv/api/direct_service.hpp @@ -16,7 +16,7 @@ #pragma once -#include +#include #include "service.hpp" #include "service_router.hpp" @@ -28,7 +28,10 @@ namespace silkworm::db::kv::api { //! This is used both client-side by 'direct' (i.e. no-gRPC) implementation and server-side by gRPC server. class DirectService : public Service { public: - explicit DirectService(ServiceRouter router, ::mdbx::env chaindata_env, StateCache* state_cache); + explicit DirectService( + ServiceRouter router, + DataStoreRef data_store, + StateCache* state_cache); ~DirectService() override = default; DirectService(const DirectService&) = delete; @@ -50,8 +53,8 @@ class DirectService : public Service { //! The router to service endpoint implementation ServiceRouter router_; - //! The MDBX chain database - ::mdbx::env chaindata_env_; + //! The data store + DataStoreRef data_store_; //! The local state cache built upon incoming state changes StateCache* state_cache_; diff --git a/silkworm/db/kv/api/direct_service_test.cpp b/silkworm/db/kv/api/direct_service_test.cpp index 8cfe4231c6..1a2d261784 100644 --- a/silkworm/db/kv/api/direct_service_test.cpp +++ b/silkworm/db/kv/api/direct_service_test.cpp @@ -18,13 +18,16 @@ #include +#include #include +#include #include #include namespace silkworm::db::kv::api { using namespace silkworm::test_util; +using test_util::make_repository; using test_util::TestDatabaseContext; struct DirectServiceTest : public test_util::KVTestBase, TestDatabaseContext { @@ -32,11 +35,13 @@ struct DirectServiceTest : public test_util::KVTestBase, TestDatabaseContext { if (!change_set) co_return; change_set_vector.push_back(*change_set); } + DataStoreRef data_store() { return {mdbx_env(), repository}; } StateChangeChannelPtr channel{std::make_shared(io_context_.get_executor())}; concurrency::Channel state_changes_calls_channel{io_context_.get_executor()}; + snapshots::SnapshotRepository repository{make_repository()}; std::unique_ptr state_cache{std::make_unique()}; - DirectService service{ServiceRouter{state_changes_calls_channel}, mdbx_env(), state_cache.get()}; + DirectService service{ServiceRouter{state_changes_calls_channel}, data_store(), state_cache.get()}; std::vector change_set_vector; }; diff --git a/silkworm/db/kv/api/local_transaction.cpp b/silkworm/db/kv/api/local_transaction.cpp index c6353e0991..889613defa 100644 --- a/silkworm/db/kv/api/local_transaction.cpp +++ b/silkworm/db/kv/api/local_transaction.cpp @@ -62,12 +62,12 @@ Task> LocalTransaction::get_cursor(const std::str std::shared_ptr LocalTransaction::create_state(boost::asio::any_io_executor&, const chain::ChainStorage&, BlockNum block_number) { // The calling thread *must* be *different* from the one which created this LocalTransaction instance - return std::make_shared(block_number, chaindata_env_); + return std::make_shared(block_number, data_store_); } std::shared_ptr LocalTransaction::create_storage() { // The calling thread *must* be the *same* which created this LocalTransaction instance - return std::make_shared(txn_); + return std::make_shared(DataModel{txn_, data_store_.repository}); } Task LocalTransaction::first_txn_num_in_block(BlockNum /*block_num*/) { diff --git a/silkworm/db/kv/api/local_transaction.hpp b/silkworm/db/kv/api/local_transaction.hpp index 9b6cdf85f6..b6172cc19e 100644 --- a/silkworm/db/kv/api/local_transaction.hpp +++ b/silkworm/db/kv/api/local_transaction.hpp @@ -24,7 +24,7 @@ #include -#include +#include #include "base_transaction.hpp" #include "cursor.hpp" @@ -35,8 +35,12 @@ namespace silkworm::db::kv::api { class LocalTransaction : public BaseTransaction { public: - explicit LocalTransaction(mdbx::env chaindata_env, StateCache* state_cache) - : BaseTransaction(state_cache), chaindata_env_{std::move(chaindata_env)}, txn_{chaindata_env_} {} + explicit LocalTransaction( + DataStoreRef data_store, + StateCache* state_cache) + : BaseTransaction(state_cache), + data_store_{std::move(data_store)}, + txn_{data_store_.chaindata_env} {} ~LocalTransaction() override = default; @@ -80,7 +84,7 @@ class LocalTransaction : public BaseTransaction { std::map> cursors_; std::map> dup_cursors_; - mdbx::env chaindata_env_; + DataStoreRef data_store_; uint32_t last_cursor_id_{0}; ROTxnManaged txn_; uint64_t tx_id_{++next_tx_id_}; diff --git a/silkworm/db/kv/state_changes_stream_test.cpp b/silkworm/db/kv/state_changes_stream_test.cpp index 65c63ca459..9d34c74d27 100644 --- a/silkworm/db/kv/state_changes_stream_test.cpp +++ b/silkworm/db/kv/state_changes_stream_test.cpp @@ -39,6 +39,8 @@ #if !defined(__APPLE__) || defined(NDEBUG) #include #endif // !defined(__APPLE__) || defined(NDEBUG) +#include +#include #include #include #include @@ -49,6 +51,7 @@ namespace silkworm::db::kv { using namespace std::chrono_literals; // NOLINT(build/namespaces) using grpc::client::RemoteClient; +using test_util::make_repository; using testing::InvokeWithoutArgs; namespace test = rpc::test; @@ -73,7 +76,9 @@ struct StateChangesStreamTest : public StateCacheTestBase { }; struct DirectStateChangesStreamTest : public StateChangesStreamTest, test_util::TestDatabaseContext { - std::shared_ptr direct_service{std::make_shared(router, mdbx_env(), state_cache.get())}; + DataStoreRef data_store() { return {mdbx_env(), repository}; } + snapshots::SnapshotRepository repository{make_repository()}; + std::shared_ptr direct_service{std::make_shared(router, data_store(), state_cache.get())}; api::DirectClient direct_client{direct_service}; StateChangesStream stream{context_, direct_client}; }; diff --git a/silkworm/db/snapshot_benchmark.cpp b/silkworm/db/snapshot_benchmark.cpp index 33b2a04658..8e843e6de2 100644 --- a/silkworm/db/snapshot_benchmark.cpp +++ b/silkworm/db/snapshot_benchmark.cpp @@ -72,8 +72,7 @@ BENCHMARK(open_snapshot); static void build_header_index(benchmark::State& state) { TemporaryDirectory tmp_dir; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); // These sample snapshot files just contain data for block range [1'500'012, 1'500'013], hence current snapshot // file name format is not sufficient to support them (see checks commented out below) @@ -91,8 +90,7 @@ BENCHMARK(build_header_index); static void build_body_index(benchmark::State& state) { TemporaryDirectory tmp_dir; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); // These sample snapshot files just contain data for block range [1'500'012, 1'500'013], hence current snapshot // file name format is not sufficient to support them (see checks commented out below) @@ -108,8 +106,7 @@ BENCHMARK(build_body_index); static void build_tx_index(benchmark::State& state) { TemporaryDirectory tmp_dir; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); // These sample snapshot files just contain data for block range [1'500'012, 1'500'013], hence current snapshot // file name format is not sufficient to support them (see checks commented out below) @@ -134,8 +131,7 @@ BENCHMARK(build_tx_index); static void reopen_folder(benchmark::State& state) { SetLogVerbosityGuard guard{log::Level::kNone}; TemporaryDirectory tmp_dir; - snapshots::SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); // These sample snapshot files just contain data for block range [1'500'012, 1'500'013], hence current snapshot // file name format is not sufficient to support them (see checks commented out below) diff --git a/silkworm/db/snapshot_repository_test.cpp b/silkworm/db/snapshot_repository_test.cpp index 80537a07d3..7362197cde 100644 --- a/silkworm/db/snapshot_repository_test.cpp +++ b/silkworm/db/snapshot_repository_test.cpp @@ -44,7 +44,7 @@ using silkworm::test_util::SetLogVerbosityGuard; TEST_CASE("SnapshotRepository::SnapshotRepository", "[silkworm][node][snapshot]") { SetLogVerbosityGuard guard{log::Level::kNone}; - CHECK_NOTHROW(make_repository(SnapshotSettings{})); + CHECK_NOTHROW(make_repository()); } TEST_CASE("SnapshotRepository::reopen_folder.partial_bundle", "[silkworm][node][snapshot]") { @@ -54,8 +54,7 @@ TEST_CASE("SnapshotRepository::reopen_folder.partial_bundle", "[silkworm][node][ test::TemporarySnapshotFile tmp_snapshot_1{tmp_dir.path(), "v1-014500-015000-headers.seg"}; test::TemporarySnapshotFile tmp_snapshot_2{tmp_dir.path(), "v1-011500-012000-bodies.seg"}; test::TemporarySnapshotFile tmp_snapshot_3{tmp_dir.path(), "v1-015000-015500-transactions.seg"}; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); repository.reopen_folder(); CHECK(repository.bundles_count() == 0); CHECK(repository.max_block_available() == 0); @@ -65,8 +64,7 @@ TEST_CASE("SnapshotRepository::view", "[silkworm][node][snapshot]") { SetLogVerbosityGuard guard{log::Level::kNone}; TemporaryDirectory tmp_dir; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); SECTION("no snapshots") { repository.reopen_folder(); @@ -140,8 +138,7 @@ TEST_CASE("SnapshotRepository::view", "[silkworm][node][snapshot]") { TEST_CASE("SnapshotRepository::find_segment", "[silkworm][node][snapshot]") { SetLogVerbosityGuard guard{log::Level::kNone}; TemporaryDirectory tmp_dir; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); // These sample snapshot files just contain data for block range [1'500'012, 1'500'013], hence current snapshot // file name format is not sufficient to support them (see checks commented out below) @@ -207,8 +204,7 @@ TEST_CASE("SnapshotRepository::find_segment", "[silkworm][node][snapshot]") { TEST_CASE("SnapshotRepository::find_block_number", "[silkworm][node][snapshot]") { SetLogVerbosityGuard guard{log::Level::kNone}; TemporaryDirectory tmp_dir; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); // These sample snapshot files just contain data for block range [1'500'012, 1'500'013], hence current snapshot // file name format is not sufficient to support them (see checks commented out below) @@ -257,8 +253,7 @@ TEST_CASE("SnapshotRepository::remove_stale_indexes", "[silkworm][node][snapshot SetLogVerbosityGuard guard{log::Level::kNone}; TemporaryDirectory tmp_dir; - SnapshotSettings settings{tmp_dir.path()}; - auto repository = make_repository(settings); + auto repository = make_repository(tmp_dir.path()); // create a snapshot file test::SampleHeaderSnapshotFile header_segment_file{tmp_dir.path()}; diff --git a/silkworm/db/snapshot_sync.cpp b/silkworm/db/snapshot_sync.cpp index 1d9389cd82..b76ccbca16 100644 --- a/silkworm/db/snapshot_sync.cpp +++ b/silkworm/db/snapshot_sync.cpp @@ -37,7 +37,6 @@ #include "datastore/mdbx/etl_mdbx_collector.hpp" #include "datastore/snapshots/bittorrent/torrent_file.hpp" #include "datastore/snapshots/common/snapshot_path.hpp" -#include "snapshot_bundle_factory_impl.hpp" #include "stages.hpp" namespace silkworm::db { @@ -64,17 +63,13 @@ static bool snapshot_file_is_fully_merged(std::string_view file_name) { SnapshotSync::SnapshotSync( snapshots::SnapshotSettings settings, ChainId chain_id, - mdbx::env chaindata_env, + db::DataStoreRef data_store, std::filesystem::path tmp_dir_path, stagedsync::StageScheduler& stage_scheduler) : settings_{std::move(settings)}, snapshots_config_{Config::lookup_known_config(chain_id, snapshot_file_is_fully_merged)}, - chaindata_env_{std::move(chaindata_env)}, - repository_{ - settings_, - std::make_unique(), - std::make_unique(), - }, + chaindata_env_{std::move(data_store.chaindata_env)}, + repository_{data_store.repository}, client_{settings_.bittorrent_settings}, snapshot_freezer_{ROAccess{chaindata_env_}, repository_, stage_scheduler, tmp_dir_path, settings_.keep_blocks}, snapshot_merger_{repository_, std::move(tmp_dir_path)}, @@ -137,9 +132,6 @@ Task SnapshotSync::setup() { update_database(rw_txn, repository_.max_block_available(), [this] { return is_stopping_latch_.try_wait(); }); rw_txn.commit_and_stop(); - // Set snapshot repository into snapshot-aware database access - DataModel::set_snapshot_repository(&repository_); - seed_frozen_local_snapshots(); if (settings_.verify_on_startup) { diff --git a/silkworm/db/snapshot_sync.hpp b/silkworm/db/snapshot_sync.hpp index 5f2d9b2e1d..78b9c5dd75 100644 --- a/silkworm/db/snapshot_sync.hpp +++ b/silkworm/db/snapshot_sync.hpp @@ -32,6 +32,7 @@ #include #include "access_layer.hpp" +#include "data_store.hpp" #include "datastore/mdbx/mdbx.hpp" #include "datastore/snapshot_merger.hpp" #include "datastore/snapshots/bittorrent/client.hpp" @@ -50,7 +51,7 @@ class SnapshotSync { SnapshotSync( snapshots::SnapshotSettings settings, ChainId chain_id, - mdbx::env chaindata_env, + db::DataStoreRef data_store, std::filesystem::path tmp_dir_path, stagedsync::StageScheduler& stage_scheduler); @@ -81,7 +82,7 @@ class SnapshotSync { const snapshots::Config snapshots_config_; mdbx::env chaindata_env_; - snapshots::SnapshotRepository repository_; + snapshots::SnapshotRepository& repository_; snapshots::bittorrent::BitTorrentClient client_; diff --git a/silkworm/db/snapshot_sync_test.cpp b/silkworm/db/snapshot_sync_test.cpp index 2f730ff012..70301d5361 100644 --- a/silkworm/db/snapshot_sync_test.cpp +++ b/silkworm/db/snapshot_sync_test.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -53,6 +54,7 @@ struct SnapshotSyncTest { SetLogVerbosityGuard guard{log::Level::kNone}; TemporaryDirectory tmp_dir; db::test_util::TempChainData context; + SnapshotRepository repository{db::test_util::make_repository(tmp_dir.path())}; TaskRunner runner; NoopStageSchedulerAdapter stage_scheduler; }; @@ -84,7 +86,7 @@ struct SnapshotSyncForTest : public SnapshotSync { : SnapshotSync{ make_settings(test.tmp_dir.path(), overrides), kMainnetConfig.chain_id, - test.context.env(), + db::DataStoreRef{test.context.env(), test.repository}, test.tmp_dir.path(), test.stage_scheduler} {} }; diff --git a/silkworm/db/state/local_state.hpp b/silkworm/db/state/local_state.hpp index a0ccbf1751..8d34fd7a76 100644 --- a/silkworm/db/state/local_state.hpp +++ b/silkworm/db/state/local_state.hpp @@ -29,12 +29,14 @@ #include #include +#include "../data_store.hpp" + namespace silkworm::db::state { class LocalState : public State { public: - explicit LocalState(BlockNum block_number, mdbx::env chaindata_env) - : block_number_{block_number}, txn_{std::move(chaindata_env)}, data_model_{txn_} {} + explicit LocalState(BlockNum block_number, DataStoreRef data_store) + : block_number_{block_number}, txn_{data_store.chaindata_env}, data_model_{txn_, data_store.repository} {} std::optional read_account(const evmc::address& address) const noexcept override; diff --git a/silkworm/db/test_util/make_repository.cpp b/silkworm/db/test_util/make_repository.cpp index b939f1abdd..6d1b08f6c2 100644 --- a/silkworm/db/test_util/make_repository.cpp +++ b/silkworm/db/test_util/make_repository.cpp @@ -16,18 +16,24 @@ #include "make_repository.hpp" +#include "../datastore/snapshots/snapshot_settings.hpp" #include "../snapshot_bundle_factory_impl.hpp" namespace silkworm::db::test_util { using namespace silkworm::snapshots; -SnapshotRepository make_repository(SnapshotSettings settings) { +SnapshotRepository make_repository(std::filesystem::path dir_path) { return SnapshotRepository{ - std::move(settings), + std::move(dir_path), std::make_unique(), std::make_unique(), }; } +SnapshotRepository make_repository() { + SnapshotSettings settings; + return make_repository(settings.repository_dir); +} + } // namespace silkworm::db::test_util diff --git a/silkworm/db/test_util/make_repository.hpp b/silkworm/db/test_util/make_repository.hpp index 32c1b4f674..2096c116a4 100644 --- a/silkworm/db/test_util/make_repository.hpp +++ b/silkworm/db/test_util/make_repository.hpp @@ -17,10 +17,10 @@ #pragma once #include "../datastore/snapshots/snapshot_repository.hpp" -#include "../datastore/snapshots/snapshot_settings.hpp" namespace silkworm::db::test_util { -snapshots::SnapshotRepository make_repository(snapshots::SnapshotSettings settings); +snapshots::SnapshotRepository make_repository(std::filesystem::path dir_path); +snapshots::SnapshotRepository make_repository(); } // namespace silkworm::db::test_util diff --git a/silkworm/db/test_util/test_database_context.cpp b/silkworm/db/test_util/test_database_context.cpp index 18a2be82d8..93683bac63 100644 --- a/silkworm/db/test_util/test_database_context.cpp +++ b/silkworm/db/test_util/test_database_context.cpp @@ -127,7 +127,7 @@ void populate_blocks(RWTxn& txn, const std::filesystem::path& tests_dir, InMemor // FIX 4b: populate receipts and logs table std::vector receipts; ExecutionProcessor processor{block, *ruleSet, state_buffer, *chain_config}; - Buffer db_buffer{txn}; + Buffer db_buffer{txn, std::make_unique(txn)}; for (auto& block_txn : block.transactions) { silkworm::Receipt receipt{}; processor.execute_transaction(block_txn, receipt); diff --git a/silkworm/node/execution/header_chain_plus_exec_test.cpp b/silkworm/node/execution/header_chain_plus_exec_test.cpp index 389222e18d..8c233e8e3a 100644 --- a/silkworm/node/execution/header_chain_plus_exec_test.cpp +++ b/silkworm/node/execution/header_chain_plus_exec_test.cpp @@ -22,12 +22,13 @@ #include #include #include +#include #include #include #include #include #include -#include +#include #include #include #include @@ -38,6 +39,7 @@ using namespace stagedsync; using namespace silkworm::db; using silkworm::execution::api::ValidChain; +using silkworm::stagedsync::test_util::make_stages_factory; using silkworm::test_util::SetLogVerbosityGuard; using silkworm::test_util::TaskRunner; @@ -85,19 +87,19 @@ TEST_CASE("Headers receiving and saving") { context.add_genesis_data(); context.commit_txn(); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + db::DataModelFactory data_model_factory = [&](db::ROTxn& tx) { return db::DataModel{tx, repository}; }; + NodeSettings node_settings = node::test_util::make_node_settings_from_temp_chain_data(context); RWAccess db_access{context.env()}; - BodiesStageFactory bodies_stage_factory = [&](SyncContext* sync_context) { - return std::make_unique(sync_context, *node_settings.chain_config, [] { return 0; }); - }; - // creating the ExecutionEngine ExecutionEngineForTest exec_engine{ runner.executor(), node_settings, + data_model_factory, /* log_timer_factory = */ std::nullopt, - std::move(bodies_stage_factory), + make_stages_factory(node_settings, data_model_factory), db_access, }; exec_engine.open(); diff --git a/silkworm/node/node.cpp b/silkworm/node/node.cpp index c90794ddc8..c1dfcc34e2 100644 --- a/silkworm/node/node.cpp +++ b/silkworm/node/node.cpp @@ -22,6 +22,8 @@ #include #include +#include +#include #include #include #include @@ -34,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -63,6 +66,16 @@ class NodeImpl final { BlockNum last_pre_validated_block() const { return chain_sync_.last_pre_validated_block(); } private: + db::DataStoreRef data_store() { + return { + chaindata_env_, // NOLINT(cppcoreguidelines-slicing) + repository_, + }; + } + db::DataModelFactory data_model_factory() { + return [this](db::ROTxn& tx) { return db::DataModel{tx, repository_}; }; + } + Task run_execution_service(); Task run_execution_server(); Task run_backend_kv_grpc_server(); @@ -75,6 +88,8 @@ class NodeImpl final { ChainConfig& chain_config_; + snapshots::SnapshotRepository repository_; + //! The execution layer server engine boost::asio::io_context execution_context_; stagedsync::ExecutionEngine execution_engine_; @@ -140,15 +155,29 @@ static stagedsync::TimerFactory make_log_timer_factory( static stagedsync::BodiesStageFactory make_bodies_stage_factory( const ChainConfig& chain_config, + db::DataModelFactory data_model_factory, const NodeImpl& node) { - return [&](stagedsync::SyncContext* sync_context) { + return [&chain_config, data_model_factory = std::move(data_model_factory), &node](stagedsync::SyncContext* sync_context) { return std::make_unique( sync_context, chain_config, + data_model_factory, [&node]() { return node.last_pre_validated_block(); }); }; }; +static stagedsync::StageContainerFactory make_stages_factory( + const NodeSettings& node_settings, + db::DataModelFactory data_model_factory, + const NodeImpl& node) { + auto bodies_stage_factory = make_bodies_stage_factory(*node_settings.chain_config, data_model_factory, node); + return stagedsync::StagesFactoryImpl::to_factory({ + node_settings, + std::move(data_model_factory), + std::move(bodies_stage_factory), + }); +} + static sentry::SessionSentryClient::StatusDataProvider make_sentry_eth_status_data_provider( db::ROAccess db_access, const ChainConfig& chain_config) { @@ -165,11 +194,17 @@ NodeImpl::NodeImpl( : settings_{settings}, chaindata_env_{init_chain_data_db(settings.node_settings)}, chain_config_{*settings_.node_settings.chain_config}, + repository_{ + settings_.snapshot_settings.repository_dir, + std::make_unique(), + std::make_unique(), + }, execution_engine_{ execution_context_.get_executor(), settings_.node_settings, + data_model_factory(), make_log_timer_factory(context_pool.any_executor(), settings_.node_settings.sync_loop_log_interval_seconds), - make_bodies_stage_factory(chain_config_, *this), + make_stages_factory(settings_.node_settings, data_model_factory(), *this), db::RWAccess{chaindata_env_}, }, execution_service_{std::make_shared(execution_engine_, execution_context_)}, @@ -178,7 +213,7 @@ NodeImpl::NodeImpl( snapshot_sync_{ settings.snapshot_settings, chain_config_.chain_id, - chaindata_env_, // NOLINT(cppcoreguidelines-slicing) + data_store(), settings_.node_settings.data_directory->temp().path(), execution_engine_.stage_scheduler(), }, @@ -191,7 +226,7 @@ NodeImpl::NodeImpl( make_sentry_eth_status_data_provider(db::ROAccess{chaindata_env_}, chain_config_))}, chain_sync_{ context_pool.any_executor(), - chaindata_env_, // NOLINT(cppcoreguidelines-slicing) + data_store(), execution_direct_client_, std::get<0>(sentry_), chain_config_, diff --git a/silkworm/node/stagedsync/execution_engine.cpp b/silkworm/node/stagedsync/execution_engine.cpp index db7e2981b4..6bef0d3fe7 100644 --- a/silkworm/node/stagedsync/execution_engine.cpp +++ b/silkworm/node/stagedsync/execution_engine.cpp @@ -32,8 +32,9 @@ using execution::api::VerificationResult; ExecutionEngine::ExecutionEngine( std::optional executor, NodeSettings& ns, + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory, + StageContainerFactory stages_factory, db::RWAccess dba) : context_pool_{executor ? std::unique_ptr>{} : std::make_unique>(1)}, executor_{executor ? std::move(*executor) : context_pool_->any_executor()}, @@ -41,8 +42,9 @@ ExecutionEngine::ExecutionEngine( main_chain_{ executor_, ns, + std::move(data_model_factory), std::move(log_timer_factory), - std::move(bodies_stage_factory), + std::move(stages_factory), std::move(dba), }, block_cache_{kDefaultCacheSize} {} @@ -77,7 +79,7 @@ BlockId ExecutionEngine::last_safe_block() const { } BlockNum ExecutionEngine::highest_frozen_block_number() const { - return db::DataModel::highest_frozen_block_number(); + return main_chain_.highest_frozen_block_number(); } void ExecutionEngine::insert_blocks(const std::vector>& blocks) { diff --git a/silkworm/node/stagedsync/execution_engine.hpp b/silkworm/node/stagedsync/execution_engine.hpp index dd47c0383d..d88c310bbc 100644 --- a/silkworm/node/stagedsync/execution_engine.hpp +++ b/silkworm/node/stagedsync/execution_engine.hpp @@ -30,6 +30,7 @@ #include #include +#include #include #include #include @@ -38,7 +39,6 @@ #include "forks/extending_fork.hpp" #include "forks/main_chain.hpp" -#include "stages/stage_bodies_factory.hpp" #include "timer_factory.hpp" namespace silkworm::stagedsync { @@ -60,8 +60,9 @@ class ExecutionEngine : public execution::api::ExecutionEngine, public Stoppable ExecutionEngine( std::optional executor, NodeSettings& ns, + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory, + StageContainerFactory stages_factory, db::RWAccess dba); ~ExecutionEngine() override = default; diff --git a/silkworm/node/stagedsync/execution_engine_test.cpp b/silkworm/node/stagedsync/execution_engine_test.cpp index 3049fbfdf9..f074f8dde9 100644 --- a/silkworm/node/stagedsync/execution_engine_test.cpp +++ b/silkworm/node/stagedsync/execution_engine_test.cpp @@ -27,13 +27,14 @@ #include #include #include +#include #include #include #include #include #include #include -#include +#include #include namespace silkworm { @@ -45,6 +46,7 @@ using execution::api::InvalidChain; using execution::api::ValidationError; using execution::api::ValidChain; +using silkworm::stagedsync::test_util::make_stages_factory; using silkworm::test_util::generate_sample_child_blocks; using silkworm::test_util::SetLogVerbosityGuard; using silkworm::test_util::TaskRunner; @@ -56,18 +58,16 @@ class ExecutionEngineForTest : public stagedsync::ExecutionEngine { using stagedsync::ExecutionEngine::main_chain_; }; -static BodiesStageFactory make_bodies_stage_factory(const ChainConfig& chain_config) { - return [chain_config](SyncContext* sync_context) { - return std::make_unique(sync_context, chain_config, [] { return 0; }); - }; -}; - TEST_CASE("ExecutionEngine Integration Test", "[node][execution][execution_engine]") { SetLogVerbosityGuard log_guard(log::Level::kNone); TaskRunner runner; Environment::set_stop_before_stage(stages::kSendersKey); // only headers, block hashes and bodies auto db_context = db::test_util::TestDatabaseContext(); + + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + db::DataModelFactory data_model_factory = [&](ROTxn& tx) { return DataModel{tx, repository}; }; + auto node_settings = NodeSettings{ .data_directory = std::make_unique(db_context.mdbx_env().get_path(), false), .chaindata_env_config = db_context.get_env_config(), @@ -81,8 +81,9 @@ TEST_CASE("ExecutionEngine Integration Test", "[node][execution][execution_engin ExecutionEngineForTest exec_engine{ runner.executor(), node_settings, + data_model_factory, /* log_timer_factory = */ std::nullopt, - make_bodies_stage_factory(*node_settings.chain_config), + make_stages_factory(node_settings, data_model_factory), db_access, }; exec_engine.open(); @@ -766,7 +767,7 @@ TEST_CASE("ExecutionEngine Integration Test", "[node][execution][execution_engin tx2.abort(); } - // TODO: temoporarily disabled, to be fixed (JG) + // TODO: temporarily disabled, to be fixed (JG) // SECTION("updates storage") { // static constexpr evmc::address kSender{0xb685342b8c54347aad148e1f22eff3eb3eb29391_address}; // auto block1 = generate_sample_child_blocks(current_head); @@ -822,16 +823,20 @@ TEST_CASE("ExecutionEngine") { context.add_genesis_data(); context.commit_txn(); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + db::DataModelFactory data_model_factory = [&](ROTxn& tx) { return DataModel{tx, repository}; }; + Environment::set_stop_before_stage(stages::kSendersKey); // only headers, block hashes and bodies NodeSettings node_settings = node::test_util::make_node_settings_from_temp_chain_data(context); - RWAccess db_access{context.env()}; + ExecutionEngineForTest exec_engine{ runner.executor(), node_settings, + data_model_factory, /* log_timer_factory = */ std::nullopt, - make_bodies_stage_factory(*node_settings.chain_config), - db_access, + make_stages_factory(node_settings, data_model_factory), + RWAccess{context.env()}, }; exec_engine.open(); diff --git a/silkworm/node/stagedsync/execution_pipeline.cpp b/silkworm/node/stagedsync/execution_pipeline.cpp index 9c0cd18bcc..81aabc7323 100644 --- a/silkworm/node/stagedsync/execution_pipeline.cpp +++ b/silkworm/node/stagedsync/execution_pipeline.cpp @@ -16,26 +16,11 @@ #include "execution_pipeline.hpp" -#include - #include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include namespace silkworm::stagedsync { @@ -48,13 +33,13 @@ static const std::chrono::milliseconds kStageDurationThresholdForLog{0}; #endif ExecutionPipeline::ExecutionPipeline( - silkworm::NodeSettings* node_settings, + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory) - : node_settings_{node_settings}, + const StageContainerFactory& stages_factory) + : data_model_factory_{std::move(data_model_factory)}, log_timer_factory_{std::move(log_timer_factory)}, - bodies_stage_factory_{std::move(bodies_stage_factory)}, - sync_context_{std::make_unique()} { + sync_context_{std::make_unique()}, + stages_{stages_factory(*sync_context_)} { load_stages(); } @@ -91,31 +76,6 @@ std::optional ExecutionPipeline::bad_block() { */ void ExecutionPipeline::load_stages() { - stages_.emplace(kHeadersKey, - std::make_unique(sync_context_.get())); - stages_.emplace(kBlockBodiesKey, bodies_stage_factory_(sync_context_.get())); - stages_.emplace(kBlockHashesKey, - std::make_unique(sync_context_.get(), node_settings_->etl())); - stages_.emplace(kSendersKey, - std::make_unique(sync_context_.get(), *node_settings_->chain_config, node_settings_->batch_size, node_settings_->etl(), node_settings_->prune_mode.senders())); - stages_.emplace(kExecutionKey, - std::make_unique(sync_context_.get(), *node_settings_->chain_config, node_settings_->batch_size, node_settings_->prune_mode)); - stages_.emplace(kHashStateKey, - std::make_unique(sync_context_.get(), node_settings_->etl())); - stages_.emplace(kIntermediateHashesKey, - std::make_unique(sync_context_.get(), node_settings_->etl())); - stages_.emplace(kHistoryIndexKey, - std::make_unique(sync_context_.get(), node_settings_->batch_size, node_settings_->etl(), node_settings_->prune_mode.history())); - stages_.emplace(kLogIndexKey, - std::make_unique(sync_context_.get(), node_settings_->batch_size, node_settings_->etl(), node_settings_->prune_mode.history())); - stages_.emplace(kCallTracesKey, - std::make_unique(sync_context_.get(), node_settings_->batch_size, node_settings_->etl(), node_settings_->prune_mode.call_traces())); - stages_.emplace(kTxLookupKey, - std::make_unique(sync_context_.get(), node_settings_->etl(), node_settings_->prune_mode.tx_index())); - stages_.emplace(kTriggersStageKey, - std::make_unique(sync_context_.get())); - stages_.emplace(kFinishKey, - std::make_unique(sync_context_.get(), node_settings_->build_info.build_description)); current_stage_ = stages_.begin(); stages_forward_order_.insert(stages_forward_order_.begin(), @@ -236,8 +196,9 @@ Stage::Result ExecutionPipeline::forward(db::RWTxn& cycle_txn, BlockNum target_h } } - head_header_hash_ = db::read_head_header_hash(cycle_txn).value_or(Hash{}); - const auto head_header = db::DataModel(cycle_txn).read_header(head_header_hash_); + db::DataModel data_model = data_model_factory_(cycle_txn); + const auto [head_header, head_header_hash] = data_model.read_head_header_and_hash(); + head_header_hash_ = head_header_hash.value_or(Hash{}); ensure(head_header.has_value(), [&]() { return "Sync pipeline, missing head header hash " + to_hex(head_header_hash_); }); head_header_number_ = head_header->number; if (head_header_number_ != target_height) { @@ -300,8 +261,9 @@ Stage::Result ExecutionPipeline::unwind(db::RWTxn& cycle_txn, BlockNum unwind_po } } - head_header_hash_ = db::read_head_header_hash(cycle_txn).value_or(Hash{}); - const auto head_header = db::DataModel(cycle_txn).read_header(head_header_hash_); + db::DataModel data_model = data_model_factory_(cycle_txn); + const auto [head_header, head_header_hash] = data_model.read_head_header_and_hash(); + head_header_hash_ = head_header_hash.value_or(Hash{}); ensure(head_header.has_value(), [&]() { return "Sync pipeline, missing head header hash " + to_hex(head_header_hash_); }); head_header_number_ = head_header->number; if (head_header_number_ != unwind_point) { @@ -359,8 +321,9 @@ Stage::Result ExecutionPipeline::prune(db::RWTxn& cycle_txn) { } } - head_header_hash_ = db::read_head_header_hash(cycle_txn).value_or(Hash{}); - const auto head_header = db::DataModel(cycle_txn).read_header(head_header_hash_); + db::DataModel data_model = data_model_factory_(cycle_txn); + const auto [head_header, head_header_hash] = data_model.read_head_header_and_hash(); + head_header_hash_ = head_header_hash.value_or(Hash{}); ensure(head_header.has_value(), [&]() { return "Sync pipeline, missing head header hash " + to_hex(head_header_hash_); }); head_header_number_ = head_header->number; diff --git a/silkworm/node/stagedsync/execution_pipeline.hpp b/silkworm/node/stagedsync/execution_pipeline.hpp index 7b2dfb5a6c..e086b26b44 100644 --- a/silkworm/node/stagedsync/execution_pipeline.hpp +++ b/silkworm/node/stagedsync/execution_pipeline.hpp @@ -17,6 +17,7 @@ #pragma once #include +#include #include #include #include @@ -25,22 +26,25 @@ #include #include +#include #include #include #include #include -#include "stages/stage_bodies_factory.hpp" #include "timer_factory.hpp" namespace silkworm::stagedsync { +using StageContainer = std::map>; +using StageContainerFactory = std::function; + class ExecutionPipeline : public Stoppable { public: - explicit ExecutionPipeline( - NodeSettings* node_settings, + ExecutionPipeline( + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory); + const StageContainerFactory& stages_factory); ~ExecutionPipeline() override = default; Stage::Result forward(db::RWTxn&, BlockNum target_height); @@ -57,12 +61,10 @@ class ExecutionPipeline : public Stoppable { StageScheduler& stage_scheduler() const; private: - silkworm::NodeSettings* node_settings_; + db::DataModelFactory data_model_factory_; std::optional log_timer_factory_; - BodiesStageFactory bodies_stage_factory_; std::unique_ptr sync_context_; // context shared across stages - using StageContainer = std::map>; StageContainer stages_; StageContainer::iterator current_stage_; diff --git a/silkworm/node/stagedsync/forks/canonical_chain.cpp b/silkworm/node/stagedsync/forks/canonical_chain.cpp index bf584f12c7..2e0a25f481 100644 --- a/silkworm/node/stagedsync/forks/canonical_chain.cpp +++ b/silkworm/node/stagedsync/forks/canonical_chain.cpp @@ -24,16 +24,19 @@ namespace silkworm::stagedsync { -CanonicalChain::CanonicalChain(db::RWTxn& tx, size_t cache_size) +CanonicalChain::CanonicalChain( + db::RWTxn& tx, + db::DataModelFactory data_model_factory, + size_t cache_size) : tx_{tx}, - data_model_{tx_}, // todo: put an header cache into the data_model_ and share the data_model_ with the owner + data_model_factory_{std::move(data_model_factory)}, canonical_hash_cache_{std::make_unique>(cache_size)} { open(); } CanonicalChain::CanonicalChain(const CanonicalChain& copy, db::RWTxn& new_tx) : tx_{new_tx}, - data_model_{tx_}, + data_model_factory_{copy.data_model_factory_}, initial_head_{copy.initial_head_}, current_head_{copy.current_head_}, canonical_hash_cache_{std::make_unique>(copy.canonical_hash_cache_->size())} { @@ -42,7 +45,7 @@ CanonicalChain::CanonicalChain(const CanonicalChain& copy, db::RWTxn& new_tx) CanonicalChain::CanonicalChain(CanonicalChain&& orig) noexcept : tx_{orig.tx_}, - data_model_{tx_}, + data_model_factory_{std::move(orig.data_model_factory_)}, initial_head_{orig.initial_head_}, current_head_{orig.current_head_}, canonical_hash_cache_{std::move(orig.canonical_hash_cache_)} { @@ -67,7 +70,7 @@ BlockId CanonicalChain::current_head() const { return current_head_; } bool CanonicalChain::cache_enabled() const { return canonical_hash_cache_->max_size() > 0; } BlockId CanonicalChain::find_forking_point(Hash header_hash) const { - std::optional header = data_model_.read_header(header_hash); + std::optional header = data_model().read_header(header_hash); if (!header) throw std::logic_error("find_forking_point precondition violation, header not found"); return find_forking_point(*header, header_hash); @@ -90,7 +93,7 @@ BlockId CanonicalChain::find_forking_point(const BlockHeader& header, Hash heade // Going further back else { - auto parent = data_model_.read_header(height - 1, parent_hash); + auto parent = data_model().read_header(height - 1, parent_hash); ensure_invariant(parent.has_value(), [&]() { return "canonical chain could not find parent with hash " + to_hex(parent_hash) + " and height " + std::to_string(height - 1); }); @@ -99,8 +102,8 @@ BlockId CanonicalChain::find_forking_point(const BlockHeader& header, Hash heade auto ancestor_height = height - 2; std::optional canon_hash; - while ((canon_hash = get_hash(ancestor_height)) && canon_hash != ancestor_hash) { - auto ancestor = data_model_.read_header(ancestor_height, ancestor_hash); + while ((canon_hash = get_hash(ancestor_height)).has_value() && (canon_hash != ancestor_hash)) { + auto ancestor = data_model().read_header(ancestor_height, ancestor_hash); ancestor_hash = ancestor->parent_hash; --ancestor_height; } @@ -138,7 +141,7 @@ void CanonicalChain::update_up_to(BlockNum height, Hash hash) { // hash can be db::write_canonical_hash(tx_, ancestor_height, ancestor_hash); if (cache_enabled()) canonical_hash_cache_->put(ancestor_height, ancestor_hash); - auto ancestor = data_model_.read_header(ancestor_height, ancestor_hash); + auto ancestor = data_model().read_header(ancestor_height, ancestor_hash); ensure_invariant(ancestor.has_value(), [&]() { return "fix canonical chain failed at ancestor= " + std::to_string(ancestor_height) + " hash=" + ancestor_hash.to_hex(); }); @@ -181,7 +184,7 @@ std::optional CanonicalChain::get_hash(BlockNum height) const { } bool CanonicalChain::has(Hash block_hash) const { - auto header = data_model_.read_header(block_hash); + auto header = data_model().read_header(block_hash); if (!header) return false; auto canonical_hash_at_same_height = get_hash(header->number); return canonical_hash_at_same_height == block_hash; diff --git a/silkworm/node/stagedsync/forks/canonical_chain.hpp b/silkworm/node/stagedsync/forks/canonical_chain.hpp index f705d7ca92..eb1be6113d 100644 --- a/silkworm/node/stagedsync/forks/canonical_chain.hpp +++ b/silkworm/node/stagedsync/forks/canonical_chain.hpp @@ -35,8 +35,11 @@ class CanonicalChain { public: static constexpr size_t kNoCache = 0; - explicit CanonicalChain(db::RWTxn&, size_t cache_size = kDefaultCacheSize); - CanonicalChain(CanonicalChain&) = delete; // tx is not copiable + explicit CanonicalChain( + db::RWTxn& tx, + db::DataModelFactory data_model_factory, + size_t cache_size = kDefaultCacheSize); + CanonicalChain(CanonicalChain&) = delete; // tx is not copyable CanonicalChain(const CanonicalChain&, db::RWTxn&); // we can copy a CanonicalChain giving a new tx CanonicalChain(CanonicalChain&&) noexcept; @@ -46,7 +49,7 @@ class CanonicalChain { BlockId find_forking_point(const BlockHeader& header, Hash header_hash) const; void advance(BlockNum height, Hash header_hash); - void update_up_to(BlockNum height, Hash header_hash); + void update_up_to(BlockNum height, Hash hash); void delete_down_to(BlockNum unwind_point); void set_current_head(BlockId); @@ -57,8 +60,10 @@ class CanonicalChain { bool has(Hash block_hash) const; private: + db::DataModel data_model() const { return data_model_factory_(tx_); } + db::RWTxn& tx_; - db::DataModel data_model_; + db::DataModelFactory data_model_factory_; BlockId initial_head_{}; BlockId current_head_{}; diff --git a/silkworm/node/stagedsync/forks/extending_fork.cpp b/silkworm/node/stagedsync/forks/extending_fork.cpp index 138671cf1d..bff0762bb9 100644 --- a/silkworm/node/stagedsync/forks/extending_fork.cpp +++ b/silkworm/node/stagedsync/forks/extending_fork.cpp @@ -70,9 +70,10 @@ void ExtendingFork::start_with(BlockId new_head, std::list( forking_point_, db::ROTxnManaged(main_chain_.tx().db()), + main_chain_.data_model_factory(), main_chain_.log_timer_factory(), - main_chain_.bodies_stage_factory(), - main_chain_.node_settings()); + main_chain_.stages_factory(), + main_chain_.node_settings().data_directory->forks().path()); fork_->extend_with(blocks_); ensure(fork_->current_head() == new_head, "fork head mismatch"); } catch (...) { diff --git a/silkworm/node/stagedsync/forks/fork.cpp b/silkworm/node/stagedsync/forks/fork.cpp index b2fbfe52d0..411db1e885 100644 --- a/silkworm/node/stagedsync/forks/fork.cpp +++ b/silkworm/node/stagedsync/forks/fork.cpp @@ -48,20 +48,21 @@ static db::MemoryOverlay create_memory_db(const std::filesystem::path& base_path Fork::Fork( BlockId forking_point, - db::ROTxnManaged&& main_chain_tx, + db::ROTxnManaged main_tx, + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory, - NodeSettings& ns) - : main_tx_{std::move(main_chain_tx)}, - memory_db_{create_memory_db(ns.data_directory->forks().path(), main_tx_)}, + const StageContainerFactory& stages_factory, + const std::filesystem::path& forks_dir_path) + : main_tx_{std::move(main_tx)}, + memory_db_{create_memory_db(forks_dir_path, main_tx_)}, memory_tx_{memory_db_}, - data_model_{memory_tx_}, + data_model_factory_{std::move(data_model_factory)}, pipeline_{ - &ns, + data_model_factory_, std::move(log_timer_factory), - std::move(bodies_stage_factory), + stages_factory, }, - canonical_chain_(memory_tx_), + canonical_chain_{memory_tx_, data_model_factory_}, current_head_{forking_point} // actual head { // go down if needed @@ -136,7 +137,7 @@ void Fork::insert_body(const Block& block, const Hash& block_hash) { // avoid calculation of block.header.hash() because is computationally expensive BlockNum block_num = block.header.number; - if (!data_model_.has_body(block_num, block_hash)) { + if (!data_model().has_body(block_num, block_hash)) { db::write_body(memory_tx_, block, block_hash, block_num); } } @@ -302,7 +303,7 @@ std::set Fork::collect_bad_headers(InvalidChain& invalid_chain) { } std::optional Fork::get_header(Hash header_hash) const { - std::optional header = data_model_.read_header(header_hash); + std::optional header = data_model().read_header(header_hash); return header; } diff --git a/silkworm/node/stagedsync/forks/fork.hpp b/silkworm/node/stagedsync/forks/fork.hpp index 25b4f825b5..92e41aef46 100644 --- a/silkworm/node/stagedsync/forks/fork.hpp +++ b/silkworm/node/stagedsync/forks/fork.hpp @@ -24,11 +24,11 @@ #include #include +#include #include #include #include -#include "../stages/stage_bodies_factory.hpp" #include "../timer_factory.hpp" #include "canonical_chain.hpp" @@ -40,10 +40,11 @@ class Fork { public: explicit Fork( BlockId forking_point, - db::ROTxnManaged&& main_chain_tx, + db::ROTxnManaged main_tx, + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory, - NodeSettings&); + const StageContainerFactory& stages_factory, + const std::filesystem::path& forks_dir_path); Fork(const Fork&) = delete; void close(); @@ -77,6 +78,7 @@ class Fork { std::optional get_header(Hash) const; protected: + db::DataModel data_model() const { return data_model_factory_(memory_tx_); } Hash insert_header(const BlockHeader&); void insert_body(const Block&, const Hash& block_hash); @@ -85,7 +87,7 @@ class Fork { db::ROTxnManaged main_tx_; db::MemoryOverlay memory_db_; mutable db::MemoryMutation memory_tx_; - db::DataModel data_model_; + db::DataModelFactory data_model_factory_; ExecutionPipeline pipeline_; CanonicalChain canonical_chain_; diff --git a/silkworm/node/stagedsync/forks/fork_test.cpp b/silkworm/node/stagedsync/forks/fork_test.cpp index f860bfcba0..70c239507b 100644 --- a/silkworm/node/stagedsync/forks/fork_test.cpp +++ b/silkworm/node/stagedsync/forks/fork_test.cpp @@ -24,10 +24,11 @@ #include #include #include +#include #include #include #include -#include +#include #include #include "main_chain.hpp" @@ -40,6 +41,7 @@ using namespace stagedsync; using namespace intx; // just for literals using execution::api::ValidChain; +using silkworm::stagedsync::test_util::make_stages_factory; class ForkForTest : public Fork { public: @@ -59,6 +61,9 @@ TEST_CASE("Fork") { context.add_genesis_data(); context.commit_txn(); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + db::DataModelFactory data_model_factory = [&](db::ROTxn& tx) { return db::DataModel{tx, repository}; }; + asio::io_context io; asio::executor_work_guard work{io.get_executor()}; @@ -67,15 +72,12 @@ TEST_CASE("Fork") { NodeSettings node_settings = node::test_util::make_node_settings_from_temp_chain_data(context); db::RWAccess db_access{context.env()}; - BodiesStageFactory bodies_stage_factory = [&](SyncContext* sync_context) { - return std::make_unique(sync_context, *node_settings.chain_config, [] { return 0; }); - }; - MainChain main_chain{ io.get_executor(), node_settings, + data_model_factory, /* log_timer_factory = */ std::nullopt, - std::move(bodies_stage_factory), + make_stages_factory(node_settings, data_model_factory), db_access, }; @@ -130,9 +132,10 @@ TEST_CASE("Fork") { ForkForTest fork{ forking_point, db::ROTxnManaged(main_chain.tx().db()), // this need to be on a different thread than main_chain + data_model_factory, /* log_timer_factory = */ std::nullopt, - main_chain.bodies_stage_factory(), - node_settings, + main_chain.stages_factory(), + node_settings.data_directory->forks().path(), }; CHECK(db::stages::read_stage_progress(fork.memory_tx_, db::stages::kHeadersKey) == 3); diff --git a/silkworm/node/stagedsync/forks/main_chain.cpp b/silkworm/node/stagedsync/forks/main_chain.cpp index 4b36af7386..869af75633 100644 --- a/silkworm/node/stagedsync/forks/main_chain.cpp +++ b/silkworm/node/stagedsync/forks/main_chain.cpp @@ -75,18 +75,19 @@ using execution::api::VerificationResult; MainChain::MainChain( boost::asio::any_io_executor executor, NodeSettings& ns, + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory, + StageContainerFactory stages_factory, db::RWAccess dba) : executor_{std::move(executor)}, node_settings_{ns}, + data_model_factory_{std::move(data_model_factory)}, log_timer_factory_{std::move(log_timer_factory)}, - bodies_stage_factory_{std::move(bodies_stage_factory)}, + stages_factory_{std::move(stages_factory)}, db_access_{std::move(dba)}, tx_{db_access_.start_rw_tx()}, - data_model_{tx_}, - pipeline_{&ns, log_timer_factory_, bodies_stage_factory_}, - interim_canonical_chain_(tx_) { + pipeline_{data_model_factory_, log_timer_factory_, stages_factory_}, + interim_canonical_chain_{tx_, data_model_factory_} { // We commit and close the one-and-only RW txn here because it must be reopened below in MainChain::open tx_.commit_and_stop(); } @@ -149,10 +150,6 @@ db::RWTxn& MainChain::tx() { return tx_; } -const BodiesStageFactory& MainChain::bodies_stage_factory() const { - return bodies_stage_factory_; -} - const std::optional& MainChain::log_timer_factory() const { return log_timer_factory_; } @@ -204,7 +201,7 @@ void MainChain::insert_body(const Block& block, const Hash& block_hash) { // avoid calculation of block.header.hash() because is computationally expensive BlockNum block_num = block.header.number; - if (data_model_.has_body(block_num, block_hash)) return; + if (data_model().has_body(block_num, block_hash)) return; db::write_body(tx_, block, block_hash, block_num); } @@ -438,7 +435,7 @@ std::optional MainChain::get_header(Hash header_hash) const { // if (cached) { // return *cached; // } - return data_model_.read_header(header_hash); + return data_model().read_header(header_hash); } std::optional MainChain::get_header(BlockNum header_height, Hash header_hash) const { @@ -447,7 +444,7 @@ std::optional MainChain::get_header(BlockNum header_height, Hash he // if (cached) { // return *cached; // } - std::optional header = data_model_.read_header(header_height, header_hash); + std::optional header = data_model().read_header(header_height, header_hash); return header; } @@ -472,21 +469,21 @@ std::optional MainChain::get_header_td(Hash header_hash) const std::optional MainChain::get_body(Hash header_hash) const { TransactionHandler tx_handler{tx_, db_access_, node_settings_.keep_db_txn_open}; BlockBody body; - bool found = data_model_.read_body(header_hash, body); + bool found = data_model().read_body(header_hash, body); if (!found) return {}; return body; } BlockNum MainChain::get_block_progress() const { TransactionHandler tx_handler{tx_, db_access_, node_settings_.keep_db_txn_open}; - return data_model_.highest_block_number(); + return data_model().highest_block_number(); } std::vector MainChain::get_last_headers(uint64_t limit) const { TransactionHandler tx_handler{tx_, db_access_, node_settings_.keep_db_txn_open}; std::vector headers; - data_model_.for_last_n_headers(limit, [&headers](BlockHeader&& header) { + data_model().for_last_n_headers(limit, [&headers](BlockHeader&& header) { headers.emplace_back(std::move(header)); }); @@ -495,7 +492,11 @@ std::vector MainChain::get_last_headers(uint64_t limit) const { std::optional MainChain::get_block_number(Hash header_hash) const { TransactionHandler tx_handler{tx_, db_access_, node_settings_.keep_db_txn_open}; - return data_model_.read_block_number(header_hash); + return data_model().read_block_number(header_hash); +} + +BlockNum MainChain::highest_frozen_block_number() const { + return data_model().highest_frozen_block_number(); } bool MainChain::is_ancestor(BlockId supposed_parent, BlockId block) const { diff --git a/silkworm/node/stagedsync/forks/main_chain.hpp b/silkworm/node/stagedsync/forks/main_chain.hpp index 90f7ec36c8..405d06228c 100644 --- a/silkworm/node/stagedsync/forks/main_chain.hpp +++ b/silkworm/node/stagedsync/forks/main_chain.hpp @@ -27,13 +27,13 @@ #include #include +#include #include #include #include #include #include -#include "../stages/stage_bodies_factory.hpp" #include "../timer_factory.hpp" #include "canonical_chain.hpp" @@ -47,8 +47,9 @@ class MainChain { explicit MainChain( boost::asio::any_io_executor executor, NodeSettings& ns, + db::DataModelFactory data_model_factory, std::optional log_timer_factory, - BodiesStageFactory bodies_stage_factory, + StageContainerFactory stages_factory, db::RWAccess dba); void open(); // needed to circumvent mdbx threading model limitations @@ -92,14 +93,17 @@ class MainChain { std::optional get_header_td(Hash) const; std::optional get_body(Hash) const; std::optional get_block_number(Hash) const; + BlockNum highest_frozen_block_number() const; NodeSettings& node_settings(); db::RWTxn& tx(); // only for testing purposes due to MDBX limitations + const db::DataModelFactory& data_model_factory() const { return data_model_factory_; } const std::optional& log_timer_factory() const; - const BodiesStageFactory& bodies_stage_factory() const; + const StageContainerFactory& stages_factory() const { return stages_factory_; } StageScheduler& stage_scheduler() const; protected: + db::DataModel data_model() const { return data_model_factory_(tx_); } Hash insert_header(const BlockHeader&); void insert_body(const Block&, const Hash& block_hash); void forward(BlockNum head_height, const Hash& head_hash); @@ -112,11 +116,11 @@ class MainChain { boost::asio::any_io_executor executor_; NodeSettings& node_settings_; + db::DataModelFactory data_model_factory_; std::optional log_timer_factory_; - BodiesStageFactory bodies_stage_factory_; + StageContainerFactory stages_factory_; mutable db::RWAccess db_access_; mutable db::RWTxnManaged tx_; - db::DataModel data_model_; bool is_first_sync_{true}; ExecutionPipeline pipeline_; diff --git a/silkworm/node/stagedsync/forks/main_chain_test.cpp b/silkworm/node/stagedsync/forks/main_chain_test.cpp index 74c1115cf5..d734992526 100644 --- a/silkworm/node/stagedsync/forks/main_chain_test.cpp +++ b/silkworm/node/stagedsync/forks/main_chain_test.cpp @@ -25,10 +25,11 @@ #include #include #include +#include #include #include #include -#include +#include #include namespace silkworm { @@ -42,6 +43,7 @@ using namespace intx; // just for literals using execution::api::InvalidChain; using execution::api::ValidationError; using execution::api::ValidChain; +using silkworm::stagedsync::test_util::make_stages_factory; class MainChainForTest : public stagedsync::MainChain { public: @@ -54,12 +56,6 @@ class MainChainForTest : public stagedsync::MainChain { using stagedsync::MainChain::tx_; }; -static BodiesStageFactory make_bodies_stage_factory(const ChainConfig& chain_config) { - return [chain_config](SyncContext* sync_context) { - return std::make_unique(sync_context, chain_config, [] { return 0; }); - }; -}; - TEST_CASE("MainChain transaction handling") { for (int i = 0; i < 2; ++i) { auto keep_db_txn_open = i == 1; @@ -74,6 +70,9 @@ TEST_CASE("MainChain transaction handling") { context.add_genesis_data(); context.commit_txn(); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + db::DataModelFactory data_model_factory = [&](db::ROTxn& tx) { return db::DataModel{tx, repository}; }; + Environment::set_stop_before_stage(stages::kSendersKey); // only headers, block hashes and bodies NodeSettings node_settings = node::test_util::make_node_settings_from_temp_chain_data(context); @@ -82,8 +81,9 @@ TEST_CASE("MainChain transaction handling") { MainChainForTest main_chain{ io.get_executor(), node_settings, + data_model_factory, /* log_timer_factory = */ std::nullopt, - make_bodies_stage_factory(*node_settings.chain_config), + make_stages_factory(node_settings, data_model_factory), db_access, }; main_chain.open(); @@ -171,6 +171,9 @@ TEST_CASE("MainChain") { context.add_genesis_data(); context.commit_txn(); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + db::DataModelFactory data_model_factory = [&](db::ROTxn& tx) { return db::DataModel{tx, repository}; }; + Environment::set_stop_before_stage(stages::kSendersKey); // only headers, block hashes and bodies NodeSettings node_settings = node::test_util::make_node_settings_from_temp_chain_data(context); @@ -178,8 +181,9 @@ TEST_CASE("MainChain") { MainChainForTest main_chain{ io.get_executor(), node_settings, + data_model_factory, /* log_timer_factory = */ std::nullopt, - make_bodies_stage_factory(*node_settings.chain_config), + make_stages_factory(node_settings, data_model_factory), db_access, }; main_chain.open(); @@ -469,8 +473,9 @@ TEST_CASE("MainChain") { MainChainForTest main_chain2{ io.get_executor(), node_settings, + main_chain.data_model_factory(), /* log_timer_factory = */ std::nullopt, - main_chain.bodies_stage_factory(), + main_chain.stages_factory(), db_access, }; main_chain2.open(); diff --git a/silkworm/node/stagedsync/stages/stage_bodies.cpp b/silkworm/node/stagedsync/stages/stage_bodies.cpp index 96198d0db1..aa6e4d58ec 100644 --- a/silkworm/node/stagedsync/stages/stage_bodies.cpp +++ b/silkworm/node/stagedsync/stages/stage_bodies.cpp @@ -27,12 +27,15 @@ namespace silkworm::stagedsync { -BodiesStage::BodyDataModel::BodyDataModel(db::RWTxn& tx, BlockNum bodies_stage_height, const ChainConfig& chain_config) - : tx_(tx), - data_model_(tx_), +BodiesStage::BodyDataModel::BodyDataModel( + db::RWTxn& tx, + db::DataModel data_model, + BlockNum bodies_stage_height, + const ChainConfig& chain_config) + : data_model_(data_model), chain_config_{chain_config}, rule_set_{protocol::rule_set_factory(chain_config)}, - chain_state_{tx}, + chain_state_{tx, std::make_unique(data_model)}, initial_height_{bodies_stage_height}, highest_height_{bodies_stage_height} { } @@ -97,9 +100,11 @@ bool BodiesStage::BodyDataModel::get_canonical_block(BlockNum height, Block& blo BodiesStage::BodiesStage( SyncContext* sync_context, const ChainConfig& chain_config, + db::DataModelFactory data_model_factory, std::function last_pre_validated_block) : Stage(sync_context, db::stages::kBlockBodiesKey), chain_config_(chain_config), + data_model_factory_(std::move(data_model_factory)), last_pre_validated_block_(std::move(last_pre_validated_block)) {} Stage::Result BodiesStage::forward(db::RWTxn& tx) { @@ -132,8 +137,12 @@ Stage::Result BodiesStage::forward(db::RWTxn& tx) { "to", std::to_string(target_height), "span", std::to_string(target_height - current_height_)}); } - - BodyDataModel body_persistence(tx, current_height_, chain_config_); + BodyDataModel body_persistence{ + tx, + data_model_factory_(tx), + current_height_, + chain_config_, + }; body_persistence.set_preverified_height(last_pre_validated_block_()); get_log_progress(); // this is a trick to set log progress initial value, please improve diff --git a/silkworm/node/stagedsync/stages/stage_bodies.hpp b/silkworm/node/stagedsync/stages/stage_bodies.hpp index e5484e7040..1dbfde4dca 100644 --- a/silkworm/node/stagedsync/stages/stage_bodies.hpp +++ b/silkworm/node/stagedsync/stages/stage_bodies.hpp @@ -34,6 +34,7 @@ class BodiesStage : public Stage { BodiesStage( SyncContext* sync_context, const ChainConfig& chain_config, + db::DataModelFactory data_model_factory, std::function last_pre_validated_block); BodiesStage(const BodiesStage&) = delete; // not copyable BodiesStage(BodiesStage&&) = delete; // nor movable @@ -45,7 +46,9 @@ class BodiesStage : public Stage { private: std::vector get_log_progress() override; // thread safe + const ChainConfig& chain_config_; + db::DataModelFactory data_model_factory_; std::function last_pre_validated_block_; std::atomic current_height_{0}; @@ -53,7 +56,11 @@ class BodiesStage : public Stage { // BodyDataModel has the responsibility to update bodies related tables class BodyDataModel { public: - explicit BodyDataModel(db::RWTxn&, BlockNum bodies_stage_height, const ChainConfig&); + explicit BodyDataModel( + db::RWTxn& tx, + db::DataModel data_model, + BlockNum bodies_stage_height, + const ChainConfig& chain_config); ~BodyDataModel() = default; void update_tables(const Block&); // make a pre-verification of the body and update body related tables @@ -74,7 +81,6 @@ class BodiesStage : public Stage { void set_preverified_height(BlockNum height); private: - db::RWTxn& tx_; db::DataModel data_model_; const ChainConfig& chain_config_; protocol::RuleSetPtr rule_set_; diff --git a/silkworm/node/stagedsync/stages/stage_bodies_test.cpp b/silkworm/node/stagedsync/stages/stage_bodies_test.cpp index d5e11ee6b0..398e8f42af 100644 --- a/silkworm/node/stagedsync/stages/stage_bodies_test.cpp +++ b/silkworm/node/stagedsync/stages/stage_bodies_test.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include namespace silkworm { @@ -39,6 +39,8 @@ TEST_CASE("BodiesStage - data model") { context.add_genesis_data(); context.commit_txn(); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + auto& chain_config = context.chain_config(); /* status: @@ -48,6 +50,7 @@ TEST_CASE("BodiesStage - data model") { */ SECTION("one invalid body after the genesis") { db::RWTxnManaged tx(context.env()); + db::DataModel data_model{tx, repository}; auto header0_hash = db::read_canonical_header_hash(tx, 0); REQUIRE(header0_hash.has_value()); @@ -63,7 +66,12 @@ TEST_CASE("BodiesStage - data model") { block1.ommers.push_back(BlockHeader{}); // generate error InvalidOmmerHeader BlockNum bodies_stage_height = 0; - BodyDataModelForTest bm(tx, bodies_stage_height, chain_config); + BodyDataModelForTest bm{ + tx, + data_model, + bodies_stage_height, + chain_config, + }; REQUIRE(bm.initial_height() == 0); REQUIRE(bm.highest_height() == 0); @@ -83,6 +91,7 @@ TEST_CASE("BodiesStage - data model") { SECTION("one valid body after the genesis") { db::RWTxnManaged tx(context.env()); + db::DataModel data_model{tx, repository}; auto header0_hash = db::read_canonical_header_hash(tx, 0); REQUIRE(header0_hash.has_value()); @@ -111,7 +120,12 @@ TEST_CASE("BodiesStage - data model") { REQUIRE(decoding_result); BlockNum bodies_stage_height = 0; - BodyDataModelForTest bm(tx, bodies_stage_height, chain_config); + BodyDataModelForTest bm{ + tx, + data_model, + bodies_stage_height, + chain_config, + }; // check internal status REQUIRE(bm.initial_height() == 0); diff --git a/silkworm/node/stagedsync/stages/stage_execution.cpp b/silkworm/node/stagedsync/stages/stage_execution.cpp index 10138fbb84..b3533cb6ca 100644 --- a/silkworm/node/stagedsync/stages/stage_execution.cpp +++ b/silkworm/node/stagedsync/stages/stage_execution.cpp @@ -162,7 +162,7 @@ void Execution::prefetch_blocks(RWTxn& txn, const BlockNum from, const BlockNum const size_t count{std::min(static_cast(to - from + 1), kMaxPrefetchedBlocks)}; size_t num_read{0}; - DataModel data_model{txn}; + DataModel data_model = data_model_factory_(txn); auto canonicals = txn.ro_cursor(table::kCanonicalHashes); Bytes starting_key{block_key(from)}; if (canonicals->seek(to_slice(starting_key))) { @@ -209,7 +209,10 @@ Stage::Result Execution::execute_batch(RWTxn& txn, BlockNum max_block_num, Analy auto log_time{std::chrono::steady_clock::now()}; try { - Buffer buffer{txn}; + Buffer buffer{ + txn, + std::make_unique(data_model_factory_(txn)), + }; buffer.set_prune_history_threshold(prune_history_threshold); buffer.set_memory_limit(batch_size_); diff --git a/silkworm/node/stagedsync/stages/stage_execution.hpp b/silkworm/node/stagedsync/stages/stage_execution.hpp index 53b11f745d..88d9e533f0 100644 --- a/silkworm/node/stagedsync/stages/stage_execution.hpp +++ b/silkworm/node/stagedsync/stages/stage_execution.hpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -30,10 +31,12 @@ class Execution final : public Stage { public: Execution( SyncContext* sync_context, + db::DataModelFactory data_model_factory, const ChainConfig& chain_config, size_t batch_size, db::PruneMode prune_mode) : Stage(sync_context, db::stages::kExecutionKey), + data_model_factory_(std::move(data_model_factory)), chain_config_(chain_config), batch_size_(batch_size), prune_mode_(prune_mode), @@ -49,6 +52,7 @@ class Execution final : public Stage { private: static constexpr size_t kMaxPrefetchedBlocks{10240}; + db::DataModelFactory data_model_factory_; const ChainConfig& chain_config_; size_t batch_size_; db::PruneMode prune_mode_; diff --git a/silkworm/node/stagedsync/stages/stage_headers.cpp b/silkworm/node/stagedsync/stages/stage_headers.cpp index 23887e6964..1dca2999a6 100644 --- a/silkworm/node/stagedsync/stages/stage_headers.cpp +++ b/silkworm/node/stagedsync/stages/stage_headers.cpp @@ -31,8 +31,13 @@ namespace silkworm::stagedsync { using namespace silkworm::db; -HeadersStage::HeaderDataModel::HeaderDataModel(RWTxn& tx, BlockNum headers_height) - : tx_(tx), data_model_(tx), previous_height_(headers_height) { +HeadersStage::HeaderDataModel::HeaderDataModel( + RWTxn& tx, + DataModel data_model, + BlockNum headers_height) + : tx_(tx), + data_model_(data_model), + previous_height_(headers_height) { auto headers_hash = read_canonical_header_hash(tx, headers_height); ensure(headers_hash.has_value(), [&]() { return "Headers stage, inconsistent canonical table: not found hash at height " + std::to_string(headers_height); }); @@ -86,8 +91,11 @@ std::optional HeadersStage::HeaderDataModel::get_canonical_header(B } // HeadersStage -HeadersStage::HeadersStage(SyncContext* sync_context) - : Stage(sync_context, stages::kHeadersKey) { +HeadersStage::HeadersStage( + SyncContext* sync_context, + DataModelFactory data_model_factory) + : Stage{sync_context, stages::kHeadersKey}, + data_model_factory_{std::move(data_model_factory)} { // User can specify to stop downloading process at some block const auto stop_at_block = Environment::get_stop_at_block(); if (stop_at_block.has_value()) { @@ -128,7 +136,11 @@ Stage::Result HeadersStage::forward(RWTxn& tx) { "span", std::to_string(segment_width)}); } - HeaderDataModel header_persistence(tx, current_height_); + HeaderDataModel header_persistence{ + tx, + data_model_factory_(tx), + current_height_, + }; get_log_progress(); // this is a trick to set log progress initial value, please improve RepeatedMeasure height_progress(current_height_); diff --git a/silkworm/node/stagedsync/stages/stage_headers.hpp b/silkworm/node/stagedsync/stages/stage_headers.hpp index cb6d439581..f06ce972e1 100644 --- a/silkworm/node/stagedsync/stages/stage_headers.hpp +++ b/silkworm/node/stagedsync/stages/stage_headers.hpp @@ -61,7 +61,9 @@ namespace silkworm::stagedsync { */ class HeadersStage : public Stage { public: - explicit HeadersStage(SyncContext*); + HeadersStage( + SyncContext* sync_context, + db::DataModelFactory data_model_factory); HeadersStage(const HeadersStage&) = delete; // not copyable HeadersStage(HeadersStage&&) = delete; // nor movable @@ -71,14 +73,18 @@ class HeadersStage : public Stage { protected: std::vector get_log_progress() override; // thread safe - std::atomic current_height_{0}; + db::DataModelFactory data_model_factory_; + std::atomic current_height_{0}; std::optional forced_target_block_; // HeaderDataModel has the responsibility to update headers related tables class HeaderDataModel { public: - explicit HeaderDataModel(db::RWTxn& tx, BlockNum headers_height); + HeaderDataModel( + db::RWTxn& tx, + db::DataModel data_model, + BlockNum headers_height); void update_tables(const BlockHeader&); // update header related tables diff --git a/silkworm/node/stagedsync/stages/stage_headers_test.cpp b/silkworm/node/stagedsync/stages/stage_headers_test.cpp index 646f2b139e..ebd7137231 100644 --- a/silkworm/node/stagedsync/stages/stage_headers_test.cpp +++ b/silkworm/node/stagedsync/stages/stage_headers_test.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include namespace silkworm { @@ -34,10 +35,12 @@ class HeadersStageForTest : public stagedsync::HeadersStage { using HeaderDataModelForTest = HeadersStageForTest::HeaderDataModel; TEST_CASE("HeadersStage - data model") { - db::test_util::TempChainData context; + test_util::TempChainData context; context.add_genesis_data(); context.commit_txn(); + snapshots::SnapshotRepository repository = test_util::make_repository(); + /* status: * h0 * input: @@ -45,6 +48,7 @@ TEST_CASE("HeadersStage - data model") { */ SECTION("one header after the genesis") { RWTxnManaged tx(context.env()); + DataModel data_model{tx, repository}; auto header0_hash = read_canonical_header_hash(tx, 0); REQUIRE(header0_hash.has_value()); @@ -53,7 +57,7 @@ TEST_CASE("HeadersStage - data model") { REQUIRE(header0.has_value()); BlockNum headers_stage_height = 0; - HeaderDataModelForTest hm(tx, headers_stage_height); + HeaderDataModelForTest hm{tx, data_model, headers_stage_height}; REQUIRE(hm.highest_height() == 0); REQUIRE(hm.highest_hash() == header0_hash); @@ -88,6 +92,7 @@ TEST_CASE("HeadersStage - data model") { */ SECTION("some header after the genesis") { RWTxnManaged tx(context.env()); + DataModel data_model{tx, repository}; // starting from an initial status auto header0 = read_canonical_header(tx, 0); @@ -115,7 +120,7 @@ TEST_CASE("HeadersStage - data model") { // updating the data model BlockNum headers_stage_height = 0; - HeaderDataModelForTest hm(tx, headers_stage_height); + HeaderDataModelForTest hm{tx, data_model, headers_stage_height}; hm.update_tables(header1); hm.update_tables(header2); @@ -133,7 +138,7 @@ TEST_CASE("HeadersStage - data model") { // Now we suppose CL triggers an unwind, resetting to h0 BlockNum headers_stage_height_fork = 0; - HeaderDataModelForTest hm_fork(tx, headers_stage_height_fork); + HeaderDataModelForTest hm_fork{tx, data_model, headers_stage_height_fork}; hm_fork.update_tables(header1b); // suppose it arrives after header2 diff --git a/silkworm/node/stagedsync/stages/stage_history_index_test.cpp b/silkworm/node/stagedsync/stages/stage_history_index_test.cpp index 1c77cecded..f2b54b1b91 100644 --- a/silkworm/node/stagedsync/stages/stage_history_index_test.cpp +++ b/silkworm/node/stagedsync/stages/stage_history_index_test.cpp @@ -89,7 +89,7 @@ TEST_CASE("Stage History Index") { block.transactions[0].s = 1; // dummy block.transactions[0].set_sender(sender); - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; Account sender_account{}; sender_account.balance = kEther; buffer.update_account(sender, std::nullopt, sender_account); @@ -449,7 +449,7 @@ TEST_CASE("HistoryIndex + Account access_layer") { TempChainData context; RWTxn& txn{context.rw_txn()}; - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(txn)}; const evmc::address miner_a{0x00000000000000000000000000000000000000aa_address}; const evmc::address miner_b{0x00000000000000000000000000000000000000bb_address}; diff --git a/silkworm/node/stagedsync/stages/stage_interhashes.cpp b/silkworm/node/stagedsync/stages/stage_interhashes.cpp index 0a668cd1cf..4e11c8f2bc 100644 --- a/silkworm/node/stagedsync/stages/stage_interhashes.cpp +++ b/silkworm/node/stagedsync/stages/stage_interhashes.cpp @@ -42,7 +42,7 @@ Stage::Result InterHashes::forward(RWTxn& txn) { try { throw_if_stopping(); - DataModel data_model{txn}; + DataModel data_model = data_model_factory_(txn); // Check stage boundaries from previous execution and previous stage execution auto previous_progress{get_progress(txn)}; @@ -135,7 +135,7 @@ Stage::Result InterHashes::unwind(RWTxn& txn) { try { throw_if_stopping(); - DataModel data_model{txn}; + DataModel data_model = data_model_factory_(txn); BlockNum previous_progress{get_progress(txn)}; if (to >= previous_progress) { diff --git a/silkworm/node/stagedsync/stages/stage_interhashes.hpp b/silkworm/node/stagedsync/stages/stage_interhashes.hpp index 5729c32efa..af3233fd37 100644 --- a/silkworm/node/stagedsync/stages/stage_interhashes.hpp +++ b/silkworm/node/stagedsync/stages/stage_interhashes.hpp @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -27,8 +28,12 @@ namespace silkworm::stagedsync { class InterHashes final : public Stage { public: - explicit InterHashes(SyncContext* sync_context, db::etl::CollectorSettings etl_settings) + InterHashes( + SyncContext* sync_context, + db::DataModelFactory data_model_factory, + db::etl::CollectorSettings etl_settings) : Stage(sync_context, db::stages::kIntermediateHashesKey), + data_model_factory_(std::move(data_model_factory)), etl_settings_(std::move(etl_settings)) {} ~InterHashes() override = default; @@ -112,6 +117,8 @@ class InterHashes final : public Stage { // The loader which (re)builds the trees std::unique_ptr trie_loader_; + db::DataModelFactory data_model_factory_; + db::etl::CollectorSettings etl_settings_; std::unique_ptr account_collector_; // To accumulate new records for kTrieOfAccounts diff --git a/silkworm/node/stagedsync/stages/stage_senders.cpp b/silkworm/node/stagedsync/stages/stage_senders.cpp index 2d5ed84eef..6a7e31e20d 100644 --- a/silkworm/node/stagedsync/stages/stage_senders.cpp +++ b/silkworm/node/stagedsync/stages/stage_senders.cpp @@ -36,11 +36,13 @@ using namespace silkworm::db; Senders::Senders( SyncContext* sync_context, + DataModelFactory data_model_factory, const ChainConfig& chain_config, size_t batch_size, etl::CollectorSettings etl_settings, BlockAmount prune_mode_senders) : Stage(sync_context, stages::kSendersKey), + data_model_factory_(std::move(data_model_factory)), chain_config_(chain_config), prune_mode_senders_(prune_mode_senders), max_batch_size_{batch_size / std::thread::hardware_concurrency() / sizeof(AddressRecovery)}, @@ -265,7 +267,7 @@ Stage::Result Senders::parallel_recover(RWTxn& txn) { results_.clear(); try { - DataModel data_model{txn}; + DataModel data_model = data_model_factory_(txn); // Check stage boundaries using previous execution of current stage and current execution of previous stage auto previous_progress{stages::read_stage_progress(txn, stages::kSendersKey)}; diff --git a/silkworm/node/stagedsync/stages/stage_senders.hpp b/silkworm/node/stagedsync/stages/stage_senders.hpp index 44d6ea929c..433048331e 100644 --- a/silkworm/node/stagedsync/stages/stage_senders.hpp +++ b/silkworm/node/stagedsync/stages/stage_senders.hpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -52,6 +53,7 @@ class Senders final : public Stage { public: Senders( SyncContext* sync_context, + db::DataModelFactory data_model_factory, const ChainConfig& chain_config, size_t batch_size, db::etl::CollectorSettings etl_settings, @@ -77,6 +79,7 @@ class Senders final : public Stage { void increment_total_processed_blocks(); void increment_total_collected_transactions(size_t delta); + db::DataModelFactory data_model_factory_; const ChainConfig& chain_config_; db::BlockAmount prune_mode_senders_; diff --git a/silkworm/node/stagedsync/stages/stage_tx_lookup.cpp b/silkworm/node/stagedsync/stages/stage_tx_lookup.cpp index 997dcb00f2..705e212298 100644 --- a/silkworm/node/stagedsync/stages/stage_tx_lookup.cpp +++ b/silkworm/node/stagedsync/stages/stage_tx_lookup.cpp @@ -50,7 +50,8 @@ Stage::Result TxLookup::forward(RWTxn& txn) { } // Snapshots already have TxLookup index, so we must start after max frozen block here - const auto highest_frozen_block_number{DataModel::highest_frozen_block_number()}; + DataModel data_model = data_model_factory_(txn); + const auto highest_frozen_block_number{data_model.highest_frozen_block_number()}; if (highest_frozen_block_number > previous_progress) { previous_progress = std::min(highest_frozen_block_number, target_progress); // If pruning is enabled, make it start from max frozen block as well @@ -124,7 +125,8 @@ Stage::Result TxLookup::unwind(RWTxn& txn) { } // Snapshots already have TxLookup index, so we must stop before max frozen block here - const auto highest_frozen_block_number{DataModel::highest_frozen_block_number()}; + DataModel data_model = data_model_factory_(txn); + const auto highest_frozen_block_number{data_model.highest_frozen_block_number()}; if (highest_frozen_block_number > to) { to = highest_frozen_block_number; } @@ -338,7 +340,7 @@ void TxLookup::collect_transaction_hashes_from_canonical_bodies(RWTxn& txn, using namespace std::chrono_literals; auto log_time{std::chrono::steady_clock::now()}; - DataModel data_model{txn}; + DataModel data_model = data_model_factory_(txn); BlockNum target_block_num{std::max(from, to)}; BlockNum start_block_num{std::min(from, to) + 1}; diff --git a/silkworm/node/stagedsync/stages/stage_tx_lookup.hpp b/silkworm/node/stagedsync/stages/stage_tx_lookup.hpp index 984d63e6f5..476f6cb579 100644 --- a/silkworm/node/stagedsync/stages/stage_tx_lookup.hpp +++ b/silkworm/node/stagedsync/stages/stage_tx_lookup.hpp @@ -16,6 +16,7 @@ #pragma once +#include #include #include #include @@ -27,9 +28,11 @@ class TxLookup : public Stage { public: TxLookup( SyncContext* sync_context, + db::DataModelFactory data_model_factory, db::etl::CollectorSettings etl_settings, db::BlockAmount prune_mode_tx_index) : Stage(sync_context, db::stages::kTxLookupKey), + data_model_factory_(std::move(data_model_factory)), etl_settings_(std::move(etl_settings)), prune_mode_tx_index_(prune_mode_tx_index) {} TxLookup(const TxLookup&) = delete; // not copyable @@ -42,6 +45,7 @@ class TxLookup : public Stage { std::vector get_log_progress() final; private: + db::DataModelFactory data_model_factory_; db::etl::CollectorSettings etl_settings_; db::BlockAmount prune_mode_tx_index_; diff --git a/silkworm/node/stagedsync/stages/stage_tx_lookup_test.cpp b/silkworm/node/stagedsync/stages/stage_tx_lookup_test.cpp index 70c93b3cd4..d0bcf88151 100644 --- a/silkworm/node/stagedsync/stages/stage_tx_lookup_test.cpp +++ b/silkworm/node/stagedsync/stages/stage_tx_lookup_test.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -32,9 +33,11 @@ using silkworm::test_util::SetLogVerbosityGuard; stagedsync::TxLookup make_tx_lookup_stage( stagedsync::SyncContext* sync_context, + snapshots::SnapshotRepository& repository, const TempChainData& chain_data) { return stagedsync::TxLookup{ sync_context, + [&](ROTxn& tx) { return DataModel{tx, repository}; }, etl::CollectorSettings{chain_data.dir().temp().path(), 256_Mebi}, chain_data.prune_mode().tx_index(), }; @@ -50,6 +53,8 @@ TEST_CASE("Stage Transaction Lookups") { RWTxn& txn{context.rw_txn()}; txn.disable_commit(); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + PooledCursor canonicals(txn, table::kCanonicalHashes); PooledCursor bodies_table(txn, table::kBlockBodies); PooledCursor transactions_table(txn, table::kBlockTransactions); @@ -89,7 +94,7 @@ TEST_CASE("Stage Transaction Lookups") { SECTION("Forward checks and unwind") { // Execute stage forward stagedsync::SyncContext sync_context{}; - stagedsync::TxLookup stage_tx_lookup = make_tx_lookup_stage(&sync_context, context); + stagedsync::TxLookup stage_tx_lookup = make_tx_lookup_stage(&sync_context, repository, context); REQUIRE(stage_tx_lookup.forward(txn) == stagedsync::Stage::Result::kSuccess); PooledCursor lookup_table(txn, table::kTxLookup); @@ -140,7 +145,7 @@ TEST_CASE("Stage Transaction Lookups") { // Execute stage forward stagedsync::SyncContext sync_context{}; - stagedsync::TxLookup stage_tx_lookup = make_tx_lookup_stage(&sync_context, context); + stagedsync::TxLookup stage_tx_lookup = make_tx_lookup_stage(&sync_context, repository, context); REQUIRE(stage_tx_lookup.forward(txn) == stagedsync::Stage::Result::kSuccess); // Only leave block 2 alive diff --git a/silkworm/node/stagedsync/stages_factory_impl.cpp b/silkworm/node/stagedsync/stages_factory_impl.cpp new file mode 100644 index 0000000000..7146d40d4a --- /dev/null +++ b/silkworm/node/stagedsync/stages_factory_impl.cpp @@ -0,0 +1,63 @@ +/* + Copyright 2024 The Silkworm Authors + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#include "stages_factory_impl.hpp" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace silkworm::stagedsync { + +using namespace db::stages; + +StageContainer StagesFactoryImpl::make(SyncContext& sync_context) const { + SyncContext* sync_context_ptr = &sync_context; + StageContainer stages; + stages.emplace(kHeadersKey, std::make_unique(sync_context_ptr, data_model_factory_)); + stages.emplace(kBlockBodiesKey, bodies_stage_factory_(sync_context_ptr)); + stages.emplace(kBlockHashesKey, std::make_unique(sync_context_ptr, settings_.etl())); + stages.emplace(kSendersKey, std::make_unique(sync_context_ptr, data_model_factory_, *settings_.chain_config, settings_.batch_size, settings_.etl(), settings_.prune_mode.senders())); + stages.emplace(kExecutionKey, std::make_unique(sync_context_ptr, data_model_factory_, *settings_.chain_config, settings_.batch_size, settings_.prune_mode)); + stages.emplace(kHashStateKey, std::make_unique(sync_context_ptr, settings_.etl())); + stages.emplace(kIntermediateHashesKey, std::make_unique(sync_context_ptr, data_model_factory_, settings_.etl())); + stages.emplace(kHistoryIndexKey, std::make_unique(sync_context_ptr, settings_.batch_size, settings_.etl(), settings_.prune_mode.history())); + stages.emplace(kLogIndexKey, std::make_unique(sync_context_ptr, settings_.batch_size, settings_.etl(), settings_.prune_mode.history())); + stages.emplace(kCallTracesKey, std::make_unique(sync_context_ptr, settings_.batch_size, settings_.etl(), settings_.prune_mode.call_traces())); + stages.emplace(kTxLookupKey, std::make_unique(sync_context_ptr, data_model_factory_, settings_.etl(), settings_.prune_mode.tx_index())); + stages.emplace(kTriggersStageKey, std::make_unique(sync_context_ptr)); + stages.emplace(kFinishKey, std::make_unique(sync_context_ptr, settings_.build_info.build_description)); + return stages; +} + +StageContainerFactory StagesFactoryImpl::to_factory(StagesFactoryImpl instance) { + return [instance = std::move(instance)](SyncContext& sync_context) -> StageContainer { + return instance.make(sync_context); + }; +} + +} // namespace silkworm::stagedsync diff --git a/silkworm/node/stagedsync/stages_factory_impl.hpp b/silkworm/node/stagedsync/stages_factory_impl.hpp new file mode 100644 index 0000000000..d5900c50a0 --- /dev/null +++ b/silkworm/node/stagedsync/stages_factory_impl.hpp @@ -0,0 +1,47 @@ +/* + Copyright 2024 The Silkworm Authors + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#pragma once + +#include +#include + +#include "execution_pipeline.hpp" +#include "stages/stage_bodies_factory.hpp" + +namespace silkworm::stagedsync { + +class StagesFactoryImpl { + public: + StagesFactoryImpl( + const NodeSettings& settings, + db::DataModelFactory data_model_factory, + BodiesStageFactory bodies_stage_factory) + : settings_{settings}, + data_model_factory_{std::move(data_model_factory)}, + bodies_stage_factory_{std::move(bodies_stage_factory)} {} + + static StageContainerFactory to_factory(StagesFactoryImpl instance); + + private: + StageContainer make(SyncContext& sync_context) const; + + const NodeSettings& settings_; + db::DataModelFactory data_model_factory_; + BodiesStageFactory bodies_stage_factory_; +}; + +} // namespace silkworm::stagedsync diff --git a/silkworm/node/stagedsync/stages_test.cpp b/silkworm/node/stagedsync/stages_test.cpp index 2eccd7bbb9..e0ffe67a7e 100644 --- a/silkworm/node/stagedsync/stages_test.cpp +++ b/silkworm/node/stagedsync/stages_test.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -46,9 +47,11 @@ static ethash::hash256 keccak256(const evmc::address& address) { static stagedsync::Execution make_execution_stage( stagedsync::SyncContext* sync_context, - const NodeSettings& node_settings) { + const NodeSettings& node_settings, + db::DataModelFactory data_model_factory) { return stagedsync::Execution{ sync_context, + std::move(data_model_factory), *node_settings.chain_config, node_settings.batch_size, node_settings.prune_mode, @@ -82,7 +85,7 @@ TEST_CASE("Sync Stages") { std::nullopt, std::nullopt, std::nullopt, std::nullopt, std::nullopt, std::nullopt, std::nullopt, std::nullopt, std::nullopt, std::nullopt); - test_util::SetLogVerbosityGuard log_guard{log::Level::kNone}; + silkworm::test_util::SetLogVerbosityGuard log_guard{log::Level::kNone}; auto chaindata_env{open_env(node_settings.chaindata_env_config)}; RWTxnManaged txn(chaindata_env); @@ -99,6 +102,9 @@ TEST_CASE("Sync Stages") { const auto tx_sequence_after_genesis{read_map_sequence(txn, table::kBlockTransactions.name)}; REQUIRE(tx_sequence_after_genesis == 2); // 2 system txs for genesis + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + db::DataModelFactory data_model_factory = [&](db::ROTxn& tx) { return db::DataModel{tx, repository}; }; + SECTION("BlockHashes") { SECTION("Forward/Unwind/Prune args validation") { stagedsync::SyncContext sync_context{}; @@ -235,6 +241,7 @@ TEST_CASE("Sync Stages") { stagedsync::SyncContext sync_context{}; stagedsync::Senders stage{ &sync_context, + data_model_factory, *node_settings.chain_config, node_settings.batch_size, node_settings.etl(), @@ -345,7 +352,7 @@ TEST_CASE("Sync Stages") { block.transactions[0].s = 1; // dummy block.transactions[0].set_sender(sender); - Buffer buffer{txn}; + Buffer buffer{txn, std::make_unique(data_model_factory(txn))}; Account sender_account{}; sender_account.balance = kEther; buffer.update_account(sender, std::nullopt, sender_account); @@ -425,10 +432,10 @@ TEST_CASE("Sync Stages") { // --------------------------------------- stagedsync::SyncContext sync_context{}; sync_context.unwind_point.emplace(2); - stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings); + stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings, data_model_factory); REQUIRE(stage.unwind(txn) == stagedsync::Stage::Result::kSuccess); - Buffer buffer2{txn}; + Buffer buffer2{txn, std::make_unique(data_model_factory(txn))}; std::optional contract_account{buffer2.read_account(contract_address)}; REQUIRE(contract_account.has_value()); @@ -482,7 +489,7 @@ TEST_CASE("Sync Stages") { // Unwind 4th block and checks if state corresponds to 3rd block stagedsync::SyncContext sync_context{}; sync_context.unwind_point.emplace(3); - stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings); + stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings, data_model_factory); REQUIRE(stage.unwind(txn) == stagedsync::Stage::Result::kSuccess); plain_state_cursor = txn.ro_cursor(table::kPlainState); @@ -501,7 +508,7 @@ TEST_CASE("Sync Stages") { SECTION("Execution Prune Default") { log::Info() << "Pruning with " << node_settings.prune_mode.to_string(); stagedsync::SyncContext sync_context{}; - stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings); + stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings, data_model_factory); REQUIRE(stage.prune(txn) == stagedsync::Stage::Result::kSuccess); // With default settings nothing should be pruned @@ -532,7 +539,7 @@ TEST_CASE("Sync Stages") { log::Info() << "Pruning with " << node_settings.prune_mode.to_string(); REQUIRE(node_settings.prune_mode.history().enabled()); stagedsync::SyncContext sync_context{}; - stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings); + stagedsync::Execution stage = make_execution_stage(&sync_context, node_settings, data_model_factory); REQUIRE(stage.prune(txn) == stagedsync::Stage::Result::kSuccess); PooledCursor account_changeset_table(txn, table::kAccountChangeSet); @@ -651,7 +658,7 @@ TEST_CASE("Sync Stages") { stagedsync::SyncContext sync_context{}; sync_context.target_height = 1; - stagedsync::Execution stage_execution = make_execution_stage(&sync_context, node_settings); + stagedsync::Execution stage_execution = make_execution_stage(&sync_context, node_settings, data_model_factory); CHECK(stage_execution.forward(txn) == StageResult::kSuccess); // Post-condition: CallTraceSet table diff --git a/silkworm/node/test_util/make_stages_factory.cpp b/silkworm/node/test_util/make_stages_factory.cpp new file mode 100644 index 0000000000..2861f8b03f --- /dev/null +++ b/silkworm/node/test_util/make_stages_factory.cpp @@ -0,0 +1,43 @@ +/* + Copyright 2024 The Silkworm Authors + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#include "make_stages_factory.hpp" + +#include "../stagedsync/stages/stage_bodies.hpp" +#include "../stagedsync/stages_factory_impl.hpp" + +namespace silkworm::stagedsync::test_util { + +BodiesStageFactory make_bodies_stage_factory(const ChainConfig& chain_config, db::DataModelFactory data_model_factory) { + return [chain_config, data_model_factory = std::move(data_model_factory)](SyncContext* sync_context) { + return std::make_unique( + sync_context, + chain_config, + data_model_factory, + [] { return 0; }); + }; +}; + +StageContainerFactory make_stages_factory(const NodeSettings& node_settings, db::DataModelFactory data_model_factory) { + auto bodies_stage_factory = make_bodies_stage_factory(*node_settings.chain_config, data_model_factory); + return StagesFactoryImpl::to_factory({ + node_settings, + std::move(data_model_factory), + std::move(bodies_stage_factory), + }); +} + +} // namespace silkworm::stagedsync::test_util diff --git a/silkworm/node/test_util/make_stages_factory.hpp b/silkworm/node/test_util/make_stages_factory.hpp new file mode 100644 index 0000000000..0160644be1 --- /dev/null +++ b/silkworm/node/test_util/make_stages_factory.hpp @@ -0,0 +1,31 @@ +/* + Copyright 2024 The Silkworm Authors + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#pragma once + +#include +#include + +#include "../common/node_settings.hpp" +#include "../stagedsync/execution_pipeline.hpp" +#include "../stagedsync/stages/stage_bodies_factory.hpp" + +namespace silkworm::stagedsync::test_util { + +BodiesStageFactory make_bodies_stage_factory(const ChainConfig& chain_config, db::DataModelFactory data_model_factory); +StageContainerFactory make_stages_factory(const NodeSettings& node_settings, db::DataModelFactory data_model_factory); + +} // namespace silkworm::stagedsync::test_util diff --git a/silkworm/node/test_util/mock_execution_engine.hpp b/silkworm/node/test_util/mock_execution_engine.hpp index 06a41de746..5c335a99cc 100644 --- a/silkworm/node/test_util/mock_execution_engine.hpp +++ b/silkworm/node/test_util/mock_execution_engine.hpp @@ -22,26 +22,41 @@ #include #include +#include #include #include #include #include #include -#include +#include namespace silkworm::execution::api { //! \brief gMock mock class for stagedsync::ExecutionEngine class MockExecutionEngine : public stagedsync::ExecutionEngine { public: - static stagedsync::BodiesStageFactory empty_bodies_stage_factory() { - return [](stagedsync::SyncContext*) { - return std::unique_ptr{}; + static db::DataModelFactory null_data_model_factory() { + return [](db::ROTxn&) -> db::DataModel { + SILKWORM_ASSERT(false); + std::abort(); + }; + }; + + static stagedsync::StageContainerFactory empty_stages_factory() { + return [](stagedsync::SyncContext&) { + return stagedsync::StageContainer{}; }; }; MockExecutionEngine(boost::asio::any_io_executor executor, NodeSettings& ns, db::RWAccess dba) - : ExecutionEngine(std::move(executor), ns, /* log_timer_factory = */ std::nullopt, empty_bodies_stage_factory(), std::move(dba)) {} + : ExecutionEngine{ + std::move(executor), + ns, + null_data_model_factory(), + /* log_timer_factory = */ std::nullopt, + empty_stages_factory(), + std::move(dba), + } {} ~MockExecutionEngine() override = default; MOCK_METHOD((void), open, (), (override)); diff --git a/silkworm/rpc/daemon.cpp b/silkworm/rpc/daemon.cpp index 04c50cd02a..a85470bce8 100644 --- a/silkworm/rpc/daemon.cpp +++ b/silkworm/rpc/daemon.cpp @@ -125,6 +125,7 @@ int Daemon::run(const DaemonSettings& settings) { // Activate the local chaindata and snapshot access (if required) std::optional chaindata_env; std::unique_ptr snapshot_repository; + std::optional data_store; if (settings.datadir) { DataDirectory data_folder{*settings.datadir}; @@ -138,27 +139,30 @@ int Daemon::run(const DaemonSettings& settings) { *chaindata_env = silkworm::db::open_env(db_config); // Create a new snapshot repository - snapshots::SnapshotSettings snapshot_settings{ - .repository_dir = data_folder.snapshots().path(), - }; snapshot_repository = std::make_unique( - std::move(snapshot_settings), + data_folder.snapshots().path(), std::make_unique(), std::make_unique()); snapshot_repository->reopen_folder(); - db::DataModel::set_snapshot_repository(snapshot_repository.get()); - // At startup check that chain configuration is valid db::ROTxnManaged ro_txn{*chaindata_env}; - db::DataModel data_access{ro_txn}; + db::DataModel data_access{ro_txn, *snapshot_repository}; if (const auto chain_config{data_access.read_chain_config()}; !chain_config) { throw std::runtime_error{"invalid chain configuration"}; } + + data_store.emplace(db::DataStoreRef{ + *chaindata_env, // NOLINT(cppcoreguidelines-slicing) + *snapshot_repository, + }); } // Create the one-and-only Silkrpc daemon - Daemon rpc_daemon{settings, chaindata_env}; + Daemon rpc_daemon{ + settings, + std::move(data_store), + }; // Check protocol version compatibility with Core Services if (!settings.skip_protocol_check) { @@ -224,21 +228,19 @@ ChannelFactory Daemon::make_channel_factory(const DaemonSettings& settings) { }; } -Daemon::Daemon(DaemonSettings settings, std::optional chaindata_env) +Daemon::Daemon( + DaemonSettings settings, + std::optional data_store) : settings_(std::move(settings)), create_channel_{make_channel_factory(settings_)}, context_pool_{settings_.context_pool_settings.num_contexts}, - worker_pool_{settings_.num_workers} { + worker_pool_{settings_.num_workers}, + data_store_{std::move(data_store)} { // Load the channel authentication token (if required) if (settings_.jwt_secret_file) { jwt_secret_ = load_jwt_token(*settings_.jwt_secret_file); } - if (chaindata_env) { - // Use the existing chaindata environment - chaindata_env_ = std::move(chaindata_env); - } - // Create shared and private state in execution contexts: order *matters* (e.g. for state cache) add_shared_services(); add_private_services(); @@ -270,8 +272,8 @@ void Daemon::add_private_services() { auto tx_pool{std::make_unique(io_context, grpc_channel, grpc_context)}; auto miner{std::make_unique(io_context, grpc_channel, grpc_context)}; std::unique_ptr database; - if (chaindata_env_) { - database = std::make_unique(state_cache, *chaindata_env_); + if (data_store_) { + database = std::make_unique(*data_store_, state_cache); } else { database = std::make_unique(backend.get(), state_cache, grpc_context, grpc_channel); } @@ -318,7 +320,7 @@ std::unique_ptr Daemon::make_kv_client(rpc::ClientContext& db::kv::api::StateChangeRunner runner{io_context.get_executor()}; db::kv::api::ServiceRouter router{runner.state_changes_calls_channel()}; return std::make_unique( - std::make_shared(router, *chaindata_env_, state_cache)); + std::make_shared(router, *data_store_, state_cache)); } void Daemon::add_execution_services(const std::vector>& engines) { @@ -364,8 +366,8 @@ void Daemon::start() { // Put the interface logs into the data folder std::filesystem::path data_folder{}; - if (chaindata_env_) { - auto chaindata_path{chaindata_env_->get_path()}; + if (data_store_) { + auto chaindata_path = data_store_->chaindata_env.get_path(); // Trick to remove any empty filename because MDBX chaindata path ends with '/' if (chaindata_path.filename().empty()) { chaindata_path = chaindata_path.parent_path(); diff --git a/silkworm/rpc/daemon.hpp b/silkworm/rpc/daemon.hpp index 7413880de9..840b7465be 100644 --- a/silkworm/rpc/daemon.hpp +++ b/silkworm/rpc/daemon.hpp @@ -21,6 +21,7 @@ #include #include +#include #include #include #include @@ -46,7 +47,9 @@ class Daemon { public: static int run(const DaemonSettings& settings); - explicit Daemon(DaemonSettings settings, std::optional chaindata_env = {}); + explicit Daemon( + DaemonSettings settings, + std::optional data_store); Daemon(const Daemon&) = delete; Daemon& operator=(const Daemon&) = delete; @@ -82,8 +85,8 @@ class Daemon { //! The pool of workers for long-running tasks. WorkerPool worker_pool_; - //! The chaindata MDBX environment or \code std::nullopt if working remotely - std::optional chaindata_env_; + //! The data store or std::nullopt if working remotely + std::optional data_store_; //! The JSON RPC API services. std::vector> rpc_services_; diff --git a/silkworm/rpc/ethdb/file/local_database.cpp b/silkworm/rpc/ethdb/file/local_database.cpp index d6b64c670c..2b786fc75f 100644 --- a/silkworm/rpc/ethdb/file/local_database.cpp +++ b/silkworm/rpc/ethdb/file/local_database.cpp @@ -23,8 +23,11 @@ namespace silkworm::rpc::ethdb::file { -LocalDatabase::LocalDatabase(StateCache* state_cache, mdbx::env chaindata_env) - : state_cache_{state_cache}, chaindata_env_{std::move(chaindata_env)} { +LocalDatabase::LocalDatabase( + db::DataStoreRef data_store, + StateCache* state_cache) + : data_store_{std::move(data_store)}, + state_cache_{state_cache} { SILK_TRACE << "LocalDatabase::ctor " << this; } @@ -34,7 +37,7 @@ LocalDatabase::~LocalDatabase() { Task> LocalDatabase::begin() { SILK_TRACE << "LocalDatabase::begin " << this << " start"; - auto txn = std::make_unique(chaindata_env_, state_cache_); + auto txn = std::make_unique(data_store_, state_cache_); co_await txn->open(); SILK_TRACE << "LocalDatabase::begin " << this << " txn: " << txn.get() << " end"; co_return txn; diff --git a/silkworm/rpc/ethdb/file/local_database.hpp b/silkworm/rpc/ethdb/file/local_database.hpp index c621e8f227..3c48bd18eb 100644 --- a/silkworm/rpc/ethdb/file/local_database.hpp +++ b/silkworm/rpc/ethdb/file/local_database.hpp @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -31,7 +32,9 @@ using db::kv::api::StateCache; class LocalDatabase : public Database { public: - explicit LocalDatabase(StateCache* state_cache, mdbx::env chaindata_env); + explicit LocalDatabase( + db::DataStoreRef data_store, + StateCache* state_cache); ~LocalDatabase() override; @@ -41,8 +44,8 @@ class LocalDatabase : public Database { Task> begin() override; private: + db::DataStoreRef data_store_; StateCache* state_cache_; - mdbx::env chaindata_env_; }; } // namespace silkworm::rpc::ethdb::file diff --git a/silkworm/rpc/test_util/api_test_database.hpp b/silkworm/rpc/test_util/api_test_database.hpp index 7a5426a684..7e5ab8358a 100644 --- a/silkworm/rpc/test_util/api_test_database.hpp +++ b/silkworm/rpc/test_util/api_test_database.hpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -78,18 +79,35 @@ class RequestHandlerForTest : public json_rpc::RequestHandler { } }; +class TestDataStoreBase : public db::test_util::TestDatabaseContext { + public: + explicit TestDataStoreBase() + : repository_{db::test_util::make_repository()}, + data_store_{mdbx_env(), repository_} {} + + db::DataStoreRef data_store() const { return data_store_; } + + private: + snapshots::SnapshotRepository repository_; + db::DataStoreRef data_store_; +}; + class LocalContextTestBase : public ServiceContextTestBase { public: - explicit LocalContextTestBase(db::kv::api::StateCache* state_cache, mdbx::env& chaindata_env) : ServiceContextTestBase() { - add_private_service(io_context_, std::make_unique(state_cache, chaindata_env)); + explicit LocalContextTestBase( + db::DataStoreRef data_store, + db::kv::api::StateCache* state_cache) { + add_private_service( + io_context_, + std::make_unique(std::move(data_store), state_cache)); } }; template class RpcApiTestBase : public LocalContextTestBase { public: - explicit RpcApiTestBase(mdbx::env& chaindata_env) - : LocalContextTestBase(&state_cache_, chaindata_env), + explicit RpcApiTestBase(db::DataStoreRef data_store) + : LocalContextTestBase{std::move(data_store), &state_cache_}, workers_{1}, socket_{io_context_}, rpc_api_{io_context_, workers_}, @@ -111,9 +129,9 @@ class RpcApiTestBase : public LocalContextTestBase { db::kv::api::CoherentStateCache state_cache_; }; -class RpcApiE2ETest : public db::test_util::TestDatabaseContext, RpcApiTestBase { +class RpcApiE2ETest : public TestDataStoreBase, RpcApiTestBase { public: - explicit RpcApiE2ETest() : RpcApiTestBase(mdbx_env()) { + explicit RpcApiE2ETest() : RpcApiTestBase{data_store()} { // Ensure JSON RPC spec has been loaded into the validator if (!jsonrpc_spec_loaded) { json_rpc::Validator::load_specification(); diff --git a/silkworm/sentry/eth/status_data_provider.cpp b/silkworm/sentry/eth/status_data_provider.cpp index a11af8b1ae..5a1978ca86 100644 --- a/silkworm/sentry/eth/status_data_provider.cpp +++ b/silkworm/sentry/eth/status_data_provider.cpp @@ -67,7 +67,7 @@ StatusDataProvider::StatusData StatusDataProvider::make_status_data( return status_data; } -StatusDataProvider::StatusData StatusDataProvider::get_status_data(uint8_t eth_version) { +StatusDataProvider::StatusData StatusDataProvider::get_status_data(uint8_t eth_version) const { if (eth_version != silkworm::sentry::eth::Protocol::kVersion) { throw std::runtime_error("StatusDataProvider::get_status_data: unsupported eth version " + std::to_string(eth_version)); } @@ -79,7 +79,7 @@ StatusDataProvider::StatusData StatusDataProvider::get_status_data(uint8_t eth_v } StatusDataProvider::StatusDataProviderFactory StatusDataProvider::to_factory_function(StatusDataProvider provider) { - return [provider = std::move(provider)](uint8_t eth_version) mutable -> Task { + return [provider = std::move(provider)](uint8_t eth_version) -> Task { co_return provider.get_status_data(eth_version); }; } diff --git a/silkworm/sentry/eth/status_data_provider.hpp b/silkworm/sentry/eth/status_data_provider.hpp index b270d37d54..904a3bb24e 100644 --- a/silkworm/sentry/eth/status_data_provider.hpp +++ b/silkworm/sentry/eth/status_data_provider.hpp @@ -36,7 +36,7 @@ class StatusDataProvider { chain_config_(chain_config) {} using StatusData = silkworm::sentry::eth::StatusData; - StatusData get_status_data(uint8_t eth_version); + StatusData get_status_data(uint8_t eth_version) const; using StatusDataProviderFactory = std::function(uint8_t eth_version)>; static StatusDataProviderFactory to_factory_function(StatusDataProvider provider); diff --git a/silkworm/sync/block_exchange.cpp b/silkworm/sync/block_exchange.cpp index b4fd4c6d58..beb07fe3d3 100644 --- a/silkworm/sync/block_exchange.cpp +++ b/silkworm/sync/block_exchange.cpp @@ -17,6 +17,7 @@ #include "block_exchange.hpp" #include +#include #include @@ -31,15 +32,14 @@ namespace silkworm { BlockExchange::BlockExchange( + db::DataStoreRef data_store, SentryClient& sentry, - db::ROAccess dba, const ChainConfig& chain_config, bool use_preverified_hashes) - : db_access_{std::move(dba)}, + : data_store_{std::move(data_store)}, sentry_{sentry}, chain_config_{chain_config}, - header_chain_{chain_config, use_preverified_hashes}, - body_sequence_{} { + header_chain_{chain_config, use_preverified_hashes} { } BlockExchange::~BlockExchange() { @@ -99,7 +99,7 @@ void BlockExchange::execution_loop() { // process an external message (replay to remote peers) or an internal message if (present) { - message->execute(db_access_, header_chain_, body_sequence_, sentry_); + message->execute(data_store_, header_chain_, body_sequence_, sentry_); ++statistics_.processed_msgs; } @@ -162,7 +162,7 @@ size_t BlockExchange::request_headers(time_point_t tp, size_t max_requests) { if (!request_message) break; - request_message->execute(db_access_, header_chain_, body_sequence_, sentry_); + request_message->execute(data_store_, header_chain_, body_sequence_, sentry_); statistics_.sent_msgs += request_message->sent_requests(); statistics_.nack_msgs += request_message->nack_requests(); @@ -187,7 +187,7 @@ size_t BlockExchange::request_bodies(time_point_t tp, size_t max_requests) { if (!request_message) break; - request_message->execute(db_access_, header_chain_, body_sequence_, sentry_); + request_message->execute(data_store_, header_chain_, body_sequence_, sentry_); statistics_.sent_msgs += request_message->sent_requests(); statistics_.nack_msgs += request_message->nack_requests(); diff --git a/silkworm/sync/block_exchange.hpp b/silkworm/sync/block_exchange.hpp index 22d49d976c..482dc579b1 100644 --- a/silkworm/sync/block_exchange.hpp +++ b/silkworm/sync/block_exchange.hpp @@ -17,14 +17,13 @@ #pragma once #include -#include #include #include #include +#include #include #include -#include #include #include #include @@ -37,8 +36,8 @@ class SentryClient; class BlockExchange : public ActiveComponent { public: BlockExchange( + db::DataStoreRef data_store, SentryClient& sentry, - db::ROAccess dba, const ChainConfig& chain_config, bool use_preverified_hashes); ~BlockExchange() override; @@ -83,7 +82,8 @@ class BlockExchange : public ActiveComponent { void collect_bodies(); void log_status(); - db::ROAccess db_access_; // only to reply remote peer's requests + // only to reply remote peer's requests + db::DataStoreRef data_store_; SentryClient& sentry_; const ChainConfig& chain_config_; HeaderChain header_chain_; diff --git a/silkworm/sync/internals/body_retrieval.cpp b/silkworm/sync/internals/body_retrieval.cpp index eede4105f1..d58ff9a720 100644 --- a/silkworm/sync/internals/body_retrieval.cpp +++ b/silkworm/sync/internals/body_retrieval.cpp @@ -20,8 +20,6 @@ namespace silkworm { -BodyRetrieval::BodyRetrieval(db::ROAccess db_access) : db_tx_{db_access.start_ro_tx()} {} - std::vector BodyRetrieval::recover(std::vector request) { std::vector response; size_t bytes = 0; diff --git a/silkworm/sync/internals/body_retrieval.hpp b/silkworm/sync/internals/body_retrieval.hpp index 0a317f292b..d12f9fc773 100644 --- a/silkworm/sync/internals/body_retrieval.hpp +++ b/silkworm/sync/internals/body_retrieval.hpp @@ -27,12 +27,13 @@ class BodyRetrieval { static const int kSoftResponseLimit = 2 * 1024 * 1024; // Target maximum size of returned blocks static const int kMaxBodiesServe = 1024; // Amount of block bodies to be fetched per retrieval request - explicit BodyRetrieval(db::ROAccess db_access); + explicit BodyRetrieval(db::ROTxn& db_tx) + : db_tx_{db_tx} {} std::vector recover(std::vector); protected: - db::ROTxnManaged db_tx_; + db::ROTxn& db_tx_; }; } // namespace silkworm diff --git a/silkworm/sync/internals/header_retrieval.cpp b/silkworm/sync/internals/header_retrieval.cpp index 4e0e9c7644..007701484d 100644 --- a/silkworm/sync/internals/header_retrieval.cpp +++ b/silkworm/sync/internals/header_retrieval.cpp @@ -22,8 +22,6 @@ namespace silkworm { -HeaderRetrieval::HeaderRetrieval(db::ROAccess db_access) : db_tx_{db_access.start_ro_tx()}, data_model_{db_tx_} {} - std::vector HeaderRetrieval::recover_by_hash(Hash origin, uint64_t amount, uint64_t skip, bool reverse) { using std::optional; uint64_t max_non_canonical = 100; @@ -127,9 +125,9 @@ std::tuple HeaderRetrieval::get_ancestor(Hash hash, BlockNum blo } while (ancestor_delta != 0) { - auto h = db::read_canonical_header_hash(db_tx_, block_num); + auto h = data_model_.read_canonical_header_hash(block_num); if (h == hash) { - auto ancestorHash = db::read_canonical_header_hash(db_tx_, block_num - ancestor_delta); + auto ancestorHash = data_model_.read_canonical_header_hash(block_num - ancestor_delta); if (!ancestorHash) { return {Hash{}, 0}; } diff --git a/silkworm/sync/internals/header_retrieval.hpp b/silkworm/sync/internals/header_retrieval.hpp index 0f6415af38..b9e9f0414e 100644 --- a/silkworm/sync/internals/header_retrieval.hpp +++ b/silkworm/sync/internals/header_retrieval.hpp @@ -31,7 +31,8 @@ class HeaderRetrieval { static const int kEstHeaderRlpSize = 500; // Approximate size of an RLP encoded block header static const int kMaxHeadersServe = 1024; // Amount of block headers to be fetched per retrieval request - explicit HeaderRetrieval(db::ROAccess); + explicit HeaderRetrieval(db::DataModel data_model) + : data_model_{data_model} {} // Headers std::vector recover_by_hash(Hash origin, uint64_t amount, uint64_t skip, bool reverse); @@ -42,7 +43,6 @@ class HeaderRetrieval { uint64_t& max_non_canonical); protected: - db::ROTxnManaged db_tx_; db::DataModel data_model_; }; diff --git a/silkworm/sync/internals/header_retrieval_test.cpp b/silkworm/sync/internals/header_retrieval_test.cpp index a72d446e04..76012c665a 100644 --- a/silkworm/sync/internals/header_retrieval_test.cpp +++ b/silkworm/sync/internals/header_retrieval_test.cpp @@ -18,6 +18,7 @@ #include +#include #include namespace silkworm { @@ -26,7 +27,13 @@ TEST_CASE("HeaderRetrieval") { db::test_util::TempChainData context; context.add_genesis_data(); context.commit_txn(); - HeaderRetrieval header_retrieval{db::ROAccess{context.env()}}; + + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + + db::ROTxnManaged tx = db::ROAccess{context.env()}.start_ro_tx(); + db::DataModel data_model{tx, repository}; + + HeaderRetrieval header_retrieval{data_model}; SECTION("recover_by_hash") { const auto headers{header_retrieval.recover_by_hash({}, 1, 0, false)}; diff --git a/silkworm/sync/messages/inbound_block_bodies.cpp b/silkworm/sync/messages/inbound_block_bodies.cpp index 14628165a6..fa942ed737 100644 --- a/silkworm/sync/messages/inbound_block_bodies.cpp +++ b/silkworm/sync/messages/inbound_block_bodies.cpp @@ -29,7 +29,7 @@ InboundBlockBodies::InboundBlockBodies(ByteView data, PeerId peer_id) SILK_TRACE << "Received message " << *this; } -void InboundBlockBodies::execute(db::ROAccess, HeaderChain&, BodySequence& bs, SentryClient& sentry) { +void InboundBlockBodies::execute(db::DataStoreRef, HeaderChain&, BodySequence& bs, SentryClient& sentry) { SILK_TRACE << "Processing message " << *this; Penalty penalty = bs.accept_requested_bodies(packet_, peer_id_); diff --git a/silkworm/sync/messages/inbound_block_bodies.hpp b/silkworm/sync/messages/inbound_block_bodies.hpp index 717ff7895e..f24b020df5 100644 --- a/silkworm/sync/messages/inbound_block_bodies.hpp +++ b/silkworm/sync/messages/inbound_block_bodies.hpp @@ -31,7 +31,7 @@ class InboundBlockBodies : public InboundMessage { std::string content() const override; uint64_t req_id() const override; - void execute(db::ROAccess db, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef db, HeaderChain&, BodySequence&, SentryClient&) override; private: PeerId peer_id_; diff --git a/silkworm/sync/messages/inbound_block_headers.cpp b/silkworm/sync/messages/inbound_block_headers.cpp index da5bd50be6..de9f2ae571 100644 --- a/silkworm/sync/messages/inbound_block_headers.cpp +++ b/silkworm/sync/messages/inbound_block_headers.cpp @@ -29,7 +29,7 @@ InboundBlockHeaders::InboundBlockHeaders(ByteView data, PeerId peer_id) SILK_TRACE << "Received message " << *this; } -void InboundBlockHeaders::execute(db::ROAccess, HeaderChain& hc, BodySequence&, SentryClient& sentry) { +void InboundBlockHeaders::execute(db::DataStoreRef, HeaderChain& hc, BodySequence&, SentryClient& sentry) { using namespace std; SILK_TRACE << "Processing message " << *this; diff --git a/silkworm/sync/messages/inbound_block_headers.hpp b/silkworm/sync/messages/inbound_block_headers.hpp index 6e57294527..189c7eeec0 100644 --- a/silkworm/sync/messages/inbound_block_headers.hpp +++ b/silkworm/sync/messages/inbound_block_headers.hpp @@ -31,7 +31,7 @@ class InboundBlockHeaders : public InboundMessage { std::string content() const override; uint64_t req_id() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; private: PeerId peer_id_; diff --git a/silkworm/sync/messages/inbound_get_block_bodies.cpp b/silkworm/sync/messages/inbound_get_block_bodies.cpp index 5a6a45dfb2..a5052edf10 100644 --- a/silkworm/sync/messages/inbound_get_block_bodies.cpp +++ b/silkworm/sync/messages/inbound_get_block_bodies.cpp @@ -32,7 +32,7 @@ InboundGetBlockBodies::InboundGetBlockBodies(ByteView data, PeerId peer_id) SILK_TRACE << "Received message " << *this; } -void InboundGetBlockBodies::execute(db::ROAccess db, HeaderChain&, BodySequence& bs, SentryClient& sentry) { +void InboundGetBlockBodies::execute(db::DataStoreRef db, HeaderChain&, BodySequence& bs, SentryClient& sentry) { using namespace std; SILK_TRACE << "Processing message " << *this; @@ -40,7 +40,8 @@ void InboundGetBlockBodies::execute(db::ROAccess db, HeaderChain&, BodySequence& if (bs.highest_block_in_output() == 0) return; - BodyRetrieval body_retrieval(db); + db::ROTxnManaged tx = db::ROAccess{db.chaindata_env}.start_ro_tx(); + BodyRetrieval body_retrieval{tx}; BlockBodiesPacket66 reply; reply.request_id = packet_.request_id; diff --git a/silkworm/sync/messages/inbound_get_block_bodies.hpp b/silkworm/sync/messages/inbound_get_block_bodies.hpp index ac64b05777..c672921176 100644 --- a/silkworm/sync/messages/inbound_get_block_bodies.hpp +++ b/silkworm/sync/messages/inbound_get_block_bodies.hpp @@ -31,7 +31,7 @@ class InboundGetBlockBodies : public InboundMessage { std::string content() const override; uint64_t req_id() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; private: PeerId peer_id_; diff --git a/silkworm/sync/messages/inbound_get_block_headers.cpp b/silkworm/sync/messages/inbound_get_block_headers.cpp index c318e75cdd..76668bab28 100644 --- a/silkworm/sync/messages/inbound_get_block_headers.cpp +++ b/silkworm/sync/messages/inbound_get_block_headers.cpp @@ -32,7 +32,7 @@ InboundGetBlockHeaders::InboundGetBlockHeaders(ByteView data, PeerId peer_id) SILK_TRACE << "Received message " << *this; } -void InboundGetBlockHeaders::execute(db::ROAccess db, HeaderChain&, BodySequence& bs, SentryClient& sentry) { +void InboundGetBlockHeaders::execute(db::DataStoreRef db, HeaderChain&, BodySequence& bs, SentryClient& sentry) { using namespace std; SILK_TRACE << "Processing message " << *this; @@ -40,7 +40,9 @@ void InboundGetBlockHeaders::execute(db::ROAccess db, HeaderChain&, BodySequence if (bs.highest_block_in_output() == 0) // skip requests in the first sync even if we already saved some headers return; - HeaderRetrieval header_retrieval(db); + db::ROTxnManaged tx = db::ROAccess{db.chaindata_env}.start_ro_tx(); + db::DataModel data_model{tx, db.repository}; + HeaderRetrieval header_retrieval(data_model); BlockHeadersPacket66 reply; reply.request_id = packet_.request_id; diff --git a/silkworm/sync/messages/inbound_get_block_headers.hpp b/silkworm/sync/messages/inbound_get_block_headers.hpp index 6213cb881b..de3035043d 100644 --- a/silkworm/sync/messages/inbound_get_block_headers.hpp +++ b/silkworm/sync/messages/inbound_get_block_headers.hpp @@ -31,7 +31,7 @@ class InboundGetBlockHeaders : public InboundMessage { std::string content() const override; uint64_t req_id() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; private: PeerId peer_id_; diff --git a/silkworm/sync/messages/inbound_new_block.cpp b/silkworm/sync/messages/inbound_new_block.cpp index 96b732382d..3d8ec90e4e 100644 --- a/silkworm/sync/messages/inbound_new_block.cpp +++ b/silkworm/sync/messages/inbound_new_block.cpp @@ -32,7 +32,7 @@ InboundNewBlock::InboundNewBlock(ByteView data, PeerId peer_id) SILK_TRACE << "Received message " << *this; } -void InboundNewBlock::execute(db::ROAccess, HeaderChain&, BodySequence& bs, SentryClient&) { +void InboundNewBlock::execute(db::DataStoreRef, HeaderChain&, BodySequence& bs, SentryClient&) { SILK_TRACE << "Processing message " << *this; // todo: complete implementation diff --git a/silkworm/sync/messages/inbound_new_block.hpp b/silkworm/sync/messages/inbound_new_block.hpp index 8460a6e483..26a11779df 100644 --- a/silkworm/sync/messages/inbound_new_block.hpp +++ b/silkworm/sync/messages/inbound_new_block.hpp @@ -31,7 +31,7 @@ class InboundNewBlock : public InboundMessage { std::string content() const override; uint64_t req_id() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; private: PeerId peer_id_; diff --git a/silkworm/sync/messages/inbound_new_block_hashes.cpp b/silkworm/sync/messages/inbound_new_block_hashes.cpp index 44ffe88d06..9332f0ff66 100644 --- a/silkworm/sync/messages/inbound_new_block_hashes.cpp +++ b/silkworm/sync/messages/inbound_new_block_hashes.cpp @@ -38,7 +38,7 @@ InboundNewBlockHashes::InboundNewBlockHashes(ByteView data, PeerId peer_id) SILK_TRACE << "Received message " << *this; } -void InboundNewBlockHashes::execute(db::ROAccess, HeaderChain& hc, BodySequence&, SentryClient& sentry) { +void InboundNewBlockHashes::execute(db::DataStoreRef, HeaderChain& hc, BodySequence&, SentryClient& sentry) { using namespace std; SILK_TRACE << "Processing message " << *this; diff --git a/silkworm/sync/messages/inbound_new_block_hashes.hpp b/silkworm/sync/messages/inbound_new_block_hashes.hpp index bb73d7b509..a00374344b 100644 --- a/silkworm/sync/messages/inbound_new_block_hashes.hpp +++ b/silkworm/sync/messages/inbound_new_block_hashes.hpp @@ -31,7 +31,7 @@ class InboundNewBlockHashes : public InboundMessage { std::string content() const override; uint64_t req_id() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; private: PeerId peer_id_; diff --git a/silkworm/sync/messages/internal_message.hpp b/silkworm/sync/messages/internal_message.hpp index dc43de4e55..8ed4c1a897 100644 --- a/silkworm/sync/messages/internal_message.hpp +++ b/silkworm/sync/messages/internal_message.hpp @@ -34,7 +34,7 @@ class InternalMessage : public Message { std::string name() const override { return "InternalMessage"; } - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; std::future& result() { return result_out_; } @@ -50,13 +50,13 @@ template InternalMessage::InternalMessage(ExecutionFunc exec) : execution_impl_{std::move(exec)} {} template -void InternalMessage::execute(db::ROAccess, HeaderChain& hc, BodySequence& bs, SentryClient&) { +void InternalMessage::execute(db::DataStoreRef, HeaderChain& hc, BodySequence& bs, SentryClient&) { R local_result = execution_impl_(hc, bs); result_in_.set_value(local_result); } template <> -inline void InternalMessage::execute(db::ROAccess, HeaderChain& hc, BodySequence& bs, SentryClient&) { +inline void InternalMessage::execute(db::DataStoreRef, HeaderChain& hc, BodySequence& bs, SentryClient&) { execution_impl_(hc, bs); result_in_.set_value(); } diff --git a/silkworm/sync/messages/internal_message_test.cpp b/silkworm/sync/messages/internal_message_test.cpp index ba92e1f36c..a951ddb6a3 100644 --- a/silkworm/sync/messages/internal_message_test.cpp +++ b/silkworm/sync/messages/internal_message_test.cpp @@ -18,6 +18,7 @@ #include +#include #include #include @@ -27,7 +28,9 @@ namespace silkworm { [[clang::no_sanitize("null")]] TEST_CASE("internal message") { db::test_util::TempChainData context; // not used in the test execution - db::ROAccess dba(context.env()); + snapshots::SnapshotRepository repository = db::test_util::make_repository(); + // not used in the test execution + db::DataStoreRef data_store{context.env(), repository}; // not used in the test execution HeaderChain hc(kMainnetConfig, /* use_preverified_hashes = */ false); // not used in the test execution @@ -43,7 +46,7 @@ namespace silkworm { REQUIRE(!command->completed_and_read()); - command->execute(dba, hc, bs, *sc); + command->execute(data_store, hc, bs, *sc); REQUIRE(!command->completed_and_read()); diff --git a/silkworm/sync/messages/message.hpp b/silkworm/sync/messages/message.hpp index 04270ef82f..3a98e542fa 100644 --- a/silkworm/sync/messages/message.hpp +++ b/silkworm/sync/messages/message.hpp @@ -18,7 +18,7 @@ #include -#include +#include namespace silkworm { @@ -31,7 +31,7 @@ class Message { virtual std::string name() const = 0; // execute: inbound message send a reply, outbound message send a request - virtual void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) = 0; + virtual void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) = 0; virtual ~Message() = default; }; diff --git a/silkworm/sync/messages/outbound_block_bodies.cpp b/silkworm/sync/messages/outbound_block_bodies.cpp index fa7ce71df2..62095730b2 100644 --- a/silkworm/sync/messages/outbound_block_bodies.cpp +++ b/silkworm/sync/messages/outbound_block_bodies.cpp @@ -22,7 +22,7 @@ namespace silkworm { -void OutboundBlockBodies::execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) { +void OutboundBlockBodies::execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) { } Bytes OutboundBlockBodies::message_data() const { diff --git a/silkworm/sync/messages/outbound_block_bodies.hpp b/silkworm/sync/messages/outbound_block_bodies.hpp index 545add7d58..efdb1ea342 100644 --- a/silkworm/sync/messages/outbound_block_bodies.hpp +++ b/silkworm/sync/messages/outbound_block_bodies.hpp @@ -29,7 +29,7 @@ class OutboundBlockBodies : public OutboundMessage { std::string name() const override { return "OutboundBlockBodies"; } std::string content() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; silkworm::sentry::eth::MessageId eth_message_id() const override { return silkworm::sentry::eth::MessageId::kBlockBodies; diff --git a/silkworm/sync/messages/outbound_block_headers.cpp b/silkworm/sync/messages/outbound_block_headers.cpp index 24d8b523f4..52d9d7646d 100644 --- a/silkworm/sync/messages/outbound_block_headers.cpp +++ b/silkworm/sync/messages/outbound_block_headers.cpp @@ -22,7 +22,7 @@ namespace silkworm { -void OutboundBlockHeaders::execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) { +void OutboundBlockHeaders::execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) { } Bytes OutboundBlockHeaders::message_data() const { diff --git a/silkworm/sync/messages/outbound_block_headers.hpp b/silkworm/sync/messages/outbound_block_headers.hpp index 7d57fabcfd..912b5bac51 100644 --- a/silkworm/sync/messages/outbound_block_headers.hpp +++ b/silkworm/sync/messages/outbound_block_headers.hpp @@ -29,7 +29,7 @@ class OutboundBlockHeaders : public OutboundMessage { std::string name() const override { return "OutboundBlockHeaders"; } std::string content() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; silkworm::sentry::eth::MessageId eth_message_id() const override { return silkworm::sentry::eth::MessageId::kBlockHeaders; diff --git a/silkworm/sync/messages/outbound_get_block_bodies.cpp b/silkworm/sync/messages/outbound_get_block_bodies.cpp index 0619f950b3..7251f68938 100644 --- a/silkworm/sync/messages/outbound_get_block_bodies.cpp +++ b/silkworm/sync/messages/outbound_get_block_bodies.cpp @@ -30,7 +30,7 @@ std::vector& OutboundGetBlockBodies::penalties() { return pena BlockNum& OutboundGetBlockBodies::min_block() { return min_block_; } bool OutboundGetBlockBodies::packet_present() const { return !packet_.request.empty(); } -void OutboundGetBlockBodies::execute(db::ROAccess, HeaderChain&, BodySequence& bs, SentryClient& sentry) { +void OutboundGetBlockBodies::execute(db::DataStoreRef, HeaderChain&, BodySequence& bs, SentryClient& sentry) { if (packet_present()) { try { auto send_outcome = send_packet(sentry); diff --git a/silkworm/sync/messages/outbound_get_block_bodies.hpp b/silkworm/sync/messages/outbound_get_block_bodies.hpp index 002f806b4b..3928873984 100644 --- a/silkworm/sync/messages/outbound_get_block_bodies.hpp +++ b/silkworm/sync/messages/outbound_get_block_bodies.hpp @@ -31,7 +31,7 @@ class OutboundGetBlockBodies : public OutboundMessage { std::string name() const override { return "OutboundGetBlockBodies"; } std::string content() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; silkworm::sentry::eth::MessageId eth_message_id() const override { return silkworm::sentry::eth::MessageId::kGetBlockBodies; diff --git a/silkworm/sync/messages/outbound_get_block_headers.cpp b/silkworm/sync/messages/outbound_get_block_headers.cpp index 76eba0e500..8966ef163e 100644 --- a/silkworm/sync/messages/outbound_get_block_headers.cpp +++ b/silkworm/sync/messages/outbound_get_block_headers.cpp @@ -29,7 +29,7 @@ GetBlockHeadersPacket66& OutboundGetBlockHeaders::packet() { return packet_; } std::vector& OutboundGetBlockHeaders::penalties() { return penalizations_; } bool OutboundGetBlockHeaders::packet_present() const { return (packet_.request.amount != 0); } -void OutboundGetBlockHeaders::execute(db::ROAccess, HeaderChain& hc, BodySequence&, SentryClient& sentry) { +void OutboundGetBlockHeaders::execute(db::DataStoreRef, HeaderChain& hc, BodySequence&, SentryClient& sentry) { if (packet_present()) { try { auto send_outcome = send_packet(sentry); diff --git a/silkworm/sync/messages/outbound_get_block_headers.hpp b/silkworm/sync/messages/outbound_get_block_headers.hpp index 30fb4ddc3d..c4a3369897 100644 --- a/silkworm/sync/messages/outbound_get_block_headers.hpp +++ b/silkworm/sync/messages/outbound_get_block_headers.hpp @@ -32,7 +32,7 @@ class OutboundGetBlockHeaders : public OutboundMessage { std::string name() const override { return "OutboundGetBlockHeaders"; } std::string content() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; silkworm::sentry::eth::MessageId eth_message_id() const override { return silkworm::sentry::eth::MessageId::kGetBlockHeaders; diff --git a/silkworm/sync/messages/outbound_message.hpp b/silkworm/sync/messages/outbound_message.hpp index fca38d43d1..8a490f7091 100644 --- a/silkworm/sync/messages/outbound_message.hpp +++ b/silkworm/sync/messages/outbound_message.hpp @@ -27,7 +27,7 @@ namespace silkworm { class OutboundMessage : public Message { public: - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override = 0; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override = 0; size_t sent_requests() const; size_t nack_requests() const; diff --git a/silkworm/sync/messages/outbound_new_block.cpp b/silkworm/sync/messages/outbound_new_block.cpp index 743e2bc106..cbad042fef 100644 --- a/silkworm/sync/messages/outbound_new_block.cpp +++ b/silkworm/sync/messages/outbound_new_block.cpp @@ -26,7 +26,7 @@ namespace silkworm { OutboundNewBlock::OutboundNewBlock(Blocks b, bool is_first_sync) : blocks_to_announce_{std::move(b)}, is_first_sync_{is_first_sync} {} -void OutboundNewBlock::execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient& sentry) { +void OutboundNewBlock::execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient& sentry) { if (is_first_sync_) return; // Don't announce blocks during first sync for (auto& block_ptr : blocks_to_announce_) { diff --git a/silkworm/sync/messages/outbound_new_block.hpp b/silkworm/sync/messages/outbound_new_block.hpp index 5baf74dd93..cf743e0d36 100644 --- a/silkworm/sync/messages/outbound_new_block.hpp +++ b/silkworm/sync/messages/outbound_new_block.hpp @@ -33,7 +33,7 @@ class OutboundNewBlock : public OutboundMessage { std::string name() const override { return "OutboundNewBlock"; } std::string content() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; silkworm::sentry::eth::MessageId eth_message_id() const override { return silkworm::sentry::eth::MessageId::kNewBlock; diff --git a/silkworm/sync/messages/outbound_new_block_hashes.cpp b/silkworm/sync/messages/outbound_new_block_hashes.cpp index bcf9f74c8a..7db1f171b0 100644 --- a/silkworm/sync/messages/outbound_new_block_hashes.cpp +++ b/silkworm/sync/messages/outbound_new_block_hashes.cpp @@ -26,7 +26,7 @@ namespace silkworm { OutboundNewBlockHashes::OutboundNewBlockHashes(bool is_first_sync) : is_first_sync_{is_first_sync} {} -void OutboundNewBlockHashes::execute(db::ROAccess, HeaderChain& hc, BodySequence&, SentryClient& sentry) { +void OutboundNewBlockHashes::execute(db::DataStoreRef, HeaderChain& hc, BodySequence&, SentryClient& sentry) { auto& announces_to_do = hc.announces_to_do(); if (is_first_sync_) { diff --git a/silkworm/sync/messages/outbound_new_block_hashes.hpp b/silkworm/sync/messages/outbound_new_block_hashes.hpp index 728ab257bb..50fb5e9a03 100644 --- a/silkworm/sync/messages/outbound_new_block_hashes.hpp +++ b/silkworm/sync/messages/outbound_new_block_hashes.hpp @@ -29,7 +29,7 @@ class OutboundNewBlockHashes : public OutboundMessage { std::string name() const override { return "OutboundNewBlockHashes"; } std::string content() const override; - void execute(db::ROAccess, HeaderChain&, BodySequence&, SentryClient&) override; + void execute(db::DataStoreRef, HeaderChain&, BodySequence&, SentryClient&) override; silkworm::sentry::eth::MessageId eth_message_id() const override { return silkworm::sentry::eth::MessageId::kNewBlockHashes; diff --git a/silkworm/sync/sync.cpp b/silkworm/sync/sync.cpp index 96108581bd..699165d571 100644 --- a/silkworm/sync/sync.cpp +++ b/silkworm/sync/sync.cpp @@ -16,6 +16,8 @@ #include "sync.hpp" +#include + #include #include "sync_pos.hpp" @@ -24,14 +26,14 @@ namespace silkworm::chainsync { Sync::Sync(const boost::asio::any_io_executor& executor, - mdbx::env chaindata_env, + db::DataStoreRef data_store, execution::api::Client& execution, - const std::shared_ptr& sentry_client, + const std::shared_ptr& sentry_client, const ChainConfig& config, bool use_preverified_hashes, const EngineRpcSettings& rpc_settings) : sync_sentry_client_{executor, sentry_client}, - block_exchange_{sync_sentry_client_, db::ROAccess{chaindata_env}, config, use_preverified_hashes} { + block_exchange_{data_store, sync_sentry_client_, config, use_preverified_hashes} { // If terminal total difficulty is present in chain config, the network will use Proof-of-Stake sooner or later if (config.terminal_total_difficulty.has_value()) { // Configure and activate the Execution Layer Engine API RPC server @@ -51,7 +53,7 @@ Sync::Sync(const boost::asio::any_io_executor& executor, .num_workers = 1, // single-client so just one worker should be OK .jwt_secret_file = rpc_settings.jwt_secret_file, }; - engine_rpc_server_ = std::make_unique(engine_rpc_settings, chaindata_env); + engine_rpc_server_ = std::make_unique(engine_rpc_settings, data_store); // Create the synchronization algorithm based on Casper + LMD-GHOST, i.e. PoS auto pos_sync = std::make_shared(block_exchange_, execution); diff --git a/silkworm/sync/sync.hpp b/silkworm/sync/sync.hpp index ffb50aa6d9..20430223d4 100644 --- a/silkworm/sync/sync.hpp +++ b/silkworm/sync/sync.hpp @@ -25,11 +25,8 @@ #include #include -#include +#include #include -#include -#include -#include #include #include @@ -43,9 +40,9 @@ namespace silkworm::chainsync { class Sync { public: Sync(const boost::asio::any_io_executor& executor, - mdbx::env chaindata_env, + db::DataStoreRef data_store, execution::api::Client& execution, - const std::shared_ptr& sentry_client, + const std::shared_ptr& sentry_client, const ChainConfig& config, bool use_preverified_hashes, const EngineRpcSettings& rpc_settings = {}); diff --git a/silkworm/sync/sync_pos_test.cpp b/silkworm/sync/sync_pos_test.cpp index 7f09fa9d03..7d17a59340 100644 --- a/silkworm/sync/sync_pos_test.cpp +++ b/silkworm/sync/sync_pos_test.cpp @@ -25,6 +25,7 @@ #include #include +#include #include #include #include @@ -36,10 +37,14 @@ namespace silkworm::chainsync { class PoSSyncTest : public rpc::test_util::ServiceContextTestBase { public: + mdbx::env_managed chaindata_env; + snapshots::SnapshotRepository repository{db::test_util::make_repository()}; + db::DataStoreRef data_store{ + chaindata_env, // NOLINT(cppcoreguidelines-slicing) + repository, + }; SentryClient sentry_client{io_context_.get_executor(), nullptr}; // TODO(canepat) mock - mdbx::env_managed chaindata_env{}; - db::ROAccess db_access{chaindata_env}; - test_util::MockBlockExchange block_exchange{sentry_client, db_access, kSepoliaConfig}; + test_util::MockBlockExchange block_exchange{data_store, sentry_client, kSepoliaConfig}; std::shared_ptr execution_service{std::make_shared()}; test_util::MockExecutionClient execution_client{execution_service}; diff --git a/silkworm/sync/test_util/mock_block_exchange.hpp b/silkworm/sync/test_util/mock_block_exchange.hpp index b55c7a7b70..87465b68db 100644 --- a/silkworm/sync/test_util/mock_block_exchange.hpp +++ b/silkworm/sync/test_util/mock_block_exchange.hpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -35,8 +34,8 @@ namespace silkworm::chainsync::test_util { //! \brief MockBlockExchange is the gMock mock class for BlockExchange. class MockBlockExchange : public BlockExchange { public: - MockBlockExchange(SentryClient& client, const db::ROAccess& dba, const ChainConfig& config) - : BlockExchange(client, dba, config, /* use_preverified_hashes = */ false) {} + MockBlockExchange(db::DataStoreRef data_store, SentryClient& client, const ChainConfig& config) + : BlockExchange{std::move(data_store), client, config, /* use_preverified_hashes = */ false} {} MOCK_METHOD((void), initial_state, (std::vector));