mirror of
https://github.com/XRPLF/rippled.git
synced 2025-11-27 14:35:52 +00:00
Simplify the Job Queue:
This is a refactor aimed at cleaning up and simplifying the existing job queue. As of now, all jobs are cancelled at the same time and in the same way, so this commit removes the per-job cancellation token. If the need for such support is demonstrated, support can be re-added. * Revise documentation for ClosureCounter and Workers. * Simplify code, removing unnecessary function arguments and deduplicating expressions * Restructure job handlers to no longer need to pass a job's handle to the job.
This commit is contained in:
committed by
Nik Bougalis
parent
df02eb125f
commit
c2a08a1f26
@@ -131,9 +131,7 @@ RCLConsensus::Adaptor::acquireLedger(LedgerHash const& hash)
|
|||||||
acquiringLedger_ = hash;
|
acquiringLedger_ = hash;
|
||||||
|
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtADVANCE,
|
jtADVANCE, "getConsensusLedger", [id = hash, &app = app_]() {
|
||||||
"getConsensusLedger",
|
|
||||||
[id = hash, &app = app_](Job&) {
|
|
||||||
app.getInboundLedgers().acquire(
|
app.getInboundLedgers().acquire(
|
||||||
id, 0, InboundLedger::Reason::CONSENSUS);
|
id, 0, InboundLedger::Reason::CONSENSUS);
|
||||||
});
|
});
|
||||||
@@ -423,9 +421,7 @@ RCLConsensus::Adaptor::onAccept(
|
|||||||
Json::Value&& consensusJson)
|
Json::Value&& consensusJson)
|
||||||
{
|
{
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtACCEPT,
|
jtACCEPT, "acceptLedger", [=, cj = std::move(consensusJson)]() mutable {
|
||||||
"acceptLedger",
|
|
||||||
[=, cj = std::move(consensusJson)](auto&) mutable {
|
|
||||||
// Note that no lock is held or acquired during this job.
|
// Note that no lock is held or acquired during this job.
|
||||||
// This is because generic Consensus guarantees that once a ledger
|
// This is because generic Consensus guarantees that once a ledger
|
||||||
// is accepted, the consensus results and capture by reference state
|
// is accepted, the consensus results and capture by reference state
|
||||||
|
|||||||
@@ -135,7 +135,7 @@ RCLValidationsAdaptor::acquire(LedgerHash const& hash)
|
|||||||
Application* pApp = &app_;
|
Application* pApp = &app_;
|
||||||
|
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtADVANCE, "getConsensusLedger", [pApp, hash](Job&) {
|
jtADVANCE, "getConsensusLedger", [pApp, hash]() {
|
||||||
pApp->getInboundLedgers().acquire(
|
pApp->getInboundLedgers().acquire(
|
||||||
hash, 0, InboundLedger::Reason::CONSENSUS);
|
hash, 0, InboundLedger::Reason::CONSENSUS);
|
||||||
});
|
});
|
||||||
|
|||||||
@@ -62,10 +62,9 @@ ConsensusTransSetSF::gotNode(
|
|||||||
auto stx = std::make_shared<STTx const>(std::ref(sit));
|
auto stx = std::make_shared<STTx const>(std::ref(sit));
|
||||||
assert(stx->getTransactionID() == nodeHash.as_uint256());
|
assert(stx->getTransactionID() == nodeHash.as_uint256());
|
||||||
auto const pap = &app_;
|
auto const pap = &app_;
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(jtTRANSACTION, "TXS->TXN", [pap, stx]() {
|
||||||
jtTRANSACTION, "TXS->TXN", [pap, stx](Job&) {
|
pap->getOPs().submitTransaction(stx);
|
||||||
pap->getOPs().submitTransaction(stx);
|
});
|
||||||
});
|
|
||||||
}
|
}
|
||||||
catch (std::exception const&)
|
catch (std::exception const&)
|
||||||
{
|
{
|
||||||
|
|||||||
@@ -981,10 +981,9 @@ pendSaveValidated(
|
|||||||
|
|
||||||
// See if we can use the JobQueue.
|
// See if we can use the JobQueue.
|
||||||
if (!isSynchronous &&
|
if (!isSynchronous &&
|
||||||
app.getJobQueue().addJob(
|
app.getJobQueue().addJob(jobType, jobName, [&app, ledger, isCurrent]() {
|
||||||
jobType, jobName, [&app, ledger, isCurrent](Job&) {
|
saveValidatedLedger(app, ledger, isCurrent);
|
||||||
saveValidatedLedger(app, ledger, isCurrent);
|
}))
|
||||||
}))
|
|
||||||
{
|
{
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -301,7 +301,7 @@ private:
|
|||||||
setPubLedger(std::shared_ptr<Ledger const> const& l);
|
setPubLedger(std::shared_ptr<Ledger const> const& l);
|
||||||
|
|
||||||
void
|
void
|
||||||
tryFill(Job& job, std::shared_ptr<Ledger const> ledger);
|
tryFill(std::shared_ptr<Ledger const> ledger);
|
||||||
|
|
||||||
void
|
void
|
||||||
getFetchPack(LedgerIndex missing, InboundLedger::Reason reason);
|
getFetchPack(LedgerIndex missing, InboundLedger::Reason reason);
|
||||||
@@ -326,7 +326,7 @@ private:
|
|||||||
findNewLedgersToPublish(std::unique_lock<std::recursive_mutex>&);
|
findNewLedgersToPublish(std::unique_lock<std::recursive_mutex>&);
|
||||||
|
|
||||||
void
|
void
|
||||||
updatePaths(Job& job);
|
updatePaths();
|
||||||
|
|
||||||
// Returns true if work started. Always called with m_mutex locked.
|
// Returns true if work started. Always called with m_mutex locked.
|
||||||
// The passed lock is a reminder to callers.
|
// The passed lock is a reminder to callers.
|
||||||
|
|||||||
@@ -68,7 +68,7 @@ OrderBookDB::setup(std::shared_ptr<ReadView const> const& ledger)
|
|||||||
update(ledger);
|
update(ledger);
|
||||||
else
|
else
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtUPDATE_PF, "OrderBookDB::update", [this, ledger](Job&) {
|
jtUPDATE_PF, "OrderBookDB::update", [this, ledger]() {
|
||||||
update(ledger);
|
update(ledger);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -527,7 +527,7 @@ InboundLedger::done()
|
|||||||
|
|
||||||
// We hold the PeerSet lock, so must dispatch
|
// We hold the PeerSet lock, so must dispatch
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtLEDGER_DATA, "AcquisitionDone", [self = shared_from_this()](Job&) {
|
jtLEDGER_DATA, "AcquisitionDone", [self = shared_from_this()]() {
|
||||||
if (self->complete_ && !self->failed_)
|
if (self->complete_ && !self->failed_)
|
||||||
{
|
{
|
||||||
self->app_.getLedgerMaster().checkAccept(self->getLedger());
|
self->app_.getLedgerMaster().checkAccept(self->getLedger());
|
||||||
|
|||||||
@@ -183,7 +183,7 @@ public:
|
|||||||
// dispatch
|
// dispatch
|
||||||
if (ledger->gotData(std::weak_ptr<Peer>(peer), packet))
|
if (ledger->gotData(std::weak_ptr<Peer>(peer), packet))
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtLEDGER_DATA, "processLedgerData", [ledger](Job&) {
|
jtLEDGER_DATA, "processLedgerData", [ledger]() {
|
||||||
ledger->runData();
|
ledger->runData();
|
||||||
});
|
});
|
||||||
|
|
||||||
@@ -198,7 +198,7 @@ public:
|
|||||||
if (packet->type() == protocol::liAS_NODE)
|
if (packet->type() == protocol::liAS_NODE)
|
||||||
{
|
{
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtLEDGER_DATA, "gotStaleData", [this, packet](Job&) {
|
jtLEDGER_DATA, "gotStaleData", [this, packet]() {
|
||||||
gotStaleData(packet);
|
gotStaleData(packet);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -240,7 +240,7 @@ LedgerDeltaAcquire::onLedgerBuilt(
|
|||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtREPLAY_TASK,
|
jtREPLAY_TASK,
|
||||||
"onLedgerBuilt",
|
"onLedgerBuilt",
|
||||||
[=, ledger = this->fullLedger_, &app = this->app_](Job&) {
|
[=, ledger = this->fullLedger_, &app = this->app_]() {
|
||||||
for (auto reason : reasons)
|
for (auto reason : reasons)
|
||||||
{
|
{
|
||||||
switch (reason)
|
switch (reason)
|
||||||
|
|||||||
@@ -699,7 +699,7 @@ LedgerMaster::getEarliestFetch()
|
|||||||
}
|
}
|
||||||
|
|
||||||
void
|
void
|
||||||
LedgerMaster::tryFill(Job& job, std::shared_ptr<Ledger const> ledger)
|
LedgerMaster::tryFill(std::shared_ptr<Ledger const> ledger)
|
||||||
{
|
{
|
||||||
std::uint32_t seq = ledger->info().seq;
|
std::uint32_t seq = ledger->info().seq;
|
||||||
uint256 prevHash = ledger->info().parentHash;
|
uint256 prevHash = ledger->info().parentHash;
|
||||||
@@ -710,7 +710,7 @@ LedgerMaster::tryFill(Job& job, std::shared_ptr<Ledger const> ledger)
|
|||||||
std::uint32_t maxHas = seq;
|
std::uint32_t maxHas = seq;
|
||||||
|
|
||||||
NodeStore::Database& nodeStore{app_.getNodeStore()};
|
NodeStore::Database& nodeStore{app_.getNodeStore()};
|
||||||
while (!job.shouldCancel() && seq > 0)
|
while (!app_.getJobQueue().isStopping() && seq > 0)
|
||||||
{
|
{
|
||||||
{
|
{
|
||||||
std::lock_guard ml(m_mutex);
|
std::lock_guard ml(m_mutex);
|
||||||
@@ -1453,7 +1453,7 @@ LedgerMaster::tryAdvance()
|
|||||||
if (!mAdvanceThread && !mValidLedger.empty())
|
if (!mAdvanceThread && !mValidLedger.empty())
|
||||||
{
|
{
|
||||||
mAdvanceThread = true;
|
mAdvanceThread = true;
|
||||||
app_.getJobQueue().addJob(jtADVANCE, "advanceLedger", [this](Job&) {
|
app_.getJobQueue().addJob(jtADVANCE, "advanceLedger", [this]() {
|
||||||
std::unique_lock sl(m_mutex);
|
std::unique_lock sl(m_mutex);
|
||||||
|
|
||||||
assert(!mValidLedger.empty() && mAdvanceThread);
|
assert(!mValidLedger.empty() && mAdvanceThread);
|
||||||
@@ -1476,7 +1476,7 @@ LedgerMaster::tryAdvance()
|
|||||||
}
|
}
|
||||||
|
|
||||||
void
|
void
|
||||||
LedgerMaster::updatePaths(Job& job)
|
LedgerMaster::updatePaths()
|
||||||
{
|
{
|
||||||
{
|
{
|
||||||
std::lock_guard ml(m_mutex);
|
std::lock_guard ml(m_mutex);
|
||||||
@@ -1487,7 +1487,7 @@ LedgerMaster::updatePaths(Job& job)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
while (!job.shouldCancel())
|
while (!app_.getJobQueue().isStopping())
|
||||||
{
|
{
|
||||||
std::shared_ptr<ReadView const> lastLedger;
|
std::shared_ptr<ReadView const> lastLedger;
|
||||||
{
|
{
|
||||||
@@ -1527,8 +1527,7 @@ LedgerMaster::updatePaths(Job& job)
|
|||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
app_.getPathRequests().updateAll(
|
app_.getPathRequests().updateAll(lastLedger);
|
||||||
lastLedger, job.getCancelCallback());
|
|
||||||
}
|
}
|
||||||
catch (SHAMapMissingNode const& mn)
|
catch (SHAMapMissingNode const& mn)
|
||||||
{
|
{
|
||||||
@@ -1591,7 +1590,7 @@ LedgerMaster::newPFWork(
|
|||||||
if (mPathFindThread < 2)
|
if (mPathFindThread < 2)
|
||||||
{
|
{
|
||||||
if (app_.getJobQueue().addJob(
|
if (app_.getJobQueue().addJob(
|
||||||
jtUPDATE_PF, name, [this](Job& j) { updatePaths(j); }))
|
jtUPDATE_PF, name, [this]() { updatePaths(); }))
|
||||||
{
|
{
|
||||||
++mPathFindThread;
|
++mPathFindThread;
|
||||||
}
|
}
|
||||||
@@ -1942,8 +1941,8 @@ LedgerMaster::fetchForHistory(
|
|||||||
mFillInProgress = seq;
|
mFillInProgress = seq;
|
||||||
}
|
}
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtADVANCE, "tryFill", [this, ledger](Job& j) {
|
jtADVANCE, "tryFill", [this, ledger]() {
|
||||||
tryFill(j, ledger);
|
tryFill(ledger);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -2124,7 +2123,7 @@ LedgerMaster::gotFetchPack(bool progress, std::uint32_t seq)
|
|||||||
{
|
{
|
||||||
if (!mGotFetchPackThread.test_and_set(std::memory_order_acquire))
|
if (!mGotFetchPackThread.test_and_set(std::memory_order_acquire))
|
||||||
{
|
{
|
||||||
app_.getJobQueue().addJob(jtLEDGER_DATA, "gotFetchPack", [&](Job&) {
|
app_.getJobQueue().addJob(jtLEDGER_DATA, "gotFetchPack", [&]() {
|
||||||
app_.getInboundLedgers().gotFetchPack();
|
app_.getInboundLedgers().gotFetchPack();
|
||||||
mGotFetchPackThread.clear(std::memory_order_release);
|
mGotFetchPackThread.clear(std::memory_order_release);
|
||||||
});
|
});
|
||||||
|
|||||||
@@ -83,7 +83,7 @@ TimeoutCounter::queueJob(ScopedLockType& sl)
|
|||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
queueJobParameter_.jobType,
|
queueJobParameter_.jobType,
|
||||||
queueJobParameter_.jobName,
|
queueJobParameter_.jobName,
|
||||||
[wptr = pmDowncast()](Job&) {
|
[wptr = pmDowncast()]() {
|
||||||
if (auto sptr = wptr.lock(); sptr)
|
if (auto sptr = wptr.lock(); sptr)
|
||||||
sptr->invokeOnTimer();
|
sptr->invokeOnTimer();
|
||||||
});
|
});
|
||||||
|
|||||||
@@ -81,7 +81,7 @@ TransactionAcquire::done()
|
|||||||
// just updates the consensus and related structures when we acquire
|
// just updates the consensus and related structures when we acquire
|
||||||
// a transaction set. No need to update them if we're shutting down.
|
// a transaction set. No need to update them if we're shutting down.
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtTXN_DATA, "completeAcquire", [pap, hash, map](Job&) {
|
jtTXN_DATA, "completeAcquire", [pap, hash, map]() {
|
||||||
pap->getInboundTransactions().giveSet(hash, map, true);
|
pap->getInboundTransactions().giveSet(hash, map, true);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -1085,7 +1085,7 @@ public:
|
|||||||
if (e.value() == boost::system::errc::success)
|
if (e.value() == boost::system::errc::success)
|
||||||
{
|
{
|
||||||
m_jobQueue->addJob(
|
m_jobQueue->addJob(
|
||||||
jtSWEEP, "sweep", [this](Job&) { doSweep(); });
|
jtSWEEP, "sweep", [this]() { doSweep(); });
|
||||||
}
|
}
|
||||||
// Recover as best we can if an unexpected error occurs.
|
// Recover as best we can if an unexpected error occurs.
|
||||||
if (e.value() != boost::system::errc::success &&
|
if (e.value() != boost::system::errc::success &&
|
||||||
|
|||||||
@@ -32,7 +32,7 @@ NodeStoreScheduler::scheduleTask(NodeStore::Task& task)
|
|||||||
if (jobQueue_.isStopped())
|
if (jobQueue_.isStopped())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
if (!jobQueue_.addJob(jtWRITE, "NodeObject::store", [&task](Job&) {
|
if (!jobQueue_.addJob(jtWRITE, "NodeObject::store", [&task]() {
|
||||||
task.performScheduledTask();
|
task.performScheduledTask();
|
||||||
}))
|
}))
|
||||||
{
|
{
|
||||||
|
|||||||
@@ -949,7 +949,7 @@ NetworkOPsImp::setHeartbeatTimer()
|
|||||||
heartbeatTimer_,
|
heartbeatTimer_,
|
||||||
mConsensus.parms().ledgerGRANULARITY,
|
mConsensus.parms().ledgerGRANULARITY,
|
||||||
[this]() {
|
[this]() {
|
||||||
m_job_queue.addJob(jtNETOP_TIMER, "NetOPs.heartbeat", [this](Job&) {
|
m_job_queue.addJob(jtNETOP_TIMER, "NetOPs.heartbeat", [this]() {
|
||||||
processHeartbeatTimer();
|
processHeartbeatTimer();
|
||||||
});
|
});
|
||||||
},
|
},
|
||||||
@@ -964,7 +964,7 @@ NetworkOPsImp::setClusterTimer()
|
|||||||
clusterTimer_,
|
clusterTimer_,
|
||||||
10s,
|
10s,
|
||||||
[this]() {
|
[this]() {
|
||||||
m_job_queue.addJob(jtNETOP_CLUSTER, "NetOPs.cluster", [this](Job&) {
|
m_job_queue.addJob(jtNETOP_CLUSTER, "NetOPs.cluster", [this]() {
|
||||||
processClusterTimer();
|
processClusterTimer();
|
||||||
});
|
});
|
||||||
},
|
},
|
||||||
@@ -1153,7 +1153,7 @@ NetworkOPsImp::submitTransaction(std::shared_ptr<STTx const> const& iTrans)
|
|||||||
|
|
||||||
auto tx = std::make_shared<Transaction>(trans, reason, app_);
|
auto tx = std::make_shared<Transaction>(trans, reason, app_);
|
||||||
|
|
||||||
m_job_queue.addJob(jtTRANSACTION, "submitTxn", [this, tx](Job&) {
|
m_job_queue.addJob(jtTRANSACTION, "submitTxn", [this, tx]() {
|
||||||
auto t = tx;
|
auto t = tx;
|
||||||
processTransaction(t, false, false, FailHard::no);
|
processTransaction(t, false, false, FailHard::no);
|
||||||
});
|
});
|
||||||
@@ -1224,9 +1224,8 @@ NetworkOPsImp::doTransactionAsync(
|
|||||||
|
|
||||||
if (mDispatchState == DispatchState::none)
|
if (mDispatchState == DispatchState::none)
|
||||||
{
|
{
|
||||||
if (m_job_queue.addJob(jtBATCH, "transactionBatch", [this](Job&) {
|
if (m_job_queue.addJob(
|
||||||
transactionBatch();
|
jtBATCH, "transactionBatch", [this]() { transactionBatch(); }))
|
||||||
}))
|
|
||||||
{
|
{
|
||||||
mDispatchState = DispatchState::scheduled;
|
mDispatchState = DispatchState::scheduled;
|
||||||
}
|
}
|
||||||
@@ -1262,10 +1261,9 @@ NetworkOPsImp::doTransactionSync(
|
|||||||
if (mTransactions.size())
|
if (mTransactions.size())
|
||||||
{
|
{
|
||||||
// More transactions need to be applied, but by another job.
|
// More transactions need to be applied, but by another job.
|
||||||
if (m_job_queue.addJob(
|
if (m_job_queue.addJob(jtBATCH, "transactionBatch", [this]() {
|
||||||
jtBATCH, "transactionBatch", [this](Job&) {
|
transactionBatch();
|
||||||
transactionBatch();
|
}))
|
||||||
}))
|
|
||||||
{
|
{
|
||||||
mDispatchState = DispatchState::scheduled;
|
mDispatchState = DispatchState::scheduled;
|
||||||
}
|
}
|
||||||
@@ -2941,7 +2939,7 @@ NetworkOPsImp::reportFeeChange()
|
|||||||
if (f != mLastFeeSummary)
|
if (f != mLastFeeSummary)
|
||||||
{
|
{
|
||||||
m_job_queue.addJob(
|
m_job_queue.addJob(
|
||||||
jtCLIENT_FEE_CHANGE, "reportFeeChange->pubServer", [this](Job&) {
|
jtCLIENT_FEE_CHANGE, "reportFeeChange->pubServer", [this]() {
|
||||||
pubServer();
|
pubServer();
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
@@ -2953,7 +2951,7 @@ NetworkOPsImp::reportConsensusStateChange(ConsensusPhase phase)
|
|||||||
m_job_queue.addJob(
|
m_job_queue.addJob(
|
||||||
jtCLIENT_CONSENSUS,
|
jtCLIENT_CONSENSUS,
|
||||||
"reportConsensusStateChange->pubConsensus",
|
"reportConsensusStateChange->pubConsensus",
|
||||||
[this, phase](Job&) { pubConsensus(phase); });
|
[this, phase]() { pubConsensus(phase); });
|
||||||
}
|
}
|
||||||
|
|
||||||
inline void
|
inline void
|
||||||
@@ -3346,7 +3344,7 @@ NetworkOPsImp::addAccountHistoryJob(SubAccountHistoryInfoWeak subInfo)
|
|||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtCLIENT_ACCT_HIST,
|
jtCLIENT_ACCT_HIST,
|
||||||
"AccountHistoryTxStream",
|
"AccountHistoryTxStream",
|
||||||
[this, dbType = databaseType, subInfo](Job&) {
|
[this, dbType = databaseType, subInfo]() {
|
||||||
auto const& accountId = subInfo.index_->accountId_;
|
auto const& accountId = subInfo.index_->accountId_;
|
||||||
auto& lastLedgerSeq = subInfo.index_->historyLastLedgerSeq_;
|
auto& lastLedgerSeq = subInfo.index_->historyLastLedgerSeq_;
|
||||||
auto& txHistoryIndex = subInfo.index_->historyTxIndex_;
|
auto& txHistoryIndex = subInfo.index_->historyTxIndex_;
|
||||||
|
|||||||
@@ -55,9 +55,7 @@ PathRequests::getLineCache(
|
|||||||
}
|
}
|
||||||
|
|
||||||
void
|
void
|
||||||
PathRequests::updateAll(
|
PathRequests::updateAll(std::shared_ptr<ReadView const> const& inLedger)
|
||||||
std::shared_ptr<ReadView const> const& inLedger,
|
|
||||||
Job::CancelCallback shouldCancel)
|
|
||||||
{
|
{
|
||||||
auto event =
|
auto event =
|
||||||
app_.getJobQueue().makeLoadEvent(jtPATH_FIND, "PathRequest::updateAll");
|
app_.getJobQueue().makeLoadEvent(jtPATH_FIND, "PathRequest::updateAll");
|
||||||
@@ -84,7 +82,7 @@ PathRequests::updateAll(
|
|||||||
{
|
{
|
||||||
for (auto const& wr : requests)
|
for (auto const& wr : requests)
|
||||||
{
|
{
|
||||||
if (shouldCancel())
|
if (app_.getJobQueue().isStopping())
|
||||||
break;
|
break;
|
||||||
|
|
||||||
auto request = wr.lock();
|
auto request = wr.lock();
|
||||||
@@ -174,7 +172,7 @@ PathRequests::updateAll(
|
|||||||
requests = requests_;
|
requests = requests_;
|
||||||
cache = getLineCache(cache->getLedger(), false);
|
cache = getLineCache(cache->getLedger(), false);
|
||||||
}
|
}
|
||||||
} while (!shouldCancel());
|
} while (!app_.getJobQueue().isStopping());
|
||||||
|
|
||||||
JLOG(mJournal.debug()) << "updateAll complete: " << processed
|
JLOG(mJournal.debug()) << "updateAll complete: " << processed
|
||||||
<< " processed and " << removed << " removed";
|
<< " processed and " << removed << " removed";
|
||||||
|
|||||||
@@ -47,12 +47,9 @@ public:
|
|||||||
/** Update all of the contained PathRequest instances.
|
/** Update all of the contained PathRequest instances.
|
||||||
|
|
||||||
@param ledger Ledger we are pathfinding in.
|
@param ledger Ledger we are pathfinding in.
|
||||||
@param shouldCancel Invocable that returns whether to cancel.
|
|
||||||
*/
|
*/
|
||||||
void
|
void
|
||||||
updateAll(
|
updateAll(std::shared_ptr<ReadView const> const& ledger);
|
||||||
std::shared_ptr<ReadView const> const& ledger,
|
|
||||||
Job::CancelCallback shouldCancel);
|
|
||||||
|
|
||||||
std::shared_ptr<RippleLineCache>
|
std::shared_ptr<RippleLineCache>
|
||||||
getLineCache(
|
getLineCache(
|
||||||
|
|||||||
@@ -31,21 +31,21 @@ namespace ripple {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* The role of a `ClosureCounter` is to assist in shutdown by letting callers
|
* The role of a `ClosureCounter` is to assist in shutdown by letting callers
|
||||||
* wait for the completion of callbacks (of a single type signature) that they
|
* wait for the completion of closures (of a specific type signature) that they
|
||||||
* previously scheduled. The lifetime of a `ClosureCounter` consists of two
|
* previously registered. These closures are typically callbacks for
|
||||||
|
* asynchronous operations. The lifetime of a `ClosureCounter` consists of two
|
||||||
* phases: the initial expanding "fork" phase, and the subsequent shrinking
|
* phases: the initial expanding "fork" phase, and the subsequent shrinking
|
||||||
* "join" phase.
|
* "join" phase.
|
||||||
*
|
*
|
||||||
* In the fork phase, callers register a callback by passing the callback and
|
* In the fork phase, callers register a closure by passing the closure and
|
||||||
* receiving a substitute in return. The substitute has the same callable
|
* receiving a substitute in return. The substitute has the same callable
|
||||||
* interface as the callback, and it informs the `ClosureCounter` whenever it
|
* interface as the closure, and it informs the `ClosureCounter` whenever it
|
||||||
* is copied or destroyed, so that it can keep an accurate count of copies.
|
* is copied or destroyed, so that it can keep an accurate count of copies.
|
||||||
*
|
*
|
||||||
* The transition to the join phase is made by a call to `join`. In this
|
* The transition to the join phase is made by a call to `join`. In this
|
||||||
* phase, every substitute returned going forward will be empty, signaling to
|
* phase, every substitute returned going forward will be null, signaling to
|
||||||
* the caller that they should just drop the callback and cancel their
|
* the caller that they should drop the closure and cancel their operation.
|
||||||
* asynchronous operation. `join` blocks until all existing callback
|
* `join` blocks until all existing closure substitutes are destroyed.
|
||||||
* substitutes are destroyed.
|
|
||||||
*
|
*
|
||||||
* \tparam Ret_t The return type of the closure.
|
* \tparam Ret_t The return type of the closure.
|
||||||
* \tparam Args_t The argument types of the closure.
|
* \tparam Args_t The argument types of the closure.
|
||||||
|
|||||||
@@ -77,7 +77,7 @@ JobQueue::Coro::post()
|
|||||||
|
|
||||||
// sp keeps 'this' alive
|
// sp keeps 'this' alive
|
||||||
if (jq_.addJob(
|
if (jq_.addJob(
|
||||||
type_, name_, [this, sp = shared_from_this()](Job&) { resume(); }))
|
type_, name_, [this, sp = shared_from_this()]() { resume(); }))
|
||||||
{
|
{
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -109,43 +109,25 @@ public:
|
|||||||
//
|
//
|
||||||
Job();
|
Job();
|
||||||
|
|
||||||
// Job (Job const& other);
|
|
||||||
|
|
||||||
Job(JobType type, std::uint64_t index);
|
Job(JobType type, std::uint64_t index);
|
||||||
|
|
||||||
/** A callback used to check for canceling a job. */
|
|
||||||
using CancelCallback = std::function<bool(void)>;
|
|
||||||
|
|
||||||
// VFALCO TODO try to remove the dependency on LoadMonitor.
|
// VFALCO TODO try to remove the dependency on LoadMonitor.
|
||||||
Job(JobType type,
|
Job(JobType type,
|
||||||
std::string const& name,
|
std::string const& name,
|
||||||
std::uint64_t index,
|
std::uint64_t index,
|
||||||
LoadMonitor& lm,
|
LoadMonitor& lm,
|
||||||
std::function<void(Job&)> const& job,
|
std::function<void()> const& job);
|
||||||
CancelCallback cancelCallback);
|
|
||||||
|
|
||||||
// Job& operator= (Job const& other);
|
|
||||||
|
|
||||||
JobType
|
JobType
|
||||||
getType() const;
|
getType() const;
|
||||||
|
|
||||||
CancelCallback
|
|
||||||
getCancelCallback() const;
|
|
||||||
|
|
||||||
/** Returns the time when the job was queued. */
|
/** Returns the time when the job was queued. */
|
||||||
clock_type::time_point const&
|
clock_type::time_point const&
|
||||||
queue_time() const;
|
queue_time() const;
|
||||||
|
|
||||||
/** Returns `true` if the running job should make a best-effort cancel. */
|
|
||||||
bool
|
|
||||||
shouldCancel() const;
|
|
||||||
|
|
||||||
void
|
void
|
||||||
doJob();
|
doJob();
|
||||||
|
|
||||||
void
|
|
||||||
rename(std::string const& n);
|
|
||||||
|
|
||||||
// These comparison operators make the jobs sort in priority order
|
// These comparison operators make the jobs sort in priority order
|
||||||
// in the job set
|
// in the job set
|
||||||
bool
|
bool
|
||||||
@@ -158,16 +140,15 @@ public:
|
|||||||
operator>=(const Job& j) const;
|
operator>=(const Job& j) const;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
CancelCallback m_cancelCallback;
|
|
||||||
JobType mType;
|
JobType mType;
|
||||||
std::uint64_t mJobIndex;
|
std::uint64_t mJobIndex;
|
||||||
std::function<void(Job&)> mJob;
|
std::function<void()> mJob;
|
||||||
std::shared_ptr<LoadEvent> m_loadEvent;
|
std::shared_ptr<LoadEvent> m_loadEvent;
|
||||||
std::string mName;
|
std::string mName;
|
||||||
clock_type::time_point m_queue_time;
|
clock_type::time_point m_queue_time;
|
||||||
};
|
};
|
||||||
|
|
||||||
using JobCounter = ClosureCounter<void, Job&>;
|
using JobCounter = ClosureCounter<void>;
|
||||||
|
|
||||||
} // namespace ripple
|
} // namespace ripple
|
||||||
|
|
||||||
|
|||||||
@@ -138,7 +138,7 @@ public:
|
|||||||
join();
|
join();
|
||||||
};
|
};
|
||||||
|
|
||||||
using JobFunction = std::function<void(Job&)>;
|
using JobFunction = std::function<void()>;
|
||||||
|
|
||||||
JobQueue(
|
JobQueue(
|
||||||
int threadCount,
|
int threadCount,
|
||||||
@@ -160,7 +160,7 @@ public:
|
|||||||
template <
|
template <
|
||||||
typename JobHandler,
|
typename JobHandler,
|
||||||
typename = std::enable_if_t<std::is_same<
|
typename = std::enable_if_t<std::is_same<
|
||||||
decltype(std::declval<JobHandler&&>()(std::declval<Job&>())),
|
decltype(std::declval<JobHandler&&>()()),
|
||||||
void>::value>>
|
void>::value>>
|
||||||
bool
|
bool
|
||||||
addJob(JobType type, std::string const& name, JobHandler&& jobHandler)
|
addJob(JobType type, std::string const& name, JobHandler&& jobHandler)
|
||||||
@@ -259,7 +259,6 @@ private:
|
|||||||
int nSuspend_ = 0;
|
int nSuspend_ = 0;
|
||||||
|
|
||||||
Workers m_workers;
|
Workers m_workers;
|
||||||
Job::CancelCallback m_cancelCallback;
|
|
||||||
|
|
||||||
// Statistics tracking
|
// Statistics tracking
|
||||||
perf::PerfLog& perfLog_;
|
perf::PerfLog& perfLog_;
|
||||||
@@ -288,23 +287,6 @@ private:
|
|||||||
std::string const& name,
|
std::string const& name,
|
||||||
JobFunction const& func);
|
JobFunction const& func);
|
||||||
|
|
||||||
// Signals an added Job for processing.
|
|
||||||
//
|
|
||||||
// Pre-conditions:
|
|
||||||
// The JobType must be valid.
|
|
||||||
// The Job must exist in mJobSet.
|
|
||||||
// The Job must not have previously been queued.
|
|
||||||
//
|
|
||||||
// Post-conditions:
|
|
||||||
// Count of waiting jobs of that type will be incremented.
|
|
||||||
// If JobQueue exists, and has at least one thread, Job will eventually
|
|
||||||
// run.
|
|
||||||
//
|
|
||||||
// Invariants:
|
|
||||||
// The calling thread owns the JobLock
|
|
||||||
void
|
|
||||||
queueJob(Job const& job, std::lock_guard<std::mutex> const& lock);
|
|
||||||
|
|
||||||
// Returns the next Job we should run now.
|
// Returns the next Job we should run now.
|
||||||
//
|
//
|
||||||
// RunnableJob:
|
// RunnableJob:
|
||||||
|
|||||||
@@ -36,10 +36,8 @@ Job::Job(
|
|||||||
std::string const& name,
|
std::string const& name,
|
||||||
std::uint64_t index,
|
std::uint64_t index,
|
||||||
LoadMonitor& lm,
|
LoadMonitor& lm,
|
||||||
std::function<void(Job&)> const& job,
|
std::function<void()> const& job)
|
||||||
CancelCallback cancelCallback)
|
: mType(type)
|
||||||
: m_cancelCallback(cancelCallback)
|
|
||||||
, mType(type)
|
|
||||||
, mJobIndex(index)
|
, mJobIndex(index)
|
||||||
, mJob(job)
|
, mJob(job)
|
||||||
, mName(name)
|
, mName(name)
|
||||||
@@ -54,27 +52,12 @@ Job::getType() const
|
|||||||
return mType;
|
return mType;
|
||||||
}
|
}
|
||||||
|
|
||||||
Job::CancelCallback
|
|
||||||
Job::getCancelCallback() const
|
|
||||||
{
|
|
||||||
assert(m_cancelCallback);
|
|
||||||
return m_cancelCallback;
|
|
||||||
}
|
|
||||||
|
|
||||||
Job::clock_type::time_point const&
|
Job::clock_type::time_point const&
|
||||||
Job::queue_time() const
|
Job::queue_time() const
|
||||||
{
|
{
|
||||||
return m_queue_time;
|
return m_queue_time;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool
|
|
||||||
Job::shouldCancel() const
|
|
||||||
{
|
|
||||||
if (m_cancelCallback)
|
|
||||||
return m_cancelCallback();
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
void
|
void
|
||||||
Job::doJob()
|
Job::doJob()
|
||||||
{
|
{
|
||||||
@@ -82,19 +65,13 @@ Job::doJob()
|
|||||||
m_loadEvent->start();
|
m_loadEvent->start();
|
||||||
m_loadEvent->setName(mName);
|
m_loadEvent->setName(mName);
|
||||||
|
|
||||||
mJob(*this);
|
mJob();
|
||||||
|
|
||||||
// Destroy the lambda, otherwise we won't include
|
// Destroy the lambda, otherwise we won't include
|
||||||
// its duration in the time measurement
|
// its duration in the time measurement
|
||||||
mJob = nullptr;
|
mJob = nullptr;
|
||||||
}
|
}
|
||||||
|
|
||||||
void
|
|
||||||
Job::rename(std::string const& newName)
|
|
||||||
{
|
|
||||||
mName = newName;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool
|
bool
|
||||||
Job::operator>(const Job& j) const
|
Job::operator>(const Job& j) const
|
||||||
{
|
{
|
||||||
|
|||||||
@@ -35,7 +35,6 @@ JobQueue::JobQueue(
|
|||||||
, m_invalidJobData(JobTypes::instance().getInvalid(), collector, logs)
|
, m_invalidJobData(JobTypes::instance().getInvalid(), collector, logs)
|
||||||
, m_processCount(0)
|
, m_processCount(0)
|
||||||
, m_workers(*this, &perfLog, "JobQueue", threadCount)
|
, m_workers(*this, &perfLog, "JobQueue", threadCount)
|
||||||
, m_cancelCallback(std::bind(&JobQueue::isStopping, this))
|
|
||||||
, perfLog_(perfLog)
|
, perfLog_(perfLog)
|
||||||
, m_collector(collector)
|
, m_collector(collector)
|
||||||
{
|
{
|
||||||
@@ -100,9 +99,27 @@ JobQueue::addRefCountedJob(
|
|||||||
|
|
||||||
{
|
{
|
||||||
std::lock_guard lock(m_mutex);
|
std::lock_guard lock(m_mutex);
|
||||||
auto result = m_jobSet.emplace(
|
auto result =
|
||||||
type, name, ++m_lastJob, data.load(), func, m_cancelCallback);
|
m_jobSet.emplace(type, name, ++m_lastJob, data.load(), func);
|
||||||
queueJob(*result.first, lock);
|
auto const& job = *result.first;
|
||||||
|
|
||||||
|
JobType const type(job.getType());
|
||||||
|
assert(type != jtINVALID);
|
||||||
|
assert(m_jobSet.find(job) != m_jobSet.end());
|
||||||
|
perfLog_.jobQueue(type);
|
||||||
|
|
||||||
|
JobTypeData& data(getJobTypeData(type));
|
||||||
|
|
||||||
|
if (data.waiting + data.running < getJobLimit(type))
|
||||||
|
{
|
||||||
|
m_workers.addTask();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
// defer the task until we go below the limit
|
||||||
|
++data.deferred;
|
||||||
|
}
|
||||||
|
++data.waiting;
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@@ -282,29 +299,6 @@ JobQueue::isStopped() const
|
|||||||
return stopped_;
|
return stopped_;
|
||||||
}
|
}
|
||||||
|
|
||||||
void
|
|
||||||
JobQueue::queueJob(Job const& job, std::lock_guard<std::mutex> const& lock)
|
|
||||||
{
|
|
||||||
JobType const type(job.getType());
|
|
||||||
assert(type != jtINVALID);
|
|
||||||
assert(m_jobSet.find(job) != m_jobSet.end());
|
|
||||||
perfLog_.jobQueue(type);
|
|
||||||
|
|
||||||
JobTypeData& data(getJobTypeData(type));
|
|
||||||
|
|
||||||
if (data.waiting + data.running < getJobLimit(type))
|
|
||||||
{
|
|
||||||
m_workers.addTask();
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
// defer the task until we go below the limit
|
|
||||||
//
|
|
||||||
++data.deferred;
|
|
||||||
}
|
|
||||||
++data.waiting;
|
|
||||||
}
|
|
||||||
|
|
||||||
void
|
void
|
||||||
JobQueue::getNextJob(Job& job)
|
JobQueue::getNextJob(Job& job)
|
||||||
{
|
{
|
||||||
@@ -313,30 +307,25 @@ JobQueue::getNextJob(Job& job)
|
|||||||
std::set<Job>::const_iterator iter;
|
std::set<Job>::const_iterator iter;
|
||||||
for (iter = m_jobSet.begin(); iter != m_jobSet.end(); ++iter)
|
for (iter = m_jobSet.begin(); iter != m_jobSet.end(); ++iter)
|
||||||
{
|
{
|
||||||
JobTypeData& data(getJobTypeData(iter->getType()));
|
JobType const type = iter->getType();
|
||||||
|
assert(type != jtINVALID);
|
||||||
|
|
||||||
assert(data.running <= getJobLimit(data.type()));
|
JobTypeData& data(getJobTypeData(type));
|
||||||
|
assert(data.running <= getJobLimit(type));
|
||||||
|
|
||||||
// Run this job if we're running below the limit.
|
// Run this job if we're running below the limit.
|
||||||
if (data.running < getJobLimit(data.type()))
|
if (data.running < getJobLimit(data.type()))
|
||||||
{
|
{
|
||||||
assert(data.waiting > 0);
|
assert(data.waiting > 0);
|
||||||
|
--data.waiting;
|
||||||
|
++data.running;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
assert(iter != m_jobSet.end());
|
assert(iter != m_jobSet.end());
|
||||||
|
|
||||||
JobType const type = iter->getType();
|
|
||||||
JobTypeData& data(getJobTypeData(type));
|
|
||||||
|
|
||||||
assert(type != jtINVALID);
|
|
||||||
|
|
||||||
job = *iter;
|
job = *iter;
|
||||||
m_jobSet.erase(iter);
|
m_jobSet.erase(iter);
|
||||||
|
|
||||||
--data.waiting;
|
|
||||||
++data.running;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void
|
void
|
||||||
|
|||||||
@@ -254,7 +254,7 @@ public:
|
|||||||
// There is a separate check in `checkpoint` for a valid
|
// There is a separate check in `checkpoint` for a valid
|
||||||
// connection in the rare case when the DatabaseCon is destroyed
|
// connection in the rare case when the DatabaseCon is destroyed
|
||||||
// after locking this weak_ptr
|
// after locking this weak_ptr
|
||||||
[wp = std::weak_ptr<Checkpointer>{shared_from_this()}](Job&) {
|
[wp = std::weak_ptr<Checkpointer>{shared_from_this()}]() {
|
||||||
if (auto self = wp.lock())
|
if (auto self = wp.lock())
|
||||||
self->checkpoint();
|
self->checkpoint();
|
||||||
}))
|
}))
|
||||||
|
|||||||
@@ -37,24 +37,35 @@ class PerfLog;
|
|||||||
/**
|
/**
|
||||||
* `Workers` is effectively a thread pool. The constructor takes a "callback"
|
* `Workers` is effectively a thread pool. The constructor takes a "callback"
|
||||||
* that has a `void processTask(int instance)` method, and a number of
|
* that has a `void processTask(int instance)` method, and a number of
|
||||||
* workers. It creates that many Workers and then waits for calls to
|
* workers. It creates that many `Worker`s and then waits for calls to
|
||||||
* `Workers::addTask()`. It holds a semaphore that counts the number of
|
* `Workers::addTask()`. It holds a semaphore that counts the number of
|
||||||
* waiting tasks, and a condition variable for the event when the last worker
|
* pending "tasks", and a condition variable for the event when the last
|
||||||
* pauses itself.
|
* worker pauses itself.
|
||||||
|
*
|
||||||
|
* A "task" is just a call to the callback's `processTask` method.
|
||||||
|
* "Adding a task" means calling that method now, or remembering to call it in
|
||||||
|
* the future.
|
||||||
|
* This is implemented with a semaphore.
|
||||||
|
* If there are any workers waiting when a task is added, then one will be
|
||||||
|
* woken to claim the task.
|
||||||
|
* If not, then the next worker to wait on the semaphore will claim the task.
|
||||||
*
|
*
|
||||||
* Creating a `Worker` creates a thread that calls `Worker::run()`. When that
|
* Creating a `Worker` creates a thread that calls `Worker::run()`. When that
|
||||||
* thread enters `Worker::run`, it increments the count of active workers in
|
* thread enters `Worker::run`, it increments the count of active workers in
|
||||||
* the parent `Workers` object and then blocks on the semaphore if there are
|
* the parent `Workers` object and then tries to claim a task, which blocks if
|
||||||
* no waiting tasks. It will be unblocked whenever the number of waiting tasks
|
* there are none pending.
|
||||||
* is incremented. That only happens in two circumstances: (1) when
|
* It will be unblocked whenever the semaphore is notified (i.e. when the
|
||||||
|
* number of pending tasks is incremented).
|
||||||
|
* That only happens in two circumstances: (1) when
|
||||||
* `Workers::addTask` is called and (2) when `Workers` wants to pause some
|
* `Workers::addTask` is called and (2) when `Workers` wants to pause some
|
||||||
* workers ("pause one worker" is considered one task), which happens when
|
* workers ("pause one worker" is considered one task), which happens when
|
||||||
* someone wants to stop the workers or shrink the threadpool. No worker
|
* someone wants to stop the workers or shrink the threadpool. No worker
|
||||||
* threads are ever destroyed until `Workers` is destroyed; it merely pauses
|
* threads are ever destroyed until `Workers` is destroyed; it merely pauses
|
||||||
* workers until then.
|
* workers until then.
|
||||||
*
|
*
|
||||||
* When an idle worker is woken, it checks whether `Workers` is trying to pause
|
* When a waiting worker is woken, it checks whether `Workers` is trying to
|
||||||
* workers. If so, it adds itself to the set of paused workers and blocks on
|
* pause workers. If so, it changes its status from active to paused and
|
||||||
|
* blocks on
|
||||||
* its own condition variable. If not, then it calls `processTask` on the
|
* its own condition variable. If not, then it calls `processTask` on the
|
||||||
* "callback" held by `Workers`.
|
* "callback" held by `Workers`.
|
||||||
*
|
*
|
||||||
@@ -62,8 +73,8 @@ class PerfLog;
|
|||||||
* to exit is only set in the destructor of `Worker`, which unblocks the
|
* to exit is only set in the destructor of `Worker`, which unblocks the
|
||||||
* paused thread and waits for it to exit. A `Worker::run` thread checks
|
* paused thread and waits for it to exit. A `Worker::run` thread checks
|
||||||
* whether it needs to exit only when it is woken from a pause (not when it is
|
* whether it needs to exit only when it is woken from a pause (not when it is
|
||||||
* woken from idle). This is why the destructor for `Workers` pauses all the
|
* woken from waiting). This is why the destructor for `Workers` pauses all
|
||||||
* workers before destroying them.
|
* the workers before destroying them.
|
||||||
*/
|
*/
|
||||||
class Workers
|
class Workers
|
||||||
{
|
{
|
||||||
|
|||||||
@@ -96,7 +96,7 @@ public:
|
|||||||
JLOG(j_.info()) << "RPCCall::fromNetwork start";
|
JLOG(j_.info()) << "RPCCall::fromNetwork start";
|
||||||
|
|
||||||
mSending = m_jobQueue.addJob(
|
mSending = m_jobQueue.addJob(
|
||||||
jtCLIENT_SUBSCRIBE, "RPCSub::sendThread", [this](Job&) {
|
jtCLIENT_SUBSCRIBE, "RPCSub::sendThread", [this]() {
|
||||||
sendThread();
|
sendThread();
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -1070,7 +1070,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMManifests> const& m)
|
|||||||
// VFALCO What's the right job type?
|
// VFALCO What's the right job type?
|
||||||
auto that = shared_from_this();
|
auto that = shared_from_this();
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtVALIDATION_ut, "receiveManifests", [this, that, m](Job&) {
|
jtVALIDATION_ut, "receiveManifests", [this, that, m]() {
|
||||||
overlay_.onManifests(m, that);
|
overlay_.onManifests(m, that);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
@@ -1608,7 +1608,7 @@ PeerImp::handleTransaction(
|
|||||||
[weak = std::weak_ptr<PeerImp>(shared_from_this()),
|
[weak = std::weak_ptr<PeerImp>(shared_from_this()),
|
||||||
flags,
|
flags,
|
||||||
checkSignature,
|
checkSignature,
|
||||||
stx](Job&) {
|
stx]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
peer->checkTransaction(flags, checkSignature, stx);
|
peer->checkTransaction(flags, checkSignature, stx);
|
||||||
});
|
});
|
||||||
@@ -1711,7 +1711,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMGetLedger> const& m)
|
|||||||
|
|
||||||
// Queue a job to process the request
|
// Queue a job to process the request
|
||||||
std::weak_ptr<PeerImp> weak = shared_from_this();
|
std::weak_ptr<PeerImp> weak = shared_from_this();
|
||||||
app_.getJobQueue().addJob(jtLEDGER_REQ, "recvGetLedger", [weak, m](Job&) {
|
app_.getJobQueue().addJob(jtLEDGER_REQ, "recvGetLedger", [weak, m]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
peer->processLedgerRequest(m);
|
peer->processLedgerRequest(m);
|
||||||
});
|
});
|
||||||
@@ -1730,7 +1730,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMProofPathRequest> const& m)
|
|||||||
fee_ = Resource::feeMediumBurdenPeer;
|
fee_ = Resource::feeMediumBurdenPeer;
|
||||||
std::weak_ptr<PeerImp> weak = shared_from_this();
|
std::weak_ptr<PeerImp> weak = shared_from_this();
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtREPLAY_REQ, "recvProofPathRequest", [weak, m](Job&) {
|
jtREPLAY_REQ, "recvProofPathRequest", [weak, m]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
{
|
{
|
||||||
auto reply =
|
auto reply =
|
||||||
@@ -1779,7 +1779,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMReplayDeltaRequest> const& m)
|
|||||||
fee_ = Resource::feeMediumBurdenPeer;
|
fee_ = Resource::feeMediumBurdenPeer;
|
||||||
std::weak_ptr<PeerImp> weak = shared_from_this();
|
std::weak_ptr<PeerImp> weak = shared_from_this();
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtREPLAY_REQ, "recvReplayDeltaRequest", [weak, m](Job&) {
|
jtREPLAY_REQ, "recvReplayDeltaRequest", [weak, m]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
{
|
{
|
||||||
auto reply =
|
auto reply =
|
||||||
@@ -1900,7 +1900,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMLedgerData> const& m)
|
|||||||
{
|
{
|
||||||
std::weak_ptr<PeerImp> weak{shared_from_this()};
|
std::weak_ptr<PeerImp> weak{shared_from_this()};
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtTXN_DATA, "recvPeerData", [weak, ledgerHash, m](Job&) {
|
jtTXN_DATA, "recvPeerData", [weak, ledgerHash, m]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
{
|
{
|
||||||
peer->app_.getInboundTransactions().gotData(
|
peer->app_.getInboundTransactions().gotData(
|
||||||
@@ -2013,9 +2013,9 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMProposeSet> const& m)
|
|||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
isTrusted ? jtPROPOSAL_t : jtPROPOSAL_ut,
|
isTrusted ? jtPROPOSAL_t : jtPROPOSAL_ut,
|
||||||
"recvPropose->checkPropose",
|
"recvPropose->checkPropose",
|
||||||
[weak, m, proposal](Job& job) {
|
[weak, isTrusted, m, proposal]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
peer->checkPropose(job, m, proposal);
|
peer->checkPropose(isTrusted, m, proposal);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -2602,7 +2602,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMValidation> const& m)
|
|||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
isTrusted ? jtVALIDATION_t : jtVALIDATION_ut,
|
isTrusted ? jtVALIDATION_t : jtVALIDATION_ut,
|
||||||
"recvValidation->checkValidation",
|
"recvValidation->checkValidation",
|
||||||
[weak, val, m](Job&) {
|
[weak, val, m]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
peer->checkValidation(val, m);
|
peer->checkValidation(val, m);
|
||||||
});
|
});
|
||||||
@@ -2655,7 +2655,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMGetObjectByHash> const& m)
|
|||||||
|
|
||||||
std::weak_ptr<PeerImp> weak = shared_from_this();
|
std::weak_ptr<PeerImp> weak = shared_from_this();
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtREQUESTED_TXN, "doTransactions", [weak, m](Job&) {
|
jtREQUESTED_TXN, "doTransactions", [weak, m]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
peer->doTransactions(m);
|
peer->doTransactions(m);
|
||||||
});
|
});
|
||||||
@@ -2795,7 +2795,7 @@ PeerImp::onMessage(std::shared_ptr<protocol::TMHaveTransactions> const& m)
|
|||||||
|
|
||||||
std::weak_ptr<PeerImp> weak = shared_from_this();
|
std::weak_ptr<PeerImp> weak = shared_from_this();
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtMISSING_TXN, "handleHaveTransactions", [weak, m](Job&) {
|
jtMISSING_TXN, "handleHaveTransactions", [weak, m]() {
|
||||||
if (auto peer = weak.lock())
|
if (auto peer = weak.lock())
|
||||||
peer->handleHaveTransactions(m);
|
peer->handleHaveTransactions(m);
|
||||||
});
|
});
|
||||||
@@ -2975,7 +2975,7 @@ PeerImp::doFetchPack(const std::shared_ptr<protocol::TMGetObjectByHash>& packet)
|
|||||||
auto elapsed = UptimeClock::now();
|
auto elapsed = UptimeClock::now();
|
||||||
auto const pap = &app_;
|
auto const pap = &app_;
|
||||||
app_.getJobQueue().addJob(
|
app_.getJobQueue().addJob(
|
||||||
jtPACK, "MakeFetchPack", [pap, weak, packet, hash, elapsed](Job&) {
|
jtPACK, "MakeFetchPack", [pap, weak, packet, hash, elapsed]() {
|
||||||
pap->getLedgerMaster().makeFetchPack(weak, packet, hash, elapsed);
|
pap->getLedgerMaster().makeFetchPack(weak, packet, hash, elapsed);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
@@ -3111,12 +3111,10 @@ PeerImp::checkTransaction(
|
|||||||
// Called from our JobQueue
|
// Called from our JobQueue
|
||||||
void
|
void
|
||||||
PeerImp::checkPropose(
|
PeerImp::checkPropose(
|
||||||
Job& job,
|
bool isTrusted,
|
||||||
std::shared_ptr<protocol::TMProposeSet> const& packet,
|
std::shared_ptr<protocol::TMProposeSet> const& packet,
|
||||||
RCLCxPeerPos peerPos)
|
RCLCxPeerPos peerPos)
|
||||||
{
|
{
|
||||||
bool isTrusted = (job.getType() == jtPROPOSAL_t);
|
|
||||||
|
|
||||||
JLOG(p_journal_.trace())
|
JLOG(p_journal_.trace())
|
||||||
<< "Checking " << (isTrusted ? "trusted" : "UNTRUSTED") << " proposal";
|
<< "Checking " << (isTrusted ? "trusted" : "UNTRUSTED") << " proposal";
|
||||||
|
|
||||||
|
|||||||
@@ -619,7 +619,7 @@ private:
|
|||||||
|
|
||||||
void
|
void
|
||||||
checkPropose(
|
checkPropose(
|
||||||
Job& job,
|
bool isTrusted,
|
||||||
std::shared_ptr<protocol::TMProposeSet> const& packet,
|
std::shared_ptr<protocol::TMProposeSet> const& packet,
|
||||||
RCLCxPeerPos peerPos);
|
RCLCxPeerPos peerPos);
|
||||||
|
|
||||||
|
|||||||
@@ -360,7 +360,7 @@ ShardArchiveHandler::next(std::lock_guard<std::mutex> const& l)
|
|||||||
// to prevent holding up the lock if the downloader
|
// to prevent holding up the lock if the downloader
|
||||||
// sleeps.
|
// sleeps.
|
||||||
auto const& url{archives_.begin()->second};
|
auto const& url{archives_.begin()->second};
|
||||||
auto wrapper = jobCounter_.wrap([this, url, dstDir](Job&) {
|
auto wrapper = jobCounter_.wrap([this, url, dstDir]() {
|
||||||
auto const ssl = (url.scheme == "https");
|
auto const ssl = (url.scheme == "https");
|
||||||
auto const defaultPort = ssl ? 443 : 80;
|
auto const defaultPort = ssl ? 443 : 80;
|
||||||
|
|
||||||
@@ -416,42 +416,41 @@ ShardArchiveHandler::complete(path dstPath)
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Make lambdas mutable captured vars can be moved from
|
// Make lambdas mutable captured vars can be moved from
|
||||||
auto wrapper =
|
auto wrapper = jobCounter_.wrap([=,
|
||||||
jobCounter_.wrap([=, dstPath = std::move(dstPath)](Job&) mutable {
|
dstPath = std::move(dstPath)]() mutable {
|
||||||
if (stopping_)
|
if (stopping_)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
// If not synced then defer and retry
|
// If not synced then defer and retry
|
||||||
auto const mode{app_.getOPs().getOperatingMode()};
|
auto const mode{app_.getOPs().getOperatingMode()};
|
||||||
if (mode != OperatingMode::FULL)
|
if (mode != OperatingMode::FULL)
|
||||||
{
|
{
|
||||||
std::lock_guard lock(m_);
|
std::lock_guard lock(m_);
|
||||||
timer_.expires_from_now(static_cast<std::chrono::seconds>(
|
timer_.expires_from_now(static_cast<std::chrono::seconds>(
|
||||||
(static_cast<std::size_t>(OperatingMode::FULL) -
|
(static_cast<std::size_t>(OperatingMode::FULL) -
|
||||||
static_cast<std::size_t>(mode)) *
|
static_cast<std::size_t>(mode)) *
|
||||||
10));
|
10));
|
||||||
|
|
||||||
auto wrapper = timerCounter_.wrap(
|
auto wrapper = timerCounter_.wrap(
|
||||||
[=, dstPath = std::move(dstPath)](
|
[=, dstPath = std::move(dstPath)](
|
||||||
boost::system::error_code const& ec) mutable {
|
boost::system::error_code const& ec) mutable {
|
||||||
if (ec != boost::asio::error::operation_aborted)
|
if (ec != boost::asio::error::operation_aborted)
|
||||||
complete(std::move(dstPath));
|
complete(std::move(dstPath));
|
||||||
});
|
});
|
||||||
|
|
||||||
if (!wrapper)
|
if (!wrapper)
|
||||||
onClosureFailed(
|
onClosureFailed(
|
||||||
"failed to wrap closure for operating mode timer",
|
"failed to wrap closure for operating mode timer", lock);
|
||||||
lock);
|
|
||||||
else
|
|
||||||
timer_.async_wait(*wrapper);
|
|
||||||
}
|
|
||||||
else
|
else
|
||||||
{
|
timer_.async_wait(*wrapper);
|
||||||
process(dstPath);
|
}
|
||||||
std::lock_guard lock(m_);
|
else
|
||||||
removeAndProceed(lock);
|
{
|
||||||
}
|
process(dstPath);
|
||||||
});
|
std::lock_guard lock(m_);
|
||||||
|
removeAndProceed(lock);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
if (!wrapper)
|
if (!wrapper)
|
||||||
{
|
{
|
||||||
|
|||||||
@@ -127,7 +127,7 @@ public:
|
|||||||
BEAST_EXPECT(*lv == -1);
|
BEAST_EXPECT(*lv == -1);
|
||||||
|
|
||||||
gate g;
|
gate g;
|
||||||
jq.addJob(jtCLIENT, "LocalValue-Test", [&](auto const& job) {
|
jq.addJob(jtCLIENT, "LocalValue-Test", [&]() {
|
||||||
this->BEAST_EXPECT(*lv == -1);
|
this->BEAST_EXPECT(*lv == -1);
|
||||||
*lv = -2;
|
*lv = -2;
|
||||||
this->BEAST_EXPECT(*lv == -2);
|
this->BEAST_EXPECT(*lv == -2);
|
||||||
@@ -166,7 +166,7 @@ public:
|
|||||||
c->join();
|
c->join();
|
||||||
}
|
}
|
||||||
|
|
||||||
jq.addJob(jtCLIENT, "LocalValue-Test", [&](auto const& job) {
|
jq.addJob(jtCLIENT, "LocalValue-Test", [&]() {
|
||||||
this->BEAST_EXPECT(*lv == -2);
|
this->BEAST_EXPECT(*lv == -2);
|
||||||
g.signal();
|
g.signal();
|
||||||
});
|
});
|
||||||
|
|||||||
@@ -37,10 +37,9 @@ class JobQueue_test : public beast::unit_test::suite
|
|||||||
{
|
{
|
||||||
// addJob() should run the Job (and return true).
|
// addJob() should run the Job (and return true).
|
||||||
std::atomic<bool> jobRan{false};
|
std::atomic<bool> jobRan{false};
|
||||||
BEAST_EXPECT(
|
BEAST_EXPECT(jQueue.addJob(jtCLIENT, "JobAddTest1", [&jobRan]() {
|
||||||
jQueue.addJob(jtCLIENT, "JobAddTest1", [&jobRan](Job&) {
|
jobRan = true;
|
||||||
jobRan = true;
|
}) == true);
|
||||||
}) == true);
|
|
||||||
|
|
||||||
// Wait for the Job to run.
|
// Wait for the Job to run.
|
||||||
while (jobRan == false)
|
while (jobRan == false)
|
||||||
@@ -58,7 +57,7 @@ class JobQueue_test : public beast::unit_test::suite
|
|||||||
// Not recommended for the faint of heart...
|
// Not recommended for the faint of heart...
|
||||||
bool unprotected;
|
bool unprotected;
|
||||||
BEAST_EXPECT(
|
BEAST_EXPECT(
|
||||||
jQueue.addJob(jtCLIENT, "JobAddTest2", [&unprotected](Job&) {
|
jQueue.addJob(jtCLIENT, "JobAddTest2", [&unprotected]() {
|
||||||
unprotected = false;
|
unprotected = false;
|
||||||
}) == false);
|
}) == false);
|
||||||
}
|
}
|
||||||
|
|||||||
Reference in New Issue
Block a user