diff --git a/src/app/ClioApplication.cpp b/src/app/ClioApplication.cpp index 6f89a19a..23cc4dff 100644 --- a/src/app/ClioApplication.cpp +++ b/src/app/ClioApplication.cpp @@ -25,6 +25,7 @@ #include "data/AmendmentCenter.hpp" #include "data/BackendFactory.hpp" #include "data/LedgerCache.hpp" +#include "data/LedgerCacheSaver.hpp" #include "etl/ETLService.hpp" #include "etl/LoadBalancer.hpp" #include "etl/NetworkValidatedLedgers.hpp" @@ -98,14 +99,15 @@ ClioApplication::run(bool const useNgWebServer) auto const threads = config_.get("io_threads"); LOG(util::LogService::info()) << "Number of io threads = " << threads; + // Similarly we need a context to run ETL on + // In the future we can remove the raw ioc and use ctx instead + // This context should be above ioc because its reference is getting into tasks inside ioc + util::async::CoroExecutionContext ctx{threads}; + // IO context to handle all incoming requests, as well as other things. // This is not the only io context in the application. boost::asio::io_context ioc{threads}; - // Similarly we need a context to run ETL on - // In the future we can remove the raw ioc and use ctx instead - util::async::CoroExecutionContext ctx{threads}; - // Rate limiter, to prevent abuse auto whitelistHandler = web::dosguard::WhitelistHandler{config_}; auto const dosguardWeights = web::dosguard::Weights::make(config_); @@ -113,21 +115,7 @@ ClioApplication::run(bool const useNgWebServer) auto sweepHandler = web::dosguard::IntervalSweepHandler{config_, ioc, dosGuard}; auto cache = data::LedgerCache{}; - appStopper_.setOnStop([&cache, this](auto&&) { - // TODO(kuznetsss): move this into Stopper::makeOnStopCallback() - auto const cacheFilePath = config_.maybeValue("cache.file.path"); - if (not cacheFilePath.has_value()) { - return; - } - - LOG(util::LogService::info()) << "Saving ledger cache to " << *cacheFilePath; - if (auto const [success, duration_ms] = util::timed([&]() { return cache.saveToFile(*cacheFilePath); }); - success.has_value()) { - LOG(util::LogService::info()) << "Successfully saved ledger cache in " << duration_ms << " ms"; - } else { - LOG(util::LogService::error()) << "Error saving LedgerCache to file"; - } - }); + auto cacheSaver = data::LedgerCacheSaver{config_, cache}; // Interface to the database auto backend = data::makeBackend(config_, cache); @@ -208,7 +196,7 @@ ClioApplication::run(bool const useNgWebServer) } appStopper_.setOnStop( - Stopper::makeOnStopCallback(httpServer.value(), *balancer, *etl, *subscriptions, *backend, ioc) + Stopper::makeOnStopCallback(httpServer.value(), *balancer, *etl, *subscriptions, *backend, cacheSaver, ioc) ); // Blocks until stopped. @@ -223,6 +211,9 @@ ClioApplication::run(bool const useNgWebServer) auto handler = std::make_shared>(config_, backend, rpcEngine, etl, dosGuard); auto const httpServer = web::makeHttpServer(config_, ioc, dosGuard, handler, cache); + appStopper_.setOnStop( + Stopper::makeOnStopCallback(*httpServer, *balancer, *etl, *subscriptions, *backend, cacheSaver, ioc) + ); // Blocks until stopped. // When stopped, shared_ptrs fall out of scope diff --git a/src/app/Stopper.hpp b/src/app/Stopper.hpp index a89fabe9..883b2e0e 100644 --- a/src/app/Stopper.hpp +++ b/src/app/Stopper.hpp @@ -20,12 +20,13 @@ #pragma once #include "data/BackendInterface.hpp" +#include "data/LedgerCacheSaver.hpp" #include "etl/ETLServiceInterface.hpp" #include "etl/LoadBalancerInterface.hpp" #include "feed/SubscriptionManagerInterface.hpp" #include "util/CoroutineGroup.hpp" #include "util/log/Logger.hpp" -#include "web/ng/Server.hpp" +#include "web/interface/Concepts.hpp" #include #include @@ -71,10 +72,11 @@ public: * @param etl The ETL service to stop. * @param subscriptions The subscription manager to stop. * @param backend The backend to stop. + * @param cacheSaver The ledger cache saver * @param ioc The io_context to stop. * @return The callback to be called on application stop. */ - template + template static std::function makeOnStopCallback( ServerType& server, @@ -82,10 +84,13 @@ public: etl::ETLServiceInterface& etl, feed::SubscriptionManagerInterface& subscriptions, data::BackendInterface& backend, + LedgerCacheSaverType& cacheSaver, boost::asio::io_context& ioc ) { return [&](boost::asio::yield_context yield) { + cacheSaver.save(); + util::CoroutineGroup coroutineGroup{yield}; coroutineGroup.spawn(yield, [&server](auto innerYield) { server.stop(innerYield); @@ -106,6 +111,8 @@ public: backend.waitForWritesToFinish(); LOG(util::LogService::info()) << "Backend writes finished"; + cacheSaver.waitToFinish(); + ioc.stop(); LOG(util::LogService::info()) << "io_context stopped"; diff --git a/src/data/CMakeLists.txt b/src/data/CMakeLists.txt index 5661040a..678defe7 100644 --- a/src/data/CMakeLists.txt +++ b/src/data/CMakeLists.txt @@ -5,6 +5,7 @@ target_sources( BackendCounters.cpp BackendInterface.cpp LedgerCache.cpp + LedgerCacheSaver.cpp LedgerHeaderCache.cpp cassandra/impl/Future.cpp cassandra/impl/Cluster.cpp diff --git a/src/data/LedgerCache.cpp b/src/data/LedgerCache.cpp index 9842fd8e..b4874862 100644 --- a/src/data/LedgerCache.cpp +++ b/src/data/LedgerCache.cpp @@ -265,7 +265,7 @@ LedgerCache::saveToFile(std::string const& path) const } impl::LedgerCacheFile file{path}; - std::unique_lock const lock{mtx_}; + std::shared_lock const lock{mtx_}; impl::LedgerCacheFile::DataView const data{.latestSeq = latestSeq_, .map = map_, .deleted = deleted_}; return file.write(data); } diff --git a/src/data/LedgerCache.hpp b/src/data/LedgerCache.hpp index ef195f53..b6346f47 100644 --- a/src/data/LedgerCache.hpp +++ b/src/data/LedgerCache.hpp @@ -145,15 +145,8 @@ public: void waitUntilCacheContainsSeq(uint32_t seq) override; - /** - * @brief Save the cache to file - * @note This operation takes about 7 seconds and it keeps mtx_ exclusively locked - * - * @param path The file path to save the cache to - * @return An error as a string if any - */ std::expected - saveToFile(std::string const& path) const; + saveToFile(std::string const& path) const override; std::expected loadFromFile(std::string const& path, uint32_t minLatestSequence) override; diff --git a/src/data/LedgerCacheInterface.hpp b/src/data/LedgerCacheInterface.hpp index 563bef35..d2ff7e48 100644 --- a/src/data/LedgerCacheInterface.hpp +++ b/src/data/LedgerCacheInterface.hpp @@ -171,6 +171,16 @@ public: virtual void waitUntilCacheContainsSeq(uint32_t seq) = 0; + /** + * @brief Save the cache to file + * @note This operation takes about 7 seconds and it keeps a shared lock of mtx_ + * + * @param path The file path to save the cache to + * @return An error as a string if any + */ + [[nodiscard]] virtual std::expected + saveToFile(std::string const& path) const = 0; + /** * @brief Load the cache from file * @note This operation takes about 7 seconds and it keeps mtx_ exclusively locked diff --git a/src/data/LedgerCacheSaver.cpp b/src/data/LedgerCacheSaver.cpp new file mode 100644 index 00000000..fef2f5cb --- /dev/null +++ b/src/data/LedgerCacheSaver.cpp @@ -0,0 +1,70 @@ +//------------------------------------------------------------------------------ +/* + This file is part of clio: https://github.com/XRPLF/clio + Copyright (c) 2025, the clio developers. + + Permission to use, copy, modify, and distribute this software for any + purpose with or without fee is hereby granted, provided that the above + copyright notice and this permission notice appear in all copies. + + THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. +*/ +//============================================================================== + +#include "data/LedgerCacheSaver.hpp" + +#include "data/LedgerCacheInterface.hpp" +#include "util/Assert.hpp" +#include "util/Profiler.hpp" +#include "util/log/Logger.hpp" + +#include +#include + +namespace data { + +LedgerCacheSaver::LedgerCacheSaver(util::config::ClioConfigDefinition const& config, LedgerCacheInterface const& cache) + : cacheFilePath_(config.maybeValue("cache.file.path")), cache_(cache) +{ +} + +LedgerCacheSaver::~LedgerCacheSaver() +{ + waitToFinish(); +} + +void +LedgerCacheSaver::save() +{ + ASSERT(not savingThread_.has_value(), "Multiple save() calls are not allowed"); + savingThread_ = std::thread([this]() { + if (not cacheFilePath_.has_value()) { + return; + } + + LOG(util::LogService::info()) << "Saving ledger cache to " << *cacheFilePath_; + if (auto const [success, durationMs] = util::timed([&]() { return cache_.get().saveToFile(*cacheFilePath_); }); + success.has_value()) { + LOG(util::LogService::info()) << "Successfully saved ledger cache in " << durationMs << " ms"; + } else { + LOG(util::LogService::error()) << "Error saving LedgerCache to file"; + } + }); +} + +void +LedgerCacheSaver::waitToFinish() +{ + if (savingThread_.has_value() and savingThread_->joinable()) { + savingThread_->join(); + } + savingThread_.reset(); +} + +} // namespace data diff --git a/src/data/LedgerCacheSaver.hpp b/src/data/LedgerCacheSaver.hpp new file mode 100644 index 00000000..7cf68f1d --- /dev/null +++ b/src/data/LedgerCacheSaver.hpp @@ -0,0 +1,93 @@ +//------------------------------------------------------------------------------ +/* + This file is part of clio: https://github.com/XRPLF/clio + Copyright (c) 2025, the clio developers. + + Permission to use, copy, modify, and distribute this software for any + purpose with or without fee is hereby granted, provided that the above + copyright notice and this permission notice appear in all copies. + + THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. +*/ +//============================================================================== + +#pragma once + +#include "data/LedgerCacheInterface.hpp" +#include "util/config/ConfigDefinition.hpp" + +#include +#include +#include +#include +#include + +namespace data { + +/** + * @brief A concept for a class that can save ledger cache asynchronously. + * + * This concept defines the interface requirements for any type that manages + * asynchronous saving of ledger cache to persistent storage. + */ +template +concept SomeLedgerCacheSaver = requires(T a) { + { a.save() } -> std::same_as; + { a.waitToFinish() } -> std::same_as; +}; + +/** + * @brief Manages asynchronous saving of ledger cache to a file. + * + * This class provides functionality to save the ledger cache to a file in a separate thread, + * allowing the main application to continue without blocking. The file path is configured + * through the application's configuration system. + */ +class LedgerCacheSaver { + std::optional cacheFilePath_; + std::reference_wrapper cache_; + std::optional savingThread_; + +public: + /** + * @brief Constructs a LedgerCacheSaver instance. + * + * @param config The configuration object containing the cache file path setting + * @param cache Reference to the ledger cache interface to be saved + */ + LedgerCacheSaver(util::config::ClioConfigDefinition const& config, LedgerCacheInterface const& cache); + + /** + * @brief Destructor that ensures the saving thread is properly joined. + * + * Waits for any ongoing save operation to complete before destruction. + */ + ~LedgerCacheSaver(); + + /** + * @brief Initiates an asynchronous save operation of the ledger cache. + * + * Spawns a new thread that saves the ledger cache to the configured file path. + * If no file path is configured, the operation is skipped. Logs the progress + * and result of the save operation. + */ + void + save(); + + /** + * @brief Waits for the saving thread to complete. + * + * Blocks until the saving operation finishes if a thread is currently active. + * Safe to call multiple times or when no save operation is in progress. + */ + void + waitToFinish(); +}; + +} // namespace data diff --git a/src/util/StopHelper.cpp b/src/util/StopHelper.cpp index 928773d0..733e3585 100644 --- a/src/util/StopHelper.cpp +++ b/src/util/StopHelper.cpp @@ -19,6 +19,8 @@ #include "util/StopHelper.hpp" +#include "util/Spawn.hpp" + #include #include @@ -37,7 +39,7 @@ void StopHelper::asyncWaitForStop(boost::asio::yield_context yield) { boost::asio::steady_timer timer{yield.get_executor(), std::chrono::steady_clock::duration::max()}; - onStopReady_.connect([&timer]() { timer.cancel(); }); + onStopReady_.connect([&]() { util::spawn(yield, [&timer](auto&&) { timer.cancel(); }); }); boost::system::error_code error; if (!*stopped_) timer.async_wait(yield[error]); diff --git a/src/util/StopHelper.hpp b/src/util/StopHelper.hpp index 226352b4..bb788ba0 100644 --- a/src/util/StopHelper.hpp +++ b/src/util/StopHelper.hpp @@ -36,6 +36,16 @@ class StopHelper { std::unique_ptr stopped_ = std::make_unique(false); public: + StopHelper() = default; + ~StopHelper() = default; + + StopHelper(StopHelper&&) = delete; + StopHelper& + operator=(StopHelper&&) = delete; + StopHelper(StopHelper const&) = delete; + StopHelper& + operator=(StopHelper const&) = delete; + /** * @brief Notify that the class is ready to stop. */ diff --git a/src/web/Server.hpp b/src/web/Server.hpp index e245eea0..8e2a3d24 100644 --- a/src/web/Server.hpp +++ b/src/web/Server.hpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,7 @@ #include #include +#include #include #include #include @@ -221,7 +223,8 @@ template < template class PlainSessionType, template class SslSessionType, SomeServerHandler HandlerType> -class Server : public std::enable_shared_from_this> { +class Server : public ServerTag, + public std::enable_shared_from_this> { using std::enable_shared_from_this>::shared_from_this; util::Logger log_{"WebServer"}; @@ -235,6 +238,7 @@ class Server : public std::enable_shared_from_this adminVerification_; std::uint32_t maxWsSendingQueueSize_; std::shared_ptr proxyIpResolver_; + std::atomic_bool isStopped_{false}; public: /** @@ -308,6 +312,13 @@ public: doAccept(); } + /** @brief Stop accepting new connections */ + void + stop(boost::asio::yield_context) + { + isStopped_ = true; + } + private: void doAccept() @@ -321,6 +332,10 @@ private: void onAccept(boost::beast::error_code ec, tcp::socket socket) { + if (isStopped_) { + return; + } + if (!ec) { auto ctxRef = ctx_ ? std::optional>{ctx_.value()} : std::nullopt; diff --git a/src/web/interface/Concepts.hpp b/src/web/interface/Concepts.hpp index b80285c2..7f6ff35f 100644 --- a/src/web/interface/Concepts.hpp +++ b/src/web/interface/Concepts.hpp @@ -22,6 +22,7 @@ #include #include +#include #include #include @@ -39,4 +40,14 @@ concept SomeServerHandler = { handler(req, ws) }; }; +/** + * @brief A tag class for server to help identify Server in templated code. + */ +struct ServerTag { + virtual ~ServerTag() = default; +}; + +template +concept SomeServer = std::derived_from; + } // namespace web diff --git a/src/web/ng/Server.hpp b/src/web/ng/Server.hpp index 166059d8..2df2fdac 100644 --- a/src/web/ng/Server.hpp +++ b/src/web/ng/Server.hpp @@ -23,6 +23,7 @@ #include "util/config/ConfigDefinition.hpp" #include "util/log/Logger.hpp" #include "web/ProxyIpResolver.hpp" +#include "web/interface/Concepts.hpp" #include "web/ng/Connection.hpp" #include "web/ng/MessageHandler.hpp" #include "web/ng/ProcessingPolicy.hpp" @@ -34,7 +35,6 @@ #include #include -#include #include #include #include @@ -42,16 +42,6 @@ namespace web::ng { -/** - * @brief A tag class for server to help identify Server in templated code. - */ -struct ServerTag { - virtual ~ServerTag() = default; -}; - -template -concept SomeServer = std::derived_from; - /** * @brief Web server class. */ diff --git a/tests/common/util/MockLedgerCache.hpp b/tests/common/util/MockLedgerCache.hpp index f4f09259..70bac942 100644 --- a/tests/common/util/MockLedgerCache.hpp +++ b/tests/common/util/MockLedgerCache.hpp @@ -79,6 +79,9 @@ struct MockLedgerCache : data::LedgerCacheInterface { MOCK_METHOD(void, waitUntilCacheContainsSeq, (uint32_t), (override)); + using SaveToFileReturnType = std::expected; + MOCK_METHOD(SaveToFileReturnType, saveToFile, (std::string const& path), (const, override)); + using LoadFromFileReturnType = std::expected; MOCK_METHOD( LoadFromFileReturnType, diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt index 7a572c67..cff464b7 100644 --- a/tests/unit/CMakeLists.txt +++ b/tests/unit/CMakeLists.txt @@ -11,6 +11,7 @@ target_sources( data/BackendCountersTests.cpp data/BackendInterfaceTests.cpp data/LedgerCacheTests.cpp + data/LedgerCacheSaverTests.cpp data/cassandra/AsyncExecutorTests.cpp data/cassandra/ExecutionStrategyTests.cpp data/cassandra/LedgerHeaderCacheTests.cpp diff --git a/tests/unit/app/StopperTests.cpp b/tests/unit/app/StopperTests.cpp index c2fc0a2b..0dd5845b 100644 --- a/tests/unit/app/StopperTests.cpp +++ b/tests/unit/app/StopperTests.cpp @@ -24,7 +24,7 @@ #include "util/MockPrometheus.hpp" #include "util/MockSubscriptionManager.hpp" #include "util/config/ConfigDefinition.hpp" -#include "web/ng/Server.hpp" +#include "web/interface/Concepts.hpp" #include #include @@ -61,16 +61,22 @@ TEST_F(StopperTest, stopCalledMultipleTimes) } struct StopperMakeCallbackTest : util::prometheus::WithPrometheus, SyncAsioContextTest { - struct ServerMock : web::ng::ServerTag { + struct ServerMock : web::ServerTag { MOCK_METHOD(void, stop, (boost::asio::yield_context), ()); }; + struct MockLedgerCacheSaver { + MOCK_METHOD(void, save, ()); + MOCK_METHOD(void, waitToFinish, ()); + }; + protected: testing::StrictMock serverMock_; testing::StrictMock loadBalancerMock_; testing::StrictMock etlServiceMock_; testing::StrictMock subscriptionManagerMock_; testing::StrictMock backendMock_{util::config::ClioConfigDefinition{}}; + testing::StrictMock cacheSaverMock_; boost::asio::io_context ioContextToStop_; bool @@ -86,10 +92,17 @@ TEST_F(StopperMakeCallbackTest, makeCallbackTest) std::thread t{[this]() { ioContextToStop_.run(); }}; auto callback = Stopper::makeOnStopCallback( - serverMock_, loadBalancerMock_, etlServiceMock_, subscriptionManagerMock_, backendMock_, ioContextToStop_ + serverMock_, + loadBalancerMock_, + etlServiceMock_, + subscriptionManagerMock_, + backendMock_, + cacheSaverMock_, + ioContextToStop_ ); testing::Sequence const s1, s2; + EXPECT_CALL(cacheSaverMock_, save).InSequence(s1).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); }); EXPECT_CALL(serverMock_, stop).InSequence(s1).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); }); EXPECT_CALL(loadBalancerMock_, stop).InSequence(s2).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); }); EXPECT_CALL(etlServiceMock_, stop).InSequence(s1, s2).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); }); @@ -99,6 +112,7 @@ TEST_F(StopperMakeCallbackTest, makeCallbackTest) EXPECT_CALL(backendMock_, waitForWritesToFinish).InSequence(s1, s2).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); }); + EXPECT_CALL(cacheSaverMock_, waitToFinish).InSequence(s1).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); }); runSpawn([&](boost::asio::yield_context yield) { callback(yield); diff --git a/tests/unit/data/LedgerCacheSaverTests.cpp b/tests/unit/data/LedgerCacheSaverTests.cpp new file mode 100644 index 00000000..782792eb --- /dev/null +++ b/tests/unit/data/LedgerCacheSaverTests.cpp @@ -0,0 +1,156 @@ +//------------------------------------------------------------------------------ +/* + This file is part of clio: https://github.com/XRPLF/clio + Copyright (c) 2025, the clio developers. + + Permission to use, copy, modify, and distribute this software for any + purpose with or without fee is hereby granted, provided that the above + copyright notice and this permission notice appear in all copies. + + THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. +*/ +//============================================================================== + +#include "data/LedgerCacheSaver.hpp" +#include "util/MockAssert.hpp" +#include "util/MockLedgerCache.hpp" +#include "util/config/ConfigDefinition.hpp" +#include "util/config/ConfigFileJson.hpp" +#include "util/config/ConfigValue.hpp" +#include "util/config/Types.hpp" + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +using namespace data; +using namespace util::config; + +struct LedgerCacheSaverTest : virtual testing::Test { + testing::StrictMock cache; + constexpr static auto kFILE_PATH = "./cache.bin"; + + static ClioConfigDefinition + generateConfig(bool cacheFilePathHasValue) + { + auto config = ClioConfigDefinition{{ + {"cache.file.path", ConfigValue{ConfigType::String}.optional()}, + }}; + + ConfigFileJson jsonFile{boost::json::object{}}; + if (cacheFilePathHasValue) { + auto const jsonObject = + boost::json::parse(fmt::format(R"JSON({{"cache": {{"file": {{"path": "{}"}}}}}})JSON", kFILE_PATH)) + .as_object(); + jsonFile = ConfigFileJson{jsonObject}; + } + auto const errors = config.parse(jsonFile); + EXPECT_FALSE(errors.has_value()); + return config; + } +}; + +TEST_F(LedgerCacheSaverTest, SaveSuccessfully) +{ + auto const config = generateConfig(true); + LedgerCacheSaver saver{config, cache}; + + EXPECT_CALL(cache, saveToFile(kFILE_PATH)).WillOnce(testing::Return(std::expected{})); + + saver.save(); + saver.waitToFinish(); +} + +TEST_F(LedgerCacheSaverTest, SaveWithError) +{ + auto const config = generateConfig(true); + LedgerCacheSaver saver{config, cache}; + + EXPECT_CALL(cache, saveToFile(kFILE_PATH)) + .WillOnce(testing::Return(std::expected(std::unexpected("Failed to save")))); + + saver.save(); + saver.waitToFinish(); +} + +TEST_F(LedgerCacheSaverTest, NoSaveWhenPathNotConfigured) +{ + auto const config = generateConfig(false); + + LedgerCacheSaver saver{config, cache}; + saver.save(); + saver.waitToFinish(); +} + +TEST_F(LedgerCacheSaverTest, DestructorWaitsForCompletion) +{ + auto const config = generateConfig(true); + + std::binary_semaphore semaphore{1}; + std::atomic_bool saveCompleted{false}; + + EXPECT_CALL(cache, saveToFile(kFILE_PATH)).WillOnce([&]() { + semaphore.release(); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + saveCompleted = true; + return std::expected{}; + }); + + { + LedgerCacheSaver saver{config, cache}; + saver.save(); + EXPECT_TRUE(semaphore.try_acquire_for(std::chrono::seconds{5})); + } + + EXPECT_TRUE(saveCompleted); +} + +TEST_F(LedgerCacheSaverTest, WaitToFinishCanBeCalledMultipleTimes) +{ + auto const config = generateConfig(true); + LedgerCacheSaver saver{config, cache}; + + EXPECT_CALL(cache, saveToFile(kFILE_PATH)); + + saver.save(); + saver.waitToFinish(); + EXPECT_NO_THROW(saver.waitToFinish()); +} + +TEST_F(LedgerCacheSaverTest, WaitToFinishWithoutSaveIsSafe) +{ + auto const config = generateConfig(true); + LedgerCacheSaver saver{config, cache}; + EXPECT_NO_THROW(saver.waitToFinish()); +} + +struct LedgerCacheSaverAssertTest : LedgerCacheSaverTest, common::util::WithMockAssert {}; + +TEST_F(LedgerCacheSaverAssertTest, MultipleSavesNotAllowed) +{ + auto const config = generateConfig(true); + + LedgerCacheSaver saver{config, cache}; + + EXPECT_CALL(cache, saveToFile(kFILE_PATH)); + saver.save(); + EXPECT_CLIO_ASSERT_FAIL({ saver.save(); }); + + saver.waitToFinish(); +}