{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE DeriveGeneric #-}
{-# LANGUAGE DerivingStrategies #-}
{-# LANGUAGE DuplicateRecordFields #-}
{-# LANGUAGE ExistentialQuantification #-}
{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GeneralizedNewtypeDeriving #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiWayIf #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE StandaloneDeriving #-}
{-# LANGUAGE TupleSections #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeFamilies #-}
{-# LANGUAGE UndecidableInstances #-}
{-# OPTIONS_GHC -fno-strictness #-}
module Ouroboros.Consensus.MiniProtocol.ChainSync.Client (
ChainDbView (..)
, ChainSyncClientException (..)
, ChainSyncClientResult (..)
, Consensus
, Our (..)
, Their (..)
, bracketChainSyncClient
, chainSyncClient
, defaultChainDbView
, InvalidBlockReason
, TraceChainSyncClientEvent (..)
) where
import Control.Monad
import Control.Monad.Except
import Control.Tracer
import Data.Kind (Type)
import Data.Map.Strict (Map)
import qualified Data.Map.Strict as Map
import Data.Proxy
import Data.Typeable
import Data.Word (Word64)
import GHC.Generics (Generic)
import GHC.Stack (HasCallStack)
import NoThunks.Class (unsafeNoThunks)
import Network.TypedProtocol.Pipelined
import Ouroboros.Network.AnchoredFragment (AnchoredFragment,
AnchoredSeq (..))
import qualified Ouroboros.Network.AnchoredFragment as AF
import qualified Ouroboros.Network.AnchoredSeq as AS
import Ouroboros.Network.Block (Tip, getTipBlockNo)
import Ouroboros.Network.Mux (ControlMessage (..), ControlMessageSTM)
import Ouroboros.Network.NodeToNode.Version (isPipeliningEnabled)
import Ouroboros.Network.PeerSelection.PeerMetric.Type
(HeaderMetricsTracer)
import Ouroboros.Network.Protocol.ChainSync.ClientPipelined
import Ouroboros.Network.Protocol.ChainSync.PipelineDecision
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Config
import Ouroboros.Consensus.Forecast
import Ouroboros.Consensus.HeaderStateHistory
(HeaderStateHistory (..), validateHeader)
import qualified Ouroboros.Consensus.HeaderStateHistory as HeaderStateHistory
import Ouroboros.Consensus.HeaderValidation hiding (validateHeader)
import Ouroboros.Consensus.Ledger.Extended
import Ouroboros.Consensus.Ledger.SupportsProtocol
import Ouroboros.Consensus.Node.NetworkProtocolVersion
import Ouroboros.Consensus.Protocol.Abstract
import Ouroboros.Consensus.Util
import Ouroboros.Consensus.Util.Assert (assertWithMsg)
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.STM (Fingerprint, Watcher (..),
WithFingerprint (..), withWatcher)
import Ouroboros.Consensus.Storage.ChainDB (ChainDB,
InvalidBlockReason)
import qualified Ouroboros.Consensus.Storage.ChainDB as ChainDB
type Consensus (client :: Type -> Type -> Type -> (Type -> Type) -> Type -> Type) blk m =
client (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
data ChainDbView m blk = ChainDbView {
ChainDbView m blk -> STM m (AnchoredFragment (Header blk))
getCurrentChain :: STM m (AnchoredFragment (Header blk))
, :: STM m (HeaderStateHistory blk)
, ChainDbView m blk
-> Point blk -> STM m (Maybe (ExtLedgerState blk))
getPastLedger :: Point blk -> STM m (Maybe (ExtLedgerState blk))
, ChainDbView m blk
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock :: STM m (WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk)))
}
defaultChainDbView ::
(IOLike m, LedgerSupportsProtocol blk)
=> ChainDB m blk -> ChainDbView m blk
defaultChainDbView :: ChainDB m blk -> ChainDbView m blk
defaultChainDbView ChainDB m blk
chainDB = ChainDbView :: forall (m :: * -> *) blk.
STM m (AnchoredFragment (Header blk))
-> STM m (HeaderStateHistory blk)
-> (Point blk -> STM m (Maybe (ExtLedgerState blk)))
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
-> ChainDbView m blk
ChainDbView {
$sel:getCurrentChain:ChainDbView :: STM m (AnchoredFragment (Header blk))
getCurrentChain = ChainDB m blk -> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) blk.
ChainDB m blk -> STM m (AnchoredFragment (Header blk))
ChainDB.getCurrentChain ChainDB m blk
chainDB
, $sel:getHeaderStateHistory:ChainDbView :: STM m (HeaderStateHistory blk)
getHeaderStateHistory = ChainDB m blk -> STM m (HeaderStateHistory blk)
forall (m :: * -> *) blk.
Monad (STM m) =>
ChainDB m blk -> STM m (HeaderStateHistory blk)
ChainDB.getHeaderStateHistory ChainDB m blk
chainDB
, $sel:getPastLedger:ChainDbView :: Point blk -> STM m (Maybe (ExtLedgerState blk))
getPastLedger = ChainDB m blk -> Point blk -> STM m (Maybe (ExtLedgerState blk))
forall (m :: * -> *) blk.
(Monad (STM m), LedgerSupportsProtocol blk) =>
ChainDB m blk -> Point blk -> STM m (Maybe (ExtLedgerState blk))
ChainDB.getPastLedger ChainDB m blk
chainDB
, $sel:getIsInvalidBlock:ChainDbView :: STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock = ChainDB m blk
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
forall (m :: * -> *) blk.
ChainDB m blk
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
ChainDB.getIsInvalidBlock ChainDB m blk
chainDB
}
newtype Their a = Their { Their a -> a
unTheir :: a }
deriving stock (Their a -> Their a -> Bool
(Their a -> Their a -> Bool)
-> (Their a -> Their a -> Bool) -> Eq (Their a)
forall a. Eq a => Their a -> Their a -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
/= :: Their a -> Their a -> Bool
$c/= :: forall a. Eq a => Their a -> Their a -> Bool
== :: Their a -> Their a -> Bool
$c== :: forall a. Eq a => Their a -> Their a -> Bool
Eq)
deriving newtype (Int -> Their a -> ShowS
[Their a] -> ShowS
Their a -> String
(Int -> Their a -> ShowS)
-> (Their a -> String) -> ([Their a] -> ShowS) -> Show (Their a)
forall a. Show a => Int -> Their a -> ShowS
forall a. Show a => [Their a] -> ShowS
forall a. Show a => Their a -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
showList :: [Their a] -> ShowS
$cshowList :: forall a. Show a => [Their a] -> ShowS
show :: Their a -> String
$cshow :: forall a. Show a => Their a -> String
showsPrec :: Int -> Their a -> ShowS
$cshowsPrec :: forall a. Show a => Int -> Their a -> ShowS
Show, Context -> Their a -> IO (Maybe ThunkInfo)
Proxy (Their a) -> String
(Context -> Their a -> IO (Maybe ThunkInfo))
-> (Context -> Their a -> IO (Maybe ThunkInfo))
-> (Proxy (Their a) -> String)
-> NoThunks (Their a)
forall a. NoThunks a => Context -> Their a -> IO (Maybe ThunkInfo)
forall a. NoThunks a => Proxy (Their a) -> String
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
showTypeOf :: Proxy (Their a) -> String
$cshowTypeOf :: forall a. NoThunks a => Proxy (Their a) -> String
wNoThunks :: Context -> Their a -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall a. NoThunks a => Context -> Their a -> IO (Maybe ThunkInfo)
noThunks :: Context -> Their a -> IO (Maybe ThunkInfo)
$cnoThunks :: forall a. NoThunks a => Context -> Their a -> IO (Maybe ThunkInfo)
NoThunks)
newtype Our a = Our { Our a -> a
unOur :: a }
deriving stock (Our a -> Our a -> Bool
(Our a -> Our a -> Bool) -> (Our a -> Our a -> Bool) -> Eq (Our a)
forall a. Eq a => Our a -> Our a -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
/= :: Our a -> Our a -> Bool
$c/= :: forall a. Eq a => Our a -> Our a -> Bool
== :: Our a -> Our a -> Bool
$c== :: forall a. Eq a => Our a -> Our a -> Bool
Eq)
deriving newtype (Int -> Our a -> ShowS
[Our a] -> ShowS
Our a -> String
(Int -> Our a -> ShowS)
-> (Our a -> String) -> ([Our a] -> ShowS) -> Show (Our a)
forall a. Show a => Int -> Our a -> ShowS
forall a. Show a => [Our a] -> ShowS
forall a. Show a => Our a -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
showList :: [Our a] -> ShowS
$cshowList :: forall a. Show a => [Our a] -> ShowS
show :: Our a -> String
$cshow :: forall a. Show a => Our a -> String
showsPrec :: Int -> Our a -> ShowS
$cshowsPrec :: forall a. Show a => Int -> Our a -> ShowS
Show, Context -> Our a -> IO (Maybe ThunkInfo)
Proxy (Our a) -> String
(Context -> Our a -> IO (Maybe ThunkInfo))
-> (Context -> Our a -> IO (Maybe ThunkInfo))
-> (Proxy (Our a) -> String)
-> NoThunks (Our a)
forall a. NoThunks a => Context -> Our a -> IO (Maybe ThunkInfo)
forall a. NoThunks a => Proxy (Our a) -> String
forall a.
(Context -> a -> IO (Maybe ThunkInfo))
-> (Context -> a -> IO (Maybe ThunkInfo))
-> (Proxy a -> String)
-> NoThunks a
showTypeOf :: Proxy (Our a) -> String
$cshowTypeOf :: forall a. NoThunks a => Proxy (Our a) -> String
wNoThunks :: Context -> Our a -> IO (Maybe ThunkInfo)
$cwNoThunks :: forall a. NoThunks a => Context -> Our a -> IO (Maybe ThunkInfo)
noThunks :: Context -> Our a -> IO (Maybe ThunkInfo)
$cnoThunks :: forall a. NoThunks a => Context -> Our a -> IO (Maybe ThunkInfo)
NoThunks)
bracketChainSyncClient
:: ( IOLike m
, Ord peer
, BlockSupportsProtocol blk
, LedgerSupportsProtocol blk
)
=> Tracer m (TraceChainSyncClientEvent blk)
-> ChainDbView m blk
-> StrictTVar m (Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> peer
-> NodeToNodeVersion
-> ( StrictTVar m (AnchoredFragment (Header blk))
-> m a
)
-> m a
bracketChainSyncClient :: Tracer m (TraceChainSyncClientEvent blk)
-> ChainDbView m blk
-> StrictTVar
m (Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> peer
-> NodeToNodeVersion
-> (StrictTVar m (AnchoredFragment (Header blk)) -> m a)
-> m a
bracketChainSyncClient Tracer m (TraceChainSyncClientEvent blk)
tracer ChainDbView { STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock :: STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
$sel:getIsInvalidBlock:ChainDbView :: forall (m :: * -> *) blk.
ChainDbView m blk
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock } StrictTVar
m (Map peer (StrictTVar m (AnchoredFragment (Header blk))))
varCandidates
peer
peer NodeToNodeVersion
version StrictTVar m (AnchoredFragment (Header blk)) -> m a
body =
m (StrictTVar m (AnchoredFragment (Header blk)))
-> (StrictTVar m (AnchoredFragment (Header blk)) -> m ())
-> (StrictTVar m (AnchoredFragment (Header blk)) -> m a)
-> m a
forall (m :: * -> *) a b c.
MonadThrow m =>
m a -> (a -> m b) -> (a -> m c) -> m c
bracket m (StrictTVar m (AnchoredFragment (Header blk)))
newCandidateVar StrictTVar m (AnchoredFragment (Header blk)) -> m ()
releaseCandidateVar
((StrictTVar m (AnchoredFragment (Header blk)) -> m a) -> m a)
-> (StrictTVar m (AnchoredFragment (Header blk)) -> m a) -> m a
forall a b. (a -> b) -> a -> b
$ \StrictTVar m (AnchoredFragment (Header blk))
varCandidate ->
String
-> Watcher
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
Fingerprint
-> m a
-> m a
forall (m :: * -> *) a fp r.
(IOLike m, Eq fp, HasCallStack) =>
String -> Watcher m a fp -> m r -> m r
withWatcher
String
"ChainSync.Client.rejectInvalidBlocks"
(StrictTVar m (AnchoredFragment (Header blk))
-> Watcher
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
Fingerprint
invalidBlockWatcher StrictTVar m (AnchoredFragment (Header blk))
varCandidate)
(m a -> m a) -> m a -> m a
forall a b. (a -> b) -> a -> b
$ StrictTVar m (AnchoredFragment (Header blk)) -> m a
body StrictTVar m (AnchoredFragment (Header blk))
varCandidate
where
newCandidateVar :: m (StrictTVar m (AnchoredFragment (Header blk)))
newCandidateVar = do
StrictTVar m (AnchoredFragment (Header blk))
varCandidate <- AnchoredFragment (Header blk)
-> m (StrictTVar m (AnchoredFragment (Header blk)))
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (AnchoredFragment (Header blk)
-> m (StrictTVar m (AnchoredFragment (Header blk))))
-> AnchoredFragment (Header blk)
-> m (StrictTVar m (AnchoredFragment (Header blk)))
forall a b. (a -> b) -> a -> b
$ Anchor (Header blk) -> AnchoredFragment (Header blk)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty Anchor (Header blk)
forall block. Anchor block
AF.AnchorGenesis
STM m () -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar
m (Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> (Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar
m (Map peer (StrictTVar m (AnchoredFragment (Header blk))))
varCandidates ((Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> STM m ())
-> (Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> STM m ()
forall a b. (a -> b) -> a -> b
$ peer
-> StrictTVar m (AnchoredFragment (Header blk))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk)))
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert peer
peer StrictTVar m (AnchoredFragment (Header blk))
varCandidate
StrictTVar m (AnchoredFragment (Header blk))
-> m (StrictTVar m (AnchoredFragment (Header blk)))
forall (m :: * -> *) a. Monad m => a -> m a
return StrictTVar m (AnchoredFragment (Header blk))
varCandidate
releaseCandidateVar :: StrictTVar m (AnchoredFragment (Header blk)) -> m ()
releaseCandidateVar StrictTVar m (AnchoredFragment (Header blk))
_ = do
STM m () -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar
m (Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> (Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar
m (Map peer (StrictTVar m (AnchoredFragment (Header blk))))
varCandidates ((Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> STM m ())
-> (Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk))))
-> STM m ()
forall a b. (a -> b) -> a -> b
$ peer
-> Map peer (StrictTVar m (AnchoredFragment (Header blk)))
-> Map peer (StrictTVar m (AnchoredFragment (Header blk)))
forall k a. Ord k => k -> Map k a -> Map k a
Map.delete peer
peer
invalidBlockWatcher :: StrictTVar m (AnchoredFragment (Header blk))
-> Watcher
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
Fingerprint
invalidBlockWatcher StrictTVar m (AnchoredFragment (Header blk))
varCandidate =
Tracer m (TraceChainSyncClientEvent blk)
-> NodeToNodeVersion
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
-> STM m (AnchoredFragment (Header blk))
-> Watcher
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
Fingerprint
forall (m :: * -> *) blk.
(IOLike m, BlockSupportsProtocol blk,
LedgerSupportsProtocol blk) =>
Tracer m (TraceChainSyncClientEvent blk)
-> NodeToNodeVersion
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
-> STM m (AnchoredFragment (Header blk))
-> Watcher
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
Fingerprint
invalidBlockRejector
Tracer m (TraceChainSyncClientEvent blk)
tracer
NodeToNodeVersion
version
STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock
(StrictTVar m (AnchoredFragment (Header blk))
-> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (AnchoredFragment (Header blk))
varCandidate)
data UnknownIntersectionState blk = UnknownIntersectionState
{ UnknownIntersectionState blk -> AnchoredFragment (Header blk)
ourFrag :: !(AnchoredFragment (Header blk))
, :: !(HeaderStateHistory blk)
}
deriving ((forall x.
UnknownIntersectionState blk
-> Rep (UnknownIntersectionState blk) x)
-> (forall x.
Rep (UnknownIntersectionState blk) x
-> UnknownIntersectionState blk)
-> Generic (UnknownIntersectionState blk)
forall x.
Rep (UnknownIntersectionState blk) x
-> UnknownIntersectionState blk
forall x.
UnknownIntersectionState blk
-> Rep (UnknownIntersectionState blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (UnknownIntersectionState blk) x
-> UnknownIntersectionState blk
forall blk x.
UnknownIntersectionState blk
-> Rep (UnknownIntersectionState blk) x
$cto :: forall blk x.
Rep (UnknownIntersectionState blk) x
-> UnknownIntersectionState blk
$cfrom :: forall blk x.
UnknownIntersectionState blk
-> Rep (UnknownIntersectionState blk) x
Generic)
instance ( LedgerSupportsProtocol blk
) => NoThunks (UnknownIntersectionState blk) where
showTypeOf :: Proxy (UnknownIntersectionState blk) -> String
showTypeOf Proxy (UnknownIntersectionState blk)
_ = TypeRep -> String
forall a. Show a => a -> String
show (TypeRep -> String) -> TypeRep -> String
forall a b. (a -> b) -> a -> b
$ Proxy (UnknownIntersectionState blk) -> TypeRep
forall k (proxy :: k -> *) (a :: k).
Typeable a =>
proxy a -> TypeRep
typeRep (Proxy (UnknownIntersectionState blk)
forall k (t :: k). Proxy t
Proxy @(UnknownIntersectionState blk))
data KnownIntersectionState blk = KnownIntersectionState
{ KnownIntersectionState blk -> AnchoredFragment (Header blk)
theirFrag :: !(AnchoredFragment (Header blk))
, :: !(HeaderStateHistory blk)
, KnownIntersectionState blk -> AnchoredFragment (Header blk)
ourFrag :: !(AnchoredFragment (Header blk))
, KnownIntersectionState blk -> Point blk
mostRecentIntersection :: !(Point blk)
}
deriving ((forall x.
KnownIntersectionState blk -> Rep (KnownIntersectionState blk) x)
-> (forall x.
Rep (KnownIntersectionState blk) x -> KnownIntersectionState blk)
-> Generic (KnownIntersectionState blk)
forall x.
Rep (KnownIntersectionState blk) x -> KnownIntersectionState blk
forall x.
KnownIntersectionState blk -> Rep (KnownIntersectionState blk) x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
forall blk x.
Rep (KnownIntersectionState blk) x -> KnownIntersectionState blk
forall blk x.
KnownIntersectionState blk -> Rep (KnownIntersectionState blk) x
$cto :: forall blk x.
Rep (KnownIntersectionState blk) x -> KnownIntersectionState blk
$cfrom :: forall blk x.
KnownIntersectionState blk -> Rep (KnownIntersectionState blk) x
Generic)
instance ( LedgerSupportsProtocol blk
) => NoThunks (KnownIntersectionState blk) where
showTypeOf :: Proxy (KnownIntersectionState blk) -> String
showTypeOf Proxy (KnownIntersectionState blk)
_ = TypeRep -> String
forall a. Show a => a -> String
show (TypeRep -> String) -> TypeRep -> String
forall a b. (a -> b) -> a -> b
$ Proxy (KnownIntersectionState blk) -> TypeRep
forall k (proxy :: k -> *) (a :: k).
Typeable a =>
proxy a -> TypeRep
typeRep (Proxy (KnownIntersectionState blk)
forall k (t :: k). Proxy t
Proxy @(KnownIntersectionState blk))
checkKnownIntersectionInvariants
:: ( HasHeader blk
, HasHeader (Header blk)
, HasAnnTip blk
, ConsensusProtocol (BlockProtocol blk)
)
=> ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk
-> Either String ()
checkKnownIntersectionInvariants :: ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> Either String ()
checkKnownIntersectionInvariants ConsensusConfig (BlockProtocol blk)
cfg KnownIntersectionState
{ AnchoredFragment (Header blk)
ourFrag :: AnchoredFragment (Header blk)
$sel:ourFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
ourFrag
, AnchoredFragment (Header blk)
theirFrag :: AnchoredFragment (Header blk)
$sel:theirFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
theirFrag
, HeaderStateHistory blk
theirHeaderStateHistory :: HeaderStateHistory blk
$sel:theirHeaderStateHistory:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> HeaderStateHistory blk
theirHeaderStateHistory
, Point blk
mostRecentIntersection :: Point blk
$sel:mostRecentIntersection:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> Point blk
mostRecentIntersection
}
| let HeaderStateHistory AnchoredSeq (WithOrigin SlotNo) (HeaderState blk) (HeaderState blk)
snapshots = HeaderStateHistory blk
theirHeaderStateHistory
historyTips :: [WithOrigin (AnnTip blk)]
historyTips = HeaderState blk -> WithOrigin (AnnTip blk)
forall blk. HeaderState blk -> WithOrigin (AnnTip blk)
headerStateTip (HeaderState blk -> WithOrigin (AnnTip blk))
-> [HeaderState blk] -> [WithOrigin (AnnTip blk)]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> AnchoredSeq (WithOrigin SlotNo) (HeaderState blk) (HeaderState blk)
-> [HeaderState blk]
forall v a b. AnchoredSeq v a b -> [b]
AS.toOldestFirst AnchoredSeq (WithOrigin SlotNo) (HeaderState blk) (HeaderState blk)
snapshots
fragmentTips :: [WithOrigin (AnnTip blk)]
fragmentTips = AnnTip blk -> WithOrigin (AnnTip blk)
forall t. t -> WithOrigin t
NotOrigin (AnnTip blk -> WithOrigin (AnnTip blk))
-> (Header blk -> AnnTip blk)
-> Header blk
-> WithOrigin (AnnTip blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Header blk -> AnnTip blk
forall blk.
(HasHeader (Header blk), HasAnnTip blk) =>
Header blk -> AnnTip blk
getAnnTip (Header blk -> WithOrigin (AnnTip blk))
-> [Header blk] -> [WithOrigin (AnnTip blk)]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> AnchoredFragment (Header blk) -> [Header blk]
forall v a b. AnchoredSeq v a b -> [b]
AF.toOldestFirst AnchoredFragment (Header blk)
theirFrag
historyAnchorPoint :: Point blk
historyAnchorPoint =
WithOrigin (RealPoint blk) -> Point blk
forall blk. WithOrigin (RealPoint blk) -> Point blk
withOriginRealPointToPoint (WithOrigin (RealPoint blk) -> Point blk)
-> WithOrigin (RealPoint blk) -> Point blk
forall a b. (a -> b) -> a -> b
$
AnnTip blk -> RealPoint blk
forall blk. HasAnnTip blk => AnnTip blk -> RealPoint blk
annTipRealPoint (AnnTip blk -> RealPoint blk)
-> WithOrigin (AnnTip blk) -> WithOrigin (RealPoint blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> HeaderState blk -> WithOrigin (AnnTip blk)
forall blk. HeaderState blk -> WithOrigin (AnnTip blk)
headerStateTip (AnchoredSeq (WithOrigin SlotNo) (HeaderState blk) (HeaderState blk)
-> HeaderState blk
forall v a b. AnchoredSeq v a b -> a
AS.anchor AnchoredSeq (WithOrigin SlotNo) (HeaderState blk) (HeaderState blk)
snapshots)
fragmentAnchorPoint :: Point blk
fragmentAnchorPoint = Point (Header blk) -> Point blk
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Point (Header blk) -> Point blk
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
theirFrag
, [WithOrigin (AnnTip blk)]
historyTips [WithOrigin (AnnTip blk)] -> [WithOrigin (AnnTip blk)] -> Bool
forall a. Eq a => a -> a -> Bool
/= [WithOrigin (AnnTip blk)]
fragmentTips Bool -> Bool -> Bool
|| Point blk
historyAnchorPoint Point blk -> Point blk -> Bool
forall a. Eq a => a -> a -> Bool
/= Point blk
fragmentAnchorPoint
= String -> Either String ()
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (String -> Either String ()) -> String -> Either String ()
forall a b. (a -> b) -> a -> b
$ Context -> String
unwords
[ String
"The tips in theirHeaderStateHistory didn't match the headers in theirFrag:"
, [WithOrigin (AnnTip blk)] -> String
forall a. Show a => a -> String
show [WithOrigin (AnnTip blk)]
historyTips
, String
"vs"
, [WithOrigin (AnnTip blk)] -> String
forall a. Show a => a -> String
show [WithOrigin (AnnTip blk)]
fragmentTips
, String
"with anchors"
, Point blk -> String
forall a. Show a => a -> String
show Point blk
historyAnchorPoint
, String
"vs"
, Point blk -> String
forall a. Show a => a -> String
show Point blk
fragmentAnchorPoint
]
| let nbHeaders :: Int
nbHeaders = AnchoredFragment (Header blk) -> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredFragment (Header blk)
ourFrag
ourAnchorPoint :: Point (Header blk)
ourAnchorPoint = AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
ourFrag
, Int
nbHeaders Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Word64 -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral Word64
k
, Point (Header blk)
ourAnchorPoint Point (Header blk) -> Point (Header blk) -> Bool
forall a. Eq a => a -> a -> Bool
/= Point (Header blk)
forall block. Point block
GenesisPoint
= String -> Either String ()
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (String -> Either String ()) -> String -> Either String ()
forall a b. (a -> b) -> a -> b
$ Context -> String
unwords
[ String
"ourFrag contains fewer than k headers and not close to genesis:"
, Int -> String
forall a. Show a => a -> String
show Int
nbHeaders
, String
"vs"
, Word64 -> String
forall a. Show a => a -> String
show Word64
k
, String
"with anchor"
, Point (Header blk) -> String
forall a. Show a => a -> String
show Point (Header blk)
ourAnchorPoint
]
| let ourFragAnchor :: Point (Header blk)
ourFragAnchor = AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
ourFrag
theirFragAnchor :: Point (Header blk)
theirFragAnchor = AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
theirFrag
, Point (Header blk)
ourFragAnchor Point (Header blk) -> Point (Header blk) -> Bool
forall a. Eq a => a -> a -> Bool
/= Point (Header blk)
theirFragAnchor
= String -> Either String ()
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (String -> Either String ()) -> String -> Either String ()
forall a b. (a -> b) -> a -> b
$ Context -> String
unwords
[ String
"ourFrag and theirFrag have different anchor points:"
, Point (Header blk) -> String
forall a. Show a => a -> String
show Point (Header blk)
ourFragAnchor
, String
"vs"
, Point (Header blk) -> String
forall a. Show a => a -> String
show Point (Header blk)
theirFragAnchor
]
| let actualMostRecentIntersection :: Maybe (Point blk)
actualMostRecentIntersection =
Point (Header blk) -> Point blk
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> Maybe (Point (Header blk)) -> Maybe (Point blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Maybe (Point (Header blk))
forall block1 block2.
(HasHeader block1, HasHeader block2,
HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> AnchoredFragment block2 -> Maybe (Point block1)
AF.intersectionPoint AnchoredFragment (Header blk)
theirFrag AnchoredFragment (Header blk)
ourFrag
, Point blk -> Maybe (Point blk)
forall a. a -> Maybe a
Just Point blk
mostRecentIntersection Maybe (Point blk) -> Maybe (Point blk) -> Bool
forall a. Eq a => a -> a -> Bool
/= Maybe (Point blk)
actualMostRecentIntersection
= String -> Either String ()
forall e (m :: * -> *) a. MonadError e m => e -> m a
throwError (String -> Either String ()) -> String -> Either String ()
forall a b. (a -> b) -> a -> b
$ Context -> String
unwords
[ String
"mostRecentIntersection not the most recent intersection"
, String
"of theirFrag and ourFrag:"
, Point blk -> String
forall a. Show a => a -> String
show Point blk
mostRecentIntersection
, String
"vs"
, Maybe (Point blk) -> String
forall a. Show a => a -> String
show Maybe (Point blk)
actualMostRecentIntersection
]
| Bool
otherwise
= () -> Either String ()
forall (m :: * -> *) a. Monad m => a -> m a
return ()
where
SecurityParam Word64
k = ConsensusConfig (BlockProtocol blk) -> SecurityParam
forall p. ConsensusProtocol p => ConsensusConfig p -> SecurityParam
protocolSecurityParam ConsensusConfig (BlockProtocol blk)
cfg
assertKnownIntersectionInvariants
:: ( HasHeader blk
, HasHeader (Header blk)
, HasAnnTip blk
, ConsensusProtocol (BlockProtocol blk)
, HasCallStack
)
=> ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk
-> KnownIntersectionState blk
assertKnownIntersectionInvariants :: ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
assertKnownIntersectionInvariants ConsensusConfig (BlockProtocol blk)
cfg KnownIntersectionState blk
kis =
Either String ()
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall a. HasCallStack => Either String () -> a -> a
assertWithMsg (ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> Either String ()
forall blk.
(HasHeader blk, HasHeader (Header blk), HasAnnTip blk,
ConsensusProtocol (BlockProtocol blk)) =>
ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> Either String ()
checkKnownIntersectionInvariants ConsensusConfig (BlockProtocol blk)
cfg KnownIntersectionState blk
kis) KnownIntersectionState blk
kis
chainSyncClient
:: forall m blk.
( IOLike m
, LedgerSupportsProtocol blk
)
=> MkPipelineDecision
-> Tracer m (TraceChainSyncClientEvent blk)
-> TopLevelConfig blk
-> ChainDbView m blk
-> NodeToNodeVersion
-> ControlMessageSTM m
-> HeaderMetricsTracer m
-> StrictTVar m (AnchoredFragment (Header blk))
-> Consensus ChainSyncClientPipelined blk m
chainSyncClient :: MkPipelineDecision
-> Tracer m (TraceChainSyncClientEvent blk)
-> TopLevelConfig blk
-> ChainDbView m blk
-> NodeToNodeVersion
-> ControlMessageSTM m
-> HeaderMetricsTracer m
-> StrictTVar m (AnchoredFragment (Header blk))
-> Consensus ChainSyncClientPipelined blk m
chainSyncClient MkPipelineDecision
mkPipelineDecision0 Tracer m (TraceChainSyncClientEvent blk)
tracer TopLevelConfig blk
cfg
ChainDbView
{ STM m (AnchoredFragment (Header blk))
getCurrentChain :: STM m (AnchoredFragment (Header blk))
$sel:getCurrentChain:ChainDbView :: forall (m :: * -> *) blk.
ChainDbView m blk -> STM m (AnchoredFragment (Header blk))
getCurrentChain
, STM m (HeaderStateHistory blk)
getHeaderStateHistory :: STM m (HeaderStateHistory blk)
$sel:getHeaderStateHistory:ChainDbView :: forall (m :: * -> *) blk.
ChainDbView m blk -> STM m (HeaderStateHistory blk)
getHeaderStateHistory
, Point blk -> STM m (Maybe (ExtLedgerState blk))
getPastLedger :: Point blk -> STM m (Maybe (ExtLedgerState blk))
$sel:getPastLedger:ChainDbView :: forall (m :: * -> *) blk.
ChainDbView m blk
-> Point blk -> STM m (Maybe (ExtLedgerState blk))
getPastLedger
, STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock :: STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
$sel:getIsInvalidBlock:ChainDbView :: forall (m :: * -> *) blk.
ChainDbView m blk
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock
}
NodeToNodeVersion
version
ControlMessageSTM m
controlMessageSTM
HeaderMetricsTracer m
headerMetricsTracer
StrictTVar m (AnchoredFragment (Header blk))
varCandidate = m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> Consensus ChainSyncClientPipelined blk m
forall header point tip (m :: * -> *) a.
m (ClientPipelinedStIdle 'Z header point tip m a)
-> ChainSyncClientPipelined header point tip m a
ChainSyncClientPipelined (m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> Consensus ChainSyncClientPipelined blk m)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> Consensus ChainSyncClientPipelined blk m
forall a b. (a -> b) -> a -> b
$
()
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState () (Stateful m blk () (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ Stateful m blk () (ClientPipelinedStIdle 'Z)
initialise
where
initialise :: Stateful m blk () (ClientPipelinedStIdle 'Z)
initialise :: Stateful m blk () (ClientPipelinedStIdle 'Z)
initialise = (Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult)
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
findIntersection (Point blk
-> Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
forall blk.
BlockSupportsProtocol blk =>
Point blk
-> Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
ForkTooDeep Point blk
forall block. Point block
GenesisPoint)
findIntersection
:: (Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult)
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
findIntersection :: (Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult)
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
findIntersection Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
mkResult = (()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
(s -> m (Consensus st blk m)) -> Stateful m blk s st
Stateful ((()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful m blk () (ClientPipelinedStIdle 'Z))
-> (()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
forall a b. (a -> b) -> a -> b
$ \() -> do
(AnchoredFragment (Header blk)
ourFrag, HeaderStateHistory blk
ourHeaderStateHistory) <- STM m (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> m (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> m (AnchoredFragment (Header blk), HeaderStateHistory blk))
-> STM m (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> m (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall a b. (a -> b) -> a -> b
$ (,)
(AnchoredFragment (Header blk)
-> HeaderStateHistory blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk))
-> STM m (AnchoredFragment (Header blk))
-> STM
m
(HeaderStateHistory blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (AnchoredFragment (Header blk))
getCurrentChain
STM
m
(HeaderStateHistory blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk))
-> STM m (HeaderStateHistory blk)
-> STM m (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> STM m (HeaderStateHistory blk)
getHeaderStateHistory
let maxOffset :: Word64
maxOffset = Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (AnchoredFragment (Header blk) -> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredFragment (Header blk)
ourFrag)
points :: [Point blk]
points = (Point (Header blk) -> Point blk)
-> [Point (Header blk)] -> [Point blk]
forall a b. (a -> b) -> [a] -> [b]
map Point (Header blk) -> Point blk
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint
([Point (Header blk)] -> [Point blk])
-> [Point (Header blk)] -> [Point blk]
forall a b. (a -> b) -> a -> b
$ [Int] -> AnchoredFragment (Header blk) -> [Point (Header blk)]
forall block.
HasHeader block =>
[Int] -> AnchoredFragment block -> [Point block]
AF.selectPoints
((Word64 -> Int) -> [Word64] -> [Int]
forall a b. (a -> b) -> [a] -> [b]
map Word64 -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> [Word64]
offsets Word64
maxOffset))
AnchoredFragment (Header blk)
ourFrag
uis :: UnknownIntersectionState blk
uis = UnknownIntersectionState :: forall blk.
AnchoredFragment (Header blk)
-> HeaderStateHistory blk -> UnknownIntersectionState blk
UnknownIntersectionState {
$sel:ourFrag:UnknownIntersectionState :: AnchoredFragment (Header blk)
ourFrag = AnchoredFragment (Header blk)
ourFrag
, $sel:ourHeaderStateHistory:UnknownIntersectionState :: HeaderStateHistory blk
ourHeaderStateHistory = HeaderStateHistory blk
ourHeaderStateHistory
}
ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) a. Monad m => a -> m a
return (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ [Point blk]
-> ClientPipelinedStIntersect
(Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall point header tip (m :: * -> *) a.
[point]
-> ClientPipelinedStIntersect header point tip m a
-> ClientPipelinedStIdle 'Z header point tip m a
SendMsgFindIntersect [Point blk]
points (ClientPipelinedStIntersect
(Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> ClientPipelinedStIntersect
(Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall a b. (a -> b) -> a -> b
$ ClientPipelinedStIntersect :: forall header point tip (m :: * -> *) a.
(point -> tip -> m (ClientPipelinedStIdle 'Z header point tip m a))
-> (tip -> m (ClientPipelinedStIdle 'Z header point tip m a))
-> ClientPipelinedStIntersect header point tip m a
ClientPipelinedStIntersect
{ recvMsgIntersectFound :: Point blk
-> Tip blk
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
recvMsgIntersectFound = \Point blk
i Tip blk
theirTip' ->
UnknownIntersectionState blk
-> Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState UnknownIntersectionState blk
uis (Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$
Point blk
-> Their (Tip blk)
-> Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z)
intersectFound (Point blk -> Point blk
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
i) (Tip blk -> Their (Tip blk)
forall a. a -> Their a
Their Tip blk
theirTip')
, recvMsgIntersectNotFound :: Tip blk
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
recvMsgIntersectNotFound = \Tip blk
theirTip' ->
ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
terminate (ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$
Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
mkResult
(AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain AnchoredFragment (Header blk)
ourFrag)
(Tip blk -> Their (Tip blk)
forall a. a -> Their a
Their Tip blk
theirTip')
}
intersectFound :: Point blk
-> Their (Tip blk)
-> Stateful m blk
(UnknownIntersectionState blk)
(ClientPipelinedStIdle 'Z)
intersectFound :: Point blk
-> Their (Tip blk)
-> Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z)
intersectFound Point blk
intersection Their (Tip blk)
theirTip
= (UnknownIntersectionState blk
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
(s -> m (Consensus st blk m)) -> Stateful m blk s st
Stateful ((UnknownIntersectionState blk
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z))
-> (UnknownIntersectionState blk
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful
m blk (UnknownIntersectionState blk) (ClientPipelinedStIdle 'Z)
forall a b. (a -> b) -> a -> b
$ \UnknownIntersectionState
{ AnchoredFragment (Header blk)
ourFrag :: AnchoredFragment (Header blk)
$sel:ourFrag:UnknownIntersectionState :: forall blk.
UnknownIntersectionState blk -> AnchoredFragment (Header blk)
ourFrag
, HeaderStateHistory blk
ourHeaderStateHistory :: HeaderStateHistory blk
$sel:ourHeaderStateHistory:UnknownIntersectionState :: forall blk. UnknownIntersectionState blk -> HeaderStateHistory blk
ourHeaderStateHistory
} -> do
Tracer m (TraceChainSyncClientEvent blk)
-> TraceChainSyncClientEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceChainSyncClientEvent blk)
tracer (TraceChainSyncClientEvent blk -> m ())
-> TraceChainSyncClientEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$
Point blk
-> Our (Tip blk)
-> Their (Tip blk)
-> TraceChainSyncClientEvent blk
forall blk.
Point blk
-> Our (Tip blk)
-> Their (Tip blk)
-> TraceChainSyncClientEvent blk
TraceFoundIntersection Point blk
intersection (AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain AnchoredFragment (Header blk)
ourFrag) Their (Tip blk)
theirTip
m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a. m a -> m a
traceException (m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ do
(AnchoredFragment (Header blk)
theirFrag, HeaderStateHistory blk
theirHeaderStateHistory) <- do
case Point blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall blk.
(BlockSupportsProtocol blk, HasAnnTip blk) =>
Point blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
attemptRollback Point blk
intersection (AnchoredFragment (Header blk)
ourFrag, HeaderStateHistory blk
ourHeaderStateHistory) of
Just (AnchoredFragment (Header blk)
c, HeaderStateHistory blk
d) -> (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> m (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall (m :: * -> *) a. Monad m => a -> m a
return (AnchoredFragment (Header blk)
c, HeaderStateHistory blk
d)
Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
Nothing -> ChainSyncClientException
-> m (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall (m' :: * -> *) x'.
MonadThrow m' =>
ChainSyncClientException -> m' x'
disconnect (ChainSyncClientException
-> m (AnchoredFragment (Header blk), HeaderStateHistory blk))
-> ChainSyncClientException
-> m (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall a b. (a -> b) -> a -> b
$
Point blk
-> Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientException
forall blk.
BlockSupportsProtocol blk =>
Point blk
-> Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientException
InvalidIntersection
Point blk
intersection
(AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain AnchoredFragment (Header blk)
ourFrag)
Their (Tip blk)
theirTip
STM m () -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar m (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (AnchoredFragment (Header blk))
varCandidate AnchoredFragment (Header blk)
theirFrag
let kis :: KnownIntersectionState blk
kis = ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall blk.
(HasHeader blk, HasHeader (Header blk), HasAnnTip blk,
ConsensusProtocol (BlockProtocol blk), HasCallStack) =>
ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
assertKnownIntersectionInvariants (TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
forall blk.
TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
configConsensus TopLevelConfig blk
cfg) (KnownIntersectionState blk -> KnownIntersectionState blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall a b. (a -> b) -> a -> b
$
KnownIntersectionState :: forall blk.
AnchoredFragment (Header blk)
-> HeaderStateHistory blk
-> AnchoredFragment (Header blk)
-> Point blk
-> KnownIntersectionState blk
KnownIntersectionState
{ $sel:theirFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
theirFrag = AnchoredFragment (Header blk)
theirFrag
, $sel:theirHeaderStateHistory:KnownIntersectionState :: HeaderStateHistory blk
theirHeaderStateHistory = HeaderStateHistory blk
theirHeaderStateHistory
, $sel:ourFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
ourFrag = AnchoredFragment (Header blk)
ourFrag
, $sel:mostRecentIntersection:KnownIntersectionState :: Point blk
mostRecentIntersection = Point blk
intersection
}
KnownIntersectionState blk
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState KnownIntersectionState blk
kis (Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ MkPipelineDecision
-> Nat 'Z
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle 'Z)
forall (n :: N).
MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
nextStep MkPipelineDecision
mkPipelineDecision0 Nat 'Z
forall (n :: N). ('Z ~ n) => Nat n
Zero Their (Tip blk)
theirTip
intersectsWithCurrentChain
:: KnownIntersectionState blk
-> STM m (Maybe (KnownIntersectionState blk))
intersectsWithCurrentChain :: KnownIntersectionState blk
-> STM m (Maybe (KnownIntersectionState blk))
intersectsWithCurrentChain kis :: KnownIntersectionState blk
kis@KnownIntersectionState
{ AnchoredFragment (Header blk)
theirFrag :: AnchoredFragment (Header blk)
$sel:theirFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
theirFrag
, HeaderStateHistory blk
theirHeaderStateHistory :: HeaderStateHistory blk
$sel:theirHeaderStateHistory:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> HeaderStateHistory blk
theirHeaderStateHistory
, AnchoredFragment (Header blk)
ourFrag :: AnchoredFragment (Header blk)
$sel:ourFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
ourFrag
} = do
AnchoredFragment (Header blk)
ourFrag' <- STM m (AnchoredFragment (Header blk))
getCurrentChain
if
| AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint AnchoredFragment (Header blk)
ourFrag Point (Header blk) -> Point (Header blk) -> Bool
forall a. Eq a => a -> a -> Bool
== AnchoredFragment (Header blk) -> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint AnchoredFragment (Header blk)
ourFrag' ->
Maybe (KnownIntersectionState blk)
-> STM m (Maybe (KnownIntersectionState blk))
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe (KnownIntersectionState blk)
-> STM m (Maybe (KnownIntersectionState blk)))
-> Maybe (KnownIntersectionState blk)
-> STM m (Maybe (KnownIntersectionState blk))
forall a b. (a -> b) -> a -> b
$ KnownIntersectionState blk -> Maybe (KnownIntersectionState blk)
forall a. a -> Maybe a
Just KnownIntersectionState blk
kis
| Just Point (Header blk)
intersection <- AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Maybe (Point (Header blk))
forall block1 block2.
(HasHeader block1, HasHeader block2,
HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> AnchoredFragment block2 -> Maybe (Point block1)
AF.intersectionPoint AnchoredFragment (Header blk)
ourFrag' AnchoredFragment (Header blk)
theirFrag ->
case AnchoredFragment (Header blk)
-> Point (Header blk)
-> Maybe
(AnchoredFragment (Header blk), AnchoredFragment (Header blk))
forall block1 block2.
(HasHeader block1, HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> Point block2
-> Maybe (AnchoredFragment block1, AnchoredFragment block1)
AF.splitAfterPoint AnchoredFragment (Header blk)
theirFrag (AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredFragment (Header blk)
ourFrag') of
Maybe
(AnchoredFragment (Header blk), AnchoredFragment (Header blk))
Nothing -> String -> STM m (Maybe (KnownIntersectionState blk))
forall a. HasCallStack => String -> a
error
String
"anchor point must be on candidate fragment if they intersect"
Just (AnchoredFragment (Header blk)
_, AnchoredFragment (Header blk)
trimmedCandidateFrag) -> Maybe (KnownIntersectionState blk)
-> STM m (Maybe (KnownIntersectionState blk))
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe (KnownIntersectionState blk)
-> STM m (Maybe (KnownIntersectionState blk)))
-> Maybe (KnownIntersectionState blk)
-> STM m (Maybe (KnownIntersectionState blk))
forall a b. (a -> b) -> a -> b
$ KnownIntersectionState blk -> Maybe (KnownIntersectionState blk)
forall a. a -> Maybe a
Just (KnownIntersectionState blk -> Maybe (KnownIntersectionState blk))
-> KnownIntersectionState blk -> Maybe (KnownIntersectionState blk)
forall a b. (a -> b) -> a -> b
$
ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall blk.
(HasHeader blk, HasHeader (Header blk), HasAnnTip blk,
ConsensusProtocol (BlockProtocol blk), HasCallStack) =>
ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
assertKnownIntersectionInvariants (TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
forall blk.
TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
configConsensus TopLevelConfig blk
cfg) (KnownIntersectionState blk -> KnownIntersectionState blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall a b. (a -> b) -> a -> b
$
KnownIntersectionState :: forall blk.
AnchoredFragment (Header blk)
-> HeaderStateHistory blk
-> AnchoredFragment (Header blk)
-> Point blk
-> KnownIntersectionState blk
KnownIntersectionState {
$sel:ourFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
ourFrag = AnchoredFragment (Header blk)
ourFrag'
, $sel:theirFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
theirFrag = AnchoredFragment (Header blk)
trimmedCandidateFrag
, $sel:theirHeaderStateHistory:KnownIntersectionState :: HeaderStateHistory blk
theirHeaderStateHistory = HeaderStateHistory blk
trimmedHeaderStateHistory'
, $sel:mostRecentIntersection:KnownIntersectionState :: Point blk
mostRecentIntersection = Point (Header blk) -> Point blk
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point (Header blk)
intersection
}
where
trimmedHeaderStateHistory' :: HeaderStateHistory blk
trimmedHeaderStateHistory' =
Int -> HeaderStateHistory blk -> HeaderStateHistory blk
forall blk. Int -> HeaderStateHistory blk -> HeaderStateHistory blk
HeaderStateHistory.trim
(AnchoredFragment (Header blk) -> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredFragment (Header blk)
trimmedCandidateFrag)
HeaderStateHistory blk
theirHeaderStateHistory
| Bool
otherwise ->
Maybe (KnownIntersectionState blk)
-> STM m (Maybe (KnownIntersectionState blk))
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (KnownIntersectionState blk)
forall a. Maybe a
Nothing
nextStep :: MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> Stateful m blk
(KnownIntersectionState blk)
(ClientPipelinedStIdle n)
nextStep :: MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
nextStep MkPipelineDecision
mkPipelineDecision Nat n
n Their (Tip blk)
theirTip = (KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
(s -> m (Consensus st blk m)) -> Stateful m blk s st
Stateful ((KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n))
-> (KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall a b. (a -> b) -> a -> b
$ \KnownIntersectionState blk
kis -> do
ControlMessageSTM m -> m ControlMessage
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically ControlMessageSTM m
controlMessageSTM m ControlMessage
-> (ControlMessage
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
ControlMessage
Terminate ->
Nat n
-> ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (n :: N).
Nat n
-> ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m)
terminateAfterDrain Nat n
n (ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ ChainSyncClientResult
AskedToTerminate
ControlMessage
_continue -> do
Maybe (KnownIntersectionState blk)
mKis' <- STM m (Maybe (KnownIntersectionState blk))
-> m (Maybe (KnownIntersectionState blk))
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Maybe (KnownIntersectionState blk))
-> m (Maybe (KnownIntersectionState blk)))
-> STM m (Maybe (KnownIntersectionState blk))
-> m (Maybe (KnownIntersectionState blk))
forall a b. (a -> b) -> a -> b
$ KnownIntersectionState blk
-> STM m (Maybe (KnownIntersectionState blk))
intersectsWithCurrentChain KnownIntersectionState blk
kis
case Maybe (KnownIntersectionState blk)
mKis' of
Just kis' :: KnownIntersectionState blk
kis'@KnownIntersectionState { AnchoredFragment (Header blk)
theirFrag :: AnchoredFragment (Header blk)
$sel:theirFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
theirFrag } -> do
STM m () -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar m (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (AnchoredFragment (Header blk))
varCandidate AnchoredFragment (Header blk)
theirFrag
let candTipBlockNo :: WithOrigin BlockNo
candTipBlockNo = AnchoredFragment (Header blk) -> WithOrigin BlockNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin BlockNo
AF.headBlockNo AnchoredFragment (Header blk)
theirFrag
Consensus (ClientPipelinedStIdle n) blk m
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (m :: * -> *) a. Monad m => a -> m a
return (Consensus (ClientPipelinedStIdle n) blk m
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Consensus (ClientPipelinedStIdle n) blk m
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$
KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> WithOrigin BlockNo
-> Consensus (ClientPipelinedStIdle n) blk m
forall (n :: N).
KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> WithOrigin BlockNo
-> Consensus (ClientPipelinedStIdle n) blk m
requestNext KnownIntersectionState blk
kis' MkPipelineDecision
mkPipelineDecision Nat n
n Their (Tip blk)
theirTip WithOrigin BlockNo
candTipBlockNo
Maybe (KnownIntersectionState blk)
Nothing ->
()
-> Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState ()
(Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ Nat n
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n)
forall s (n :: N).
NoThunks s =>
Nat n
-> Stateful m blk s (ClientPipelinedStIdle 'Z)
-> Stateful m blk s (ClientPipelinedStIdle n)
drainThePipe Nat n
n
(Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n)
forall a b. (a -> b) -> a -> b
$ (Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult)
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
findIntersection Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
forall blk.
BlockSupportsProtocol blk =>
Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
NoMoreIntersection
drainThePipe :: forall s n. NoThunks s
=> Nat n
-> Stateful m blk s (ClientPipelinedStIdle 'Z)
-> Stateful m blk s (ClientPipelinedStIdle n)
drainThePipe :: Nat n
-> Stateful m blk s (ClientPipelinedStIdle 'Z)
-> Stateful m blk s (ClientPipelinedStIdle n)
drainThePipe Nat n
n0 Stateful m blk s (ClientPipelinedStIdle 'Z)
m = (s -> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful m blk s (ClientPipelinedStIdle n)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
(s -> m (Consensus st blk m)) -> Stateful m blk s st
Stateful ((s -> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful m blk s (ClientPipelinedStIdle n))
-> (s -> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful m blk s (ClientPipelinedStIdle n)
forall a b. (a -> b) -> a -> b
$ Nat n -> s -> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (n' :: N).
Nat n' -> s -> m (Consensus (ClientPipelinedStIdle n') blk m)
go Nat n
n0
where
go :: forall n'. Nat n'
-> s
-> m (Consensus (ClientPipelinedStIdle n') blk m)
go :: Nat n' -> s -> m (Consensus (ClientPipelinedStIdle n') blk m)
go Nat n'
n s
s = case Nat n'
n of
Nat n'
Zero -> s
-> Stateful m blk s (ClientPipelinedStIdle 'Z)
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState s
s Stateful m blk s (ClientPipelinedStIdle 'Z)
m
Succ Nat n
n' -> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) a. Monad m => a -> m a
return (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ Maybe
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ClientStNext
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (m :: * -> *) (n1 :: N) header point tip a.
Maybe (m (ClientPipelinedStIdle ('S n1) header point tip m a))
-> ClientStNext n1 header point tip m a
-> ClientPipelinedStIdle ('S n1) header point tip m a
CollectResponse Maybe
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
forall a. Maybe a
Nothing (ClientStNext
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> ClientStNext
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall a b. (a -> b) -> a -> b
$ ClientStNext :: forall (n :: N) header point tip (m :: * -> *) a.
(header -> tip -> m (ClientPipelinedStIdle n header point tip m a))
-> (point
-> tip -> m (ClientPipelinedStIdle n header point tip m a))
-> ClientStNext n header point tip m a
ClientStNext
{ recvMsgRollForward :: Header blk
-> Tip blk
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
recvMsgRollForward = \Header blk
_hdr Tip blk
_tip -> Nat n
-> s
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (n' :: N).
Nat n' -> s -> m (Consensus (ClientPipelinedStIdle n') blk m)
go Nat n
n' s
s
, recvMsgRollBackward :: Point blk
-> Tip blk
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
recvMsgRollBackward = \Point blk
_pt Tip blk
_tip -> Nat n
-> s
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (n' :: N).
Nat n' -> s -> m (Consensus (ClientPipelinedStIdle n') blk m)
go Nat n
n' s
s
}
requestNext :: KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> WithOrigin BlockNo
-> Consensus (ClientPipelinedStIdle n) blk m
requestNext :: KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> WithOrigin BlockNo
-> Consensus (ClientPipelinedStIdle n) blk m
requestNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision Nat n
n Their (Tip blk)
theirTip WithOrigin BlockNo
candTipBlockNo =
case (Nat n
n, (PipelineDecision n, MkPipelineDecision)
decision) of
(Nat n
Zero, (PipelineDecision n
Request, MkPipelineDecision
mkPipelineDecision')) ->
ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall header point tip (m :: * -> *) a.
ClientStNext 'Z header point tip m a
-> m (ClientStNext 'Z header point tip m a)
-> ClientPipelinedStIdle 'Z header point tip m a
SendMsgRequestNext
(KnownIntersectionState blk
-> MkPipelineDecision
-> Nat 'Z
-> ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (n :: N).
KnownIntersectionState blk
-> MkPipelineDecision -> Nat n -> Consensus (ClientStNext n) blk m
handleNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision' Nat 'Z
forall (n :: N). ('Z ~ n) => Nat n
Zero)
(ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) a. Monad m => a -> m a
return (ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ KnownIntersectionState blk
-> MkPipelineDecision
-> Nat 'Z
-> ClientStNext
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (n :: N).
KnownIntersectionState blk
-> MkPipelineDecision -> Nat n -> Consensus (ClientStNext n) blk m
handleNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision' Nat 'Z
forall (n :: N). ('Z ~ n) => Nat n
Zero)
(Nat n
_, (PipelineDecision n
Pipeline, MkPipelineDecision
mkPipelineDecision')) ->
ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> Consensus (ClientPipelinedStIdle n) blk m
forall (n :: N) header point tip (m :: * -> *) a.
ClientPipelinedStIdle ('S n) header point tip m a
-> ClientPipelinedStIdle n header point tip m a
SendMsgRequestNextPipelined
(KnownIntersectionState blk
-> MkPipelineDecision
-> Nat ('S n)
-> Their (Tip blk)
-> WithOrigin BlockNo
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (n :: N).
KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> WithOrigin BlockNo
-> Consensus (ClientPipelinedStIdle n) blk m
requestNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision' (Nat n -> Nat ('S n)
forall (m :: N) (n :: N). (m ~ 'S n) => Nat n -> Nat m
Succ Nat n
n) Their (Tip blk)
theirTip WithOrigin BlockNo
candTipBlockNo)
(Succ Nat n
n', (PipelineDecision n
CollectOrPipeline, MkPipelineDecision
mkPipelineDecision')) ->
Maybe
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ClientStNext
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (m :: * -> *) (n1 :: N) header point tip a.
Maybe (m (ClientPipelinedStIdle ('S n1) header point tip m a))
-> ClientStNext n1 header point tip m a
-> ClientPipelinedStIdle ('S n1) header point tip m a
CollectResponse
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> Maybe
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
forall a. a -> Maybe a
Just (m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> Maybe
(m (ClientPipelinedStIdle
('S n)
(Header blk)
(Point blk)
(Tip blk)
m
ChainSyncClientResult)))
-> m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> Maybe
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
forall a b. (a -> b) -> a -> b
$ ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ ClientPipelinedStIdle
('S ('S n))
(Header blk)
(Point blk)
(Tip blk)
m
ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (n :: N) header point tip (m :: * -> *) a.
ClientPipelinedStIdle ('S n) header point tip m a
-> ClientPipelinedStIdle n header point tip m a
SendMsgRequestNextPipelined (ClientPipelinedStIdle
('S ('S n))
(Header blk)
(Point blk)
(Tip blk)
m
ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> ClientPipelinedStIdle
('S ('S n))
(Header blk)
(Point blk)
(Tip blk)
m
ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall a b. (a -> b) -> a -> b
$
KnownIntersectionState blk
-> MkPipelineDecision
-> Nat ('S ('S n))
-> Their (Tip blk)
-> WithOrigin BlockNo
-> ClientPipelinedStIdle
('S ('S n))
(Header blk)
(Point blk)
(Tip blk)
m
ChainSyncClientResult
forall (n :: N).
KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> WithOrigin BlockNo
-> Consensus (ClientPipelinedStIdle n) blk m
requestNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision' (Nat n -> Nat ('S ('S n))
forall (m :: N) (n :: N). (m ~ 'S n) => Nat n -> Nat m
Succ Nat n
n) Their (Tip blk)
theirTip WithOrigin BlockNo
candTipBlockNo)
(KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> ClientStNext
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (n :: N).
KnownIntersectionState blk
-> MkPipelineDecision -> Nat n -> Consensus (ClientStNext n) blk m
handleNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision' Nat n
n')
(Succ Nat n
n', (PipelineDecision n
Collect, MkPipelineDecision
mkPipelineDecision')) ->
Maybe
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> ClientStNext
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (m :: * -> *) (n1 :: N) header point tip a.
Maybe (m (ClientPipelinedStIdle ('S n1) header point tip m a))
-> ClientStNext n1 header point tip m a
-> ClientPipelinedStIdle ('S n1) header point tip m a
CollectResponse
Maybe
(m (ClientPipelinedStIdle
('S n) (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
forall a. Maybe a
Nothing
(KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> ClientStNext
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall (n :: N).
KnownIntersectionState blk
-> MkPipelineDecision -> Nat n -> Consensus (ClientStNext n) blk m
handleNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision' Nat n
n')
where
theirTipBlockNo :: WithOrigin BlockNo
theirTipBlockNo = Tip blk -> WithOrigin BlockNo
forall b. Tip b -> WithOrigin BlockNo
getTipBlockNo (Their (Tip blk) -> Tip blk
forall a. Their a -> a
unTheir Their (Tip blk)
theirTip)
decision :: (PipelineDecision n, MkPipelineDecision)
decision = MkPipelineDecision
-> Nat n
-> WithOrigin BlockNo
-> WithOrigin BlockNo
-> (PipelineDecision n, MkPipelineDecision)
forall (n :: N).
MkPipelineDecision
-> Nat n
-> WithOrigin BlockNo
-> WithOrigin BlockNo
-> (PipelineDecision n, MkPipelineDecision)
runPipelineDecision
MkPipelineDecision
mkPipelineDecision
Nat n
n
WithOrigin BlockNo
candTipBlockNo
WithOrigin BlockNo
theirTipBlockNo
handleNext :: KnownIntersectionState blk
-> MkPipelineDecision
-> Nat n
-> Consensus (ClientStNext n) blk m
handleNext :: KnownIntersectionState blk
-> MkPipelineDecision -> Nat n -> Consensus (ClientStNext n) blk m
handleNext KnownIntersectionState blk
kis MkPipelineDecision
mkPipelineDecision Nat n
n = ClientStNext :: forall (n :: N) header point tip (m :: * -> *) a.
(header -> tip -> m (ClientPipelinedStIdle n header point tip m a))
-> (point
-> tip -> m (ClientPipelinedStIdle n header point tip m a))
-> ClientStNext n header point tip m a
ClientStNext
{ recvMsgRollForward :: Header blk
-> Tip blk
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
recvMsgRollForward = \Header blk
hdr Tip blk
theirTip -> do
Tracer m (TraceChainSyncClientEvent blk)
-> TraceChainSyncClientEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceChainSyncClientEvent blk)
tracer (TraceChainSyncClientEvent blk -> m ())
-> TraceChainSyncClientEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ Header blk -> TraceChainSyncClientEvent blk
forall blk. Header blk -> TraceChainSyncClientEvent blk
TraceDownloadedHeader Header blk
hdr
KnownIntersectionState blk
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState KnownIntersectionState blk
kis (Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$
MkPipelineDecision
-> Nat n
-> Header blk
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall (n :: N).
MkPipelineDecision
-> Nat n
-> Header blk
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
rollForward MkPipelineDecision
mkPipelineDecision Nat n
n Header blk
hdr (Tip blk -> Their (Tip blk)
forall a. a -> Their a
Their Tip blk
theirTip)
, recvMsgRollBackward :: Point blk
-> Tip blk
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
recvMsgRollBackward = \Point blk
intersection Tip blk
theirTip -> do
let intersection' :: Point blk
intersection' :: Point blk
intersection' = Point blk -> Point blk
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
intersection
Tracer m (TraceChainSyncClientEvent blk)
-> TraceChainSyncClientEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceChainSyncClientEvent blk)
tracer (TraceChainSyncClientEvent blk -> m ())
-> TraceChainSyncClientEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ Point blk -> TraceChainSyncClientEvent blk
forall blk. Point blk -> TraceChainSyncClientEvent blk
TraceRolledBack Point blk
intersection'
KnownIntersectionState blk
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState KnownIntersectionState blk
kis (Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (ClientPipelinedStIdle
n (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$
MkPipelineDecision
-> Nat n
-> Point blk
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall (n :: N).
MkPipelineDecision
-> Nat n
-> Point blk
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
rollBackward MkPipelineDecision
mkPipelineDecision Nat n
n Point blk
intersection' (Tip blk -> Their (Tip blk)
forall a. a -> Their a
Their Tip blk
theirTip)
}
rollForward :: MkPipelineDecision
-> Nat n
-> Header blk
-> Their (Tip blk)
-> Stateful m blk
(KnownIntersectionState blk)
(ClientPipelinedStIdle n)
rollForward :: MkPipelineDecision
-> Nat n
-> Header blk
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
rollForward MkPipelineDecision
mkPipelineDecision Nat n
n Header blk
hdr Their (Tip blk)
theirTip
= (KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
(s -> m (Consensus st blk m)) -> Stateful m blk s st
Stateful ((KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n))
-> (KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall a b. (a -> b) -> a -> b
$ \KnownIntersectionState blk
kis -> m (Consensus (ClientPipelinedStIdle n) blk m)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a. m a -> m a
traceException (m (Consensus (ClientPipelinedStIdle n) blk m)
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> m (Consensus (ClientPipelinedStIdle n) blk m)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ do
Time
now <- m Time
forall (m :: * -> *). MonadMonotonicTime m => m Time
getMonotonicTime
let hdrPoint :: Point blk
hdrPoint = Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint Header blk
hdr
HeaderHash blk -> Maybe (InvalidBlockReason blk)
isInvalidBlock <- STM m (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> m (HeaderHash blk -> Maybe (InvalidBlockReason blk))
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> m (HeaderHash blk -> Maybe (InvalidBlockReason blk)))
-> STM m (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> m (HeaderHash blk -> Maybe (InvalidBlockReason blk))
forall a b. (a -> b) -> a -> b
$ WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> HeaderHash blk -> Maybe (InvalidBlockReason blk)
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
-> STM m (HeaderHash blk -> Maybe (InvalidBlockReason blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock
let disconnectWhenInvalid :: ChainHash blk -> m ()
disconnectWhenInvalid = \case
ChainHash blk
GenesisHash -> () -> m ()
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
BlockHash HeaderHash blk
hash ->
Maybe (InvalidBlockReason blk)
-> (InvalidBlockReason blk -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust (HeaderHash blk -> Maybe (InvalidBlockReason blk)
isInvalidBlock HeaderHash blk
hash) ((InvalidBlockReason blk -> m ()) -> m ())
-> (InvalidBlockReason blk -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \InvalidBlockReason blk
reason ->
ChainSyncClientException -> m ()
forall (m' :: * -> *) x'.
MonadThrow m' =>
ChainSyncClientException -> m' x'
disconnect (ChainSyncClientException -> m ())
-> ChainSyncClientException -> m ()
forall a b. (a -> b) -> a -> b
$ Point blk
-> HeaderHash blk
-> InvalidBlockReason blk
-> ChainSyncClientException
forall blk.
LedgerSupportsProtocol blk =>
Point blk
-> HeaderHash blk
-> InvalidBlockReason blk
-> ChainSyncClientException
InvalidBlock Point blk
hdrPoint HeaderHash blk
hash InvalidBlockReason blk
reason
ChainHash blk -> m ()
disconnectWhenInvalid (ChainHash blk -> m ()) -> ChainHash blk -> m ()
forall a b. (a -> b) -> a -> b
$
if NodeToNodeVersion -> Bool
isPipeliningEnabled NodeToNodeVersion
version
then Header blk -> ChainHash blk
forall blk. GetPrevHash blk => Header blk -> ChainHash blk
headerPrevHash Header blk
hdr
else HeaderHash blk -> ChainHash blk
forall b. HeaderHash b -> ChainHash b
BlockHash (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr)
IntersectCheck blk
intersectCheck <- STM m (IntersectCheck blk) -> m (IntersectCheck blk)
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (IntersectCheck blk) -> m (IntersectCheck blk))
-> STM m (IntersectCheck blk) -> m (IntersectCheck blk)
forall a b. (a -> b) -> a -> b
$ do
Maybe (KnownIntersectionState blk)
mKis' <- KnownIntersectionState blk
-> STM m (Maybe (KnownIntersectionState blk))
intersectsWithCurrentChain KnownIntersectionState blk
kis
case Maybe (KnownIntersectionState blk)
mKis' of
Maybe (KnownIntersectionState blk)
Nothing -> IntersectCheck blk -> STM m (IntersectCheck blk)
forall (m :: * -> *) a. Monad m => a -> m a
return IntersectCheck blk
forall blk. IntersectCheck blk
NoLongerIntersects
Just kis' :: KnownIntersectionState blk
kis'@KnownIntersectionState { Point blk
mostRecentIntersection :: Point blk
$sel:mostRecentIntersection:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> Point blk
mostRecentIntersection } -> do
Forecast (LedgerView (BlockProtocol blk))
forecast <-
Forecast (LedgerView (BlockProtocol blk))
-> (ExtLedgerState blk
-> Forecast (LedgerView (BlockProtocol blk)))
-> Maybe (ExtLedgerState blk)
-> Forecast (LedgerView (BlockProtocol blk))
forall b a. b -> (a -> b) -> Maybe a -> b
maybe
(String -> Forecast (LedgerView (BlockProtocol blk))
forall a. HasCallStack => String -> a
error (String -> Forecast (LedgerView (BlockProtocol blk)))
-> String -> Forecast (LedgerView (BlockProtocol blk))
forall a b. (a -> b) -> a -> b
$
String
"intersection not within last k blocks: " String -> ShowS
forall a. Semigroup a => a -> a -> a
<> Point blk -> String
forall a. Show a => a -> String
show Point blk
mostRecentIntersection)
(LedgerConfig blk
-> LedgerState blk -> Forecast (LedgerView (BlockProtocol blk))
forall blk.
(LedgerSupportsProtocol blk, HasCallStack) =>
LedgerConfig blk
-> LedgerState blk -> Forecast (LedgerView (BlockProtocol blk))
ledgerViewForecastAt (TopLevelConfig blk -> LedgerConfig blk
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig blk
cfg) (LedgerState blk -> Forecast (LedgerView (BlockProtocol blk)))
-> (ExtLedgerState blk -> LedgerState blk)
-> ExtLedgerState blk
-> Forecast (LedgerView (BlockProtocol blk))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState)
(Maybe (ExtLedgerState blk)
-> Forecast (LedgerView (BlockProtocol blk)))
-> STM m (Maybe (ExtLedgerState blk))
-> STM m (Forecast (LedgerView (BlockProtocol blk)))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Point blk -> STM m (Maybe (ExtLedgerState blk))
getPastLedger Point blk
mostRecentIntersection
case Except
OutsideForecastRange (Ticked (LedgerView (BlockProtocol blk)))
-> Either
OutsideForecastRange (Ticked (LedgerView (BlockProtocol blk)))
forall e a. Except e a -> Either e a
runExcept (Except
OutsideForecastRange (Ticked (LedgerView (BlockProtocol blk)))
-> Either
OutsideForecastRange (Ticked (LedgerView (BlockProtocol blk))))
-> Except
OutsideForecastRange (Ticked (LedgerView (BlockProtocol blk)))
-> Either
OutsideForecastRange (Ticked (LedgerView (BlockProtocol blk)))
forall a b. (a -> b) -> a -> b
$ Forecast (LedgerView (BlockProtocol blk))
-> SlotNo
-> Except
OutsideForecastRange (Ticked (LedgerView (BlockProtocol blk)))
forall a.
Forecast a -> SlotNo -> Except OutsideForecastRange (Ticked a)
forecastFor Forecast (LedgerView (BlockProtocol blk))
forecast (Header blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot Header blk
hdr) of
Left OutsideForecastRange{} ->
STM m (IntersectCheck blk)
forall (m :: * -> *) a. MonadSTM m => STM m a
retry
Right Ticked (LedgerView (BlockProtocol blk))
ledgerView ->
IntersectCheck blk -> STM m (IntersectCheck blk)
forall (m :: * -> *) a. Monad m => a -> m a
return (IntersectCheck blk -> STM m (IntersectCheck blk))
-> IntersectCheck blk -> STM m (IntersectCheck blk)
forall a b. (a -> b) -> a -> b
$ KnownIntersectionState blk
-> Ticked (LedgerView (BlockProtocol blk)) -> IntersectCheck blk
forall blk.
KnownIntersectionState blk
-> Ticked (LedgerView (BlockProtocol blk)) -> IntersectCheck blk
Intersects KnownIntersectionState blk
kis' Ticked (LedgerView (BlockProtocol blk))
ledgerView
case IntersectCheck blk
intersectCheck of
IntersectCheck blk
NoLongerIntersects ->
()
-> Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState ()
(Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ Nat n
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n)
forall s (n :: N).
NoThunks s =>
Nat n
-> Stateful m blk s (ClientPipelinedStIdle 'Z)
-> Stateful m blk s (ClientPipelinedStIdle n)
drainThePipe Nat n
n
(Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n)
forall a b. (a -> b) -> a -> b
$ (Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult)
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
findIntersection Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
forall blk.
BlockSupportsProtocol blk =>
Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
NoMoreIntersection
Intersects KnownIntersectionState blk
kis' Ticked (LedgerView (BlockProtocol blk))
ledgerView -> do
let KnownIntersectionState {
AnchoredFragment (Header blk)
ourFrag :: AnchoredFragment (Header blk)
$sel:ourFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
ourFrag
, AnchoredFragment (Header blk)
theirFrag :: AnchoredFragment (Header blk)
$sel:theirFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
theirFrag
, HeaderStateHistory blk
theirHeaderStateHistory :: HeaderStateHistory blk
$sel:theirHeaderStateHistory:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> HeaderStateHistory blk
theirHeaderStateHistory
, Point blk
mostRecentIntersection :: Point blk
$sel:mostRecentIntersection:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> Point blk
mostRecentIntersection
} = KnownIntersectionState blk
kis'
let expectPrevHash :: ChainHash blk
expectPrevHash = ChainHash (Header blk) -> ChainHash blk
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
ChainHash b -> ChainHash b'
castHash (AnchoredFragment (Header blk) -> ChainHash (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> ChainHash block
AF.headHash AnchoredFragment (Header blk)
theirFrag)
actualPrevHash :: ChainHash blk
actualPrevHash = Header blk -> ChainHash blk
forall blk. GetPrevHash blk => Header blk -> ChainHash blk
headerPrevHash Header blk
hdr
Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (ChainHash blk
actualPrevHash ChainHash blk -> ChainHash blk -> Bool
forall a. Eq a => a -> a -> Bool
/= ChainHash blk
expectPrevHash) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
ChainSyncClientException -> m ()
forall (m' :: * -> *) x'.
MonadThrow m' =>
ChainSyncClientException -> m' x'
disconnect (ChainSyncClientException -> m ())
-> ChainSyncClientException -> m ()
forall a b. (a -> b) -> a -> b
$
ChainHash blk
-> ChainHash blk
-> Our (Tip blk)
-> Their (Tip blk)
-> ChainSyncClientException
forall blk.
BlockSupportsProtocol blk =>
ChainHash blk
-> ChainHash blk
-> Our (Tip blk)
-> Their (Tip blk)
-> ChainSyncClientException
DoesntFit
ChainHash blk
actualPrevHash
ChainHash blk
expectPrevHash
(AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain AnchoredFragment (Header blk)
ourFrag)
Their (Tip blk)
theirTip
HeaderStateHistory blk
theirHeaderStateHistory' <-
case Except (HeaderError blk) (HeaderStateHistory blk)
-> Either (HeaderError blk) (HeaderStateHistory blk)
forall e a. Except e a -> Either e a
runExcept (Except (HeaderError blk) (HeaderStateHistory blk)
-> Either (HeaderError blk) (HeaderStateHistory blk))
-> Except (HeaderError blk) (HeaderStateHistory blk)
-> Either (HeaderError blk) (HeaderStateHistory blk)
forall a b. (a -> b) -> a -> b
$ TopLevelConfig blk
-> Ticked (LedgerView (BlockProtocol blk))
-> Header blk
-> HeaderStateHistory blk
-> Except (HeaderError blk) (HeaderStateHistory blk)
forall blk.
(BlockSupportsProtocol blk, ValidateEnvelope blk) =>
TopLevelConfig blk
-> Ticked (LedgerView (BlockProtocol blk))
-> Header blk
-> HeaderStateHistory blk
-> Except (HeaderError blk) (HeaderStateHistory blk)
validateHeader TopLevelConfig blk
cfg Ticked (LedgerView (BlockProtocol blk))
ledgerView Header blk
hdr HeaderStateHistory blk
theirHeaderStateHistory of
Right HeaderStateHistory blk
theirHeaderStateHistory' -> HeaderStateHistory blk -> m (HeaderStateHistory blk)
forall (m :: * -> *) a. Monad m => a -> m a
return HeaderStateHistory blk
theirHeaderStateHistory'
Left HeaderError blk
vErr ->
ChainSyncClientException -> m (HeaderStateHistory blk)
forall (m' :: * -> *) x'.
MonadThrow m' =>
ChainSyncClientException -> m' x'
disconnect (ChainSyncClientException -> m (HeaderStateHistory blk))
-> ChainSyncClientException -> m (HeaderStateHistory blk)
forall a b. (a -> b) -> a -> b
$
Point blk
-> HeaderError blk
-> Our (Tip blk)
-> Their (Tip blk)
-> ChainSyncClientException
forall blk.
(BlockSupportsProtocol blk, ValidateEnvelope blk) =>
Point blk
-> HeaderError blk
-> Our (Tip blk)
-> Their (Tip blk)
-> ChainSyncClientException
HeaderError Point blk
hdrPoint HeaderError blk
vErr (AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain AnchoredFragment (Header blk)
ourFrag) Their (Tip blk)
theirTip
let theirFrag' :: AnchoredFragment (Header blk)
theirFrag' = AnchoredFragment (Header blk)
theirFrag AnchoredFragment (Header blk)
-> Header blk -> AnchoredFragment (Header blk)
forall v a b.
Anchorable v a b =>
AnchoredSeq v a b -> b -> AnchoredSeq v a b
:> Header blk
hdr
mostRecentIntersection' :: Point blk
mostRecentIntersection'
| Just Header blk
ourSuccessor <-
Point (Header blk)
-> AnchoredFragment (Header blk) -> Maybe (Header blk)
forall block.
HasHeader block =>
Point block -> AnchoredFragment block -> Maybe block
AF.successorBlock (Point blk -> Point (Header blk)
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
mostRecentIntersection) AnchoredFragment (Header blk)
ourFrag
, Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
ourSuccessor HeaderHash blk -> HeaderHash blk -> Bool
forall a. Eq a => a -> a -> Bool
== Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr
= Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint Header blk
hdr
| Bool
otherwise
= Point blk
mostRecentIntersection
kis'' :: KnownIntersectionState blk
kis'' = ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall blk.
(HasHeader blk, HasHeader (Header blk), HasAnnTip blk,
ConsensusProtocol (BlockProtocol blk), HasCallStack) =>
ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
assertKnownIntersectionInvariants (TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
forall blk.
TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
configConsensus TopLevelConfig blk
cfg) (KnownIntersectionState blk -> KnownIntersectionState blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall a b. (a -> b) -> a -> b
$
KnownIntersectionState :: forall blk.
AnchoredFragment (Header blk)
-> HeaderStateHistory blk
-> AnchoredFragment (Header blk)
-> Point blk
-> KnownIntersectionState blk
KnownIntersectionState {
$sel:theirFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
theirFrag = AnchoredFragment (Header blk)
theirFrag'
, $sel:theirHeaderStateHistory:KnownIntersectionState :: HeaderStateHistory blk
theirHeaderStateHistory = HeaderStateHistory blk
theirHeaderStateHistory'
, $sel:ourFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
ourFrag = AnchoredFragment (Header blk)
ourFrag
, $sel:mostRecentIntersection:KnownIntersectionState :: Point blk
mostRecentIntersection = Point blk
mostRecentIntersection'
}
STM m () -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar m (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (AnchoredFragment (Header blk))
varCandidate AnchoredFragment (Header blk)
theirFrag'
let slotNo :: SlotNo
slotNo = Header blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot Header blk
hdr
STM m () -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ HeaderMetricsTracer m -> (SlotNo, Time) -> STM m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith HeaderMetricsTracer m
headerMetricsTracer (SlotNo
slotNo, Time
now)
KnownIntersectionState blk
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState KnownIntersectionState blk
kis'' (Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall (n :: N).
MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
nextStep MkPipelineDecision
mkPipelineDecision Nat n
n Their (Tip blk)
theirTip
rollBackward :: MkPipelineDecision
-> Nat n
-> Point blk
-> Their (Tip blk)
-> Stateful m blk
(KnownIntersectionState blk)
(ClientPipelinedStIdle n)
rollBackward :: MkPipelineDecision
-> Nat n
-> Point blk
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
rollBackward MkPipelineDecision
mkPipelineDecision Nat n
n Point blk
rollBackPoint
Their (Tip blk)
theirTip
= (KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
(s -> m (Consensus st blk m)) -> Stateful m blk s st
Stateful ((KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n))
-> (KnownIntersectionState blk
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall a b. (a -> b) -> a -> b
$ \KnownIntersectionState
{ AnchoredFragment (Header blk)
theirFrag :: AnchoredFragment (Header blk)
$sel:theirFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
theirFrag
, HeaderStateHistory blk
theirHeaderStateHistory :: HeaderStateHistory blk
$sel:theirHeaderStateHistory:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> HeaderStateHistory blk
theirHeaderStateHistory
, AnchoredFragment (Header blk)
ourFrag :: AnchoredFragment (Header blk)
$sel:ourFrag:KnownIntersectionState :: forall blk.
KnownIntersectionState blk -> AnchoredFragment (Header blk)
ourFrag
, Point blk
mostRecentIntersection :: Point blk
$sel:mostRecentIntersection:KnownIntersectionState :: forall blk. KnownIntersectionState blk -> Point blk
mostRecentIntersection
} -> m (Consensus (ClientPipelinedStIdle n) blk m)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a. m a -> m a
traceException (m (Consensus (ClientPipelinedStIdle n) blk m)
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> m (Consensus (ClientPipelinedStIdle n) blk m)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ do
case Point blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall blk.
(BlockSupportsProtocol blk, HasAnnTip blk) =>
Point blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
attemptRollback Point blk
rollBackPoint (AnchoredFragment (Header blk)
theirFrag, HeaderStateHistory blk
theirHeaderStateHistory) of
Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
Nothing ->
Nat n
-> ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (n :: N).
Nat n
-> ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m)
terminateAfterDrain Nat n
n (ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$
Point blk
-> Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
forall blk.
BlockSupportsProtocol blk =>
Point blk
-> Our (Tip blk) -> Their (Tip blk) -> ChainSyncClientResult
RolledBackPastIntersection
Point blk
rollBackPoint
(AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain AnchoredFragment (Header blk)
ourFrag)
Their (Tip blk)
theirTip
Just (AnchoredFragment (Header blk)
theirFrag', HeaderStateHistory blk
theirHeaderStateHistory') -> do
let mostRecentIntersection' :: Point blk
mostRecentIntersection'
| Point (Header blk) -> AnchoredFragment (Header blk) -> Bool
forall block.
HasHeader block =>
Point block -> AnchoredFragment block -> Bool
AF.withinFragmentBounds (Point blk -> Point (Header blk)
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
rollBackPoint) AnchoredFragment (Header blk)
ourFrag
= Point blk
rollBackPoint
| Bool
otherwise
= Point blk
mostRecentIntersection
kis' :: KnownIntersectionState blk
kis' = ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall blk.
(HasHeader blk, HasHeader (Header blk), HasAnnTip blk,
ConsensusProtocol (BlockProtocol blk), HasCallStack) =>
ConsensusConfig (BlockProtocol blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
assertKnownIntersectionInvariants (TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
forall blk.
TopLevelConfig blk -> ConsensusConfig (BlockProtocol blk)
configConsensus TopLevelConfig blk
cfg) (KnownIntersectionState blk -> KnownIntersectionState blk)
-> KnownIntersectionState blk -> KnownIntersectionState blk
forall a b. (a -> b) -> a -> b
$
KnownIntersectionState :: forall blk.
AnchoredFragment (Header blk)
-> HeaderStateHistory blk
-> AnchoredFragment (Header blk)
-> Point blk
-> KnownIntersectionState blk
KnownIntersectionState {
$sel:theirFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
theirFrag = AnchoredFragment (Header blk)
theirFrag'
, $sel:theirHeaderStateHistory:KnownIntersectionState :: HeaderStateHistory blk
theirHeaderStateHistory = HeaderStateHistory blk
theirHeaderStateHistory'
, $sel:ourFrag:KnownIntersectionState :: AnchoredFragment (Header blk)
ourFrag = AnchoredFragment (Header blk)
ourFrag
, $sel:mostRecentIntersection:KnownIntersectionState :: Point blk
mostRecentIntersection = Point blk
mostRecentIntersection'
}
STM m () -> m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m () -> m ()) -> STM m () -> m ()
forall a b. (a -> b) -> a -> b
$ StrictTVar m (AnchoredFragment (Header blk))
-> AnchoredFragment (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (AnchoredFragment (Header blk))
varCandidate AnchoredFragment (Header blk)
theirFrag'
KnownIntersectionState blk
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState KnownIntersectionState blk
kis' (Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
forall (n :: N).
MkPipelineDecision
-> Nat n
-> Their (Tip blk)
-> Stateful
m blk (KnownIntersectionState blk) (ClientPipelinedStIdle n)
nextStep MkPipelineDecision
mkPipelineDecision Nat n
n Their (Tip blk)
theirTip
terminate :: ChainSyncClientResult -> m (Consensus (ClientPipelinedStIdle 'Z) blk m)
terminate :: ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
terminate ChainSyncClientResult
res = do
Tracer m (TraceChainSyncClientEvent blk)
-> TraceChainSyncClientEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceChainSyncClientEvent blk)
tracer (ChainSyncClientResult -> TraceChainSyncClientEvent blk
forall blk. ChainSyncClientResult -> TraceChainSyncClientEvent blk
TraceTermination ChainSyncClientResult
res)
ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ChainSyncClientResult
-> ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult
forall a header point tip (m :: * -> *).
a -> ClientPipelinedStIdle 'Z header point tip m a
SendMsgDone ChainSyncClientResult
res)
terminateAfterDrain :: Nat n -> ChainSyncClientResult -> m (Consensus (ClientPipelinedStIdle n) blk m)
terminateAfterDrain :: Nat n
-> ChainSyncClientResult
-> m (Consensus (ClientPipelinedStIdle n) blk m)
terminateAfterDrain Nat n
n ChainSyncClientResult
result =
()
-> Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
NoThunks s =>
s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState ()
(Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m))
-> Stateful m blk () (ClientPipelinedStIdle n)
-> m (Consensus (ClientPipelinedStIdle n) blk m)
forall a b. (a -> b) -> a -> b
$ Nat n
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n)
forall s (n :: N).
NoThunks s =>
Nat n
-> Stateful m blk s (ClientPipelinedStIdle 'Z)
-> Stateful m blk s (ClientPipelinedStIdle n)
drainThePipe Nat n
n
(Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
-> Stateful m blk () (ClientPipelinedStIdle n)
forall a b. (a -> b) -> a -> b
$ (()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
forall (m :: * -> *) blk s
(st :: * -> * -> * -> (* -> *) -> * -> *).
(s -> m (Consensus st blk m)) -> Stateful m blk s st
Stateful ((()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful m blk () (ClientPipelinedStIdle 'Z))
-> (()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> Stateful m blk () (ClientPipelinedStIdle 'Z)
forall a b. (a -> b) -> a -> b
$ m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> ()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. a -> b -> a
const (m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> ()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult))
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
-> ()
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
forall a b. (a -> b) -> a -> b
$ ChainSyncClientResult
-> m (ClientPipelinedStIdle
'Z (Header blk) (Point blk) (Tip blk) m ChainSyncClientResult)
terminate ChainSyncClientResult
result
disconnect :: forall m' x'. MonadThrow m'
=> ChainSyncClientException -> m' x'
disconnect :: ChainSyncClientException -> m' x'
disconnect = ChainSyncClientException -> m' x'
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO
traceException :: m a -> m a
traceException :: m a -> m a
traceException m a
m = m a
m m a -> (ChainSyncClientException -> m a) -> m a
forall (m :: * -> *) e a.
(MonadCatch m, Exception e) =>
m a -> (e -> m a) -> m a
`catch` \(ChainSyncClientException
e :: ChainSyncClientException) -> do
Tracer m (TraceChainSyncClientEvent blk)
-> TraceChainSyncClientEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceChainSyncClientEvent blk)
tracer (TraceChainSyncClientEvent blk -> m ())
-> TraceChainSyncClientEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ ChainSyncClientException -> TraceChainSyncClientEvent blk
forall blk.
ChainSyncClientException -> TraceChainSyncClientEvent blk
TraceException ChainSyncClientException
e
ChainSyncClientException -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO ChainSyncClientException
e
ourTipFromChain :: AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain :: AnchoredFragment (Header blk) -> Our (Tip blk)
ourTipFromChain = Tip blk -> Our (Tip blk)
forall a. a -> Our a
Our (Tip blk -> Our (Tip blk))
-> (AnchoredFragment (Header blk) -> Tip blk)
-> AnchoredFragment (Header blk)
-> Our (Tip blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Anchor (Header blk) -> Tip blk
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Tip b
AF.anchorToTip (Anchor (Header blk) -> Tip blk)
-> (AnchoredFragment (Header blk) -> Anchor (Header blk))
-> AnchoredFragment (Header blk)
-> Tip blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment (Header blk) -> Anchor (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> a
AF.headAnchor
offsets :: Word64 -> [Word64]
offsets :: Word64 -> [Word64]
offsets Word64
maxOffset = [Word64
0] [Word64] -> [Word64] -> [Word64]
forall a. [a] -> [a] -> [a]
++ (Word64 -> Bool) -> [Word64] -> [Word64]
forall a. (a -> Bool) -> [a] -> [a]
takeWhile (Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
< Word64
l) [Word64 -> Word64
fib Word64
n | Word64
n <- [Word64
2..]] [Word64] -> [Word64] -> [Word64]
forall a. [a] -> [a] -> [a]
++ [Word64
l]
where
l :: Word64
l = Word64
k Word64 -> Word64 -> Word64
forall a. Ord a => a -> a -> a
`min` Word64
maxOffset
k :: Word64
k :: Word64
k = SecurityParam -> Word64
maxRollbacks (SecurityParam -> Word64) -> SecurityParam -> Word64
forall a b. (a -> b) -> a -> b
$ TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg
attemptRollback ::
( BlockSupportsProtocol blk
, HasAnnTip blk
)
=> Point blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
attemptRollback :: Point blk
-> (AnchoredFragment (Header blk), HeaderStateHistory blk)
-> Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
attemptRollback Point blk
rollBackPoint (AnchoredFragment (Header blk)
frag, HeaderStateHistory blk
state) = do
AnchoredFragment (Header blk)
frag' <- Point (Header blk)
-> AnchoredFragment (Header blk)
-> Maybe (AnchoredFragment (Header blk))
forall block.
HasHeader block =>
Point block
-> AnchoredFragment block -> Maybe (AnchoredFragment block)
AF.rollback (Point blk -> Point (Header blk)
forall b b'.
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
rollBackPoint) AnchoredFragment (Header blk)
frag
HeaderStateHistory blk
state' <- Point blk
-> HeaderStateHistory blk -> Maybe (HeaderStateHistory blk)
forall blk.
(BlockSupportsProtocol blk, HasAnnTip blk) =>
Point blk
-> HeaderStateHistory blk -> Maybe (HeaderStateHistory blk)
HeaderStateHistory.rewind Point blk
rollBackPoint HeaderStateHistory blk
state
(AnchoredFragment (Header blk), HeaderStateHistory blk)
-> Maybe (AnchoredFragment (Header blk), HeaderStateHistory blk)
forall (m :: * -> *) a. Monad m => a -> m a
return (AnchoredFragment (Header blk)
frag', HeaderStateHistory blk
state')
invalidBlockRejector
:: forall m blk.
( IOLike m
, BlockSupportsProtocol blk
, LedgerSupportsProtocol blk
)
=> Tracer m (TraceChainSyncClientEvent blk)
-> NodeToNodeVersion
-> STM m (WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk)))
-> STM m (AnchoredFragment (Header blk))
-> Watcher m
(WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk)))
Fingerprint
invalidBlockRejector :: Tracer m (TraceChainSyncClientEvent blk)
-> NodeToNodeVersion
-> STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
-> STM m (AnchoredFragment (Header blk))
-> Watcher
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
Fingerprint
invalidBlockRejector Tracer m (TraceChainSyncClientEvent blk)
tracer NodeToNodeVersion
version STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock STM m (AnchoredFragment (Header blk))
getCandidate =
Watcher :: forall (m :: * -> *) a fp.
(a -> fp) -> Maybe fp -> (a -> m ()) -> STM m a -> Watcher m a fp
Watcher {
wFingerprint :: WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> Fingerprint
wFingerprint = WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> Fingerprint
forall a. WithFingerprint a -> Fingerprint
getFingerprint
, wInitial :: Maybe Fingerprint
wInitial = Maybe Fingerprint
forall a. Maybe a
Nothing
, wNotify :: WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> m ()
wNotify = (HeaderHash blk -> Maybe (InvalidBlockReason blk)) -> m ()
checkInvalid ((HeaderHash blk -> Maybe (InvalidBlockReason blk)) -> m ())
-> (WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. WithFingerprint (HeaderHash blk -> Maybe (InvalidBlockReason blk))
-> HeaderHash blk -> Maybe (InvalidBlockReason blk)
forall a. WithFingerprint a -> a
forgetFingerprint
, wReader :: STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
wReader = STM
m
(WithFingerprint
(HeaderHash blk -> Maybe (InvalidBlockReason blk)))
getIsInvalidBlock
}
where
checkInvalid :: (HeaderHash blk -> Maybe (InvalidBlockReason blk)) -> m ()
checkInvalid :: (HeaderHash blk -> Maybe (InvalidBlockReason blk)) -> m ()
checkInvalid HeaderHash blk -> Maybe (InvalidBlockReason blk)
isInvalidBlock = do
AnchoredFragment (Header blk)
theirFrag <- STM m (AnchoredFragment (Header blk))
-> m (AnchoredFragment (Header blk))
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically STM m (AnchoredFragment (Header blk))
getCandidate
((Header blk, InvalidBlockReason blk) -> m ())
-> Maybe (Header blk, InvalidBlockReason blk) -> m ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
(a -> m b) -> t a -> m ()
mapM_ ((Header blk -> InvalidBlockReason blk -> m ())
-> (Header blk, InvalidBlockReason blk) -> m ()
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry Header blk -> InvalidBlockReason blk -> m ()
disconnect) (Maybe (Header blk, InvalidBlockReason blk) -> m ())
-> Maybe (Header blk, InvalidBlockReason blk) -> m ()
forall a b. (a -> b) -> a -> b
$ (Header blk -> Maybe (Header blk, InvalidBlockReason blk))
-> [Header blk] -> Maybe (Header blk, InvalidBlockReason blk)
forall a b (f :: * -> *).
Foldable f =>
(a -> Maybe b) -> f a -> Maybe b
firstJust
(\Header blk
hdr -> (Header blk
hdr,) (InvalidBlockReason blk -> (Header blk, InvalidBlockReason blk))
-> Maybe (InvalidBlockReason blk)
-> Maybe (Header blk, InvalidBlockReason blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> HeaderHash blk -> Maybe (InvalidBlockReason blk)
isInvalidBlock (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr))
( (if NodeToNodeVersion -> Bool
isPipeliningEnabled NodeToNodeVersion
version then Int -> [Header blk] -> [Header blk]
forall a. Int -> [a] -> [a]
drop Int
1 else [Header blk] -> [Header blk]
forall a. a -> a
id)
([Header blk] -> [Header blk]) -> [Header blk] -> [Header blk]
forall a b. (a -> b) -> a -> b
$ AnchoredFragment (Header blk) -> [Header blk]
forall v a b. AnchoredSeq v a b -> [b]
AF.toNewestFirst AnchoredFragment (Header blk)
theirFrag
)
disconnect :: Header blk -> InvalidBlockReason blk -> m ()
disconnect :: Header blk -> InvalidBlockReason blk -> m ()
disconnect Header blk
invalidHeader InvalidBlockReason blk
reason = do
let ex :: ChainSyncClientException
ex = Point blk
-> HeaderHash blk
-> InvalidBlockReason blk
-> ChainSyncClientException
forall blk.
LedgerSupportsProtocol blk =>
Point blk
-> HeaderHash blk
-> InvalidBlockReason blk
-> ChainSyncClientException
InvalidBlock
(Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint Header blk
invalidHeader)
(Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
invalidHeader)
InvalidBlockReason blk
reason
Tracer m (TraceChainSyncClientEvent blk)
-> TraceChainSyncClientEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceChainSyncClientEvent blk)
tracer (TraceChainSyncClientEvent blk -> m ())
-> TraceChainSyncClientEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ ChainSyncClientException -> TraceChainSyncClientEvent blk
forall blk.
ChainSyncClientException -> TraceChainSyncClientEvent blk
TraceException ChainSyncClientException
ex
ChainSyncClientException -> m ()
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO ChainSyncClientException
ex
data IntersectCheck blk =
NoLongerIntersects
| Intersects
(KnownIntersectionState blk)
(Ticked (LedgerView (BlockProtocol blk)))
newtype Stateful m blk s st = Stateful (s -> m (Consensus st blk m))
continueWithState :: forall m blk s st. NoThunks s
=> s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState :: s -> Stateful m blk s st -> m (Consensus st blk m)
continueWithState !s
s (Stateful s -> m (Consensus st blk m)
f) =
Maybe String -> m (Consensus st blk m) -> m (Consensus st blk m)
forall a. HasCallStack => Maybe String -> a -> a
checkInvariant (ThunkInfo -> String
forall a. Show a => a -> String
show (ThunkInfo -> String) -> Maybe ThunkInfo -> Maybe String
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> s -> Maybe ThunkInfo
forall a. NoThunks a => a -> Maybe ThunkInfo
unsafeNoThunks s
s) (m (Consensus st blk m) -> m (Consensus st blk m))
-> m (Consensus st blk m) -> m (Consensus st blk m)
forall a b. (a -> b) -> a -> b
$ s -> m (Consensus st blk m)
f s
s
data ChainSyncClientResult =
forall blk. BlockSupportsProtocol blk =>
ForkTooDeep
(Point blk)
(Our (Tip blk))
(Their (Tip blk))
| forall blk. BlockSupportsProtocol blk =>
NoMoreIntersection
(Our (Tip blk))
(Their (Tip blk))
| forall blk. BlockSupportsProtocol blk =>
RolledBackPastIntersection
(Point blk)
(Our (Tip blk))
(Their (Tip blk))
| AskedToTerminate
deriving instance Show ChainSyncClientResult
instance Eq ChainSyncClientResult where
ForkTooDeep (Point blk
a :: Point blk) Our (Tip blk)
b Their (Tip blk)
c == :: ChainSyncClientResult -> ChainSyncClientResult -> Bool
== ForkTooDeep (Point blk
a' :: Point blk') Our (Tip blk)
b' Their (Tip blk)
c' =
case (Typeable blk, Typeable blk) => Maybe (blk :~: blk)
forall k (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
eqT @blk @blk' of
Maybe (blk :~: blk)
Nothing -> Bool
False
Just blk :~: blk
Refl -> (Point blk
a, Our (Tip blk)
b, Their (Tip blk)
c) (Point blk, Our (Tip blk), Their (Tip blk))
-> (Point blk, Our (Tip blk), Their (Tip blk)) -> Bool
forall a. Eq a => a -> a -> Bool
== (Point blk
Point blk
a', Our (Tip blk)
Our (Tip blk)
b', Their (Tip blk)
Their (Tip blk)
c')
ForkTooDeep{} == ChainSyncClientResult
_ = Bool
False
NoMoreIntersection (Our (Tip blk)
a :: Our (Tip blk)) Their (Tip blk)
b == NoMoreIntersection (Our (Tip blk)
a' :: Our (Tip blk')) Their (Tip blk)
b' =
case (Typeable blk, Typeable blk) => Maybe (blk :~: blk)
forall k (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
eqT @blk @blk' of
Maybe (blk :~: blk)
Nothing -> Bool
False
Just blk :~: blk
Refl -> (Our (Tip blk)
a, Their (Tip blk)
b) (Our (Tip blk), Their (Tip blk))
-> (Our (Tip blk), Their (Tip blk)) -> Bool
forall a. Eq a => a -> a -> Bool
== (Our (Tip blk)
Our (Tip blk)
a', Their (Tip blk)
Their (Tip blk)
b')
NoMoreIntersection{} == ChainSyncClientResult
_ = Bool
False
RolledBackPastIntersection (Point blk
a :: Point blk) Our (Tip blk)
b Their (Tip blk)
c == RolledBackPastIntersection (Point blk
a' :: Point blk') Our (Tip blk)
b' Their (Tip blk)
c' =
case (Typeable blk, Typeable blk) => Maybe (blk :~: blk)
forall k (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
eqT @blk @blk' of
Maybe (blk :~: blk)
Nothing -> Bool
False
Just blk :~: blk
Refl -> (Point blk
a, Our (Tip blk)
b, Their (Tip blk)
c) (Point blk, Our (Tip blk), Their (Tip blk))
-> (Point blk, Our (Tip blk), Their (Tip blk)) -> Bool
forall a. Eq a => a -> a -> Bool
== (Point blk
Point blk
a', Our (Tip blk)
Our (Tip blk)
b', Their (Tip blk)
Their (Tip blk)
c')
RolledBackPastIntersection{} == ChainSyncClientResult
_ = Bool
False
ChainSyncClientResult
AskedToTerminate == ChainSyncClientResult
AskedToTerminate = Bool
True
ChainSyncClientResult
AskedToTerminate == ChainSyncClientResult
_ = Bool
False
data ChainSyncClientException =
forall blk. (BlockSupportsProtocol blk, ValidateEnvelope blk) =>
(Point blk)
(HeaderError blk)
(Our (Tip blk))
(Their (Tip blk))
| forall blk. BlockSupportsProtocol blk =>
InvalidIntersection
(Point blk)
(Our (Tip blk))
(Their (Tip blk))
| forall blk. BlockSupportsProtocol blk =>
DoesntFit
(ChainHash blk)
(ChainHash blk)
(Our (Tip blk))
(Their (Tip blk))
| forall blk. LedgerSupportsProtocol blk =>
InvalidBlock
(Point blk)
(HeaderHash blk)
(InvalidBlockReason blk)
deriving instance Show ChainSyncClientException
instance Eq ChainSyncClientException where
HeaderError (Point blk
a :: Point blk) HeaderError blk
b Our (Tip blk)
c Their (Tip blk)
d == :: ChainSyncClientException -> ChainSyncClientException -> Bool
== HeaderError (Point blk
a' :: Point blk') HeaderError blk
b' Our (Tip blk)
c' Their (Tip blk)
d' =
case (Typeable blk, Typeable blk) => Maybe (blk :~: blk)
forall k (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
eqT @blk @blk' of
Maybe (blk :~: blk)
Nothing -> Bool
False
Just blk :~: blk
Refl -> (Point blk
a, HeaderError blk
b, Our (Tip blk)
c, Their (Tip blk)
d) (Point blk, HeaderError blk, Our (Tip blk), Their (Tip blk))
-> (Point blk, HeaderError blk, Our (Tip blk), Their (Tip blk))
-> Bool
forall a. Eq a => a -> a -> Bool
== (Point blk
Point blk
a', HeaderError blk
HeaderError blk
b', Our (Tip blk)
Our (Tip blk)
c', Their (Tip blk)
Their (Tip blk)
d')
HeaderError{} == ChainSyncClientException
_ = Bool
False
InvalidIntersection (Point blk
a :: Point blk) Our (Tip blk)
b Their (Tip blk)
c == InvalidIntersection (Point blk
a' :: Point blk') Our (Tip blk)
b' Their (Tip blk)
c' =
case (Typeable blk, Typeable blk) => Maybe (blk :~: blk)
forall k (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
eqT @blk @blk' of
Maybe (blk :~: blk)
Nothing -> Bool
False
Just blk :~: blk
Refl -> (Point blk
a, Our (Tip blk)
b, Their (Tip blk)
c) (Point blk, Our (Tip blk), Their (Tip blk))
-> (Point blk, Our (Tip blk), Their (Tip blk)) -> Bool
forall a. Eq a => a -> a -> Bool
== (Point blk
Point blk
a', Our (Tip blk)
Our (Tip blk)
b', Their (Tip blk)
Their (Tip blk)
c')
InvalidIntersection{} == ChainSyncClientException
_ = Bool
False
DoesntFit (ChainHash blk
a :: ChainHash blk) ChainHash blk
b Our (Tip blk)
c Their (Tip blk)
d == DoesntFit (ChainHash blk
a' :: ChainHash blk') ChainHash blk
b' Our (Tip blk)
c' Their (Tip blk)
d' =
case (Typeable blk, Typeable blk) => Maybe (blk :~: blk)
forall k (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
eqT @blk @blk' of
Maybe (blk :~: blk)
Nothing -> Bool
False
Just blk :~: blk
Refl -> (ChainHash blk
a, ChainHash blk
b, Our (Tip blk)
c, Their (Tip blk)
d) (ChainHash blk, ChainHash blk, Our (Tip blk), Their (Tip blk))
-> (ChainHash blk, ChainHash blk, Our (Tip blk), Their (Tip blk))
-> Bool
forall a. Eq a => a -> a -> Bool
== (ChainHash blk
ChainHash blk
a', ChainHash blk
ChainHash blk
b', Our (Tip blk)
Our (Tip blk)
c', Their (Tip blk)
Their (Tip blk)
d')
DoesntFit{} == ChainSyncClientException
_ = Bool
False
InvalidBlock (Point blk
a :: Point blk) HeaderHash blk
b InvalidBlockReason blk
c == InvalidBlock (Point blk
a' :: Point blk') HeaderHash blk
b' InvalidBlockReason blk
c' =
case (Typeable blk, Typeable blk) => Maybe (blk :~: blk)
forall k (a :: k) (b :: k).
(Typeable a, Typeable b) =>
Maybe (a :~: b)
eqT @blk @blk' of
Maybe (blk :~: blk)
Nothing -> Bool
False
Just blk :~: blk
Refl -> (Point blk
a, HeaderHash blk
b, InvalidBlockReason blk
c) (Point blk, HeaderHash blk, InvalidBlockReason blk)
-> (Point blk, HeaderHash blk, InvalidBlockReason blk) -> Bool
forall a. Eq a => a -> a -> Bool
== (Point blk
Point blk
a', HeaderHash blk
HeaderHash blk
b', InvalidBlockReason blk
InvalidBlockReason blk
c')
InvalidBlock{} == ChainSyncClientException
_ = Bool
False
instance Exception ChainSyncClientException
data TraceChainSyncClientEvent blk
= (Header blk)
| TraceRolledBack (Point blk)
| TraceFoundIntersection (Point blk) (Our (Tip blk)) (Their (Tip blk))
| TraceException ChainSyncClientException
| TraceTermination ChainSyncClientResult
deriving instance ( BlockSupportsProtocol blk
, Eq (ValidationErr (BlockProtocol blk))
, Eq (Header blk)
)
=> Eq (TraceChainSyncClientEvent blk)
deriving instance ( BlockSupportsProtocol blk
, Show (Header blk)
)
=> Show (TraceChainSyncClientEvent blk)