From 2a2310efebeedb8fc1ab284b6bb80c4f37ed5435 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Thu, 14 Nov 2024 15:27:47 -0800
Subject: [PATCH 01/14] WIP: Background `tryAdd` functionality in
 TransactionQueue

This is a *draft* change that will resolve #4316 when it is complete.
The change makes `TransactionQueue` thread safe and runs the `tryAdd`
function in the background when the feature is enabled. The
implementation closely follows the
[design document](https://docs.google.com/document/d/1pU__XfEp-rR-17TNsuj-VhY6JfyendaFSYLTiq6tIj4/edit?usp=sharing)
I wrote.  The implementation still requires the main thread to
re-broadcast the transactions (for now). I've opened this PR for
visibility / early feedback on the implementation.

This change is very much a work in progress, with the following tasks
remaining:

* [ ] Fix catchup. I seem to have broken catchup in rebasing these
      changes on master. I need to figure out what is going on there and fix
      it.
* [ ] Fix failing tests. These are failing because they don't update
      `TransactionQueue`s new snapshots correctly.
* [ ] Rigorous testing, both for correctness and performance.
* [ ] I'd like to take a look at pushing the cut-point out a bit to
      enable flooding in the background as well. If this is a relatively
      simple change, I'd like to roll it into this PR. If it looks hairy,
      then I'll leave it for a separate change later.
---
 src/herder/HerderImpl.cpp                     |  94 ++++++---
 src/herder/HerderImpl.h                       |   5 +-
 src/herder/TransactionQueue.cpp               | 192 +++++++++++-------
 src/herder/TransactionQueue.h                 |  73 +++++--
 src/herder/TxQueueLimiter.cpp                 |  31 ++-
 src/herder/TxQueueLimiter.h                   |  14 +-
 src/herder/TxSetUtils.cpp                     |   4 +-
 src/herder/test/TransactionQueueTests.cpp     |  62 +++++-
 src/ledger/LedgerManager.h                    |  13 ++
 src/ledger/LedgerManagerImpl.cpp              | 116 +++++++++--
 src/ledger/LedgerStateSnapshot.cpp            |   5 +
 src/ledger/LedgerStateSnapshot.h              |   1 +
 src/main/AppConnector.cpp                     |  38 +++-
 src/main/AppConnector.h                       |  23 ++-
 src/main/Application.h                        |   8 +-
 src/main/ApplicationImpl.cpp                  |   8 +-
 src/main/ApplicationImpl.h                    |   8 +-
 src/main/Config.cpp                           |   3 +
 src/main/Config.h                             |   5 +
 src/main/test/CommandHandlerTests.cpp         |   3 +-
 src/simulation/ApplyLoad.cpp                  |   4 +-
 src/test/FuzzerImpl.cpp                       |   7 +-
 src/transactions/FeeBumpTransactionFrame.cpp  |  13 +-
 src/transactions/FeeBumpTransactionFrame.h    |   9 +-
 src/transactions/OperationFrame.cpp           |  23 +--
 src/transactions/OperationFrame.h             |   4 +-
 src/transactions/TransactionFrame.cpp         |  74 +++----
 src/transactions/TransactionFrame.h           |  20 +-
 src/transactions/TransactionFrameBase.cpp     |  60 ++++++
 src/transactions/TransactionFrameBase.h       |  60 +++++-
 src/transactions/TransactionUtils.cpp         |  11 +-
 src/transactions/TransactionUtils.h           |   9 +-
 .../test/InvokeHostFunctionTests.cpp          |  17 +-
 src/transactions/test/SorobanTxTestUtils.cpp  |  10 +-
 .../test/TransactionTestFrame.cpp             |  12 +-
 src/transactions/test/TransactionTestFrame.h  |   9 +-
 src/transactions/test/TxEnvelopeTests.cpp     |   4 +-
 37 files changed, 744 insertions(+), 308 deletions(-)

diff --git a/src/herder/HerderImpl.cpp b/src/herder/HerderImpl.cpp
index 91c5fb19ff..8b0513dabe 100644
--- a/src/herder/HerderImpl.cpp
+++ b/src/herder/HerderImpl.cpp
@@ -83,10 +83,7 @@ HerderImpl::SCPMetrics::SCPMetrics(Application& app)
 }
 
 HerderImpl::HerderImpl(Application& app)
-    : mTransactionQueue(app, TRANSACTION_QUEUE_TIMEOUT_LEDGERS,
-                        TRANSACTION_QUEUE_BAN_LEDGERS,
-                        TRANSACTION_QUEUE_SIZE_MULTIPLIER)
-    , mPendingEnvelopes(app, *this)
+    : mPendingEnvelopes(app, *this)
     , mHerderSCPDriver(app, *this, mUpgrades, mPendingEnvelopes)
     , mLastSlotSaved(0)
     , mTrackingTimer(app)
@@ -275,7 +272,10 @@ HerderImpl::shutdown()
                    "Shutdown interrupting quorum transitive closure analysis.");
         mLastQuorumMapIntersectionState.mInterruptFlag = true;
     }
-    mTransactionQueue.shutdown();
+    if (mTransactionQueue)
+    {
+        mTransactionQueue->shutdown();
+    }
     if (mSorobanTransactionQueue)
     {
         mSorobanTransactionQueue->shutdown();
@@ -603,7 +603,7 @@ HerderImpl::recvTransaction(TransactionFrameBasePtr tx, bool submittedFromSelf)
         mSorobanTransactionQueue->sourceAccountPending(tx->getSourceID()) &&
         !tx->isSoroban();
     bool hasClassic =
-        mTransactionQueue.sourceAccountPending(tx->getSourceID()) &&
+        mTransactionQueue->sourceAccountPending(tx->getSourceID()) &&
         tx->isSoroban();
     if (hasSoroban || hasClassic)
     {
@@ -617,11 +617,31 @@ HerderImpl::recvTransaction(TransactionFrameBasePtr tx, bool submittedFromSelf)
     }
     else if (!tx->isSoroban())
     {
-        result = mTransactionQueue.tryAdd(tx, submittedFromSelf);
+        if (mApp.getConfig().BACKGROUND_TX_QUEUE && !submittedFromSelf)
+        {
+            mApp.postOnOverlayThread(
+                [this, tx]() { mTransactionQueue->tryAdd(tx, false); },
+                "try add tx");
+            result.code = TransactionQueue::AddResultCode::ADD_STATUS_UNKNOWN;
+        }
+        else
+        {
+            result = mTransactionQueue->tryAdd(tx, submittedFromSelf);
+        }
     }
     else if (mSorobanTransactionQueue)
     {
-        result = mSorobanTransactionQueue->tryAdd(tx, submittedFromSelf);
+        if (mApp.getConfig().BACKGROUND_TX_QUEUE && !submittedFromSelf)
+        {
+            mApp.postOnOverlayThread(
+                [this, tx]() { mSorobanTransactionQueue->tryAdd(tx, false); },
+                "try add tx");
+            result.code = TransactionQueue::AddResultCode::ADD_STATUS_UNKNOWN;
+        }
+        else
+        {
+            result = mSorobanTransactionQueue->tryAdd(tx, submittedFromSelf);
+        }
     }
     else
     {
@@ -923,7 +943,7 @@ HerderImpl::externalizeValue(TxSetXDRFrameConstPtr txSet, uint32_t ledgerSeq,
 bool
 HerderImpl::sourceAccountPending(AccountID const& accountID) const
 {
-    bool accPending = mTransactionQueue.sourceAccountPending(accountID);
+    bool accPending = mTransactionQueue->sourceAccountPending(accountID);
     if (mSorobanTransactionQueue)
     {
         accPending = accPending ||
@@ -1092,7 +1112,7 @@ HerderImpl::getPendingEnvelopes()
 ClassicTransactionQueue&
 HerderImpl::getTransactionQueue()
 {
-    return mTransactionQueue;
+    return *mTransactionQueue;
 }
 SorobanTransactionQueue&
 HerderImpl::getSorobanTransactionQueue()
@@ -1391,7 +1411,7 @@ HerderImpl::triggerNextLedger(uint32_t ledgerSeqToTrigger,
     // it's guaranteed to be up-to-date
     auto const& lcl = mLedgerManager.getLastClosedLedgerHeader();
     PerPhaseTransactionList txPhases;
-    txPhases.emplace_back(mTransactionQueue.getTransactions(lcl.header));
+    txPhases.emplace_back(mTransactionQueue->getTransactions(lcl.header));
 
     if (protocolVersionStartsFrom(lcl.header.ledgerVersion,
                                   SOROBAN_PROTOCOL_VERSION))
@@ -1470,7 +1490,7 @@ HerderImpl::triggerNextLedger(uint32_t ledgerSeqToTrigger,
             invalidTxPhases[static_cast<size_t>(TxSetPhase::SOROBAN)]);
     }
 
-    mTransactionQueue.ban(
+    mTransactionQueue->ban(
         invalidTxPhases[static_cast<size_t>(TxSetPhase::CLASSIC)]);
 
     auto txSetHash = proposedSet->getContentsHash();
@@ -2172,9 +2192,11 @@ HerderImpl::maybeSetupSorobanQueue(uint32_t protocolVersion)
     {
         if (!mSorobanTransactionQueue)
         {
+            releaseAssert(mTxQueueBucketSnapshot);
             mSorobanTransactionQueue =
                 std::make_unique<SorobanTransactionQueue>(
-                    mApp, TRANSACTION_QUEUE_TIMEOUT_LEDGERS,
+                    mApp, mTxQueueBucketSnapshot,
+                    TRANSACTION_QUEUE_TIMEOUT_LEDGERS,
                     TRANSACTION_QUEUE_BAN_LEDGERS,
                     SOROBAN_TRANSACTION_QUEUE_SIZE_MULTIPLIER);
         }
@@ -2189,6 +2211,15 @@ HerderImpl::maybeSetupSorobanQueue(uint32_t protocolVersion)
 void
 HerderImpl::start()
 {
+    releaseAssert(!mTxQueueBucketSnapshot);
+    mTxQueueBucketSnapshot = mApp.getBucketManager()
+                                 .getBucketSnapshotManager()
+                                 .copySearchableLiveBucketListSnapshot();
+    releaseAssert(!mTransactionQueue);
+    mTransactionQueue = std::make_unique<ClassicTransactionQueue>(
+        mApp, mTxQueueBucketSnapshot, TRANSACTION_QUEUE_TIMEOUT_LEDGERS,
+        TRANSACTION_QUEUE_BAN_LEDGERS, TRANSACTION_QUEUE_SIZE_MULTIPLIER);
+
     mMaxTxSize = mApp.getHerder().getMaxClassicTxSize();
     {
         uint32_t version = mApp.getLedgerManager()
@@ -2333,23 +2364,23 @@ HerderImpl::updateTransactionQueue(TxSetXDRFrameConstPtr externalizedTxSet)
 
     auto lhhe = mLedgerManager.getLastClosedLedgerHeader();
 
-    auto updateQueue = [&](auto& queue, auto const& applied) {
-        queue.removeApplied(applied);
-        queue.shift();
-
-        auto txs = queue.getTransactions(lhhe.header);
-
-        auto invalidTxs = TxSetUtils::getInvalidTxList(
+    auto filterInvalidTxs = [&](TxFrameList const& txs) {
+        return TxSetUtils::getInvalidTxList(
             txs, mApp, 0,
-            getUpperBoundCloseTimeOffset(mApp, lhhe.header.scpValue.closeTime));
-        queue.ban(invalidTxs);
-
-        queue.rebroadcast();
+            getUpperBoundCloseTimeOffset(mApp.getAppConnector(),
+                                         lhhe.header.scpValue.closeTime));
     };
+    // Update bucket list snapshot, if needed. Note that this modifies the
+    // pointer itself on update, so we need to pass the potentially new pointer
+    // to the tx queues.
+    mApp.getBucketManager()
+        .getBucketSnapshotManager()
+        .maybeCopySearchableBucketListSnapshot(mTxQueueBucketSnapshot);
     if (txsPerPhase.size() > static_cast<size_t>(TxSetPhase::CLASSIC))
     {
-        updateQueue(mTransactionQueue,
-                    txsPerPhase[static_cast<size_t>(TxSetPhase::CLASSIC)]);
+        mTransactionQueue->update(
+            txsPerPhase[static_cast<size_t>(TxSetPhase::CLASSIC)], lhhe.header,
+            mTxQueueBucketSnapshot, filterInvalidTxs);
     }
 
     // Even if we're in protocol 20, still check for number of phases, in case
@@ -2358,8 +2389,9 @@ HerderImpl::updateTransactionQueue(TxSetXDRFrameConstPtr externalizedTxSet)
     if (mSorobanTransactionQueue != nullptr &&
         txsPerPhase.size() > static_cast<size_t>(TxSetPhase::SOROBAN))
     {
-        updateQueue(*mSorobanTransactionQueue,
-                    txsPerPhase[static_cast<size_t>(TxSetPhase::SOROBAN)]);
+        mSorobanTransactionQueue->update(
+            txsPerPhase[static_cast<size_t>(TxSetPhase::SOROBAN)], lhhe.header,
+            mTxQueueBucketSnapshot, filterInvalidTxs);
     }
 }
 
@@ -2476,7 +2508,7 @@ HerderImpl::isNewerNominationOrBallotSt(SCPStatement const& oldSt,
 size_t
 HerderImpl::getMaxQueueSizeOps() const
 {
-    return mTransactionQueue.getMaxQueueSizeOps();
+    return mTransactionQueue->getMaxQueueSizeOps();
 }
 
 size_t
@@ -2490,7 +2522,7 @@ HerderImpl::getMaxQueueSizeSorobanOps() const
 bool
 HerderImpl::isBannedTx(Hash const& hash) const
 {
-    auto banned = mTransactionQueue.isBanned(hash);
+    auto banned = mTransactionQueue->isBanned(hash);
     if (mSorobanTransactionQueue)
     {
         banned = banned || mSorobanTransactionQueue->isBanned(hash);
@@ -2501,7 +2533,7 @@ HerderImpl::isBannedTx(Hash const& hash) const
 TransactionFrameBaseConstPtr
 HerderImpl::getTx(Hash const& hash) const
 {
-    auto classic = mTransactionQueue.getTx(hash);
+    auto classic = mTransactionQueue->getTx(hash);
     if (!classic && mSorobanTransactionQueue)
     {
         return mSorobanTransactionQueue->getTx(hash);
diff --git a/src/herder/HerderImpl.h b/src/herder/HerderImpl.h
index 015cde5e8f..f3b05751e0 100644
--- a/src/herder/HerderImpl.h
+++ b/src/herder/HerderImpl.h
@@ -248,7 +248,7 @@ class HerderImpl : public Herder
     void purgeOldPersistedTxSets();
     void writeDebugTxSet(LedgerCloseData const& lcd);
 
-    ClassicTransactionQueue mTransactionQueue;
+    std::unique_ptr<ClassicTransactionQueue> mTransactionQueue;
     std::unique_ptr<SorobanTransactionQueue> mSorobanTransactionQueue;
 
     void updateTransactionQueue(TxSetXDRFrameConstPtr txSet);
@@ -301,6 +301,9 @@ class HerderImpl : public Herder
     Application& mApp;
     LedgerManager& mLedgerManager;
 
+    // Bucket list snapshot to use for transaction queues
+    SearchableSnapshotConstPtr mTxQueueBucketSnapshot;
+
     struct SCPMetrics
     {
         medida::Meter& mLostSync;
diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index 10cf4f48e9..9f4decf018 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -73,17 +73,20 @@ TransactionQueue::AddResult::AddResult(AddResultCode addCode,
     txResult->setResultCode(txErrorCode);
 }
 
-TransactionQueue::TransactionQueue(Application& app, uint32 pendingDepth,
-                                   uint32 banDepth, uint32 poolLedgerMultiplier,
-                                   bool isSoroban)
-    : mApp(app)
-    , mPendingDepth(pendingDepth)
+TransactionQueue::TransactionQueue(Application& app,
+                                   SearchableSnapshotConstPtr bucketSnapshot,
+                                   uint32 pendingDepth, uint32 banDepth,
+                                   uint32 poolLedgerMultiplier, bool isSoroban)
+    : mPendingDepth(pendingDepth)
     , mBannedTransactions(banDepth)
     , mBroadcastTimer(app)
+    , mValidationSnapshot(
+          std::make_shared<ImmutableValidationSnapshot const>(app))
+    , mBucketSnapshot(bucketSnapshot)
+    , mTxQueueLimiter(poolLedgerMultiplier, isSoroban, mValidationSnapshot,
+                      bucketSnapshot)
+    , mAppConn(app.getAppConnector())
 {
-    mTxQueueLimiter =
-        std::make_unique<TxQueueLimiter>(poolLedgerMultiplier, app, isSoroban);
-
     auto const& filteredTypes =
         app.getConfig().EXCLUDE_TRANSACTIONS_CONTAINING_OPERATION_TYPE;
     mFilteredTypes.insert(filteredTypes.begin(), filteredTypes.end());
@@ -91,11 +94,11 @@ TransactionQueue::TransactionQueue(Application& app, uint32 pendingDepth,
         rand_uniform<uint64>(0, std::numeric_limits<uint64>::max());
 }
 
-ClassicTransactionQueue::ClassicTransactionQueue(Application& app,
-                                                 uint32 pendingDepth,
-                                                 uint32 banDepth,
-                                                 uint32 poolLedgerMultiplier)
-    : TransactionQueue(app, pendingDepth, banDepth, poolLedgerMultiplier, false)
+ClassicTransactionQueue::ClassicTransactionQueue(
+    Application& app, SearchableSnapshotConstPtr bucketSnapshot,
+    uint32 pendingDepth, uint32 banDepth, uint32 poolLedgerMultiplier)
+    : TransactionQueue(app, bucketSnapshot, pendingDepth, banDepth,
+                       poolLedgerMultiplier, false)
     // Arb tx damping is only relevant to classic txs
     , mArbTxSeenCounter(
           app.getMetrics().NewCounter({"herder", "arb-tx", "seen"}))
@@ -123,7 +126,7 @@ ClassicTransactionQueue::allowTxBroadcast(TimestampedTx const& tx)
     bool allowTx{true};
 
     int32_t const signedAllowance =
-        mApp.getConfig().FLOOD_ARB_TX_BASE_ALLOWANCE;
+        mValidationSnapshot->getConfig().FLOOD_ARB_TX_BASE_ALLOWANCE;
     if (signedAllowance >= 0)
     {
         uint32_t const allowance = static_cast<uint32_t>(signedAllowance);
@@ -165,7 +168,8 @@ ClassicTransactionQueue::allowTxBroadcast(TimestampedTx const& tx)
             if (!allowTx)
             {
                 std::geometric_distribution<uint32_t> dist(
-                    mApp.getConfig().FLOOD_ARB_TX_DAMPING_FACTOR);
+                    mValidationSnapshot->getConfig()
+                        .FLOOD_ARB_TX_DAMPING_FACTOR);
                 uint32_t k = maxBroadcast - allowance;
                 allowTx = dist(gRandomEngine) >= k;
             }
@@ -266,6 +270,7 @@ isDuplicateTx(TransactionFrameBasePtr oldTx, TransactionFrameBasePtr newTx)
 bool
 TransactionQueue::sourceAccountPending(AccountID const& accountID) const
 {
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     return mAccountStates.find(accountID) != mAccountStates.end();
 }
 
@@ -348,6 +353,9 @@ TransactionQueue::canAdd(
 
     stateIter = mAccountStates.find(tx->getSourceID());
     TransactionFrameBasePtr currentTx;
+    LedgerSnapshot const ls(mBucketSnapshot);
+    LedgerHeader const& lh = ls.getLedgerHeader().current();
+    uint32_t ledgerVersion = lh.ledgerVersion;
     if (stateIter != mAccountStates.end())
     {
         auto const& transaction = stateIter->second.mTransaction;
@@ -380,13 +388,7 @@ TransactionQueue::canAdd(
             {
                 auto txResult = tx->createSuccessResult();
                 if (!tx->checkSorobanResourceAndSetError(
-                        mApp.getAppConnector(),
-                        mApp.getLedgerManager()
-                            .getLastClosedSorobanNetworkConfig(),
-                        mApp.getLedgerManager()
-                            .getLastClosedLedgerHeader()
-                            .header.ledgerVersion,
-                        txResult))
+                        *mValidationSnapshot, ledgerVersion, txResult))
                 {
                     return AddResult(AddResultCode::ADD_STATUS_ERROR, txResult);
                 }
@@ -426,14 +428,12 @@ TransactionQueue::canAdd(
         }
     }
 
-    LedgerSnapshot ls(mApp);
-    uint32_t ledgerVersion = ls.getLedgerHeader().current().ledgerVersion;
     // Subtle: transactions are rejected based on the source account limit
     // prior to this point. This is safe because we can't evict transactions
     // from the same source account, so a newer transaction won't replace an
     // old one.
     auto canAddRes =
-        mTxQueueLimiter->canAddTx(tx, currentTx, txsToEvict, ledgerVersion);
+        mTxQueueLimiter.canAddTx(tx, currentTx, txsToEvict, ledgerVersion);
     if (!canAddRes.first)
     {
         ban({tx});
@@ -448,20 +448,21 @@ TransactionQueue::canAdd(
             TransactionQueue::AddResultCode::ADD_STATUS_TRY_AGAIN_LATER);
     }
 
-    auto closeTime = mApp.getLedgerManager()
-                         .getLastClosedLedgerHeader()
-                         .header.scpValue.closeTime;
+    auto closeTime = lh.scpValue.closeTime;
     if (protocolVersionStartsFrom(ledgerVersion, ProtocolVersion::V_19))
     {
         // This is done so minSeqLedgerGap is validated against the next
         // ledgerSeq, which is what will be used at apply time
-        ls.getLedgerHeader().currentToModify().ledgerSeq =
-            mApp.getLedgerManager().getLastClosedLedgerNum() + 1;
+        ++ls.getLedgerHeader().currentToModify().ledgerSeq;
+        // TODO: ^^ I think this is the right thing to do. Was previously the
+        // commented out line below.
+        // ls.getLedgerHeader().currentToModify().ledgerSeq =
+        //     mApp.getLedgerManager().getLastClosedLedgerNum() + 1;
     }
 
     auto txResult =
-        tx->checkValid(mApp.getAppConnector(), ls, 0, 0,
-                       getUpperBoundCloseTimeOffset(mApp, closeTime));
+        tx->checkValid(*mValidationSnapshot, ls, 0, 0,
+                       getUpperBoundCloseTimeOffset(mAppConn, closeTime));
     if (!txResult->isSuccess())
     {
         return AddResult(TransactionQueue::AddResultCode::ADD_STATUS_ERROR,
@@ -493,7 +494,7 @@ TransactionQueue::canAdd(
         releaseAssertOrThrow(sorobanTxData);
 
         sorobanTxData->pushValidationTimeDiagnosticError(
-            mApp.getConfig(), SCE_CONTEXT, SCEC_INVALID_INPUT,
+            mValidationSnapshot->getConfig(), SCE_CONTEXT, SCEC_INVALID_INPUT,
             "non-source auth Soroban tx uses memo or muxed source account");
 
         return AddResult(TransactionQueue::AddResultCode::ADD_STATUS_ERROR,
@@ -522,7 +523,7 @@ void
 TransactionQueue::prepareDropTransaction(AccountState& as)
 {
     releaseAssert(as.mTransaction);
-    mTxQueueLimiter->removeTransaction(as.mTransaction->mTx);
+    mTxQueueLimiter.removeTransaction(as.mTransaction->mTx);
     mKnownTxHashes.erase(as.mTransaction->mTx->getFullHash());
     CLOG_DEBUG(Tx, "Dropping {} transaction",
                hexAbbrev(as.mTransaction->mTx->getFullHash()));
@@ -644,6 +645,7 @@ TransactionQueue::AddResult
 TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
 {
     ZoneScoped;
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
 
     auto c1 =
         tx->getEnvelope().type() == ENVELOPE_TYPE_TX_FEE_BUMP &&
@@ -682,12 +684,12 @@ TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
         // Drop current transaction associated with this account, replace
         // with `tx`
         prepareDropTransaction(stateIter->second);
-        *oldTx = {tx, false, mApp.getClock().now(), submittedFromSelf};
+        *oldTx = {tx, false, mAppConn.now(), submittedFromSelf};
     }
     else
     {
         // New transaction for this account, insert it and update age
-        stateIter->second.mTransaction = {tx, false, mApp.getClock().now(),
+        stateIter->second.mTransaction = {tx, false, mAppConn.now(),
                                           submittedFromSelf};
         mQueueMetrics->mSizeByAge[stateIter->second.mAge]->inc();
     }
@@ -698,13 +700,14 @@ TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
 
     // make space so that we can add this transaction
     // this will succeed as `canAdd` ensures that this is the case
-    mTxQueueLimiter->evictTransactions(
+    mTxQueueLimiter.evictTransactions(
         txsToEvict, *tx,
         [&](TransactionFrameBasePtr const& txToEvict) { ban({txToEvict}); });
-    mTxQueueLimiter->addTransaction(tx);
+    mTxQueueLimiter.addTransaction(tx);
     mKnownTxHashes[tx->getFullHash()] = tx;
 
-    broadcast(false);
+    mAppConn.postOnMainThread([this]() { broadcast(false); },
+                              "tx queue broadcast");
 
     return res;
 }
@@ -742,7 +745,7 @@ TransactionQueue::removeApplied(Transactions const& appliedTxs)
 {
     ZoneScoped;
 
-    auto now = mApp.getClock().now();
+    auto now = mAppConn.now();
     for (auto const& appliedTx : appliedTxs)
     {
         // If the source account is not in mAccountStates, then it has no
@@ -803,6 +806,7 @@ void
 TransactionQueue::ban(Transactions const& banTxs)
 {
     ZoneScoped;
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     auto& bannedFront = mBannedTransactions.front();
 
     // Group the transactions by source account and ban all the transactions
@@ -848,6 +852,7 @@ TransactionQueue::AccountState
 TransactionQueue::getAccountTransactionQueueInfo(
     AccountID const& accountID) const
 {
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     auto i = mAccountStates.find(accountID);
     if (i == std::end(mAccountStates))
     {
@@ -859,6 +864,7 @@ TransactionQueue::getAccountTransactionQueueInfo(
 size_t
 TransactionQueue::countBanned(int index) const
 {
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     return mBannedTransactions[index].size();
 }
 #endif
@@ -924,7 +930,7 @@ TransactionQueue::shift()
     {
         mQueueMetrics->mSizeByAge[i]->set_count(sizes[i]);
     }
-    mTxQueueLimiter->resetEvictionState();
+    mTxQueueLimiter.resetEvictionState();
     // pick a new randomizing seed for tie breaking
     mBroadcastSeed =
         rand_uniform<uint64>(0, std::numeric_limits<uint64>::max());
@@ -933,6 +939,7 @@ TransactionQueue::shift()
 bool
 TransactionQueue::isBanned(Hash const& hash) const
 {
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     return std::any_of(
         std::begin(mBannedTransactions), std::end(mBannedTransactions),
         [&](UnorderedSet<Hash> const& transactions) {
@@ -944,6 +951,7 @@ TxFrameList
 TransactionQueue::getTransactions(LedgerHeader const& lcl) const
 {
     ZoneScoped;
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     TxFrameList txs;
 
     uint32_t const nextLedgerSeq = lcl.ledgerSeq + 1;
@@ -964,6 +972,7 @@ TransactionFrameBaseConstPtr
 TransactionQueue::getTx(Hash const& hash) const
 {
     ZoneScoped;
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     auto it = mKnownTxHashes.find(hash);
     if (it != mKnownTxHashes.end())
     {
@@ -978,10 +987,11 @@ TransactionQueue::getTx(Hash const& hash) const
 std::pair<Resource, std::optional<Resource>>
 ClassicTransactionQueue::getMaxResourcesToFloodThisPeriod() const
 {
-    auto& cfg = mApp.getConfig();
+    auto& cfg = mValidationSnapshot->getConfig();
     double opRatePerLedger = cfg.FLOOD_OP_RATE_PER_LEDGER;
 
-    auto maxOps = mApp.getLedgerManager().getLastMaxTxSetSizeOps();
+    auto maxOps =
+        LedgerManager::getMaxTxSetSizeOps(mBucketSnapshot->getLedgerHeader());
     double opsToFloodLedgerDbl = opRatePerLedger * maxOps;
     releaseAssertOrThrow(opsToFloodLedgerDbl >= 0.0);
     releaseAssertOrThrow(isRepresentableAsInt64(opsToFloodLedgerDbl));
@@ -1022,6 +1032,9 @@ ClassicTransactionQueue::getMaxResourcesToFloodThisPeriod() const
 TransactionQueue::BroadcastStatus
 TransactionQueue::broadcastTx(TimestampedTx& tx)
 {
+    // Must be main thread because we are accessing the overlay manager
+    releaseAssert(threadIsMain());
+
     if (tx.mBroadcasted)
     {
         return BroadcastStatus::BROADCAST_STATUS_ALREADY;
@@ -1050,18 +1063,18 @@ TransactionQueue::broadcastTx(TimestampedTx& tx)
         // useful work from other sources.
         return BroadcastStatus::BROADCAST_STATUS_SKIPPED;
     }
-    return mApp.getOverlayManager().broadcastMessage(
+    return mAppConn.getOverlayManager().broadcastMessage(
                tx.mTx->toStellarMessage(),
                std::make_optional<Hash>(tx.mTx->getFullHash()))
                ? BroadcastStatus::BROADCAST_STATUS_SUCCESS
                : BroadcastStatus::BROADCAST_STATUS_ALREADY;
 }
 
-SorobanTransactionQueue::SorobanTransactionQueue(Application& app,
-                                                 uint32 pendingDepth,
-                                                 uint32 banDepth,
-                                                 uint32 poolLedgerMultiplier)
-    : TransactionQueue(app, pendingDepth, banDepth, poolLedgerMultiplier, true)
+SorobanTransactionQueue::SorobanTransactionQueue(
+    Application& app, SearchableSnapshotConstPtr bucketSnapshot,
+    uint32 pendingDepth, uint32 banDepth, uint32 poolLedgerMultiplier)
+    : TransactionQueue(app, bucketSnapshot, pendingDepth, banDepth,
+                       poolLedgerMultiplier, true)
 {
 
     std::vector<medida::Counter*> sizeByAge;
@@ -1084,10 +1097,11 @@ SorobanTransactionQueue::SorobanTransactionQueue(Application& app,
 std::pair<Resource, std::optional<Resource>>
 SorobanTransactionQueue::getMaxResourcesToFloodThisPeriod() const
 {
-    auto const& cfg = mApp.getConfig();
+    auto const& cfg = mValidationSnapshot->getConfig();
     double ratePerLedger = cfg.FLOOD_SOROBAN_RATE_PER_LEDGER;
 
-    auto sorRes = mApp.getLedgerManager().maxLedgerResources(true);
+    auto sorRes = LedgerManager::maxSorobanLedgerResources(
+        mValidationSnapshot->getSorobanNetworkConfig());
 
     auto totalFloodPerLedger = multiplyByDouble(sorRes, ratePerLedger);
 
@@ -1102,6 +1116,9 @@ SorobanTransactionQueue::getMaxResourcesToFloodThisPeriod() const
 bool
 SorobanTransactionQueue::broadcastSome()
 {
+    // Must be main thread for call to `broadcastTx`
+    releaseAssert(threadIsMain());
+
     // broadcast transactions in surge pricing order:
     // loop over transactions by picking from the account queue with the
     // highest base fee not broadcasted so far.
@@ -1154,8 +1171,8 @@ SorobanTransactionQueue::broadcastSome()
         std::make_shared<SorobanGenericLaneConfig>(resToFlood), mBroadcastSeed);
     queue.visitTopTxs(txsToBroadcast, visitor, mBroadcastOpCarryover);
 
-    Resource maxPerTx =
-        mApp.getLedgerManager().maxSorobanTransactionResources();
+    Resource maxPerTx = LedgerManager::maxSorobanTransactionResources(
+        mValidationSnapshot->getSorobanNetworkConfig());
     for (auto& resLeft : mBroadcastOpCarryover)
     {
         // Limit carry-over to 1 maximum resource transaction
@@ -1167,24 +1184,20 @@ SorobanTransactionQueue::broadcastSome()
 size_t
 SorobanTransactionQueue::getMaxQueueSizeOps() const
 {
-    if (protocolVersionStartsFrom(mApp.getLedgerManager()
-                                      .getLastClosedLedgerHeader()
-                                      .header.ledgerVersion,
-                                  SOROBAN_PROTOCOL_VERSION))
-    {
-        auto res = mTxQueueLimiter->maxScaledLedgerResources(true);
-        releaseAssert(res.size() == NUM_SOROBAN_TX_RESOURCES);
-        return res.getVal(Resource::Type::OPERATIONS);
-    }
-    else
-    {
-        return 0;
-    }
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    // TODO: I removed a conditional checking that the protocol version is
+    // post-soroban here. I think that check is now unnecessary, right?
+    auto res = mTxQueueLimiter.maxScaledLedgerResources(true);
+    releaseAssert(res.size() == NUM_SOROBAN_TX_RESOURCES);
+    return res.getVal(Resource::Type::OPERATIONS);
 }
 
 bool
 ClassicTransactionQueue::broadcastSome()
 {
+    // Must be main thread for call to `broadcastTx`
+    releaseAssert(threadIsMain());
+
     // broadcast transactions in surge pricing order:
     // loop over transactions by picking from the account queue with the
     // highest base fee not broadcasted so far.
@@ -1260,6 +1273,13 @@ ClassicTransactionQueue::broadcastSome()
 void
 TransactionQueue::broadcast(bool fromCallback)
 {
+    // Must be called from the main thread due to the use of `mBroadcastTimer`
+    releaseAssert(threadIsMain());
+
+    // NOTE: Although this is not a public function, it can be called from
+    // `mBroadcastTimer` and so it needs to be synchronized.
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+
     if (mShutdown || (!fromCallback && mWaiting))
     {
         return;
@@ -1282,6 +1302,9 @@ TransactionQueue::broadcast(bool fromCallback)
         mWaiting = true;
         mBroadcastTimer.expires_from_now(
             std::chrono::milliseconds(getFloodPeriod()));
+        // TODO: This use of mBroadcastTimer is OK because this function can
+        // only be called from the main thread. If I push the cut point out to
+        // allow for background broadcasting then I need to replace this timer.
         mBroadcastTimer.async_wait([&]() { broadcast(true); },
                                    &VirtualTimer::onFailureNoop);
     }
@@ -1290,6 +1313,9 @@ TransactionQueue::broadcast(bool fromCallback)
 void
 TransactionQueue::rebroadcast()
 {
+    // For `broadcast` call
+    releaseAssert(threadIsMain());
+
     // force to rebroadcast everything
     for (auto& m : mAccountStates)
     {
@@ -1305,10 +1331,37 @@ TransactionQueue::rebroadcast()
 void
 TransactionQueue::shutdown()
 {
+    releaseAssert(threadIsMain());
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     mShutdown = true;
     mBroadcastTimer.cancel();
 }
 
+void
+TransactionQueue::update(
+    Transactions const& applied, LedgerHeader const& lcl,
+    SearchableSnapshotConstPtr const newBucketSnapshot,
+    std::function<TxFrameList(TxFrameList const&)> const& filterInvalidTxs)
+{
+    ZoneScoped;
+    releaseAssert(threadIsMain());
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+
+    mValidationSnapshot =
+        std::make_shared<ImmutableValidationSnapshot>(mAppConn);
+    mBucketSnapshot = newBucketSnapshot;
+    mTxQueueLimiter.updateSnapshots(mValidationSnapshot, mBucketSnapshot);
+
+    removeApplied(applied);
+    shift();
+
+    auto txs = getTransactions(lcl);
+    auto invalidTxs = filterInvalidTxs(txs);
+    ban(invalidTxs);
+
+    rebroadcast();
+}
+
 static bool
 containsFilteredOperation(std::vector<Operation> const& ops,
                           UnorderedSet<OperationType> const& filteredTypes)
@@ -1350,12 +1403,14 @@ TransactionQueue::isFiltered(TransactionFrameBasePtr tx) const
 size_t
 TransactionQueue::getQueueSizeOps() const
 {
-    return mTxQueueLimiter->size();
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    return mTxQueueLimiter.size();
 }
 
 std::optional<int64_t>
 TransactionQueue::getInQueueSeqNum(AccountID const& account) const
 {
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
     auto stateIter = mAccountStates.find(account);
     if (stateIter == mAccountStates.end())
     {
@@ -1372,7 +1427,8 @@ TransactionQueue::getInQueueSeqNum(AccountID const& account) const
 size_t
 ClassicTransactionQueue::getMaxQueueSizeOps() const
 {
-    auto res = mTxQueueLimiter->maxScaledLedgerResources(false);
+    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    auto res = mTxQueueLimiter.maxScaledLedgerResources(false);
     releaseAssert(res.size() == NUM_CLASSIC_TX_RESOURCES);
     return res.getVal(Resource::Type::OPERATIONS);
 }
diff --git a/src/herder/TransactionQueue.h b/src/herder/TransactionQueue.h
index 81b2409853..f47df0c307 100644
--- a/src/herder/TransactionQueue.h
+++ b/src/herder/TransactionQueue.h
@@ -69,7 +69,8 @@ class TransactionQueue
         ADD_STATUS_ERROR,
         ADD_STATUS_TRY_AGAIN_LATER,
         ADD_STATUS_FILTERED,
-        ADD_STATUS_COUNT
+        ADD_STATUS_COUNT,
+        ADD_STATUS_UNKNOWN // TODO: rename?
     };
 
     struct AddResult
@@ -117,30 +118,31 @@ class TransactionQueue
         std::optional<TimestampedTx> mTransaction;
     };
 
-    explicit TransactionQueue(Application& app, uint32 pendingDepth,
-                              uint32 banDepth, uint32 poolLedgerMultiplier,
-                              bool isSoroban);
+    explicit TransactionQueue(Application& app,
+                              SearchableSnapshotConstPtr bucketSnapshot,
+                              uint32 pendingDepth, uint32 banDepth,
+                              uint32 poolLedgerMultiplier, bool isSoroban);
     virtual ~TransactionQueue();
 
     static std::vector<AssetPair>
     findAllAssetPairsInvolvedInPaymentLoops(TransactionFrameBasePtr tx);
 
     AddResult tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf);
-    void removeApplied(Transactions const& txs);
     // Ban transactions that are no longer valid or have insufficient fee;
     // transaction per account limit applies here, so `txs` should have no
     // duplicate source accounts
     void ban(Transactions const& txs);
 
-    /**
-     * Increase age of each AccountState that has at least one transaction in
-     * mTransactions. Also increments the age for each banned transaction, and
-     * unbans transactions for which age equals banDepth.
-     */
-    void shift();
-    void rebroadcast();
     void shutdown();
 
+    // TODO: Better docs
+    // TODO: More descriptive name
+    // Update internal queue structures after a ledger closes
+    void update(
+        Transactions const& applied, LedgerHeader const& lcl,
+        SearchableSnapshotConstPtr newBucketSnapshot,
+        std::function<TxFrameList(TxFrameList const&)> const& filterInvalidTxs);
+
     bool isBanned(Hash const& hash) const;
     TransactionFrameBaseConstPtr getTx(Hash const& hash) const;
     TxFrameList getTransactions(LedgerHeader const& lcl) const;
@@ -170,7 +172,6 @@ class TransactionQueue
      */
     using BannedTransactions = std::deque<UnorderedSet<Hash>>;
 
-    Application& mApp;
     uint32 const mPendingDepth;
 
     AccountStates mAccountStates;
@@ -201,6 +202,9 @@ class TransactionQueue
 
     bool mShutdown{false};
     bool mWaiting{false};
+    // TODO: VirtualTimer is not thread-safe. Right now it's only used in
+    // functions that are called from the main thread. However, if I move
+    // broadcasting to the background I will need to be careful with this.
     VirtualTimer mBroadcastTimer;
 
     virtual std::pair<Resource, std::optional<Resource>>
@@ -230,15 +234,37 @@ class TransactionQueue
 
     bool isFiltered(TransactionFrameBasePtr tx) const;
 
-    std::unique_ptr<TxQueueLimiter> mTxQueueLimiter;
+    // Snapshots to use for transaction validation
+    ImmutableValidationSnapshotPtr mValidationSnapshot;
+    SearchableSnapshotConstPtr mBucketSnapshot;
+
+    TxQueueLimiter mTxQueueLimiter;
     UnorderedMap<AssetPair, uint32_t, AssetPairHash> mArbitrageFloodDamping;
 
     UnorderedMap<Hash, TransactionFrameBasePtr> mKnownTxHashes;
 
     size_t mBroadcastSeed;
 
+    mutable std::recursive_mutex mTxQueueMutex;
+
+  private:
+    AppConnector& mAppConn;
+
+    void removeApplied(Transactions const& txs);
+
+    /**
+     * Increase age of each AccountState that has at least one transaction in
+     * mTransactions. Also increments the age for each banned transaction, and
+     * unbans transactions for which age equals banDepth.
+     */
+    void shift();
+
+    void rebroadcast();
+
 #ifdef BUILD_TESTS
   public:
+    friend class TransactionQueueTest;
+
     size_t getQueueSizeOps() const;
     std::optional<int64_t> getInQueueSeqNum(AccountID const& account) const;
     std::function<void(TransactionFrameBasePtr&)> mTxBroadcastedEvent;
@@ -248,12 +274,15 @@ class TransactionQueue
 class SorobanTransactionQueue : public TransactionQueue
 {
   public:
-    SorobanTransactionQueue(Application& app, uint32 pendingDepth,
-                            uint32 banDepth, uint32 poolLedgerMultiplier);
+    SorobanTransactionQueue(Application& app,
+                            SearchableSnapshotConstPtr bucketSnapshot,
+                            uint32 pendingDepth, uint32 banDepth,
+                            uint32 poolLedgerMultiplier);
     int
     getFloodPeriod() const override
     {
-        return mApp.getConfig().FLOOD_SOROBAN_TX_PERIOD_MS;
+        std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+        return mValidationSnapshot->getConfig().FLOOD_SOROBAN_TX_PERIOD_MS;
     }
 
     size_t getMaxQueueSizeOps() const override;
@@ -261,6 +290,7 @@ class SorobanTransactionQueue : public TransactionQueue
     void
     clearBroadcastCarryover()
     {
+        std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
         mBroadcastOpCarryover.clear();
         mBroadcastOpCarryover.resize(1, Resource::makeEmptySoroban());
     }
@@ -282,13 +312,16 @@ class SorobanTransactionQueue : public TransactionQueue
 class ClassicTransactionQueue : public TransactionQueue
 {
   public:
-    ClassicTransactionQueue(Application& app, uint32 pendingDepth,
-                            uint32 banDepth, uint32 poolLedgerMultiplier);
+    ClassicTransactionQueue(Application& app,
+                            SearchableSnapshotConstPtr bucketSnapshot,
+                            uint32 pendingDepth, uint32 banDepth,
+                            uint32 poolLedgerMultiplier);
 
     int
     getFloodPeriod() const override
     {
-        return mApp.getConfig().FLOOD_TX_PERIOD_MS;
+        std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+        return mValidationSnapshot->getConfig().FLOOD_TX_PERIOD_MS;
     }
 
     size_t getMaxQueueSizeOps() const override;
diff --git a/src/herder/TxQueueLimiter.cpp b/src/herder/TxQueueLimiter.cpp
index 265ce8fb72..ef5d3d565e 100644
--- a/src/herder/TxQueueLimiter.cpp
+++ b/src/herder/TxQueueLimiter.cpp
@@ -29,14 +29,15 @@ computeBetterFee(std::pair<int64, uint32_t> const& evictedBid,
 
 }
 
-TxQueueLimiter::TxQueueLimiter(uint32 multiplier, Application& app,
-                               bool isSoroban)
+TxQueueLimiter::TxQueueLimiter(uint32 multiplier, bool isSoroban,
+                               ImmutableValidationSnapshotPtr vs,
+                               SearchableSnapshotConstPtr bls)
     : mPoolLedgerMultiplier(multiplier)
-    , mLedgerManager(app.getLedgerManager())
-    , mApp(app)
     , mIsSoroban(isSoroban)
+    , mValidationSnapshot(vs)
+    , mBucketSnapshot(bls)
 {
-    auto maxDexOps = app.getConfig().MAX_DEX_TX_OPERATIONS_IN_TX_SET;
+    auto maxDexOps = vs->getConfig().MAX_DEX_TX_OPERATIONS_IN_TX_SET;
     if (maxDexOps && !mIsSoroban)
     {
         mMaxDexOperations =
@@ -60,8 +61,12 @@ TxQueueLimiter::size() const
 Resource
 TxQueueLimiter::maxScaledLedgerResources(bool isSoroban) const
 {
-    return multiplyByDouble(mLedgerManager.maxLedgerResources(isSoroban),
-                            mPoolLedgerMultiplier);
+    Resource const r = isSoroban
+                           ? LedgerManager::maxSorobanLedgerResources(
+                                 mValidationSnapshot->getSorobanNetworkConfig())
+                           : LedgerManager::maxClassicLedgerResources(
+                                 mBucketSnapshot->getLedgerHeader());
+    return multiplyByDouble(r, mPoolLedgerMultiplier);
 }
 
 void
@@ -84,9 +89,7 @@ TxQueueLimiter::canAddTx(
     std::vector<std::pair<TransactionFrameBasePtr, bool>>& txsToEvict)
 {
     return canAddTx(newTx, oldTx, txsToEvict,
-                    mApp.getLedgerManager()
-                        .getLastClosedLedgerHeader()
-                        .header.ledgerVersion);
+                    mBucketSnapshot->getLedgerHeader().ledgerVersion);
 }
 #endif
 
@@ -236,6 +239,14 @@ TxQueueLimiter::reset(uint32_t ledgerVersion)
     resetEvictionState();
 }
 
+void
+TxQueueLimiter::updateSnapshots(ImmutableValidationSnapshotPtr vs,
+                                SearchableSnapshotConstPtr bls)
+{
+    mValidationSnapshot = vs;
+    mBucketSnapshot = bls;
+}
+
 void
 TxQueueLimiter::resetEvictionState()
 {
diff --git a/src/herder/TxQueueLimiter.h b/src/herder/TxQueueLimiter.h
index e913af6436..9ddcd26820 100644
--- a/src/herder/TxQueueLimiter.h
+++ b/src/herder/TxQueueLimiter.h
@@ -17,7 +17,6 @@ class TxQueueLimiter
 {
     // number of ledgers we can pool in memory
     uint32 const mPoolLedgerMultiplier;
-    LedgerManager& mLedgerManager;
 
     // all known transactions
     std::unique_ptr<SurgePricingPriorityQueue> mTxs;
@@ -33,11 +32,16 @@ class TxQueueLimiter
     // limits.
     std::shared_ptr<SurgePricingLaneConfig> mSurgePricingLaneConfig;
 
-    Application& mApp;
     bool const mIsSoroban;
 
+    // State snapshots used to compute limits
+    ImmutableValidationSnapshotPtr mValidationSnapshot;
+    SearchableSnapshotConstPtr mBucketSnapshot;
+
   public:
-    TxQueueLimiter(uint32 multiplier, Application& app, bool isSoroban);
+    TxQueueLimiter(uint32 multiplier, bool isSoroban,
+                   ImmutableValidationSnapshotPtr vs,
+                   SearchableSnapshotConstPtr bls);
     ~TxQueueLimiter();
 
     void addTransaction(TransactionFrameBasePtr const& tx);
@@ -80,5 +84,9 @@ class TxQueueLimiter
 
     // Resets the internal transaction container and the eviction state.
     void reset(uint32_t ledgerVersion);
+
+    // Update snapshots. Should be called after ledger close
+    void updateSnapshots(ImmutableValidationSnapshotPtr vs,
+                         SearchableSnapshotConstPtr bls);
 };
 }
diff --git a/src/herder/TxSetUtils.cpp b/src/herder/TxSetUtils.cpp
index f81456b584..1a51b5e1c7 100644
--- a/src/herder/TxSetUtils.cpp
+++ b/src/herder/TxSetUtils.cpp
@@ -176,10 +176,10 @@ TxSetUtils::getInvalidTxList(TxFrameList const& txs, Application& app,
 
     TxFrameList invalidTxs;
 
+    AppValidationWrapper avw(app.getAppConnector(), false);
     for (auto const& tx : txs)
     {
-        auto txResult = tx->checkValid(app.getAppConnector(), ls, 0,
-                                       lowerBoundCloseTimeOffset,
+        auto txResult = tx->checkValid(avw, ls, 0, lowerBoundCloseTimeOffset,
                                        upperBoundCloseTimeOffset);
         if (!txResult->isSuccess())
         {
diff --git a/src/herder/test/TransactionQueueTests.cpp b/src/herder/test/TransactionQueueTests.cpp
index 9eaa029c81..c6fcc42b5e 100644
--- a/src/herder/test/TransactionQueueTests.cpp
+++ b/src/herder/test/TransactionQueueTests.cpp
@@ -70,7 +70,10 @@ invalidTransaction(Application& app, TestAccount& account, int sequenceDelta)
         app, account, account.getLastSequenceNumber() + sequenceDelta,
         {payment(account.getPublicKey(), -1)});
 }
+}
 
+namespace stellar
+{
 class TransactionQueueTest
 {
   public:
@@ -258,7 +261,10 @@ TEST_CASE("TransactionQueue complex scenarios", "[herder][transactionqueue]")
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto queue = ClassicTransactionQueue{*app, 4, 2, 2};
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -527,7 +533,10 @@ testTransactionQueueBasicScenarios()
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto queue = ClassicTransactionQueue{*app, 4, 2, 2};
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -770,7 +779,10 @@ TEST_CASE("TransactionQueue hitting the rate limit",
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto queue = ClassicTransactionQueue{*app, 4, 2, 2};
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -851,7 +863,10 @@ TEST_CASE("TransactionQueue with PreconditionsV2", "[herder][transactionqueue]")
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto queue = ClassicTransactionQueue{*app, 4, 2, 2};
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -1701,7 +1716,12 @@ TEST_CASE("TransactionQueue limits", "[herder][transactionqueue]")
     auto account6 = root.create("a6", minBalance2);
     auto account7 = root.create("a7", minBalance2);
 
-    TxQueueLimiter limiter(1, *app, false);
+    auto vs = std::make_shared<ImmutableValidationSnapshot const>(
+        app->getAppConnector());
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    TxQueueLimiter limiter(1, false, vs, bls);
 
     struct SetupElement
     {
@@ -1882,7 +1902,12 @@ TEST_CASE("TransactionQueue limiter with DEX separation",
     auto account6 = root.create("a6", minBalance2);
 
     // 3 * 3 = 9 operations limit, 3 * 1 = 3 DEX operations limit.
-    TxQueueLimiter limiter(3, *app, false);
+    auto vs = std::make_shared<ImmutableValidationSnapshot const>(
+        app->getAppConnector());
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    TxQueueLimiter limiter(3, false, vs, bls);
 
     std::vector<TransactionFrameBasePtr> txs;
 
@@ -2130,7 +2155,10 @@ TEST_CASE("transaction queue starting sequence boundary",
         acc1.bumpSequence(startingSeq - 1);
         REQUIRE(acc1.loadSequenceNumber() == startingSeq - 1);
 
-        ClassicTransactionQueue tq(*app, 4, 10, 4);
+        auto bls = app->getBucketManager()
+                       .getBucketSnapshotManager()
+                       .copySearchableLiveBucketListSnapshot();
+        ClassicTransactionQueue tq(*app, bls, 4, 10, 4);
         REQUIRE(tq.tryAdd(transaction(*app, acc1, 1, 1, 100), false).code ==
                 TransactionQueue::AddResultCode::ADD_STATUS_PENDING);
 
@@ -2511,12 +2539,18 @@ TEST_CASE("transaction queue with fee-bump", "[herder][transactionqueue]")
 
     SECTION("classic")
     {
-        auto queue = ClassicTransactionQueue{*app, 4, 2, 2};
+        auto bls = app->getBucketManager()
+                       .getBucketSnapshotManager()
+                       .copySearchableLiveBucketListSnapshot();
+        auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
         testFeeBump(queue, /* isSoroban */ false);
     }
     SECTION("soroban")
     {
-        auto queue = SorobanTransactionQueue{*app, 4, 2, 2};
+        auto bls = app->getBucketManager()
+                       .getBucketSnapshotManager()
+                       .copySearchableLiveBucketListSnapshot();
+        auto queue = SorobanTransactionQueue{*app, bls, 4, 2, 2};
         testFeeBump(queue, /* isSoroban */ true);
     }
 }
@@ -2688,12 +2722,18 @@ TEST_CASE("replace by fee", "[herder][transactionqueue]")
 
     SECTION("classic")
     {
-        auto queue = ClassicTransactionQueue{*app, 4, 2, 2};
+        auto bls = app->getBucketManager()
+                       .getBucketSnapshotManager()
+                       .copySearchableLiveBucketListSnapshot();
+        auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
         testReplaceByFee(queue, /* isSoroban */ false);
     }
     SECTION("soroban")
     {
-        auto queue = SorobanTransactionQueue{*app, 4, 2, 2};
+        auto bls = app->getBucketManager()
+                       .getBucketSnapshotManager()
+                       .copySearchableLiveBucketListSnapshot();
+        auto queue = SorobanTransactionQueue{*app, bls, 4, 2, 2};
         testReplaceByFee(queue, /* isSoroban */ true);
     }
 }
diff --git a/src/ledger/LedgerManager.h b/src/ledger/LedgerManager.h
index 5094196de8..03d3a32f41 100644
--- a/src/ledger/LedgerManager.h
+++ b/src/ledger/LedgerManager.h
@@ -201,6 +201,19 @@ class LedgerManager
     // Genesis ledger
     static LedgerHeader genesisLedger();
 
+    // TODO: Docs
+    static Resource maxClassicLedgerResources(LedgerHeader const& conf);
+
+    // TODO: Docs
+    static Resource maxSorobanLedgerResources(SorobanNetworkConfig const& conf);
+
+    // TODO: Docs
+    static Resource
+    maxSorobanTransactionResources(SorobanNetworkConfig const& conf);
+
+    // TODO: Docs
+    static uint32_t getMaxTxSetSizeOps(LedgerHeader const& header);
+
     // Called by Herder to inform LedgerManager that a SCP has agreed on a new
     // close event. This is the most common cause of LedgerManager advancing
     // from one ledger to the next: the network reached consensus on
diff --git a/src/ledger/LedgerManagerImpl.cpp b/src/ledger/LedgerManagerImpl.cpp
index 8dcc8a553c..d569fb234f 100644
--- a/src/ledger/LedgerManagerImpl.cpp
+++ b/src/ledger/LedgerManagerImpl.cpp
@@ -125,6 +125,94 @@ LedgerManager::ledgerAbbrev(LedgerHeaderHistoryEntry const& he)
     return ledgerAbbrev(he.header, he.hash);
 }
 
+Resource
+LedgerManager::maxClassicLedgerResources(LedgerHeader const& header)
+{
+    return Resource(LedgerManager::getMaxTxSetSizeOps(header));
+}
+
+Resource
+LedgerManager::maxSorobanLedgerResources(SorobanNetworkConfig const& conf)
+{
+    ZoneScoped std::vector<int64_t> limits = {
+        conf.ledgerMaxTxCount(),
+        conf.ledgerMaxInstructions(),
+        conf.ledgerMaxTransactionSizesBytes(),
+        conf.ledgerMaxReadBytes(),
+        conf.ledgerMaxWriteBytes(),
+        conf.ledgerMaxReadLedgerEntries(),
+        conf.ledgerMaxWriteLedgerEntries()};
+    return Resource(limits);
+}
+
+Resource
+LedgerManager::maxSorobanTransactionResources(SorobanNetworkConfig const& conf)
+{
+    ZoneScoped int64_t const opCount = 1;
+    std::vector<int64_t> limits = {opCount,
+                                   conf.txMaxInstructions(),
+                                   conf.txMaxSizeBytes(),
+                                   conf.txMaxReadBytes(),
+                                   conf.txMaxWriteBytes(),
+                                   conf.txMaxReadLedgerEntries(),
+                                   conf.txMaxWriteLedgerEntries()};
+    return Resource(limits);
+}
+
+uint32_t
+LedgerManager::getMaxTxSetSizeOps(LedgerHeader const& header)
+{
+    auto n = header.maxTxSetSize;
+    return protocolVersionStartsFrom(header.ledgerVersion,
+                                     ProtocolVersion::V_11)
+               ? n
+               : (n * MAX_OPS_PER_TX);
+}
+
+Resource
+LedgerManager::maxClassicLedgerResources(LedgerHeader const& header)
+{
+    return Resource(LedgerManager::getMaxTxSetSizeOps(header));
+}
+
+Resource
+LedgerManager::maxSorobanLedgerResources(SorobanNetworkConfig const& conf)
+{
+    ZoneScoped std::vector<int64_t> limits = {
+        conf.ledgerMaxTxCount(),
+        conf.ledgerMaxInstructions(),
+        conf.ledgerMaxTransactionSizesBytes(),
+        conf.ledgerMaxReadBytes(),
+        conf.ledgerMaxWriteBytes(),
+        conf.ledgerMaxReadLedgerEntries(),
+        conf.ledgerMaxWriteLedgerEntries()};
+    return Resource(limits);
+}
+
+Resource
+LedgerManager::maxSorobanTransactionResources(SorobanNetworkConfig const& conf)
+{
+    ZoneScoped int64_t const opCount = 1;
+    std::vector<int64_t> limits = {opCount,
+                                   conf.txMaxInstructions(),
+                                   conf.txMaxSizeBytes(),
+                                   conf.txMaxReadBytes(),
+                                   conf.txMaxWriteBytes(),
+                                   conf.txMaxReadLedgerEntries(),
+                                   conf.txMaxWriteLedgerEntries()};
+    return Resource(limits);
+}
+
+uint32_t
+LedgerManager::getMaxTxSetSizeOps(LedgerHeader const& header)
+{
+    auto n = header.maxTxSetSize;
+    return protocolVersionStartsFrom(header.ledgerVersion,
+                                     ProtocolVersion::V_11)
+               ? n
+               : (n * MAX_OPS_PER_TX);
+}
+
 LedgerManagerImpl::LedgerApplyMetrics::LedgerApplyMetrics(
     medida::MetricsRegistry& registry)
     : mTransactionApply(registry.NewTimer({"ledger", "transaction", "apply"}))
@@ -417,27 +505,24 @@ uint32_t
 LedgerManagerImpl::getLastMaxTxSetSizeOps() const
 {
     releaseAssert(threadIsMain());
-    auto n = getLCLState().ledgerHeader.header.maxTxSetSize;
-    return protocolVersionStartsFrom(
-               getLCLState().ledgerHeader.header.ledgerVersion,
-               ProtocolVersion::V_11)
-               ? n
-               : (n * MAX_OPS_PER_TX);
+    return LedgerManager::getMaxTxSetSizeOps(mLastClosedLedger.header);
 }
 
 Resource
 LedgerManagerImpl::maxLedgerResources(bool isSoroban)
 {
     ZoneScoped;
+    releaseAssert(threadIsMain());
 
     if (isSoroban)
     {
-        return getLastClosedSorobanNetworkConfig().maxLedgerResources();
+        return LedgerManager::maxSorobanLedgerResources(
+            getSorobanNetworkConfigReadOnly());
     }
     else
     {
-        uint32_t maxOpsLedger = getLastMaxTxSetSizeOps();
-        return Resource(maxOpsLedger);
+        return LedgerManager::maxClassicLedgerResources(
+            mLastClosedLedger.header);
     }
 }
 
@@ -446,17 +531,8 @@ LedgerManagerImpl::maxSorobanTransactionResources()
 {
     ZoneScoped;
 
-    auto const& conf =
-        mApp.getLedgerManager().getLastClosedSorobanNetworkConfig();
-    int64_t const opCount = 1;
-    std::vector<int64_t> limits = {opCount,
-                                   conf.txMaxInstructions(),
-                                   conf.txMaxSizeBytes(),
-                                   conf.txMaxReadBytes(),
-                                   conf.txMaxWriteBytes(),
-                                   conf.txMaxReadLedgerEntries(),
-                                   conf.txMaxWriteLedgerEntries()};
-    return Resource(limits);
+    return LedgerManager::maxSorobanTransactionResources(
+        mApp.getLedgerManager().getSorobanNetworkConfigReadOnly());
 }
 
 int64_t
diff --git a/src/ledger/LedgerStateSnapshot.cpp b/src/ledger/LedgerStateSnapshot.cpp
index badce420fc..1713ae35b0 100644
--- a/src/ledger/LedgerStateSnapshot.cpp
+++ b/src/ledger/LedgerStateSnapshot.cpp
@@ -239,6 +239,11 @@ LedgerSnapshot::LedgerSnapshot(Application& app)
             app.getLedgerManager().getLastClosedSnaphot());
 }
 
+LedgerSnapshot::LedgerSnapshot(SearchableSnapshotConstPtr snapshot)
+    : mGetter(std::make_unique<BucketSnapshotState>(snapshot))
+{
+}
+
 LedgerHeaderWrapper
 LedgerSnapshot::getLedgerHeader() const
 {
diff --git a/src/ledger/LedgerStateSnapshot.h b/src/ledger/LedgerStateSnapshot.h
index a4e0653a1c..858cc4da53 100644
--- a/src/ledger/LedgerStateSnapshot.h
+++ b/src/ledger/LedgerStateSnapshot.h
@@ -143,6 +143,7 @@ class LedgerSnapshot : public NonMovableOrCopyable
   public:
     LedgerSnapshot(AbstractLedgerTxn& ltx);
     LedgerSnapshot(Application& app);
+    explicit LedgerSnapshot(SearchableSnapshotConstPtr snapshot);
     LedgerHeaderWrapper getLedgerHeader() const;
     LedgerEntryWrapper getAccount(AccountID const& account) const;
     LedgerEntryWrapper
diff --git a/src/main/AppConnector.cpp b/src/main/AppConnector.cpp
index b18f0506bd..907a0b9670 100644
--- a/src/main/AppConnector.cpp
+++ b/src/main/AppConnector.cpp
@@ -14,33 +14,33 @@ namespace stellar
 {
 
 AppConnector::AppConnector(Application& app)
-    : mApp(app), mConfig(app.getConfig())
+    : mApp(app), mConfig(std::make_shared<const Config>(app.getConfig()))
 {
 }
 
 Herder&
-AppConnector::getHerder()
+AppConnector::getHerder() const
 {
     releaseAssert(threadIsMain());
     return mApp.getHerder();
 }
 
 LedgerManager&
-AppConnector::getLedgerManager()
+AppConnector::getLedgerManager() const
 {
     releaseAssert(threadIsMain());
     return mApp.getLedgerManager();
 }
 
 OverlayManager&
-AppConnector::getOverlayManager()
+AppConnector::getOverlayManager() const
 {
     releaseAssert(threadIsMain());
     return mApp.getOverlayManager();
 }
 
 BanManager&
-AppConnector::getBanManager()
+AppConnector::getBanManager() const
 {
     releaseAssert(threadIsMain());
     return mApp.getBanManager();
@@ -59,6 +59,17 @@ AppConnector::getSorobanNetworkConfigForApply() const
     return mApp.getLedgerManager().getSorobanNetworkConfigForApply();
 }
 
+std::optional<SorobanNetworkConfig>
+AppConnector::maybeGetSorobanNetworkConfigReadOnly() const
+{
+    releaseAssert(threadIsMain());
+    if (mApp.getLedgerManager().hasSorobanNetworkConfig())
+    {
+        return mApp.getLedgerManager().getSorobanNetworkConfigReadOnly();
+    }
+    return std::nullopt;
+}
+
 medida::MetricsRegistry&
 AppConnector::getMetrics() const
 {
@@ -103,6 +114,12 @@ AppConnector::postOnOverlayThread(std::function<void()>&& f,
 
 Config const&
 AppConnector::getConfig() const
+{
+    return *mConfig;
+}
+
+std::shared_ptr<Config const>
+AppConnector::getConfigPtr() const
 {
     return mConfig;
 }
@@ -119,6 +136,17 @@ AppConnector::now() const
     return mApp.getClock().now();
 }
 
+VirtualClock::system_time_point
+AppConnector::system_now() const
+{
+    // TODO: Is this thread safe? It looks like it is when in REAL_TIME mode,
+    // but I'm not so sure about VIRTUAL_TIME mode as that mode has a
+    // `mVirtualNow` that looks like it can change during access? The same is
+    // true for `AppConnector::now`, which is marked "thread safe" in the header
+    // file. Maybe both of these need some hardening though?
+    return mApp.getClock().system_now();
+}
+
 bool
 AppConnector::shouldYield() const
 {
diff --git a/src/main/AppConnector.h b/src/main/AppConnector.h
index 9d78ab9e66..eed9c67de0 100644
--- a/src/main/AppConnector.h
+++ b/src/main/AppConnector.h
@@ -25,16 +25,16 @@ class AppConnector
     Application& mApp;
     // Copy config for threads to use, and avoid warnings from thread sanitizer
     // about accessing mApp
-    Config const mConfig;
+    std::shared_ptr<const Config> const mConfig;
 
   public:
     AppConnector(Application& app);
 
     // Methods that can only be called from main thread
-    Herder& getHerder();
-    LedgerManager& getLedgerManager();
-    OverlayManager& getOverlayManager();
-    BanManager& getBanManager();
+    Herder& getHerder() const;
+    LedgerManager& getLedgerManager() const;
+    OverlayManager& getOverlayManager() const;
+    BanManager& getBanManager() const;
     bool shouldYield() const;
     SorobanMetrics& getSorobanMetrics() const;
     void checkOnOperationApply(Operation const& operation,
@@ -49,7 +49,9 @@ class AppConnector
     void postOnOverlayThread(std::function<void()>&& f,
                              std::string const& message);
     VirtualClock::time_point now() const;
+    VirtualClock::system_time_point system_now() const;
     Config const& getConfig() const;
+    std::shared_ptr<Config const> getConfigPtr() const;
     bool overlayShuttingDown() const;
     OverlayMetrics& getOverlayMetrics();
     // This method is always exclusively called from one thread
@@ -57,6 +59,17 @@ class AppConnector
     checkScheduledAndCache(std::shared_ptr<CapacityTrackedMessage> msgTracker);
     SorobanNetworkConfig const& getLastClosedSorobanNetworkConfig() const;
     SorobanNetworkConfig const& getSorobanNetworkConfigForApply() const;
+    // TODO: Docs. Mention that the difference between this and
+    // `getSorobanNetowrkConfig` is that:
+    // 1. This makes a copy, which is safe to use in other threads (TODO: Really
+    // double check this. I don't see any references or pointers in
+    // SorobanNetworkConfig, but there is a `mutable` field, which needs to be
+    // investigated as it throws `const` functions into question).
+    // 2. This returns nullopt when the network config is not set, while
+    // `getSorobanNetworkConfig` will throw an assertion error in that case.
+    std::optional<SorobanNetworkConfig>
+    maybeGetSorobanNetworkConfigReadOnly() const;
+
     bool threadIsType(Application::ThreadType type) const;
 
     medida::MetricsRegistry& getMetrics() const;
diff --git a/src/main/Application.h b/src/main/Application.h
index f7fb181e72..f5b1690c35 100644
--- a/src/main/Application.h
+++ b/src/main/Application.h
@@ -188,7 +188,7 @@ class Application
 
     // Return a reference to the Application-local copy of the Config object
     // that the Application was constructed with.
-    virtual Config const& getConfig() = 0;
+    virtual Config const& getConfig() const = 0;
 
     // Gets the current execution-state of the Application
     // (derived from the state of other modules
@@ -216,7 +216,7 @@ class Application
     // Get references to each of the "subsystem" objects.
     virtual TmpDirManager& getTmpDirManager() = 0;
     virtual LedgerManager& getLedgerManager() = 0;
-    virtual BucketManager& getBucketManager() = 0;
+    virtual BucketManager& getBucketManager() const = 0;
     virtual LedgerApplyManager& getLedgerApplyManager() = 0;
     virtual HistoryArchiveManager& getHistoryArchiveManager() = 0;
     virtual HistoryManager& getHistoryManager() = 0;
@@ -319,7 +319,7 @@ class Application
     // instances
     virtual Hash const& getNetworkID() const = 0;
 
-    virtual AbstractLedgerTxnParent& getLedgerTxnRoot() = 0;
+    virtual AbstractLedgerTxnParent& getLedgerTxnRoot() const = 0;
 
     virtual void validateAndLogConfig() = 0;
 
@@ -343,7 +343,7 @@ class Application
     // Returns true iff the calling thread has the same type as `type`
     virtual bool threadIsType(ThreadType type) const = 0;
 
-    virtual AppConnector& getAppConnector() = 0;
+    virtual AppConnector& getAppConnector() const = 0;
 
   protected:
     Application()
diff --git a/src/main/ApplicationImpl.cpp b/src/main/ApplicationImpl.cpp
index 1cb984b205..afaa4e0f37 100644
--- a/src/main/ApplicationImpl.cpp
+++ b/src/main/ApplicationImpl.cpp
@@ -1134,7 +1134,7 @@ ApplicationImpl::applyCfgCommands()
 }
 
 Config const&
-ApplicationImpl::getConfig()
+ApplicationImpl::getConfig() const
 {
     return mConfig;
 }
@@ -1281,7 +1281,7 @@ ApplicationImpl::getLedgerManager()
 }
 
 BucketManager&
-ApplicationImpl::getBucketManager()
+ApplicationImpl::getBucketManager() const
 {
     return *mBucketManager;
 }
@@ -1513,7 +1513,7 @@ ApplicationImpl::createDatabase()
 }
 
 AbstractLedgerTxnParent&
-ApplicationImpl::getLedgerTxnRoot()
+ApplicationImpl::getLedgerTxnRoot() const
 {
 #ifdef BUILD_TESTS
     if (mConfig.MODE_USES_IN_MEMORY_LEDGER)
@@ -1526,7 +1526,7 @@ ApplicationImpl::getLedgerTxnRoot()
 }
 
 AppConnector&
-ApplicationImpl::getAppConnector()
+ApplicationImpl::getAppConnector() const
 {
     return *mAppConnector;
 }
diff --git a/src/main/ApplicationImpl.h b/src/main/ApplicationImpl.h
index 98ad3d3cc4..fd0db8f684 100644
--- a/src/main/ApplicationImpl.h
+++ b/src/main/ApplicationImpl.h
@@ -49,7 +49,7 @@ class ApplicationImpl : public Application
 
     virtual uint64_t timeNow() override;
 
-    virtual Config const& getConfig() override;
+    virtual Config const& getConfig() const override;
 
     virtual State getState() const override;
     virtual std::string getStateHuman() const override;
@@ -61,7 +61,7 @@ class ApplicationImpl : public Application
     virtual void clearMetrics(std::string const& domain) override;
     virtual TmpDirManager& getTmpDirManager() override;
     virtual LedgerManager& getLedgerManager() override;
-    virtual BucketManager& getBucketManager() override;
+    virtual BucketManager& getBucketManager() const override;
     virtual LedgerApplyManager& getLedgerApplyManager() override;
     virtual HistoryArchiveManager& getHistoryArchiveManager() override;
     virtual HistoryManager& getHistoryManager() override;
@@ -77,7 +77,7 @@ class ApplicationImpl : public Application
     virtual WorkScheduler& getWorkScheduler() override;
     virtual BanManager& getBanManager() override;
     virtual StatusManager& getStatusManager() override;
-    virtual AppConnector& getAppConnector() override;
+    virtual AppConnector& getAppConnector() const override;
 
     virtual asio::io_context& getWorkerIOContext() override;
     virtual asio::io_context& getEvictionIOContext() override;
@@ -138,7 +138,7 @@ class ApplicationImpl : public Application
 
     virtual Hash const& getNetworkID() const override;
 
-    virtual AbstractLedgerTxnParent& getLedgerTxnRoot() override;
+    virtual AbstractLedgerTxnParent& getLedgerTxnRoot() const override;
 
   private:
     VirtualClock& mVirtualClock;
diff --git a/src/main/Config.cpp b/src/main/Config.cpp
index a30bcc0012..b8b50cb160 100644
--- a/src/main/Config.cpp
+++ b/src/main/Config.cpp
@@ -160,6 +160,7 @@ Config::Config() : NODE_SEED(SecretKey::random())
     CATCHUP_RECENT = 0;
     BACKGROUND_OVERLAY_PROCESSING = true;
     EXPERIMENTAL_PARALLEL_LEDGER_APPLY = false;
+    BACKGROUND_TX_QUEUE = false;
     BUCKETLIST_DB_INDEX_PAGE_SIZE_EXPONENT = 14; // 2^14 == 16 kb
     BUCKETLIST_DB_INDEX_CUTOFF = 20;             // 20 mb
     BUCKETLIST_DB_MEMORY_FOR_CACHING = 0;
@@ -1079,6 +1080,8 @@ Config::processConfig(std::shared_ptr<cpptoml::table> t)
                      ARTIFICIALLY_DELAY_LEDGER_CLOSE_FOR_TESTING =
                          std::chrono::milliseconds(readInt<uint32_t>(item));
                  }},
+                {"EXPERIMENTAL_BACKGROUND_TX_QUEUE",
+                 [&]() { BACKGROUND_TX_QUEUE = readBool(item); }},
                 // https://github.com/stellar/stellar-core/issues/4581
                 {"BACKGROUND_EVICTION_SCAN",
                  [&]() {
diff --git a/src/main/Config.h b/src/main/Config.h
index 06642affb2..cb473f0173 100644
--- a/src/main/Config.h
+++ b/src/main/Config.h
@@ -480,6 +480,11 @@ class Config : public std::enable_shared_from_this<Config>
     // Enable parallel block application (experimental)
     bool EXPERIMENTAL_PARALLEL_LEDGER_APPLY;
 
+    // TODO: Docs, both here and in the example cfg
+    // TODO: Require BACKGROUND_OVERLAY_PROCESSING to be set to true if this is
+    // enabled?
+    bool BACKGROUND_TX_QUEUE;
+
     // When set to true, BucketListDB indexes are persisted on-disk so that the
     // BucketList does not need to be reindexed on startup. Defaults to true.
     // This should only be set to false for testing purposes
diff --git a/src/main/test/CommandHandlerTests.cpp b/src/main/test/CommandHandlerTests.cpp
index 3e4f5df9d9..3decfbb758 100644
--- a/src/main/test/CommandHandlerTests.cpp
+++ b/src/main/test/CommandHandlerTests.cpp
@@ -502,7 +502,8 @@ TEST_CASE("manualclose", "[commandhandler]")
             setMinTime(txFrame, 0);
             TimePoint const maxTime =
                 lastCloseTime() + defaultManualCloseTimeInterval +
-                getUpperBoundCloseTimeOffset(*app, lastCloseTime());
+                getUpperBoundCloseTimeOffset(app->getAppConnector(),
+                                             lastCloseTime());
             setMaxTime(txFrame, maxTime);
             txFrame->getMutableEnvelope().v1().signatures.clear();
             txFrame->addSignature(root);
diff --git a/src/simulation/ApplyLoad.cpp b/src/simulation/ApplyLoad.cpp
index bbf9eb6e08..0aa875d9bf 100644
--- a/src/simulation/ApplyLoad.cpp
+++ b/src/simulation/ApplyLoad.cpp
@@ -396,6 +396,7 @@ ApplyLoad::benchmark()
                      gRandomEngine);
 
     bool limitHit = false;
+    AppValidationWrapper avw(mApp.getAppConnector(), false);
     for (auto accountIndex : shuffledAccounts)
     {
         auto it = accounts.find(accountIndex);
@@ -407,8 +408,7 @@ ApplyLoad::benchmark()
 
         {
             LedgerTxn ltx(mApp.getLedgerTxnRoot());
-            auto res = tx.second->checkValid(mApp.getAppConnector(), ltx, 0, 0,
-                                             UINT64_MAX);
+            auto res = tx.second->checkValid(avw, ltx, 0, 0, UINT64_MAX);
             releaseAssert((res && res->isSuccess()));
         }
 
diff --git a/src/test/FuzzerImpl.cpp b/src/test/FuzzerImpl.cpp
index fe8eb0186d..fa47428dc1 100644
--- a/src/test/FuzzerImpl.cpp
+++ b/src/test/FuzzerImpl.cpp
@@ -921,16 +921,15 @@ class FuzzTransactionFrame : public TransactionFrame
 
         // attempt application of transaction without processing the fee or
         // committing the LedgerTxn
+        AppValidationWrapper avw(app.getAppConnector(), false);
         SignatureChecker signatureChecker{
             ltx.loadHeader().current().ledgerVersion, getContentsHash(),
             mEnvelope.v1().signatures};
         LedgerSnapshot ltxStmt(ltx);
         // if any ill-formed Operations, do not attempt transaction application
         auto isInvalidOperation = [&](auto const& op, auto& opResult) {
-            return !op->checkValid(
-                app.getAppConnector(), signatureChecker,
-                app.getAppConnector().getLastClosedSorobanNetworkConfig(),
-                ltxStmt, false, opResult, mTxResult->getSorobanData());
+            return !op->checkValid(avw, signatureChecker, ltxStmt, false,
+                                   opResult, mTxResult->getSorobanData());
         };
 
         auto const& ops = getOperations();
diff --git a/src/transactions/FeeBumpTransactionFrame.cpp b/src/transactions/FeeBumpTransactionFrame.cpp
index 13133648fa..555a514439 100644
--- a/src/transactions/FeeBumpTransactionFrame.cpp
+++ b/src/transactions/FeeBumpTransactionFrame.cpp
@@ -154,14 +154,13 @@ FeeBumpTransactionFrame::checkSignature(SignatureChecker& signatureChecker,
 }
 
 MutableTxResultPtr
-FeeBumpTransactionFrame::checkValid(AppConnector& app, LedgerSnapshot const& ls,
+FeeBumpTransactionFrame::checkValid(ValidationConnector const& vc,
+                                    LedgerSnapshot const& ls,
                                     SequenceNumber current,
                                     uint64_t lowerBoundCloseTimeOffset,
                                     uint64_t upperBoundCloseTimeOffset) const
 {
-    if (!isTransactionXDRValidForProtocol(
-            ls.getLedgerHeader().current().ledgerVersion, app.getConfig(),
-            mEnvelope) ||
+    if (!isTransactionXDRValidForCurrentProtocol(vc, mEnvelope) ||
         !XDRProvidesValidFee())
     {
         auto txResult = createSuccessResult();
@@ -189,7 +188,7 @@ FeeBumpTransactionFrame::checkValid(AppConnector& app, LedgerSnapshot const& ls,
     }
 
     auto innerTxResult = mInnerTx->checkValidWithOptionallyChargedFee(
-        app, ls, current, false, lowerBoundCloseTimeOffset,
+        vc, ls, current, false, lowerBoundCloseTimeOffset,
         upperBoundCloseTimeOffset);
     auto finalTxResult = createSuccessResultWithNewInnerTx(
         std::move(txResult), std::move(innerTxResult), mInnerTx);
@@ -199,10 +198,10 @@ FeeBumpTransactionFrame::checkValid(AppConnector& app, LedgerSnapshot const& ls,
 
 bool
 FeeBumpTransactionFrame::checkSorobanResourceAndSetError(
-    AppConnector& app, SorobanNetworkConfig const& cfg, uint32_t ledgerVersion,
+    ValidationConnector const& vc, uint32_t ledgerVersion,
     MutableTxResultPtr txResult) const
 {
-    return mInnerTx->checkSorobanResourceAndSetError(app, cfg, ledgerVersion,
+    return mInnerTx->checkSorobanResourceAndSetError(vc, ledgerVersion,
                                                      txResult);
 }
 
diff --git a/src/transactions/FeeBumpTransactionFrame.h b/src/transactions/FeeBumpTransactionFrame.h
index 8d005612c5..1afdd1d6c7 100644
--- a/src/transactions/FeeBumpTransactionFrame.h
+++ b/src/transactions/FeeBumpTransactionFrame.h
@@ -77,12 +77,13 @@ class FeeBumpTransactionFrame : public TransactionFrameBase
                           MutableTxResultPtr txResult) const override;
 
     MutableTxResultPtr
-    checkValid(AppConnector& app, LedgerSnapshot const& ls,
+    checkValid(ValidationConnector const& app, LedgerSnapshot const& ls,
                SequenceNumber current, uint64_t lowerBoundCloseTimeOffset,
                uint64_t upperBoundCloseTimeOffset) const override;
-    bool checkSorobanResourceAndSetError(
-        AppConnector& app, SorobanNetworkConfig const& cfg,
-        uint32_t ledgerVersion, MutableTxResultPtr txResult) const override;
+    bool
+    checkSorobanResourceAndSetError(ValidationConnector const& app,
+                                    uint32_t ledgerVersion,
+                                    MutableTxResultPtr txResult) const override;
 
     MutableTxResultPtr createSuccessResult() const override;
 
diff --git a/src/transactions/OperationFrame.cpp b/src/transactions/OperationFrame.cpp
index 00d2d46041..61fcf9bdac 100644
--- a/src/transactions/OperationFrame.cpp
+++ b/src/transactions/OperationFrame.cpp
@@ -144,12 +144,10 @@ OperationFrame::apply(AppConnector& app, SignatureChecker& signatureChecker,
     ZoneScoped;
     CLOG_TRACE(Tx, "{}", xdrToCerealString(mOperation, "Operation"));
 
+    AppValidationWrapper avw(app, true);
     LedgerSnapshot ltxState(ltx);
-    std::optional<SorobanNetworkConfig> cfg =
-        isSoroban() ? std::make_optional(app.getSorobanNetworkConfigForApply())
-                    : std::nullopt;
-    bool applyRes = checkValid(app, signatureChecker, cfg, ltxState, true, res,
-                               sorobanData);
+    bool applyRes =
+        checkValid(avw, signatureChecker, ltxState, true, res, sorobanData);
     if (applyRes)
     {
         applyRes = doApply(app, ltx, sorobanBasePrngSeed, res, sorobanData);
@@ -221,7 +219,7 @@ OperationFrame::getSourceID() const
 // make sure sig is correct
 // verifies that the operation is well formed (operation specific)
 bool
-OperationFrame::checkValid(AppConnector& app,
+OperationFrame::checkValid(ValidationConnector const& vc,
                            SignatureChecker& signatureChecker,
                            std::optional<SorobanNetworkConfig> const& cfg,
                            LedgerSnapshot const& ls, bool forApply,
@@ -230,9 +228,8 @@ OperationFrame::checkValid(AppConnector& app,
 {
     ZoneScoped;
     bool validationResult = false;
-    auto validate = [this, &res, forApply, &signatureChecker, &app,
-                     &sorobanData, &validationResult,
-                     &cfg](LedgerSnapshot const& ls) {
+    auto validate = [this, &res, forApply, &signatureChecker, &vc, &sorobanData,
+                     &validationResult](LedgerSnapshot const& ls) {
         if (!isOpSupported(ls.getLedgerHeader().current()))
         {
             res.code(opNOT_SUPPORTED);
@@ -267,9 +264,11 @@ OperationFrame::checkValid(AppConnector& app,
             isSoroban())
         {
             releaseAssertOrThrow(sorobanData);
-            releaseAssertOrThrow(cfg);
-            validationResult = doCheckValidForSoroban(
-                cfg.value(), app.getConfig(), ledgerVersion, res, *sorobanData);
+            auto const& sorobanConfig = vc.getSorobanNetworkConfig();
+
+            validationResult =
+                doCheckValidForSoroban(sorobanConfig, vc.getConfig(),
+                                       ledgerVersion, res, *sorobanData);
         }
         else
         {
diff --git a/src/transactions/OperationFrame.h b/src/transactions/OperationFrame.h
index a18778d54c..c2ae9d7965 100644
--- a/src/transactions/OperationFrame.h
+++ b/src/transactions/OperationFrame.h
@@ -73,8 +73,8 @@ class OperationFrame
 
     AccountID getSourceID() const;
 
-    bool checkValid(AppConnector& app, SignatureChecker& signatureChecker,
-                    std::optional<SorobanNetworkConfig> const& cfg,
+    bool checkValid(ValidationConnector const& vc,
+                    SignatureChecker& signatureChecker,
                     LedgerSnapshot const& ls, bool forApply,
                     OperationResult& res,
                     std::shared_ptr<SorobanTxData> sorobanData) const;
diff --git a/src/transactions/TransactionFrame.cpp b/src/transactions/TransactionFrame.cpp
index 53bfb5635d..d4a893bb1a 100644
--- a/src/transactions/TransactionFrame.cpp
+++ b/src/transactions/TransactionFrame.cpp
@@ -984,8 +984,7 @@ TransactionFrame::isTooEarlyForAccount(LedgerHeaderWrapper const& header,
 
 bool
 TransactionFrame::commonValidPreSeqNum(
-    AppConnector& app, std::optional<SorobanNetworkConfig> const& cfg,
-    LedgerSnapshot const& ls, bool chargeFee,
+    ValidationConnector const& vc, LedgerSnapshot const& ls, bool chargeFee,
     uint64_t lowerBoundCloseTimeOffset, uint64_t upperBoundCloseTimeOffset,
     std::optional<FeePair> sorobanResourceFee,
     MutableTxResultPtr txResult) const
@@ -1055,9 +1054,7 @@ TransactionFrame::commonValidPreSeqNum(
             return false;
         }
 
-        releaseAssert(cfg);
-        if (!checkSorobanResourceAndSetError(app, cfg.value(), ledgerVersion,
-                                             txResult))
+        if (!checkSorobanResourceAndSetError(vc, ledgerVersion, txResult))
         {
             return false;
         }
@@ -1067,7 +1064,7 @@ TransactionFrame::commonValidPreSeqNum(
         if (sorobanData.resourceFee > getFullFee())
         {
             sorobanTxData.pushValidationTimeDiagnosticError(
-                app.getConfig(), SCE_STORAGE, SCEC_EXCEEDED_LIMIT,
+                vc.getConfig(), SCE_STORAGE, SCEC_EXCEEDED_LIMIT,
                 "transaction `sorobanData.resourceFee` is higher than the "
                 "full transaction fee",
                 {makeU64SCVal(sorobanData.resourceFee),
@@ -1081,7 +1078,7 @@ TransactionFrame::commonValidPreSeqNum(
             INT64_MAX - sorobanResourceFee->non_refundable_fee)
         {
             sorobanTxData.pushValidationTimeDiagnosticError(
-                app.getConfig(), SCE_STORAGE, SCEC_INVALID_INPUT,
+                vc.getConfig(), SCE_STORAGE, SCEC_INVALID_INPUT,
                 "transaction resource fees cannot be added",
                 {makeU64SCVal(sorobanResourceFee->refundable_fee),
                  makeU64SCVal(sorobanResourceFee->non_refundable_fee)});
@@ -1094,7 +1091,7 @@ TransactionFrame::commonValidPreSeqNum(
         if (sorobanData.resourceFee < resourceFees)
         {
             sorobanTxData.pushValidationTimeDiagnosticError(
-                app.getConfig(), SCE_STORAGE, SCEC_EXCEEDED_LIMIT,
+                vc.getConfig(), SCE_STORAGE, SCEC_EXCEEDED_LIMIT,
                 "transaction `sorobanData.resourceFee` is lower than the "
                 "actual Soroban resource fee",
                 {makeU64SCVal(sorobanData.resourceFee),
@@ -1113,7 +1110,7 @@ TransactionFrame::commonValidPreSeqNum(
                 if (!set.emplace(lk).second)
                 {
                     sorobanTxData.pushValidationTimeDiagnosticError(
-                        app.getConfig(), SCE_STORAGE, SCEC_INVALID_INPUT,
+                        vc.getConfig(), SCE_STORAGE, SCEC_INVALID_INPUT,
                         "Found duplicate key in the Soroban footprint; every "
                         "key across read-only and read-write footprints has to "
                         "be unique.",
@@ -1292,8 +1289,7 @@ TransactionFrame::isBadSeq(LedgerHeaderWrapper const& header,
 }
 
 TransactionFrame::ValidationType
-TransactionFrame::commonValid(AppConnector& app,
-                              std::optional<SorobanNetworkConfig> const& cfg,
+TransactionFrame::commonValid(ValidationConnector const& vc,
                               SignatureChecker& signatureChecker,
                               LedgerSnapshot const& ls, SequenceNumber current,
                               bool applying, bool chargeFee,
@@ -1307,9 +1303,9 @@ TransactionFrame::commonValid(AppConnector& app,
     ValidationType res = ValidationType::kInvalid;
 
     auto validate = [this, &signatureChecker, applying,
-                     lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset, &app,
-                     chargeFee, sorobanResourceFee, txResult, &current, &res,
-                     cfg](LedgerSnapshot const& ls) {
+                     lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset, &vc,
+                     chargeFee, sorobanResourceFee, txResult, &current,
+                     &res](LedgerSnapshot const& ls) {
         if (applying &&
             (lowerBoundCloseTimeOffset != 0 || upperBoundCloseTimeOffset != 0))
         {
@@ -1317,9 +1313,9 @@ TransactionFrame::commonValid(AppConnector& app,
                 "Applying transaction with non-current closeTime");
         }
 
-        if (!commonValidPreSeqNum(
-                app, cfg, ls, chargeFee, lowerBoundCloseTimeOffset,
-                upperBoundCloseTimeOffset, sorobanResourceFee, txResult))
+        if (!commonValidPreSeqNum(vc, ls, chargeFee, lowerBoundCloseTimeOffset,
+                                  upperBoundCloseTimeOffset, sorobanResourceFee,
+                                  txResult))
         {
             return;
         }
@@ -1524,8 +1520,8 @@ TransactionFrame::removeAccountSigner(AbstractLedgerTxn& ltxOuter,
 
 MutableTxResultPtr
 TransactionFrame::checkValidWithOptionallyChargedFee(
-    AppConnector& app, LedgerSnapshot const& ls, SequenceNumber current,
-    bool chargeFee, uint64_t lowerBoundCloseTimeOffset,
+    ValidationConnector const& vc, LedgerSnapshot const& ls,
+    SequenceNumber current, bool chargeFee, uint64_t lowerBoundCloseTimeOffset,
     uint64_t upperBoundCloseTimeOffset) const
 {
     ZoneScoped;
@@ -1552,20 +1548,17 @@ TransactionFrame::checkValidWithOptionallyChargedFee(
         getSignatures(mEnvelope)};
 
     std::optional<FeePair> sorobanResourceFee;
-    std::optional<SorobanNetworkConfig> sorobanConfig;
     if (protocolVersionStartsFrom(ls.getLedgerHeader().current().ledgerVersion,
                                   SOROBAN_PROTOCOL_VERSION) &&
         isSoroban())
     {
-        sorobanConfig =
-            app.getLedgerManager().getLastClosedSorobanNetworkConfig();
         sorobanResourceFee = computePreApplySorobanResourceFee(
-            ls.getLedgerHeader().current().ledgerVersion, sorobanConfig.value(),
-            app.getConfig());
+            ls.getLedgerHeader().current().ledgerVersion,
+            vc.getSorobanNetworkConfig(), vc.getConfig());
     }
-    bool res = commonValid(app, sorobanConfig, signatureChecker, ls, current,
-                           false, chargeFee, lowerBoundCloseTimeOffset,
-                           upperBoundCloseTimeOffset, sorobanResourceFee,
+    bool res = commonValid(vc, signatureChecker, ls, current, false, chargeFee,
+                           lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset,
+                           sorobanResourceFee,
                            txResult) == ValidationType::kMaybeValid;
     if (res)
     {
@@ -1574,8 +1567,8 @@ TransactionFrame::checkValidWithOptionallyChargedFee(
             auto const& op = mOperations[i];
             auto& opResult = txResult->getOpResultAt(i);
 
-            if (!op->checkValid(app, signatureChecker, sorobanConfig, ls, false,
-                                opResult, txResult->getSorobanData()))
+            if (!op->checkValid(vc, signatureChecker, ls, false, opResult,
+                                txResult->getSorobanData()))
             {
                 // it's OK to just fast fail here and not try to call
                 // checkValid on all operations as the resulting object
@@ -1595,8 +1588,8 @@ TransactionFrame::checkValidWithOptionallyChargedFee(
 }
 
 MutableTxResultPtr
-TransactionFrame::checkValid(AppConnector& app, LedgerSnapshot const& ls,
-                             SequenceNumber current,
+TransactionFrame::checkValid(ValidationConnector const& vc,
+                             LedgerSnapshot const& ls, SequenceNumber current,
                              uint64_t lowerBoundCloseTimeOffset,
                              uint64_t upperBoundCloseTimeOffset) const
 {
@@ -1604,26 +1597,24 @@ TransactionFrame::checkValid(AppConnector& app, LedgerSnapshot const& ls,
     // `checkValidWithOptionallyChargedFee` in order to not validate the
     // envelope XDR twice for the fee bump transactions (they use
     // `checkValidWithOptionallyChargedFee` for the inner tx).
-    if (!isTransactionXDRValidForProtocol(
-            ls.getLedgerHeader().current().ledgerVersion, app.getConfig(),
-            mEnvelope))
+    if (!isTransactionXDRValidForCurrentProtocol(vc, mEnvelope))
     {
         auto txResult = createSuccessResult();
         txResult->setResultCode(txMALFORMED);
         return txResult;
     }
-    return checkValidWithOptionallyChargedFee(app, ls, current, true,
+    return checkValidWithOptionallyChargedFee(vc, ls, current, true,
                                               lowerBoundCloseTimeOffset,
                                               upperBoundCloseTimeOffset);
 }
 
 bool
 TransactionFrame::checkSorobanResourceAndSetError(
-    AppConnector& app, SorobanNetworkConfig const& cfg, uint32_t ledgerVersion,
+    ValidationConnector const& vc, uint32_t ledgerVersion,
     MutableTxResultPtr txResult) const
 {
-    if (!validateSorobanResources(cfg, app.getConfig(), ledgerVersion,
-                                  *txResult->getSorobanData()))
+    if (!validateSorobanResources(vc.getSorobanNetworkConfig(), vc.getConfig(),
+                                  ledgerVersion, *txResult->getSorobanData()))
     {
         txResult->setInnermostResultCode(txSOROBAN_INVALID);
         return false;
@@ -1961,7 +1952,6 @@ TransactionFrame::apply(AppConnector& app, AbstractLedgerTxn& ltx,
                                       SOROBAN_PROTOCOL_VERSION) &&
             isSoroban())
         {
-            sorobanConfig = app.getSorobanNetworkConfigForApply();
             sorobanResourceFee = computePreApplySorobanResourceFee(
                 ledgerVersion, *sorobanConfig, app.getConfig());
 
@@ -1974,9 +1964,9 @@ TransactionFrame::apply(AppConnector& app, AbstractLedgerTxn& ltx,
         }
         LedgerTxn ltxTx(ltx);
         LedgerSnapshot ltxStmt(ltxTx);
-        auto cv =
-            commonValid(app, sorobanConfig, *signatureChecker, ltxStmt, 0, true,
-                        chargeFee, 0, 0, sorobanResourceFee, txResult);
+        AppValidationWrapper avw(app, true);
+        auto cv = commonValid(avw, *signatureChecker, ltxStmt, 0, true,
+                              chargeFee, 0, 0, sorobanResourceFee, txResult);
         if (cv >= ValidationType::kInvalidUpdateSeqNum)
         {
             processSeqNum(ltxTx);
diff --git a/src/transactions/TransactionFrame.h b/src/transactions/TransactionFrame.h
index 96811b6312..dc15784d16 100644
--- a/src/transactions/TransactionFrame.h
+++ b/src/transactions/TransactionFrame.h
@@ -94,8 +94,7 @@ class TransactionFrame : public TransactionFrameBase
                               LedgerEntryWrapper const& sourceAccount,
                               uint64_t lowerBoundCloseTimeOffset) const;
 
-    bool commonValidPreSeqNum(AppConnector& app,
-                              std::optional<SorobanNetworkConfig> const& cfg,
+    bool commonValidPreSeqNum(ValidationConnector const& vc,
                               LedgerSnapshot const& ls, bool chargeFee,
                               uint64_t lowerBoundCloseTimeOffset,
                               uint64_t upperBoundCloseTimeOffset,
@@ -105,8 +104,7 @@ class TransactionFrame : public TransactionFrameBase
     virtual bool isBadSeq(LedgerHeaderWrapper const& header,
                           int64_t seqNum) const;
 
-    ValidationType commonValid(AppConnector& app,
-                               std::optional<SorobanNetworkConfig> const& cfg,
+    ValidationType commonValid(ValidationConnector const& vc,
                                SignatureChecker& signatureChecker,
                                LedgerSnapshot const& ls, SequenceNumber current,
                                bool applying, bool chargeFee,
@@ -210,16 +208,18 @@ class TransactionFrame : public TransactionFrameBase
     bool checkExtraSigners(SignatureChecker& signatureChecker) const;
 
     MutableTxResultPtr checkValidWithOptionallyChargedFee(
-        AppConnector& app, LedgerSnapshot const& ls, SequenceNumber current,
-        bool chargeFee, uint64_t lowerBoundCloseTimeOffset,
+        ValidationConnector const& vc, LedgerSnapshot const& ls,
+        SequenceNumber current, bool chargeFee,
+        uint64_t lowerBoundCloseTimeOffset,
         uint64_t upperBoundCloseTimeOffset) const;
     MutableTxResultPtr
-    checkValid(AppConnector& app, LedgerSnapshot const& ls,
+    checkValid(ValidationConnector const& vc, LedgerSnapshot const& ls,
                SequenceNumber current, uint64_t lowerBoundCloseTimeOffset,
                uint64_t upperBoundCloseTimeOffset) const override;
-    bool checkSorobanResourceAndSetError(
-        AppConnector& app, SorobanNetworkConfig const& cfg,
-        uint32_t ledgerVersion, MutableTxResultPtr txResult) const override;
+    bool
+    checkSorobanResourceAndSetError(ValidationConnector const& vc,
+                                    uint32_t ledgerVersion,
+                                    MutableTxResultPtr txResult) const override;
 
     MutableTxResultPtr createSuccessResult() const override;
 
diff --git a/src/transactions/TransactionFrameBase.cpp b/src/transactions/TransactionFrameBase.cpp
index a1cc4dcc74..8bfdb95c57 100644
--- a/src/transactions/TransactionFrameBase.cpp
+++ b/src/transactions/TransactionFrameBase.cpp
@@ -3,12 +3,72 @@
 // of this distribution or at http://www.apache.org/licenses/LICENSE-2.0
 
 #include "transactions/TransactionFrameBase.h"
+#include "ledger/LedgerManager.h"
+#include "main/AppConnector.h"
 #include "transactions/FeeBumpTransactionFrame.h"
 #include "transactions/TransactionFrame.h"
 
 namespace stellar
 {
 
+AppValidationWrapper::AppValidationWrapper(AppConnector const& app,
+                                           bool forApply)
+    : mApp(app), mForApply(forApply)
+{
+}
+
+Config const&
+AppValidationWrapper::getConfig() const
+{
+    return mApp.getConfig();
+}
+
+SorobanNetworkConfig const&
+AppValidationWrapper::getSorobanNetworkConfig() const
+{
+    return mForApply ? mApp.getLedgerManager().getSorobanNetworkConfigForApply()
+                     : mApp.getSorobanNetworkConfigReadOnly();
+}
+
+uint32_t
+AppValidationWrapper::getCurrentProtocolVersion() const
+{
+    return mApp.getLedgerManager()
+        .getLastClosedLedgerHeader()
+        .header.ledgerVersion;
+}
+
+ImmutableValidationSnapshot::ImmutableValidationSnapshot(
+    AppConnector const& app)
+    : mConfig(app.getConfigPtr())
+    , mSorobanNetworkConfig(app.maybeGetSorobanNetworkConfigReadOnly())
+    , mCurrentProtocolVersion(app.getLedgerManager()
+                                  .getLastClosedLedgerHeader()
+                                  .header.ledgerVersion)
+{
+    releaseAssert(threadIsMain());
+}
+
+Config const&
+ImmutableValidationSnapshot::getConfig() const
+{
+    return *mConfig;
+}
+
+SorobanNetworkConfig const&
+ImmutableValidationSnapshot::getSorobanNetworkConfig() const
+{
+    // TODO: This can throw. Check and throw a more usefull exception instead.
+    // Also document this.
+    return mSorobanNetworkConfig.value();
+}
+
+uint32_t
+ImmutableValidationSnapshot::getCurrentProtocolVersion() const
+{
+    return mCurrentProtocolVersion;
+}
+
 TransactionFrameBasePtr
 TransactionFrameBase::makeTransactionFromWire(Hash const& networkID,
                                               TransactionEnvelope const& env)
diff --git a/src/transactions/TransactionFrameBase.h b/src/transactions/TransactionFrameBase.h
index 94763cdeae..369b2b4d86 100644
--- a/src/transactions/TransactionFrameBase.h
+++ b/src/transactions/TransactionFrameBase.h
@@ -35,6 +35,57 @@ using TransactionFrameBasePtr = std::shared_ptr<TransactionFrameBase const>;
 using TransactionFrameBaseConstPtr =
     std::shared_ptr<TransactionFrameBase const>;
 
+// TODO: Explain why this exists. Allows common validation flows between "apply"
+// and "validate" to work whether given an immutable snapshot (for use in
+// "validate") or a wrapper around the AppConnector (for use in "apply").
+// AppValidationWrapper is also usable outside of "apply" by setting "forApply"
+// to false. This is useful for some of our testing infrastructure.
+class ValidationConnector
+{
+  public:
+    virtual ~ValidationConnector() = default;
+    virtual Config const& getConfig() const = 0;
+    virtual SorobanNetworkConfig const& getSorobanNetworkConfig() const = 0;
+    virtual uint32_t getCurrentProtocolVersion() const = 0;
+};
+
+// TODO: Docs
+class AppValidationWrapper : public ValidationConnector
+{
+  public:
+    explicit AppValidationWrapper(AppConnector const& app, bool forApply);
+    ~AppValidationWrapper() override = default;
+
+    Config const& getConfig() const override;
+    SorobanNetworkConfig const& getSorobanNetworkConfig() const override;
+    uint32_t getCurrentProtocolVersion() const override;
+
+  private:
+    AppConnector const& mApp;
+    bool const mForApply;
+};
+
+// TODO: Docs
+class ImmutableValidationSnapshot : public ValidationConnector
+{
+  public:
+    explicit ImmutableValidationSnapshot(AppConnector const& app);
+    ~ImmutableValidationSnapshot() override = default;
+
+    Config const& getConfig() const override;
+    SorobanNetworkConfig const& getSorobanNetworkConfig() const override;
+    uint32_t getCurrentProtocolVersion() const override;
+
+  private:
+    std::shared_ptr<const Config> const mConfig;
+    std::optional<const SorobanNetworkConfig> const mSorobanNetworkConfig;
+    uint32_t const mCurrentProtocolVersion;
+};
+
+// TODO: Rename to indicate constness / Immutableness?
+using ImmutableValidationSnapshotPtr =
+    std::shared_ptr<ImmutableValidationSnapshot const>;
+
 class TransactionFrameBase
 {
   public:
@@ -46,12 +97,13 @@ class TransactionFrameBase
                        TransactionMetaFrame& meta, MutableTxResultPtr txResult,
                        Hash const& sorobanBasePrngSeed = Hash{}) const = 0;
     virtual MutableTxResultPtr
-    checkValid(AppConnector& app, LedgerSnapshot const& ls,
+    checkValid(ValidationConnector const& vc, LedgerSnapshot const& ls,
                SequenceNumber current, uint64_t lowerBoundCloseTimeOffset,
                uint64_t upperBoundCloseTimeOffset) const = 0;
-    virtual bool checkSorobanResourceAndSetError(
-        AppConnector& app, SorobanNetworkConfig const& cfg,
-        uint32_t ledgerVersion, MutableTxResultPtr txResult) const = 0;
+    virtual bool
+    checkSorobanResourceAndSetError(ValidationConnector const& vc,
+                                    uint32_t ledgerVersion,
+                                    MutableTxResultPtr txResult) const = 0;
 
     virtual MutableTxResultPtr createSuccessResult() const = 0;
 
diff --git a/src/transactions/TransactionUtils.cpp b/src/transactions/TransactionUtils.cpp
index 15a177906b..bd15fa32f1 100644
--- a/src/transactions/TransactionUtils.cpp
+++ b/src/transactions/TransactionUtils.cpp
@@ -1257,9 +1257,9 @@ trustLineFlagIsValid(uint32_t flag, LedgerTxnHeader const& header)
 }
 
 uint64_t
-getUpperBoundCloseTimeOffset(Application& app, uint64_t lastCloseTime)
+getUpperBoundCloseTimeOffset(AppConnector& app, uint64_t lastCloseTime)
 {
-    uint64_t currentTime = VirtualClock::to_time_t(app.getClock().system_now());
+    uint64_t currentTime = VirtualClock::to_time_t(app.system_now());
 
     // account for the time between closeTime and now
     uint64_t closeTimeDrift =
@@ -1980,10 +1980,11 @@ hasMuxedAccount(TransactionEnvelope const& e)
 }
 
 bool
-isTransactionXDRValidForProtocol(uint32_t currProtocol, Config const& cfg,
-                                 TransactionEnvelope const& envelope)
+isTransactionXDRValidForCurrentProtocol(ValidationConnector const& vc,
+                                        TransactionEnvelope const& envelope)
 {
-    uint32_t maxProtocol = cfg.CURRENT_LEDGER_PROTOCOL_VERSION;
+    uint32_t maxProtocol = vc.getConfig().CURRENT_LEDGER_PROTOCOL_VERSION;
+    uint32_t currProtocol = vc.getCurrentProtocolVersion();
     // If we could parse the XDR when ledger is using the maximum supported
     // protocol version, then XDR has to be valid.
     // This check also is pointless before protocol 21 as Soroban environment
diff --git a/src/transactions/TransactionUtils.h b/src/transactions/TransactionUtils.h
index 2963fccf13..0c1d676ea1 100644
--- a/src/transactions/TransactionUtils.h
+++ b/src/transactions/TransactionUtils.h
@@ -29,6 +29,7 @@ class SorobanNetworkConfig;
 class TransactionFrame;
 class TransactionFrameBase;
 class SorobanTxData;
+class ValidationConnector;
 struct ClaimAtom;
 struct LedgerHeader;
 struct LedgerKey;
@@ -260,10 +261,12 @@ bool accountFlagMaskCheckIsValid(uint32_t flag, uint32_t ledgerVersion);
 
 bool hasMuxedAccount(TransactionEnvelope const& e);
 
-bool isTransactionXDRValidForProtocol(uint32_t currProtocol, Config const& cfg,
-                                      TransactionEnvelope const& envelope);
+bool
+isTransactionXDRValidForCurrentProtocol(ValidationConnector const& vc,
+                                        TransactionEnvelope const& envelope);
 
-uint64_t getUpperBoundCloseTimeOffset(Application& app, uint64_t lastCloseTime);
+uint64_t getUpperBoundCloseTimeOffset(AppConnector& app,
+                                      uint64_t lastCloseTime);
 
 bool hasAccountEntryExtV2(AccountEntry const& ae);
 bool hasAccountEntryExtV3(AccountEntry const& ae);
diff --git a/src/transactions/test/InvokeHostFunctionTests.cpp b/src/transactions/test/InvokeHostFunctionTests.cpp
index ed325d89fc..ec5311db46 100644
--- a/src/transactions/test/InvokeHostFunctionTests.cpp
+++ b/src/transactions/test/InvokeHostFunctionTests.cpp
@@ -421,8 +421,8 @@ TEST_CASE("basic contract invocation", "[tx][soroban]")
                                                      addContractKeys);
         auto tx = invocation.createTx(&rootAccount);
 
-        auto result =
-            tx->checkValid(test.getApp().getAppConnector(), rootLtx, 0, 0, 0);
+        AppValidationWrapper avw(test.getApp().getAppConnector(), false);
+        auto result = tx->checkValid(avw, rootLtx, 0, 0, 0);
 
         REQUIRE(tx->getFullFee() ==
                 spec.getInclusionFee() + spec.getResourceFee());
@@ -742,6 +742,7 @@ TEST_CASE("Soroban footprint validation", "[tx][soroban]")
 {
     SorobanTest test;
     auto const& cfg = test.getNetworkCfg();
+    AppValidationWrapper const avw(test.getApp().getAppConnector(), false);
 
     auto& addContract =
         test.deployWasmContract(rust_bridge::get_test_wasm_add_i32());
@@ -760,8 +761,7 @@ TEST_CASE("Soroban footprint validation", "[tx][soroban]")
         MutableTxResultPtr result;
         {
             LedgerTxn ltx(test.getApp().getLedgerTxnRoot());
-            result =
-                tx->checkValid(test.getApp().getAppConnector(), ltx, 0, 0, 0);
+            result = tx->checkValid(avw, ltx, 0, 0, 0);
         }
         REQUIRE(result->isSuccess() == shouldBeValid);
 
@@ -777,8 +777,7 @@ TEST_CASE("Soroban footprint validation", "[tx][soroban]")
         MutableTxResultPtr result;
         {
             LedgerTxn ltx(test.getApp().getLedgerTxnRoot());
-            result =
-                tx->checkValid(test.getApp().getAppConnector(), ltx, 0, 0, 0);
+            result = tx->checkValid(avw, ltx, 0, 0, 0);
         }
         REQUIRE(result->isSuccess() == shouldBeValid);
         if (!shouldBeValid)
@@ -806,8 +805,7 @@ TEST_CASE("Soroban footprint validation", "[tx][soroban]")
         MutableTxResultPtr result;
         {
             LedgerTxn ltx(test.getApp().getLedgerTxnRoot());
-            result =
-                tx->checkValid(test.getApp().getAppConnector(), ltx, 0, 0, 0);
+            result = tx->checkValid(avw, ltx, 0, 0, 0);
         }
         REQUIRE(result->isSuccess() == shouldBeValid);
         if (!shouldBeValid)
@@ -1456,8 +1454,9 @@ TEST_CASE("transaction validation diagnostics", "[tx][soroban]")
             .createTx();
     MutableTxResultPtr result;
     {
+        AppValidationWrapper const avw(test.getApp().getAppConnector(), false);
         LedgerTxn ltx(test.getApp().getLedgerTxnRoot());
-        result = tx->checkValid(test.getApp().getAppConnector(), ltx, 0, 0, 0);
+        result = tx->checkValid(avw, ltx, 0, 0, 0);
     }
     REQUIRE(!test.isTxValid(tx));
 
diff --git a/src/transactions/test/SorobanTxTestUtils.cpp b/src/transactions/test/SorobanTxTestUtils.cpp
index f9d7e4f033..b498826c94 100644
--- a/src/transactions/test/SorobanTxTestUtils.cpp
+++ b/src/transactions/test/SorobanTxTestUtils.cpp
@@ -820,8 +820,9 @@ SorobanTest::invokeArchivalOp(TransactionFrameBaseConstPtr tx,
 {
     MutableTxResultPtr result;
     {
+        AppValidationWrapper const avw(getApp().getAppConnector(), false);
         LedgerTxn ltx(getApp().getLedgerTxnRoot());
-        result = tx->checkValid(getApp().getAppConnector(), ltx, 0, 0, 0);
+        result = tx->checkValid(avw, ltx, 0, 0, 0);
     }
     REQUIRE(result->isSuccess());
     int64_t initBalance = getRoot().getBalance();
@@ -1100,8 +1101,9 @@ SorobanTest::createRestoreTx(SorobanResources const& resources, uint32_t fee,
 bool
 SorobanTest::isTxValid(TransactionFrameBaseConstPtr tx)
 {
+    AppValidationWrapper const avw(getApp().getAppConnector(), false);
     LedgerTxn ltx(getApp().getLedgerTxnRoot());
-    auto ret = tx->checkValid(getApp().getAppConnector(), ltx, 0, 0, 0);
+    auto ret = tx->checkValid(avw, ltx, 0, 0, 0);
     return ret->isSuccess();
 }
 
@@ -1110,9 +1112,9 @@ SorobanTest::invokeTx(TransactionFrameBaseConstPtr tx,
                       TransactionMetaFrame* txMeta)
 {
     {
+        AppValidationWrapper const avw(getApp().getAppConnector(), false);
         LedgerTxn ltx(getApp().getLedgerTxnRoot());
-        REQUIRE(tx->checkValid(getApp().getAppConnector(), ltx, 0, 0, 0)
-                    ->isSuccess());
+        REQUIRE(tx->checkValid(avw, ltx, 0, 0, 0)->isSuccess());
     }
 
     auto resultSet = closeLedger(*mApp, {tx});
diff --git a/src/transactions/test/TransactionTestFrame.cpp b/src/transactions/test/TransactionTestFrame.cpp
index 7373b7eec4..300410f22f 100644
--- a/src/transactions/test/TransactionTestFrame.cpp
+++ b/src/transactions/test/TransactionTestFrame.cpp
@@ -95,20 +95,22 @@ TransactionTestFrame::checkValid(AppConnector& app, AbstractLedgerTxn& ltxOuter,
                                  uint64_t upperBoundCloseTimeOffset) const
 {
     LedgerTxn ltx(ltxOuter);
+    AppValidationWrapper const avw(app, false);
     auto ls = LedgerSnapshot(ltx);
     mTransactionTxResult = mTransactionFrame->checkValid(
-        app, ls, current, lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset);
+        avw, ls, current, lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset);
     return mTransactionTxResult;
 }
 
 MutableTxResultPtr
-TransactionTestFrame::checkValid(AppConnector& app, LedgerSnapshot const& ls,
+TransactionTestFrame::checkValid(ValidationConnector const& vc,
+                                 LedgerSnapshot const& ls,
                                  SequenceNumber current,
                                  uint64_t lowerBoundCloseTimeOffset,
                                  uint64_t upperBoundCloseTimeOffset) const
 {
     mTransactionTxResult = mTransactionFrame->checkValid(
-        app, ls, current, lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset);
+        vc, ls, current, lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset);
     return mTransactionTxResult;
 }
 
@@ -142,11 +144,11 @@ TransactionTestFrame::checkValidForTesting(AppConnector& app,
 
 bool
 TransactionTestFrame::checkSorobanResourceAndSetError(
-    AppConnector& app, SorobanNetworkConfig const& cfg, uint32_t ledgerVersion,
+    ValidationConnector const& vc, uint32_t ledgerVersion,
     MutableTxResultPtr txResult) const
 {
     auto ret = mTransactionFrame->checkSorobanResourceAndSetError(
-        app, cfg, ledgerVersion, txResult);
+        vc, ledgerVersion, txResult);
     mTransactionTxResult = txResult;
     return ret;
 }
diff --git a/src/transactions/test/TransactionTestFrame.h b/src/transactions/test/TransactionTestFrame.h
index 4f6f577057..7ee640eae9 100644
--- a/src/transactions/test/TransactionTestFrame.h
+++ b/src/transactions/test/TransactionTestFrame.h
@@ -67,12 +67,13 @@ class TransactionTestFrame : public TransactionFrameBase
                                   uint64_t lowerBoundCloseTimeOffset,
                                   uint64_t upperBoundCloseTimeOffset) const;
     MutableTxResultPtr
-    checkValid(AppConnector& app, LedgerSnapshot const& ls,
+    checkValid(ValidationConnector const& vc, LedgerSnapshot const& ls,
                SequenceNumber current, uint64_t lowerBoundCloseTimeOffset,
                uint64_t upperBoundCloseTimeOffset) const override;
-    bool checkSorobanResourceAndSetError(
-        AppConnector& app, SorobanNetworkConfig const& cfg,
-        uint32_t ledgerVersion, MutableTxResultPtr txResult) const override;
+    bool
+    checkSorobanResourceAndSetError(ValidationConnector const& vc,
+                                    uint32_t ledgerVersion,
+                                    MutableTxResultPtr txResult) const override;
 
     MutableTxResultPtr createSuccessResult() const override;
 
diff --git a/src/transactions/test/TxEnvelopeTests.cpp b/src/transactions/test/TxEnvelopeTests.cpp
index 5964e6cdcd..20c7e0515c 100644
--- a/src/transactions/test/TxEnvelopeTests.cpp
+++ b/src/transactions/test/TxEnvelopeTests.cpp
@@ -2074,8 +2074,8 @@ TEST_CASE_VERSIONS("txenvelope", "[tx][envelope]")
                                     VirtualClock::from_time_t(closeTime + 5));
                             }
 
-                            auto offset =
-                                getUpperBoundCloseTimeOffset(*app, closeTime);
+                            auto offset = getUpperBoundCloseTimeOffset(
+                                app->getAppConnector(), closeTime);
                             auto upperBoundCloseTime = closeTime + offset;
 
                             SECTION("success")

From 341f123b6c0ea61001b65836b61cabe082b5aa33 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Fri, 10 Jan 2025 11:31:17 -0800
Subject: [PATCH 02/14] Restore protocol version check in
 `SorobanTransactionQueue::getMaxQueueSizeOps`

---
 src/herder/TransactionQueue.cpp | 18 ++++++++++++------
 1 file changed, 12 insertions(+), 6 deletions(-)

diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index 9f4decf018..5ce8f8503c 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -1184,12 +1184,18 @@ SorobanTransactionQueue::broadcastSome()
 size_t
 SorobanTransactionQueue::getMaxQueueSizeOps() const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
-    // TODO: I removed a conditional checking that the protocol version is
-    // post-soroban here. I think that check is now unnecessary, right?
-    auto res = mTxQueueLimiter.maxScaledLedgerResources(true);
-    releaseAssert(res.size() == NUM_SOROBAN_TX_RESOURCES);
-    return res.getVal(Resource::Type::OPERATIONS);
+    if (protocolVersionStartsFrom(
+            mBucketSnapshot->getLedgerHeader().ledgerVersion,
+            SOROBAN_PROTOCOL_VERSION))
+    {
+        auto res = mTxQueueLimiter.maxScaledLedgerResources(true);
+        releaseAssert(res.size() == NUM_SOROBAN_TX_RESOURCES);
+        return res.getVal(Resource::Type::OPERATIONS);
+    }
+    else
+    {
+        return 0;
+    }
 }
 
 bool

From 38fea58d1cf2ccaee2c03e52f71202f1d27c1246 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Tue, 14 Jan 2025 15:30:27 -0800
Subject: [PATCH 03/14] Fix missing soroban network config issue

This is what was breaking catchup
---
 src/transactions/TransactionFrame.cpp | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/src/transactions/TransactionFrame.cpp b/src/transactions/TransactionFrame.cpp
index d4a893bb1a..e9f6524e1c 100644
--- a/src/transactions/TransactionFrame.cpp
+++ b/src/transactions/TransactionFrame.cpp
@@ -1947,13 +1947,13 @@ TransactionFrame::apply(AppConnector& app, AbstractLedgerTxn& ltx,
         //  we'll skip trying to apply operations but we'll still
         //  process the sequence number if needed
         std::optional<FeePair> sorobanResourceFee;
-        std::optional<SorobanNetworkConfig> sorobanConfig;
+        AppValidationWrapper avw(app, true);
         if (protocolVersionStartsFrom(ledgerVersion,
                                       SOROBAN_PROTOCOL_VERSION) &&
             isSoroban())
         {
             sorobanResourceFee = computePreApplySorobanResourceFee(
-                ledgerVersion, *sorobanConfig, app.getConfig());
+                ledgerVersion, avw.getSorobanNetworkConfig(), app.getConfig());
 
             auto& sorobanData = *txResult->getSorobanData();
             sorobanData.setSorobanConsumedNonRefundableFee(
@@ -1964,7 +1964,6 @@ TransactionFrame::apply(AppConnector& app, AbstractLedgerTxn& ltx,
         }
         LedgerTxn ltxTx(ltx);
         LedgerSnapshot ltxStmt(ltxTx);
-        AppValidationWrapper avw(app, true);
         auto cv = commonValid(avw, *signatureChecker, ltxStmt, 0, true,
                               chargeFee, 0, 0, sorobanResourceFee, txResult);
         if (cv >= ValidationType::kInvalidUpdateSeqNum)

From 18a93903c313608f7fe62e372036acdd420a6c64 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Wed, 15 Jan 2025 15:35:20 -0800
Subject: [PATCH 04/14] recursive_mutex -> mutex

---
 src/herder/TransactionQueue.cpp | 88 +++++++++++++++++++++------------
 src/herder/TransactionQueue.h   | 60 +++++++++++++++-------
 2 files changed, 99 insertions(+), 49 deletions(-)

diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index 5ce8f8503c..0603048262 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -270,7 +270,7 @@ isDuplicateTx(TransactionFrameBasePtr oldTx, TransactionFrameBasePtr newTx)
 bool
 TransactionQueue::sourceAccountPending(AccountID const& accountID) const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     return mAccountStates.find(accountID) != mAccountStates.end();
 }
 
@@ -334,7 +334,7 @@ TransactionQueue::canAdd(
     std::vector<std::pair<TransactionFrameBasePtr, bool>>& txsToEvict)
 {
     ZoneScoped;
-    if (isBanned(tx->getFullHash()))
+    if (isBannedInternal(tx->getFullHash()))
     {
         return AddResult(
             TransactionQueue::AddResultCode::ADD_STATUS_TRY_AGAIN_LATER);
@@ -436,7 +436,7 @@ TransactionQueue::canAdd(
         mTxQueueLimiter.canAddTx(tx, currentTx, txsToEvict, ledgerVersion);
     if (!canAddRes.first)
     {
-        ban({tx});
+        banInternal({tx});
         if (canAddRes.second != 0)
         {
             AddResult result(TransactionQueue::AddResultCode::ADD_STATUS_ERROR,
@@ -454,10 +454,6 @@ TransactionQueue::canAdd(
         // This is done so minSeqLedgerGap is validated against the next
         // ledgerSeq, which is what will be used at apply time
         ++ls.getLedgerHeader().currentToModify().ledgerSeq;
-        // TODO: ^^ I think this is the right thing to do. Was previously the
-        // commented out line below.
-        // ls.getLedgerHeader().currentToModify().ledgerSeq =
-        //     mApp.getLedgerManager().getLastClosedLedgerNum() + 1;
     }
 
     auto txResult =
@@ -645,7 +641,7 @@ TransactionQueue::AddResult
 TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
 {
     ZoneScoped;
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
 
     auto c1 =
         tx->getEnvelope().type() == ENVELOPE_TYPE_TX_FEE_BUMP &&
@@ -701,8 +697,9 @@ TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
     // make space so that we can add this transaction
     // this will succeed as `canAdd` ensures that this is the case
     mTxQueueLimiter.evictTransactions(
-        txsToEvict, *tx,
-        [&](TransactionFrameBasePtr const& txToEvict) { ban({txToEvict}); });
+        txsToEvict, *tx, [&](TransactionFrameBasePtr const& txToEvict) {
+            banInternal({txToEvict});
+        });
     mTxQueueLimiter.addTransaction(tx);
     mKnownTxHashes[tx->getFullHash()] = tx;
 
@@ -806,7 +803,14 @@ void
 TransactionQueue::ban(Transactions const& banTxs)
 {
     ZoneScoped;
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    banInternal(banTxs);
+}
+
+void
+TransactionQueue::banInternal(Transactions const& banTxs)
+{
+    ZoneScoped;
     auto& bannedFront = mBannedTransactions.front();
 
     // Group the transactions by source account and ban all the transactions
@@ -852,7 +856,7 @@ TransactionQueue::AccountState
 TransactionQueue::getAccountTransactionQueueInfo(
     AccountID const& accountID) const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     auto i = mAccountStates.find(accountID);
     if (i == std::end(mAccountStates))
     {
@@ -864,7 +868,7 @@ TransactionQueue::getAccountTransactionQueueInfo(
 size_t
 TransactionQueue::countBanned(int index) const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     return mBannedTransactions[index].size();
 }
 #endif
@@ -939,7 +943,13 @@ TransactionQueue::shift()
 bool
 TransactionQueue::isBanned(Hash const& hash) const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    return isBannedInternal(hash);
+}
+
+bool
+TransactionQueue::isBannedInternal(Hash const& hash) const
+{
     return std::any_of(
         std::begin(mBannedTransactions), std::end(mBannedTransactions),
         [&](UnorderedSet<Hash> const& transactions) {
@@ -951,7 +961,14 @@ TxFrameList
 TransactionQueue::getTransactions(LedgerHeader const& lcl) const
 {
     ZoneScoped;
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    return getTransactionsInternal(lcl);
+}
+
+TxFrameList
+TransactionQueue::getTransactionsInternal(LedgerHeader const& lcl) const
+{
+    ZoneScoped;
     TxFrameList txs;
 
     uint32_t const nextLedgerSeq = lcl.ledgerSeq + 1;
@@ -972,7 +989,7 @@ TransactionFrameBaseConstPtr
 TransactionQueue::getTx(Hash const& hash) const
 {
     ZoneScoped;
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     auto it = mKnownTxHashes.find(hash);
     if (it != mKnownTxHashes.end())
     {
@@ -1184,6 +1201,8 @@ SorobanTransactionQueue::broadcastSome()
 size_t
 SorobanTransactionQueue::getMaxQueueSizeOps() const
 {
+    ZoneScoped;
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     if (protocolVersionStartsFrom(
             mBucketSnapshot->getLedgerHeader().ledgerVersion,
             SOROBAN_PROTOCOL_VERSION))
@@ -1264,7 +1283,7 @@ ClassicTransactionQueue::broadcastSome()
         std::make_shared<DexLimitingLaneConfig>(opsToFlood, dexOpsToFlood),
         mBroadcastSeed);
     queue.visitTopTxs(txsToBroadcast, visitor, mBroadcastOpCarryover);
-    ban(banningTxs);
+    banInternal(banningTxs);
     // carry over remainder, up to MAX_OPS_PER_TX ops
     // reason is that if we add 1 next round, we can flood a "worst case fee
     // bump" tx
@@ -1277,15 +1296,12 @@ ClassicTransactionQueue::broadcastSome()
 }
 
 void
-TransactionQueue::broadcast(bool fromCallback)
+TransactionQueue::broadcast(bool fromCallback,
+                            std::lock_guard<std::mutex> const& guard)
 {
     // Must be called from the main thread due to the use of `mBroadcastTimer`
     releaseAssert(threadIsMain());
 
-    // NOTE: Although this is not a public function, it can be called from
-    // `mBroadcastTimer` and so it needs to be synchronized.
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
-
     if (mShutdown || (!fromCallback && mWaiting))
     {
         return;
@@ -1317,7 +1333,14 @@ TransactionQueue::broadcast(bool fromCallback)
 }
 
 void
-TransactionQueue::rebroadcast()
+TransactionQueue::broadcast(bool fromCallback)
+{
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    broadcast(fromCallback, guard);
+}
+
+void
+TransactionQueue::rebroadcast(std::lock_guard<std::mutex> const& guard)
 {
     // For `broadcast` call
     releaseAssert(threadIsMain());
@@ -1331,14 +1354,14 @@ TransactionQueue::rebroadcast()
             as.mTransaction->mBroadcasted = false;
         }
     }
-    broadcast(false);
+    broadcast(false, guard);
 }
 
 void
 TransactionQueue::shutdown()
 {
     releaseAssert(threadIsMain());
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     mShutdown = true;
     mBroadcastTimer.cancel();
 }
@@ -1351,7 +1374,7 @@ TransactionQueue::update(
 {
     ZoneScoped;
     releaseAssert(threadIsMain());
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
 
     mValidationSnapshot =
         std::make_shared<ImmutableValidationSnapshot>(mAppConn);
@@ -1361,11 +1384,11 @@ TransactionQueue::update(
     removeApplied(applied);
     shift();
 
-    auto txs = getTransactions(lcl);
+    auto txs = getTransactionsInternal(lcl);
     auto invalidTxs = filterInvalidTxs(txs);
-    ban(invalidTxs);
+    banInternal(invalidTxs);
 
-    rebroadcast();
+    rebroadcast(guard);
 }
 
 static bool
@@ -1409,14 +1432,14 @@ TransactionQueue::isFiltered(TransactionFrameBasePtr tx) const
 size_t
 TransactionQueue::getQueueSizeOps() const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     return mTxQueueLimiter.size();
 }
 
 std::optional<int64_t>
 TransactionQueue::getInQueueSeqNum(AccountID const& account) const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     auto stateIter = mAccountStates.find(account);
     if (stateIter == mAccountStates.end())
     {
@@ -1433,7 +1456,8 @@ TransactionQueue::getInQueueSeqNum(AccountID const& account) const
 size_t
 ClassicTransactionQueue::getMaxQueueSizeOps() const
 {
-    std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+    ZoneScoped;
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
     auto res = mTxQueueLimiter.maxScaledLedgerResources(false);
     releaseAssert(res.size() == NUM_CLASSIC_TX_RESOURCES);
     return res.getVal(Resource::Type::OPERATIONS);
diff --git a/src/herder/TransactionQueue.h b/src/herder/TransactionQueue.h
index f47df0c307..f5bfbcbc1c 100644
--- a/src/herder/TransactionQueue.h
+++ b/src/herder/TransactionQueue.h
@@ -210,10 +210,15 @@ class TransactionQueue
     virtual std::pair<Resource, std::optional<Resource>>
     getMaxResourcesToFloodThisPeriod() const = 0;
     virtual bool broadcastSome() = 0;
-    virtual int getFloodPeriod() const = 0;
     virtual bool allowTxBroadcast(TimestampedTx const& tx) = 0;
 
+    // TODO: Explain that there's an overload that takes a guard because this
+    // function is called internally, and also scheduled on a timer. Any async
+    // call should call the first overload (which grabs a lock), and any
+    // internal call should call the second overload (which enforces that the
+    // lock is already held).
     void broadcast(bool fromCallback);
+    void broadcast(bool fromCallback, std::lock_guard<std::mutex> const& guard);
     // broadcasts a single transaction
     enum class BroadcastStatus
     {
@@ -234,6 +239,12 @@ class TransactionQueue
 
     bool isFiltered(TransactionFrameBasePtr tx) const;
 
+    // TODO: Docs
+    // Protected versions of public functions that contain the actual
+    // implementation so they can be called internally when the lock is already
+    // held.
+    void banInternal(Transactions const& banTxs);
+
     // Snapshots to use for transaction validation
     ImmutableValidationSnapshotPtr mValidationSnapshot;
     SearchableSnapshotConstPtr mBucketSnapshot;
@@ -245,7 +256,7 @@ class TransactionQueue
 
     size_t mBroadcastSeed;
 
-    mutable std::recursive_mutex mTxQueueMutex;
+    mutable std::mutex mTxQueueMutex;
 
   private:
     AppConnector& mAppConn;
@@ -259,10 +270,24 @@ class TransactionQueue
      */
     void shift();
 
-    void rebroadcast();
+    // TODO: Explain that this takes a lock guard due to the `broadcast` call
+    // that it makes.
+    void rebroadcast(std::lock_guard<std::mutex> const& guard);
+
+    // TODO: Docs
+    // Private versions of public functions that contain the actual
+    // implementation so they can be called internally when the lock is already
+    // held.
+    bool isBannedInternal(Hash const& hash) const;
+    TxFrameList getTransactionsInternal(LedgerHeader const& lcl) const;
+
+    virtual int getFloodPeriod() const = 0;
 
 #ifdef BUILD_TESTS
   public:
+    // TODO: These tests invoke protected/private functions directly that assume
+    // things are properly locked. I need to make sure these tests operate in a
+    // thread-safe manner or change them to not require private member access.
     friend class TransactionQueueTest;
 
     size_t getQueueSizeOps() const;
@@ -278,19 +303,13 @@ class SorobanTransactionQueue : public TransactionQueue
                             SearchableSnapshotConstPtr bucketSnapshot,
                             uint32 pendingDepth, uint32 banDepth,
                             uint32 poolLedgerMultiplier);
-    int
-    getFloodPeriod() const override
-    {
-        std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
-        return mValidationSnapshot->getConfig().FLOOD_SOROBAN_TX_PERIOD_MS;
-    }
 
     size_t getMaxQueueSizeOps() const override;
 #ifdef BUILD_TESTS
     void
     clearBroadcastCarryover()
     {
-        std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
+        std::lock_guard<std::mutex> guard(mTxQueueMutex);
         mBroadcastOpCarryover.clear();
         mBroadcastOpCarryover.resize(1, Resource::makeEmptySoroban());
     }
@@ -307,6 +326,13 @@ class SorobanTransactionQueue : public TransactionQueue
     {
         return true;
     }
+
+    int
+    getFloodPeriod() const override
+    {
+        return mValidationSnapshot->getConfig().FLOOD_SOROBAN_TX_PERIOD_MS;
+    }
+
 };
 
 class ClassicTransactionQueue : public TransactionQueue
@@ -317,13 +343,6 @@ class ClassicTransactionQueue : public TransactionQueue
                             uint32 pendingDepth, uint32 banDepth,
                             uint32 poolLedgerMultiplier);
 
-    int
-    getFloodPeriod() const override
-    {
-        std::lock_guard<std::recursive_mutex> guard(mTxQueueMutex);
-        return mValidationSnapshot->getConfig().FLOOD_TX_PERIOD_MS;
-    }
-
     size_t getMaxQueueSizeOps() const override;
 
   private:
@@ -335,6 +354,13 @@ class ClassicTransactionQueue : public TransactionQueue
     virtual bool broadcastSome() override;
     std::vector<Resource> mBroadcastOpCarryover;
     virtual bool allowTxBroadcast(TimestampedTx const& tx) override;
+
+    int
+    getFloodPeriod() const override
+    {
+        return mValidationSnapshot->getConfig().FLOOD_TX_PERIOD_MS;
+    }
+
 };
 
 extern std::array<const char*,

From 8b1c6042b71ffd264ac54477a65e6c5d5a5e2ae4 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Wed, 15 Jan 2025 15:53:19 -0800
Subject: [PATCH 05/14] Remove redundant passing around of protocol version

---
 src/herder/TransactionQueue.cpp                |  2 +-
 src/transactions/FeeBumpTransactionFrame.cpp   |  6 ++----
 src/transactions/FeeBumpTransactionFrame.h     |  1 -
 src/transactions/TransactionFrame.cpp          | 15 +++++++--------
 src/transactions/TransactionFrame.h            |  5 +----
 src/transactions/TransactionFrameBase.h        |  1 -
 src/transactions/test/TransactionTestFrame.cpp |  6 ++----
 src/transactions/test/TransactionTestFrame.h   |  1 -
 8 files changed, 13 insertions(+), 24 deletions(-)

diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index 0603048262..c2f92b31a6 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -388,7 +388,7 @@ TransactionQueue::canAdd(
             {
                 auto txResult = tx->createSuccessResult();
                 if (!tx->checkSorobanResourceAndSetError(
-                        *mValidationSnapshot, ledgerVersion, txResult))
+                        *mValidationSnapshot, txResult))
                 {
                     return AddResult(AddResultCode::ADD_STATUS_ERROR, txResult);
                 }
diff --git a/src/transactions/FeeBumpTransactionFrame.cpp b/src/transactions/FeeBumpTransactionFrame.cpp
index 555a514439..d3e90710d8 100644
--- a/src/transactions/FeeBumpTransactionFrame.cpp
+++ b/src/transactions/FeeBumpTransactionFrame.cpp
@@ -198,11 +198,9 @@ FeeBumpTransactionFrame::checkValid(ValidationConnector const& vc,
 
 bool
 FeeBumpTransactionFrame::checkSorobanResourceAndSetError(
-    ValidationConnector const& vc, uint32_t ledgerVersion,
-    MutableTxResultPtr txResult) const
+    ValidationConnector const& vc, MutableTxResultPtr txResult) const
 {
-    return mInnerTx->checkSorobanResourceAndSetError(vc, ledgerVersion,
-                                                     txResult);
+    return mInnerTx->checkSorobanResourceAndSetError(vc, txResult);
 }
 
 bool
diff --git a/src/transactions/FeeBumpTransactionFrame.h b/src/transactions/FeeBumpTransactionFrame.h
index 1afdd1d6c7..755f53919d 100644
--- a/src/transactions/FeeBumpTransactionFrame.h
+++ b/src/transactions/FeeBumpTransactionFrame.h
@@ -82,7 +82,6 @@ class FeeBumpTransactionFrame : public TransactionFrameBase
                uint64_t upperBoundCloseTimeOffset) const override;
     bool
     checkSorobanResourceAndSetError(ValidationConnector const& app,
-                                    uint32_t ledgerVersion,
                                     MutableTxResultPtr txResult) const override;
 
     MutableTxResultPtr createSuccessResult() const override;
diff --git a/src/transactions/TransactionFrame.cpp b/src/transactions/TransactionFrame.cpp
index e9f6524e1c..ccceb68641 100644
--- a/src/transactions/TransactionFrame.cpp
+++ b/src/transactions/TransactionFrame.cpp
@@ -646,11 +646,12 @@ TransactionFrame::validateSorobanOpsConsistency() const
 }
 
 bool
-TransactionFrame::validateSorobanResources(SorobanNetworkConfig const& config,
-                                           Config const& appConfig,
-                                           uint32_t protocolVersion,
+TransactionFrame::validateSorobanResources(ValidationConnector const& vc,
                                            SorobanTxData& sorobanData) const
 {
+    SorobanNetworkConfig const& config = vc.getSorobanNetworkConfig();
+    Config const& appConfig = vc.getConfig();
+    uint32_t protocolVersion = vc.getCurrentProtocolVersion();
     auto const& resources = sorobanResources();
     auto const& readEntries = resources.footprint.readOnly;
     auto const& writeEntries = resources.footprint.readWrite;
@@ -1054,7 +1055,7 @@ TransactionFrame::commonValidPreSeqNum(
             return false;
         }
 
-        if (!checkSorobanResourceAndSetError(vc, ledgerVersion, txResult))
+        if (!checkSorobanResourceAndSetError(vc, txResult))
         {
             return false;
         }
@@ -1610,11 +1611,9 @@ TransactionFrame::checkValid(ValidationConnector const& vc,
 
 bool
 TransactionFrame::checkSorobanResourceAndSetError(
-    ValidationConnector const& vc, uint32_t ledgerVersion,
-    MutableTxResultPtr txResult) const
+    ValidationConnector const& vc, MutableTxResultPtr txResult) const
 {
-    if (!validateSorobanResources(vc.getSorobanNetworkConfig(), vc.getConfig(),
-                                  ledgerVersion, *txResult->getSorobanData()))
+    if (!validateSorobanResources(vc, *txResult->getSorobanData()))
     {
         txResult->setInnermostResultCode(txSOROBAN_INVALID);
         return false;
diff --git a/src/transactions/TransactionFrame.h b/src/transactions/TransactionFrame.h
index dc15784d16..66d5fc4966 100644
--- a/src/transactions/TransactionFrame.h
+++ b/src/transactions/TransactionFrame.h
@@ -136,9 +136,7 @@ class TransactionFrame : public TransactionFrameBase
     bool extraSignersExist() const;
 
     bool validateSorobanOpsConsistency() const;
-    bool validateSorobanResources(SorobanNetworkConfig const& config,
-                                  Config const& appConfig,
-                                  uint32_t protocolVersion,
+    bool validateSorobanResources(ValidationConnector const& vc,
                                   SorobanTxData& sorobanData) const;
     int64_t refundSorobanFee(AbstractLedgerTxn& ltx, AccountID const& feeSource,
                              MutableTransactionResultBase& txResult) const;
@@ -218,7 +216,6 @@ class TransactionFrame : public TransactionFrameBase
                uint64_t upperBoundCloseTimeOffset) const override;
     bool
     checkSorobanResourceAndSetError(ValidationConnector const& vc,
-                                    uint32_t ledgerVersion,
                                     MutableTxResultPtr txResult) const override;
 
     MutableTxResultPtr createSuccessResult() const override;
diff --git a/src/transactions/TransactionFrameBase.h b/src/transactions/TransactionFrameBase.h
index 369b2b4d86..94c2e0a106 100644
--- a/src/transactions/TransactionFrameBase.h
+++ b/src/transactions/TransactionFrameBase.h
@@ -102,7 +102,6 @@ class TransactionFrameBase
                uint64_t upperBoundCloseTimeOffset) const = 0;
     virtual bool
     checkSorobanResourceAndSetError(ValidationConnector const& vc,
-                                    uint32_t ledgerVersion,
                                     MutableTxResultPtr txResult) const = 0;
 
     virtual MutableTxResultPtr createSuccessResult() const = 0;
diff --git a/src/transactions/test/TransactionTestFrame.cpp b/src/transactions/test/TransactionTestFrame.cpp
index 300410f22f..1da3fa1291 100644
--- a/src/transactions/test/TransactionTestFrame.cpp
+++ b/src/transactions/test/TransactionTestFrame.cpp
@@ -144,11 +144,9 @@ TransactionTestFrame::checkValidForTesting(AppConnector& app,
 
 bool
 TransactionTestFrame::checkSorobanResourceAndSetError(
-    ValidationConnector const& vc, uint32_t ledgerVersion,
-    MutableTxResultPtr txResult) const
+    ValidationConnector const& vc, MutableTxResultPtr txResult) const
 {
-    auto ret = mTransactionFrame->checkSorobanResourceAndSetError(
-        vc, ledgerVersion, txResult);
+    auto ret = mTransactionFrame->checkSorobanResourceAndSetError(vc, txResult);
     mTransactionTxResult = txResult;
     return ret;
 }
diff --git a/src/transactions/test/TransactionTestFrame.h b/src/transactions/test/TransactionTestFrame.h
index 7ee640eae9..daf0531ab8 100644
--- a/src/transactions/test/TransactionTestFrame.h
+++ b/src/transactions/test/TransactionTestFrame.h
@@ -72,7 +72,6 @@ class TransactionTestFrame : public TransactionFrameBase
                uint64_t upperBoundCloseTimeOffset) const override;
     bool
     checkSorobanResourceAndSetError(ValidationConnector const& vc,
-                                    uint32_t ledgerVersion,
                                     MutableTxResultPtr txResult) const override;
 
     MutableTxResultPtr createSuccessResult() const override;

From 8fb9865600469c277ed6d2a52d126628f615fa7e Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Wed, 15 Jan 2025 16:49:59 -0800
Subject: [PATCH 06/14] Fix error introduced in rebase

---
 src/transactions/OperationFrame.cpp | 1 -
 1 file changed, 1 deletion(-)

diff --git a/src/transactions/OperationFrame.cpp b/src/transactions/OperationFrame.cpp
index 61fcf9bdac..a3a0aed1d4 100644
--- a/src/transactions/OperationFrame.cpp
+++ b/src/transactions/OperationFrame.cpp
@@ -221,7 +221,6 @@ OperationFrame::getSourceID() const
 bool
 OperationFrame::checkValid(ValidationConnector const& vc,
                            SignatureChecker& signatureChecker,
-                           std::optional<SorobanNetworkConfig> const& cfg,
                            LedgerSnapshot const& ls, bool forApply,
                            OperationResult& res,
                            std::shared_ptr<SorobanTxData> sorobanData) const

From a08cd2738646ef94c6ccb08f2fd5719b172c791f Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Thu, 16 Jan 2025 13:47:22 -0800
Subject: [PATCH 07/14] Fix tx queue tests

---
 src/herder/TransactionQueue.cpp           | 15 +++++++-
 src/herder/TransactionQueue.h             |  6 ++-
 src/herder/test/TransactionQueueTests.cpp | 47 +++++++++++++++--------
 3 files changed, 48 insertions(+), 20 deletions(-)

diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index c2f92b31a6..9f91bbd3a9 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -387,8 +387,8 @@ TransactionQueue::canAdd(
             if (tx->isSoroban())
             {
                 auto txResult = tx->createSuccessResult();
-                if (!tx->checkSorobanResourceAndSetError(
-                        *mValidationSnapshot, txResult))
+                if (!tx->checkSorobanResourceAndSetError(*mValidationSnapshot,
+                                                         txResult))
                 {
                     return AddResult(AddResultCode::ADD_STATUS_ERROR, txResult);
                 }
@@ -1429,6 +1429,17 @@ TransactionQueue::isFiltered(TransactionFrameBasePtr tx) const
 }
 
 #ifdef BUILD_TESTS
+void
+TransactionQueue::updateSnapshots(
+    SearchableSnapshotConstPtr const& newBucketSnapshot)
+{
+    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    mValidationSnapshot =
+        std::make_shared<ImmutableValidationSnapshot>(mAppConn);
+    mBucketSnapshot = newBucketSnapshot;
+    mTxQueueLimiter.updateSnapshots(mValidationSnapshot, mBucketSnapshot);
+}
+
 size_t
 TransactionQueue::getQueueSizeOps() const
 {
diff --git a/src/herder/TransactionQueue.h b/src/herder/TransactionQueue.h
index f5bfbcbc1c..1d8047be39 100644
--- a/src/herder/TransactionQueue.h
+++ b/src/herder/TransactionQueue.h
@@ -290,9 +290,13 @@ class TransactionQueue
     // thread-safe manner or change them to not require private member access.
     friend class TransactionQueueTest;
 
+    // TODO: Docs
+    void updateSnapshots(SearchableSnapshotConstPtr const& newBucketSnapshot);
+
     size_t getQueueSizeOps() const;
     std::optional<int64_t> getInQueueSeqNum(AccountID const& account) const;
     std::function<void(TransactionFrameBasePtr&)> mTxBroadcastedEvent;
+
 #endif
 };
 
@@ -332,7 +336,6 @@ class SorobanTransactionQueue : public TransactionQueue
     {
         return mValidationSnapshot->getConfig().FLOOD_SOROBAN_TX_PERIOD_MS;
     }
-
 };
 
 class ClassicTransactionQueue : public TransactionQueue
@@ -360,7 +363,6 @@ class ClassicTransactionQueue : public TransactionQueue
     {
         return mValidationSnapshot->getConfig().FLOOD_TX_PERIOD_MS;
     }
-
 };
 
 extern std::array<const char*,
diff --git a/src/herder/test/TransactionQueueTests.cpp b/src/herder/test/TransactionQueueTests.cpp
index c6fcc42b5e..38685d8a66 100644
--- a/src/herder/test/TransactionQueueTests.cpp
+++ b/src/herder/test/TransactionQueueTests.cpp
@@ -249,6 +249,13 @@ class TransactionQueueTest
         }
     }
 
+    // TODO: Docs
+    void
+    updateSnapshots(SearchableSnapshotConstPtr const& newBucketSnapshot)
+    {
+        mTransactionQueue.updateSnapshots(newBucketSnapshot);
+    }
+
   private:
     TransactionQueue& mTransactionQueue;
 };
@@ -261,10 +268,6 @@ TEST_CASE("TransactionQueue complex scenarios", "[herder][transactionqueue]")
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto bls = app->getBucketManager()
-                   .getBucketSnapshotManager()
-                   .copySearchableLiveBucketListSnapshot();
-    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -283,6 +286,11 @@ TEST_CASE("TransactionQueue complex scenarios", "[herder][transactionqueue]")
     auto txSeqA2T2 = transaction(*app, account2, 2, 1, 200);
     auto txSeqA3T1 = transaction(*app, account3, 1, 1, 100);
 
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
+
     SECTION("multiple good sequence numbers, with four shifts")
     {
         TransactionQueueTest test{queue};
@@ -533,10 +541,6 @@ testTransactionQueueBasicScenarios()
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto bls = app->getBucketManager()
-                   .getBucketSnapshotManager()
-                   .copySearchableLiveBucketListSnapshot();
-    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -555,6 +559,11 @@ testTransactionQueueBasicScenarios()
     auto txSeqA2T2 = transaction(*app, account2, 2, 1, 200);
     auto txSeqA3T1 = transaction(*app, account3, 1, 1, 100);
 
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
+
     SECTION("simple sequence")
     {
         TransactionQueueTest test{queue};
@@ -779,10 +788,6 @@ TEST_CASE("TransactionQueue hitting the rate limit",
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto bls = app->getBucketManager()
-                   .getBucketSnapshotManager()
-                   .copySearchableLiveBucketListSnapshot();
-    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -793,6 +798,11 @@ TEST_CASE("TransactionQueue hitting the rate limit",
     auto account5 = root.create("a5", minBalance2);
     auto account6 = root.create("a6", minBalance2);
 
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
+
     TransactionQueueTest testQueue{queue};
     std::vector<TransactionFrameBasePtr> txs;
     auto addTx = [&](TransactionFrameBasePtr tx) {
@@ -863,10 +873,6 @@ TEST_CASE("TransactionQueue with PreconditionsV2", "[herder][transactionqueue]")
     cfg.TESTING_UPGRADE_MAX_TX_SET_SIZE = 4;
     cfg.FLOOD_TX_PERIOD_MS = 100;
     auto app = createTestApplication(clock, cfg);
-    auto bls = app->getBucketManager()
-                   .getBucketSnapshotManager()
-                   .copySearchableLiveBucketListSnapshot();
-    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
     auto const minBalance2 = app->getLedgerManager().getLastMinBalance(2);
 
     auto root = TestAccount::createRoot(*app);
@@ -902,6 +908,11 @@ TEST_CASE("TransactionQueue with PreconditionsV2", "[herder][transactionqueue]")
     auto txSeqA1S3MinSeqLedgerGap = transactionWithV2Precondition(
         *app, account1, 3, 200, condMinSeqLedgerGap);
 
+    auto bls = app->getBucketManager()
+                   .getBucketSnapshotManager()
+                   .copySearchableLiveBucketListSnapshot();
+    auto queue = ClassicTransactionQueue{*app, bls, 4, 2, 2};
+
     SECTION("fee bump new tx with minSeqNum past lastSeq")
     {
         PreconditionsV2 cond;
@@ -2484,6 +2495,10 @@ TEST_CASE("transaction queue with fee-bump", "[herder][transactionqueue]")
                     auto fb2 = feeBump(*app, account3, tx2,
                                        fb1->getInclusionFee() * 10);
 
+                    auto bls = app->getBucketManager()
+                                   .getBucketSnapshotManager()
+                                   .copySearchableLiveBucketListSnapshot();
+                    test.updateSnapshots(bls);
                     test.add(
                         fb2,
                         TransactionQueue::AddResultCode::ADD_STATUS_PENDING);

From 4beceee729901b9429dd8e0af9f281cce7e46dee Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Fri, 17 Jan 2025 15:55:31 -0800
Subject: [PATCH 08/14] Fix integration with bg ledger close

---
 src/herder/TxSetUtils.cpp                     |  2 +-
 src/simulation/ApplyLoad.cpp                  |  2 +-
 src/test/FuzzerImpl.cpp                       |  2 +-
 src/transactions/OperationFrame.cpp           |  3 ++-
 src/transactions/TransactionFrame.cpp         |  2 +-
 src/transactions/TransactionFrameBase.cpp     | 23 +++++++++++++++----
 src/transactions/TransactionFrameBase.h       |  6 +++--
 .../test/InvokeHostFunctionTests.cpp          |  9 +++++---
 src/transactions/test/SorobanTxTestUtils.cpp  |  9 +++++---
 .../test/TransactionTestFrame.cpp             |  2 +-
 10 files changed, 42 insertions(+), 18 deletions(-)

diff --git a/src/herder/TxSetUtils.cpp b/src/herder/TxSetUtils.cpp
index 1a51b5e1c7..ca9fa1ddfe 100644
--- a/src/herder/TxSetUtils.cpp
+++ b/src/herder/TxSetUtils.cpp
@@ -176,7 +176,7 @@ TxSetUtils::getInvalidTxList(TxFrameList const& txs, Application& app,
 
     TxFrameList invalidTxs;
 
-    AppValidationWrapper avw(app.getAppConnector(), false);
+    AppValidationWrapper avw(app.getAppConnector(), false, std::nullopt);
     for (auto const& tx : txs)
     {
         auto txResult = tx->checkValid(avw, ls, 0, lowerBoundCloseTimeOffset,
diff --git a/src/simulation/ApplyLoad.cpp b/src/simulation/ApplyLoad.cpp
index 0aa875d9bf..e3126f6988 100644
--- a/src/simulation/ApplyLoad.cpp
+++ b/src/simulation/ApplyLoad.cpp
@@ -396,7 +396,7 @@ ApplyLoad::benchmark()
                      gRandomEngine);
 
     bool limitHit = false;
-    AppValidationWrapper avw(mApp.getAppConnector(), false);
+    AppValidationWrapper avw(mApp.getAppConnector(), false, std::nullopt);
     for (auto accountIndex : shuffledAccounts)
     {
         auto it = accounts.find(accountIndex);
diff --git a/src/test/FuzzerImpl.cpp b/src/test/FuzzerImpl.cpp
index fa47428dc1..aaae85b8a8 100644
--- a/src/test/FuzzerImpl.cpp
+++ b/src/test/FuzzerImpl.cpp
@@ -921,7 +921,7 @@ class FuzzTransactionFrame : public TransactionFrame
 
         // attempt application of transaction without processing the fee or
         // committing the LedgerTxn
-        AppValidationWrapper avw(app.getAppConnector(), false);
+        AppValidationWrapper avw(app.getAppConnector(), false, std::nullopt);
         SignatureChecker signatureChecker{
             ltx.loadHeader().current().ledgerVersion, getContentsHash(),
             mEnvelope.v1().signatures};
diff --git a/src/transactions/OperationFrame.cpp b/src/transactions/OperationFrame.cpp
index a3a0aed1d4..097967cb99 100644
--- a/src/transactions/OperationFrame.cpp
+++ b/src/transactions/OperationFrame.cpp
@@ -144,8 +144,9 @@ OperationFrame::apply(AppConnector& app, SignatureChecker& signatureChecker,
     ZoneScoped;
     CLOG_TRACE(Tx, "{}", xdrToCerealString(mOperation, "Operation"));
 
-    AppValidationWrapper avw(app, true);
     LedgerSnapshot ltxState(ltx);
+    AppValidationWrapper avw(
+        app, true, ltxState.getLedgerHeader().current().ledgerVersion);
     bool applyRes =
         checkValid(avw, signatureChecker, ltxState, true, res, sorobanData);
     if (applyRes)
diff --git a/src/transactions/TransactionFrame.cpp b/src/transactions/TransactionFrame.cpp
index ccceb68641..9a8768290b 100644
--- a/src/transactions/TransactionFrame.cpp
+++ b/src/transactions/TransactionFrame.cpp
@@ -1946,7 +1946,7 @@ TransactionFrame::apply(AppConnector& app, AbstractLedgerTxn& ltx,
         //  we'll skip trying to apply operations but we'll still
         //  process the sequence number if needed
         std::optional<FeePair> sorobanResourceFee;
-        AppValidationWrapper avw(app, true);
+        AppValidationWrapper avw(app, true, ledgerVersion);
         if (protocolVersionStartsFrom(ledgerVersion,
                                       SOROBAN_PROTOCOL_VERSION) &&
             isSoroban())
diff --git a/src/transactions/TransactionFrameBase.cpp b/src/transactions/TransactionFrameBase.cpp
index 8bfdb95c57..5eb61f48a2 100644
--- a/src/transactions/TransactionFrameBase.cpp
+++ b/src/transactions/TransactionFrameBase.cpp
@@ -11,10 +11,14 @@
 namespace stellar
 {
 
-AppValidationWrapper::AppValidationWrapper(AppConnector const& app,
-                                           bool forApply)
-    : mApp(app), mForApply(forApply)
+AppValidationWrapper::AppValidationWrapper(
+    AppConnector const& app, bool forApply,
+    std::optional<uint32_t> protocolVersion)
+    : mApp(app), mForApply(forApply), mProtocolVersion(protocolVersion)
 {
+    // Must supply protocolVersion if running this on a background thread (such
+    // as in the background apply flow).
+    releaseAssert(threadIsMain() || protocolVersion.has_value());
 }
 
 Config const&
@@ -26,13 +30,24 @@ AppValidationWrapper::getConfig() const
 SorobanNetworkConfig const&
 AppValidationWrapper::getSorobanNetworkConfig() const
 {
-    return mForApply ? mApp.getLedgerManager().getSorobanNetworkConfigForApply()
+    return mForApply ? mApp.getSorobanNetworkConfigForApply()
                      : mApp.getSorobanNetworkConfigReadOnly();
 }
 
 uint32_t
 AppValidationWrapper::getCurrentProtocolVersion() const
 {
+    if (mProtocolVersion.has_value())
+    {
+        return mProtocolVersion.value();
+    }
+    // TODO: This use of `getLedgerManager` doesn't play nice with background
+    // ledger close. `getLedgerManager` mandates that it runs in the main
+    // thread, but there are calls to this function via the apply flow that are
+    // not in the main thread. I've instead added an optional protocol version
+    // to store, and fall back on this when it isn't available. I don't like
+    // this solution much though, and would prefer to just use the ledger
+    // snapshot directly once I get rid of ValidationConnector.
     return mApp.getLedgerManager()
         .getLastClosedLedgerHeader()
         .header.ledgerVersion;
diff --git a/src/transactions/TransactionFrameBase.h b/src/transactions/TransactionFrameBase.h
index 94c2e0a106..ef847e906c 100644
--- a/src/transactions/TransactionFrameBase.h
+++ b/src/transactions/TransactionFrameBase.h
@@ -46,14 +46,15 @@ class ValidationConnector
     virtual ~ValidationConnector() = default;
     virtual Config const& getConfig() const = 0;
     virtual SorobanNetworkConfig const& getSorobanNetworkConfig() const = 0;
-    virtual uint32_t getCurrentProtocolVersion() const = 0;
+    virtual uint34_t getCurrentProtocolVersion() const = 0;
 };
 
 // TODO: Docs
 class AppValidationWrapper : public ValidationConnector
 {
   public:
-    explicit AppValidationWrapper(AppConnector const& app, bool forApply);
+    explicit AppValidationWrapper(AppConnector const& app, bool forApply,
+                                  std::optional<uint32_t> protocolVersion);
     ~AppValidationWrapper() override = default;
 
     Config const& getConfig() const override;
@@ -63,6 +64,7 @@ class AppValidationWrapper : public ValidationConnector
   private:
     AppConnector const& mApp;
     bool const mForApply;
+    std::optional<uint32_t> const mProtocolVersion;
 };
 
 // TODO: Docs
diff --git a/src/transactions/test/InvokeHostFunctionTests.cpp b/src/transactions/test/InvokeHostFunctionTests.cpp
index ec5311db46..ef1855dc63 100644
--- a/src/transactions/test/InvokeHostFunctionTests.cpp
+++ b/src/transactions/test/InvokeHostFunctionTests.cpp
@@ -421,7 +421,8 @@ TEST_CASE("basic contract invocation", "[tx][soroban]")
                                                      addContractKeys);
         auto tx = invocation.createTx(&rootAccount);
 
-        AppValidationWrapper avw(test.getApp().getAppConnector(), false);
+        AppValidationWrapper avw(test.getApp().getAppConnector(), false,
+                                 std::nullopt);
         auto result = tx->checkValid(avw, rootLtx, 0, 0, 0);
 
         REQUIRE(tx->getFullFee() ==
@@ -742,7 +743,8 @@ TEST_CASE("Soroban footprint validation", "[tx][soroban]")
 {
     SorobanTest test;
     auto const& cfg = test.getNetworkCfg();
-    AppValidationWrapper const avw(test.getApp().getAppConnector(), false);
+    AppValidationWrapper const avw(test.getApp().getAppConnector(), false,
+                                   std::nullopt);
 
     auto& addContract =
         test.deployWasmContract(rust_bridge::get_test_wasm_add_i32());
@@ -1454,7 +1456,8 @@ TEST_CASE("transaction validation diagnostics", "[tx][soroban]")
             .createTx();
     MutableTxResultPtr result;
     {
-        AppValidationWrapper const avw(test.getApp().getAppConnector(), false);
+        AppValidationWrapper const avw(test.getApp().getAppConnector(), false,
+                                       std::nullopt);
         LedgerTxn ltx(test.getApp().getLedgerTxnRoot());
         result = tx->checkValid(avw, ltx, 0, 0, 0);
     }
diff --git a/src/transactions/test/SorobanTxTestUtils.cpp b/src/transactions/test/SorobanTxTestUtils.cpp
index b498826c94..889ccc8e47 100644
--- a/src/transactions/test/SorobanTxTestUtils.cpp
+++ b/src/transactions/test/SorobanTxTestUtils.cpp
@@ -820,7 +820,8 @@ SorobanTest::invokeArchivalOp(TransactionFrameBaseConstPtr tx,
 {
     MutableTxResultPtr result;
     {
-        AppValidationWrapper const avw(getApp().getAppConnector(), false);
+        AppValidationWrapper const avw(getApp().getAppConnector(), false,
+                                       std::nullopt);
         LedgerTxn ltx(getApp().getLedgerTxnRoot());
         result = tx->checkValid(avw, ltx, 0, 0, 0);
     }
@@ -1101,7 +1102,8 @@ SorobanTest::createRestoreTx(SorobanResources const& resources, uint32_t fee,
 bool
 SorobanTest::isTxValid(TransactionFrameBaseConstPtr tx)
 {
-    AppValidationWrapper const avw(getApp().getAppConnector(), false);
+    AppValidationWrapper const avw(getApp().getAppConnector(), false,
+                                   std::nullopt);
     LedgerTxn ltx(getApp().getLedgerTxnRoot());
     auto ret = tx->checkValid(avw, ltx, 0, 0, 0);
     return ret->isSuccess();
@@ -1112,7 +1114,8 @@ SorobanTest::invokeTx(TransactionFrameBaseConstPtr tx,
                       TransactionMetaFrame* txMeta)
 {
     {
-        AppValidationWrapper const avw(getApp().getAppConnector(), false);
+        AppValidationWrapper const avw(getApp().getAppConnector(), false,
+                                       std::nullopt);
         LedgerTxn ltx(getApp().getLedgerTxnRoot());
         REQUIRE(tx->checkValid(avw, ltx, 0, 0, 0)->isSuccess());
     }
diff --git a/src/transactions/test/TransactionTestFrame.cpp b/src/transactions/test/TransactionTestFrame.cpp
index 1da3fa1291..41428f5dc4 100644
--- a/src/transactions/test/TransactionTestFrame.cpp
+++ b/src/transactions/test/TransactionTestFrame.cpp
@@ -95,7 +95,7 @@ TransactionTestFrame::checkValid(AppConnector& app, AbstractLedgerTxn& ltxOuter,
                                  uint64_t upperBoundCloseTimeOffset) const
 {
     LedgerTxn ltx(ltxOuter);
-    AppValidationWrapper const avw(app, false);
+    AppValidationWrapper const avw(app, false, std::nullopt);
     auto ls = LedgerSnapshot(ltx);
     mTransactionTxResult = mTransactionFrame->checkValid(
         avw, ls, current, lowerBoundCloseTimeOffset, upperBoundCloseTimeOffset);

From 9d788080bb08646e4278696d3917d4b263bfd7d0 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Tue, 21 Jan 2025 10:59:31 -0800
Subject: [PATCH 09/14] Typo

---
 src/transactions/TransactionFrameBase.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/transactions/TransactionFrameBase.h b/src/transactions/TransactionFrameBase.h
index ef847e906c..a8b1020651 100644
--- a/src/transactions/TransactionFrameBase.h
+++ b/src/transactions/TransactionFrameBase.h
@@ -46,7 +46,7 @@ class ValidationConnector
     virtual ~ValidationConnector() = default;
     virtual Config const& getConfig() const = 0;
     virtual SorobanNetworkConfig const& getSorobanNetworkConfig() const = 0;
-    virtual uint34_t getCurrentProtocolVersion() const = 0;
+    virtual uint32_t getCurrentProtocolVersion() const = 0;
 };
 
 // TODO: Docs

From 0e02e8222ebaad9d0d1d2a410dad7af7865f030b Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Tue, 21 Jan 2025 18:35:26 -0800
Subject: [PATCH 10/14] Push cut point out through receiving messages

Push cut point out a tiny bit

todo

Identify where to push back next

Atomic ledger state

Refactor out tx metrics from queue adding

In preparation for moving the cut point

Consistency

Getting close to having this all pushed out

Just need to wire up the final bit

Remove old post to bg thread

Indirection around tx queue storage

Feed tx queues to peer

Wire up background flow

Debug output

Fix tracy build

More debug info around broadcasting

Maybe fix deadlock

Fix deadlock with TCPPeer

Reduce main thread posts on broadcast

Add thread

Though it's not being used anywhere

Actually run on new thread

Another memory leak fix

Fix `maybeExecuteInBackground` to check thread type

Save main thread type
---
 src/herder/Herder.h                   |   8 +
 src/herder/HerderImpl.cpp             | 143 +++++++-------
 src/herder/HerderImpl.h               |   9 +-
 src/herder/TransactionQueue.cpp       | 139 +++++++++++++-
 src/herder/TransactionQueue.h         | 101 +++++++---
 src/ledger/LedgerManager.h            |   6 +
 src/ledger/LedgerManagerImpl.cpp      |  27 +--
 src/ledger/LedgerManagerImpl.h        |   2 +-
 src/main/AppConnector.cpp             |  14 ++
 src/main/AppConnector.h               |   6 +-
 src/main/Application.h                |   7 +-
 src/main/ApplicationImpl.cpp          |  39 ++++
 src/main/ApplicationImpl.h            |   7 +
 src/overlay/Floodgate.cpp             |   3 +-
 src/overlay/Floodgate.h               |   3 +-
 src/overlay/OverlayManager.h          |  14 +-
 src/overlay/OverlayManagerImpl.cpp    |  91 +++++----
 src/overlay/OverlayManagerImpl.h      |   6 +-
 src/overlay/Peer.cpp                  | 260 +++++++++++++++++---------
 src/overlay/Peer.h                    |   3 +
 src/overlay/TCPPeer.cpp               |  10 +-
 src/overlay/test/ItemFetcherTests.cpp |   3 +-
 22 files changed, 630 insertions(+), 271 deletions(-)

diff --git a/src/herder/Herder.h b/src/herder/Herder.h
index ba64929101..6b701c5fe1 100644
--- a/src/herder/Herder.h
+++ b/src/herder/Herder.h
@@ -225,5 +225,13 @@ class Herder
     virtual TransactionFrameBaseConstPtr getTx(Hash const& hash) const = 0;
 
     virtual void beginApply() = 0;
+
+    // TODO: Docs
+    virtual TransactionQueuesPtr getTransactionQueues() const = 0;
+
+    // TODO: Docs
+    static TransactionQueue::AddResult
+    recvTransaction(TransactionQueuesPtr txQueues, TransactionFrameBasePtr tx,
+                    bool submittedFromSelf);
 };
 }
diff --git a/src/herder/HerderImpl.cpp b/src/herder/HerderImpl.cpp
index 8b0513dabe..b27ea16593 100644
--- a/src/herder/HerderImpl.cpp
+++ b/src/herder/HerderImpl.cpp
@@ -272,14 +272,7 @@ HerderImpl::shutdown()
                    "Shutdown interrupting quorum transitive closure analysis.");
         mLastQuorumMapIntersectionState.mInterruptFlag = true;
     }
-    if (mTransactionQueue)
-    {
-        mTransactionQueue->shutdown();
-    }
-    if (mSorobanTransactionQueue)
-    {
-        mSorobanTransactionQueue->shutdown();
-    }
+    mTransactionQueues->shutdown();
 
     mTxSetGarbageCollectTimer.cancel();
 }
@@ -589,22 +582,35 @@ HerderImpl::emitEnvelope(SCPEnvelope const& envelope)
     broadcast(envelope);
 }
 
+// TODO: Move to Herder.cpp?
 TransactionQueue::AddResult
-HerderImpl::recvTransaction(TransactionFrameBasePtr tx, bool submittedFromSelf)
+Herder::recvTransaction(TransactionQueuesPtr txQueues,
+                            TransactionFrameBasePtr tx, bool submittedFromSelf)
 {
     ZoneScoped;
+    ClassicTransactionQueue& classicTxQueue =
+        txQueues->getClassicTransactionQueue();
     TransactionQueue::AddResult result(
         TransactionQueue::AddResultCode::ADD_STATUS_COUNT);
 
     // Allow txs of the same kind to reach the tx queue in case it can be
     // replaced by fee
+    // TODO: Is there a potential TOCTOU issue here as sourceAccountPending
+    // could change before adding? I think no because the other competing thread
+    // would be whatever is handling ledger close. However, that will only
+    // decrease the sourceAccountPending value, which means this erroneously
+    // rejects (which is safe). I guess it's possible for a user-submitted
+    // transaction to come in and conflict with the overlay thread, but that
+    // would require them to be simultaneously running two clients and
+    // submitting from both of them. Still, it might be safest to use some kind
+    // of atomic function that handles both this check AND the add.
     bool hasSoroban =
-        mSorobanTransactionQueue &&
-        mSorobanTransactionQueue->sourceAccountPending(tx->getSourceID()) &&
+        txQueues->hasSorobanTransactionQueue() &&
+        txQueues->getSorobanTransactionQueue().sourceAccountPending(
+            tx->getSourceID()) &&
         !tx->isSoroban();
-    bool hasClassic =
-        mTransactionQueue->sourceAccountPending(tx->getSourceID()) &&
-        tx->isSoroban();
+    bool hasClassic = classicTxQueue.sourceAccountPending(tx->getSourceID()) &&
+                      tx->isSoroban();
     if (hasSoroban || hasClassic)
     {
         CLOG_DEBUG(Herder,
@@ -617,31 +623,12 @@ HerderImpl::recvTransaction(TransactionFrameBasePtr tx, bool submittedFromSelf)
     }
     else if (!tx->isSoroban())
     {
-        if (mApp.getConfig().BACKGROUND_TX_QUEUE && !submittedFromSelf)
-        {
-            mApp.postOnOverlayThread(
-                [this, tx]() { mTransactionQueue->tryAdd(tx, false); },
-                "try add tx");
-            result.code = TransactionQueue::AddResultCode::ADD_STATUS_UNKNOWN;
-        }
-        else
-        {
-            result = mTransactionQueue->tryAdd(tx, submittedFromSelf);
-        }
+        result = classicTxQueue.tryAdd(tx, submittedFromSelf);
     }
-    else if (mSorobanTransactionQueue)
+    else if (txQueues->hasSorobanTransactionQueue())
     {
-        if (mApp.getConfig().BACKGROUND_TX_QUEUE && !submittedFromSelf)
-        {
-            mApp.postOnOverlayThread(
-                [this, tx]() { mSorobanTransactionQueue->tryAdd(tx, false); },
-                "try add tx");
-            result.code = TransactionQueue::AddResultCode::ADD_STATUS_UNKNOWN;
-        }
-        else
-        {
-            result = mSorobanTransactionQueue->tryAdd(tx, submittedFromSelf);
-        }
+        result = txQueues->getSorobanTransactionQueue().tryAdd(
+            tx, submittedFromSelf);
     }
     else
     {
@@ -661,6 +648,13 @@ HerderImpl::recvTransaction(TransactionFrameBasePtr tx, bool submittedFromSelf)
     return result;
 }
 
+TransactionQueue::AddResult
+HerderImpl::recvTransaction(TransactionFrameBasePtr tx, bool submittedFromSelf)
+{
+    ZoneScoped;
+    return Herder::recvTransaction(mTransactionQueues, tx, submittedFromSelf);
+}
+
 bool
 HerderImpl::checkCloseTime(SCPEnvelope const& envelope, bool enforceRecent)
 {
@@ -943,13 +937,7 @@ HerderImpl::externalizeValue(TxSetXDRFrameConstPtr txSet, uint32_t ledgerSeq,
 bool
 HerderImpl::sourceAccountPending(AccountID const& accountID) const
 {
-    bool accPending = mTransactionQueue->sourceAccountPending(accountID);
-    if (mSorobanTransactionQueue)
-    {
-        accPending = accPending ||
-                     mSorobanTransactionQueue->sourceAccountPending(accountID);
-    }
-    return accPending;
+    return mTransactionQueues->sourceAccountPending(accountID);
 }
 
 #endif
@@ -1112,13 +1100,12 @@ HerderImpl::getPendingEnvelopes()
 ClassicTransactionQueue&
 HerderImpl::getTransactionQueue()
 {
-    return *mTransactionQueue;
+    return mTransactionQueues->getClassicTransactionQueue();
 }
 SorobanTransactionQueue&
 HerderImpl::getSorobanTransactionQueue()
 {
-    releaseAssert(mSorobanTransactionQueue);
-    return *mSorobanTransactionQueue;
+    return mTransactionQueues->getSorobanTransactionQueue();
 }
 #endif
 
@@ -1411,14 +1398,16 @@ HerderImpl::triggerNextLedger(uint32_t ledgerSeqToTrigger,
     // it's guaranteed to be up-to-date
     auto const& lcl = mLedgerManager.getLastClosedLedgerHeader();
     PerPhaseTransactionList txPhases;
-    txPhases.emplace_back(mTransactionQueue->getTransactions(lcl.header));
+    txPhases.emplace_back(
+        mTransactionQueues->getClassicTransactionQueue().getTransactions(
+            lcl.header));
 
     if (protocolVersionStartsFrom(lcl.header.ledgerVersion,
                                   SOROBAN_PROTOCOL_VERSION))
     {
-        releaseAssert(mSorobanTransactionQueue);
         txPhases.emplace_back(
-            mSorobanTransactionQueue->getTransactions(lcl.header));
+            mTransactionQueues->getSorobanTransactionQueue().getTransactions(
+                lcl.header));
     }
 
     // We pick as next close time the current time unless it's before the last
@@ -1485,12 +1474,11 @@ HerderImpl::triggerNextLedger(uint32_t ledgerSeqToTrigger,
     if (protocolVersionStartsFrom(lcl.header.ledgerVersion,
                                   SOROBAN_PROTOCOL_VERSION))
     {
-        releaseAssert(mSorobanTransactionQueue);
-        mSorobanTransactionQueue->ban(
+        mTransactionQueues->getSorobanTransactionQueue().ban(
             invalidTxPhases[static_cast<size_t>(TxSetPhase::SOROBAN)]);
     }
 
-    mTransactionQueue->ban(
+    mTransactionQueues->getClassicTransactionQueue().ban(
         invalidTxPhases[static_cast<size_t>(TxSetPhase::CLASSIC)]);
 
     auto txSetHash = proposedSet->getContentsHash();
@@ -2190,18 +2178,18 @@ HerderImpl::maybeSetupSorobanQueue(uint32_t protocolVersion)
 {
     if (protocolVersionStartsFrom(protocolVersion, SOROBAN_PROTOCOL_VERSION))
     {
-        if (!mSorobanTransactionQueue)
+        if (!mTransactionQueues->hasSorobanTransactionQueue())
         {
             releaseAssert(mTxQueueBucketSnapshot);
-            mSorobanTransactionQueue =
+            mTransactionQueues->setSorobanTransactionQueue(
                 std::make_unique<SorobanTransactionQueue>(
                     mApp, mTxQueueBucketSnapshot,
                     TRANSACTION_QUEUE_TIMEOUT_LEDGERS,
                     TRANSACTION_QUEUE_BAN_LEDGERS,
-                    SOROBAN_TRANSACTION_QUEUE_SIZE_MULTIPLIER);
+                    SOROBAN_TRANSACTION_QUEUE_SIZE_MULTIPLIER));
         }
     }
-    else if (mSorobanTransactionQueue)
+    else if (mTransactionQueues->hasSorobanTransactionQueue())
     {
         throw std::runtime_error(
             "Invalid state: Soroban queue initialized before v20");
@@ -2215,10 +2203,10 @@ HerderImpl::start()
     mTxQueueBucketSnapshot = mApp.getBucketManager()
                                  .getBucketSnapshotManager()
                                  .copySearchableLiveBucketListSnapshot();
-    releaseAssert(!mTransactionQueue);
-    mTransactionQueue = std::make_unique<ClassicTransactionQueue>(
-        mApp, mTxQueueBucketSnapshot, TRANSACTION_QUEUE_TIMEOUT_LEDGERS,
-        TRANSACTION_QUEUE_BAN_LEDGERS, TRANSACTION_QUEUE_SIZE_MULTIPLIER);
+    mTransactionQueues->setClassicTransactionQueue(
+        std::make_unique<ClassicTransactionQueue>(
+            mApp, mTxQueueBucketSnapshot, TRANSACTION_QUEUE_TIMEOUT_LEDGERS,
+            TRANSACTION_QUEUE_BAN_LEDGERS, TRANSACTION_QUEUE_SIZE_MULTIPLIER));
 
     mMaxTxSize = mApp.getHerder().getMaxClassicTxSize();
     {
@@ -2378,7 +2366,7 @@ HerderImpl::updateTransactionQueue(TxSetXDRFrameConstPtr externalizedTxSet)
         .maybeCopySearchableBucketListSnapshot(mTxQueueBucketSnapshot);
     if (txsPerPhase.size() > static_cast<size_t>(TxSetPhase::CLASSIC))
     {
-        mTransactionQueue->update(
+        mTransactionQueues->getClassicTransactionQueue().update(
             txsPerPhase[static_cast<size_t>(TxSetPhase::CLASSIC)], lhhe.header,
             mTxQueueBucketSnapshot, filterInvalidTxs);
     }
@@ -2386,10 +2374,10 @@ HerderImpl::updateTransactionQueue(TxSetXDRFrameConstPtr externalizedTxSet)
     // Even if we're in protocol 20, still check for number of phases, in case
     // we're dealing with the upgrade ledger that contains old-style transaction
     // set
-    if (mSorobanTransactionQueue != nullptr &&
+    if (mTransactionQueues->hasSorobanTransactionQueue() &&
         txsPerPhase.size() > static_cast<size_t>(TxSetPhase::SOROBAN))
     {
-        mSorobanTransactionQueue->update(
+        mTransactionQueues->getSorobanTransactionQueue().update(
             txsPerPhase[static_cast<size_t>(TxSetPhase::SOROBAN)], lhhe.header,
             mTxQueueBucketSnapshot, filterInvalidTxs);
     }
@@ -2508,37 +2496,34 @@ HerderImpl::isNewerNominationOrBallotSt(SCPStatement const& oldSt,
 size_t
 HerderImpl::getMaxQueueSizeOps() const
 {
-    return mTransactionQueue->getMaxQueueSizeOps();
+    return mTransactionQueues->getClassicTransactionQueue()
+        .getMaxQueueSizeOps();
 }
 
 size_t
 HerderImpl::getMaxQueueSizeSorobanOps() const
 {
-    return mSorobanTransactionQueue
-               ? mSorobanTransactionQueue->getMaxQueueSizeOps()
+    return mTransactionQueues->hasSorobanTransactionQueue()
+               ? mTransactionQueues->getSorobanTransactionQueue()
+                     .getMaxQueueSizeOps()
                : 0;
 }
 
 bool
 HerderImpl::isBannedTx(Hash const& hash) const
 {
-    auto banned = mTransactionQueue->isBanned(hash);
-    if (mSorobanTransactionQueue)
-    {
-        banned = banned || mSorobanTransactionQueue->isBanned(hash);
-    }
-    return banned;
+    return mTransactionQueues->isBanned(hash);
 }
 
 TransactionFrameBaseConstPtr
 HerderImpl::getTx(Hash const& hash) const
 {
-    auto classic = mTransactionQueue->getTx(hash);
-    if (!classic && mSorobanTransactionQueue)
-    {
-        return mSorobanTransactionQueue->getTx(hash);
-    }
-    return classic;
+    return mTransactionQueues->getTx(hash);
+}
+
+TransactionQueuesPtr HerderImpl::getTransactionQueues() const {
+    releaseAssert(mTransactionQueues);
+    return mTransactionQueues;
 }
 
 }
diff --git a/src/herder/HerderImpl.h b/src/herder/HerderImpl.h
index f3b05751e0..6ebfb994a3 100644
--- a/src/herder/HerderImpl.h
+++ b/src/herder/HerderImpl.h
@@ -198,6 +198,8 @@ class HerderImpl : public Herder
 
     virtual void beginApply() override;
 
+    TransactionQueuesPtr getTransactionQueues() const override;
+
     void startTxSetGCTimer();
 
 #ifdef BUILD_TESTS
@@ -248,8 +250,11 @@ class HerderImpl : public Herder
     void purgeOldPersistedTxSets();
     void writeDebugTxSet(LedgerCloseData const& lcd);
 
-    std::unique_ptr<ClassicTransactionQueue> mTransactionQueue;
-    std::unique_ptr<SorobanTransactionQueue> mSorobanTransactionQueue;
+    // TODO: Need some way to get these queues
+    // TODO: Maybe something else should create this and pass it in somehow,
+    // either via Application or explicitly in the constructor for HerderImpl.
+    TransactionQueuesPtr const mTransactionQueues =
+        std::make_shared<TransactionQueues>();
 
     void updateTransactionQueue(TxSetXDRFrameConstPtr txSet);
     void maybeSetupSorobanQueue(uint32_t protocolVersion);
diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index 9f91bbd3a9..d158a3c2bd 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -52,21 +52,21 @@ std::array<const char*,
     TX_STATUS_STRING = std::array{"PENDING", "DUPLICATE", "ERROR",
                                   "TRY_AGAIN_LATER", "FILTERED"};
 
-TransactionQueue::AddResult::AddResult(AddResultCode addCode)
+TxQueueAddResult::TxQueueAddResult(TxQueueAddResultCode addCode)
     : code(addCode), txResult()
 {
 }
 
-TransactionQueue::AddResult::AddResult(AddResultCode addCode,
-                                       MutableTxResultPtr payload)
+TxQueueAddResult::TxQueueAddResult(TxQueueAddResultCode addCode,
+                                   MutableTxResultPtr payload)
     : code(addCode), txResult(payload)
 {
     releaseAssert(txResult);
 }
 
-TransactionQueue::AddResult::AddResult(AddResultCode addCode,
-                                       TransactionFrameBasePtr tx,
-                                       TransactionResultCode txErrorCode)
+TxQueueAddResult::TxQueueAddResult(TxQueueAddResultCode addCode,
+                                   TransactionFrameBasePtr tx,
+                                   TransactionResultCode txErrorCode)
     : code(addCode), txResult(tx->createSuccessResult())
 {
     releaseAssert(txErrorCode != txSUCCESS);
@@ -642,6 +642,8 @@ TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
 {
     ZoneScoped;
     std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    CLOG_DEBUG(Tx, "Try add tx {} in {}", hexAbbrev(tx->getFullHash()),
+               threadIsMain() ? "foreground" : "background");
 
     auto c1 =
         tx->getEnvelope().type() == ENVELOPE_TYPE_TX_FEE_BUMP &&
@@ -703,8 +705,21 @@ TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
     mTxQueueLimiter.addTransaction(tx);
     mKnownTxHashes[tx->getFullHash()] = tx;
 
-    mAppConn.postOnMainThread([this]() { broadcast(false); },
-                              "tx queue broadcast");
+    if (threadIsMain())
+    {
+        broadcast(false, guard);
+    }
+    else if (!mWaiting && !mPendingMainThreadBroadcast)
+    {
+        // NOTE: If mWaiting is set, then the broadcast timer is already running
+        // and there is no need to take up main thread time to start it.
+        // Similarly, if mPendingMainThreadBroadcast is set, then there is
+        // already something enqueued to be broadcast on the main thread, which
+        // will result in this transaction being broadcast too.
+        mPendingMainThreadBroadcast = true;
+        mAppConn.postOnMainThread([this]() { broadcast(false); },
+                                  "tx queue broadcast");
+    }
 
     return res;
 }
@@ -1052,8 +1067,12 @@ TransactionQueue::broadcastTx(TimestampedTx& tx)
     // Must be main thread because we are accessing the overlay manager
     releaseAssert(threadIsMain());
 
+    // TODO: Remove
+    std::string txStr = hexAbbrev(tx.mTx->getFullHash());
+
     if (tx.mBroadcasted)
     {
+        CLOG_DEBUG(Tx, "Transaction {} already broadcasted", txStr);
         return BroadcastStatus::BROADCAST_STATUS_ALREADY;
     }
 
@@ -1078,8 +1097,10 @@ TransactionQueue::broadcastTx(TimestampedTx& tx)
         // false to our caller so that they will not count this tx against
         // the per-timeslice counters -- we want to allow the caller to try
         // useful work from other sources.
+        CLOG_DEBUG(Tx, "Transaction {} skipped", txStr);
         return BroadcastStatus::BROADCAST_STATUS_SKIPPED;
     }
+    CLOG_DEBUG(Tx, "Broadcasting transaction {}", txStr);
     return mAppConn.getOverlayManager().broadcastMessage(
                tx.mTx->toStellarMessage(),
                std::make_optional<Hash>(tx.mTx->getFullHash()))
@@ -1302,6 +1323,7 @@ TransactionQueue::broadcast(bool fromCallback,
     // Must be called from the main thread due to the use of `mBroadcastTimer`
     releaseAssert(threadIsMain());
 
+    mPendingMainThreadBroadcast = false;
     if (mShutdown || (!fromCallback && mWaiting))
     {
         return;
@@ -1473,4 +1495,105 @@ ClassicTransactionQueue::getMaxQueueSizeOps() const
     releaseAssert(res.size() == NUM_CLASSIC_TX_RESOURCES);
     return res.getVal(Resource::Type::OPERATIONS);
 }
+
+void
+TransactionQueues::setClassicTransactionQueue(
+    std::unique_ptr<ClassicTransactionQueue> classicTransactionQueue)
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    releaseAssert(!mClassicTransactionQueue);
+    mClassicTransactionQueue = std::move(classicTransactionQueue);
+}
+
+void
+TransactionQueues::setSorobanTransactionQueue(
+    std::unique_ptr<SorobanTransactionQueue> sorobanTransactionQueue)
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    releaseAssert(!mSorobanTransactionQueue);
+    mSorobanTransactionQueue = std::move(sorobanTransactionQueue);
+}
+
+bool
+TransactionQueues::hasClassicTransactionQueue() const
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    return mClassicTransactionQueue != nullptr;
+}
+
+bool
+TransactionQueues::hasSorobanTransactionQueue() const
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    return mSorobanTransactionQueue != nullptr;
+}
+
+ClassicTransactionQueue&
+TransactionQueues::getClassicTransactionQueue() const
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    releaseAssert(mClassicTransactionQueue);
+    return *mClassicTransactionQueue;
+}
+
+SorobanTransactionQueue&
+TransactionQueues::getSorobanTransactionQueue() const
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    releaseAssert(mSorobanTransactionQueue);
+    return *mSorobanTransactionQueue;
+}
+
+void
+TransactionQueues::shutdown()
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    if (mClassicTransactionQueue)
+    {
+        mClassicTransactionQueue->shutdown();
+    }
+    if (mSorobanTransactionQueue)
+    {
+        mSorobanTransactionQueue->shutdown();
+    }
+}
+
+bool
+TransactionQueues::sourceAccountPending(AccountID const& accountID) const
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    releaseAssert(mClassicTransactionQueue);
+    bool accPending = mClassicTransactionQueue->sourceAccountPending(accountID);
+    if (mSorobanTransactionQueue)
+    {
+        accPending = accPending ||
+                     mSorobanTransactionQueue->sourceAccountPending(accountID);
+    }
+    return accPending;
+}
+
+bool
+TransactionQueues::isBanned(Hash const& hash) const
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    auto banned = mClassicTransactionQueue->isBanned(hash);
+    if (mSorobanTransactionQueue)
+    {
+        banned = banned || mSorobanTransactionQueue->isBanned(hash);
+    }
+    return banned;
+}
+
+TransactionFrameBaseConstPtr
+TransactionQueues::getTx(Hash const& hash) const
+{
+    std::lock_guard<std::mutex> guard(mMutex);
+    auto classic = mClassicTransactionQueue->getTx(hash);
+    if (!classic && mSorobanTransactionQueue)
+    {
+        return mSorobanTransactionQueue->getTx(hash);
+    }
+    return classic;
+}
+
 }
diff --git a/src/herder/TransactionQueue.h b/src/herder/TransactionQueue.h
index 1d8047be39..4e279b9759 100644
--- a/src/herder/TransactionQueue.h
+++ b/src/herder/TransactionQueue.h
@@ -57,39 +57,52 @@ class Application;
  *   unbans any transactions that have been banned for more than banDepth
  *   ledgers.
  */
-class TransactionQueue
+
+// TODO:
+// * Dig into flow control, make sure it holds some kind of lock to prevent ASIO
+// overlay queue from growing too much
+//     * Might want to put bg tx queue on its own thread with intermediate
+//     priority (lower than SCP, higher than bucket maintenance). Otherwise,
+//     running this on the overlay thread might delay SCP message processing as
+//     incoming SCP messages need to wait for tx queue additions to occur, which
+//     is bad.
+//         * My note: Try both approaches and benchmark
+
+enum class TxQueueAddResultCode
 {
-  public:
-    static uint64_t const FEE_MULTIPLIER;
+    ADD_STATUS_PENDING = 0,
+    ADD_STATUS_DUPLICATE,
+    ADD_STATUS_ERROR,
+    ADD_STATUS_TRY_AGAIN_LATER,
+    ADD_STATUS_FILTERED,
+    ADD_STATUS_COUNT
+};
 
-    enum class AddResultCode
-    {
-        ADD_STATUS_PENDING = 0,
-        ADD_STATUS_DUPLICATE,
-        ADD_STATUS_ERROR,
-        ADD_STATUS_TRY_AGAIN_LATER,
-        ADD_STATUS_FILTERED,
-        ADD_STATUS_COUNT,
-        ADD_STATUS_UNKNOWN // TODO: rename?
-    };
+struct TxQueueAddResult
+{
+    TxQueueAddResultCode code;
+    MutableTxResultPtr txResult;
 
-    struct AddResult
-    {
-        TransactionQueue::AddResultCode code;
-        MutableTxResultPtr txResult;
+    // AddResult with no txResult
+    explicit TxQueueAddResult(TxQueueAddResultCode addCode);
 
-        // AddResult with no txResult
-        explicit AddResult(TransactionQueue::AddResultCode addCode);
+    // AddResult from existing transaction result
+    explicit TxQueueAddResult(TxQueueAddResultCode addCode,
+                              MutableTxResultPtr payload);
 
-        // AddResult from existing transaction result
-        explicit AddResult(TransactionQueue::AddResultCode addCode,
-                           MutableTxResultPtr payload);
+    // AddResult with error txResult with the specified txErrorCode
+    explicit TxQueueAddResult(TxQueueAddResultCode addCode,
+                              TransactionFrameBasePtr tx,
+                              TransactionResultCode txErrorCode);
+};
 
-        // AddResult with error txResult with the specified txErrorCode
-        explicit AddResult(TransactionQueue::AddResultCode addCode,
-                           TransactionFrameBasePtr tx,
-                           TransactionResultCode txErrorCode);
-    };
+class TransactionQueue
+{
+  public:
+    static uint64_t const FEE_MULTIPLIER;
+
+    using AddResultCode = TxQueueAddResultCode;
+    using AddResult = TxQueueAddResult;
 
     /**
      * AccountState stores the following information:
@@ -202,6 +215,7 @@ class TransactionQueue
 
     bool mShutdown{false};
     bool mWaiting{false};
+    bool mPendingMainThreadBroadcast{false}; // TODO: I don't love this solution
     // TODO: VirtualTimer is not thread-safe. Right now it's only used in
     // functions that are called from the main thread. However, if I move
     // broadcasting to the background I will need to be careful with this.
@@ -365,6 +379,39 @@ class ClassicTransactionQueue : public TransactionQueue
     }
 };
 
+// TODO: Rename?
+// TODO: Docs. A thread-safe container for transaction queues that allows for
+// delayed-initialization of the queues.
+// TODO: Doc comments on methods.
+class TransactionQueues : public NonMovableOrCopyable
+{
+  public:
+    TransactionQueues() = default;
+
+    void setClassicTransactionQueue(
+        std::unique_ptr<ClassicTransactionQueue> classicTransactionQueue);
+    void setSorobanTransactionQueue(
+        std::unique_ptr<SorobanTransactionQueue> sorobanTransactionQueue);
+
+    bool hasClassicTransactionQueue() const;
+    bool hasSorobanTransactionQueue() const;
+
+    ClassicTransactionQueue& getClassicTransactionQueue() const;
+    SorobanTransactionQueue& getSorobanTransactionQueue() const;
+
+    // Convenience functions that operate on both queues (if they exist)
+    void shutdown();
+    bool sourceAccountPending(AccountID const& accountID) const;
+    bool isBanned(Hash const& hash) const;
+    TransactionFrameBaseConstPtr getTx(Hash const& hash) const;
+
+  private:
+    mutable std::mutex mMutex;
+    std::unique_ptr<ClassicTransactionQueue> mClassicTransactionQueue = nullptr;
+    std::unique_ptr<SorobanTransactionQueue> mSorobanTransactionQueue = nullptr;
+};
+using TransactionQueuesPtr = std::shared_ptr<TransactionQueues>;
+
 extern std::array<const char*,
                   static_cast<int>(
                       TransactionQueue::AddResultCode::ADD_STATUS_COUNT)>
diff --git a/src/ledger/LedgerManager.h b/src/ledger/LedgerManager.h
index 03d3a32f41..a9caa76f40 100644
--- a/src/ledger/LedgerManager.h
+++ b/src/ledger/LedgerManager.h
@@ -179,9 +179,15 @@ class LedgerManager
 
     virtual void moveToSynced() = 0;
     virtual void beginApply() = 0;
+    // TODO: Mention this must be thread safe in implementation. Beware TOCTOU,
+    // but for non-critical uses (such as tx queue, which is "best effort"
+    // anyway), this is fine.
     virtual State getState() const = 0;
     virtual std::string getStateHuman() const = 0;
 
+    // TODO: Mention this is thread-safe because `getState` is thread-safe.
+    // Beware TOCTOU, but for non-critical uses (such as tx queue, which is
+    // "best effort" anyway), this is fine.
     bool
     isSynced() const
     {
diff --git a/src/ledger/LedgerManagerImpl.cpp b/src/ledger/LedgerManagerImpl.cpp
index d569fb234f..1e12adb851 100644
--- a/src/ledger/LedgerManagerImpl.cpp
+++ b/src/ledger/LedgerManagerImpl.cpp
@@ -134,21 +134,22 @@ LedgerManager::maxClassicLedgerResources(LedgerHeader const& header)
 Resource
 LedgerManager::maxSorobanLedgerResources(SorobanNetworkConfig const& conf)
 {
-    ZoneScoped std::vector<int64_t> limits = {
-        conf.ledgerMaxTxCount(),
-        conf.ledgerMaxInstructions(),
-        conf.ledgerMaxTransactionSizesBytes(),
-        conf.ledgerMaxReadBytes(),
-        conf.ledgerMaxWriteBytes(),
-        conf.ledgerMaxReadLedgerEntries(),
-        conf.ledgerMaxWriteLedgerEntries()};
+    ZoneScoped;
+    std::vector<int64_t> limits = {conf.ledgerMaxTxCount(),
+                                   conf.ledgerMaxInstructions(),
+                                   conf.ledgerMaxTransactionSizesBytes(),
+                                   conf.ledgerMaxReadBytes(),
+                                   conf.ledgerMaxWriteBytes(),
+                                   conf.ledgerMaxReadLedgerEntries(),
+                                   conf.ledgerMaxWriteLedgerEntries()};
     return Resource(limits);
 }
 
 Resource
 LedgerManager::maxSorobanTransactionResources(SorobanNetworkConfig const& conf)
 {
-    ZoneScoped int64_t const opCount = 1;
+    ZoneScoped;
+    int64_t const opCount = 1;
     std::vector<int64_t> limits = {opCount,
                                    conf.txMaxInstructions(),
                                    conf.txMaxSizeBytes(),
@@ -277,10 +278,10 @@ LedgerManagerImpl::setState(State s)
     if (s != getState())
     {
         std::string oldState = getStateHuman();
-        mState = s;
+        mState.store(s);
         mApp.syncOwnMetrics();
         CLOG_INFO(Ledger, "Changing state {} -> {}", oldState, getStateHuman());
-        if (mState != LM_CATCHING_UP_STATE)
+        if (mState.load() != LM_CATCHING_UP_STATE)
         {
             mApp.getLedgerApplyManager().logAndUpdateCatchupStatus(true);
         }
@@ -290,7 +291,7 @@ LedgerManagerImpl::setState(State s)
 LedgerManager::State
 LedgerManagerImpl::getState() const
 {
-    return mState;
+    return mState.load();
 }
 
 std::string
@@ -704,7 +705,7 @@ LedgerManagerImpl::valueExternalized(LedgerCloseData const& ledgerData,
     if (res == LedgerApplyManager::ProcessLedgerResult::
                    WAIT_TO_APPLY_BUFFERED_OR_CATCHUP)
     {
-        if (mState != LM_CATCHING_UP_STATE)
+        if (mState.load() != LM_CATCHING_UP_STATE)
         {
             // Out of sync, buffer what we just heard and start catchup.
             CLOG_INFO(Ledger,
diff --git a/src/ledger/LedgerManagerImpl.h b/src/ledger/LedgerManagerImpl.h
index 61abb7152c..8168adc657 100644
--- a/src/ledger/LedgerManagerImpl.h
+++ b/src/ledger/LedgerManagerImpl.h
@@ -143,7 +143,7 @@ class LedgerManagerImpl : public LedgerManager
                                     ApplicableTxSetFrame const& txSet,
                                     Config const& config);
 
-    State mState;
+    std::atomic<State> mState;
 
 #ifdef BUILD_TESTS
     std::vector<TransactionMetaFrame> mLastLedgerTxMeta;
diff --git a/src/main/AppConnector.cpp b/src/main/AppConnector.cpp
index 907a0b9670..d54b895e27 100644
--- a/src/main/AppConnector.cpp
+++ b/src/main/AppConnector.cpp
@@ -112,6 +112,13 @@ AppConnector::postOnOverlayThread(std::function<void()>&& f,
     mApp.postOnOverlayThread(std::move(f), message);
 }
 
+void
+AppConnector::postOnTxQueueThread(std::function<void()>&& f,
+                                  std::string const& message)
+{
+    mApp.postOnTxQueueThread(std::move(f), message);
+}
+
 Config const&
 AppConnector::getConfig() const
 {
@@ -161,6 +168,13 @@ AppConnector::getOverlayMetrics()
     return mApp.getOverlayManager().getOverlayMetrics();
 }
 
+bool
+AppConnector::ledgerIsSynced() const
+{
+    // Ledger manager's state enum is atomic
+    return mApp.getLedgerManager().isSynced();
+}
+
 bool
 AppConnector::checkScheduledAndCache(
     std::shared_ptr<CapacityTrackedMessage> msgTracker)
diff --git a/src/main/AppConnector.h b/src/main/AppConnector.h
index eed9c67de0..0f0e4a7c7d 100644
--- a/src/main/AppConnector.h
+++ b/src/main/AppConnector.h
@@ -40,7 +40,7 @@ class AppConnector
     void checkOnOperationApply(Operation const& operation,
                                OperationResult const& opres,
                                LedgerTxnDelta const& ltxDelta);
-    Hash const& getNetworkID() const;
+    Hash const& getNetworkID() const; // TODO: Is this *really* not thread safe?
 
     // Thread-safe methods
     void postOnMainThread(
@@ -48,12 +48,15 @@ class AppConnector
         Scheduler::ActionType type = Scheduler::ActionType::NORMAL_ACTION);
     void postOnOverlayThread(std::function<void()>&& f,
                              std::string const& message);
+    void postOnTxQueueThread(std::function<void()>&& f,
+                             std::string const& message);
     VirtualClock::time_point now() const;
     VirtualClock::system_time_point system_now() const;
     Config const& getConfig() const;
     std::shared_ptr<Config const> getConfigPtr() const;
     bool overlayShuttingDown() const;
     OverlayMetrics& getOverlayMetrics();
+    bool ledgerIsSynced() const;
     // This method is always exclusively called from one thread
     bool
     checkScheduledAndCache(std::shared_ptr<CapacityTrackedMessage> msgTracker);
@@ -69,6 +72,7 @@ class AppConnector
     // `getSorobanNetworkConfig` will throw an assertion error in that case.
     std::optional<SorobanNetworkConfig>
     maybeGetSorobanNetworkConfigReadOnly() const;
+    bool threadIsType(Application::ThreadType type) const;
 
     bool threadIsType(Application::ThreadType type) const;
 
diff --git a/src/main/Application.h b/src/main/Application.h
index f5b1690c35..56646fc3a3 100644
--- a/src/main/Application.h
+++ b/src/main/Application.h
@@ -164,14 +164,15 @@ class Application
         APP_NUM_STATE
     };
 
-    // Types of threads that may be running
+    // TODO: Docs
     enum class ThreadType
     {
         MAIN,
         WORKER,
         EVICTION,
         OVERLAY,
-        APPLY
+        LEDGER_CLOSE,
+        TX_QUEUE
     };
 
     virtual ~Application(){};
@@ -253,6 +254,8 @@ class Application
                                                 std::string jobName) = 0;
     virtual void postOnOverlayThread(std::function<void()>&& f,
                                      std::string jobName) = 0;
+    virtual void postOnTxQueueThread(std::function<void()>&& f,
+                                     std::string jobName) = 0;
     virtual void postOnLedgerCloseThread(std::function<void()>&& f,
                                          std::string jobName) = 0;
 
diff --git a/src/main/ApplicationImpl.cpp b/src/main/ApplicationImpl.cpp
index afaa4e0f37..b9b9d300aa 100644
--- a/src/main/ApplicationImpl.cpp
+++ b/src/main/ApplicationImpl.cpp
@@ -92,6 +92,12 @@ ApplicationImpl::ApplicationImpl(VirtualClock& clock, Config const& cfg)
     , mOverlayWork(mOverlayIOContext ? std::make_unique<asio::io_context::work>(
                                            *mOverlayIOContext)
                                      : nullptr)
+    , mTxQueueIOContext(mConfig.BACKGROUND_TX_QUEUE
+                            ? std::make_unique<asio::io_context>(1)
+                            : nullptr)
+    , mTxQueueWork(mTxQueueIOContext ? std::make_unique<asio::io_context::work>(
+                                           *mTxQueueIOContext)
+                                     : nullptr)
     , mLedgerCloseIOContext(mConfig.parallelLedgerClose()
                                 ? std::make_unique<asio::io_context>(1)
                                 : nullptr)
@@ -114,6 +120,8 @@ ApplicationImpl::ApplicationImpl(VirtualClock& clock, Config const& cfg)
           mMetrics->NewTimer({"app", "post-on-background-thread", "delay"}))
     , mPostOnOverlayThreadDelay(
           mMetrics->NewTimer({"app", "post-on-overlay-thread", "delay"}))
+    , mPostOnTxQueueThreadDelay(
+          mMetrics->NewTimer({"app", "post-on-tx-queue-thread", "delay"}))
     , mPostOnLedgerCloseThreadDelay(
           mMetrics->NewTimer({"app", "post-on-ledger-close-thread", "delay"}))
     , mStartedOn(clock.system_now())
@@ -188,6 +196,14 @@ ApplicationImpl::ApplicationImpl(VirtualClock& clock, Config const& cfg)
         mThreadTypes[mOverlayThread->get_id()] = ThreadType::OVERLAY;
     }
 
+    if (mConfig.BACKGROUND_TX_QUEUE)
+    {
+        // TODO: Keep priority unchanged as tx queue processes time-sensitive
+        // tasks? Or should tx queue priority be downgraded?
+        mTxQueueThread = std::thread{[this]() { mTxQueueIOContext->run(); }};
+        mThreadTypes[mTxQueueThread->get_id()] = ThreadType::TX_QUEUE;
+    }
+
     if (mConfig.parallelLedgerClose())
     {
         mLedgerCloseThread =
@@ -877,6 +893,10 @@ ApplicationImpl::joinAllThreads()
     {
         mOverlayWork.reset();
     }
+    if (mTxQueueWork)
+    {
+        mTxQueueWork.reset();
+    }
     if (mEvictionWork)
     {
         mEvictionWork.reset();
@@ -894,6 +914,12 @@ ApplicationImpl::joinAllThreads()
         mOverlayThread->join();
     }
 
+    if (mTxQueueThread)
+    {
+        LOG_INFO(DEFAULT_LOG, "Joining the tx queue thread");
+        mTxQueueThread->join();
+    }
+
     if (mEvictionThread)
     {
         LOG_INFO(DEFAULT_LOG, "Joining eviction thread");
@@ -1460,6 +1486,19 @@ ApplicationImpl::postOnOverlayThread(std::function<void()>&& f,
     });
 }
 
+void
+ApplicationImpl::postOnTxQueueThread(std::function<void()>&& f,
+                                     std::string jobName)
+{
+    releaseAssert(mTxQueueIOContext);
+    LogSlowExecution isSlow{std::move(jobName), LogSlowExecution::Mode::MANUAL,
+                            "executed after"};
+    asio::post(*mTxQueueIOContext, [this, f = std::move(f), isSlow]() {
+        mPostOnTxQueueThreadDelay.Update(isSlow.checkElapsedTime());
+        f();
+    });
+}
+
 void
 ApplicationImpl::postOnLedgerCloseThread(std::function<void()>&& f,
                                          std::string jobName)
diff --git a/src/main/ApplicationImpl.h b/src/main/ApplicationImpl.h
index fd0db8f684..1675e63d67 100644
--- a/src/main/ApplicationImpl.h
+++ b/src/main/ApplicationImpl.h
@@ -93,6 +93,8 @@ class ApplicationImpl : public Application
 
     virtual void postOnOverlayThread(std::function<void()>&& f,
                                      std::string jobName) override;
+    virtual void postOnTxQueueThread(std::function<void()>&& f,
+                                     std::string jobName) override;
     virtual void postOnLedgerCloseThread(std::function<void()>&& f,
                                          std::string jobName) override;
     virtual void start() override;
@@ -163,6 +165,9 @@ class ApplicationImpl : public Application
     std::unique_ptr<asio::io_context> mOverlayIOContext;
     std::unique_ptr<asio::io_context::work> mOverlayWork;
 
+    std::unique_ptr<asio::io_context> mTxQueueIOContext;
+    std::unique_ptr<asio::io_context::work> mTxQueueWork;
+
     std::unique_ptr<asio::io_context> mLedgerCloseIOContext;
     std::unique_ptr<asio::io_context::work> mLedgerCloseWork;
 
@@ -214,6 +219,7 @@ class ApplicationImpl : public Application
 
     std::vector<std::thread> mWorkerThreads;
     std::optional<std::thread> mOverlayThread;
+    std::optional<std::thread> mTxQueueThread;
     std::optional<std::thread> mLedgerCloseThread;
 
     // Unlike mWorkerThreads (which are low priority), eviction scans require a
@@ -240,6 +246,7 @@ class ApplicationImpl : public Application
     medida::Timer& mPostOnMainThreadDelay;
     medida::Timer& mPostOnBackgroundThreadDelay;
     medida::Timer& mPostOnOverlayThreadDelay;
+    medida::Timer& mPostOnTxQueueThreadDelay;
     medida::Timer& mPostOnLedgerCloseThreadDelay;
 
     VirtualClock::system_time_point mStartedOn;
diff --git a/src/overlay/Floodgate.cpp b/src/overlay/Floodgate.cpp
index 65b8e6b2cb..c424b9b4c4 100644
--- a/src/overlay/Floodgate.cpp
+++ b/src/overlay/Floodgate.cpp
@@ -56,8 +56,7 @@ Floodgate::clearBelow(uint32_t maxLedger)
 }
 
 bool
-Floodgate::addRecord(StellarMessage const& msg, Peer::pointer peer,
-                     Hash const& index)
+Floodgate::addRecord(Peer::pointer peer, Hash const& index)
 {
     ZoneScoped;
     if (mShuttingDown)
diff --git a/src/overlay/Floodgate.h b/src/overlay/Floodgate.h
index ac18bae504..a5b663d9b6 100644
--- a/src/overlay/Floodgate.h
+++ b/src/overlay/Floodgate.h
@@ -56,8 +56,7 @@ class Floodgate
     void clearBelow(uint32_t maxLedger);
     // returns true if this is a new record
     // fills msgID with msg's hash
-    bool addRecord(StellarMessage const& msg, Peer::pointer fromPeer,
-                   Hash const& msgID);
+    bool addRecord(Peer::pointer fromPeer, Hash const& msgID);
 
     // returns true if msg was sent to at least one peer
     // The hash required for transactions
diff --git a/src/overlay/OverlayManager.h b/src/overlay/OverlayManager.h
index 5e8e0edf46..6809b59dfb 100644
--- a/src/overlay/OverlayManager.h
+++ b/src/overlay/OverlayManager.h
@@ -51,6 +51,7 @@ class PeerBareAddress;
 class PeerManager;
 class SurveyManager;
 struct StellarMessage;
+struct TxQueueAddResult;
 
 class OverlayManager
 {
@@ -83,15 +84,22 @@ class OverlayManager
     // that, call broadcastMessage, above.
     // Returns true if this is a new message
     // fills msgID with msg's hash
-    virtual bool recvFloodedMsgID(StellarMessage const& msg, Peer::pointer peer,
-                                  Hash const& msgID) = 0;
+    virtual bool recvFloodedMsgID(Peer::pointer peer, Hash const& msgID) = 0;
 
     bool
     recvFloodedMsg(StellarMessage const& msg, Peer::pointer peer)
     {
-        return recvFloodedMsgID(msg, peer, xdrBlake2(msg));
+        return recvFloodedMsgID(peer, xdrBlake2(msg));
     }
 
+    // TODO: Docs
+    // TODO: This is going to be called in a lambda. Be careful about those
+    // pointers and references
+    virtual void
+    recordAddTransactionStats(TxQueueAddResult const& addResult,
+                              Hash const& txHash, Peer::pointer peer,
+                              Hash const& index) = 0;
+
     // Process incoming transaction, pass it down to the transaction queue
     virtual void recvTransaction(StellarMessage const& msg, Peer::pointer peer,
                                  Hash const& index) = 0;
diff --git a/src/overlay/OverlayManagerImpl.cpp b/src/overlay/OverlayManagerImpl.cpp
index 4bc525da67..956d223a96 100644
--- a/src/overlay/OverlayManagerImpl.cpp
+++ b/src/overlay/OverlayManagerImpl.cpp
@@ -1150,11 +1150,10 @@ OverlayManagerImpl::shufflePeerList(std::vector<Peer::pointer>& peerList)
 }
 
 bool
-OverlayManagerImpl::recvFloodedMsgID(StellarMessage const& msg,
-                                     Peer::pointer peer, Hash const& msgID)
+OverlayManagerImpl::recvFloodedMsgID(Peer::pointer peer, Hash const& msgID)
 {
     ZoneScoped;
-    return mFloodGate.addRecord(msg, peer, msgID);
+    return mFloodGate.addRecord(peer, msgID);
 }
 
 bool
@@ -1182,6 +1181,44 @@ OverlayManagerImpl::checkScheduledAndCache(
     return false;
 }
 
+void
+OverlayManagerImpl::recordAddTransactionStats(TxQueueAddResult const& addResult,
+                                              Hash const& txHash,
+                                              Peer::pointer peer,
+                                              Hash const& index)
+{
+    ZoneScoped;
+
+    auto const& om = getOverlayMetrics();
+    switch (addResult.code)
+    {
+    case TxQueueAddResultCode::ADD_STATUS_PENDING:
+        om.mPulledRelevantTxs.Mark();
+        // record that this peer sent us this transaction
+        // add it to the floodmap so that this peer gets credit for it
+        recvFloodedMsgID(peer, index);
+        CLOG_DEBUG(
+            Overlay,
+            "Peer::recvTransaction Received unique transaction {} from {}",
+            hexAbbrev(txHash), peer->toString());
+        break;
+    case TxQueueAddResultCode::ADD_STATUS_DUPLICATE:
+        om.mPulledIrrelevantTxs.Mark();
+        // record that this peer sent us this transaction
+        // add it to the floodmap so that this peer gets credit for it
+        recvFloodedMsgID(peer, index);
+        CLOG_DEBUG(
+            Overlay,
+            "Peer::recvTransaction Received duplicate transaction {} from {}",
+            hexAbbrev(txHash), peer->toString());
+        break;
+    default:
+        om.mPulledIrrelevantTxs.Mark();
+        forgetFloodedMsg(index);
+        break;
+    }
+}
+
 void
 OverlayManagerImpl::recvTransaction(StellarMessage const& msg,
                                     Peer::pointer peer, Hash const& index)
@@ -1191,45 +1228,23 @@ OverlayManagerImpl::recvTransaction(StellarMessage const& msg,
         mApp.getNetworkID(), msg.transaction());
     if (transaction)
     {
-        // record that this peer sent us this transaction
-        // add it to the floodmap so that this peer gets credit for it
-        recvFloodedMsgID(msg, peer, index);
-
-        mTxDemandsManager.recordTxPullLatency(transaction->getFullHash(), peer);
 
         // add it to our current set
         // and make sure it is valid
+        // TODO: I think I could pull this one call into Peer::recvTransaction
+        // and move basically all of the rest of this function to a new function
+        // called something like "recordTransactionStats" or something. Then,
+        // Peer:recvTransaction would invoke HerderImpl::recvTransaction in the
+        // background, and then pass the result to the new function on the main
+        // thread. That way I don't have to make OverlayManagerImpl and its
+        // dependencies (Floodgate, Peer, TxDemandsManager, maybe more), or much
+        // of Peer thread safe. Note that the recordTransactionStats function
+        // would probably need to take a shared ptr to the message so that the
+        // message doesn't get deleted before the function is called. The lambda
+        // capture will need to copy this pointer in.
         auto addResult = mApp.getHerder().recvTransaction(transaction, false);
-        bool pulledRelevantTx = false;
-        if (!(addResult.code ==
-                  TransactionQueue::AddResultCode::ADD_STATUS_PENDING ||
-              addResult.code ==
-                  TransactionQueue::AddResultCode::ADD_STATUS_DUPLICATE))
-        {
-            forgetFloodedMsg(index);
-            CLOG_DEBUG(Overlay,
-                       "Peer::recvTransaction Discarded transaction {} from {}",
-                       hexAbbrev(transaction->getFullHash()), peer->toString());
-        }
-        else
-        {
-            bool dup = addResult.code ==
-                       TransactionQueue::AddResultCode::ADD_STATUS_DUPLICATE;
-            if (!dup)
-            {
-                pulledRelevantTx = true;
-            }
-            CLOG_DEBUG(
-                Overlay,
-                "Peer::recvTransaction Received {} transaction {} from {}",
-                (dup ? "duplicate" : "unique"),
-                hexAbbrev(transaction->getFullHash()), peer->toString());
-        }
-
-        auto const& om = getOverlayMetrics();
-        auto& meter =
-            pulledRelevantTx ? om.mPulledRelevantTxs : om.mPulledIrrelevantTxs;
-        meter.Mark();
+        recordAddTransactionStats(addResult, transaction->getFullHash(), peer,
+                                  index);
     }
 }
 
diff --git a/src/overlay/OverlayManagerImpl.h b/src/overlay/OverlayManagerImpl.h
index 60d9111762..5f58a6034b 100644
--- a/src/overlay/OverlayManagerImpl.h
+++ b/src/overlay/OverlayManagerImpl.h
@@ -112,11 +112,13 @@ class OverlayManagerImpl : public OverlayManager
     ~OverlayManagerImpl();
 
     void clearLedgersBelow(uint32_t ledgerSeq, uint32_t lclSeq) override;
-    bool recvFloodedMsgID(StellarMessage const& msg, Peer::pointer peer,
-                          Hash const& msgID) override;
+    bool recvFloodedMsgID(Peer::pointer peer, Hash const& msgID) override;
     void recvTransaction(StellarMessage const& msg, Peer::pointer peer,
                          Hash const& index) override;
     void forgetFloodedMsg(Hash const& msgID) override;
+    void recordAddTransactionStats(TxQueueAddResult const& addResult,
+                                   Hash const& txHash, Peer::pointer peer,
+                                   Hash const& index) override;
     void recvTxDemand(FloodDemand const& dmd, Peer::pointer peer) override;
     bool
     broadcastMessage(std::shared_ptr<StellarMessage const> msg,
diff --git a/src/overlay/Peer.cpp b/src/overlay/Peer.cpp
index f9dccef03f..f969d2a6ef 100644
--- a/src/overlay/Peer.cpp
+++ b/src/overlay/Peer.cpp
@@ -71,8 +71,10 @@ Peer::Peer(Application& app, PeerRole role)
     , mRecurringTimer(app)
     , mDelayedExecutionTimer(app)
     , mTxAdverts(std::make_shared<TxAdverts>(app))
+    , mTransactionQueues(mAppConnector.getHerder().getTransactionQueues())
 {
     releaseAssert(threadIsMain());
+    releaseAssert(mTransactionQueues); // TODO: Remove?
     mPingSentTime = PING_NOT_SENT;
     mLastPing = std::chrono::hours(24); // some default very high value
     auto bytes = randomBytes(mSendNonce.size());
@@ -876,94 +878,119 @@ Peer::recvAuthenticatedMessage(AuthenticatedMessage&& msg)
 {
     ZoneScoped;
     releaseAssert(!threadIsMain() || !useBackgroundThread());
-    RECURSIVE_LOCK_GUARD(mStateMutex, guard);
 
-    if (shouldAbort(guard))
-    {
-        return false;
-    }
+    // TODO: Remove if I get rid of the special lock scoping vv
+    std::shared_ptr<CapacityTrackedMessage> msgTracker = nullptr;
 
-    std::string errorMsg;
-    if (getState(guard) >= GOT_HELLO && msg.v0().message.type() != ERROR_MSG)
+    // TODO: Move back if I git rid of lock scoping vv
+    Scheduler::ActionType type = Scheduler::ActionType::NORMAL_ACTION;
+    std::string queueName;
     {
-        if (!mHmac.checkAuthenticatedMessage(msg, errorMsg))
+        RECURSIVE_LOCK_GUARD(mStateMutex, guard);
+
+        if (shouldAbort(guard))
         {
-            if (!threadIsMain())
-            {
-                mAppConnector.postOnMainThread(
-                    [self = shared_from_this(), errorMsg]() {
-                        self->sendErrorAndDrop(ERR_AUTH, errorMsg);
-                    },
-                    "Peer::sendErrorAndDrop");
-            }
-            else
+            return false;
+        }
+
+        std::string errorMsg;
+        if (getState(guard) >= GOT_HELLO &&
+            msg.v0().message.type() != ERROR_MSG)
+        {
+            if (!mHmac.checkAuthenticatedMessage(msg, errorMsg))
             {
-                sendErrorAndDrop(ERR_AUTH, errorMsg);
+                if (!threadIsMain())
+                {
+                    mAppConnector.postOnMainThread(
+                        [self = shared_from_this(), errorMsg]() {
+                            self->sendErrorAndDrop(ERR_AUTH, errorMsg);
+                        },
+                        "Peer::sendErrorAndDrop");
+                }
+                else
+                {
+                    sendErrorAndDrop(ERR_AUTH, errorMsg);
+                }
+                return false;
             }
-            return false;
         }
-    }
 
-    // NOTE: Additionally, we may use state snapshots to verify TRANSACTION type
-    // messages in the background.
+        // NOTE: Additionally, we may use state snapshots to verify TRANSACTION
+        // type messages in the background.
 
-    // Start tracking capacity here, so read throttling is applied
-    // appropriately. Flow control might not be started at that time
-    auto msgTracker = std::make_shared<CapacityTrackedMessage>(
-        shared_from_this(), msg.v0().message);
+        // Start tracking capacity here, so read throttling is applied
+        // appropriately. Flow control might not be started at that time
+        msgTracker = std::make_shared<CapacityTrackedMessage>(
+            shared_from_this(), msg.v0().message);
 
-    std::string cat;
-    Scheduler::ActionType type = Scheduler::ActionType::NORMAL_ACTION;
+        std::string cat;
 
-    switch (msgTracker->getMessage().type())
-    {
-    case HELLO:
-    case AUTH:
-        cat = AUTH_ACTION_QUEUE;
-        break;
-    // control messages
-    case PEERS:
-    case ERROR_MSG:
-    case SEND_MORE:
-    case SEND_MORE_EXTENDED:
-        cat = "CTRL";
-        break;
-    // high volume flooding
-    case TRANSACTION:
-    case FLOOD_ADVERT:
-    case FLOOD_DEMAND:
-    {
-        cat = "TX";
-        type = Scheduler::ActionType::DROPPABLE_ACTION;
-        break;
-    }
+        switch (msgTracker->getMessage().type())
+        {
+        case HELLO:
+        case AUTH:
+            cat = AUTH_ACTION_QUEUE;
+            break;
+        // control messages
+        case PEERS:
+        case ERROR_MSG:
+        case SEND_MORE:
+        case SEND_MORE_EXTENDED:
+            cat = "CTRL";
+            break;
+        // high volume flooding
+        case TRANSACTION:
+        case FLOOD_ADVERT:
+        case FLOOD_DEMAND:
+        {
+            cat = "TX";
+            type = Scheduler::ActionType::DROPPABLE_ACTION;
+            break;
+        }
 
-    // consensus, inbound
-    case GET_TX_SET:
-    case GET_SCP_QUORUMSET:
-    case GET_SCP_STATE:
-        cat = "SCPQ";
-        type = Scheduler::ActionType::DROPPABLE_ACTION;
-        break;
+        // consensus, inbound
+        case GET_TX_SET:
+        case GET_SCP_QUORUMSET:
+        case GET_SCP_STATE:
+            cat = "SCPQ";
+            type = Scheduler::ActionType::DROPPABLE_ACTION;
+            break;
 
-    // consensus, self
-    case DONT_HAVE:
-    case TX_SET:
-    case GENERALIZED_TX_SET:
-    case SCP_QUORUMSET:
-    case SCP_MESSAGE:
-        cat = "SCP";
-        break;
+        // consensus, self
+        case DONT_HAVE:
+        case TX_SET:
+        case GENERALIZED_TX_SET:
+        case SCP_QUORUMSET:
+        case SCP_MESSAGE:
+            cat = "SCP";
+            break;
 
-    default:
-        cat = "MISC";
+        default:
+            cat = "MISC";
+        }
+
+        // processing of incoming messages during authenticated must be
+        // in-order, so while not authenticated, place all messages onto
+        // AUTH_ACTION_QUEUE scheduler queue
+        queueName = isAuthenticated(guard) ? cat : AUTH_ACTION_QUEUE;
+        type = isAuthenticated(guard) ? type
+                                      : Scheduler::ActionType::NORMAL_ACTION;
+
+        // TODO: This scope (ending here) exists to ensure this doesn't hold the
+        // state lock upon entry to the transaction queue. This can cause
+        // deadlocks! I think it's safe to release the lock here as there's no
+        // longer any state querying. In practice though, if I end up posting
+        // the tryAdd action onto some tx-queue specific thread, then I can
+        // remove the scoping I added here and the lock will be released upon
+        // return from this function (like it always has).
+
+        // TODO: Really investigate whether this peer+transaction queue locking
+        // each other issue can come up anywhere else.
     }
 
-    // processing of incoming messages during authenticated must be in-order, so
-    // while not authenticated, place all messages onto AUTH_ACTION_QUEUE
-    // scheduler queue
-    auto queueName = isAuthenticated(guard) ? cat : AUTH_ACTION_QUEUE;
-    type = isAuthenticated(guard) ? type : Scheduler::ActionType::NORMAL_ACTION;
+    // TODO: vv Remove asserts if I get rid of the scoping above
+    releaseAssert(msgTracker);
+    releaseAssert(!queueName.empty());
 
     // If a message is already scheduled, drop
     if (mAppConnector.checkScheduledAndCache(msgTracker))
@@ -980,16 +1007,44 @@ Peer::recvAuthenticatedMessage(AuthenticatedMessage&& msg)
                                                   envelope.statement));
     }
 
-    // Subtle: move `msgTracker` shared_ptr into the lambda, to ensure
-    // its destructor is invoked from main thread only. Note that we can't use
-    // unique_ptr here, because std::function requires its callable
-    // to be copyable (C++23 fixes this with std::move_only_function, but we're
-    // not there yet)
-    mAppConnector.postOnMainThread(
-        [self = shared_from_this(), t = std::move(msgTracker)]() {
-            self->recvMessage(t);
-        },
-        std::move(queueName), type);
+    // TODO: Special case for TRANSACTION here to add to tx queue in the
+    // background.
+    if (msgTracker->getMessage().type() == TRANSACTION &&
+        mAppConnector.getConfig().BACKGROUND_TX_QUEUE)
+    {
+        // TODO: This assert might not be relevant. It's theoretically possible
+        // to enable background tx queue without enabling background overlay
+        // (under this implementation). Whether or not that's a good idea is a
+        // separate question.
+        releaseAssert(!threadIsMain());
+        // Subtle: move `msgTracker` shared_ptr into the lambda, to ensure
+        // its destructor is invoked from main thread only. Note that we can't
+        // use unique_ptr here, because std::function requires its callable to
+        // be copyable (C++23 fixes this with std::move_only_function, but we're
+        // not there yet)
+        mAppConnector.postOnTxQueueThread(
+            [self = shared_from_this(), t = std::move(msgTracker)]() {
+                self->recvMessage(t);
+            },
+            "Peer::recvMessage"); // TODO: Change message to something better
+        // TODO: If I end up running this on a different thread then I need to
+        // be sure to std::move `msgTracker` into the lambda as-per the note
+        // below.
+    }
+    else
+    {
+
+        // Subtle: move `msgTracker` shared_ptr into the lambda, to ensure
+        // its destructor is invoked from main thread only. Note that we can't
+        // use unique_ptr here, because std::function requires its callable to
+        // be copyable (C++23 fixes this with std::move_only_function, but we're
+        // not there yet)
+        mAppConnector.postOnMainThread(
+            [self = shared_from_this(), t = std::move(msgTracker)]() {
+                self->recvMessage(t);
+            },
+            std::move(queueName), type);
+    }
 
     // msgTracker should be null now
     releaseAssert(!msgTracker);
@@ -1000,7 +1055,7 @@ void
 Peer::recvMessage(std::shared_ptr<CapacityTrackedMessage> msgTracker)
 {
     ZoneScoped;
-    releaseAssert(threadIsMain());
+    // TODO: Note in the docs that this function may be called in the background
 
     auto const& stellarMsg = msgTracker->getMessage();
 
@@ -1019,7 +1074,7 @@ Peer::recvMessage(std::shared_ptr<CapacityTrackedMessage> msgTracker)
     bool ignoreIfOutOfSync = msgType == TRANSACTION ||
                              msgType == FLOOD_ADVERT || msgType == FLOOD_DEMAND;
 
-    if (!mAppConnector.getLedgerManager().isSynced() && ignoreIfOutOfSync)
+    if (!mAppConnector.ledgerIsSynced() && ignoreIfOutOfSync)
     {
         // For transactions, exit early during the state rebuild, as we
         // can't properly verify them
@@ -1028,7 +1083,36 @@ Peer::recvMessage(std::shared_ptr<CapacityTrackedMessage> msgTracker)
 
     try
     {
-        recvRawMessage(msgTracker);
+        if (msgType == TRANSACTION &&
+            mAppConnector.getConfig().BACKGROUND_TX_QUEUE)
+        {
+            // TODO: Refactor this to more closely mirror recvTransaction flow?
+            // Also need to get all the metrics/etc from
+            // OverlayManagerImpl::recordAddTransactionStats in here.
+            releaseAssert(!threadIsMain());
+            releaseAssert(msgTracker->maybeGetHash().has_value());
+            Hash const index = msgTracker->maybeGetHash().value();
+            // TODO: If `getNetworkID` is truly not thread safe then we might
+            // need to keep a copy of it somewhere safe.
+            auto transaction = TransactionFrameBase::makeTransactionFromWire(
+                mAppConnector.getNetworkID(),
+                msgTracker->getMessage().transaction());
+            releaseAssert(transaction);
+            TransactionQueue::AddResult addResult =
+                Herder::recvTransaction(mTransactionQueues, transaction, false);
+            mAppConnector.postOnMainThread(
+                [self = shared_from_this(), index, addResult, transaction]() {
+                    self->mAppConnector.getOverlayManager()
+                        .recordAddTransactionStats(
+                            addResult, transaction->getFullHash(), self, index);
+                },
+                "Peer::recvMessage recordAddTransactionStats");
+        }
+        else
+        {
+            releaseAssert(threadIsMain());
+            recvRawMessage(msgTracker);
+        }
     }
     catch (CryptoError const& e)
     {
@@ -1070,6 +1154,9 @@ Peer::recvRawMessage(std::shared_ptr<CapacityTrackedMessage> msgTracker)
     // No need to hold the lock for the whole duration of the function, just
     // need to check state for a potential early exit. If the peer gets dropped
     // after, we'd still process the message, but that's harmless.
+    // TODO: I think it's OK that the flow I developed skips this? The message
+    // type is skips most of these checks, except for the authenticated check.
+    // Maybe I should add an isAuthenticated check? Or a shouldAbort check?
     {
         RECURSIVE_LOCK_GUARD(mStateMutex, guard);
         if (shouldAbort(guard))
@@ -1189,6 +1276,7 @@ Peer::recvRawMessage(std::shared_ptr<CapacityTrackedMessage> msgTracker)
 
     case TRANSACTION:
     {
+        // TODO: Losing this metric right now vv
         auto t = mOverlayMetrics.mRecvTransactionTimer.TimeScope();
         recvTransaction(*msgTracker);
     }
@@ -1487,7 +1575,7 @@ Peer::recvSCPMessage(CapacityTrackedMessage const& msg)
     // add it to the floodmap so that this peer gets credit for it
     releaseAssert(msg.maybeGetHash());
     mAppConnector.getOverlayManager().recvFloodedMsgID(
-        msg.getMessage(), shared_from_this(), msg.maybeGetHash().value());
+        shared_from_this(), msg.maybeGetHash().value());
 
     auto res = mAppConnector.getHerder().recvSCPEnvelope(envelope);
     if (res == Herder::ENVELOPE_STATUS_DISCARDED)
diff --git a/src/overlay/Peer.h b/src/overlay/Peer.h
index 53426d53ac..fb7b6aa644 100644
--- a/src/overlay/Peer.h
+++ b/src/overlay/Peer.h
@@ -39,6 +39,7 @@ struct OverlayMetrics;
 class FlowControl;
 class TxAdverts;
 class CapacityTrackedMessage;
+class TransactionQueues;
 
 // Peer class represents a connected peer (either inbound or outbound)
 //
@@ -263,6 +264,8 @@ class Peer : public std::enable_shared_from_this<Peer>,
     QueryInfo mTxSetQueryInfo;
     bool mPeersReceived{false};
 
+    std::shared_ptr<TransactionQueues> const mTransactionQueues;
+
     static Hash pingIDfromTimePoint(VirtualClock::time_point const& tp);
     void pingPeer();
     void maybeProcessPingResponse(Hash const& id);
diff --git a/src/overlay/TCPPeer.cpp b/src/overlay/TCPPeer.cpp
index f3129f94ab..85d2e30b26 100644
--- a/src/overlay/TCPPeer.cpp
+++ b/src/overlay/TCPPeer.cpp
@@ -528,10 +528,14 @@ TCPPeer::startRead()
     ZoneScoped;
     releaseAssert(!threadIsMain() || !useBackgroundThread());
     releaseAssert(canRead());
-    RECURSIVE_LOCK_GUARD(mStateMutex, guard);
-    if (shouldAbort(guard))
     {
-        return;
+        RECURSIVE_LOCK_GUARD(mStateMutex, guard);
+        if (shouldAbort(guard))
+        {
+            return;
+        }
+        // TODO: Remove this outer scoping if I add separate thread for bg tx
+        // queue
     }
 
     mThreadVars.getIncomingHeader().clear();
diff --git a/src/overlay/test/ItemFetcherTests.cpp b/src/overlay/test/ItemFetcherTests.cpp
index e2d138a488..d304c58e0f 100644
--- a/src/overlay/test/ItemFetcherTests.cpp
+++ b/src/overlay/test/ItemFetcherTests.cpp
@@ -465,8 +465,7 @@ TEST_CASE("next peer strategy", "[overlay][ItemFetcher]")
         {
             StellarMessage msg(SCP_MESSAGE);
             msg.envelope() = hundredEnvelope1;
-            app->getOverlayManager().recvFloodedMsgID(msg, peer1,
-                                                      xdrBlake2(msg));
+            app->getOverlayManager().recvFloodedMsgID(peer1, xdrBlake2(msg));
             tracker->tryNextPeer();
             REQUIRE(askCount == 2);
             auto trPeer1b = tracker->getLastAskedPeer();

From 357bfee19b076511c15b43932fb16d49acb38111 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Fri, 31 Jan 2025 13:42:53 -0800
Subject: [PATCH 11/14] Prioritize main thread in locking

---
 src/herder/TransactionQueue.cpp | 53 +++++++++++++++++++++------------
 src/herder/TransactionQueue.h   | 36 ++++++++++++++++++----
 2 files changed, 65 insertions(+), 24 deletions(-)

diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index d158a3c2bd..3f4cdc9bb1 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -270,7 +270,7 @@ isDuplicateTx(TransactionFrameBasePtr oldTx, TransactionFrameBasePtr newTx)
 bool
 TransactionQueue::sourceAccountPending(AccountID const& accountID) const
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     return mAccountStates.find(accountID) != mAccountStates.end();
 }
 
@@ -328,6 +328,22 @@ validateSorobanMemo(TransactionFrameBasePtr tx)
     return true;
 }
 
+TransactionQueue::TxQueueLock
+TransactionQueue::lock() const
+{
+    if (threadIsMain())
+    {
+        mMainThreadWaiting.store(true);
+        std::unique_lock<std::mutex> lock(mTxQueueMutex);
+        mMainThreadWaiting.store(false);
+        return TxQueueLock(std::move(lock), mTxQueueCv);
+    }
+    std::unique_lock<std::mutex> lock(mTxQueueMutex);
+    mTxQueueCv->wait(lock, [this] { return !mMainThreadWaiting.load(); });
+    return TxQueueLock(std::move(lock), mTxQueueCv);
+}
+
+
 TransactionQueue::AddResult
 TransactionQueue::canAdd(
     TransactionFrameBasePtr tx, AccountStates::iterator& stateIter,
@@ -641,7 +657,7 @@ TransactionQueue::AddResult
 TransactionQueue::tryAdd(TransactionFrameBasePtr tx, bool submittedFromSelf)
 {
     ZoneScoped;
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     CLOG_DEBUG(Tx, "Try add tx {} in {}", hexAbbrev(tx->getFullHash()),
                threadIsMain() ? "foreground" : "background");
 
@@ -818,7 +834,7 @@ void
 TransactionQueue::ban(Transactions const& banTxs)
 {
     ZoneScoped;
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     banInternal(banTxs);
 }
 
@@ -871,7 +887,7 @@ TransactionQueue::AccountState
 TransactionQueue::getAccountTransactionQueueInfo(
     AccountID const& accountID) const
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     auto i = mAccountStates.find(accountID);
     if (i == std::end(mAccountStates))
     {
@@ -883,7 +899,7 @@ TransactionQueue::getAccountTransactionQueueInfo(
 size_t
 TransactionQueue::countBanned(int index) const
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     return mBannedTransactions[index].size();
 }
 #endif
@@ -958,7 +974,7 @@ TransactionQueue::shift()
 bool
 TransactionQueue::isBanned(Hash const& hash) const
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     return isBannedInternal(hash);
 }
 
@@ -976,7 +992,7 @@ TxFrameList
 TransactionQueue::getTransactions(LedgerHeader const& lcl) const
 {
     ZoneScoped;
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     return getTransactionsInternal(lcl);
 }
 
@@ -1004,7 +1020,7 @@ TransactionFrameBaseConstPtr
 TransactionQueue::getTx(Hash const& hash) const
 {
     ZoneScoped;
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     auto it = mKnownTxHashes.find(hash);
     if (it != mKnownTxHashes.end())
     {
@@ -1223,7 +1239,7 @@ size_t
 SorobanTransactionQueue::getMaxQueueSizeOps() const
 {
     ZoneScoped;
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     if (protocolVersionStartsFrom(
             mBucketSnapshot->getLedgerHeader().ledgerVersion,
             SOROBAN_PROTOCOL_VERSION))
@@ -1317,8 +1333,7 @@ ClassicTransactionQueue::broadcastSome()
 }
 
 void
-TransactionQueue::broadcast(bool fromCallback,
-                            std::lock_guard<std::mutex> const& guard)
+TransactionQueue::broadcast(bool fromCallback, TxQueueLock const& guard)
 {
     // Must be called from the main thread due to the use of `mBroadcastTimer`
     releaseAssert(threadIsMain());
@@ -1357,12 +1372,12 @@ TransactionQueue::broadcast(bool fromCallback,
 void
 TransactionQueue::broadcast(bool fromCallback)
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     broadcast(fromCallback, guard);
 }
 
 void
-TransactionQueue::rebroadcast(std::lock_guard<std::mutex> const& guard)
+TransactionQueue::rebroadcast(TxQueueLock const& guard)
 {
     // For `broadcast` call
     releaseAssert(threadIsMain());
@@ -1383,7 +1398,7 @@ void
 TransactionQueue::shutdown()
 {
     releaseAssert(threadIsMain());
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     mShutdown = true;
     mBroadcastTimer.cancel();
 }
@@ -1396,7 +1411,7 @@ TransactionQueue::update(
 {
     ZoneScoped;
     releaseAssert(threadIsMain());
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
 
     mValidationSnapshot =
         std::make_shared<ImmutableValidationSnapshot>(mAppConn);
@@ -1455,7 +1470,7 @@ void
 TransactionQueue::updateSnapshots(
     SearchableSnapshotConstPtr const& newBucketSnapshot)
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     mValidationSnapshot =
         std::make_shared<ImmutableValidationSnapshot>(mAppConn);
     mBucketSnapshot = newBucketSnapshot;
@@ -1465,14 +1480,14 @@ TransactionQueue::updateSnapshots(
 size_t
 TransactionQueue::getQueueSizeOps() const
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     return mTxQueueLimiter.size();
 }
 
 std::optional<int64_t>
 TransactionQueue::getInQueueSeqNum(AccountID const& account) const
 {
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     auto stateIter = mAccountStates.find(account);
     if (stateIter == mAccountStates.end())
     {
@@ -1490,7 +1505,7 @@ size_t
 ClassicTransactionQueue::getMaxQueueSizeOps() const
 {
     ZoneScoped;
-    std::lock_guard<std::mutex> guard(mTxQueueMutex);
+    TxQueueLock guard = lock();
     auto res = mTxQueueLimiter.maxScaledLedgerResources(false);
     releaseAssert(res.size() == NUM_CLASSIC_TX_RESOURCES);
     return res.getVal(Resource::Type::OPERATIONS);
diff --git a/src/herder/TransactionQueue.h b/src/herder/TransactionQueue.h
index 4e279b9759..1b7e715905 100644
--- a/src/herder/TransactionQueue.h
+++ b/src/herder/TransactionQueue.h
@@ -170,6 +170,27 @@ class TransactionQueue
 #endif
 
   protected:
+    // TODO: Docs?
+    // TODO: Move?
+    class TxQueueLock : NonMovableOrCopyable
+    {
+      public:
+        TxQueueLock(std::unique_lock<std::mutex>&& lock,
+                    std::shared_ptr<std::condition_variable> cv)
+            : mLock(std::move(lock)), mCv(cv)
+        {
+        }
+        ~TxQueueLock()
+        {
+            // Wake threads on destruction
+            mCv->notify_all();
+        }
+
+      private:
+        std::unique_lock<std::mutex> mLock;
+        std::shared_ptr<std::condition_variable> mCv;
+    };
+
     /**
      * The AccountState for every account. As noted above, an AccountID is in
      * AccountStates iff at least one of the following is true for the
@@ -232,7 +253,7 @@ class TransactionQueue
     // internal call should call the second overload (which enforces that the
     // lock is already held).
     void broadcast(bool fromCallback);
-    void broadcast(bool fromCallback, std::lock_guard<std::mutex> const& guard);
+    void broadcast(bool fromCallback, TxQueueLock const& guard);
     // broadcasts a single transaction
     enum class BroadcastStatus
     {
@@ -259,6 +280,8 @@ class TransactionQueue
     // held.
     void banInternal(Transactions const& banTxs);
 
+    TxQueueLock lock() const;
+
     // Snapshots to use for transaction validation
     ImmutableValidationSnapshotPtr mValidationSnapshot;
     SearchableSnapshotConstPtr mBucketSnapshot;
@@ -270,11 +293,14 @@ class TransactionQueue
 
     size_t mBroadcastSeed;
 
-    mutable std::mutex mTxQueueMutex;
-
   private:
     AppConnector& mAppConn;
 
+    mutable std::mutex mTxQueueMutex;
+    mutable std::shared_ptr<std::condition_variable> mTxQueueCv =
+        std::make_shared<std::condition_variable>();
+    mutable std::atomic<bool> mMainThreadWaiting{false};
+
     void removeApplied(Transactions const& txs);
 
     /**
@@ -286,7 +312,7 @@ class TransactionQueue
 
     // TODO: Explain that this takes a lock guard due to the `broadcast` call
     // that it makes.
-    void rebroadcast(std::lock_guard<std::mutex> const& guard);
+    void rebroadcast(TxQueueLock const& guard);
 
     // TODO: Docs
     // Private versions of public functions that contain the actual
@@ -327,7 +353,7 @@ class SorobanTransactionQueue : public TransactionQueue
     void
     clearBroadcastCarryover()
     {
-        std::lock_guard<std::mutex> guard(mTxQueueMutex);
+        TxQueueLock lock = TransactionQueue::lock();
         mBroadcastOpCarryover.clear();
         mBroadcastOpCarryover.resize(1, Resource::makeEmptySoroban());
     }

From 39de3e64c6948a330aa4db61d7837cd283e81beb Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Tue, 4 Feb 2025 09:45:06 -0800
Subject: [PATCH 12/14] Some cleanup around separate tx queue thread

---
 src/herder/HerderImpl.cpp          |   2 +-
 src/herder/HerderImpl.h            |   3 -
 src/herder/TransactionQueue.h      |   5 +
 src/main/ApplicationImpl.cpp       |   5 +-
 src/overlay/OverlayManagerImpl.cpp |  11 --
 src/overlay/Peer.cpp               | 172 ++++++++++++-----------------
 src/overlay/TCPPeer.cpp            |  10 +-
 7 files changed, 85 insertions(+), 123 deletions(-)

diff --git a/src/herder/HerderImpl.cpp b/src/herder/HerderImpl.cpp
index b27ea16593..cfdff15890 100644
--- a/src/herder/HerderImpl.cpp
+++ b/src/herder/HerderImpl.cpp
@@ -600,7 +600,7 @@ Herder::recvTransaction(TransactionQueuesPtr txQueues,
     // would be whatever is handling ledger close. However, that will only
     // decrease the sourceAccountPending value, which means this erroneously
     // rejects (which is safe). I guess it's possible for a user-submitted
-    // transaction to come in and conflict with the overlay thread, but that
+    // transaction to come in and conflict with the tx queue thread, but that
     // would require them to be simultaneously running two clients and
     // submitting from both of them. Still, it might be safest to use some kind
     // of atomic function that handles both this check AND the add.
diff --git a/src/herder/HerderImpl.h b/src/herder/HerderImpl.h
index 6ebfb994a3..1d0a829fe0 100644
--- a/src/herder/HerderImpl.h
+++ b/src/herder/HerderImpl.h
@@ -250,9 +250,6 @@ class HerderImpl : public Herder
     void purgeOldPersistedTxSets();
     void writeDebugTxSet(LedgerCloseData const& lcd);
 
-    // TODO: Need some way to get these queues
-    // TODO: Maybe something else should create this and pass it in somehow,
-    // either via Application or explicitly in the constructor for HerderImpl.
     TransactionQueuesPtr const mTransactionQueues =
         std::make_shared<TransactionQueues>();
 
diff --git a/src/herder/TransactionQueue.h b/src/herder/TransactionQueue.h
index 1b7e715905..8ce50cd18e 100644
--- a/src/herder/TransactionQueue.h
+++ b/src/herder/TransactionQueue.h
@@ -67,6 +67,8 @@ class Application;
 //     incoming SCP messages need to wait for tx queue additions to occur, which
 //     is bad.
 //         * My note: Try both approaches and benchmark
+//         * Follow up note: The priority locking scheme also helps to address
+//           this.
 
 enum class TxQueueAddResultCode
 {
@@ -172,6 +174,9 @@ class TransactionQueue
   protected:
     // TODO: Docs?
     // TODO: Move?
+    // TODO: It might be worth benchmarking this against the solution that does
+    // not use priority locking (just uses std::mutex). The added complexity of
+    // this may not be worth it.
     class TxQueueLock : NonMovableOrCopyable
     {
       public:
diff --git a/src/main/ApplicationImpl.cpp b/src/main/ApplicationImpl.cpp
index b9b9d300aa..4b5a687a3e 100644
--- a/src/main/ApplicationImpl.cpp
+++ b/src/main/ApplicationImpl.cpp
@@ -199,7 +199,10 @@ ApplicationImpl::ApplicationImpl(VirtualClock& clock, Config const& cfg)
     if (mConfig.BACKGROUND_TX_QUEUE)
     {
         // TODO: Keep priority unchanged as tx queue processes time-sensitive
-        // tasks? Or should tx queue priority be downgraded?
+        // tasks? Or should tx queue priority be downgraded? The priority
+        // locking mechanism in TransactionQueue is designed to prevent tx queue
+        // from starving other work, so it may be fine to keep priority
+        // unchanged.
         mTxQueueThread = std::thread{[this]() { mTxQueueIOContext->run(); }};
         mThreadTypes[mTxQueueThread->get_id()] = ThreadType::TX_QUEUE;
     }
diff --git a/src/overlay/OverlayManagerImpl.cpp b/src/overlay/OverlayManagerImpl.cpp
index 956d223a96..f657bc6921 100644
--- a/src/overlay/OverlayManagerImpl.cpp
+++ b/src/overlay/OverlayManagerImpl.cpp
@@ -1231,17 +1231,6 @@ OverlayManagerImpl::recvTransaction(StellarMessage const& msg,
 
         // add it to our current set
         // and make sure it is valid
-        // TODO: I think I could pull this one call into Peer::recvTransaction
-        // and move basically all of the rest of this function to a new function
-        // called something like "recordTransactionStats" or something. Then,
-        // Peer:recvTransaction would invoke HerderImpl::recvTransaction in the
-        // background, and then pass the result to the new function on the main
-        // thread. That way I don't have to make OverlayManagerImpl and its
-        // dependencies (Floodgate, Peer, TxDemandsManager, maybe more), or much
-        // of Peer thread safe. Note that the recordTransactionStats function
-        // would probably need to take a shared ptr to the message so that the
-        // message doesn't get deleted before the function is called. The lambda
-        // capture will need to copy this pointer in.
         auto addResult = mApp.getHerder().recvTransaction(transaction, false);
         recordAddTransactionStats(addResult, transaction->getFullHash(), peer,
                                   index);
diff --git a/src/overlay/Peer.cpp b/src/overlay/Peer.cpp
index f969d2a6ef..3c0e662b79 100644
--- a/src/overlay/Peer.cpp
+++ b/src/overlay/Peer.cpp
@@ -878,119 +878,94 @@ Peer::recvAuthenticatedMessage(AuthenticatedMessage&& msg)
 {
     ZoneScoped;
     releaseAssert(!threadIsMain() || !useBackgroundThread());
+    RECURSIVE_LOCK_GUARD(mStateMutex, guard);
 
-    // TODO: Remove if I get rid of the special lock scoping vv
-    std::shared_ptr<CapacityTrackedMessage> msgTracker = nullptr;
-
-    // TODO: Move back if I git rid of lock scoping vv
-    Scheduler::ActionType type = Scheduler::ActionType::NORMAL_ACTION;
-    std::string queueName;
+    if (shouldAbort(guard))
     {
-        RECURSIVE_LOCK_GUARD(mStateMutex, guard);
-
-        if (shouldAbort(guard))
-        {
-            return false;
-        }
+        return false;
+    }
 
-        std::string errorMsg;
-        if (getState(guard) >= GOT_HELLO &&
-            msg.v0().message.type() != ERROR_MSG)
+    std::string errorMsg;
+    if (getState(guard) >= GOT_HELLO && msg.v0().message.type() != ERROR_MSG)
+    {
+        if (!mHmac.checkAuthenticatedMessage(msg, errorMsg))
         {
-            if (!mHmac.checkAuthenticatedMessage(msg, errorMsg))
+            if (!threadIsMain())
             {
-                if (!threadIsMain())
-                {
-                    mAppConnector.postOnMainThread(
-                        [self = shared_from_this(), errorMsg]() {
-                            self->sendErrorAndDrop(ERR_AUTH, errorMsg);
-                        },
-                        "Peer::sendErrorAndDrop");
-                }
-                else
-                {
-                    sendErrorAndDrop(ERR_AUTH, errorMsg);
-                }
-                return false;
+                mAppConnector.postOnMainThread(
+                    [self = shared_from_this(), errorMsg]() {
+                        self->sendErrorAndDrop(ERR_AUTH, errorMsg);
+                    },
+                    "Peer::sendErrorAndDrop");
+            }
+            else
+            {
+                sendErrorAndDrop(ERR_AUTH, errorMsg);
             }
+            return false;
         }
+    }
 
-        // NOTE: Additionally, we may use state snapshots to verify TRANSACTION
-        // type messages in the background.
-
-        // Start tracking capacity here, so read throttling is applied
-        // appropriately. Flow control might not be started at that time
-        msgTracker = std::make_shared<CapacityTrackedMessage>(
-            shared_from_this(), msg.v0().message);
-
-        std::string cat;
-
-        switch (msgTracker->getMessage().type())
-        {
-        case HELLO:
-        case AUTH:
-            cat = AUTH_ACTION_QUEUE;
-            break;
-        // control messages
-        case PEERS:
-        case ERROR_MSG:
-        case SEND_MORE:
-        case SEND_MORE_EXTENDED:
-            cat = "CTRL";
-            break;
-        // high volume flooding
-        case TRANSACTION:
-        case FLOOD_ADVERT:
-        case FLOOD_DEMAND:
-        {
-            cat = "TX";
-            type = Scheduler::ActionType::DROPPABLE_ACTION;
-            break;
-        }
+    // NOTE: Additionally, we may use state snapshots to verify TRANSACTION type
+    // messages in the background.
 
-        // consensus, inbound
-        case GET_TX_SET:
-        case GET_SCP_QUORUMSET:
-        case GET_SCP_STATE:
-            cat = "SCPQ";
-            type = Scheduler::ActionType::DROPPABLE_ACTION;
-            break;
+    // Start tracking capacity here, so read throttling is applied
+    // appropriately. Flow control might not be started at that time
+    auto msgTracker = std::make_shared<CapacityTrackedMessage>(
+        shared_from_this(), msg.v0().message);
 
-        // consensus, self
-        case DONT_HAVE:
-        case TX_SET:
-        case GENERALIZED_TX_SET:
-        case SCP_QUORUMSET:
-        case SCP_MESSAGE:
-            cat = "SCP";
-            break;
+    std::string cat;
+    Scheduler::ActionType type = Scheduler::ActionType::NORMAL_ACTION;
 
-        default:
-            cat = "MISC";
-        }
+    switch (msgTracker->getMessage().type())
+    {
+    case HELLO:
+    case AUTH:
+        cat = AUTH_ACTION_QUEUE;
+        break;
+    // control messages
+    case PEERS:
+    case ERROR_MSG:
+    case SEND_MORE:
+    case SEND_MORE_EXTENDED:
+        cat = "CTRL";
+        break;
+    // high volume flooding
+    case TRANSACTION:
+    case FLOOD_ADVERT:
+    case FLOOD_DEMAND:
+    {
+        cat = "TX";
+        type = Scheduler::ActionType::DROPPABLE_ACTION;
+        break;
+    }
 
-        // processing of incoming messages during authenticated must be
-        // in-order, so while not authenticated, place all messages onto
-        // AUTH_ACTION_QUEUE scheduler queue
-        queueName = isAuthenticated(guard) ? cat : AUTH_ACTION_QUEUE;
-        type = isAuthenticated(guard) ? type
-                                      : Scheduler::ActionType::NORMAL_ACTION;
+    // consensus, inbound
+    case GET_TX_SET:
+    case GET_SCP_QUORUMSET:
+    case GET_SCP_STATE:
+        cat = "SCPQ";
+        type = Scheduler::ActionType::DROPPABLE_ACTION;
+        break;
 
-        // TODO: This scope (ending here) exists to ensure this doesn't hold the
-        // state lock upon entry to the transaction queue. This can cause
-        // deadlocks! I think it's safe to release the lock here as there's no
-        // longer any state querying. In practice though, if I end up posting
-        // the tryAdd action onto some tx-queue specific thread, then I can
-        // remove the scoping I added here and the lock will be released upon
-        // return from this function (like it always has).
+    // consensus, self
+    case DONT_HAVE:
+    case TX_SET:
+    case GENERALIZED_TX_SET:
+    case SCP_QUORUMSET:
+    case SCP_MESSAGE:
+        cat = "SCP";
+        break;
 
-        // TODO: Really investigate whether this peer+transaction queue locking
-        // each other issue can come up anywhere else.
+    default:
+        cat = "MISC";
     }
 
-    // TODO: vv Remove asserts if I get rid of the scoping above
-    releaseAssert(msgTracker);
-    releaseAssert(!queueName.empty());
+    // processing of incoming messages during authenticated must be in-order, so
+    // while not authenticated, place all messages onto AUTH_ACTION_QUEUE
+    // scheduler queue
+    auto queueName = isAuthenticated(guard) ? cat : AUTH_ACTION_QUEUE;
+    type = isAuthenticated(guard) ? type : Scheduler::ActionType::NORMAL_ACTION;
 
     // If a message is already scheduled, drop
     if (mAppConnector.checkScheduledAndCache(msgTracker))
@@ -1027,9 +1002,6 @@ Peer::recvAuthenticatedMessage(AuthenticatedMessage&& msg)
                 self->recvMessage(t);
             },
             "Peer::recvMessage"); // TODO: Change message to something better
-        // TODO: If I end up running this on a different thread then I need to
-        // be sure to std::move `msgTracker` into the lambda as-per the note
-        // below.
     }
     else
     {
diff --git a/src/overlay/TCPPeer.cpp b/src/overlay/TCPPeer.cpp
index 85d2e30b26..f3129f94ab 100644
--- a/src/overlay/TCPPeer.cpp
+++ b/src/overlay/TCPPeer.cpp
@@ -528,14 +528,10 @@ TCPPeer::startRead()
     ZoneScoped;
     releaseAssert(!threadIsMain() || !useBackgroundThread());
     releaseAssert(canRead());
+    RECURSIVE_LOCK_GUARD(mStateMutex, guard);
+    if (shouldAbort(guard))
     {
-        RECURSIVE_LOCK_GUARD(mStateMutex, guard);
-        if (shouldAbort(guard))
-        {
-            return;
-        }
-        // TODO: Remove this outer scoping if I add separate thread for bg tx
-        // queue
+        return;
     }
 
     mThreadVars.getIncomingHeader().clear();

From 5b2dc94563af3bc36355ecfdd769a566cb92d9f2 Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Thu, 6 Mar 2025 14:24:06 -0800
Subject: [PATCH 13/14] Post-rebase fixes

---
 src/herder/TxSetFrame.cpp                 |  6 ++-
 src/herder/test/HerderTests.cpp           |  2 +
 src/herder/test/TxSetTests.cpp            | 10 ++--
 src/ledger/LedgerManagerImpl.cpp          | 56 +++--------------------
 src/main/AppConnector.cpp                 |  6 ++-
 src/main/AppConnector.h                   |  1 -
 src/main/Application.h                    |  2 +-
 src/main/ApplicationImpl.cpp              |  5 +-
 src/transactions/TransactionFrameBase.cpp |  2 +-
 src/transactions/TransactionUtils.h       |  1 +
 10 files changed, 26 insertions(+), 65 deletions(-)

diff --git a/src/herder/TxSetFrame.cpp b/src/herder/TxSetFrame.cpp
index 3fe1473cf5..4b4faf7f3a 100644
--- a/src/herder/TxSetFrame.cpp
+++ b/src/herder/TxSetFrame.cpp
@@ -1817,10 +1817,12 @@ TxSetPhaseFrame::txsAreValid(Application& app,
     LedgerSnapshot ls(app);
     ls.getLedgerHeader().currentToModify().ledgerSeq =
         app.getLedgerManager().getLastClosedLedgerNum() + 1;
+
+    // TODO: I *think* forApply should be false here? Is that right?
+    AppValidationWrapper avw(app.getAppConnector(), false, std::nullopt);
     for (auto const& tx : *this)
     {
-        auto txResult = tx->checkValid(app.getAppConnector(), ls, 0,
-                                       lowerBoundCloseTimeOffset,
+        auto txResult = tx->checkValid(avw, ls, 0, lowerBoundCloseTimeOffset,
                                        upperBoundCloseTimeOffset);
         if (!txResult->isSuccess())
         {
diff --git a/src/herder/test/HerderTests.cpp b/src/herder/test/HerderTests.cpp
index 1be7aa430f..9df070f6da 100644
--- a/src/herder/test/HerderTests.cpp
+++ b/src/herder/test/HerderTests.cpp
@@ -3225,6 +3225,8 @@ TEST_CASE("overlay parallel processing")
 
 // Background ledger close requires postgres
 #ifdef USE_POSTGRES
+    // TODO: This is broken wtih background tx queue. Fix it. (note that this
+    // test doesn't run at all when --disable-postgres is set)
     SECTION("background ledger close")
     {
         // Set threshold to 1 so all have to vote
diff --git a/src/herder/test/TxSetTests.cpp b/src/herder/test/TxSetTests.cpp
index 4011aa2e26..6dd9917fec 100644
--- a/src/herder/test/TxSetTests.cpp
+++ b/src/herder/test/TxSetTests.cpp
@@ -991,9 +991,10 @@ TEST_CASE("applicable txset validation - transactions belong to correct phase",
                         1)},
                     2000);
             }
+            AppValidationWrapper avw(app->getAppConnector(), false,
+                                     std::nullopt);
             LedgerSnapshot ls(*app);
-            REQUIRE(tx->checkValid(app->getAppConnector(), ls, 0, 0, 0)
-                        ->isSuccess());
+            REQUIRE(tx->checkValid(avw, ls, 0, 0, 0)->isSuccess());
             return tx;
         };
 
@@ -1117,9 +1118,10 @@ TEST_CASE("applicable txset validation - Soroban resources", "[txset][soroban]")
             auto tx = sorobanTransactionFrameFromOps(
                 app->getNetworkID(), source, {op}, {}, resources, 2000,
                 100'000'000);
+            AppValidationWrapper avw(app->getAppConnector(), false,
+                                     std::nullopt);
             LedgerSnapshot ls(*app);
-            REQUIRE(tx->checkValid(app->getAppConnector(), ls, 0, 0, 0)
-                        ->isSuccess());
+            REQUIRE(tx->checkValid(avw, ls, 0, 0, 0)->isSuccess());
             return tx;
         };
 
diff --git a/src/ledger/LedgerManagerImpl.cpp b/src/ledger/LedgerManagerImpl.cpp
index 1e12adb851..fe4b7a26b8 100644
--- a/src/ledger/LedgerManagerImpl.cpp
+++ b/src/ledger/LedgerManagerImpl.cpp
@@ -145,21 +145,6 @@ LedgerManager::maxSorobanLedgerResources(SorobanNetworkConfig const& conf)
     return Resource(limits);
 }
 
-Resource
-LedgerManager::maxSorobanTransactionResources(SorobanNetworkConfig const& conf)
-{
-    ZoneScoped;
-    int64_t const opCount = 1;
-    std::vector<int64_t> limits = {opCount,
-                                   conf.txMaxInstructions(),
-                                   conf.txMaxSizeBytes(),
-                                   conf.txMaxReadBytes(),
-                                   conf.txMaxWriteBytes(),
-                                   conf.txMaxReadLedgerEntries(),
-                                   conf.txMaxWriteLedgerEntries()};
-    return Resource(limits);
-}
-
 uint32_t
 LedgerManager::getMaxTxSetSizeOps(LedgerHeader const& header)
 {
@@ -170,30 +155,11 @@ LedgerManager::getMaxTxSetSizeOps(LedgerHeader const& header)
                : (n * MAX_OPS_PER_TX);
 }
 
-Resource
-LedgerManager::maxClassicLedgerResources(LedgerHeader const& header)
-{
-    return Resource(LedgerManager::getMaxTxSetSizeOps(header));
-}
-
-Resource
-LedgerManager::maxSorobanLedgerResources(SorobanNetworkConfig const& conf)
-{
-    ZoneScoped std::vector<int64_t> limits = {
-        conf.ledgerMaxTxCount(),
-        conf.ledgerMaxInstructions(),
-        conf.ledgerMaxTransactionSizesBytes(),
-        conf.ledgerMaxReadBytes(),
-        conf.ledgerMaxWriteBytes(),
-        conf.ledgerMaxReadLedgerEntries(),
-        conf.ledgerMaxWriteLedgerEntries()};
-    return Resource(limits);
-}
-
 Resource
 LedgerManager::maxSorobanTransactionResources(SorobanNetworkConfig const& conf)
 {
-    ZoneScoped int64_t const opCount = 1;
+    ZoneScoped;
+    int64_t const opCount = 1;
     std::vector<int64_t> limits = {opCount,
                                    conf.txMaxInstructions(),
                                    conf.txMaxSizeBytes(),
@@ -204,16 +170,6 @@ LedgerManager::maxSorobanTransactionResources(SorobanNetworkConfig const& conf)
     return Resource(limits);
 }
 
-uint32_t
-LedgerManager::getMaxTxSetSizeOps(LedgerHeader const& header)
-{
-    auto n = header.maxTxSetSize;
-    return protocolVersionStartsFrom(header.ledgerVersion,
-                                     ProtocolVersion::V_11)
-               ? n
-               : (n * MAX_OPS_PER_TX);
-}
-
 LedgerManagerImpl::LedgerApplyMetrics::LedgerApplyMetrics(
     medida::MetricsRegistry& registry)
     : mTransactionApply(registry.NewTimer({"ledger", "transaction", "apply"}))
@@ -506,7 +462,7 @@ uint32_t
 LedgerManagerImpl::getLastMaxTxSetSizeOps() const
 {
     releaseAssert(threadIsMain());
-    return LedgerManager::getMaxTxSetSizeOps(mLastClosedLedger.header);
+    return LedgerManager::getMaxTxSetSizeOps(getLCLState().ledgerHeader.header);
 }
 
 Resource
@@ -518,12 +474,12 @@ LedgerManagerImpl::maxLedgerResources(bool isSoroban)
     if (isSoroban)
     {
         return LedgerManager::maxSorobanLedgerResources(
-            getSorobanNetworkConfigReadOnly());
+            getLastClosedSorobanNetworkConfig());
     }
     else
     {
         return LedgerManager::maxClassicLedgerResources(
-            mLastClosedLedger.header);
+            getLCLState().ledgerHeader.header);
     }
 }
 
@@ -533,7 +489,7 @@ LedgerManagerImpl::maxSorobanTransactionResources()
     ZoneScoped;
 
     return LedgerManager::maxSorobanTransactionResources(
-        mApp.getLedgerManager().getSorobanNetworkConfigReadOnly());
+        getLastClosedSorobanNetworkConfig());
 }
 
 int64_t
diff --git a/src/main/AppConnector.cpp b/src/main/AppConnector.cpp
index d54b895e27..d615ee1027 100644
--- a/src/main/AppConnector.cpp
+++ b/src/main/AppConnector.cpp
@@ -56,6 +56,8 @@ AppConnector::getLastClosedSorobanNetworkConfig() const
 SorobanNetworkConfig const&
 AppConnector::getSorobanNetworkConfigForApply() const
 {
+    releaseAssert(threadIsMain() ||
+                  mApp.threadIsType(Application::ThreadType::APPLY));
     return mApp.getLedgerManager().getSorobanNetworkConfigForApply();
 }
 
@@ -63,9 +65,9 @@ std::optional<SorobanNetworkConfig>
 AppConnector::maybeGetSorobanNetworkConfigReadOnly() const
 {
     releaseAssert(threadIsMain());
-    if (mApp.getLedgerManager().hasSorobanNetworkConfig())
+    if (mApp.getLedgerManager().hasLastClosedSorobanNetworkConfig())
     {
-        return mApp.getLedgerManager().getSorobanNetworkConfigReadOnly();
+        return mApp.getLedgerManager().getLastClosedSorobanNetworkConfig();
     }
     return std::nullopt;
 }
diff --git a/src/main/AppConnector.h b/src/main/AppConnector.h
index 0f0e4a7c7d..673b8d5dc8 100644
--- a/src/main/AppConnector.h
+++ b/src/main/AppConnector.h
@@ -72,7 +72,6 @@ class AppConnector
     // `getSorobanNetworkConfig` will throw an assertion error in that case.
     std::optional<SorobanNetworkConfig>
     maybeGetSorobanNetworkConfigReadOnly() const;
-    bool threadIsType(Application::ThreadType type) const;
 
     bool threadIsType(Application::ThreadType type) const;
 
diff --git a/src/main/Application.h b/src/main/Application.h
index 56646fc3a3..b4d2db1f0d 100644
--- a/src/main/Application.h
+++ b/src/main/Application.h
@@ -171,7 +171,7 @@ class Application
         WORKER,
         EVICTION,
         OVERLAY,
-        LEDGER_CLOSE,
+        APPLY,
         TX_QUEUE
     };
 
diff --git a/src/main/ApplicationImpl.cpp b/src/main/ApplicationImpl.cpp
index 4b5a687a3e..b9b9d300aa 100644
--- a/src/main/ApplicationImpl.cpp
+++ b/src/main/ApplicationImpl.cpp
@@ -199,10 +199,7 @@ ApplicationImpl::ApplicationImpl(VirtualClock& clock, Config const& cfg)
     if (mConfig.BACKGROUND_TX_QUEUE)
     {
         // TODO: Keep priority unchanged as tx queue processes time-sensitive
-        // tasks? Or should tx queue priority be downgraded? The priority
-        // locking mechanism in TransactionQueue is designed to prevent tx queue
-        // from starving other work, so it may be fine to keep priority
-        // unchanged.
+        // tasks? Or should tx queue priority be downgraded?
         mTxQueueThread = std::thread{[this]() { mTxQueueIOContext->run(); }};
         mThreadTypes[mTxQueueThread->get_id()] = ThreadType::TX_QUEUE;
     }
diff --git a/src/transactions/TransactionFrameBase.cpp b/src/transactions/TransactionFrameBase.cpp
index 5eb61f48a2..7854fad848 100644
--- a/src/transactions/TransactionFrameBase.cpp
+++ b/src/transactions/TransactionFrameBase.cpp
@@ -31,7 +31,7 @@ SorobanNetworkConfig const&
 AppValidationWrapper::getSorobanNetworkConfig() const
 {
     return mForApply ? mApp.getSorobanNetworkConfigForApply()
-                     : mApp.getSorobanNetworkConfigReadOnly();
+                     : mApp.getLastClosedSorobanNetworkConfig();
 }
 
 uint32_t
diff --git a/src/transactions/TransactionUtils.h b/src/transactions/TransactionUtils.h
index 0c1d676ea1..204badd415 100644
--- a/src/transactions/TransactionUtils.h
+++ b/src/transactions/TransactionUtils.h
@@ -17,6 +17,7 @@ namespace stellar
 {
 
 class Application;
+class AppConnector;
 class Config;
 class ConstLedgerTxnEntry;
 class ConstTrustLineWrapper;

From d3e9cccf2a9d2785a7d0ca9db7340c069869e77a Mon Sep 17 00:00:00 2001
From: Brett Boston <brett.boston@stellar.org>
Date: Wed, 12 Mar 2025 15:56:49 -0700
Subject: [PATCH 14/14] Formatting

---
 src/herder/HerderImpl.cpp       | 6 ++++--
 src/herder/TransactionQueue.cpp | 1 -
 src/overlay/OverlayManager.h    | 8 ++++----
 3 files changed, 8 insertions(+), 7 deletions(-)

diff --git a/src/herder/HerderImpl.cpp b/src/herder/HerderImpl.cpp
index cfdff15890..e9c702d5ee 100644
--- a/src/herder/HerderImpl.cpp
+++ b/src/herder/HerderImpl.cpp
@@ -585,7 +585,7 @@ HerderImpl::emitEnvelope(SCPEnvelope const& envelope)
 // TODO: Move to Herder.cpp?
 TransactionQueue::AddResult
 Herder::recvTransaction(TransactionQueuesPtr txQueues,
-                            TransactionFrameBasePtr tx, bool submittedFromSelf)
+                        TransactionFrameBasePtr tx, bool submittedFromSelf)
 {
     ZoneScoped;
     ClassicTransactionQueue& classicTxQueue =
@@ -2521,7 +2521,9 @@ HerderImpl::getTx(Hash const& hash) const
     return mTransactionQueues->getTx(hash);
 }
 
-TransactionQueuesPtr HerderImpl::getTransactionQueues() const {
+TransactionQueuesPtr
+HerderImpl::getTransactionQueues() const
+{
     releaseAssert(mTransactionQueues);
     return mTransactionQueues;
 }
diff --git a/src/herder/TransactionQueue.cpp b/src/herder/TransactionQueue.cpp
index 3f4cdc9bb1..4ad4fb593f 100644
--- a/src/herder/TransactionQueue.cpp
+++ b/src/herder/TransactionQueue.cpp
@@ -343,7 +343,6 @@ TransactionQueue::lock() const
     return TxQueueLock(std::move(lock), mTxQueueCv);
 }
 
-
 TransactionQueue::AddResult
 TransactionQueue::canAdd(
     TransactionFrameBasePtr tx, AccountStates::iterator& stateIter,
diff --git a/src/overlay/OverlayManager.h b/src/overlay/OverlayManager.h
index 6809b59dfb..595e8adb97 100644
--- a/src/overlay/OverlayManager.h
+++ b/src/overlay/OverlayManager.h
@@ -95,10 +95,10 @@ class OverlayManager
     // TODO: Docs
     // TODO: This is going to be called in a lambda. Be careful about those
     // pointers and references
-    virtual void
-    recordAddTransactionStats(TxQueueAddResult const& addResult,
-                              Hash const& txHash, Peer::pointer peer,
-                              Hash const& index) = 0;
+    virtual void recordAddTransactionStats(TxQueueAddResult const& addResult,
+                                           Hash const& txHash,
+                                           Peer::pointer peer,
+                                           Hash const& index) = 0;
 
     // Process incoming transaction, pass it down to the transaction queue
     virtual void recvTransaction(StellarMessage const& msg, Peer::pointer peer,