{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE MultiWayIf #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE TypeOperators #-}
{-# LANGUAGE UndecidableInstances #-}
module Ouroboros.Consensus.Storage.ChainDB.Impl.ChainSel (
addBlockAsync
, chainSelSync
, chainSelectionForBlock
, initialChainSelection
, triggerChainSelectionAsync
, olderThanK
) where
import Cardano.Ledger.BaseTypes (unNonZero)
import Control.Exception (assert)
import Control.Monad (forM, forM_, when)
import Control.Monad.Except ()
import Control.Monad.Trans.Class (lift)
import Control.Monad.Trans.State.Strict
import Control.ResourceRegistry (ResourceRegistry, withRegistry)
import Control.Tracer (Tracer, nullTracer, traceWith)
import Data.Foldable (for_)
import Data.Function (on)
import Data.Functor.Contravariant ((>$<))
import Data.List (sortBy)
import Data.List.NonEmpty (NonEmpty)
import qualified Data.List.NonEmpty as NE
import Data.Map.Strict (Map)
import qualified Data.Map.Strict as Map
import Data.Maybe (fromJust, isJust, isNothing)
import Data.Maybe.Strict (StrictMaybe (..), strictMaybeToMaybe)
import Data.Set (Set)
import qualified Data.Set as Set
import GHC.Stack (HasCallStack)
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.Config
import Ouroboros.Consensus.Fragment.Diff (ChainDiff (..))
import qualified Ouroboros.Consensus.Fragment.Diff as Diff
import Ouroboros.Consensus.Fragment.Validated (ValidatedFragment)
import qualified Ouroboros.Consensus.Fragment.Validated as VF
import Ouroboros.Consensus.Fragment.ValidatedDiff
(ValidatedChainDiff (..))
import qualified Ouroboros.Consensus.Fragment.ValidatedDiff as ValidatedDiff
import Ouroboros.Consensus.HardFork.Abstract
import qualified Ouroboros.Consensus.HardFork.History as History
import Ouroboros.Consensus.HeaderValidation (HeaderWithTime (..),
mkHeaderWithTime)
import Ouroboros.Consensus.Ledger.Abstract
import Ouroboros.Consensus.Ledger.Extended
import Ouroboros.Consensus.Ledger.Inspect
import Ouroboros.Consensus.Ledger.SupportsProtocol
import Ouroboros.Consensus.Storage.ChainDB.API (AddBlockPromise (..),
AddBlockResult (..), BlockComponent (..), ChainType (..),
LoE (..))
import Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment
(InvalidBlockPunishment, noPunishment)
import qualified Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment as InvalidBlockPunishment
import Ouroboros.Consensus.Storage.ChainDB.Impl.BlockCache
(BlockCache)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.BlockCache as BlockCache
import Ouroboros.Consensus.Storage.ChainDB.Impl.Paths
(LookupBlockInfo)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Paths as Paths
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.Query as Query
import Ouroboros.Consensus.Storage.ChainDB.Impl.Types
import Ouroboros.Consensus.Storage.ImmutableDB (ImmutableDB)
import qualified Ouroboros.Consensus.Storage.ImmutableDB as ImmutableDB
import Ouroboros.Consensus.Storage.LedgerDB
import qualified Ouroboros.Consensus.Storage.LedgerDB as LedgerDB
import Ouroboros.Consensus.Storage.VolatileDB (VolatileDB)
import qualified Ouroboros.Consensus.Storage.VolatileDB as VolatileDB
import Ouroboros.Consensus.Util
import Ouroboros.Consensus.Util.AnchoredFragment
import Ouroboros.Consensus.Util.Enclose (encloseWith)
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.STM (WithFingerprint (..))
import Ouroboros.Network.AnchoredFragment (Anchor, AnchoredFragment,
AnchoredSeq (..))
import qualified Ouroboros.Network.AnchoredFragment as AF
import qualified Ouroboros.Network.AnchoredSeq as AS
import Ouroboros.Network.Protocol.LocalStateQuery.Type (Target (..))
initialChainSelection ::
forall m blk.
( IOLike m
, LedgerSupportsProtocol blk
, BlockSupportsDiffusionPipelining blk
)
=> ImmutableDB m blk
-> VolatileDB m blk
-> LedgerDB.LedgerDB' m blk
-> ResourceRegistry m
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> LoE ()
-> m (ChainAndLedger m blk)
initialChainSelection :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
BlockSupportsDiffusionPipelining blk) =>
ImmutableDB m blk
-> VolatileDB m blk
-> LedgerDB' m blk
-> ResourceRegistry m
-> Tracer m (TraceInitChainSelEvent blk)
-> TopLevelConfig blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> LoE ()
-> m (ChainAndLedger m blk)
initialChainSelection ImmutableDB m blk
immutableDB VolatileDB m blk
volatileDB LedgerDB' m blk
lgrDB ResourceRegistry m
rr Tracer m (TraceInitChainSelEvent blk)
tracer TopLevelConfig blk
cfg StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
LoE ()
loE = do
(i :: Anchor blk, succsOf) <- STM m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
-> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
-> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk)))
-> STM m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
-> m (Anchor blk, ChainHash blk -> Set (HeaderHash blk))
forall a b. (a -> b) -> a -> b
$ do
invalid <- WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
(,)
<$> ImmutableDB.getTipAnchor immutableDB
<*> (ignoreInvalidSuc volatileDB invalid <$>
VolatileDB.filterByPredecessor volatileDB)
curForker <- LedgerDB.getForkerAtTarget lgrDB rr VolatileTip >>= \case
Left{} -> [Char] -> m (Forker' m blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"Unreachable, VolatileTip MUST be in the LedgerDB"
Right Forker' m blk
frk -> Forker' m blk -> m (Forker' m blk)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Forker' m blk
frk
chains <- constructChains i succsOf
let curChain = Anchor (Header blk) -> AnchoredFragment (Header blk)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
Empty (Anchor blk -> Anchor (Header blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor Anchor blk
i)
curChainAndLedger <- VF.newM curChain curForker
case NE.nonEmpty (filter (preferAnchoredCandidate bcfg curChain) chains) of
Maybe (NonEmpty (AnchoredFragment (Header blk)))
Nothing -> ValidatedFragment (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
Just NonEmpty (AnchoredFragment (Header blk))
chains' ->
HasCallStack =>
ValidatedFragment (Header blk) (Forker' m blk)
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
ValidatedFragment (Header blk) (Forker' m blk)
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection' ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger NonEmpty (AnchoredFragment (Header blk))
chains' m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (ValidatedFragment (Header blk) (Forker' m blk)))
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
Nothing -> ValidatedFragment (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
Just ValidatedChainDiff (Header blk) (Forker' m blk)
newChain -> Forker' m blk -> m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> m ()
forkerClose Forker' m blk
curForker m ()
-> m (ValidatedFragment (Header blk) (Forker' m blk))
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a b. m a -> m b -> m b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> ValidatedChainDiff (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
toChainAndLedger ValidatedChainDiff (Header blk) (Forker' m blk)
newChain
where
bcfg :: BlockConfig blk
bcfg :: BlockConfig blk
bcfg = TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg
SecurityParam NonZero Word64
k = TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cfg
toChainAndLedger
:: ValidatedChainDiff (Header blk) (Forker' m blk)
-> m (ChainAndLedger m blk)
toChainAndLedger :: ValidatedChainDiff (Header blk) (Forker' m blk)
-> m (ValidatedFragment (Header blk) (Forker' m blk))
toChainAndLedger (ValidatedChainDiff ChainDiff (Header blk)
chainDiff Forker' m blk
ledger) =
case ChainDiff (Header blk)
chainDiff of
ChainDiff Word64
rollback AnchoredFragment (Header blk)
suffix
| Word64
rollback Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
0
-> AnchoredFragment (Header blk)
-> Forker' m blk
-> m (ValidatedFragment (Header blk) (Forker' m blk))
forall (m :: * -> *) l b.
(MonadSTM m, GetTipSTM m l, HasHeader b,
HeaderHash b ~ HeaderHash l, HasCallStack) =>
AnchoredFragment b -> l -> m (ValidatedFragment b l)
VF.newM AnchoredFragment (Header blk)
suffix Forker' m blk
ledger
| Bool
otherwise
-> [Char] -> m (ValidatedFragment (Header blk) (Forker' m blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"constructed an initial chain with rollback"
constructChains ::
Anchor blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> m [AnchoredFragment (Header blk)]
constructChains :: Anchor blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> m [AnchoredFragment (Header blk)]
constructChains Anchor blk
i ChainHash blk -> Set (HeaderHash blk)
succsOf = (StateT
(Map (HeaderHash blk) (Header blk))
m
[AnchoredFragment (Header blk)]
-> Map (HeaderHash blk) (Header blk)
-> m [AnchoredFragment (Header blk)])
-> Map (HeaderHash blk) (Header blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
[AnchoredFragment (Header blk)]
-> m [AnchoredFragment (Header blk)]
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
(Map (HeaderHash blk) (Header blk))
m
[AnchoredFragment (Header blk)]
-> Map (HeaderHash blk) (Header blk)
-> m [AnchoredFragment (Header blk)]
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m a
evalStateT Map (HeaderHash blk) (Header blk)
forall k a. Map k a
Map.empty (StateT
(Map (HeaderHash blk) (Header blk))
m
[AnchoredFragment (Header blk)]
-> m [AnchoredFragment (Header blk)])
-> StateT
(Map (HeaderHash blk) (Header blk))
m
[AnchoredFragment (Header blk)]
-> m [AnchoredFragment (Header blk)]
forall a b. (a -> b) -> a -> b
$
(NonEmpty (HeaderHash blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(AnchoredFragment (Header blk)))
-> [NonEmpty (HeaderHash blk)]
-> StateT
(Map (HeaderHash blk) (Header blk))
m
[AnchoredFragment (Header blk)]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM NonEmpty (HeaderHash blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(AnchoredFragment (Header blk))
constructChain [NonEmpty (HeaderHash blk)]
suffixesAfterI
where
suffixesAfterI :: [NonEmpty (HeaderHash blk)]
suffixesAfterI :: [NonEmpty (HeaderHash blk)]
suffixesAfterI = (ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
forall blk.
(ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
Paths.maximalCandidates ChainHash blk -> Set (HeaderHash blk)
succsOf (NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero (NonZero Word64 -> Word64)
-> Maybe (NonZero Word64) -> Maybe Word64
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Maybe (NonZero Word64)
limit) (Anchor blk -> Point blk
forall block. Anchor block -> Point block
AF.anchorToPoint Anchor blk
i)
where
limit :: Maybe (NonZero Word64)
limit = case LoE ()
loE of
LoE ()
LoEDisabled -> Maybe (NonZero Word64)
forall a. Maybe a
Nothing
LoEEnabled () -> NonZero Word64 -> Maybe (NonZero Word64)
forall a. a -> Maybe a
Just NonZero Word64
k
constructChain ::
NonEmpty (HeaderHash blk)
-> StateT (Map (HeaderHash blk) (Header blk))
m
(AnchoredFragment (Header blk))
constructChain :: NonEmpty (HeaderHash blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(AnchoredFragment (Header blk))
constructChain NonEmpty (HeaderHash blk)
hashes =
Anchor (Header blk)
-> [Header blk] -> AnchoredFragment (Header blk)
forall v a b. Anchorable v a b => a -> [b] -> AnchoredSeq v a b
AF.fromOldestFirst (Anchor blk -> Anchor (Header blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor Anchor blk
i) ([Header blk] -> AnchoredFragment (Header blk))
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(AnchoredFragment (Header blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
(HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> [HeaderHash blk]
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM (VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) blk.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache VolatileDB m blk
volatileDB) (NonEmpty (HeaderHash blk) -> [HeaderHash blk]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (HeaderHash blk)
hashes)
chainSelection' ::
HasCallStack
=> ChainAndLedger m blk
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection' :: HasCallStack =>
ValidatedFragment (Header blk) (Forker' m blk)
-> NonEmpty (AnchoredFragment (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection' ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger NonEmpty (AnchoredFragment (Header blk))
candidates =
STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (Forker' m blk -> STM m (Point blk)
forall (l :: LedgerStateKind) blk (m :: * -> *).
(GetTip l, HeaderHash l ~ HeaderHash blk, Functor (STM m)) =>
Forker m l blk -> STM m (Point blk)
forkerCurrentPoint Forker' m blk
ledger) m (Point blk)
-> (Point blk
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \Point blk
curpt ->
Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((AnchoredFragment (Header blk) -> Bool)
-> NonEmpty (AnchoredFragment (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all ((Point blk
curpt Point blk -> Point blk -> Bool
forall a. Eq a => a -> a -> Bool
==) (Point blk -> Bool)
-> (AnchoredFragment (Header blk) -> Point blk)
-> AnchoredFragment (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (Header blk) -> Point blk)
-> (AnchoredFragment (Header blk) -> Point (Header blk))
-> AnchoredFragment (Header blk)
-> Point blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment (Header blk) -> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint) NonEmpty (AnchoredFragment (Header blk))
candidates) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((AnchoredFragment (Header blk) -> Bool)
-> NonEmpty (AnchoredFragment (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all (BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain) NonEmpty (AnchoredFragment (Header blk))
candidates) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$ do
cse <- m (ChainSelEnv m blk)
chainSelEnv
chainSelection cse rr (Diff.extend <$> candidates)
where
curChain :: AnchoredFragment (Header blk)
curChain = ValidatedFragment (Header blk) (Forker' m blk)
-> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
ledger :: Forker' m blk
ledger = ValidatedFragment (Header blk) (Forker' m blk) -> Forker' m blk
forall b l. ValidatedFragment b l -> l
VF.validatedLedger ValidatedFragment (Header blk) (Forker' m blk)
curChainAndLedger
chainSelEnv :: m (ChainSelEnv m blk)
chainSelEnv = do
varTentativeState <- TentativeHeaderState blk
-> m (StrictTVar m (TentativeHeaderState blk))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (Proxy blk -> TentativeHeaderState blk
forall blk.
BlockSupportsDiffusionPipelining blk =>
Proxy blk -> TentativeHeaderState blk
initialTentativeHeaderState (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @blk))
varTentativeHeader <- newTVarIO SNothing
pure ChainSelEnv
{ lgrDB
, bcfg
, varInvalid
, blockCache = BlockCache.empty
, curChainAndLedger
, validationTracer = InitChainSelValidation >$< tracer
, pipeliningTracer = nullTracer
, varTentativeState
, varTentativeHeader
, punish = Nothing
, getTentativeFollowers = pure []
}
addBlockAsync ::
forall m blk. (IOLike m, HasHeader blk)
=> ChainDbEnv m blk
-> InvalidBlockPunishment m
-> blk
-> m (AddBlockPromise m blk)
addBlockAsync :: forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
ChainDbEnv m blk
-> InvalidBlockPunishment m -> blk -> m (AddBlockPromise m blk)
addBlockAsync CDB { Tracer m (TraceEvent blk)
cdbTracer :: Tracer m (TraceEvent blk)
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbTracer, ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbChainSelQueue :: ChainSelQueue m blk
cdbChainSelQueue } =
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk
-> InvalidBlockPunishment m
-> blk
-> m (AddBlockPromise m blk)
forall (m :: * -> *) blk.
(IOLike m, HasHeader blk) =>
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk
-> InvalidBlockPunishment m
-> blk
-> m (AddBlockPromise m blk)
addBlockToAdd (TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer) ChainSelQueue m blk
cdbChainSelQueue
triggerChainSelectionAsync ::
forall m blk.
IOLike m =>
ChainDbEnv m blk ->
m (ChainSelectionPromise m)
triggerChainSelectionAsync :: forall (m :: * -> *) blk.
IOLike m =>
ChainDbEnv m blk -> m (ChainSelectionPromise m)
triggerChainSelectionAsync CDB {Tracer m (TraceEvent blk)
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbTracer :: Tracer m (TraceEvent blk)
cdbTracer, ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbChainSelQueue :: ChainSelQueue m blk
cdbChainSelQueue} =
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk -> m (ChainSelectionPromise m)
forall (m :: * -> *) blk.
IOLike m =>
Tracer m (TraceAddBlockEvent blk)
-> ChainSelQueue m blk -> m (ChainSelectionPromise m)
addReprocessLoEBlocks (TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer) ChainSelQueue m blk
cdbChainSelQueue
chainSelSync ::
forall m blk.
( IOLike m
, LedgerSupportsProtocol blk
, BlockSupportsDiffusionPipelining blk
, InspectLedger blk
, HasHardForkHistory blk
, HasCallStack
)
=> ChainDbEnv m blk
-> ChainSelMessage m blk
-> Electric m ()
chainSelSync :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
BlockSupportsDiffusionPipelining blk, InspectLedger blk,
HasHardForkHistory blk, HasCallStack) =>
ChainDbEnv m blk -> ChainSelMessage m blk -> Electric m ()
chainSelSync cdb :: ChainDbEnv m blk
cdb@CDB{m (LoE (AnchoredFragment (HeaderWithTime blk)))
Tracer m (TraceEvent blk)
DiffTime
ResourceRegistry m
StrictTVar m (m ())
StrictTVar m (Map FollowerKey (FollowerHandle m blk))
StrictTVar m (Map IteratorKey (m ()))
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m ChainSelStarvation
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
StrictTVar m (InternalChain blk)
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LedgerDB' m blk
ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLedgerDB :: LedgerDB' m blk
cdbChain :: StrictTVar m (InternalChain blk)
cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: TopLevelConfig blk
cdbInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbCopyFuse :: Fuse m
cdbChainSelFuse :: Fuse m
cdbTracer :: Tracer m (TraceEvent blk)
cdbRegistry :: ResourceRegistry m
cdbGcDelay :: DiffTime
cdbGcInterval :: DiffTime
cdbKillBgThreads :: StrictTVar m (m ())
cdbChainSelQueue :: ChainSelQueue m blk
cdbLoE :: m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
cdbChainSelStarvation :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbLedgerDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
..} (ChainSelReprocessLoEBlocks StrictTMVar m ()
varProcessed) = do
m (LoE (AnchoredFragment (HeaderWithTime blk)))
-> Electric m (LoE (AnchoredFragment (HeaderWithTime blk)))
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbLoE Electric m (LoE (AnchoredFragment (HeaderWithTime blk)))
-> (LoE (AnchoredFragment (HeaderWithTime blk)) -> Electric m ())
-> Electric m ()
forall a b. Electric m a -> (a -> Electric m b) -> Electric m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
LoE (AnchoredFragment (HeaderWithTime blk))
LoEDisabled -> () -> Electric m ()
forall a. a -> Electric m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
LoEEnabled AnchoredFragment (HeaderWithTime blk)
_ -> do
(succsOf, chain) <- m (ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
m
(ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
m
(ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
m
(ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ STM
m
(ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
m
(ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM
m
(ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (ChainHash blk -> Set (HeaderHash blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ do
invalid <- WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid
(,)
<$> (ignoreInvalidSuc cdbVolatileDB invalid <$>
VolatileDB.filterByPredecessor cdbVolatileDB)
<*> Query.getCurrentChain cdb
let
succsOf' = Set (HeaderHash blk) -> [HeaderHash blk]
forall a. Set a -> [a]
Set.toList (Set (HeaderHash blk) -> [HeaderHash blk])
-> (Point (Header blk) -> Set (HeaderHash blk))
-> Point (Header blk)
-> [HeaderHash blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainHash blk -> Set (HeaderHash blk)
succsOf (ChainHash blk -> Set (HeaderHash blk))
-> (Point (Header blk) -> ChainHash blk)
-> Point (Header blk)
-> Set (HeaderHash blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point blk -> ChainHash blk
forall {k} (block :: k). Point block -> ChainHash block
pointHash (Point blk -> ChainHash blk)
-> (Point (Header blk) -> Point blk)
-> Point (Header blk)
-> ChainHash blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint
loeHashes = Point (Header blk) -> [HeaderHash blk]
succsOf' (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
chain)
firstHeader = (Anchor (Header blk) -> Maybe (Header blk))
-> (Header blk -> Maybe (Header blk))
-> Either (Anchor (Header blk)) (Header blk)
-> Maybe (Header blk)
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either (Maybe (Header blk) -> Anchor (Header blk) -> Maybe (Header blk)
forall a b. a -> b -> a
const Maybe (Header blk)
forall a. Maybe a
Nothing) Header blk -> Maybe (Header blk)
forall a. a -> Maybe a
Just (Either (Anchor (Header blk)) (Header blk) -> Maybe (Header blk))
-> Either (Anchor (Header blk)) (Header blk) -> Maybe (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Either (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Either a b
AF.last AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
chain
getHeaderFromHash HeaderHash blk
hash =
case Maybe (Header blk)
firstHeader of
Just Header blk
header | Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
header HeaderHash blk -> HeaderHash blk -> Bool
forall a. Eq a => a -> a -> Bool
== HeaderHash blk
hash -> Header blk -> m (Header blk)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Header blk
header
Maybe (Header blk)
_ -> VolatileDB m blk
-> BlockComponent blk (Header blk)
-> HeaderHash blk
-> m (Header blk)
forall (m :: * -> *) blk b.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk -> BlockComponent blk b -> HeaderHash blk -> m b
VolatileDB.getKnownBlockComponent VolatileDB m blk
cdbVolatileDB BlockComponent blk (Header blk)
forall blk. BlockComponent blk (Header blk)
GetHeader HeaderHash blk
hash
loeHeaders <- lift (mapM getHeaderFromHash loeHashes)
for_ loeHeaders $ \Header blk
hdr ->
ChainDbEnv m blk
-> BlockCache blk
-> Header blk
-> InvalidBlockPunishment m
-> Electric m ()
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
BlockSupportsDiffusionPipelining blk, InspectLedger blk,
HasHardForkHistory blk, HasCallStack) =>
ChainDbEnv m blk
-> BlockCache blk
-> Header blk
-> InvalidBlockPunishment m
-> Electric m ()
chainSelectionForBlock ChainDbEnv m blk
cdb BlockCache blk
forall blk. BlockCache blk
BlockCache.empty Header blk
hdr InvalidBlockPunishment m
forall (m :: * -> *). Applicative m => InvalidBlockPunishment m
noPunishment
m () -> Electric m ()
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m () -> Electric m ()) -> m () -> Electric m ()
forall a b. (a -> b) -> a -> b
$ STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
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
$ StrictTMVar m () -> () -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m ()
putTMVar StrictTMVar m ()
varProcessed ()
chainSelSync cdb :: ChainDbEnv m blk
cdb@CDB {m (LoE (AnchoredFragment (HeaderWithTime blk)))
Tracer m (TraceEvent blk)
DiffTime
ResourceRegistry m
StrictTVar m (m ())
StrictTVar m (Map FollowerKey (FollowerHandle m blk))
StrictTVar m (Map IteratorKey (m ()))
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m ChainSelStarvation
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
StrictTVar m (InternalChain blk)
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LedgerDB' m blk
ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbChainSelStarvation :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbLedgerDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLedgerDB :: LedgerDB' m blk
cdbChain :: StrictTVar m (InternalChain blk)
cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: TopLevelConfig blk
cdbInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbCopyFuse :: Fuse m
cdbChainSelFuse :: Fuse m
cdbTracer :: Tracer m (TraceEvent blk)
cdbRegistry :: ResourceRegistry m
cdbGcDelay :: DiffTime
cdbGcInterval :: DiffTime
cdbKillBgThreads :: StrictTVar m (m ())
cdbChainSelQueue :: ChainSelQueue m blk
cdbLoE :: m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
..} (ChainSelAddBlock BlockToAdd { blockToAdd :: forall (m :: * -> *) blk. BlockToAdd m blk -> blk
blockToAdd = blk
b, StrictTMVar m Bool
StrictTMVar m (AddBlockResult blk)
InvalidBlockPunishment m
blockPunish :: InvalidBlockPunishment m
varBlockWrittenToDisk :: StrictTMVar m Bool
varBlockProcessed :: StrictTMVar m (AddBlockResult blk)
varBlockProcessed :: forall (m :: * -> *) blk.
BlockToAdd m blk -> StrictTMVar m (AddBlockResult blk)
varBlockWrittenToDisk :: forall (m :: * -> *) blk. BlockToAdd m blk -> StrictTMVar m Bool
blockPunish :: forall (m :: * -> *) blk.
BlockToAdd m blk -> InvalidBlockPunishment m
.. }) = do
(isMember, invalid, curChain) <- m (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
m
(HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) a. Monad m => m a -> Electric m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
m
(HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Electric
m
(HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ STM
m
(HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
m
(HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM
m
(HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. (a -> b) -> a -> b
$ (,,)
((HeaderHash blk -> Bool)
-> InvalidBlocks blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM m (HeaderHash blk -> Bool)
-> STM
m
(InvalidBlocks blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> VolatileDB m blk -> STM m (HeaderHash blk -> Bool)
forall (m :: * -> *) blk.
Functor (STM m) =>
VolatileDB m blk -> STM m (HeaderHash blk -> Bool)
VolatileDB.getIsMember VolatileDB m blk
cdbVolatileDB
STM
m
(InvalidBlocks blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM m (InvalidBlocks blk)
-> STM
m
(AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid)
STM
m
(AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> (HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> STM
m
(HeaderHash blk -> Bool, InvalidBlocks blk,
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> ChainDbEnv m blk
-> STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk),
ConsensusProtocol (BlockProtocol blk)) =>
ChainDbEnv m blk -> STM m (AnchoredFragment (Header blk))
Query.getCurrentChain ChainDbEnv m blk
cdb
let immBlockNo = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> WithOrigin BlockNo
forall block. AnchoredFragment block -> WithOrigin BlockNo
AF.anchorBlockNo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain
if
| olderThanK hdr isEBB immBlockNo -> do
lift $ traceWith addBlockTracer $ IgnoreBlockOlderThanK (blockRealPoint b)
lift $ deliverWrittenToDisk False
| isMember (blockHash b) -> do
lift $ traceWith addBlockTracer $ IgnoreBlockAlreadyInVolatileDB (blockRealPoint b)
lift $ deliverWrittenToDisk True
| Just (InvalidBlockInfo reason _) <- Map.lookup (blockHash b) invalid -> do
lift $ traceWith addBlockTracer $ IgnoreInvalidBlock (blockRealPoint b) reason
lift $ deliverWrittenToDisk False
lift $ InvalidBlockPunishment.enact
blockPunish
InvalidBlockPunishment.BlockItself
| otherwise -> do
let traceEv = RealPoint blk
-> BlockNo -> IsEBB -> Enclosing -> TraceAddBlockEvent blk
forall blk.
RealPoint blk
-> BlockNo -> IsEBB -> Enclosing -> TraceAddBlockEvent blk
AddedBlockToVolatileDB (blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint blk
b) (blk -> BlockNo
forall b. HasHeader b => b -> BlockNo
blockNo blk
b) IsEBB
isEBB
lift $ encloseWith (traceEv >$< addBlockTracer) $
VolatileDB.putBlock cdbVolatileDB b
lift $ deliverWrittenToDisk True
chainSelectionForBlock cdb (BlockCache.singleton b) hdr blockPunish
newTip <- lift $ atomically $ Query.getTipPoint cdb
lift $ deliverProcessed newTip
where
addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
addBlockTracer = TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer
hdr :: Header blk
hdr :: Header blk
hdr = blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader blk
b
isEBB :: IsEBB
isEBB :: IsEBB
isEBB = Header blk -> IsEBB
forall blk. GetHeader blk => Header blk -> IsEBB
headerToIsEBB Header blk
hdr
deliverWrittenToDisk :: Bool -> m ()
deliverWrittenToDisk :: Bool -> m ()
deliverWrittenToDisk Bool
writtenToDisk = STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
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
$
StrictTMVar m Bool -> Bool -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m ()
putTMVar StrictTMVar m Bool
varBlockWrittenToDisk Bool
writtenToDisk
deliverProcessed :: Point blk -> m ()
deliverProcessed :: Point blk -> m ()
deliverProcessed Point blk
tip = STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
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
$
StrictTMVar m (AddBlockResult blk)
-> AddBlockResult blk -> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTMVar m a -> a -> STM m ()
putTMVar StrictTMVar m (AddBlockResult blk)
varBlockProcessed (Point blk -> AddBlockResult blk
forall blk. Point blk -> AddBlockResult blk
SuccesfullyAddedBlock Point blk
tip)
olderThanK ::
HasHeader (Header blk)
=> Header blk
-> IsEBB
-> WithOrigin BlockNo
-> Bool
olderThanK :: forall blk.
HasHeader (Header blk) =>
Header blk -> IsEBB -> WithOrigin BlockNo -> Bool
olderThanK Header blk
hdr IsEBB
isEBB WithOrigin BlockNo
immBlockNo
| BlockNo -> WithOrigin BlockNo
forall t. t -> WithOrigin t
NotOrigin BlockNo
bNo WithOrigin BlockNo -> WithOrigin BlockNo -> Bool
forall a. Eq a => a -> a -> Bool
== WithOrigin BlockNo
immBlockNo
, IsEBB
isEBB IsEBB -> IsEBB -> Bool
forall a. Eq a => a -> a -> Bool
== IsEBB
IsEBB
= Bool
False
| Bool
otherwise
= BlockNo -> WithOrigin BlockNo
forall t. t -> WithOrigin t
NotOrigin BlockNo
bNo WithOrigin BlockNo -> WithOrigin BlockNo -> Bool
forall a. Ord a => a -> a -> Bool
<= WithOrigin BlockNo
immBlockNo
where
bNo :: BlockNo
bNo = Header blk -> BlockNo
forall b. HasHeader b => b -> BlockNo
blockNo Header blk
hdr
data ChainSwitchType = AddingBlocks | SwitchingToAFork
deriving (Int -> ChainSwitchType -> ShowS
[ChainSwitchType] -> ShowS
ChainSwitchType -> [Char]
(Int -> ChainSwitchType -> ShowS)
-> (ChainSwitchType -> [Char])
-> ([ChainSwitchType] -> ShowS)
-> Show ChainSwitchType
forall a.
(Int -> a -> ShowS) -> (a -> [Char]) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> ChainSwitchType -> ShowS
showsPrec :: Int -> ChainSwitchType -> ShowS
$cshow :: ChainSwitchType -> [Char]
show :: ChainSwitchType -> [Char]
$cshowList :: [ChainSwitchType] -> ShowS
showList :: [ChainSwitchType] -> ShowS
Show, ChainSwitchType -> ChainSwitchType -> Bool
(ChainSwitchType -> ChainSwitchType -> Bool)
-> (ChainSwitchType -> ChainSwitchType -> Bool)
-> Eq ChainSwitchType
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: ChainSwitchType -> ChainSwitchType -> Bool
== :: ChainSwitchType -> ChainSwitchType -> Bool
$c/= :: ChainSwitchType -> ChainSwitchType -> Bool
/= :: ChainSwitchType -> ChainSwitchType -> Bool
Eq)
chainSelectionForBlock ::
forall m blk.
( IOLike m
, LedgerSupportsProtocol blk
, BlockSupportsDiffusionPipelining blk
, InspectLedger blk
, HasHardForkHistory blk
, HasCallStack
)
=> ChainDbEnv m blk
-> BlockCache blk
-> Header blk
-> InvalidBlockPunishment m
-> Electric m ()
chainSelectionForBlock :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
BlockSupportsDiffusionPipelining blk, InspectLedger blk,
HasHardForkHistory blk, HasCallStack) =>
ChainDbEnv m blk
-> BlockCache blk
-> Header blk
-> InvalidBlockPunishment m
-> Electric m ()
chainSelectionForBlock cdb :: ChainDbEnv m blk
cdb@CDB{m (LoE (AnchoredFragment (HeaderWithTime blk)))
Tracer m (TraceEvent blk)
DiffTime
ResourceRegistry m
StrictTVar m (m ())
StrictTVar m (Map FollowerKey (FollowerHandle m blk))
StrictTVar m (Map IteratorKey (m ()))
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m ChainSelStarvation
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
StrictTVar m FollowerKey
StrictTVar m IteratorKey
StrictTVar m (InternalChain blk)
Fuse m
TopLevelConfig blk
VolatileDB m blk
ImmutableDB m blk
LedgerDB' m blk
ChainSelQueue m blk
cdbChainSelQueue :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ChainSelQueue m blk
cdbTracer :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> Tracer m (TraceEvent blk)
cdbChainSelStarvation :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m ChainSelStarvation
cdbLoE :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbKillBgThreads :: forall (m :: * -> *) blk. ChainDbEnv m blk -> StrictTVar m (m ())
cdbGcInterval :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbGcDelay :: forall (m :: * -> *) blk. ChainDbEnv m blk -> DiffTime
cdbRegistry :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ResourceRegistry m
cdbChainSelFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbCopyFuse :: forall (m :: * -> *) blk. ChainDbEnv m blk -> Fuse m
cdbNextFollowerKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m FollowerKey
cdbNextIteratorKey :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m IteratorKey
cdbInvalid :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbTopLevelConfig :: forall (m :: * -> *) blk. ChainDbEnv m blk -> TopLevelConfig blk
cdbFollowers :: forall (m :: * -> *) blk.
ChainDbEnv m blk
-> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbIterators :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (Map IteratorKey (m ()))
cdbTentativeHeader :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
cdbTentativeState :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (TentativeHeaderState blk)
cdbChain :: forall (m :: * -> *) blk.
ChainDbEnv m blk -> StrictTVar m (InternalChain blk)
cdbLedgerDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> LedgerDB' m blk
cdbVolatileDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> VolatileDB m blk
cdbImmutableDB :: forall (m :: * -> *) blk. ChainDbEnv m blk -> ImmutableDB m blk
cdbImmutableDB :: ImmutableDB m blk
cdbVolatileDB :: VolatileDB m blk
cdbLedgerDB :: LedgerDB' m blk
cdbChain :: StrictTVar m (InternalChain blk)
cdbTentativeState :: StrictTVar m (TentativeHeaderState blk)
cdbTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
cdbIterators :: StrictTVar m (Map IteratorKey (m ()))
cdbFollowers :: StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbTopLevelConfig :: TopLevelConfig blk
cdbInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbNextIteratorKey :: StrictTVar m IteratorKey
cdbNextFollowerKey :: StrictTVar m FollowerKey
cdbCopyFuse :: Fuse m
cdbChainSelFuse :: Fuse m
cdbTracer :: Tracer m (TraceEvent blk)
cdbRegistry :: ResourceRegistry m
cdbGcDelay :: DiffTime
cdbGcInterval :: DiffTime
cdbKillBgThreads :: StrictTVar m (m ())
cdbChainSelQueue :: ChainSelQueue m blk
cdbLoE :: m (LoE (AnchoredFragment (HeaderWithTime blk)))
cdbChainSelStarvation :: StrictTVar m ChainSelStarvation
..} BlockCache blk
blockCache Header blk
hdr InvalidBlockPunishment m
punish = m () -> Electric m ()
forall {k} (m :: k -> *) (a :: k). m a -> Electric m a
electric (m () -> Electric m ()) -> m () -> Electric m ()
forall a b. (a -> b) -> a -> b
$ (ResourceRegistry m -> m ()) -> m ()
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
(ResourceRegistry m -> m a) -> m a
withRegistry ((ResourceRegistry m -> m ()) -> m ())
-> (ResourceRegistry m -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \ResourceRegistry m
rr -> do
(invalid, succsOf, lookupBlockInfo, curChain, tipPoint)
<- STM
m
(InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk)
-> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
m
(InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk)
-> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
-> STM
m
(InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk)
-> m (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk)
forall a b. (a -> b) -> a -> b
$ (,,,,)
(InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
-> STM m (InvalidBlocks blk)
-> STM
m
((ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk)
-> STM m (WithFingerprint (InvalidBlocks blk))
-> STM m (InvalidBlocks blk)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid)
STM
m
((ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
-> STM m (ChainHash blk -> Set (HeaderHash blk))
-> STM
m
((HeaderHash blk -> Maybe (BlockInfo blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> VolatileDB m blk
-> HasCallStack => STM m (ChainHash blk -> Set (HeaderHash blk))
forall (m :: * -> *) blk.
VolatileDB m blk
-> HasCallStack => STM m (ChainHash blk -> Set (HeaderHash blk))
VolatileDB.filterByPredecessor VolatileDB m blk
cdbVolatileDB
STM
m
((HeaderHash blk -> Maybe (BlockInfo blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
-> STM m (HeaderHash blk -> Maybe (BlockInfo blk))
-> STM
m
(AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> VolatileDB m blk
-> HasCallStack => STM m (HeaderHash blk -> Maybe (BlockInfo blk))
forall (m :: * -> *) blk.
VolatileDB m blk
-> HasCallStack => STM m (HeaderHash blk -> Maybe (BlockInfo blk))
VolatileDB.getBlockInfo VolatileDB m blk
cdbVolatileDB
STM
m
(AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
-> STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> STM
m
(Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> ChainDbEnv m blk
-> STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk),
ConsensusProtocol (BlockProtocol blk)) =>
ChainDbEnv m blk -> STM m (AnchoredFragment (Header blk))
Query.getCurrentChain ChainDbEnv m blk
cdb
STM
m
(Point blk
-> (InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk))
-> STM m (Point blk)
-> STM
m
(InvalidBlocks blk, ChainHash blk -> Set (HeaderHash blk),
HeaderHash blk -> Maybe (BlockInfo blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
Point blk)
forall a b. STM m (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*> ChainDbEnv m blk -> STM m (Point blk)
forall (m :: * -> *) blk.
(IOLike m, HasHeader (Header blk)) =>
ChainDbEnv m blk -> STM m (Point blk)
Query.getTipPoint ChainDbEnv m blk
cdb
LedgerDB.withTipForker cdbLedgerDB rr $ \Forker m (ExtLedgerState blk) blk
curForker -> do
curChainAndLedger :: ChainAndLedger m blk <-
Bool -> m (ChainAndLedger m blk) -> m (ChainAndLedger m blk)
forall a. HasCallStack => Bool -> a -> a
assert (Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain) Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
<= NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero NonZero Word64
k) (m (ChainAndLedger m blk) -> m (ChainAndLedger m blk))
-> m (ChainAndLedger m blk) -> m (ChainAndLedger m blk)
forall a b. (a -> b) -> a -> b
$
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Forker m (ExtLedgerState blk) blk -> m (ChainAndLedger m blk)
forall (m :: * -> *) l b.
(MonadSTM m, GetTipSTM m l, HasHeader b,
HeaderHash b ~ HeaderHash l, HasCallStack) =>
AnchoredFragment b -> l -> m (ValidatedFragment b l)
VF.newM AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain Forker m (ExtLedgerState blk) blk
curForker
let
immBlockNo :: WithOrigin BlockNo
immBlockNo = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> WithOrigin BlockNo
forall block. AnchoredFragment block -> WithOrigin BlockNo
AF.anchorBlockNo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain
lookupBlockInfo' = ChainDbEnv m blk
-> InvalidBlocks blk
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> HeaderHash blk
-> Maybe (BlockInfo blk)
forall blk (proxy :: * -> *) a.
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (HeaderHash blk -> Maybe a)
-> HeaderHash blk
-> Maybe a
ignoreInvalid ChainDbEnv m blk
cdb InvalidBlocks blk
invalid HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo
succsOf' = ChainDbEnv m blk
-> InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
forall blk (proxy :: * -> *).
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
ignoreInvalidSuc ChainDbEnv m blk
cdb InvalidBlocks blk
invalid ChainHash blk -> Set (HeaderHash blk)
succsOf
assert (isJust $ lookupBlockInfo (headerHash hdr)) $ return ()
let
sanitizeLoEFrag :: AnchoredFragment (HeaderWithTime blk)
-> AnchoredFragment (HeaderWithTime blk)
sanitizeLoEFrag AnchoredFragment (HeaderWithTime blk)
loeFrag0 =
case AnchoredFragment (HeaderWithTime blk)
-> Point (Header blk)
-> Maybe
(AnchoredFragment (HeaderWithTime blk),
AnchoredFragment (HeaderWithTime blk))
forall block1 block2.
(HasHeader block1, HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> Point block2
-> Maybe (AnchoredFragment block1, AnchoredFragment block1)
AF.splitAfterPoint AnchoredFragment (HeaderWithTime blk)
loeFrag0 (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall block. AnchoredFragment block -> Point block
AF.anchorPoint AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain) of
Just (AnchoredFragment (HeaderWithTime blk)
_, AnchoredFragment (HeaderWithTime blk)
frag) -> AnchoredFragment (HeaderWithTime blk)
frag
Maybe
(AnchoredFragment (HeaderWithTime blk),
AnchoredFragment (HeaderWithTime blk))
Nothing -> Anchor (HeaderWithTime blk)
-> AnchoredFragment (HeaderWithTime blk)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty (Anchor (HeaderWithTime blk)
-> AnchoredFragment (HeaderWithTime blk))
-> Anchor (HeaderWithTime blk)
-> AnchoredFragment (HeaderWithTime blk)
forall a b. (a -> b) -> a -> b
$ Anchor (Header blk) -> Anchor (HeaderWithTime blk)
forall a b. (HeaderHash a ~ HeaderHash b) => Anchor a -> Anchor b
AF.castAnchor (Anchor (Header blk) -> Anchor (HeaderWithTime blk))
-> Anchor (Header blk) -> Anchor (HeaderWithTime blk)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Anchor (Header blk)
forall v a b. AnchoredSeq v a b -> a
AF.anchor AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain
loeFrag <- fmap sanitizeLoEFrag <$> cdbLoE
traceWith addBlockTracer
(ChainSelectionLoEDebug curChain (AF.mapAnchoredFragment hwtHeader <$> loeFrag))
if
| olderThanK hdr isEBB immBlockNo -> do
traceWith addBlockTracer $ IgnoreBlockOlderThanK p
| Just (InvalidBlockInfo reason _) <- Map.lookup (headerHash hdr) invalid -> do
traceWith addBlockTracer $ IgnoreInvalidBlock p reason
InvalidBlockPunishment.enact
punish
InvalidBlockPunishment.BlockItself
| pointHash tipPoint == headerPrevHash hdr -> do
traceWith addBlockTracer (TryAddToCurrentChain p)
addToCurrentChain rr succsOf' curChainAndLedger loeFrag
| Just diff <- Paths.isReachable lookupBlockInfo' curChain p -> do
traceWith addBlockTracer (TrySwitchToAFork p diff)
switchToAFork rr succsOf' lookupBlockInfo' curChainAndLedger loeFrag diff
| otherwise -> do
traceWith addBlockTracer (StoreButDontChange p)
where
SecurityParam NonZero Word64
k = TopLevelConfig blk -> SecurityParam
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> SecurityParam
configSecurityParam TopLevelConfig blk
cdbTopLevelConfig
p :: RealPoint blk
p :: RealPoint blk
p = Header blk -> RealPoint blk
forall blk. HasHeader (Header blk) => Header blk -> RealPoint blk
headerRealPoint Header blk
hdr
isEBB :: IsEBB
isEBB :: IsEBB
isEBB = Header blk -> IsEBB
forall blk. GetHeader blk => Header blk -> IsEBB
headerToIsEBB Header blk
hdr
addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
addBlockTracer :: Tracer m (TraceAddBlockEvent blk)
addBlockTracer = TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceAddBlockEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer
mkChainSelEnv :: ChainAndLedger m blk -> ChainSelEnv m blk
mkChainSelEnv :: ChainAndLedger m blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger m blk
curChainAndLedger = ChainSelEnv
{ lgrDB :: LedgerDB' m blk
lgrDB = LedgerDB' m blk
cdbLedgerDB
, bcfg :: BlockConfig blk
bcfg = TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cdbTopLevelConfig
, varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid = StrictTVar m (WithFingerprint (InvalidBlocks blk))
cdbInvalid
, varTentativeState :: StrictTVar m (TentativeHeaderState blk)
varTentativeState = StrictTVar m (TentativeHeaderState blk)
cdbTentativeState
, varTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader = StrictTVar m (StrictMaybe (Header blk))
cdbTentativeHeader
, getTentativeFollowers :: STM m [FollowerHandle m blk]
getTentativeFollowers =
(FollowerHandle m blk -> Bool)
-> [FollowerHandle m blk] -> [FollowerHandle m blk]
forall a. (a -> Bool) -> [a] -> [a]
filter ((ChainType
TentativeChain ChainType -> ChainType -> Bool
forall a. Eq a => a -> a -> Bool
==) (ChainType -> Bool)
-> (FollowerHandle m blk -> ChainType)
-> FollowerHandle m blk
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FollowerHandle m blk -> ChainType
forall (m :: * -> *) blk. FollowerHandle m blk -> ChainType
fhChainType) ([FollowerHandle m blk] -> [FollowerHandle m blk])
-> (Map FollowerKey (FollowerHandle m blk)
-> [FollowerHandle m blk])
-> Map FollowerKey (FollowerHandle m blk)
-> [FollowerHandle m blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk]
forall k a. Map k a -> [a]
Map.elems
(Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk])
-> STM m (Map FollowerKey (FollowerHandle m blk))
-> STM m [FollowerHandle m blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
-> STM m (Map FollowerKey (FollowerHandle m blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbFollowers
, blockCache :: BlockCache blk
blockCache = BlockCache blk
blockCache
, curChainAndLedger :: ChainAndLedger m blk
curChainAndLedger = ChainAndLedger m blk
curChainAndLedger
, validationTracer :: Tracer m (TraceValidationEvent blk)
validationTracer =
TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> (TraceValidationEvent blk -> TraceAddBlockEvent blk)
-> TraceValidationEvent blk
-> TraceEvent blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TraceValidationEvent blk -> TraceAddBlockEvent blk
forall blk. TraceValidationEvent blk -> TraceAddBlockEvent blk
AddBlockValidation (TraceValidationEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TraceValidationEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer
, pipeliningTracer :: Tracer m (TracePipeliningEvent blk)
pipeliningTracer =
TraceAddBlockEvent blk -> TraceEvent blk
forall blk. TraceAddBlockEvent blk -> TraceEvent blk
TraceAddBlockEvent (TraceAddBlockEvent blk -> TraceEvent blk)
-> (TracePipeliningEvent blk -> TraceAddBlockEvent blk)
-> TracePipeliningEvent blk
-> TraceEvent blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TracePipeliningEvent blk -> TraceAddBlockEvent blk
forall blk. TracePipeliningEvent blk -> TraceAddBlockEvent blk
PipeliningEvent (TracePipeliningEvent blk -> TraceEvent blk)
-> Tracer m (TraceEvent blk) -> Tracer m (TracePipeliningEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceEvent blk)
cdbTracer
, punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
punish = (RealPoint blk, InvalidBlockPunishment m)
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
forall a. a -> Maybe a
Just (RealPoint blk
p, InvalidBlockPunishment m
punish)
}
addToCurrentChain ::
HasCallStack
=> ResourceRegistry m
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainAndLedger m blk
-> LoE (AnchoredFragment (HeaderWithTime blk))
-> m ()
addToCurrentChain :: HasCallStack =>
ResourceRegistry m
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainAndLedger m blk
-> LoE (AnchoredFragment (HeaderWithTime blk))
-> m ()
addToCurrentChain ResourceRegistry m
rr ChainHash blk -> Set (HeaderHash blk)
succsOf ChainAndLedger m blk
curChainAndLedger LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag = do
let suffixesAfterB :: [NonEmpty (HeaderHash blk)]
suffixesAfterB = (ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
forall blk.
(ChainHash blk -> Set (HeaderHash blk))
-> Maybe Word64 -> Point blk -> [NonEmpty (HeaderHash blk)]
Paths.maximalCandidates ChainHash blk -> Set (HeaderHash blk)
succsOf Maybe Word64
forall a. Maybe a
Nothing (RealPoint blk -> Point blk
forall blk. RealPoint blk -> Point blk
realPointToPoint RealPoint blk
p)
candidates <- case [NonEmpty (HeaderHash blk)]
-> Maybe (NonEmpty (NonEmpty (HeaderHash blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [NonEmpty (HeaderHash blk)]
suffixesAfterB of
Maybe (NonEmpty (NonEmpty (HeaderHash blk)))
Nothing ->
NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. (a -> b) -> a -> b
$ Anchor (Header blk)
-> [Header blk]
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => a -> [b] -> AnchoredSeq v a b
AF.fromOldestFirst Anchor (Header blk)
curHead [Header blk
hdr] AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> [AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
-> NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall a. a -> [a] -> NonEmpty a
NE.:| []
Just NonEmpty (NonEmpty (HeaderHash blk))
suffixesAfterB' ->
(StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> Map (HeaderHash blk) (Header blk)
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> Map (HeaderHash blk) (Header blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> Map (HeaderHash blk) (Header blk)
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m a
evalStateT Map (HeaderHash blk) (Header blk)
forall k a. Map k a
Map.empty (StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> m (NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. (a -> b) -> a -> b
$ NonEmpty (NonEmpty (HeaderHash blk))
-> (NonEmpty (HeaderHash blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
t a -> (a -> m b) -> m (t b)
forM NonEmpty (NonEmpty (HeaderHash blk))
suffixesAfterB' ((NonEmpty (HeaderHash blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))))
-> (NonEmpty (HeaderHash blk)
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
-> StateT
(Map (HeaderHash blk) (Header blk))
m
(NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)))
forall a b. (a -> b) -> a -> b
$ \NonEmpty (HeaderHash blk)
hashes -> do
hdrs <- (HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> [HeaderHash blk]
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM (VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) blk.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache VolatileDB m blk
cdbVolatileDB) ([HeaderHash blk]
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk])
-> [HeaderHash blk]
-> StateT (Map (HeaderHash blk) (Header blk)) m [Header blk]
forall a b. (a -> b) -> a -> b
$
NonEmpty (HeaderHash blk) -> [HeaderHash blk]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (HeaderHash blk)
hashes
return $ AF.fromOldestFirst curHead (hdr : hdrs)
let chainDiffs = [ChainDiff (Header blk)]
-> Maybe (NonEmpty (ChainDiff (Header blk)))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty
([ChainDiff (Header blk)]
-> Maybe (NonEmpty (ChainDiff (Header blk))))
-> [ChainDiff (Header blk)]
-> Maybe (NonEmpty (ChainDiff (Header blk)))
forall a b. (a -> b) -> a -> b
$ (ChainDiff (Header blk) -> Bool)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. (a -> Bool) -> [a] -> [a]
filter (BlockConfig blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate (ChainSelEnv m blk -> BlockConfig blk
forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg ChainSelEnv m blk
chainSelEnv) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Bool)
-> (ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix)
([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ (ChainDiff (Header blk) -> ChainDiff (Header blk))
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (LoE (AnchoredFragment (HeaderWithTime blk))
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
forall blk'.
(HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
LoE (AnchoredFragment blk')
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag ChainAndLedger m blk
curChainAndLedger)
([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ChainDiff (Header blk))
-> [AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
-> [ChainDiff (Header blk)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ChainDiff (Header blk)
forall b. AnchoredFragment b -> ChainDiff b
Diff.extend
([AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
-> [ChainDiff (Header blk)])
-> [AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
-> [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> [AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
candidates
case chainDiffs of
Maybe (NonEmpty (ChainDiff (Header blk)))
Nothing -> () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
Just NonEmpty (ChainDiff (Header blk))
chainDiffs' ->
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk)))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr NonEmpty (ChainDiff (Header blk))
chainDiffs' m (Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk)))
-> (Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk))
-> m ())
-> m ()
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk))
Nothing ->
() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
Just ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff ->
HasCallStack =>
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff
(ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader ChainSelEnv m blk
chainSelEnv)
ChainSwitchType
AddingBlocks
where
chainSelEnv :: ChainSelEnv m blk
chainSelEnv = ChainAndLedger m blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger m blk
curChainAndLedger
curChain :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain = ChainAndLedger m blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChainAndLedger
curHead :: Anchor (Header blk)
curHead = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Anchor (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> a
AF.headAnchor AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain
trimToLoE ::
(HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
LoE (AnchoredFragment blk') ->
ChainAndLedger m blk ->
ChainDiff (Header blk) ->
ChainDiff (Header blk)
trimToLoE :: forall blk'.
(HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
LoE (AnchoredFragment blk')
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment blk')
LoEDisabled ChainAndLedger m blk
_ ChainDiff (Header blk)
diff = ChainDiff (Header blk)
diff
trimToLoE (LoEEnabled AnchoredFragment blk'
loe) ChainAndLedger m blk
curChain ChainDiff (Header blk)
diff =
case AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ChainDiff (Header blk)
-> Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall b.
HasHeader b =>
AnchoredFragment b -> ChainDiff b -> Maybe (AnchoredFragment b)
Diff.apply (ChainAndLedger m blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChain) ChainDiff (Header blk)
diff of
Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
Nothing -> [Char] -> ChainDiff (Header blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"trimToLoE: precondition 1 violated: the given 'ChainDiff' must apply on top of the given 'ChainAndLedger'"
Just AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
cand ->
case AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredFragment blk'
-> Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredFragment blk',
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredFragment blk')
forall block1 block2.
(HasHeader block1, HasHeader block2,
HeaderHash block1 ~ HeaderHash block2) =>
AnchoredFragment block1
-> AnchoredFragment block2
-> Maybe
(AnchoredFragment block1, AnchoredFragment block2,
AnchoredFragment block1, AnchoredFragment block2)
AF.intersect AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
cand AnchoredFragment blk'
loe of
Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredFragment blk',
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredFragment blk')
Nothing -> [Char] -> ChainDiff (Header blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"trimToLoE: precondition 2 violated: the LoE fragment must intersect with the current selection"
Just (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candPrefix, AnchoredFragment blk'
_, AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candSuffix, AnchoredFragment blk'
loeSuffix) ->
let trimmedCandSuffix :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCandSuffix = Int
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall v a b.
Anchorable v a b =>
Int -> AnchoredSeq v a b -> AnchoredSeq v a b
AF.takeOldest (Word64 -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Word64 -> Int) -> Word64 -> Int
forall a b. (a -> b) -> a -> b
$ NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero NonZero Word64
k) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candSuffix
trimmedCand :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCand =
if AnchoredFragment blk' -> Bool
forall v a b. AnchoredSeq v a b -> Bool
AF.null AnchoredFragment blk'
loeSuffix
then Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall a. HasCallStack => Maybe a -> a
fromJust (Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
forall block.
HasHeader block =>
AnchoredFragment block
-> AnchoredFragment block -> Maybe (AnchoredFragment block)
AF.join AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candPrefix AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCandSuffix
else AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
candPrefix
in AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ChainDiff (Header blk)
forall b b'.
(HasHeader b, HasHeader b', HeaderHash b ~ HeaderHash b',
HasCallStack) =>
AnchoredFragment b' -> AnchoredFragment b -> ChainDiff b
Diff.diff (ChainAndLedger m blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChain) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
trimmedCand
switchToAFork ::
HasCallStack
=> ResourceRegistry m
-> (ChainHash blk -> Set (HeaderHash blk))
-> LookupBlockInfo blk
-> ChainAndLedger m blk
-> LoE (AnchoredFragment (HeaderWithTime blk))
-> ChainDiff (HeaderFields blk)
-> m ()
switchToAFork :: HasCallStack =>
ResourceRegistry m
-> (ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> ChainAndLedger m blk
-> LoE (AnchoredFragment (HeaderWithTime blk))
-> ChainDiff (HeaderFields blk)
-> m ()
switchToAFork ResourceRegistry m
rr ChainHash blk -> Set (HeaderHash blk)
succsOf HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo ChainAndLedger m blk
curChainAndLedger LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag ChainDiff (HeaderFields blk)
diff = do
let initCache :: Map (HeaderHash blk) (Header blk)
initCache = HeaderHash blk -> Header blk -> Map (HeaderHash blk) (Header blk)
forall k a. k -> a -> Map k a
Map.singleton (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr) Header blk
hdr
chainDiffs <-
([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
( (ChainDiff (Header blk) -> Bool)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. (a -> Bool) -> [a] -> [a]
filter
( BlockConfig blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate (ChainSelEnv m blk -> BlockConfig blk
forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg ChainSelEnv m blk
chainSelEnv) AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain
(AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Bool)
-> (ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix
)
)
(m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk)
-> m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)])
-> m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((ChainDiff (Header blk) -> ChainDiff (Header blk))
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (LoE (AnchoredFragment (HeaderWithTime blk))
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
forall blk'.
(HasHeader blk', HeaderHash blk ~ HeaderHash blk') =>
LoE (AnchoredFragment blk')
-> ChainAndLedger m blk
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
trimToLoE LoE (AnchoredFragment (HeaderWithTime blk))
loeFrag ChainAndLedger m blk
curChainAndLedger))
(m [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk)
-> m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
-> Map (HeaderHash blk) (Header blk) -> m [ChainDiff (Header blk)])
-> Map (HeaderHash blk) (Header blk)
-> StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
-> m [ChainDiff (Header blk)]
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
-> Map (HeaderHash blk) (Header blk) -> m [ChainDiff (Header blk)]
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m a
evalStateT Map (HeaderHash blk) (Header blk)
initCache
(StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
-> m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk)
-> StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk)
-> m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ChainDiff (HeaderFields blk)
-> StateT
(Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk)))
-> [ChainDiff (HeaderFields blk)]
-> StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM ChainDiff (HeaderFields blk)
-> StateT
(Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk))
translateToHeaders
([ChainDiff (HeaderFields blk)]
-> StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)])
-> (ChainDiff (HeaderFields blk) -> [ChainDiff (HeaderFields blk)])
-> ChainDiff (HeaderFields blk)
-> StateT
(Map (HeaderHash blk) (Header blk)) m [ChainDiff (Header blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ChainDiff (HeaderFields blk) -> Bool)
-> NonEmpty (ChainDiff (HeaderFields blk))
-> [ChainDiff (HeaderFields blk)]
forall a. (a -> Bool) -> NonEmpty a -> [a]
NE.filter (Bool -> Bool
not (Bool -> Bool)
-> (ChainDiff (HeaderFields blk) -> Bool)
-> ChainDiff (HeaderFields blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (HeaderFields blk) -> Bool
forall b. HasHeader b => ChainDiff b -> Bool
Diff.rollbackExceedsSuffix)
(NonEmpty (ChainDiff (HeaderFields blk))
-> [ChainDiff (HeaderFields blk)])
-> (ChainDiff (HeaderFields blk)
-> NonEmpty (ChainDiff (HeaderFields blk)))
-> ChainDiff (HeaderFields blk)
-> [ChainDiff (HeaderFields blk)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ChainHash blk -> Set (HeaderHash blk))
-> (HeaderHash blk -> Maybe (BlockInfo blk))
-> ChainDiff (HeaderFields blk)
-> NonEmpty (ChainDiff (HeaderFields blk))
forall blk.
HasHeader blk =>
(ChainHash blk -> Set (HeaderHash blk))
-> LookupBlockInfo blk
-> ChainDiff (HeaderFields blk)
-> NonEmpty (ChainDiff (HeaderFields blk))
Paths.extendWithSuccessors ChainHash blk -> Set (HeaderHash blk)
succsOf HeaderHash blk -> Maybe (BlockInfo blk)
lookupBlockInfo
(ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)])
-> ChainDiff (HeaderFields blk) -> m [ChainDiff (Header blk)]
forall a b. (a -> b) -> a -> b
$ ChainDiff (HeaderFields blk)
diff
case NE.nonEmpty chainDiffs of
Maybe (NonEmpty (ChainDiff (Header blk)))
Nothing -> () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
Just NonEmpty (ChainDiff (Header blk))
chainDiffs' ->
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk)))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr NonEmpty (ChainDiff (Header blk))
chainDiffs' m (Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk)))
-> (Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk))
-> m ())
-> m ()
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
Maybe
(ValidatedChainDiff
(Header blk) (Forker m (ExtLedgerState blk) blk))
Nothing ->
() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()
Just ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff ->
HasCallStack =>
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
validatedChainDiff
(ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader ChainSelEnv m blk
chainSelEnv)
ChainSwitchType
SwitchingToAFork
where
chainSelEnv :: ChainSelEnv m blk
chainSelEnv = ChainAndLedger m blk -> ChainSelEnv m blk
mkChainSelEnv ChainAndLedger m blk
curChainAndLedger
curChain :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain = ChainAndLedger m blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChainAndLedger
mkSelectionChangedInfo ::
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> ExtLedgerState blk EmptyMK
-> SelectionChangedInfo blk
mkSelectionChangedInfo :: AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ExtLedgerState blk EmptyMK
-> SelectionChangedInfo blk
mkSelectionChangedInfo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
oldChain AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain ExtLedgerState blk EmptyMK
newTip =
SelectionChangedInfo {
newTipPoint :: RealPoint blk
newTipPoint = RealPoint (Header blk) -> RealPoint blk
forall blk blk'.
Coercible (HeaderHash blk) (HeaderHash blk') =>
RealPoint blk -> RealPoint blk'
castRealPoint RealPoint (Header blk)
tipPoint
, newTipEpoch :: EpochNo
newTipEpoch = EpochNo
tipEpoch
, newTipSlotInEpoch :: Word64
newTipSlotInEpoch = Word64
tipSlotInEpoch
, newTipTrigger :: RealPoint blk
newTipTrigger = RealPoint blk
p
, SelectView (BlockProtocol blk)
newTipSelectView :: SelectView (BlockProtocol blk)
newTipSelectView :: SelectView (BlockProtocol blk)
newTipSelectView
, oldTipSelectView :: Maybe (SelectView (BlockProtocol blk))
oldTipSelectView =
BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
forall blk.
BlockSupportsProtocol blk =>
BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
selectView (TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg)
(Header blk -> SelectView (BlockProtocol blk))
-> Maybe (Header blk) -> Maybe (SelectView (BlockProtocol blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Either (Anchor (Header blk)) (Header blk) -> Maybe (Header blk)
forall a b. Either a b -> Maybe b
eitherToMaybe (AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Either (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Either a b
AF.head AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
oldChain)
}
where
cfg :: TopLevelConfig blk
cfg :: TopLevelConfig blk
cfg = TopLevelConfig blk
cdbTopLevelConfig
ledger :: LedgerState blk EmptyMK
ledger :: LedgerState blk EmptyMK
ledger = ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
newTip
summary :: History.Summary (HardForkIndices blk)
summary :: Summary (HardForkIndices blk)
summary = LedgerConfig blk
-> LedgerState blk EmptyMK -> Summary (HardForkIndices blk)
forall blk (mk :: MapKind).
HasHardForkHistory blk =>
LedgerConfig blk
-> LedgerState blk mk -> Summary (HardForkIndices blk)
forall (mk :: MapKind).
LedgerConfig blk
-> LedgerState blk mk -> Summary (HardForkIndices blk)
hardForkSummary
(TopLevelConfig blk -> LedgerConfig blk
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig blk
cfg)
LedgerState blk EmptyMK
ledger
(RealPoint (Header blk)
tipPoint, (EpochNo
tipEpoch, Word64
tipSlotInEpoch), SelectView (BlockProtocol blk)
newTipSelectView) =
case AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Either (Anchor (Header blk)) (Header blk)
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Either a b
AF.head AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain of
Left Anchor (Header blk)
_anchor -> [Char]
-> (RealPoint (Header blk), (EpochNo, Word64),
SelectView (BlockProtocol blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"cannot have switched to an empty chain"
Right Header blk
tipHdr ->
let query :: Qry (EpochNo, Word64)
query = SlotNo -> Qry (EpochNo, Word64)
History.slotToEpoch' (Header blk -> SlotNo
forall b. HasHeader b => b -> SlotNo
blockSlot Header blk
tipHdr)
tipEpochData :: (EpochNo, Word64)
tipEpochData = Qry (EpochNo, Word64)
-> Summary (HardForkIndices blk) -> (EpochNo, Word64)
forall a (xs :: [*]). HasCallStack => Qry a -> Summary xs -> a
History.runQueryPure Qry (EpochNo, Word64)
query Summary (HardForkIndices blk)
summary
sv :: SelectView (BlockProtocol blk)
sv = BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
forall blk.
BlockSupportsProtocol blk =>
BlockConfig blk -> Header blk -> SelectView (BlockProtocol blk)
selectView (TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg) Header blk
tipHdr
in (Header blk -> RealPoint (Header blk)
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint Header blk
tipHdr, (EpochNo, Word64)
tipEpochData, SelectView (BlockProtocol blk)
sv)
switchTo
:: HasCallStack
=> ValidatedChainDiff (Header blk) (Forker' m blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo :: HasCallStack =>
ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> StrictTVar m (StrictMaybe (Header blk))
-> ChainSwitchType
-> m ()
switchTo ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
vChainDiff StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader ChainSwitchType
chainSwitchType = do
Tracer m (TraceAddBlockEvent blk) -> TraceAddBlockEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceAddBlockEvent blk)
addBlockTracer (TraceAddBlockEvent blk -> m ()) -> TraceAddBlockEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$
Point blk -> TraceAddBlockEvent blk
forall blk. Point blk -> TraceAddBlockEvent blk
ChangingSelection
(Point blk -> TraceAddBlockEvent blk)
-> Point blk -> TraceAddBlockEvent blk
forall a b. (a -> b) -> a -> b
$ Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
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
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall block.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint
(AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Point (Header blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
getSuffix
(ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
-> ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
-> ChainDiff (Header blk)
forall b l. ValidatedChainDiff b l -> ChainDiff b
getChainDiff ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
vChainDiff
(curChain, newChain, events, prevTentativeHeader, newLedger) <- STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
[LedgerEvent blk], StrictMaybe (Header blk),
ExtLedgerState blk EmptyMK)
-> m (AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
[LedgerEvent blk], StrictMaybe (Header blk),
ExtLedgerState blk EmptyMK)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
[LedgerEvent blk], StrictMaybe (Header blk),
ExtLedgerState blk EmptyMK)
-> m (AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
[LedgerEvent blk], StrictMaybe (Header blk),
ExtLedgerState blk EmptyMK))
-> STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
[LedgerEvent blk], StrictMaybe (Header blk),
ExtLedgerState blk EmptyMK)
-> m (AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
[LedgerEvent blk], StrictMaybe (Header blk),
ExtLedgerState blk EmptyMK)
forall a b. (a -> b) -> a -> b
$ do
InternalChain curChain curChainWithTime <- StrictTVar m (InternalChain blk) -> STM m (InternalChain blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (InternalChain blk)
cdbChain
curLedger <- getVolatileTip cdbLedgerDB
newLedger <- forkerGetLedgerState newForker
case Diff.apply curChain chainDiff of
Maybe
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk))
Nothing ->
[Char]
-> STM
m
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk),
[LedgerEvent blk], StrictMaybe (Header blk),
ExtLedgerState blk EmptyMK)
forall a. HasCallStack => [Char] -> a
error [Char]
"chainDiff doesn't fit onto current chain"
Just AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain -> do
let lcfg :: LedgerConfig blk
lcfg = TopLevelConfig blk -> LedgerConfig blk
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig blk
cdbTopLevelConfig
diffWithTime :: ChainDiff (HeaderWithTime blk)
diffWithTime =
(Header blk -> HeaderWithTime blk)
-> ChainDiff (Header blk) -> ChainDiff (HeaderWithTime blk)
forall a b.
(HasHeader b, HeaderHash a ~ HeaderHash b) =>
(a -> b) -> ChainDiff a -> ChainDiff b
Diff.map
(LedgerConfig blk
-> LedgerState blk EmptyMK -> Header blk -> HeaderWithTime blk
forall blk (mk :: MapKind).
(HasHardForkHistory blk, HasHeader (Header blk)) =>
LedgerConfig blk
-> LedgerState blk mk -> Header blk -> HeaderWithTime blk
mkHeaderWithTime
LedgerConfig blk
lcfg
(ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
newLedger)
)
ChainDiff (Header blk)
chainDiff
newChainWithTime :: AnchoredFragment (HeaderWithTime blk)
newChainWithTime =
case AnchoredFragment (HeaderWithTime blk)
-> ChainDiff (HeaderWithTime blk)
-> Maybe (AnchoredFragment (HeaderWithTime blk))
forall b.
HasHeader b =>
AnchoredFragment b -> ChainDiff b -> Maybe (AnchoredFragment b)
Diff.apply AnchoredFragment (HeaderWithTime blk)
curChainWithTime ChainDiff (HeaderWithTime blk)
diffWithTime of
Maybe (AnchoredFragment (HeaderWithTime blk))
Nothing -> [Char] -> AnchoredFragment (HeaderWithTime blk)
forall a. HasCallStack => [Char] -> a
error [Char]
"chainDiff failed for HeaderWithTime"
Just AnchoredFragment (HeaderWithTime blk)
x -> AnchoredFragment (HeaderWithTime blk)
x
StrictTVar m (InternalChain blk) -> InternalChain blk -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (InternalChain blk)
cdbChain (InternalChain blk -> STM m ()) -> InternalChain blk -> STM m ()
forall a b. (a -> b) -> a -> b
$ AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredFragment (HeaderWithTime blk) -> InternalChain blk
forall blk.
AnchoredFragment (Header blk)
-> AnchoredFragment (HeaderWithTime blk) -> InternalChain blk
InternalChain AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain AnchoredFragment (HeaderWithTime blk)
newChainWithTime
Forker m (ExtLedgerState blk) blk -> STM m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> STM m ()
forkerCommit Forker m (ExtLedgerState blk) blk
newForker
let events :: [LedgerEvent blk]
events :: [LedgerEvent blk]
events = TopLevelConfig blk
-> LedgerState blk EmptyMK
-> LedgerState blk EmptyMK
-> [LedgerEvent blk]
forall blk (mk1 :: MapKind) (mk2 :: MapKind).
InspectLedger blk =>
TopLevelConfig blk
-> LedgerState blk mk1 -> LedgerState blk mk2 -> [LedgerEvent blk]
forall (mk1 :: MapKind) (mk2 :: MapKind).
TopLevelConfig blk
-> LedgerState blk mk1 -> LedgerState blk mk2 -> [LedgerEvent blk]
inspectLedger
TopLevelConfig blk
cdbTopLevelConfig
(ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
curLedger)
(ExtLedgerState blk EmptyMK -> LedgerState blk EmptyMK
forall blk (mk :: MapKind).
ExtLedgerState blk mk -> LedgerState blk mk
ledgerState ExtLedgerState blk EmptyMK
newLedger)
prevTentativeHeader <- StrictTVar m (StrictMaybe (Header blk))
-> StrictMaybe (Header blk) -> STM m (StrictMaybe (Header blk))
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> a -> STM m a
swapTVar StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader StrictMaybe (Header blk)
forall a. StrictMaybe a
SNothing
case chainSwitchType of
ChainSwitchType
AddingBlocks -> () -> STM m ()
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
ChainSwitchType
SwitchingToAFork -> do
let ipoint :: Point blk
ipoint = Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
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
$ ChainDiff (Header blk) -> Point (Header blk)
forall b. ChainDiff b -> Point b
Diff.getAnchorPoint ChainDiff (Header blk)
chainDiff
followerHandles <- Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk]
forall k a. Map k a -> [a]
Map.elems (Map FollowerKey (FollowerHandle m blk) -> [FollowerHandle m blk])
-> STM m (Map FollowerKey (FollowerHandle m blk))
-> STM m [FollowerHandle m blk]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (Map FollowerKey (FollowerHandle m blk))
-> STM m (Map FollowerKey (FollowerHandle m blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (Map FollowerKey (FollowerHandle m blk))
cdbFollowers
forM_ followerHandles $ switchFollowerToFork curChain newChain ipoint
return (curChain, newChain, events, prevTentativeHeader, newLedger)
let mkTraceEvent = case ChainSwitchType
chainSwitchType of
ChainSwitchType
AddingBlocks -> [LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> TraceAddBlockEvent blk
forall blk.
[LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
AddedToCurrentChain
ChainSwitchType
SwitchingToAFork -> [LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> TraceAddBlockEvent blk
forall blk.
[LedgerEvent blk]
-> SelectionChangedInfo blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> TraceAddBlockEvent blk
SwitchedToAFork
selChangedInfo = AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> ExtLedgerState blk EmptyMK
-> SelectionChangedInfo blk
mkSelectionChangedInfo AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
curChain AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
newChain ExtLedgerState blk EmptyMK
newLedger
traceWith addBlockTracer $
mkTraceEvent events selChangedInfo curChain newChain
whenJust (strictMaybeToMaybe prevTentativeHeader) $ traceWith $
PipeliningEvent . OutdatedTentativeHeader >$< addBlockTracer
forkerClose newForker
where
switchFollowerToFork :: AnchoredFragment (Header blk)
-> AnchoredFragment block
-> Point blk
-> FollowerHandle m blk
-> STM m ()
switchFollowerToFork AnchoredFragment (Header blk)
curChain AnchoredFragment block
newChain Point blk
ipoint =
let oldPoints :: Set (Point blk)
oldPoints = [Point blk] -> Set (Point blk)
forall a. Ord a => [a] -> Set a
Set.fromList ([Point blk] -> Set (Point blk))
-> (AnchoredFragment (Header blk) -> [Point blk])
-> AnchoredFragment (Header blk)
-> Set (Point blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Header blk -> Point blk) -> [Header blk] -> [Point blk]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint ([Header blk] -> [Point blk])
-> (AnchoredFragment (Header blk) -> [Header blk])
-> AnchoredFragment (Header blk)
-> [Point blk]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment (Header blk) -> [Header blk]
forall v a b. AnchoredSeq v a b -> [b]
AS.toOldestFirst
(AnchoredFragment (Header blk) -> Set (Point blk))
-> AnchoredFragment (Header blk) -> Set (Point blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix
(ChainDiff (Header blk) -> AnchoredFragment (Header blk))
-> ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall a b. (a -> b) -> a -> b
$ AnchoredFragment block
-> AnchoredFragment (Header blk) -> ChainDiff (Header blk)
forall b b'.
(HasHeader b, HasHeader b', HeaderHash b ~ HeaderHash b',
HasCallStack) =>
AnchoredFragment b' -> AnchoredFragment b -> ChainDiff b
Diff.diff AnchoredFragment block
newChain AnchoredFragment (Header blk)
curChain
in Bool
-> (FollowerHandle m blk -> STM m ())
-> FollowerHandle m blk
-> STM m ()
forall a. HasCallStack => Bool -> a -> a
assert (Point block -> AnchoredFragment block -> Bool
forall block.
HasHeader block =>
Point block -> AnchoredFragment block -> Bool
AF.withinFragmentBounds (Point blk -> Point block
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
ipoint) AnchoredFragment block
newChain) ((FollowerHandle m blk -> STM m ())
-> FollowerHandle m blk -> STM m ())
-> (FollowerHandle m blk -> STM m ())
-> FollowerHandle m blk
-> STM m ()
forall a b. (a -> b) -> a -> b
$
\FollowerHandle m blk
followerHandle -> FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
forall (m :: * -> *) blk.
FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
fhSwitchFork FollowerHandle m blk
followerHandle Point blk
ipoint Set (Point blk)
oldPoints
ValidatedChainDiff ChainDiff (Header blk)
chainDiff Forker m (ExtLedgerState blk) blk
newForker = ValidatedChainDiff (Header blk) (Forker m (ExtLedgerState blk) blk)
vChainDiff
translateToHeaders
:: ChainDiff (HeaderFields blk)
-> StateT (Map (HeaderHash blk) (Header blk))
m
(ChainDiff (Header blk))
translateToHeaders :: ChainDiff (HeaderFields blk)
-> StateT
(Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk))
translateToHeaders =
(HeaderFields blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> ChainDiff (HeaderFields blk)
-> StateT
(Map (HeaderHash blk) (Header blk)) m (ChainDiff (Header blk))
forall a b (m :: * -> *).
(HasHeader b, HeaderHash a ~ HeaderHash b, Monad m) =>
(a -> m b) -> ChainDiff a -> m (ChainDiff b)
Diff.mapM (VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) blk.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
getKnownHeaderThroughCache VolatileDB m blk
cdbVolatileDB (HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> (HeaderFields blk -> HeaderHash blk)
-> HeaderFields blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. HeaderFields blk -> HeaderHash blk
forall k (b :: k). HeaderFields b -> HeaderHash b
headerFieldHash)
getKnownHeaderThroughCache ::
(MonadThrow m, HasHeader blk)
=> VolatileDB m blk
-> HeaderHash blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
VolatileDB m blk
volatileDB HeaderHash blk
hash = (Map (HeaderHash blk) (Header blk) -> Maybe (Header blk))
-> StateT
(Map (HeaderHash blk) (Header blk)) m (Maybe (Header blk))
forall (m :: * -> *) s a. Monad m => (s -> a) -> StateT s m a
gets (HeaderHash blk
-> Map (HeaderHash blk) (Header blk) -> Maybe (Header blk)
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup HeaderHash blk
hash) StateT (Map (HeaderHash blk) (Header blk)) m (Maybe (Header blk))
-> (Maybe (Header blk)
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall a b.
StateT (Map (HeaderHash blk) (Header blk)) m a
-> (a -> StateT (Map (HeaderHash blk) (Header blk)) m b)
-> StateT (Map (HeaderHash blk) (Header blk)) m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
Just Header blk
hdr -> Header blk
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall a. a -> StateT (Map (HeaderHash blk) (Header blk)) m a
forall (m :: * -> *) a. Monad m => a -> m a
return Header blk
hdr
Maybe (Header blk)
Nothing -> do
hdr <- m (Header blk)
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall (m :: * -> *) a.
Monad m =>
m a -> StateT (Map (HeaderHash blk) (Header blk)) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m (Header blk)
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk))
-> m (Header blk)
-> StateT (Map (HeaderHash blk) (Header blk)) m (Header blk)
forall a b. (a -> b) -> a -> b
$ VolatileDB m blk
-> BlockComponent blk (Header blk)
-> HeaderHash blk
-> m (Header blk)
forall (m :: * -> *) blk b.
(MonadThrow m, HasHeader blk) =>
VolatileDB m blk -> BlockComponent blk b -> HeaderHash blk -> m b
VolatileDB.getKnownBlockComponent VolatileDB m blk
volatileDB BlockComponent blk (Header blk)
forall blk. BlockComponent blk (Header blk)
GetHeader HeaderHash blk
hash
modify (Map.insert hash hdr)
return hdr
data ChainSelEnv m blk = ChainSelEnv
{ forall (m :: * -> *) blk. ChainSelEnv m blk -> LedgerDB' m blk
lgrDB :: LedgerDB.LedgerDB' m blk
, forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TraceValidationEvent blk)
validationTracer :: Tracer m (TraceValidationEvent blk)
, forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TracePipeliningEvent blk)
pipeliningTracer :: Tracer m (TracePipeliningEvent blk)
, forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
bcfg :: BlockConfig blk
, forall (m :: * -> *) blk.
ChainSelEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
, forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (TentativeHeaderState blk)
varTentativeState :: StrictTVar m (TentativeHeaderState blk)
, :: StrictTVar m (StrictMaybe (Header blk))
, forall (m :: * -> *) blk.
ChainSelEnv m blk -> STM m [FollowerHandle m blk]
getTentativeFollowers :: STM m [FollowerHandle m blk]
, forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockCache blk
blockCache :: BlockCache blk
, forall (m :: * -> *) blk. ChainSelEnv m blk -> ChainAndLedger m blk
curChainAndLedger :: ChainAndLedger m blk
, forall (m :: * -> *) blk.
ChainSelEnv m blk
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
}
chainSelection ::
forall m blk.
( IOLike m
, LedgerSupportsProtocol blk
, BlockSupportsDiffusionPipelining blk
, HasCallStack
)
=> ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk,
BlockSupportsDiffusionPipelining blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> NonEmpty (ChainDiff (Header blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
chainSelection ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr NonEmpty (ChainDiff (Header blk))
chainDiffs =
Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((ChainDiff (Header blk) -> Bool)
-> NonEmpty (ChainDiff (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all (BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain (AnchoredFragment (Header blk) -> Bool)
-> (ChainDiff (Header blk) -> AnchoredFragment (Header blk))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix)
NonEmpty (ChainDiff (Header blk))
chainDiffs) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert ((ChainDiff (Header blk) -> Bool)
-> NonEmpty (ChainDiff (Header blk)) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
all (Maybe (AnchoredFragment (Header blk)) -> Bool
forall a. Maybe a -> Bool
isJust (Maybe (AnchoredFragment (Header blk)) -> Bool)
-> (ChainDiff (Header blk)
-> Maybe (AnchoredFragment (Header blk)))
-> ChainDiff (Header blk)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. AnchoredFragment (Header blk)
-> ChainDiff (Header blk) -> Maybe (AnchoredFragment (Header blk))
forall b.
HasHeader b =>
AnchoredFragment b -> ChainDiff b -> Maybe (AnchoredFragment b)
Diff.apply AnchoredFragment (Header blk)
curChain)
NonEmpty (ChainDiff (Header blk))
chainDiffs) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
[ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
go ([ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
sortCandidates (NonEmpty (ChainDiff (Header blk)) -> [ChainDiff (Header blk)]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (ChainDiff (Header blk))
chainDiffs))
where
ChainSelEnv {Maybe (RealPoint blk, InvalidBlockPunishment m)
Tracer m (TracePipeliningEvent blk)
Tracer m (TraceValidationEvent blk)
STM m [FollowerHandle m blk]
StrictTVar m (StrictMaybe (Header blk))
StrictTVar m (TentativeHeaderState blk)
StrictTVar m (WithFingerprint (InvalidBlocks blk))
BlockConfig blk
BlockCache blk
ChainAndLedger m blk
LedgerDB' m blk
lgrDB :: forall (m :: * -> *) blk. ChainSelEnv m blk -> LedgerDB' m blk
bcfg :: forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockConfig blk
varInvalid :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
blockCache :: forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockCache blk
curChainAndLedger :: forall (m :: * -> *) blk. ChainSelEnv m blk -> ChainAndLedger m blk
validationTracer :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TraceValidationEvent blk)
pipeliningTracer :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TracePipeliningEvent blk)
varTentativeState :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (TentativeHeaderState blk)
varTentativeHeader :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> StrictTVar m (StrictMaybe (Header blk))
punish :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
getTentativeFollowers :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> STM m [FollowerHandle m blk]
bcfg :: BlockConfig blk
lgrDB :: LedgerDB' m blk
validationTracer :: Tracer m (TraceValidationEvent blk)
pipeliningTracer :: Tracer m (TracePipeliningEvent blk)
varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varTentativeState :: StrictTVar m (TentativeHeaderState blk)
varTentativeHeader :: StrictTVar m (StrictMaybe (Header blk))
getTentativeFollowers :: STM m [FollowerHandle m blk]
blockCache :: BlockCache blk
curChainAndLedger :: ChainAndLedger m blk
punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
..} = ChainSelEnv m blk
chainSelEnv
curChain :: AnchoredFragment (Header blk)
curChain = ChainAndLedger m blk -> AnchoredFragment (Header blk)
forall b l. ValidatedFragment b l -> AnchoredFragment b
VF.validatedFragment ChainAndLedger m blk
curChainAndLedger
sortCandidates :: [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
sortCandidates :: [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
sortCandidates =
(ChainDiff (Header blk) -> ChainDiff (Header blk) -> Ordering)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. (a -> a -> Ordering) -> [a] -> [a]
sortBy ((AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Ordering)
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Ordering
forall a b c. (a -> b -> c) -> b -> a -> c
flip (BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Ordering
forall blk (h :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
HasHeader (h blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h blk) -> Ordering
compareAnchoredFragments BlockConfig blk
bcfg) (AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Ordering)
-> (ChainDiff (Header blk) -> AnchoredFragment (Header blk))
-> ChainDiff (Header blk)
-> ChainDiff (Header blk)
-> Ordering
forall b c a. (b -> b -> c) -> (a -> b) -> a -> a -> c
`on` ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix)
go ::
[ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
go :: [ChainDiff (Header blk)]
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
go [] = Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a. Maybe a
Nothing
go (ChainDiff (Header blk)
candidate:[ChainDiff (Header blk)]
candidates0) = do
mTentativeHeader <- m (Maybe (Header blk, TentativeHeaderState blk))
setTentativeHeader
validateCandidate chainSelEnv rr candidate >>= \case
ValidationResult m blk
InsufficientSuffix ->
Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert (Maybe (Header blk, TentativeHeaderState blk) -> Bool
forall a. Maybe a -> Bool
isNothing Maybe (Header blk, TentativeHeaderState blk)
mTentativeHeader) (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$ do
candidates1 <- [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
truncateRejectedBlocks [ChainDiff (Header blk)]
candidates0
go (sortCandidates candidates1)
FullyValid validatedCandidate :: ValidatedChainDiff (Header blk) (Forker' m blk)
validatedCandidate@(ValidatedChainDiff ChainDiff (Header blk)
candidate' Forker' m blk
_) ->
Bool
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. HasCallStack => Bool -> a -> a
assert (ChainDiff (Header blk) -> Point (Header blk)
forall b. HasHeader b => ChainDiff b -> Point b
Diff.getTip ChainDiff (Header blk)
candidate Point (Header blk) -> Point (Header blk) -> Bool
forall a. Eq a => a -> a -> Bool
== ChainDiff (Header blk) -> Point (Header blk)
forall b. HasHeader b => ChainDiff b -> Point b
Diff.getTip ChainDiff (Header blk)
candidate') (m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$
Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))))
-> Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
-> m (Maybe (ValidatedChainDiff (Header blk) (Forker' m blk)))
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (Forker' m blk)
-> Maybe (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a. a -> Maybe a
Just ValidatedChainDiff (Header blk) (Forker' m blk)
validatedCandidate
ValidPrefix ChainDiff (Header blk)
candidate' -> do
Maybe (Header blk, TentativeHeaderState blk)
-> ((Header blk, TentativeHeaderState blk) -> m ()) -> m ()
forall (f :: * -> *) a.
Applicative f =>
Maybe a -> (a -> f ()) -> f ()
whenJust Maybe (Header blk, TentativeHeaderState blk)
mTentativeHeader (Header blk, TentativeHeaderState blk) -> m ()
clearTentativeHeader
candidates1 <- [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
truncateRejectedBlocks [ChainDiff (Header blk)]
candidates0
let candidates2
| BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk (h :: * -> *) (h' :: * -> *).
(BlockSupportsProtocol blk, HasCallStack, GetHeader1 h,
GetHeader1 h', HeaderHash (h blk) ~ HeaderHash (h' blk),
HasHeader (h blk), HasHeader (h' blk)) =>
BlockConfig blk
-> AnchoredFragment (h blk) -> AnchoredFragment (h' blk) -> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
curChain (ChainDiff (Header blk) -> AnchoredFragment (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix ChainDiff (Header blk)
candidate')
= ChainDiff (Header blk)
candidate'ChainDiff (Header blk)
-> [ChainDiff (Header blk)] -> [ChainDiff (Header blk)]
forall a. a -> [a] -> [a]
:[ChainDiff (Header blk)]
candidates1
| Bool
otherwise
= [ChainDiff (Header blk)]
candidates1
go (sortCandidates candidates2)
where
setTentativeHeader :: m (Maybe (Header blk, TentativeHeaderState blk))
setTentativeHeader :: m (Maybe (Header blk, TentativeHeaderState blk))
setTentativeHeader = do
pipeliningResult <-
(\TentativeHeaderState blk
ts -> BlockConfig blk
-> TentativeHeaderState blk
-> ChainDiff (Header blk)
-> Maybe (Header blk, TentativeHeaderState blk)
forall blk.
(HasHeader (Header blk), BlockSupportsDiffusionPipelining blk) =>
BlockConfig blk
-> TentativeHeaderState blk
-> ChainDiff (Header blk)
-> Maybe (Header blk, TentativeHeaderState blk)
isPipelineable BlockConfig blk
bcfg TentativeHeaderState blk
ts ChainDiff (Header blk)
candidate)
(TentativeHeaderState blk
-> Maybe (Header blk, TentativeHeaderState blk))
-> m (TentativeHeaderState blk)
-> m (Maybe (Header blk, TentativeHeaderState blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StrictTVar m (TentativeHeaderState blk)
-> m (TentativeHeaderState blk)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> m a
readTVarIO StrictTVar m (TentativeHeaderState blk)
varTentativeState
whenJust pipeliningResult $ \(Header blk
tentativeHeader, TentativeHeaderState blk
_) -> do
let setTentative :: Enclosing -> TracePipeliningEvent blk
setTentative = Header blk -> Enclosing -> TracePipeliningEvent blk
forall blk. Header blk -> Enclosing -> TracePipeliningEvent blk
SetTentativeHeader Header blk
tentativeHeader
Tracer m Enclosing -> m () -> m ()
forall (m :: * -> *) a.
Applicative m =>
Tracer m Enclosing -> m a -> m a
encloseWith (Enclosing -> TracePipeliningEvent blk
setTentative (Enclosing -> TracePipeliningEvent blk)
-> Tracer m (TracePipeliningEvent blk) -> Tracer m Enclosing
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TracePipeliningEvent blk)
pipeliningTracer) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
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 (StrictMaybe (Header blk))
-> StrictMaybe (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader (StrictMaybe (Header blk) -> STM m ())
-> StrictMaybe (Header blk) -> STM m ()
forall a b. (a -> b) -> a -> b
$ Header blk -> StrictMaybe (Header blk)
forall a. a -> StrictMaybe a
SJust Header blk
tentativeHeader
m ()
forall (m :: * -> *). MonadFork m => m ()
yield
pure pipeliningResult
clearTentativeHeader :: (Header blk, TentativeHeaderState blk) -> m ()
clearTentativeHeader :: (Header blk, TentativeHeaderState blk) -> m ()
clearTentativeHeader (Header blk
tentativeHeader, TentativeHeaderState blk
tentativeSt) = do
STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
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
$ do
StrictTVar m (StrictMaybe (Header blk))
-> StrictMaybe (Header blk) -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (StrictMaybe (Header blk))
varTentativeHeader StrictMaybe (Header blk)
forall a. StrictMaybe a
SNothing
StrictTVar m (TentativeHeaderState blk)
-> TentativeHeaderState blk -> STM m ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar m (TentativeHeaderState blk)
varTentativeState TentativeHeaderState blk
tentativeSt
(FollowerHandle m blk -> STM m ()) -> STM m ()
forTentativeFollowers ((FollowerHandle m blk -> STM m ()) -> STM m ())
-> (FollowerHandle m blk -> STM m ()) -> STM m ()
forall a b. (a -> b) -> a -> b
$ \FollowerHandle m blk
followerHandle -> do
let curTipPoint :: Point blk
curTipPoint = Point (Header blk) -> Point blk
forall {k1} {k2} (b :: k1) (b' :: k2).
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.
HasHeader block =>
AnchoredFragment block -> Point block
AF.headPoint AnchoredFragment (Header blk)
curChain
oldPoints :: Set (Point blk)
oldPoints = Point blk -> Set (Point blk)
forall a. a -> Set a
Set.singleton (Point blk -> Set (Point blk)) -> Point blk -> Set (Point blk)
forall a b. (a -> b) -> a -> b
$ Header blk -> Point blk
forall blk. HasHeader (Header blk) => Header blk -> Point blk
headerPoint Header blk
tentativeHeader
FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
forall (m :: * -> *) blk.
FollowerHandle m blk -> Point blk -> Set (Point blk) -> STM m ()
fhSwitchFork FollowerHandle m blk
followerHandle Point blk
curTipPoint Set (Point blk)
oldPoints
Tracer m (TracePipeliningEvent blk)
-> TracePipeliningEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TracePipeliningEvent blk)
pipeliningTracer (TracePipeliningEvent blk -> m ())
-> TracePipeliningEvent blk -> m ()
forall a b. (a -> b) -> a -> b
$ Header blk -> TracePipeliningEvent blk
forall blk. Header blk -> TracePipeliningEvent blk
TrapTentativeHeader Header blk
tentativeHeader
where
forTentativeFollowers :: (FollowerHandle m blk -> STM m ()) -> STM m ()
forTentativeFollowers FollowerHandle m blk -> STM m ()
f = STM m [FollowerHandle m blk]
getTentativeFollowers STM m [FollowerHandle m blk]
-> ([FollowerHandle m blk] -> STM m ()) -> STM m ()
forall a b. STM m a -> (a -> STM m b) -> STM m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= (FollowerHandle m blk -> STM m ())
-> [FollowerHandle m blk] -> STM m ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
(a -> m b) -> t a -> m ()
mapM_ FollowerHandle m blk -> STM m ()
f
truncateRejectedBlocks ::
[ChainDiff (Header blk)]
-> m [ChainDiff (Header blk)]
truncateRejectedBlocks :: [ChainDiff (Header blk)] -> m [ChainDiff (Header blk)]
truncateRejectedBlocks [ChainDiff (Header blk)]
cands = do
invalid <- STM m (WithFingerprint (InvalidBlocks blk))
-> m (WithFingerprint (InvalidBlocks blk))
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (WithFingerprint (InvalidBlocks blk))
-> m (WithFingerprint (InvalidBlocks blk)))
-> STM m (WithFingerprint (InvalidBlocks blk))
-> m (WithFingerprint (InvalidBlocks blk))
forall a b. (a -> b) -> a -> b
$ StrictTVar m (WithFingerprint (InvalidBlocks blk))
-> STM m (WithFingerprint (InvalidBlocks blk))
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
let isRejected Header blk
hdr =
HeaderHash blk -> InvalidBlocks blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.member (Header blk -> HeaderHash blk
forall blk. HasHeader (Header blk) => Header blk -> HeaderHash blk
headerHash Header blk
hdr) (WithFingerprint (InvalidBlocks blk) -> InvalidBlocks blk
forall a. WithFingerprint a -> a
forgetFingerprint WithFingerprint (InvalidBlocks blk)
invalid)
return $ filter (preferAnchoredCandidate bcfg curChain . Diff.getSuffix)
$ map (Diff.takeWhileOldest (not . isRejected)) cands
data ValidationResult m blk =
FullyValid (ValidatedChainDiff (Header blk) (Forker' m blk))
| ValidPrefix (ChainDiff (Header blk))
| InsufficientSuffix
ledgerValidateCandidate ::
forall m blk.
( IOLike m
, LedgerSupportsProtocol blk
, HasCallStack
)
=> ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
ledgerValidateCandidate :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
ledgerValidateCandidate ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr chainDiff :: ChainDiff (Header blk)
chainDiff@(ChainDiff Word64
rollback AnchoredFragment (Header blk)
suffix) =
LedgerDB m (ExtLedgerState blk) blk
-> (ExtLedgerState blk ~ ExtLedgerState blk) =>
ResourceRegistry m
-> (TraceValidateEvent blk -> m ())
-> BlockCache blk
-> Word64
-> [Header blk]
-> m (ValidateResult m (ExtLedgerState blk) blk)
forall (m :: * -> *) (l :: LedgerStateKind) blk.
LedgerDB m l blk
-> (l ~ ExtLedgerState blk) =>
ResourceRegistry m
-> (TraceValidateEvent blk -> m ())
-> BlockCache blk
-> Word64
-> [Header blk]
-> m (ValidateResult m l blk)
LedgerDB.validateFork LedgerDB m (ExtLedgerState blk) blk
lgrDB ResourceRegistry m
rr TraceValidateEvent blk -> m ()
traceUpdate BlockCache blk
blockCache Word64
rollback [Header blk]
newBlocks m (ValidateResult m (ExtLedgerState blk) blk)
-> (ValidateResult m (ExtLedgerState blk) blk
-> m (ValidatedChainDiff (Header blk) (Forker' m blk)))
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
ValidateExceededRollBack {} ->
[Char] -> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall a. HasCallStack => [Char] -> a
error [Char]
"found candidate requiring rolling back past the immutable tip"
ValidateLedgerError (AnnLedgerError Forker' m blk
ledger' RealPoint blk
pt LedgerErr (ExtLedgerState blk)
e) -> do
lastValid <- STM m (Point blk) -> m (Point blk)
forall a. HasCallStack => STM m a -> m a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM m (Point blk) -> m (Point blk))
-> STM m (Point blk) -> m (Point blk)
forall a b. (a -> b) -> a -> b
$ Forker' m blk -> STM m (Point blk)
forall (l :: LedgerStateKind) blk (m :: * -> *).
(GetTip l, HeaderHash l ~ HeaderHash blk, Functor (STM m)) =>
Forker m l blk -> STM m (Point blk)
forkerCurrentPoint Forker' m blk
ledger'
let chainDiff' = Point (Header blk)
-> ChainDiff (Header blk) -> ChainDiff (Header blk)
forall b.
(HasHeader b, HasCallStack) =>
Point b -> ChainDiff b -> ChainDiff b
Diff.truncate (Point blk -> Point (Header blk)
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint Point blk
lastValid) ChainDiff (Header blk)
chainDiff
traceWith validationTracer (InvalidBlock e pt)
addInvalidBlock e pt
traceWith validationTracer (ValidCandidate (Diff.getSuffix chainDiff'))
whenJust punish $ \(RealPoint blk
addedPt, InvalidBlockPunishment m
punishment) -> do
let m :: m ()
m = InvalidBlockPunishment m -> Invalidity -> m ()
forall (m :: * -> *).
InvalidBlockPunishment m -> Invalidity -> m ()
InvalidBlockPunishment.enact InvalidBlockPunishment m
punishment
(Invalidity -> m ()) -> Invalidity -> m ()
forall a b. (a -> b) -> a -> b
$ if RealPoint blk
addedPt RealPoint blk -> RealPoint blk -> Bool
forall a. Eq a => a -> a -> Bool
== RealPoint blk
pt
then Invalidity
InvalidBlockPunishment.BlockItself
else Invalidity
InvalidBlockPunishment.BlockPrefix
case RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint blk
pt SlotNo -> SlotNo -> Ordering
forall a. Ord a => a -> a -> Ordering
`compare` RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint blk
addedPt of
Ordering
LT -> m ()
m
Ordering
GT -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
Ordering
EQ -> Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Point blk
lastValid Point blk -> Point blk -> Bool
forall a. Eq a => a -> a -> Bool
/= RealPoint blk -> Point blk
forall blk. RealPoint blk -> Point blk
realPointToPoint RealPoint blk
addedPt) m ()
m
ValidatedDiff.newM chainDiff' ledger'
ValidateSuccessful Forker' m blk
ledger' -> do
Tracer m (TraceValidationEvent blk)
-> TraceValidationEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith Tracer m (TraceValidationEvent blk)
validationTracer (AnchoredFragment (Header blk) -> TraceValidationEvent blk
forall blk.
AnchoredFragment (Header blk) -> TraceValidationEvent blk
ValidCandidate AnchoredFragment (Header blk)
suffix)
ChainDiff (Header blk)
-> Forker' m blk
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall (m :: * -> *) b l.
(MonadSTM m, GetTipSTM m l, HasHeader b,
HeaderHash l ~ HeaderHash b, HasCallStack) =>
ChainDiff b -> l -> m (ValidatedChainDiff b l)
ValidatedDiff.newM ChainDiff (Header blk)
chainDiff Forker' m blk
ledger'
where
ChainSelEnv {
LedgerDB m (ExtLedgerState blk) blk
lgrDB :: forall (m :: * -> *) blk. ChainSelEnv m blk -> LedgerDB' m blk
lgrDB :: LedgerDB m (ExtLedgerState blk) blk
lgrDB
, Tracer m (TraceValidationEvent blk)
validationTracer :: forall (m :: * -> *) blk.
ChainSelEnv m blk -> Tracer m (TraceValidationEvent blk)
validationTracer :: Tracer m (TraceValidationEvent blk)
validationTracer
, BlockCache blk
blockCache :: forall (m :: * -> *) blk. ChainSelEnv m blk -> BlockCache blk
blockCache :: BlockCache blk
blockCache
, StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid :: StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid
, Maybe (RealPoint blk, InvalidBlockPunishment m)
punish :: forall (m :: * -> *) blk.
ChainSelEnv m blk
-> Maybe (RealPoint blk, InvalidBlockPunishment m)
punish :: Maybe (RealPoint blk, InvalidBlockPunishment m)
punish
} = ChainSelEnv m blk
chainSelEnv
traceUpdate :: TraceValidateEvent blk -> m ()
traceUpdate = Tracer m (TraceValidateEvent blk) -> TraceValidateEvent blk -> m ()
forall (m :: * -> *) a. Tracer m a -> a -> m ()
traceWith (Tracer m (TraceValidateEvent blk)
-> TraceValidateEvent blk -> m ())
-> Tracer m (TraceValidateEvent blk)
-> TraceValidateEvent blk
-> m ()
forall a b. (a -> b) -> a -> b
$ TraceValidateEvent blk -> TraceValidationEvent blk
forall blk. TraceValidateEvent blk -> TraceValidationEvent blk
UpdateLedgerDbTraceEvent (TraceValidateEvent blk -> TraceValidationEvent blk)
-> Tracer m (TraceValidationEvent blk)
-> Tracer m (TraceValidateEvent blk)
forall (f :: * -> *) a b. Contravariant f => (a -> b) -> f b -> f a
>$< Tracer m (TraceValidationEvent blk)
validationTracer
newBlocks :: [Header blk]
newBlocks :: [Header blk]
newBlocks = AnchoredFragment (Header blk) -> [Header blk]
forall v a b. AnchoredSeq v a b -> [b]
AF.toOldestFirst AnchoredFragment (Header blk)
suffix
addInvalidBlock :: ExtValidationError blk -> RealPoint blk -> m ()
addInvalidBlock :: ExtValidationError blk -> RealPoint blk -> m ()
addInvalidBlock ExtValidationError blk
e (RealPoint SlotNo
slot HeaderHash blk
hash) = STM m () -> m ()
forall a. HasCallStack => STM m a -> m a
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 (WithFingerprint (InvalidBlocks blk))
-> (WithFingerprint (InvalidBlocks blk)
-> WithFingerprint (InvalidBlocks blk))
-> STM m ()
forall (m :: * -> *) a.
MonadSTM m =>
StrictTVar m a -> (a -> a) -> STM m ()
modifyTVar StrictTVar m (WithFingerprint (InvalidBlocks blk))
varInvalid ((WithFingerprint (InvalidBlocks blk)
-> WithFingerprint (InvalidBlocks blk))
-> STM m ())
-> (WithFingerprint (InvalidBlocks blk)
-> WithFingerprint (InvalidBlocks blk))
-> STM m ()
forall a b. (a -> b) -> a -> b
$ \(WithFingerprint InvalidBlocks blk
invalid Fingerprint
fp) ->
InvalidBlocks blk
-> Fingerprint -> WithFingerprint (InvalidBlocks blk)
forall a. a -> Fingerprint -> WithFingerprint a
WithFingerprint
(HeaderHash blk
-> InvalidBlockInfo blk -> InvalidBlocks blk -> InvalidBlocks blk
forall k a. Ord k => k -> a -> Map k a -> Map k a
Map.insert HeaderHash blk
hash (ExtValidationError blk -> SlotNo -> InvalidBlockInfo blk
forall blk.
ExtValidationError blk -> SlotNo -> InvalidBlockInfo blk
InvalidBlockInfo ExtValidationError blk
e SlotNo
slot) InvalidBlocks blk
invalid)
(Fingerprint -> Fingerprint
forall a. Enum a => a -> a
succ Fingerprint
fp)
validateCandidate ::
( IOLike m
, LedgerSupportsProtocol blk
, HasCallStack
)
=> ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidationResult m blk)
validateCandidate :: forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidationResult m blk)
validateCandidate ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr ChainDiff (Header blk)
chainDiff =
ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
ChainSelEnv m blk
-> ResourceRegistry m
-> ChainDiff (Header blk)
-> m (ValidatedChainDiff (Header blk) (Forker' m blk))
ledgerValidateCandidate ChainSelEnv m blk
chainSelEnv ResourceRegistry m
rr ChainDiff (Header blk)
chainDiff m (ValidatedChainDiff (Header blk) (Forker' m blk))
-> (ValidatedChainDiff (Header blk) (Forker' m blk)
-> m (ValidationResult m blk))
-> m (ValidationResult m blk)
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff
| ValidatedChainDiff (Header blk) (Forker' m blk) -> Bool
forall b l. HasHeader b => ValidatedChainDiff b l -> Bool
ValidatedDiff.rollbackExceedsSuffix ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff
-> ValidatedChainDiff (Header blk) (Forker' m blk) -> m ()
forall b (m :: * -> *) blk.
ValidatedChainDiff b (Forker' m blk) -> m ()
cleanup ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff m () -> m (ValidationResult m blk) -> m (ValidationResult m blk)
forall a b. m a -> m b -> m b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> ValidationResult m blk -> m (ValidationResult m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ValidationResult m blk
forall (m :: * -> *) blk. ValidationResult m blk
InsufficientSuffix
| AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length (ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix ChainDiff (Header blk)
chainDiff) Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== AnchoredSeq (WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
-> Int
forall v a b. Anchorable v a b => AnchoredSeq v a b -> Int
AF.length (ChainDiff (Header blk)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (Header blk)) (Header blk)
forall b. ChainDiff b -> AnchoredFragment b
Diff.getSuffix ChainDiff (Header blk)
chainDiff')
-> ValidationResult m blk -> m (ValidationResult m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidationResult m blk -> m (ValidationResult m blk))
-> ValidationResult m blk -> m (ValidationResult m blk)
forall a b. (a -> b) -> a -> b
$ ValidatedChainDiff (Header blk) (Forker' m blk)
-> ValidationResult m blk
forall (m :: * -> *) blk.
ValidatedChainDiff (Header blk) (Forker' m blk)
-> ValidationResult m blk
FullyValid ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff
| Bool
otherwise
-> ValidationResult m blk -> m (ValidationResult m blk)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ValidationResult m blk -> m (ValidationResult m blk))
-> ValidationResult m blk -> m (ValidationResult m blk)
forall a b. (a -> b) -> a -> b
$ ChainDiff (Header blk) -> ValidationResult m blk
forall (m :: * -> *) blk.
ChainDiff (Header blk) -> ValidationResult m blk
ValidPrefix ChainDiff (Header blk)
chainDiff'
where
chainDiff' :: ChainDiff (Header blk)
chainDiff' = ValidatedChainDiff (Header blk) (Forker' m blk)
-> ChainDiff (Header blk)
forall b l. ValidatedChainDiff b l -> ChainDiff b
ValidatedDiff.getChainDiff ValidatedChainDiff (Header blk) (Forker' m blk)
validatedChainDiff
where
cleanup :: ValidatedChainDiff b (Forker' m blk) -> m ()
cleanup :: forall b (m :: * -> *) blk.
ValidatedChainDiff b (Forker' m blk) -> m ()
cleanup = Forker m (ExtLedgerState blk) blk -> m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> m ()
forkerClose (Forker m (ExtLedgerState blk) blk -> m ())
-> (ValidatedChainDiff b (Forker m (ExtLedgerState blk) blk)
-> Forker m (ExtLedgerState blk) blk)
-> ValidatedChainDiff b (Forker m (ExtLedgerState blk) blk)
-> m ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ValidatedChainDiff b (Forker m (ExtLedgerState blk) blk)
-> Forker m (ExtLedgerState blk) blk
forall b l. ValidatedChainDiff b l -> l
getLedger
type ChainAndLedger m blk = ValidatedFragment (Header blk) (Forker' m blk)
isPipelineable ::
(HasHeader (Header blk), BlockSupportsDiffusionPipelining blk)
=> BlockConfig blk
-> TentativeHeaderState blk
-> ChainDiff (Header blk)
-> Maybe (Header blk, TentativeHeaderState blk)
isPipelineable :: forall blk.
(HasHeader (Header blk), BlockSupportsDiffusionPipelining blk) =>
BlockConfig blk
-> TentativeHeaderState blk
-> ChainDiff (Header blk)
-> Maybe (Header blk, TentativeHeaderState blk)
isPipelineable BlockConfig blk
bcfg TentativeHeaderState blk
st ChainDiff {Word64
AnchoredFragment (Header blk)
getSuffix :: forall b. ChainDiff b -> AnchoredFragment b
getRollback :: Word64
getSuffix :: AnchoredFragment (Header blk)
getRollback :: forall b. ChainDiff b -> Word64
..}
|
AF.Empty Anchor (Header blk)
_ :> Header blk
hdr <- AnchoredFragment (Header blk)
getSuffix
, Just TentativeHeaderState blk
st' <- BlockConfig blk
-> Header blk
-> TentativeHeaderState blk
-> Maybe (TentativeHeaderState blk)
forall blk.
BlockSupportsDiffusionPipelining blk =>
BlockConfig blk
-> Header blk
-> TentativeHeaderState blk
-> Maybe (TentativeHeaderState blk)
updateTentativeHeaderState BlockConfig blk
bcfg Header blk
hdr TentativeHeaderState blk
st
, Word64
getRollback Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
0
= (Header blk, TentativeHeaderState blk)
-> Maybe (Header blk, TentativeHeaderState blk)
forall a. a -> Maybe a
Just (Header blk
hdr, TentativeHeaderState blk
st')
| Bool
otherwise = Maybe (Header blk, TentativeHeaderState blk)
forall a. Maybe a
Nothing
ignoreInvalid ::
HasHeader blk
=> proxy blk
-> InvalidBlocks blk
-> (HeaderHash blk -> Maybe a)
-> (HeaderHash blk -> Maybe a)
ignoreInvalid :: forall blk (proxy :: * -> *) a.
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (HeaderHash blk -> Maybe a)
-> HeaderHash blk
-> Maybe a
ignoreInvalid proxy blk
_ InvalidBlocks blk
invalid HeaderHash blk -> Maybe a
getter HeaderHash blk
hash
| HeaderHash blk -> InvalidBlocks blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
Map.member HeaderHash blk
hash InvalidBlocks blk
invalid = Maybe a
forall a. Maybe a
Nothing
| Bool
otherwise = HeaderHash blk -> Maybe a
getter HeaderHash blk
hash
ignoreInvalidSuc ::
HasHeader blk
=> proxy blk
-> InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> (ChainHash blk -> Set (HeaderHash blk))
ignoreInvalidSuc :: forall blk (proxy :: * -> *).
HasHeader blk =>
proxy blk
-> InvalidBlocks blk
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
ignoreInvalidSuc proxy blk
_ InvalidBlocks blk
invalid ChainHash blk -> Set (HeaderHash blk)
succsOf =
(HeaderHash blk -> Bool)
-> Set (HeaderHash blk) -> Set (HeaderHash blk)
forall a. (a -> Bool) -> Set a -> Set a
Set.filter (HeaderHash blk -> InvalidBlocks blk -> Bool
forall k a. Ord k => k -> Map k a -> Bool
`Map.notMember` InvalidBlocks blk
invalid) (Set (HeaderHash blk) -> Set (HeaderHash blk))
-> (ChainHash blk -> Set (HeaderHash blk))
-> ChainHash blk
-> Set (HeaderHash blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChainHash blk -> Set (HeaderHash blk)
succsOf