0
0
Fork 0
mirror of https://github.com/bitcoin/bitcoin.git synced 2025-02-08 10:31:50 -05:00

assumeutxo cleanup: Move IsInitialBlockDownload & NotifyHeaderTip to ChainstateManager

This change makes IsInitialBlockDownload and NotifyHeaderTip functions no
longer tied to individual Chainstate objects. It makes them work with the
ChainstateManager object instead so code is simpler and it is no longer
possible to call them incorrectly with an inactive Chainstate.

This change also makes m_cached_finished_ibd caching easier to reason about,
because now there is only one cached value instead of two (for background and
snapshot chainstates) so the cached IBD state now no longer gets reset when a
snapshot is loaded.

There should be no change in behavior because these functions were always
called on the active ChainState objects.

These changes were discussed previously
https://github.com/bitcoin/bitcoin/pull/27746#discussion_r1246868905 and
https://github.com/bitcoin/bitcoin/pull/27746#discussion_r1237552792 as
possible followups for that PR.
This commit is contained in:
Ryan Ofsky 2023-08-04 16:43:39 -04:00
parent 9b066da8af
commit 94a98fbd1d
14 changed files with 80 additions and 67 deletions

View file

@ -163,7 +163,7 @@ int main(int argc, char* argv[])
<< "\t" << "Reindexing: " << std::boolalpha << node::fReindex.load() << std::noboolalpha << std::endl
<< "\t" << "Snapshot Active: " << std::boolalpha << chainman.IsSnapshotActive() << std::noboolalpha << std::endl
<< "\t" << "Active Height: " << chainman.ActiveHeight() << std::endl
<< "\t" << "Active IBD: " << std::boolalpha << chainman.ActiveChainstate().IsInitialBlockDownload() << std::noboolalpha << std::endl;
<< "\t" << "Active IBD: " << std::boolalpha << chainman.IsInitialBlockDownload() << std::noboolalpha << std::endl;
CBlockIndex* tip = chainman.ActiveTip();
if (tip) {
std::cout << "\t" << tip->ToString() << std::endl;

View file

@ -2010,7 +2010,7 @@ void PeerManagerImpl::BlockChecked(const CBlock& block, const BlockValidationSta
// the tip yet so we have no way to check this directly here. Instead we
// just check that there are currently no other blocks in flight.
else if (state.IsValid() &&
!m_chainman.ActiveChainstate().IsInitialBlockDownload() &&
!m_chainman.IsInitialBlockDownload() &&
mapBlocksInFlight.count(hash) == mapBlocksInFlight.size()) {
if (it != mapBlockSource.end()) {
MaybeSetPeerAsAnnouncingHeaderAndIDs(it->second.first);
@ -2729,7 +2729,7 @@ void PeerManagerImpl::UpdatePeerStateForReceivedHeaders(CNode& pfrom, Peer& peer
// If we're in IBD, we want outbound peers that will serve us a useful
// chain. Disconnect peers that are on chains with insufficient work.
if (m_chainman.ActiveChainstate().IsInitialBlockDownload() && !may_have_more_headers) {
if (m_chainman.IsInitialBlockDownload() && !may_have_more_headers) {
// If the peer has no more headers to give us, then we know we have
// their tip.
if (nodestate->pindexBestKnownBlock && nodestate->pindexBestKnownBlock->nChainWork < m_chainman.MinimumChainWork()) {
@ -3808,7 +3808,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
LogPrint(BCLog::NET, "got inv: %s %s peer=%d\n", inv.ToString(), fAlreadyHave ? "have" : "new", pfrom.GetId());
AddKnownTx(*peer, inv.hash);
if (!fAlreadyHave && !m_chainman.ActiveChainstate().IsInitialBlockDownload()) {
if (!fAlreadyHave && !m_chainman.IsInitialBlockDownload()) {
AddTxAnnouncement(pfrom, gtxid, current_time);
}
} else {
@ -4080,7 +4080,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
// Stop processing the transaction early if we are still in IBD since we don't
// have enough information to validate it yet. Sending unsolicited transactions
// is not considered a protocol violation, so don't punish the peer.
if (m_chainman.ActiveChainstate().IsInitialBlockDownload()) return;
if (m_chainman.IsInitialBlockDownload()) return;
CTransactionRef ptx;
vRecv >> ptx;
@ -4284,7 +4284,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
const CBlockIndex* prev_block = m_chainman.m_blockman.LookupBlockIndex(cmpctblock.header.hashPrevBlock);
if (!prev_block) {
// Doesn't connect (or is genesis), instead of DoSing in AcceptBlockHeader, request deeper headers
if (!m_chainman.ActiveChainstate().IsInitialBlockDownload()) {
if (!m_chainman.IsInitialBlockDownload()) {
MaybeSendGetHeaders(pfrom, GetLocator(m_chainman.m_best_header), *peer);
}
return;
@ -5228,7 +5228,7 @@ void PeerManagerImpl::MaybeSendAddr(CNode& node, Peer& peer, std::chrono::micros
LOCK(peer.m_addr_send_times_mutex);
// Periodically advertise our local address to the peer.
if (fListen && !m_chainman.ActiveChainstate().IsInitialBlockDownload() &&
if (fListen && !m_chainman.IsInitialBlockDownload() &&
peer.m_next_local_addr_send < current_time) {
// If we've sent before, clear the bloom filter for the peer, so that our
// self-announcement will actually go out.
@ -5323,7 +5323,7 @@ void PeerManagerImpl::MaybeSendFeefilter(CNode& pto, Peer& peer, std::chrono::mi
CAmount currentFilter = m_mempool.GetMinFee().GetFeePerK();
static FeeFilterRounder g_filter_rounder{CFeeRate{DEFAULT_MIN_RELAY_TX_FEE}};
if (m_chainman.ActiveChainstate().IsInitialBlockDownload()) {
if (m_chainman.IsInitialBlockDownload()) {
// Received tx-inv messages are discarded when the active
// chainstate is in IBD, so tell the peer to not send them.
currentFilter = MAX_MONEY;
@ -5827,7 +5827,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
// Message: getdata (blocks)
//
std::vector<CInv> vGetData;
if (CanServeBlocks(*peer) && ((sync_blocks_and_headers_from_peer && !IsLimitedPeer(*peer)) || !m_chainman.ActiveChainstate().IsInitialBlockDownload()) && state.vBlocksInFlight.size() < MAX_BLOCKS_IN_TRANSIT_PER_PEER) {
if (CanServeBlocks(*peer) && ((sync_blocks_and_headers_from_peer && !IsLimitedPeer(*peer)) || !m_chainman.IsInitialBlockDownload()) && state.vBlocksInFlight.size() < MAX_BLOCKS_IN_TRANSIT_PER_PEER) {
std::vector<const CBlockIndex*> vToDownload;
NodeId staller = -1;
FindNextBlocksToDownload(*peer, MAX_BLOCKS_IN_TRANSIT_PER_PEER - state.vBlocksInFlight.size(), vToDownload, staller);

View file

@ -298,8 +298,9 @@ public:
{
return GuessVerificationProgress(chainman().GetParams().TxData(), WITH_LOCK(::cs_main, return chainman().ActiveChain().Tip()));
}
bool isInitialBlockDownload() override {
return chainman().ActiveChainstate().IsInitialBlockDownload();
bool isInitialBlockDownload() override
{
return chainman().IsInitialBlockDownload();
}
bool isLoadingBlocks() override { return chainman().m_blockman.LoadingBlocks(); }
void setNetworkActive(bool active) override
@ -720,7 +721,7 @@ public:
bool isReadyToBroadcast() override { return !chainman().m_blockman.LoadingBlocks() && !isInitialBlockDownload(); }
bool isInitialBlockDownload() override
{
return chainman().ActiveChainstate().IsInitialBlockDownload();
return chainman().IsInitialBlockDownload();
}
bool shutdownRequested() override { return ShutdownRequested(); }
void initMessage(const std::string& message) override { ::uiInterface.InitMessage(message); }

View file

@ -1260,7 +1260,7 @@ RPCHelpMan getblockchaininfo()
obj.pushKV("time", tip.GetBlockTime());
obj.pushKV("mediantime", tip.GetMedianTimePast());
obj.pushKV("verificationprogress", GuessVerificationProgress(chainman.GetParams().TxData(), &tip));
obj.pushKV("initialblockdownload", active_chainstate.IsInitialBlockDownload());
obj.pushKV("initialblockdownload", chainman.IsInitialBlockDownload());
obj.pushKV("chainwork", tip.nChainWork.GetHex());
obj.pushKV("size_on_disk", chainman.m_blockman.CalculateCurrentUsage());
obj.pushKV("pruned", chainman.m_blockman.IsPruneMode());

View file

@ -752,9 +752,10 @@ static RPCHelpMan importmempool()
const NodeContext& node{EnsureAnyNodeContext(request.context)};
CTxMemPool& mempool{EnsureMemPool(node)};
Chainstate& chainstate = EnsureChainman(node).ActiveChainstate();
ChainstateManager& chainman = EnsureChainman(node);
Chainstate& chainstate = chainman.ActiveChainstate();
if (chainstate.IsInitialBlockDownload()) {
if (chainman.IsInitialBlockDownload()) {
throw JSONRPCError(RPC_CLIENT_IN_INITIAL_DOWNLOAD, "Can only import the mempool after the block download and sync is done.");
}

View file

@ -706,7 +706,7 @@ static RPCHelpMan getblocktemplate()
throw JSONRPCError(RPC_CLIENT_NOT_CONNECTED, PACKAGE_NAME " is not connected!");
}
if (active_chainstate.IsInitialBlockDownload()) {
if (chainman.IsInitialBlockDownload()) {
throw JSONRPCError(RPC_CLIENT_IN_INITIAL_DOWNLOAD, PACKAGE_NAME " is in initial sync and waiting for blocks...");
}
}

View file

@ -63,9 +63,9 @@ FUZZ_TARGET(process_message, .init = initialize_process_message)
FuzzedDataProvider fuzzed_data_provider(buffer.data(), buffer.size());
ConnmanTestMsg& connman = *static_cast<ConnmanTestMsg*>(g_setup->m_node.connman.get());
TestChainState& chainstate = *static_cast<TestChainState*>(&g_setup->m_node.chainman->ActiveChainstate());
auto& chainman = static_cast<TestChainstateManager&>(*g_setup->m_node.chainman);
SetMockTime(1610000000); // any time to successfully reset ibd
chainstate.ResetIbd();
chainman.ResetIbd();
LOCK(NetEventsInterface::g_msgproc_mutex);

View file

@ -38,9 +38,9 @@ FUZZ_TARGET(process_messages, .init = initialize_process_messages)
FuzzedDataProvider fuzzed_data_provider(buffer.data(), buffer.size());
ConnmanTestMsg& connman = *static_cast<ConnmanTestMsg*>(g_setup->m_node.connman.get());
TestChainState& chainstate = *static_cast<TestChainState*>(&g_setup->m_node.chainman->ActiveChainstate());
auto& chainman = static_cast<TestChainstateManager&>(*g_setup->m_node.chainman);
SetMockTime(1610000000); // any time to successfully reset ibd
chainstate.ResetIbd();
chainman.ResetIbd();
LOCK(NetEventsInterface::g_msgproc_mutex);

View file

@ -841,11 +841,10 @@ BOOST_AUTO_TEST_CASE(initial_advertise_from_version_message)
const int64_t time{0};
const CNetMsgMaker msg_maker{PROTOCOL_VERSION};
// Force Chainstate::IsInitialBlockDownload() to return false.
// Force ChainstateManager::IsInitialBlockDownload() to return false.
// Otherwise PushAddress() isn't called by PeerManager::ProcessMessage().
TestChainState& chainstate =
*static_cast<TestChainState*>(&m_node.chainman->ActiveChainstate());
chainstate.JumpOutOfIbd();
auto& chainman = static_cast<TestChainstateManager&>(*m_node.chainman);
chainman.JumpOutOfIbd();
m_node.peerman->InitializeNode(peer, NODE_NETWORK);
@ -895,7 +894,7 @@ BOOST_AUTO_TEST_CASE(initial_advertise_from_version_message)
BOOST_CHECK(sent);
CaptureMessage = CaptureMessageOrig;
chainstate.ResetIbd();
chainman.ResetIbd();
m_node.args->ForceSetArg("-capturemessages", "0");
m_node.args->ForceSetArg("-bind", "");
// PeerManager::ProcessMessage() calls AddTimeData() which changes the internal state

View file

@ -9,13 +9,13 @@
#include <validation.h>
#include <validationinterface.h>
void TestChainState::ResetIbd()
void TestChainstateManager::ResetIbd()
{
m_cached_finished_ibd = false;
assert(IsInitialBlockDownload());
}
void TestChainState::JumpOutOfIbd()
void TestChainstateManager::JumpOutOfIbd()
{
Assert(IsInitialBlockDownload());
m_cached_finished_ibd = true;

View file

@ -9,7 +9,7 @@
class CValidationInterface;
struct TestChainState : public Chainstate {
struct TestChainstateManager : public ChainstateManager {
/** Reset the ibd cache to its initial state */
void ResetIbd();
/** Toggle IsInitialBlockDownload from true to false */

View file

@ -13,6 +13,7 @@
#include <test/util/logging.h>
#include <test/util/random.h>
#include <test/util/setup_common.h>
#include <test/util/validation.h>
#include <timedata.h>
#include <uint256.h>
#include <validation.h>
@ -143,14 +144,21 @@ BOOST_FIXTURE_TEST_CASE(chainstatemanager_rebalance_caches, TestChain100Setup)
c2.InitCoinsDB(
/*cache_size_bytes=*/1 << 23, /*in_memory=*/true, /*should_wipe=*/false);
// Reset IBD state so IsInitialBlockDownload() returns true and causes
// MaybeRebalancesCaches() to prioritize the snapshot chainstate, giving it
// more cache space than the snapshot chainstate. Calling ResetIbd() is
// necessary because m_cached_finished_ibd is already latched to true before
// the test starts due to the test setup. After ResetIbd() is called.
// IsInitialBlockDownload will return true because at this point the active
// chainstate has a null chain tip.
static_cast<TestChainstateManager&>(manager).ResetIbd();
{
LOCK(::cs_main);
c2.InitCoinsCache(1 << 23);
manager.MaybeRebalanceCaches();
}
// Since both chainstates are considered to be in initial block download,
// the snapshot chainstate should take priority.
BOOST_CHECK_CLOSE(c1.m_coinstip_cache_size_bytes, max_cache * 0.05, 1);
BOOST_CHECK_CLOSE(c1.m_coinsdb_cache_size_bytes, max_cache * 0.05, 1);
BOOST_CHECK_CLOSE(c2.m_coinstip_cache_size_bytes, max_cache * 0.95, 1);

View file

@ -275,8 +275,9 @@ static void LimitMempoolSize(CTxMemPool& pool, CCoinsViewCache& coins_cache)
static bool IsCurrentForFeeEstimation(Chainstate& active_chainstate) EXCLUSIVE_LOCKS_REQUIRED(cs_main)
{
AssertLockHeld(cs_main);
if (active_chainstate.IsInitialBlockDownload())
if (active_chainstate.m_chainman.IsInitialBlockDownload()) {
return false;
}
if (active_chainstate.m_chain.Tip()->GetBlockTime() < count_seconds(GetTime<std::chrono::seconds>() - MAX_FEE_ESTIMATION_TIP_AGE))
return false;
if (active_chainstate.m_chain.Height() < active_chainstate.m_chainman.m_best_header->nHeight - 1) {
@ -1622,7 +1623,7 @@ void Chainstate::InitCoinsCache(size_t cache_size_bytes)
// `const` so that `CValidationInterface` clients (which are given a `const Chainstate*`)
// can call it.
//
bool Chainstate::IsInitialBlockDownload() const
bool ChainstateManager::IsInitialBlockDownload() const
{
// Optimization: pre-test latch before taking the lock.
if (m_cached_finished_ibd.load(std::memory_order_relaxed))
@ -1631,15 +1632,17 @@ bool Chainstate::IsInitialBlockDownload() const
LOCK(cs_main);
if (m_cached_finished_ibd.load(std::memory_order_relaxed))
return false;
if (m_chainman.m_blockman.LoadingBlocks()) {
if (m_blockman.LoadingBlocks()) {
return true;
}
if (m_chain.Tip() == nullptr)
return true;
if (m_chain.Tip()->nChainWork < m_chainman.MinimumChainWork()) {
CChain& chain{ActiveChain()};
if (chain.Tip() == nullptr) {
return true;
}
if (m_chain.Tip()->Time() < Now<NodeSeconds>() - m_chainman.m_options.max_tip_age) {
if (chain.Tip()->nChainWork < MinimumChainWork()) {
return true;
}
if (chain.Tip()->Time() < Now<NodeSeconds>() - m_options.max_tip_age) {
return true;
}
LogPrintf("Leaving InitialBlockDownload (latching to false)\n");
@ -1653,7 +1656,7 @@ void Chainstate::CheckForkWarningConditions()
// Before we get past initial download, we cannot reliably alert about forks
// (we assume we don't get stuck on a fork before finishing our initial sync)
if (IsInitialBlockDownload()) {
if (m_chainman.IsInitialBlockDownload()) {
return;
}
@ -2474,7 +2477,7 @@ bool Chainstate::FlushStateToDisk(
} else {
LOG_TIME_MILLIS_WITH_CATEGORY("find files to prune", BCLog::BENCH);
m_blockman.FindFilesToPrune(setFilesToPrune, m_chainman.GetParams().PruneAfterHeight(), m_chain.Height(), last_prune, IsInitialBlockDownload());
m_blockman.FindFilesToPrune(setFilesToPrune, m_chainman.GetParams().PruneAfterHeight(), m_chain.Height(), last_prune, m_chainman.IsInitialBlockDownload());
m_blockman.m_check_for_pruning = false;
}
if (!setFilesToPrune.empty()) {
@ -2643,7 +2646,7 @@ void Chainstate::UpdateTip(const CBlockIndex* pindexNew)
}
bilingual_str warning_messages;
if (!this->IsInitialBlockDownload()) {
if (!m_chainman.IsInitialBlockDownload()) {
const CBlockIndex* pindex = pindexNew;
for (int bit = 0; bit < VERSIONBITS_NUM_BITS; bit++) {
WarningBitsConditionChecker checker(m_chainman, bit);
@ -3055,24 +3058,25 @@ static SynchronizationState GetSynchronizationState(bool init)
return SynchronizationState::INIT_DOWNLOAD;
}
static bool NotifyHeaderTip(Chainstate& chainstate) LOCKS_EXCLUDED(cs_main) {
static bool NotifyHeaderTip(ChainstateManager& chainman) LOCKS_EXCLUDED(cs_main)
{
bool fNotify = false;
bool fInitialBlockDownload = false;
static CBlockIndex* pindexHeaderOld = nullptr;
CBlockIndex* pindexHeader = nullptr;
{
LOCK(cs_main);
pindexHeader = chainstate.m_chainman.m_best_header;
pindexHeader = chainman.m_best_header;
if (pindexHeader != pindexHeaderOld) {
fNotify = true;
fInitialBlockDownload = chainstate.IsInitialBlockDownload();
fInitialBlockDownload = chainman.IsInitialBlockDownload();
pindexHeaderOld = pindexHeader;
}
}
// Send block tip changed notifications without cs_main
if (fNotify) {
chainstate.m_chainman.GetNotifications().headerTip(GetSynchronizationState(fInitialBlockDownload), pindexHeader->nHeight, pindexHeader->nTime, false);
chainman.GetNotifications().headerTip(GetSynchronizationState(fInitialBlockDownload), pindexHeader->nHeight, pindexHeader->nTime, false);
}
return fNotify;
}
@ -3171,7 +3175,7 @@ bool Chainstate::ActivateBestChain(BlockValidationState& state, std::shared_ptr<
if (!blocks_connected) return true;
const CBlockIndex* pindexFork = m_chain.FindFork(starting_tip);
bool fInitialDownload = IsInitialBlockDownload();
bool fInitialDownload = m_chainman.IsInitialBlockDownload();
// Notify external listeners about the new tip.
// Enqueue while holding cs_main to ensure that UpdatedBlockTip is called in the order in which blocks are connected
@ -3389,7 +3393,7 @@ bool Chainstate::InvalidateBlock(BlockValidationState& state, CBlockIndex* pinde
// parameter indicating the source of the tip change so hooks can
// distinguish user-initiated invalidateblock changes from other
// changes.
(void)m_chainman.GetNotifications().blockTip(GetSynchronizationState(IsInitialBlockDownload()), *to_mark_failed->pprev);
(void)m_chainman.GetNotifications().blockTip(GetSynchronizationState(m_chainman.IsInitialBlockDownload()), *to_mark_failed->pprev);
}
return true;
}
@ -3874,7 +3878,7 @@ bool ChainstateManager::AcceptBlockHeader(const CBlockHeader& block, BlockValida
const auto msg = strprintf(
"Saw new header hash=%s height=%d", hash.ToString(), pindex->nHeight);
if (ActiveChainstate().IsInitialBlockDownload()) {
if (IsInitialBlockDownload()) {
LogPrintLevel(BCLog::VALIDATION, BCLog::Level::Debug, "%s\n", msg);
} else {
LogPrintf("%s\n", msg);
@ -3902,8 +3906,8 @@ bool ChainstateManager::ProcessNewBlockHeaders(const std::vector<CBlockHeader>&
}
}
}
if (NotifyHeaderTip(ActiveChainstate())) {
if (ActiveChainstate().IsInitialBlockDownload() && ppindex && *ppindex) {
if (NotifyHeaderTip(*this)) {
if (IsInitialBlockDownload() && ppindex && *ppindex) {
const CBlockIndex& last_accepted{**ppindex};
const int64_t blocks_left{(GetTime() - last_accepted.GetBlockTime()) / GetConsensus().nPowTargetSpacing};
const double progress{100.0 * last_accepted.nHeight / (last_accepted.nHeight + blocks_left)};
@ -3916,7 +3920,6 @@ bool ChainstateManager::ProcessNewBlockHeaders(const std::vector<CBlockHeader>&
void ChainstateManager::ReportHeadersPresync(const arith_uint256& work, int64_t height, int64_t timestamp)
{
AssertLockNotHeld(cs_main);
const auto& chainstate = ActiveChainstate();
{
LOCK(cs_main);
// Don't report headers presync progress if we already have a post-minchainwork header chain.
@ -3929,7 +3932,7 @@ void ChainstateManager::ReportHeadersPresync(const arith_uint256& work, int64_t
if (now < m_last_presync_update + std::chrono::milliseconds{250}) return;
m_last_presync_update = now;
}
bool initial_download = chainstate.IsInitialBlockDownload();
bool initial_download = IsInitialBlockDownload();
GetNotifications().headerTip(GetSynchronizationState(initial_download), height, timestamp, /*presync=*/true);
if (initial_download) {
const int64_t blocks_left{(GetTime() - timestamp) / GetConsensus().nPowTargetSpacing};
@ -4002,7 +4005,7 @@ bool ChainstateManager::AcceptBlock(const std::shared_ptr<const CBlock>& pblock,
// Header is valid/has work, merkle tree and segwit merkle tree are good...RELAY NOW
// (but if it does not build on our best tip, let the SendMessages loop relay it)
if (!ActiveChainstate().IsInitialBlockDownload() && ActiveTip() == pindex->pprev)
if (!IsInitialBlockDownload() && ActiveTip() == pindex->pprev)
GetMainSignals().NewPoWValidBlock(pindex, pblock);
// Write block to history file
@ -4061,7 +4064,7 @@ bool ChainstateManager::ProcessNewBlock(const std::shared_ptr<const CBlock>& blo
}
}
NotifyHeaderTip(ActiveChainstate());
NotifyHeaderTip(*this);
BlockValidationState state; // Only used to report errors, not invalidity - ignore it
if (!ActiveChainstate().ActivateBestChain(state, block)) {
@ -4629,7 +4632,7 @@ void ChainstateManager::LoadExternalBlockFile(
}
}
NotifyHeaderTip(ActiveChainstate());
NotifyHeaderTip(*this);
if (!blocks_with_unknown_parent) continue;
@ -4655,7 +4658,7 @@ void ChainstateManager::LoadExternalBlockFile(
}
range.first++;
blocks_with_unknown_parent->erase(it);
NotifyHeaderTip(ActiveChainstate());
NotifyHeaderTip(*this);
}
}
} catch (const std::exception& e) {
@ -5578,7 +5581,7 @@ void ChainstateManager::MaybeRebalanceCaches()
// If both chainstates exist, determine who needs more cache based on IBD status.
//
// Note: shrink caches first so that we don't inadvertently overwhelm available memory.
if (m_snapshot_chainstate->IsInitialBlockDownload()) {
if (IsInitialBlockDownload()) {
m_ibd_chainstate->ResizeCoinsCaches(
m_total_coinstip_cache * 0.05, m_total_coinsdb_cache * 0.05);
m_snapshot_chainstate->ResizeCoinsCaches(

View file

@ -473,14 +473,6 @@ protected:
*/
Mutex m_chainstate_mutex;
/**
* Whether this chainstate is undergoing initial block download.
*
* Mutable because we need to be able to mark IsInitialBlockDownload()
* const, which latches this for caching purposes.
*/
mutable std::atomic<bool> m_cached_finished_ibd{false};
//! Optional mempool that is kept in sync with the chain.
//! Only the active chainstate has a mempool.
CTxMemPool* m_mempool;
@ -707,9 +699,6 @@ public:
void ClearBlockIndexCandidates() EXCLUSIVE_LOCKS_REQUIRED(::cs_main);
/** Check whether we are doing an initial block download (synchronizing from disk or network) */
bool IsInitialBlockDownload() const;
/** Find the last common block of this chain and a locator. */
const CBlockIndex* FindForkInGlobalIndex(const CBlockLocator& locator) const EXCLUSIVE_LOCKS_REQUIRED(cs_main);
@ -951,6 +940,15 @@ public:
//! chainstate to avoid duplicating block metadata.
node::BlockManager m_blockman;
/**
* Whether initial block download has ended and IsInitialBlockDownload
* should return false from now on.
*
* Mutable because we need to be able to mark IsInitialBlockDownload()
* const, which latches this for caching purposes.
*/
mutable std::atomic<bool> m_cached_finished_ibd{false};
/**
* Every received block is assigned a unique and increasing identifier, so we
* know which one to give priority in case of a fork.
@ -1067,6 +1065,9 @@ public:
return m_snapshot_chainstate && m_ibd_chainstate && m_ibd_chainstate->m_disabled;
}
/** Check whether we are doing an initial block download (synchronizing from disk or network) */
bool IsInitialBlockDownload() const;
/**
* Import blocks from an external file
*