Skip to content

Commit

Permalink
Adapt tests to use ValidationPolicy and ComputeLedgerEvents
Browse files Browse the repository at this point in the history
  • Loading branch information
jasagredo committed Feb 17, 2025
1 parent 534d6ac commit 4d3170d
Show file tree
Hide file tree
Showing 22 changed files with 99 additions and 103 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ exampleBlock =
cfg
(BlockNo 1)
(SlotNo 1)
(applyChainTick ledgerConfig (SlotNo 1) ledgerStateAfterEBB)
(applyChainTick OmitLedgerEvents ledgerConfig (SlotNo 1) ledgerStateAfterEBB)
[ValidatedByronTx exampleGenTx]
(fakeMkIsLeader leaderCredentials)
where
Expand Down Expand Up @@ -180,14 +180,14 @@ emptyLedgerState = ByronLedgerState {

ledgerStateAfterEBB :: LedgerState ByronBlock
ledgerStateAfterEBB =
reapplyLedgerBlock ledgerConfig exampleEBB
. applyChainTick ledgerConfig (SlotNo 0)
reapplyLedgerBlock OmitLedgerEvents ledgerConfig exampleEBB
. applyChainTick OmitLedgerEvents ledgerConfig (SlotNo 0)
$ emptyLedgerState

exampleLedgerState :: LedgerState ByronBlock
exampleLedgerState =
reapplyLedgerBlock ledgerConfig exampleBlock
. applyChainTick ledgerConfig (SlotNo 1)
reapplyLedgerBlock OmitLedgerEvents ledgerConfig exampleBlock
. applyChainTick OmitLedgerEvents ledgerConfig (SlotNo 1)
$ ledgerStateAfterEBB

exampleHeaderState :: HeaderState ByronBlock
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import qualified Ouroboros.Consensus.ByronSpec.Ledger.Rules as Rules
import Ouroboros.Consensus.Ledger.Abstract
import Ouroboros.Consensus.Ledger.CommonProtocolParams
import Ouroboros.Consensus.Ticked
import Ouroboros.Consensus.Util ((..:))

{-------------------------------------------------------------------------------
State
Expand Down Expand Up @@ -103,7 +102,7 @@ instance IsLedger (LedgerState ByronSpecBlock) where
type AuxLedgerEvent (LedgerState ByronSpecBlock) =
VoidLedgerEvent (LedgerState ByronSpecBlock)

applyChainTickLedgerResult cfg slot (ByronSpecLedgerState tip state) =
applyChainTickLedgerResult _evs cfg slot (ByronSpecLedgerState tip state) =
pureLedgerResult
$ TickedByronSpecLedgerState {
untickedByronSpecLedgerTip = tip
Expand All @@ -118,7 +117,7 @@ instance IsLedger (LedgerState ByronSpecBlock) where
-------------------------------------------------------------------------------}

instance ApplyBlock (LedgerState ByronSpecBlock) ByronSpecBlock where
applyBlockLedgerResult cfg block (TickedByronSpecLedgerState _tip state) =
applyBlockLedgerResultWithValidation _ _ cfg block (TickedByronSpecLedgerState _tip state) =
withExcept ByronSpecLedgerError
$ fmap (pureLedgerResult . ByronSpecLedgerState (Just (blockSlot block)))
$ -- Note that the CHAIN rule also applies the chain tick. So even
Expand All @@ -131,14 +130,9 @@ instance ApplyBlock (LedgerState ByronSpecBlock) ByronSpecBlock where
(byronSpecBlock block)
state

applyBlockLedgerResult = defaultApplyBlockLedgerResult
reapplyBlockLedgerResult =
-- The spec doesn't have a "reapply" mode
dontExpectError ..: applyBlockLedgerResult
where
dontExpectError :: Except a b -> b
dontExpectError mb = case runExcept mb of
Left _ -> error "reapplyBlockLedgerResult: unexpected error"
Right b -> b
defaultReapplyBlockLedgerResult (error . ("reapplyBlockLedgerResult: unexpected error " ++) . show)

{-------------------------------------------------------------------------------
CommonProtocolParams
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@ import Ouroboros.Consensus.HardFork.Combinator.Ledger
(tickedHardForkLedgerStatePerEra)
import Ouroboros.Consensus.HardFork.Combinator.State.Types
(currentState, getHardForkState)
import Ouroboros.Consensus.Ledger.Basics (LedgerConfig, LedgerState,
applyChainTick)
import Ouroboros.Consensus.Ledger.Basics (ComputeLedgerEvents (..),
LedgerConfig, LedgerState, applyChainTick)
import Ouroboros.Consensus.NodeId (CoreNodeId (..))
import Ouroboros.Consensus.Protocol.TPraos (TPraos)
import Ouroboros.Consensus.Shelley.Ledger (ShelleyBlock, mkShelleyTx)
Expand Down Expand Up @@ -211,7 +211,7 @@ migrateUTxO migrationInfo curSlot lcfg lst
mbUTxO =
fmap getUTxOShelley $
ejectShelleyTickedLedgerState $
applyChainTick lcfg curSlot $
applyChainTick OmitLedgerEvents lcfg curSlot $
lst

MigrationInfo
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,8 @@ import Ouroboros.Consensus.Ledger.Abstract
(ApplyBlock (reapplyBlockLedgerResult), LedgerCfg,
LedgerConfig, applyBlockLedgerResult, applyChainTick,
tickThenApply, tickThenApplyLedgerResult, tickThenReapply)
import Ouroboros.Consensus.Ledger.Basics (LedgerResult (..),
LedgerState, getTipSlot)
import Ouroboros.Consensus.Ledger.Basics (ComputeLedgerEvents (..),
LedgerResult (..), LedgerState, getTipSlot)
import Ouroboros.Consensus.Ledger.Extended
import Ouroboros.Consensus.Ledger.SupportsMempool
(LedgerSupportsMempool)
Expand All @@ -74,7 +74,7 @@ import qualified Ouroboros.Consensus.Storage.ImmutableDB as ImmutableDB
import Ouroboros.Consensus.Storage.LedgerDB (DiskSnapshot (..),
writeSnapshot)
import Ouroboros.Consensus.Storage.Serialisation (encodeDisk)
import Ouroboros.Consensus.Util (Flag (..), (..:))
import Ouroboros.Consensus.Util (Flag (..), (...:))
import qualified Ouroboros.Consensus.Util.IOLike as IOLike
import Ouroboros.Network.SizeInBytes
import System.FS.API (SomeHasFS (..))
Expand Down Expand Up @@ -394,7 +394,7 @@ storeLedgerStateAt slotNo ledgerAppMode doChecksum env = do
process :: ExtLedgerState blk -> blk -> IO (NextStep, ExtLedgerState blk)
process oldLedger blk = do
let ledgerCfg = ExtLedgerCfg cfg
case runExcept $ tickThenXApply ledgerCfg blk oldLedger of
case runExcept $ tickThenXApply OmitLedgerEvents ledgerCfg blk oldLedger of
Right newLedger -> do
when (blockSlot blk >= slotNo) $ storeLedgerState newLedger
when (blockSlot blk > slotNo) $ issueWarning blk
Expand All @@ -406,7 +406,7 @@ storeLedgerStateAt slotNo ledgerAppMode doChecksum env = do
pure (Stop, oldLedger)

tickThenXApply = case ledgerAppMode of
LedgerReapply -> pure ..: tickThenReapply
LedgerReapply -> pure ...: tickThenReapply
LedgerApply -> tickThenApply

continue :: blk -> NextStep
Expand Down Expand Up @@ -473,7 +473,7 @@ checkNoThunksEvery
process :: ExtLedgerState blk -> blk -> IO (ExtLedgerState blk)
process oldLedger blk = do
let ledgerCfg = ExtLedgerCfg cfg
appliedResult = tickThenApplyLedgerResult ledgerCfg blk oldLedger
appliedResult = tickThenApplyLedgerResult OmitLedgerEvents ledgerCfg blk oldLedger
newLedger = either (error . show) lrResult $ runExcept $ appliedResult
bn = blockNo blk
when (unBlockNo bn `mod` nBlocks == 0 ) $ IOLike.evaluate (ledgerState newLedger) >>= checkNoThunks bn
Expand Down Expand Up @@ -511,7 +511,7 @@ traceLedgerProcessing
-> IO (ExtLedgerState blk)
process oldLedger blk = do
let ledgerCfg = ExtLedgerCfg cfg
appliedResult = tickThenApplyLedgerResult ledgerCfg blk oldLedger
appliedResult = tickThenApplyLedgerResult OmitLedgerEvents ledgerCfg blk oldLedger
newLedger = either (error . show) lrResult $ runExcept $ appliedResult
traces =
(HasAnalysis.emitTraces $
Expand Down Expand Up @@ -667,18 +667,18 @@ benchmarkLedgerOps mOutfile ledgerAppMode AnalysisEnv {db, registry, startFrom,
-> ExtLedgerState blk
-> IO (Ticked (LedgerState blk))
tickTheLedgerState slot st =
pure $ applyChainTick lcfg slot (ledgerState st)
pure $ applyChainTick OmitLedgerEvents lcfg slot (ledgerState st)

applyTheBlock ::
Ticked (LedgerState blk)
-> IO (LedgerState blk)
applyTheBlock tickedLedgerSt = case ledgerAppMode of
LedgerApply ->
case runExcept (lrResult <$> applyBlockLedgerResult lcfg blk tickedLedgerSt) of
case runExcept (lrResult <$> applyBlockLedgerResult OmitLedgerEvents lcfg blk tickedLedgerSt) of
Left err -> fail $ "benchmark doesn't support invalid blocks: " <> show rp <> " " <> show err
Right x -> pure x
LedgerReapply ->
pure $! lrResult $ reapplyBlockLedgerResult lcfg blk tickedLedgerSt
pure $! lrResult $ reapplyBlockLedgerResult OmitLedgerEvents lcfg blk tickedLedgerSt

withFile :: Maybe FilePath -> (IO.Handle -> IO r) -> IO r
withFile (Just outfile) = IO.withFile outfile IO.WriteMode
Expand Down Expand Up @@ -707,7 +707,7 @@ getBlockApplicationMetrics (NumberOfBlocks nrBlocks) mOutFile env = do

process :: IO.Handle -> ExtLedgerState blk -> blk -> IO (ExtLedgerState blk)
process outFileHandle currLedgerSt blk = do
let nextLedgerSt = tickThenReapply (ExtLedgerCfg cfg) blk currLedgerSt
let nextLedgerSt = tickThenReapply OmitLedgerEvents (ExtLedgerCfg cfg) blk currLedgerSt
when (unBlockNo (blockNo blk) `mod` nrBlocks == 0) $ do
let blockApplication =
HasAnalysis.WithLedgerState blk
Expand Down Expand Up @@ -830,7 +830,7 @@ reproMempoolForge numBlks env = do
do
let slot = blockSlot blk
(ticked, durTick, mutTick, gcTick) <- timed $ IOLike.evaluate $
applyChainTick lCfg slot (ledgerState st)
applyChainTick OmitLedgerEvents lCfg slot (ledgerState st)
((), durSnap, mutSnap, gcSnap) <- timed $ IOLike.atomically $ do
snap <- Mempool.getSnapshotFor mempool $ Mempool.ForgeInKnownSlot slot ticked

Expand Down Expand Up @@ -858,7 +858,7 @@ reproMempoolForge numBlks env = do
-- since it currently matches the call in the forging thread, which is
-- the primary intention of this Analysis. Maybe GHC's CSE is already
-- doing this sharing optimization?
IOLike.atomically $ IOLike.writeTVar ref $! tickThenReapply elCfg blk st
IOLike.atomically $ IOLike.writeTVar ref $! tickThenReapply OmitLedgerEvents elCfg blk st

-- this flushes blk from the mempool, since every tx in it is now on the chain
void $ Mempool.syncWithLedger mempool
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,7 @@ runForge epochSize_ nextSlot opts chainDB blockForging cfg genTxs = do
let tickedLedgerState :: Ticked (LedgerState blk)
tickedLedgerState =
applyChainTick
OmitLedgerEvents
(configLedger cfg)
currentSlot
(ledgerState unticked)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ instance HashAlgorithm h => TxGen (ShelleyBlock (TPraos (MockCrypto h)) (MockShe
then pure []
else do
n <- choose (0, 20)
go [] n $ applyChainTick lcfg curSlotNo lst
go [] n $ applyChainTick OmitLedgerEvents lcfg curSlotNo lst
where
ShelleyTxGenExtra
{ stgeGenEnv
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -257,7 +257,7 @@ byronPBftParams ByronSpecGenesis{..} =
instance TxGen DualByronBlock where
testGenTxs _coreNodeId _numCoreNodes curSlotNo cfg () = \st -> do
n <- choose (0, 20)
go [] n $ applyChainTick (configLedger cfg) curSlotNo st
go [] n $ applyChainTick OmitLedgerEvents (configLedger cfg) curSlotNo st
where
-- Attempt to produce @n@ transactions
-- Stops when the transaction generator cannot produce more txs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -314,7 +314,7 @@ prop_simple_real_tpraos_convergence TestSetup
-- slots to reach the epoch transition but the last several
-- slots end up empty.
Shelley.tickedShelleyLedgerState $
applyChainTick ledgerConfig sentinel lsUnticked
applyChainTick OmitLedgerEvents ledgerConfig sentinel lsUnticked

msg =
"The ticked final ledger state of " <> show nid <>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -622,11 +622,11 @@ runThreadNetwork systemTime ThreadNetworkArgs
snap1 <- getSnapshotFor mempool $
-- This node would include these crucial txs if it leads in
-- this slot.
ForgeInKnownSlot slot $ applyChainTick lcfg slot ledger
ForgeInKnownSlot slot $ applyChainTick OmitLedgerEvents lcfg slot ledger
snap2 <- getSnapshotFor mempool $
-- Other nodes might include these crucial txs when leading
-- in the next slot.
ForgeInKnownSlot (succ slot) $ applyChainTick lcfg (succ slot) ledger
ForgeInKnownSlot (succ slot) $ applyChainTick OmitLedgerEvents lcfg (succ slot) ledger
-- This loop will repeat for the next slot, so we only need to
-- check for this one and the next.
pure (snap1, snap2)
Expand Down Expand Up @@ -887,10 +887,10 @@ runThreadNetwork systemTime ThreadNetworkArgs

-- fail if the EBB is invalid
-- if it is valid, we retick to the /same/ slot
let apply = applyLedgerBlock (configLedger pInfoConfig)
let apply = applyLedgerBlock OmitLedgerEvents (configLedger pInfoConfig)
tickedLdgSt' <- case Exc.runExcept $ apply ebb tickedLdgSt of
Left e -> Exn.throw $ JitEbbError @blk e
Right st -> pure $ applyChainTick
Right st -> pure $ applyChainTick OmitLedgerEvents
(configLedger pInfoConfig)
currentSlot
st
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@ module Test.Consensus.HardFork.Combinator.A (
import Cardano.Slotting.EpochInfo
import Codec.Serialise
import Control.Monad (guard)
import Control.Monad.Except (runExcept)
import qualified Data.Binary as B
import Data.ByteString as Strict
import qualified Data.ByteString.Lazy as Lazy
Expand Down Expand Up @@ -81,7 +80,7 @@ import Ouroboros.Consensus.Node.Serialisation
import Ouroboros.Consensus.Protocol.Abstract
import Ouroboros.Consensus.Storage.ImmutableDB (simpleChunkInfo)
import Ouroboros.Consensus.Storage.Serialisation
import Ouroboros.Consensus.Util (repeatedlyM, (..:), (.:))
import Ouroboros.Consensus.Util (repeatedlyM, (.:))
import Ouroboros.Consensus.Util.Condense
import Ouroboros.Consensus.Util.Orphans ()
import Ouroboros.Network.Block (Serialised, unwrapCBORinCBOR,
Expand Down Expand Up @@ -213,10 +212,10 @@ instance IsLedger (LedgerState BlockA) where
type AuxLedgerEvent (LedgerState BlockA) =
VoidLedgerEvent (LedgerState BlockA)

applyChainTickLedgerResult _ _ = pureLedgerResult . TickedLedgerStateA
applyChainTickLedgerResult _ _ _ = pureLedgerResult . TickedLedgerStateA

instance ApplyBlock (LedgerState BlockA) BlockA where
applyBlockLedgerResult cfg blk =
applyBlockLedgerResultWithValidation _ _ cfg blk =
fmap (pureLedgerResult . setTip)
. repeatedlyM
(fmap fst .: applyTx cfg DoNotIntervene (blockSlot blk))
Expand All @@ -225,13 +224,9 @@ instance ApplyBlock (LedgerState BlockA) BlockA where
setTip :: TickedLedgerState BlockA -> LedgerState BlockA
setTip (TickedLedgerStateA st) = st { lgrA_tip = blockPoint blk }

applyBlockLedgerResult = defaultApplyBlockLedgerResult
reapplyBlockLedgerResult =
dontExpectError ..: applyBlockLedgerResult
where
dontExpectError :: Except a b -> b
dontExpectError mb = case runExcept mb of
Left _ -> error "reapplyBlockLedgerResult: unexpected error"
Right b -> b
defaultReapplyBlockLedgerResult absurd

instance UpdateLedger BlockA

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -186,11 +186,12 @@ instance IsLedger (LedgerState BlockB) where
type AuxLedgerEvent (LedgerState BlockB) =
VoidLedgerEvent (LedgerState BlockB)

applyChainTickLedgerResult _ _ = pureLedgerResult . TickedLedgerStateB
applyChainTickLedgerResult _ _ _ = pureLedgerResult . TickedLedgerStateB

instance ApplyBlock (LedgerState BlockB) BlockB where
applyBlockLedgerResult = \_ b _ -> return $ pureLedgerResult $ LgrB (blockPoint b)
reapplyBlockLedgerResult = \_ b _ -> pureLedgerResult $ LgrB (blockPoint b)
applyBlockLedgerResultWithValidation = \_ _ _ b _ -> return $ pureLedgerResult $ LgrB (blockPoint b)
applyBlockLedgerResult = defaultApplyBlockLedgerResult
reapplyBlockLedgerResult = defaultReapplyBlockLedgerResult absurd

instance UpdateLedger BlockB

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import Ouroboros.Consensus.Block.Abstract
import Ouroboros.Consensus.Config
(TopLevelConfig (topLevelConfigLedger), configCodec)
import Ouroboros.Consensus.HardFork.History.EraParams (eraEpochSize)
import Ouroboros.Consensus.Ledger.Basics (ComputeLedgerEvents (..))
import Ouroboros.Consensus.Ledger.Extended (ExtLedgerState)
import Ouroboros.Consensus.Protocol.Abstract
import Ouroboros.Consensus.Storage.ChainDB hiding
Expand Down Expand Up @@ -117,7 +118,7 @@ fromMinimalChainDbArgs MinimalChainDbArgs {..} = ChainDbArgs {
, lgrGenesis = return mcdbInitLedger
, lgrHasFS = SomeHasFS $ simHasFS (nodeDBsLgr mcdbNodeDBs)
, lgrTracer = nullTracer
, lgrConfig = configLedgerDb mcdbTopLevelConfig
, lgrConfig = configLedgerDb mcdbTopLevelConfig OmitLedgerEvents
}
, cdbsArgs = ChainDbSpecificArgs {
cdbsBlocksToAddSize = 1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -490,7 +490,7 @@ instance ( Typeable ptype

instance PayloadSemantics ptype
=> ApplyBlock (LedgerState (TestBlockWith ptype)) (TestBlockWith ptype) where
applyBlockLedgerResult _ tb@TestBlockWith{..} (TickedTestLedger TestLedger{..})
applyBlockLedgerResultWithValidation _validation _events _ tb@TestBlockWith{..} (TickedTestLedger TestLedger{..})
| blockPrevHash tb /= pointHash lastAppliedPoint
= throwError $ InvalidHash (pointHash lastAppliedPoint) (blockPrevHash tb)
| tbValid == Invalid
Expand All @@ -504,15 +504,9 @@ instance PayloadSemantics ptype
, payloadDependentState = st'
}

reapplyBlockLedgerResult _ tb@TestBlockWith{..} (TickedTestLedger TestLedger{..}) =
case applyPayload payloadDependentState tbPayload of
Left err -> error $ "Found an error when reapplying a block: " ++ show err
Right st' -> pureLedgerResult
$ TestLedger {
lastAppliedPoint = Chain.blockPoint tb
, payloadDependentState = st'
}

applyBlockLedgerResult = defaultApplyBlockLedgerResult
reapplyBlockLedgerResult =
defaultReapplyBlockLedgerResult (error . ("Found an error when reapplying a block: " ++) . show)

data instance LedgerState (TestBlockWith ptype) =
TestLedger {
Expand Down Expand Up @@ -573,7 +567,7 @@ instance PayloadSemantics ptype => IsLedger (LedgerState (TestBlockWith ptype))
type AuxLedgerEvent (LedgerState (TestBlockWith ptype)) =
VoidLedgerEvent (LedgerState (TestBlockWith ptype))

applyChainTickLedgerResult _ _ = pureLedgerResult . TickedTestLedger
applyChainTickLedgerResult _ _ _ = pureLedgerResult . TickedTestLedger

instance PayloadSemantics ptype => UpdateLedger (TestBlockWith ptype)

Expand Down
Loading

0 comments on commit 4d3170d

Please sign in to comment.