From 0feda31a7d546ef55e3267e6d98d775ec9a3cc56 Mon Sep 17 00:00:00 2001 From: Edsko de Vries Date: Wed, 5 Feb 2020 07:48:02 +0100 Subject: [PATCH] Header validation Closes #1571 Closes #1617 --- ouroboros-consensus/ouroboros-consensus.cabal | 1 + .../src/Ouroboros/Consensus/Block.hs | 8 +- .../Ouroboros/Consensus/ChainSyncClient.hs | 113 ++--- .../Ouroboros/Consensus/HeaderValidation.hs | 430 ++++++++++++++++++ .../Ouroboros/Consensus/Ledger/Abstract.hs | 6 +- .../Ouroboros/Consensus/Ledger/Byron/Block.hs | 54 +++ .../src/Ouroboros/Consensus/Ledger/Dual.hs | 22 +- .../Ouroboros/Consensus/Ledger/Dual/Byron.hs | 3 +- .../Ouroboros/Consensus/Ledger/Extended.hs | 87 ++-- .../Ouroboros/Consensus/Ledger/Mock/Block.hs | 11 + .../src/Ouroboros/Consensus/Mempool/Impl.hs | 37 +- .../src/Ouroboros/Consensus/Node.hs | 2 + .../Ouroboros/Consensus/Node/ErrorPolicy.hs | 2 +- .../Consensus/Node/ProtocolInfo/Byron.hs | 5 +- .../Consensus/Node/ProtocolInfo/Mock/BFT.hs | 4 +- .../Consensus/Node/ProtocolInfo/Mock/PBFT.hs | 3 +- .../Consensus/Node/ProtocolInfo/Mock/Praos.hs | 5 +- .../Node/ProtocolInfo/Mock/PraosRule.hs | 5 +- .../Ouroboros/Consensus/Node/Run/Abstract.hs | 3 + .../src/Ouroboros/Consensus/Node/Run/Byron.hs | 3 + .../Ouroboros/Consensus/Node/Run/DualByron.hs | 6 +- .../src/Ouroboros/Consensus/Node/Run/Mock.hs | 2 + .../src/Ouroboros/Consensus/NodeKernel.hs | 12 +- .../src/Ouroboros/Consensus/Util/CBOR.hs | 118 +++-- .../src/Ouroboros/Storage/ChainDB/API.hs | 7 +- .../Ouroboros/Storage/ChainDB/Impl/Args.hs | 7 + .../Ouroboros/Storage/ChainDB/Impl/LgrDB.hs | 34 +- .../Test/Consensus/LocalStateQueryServer.hs | 2 + .../test-consensus/Test/ThreadNet/Network.hs | 9 +- .../test-consensus/Test/ThreadNet/PBFT.hs | 7 +- .../test-consensus/Test/ThreadNet/RealPBFT.hs | 13 +- .../Ouroboros/Storage/ChainDB/AddBlock.hs | 2 + .../Test/Ouroboros/Storage/ChainDB/Model.hs | 7 +- .../Ouroboros/Storage/ChainDB/Model/Test.hs | 2 + .../Ouroboros/Storage/ChainDB/StateMachine.hs | 24 +- .../Test/Ouroboros/Storage/TestBlock.hs | 11 +- .../test-util/Test/Util/TestBlock.hs | 12 +- .../src/Ouroboros/Network/AnchoredFragment.hs | 9 + 38 files changed, 893 insertions(+), 195 deletions(-) create mode 100644 ouroboros-consensus/src/Ouroboros/Consensus/HeaderValidation.hs diff --git a/ouroboros-consensus/ouroboros-consensus.cabal b/ouroboros-consensus/ouroboros-consensus.cabal index 63c43a667e4..139e9731672 100644 --- a/ouroboros-consensus/ouroboros-consensus.cabal +++ b/ouroboros-consensus/ouroboros-consensus.cabal @@ -42,6 +42,7 @@ library Ouroboros.Consensus.ChainSyncClient Ouroboros.Consensus.ChainSyncServer Ouroboros.Consensus.Crypto.DSIGN.Cardano + Ouroboros.Consensus.HeaderValidation Ouroboros.Consensus.Ledger.Abstract Ouroboros.Consensus.Ledger.Byron Ouroboros.Consensus.Ledger.Byron.Block diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Block.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Block.hs index 23750967493..9dd3d208fca 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Block.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Block.hs @@ -25,7 +25,7 @@ module Ouroboros.Consensus.Block ( , fromIsEBB ) where -import Codec.Serialise (Serialise) +import Codec.Serialise (Serialise (..)) import GHC.Generics (Generic) import Cardano.Prelude (NoUnexpectedThunks) @@ -123,7 +123,11 @@ class ( GetHeader blk data IsEBB = IsEBB | IsNotEBB - deriving (Eq, Show, Generic, NoUnexpectedThunks, Serialise) + deriving (Eq, Show, Generic, NoUnexpectedThunks) + +instance Serialise IsEBB where + encode = encode . fromIsEBB + decode = toIsEBB <$> decode instance Condense IsEBB where condense = show diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/ChainSyncClient.hs b/ouroboros-consensus/src/Ouroboros/Consensus/ChainSyncClient.hs index f8d63316a22..d5b962c3bfc 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/ChainSyncClient.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/ChainSyncClient.hs @@ -53,6 +53,7 @@ import Ouroboros.Network.Protocol.ChainSync.PipelineDecision import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Protocol.Abstract @@ -178,7 +179,7 @@ bracketChainSyncClient tracer ChainDbView { getIsInvalidBlock } varCandidates -- | State used when the intersection between the candidate and the current -- chain is unknown. data UnknownIntersectionState blk = UnknownIntersectionState - { ourFrag :: !(AnchoredFragment (Header blk)) + { ourFrag :: !(AnchoredFragment (Header blk)) -- ^ A view of the current chain fragment. Note that this might be -- temporarily out of date w.r.t. the actual current chain until we update -- it again. @@ -187,10 +188,10 @@ data UnknownIntersectionState blk = UnknownIntersectionState -- with the candidate. -- -- INVARIANT: 'ourFrag' contains @k@ headers, unless close to genesis. - , ourChainState :: !(ChainState (BlockProtocol blk)) - -- ^ 'ChainState' corresponding to the tip (most recent block) of + , ourHeaderState :: !(HeaderState blk) + -- ^ 'HeaderState' corresponding to the tip (most recent block) of -- 'ourFrag'. - , ourTip :: !(Our (Tip blk)) + , ourTip :: !(Our (Tip blk)) -- ^ INVARIANT: must correspond to the tip of 'ourFrag'. } deriving (Generic) @@ -202,12 +203,12 @@ instance ( ProtocolLedgerView blk -- | State used when the intersection between the candidate and the current -- chain is known. data KnownIntersectionState blk = KnownIntersectionState - { theirFrag :: !(AnchoredFragment (Header blk)) + { theirFrag :: !(AnchoredFragment (Header blk)) -- ^ The candidate, the synched fragment of their chain. - , theirChainState :: !(ChainState (BlockProtocol blk)) - -- ^ 'ChainState' corresponding to the tip (most recent block) of + , theirHeaderState :: !(HeaderState blk) + -- ^ 'HeaderState' corresponding to the tip (most recent block) of -- 'theirFrag'. - , ourFrag :: !(AnchoredFragment (Header blk)) + , ourFrag :: !(AnchoredFragment (Header blk)) -- ^ A view of the current chain fragment used to maintain the invariants -- with. Note that this might be temporarily out of date w.r.t. the actual -- current chain until we update it again. @@ -218,7 +219,7 @@ data KnownIntersectionState blk = KnownIntersectionState -- this follows that both fragments intersect. This also means that -- 'theirFrag' forks off within the last @k@ headers/blocks of the -- 'ourFrag'. - , ourTip :: !(Our (Tip blk)) + , ourTip :: !(Our (Tip blk)) -- ^ INVARIANT: must correspond to the tip of 'ourFrag'. } deriving (Generic) @@ -272,9 +273,9 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime -- ^ Exception to throw when no intersection is found. -> Stateful m blk () (ClientPipelinedStIdle Z) findIntersection mkEx = Stateful $ \() -> do - (ourFrag, ourChainState, ourTip) <- atomically $ (,,) + (ourFrag, ourHeaderState, ourTip) <- atomically $ (,,) <$> getCurrentChain - <*> (ouroborosChainState <$> getCurrentLedger) + <*> (headerState <$> getCurrentLedger) <*> (Our <$> getOurTip) -- We select points from the last @k@ headers of our current chain. This -- means that if an intersection is found for one of these points, it @@ -285,9 +286,9 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime (map fromIntegral (offsets maxOffset)) ourFrag uis = UnknownIntersectionState - { ourFrag = ourFrag - , ourChainState = ourChainState - , ourTip = ourTip + { ourFrag = ourFrag + , ourHeaderState = ourHeaderState + , ourTip = ourTip } return $ SendMsgFindIntersect points $ ClientPipelinedStIntersect { recvMsgIntersectFound = \i theirTip' -> @@ -307,7 +308,7 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime intersectFound intersection theirTip = Stateful $ \UnknownIntersectionState { ourFrag - , ourChainState + , ourHeaderState , ourTip = ourTip } -> do traceWith tracer $ TraceFoundIntersection intersection ourTip theirTip @@ -319,7 +320,7 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime -- to fork", which means that a roll back is always followed by -- applying at least as many blocks that we rolled back. -- - -- This is important for 'rewindChainState', which can only roll back + -- This is important for 'rewindHeaderState', which can only roll back -- up to @k@ blocks, /once/, i.e., we cannot keep rolling back the -- same chain state multiple times, because that would mean that we -- store the chain state for the /whole chain/, all the way to @@ -329,10 +330,8 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime -- it is followed by rolling forward again), but we need some -- guarantees that the ChainSync protocol /does/ in fact give us a -- switch-to-fork instead of a true rollback. - (theirFrag, theirChainState) <- do - let i = castPoint intersection - case (,) <$> AF.rollback i ourFrag - <*> rewindChainState cfg ourChainState i of + (theirFrag, theirHeaderState) <- do + case attemptRollback cfg intersection (ourFrag, ourHeaderState) of Just (c, d) -> return (c, d) -- The @intersection@ is not on the candidate chain, even though -- we sent only points from the candidate chain to find an @@ -345,10 +344,10 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime } atomically $ writeTVar varCandidate theirFrag let kis = KnownIntersectionState - { theirFrag = theirFrag - , theirChainState = theirChainState - , ourFrag = ourFrag - , ourTip = ourTip + { theirFrag = theirFrag + , theirHeaderState = theirHeaderState + , ourFrag = ourFrag + , ourTip = ourTip } continueWithState kis $ nextStep mkPipelineDecision0 Zero theirTip @@ -515,7 +514,7 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime (ClientPipelinedStIdle n) rollForward mkPipelineDecision n hdr theirTip = Stateful $ \kis@KnownIntersectionState - { theirChainState + { theirHeaderState , theirFrag , ourTip } -> traceException $ do @@ -551,24 +550,20 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime , _theirTip = theirTip } - theirChainState' <- - case runExcept $ applyChainState - cfg - ledgerView - (validateView cfg hdr) - theirChainState of - Right theirChainState' -> return theirChainState' - Left vErr -> disconnect ChainError - { _newPoint = hdrPoint - , _chainValidationErr = vErr - , _ourTip = ourTip - , _theirTip = theirTip + theirHeaderState' <- + case runExcept $ validateHeader cfg ledgerView hdr theirHeaderState of + Right theirHeaderState' -> return theirHeaderState' + Left vErr -> disconnect HeaderError + { _newPoint = hdrPoint + , _headerErr = vErr + , _ourTip = ourTip + , _theirTip = theirTip } let theirFrag' = theirFrag :> hdr kis' = kis - { theirFrag = theirFrag' - , theirChainState = theirChainState' + { theirFrag = theirFrag' + , theirHeaderState = theirHeaderState' } atomically $ writeTVar varCandidate theirFrag' @@ -621,13 +616,11 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime theirTip = Stateful $ \kis@KnownIntersectionState { theirFrag - , theirChainState + , theirHeaderState , ourTip } -> traceException $ do - (theirFrag', theirChainState') <- do - let i = castPoint intersection - case (,) <$> AF.rollback i theirFrag - <*> rewindChainState cfg theirChainState i of + (theirFrag', theirHeaderState') <- do + case attemptRollback cfg intersection (theirFrag, theirHeaderState) of Just (c, d) -> return (c,d) -- Remember that we use our current chain fragment as the starting -- point for the candidate's chain. Our fragment contained @k@ @@ -658,8 +651,8 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime } let kis' = kis - { theirFrag = theirFrag' - , theirChainState = theirChainState' + { theirFrag = theirFrag' + , theirHeaderState = theirHeaderState' } atomically $ writeTVar varCandidate theirFrag' @@ -705,6 +698,18 @@ chainSyncClient mkPipelineDecision0 tracer cfg btime k :: Word64 k = maxRollbacks $ protocolSecurityParam cfg +attemptRollback :: ( SupportedBlock blk + , Serialise (HeaderHash blk) + ) + => NodeConfig (BlockProtocol blk) + -> Point blk + -> (AnchoredFragment (Header blk), HeaderState blk) + -> Maybe (AnchoredFragment (Header blk), HeaderState blk) +attemptRollback cfg intersection (frag, state) = do + frag' <- AF.rollback (castPoint intersection) frag + state' <- rewindHeaderState cfg intersection state + return (frag', state') + -- | Watch the invalid block checker function for changes (using its -- fingerprint). Whenever it changes, i.e., a new invalid block is detected, -- check whether the current candidate fragment contains any header that is @@ -799,13 +804,13 @@ data ChainSyncClientException = , _theirTip :: Their (Tip blk) } - -- | The chain validation threw an error. + -- | Header validation threw an error. | forall blk. SupportedBlock blk => - ChainError - { _newPoint :: Point blk - , _chainValidationErr :: ValidationErr (BlockProtocol blk) - , _ourTip :: Our (Tip blk) - , _theirTip :: Their (Tip blk) + HeaderError + { _newPoint :: Point blk + , _headerErr :: HeaderError blk + , _ourTip :: Our (Tip blk) + , _theirTip :: Their (Tip blk) } -- | The upstream node rolled forward to a point too far in our past. @@ -882,11 +887,11 @@ instance Eq ChainSyncClientException where Just Refl -> (a, b, c) == (a', b', c') ForkTooDeep{} == _ = False - ChainError (a :: Point blk) b c d == ChainError (a' :: Point blk') b' c' d' = + HeaderError (a :: Point blk) b c d == HeaderError (a' :: Point blk') b' c' d' = case eqT @blk @blk' of Nothing -> False Just Refl -> (a, b, c, d) == (a', b', c', d') - ChainError{} == _ = False + HeaderError{} == _ = False InvalidRollForward (a :: Point blk) b c == InvalidRollForward (a' :: Point blk') b' c' = case eqT @blk @blk' of diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/HeaderValidation.hs b/ouroboros-consensus/src/Ouroboros/Consensus/HeaderValidation.hs new file mode 100644 index 00000000000..0cc8fbe6ba9 --- /dev/null +++ b/ouroboros-consensus/src/Ouroboros/Consensus/HeaderValidation.hs @@ -0,0 +1,430 @@ +{-# LANGUAGE DefaultSignatures #-} +{-# LANGUAGE DeriveAnyClass #-} +{-# LANGUAGE DeriveGeneric #-} +{-# LANGUAGE FlexibleContexts #-} +{-# LANGUAGE LambdaCase #-} +{-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE RankNTypes #-} +{-# LANGUAGE RecordWildCards #-} +{-# LANGUAGE ScopedTypeVariables #-} +{-# LANGUAGE StandaloneDeriving #-} +{-# LANGUAGE TypeApplications #-} +{-# LANGUAGE TypeFamilies #-} +{-# LANGUAGE UndecidableInstances #-} + +-- | Header validation +module Ouroboros.Consensus.HeaderValidation ( + validateHeader + -- * Annotated tips + , AnnTip(..) + , annTipPoint + , castAnnTip + , HasAnnTip(..) + , getAnnTip + -- * Header state + , HeaderState(..) + , headerStateTip + , headerStatePush + , genesisHeaderState + , castHeaderState + , rewindHeaderState + -- * Validate header envelope + , HeaderEnvelopeError(..) + , castHeaderEnvelopeError + , ValidateEnvelope(..) + -- * Errors + , HeaderError(..) + , castHeaderError + -- * Serialization + , encodeAnnTip + , decodeAnnTip + , encodeHeaderState + , decodeHeaderState + ) where + +import Codec.CBOR.Decoding (Decoder) +import Codec.CBOR.Encoding (Encoding, encodeListLen) +import Codec.Serialise (Serialise, decode, encode) +import Control.Monad.Except +import Data.Foldable (toList) +import Data.Proxy +import Data.Sequence.Strict (StrictSeq ((:<|), (:|>), Empty)) +import qualified Data.Sequence.Strict as Seq +import GHC.Generics (Generic) + +import Cardano.Binary (enforceSize) +import Cardano.Prelude (NoUnexpectedThunks) +import Cardano.Slotting.Slot (WithOrigin (..)) + +import Ouroboros.Network.Block hiding (Tip (..)) + +import Ouroboros.Consensus.Block +import Ouroboros.Consensus.Protocol.Abstract +import qualified Ouroboros.Consensus.Util.CBOR as Util.CBOR + +{------------------------------------------------------------------------------- + Preliminary: annotated tip +-------------------------------------------------------------------------------} + +-- | Annotated information about the tip of the chain +-- +-- The annotation is the additional information we need to validate the +-- header envelope. Under normal circumstances no additional information is +-- required, but for instance for Byron we need to know if the previous header +-- was an EBB. +data AnnTip blk = AnnTip { + annTipSlotNo :: !SlotNo + , annTipHash :: !(HeaderHash blk) + , annTipBlockNo :: !BlockNo + , annTipInfo :: !(TipInfo blk) + } + deriving (Generic) + +deriving instance HasAnnTip blk => Show (AnnTip blk) +deriving instance HasAnnTip blk => Eq (AnnTip blk) +deriving instance HasAnnTip blk => NoUnexpectedThunks (AnnTip blk) + +annTipPoint :: AnnTip blk -> Point blk +annTipPoint AnnTip{..} = BlockPoint annTipSlotNo annTipHash + +castAnnTip :: ( HeaderHash blk ~ HeaderHash blk' + , TipInfo blk ~ TipInfo blk' + ) + => AnnTip blk -> AnnTip blk' +castAnnTip AnnTip{..} = AnnTip{..} + +class ( StandardHash blk + , Show (TipInfo blk) + , Eq (TipInfo blk) + , NoUnexpectedThunks (TipInfo blk) + ) => HasAnnTip blk where + type TipInfo blk :: * + type TipInfo blk = () + + getTipInfo :: Header blk -> TipInfo blk + + default getTipInfo :: TipInfo blk ~ () => Header blk -> TipInfo blk + getTipInfo _ = () + +getAnnTip :: (HasHeader (Header blk), HasAnnTip blk) + => Header blk -> AnnTip blk +getAnnTip hdr = AnnTip { + annTipSlotNo = blockSlot hdr + , annTipHash = blockHash hdr + , annTipBlockNo = blockNo hdr + , annTipInfo = getTipInfo hdr + } + +{------------------------------------------------------------------------------- + State +-------------------------------------------------------------------------------} + +-- | State required to validate the header +-- +-- See 'validateHeader' for details +data HeaderState blk = HeaderState { + -- | Protocol-specific chain state + headerStateChain :: !(ChainState (BlockProtocol blk)) + + -- | The most recent @k@ tips + , headerStateTips :: !(StrictSeq (AnnTip blk)) + + -- | Tip /before/ 'headerStateTips' + , headerStateAnchor :: !(WithOrigin (AnnTip blk)) + } + deriving (Generic) + +headerStateTip :: HeaderState blk -> WithOrigin (AnnTip blk) +headerStateTip HeaderState{..} = + case headerStateTips of + Empty -> headerStateAnchor + _ :|> tip -> At tip + +headerStatePush :: forall blk. + SecurityParam + -> ChainState (BlockProtocol blk) + -> AnnTip blk + -> HeaderState blk + -> HeaderState blk +headerStatePush (SecurityParam k) chainState newTip HeaderState{..} = + case trim pushed of + Nothing -> HeaderState chainState pushed headerStateAnchor + Just (newAnchor, trimmed) -> HeaderState chainState trimmed (At newAnchor) + where + pushed :: StrictSeq (AnnTip blk) + pushed = headerStateTips :|> newTip + + trim :: StrictSeq (AnnTip blk) -> Maybe (AnnTip blk, StrictSeq (AnnTip blk)) + trim (newAnchor :<| trimmed) | Seq.length trimmed >= fromIntegral k = + Just (newAnchor, trimmed) + trim _otherwise = Nothing + +deriving instance (SupportedBlock blk, HasAnnTip blk) => Show (HeaderState blk) +deriving instance (SupportedBlock blk, HasAnnTip blk) => NoUnexpectedThunks (HeaderState blk) +deriving instance ( SupportedBlock blk + , HasAnnTip blk + , Eq (ChainState (BlockProtocol blk)) + ) => Eq (HeaderState blk) + +genesisHeaderState :: ChainState (BlockProtocol blk) -> HeaderState blk +genesisHeaderState state = HeaderState state Seq.Empty Origin + +castHeaderState :: ( ChainState (BlockProtocol blk ) + ~ ChainState (BlockProtocol blk') + , HeaderHash blk + ~ HeaderHash blk' + , TipInfo blk + ~ TipInfo blk' + ) + => HeaderState blk -> HeaderState blk' +castHeaderState HeaderState{..} = HeaderState{ + headerStateChain = headerStateChain + , headerStateTips = castSeq castAnnTip $ headerStateTips + , headerStateAnchor = fmap castAnnTip $ headerStateAnchor + } + where + -- This is unfortunate. We're doing busy-work on a strict-sequence, + -- mapping a function that actually doesn't change anything :/ + castSeq :: (a -> b) -> StrictSeq a -> StrictSeq b + castSeq f = Seq.fromList . map f . toList + +rewindHeaderState :: forall blk. + ( SupportedBlock blk + , Serialise (HeaderHash blk) + ) + => NodeConfig (BlockProtocol blk) + -> Point blk + -> HeaderState blk -> Maybe (HeaderState blk) +rewindHeaderState cfg p HeaderState{..} = do + chainState' <- rewindChainState cfg headerStateChain p + return $ HeaderState { + headerStateChain = chainState' + , headerStateTips = Seq.dropWhileR rolledBack headerStateTips + , headerStateAnchor = headerStateAnchor + } + where + rolledBack :: AnnTip blk -> Bool + rolledBack t = annTipPoint t > p + +{------------------------------------------------------------------------------- + Validate header envelope +-------------------------------------------------------------------------------} + +data HeaderEnvelopeError blk = + -- | Invalid block number + -- + -- We record both the expected and actual block number + UnexpectedBlockNo !BlockNo !BlockNo + + -- | Invalid slot number + -- + -- We record both the expected (minimum) and actual slot number + | UnexpectedSlotNo !SlotNo !SlotNo + + -- | Invalid hash (in the reference to the previous block) + -- + -- We record both the expected and actual hash + | UnexpectedPrevHash !(ChainHash blk) !(ChainHash blk) + deriving (Generic) + +deriving instance SupportedBlock blk => Eq (HeaderEnvelopeError blk) +deriving instance SupportedBlock blk => Show (HeaderEnvelopeError blk) +deriving instance SupportedBlock blk => NoUnexpectedThunks (HeaderEnvelopeError blk) + +castHeaderEnvelopeError :: HeaderHash blk ~ HeaderHash blk' + => HeaderEnvelopeError blk -> HeaderEnvelopeError blk' +castHeaderEnvelopeError = \case + UnexpectedBlockNo expected actual -> UnexpectedBlockNo expected actual + UnexpectedSlotNo expected actual -> UnexpectedSlotNo expected actual + UnexpectedPrevHash expected actual -> UnexpectedPrevHash expected' actual' + where + expected' = castHash expected + actual' = castHash actual + +class HasAnnTip blk => ValidateEnvelope blk where + -- | Validate the header envelope + validateEnvelope :: NodeConfig (BlockProtocol blk) + -> WithOrigin (AnnTip blk) + -> Header blk + -> Except (HeaderEnvelopeError blk) () + + -- | The block number of the first block on the chain + firstBlockNo :: Proxy blk -> BlockNo + firstBlockNo _ = BlockNo 0 + + -- | The smallest possible 'SlotNo' + -- + -- The /actual/ 'SlotNo' of the first block on the chain may well be larger + -- than this value of course (if some slots were empty) + minimumPossibleSlotNo :: Proxy blk -> SlotNo + minimumPossibleSlotNo _ = SlotNo 0 + + default validateEnvelope :: HasHeader (Header blk) + => NodeConfig (BlockProtocol blk) + -> WithOrigin (AnnTip blk) + -> Header blk + -> Except (HeaderEnvelopeError blk) () + validateEnvelope _cfg oldTip hdr = do + when (actualBlockNo /= expectedBlockNo) $ + throwError $ UnexpectedBlockNo expectedBlockNo actualBlockNo + when (actualSlotNo < expectedSlotNo) $ + throwError $ UnexpectedSlotNo expectedSlotNo actualSlotNo + when (actualPrevHash /= expectedPrevHash) $ + throwError $ UnexpectedPrevHash expectedPrevHash actualPrevHash + where + actualSlotNo :: SlotNo + actualBlockNo :: BlockNo + actualPrevHash :: ChainHash blk + + actualSlotNo = blockSlot hdr + actualBlockNo = blockNo hdr + actualPrevHash = castHash $ blockPrevHash hdr + + expectedSlotNo :: SlotNo -- Lower bound only + expectedBlockNo :: BlockNo + expectedPrevHash :: ChainHash blk + + (expectedSlotNo, expectedBlockNo, expectedPrevHash) = + case oldTip of + At (AnnTip s h b _) -> ( succ s, succ b, BlockHash h ) + Origin -> ( minimumPossibleSlotNo proxy + , firstBlockNo proxy + , GenesisHash + ) + where + proxy = Proxy @blk + +{------------------------------------------------------------------------------- + Errors +-------------------------------------------------------------------------------} + +-- | Invalid header +data HeaderError blk = + -- | Invalid consensus protocol fields + HeaderProtocolError (ValidationErr (BlockProtocol blk)) + + -- | Failed to validate the envelope + | HeaderEnvelopeError (HeaderEnvelopeError blk) + deriving (Generic) + +deriving instance SupportedBlock blk => Eq (HeaderError blk) +deriving instance SupportedBlock blk => Show (HeaderError blk) +deriving instance SupportedBlock blk => NoUnexpectedThunks (HeaderError blk) + +castHeaderError :: ( ValidationErr (BlockProtocol blk ) + ~ ValidationErr (BlockProtocol blk') + , HeaderHash blk + ~ HeaderHash blk' + ) + => HeaderError blk -> HeaderError blk' +castHeaderError (HeaderProtocolError e) = HeaderProtocolError e +castHeaderError (HeaderEnvelopeError e) = HeaderEnvelopeError $ + castHeaderEnvelopeError e + +{------------------------------------------------------------------------------- + Validation proper +-------------------------------------------------------------------------------} + +-- | Header validation +-- +-- Header validation (as opposed to block validation) is done by the chain sync +-- client: as we download headers from other network nodes, we validate those +-- headers before deciding whether or not to download the corresponding blocks. +-- +-- Before we /adopt/ any blocks we have downloaded, however, we will do a full +-- block validation. As such, the header validation check can omit some checks +-- (provided that we do those checks when we do the full validation); at worst, +-- this would mean we might download some blocks that we will reject as being +-- invalid where we could have detected that sooner. +-- +-- For this reason, the header validation currently only checks two things: +-- +-- o It verifies the consensus part of the header. +-- +-- For example, for Praos this means checking the VRF proofs. +-- +-- o It verifies the 'HasHeader' part of the header. +-- +-- By default, we verify that +-- +-- x Block numbers are consecutive +-- x The block number of the first block is 'firstBlockNo' +-- x Slot numbers are strictly increasing +-- x The slot number of the first block is at least 'minimumPossibleSlotNo' +-- x Hashes line up +-- +-- /If/ a particular ledger wants to verify additional fields in the header, +-- it will get the chance to do so in 'applyLedgerBlock', which is passed the +-- entire block (not just the block body). +validateHeader :: (SupportedBlock blk, ValidateEnvelope blk) + => NodeConfig (BlockProtocol blk) + -> LedgerView (BlockProtocol blk) + -> Header blk + -> HeaderState blk + -> Except (HeaderError blk) (HeaderState blk) +validateHeader cfg ledgerView hdr st = do + withExcept HeaderEnvelopeError $ + validateEnvelope cfg (headerStateTip st) hdr + chainState' <- withExcept HeaderProtocolError $ + applyChainState + cfg + ledgerView + (validateView cfg hdr) + (headerStateChain st) + return $ headerStatePush + (protocolSecurityParam cfg) + chainState' + (getAnnTip hdr) + st + +{------------------------------------------------------------------------------- + Serialisation +-------------------------------------------------------------------------------} + +encodeAnnTip :: (HeaderHash blk -> Encoding) + -> (TipInfo blk -> Encoding) + -> (AnnTip blk -> Encoding) +encodeAnnTip encodeHash encodeInfo AnnTip{..} = mconcat [ + encodeListLen 4 + , encode annTipSlotNo + , encodeHash annTipHash + , encode annTipBlockNo + , encodeInfo annTipInfo + ] + +decodeAnnTip :: (forall s. Decoder s (HeaderHash blk)) + -> (forall s. Decoder s (TipInfo blk)) + -> (forall s. Decoder s (AnnTip blk)) +decodeAnnTip decodeHash decodeInfo = do + enforceSize "AnnTip" 4 + annTipSlotNo <- decode + annTipHash <- decodeHash + annTipBlockNo <- decode + annTipInfo <- decodeInfo + return AnnTip{..} + +encodeHeaderState :: (ChainState (BlockProtocol blk) -> Encoding) + -> (HeaderHash blk -> Encoding) + -> (TipInfo blk -> Encoding) + -> (HeaderState blk -> Encoding) +encodeHeaderState encodeChainState encodeHash encodeInfo HeaderState{..} = mconcat [ + encodeListLen 3 + , encodeChainState headerStateChain + , Util.CBOR.encodeSeq encodeAnnTip' headerStateTips + , Util.CBOR.encodeWithOrigin encodeAnnTip' headerStateAnchor + ] + where + encodeAnnTip' = encodeAnnTip encodeHash encodeInfo + +decodeHeaderState :: (forall s. Decoder s (ChainState (BlockProtocol blk))) + -> (forall s. Decoder s (HeaderHash blk)) + -> (forall s. Decoder s (TipInfo blk)) + -> (forall s. Decoder s (HeaderState blk)) +decodeHeaderState decodeChainState decodeHash decodeInfo = do + enforceSize "HeaderState" 3 + headerStateChain <- decodeChainState + headerStateTips <- Util.CBOR.decodeSeq decodeAnnTip' + headerStateAnchor <- Util.CBOR.decodeWithOrigin decodeAnnTip' + return HeaderState{..} + where + decodeAnnTip' = decodeAnnTip decodeHash decodeInfo diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Abstract.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Abstract.hs index de673d7b1a0..f93fdce442f 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Abstract.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Abstract.hs @@ -36,6 +36,7 @@ import Ouroboros.Network.Protocol.LocalStateQuery.Type (ShowQuery (..)) import Ouroboros.Consensus.Block +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Protocol.Abstract {------------------------------------------------------------------------------- @@ -141,7 +142,10 @@ deriving instance NoUnexpectedThunks (LedgerState blk) => NoUnexpectedThunks (TickedLedgerState blk) -- | Link protocol to ledger -class (SupportedBlock blk, UpdateLedger blk) => ProtocolLedgerView blk where +class ( SupportedBlock blk + , UpdateLedger blk + , ValidateEnvelope blk + ) => ProtocolLedgerView blk where -- | Extract the ledger environment from the node config ledgerConfigView :: NodeConfig (BlockProtocol blk) -> LedgerConfig blk diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Byron/Block.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Byron/Block.hs index 949239d358d..534e9fceb58 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Byron/Block.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Byron/Block.hs @@ -38,6 +38,8 @@ module Ouroboros.Consensus.Ledger.Byron.Block ( , byronAddHeaderEnvelope ) where +import Control.Arrow ((&&&)) +import Control.Monad.Except import Data.Binary (Get, Put) import qualified Data.Binary.Get as Get import qualified Data.Binary.Put as Put @@ -59,6 +61,7 @@ import Codec.Serialise (Serialise (..)) import Cardano.Binary import Cardano.Prelude (NoUnexpectedThunks (..)) +import Cardano.Slotting.Slot (WithOrigin (..), withOrigin) import qualified Crypto.Hash as Crypto @@ -73,6 +76,7 @@ import qualified Cardano.Crypto.Hashing as CC import Ouroboros.Network.Block import Ouroboros.Consensus.Block +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Byron.Conversions import Ouroboros.Consensus.Ledger.Byron.Orphans () import Ouroboros.Consensus.Util.Condense @@ -250,6 +254,56 @@ byronHeaderIsEBB = go . byronHeaderRaw byronBlockIsEBB :: ByronBlock -> IsEBB byronBlockIsEBB = byronHeaderIsEBB . getHeader +{------------------------------------------------------------------------------- + Envelope +-------------------------------------------------------------------------------} + +instance HasAnnTip ByronBlock where + type TipInfo ByronBlock = IsEBB + getTipInfo = byronHeaderIsEBB + +instance ValidateEnvelope ByronBlock where + validateEnvelope _cfg oldTip hdr = do + when (actualBlockNo /= expectedBlockNo) $ + throwError $ UnexpectedBlockNo expectedBlockNo actualBlockNo + when (actualSlotNo < expectedSlotNo) $ + throwError $ UnexpectedSlotNo expectedSlotNo actualSlotNo + when (actualPrevHash /= expectedPrevHash) $ + throwError $ UnexpectedPrevHash expectedPrevHash actualPrevHash + where + newIsEBB :: IsEBB + newIsEBB = byronHeaderIsEBB hdr + + actualSlotNo :: SlotNo + actualBlockNo :: BlockNo + actualPrevHash :: ChainHash ByronBlock + + actualSlotNo = blockSlot hdr + actualBlockNo = blockNo hdr + actualPrevHash = castHash $ blockPrevHash hdr + + expectedSlotNo :: SlotNo -- Lower bound only + expectedBlockNo :: BlockNo + expectedPrevHash :: ChainHash ByronBlock + + (expectedSlotNo, expectedBlockNo, expectedPrevHash) = ( + nextSlotNo ((annTipInfo &&& annTipSlotNo) <$> oldTip) newIsEBB + , nextBlockNo ((annTipInfo &&& annTipBlockNo) <$> oldTip) newIsEBB + , withOrigin GenesisHash (BlockHash . annTipHash) oldTip + ) + + -- EBB shares its slot number with its successor + nextSlotNo :: WithOrigin (IsEBB, SlotNo) -> IsEBB -> SlotNo + nextSlotNo Origin _ = SlotNo 0 + nextSlotNo (At (IsEBB, s)) IsNotEBB = s + nextSlotNo (At (_ , s)) _ = succ s + + -- EBB shares its block number with its predecessor + nextBlockNo :: WithOrigin (IsEBB, BlockNo) -> IsEBB -> BlockNo + nextBlockNo Origin _ = BlockNo 0 + nextBlockNo (At (IsNotEBB, b)) IsEBB = b + nextBlockNo (At (_ , b)) _ = succ b + {------------------------------------------------------------------------------- Serialisation -------------------------------------------------------------------------------} diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual.hs index 26ee9c783fd..26f866ffa6f 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual.hs @@ -66,6 +66,7 @@ import Ouroboros.Network.Block import Ouroboros.Storage.Common import Ouroboros.Consensus.Block +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Mempool.API @@ -328,16 +329,16 @@ dualExtLedgerStateMain :: Bridge m a => ExtLedgerState (DualBlock m a) -> ExtLedgerState m dualExtLedgerStateMain ExtLedgerState{..} = ExtLedgerState{ - ledgerState = dualLedgerStateMain ledgerState - , ouroborosChainState = ouroborosChainState + ledgerState = dualLedgerStateMain ledgerState + , headerState = castHeaderState headerState } dualExtValidationErrorMain :: Bridge m a => ExtValidationError (DualBlock m a) -> ExtValidationError m dualExtValidationErrorMain = \case - ExtValidationErrorLedger e -> ExtValidationErrorLedger (dualLedgerErrorMain e) - ExtValidationErrorOuroboros e -> ExtValidationErrorOuroboros e + ExtValidationErrorLedger e -> ExtValidationErrorLedger (dualLedgerErrorMain e) + ExtValidationErrorHeader e -> ExtValidationErrorHeader (castHeaderError e) {------------------------------------------------------------------------------- ProtocolLedgerView @@ -346,6 +347,19 @@ dualExtValidationErrorMain = \case in the consensus protocol, and has no 'ProtocolLedgerView' instance. -------------------------------------------------------------------------------} +instance Bridge m a => HasAnnTip (DualBlock m a) where + type TipInfo (DualBlock m a) = TipInfo m + getTipInfo = getTipInfo . dualHeaderMain + +instance Bridge m a => ValidateEnvelope (DualBlock m a) where + validateEnvelope cfg t = + withExcept castHeaderEnvelopeError + . validateEnvelope (extNodeConfigP cfg) (castAnnTip <$> t) + . dualHeaderMain + + firstBlockNo _ = firstBlockNo (Proxy @m) + minimumPossibleSlotNo _ = minimumPossibleSlotNo (Proxy @m) + instance Bridge m a => ProtocolLedgerView (DualBlock m a) where ledgerConfigView cfg = DualLedgerConfig { dualLedgerConfigMain = ledgerConfigView $ extNodeConfigP cfg diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual/Byron.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual/Byron.hs index 9854d860760..cd46791e818 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual/Byron.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Dual/Byron.hs @@ -57,6 +57,7 @@ import qualified Cardano.Chain.UTxO as Impl import Ouroboros.Network.Block import Ouroboros.Consensus.Crypto.DSIGN.Cardano +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Byron import Ouroboros.Consensus.Ledger.ByronSpec import qualified Ouroboros.Consensus.Ledger.ByronSpec.Genesis as Genesis @@ -295,7 +296,7 @@ protocolInfoDualByron abstractGenesis@ByronSpecGenesis{..} params mLeader = , dualLedgerStateAux = initAbstractState , dualLedgerStateBridge = initBridge } - , ouroborosChainState = CS.empty + , headerState = genesisHeaderState CS.empty } } where diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Extended.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Extended.hs index e8b5af9d97d..f8e42db798f 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Extended.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Extended.hs @@ -29,7 +29,10 @@ import GHC.Stack import Cardano.Prelude (NoUnexpectedThunks (..)) +import Ouroboros.Network.Block (HeaderHash) + import Ouroboros.Consensus.Block +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Protocol.Abstract import Ouroboros.Consensus.Util (repeatedlyM) @@ -40,16 +43,16 @@ import Ouroboros.Consensus.Util (repeatedlyM) -- | Extended ledger state -- --- This is the combination of the ouroboros state and the ledger state proper. +-- This is the combination of the header state and the ledger state proper. data ExtLedgerState blk = ExtLedgerState { - ledgerState :: !(LedgerState blk) - , ouroborosChainState :: !(ChainState (BlockProtocol blk)) + ledgerState :: !(LedgerState blk) + , headerState :: !(HeaderState blk) } deriving (Generic) data ExtValidationError blk = ExtValidationErrorLedger !(LedgerError blk) - | ExtValidationErrorOuroboros !(ValidationErr (BlockProtocol blk)) + | ExtValidationErrorHeader !(HeaderError blk) deriving (Generic) instance ProtocolLedgerView blk => NoUnexpectedThunks (ExtValidationError blk) @@ -100,25 +103,26 @@ applyExtLedgerState :: (ProtocolLedgerView blk, HasCallStack) -> ExtLedgerState blk -> Except (ExtValidationError blk) (ExtLedgerState blk) applyExtLedgerState prevApplied cfg blk ExtLedgerState{..} = do - ledgerState' <- case prevApplied of - BlockNotPreviouslyApplied -> - withExcept ExtValidationErrorLedger $ - applyLedgerBlock - (ledgerConfigView cfg) - blk - ledgerState - BlockPreviouslyApplied -> pure $ - reapplyLedgerBlock - (ledgerConfigView cfg) - blk - ledgerState - ouroborosChainState' <- withExcept ExtValidationErrorOuroboros $ - applyChainState - cfg - (protocolLedgerView cfg ledgerState') - (validateView cfg (getHeader blk)) - ouroborosChainState - return $ ExtLedgerState ledgerState' ouroborosChainState' + -- TODO: Flip order, and use anachronistic ledger view + ledgerState' <- case prevApplied of + BlockNotPreviouslyApplied -> + withExcept ExtValidationErrorLedger $ + applyLedgerBlock + (ledgerConfigView cfg) + blk + ledgerState + BlockPreviouslyApplied -> pure $ + reapplyLedgerBlock + (ledgerConfigView cfg) + blk + ledgerState + headerState' <- withExcept ExtValidationErrorHeader $ + validateHeader + cfg + (protocolLedgerView cfg ledgerState') + (getHeader blk) + headerState + return $ ExtLedgerState ledgerState' headerState' foldExtLedgerState :: (ProtocolLedgerView blk, HasCallStack) => BlockPreviouslyApplied @@ -132,20 +136,39 @@ foldExtLedgerState prevApplied = repeatedlyM . (applyExtLedgerState prevApplied) Serialisation -------------------------------------------------------------------------------} -encodeExtLedgerState :: (LedgerState blk -> Encoding) +encodeExtLedgerState :: (LedgerState blk -> Encoding) -> (ChainState (BlockProtocol blk) -> Encoding) + -> (HeaderHash blk -> Encoding) + -> (TipInfo blk -> Encoding) -> ExtLedgerState blk -> Encoding -encodeExtLedgerState encodeLedger +encodeExtLedgerState encodeLedgerState encodeChainState + encodeHash + encodeInfo ExtLedgerState{..} = mconcat [ - encodeLedger ledgerState - , encodeChainState ouroborosChainState + encodeLedgerState ledgerState + , encodeHeaderState' headerState ] + where + encodeHeaderState' = encodeHeaderState + encodeChainState + encodeHash + encodeInfo -decodeExtLedgerState :: (forall s. Decoder s (LedgerState blk)) +decodeExtLedgerState :: (forall s. Decoder s (LedgerState blk)) -> (forall s. Decoder s (ChainState (BlockProtocol blk))) - -> forall s. Decoder s (ExtLedgerState blk) -decodeExtLedgerState decodeLedger decodeChainState = do - ledgerState <- decodeLedger - ouroborosChainState <- decodeChainState + -> (forall s. Decoder s (HeaderHash blk)) + -> (forall s. Decoder s (TipInfo blk)) + -> (forall s. Decoder s (ExtLedgerState blk)) +decodeExtLedgerState decodeLedgerState + decodeChainState + decodeHash + decodeInfo = do + ledgerState <- decodeLedgerState + headerState <- decodeHeaderState' return ExtLedgerState{..} + where + decodeHeaderState' = decodeHeaderState + decodeChainState + decodeHash + decodeInfo diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Mock/Block.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Mock/Block.hs index f7381c36aaa..94fad5efb6c 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Mock/Block.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Ledger/Mock/Block.hs @@ -73,6 +73,7 @@ import Cardano.Prelude (NoUnexpectedThunks (..)) import Ouroboros.Network.Block import Ouroboros.Consensus.Block +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Mock.Address import Ouroboros.Consensus.Ledger.Mock.State @@ -223,6 +224,16 @@ instance Mock.HasUtxo SimpleBody where confirmed = Mock.confirmed . simpleTxs updateUtxo = Mock.updateUtxo . simpleTxs +{------------------------------------------------------------------------------- + Envelope validation +-------------------------------------------------------------------------------} + +instance (SimpleCrypto c, Typeable ext) => HasAnnTip (SimpleBlock c ext) + -- Use defaults + +instance (SimpleCrypto c, Typeable ext) => ValidateEnvelope (SimpleBlock c ext) + -- Use defaults + {------------------------------------------------------------------------------- Update the ledger -------------------------------------------------------------------------------} diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Mempool/Impl.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Mempool/Impl.hs index 76e3508bc09..3f2671771ad 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Mempool/Impl.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Mempool/Impl.hs @@ -4,6 +4,7 @@ {-# LANGUAGE NamedFieldPuns #-} {-# LANGUAGE ScopedTypeVariables #-} {-# LANGUAGE StandaloneDeriving #-} +{-# LANGUAGE TypeApplications #-} {-# LANGUAGE UndecidableInstances #-} module Ouroboros.Consensus.Mempool.Impl ( @@ -28,12 +29,12 @@ import Control.Tracer import Ouroboros.Network.Block (ChainHash, Point, SlotNo, StandardHash) -import qualified Ouroboros.Network.Block as Block import Ouroboros.Network.Point (WithOrigin (..)) import Ouroboros.Storage.ChainDB (ChainDB) import qualified Ouroboros.Storage.ChainDB.API as ChainDB +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Mempool.API @@ -49,7 +50,7 @@ import Ouroboros.Consensus.Util.STM (onEachChange) Top-level API -------------------------------------------------------------------------------} -openMempool :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk)) +openMempool :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk), ValidateEnvelope blk) => ResourceRegistry m -> LedgerInterface m blk -> LedgerConfig blk @@ -66,7 +67,7 @@ openMempool registry ledger cfg capacity tracer = do -- -- Intended for testing purposes. openMempoolWithoutSyncThread - :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk)) + :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk), ValidateEnvelope blk) => LedgerInterface m blk -> LedgerConfig blk -> MempoolCapacityBytes @@ -75,7 +76,7 @@ openMempoolWithoutSyncThread openMempoolWithoutSyncThread ledger cfg capacity tracer = mkMempool <$> initMempoolEnv ledger cfg capacity tracer -mkMempool :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk)) +mkMempool :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk), ValidateEnvelope blk) => MempoolEnv m blk -> Mempool m blk TicketNo mkMempool env = Mempool { tryAddTxs = implTryAddTxs env @@ -164,7 +165,7 @@ initInternalState lastTicketNo st = IS { , isLastTicketNo = lastTicketNo } -initMempoolEnv :: (IOLike m, ApplyTx blk) +initMempoolEnv :: (IOLike m, ApplyTx blk, ValidateEnvelope blk) => LedgerInterface m blk -> LedgerConfig blk -> MempoolCapacityBytes @@ -184,7 +185,11 @@ initMempoolEnv ledgerInterface cfg capacity tracer = do -- | Spawn a thread which syncs the 'Mempool' state whenever the 'LedgerState' -- changes. -forkSyncStateOnTipPointChange :: forall m blk. (IOLike m, ApplyTx blk) +forkSyncStateOnTipPointChange :: forall m blk. ( + IOLike m + , ApplyTx blk + , ValidateEnvelope blk + ) => ResourceRegistry m -> MempoolEnv m blk -> m () @@ -285,7 +290,7 @@ implTryAddTxs mpEnv = go [] go ((firstTx, Just err):acc) toAdd' implRemoveTxs - :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk)) + :: (IOLike m, ApplyTx blk, HasTxId (GenTx blk), ValidateEnvelope blk) => MempoolEnv m blk -> [GenTxId blk] -> m () @@ -320,7 +325,7 @@ implRemoveTxs mpEnv txIds = do toRemove = Set.fromList txIds -implSyncWithLedger :: (IOLike m, ApplyTx blk) +implSyncWithLedger :: (IOLike m, ApplyTx blk, ValidateEnvelope blk) => MempoolEnv m blk -> m (MempoolSnapshot blk TicketNo) implSyncWithLedger mpEnv@MempoolEnv{mpEnvTracer, mpEnvStateVar} = do (removed, mempoolSize, snapshot) <- atomically $ do @@ -340,7 +345,7 @@ implGetSnapshot :: (IOLike m, ApplyTx blk) implGetSnapshot MempoolEnv{mpEnvStateVar} = implSnapshotFromIS <$> readTVar mpEnvStateVar -implGetSnapshotFor :: forall m blk. (IOLike m, ApplyTx blk) +implGetSnapshotFor :: forall m blk. (IOLike m, ApplyTx blk, ValidateEnvelope blk) => MempoolEnv m blk -> BlockSlot -> LedgerState blk @@ -546,7 +551,7 @@ extendVRNew cfg tx vr = assert (isNothing vrNewValid) $ -- | Validate the internal state against the current ledger state and the -- given 'BlockSlot', revalidating if necessary. -validateIS :: forall m blk. (IOLike m, ApplyTx blk) +validateIS :: forall m blk. (IOLike m, ApplyTx blk, ValidateEnvelope blk) => MempoolEnv m blk -> BlockSlot -> STM m (ValidationResult blk) @@ -565,7 +570,7 @@ validateIS MempoolEnv{mpEnvLedger, mpEnvLedgerCfg, mpEnvStateVar} blockSlot = -- When these don't match, the transaction in the internal state will be -- revalidated ('revalidateTxsFor'). validateStateFor - :: forall blk. ApplyTx blk + :: forall blk. (ApplyTx blk, ValidateEnvelope blk) => LedgerConfig blk -> BlockSlot -> LedgerState blk @@ -601,7 +606,7 @@ revalidateTxsFor cfg st lastTicketNo txTickets = -- | Tick the 'LedgerState' using the given 'BlockSlot'. tickLedgerState - :: UpdateLedger blk + :: forall blk. (UpdateLedger blk, ValidateEnvelope blk) => LedgerConfig blk -> BlockSlot -> LedgerState blk @@ -614,9 +619,9 @@ tickLedgerState cfg blockSlot st = applyChainTick cfg slot st slot = case blockSlot of TxsForBlockInSlot s -> s TxsForUnknownBlock -> + -- TODO: We should use time here instead + -- + -- Once we do, the ValidateEnvelope constraint can go. case ledgerTipSlot st of - -- TODO: We should not make assumptions about the underlying - -- ledger. We will fix this in - -- - Origin -> Block.SlotNo 0 + Origin -> minimumPossibleSlotNo (Proxy @blk) At s -> succ s diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node.hs index b95a103fb6c..f1cf8c65a9a 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node.hs @@ -226,11 +226,13 @@ mkChainDbArgs tracer registry btime dbPath cfg initLedger , ChainDB.cdbDecodeChainState = nodeDecodeChainState (Proxy @blk) cfg , ChainDB.cdbDecodeHash = nodeDecodeHeaderHash (Proxy @blk) , ChainDB.cdbDecodeLedger = nodeDecodeLedgerState cfg + , ChainDB.cdbDecodeTipInfo = nodeDecodeTipInfo (Proxy @blk) , ChainDB.cdbEncodeBlock = nodeEncodeBlockWithInfo cfg , ChainDB.cdbEncodeHeader = nodeEncodeHeader cfg , ChainDB.cdbEncodeChainState = nodeEncodeChainState (Proxy @blk) cfg , ChainDB.cdbEncodeHash = nodeEncodeHeaderHash (Proxy @blk) , ChainDB.cdbEncodeLedger = nodeEncodeLedgerState cfg + , ChainDB.cdbEncodeTipInfo = nodeEncodeTipInfo (Proxy @blk) , ChainDB.cdbEpochInfo = epochInfo , ChainDB.cdbHashInfo = nodeHashInfo (Proxy @blk) , ChainDB.cdbGenesis = return initLedger diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ErrorPolicy.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ErrorPolicy.hs index 0d2857db126..c54ea2826f4 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ErrorPolicy.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ErrorPolicy.hs @@ -99,7 +99,7 @@ consensusErrorPolicy = ErrorPolicies { case e of HeaderExceedsClockSkew{} -> Just misconfiguredPeer ForkTooDeep{} -> Just distantPeer - ChainError{} -> Just theyBuggyOrEvil + HeaderError{} -> Just theyBuggyOrEvil InvalidRollForward{} -> Just distantPeer InvalidRollBack{} -> Just theyBuggyOrEvil InvalidIntersection{} -> Just theyBuggyOrEvil diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Byron.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Byron.hs index 6282ca75a38..a6f9b490f55 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Byron.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Byron.hs @@ -38,6 +38,7 @@ import qualified Cardano.Crypto as Crypto import Ouroboros.Consensus.BlockchainTime import Ouroboros.Consensus.Crypto.DSIGN.Cardano +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Byron import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -127,8 +128,8 @@ protocolInfoByron genesisConfig mSigThresh pVer sVer mLeader = Just cred -> PBftIsALeader $ pbftLeaderOrNot cred } , pInfoInitLedger = ExtLedgerState { - ledgerState = initByronLedgerState genesisConfig Nothing - , ouroborosChainState = CS.empty + ledgerState = initByronLedgerState genesisConfig Nothing + , headerState = genesisHeaderState CS.empty } , pInfoInitState = () } diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/BFT.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/BFT.hs index a804c382a5a..423440351d7 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/BFT.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/BFT.hs @@ -7,6 +7,7 @@ import qualified Data.Map.Strict as Map import Cardano.Crypto.DSIGN import Ouroboros.Consensus.BlockchainTime +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Ledger.Mock import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -34,7 +35,8 @@ protocolInfoBft numCoreNodes nid securityParam slotLengths = | n <- enumCoreNodes numCoreNodes ] } - , pInfoInitLedger = ExtLedgerState (genesisSimpleLedgerState addrDist) () + , pInfoInitLedger = ExtLedgerState (genesisSimpleLedgerState addrDist) + (genesisHeaderState ()) , pInfoInitState = () } where diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PBFT.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PBFT.hs index 48882d76e4b..174db6179d5 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PBFT.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PBFT.hs @@ -10,6 +10,7 @@ import qualified Data.Bimap as Bimap import Cardano.Crypto.DSIGN +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Ledger.Mock import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -34,7 +35,7 @@ protocolInfoMockPBFT params nid = } } , pInfoInitLedger = ExtLedgerState (genesisSimpleLedgerState addrDist) - CS.empty + (genesisHeaderState CS.empty) , pInfoInitState = () } where diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/Praos.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/Praos.hs index 7fb06848644..5b7e275d1fe 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/Praos.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/Praos.hs @@ -12,6 +12,7 @@ import qualified Data.Map as Map import Cardano.Crypto.KES import Cardano.Crypto.VRF +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Ledger.Mock import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -35,8 +36,8 @@ protocolInfoPraos numCoreNodes nid params = , praosVerKeys = verKeys } , pInfoInitLedger = ExtLedgerState { - ledgerState = genesisSimpleLedgerState addrDist - , ouroborosChainState = [] + ledgerState = genesisSimpleLedgerState addrDist + , headerState = genesisHeaderState [] } , pInfoInitState = PraosNodeState $ SignKeyMockKES (fst $ verKeys Map.! nid) -- key ID diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PraosRule.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PraosRule.hs index 9bee5546a82..55c7b6ad8e6 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PraosRule.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/ProtocolInfo/Mock/PraosRule.hs @@ -9,6 +9,7 @@ import qualified Data.Map as Map import Cardano.Crypto.KES import Cardano.Crypto.VRF +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Ledger.Mock import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -39,8 +40,8 @@ protocolInfoPraosRule numCoreNodes , lsNodeConfigNodeId = nid } , pInfoInitLedger = ExtLedgerState - { ledgerState = genesisSimpleLedgerState addrDist - , ouroborosChainState = () + { ledgerState = genesisSimpleLedgerState addrDist + , headerState = genesisHeaderState () } , pInfoInitState = () } diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Abstract.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Abstract.hs index c73f8f68857..de0d676dfc3 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Abstract.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Abstract.hs @@ -26,6 +26,7 @@ import Ouroboros.Network.Protocol.LocalStateQuery.Codec (Some (..)) import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime (SystemStart) +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Byron import Ouroboros.Consensus.Ledger.Extended @@ -132,6 +133,7 @@ class ( ProtocolLedgerView blk nodeEncodeLedgerState :: NodeConfig (BlockProtocol blk) -> LedgerState blk -> Encoding nodeEncodeChainState :: Proxy blk -> NodeConfig (BlockProtocol blk) -> ChainState (BlockProtocol blk) -> Encoding nodeEncodeApplyTxError :: Proxy blk -> ApplyTxErr blk -> Encoding + nodeEncodeTipInfo :: Proxy blk -> TipInfo blk -> Encoding nodeEncodeQuery :: Query blk result -> Encoding nodeEncodeResult :: Query blk result -> result -> Encoding @@ -144,5 +146,6 @@ class ( ProtocolLedgerView blk nodeDecodeLedgerState :: forall s. NodeConfig (BlockProtocol blk) -> Decoder s (LedgerState blk) nodeDecodeChainState :: forall s. Proxy blk -> NodeConfig (BlockProtocol blk) -> Decoder s (ChainState (BlockProtocol blk)) nodeDecodeApplyTxError :: forall s. Proxy blk -> Decoder s (ApplyTxErr blk) + nodeDecodeTipInfo :: forall s. Proxy blk -> Decoder s (TipInfo blk) nodeDecodeQuery :: forall s. Decoder s (Some (Query blk)) nodeDecodeResult :: Query blk result -> forall s. Decoder s result diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Byron.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Byron.hs index 1f661e693a7..5b325c0f1de 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Byron.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Byron.hs @@ -8,6 +8,7 @@ module Ouroboros.Consensus.Node.Run.Byron ( extractEpochSlots ) where +import Codec.Serialise (decode, encode) import Data.Coerce (coerce) import qualified Cardano.Chain.Block as Cardano.Block @@ -95,6 +96,7 @@ instance RunNode ByronBlock where nodeEncodeLedgerState = const encodeByronLedgerState nodeEncodeChainState = \_proxy _cfg -> encodeByronChainState nodeEncodeApplyTxError = const encodeByronApplyTxError + nodeEncodeTipInfo = const encode nodeEncodeQuery = encodeByronQuery nodeEncodeResult = encodeByronResult @@ -109,6 +111,7 @@ instance RunNode ByronBlock where pbftParams (extNodeConfigP cfg) in decodeByronChainState k nodeDecodeApplyTxError = const decodeByronApplyTxError + nodeDecodeTipInfo = const decode nodeDecodeQuery = decodeByronQuery nodeDecodeResult = decodeByronResult diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/DualByron.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/DualByron.hs index 1a46f0871ca..8f5002cbca9 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/DualByron.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/DualByron.hs @@ -63,8 +63,10 @@ instance RunNode DualByronBlock where nodeMaxBlockSize = nodeMaxBlockSize . dualLedgerStateMain nodeBlockEncodingOverhead = nodeBlockEncodingOverhead . dualLedgerStateMain - -- The hash we use is the hash of the concrete block - nodeHashInfo = \_p -> nodeHashInfo pb + -- Envelope + nodeHashInfo = \_p -> nodeHashInfo pb + nodeEncodeTipInfo = \_p -> nodeEncodeTipInfo pb + nodeDecodeTipInfo = \_p -> nodeDecodeTipInfo pb -- We can look at the concrete header to see if this is an EBB nodeIsEBB = nodeIsEBB . dualHeaderMain diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Mock.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Mock.hs index 3b334e5f2ab..4610d0bd7d5 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Mock.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Node/Run/Mock.hs @@ -66,6 +66,7 @@ instance ( ProtocolLedgerView (SimpleBlock SimpleMockCrypto ext) nodeEncodeLedgerState = const encode nodeEncodeChainState = const mockEncodeChainState nodeEncodeApplyTxError = const encode + nodeEncodeTipInfo = const encode nodeEncodeQuery = \case {} nodeEncodeResult = \case {} @@ -77,5 +78,6 @@ instance ( ProtocolLedgerView (SimpleBlock SimpleMockCrypto ext) nodeDecodeLedgerState = const decode nodeDecodeChainState = const mockDecodeChainState nodeDecodeApplyTxError = const decode + nodeDecodeTipInfo = const decode nodeDecodeQuery = error "Mock.nodeDecodeQuery" nodeDecodeResult = \case {} diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/NodeKernel.hs b/ouroboros-consensus/src/Ouroboros/Consensus/NodeKernel.hs index 53abf55da82..be220fdd2a8 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/NodeKernel.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/NodeKernel.hs @@ -7,6 +7,7 @@ {-# LANGUAGE RankNTypes #-} {-# LANGUAGE RecordWildCards #-} {-# LANGUAGE ScopedTypeVariables #-} +{-# LANGUAGE TypeApplications #-} {-# LANGUAGE TypeFamilies #-} {-# OPTIONS_GHC -Wredundant-constraints -Werror=missing-fields #-} @@ -29,6 +30,7 @@ import Control.Monad import Crypto.Random (ChaChaDRG) import Data.Map.Strict (Map) import Data.Maybe (isJust, isNothing) +import Data.Proxy import Data.Word (Word16, Word32) import Cardano.Prelude (UseIsNormalForm (..)) @@ -52,6 +54,7 @@ import qualified Ouroboros.Network.TxSubmission.Outbound as Outbound import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime import Ouroboros.Consensus.ChainSyncClient +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Mempool @@ -392,7 +395,7 @@ forkBlockProduction maxBlockSizeOverride IS{..} BlockProduction{..} = cfg currentSlot ledgerView - (ouroborosChainState extLedger) + (headerStateChain (headerState extLedger)) case mIsLeader of Just p -> return p Nothing -> do @@ -546,12 +549,7 @@ mkCurrentBlockContext mkCurrentBlockContext currentSlot c = case c of Empty AF.AnchorGenesis -> -- The chain is entirely empty. - -- - -- TODO: We should not make assumptions about the underlying - -- ledger. We will fix this in - -- - let firstBlockNo = BlockNo 0 in - Right $ BlockContext firstBlockNo genesisPoint + Right $ BlockContext (firstBlockNo (Proxy @blk)) genesisPoint Empty (AF.Anchor anchorSlot anchorHash anchorBlockNo) -> let p :: Point blk = BlockPoint anchorSlot anchorHash diff --git a/ouroboros-consensus/src/Ouroboros/Consensus/Util/CBOR.hs b/ouroboros-consensus/src/Ouroboros/Consensus/Util/CBOR.hs index cde6647b1ee..794bc90a96e 100644 --- a/ouroboros-consensus/src/Ouroboros/Consensus/Util/CBOR.hs +++ b/ouroboros-consensus/src/Ouroboros/Consensus/Util/CBOR.hs @@ -1,4 +1,5 @@ {-# LANGUAGE BangPatterns #-} +{-# LANGUAGE LambdaCase #-} {-# LANGUAGE RankNTypes #-} {-# LANGUAGE RecordWildCards #-} {-# LANGUAGE ScopedTypeVariables #-} @@ -16,10 +17,20 @@ module Ouroboros.Consensus.Util.CBOR ( , readIncremental , readIncrementalOffsets , withStreamIncrementalOffsets + -- * Encoding/decoding containers + , encodeList + , decodeList + , encodeSeq + , decodeSeq + , encodeMaybe + , decodeMaybe + , encodeWithOrigin + , decodeWithOrigin ) where -import qualified Codec.CBOR.Decoding as CBOR (Decoder) -import qualified Codec.CBOR.Read as CBOR +import qualified Codec.CBOR.Decoding as CBOR.D +import qualified Codec.CBOR.Encoding as CBOR.E +import qualified Codec.CBOR.Read as CBOR.R import Control.Exception (assert, throwIO) import Control.Monad import Control.Monad.ST @@ -27,13 +38,20 @@ import Data.ByteString (ByteString) import qualified Data.ByteString as BS import Data.ByteString.Builder.Extra (defaultChunkSize) import qualified Data.ByteString.Lazy as LBS +import Data.Foldable (toList) import Data.IORef +import Data.Sequence.Strict (StrictSeq) +import qualified Data.Sequence.Strict as Seq import Data.Word (Word64) import GHC.Stack (HasCallStack) + import qualified Streaming as S import Streaming.Prelude (Of (..), Stream) import qualified Streaming.Prelude as S +import Cardano.Slotting.Slot (WithOrigin, withOriginFromMaybe, + withOriginToMaybe) + import Ouroboros.Consensus.Util.IOLike import Ouroboros.Storage.FS.API @@ -45,16 +63,18 @@ import Ouroboros.Storage.FS.API.Types data IDecodeIO a = Partial (Maybe ByteString -> IO (IDecodeIO a)) - | Done !ByteString !CBOR.ByteOffset a - | Fail !ByteString !CBOR.ByteOffset CBOR.DeserialiseFailure + | Done !ByteString !CBOR.R.ByteOffset a + | Fail !ByteString !CBOR.R.ByteOffset CBOR.R.DeserialiseFailure -fromIDecode :: CBOR.IDecode RealWorld a -> IDecodeIO a -fromIDecode (CBOR.Partial k) = Partial $ fmap fromIDecode . stToIO . k -fromIDecode (CBOR.Done bs off x) = Done bs off x -fromIDecode (CBOR.Fail bs off e) = Fail bs off e +fromIDecode :: CBOR.R.IDecode RealWorld a -> IDecodeIO a +fromIDecode (CBOR.R.Partial k) = Partial $ fmap fromIDecode . stToIO . k +fromIDecode (CBOR.R.Done bs off x) = Done bs off x +fromIDecode (CBOR.R.Fail bs off e) = Fail bs off e -deserialiseIncrementalIO :: (forall s. CBOR.Decoder s a) -> IO (IDecodeIO a) -deserialiseIncrementalIO = fmap fromIDecode . stToIO . CBOR.deserialiseIncremental +deserialiseIncrementalIO :: (forall s. CBOR.D.Decoder s a) -> IO (IDecodeIO a) +deserialiseIncrementalIO = fmap fromIDecode + . stToIO + . CBOR.R.deserialiseIncremental {------------------------------------------------------------------------------- Higher-level incremental interface @@ -64,7 +84,7 @@ data Decoder m = Decoder { -- | Decode next failure -- -- May throw 'CBOR.DeserialiseFailure' - decodeNext :: forall a. (forall s. CBOR.Decoder s a) -> m a + decodeNext :: forall a. (forall s. CBOR.D.Decoder s a) -> m a } -- | Construct incremental decoder given a way to get chunks @@ -73,7 +93,7 @@ data Decoder m = Decoder { initDecoderIO :: IO ByteString -> IO (Decoder IO) initDecoderIO getChunk = do leftover <- newIORef BS.empty - let go :: forall a. (forall s. CBOR.Decoder s a) -> IO a + let go :: forall a. (forall s. CBOR.D.Decoder s a) -> IO a go decoder = do i <- deserialiseIncrementalIO decoder case i of @@ -102,7 +122,7 @@ initDecoderIO getChunk = do data ReadIncrementalErr = -- | Could not deserialise the data - ReadFailed CBOR.DeserialiseFailure + ReadFailed CBOR.R.DeserialiseFailure -- | Deserialisation was successful, but there was additional data | TrailingBytes ByteString @@ -120,26 +140,26 @@ data ReadIncrementalErr = -- 'readIncrementalOffsets'. readIncremental :: forall m h a. IOLike m => HasFS m h - -> (forall s . CBOR.Decoder s a) + -> (forall s . CBOR.D.Decoder s a) -> FsPath -> m (Either ReadIncrementalErr a) readIncremental hasFS@HasFS{..} decoder fp = withLiftST $ \liftST -> do withFile hasFS fp ReadMode $ \h -> - go liftST h =<< liftST (CBOR.deserialiseIncremental decoder) + go liftST h =<< liftST (CBOR.R.deserialiseIncremental decoder) where go :: (forall x. ST s x -> m x) -> Handle h - -> CBOR.IDecode s a + -> CBOR.R.IDecode s a -> m (Either ReadIncrementalErr a) - go liftST h (CBOR.Partial k) = do + go liftST h (CBOR.R.Partial k) = do bs <- hGetSome h (fromIntegral defaultChunkSize) dec' <- liftST $ k (checkEmpty bs) go liftST h dec' - go _ _ (CBOR.Done leftover _ a) = + go _ _ (CBOR.R.Done leftover _ a) = return $ if BS.null leftover then Right a else Left $ TrailingBytes leftover - go _ _ (CBOR.Fail _ _ err) = + go _ _ (CBOR.R.Fail _ _ err) = return $ Left $ ReadFailed err checkEmpty :: ByteString -> Maybe ByteString @@ -155,7 +175,7 @@ readIncremental hasFS@HasFS{..} decoder fp = withLiftST $ \liftST -> do readIncrementalOffsets :: forall m h a. (IOLike m, HasCallStack) => HasFS m h - -> (forall s . CBOR.Decoder s (LBS.ByteString -> a)) + -> (forall s . CBOR.D.Decoder s (LBS.ByteString -> a)) -> FsPath -> m ([(Word64, (Word64, a))], Maybe (ReadIncrementalErr, Word64)) -- ^ ((the offset of the start of @a@ in the file, @@ -176,7 +196,7 @@ readIncrementalOffsets hasFS decoder fp = withStreamIncrementalOffsets :: forall m h a r. (IOLike m, HasCallStack) => HasFS m h - -> (forall s . CBOR.Decoder s (LBS.ByteString -> a)) + -> (forall s . CBOR.D.Decoder s (LBS.ByteString -> a)) -> FsPath -> (Stream (Of (Word64, (Word64, a))) m (Maybe (ReadIncrementalErr, Word64)) -> m r) -> m r @@ -188,7 +208,7 @@ withStreamIncrementalOffsets hasFS@HasFS{..} decoder fp = \k -> -- If the file is empty, we will immediately get "end of input" return Nothing else - S.lift (liftST (CBOR.deserialiseIncremental decoder)) >>= + S.lift (liftST (CBOR.R.deserialiseIncremental decoder)) >>= go liftST h 0 Nothing [] fileSize where -- TODO stream from HasFS? @@ -198,10 +218,10 @@ withStreamIncrementalOffsets hasFS@HasFS{..} decoder fp = \k -> -> Maybe ByteString -- ^ Unconsumed bytes from last time -> [ByteString] -- ^ Chunks pushed for this item (rev order) -> Word64 -- ^ Total file size - -> CBOR.IDecode s (LBS.ByteString -> a) + -> CBOR.R.IDecode s (LBS.ByteString -> a) -> Stream (Of (Word64, (Word64, a))) m (Maybe (ReadIncrementalErr, Word64)) go liftST h offset mbUnconsumed bss fileSize dec = case dec of - CBOR.Partial k -> do + CBOR.R.Partial k -> do -- First use the unconsumed bytes from a previous read before read -- some more bytes from the file. bs <- case mbUnconsumed of @@ -210,7 +230,7 @@ withStreamIncrementalOffsets hasFS@HasFS{..} decoder fp = \k -> dec' <- S.lift $ liftST $ k (checkEmpty bs) go liftST h offset Nothing (bs:bss) fileSize dec' - CBOR.Done leftover size mkA -> do + CBOR.R.Done leftover size mkA -> do let nextOffset = offset + fromIntegral size -- We've been keeping track of the bytes pushed into the decoder -- for this item so far in bss. Now there's some trailing data to @@ -237,11 +257,55 @@ withStreamIncrementalOffsets hasFS@HasFS{..} decoder fp = \k -> -> return Nothing -- Some more bytes, so try to read the next @a@. mbLeftover -> - S.lift (liftST (CBOR.deserialiseIncremental decoder)) >>= + S.lift (liftST (CBOR.R.deserialiseIncremental decoder)) >>= go liftST h nextOffset mbLeftover [] fileSize - CBOR.Fail _ _ err -> return $ Just (ReadFailed err, offset) + CBOR.R.Fail _ _ err -> return $ Just (ReadFailed err, offset) checkEmpty :: ByteString -> Maybe ByteString checkEmpty bs | BS.null bs = Nothing | otherwise = Just bs + +{------------------------------------------------------------------------------- + Encoding/decoding lists +-------------------------------------------------------------------------------} + +encodeList :: (a -> CBOR.E.Encoding) -> [a] -> CBOR.E.Encoding +encodeList _ [] = CBOR.E.encodeListLen 0 +encodeList enc xs = mconcat [ + CBOR.E.encodeListLenIndef + , foldr (\x r -> enc x <> r) CBOR.E.encodeBreak xs + ] + +decodeList :: CBOR.D.Decoder s a -> CBOR.D.Decoder s [a] +decodeList dec = do + mn <- CBOR.D.decodeListLenOrIndef + case mn of + Nothing -> CBOR.D.decodeSequenceLenIndef (flip (:)) [] reverse dec + Just n -> CBOR.D.decodeSequenceLenN (flip (:)) [] reverse n dec + +encodeSeq :: (a -> CBOR.E.Encoding) -> StrictSeq a -> CBOR.E.Encoding +encodeSeq f = encodeList f . toList + +decodeSeq :: CBOR.D.Decoder s a -> CBOR.D.Decoder s (StrictSeq a) +decodeSeq f = Seq.fromList <$> decodeList f + +encodeMaybe :: (a -> CBOR.E.Encoding) -> Maybe a -> CBOR.E.Encoding +encodeMaybe enc = \case + Nothing -> CBOR.E.encodeListLen 0 + Just x -> CBOR.E.encodeListLen 1 <> enc x + +decodeMaybe :: CBOR.D.Decoder s a -> CBOR.D.Decoder s (Maybe a) +decodeMaybe dec = do + n <- CBOR.D.decodeListLen + case n of + 0 -> return Nothing + 1 -> do !x <- dec + return (Just x) + _ -> fail "unknown tag" + +encodeWithOrigin :: (a -> CBOR.E.Encoding) -> WithOrigin a -> CBOR.E.Encoding +encodeWithOrigin f = encodeMaybe f . withOriginToMaybe + +decodeWithOrigin :: CBOR.D.Decoder s a -> CBOR.D.Decoder s (WithOrigin a) +decodeWithOrigin f = withOriginFromMaybe <$> decodeMaybe f diff --git a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs index ce2e94297fa..7331967ecf8 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/API.hs @@ -305,12 +305,7 @@ data ChainDB m blk = ChainDB { getCurrentTip :: (Monad (STM m), HasHeader (Header blk)) => ChainDB m blk -> STM m (Network.Tip blk) -getCurrentTip chainDB = - mkTip . AF.headAnchor <$> getCurrentChain chainDB - where - mkTip :: AF.Anchor (Header blk) -> Network.Tip blk - mkTip AF.AnchorGenesis = Network.TipGenesis - mkTip (AF.Anchor s h b) = Network.Tip s h b +getCurrentTip = fmap (AF.anchorToTip . AF.headAnchor) . getCurrentChain getTipBlockNo :: (Monad (STM m), HasHeader (Header blk)) => ChainDB m blk -> STM m (WithOrigin BlockNo) diff --git a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/Args.hs b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/Args.hs index 058e5730def..470fd6f313f 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/Args.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/Args.hs @@ -22,6 +22,7 @@ import Ouroboros.Network.Block (HeaderHash) import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime (BlockchainTime) +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Protocol.Abstract @@ -54,6 +55,7 @@ data ChainDbArgs m blk = forall h1 h2 h3. ChainDbArgs { -- ^ The given encoding will include the header envelope -- ('cdbAddHdrEnv'). , cdbDecodeLedger :: forall s. Decoder s (LedgerState blk) + , cdbDecodeTipInfo :: forall s. Decoder s (TipInfo blk) , cdbDecodeChainState :: forall s. Decoder s (ChainState (BlockProtocol blk)) -- Encoders @@ -67,6 +69,7 @@ data ChainDbArgs m blk = forall h1 h2 h3. ChainDbArgs { -- often be encoding the in-memory headers. (It is cheap for Byron -- headers, as we store the serialisation in the annotation.) , cdbEncodeLedger :: LedgerState blk -> Encoding + , cdbEncodeTipInfo :: TipInfo blk -> Encoding , cdbEncodeChainState :: ChainState (BlockProtocol blk) -> Encoding -- Error handling @@ -198,9 +201,11 @@ fromChainDbArgs ChainDbArgs{..} = ( , lgrDecodeLedger = cdbDecodeLedger , lgrDecodeChainState = cdbDecodeChainState , lgrDecodeHash = cdbDecodeHash + , lgrDecodeTipInfo = cdbDecodeTipInfo , lgrEncodeLedger = cdbEncodeLedger , lgrEncodeChainState = cdbEncodeChainState , lgrEncodeHash = cdbEncodeHash + , lgrEncodeTipInfo = cdbEncodeTipInfo , lgrParams = cdbParamsLgrDB , lgrDiskPolicy = cdbDiskPolicy , lgrGenesis = cdbGenesis @@ -234,12 +239,14 @@ toChainDbArgs ImmDB.ImmDbArgs{..} , cdbDecodeBlock = immDecodeBlock , cdbDecodeHeader = immDecodeHeader , cdbDecodeLedger = lgrDecodeLedger + , cdbDecodeTipInfo = lgrDecodeTipInfo , cdbDecodeChainState = lgrDecodeChainState -- Encoders , cdbEncodeHash = immEncodeHash , cdbEncodeBlock = immEncodeBlock , cdbEncodeHeader = cdbsEncodeHeader , cdbEncodeLedger = lgrEncodeLedger + , cdbEncodeTipInfo = lgrEncodeTipInfo , cdbEncodeChainState = lgrEncodeChainState -- Error handling , cdbErrImmDb = immErr diff --git a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/LgrDB.hs b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/LgrDB.hs index ed49640b066..184bbc916c0 100644 --- a/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/LgrDB.hs +++ b/ouroboros-consensus/src/Ouroboros/Storage/ChainDB/Impl/LgrDB.hs @@ -75,6 +75,7 @@ import qualified Ouroboros.Network.Block as Block import Ouroboros.Network.Point (WithOrigin (At)) import Ouroboros.Consensus.Block +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Protocol.Abstract @@ -147,11 +148,13 @@ data LgrDbArgs m blk = forall h. LgrDbArgs { lgrNodeConfig :: NodeConfig (BlockProtocol blk) , lgrHasFS :: HasFS m h , lgrDecodeLedger :: forall s. Decoder s (LedgerState blk) + , lgrDecodeHash :: forall s. Decoder s (HeaderHash blk) + , lgrDecodeTipInfo :: forall s. Decoder s (TipInfo blk) , lgrDecodeChainState :: forall s. Decoder s (ChainState (BlockProtocol blk)) - , lgrDecodeHash :: forall s. Decoder s (HeaderHash blk) , lgrEncodeLedger :: LedgerState blk -> Encoding + , lgrEncodeHash :: HeaderHash blk -> Encoding + , lgrEncodeTipInfo :: TipInfo blk -> Encoding , lgrEncodeChainState :: ChainState (BlockProtocol blk) -> Encoding - , lgrEncodeHash :: HeaderHash blk -> Encoding , lgrParams :: LedgerDbParams , lgrDiskPolicy :: DiskPolicy , lgrGenesis :: m (ExtLedgerState blk) @@ -168,9 +171,11 @@ data LgrDbArgs m blk = forall h. LgrDbArgs { -- * 'lgrDecodeLedger' -- * 'lgrDecodeChainState' -- * 'lgrDecodeHash' +-- * 'lgrDecodeTipInfo' -- * 'lgrEncodeLedger' -- * 'lgrEncodeChainState' -- * 'lgrEncodeHash' +-- * 'lgrEncodeTipInfo' -- * 'lgrMemPolicy' -- * 'lgrGenesis' defaultArgs :: FilePath -> LgrDbArgs IO blk @@ -179,11 +184,13 @@ defaultArgs fp = LgrDbArgs { -- Fields without a default , lgrNodeConfig = error "no default for lgrNodeConfig" , lgrDecodeLedger = error "no default for lgrDecodeLedger" - , lgrDecodeChainState = error "no default for lgrDecodeChainState" , lgrDecodeHash = error "no default for lgrDecodeHash" + , lgrDecodeTipInfo = error "no default for lgrDecodeTipInfo" + , lgrDecodeChainState = error "no default for lgrDecodeChainState" , lgrEncodeLedger = error "no default for lgrEncodeLedger" - , lgrEncodeChainState = error "no default for lgrEncodeChainState" , lgrEncodeHash = error "no default for lgrEncodeHash" + , lgrEncodeTipInfo = error "no default for lgrEncodeTipInfo" + , lgrEncodeChainState = error "no default for lgrEncodeChainState" , lgrParams = error "no default for lgrParams" , lgrDiskPolicy = error "no default for lgrDiskPolicy" , lgrGenesis = error "no default for lgrGenesis" @@ -261,7 +268,7 @@ reopen LgrDB{..} immDB replayTracer = do atomically $ writeTVar varDB db return replayed -initFromDisk :: (IOLike m, HasHeader blk, HasCallStack) +initFromDisk :: forall blk m. (IOLike m, HasHeader blk, HasCallStack) => LgrDbArgs m blk -> Tracer m (TraceReplayEvent (Point blk) () (Point blk)) -> Conf m blk @@ -273,12 +280,19 @@ initFromDisk args@LgrDbArgs{..} replayTracer lgrDbConf immDB = wrapFailure args replayTracer lgrTracer lgrHasFS - (decodeExtLedgerState lgrDecodeLedger lgrDecodeChainState) + decodeExtLedgerState' (Block.decodePoint lgrDecodeHash) lgrParams lgrDbConf (streamAPI immDB) return (db, replayed) + where + decodeExtLedgerState' :: forall s. Decoder s (ExtLedgerState blk) + decodeExtLedgerState' = decodeExtLedgerState + lgrDecodeLedger + lgrDecodeChainState + lgrDecodeHash + lgrDecodeTipInfo -- | For testing purposes mkLgrDB :: Conf m blk @@ -359,9 +373,15 @@ takeSnapshot lgrDB@LgrDB{ args = args@LgrDbArgs{..} } = wrapFailure args $ do second tipToPoint <$> LedgerDB.takeSnapshot lgrTracer lgrHasFS - (encodeExtLedgerState lgrEncodeLedger lgrEncodeChainState) + encodeExtLedgerState' (Block.encodePoint lgrEncodeHash) ledgerDB + where + encodeExtLedgerState' = encodeExtLedgerState + lgrEncodeLedger + lgrEncodeChainState + lgrEncodeHash + lgrEncodeTipInfo trimSnapshots :: MonadThrow m => LgrDB m blk -> m [DiskSnapshot] trimSnapshots LgrDB{ args = args@LgrDbArgs{..} } = wrapFailure args $ diff --git a/ouroboros-consensus/test-consensus/Test/Consensus/LocalStateQueryServer.hs b/ouroboros-consensus/test-consensus/Test/Consensus/LocalStateQueryServer.hs index 443ce576646..262af081426 100644 --- a/ouroboros-consensus/test-consensus/Test/Consensus/LocalStateQueryServer.hs +++ b/ouroboros-consensus/test-consensus/Test/Consensus/LocalStateQueryServer.hs @@ -224,9 +224,11 @@ initLgrDB k chain = do , lgrDecodeLedger = error "lgrDecodeLedger" , lgrDecodeChainState = error "lgrDecodeChainState" , lgrDecodeHash = error "lgrDecodeHash" + , lgrDecodeTipInfo = error "lgrDecodeTipInfo" , lgrEncodeLedger = error "lgrEncodeLedger" , lgrEncodeChainState = error "lgrEncodeChainState" , lgrEncodeHash = error "lgrEncodeHash" + , lgrEncodeTipInfo = error "lgrEncodeTipInfo" , lgrParams = params , lgrDiskPolicy = error "lgrDiskPolicy" , lgrGenesis = return testInitExtLedger diff --git a/ouroboros-consensus/test-consensus/Test/ThreadNet/Network.hs b/ouroboros-consensus/test-consensus/Test/ThreadNet/Network.hs index ed917f97a75..b8419a22860 100644 --- a/ouroboros-consensus/test-consensus/Test/ThreadNet/Network.hs +++ b/ouroboros-consensus/test-consensus/Test/ThreadNet/Network.hs @@ -73,6 +73,7 @@ import qualified Ouroboros.Consensus.BlockFetchServer as BFServer import Ouroboros.Consensus.ChainSyncClient (ClockSkew (..)) import qualified Ouroboros.Consensus.ChainSyncClient as CSClient import Ouroboros.Consensus.ChainSyncServer (Tip) +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Ledger.Mock @@ -484,12 +485,8 @@ runThreadNetwork ThreadNetworkArgs Origin -> True At s -> s >= (ebbSlotNo - min ebbSlotNo (2 * k)) bno <- ChainDB.getTipBlockNo chainDB - -- TODO: We should not make assumptions about the underlying - -- ledger. We will fix this in - -- - let firstBlockNo = BlockNo 0 -- The EBB shares its BlockNo with its predecessor (if there is one) - pure (mSlot, fromWithOrigin firstBlockNo bno, pointHash p) + pure (mSlot, fromWithOrigin (firstBlockNo (Proxy @blk)) bno, pointHash p) when (prevSlot < At ebbSlotNo) $ do let ebb = forgeEBB cfg ebbSlotNo ebbBlockNo prevHash ChainDB.addBlock chainDB ebb @@ -518,12 +515,14 @@ runThreadNetwork ThreadNetworkArgs , cdbDecodeHeader = nodeDecodeHeader cfg , cdbDecodeLedger = nodeDecodeLedgerState cfg , cdbDecodeChainState = nodeDecodeChainState (Proxy @blk) cfg + , cdbDecodeTipInfo = nodeDecodeTipInfo (Proxy @blk) -- Encoders , cdbEncodeHash = nodeEncodeHeaderHash (Proxy @blk) , cdbEncodeBlock = nodeEncodeBlockWithInfo cfg , cdbEncodeHeader = nodeEncodeHeader cfg , cdbEncodeLedger = nodeEncodeLedgerState cfg , cdbEncodeChainState = nodeEncodeChainState (Proxy @blk) cfg + , cdbEncodeTipInfo = nodeEncodeTipInfo (Proxy @blk) -- Error handling , cdbErrImmDb = EH.monadCatch , cdbErrVolDb = EH.monadCatch diff --git a/ouroboros-consensus/test-consensus/Test/ThreadNet/PBFT.hs b/ouroboros-consensus/test-consensus/Test/ThreadNet/PBFT.hs index c2dc1ec0975..41d1c8196f5 100644 --- a/ouroboros-consensus/test-consensus/Test/ThreadNet/PBFT.hs +++ b/ouroboros-consensus/test-consensus/Test/ThreadNet/PBFT.hs @@ -12,6 +12,7 @@ import Test.Tasty.QuickCheck import Ouroboros.Network.Block (SlotNo (..)) import Ouroboros.Consensus.BlockchainTime +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Extended (ExtValidationError (..)) import Ouroboros.Consensus.Ledger.Mock.Block import Ouroboros.Consensus.Ledger.Mock.Block.PBFT @@ -67,9 +68,9 @@ expectedBlockRejection (NumCoreNodes nn) BlockRejection , brRejector = CoreId (CoreNodeId i) } | ownBlock = case err of - ExtValidationErrorOuroboros - PBftExceededSignThreshold{} -> True - _ -> False + ExtValidationErrorHeader + (HeaderProtocolError PBftExceededSignThreshold{}) -> True + _ -> False where -- Because of round-robin and the fact that the id divides slot, we know -- the node lead but rejected its own block. This is the only case we diff --git a/ouroboros-consensus/test-consensus/Test/ThreadNet/RealPBFT.hs b/ouroboros-consensus/test-consensus/Test/ThreadNet/RealPBFT.hs index ead1d9e6ee4..8db26b46759 100644 --- a/ouroboros-consensus/test-consensus/Test/ThreadNet/RealPBFT.hs +++ b/ouroboros-consensus/test-consensus/Test/ThreadNet/RealPBFT.hs @@ -38,6 +38,7 @@ import Ouroboros.Consensus.Block (BlockProtocol, getHeader) import Ouroboros.Consensus.BlockchainTime import Ouroboros.Consensus.BlockchainTime.Mock import qualified Ouroboros.Consensus.Crypto.DSIGN.Cardano as Crypto +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Byron (ByronBlock) import qualified Ouroboros.Consensus.Ledger.Byron as Byron import Ouroboros.Consensus.Ledger.Extended (ExtValidationError (..)) @@ -296,11 +297,13 @@ expectedBlockRejection , brRejector = CoreId (CoreNodeId i) } | ownBlock = case err of - ExtValidationErrorOuroboros - PBftExceededSignThreshold{} -> True -- TODO validate this against Ref - -- implementation? - ExtValidationErrorOuroboros - PBftNotGenesisDelegate{} -> + ExtValidationErrorHeader + (HeaderProtocolError PBftExceededSignThreshold{}) -> + -- TODO validate this against Ref implementation? + True + + ExtValidationErrorHeader + (HeaderProtocolError PBftNotGenesisDelegate{}) -> -- only if it rekeyed within before a restarts latest possible -- maturation not $ null $ diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/AddBlock.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/AddBlock.hs index 31369ebf852..81cb4a93722 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/AddBlock.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/AddBlock.hs @@ -244,6 +244,7 @@ mkArgs cfg initLedger tracer registry hashInfo , cdbDecodeHeader = const <$> decode , cdbDecodeLedger = decode , cdbDecodeChainState = decode + , cdbDecodeTipInfo = decode -- Encoders , cdbEncodeHash = encode @@ -251,6 +252,7 @@ mkArgs cfg initLedger tracer registry hashInfo , cdbEncodeHeader = encode , cdbEncodeLedger = encode , cdbEncodeChainState = encode + , cdbEncodeTipInfo = encode -- Error handling , cdbErrImmDb = EH.monadCatch diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model.hs index 24a1aaa28db..28be570013b 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model.hs @@ -82,6 +82,7 @@ import qualified Data.List.NonEmpty as NE import Data.Map.Strict (Map) import qualified Data.Map.Strict as Map import Data.Maybe (fromMaybe, isJust) +import Data.Proxy import GHC.Generics (Generic) import Ouroboros.Network.AnchoredFragment (AnchoredFragment) @@ -96,6 +97,7 @@ import qualified Ouroboros.Network.MockChain.ProducerState as CPS import Ouroboros.Network.Point (WithOrigin (..)) import Ouroboros.Consensus.Block +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Protocol.Abstract @@ -365,11 +367,8 @@ addBlock cfg blk m secParam = protocolSecurityParam cfg -- If we switch to PBFT for these tests, this case is not required anymore - -- TODO: We should not make assumptions about the underlying - -- ledger. We will fix this in - -- addingGenesisEBBToEmptyDB = tipPoint m == GenesisPoint - && Block.blockNo blk == Block.BlockNo 0 + && Block.blockNo blk == firstBlockNo (Proxy @blk) slot = Block.blockSlot blk diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model/Test.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model/Test.hs index c898c801461..5022f8dd68b 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model/Test.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/Model/Test.hs @@ -63,6 +63,8 @@ prop_getChain_addChain bc = prop_alwaysPickPreferredChain :: BlockTree -> Permutation -> Property prop_alwaysPickPreferredChain bt p = + counterexample ("blocks: " ++ show blocks) $ + counterexample ("invalid: " ++ show (M.invalid model)) $ conjoin [ not $ preferCandidate' candidate | candidate <- treeToChains bt diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/StateMachine.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/StateMachine.hs index 11d45277888..7855462ea12 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/StateMachine.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/ChainDB/StateMachine.hs @@ -32,13 +32,15 @@ import Data.Bifunctor import qualified Data.Bifunctor.TH as TH import Data.Bitraversable import Data.ByteString.Lazy (ByteString) +import Data.Foldable (toList) import Data.Functor.Classes (Eq1, Show1) import Data.Functor.Identity (Identity (..)) import Data.List (sortOn) import qualified Data.Map as Map import Data.Ord (Down (..)) import Data.Proxy -import Data.TreeDiff (ToExpr) +import Data.Sequence.Strict (StrictSeq) +import Data.TreeDiff (ToExpr (..)) import Data.Typeable import Data.Word (Word16, Word32) import GHC.Generics (Generic) @@ -73,6 +75,7 @@ import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime import Ouroboros.Consensus.BlockchainTime.Mock (settableBlockchainTime) +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -1029,7 +1032,7 @@ deriving instance Generic (Chain blk) deriving instance Generic (ChainProducerState blk) deriving instance Generic (ReaderState blk) -deriving instance ( ToExpr (ChainState (BlockProtocol blk)) +deriving instance ( ToExpr (HeaderState blk) , ToExpr (LedgerState blk) ) => ToExpr (ExtLedgerState blk) @@ -1054,15 +1057,15 @@ deriving instance ( ToExpr (HeaderHash blk) ) => ToExpr (InvalidBlockReason blk) deriving instance ( ToExpr blk - , ToExpr (HeaderHash blk) - , ToExpr (ChainState (BlockProtocol blk)) + , ToExpr (HeaderHash blk) + , ToExpr (HeaderState blk) , ToExpr (LedgerState blk) , ToExpr (ExtValidationError blk) ) => ToExpr (DBModel blk) deriving instance ( ToExpr blk - , ToExpr (HeaderHash blk) - , ToExpr (ChainState (BlockProtocol blk)) + , ToExpr (HeaderHash blk) + , ToExpr (HeaderState blk) , ToExpr (LedgerState blk) , ToExpr (ExtValidationError blk) ) @@ -1077,10 +1080,17 @@ deriving instance ToExpr TestBody deriving instance ToExpr TestBodyHash deriving instance ToExpr TestBlockError deriving instance ToExpr Blk +deriving instance ToExpr (AnnTip Blk) deriving instance ToExpr (LedgerState Blk) +deriving instance ToExpr (HeaderState Blk) +deriving instance ToExpr (HeaderError Blk) +deriving instance ToExpr (HeaderEnvelopeError Blk) deriving instance ToExpr BftValidationErr deriving instance ToExpr (ExtValidationError Blk) +instance ToExpr a => ToExpr (StrictSeq a) where + toExpr = toExpr . toList + {------------------------------------------------------------------------------- Labelling -------------------------------------------------------------------------------} @@ -1420,6 +1430,7 @@ mkArgs cfg initLedger tracer registry varCurSlot , cdbDecodeHeader = const <$> decode , cdbDecodeLedger = decode , cdbDecodeChainState = decode + , cdbDecodeTipInfo = decode -- Encoders , cdbEncodeHash = encode @@ -1427,6 +1438,7 @@ mkArgs cfg initLedger tracer registry varCurSlot , cdbEncodeHeader = encode , cdbEncodeLedger = encode , cdbEncodeChainState = encode + , cdbEncodeTipInfo = encode -- Error handling , cdbErrImmDb = EH.monadCatch diff --git a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/TestBlock.hs b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/TestBlock.hs index 1383bddfe4f..fb673f08e90 100644 --- a/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/TestBlock.hs +++ b/ouroboros-consensus/test-storage/Test/Ouroboros/Storage/TestBlock.hs @@ -47,6 +47,7 @@ import qualified Ouroboros.Network.MockChain.Chain as Chain import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -346,6 +347,12 @@ instance UpdateLedger TestBlock where ledgerTipPoint = lastAppliedPoint +instance HasAnnTip TestBlock where + -- Use defaults + +instance ValidateEnvelope TestBlock where + -- Use defaults + instance ProtocolLedgerView TestBlock where ledgerConfigView _ = LedgerConfig protocolLedgerView _ _ = () @@ -356,8 +363,8 @@ testInitLedger = TestLedger GenesisPoint GenesisHash testInitExtLedger :: ExtLedgerState TestBlock testInitExtLedger = ExtLedgerState { - ledgerState = testInitLedger - , ouroborosChainState = () + ledgerState = testInitLedger + , headerState = genesisHeaderState () } -- | Trivial test configuration with a single core node diff --git a/ouroboros-consensus/test-util/Test/Util/TestBlock.hs b/ouroboros-consensus/test-util/Test/Util/TestBlock.hs index b496b6bf443..7ad03d9a43c 100644 --- a/ouroboros-consensus/test-util/Test/Util/TestBlock.hs +++ b/ouroboros-consensus/test-util/Test/Util/TestBlock.hs @@ -80,6 +80,7 @@ import qualified Ouroboros.Network.MockChain.Chain as Chain import Ouroboros.Consensus.Block import Ouroboros.Consensus.BlockchainTime +import Ouroboros.Consensus.HeaderValidation import Ouroboros.Consensus.Ledger.Abstract import Ouroboros.Consensus.Ledger.Extended import Ouroboros.Consensus.Node.ProtocolInfo.Abstract @@ -298,6 +299,13 @@ instance UpdateLedger TestBlock where ledgerTipPoint = lastAppliedPoint +instance HasAnnTip TestBlock where + -- Use defaults + +instance ValidateEnvelope TestBlock where + -- The block number of a test block is derived from the length of the hash + firstBlockNo _ = Block.BlockNo 1 + instance ProtocolLedgerView TestBlock where ledgerConfigView _ = LedgerConfig protocolLedgerView _ _ = () @@ -322,8 +330,8 @@ testInitLedger = TestLedger Block.genesisPoint testInitExtLedger :: ExtLedgerState TestBlock testInitExtLedger = ExtLedgerState { - ledgerState = testInitLedger - , ouroborosChainState = () + ledgerState = testInitLedger + , headerState = genesisHeaderState () } -- | Trivial test configuration with a single core node diff --git a/ouroboros-network/src/Ouroboros/Network/AnchoredFragment.hs b/ouroboros-network/src/Ouroboros/Network/AnchoredFragment.hs index 0412805b2e7..ab827a6d0ff 100644 --- a/ouroboros-network/src/Ouroboros/Network/AnchoredFragment.hs +++ b/ouroboros-network/src/Ouroboros/Network/AnchoredFragment.hs @@ -22,6 +22,7 @@ module Ouroboros.Network.AnchoredFragment ( anchorToBlockNo, anchorToHash, anchorIsGenesis, + anchorToTip, castAnchor, valid, @@ -222,6 +223,14 @@ anchorToHash :: Anchor block -> ChainHash block anchorToHash AnchorGenesis = GenesisHash anchorToHash (Anchor _s h _b) = BlockHash h +-- | Translate 'Anchor' to 'Tip' +-- +-- Right now this is in fact an isomorphism, but these two types are logically +-- independent. +anchorToTip :: (HeaderHash a ~ HeaderHash b) => Anchor a -> Tip b +anchorToTip AnchorGenesis = TipGenesis +anchorToTip (Anchor s h b) = Tip s h b + mkAnchoredFragment :: HasHeader block => Anchor block -> ChainFragment block -> AnchoredFragment block