{-# LANGUAGE FlexibleContexts #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
module Ouroboros.Consensus.MiniProtocol.BlockFetch.ClientInterface (
ChainDbView (..)
, SlotForgeTimeOracle
, defaultChainDbView
, initSlotForgeTimeOracle
, mkBlockFetchConsensusInterface
, readFetchModeDefault
) where
import Control.Monad
import Data.Map.Strict (Map)
import Data.Time.Clock (UTCTime)
import GHC.Stack (HasCallStack)
import Ouroboros.Consensus.Block hiding (blockMatchesHeader)
import qualified Ouroboros.Consensus.Block as Block
import Ouroboros.Consensus.BlockchainTime
import Ouroboros.Consensus.Config
import qualified Ouroboros.Consensus.Config.SupportsNode as SupportsNode
import qualified Ouroboros.Consensus.HardFork.Abstract as History
import qualified Ouroboros.Consensus.HardFork.History as History
import Ouroboros.Consensus.Ledger.Abstract
import Ouroboros.Consensus.Ledger.Extended
import Ouroboros.Consensus.Storage.ChainDB.API (ChainDB)
import qualified Ouroboros.Consensus.Storage.ChainDB.API as ChainDB
import Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment
(InvalidBlockPunishment)
import qualified Ouroboros.Consensus.Storage.ChainDB.API.Types.InvalidBlockPunishment as InvalidBlockPunishment
import Ouroboros.Consensus.Util.AnchoredFragment
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.Orphans ()
import Ouroboros.Network.AnchoredFragment (AnchoredFragment)
import qualified Ouroboros.Network.AnchoredFragment as AF
import Ouroboros.Network.Block (MaxSlotNo)
import Ouroboros.Network.BlockFetch.ConsensusInterface
(BlockFetchConsensusInterface (..), FetchMode (..),
FromConsensus (..))
import Ouroboros.Network.PeerSelection.Bootstrap (UseBootstrapPeers,
requiresBootstrapPeers)
import Ouroboros.Network.PeerSelection.LedgerPeers.Type
(LedgerStateJudgement)
import Ouroboros.Network.SizeInBytes
data ChainDbView m blk = ChainDbView {
forall (m :: * -> *) blk.
ChainDbView m blk -> STM m (AnchoredFragment (Header blk))
getCurrentChain :: STM m (AnchoredFragment (Header blk))
, forall (m :: * -> *) blk.
ChainDbView m blk -> STM m (Point blk -> Bool)
getIsFetched :: STM m (Point blk -> Bool)
, forall (m :: * -> *) blk. ChainDbView m blk -> STM m MaxSlotNo
getMaxSlotNo :: STM m MaxSlotNo
, forall (m :: * -> *) blk.
ChainDbView m blk -> InvalidBlockPunishment m -> blk -> m Bool
addBlockWaitWrittenToDisk :: InvalidBlockPunishment m -> blk -> m Bool
}
defaultChainDbView :: IOLike m => ChainDB m blk -> ChainDbView m blk
defaultChainDbView :: forall (m :: * -> *) blk.
IOLike m =>
ChainDB m blk -> ChainDbView m blk
defaultChainDbView ChainDB m blk
chainDB = ChainDbView {
getCurrentChain :: STM m (AnchoredFragment (Header blk))
getCurrentChain = ChainDB m blk -> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) blk.
ChainDB m blk -> STM m (AnchoredFragment (Header blk))
ChainDB.getCurrentChain ChainDB m blk
chainDB
, getIsFetched :: STM m (Point blk -> Bool)
getIsFetched = ChainDB m blk -> STM m (Point blk -> Bool)
forall (m :: * -> *) blk.
ChainDB m blk -> STM m (Point blk -> Bool)
ChainDB.getIsFetched ChainDB m blk
chainDB
, getMaxSlotNo :: STM m MaxSlotNo
getMaxSlotNo = ChainDB m blk -> STM m MaxSlotNo
forall (m :: * -> *) blk. ChainDB m blk -> STM m MaxSlotNo
ChainDB.getMaxSlotNo ChainDB m blk
chainDB
, addBlockWaitWrittenToDisk :: InvalidBlockPunishment m -> blk -> m Bool
addBlockWaitWrittenToDisk = ChainDB m blk -> InvalidBlockPunishment m -> blk -> m Bool
forall (m :: * -> *) blk.
IOLike m =>
ChainDB m blk -> InvalidBlockPunishment m -> blk -> m Bool
ChainDB.addBlockWaitWrittenToDisk ChainDB m blk
chainDB
}
type SlotForgeTimeOracle m blk = RealPoint blk -> STM m UTCTime
initSlotForgeTimeOracle ::
forall m blk.
( IOLike m
, BlockSupportsProtocol blk
, History.HasHardForkHistory blk
, SupportsNode.ConfigSupportsNode blk
, IsLedger (LedgerState blk)
)
=> TopLevelConfig blk
-> ChainDB m blk
-> m (SlotForgeTimeOracle m blk)
initSlotForgeTimeOracle :: forall (m :: * -> *) blk.
(IOLike m, BlockSupportsProtocol blk, HasHardForkHistory blk,
ConfigSupportsNode blk, IsLedger (LedgerState blk)) =>
TopLevelConfig blk
-> ChainDB m blk -> m (SlotForgeTimeOracle m blk)
initSlotForgeTimeOracle TopLevelConfig blk
cfg ChainDB m blk
chainDB = do
RunWithCachedSummary (HardForkIndices blk) m
cache <-
STM m (Summary (HardForkIndices blk))
-> m (RunWithCachedSummary (HardForkIndices blk) m)
forall (m :: * -> *) (xs :: [*]).
MonadSTM m =>
STM m (Summary xs) -> m (RunWithCachedSummary xs m)
History.runWithCachedSummary
(ExtLedgerState blk -> Summary (HardForkIndices blk)
toSummary (ExtLedgerState blk -> Summary (HardForkIndices blk))
-> STM m (ExtLedgerState blk)
-> STM m (Summary (HardForkIndices blk))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ChainDB m blk -> STM m (ExtLedgerState blk)
forall (m :: * -> *) blk.
(Monad (STM m), IsLedger (LedgerState blk)) =>
ChainDB m blk -> STM m (ExtLedgerState blk)
ChainDB.getCurrentLedger ChainDB m blk
chainDB)
let slotForgeTime :: RealPoint blk -> STM m UTCTime
slotForgeTime RealPoint blk
rp =
(Either PastHorizonException RelativeTime -> UTCTime)
-> STM m (Either PastHorizonException RelativeTime)
-> STM m UTCTime
forall a b. (a -> b) -> STM m a -> STM m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
((PastHorizonException -> UTCTime)
-> (RelativeTime -> UTCTime)
-> Either PastHorizonException RelativeTime
-> UTCTime
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either PastHorizonException -> UTCTime
forall {a} {a}. Show a => a -> a
errMsg RelativeTime -> UTCTime
toAbsolute)
(STM m (Either PastHorizonException RelativeTime) -> STM m UTCTime)
-> STM m (Either PastHorizonException RelativeTime)
-> STM m UTCTime
forall a b. (a -> b) -> a -> b
$ RunWithCachedSummary (HardForkIndices blk) m
-> forall a. Qry a -> STM m (Either PastHorizonException a)
forall (xs :: [*]) (m :: * -> *).
RunWithCachedSummary xs m
-> forall a. Qry a -> STM m (Either PastHorizonException a)
History.cachedRunQuery
RunWithCachedSummary (HardForkIndices blk) m
cache
((RelativeTime, SlotLength) -> RelativeTime
forall a b. (a, b) -> a
fst ((RelativeTime, SlotLength) -> RelativeTime)
-> Qry (RelativeTime, SlotLength) -> Qry RelativeTime
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> SlotNo -> Qry (RelativeTime, SlotLength)
History.slotToWallclock (RealPoint blk -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint blk
rp))
where
errMsg :: a -> a
errMsg a
err =
[Char] -> a
forall a. HasCallStack => [Char] -> a
error ([Char] -> a) -> [Char] -> a
forall a b. (a -> b) -> a -> b
$
[Char]
"Consensus could not determine forge UTCTime!"
[Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> [Char]
" " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> RealPoint blk -> [Char]
forall a. Show a => a -> [Char]
show RealPoint blk
rp
[Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> [Char]
" " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> a -> [Char]
forall a. Show a => a -> [Char]
show a
err
SlotForgeTimeOracle m blk -> m (SlotForgeTimeOracle m blk)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure SlotForgeTimeOracle m blk
forall {blk}. StandardHash blk => RealPoint blk -> STM m UTCTime
slotForgeTime
where
toSummary ::
ExtLedgerState blk
-> History.Summary (History.HardForkIndices blk)
toSummary :: ExtLedgerState blk -> Summary (HardForkIndices blk)
toSummary = LedgerConfig blk
-> LedgerState blk -> Summary (HardForkIndices blk)
forall blk.
HasHardForkHistory blk =>
LedgerConfig blk
-> LedgerState blk -> Summary (HardForkIndices blk)
History.hardForkSummary (TopLevelConfig blk -> LedgerConfig blk
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig blk
cfg) (LedgerState blk -> Summary (HardForkIndices blk))
-> (ExtLedgerState blk -> LedgerState blk)
-> ExtLedgerState blk
-> Summary (HardForkIndices blk)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ExtLedgerState blk -> LedgerState blk
forall blk. ExtLedgerState blk -> LedgerState blk
ledgerState
toAbsolute :: RelativeTime -> UTCTime
toAbsolute :: RelativeTime -> UTCTime
toAbsolute =
SystemStart -> RelativeTime -> UTCTime
fromRelativeTime (BlockConfig blk -> SystemStart
forall blk.
ConfigSupportsNode blk =>
BlockConfig blk -> SystemStart
SupportsNode.getSystemStart (TopLevelConfig blk -> BlockConfig blk
forall blk. TopLevelConfig blk -> BlockConfig blk
configBlock TopLevelConfig blk
cfg))
readFetchModeDefault ::
(MonadSTM m, HasHeader blk)
=> BlockchainTime m
-> STM m (AnchoredFragment blk)
-> STM m UseBootstrapPeers
-> STM m LedgerStateJudgement
-> STM m FetchMode
readFetchModeDefault :: forall (m :: * -> *) blk.
(MonadSTM m, HasHeader blk) =>
BlockchainTime m
-> STM m (AnchoredFragment blk)
-> STM m UseBootstrapPeers
-> STM m LedgerStateJudgement
-> STM m FetchMode
readFetchModeDefault BlockchainTime m
btime STM m (AnchoredFragment blk)
getCurrentChain
STM m UseBootstrapPeers
getUseBootstrapPeers STM m LedgerStateJudgement
getLedgerStateJudgement = do
CurrentSlot
mCurSlot <- BlockchainTime m -> STM m CurrentSlot
forall (m :: * -> *). BlockchainTime m -> STM m CurrentSlot
getCurrentSlot BlockchainTime m
btime
Bool
usingBootstrapPeers <- UseBootstrapPeers -> LedgerStateJudgement -> Bool
requiresBootstrapPeers (UseBootstrapPeers -> LedgerStateJudgement -> Bool)
-> STM m UseBootstrapPeers -> STM m (LedgerStateJudgement -> Bool)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m UseBootstrapPeers
getUseBootstrapPeers
STM m (LedgerStateJudgement -> Bool)
-> STM m LedgerStateJudgement -> STM m Bool
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
<*> STM m LedgerStateJudgement
getLedgerStateJudgement
case (Bool
usingBootstrapPeers, CurrentSlot
mCurSlot) of
(Bool
True, CurrentSlot
_) -> FetchMode -> STM m FetchMode
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return FetchMode
FetchModeBulkSync
(Bool
False, CurrentSlot
CurrentSlotUnknown) -> FetchMode -> STM m FetchMode
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return FetchMode
FetchModeBulkSync
(Bool
False, CurrentSlot SlotNo
curSlot) -> do
WithOrigin SlotNo
curChainSlot <- AnchoredFragment blk -> WithOrigin SlotNo
forall block.
HasHeader block =>
AnchoredFragment block -> WithOrigin SlotNo
AF.headSlot (AnchoredFragment blk -> WithOrigin SlotNo)
-> STM m (AnchoredFragment blk) -> STM m (WithOrigin SlotNo)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> STM m (AnchoredFragment blk)
getCurrentChain
let slotsBehind :: Word64
slotsBehind = case WithOrigin SlotNo
curChainSlot of
WithOrigin SlotNo
Origin -> SlotNo -> Word64
unSlotNo SlotNo
curSlot Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
+ Word64
1
NotOrigin SlotNo
slot -> SlotNo -> Word64
unSlotNo SlotNo
curSlot Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
- SlotNo -> Word64
unSlotNo SlotNo
slot
maxSlotsBehind :: Word64
maxSlotsBehind = Word64
1000
FetchMode -> STM m FetchMode
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return (FetchMode -> STM m FetchMode) -> FetchMode -> STM m FetchMode
forall a b. (a -> b) -> a -> b
$ if Word64
slotsBehind Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
< Word64
maxSlotsBehind
then FetchMode
FetchModeDeadline
else FetchMode
FetchModeBulkSync
mkBlockFetchConsensusInterface ::
forall m peer blk.
( IOLike m
, BlockSupportsDiffusionPipelining blk
, BlockSupportsProtocol blk
)
=> BlockConfig blk
-> ChainDbView m blk
-> STM m (Map peer (AnchoredFragment (Header blk)))
-> (Header blk -> SizeInBytes)
-> SlotForgeTimeOracle m blk
-> STM m FetchMode
-> DiffusionPipeliningSupport
-> BlockFetchConsensusInterface peer (Header blk) blk m
mkBlockFetchConsensusInterface :: forall (m :: * -> *) peer blk.
(IOLike m, BlockSupportsDiffusionPipelining blk,
BlockSupportsProtocol blk) =>
BlockConfig blk
-> ChainDbView m blk
-> STM m (Map peer (AnchoredFragment (Header blk)))
-> (Header blk -> SizeInBytes)
-> SlotForgeTimeOracle m blk
-> STM m FetchMode
-> DiffusionPipeliningSupport
-> BlockFetchConsensusInterface peer (Header blk) blk m
mkBlockFetchConsensusInterface
BlockConfig blk
bcfg ChainDbView m blk
chainDB STM m (Map peer (AnchoredFragment (Header blk)))
getCandidates Header blk -> SizeInBytes
blockFetchSize SlotForgeTimeOracle m blk
slotForgeTime STM m FetchMode
readFetchMode DiffusionPipeliningSupport
pipelining =
BlockFetchConsensusInterface {STM m (Map peer (AnchoredFragment (Header blk)))
STM m (AnchoredFragment (Header blk))
STM m MaxSlotNo
STM m FetchMode
STM m (Point blk -> Bool)
STM m (Point blk -> blk -> m ())
HasCallStack =>
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Bool
HasCallStack =>
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Ordering
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Bool
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Ordering
FromConsensus blk -> STM m UTCTime
FromConsensus (Header blk) -> STM m UTCTime
Header blk -> SizeInBytes
Header blk -> blk -> Bool
headerForgeUTCTime :: FromConsensus (Header blk) -> STM m UTCTime
blockForgeUTCTime :: FromConsensus blk -> STM m UTCTime
readFetchMode :: STM m FetchMode
blockFetchSize :: Header blk -> SizeInBytes
readFetchMode :: STM m FetchMode
blockMatchesHeader :: Header blk -> blk -> Bool
readCandidateChains :: STM m (Map peer (AnchoredFragment (Header blk)))
readCurrentChain :: STM m (AnchoredFragment (Header blk))
readFetchedBlocks :: STM m (Point blk -> Bool)
mkAddFetchedBlock :: STM m (Point blk -> blk -> m ())
readFetchedMaxSlotNo :: STM m MaxSlotNo
plausibleCandidateChain :: HasCallStack =>
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Bool
compareCandidateChains :: AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Ordering
headerForgeUTCTime :: FromConsensus (Header blk) -> STM m UTCTime
blockForgeUTCTime :: FromConsensus blk -> STM m UTCTime
readCandidateChains :: STM m (Map peer (AnchoredFragment (Header blk)))
readCurrentChain :: STM m (AnchoredFragment (Header blk))
readFetchedBlocks :: STM m (Point blk -> Bool)
mkAddFetchedBlock :: STM m (Point blk -> blk -> m ())
readFetchedMaxSlotNo :: STM m MaxSlotNo
plausibleCandidateChain :: HasCallStack =>
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Bool
compareCandidateChains :: HasCallStack =>
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Ordering
blockFetchSize :: Header blk -> SizeInBytes
blockMatchesHeader :: Header blk -> blk -> Bool
..}
where
blockMatchesHeader :: Header blk -> blk -> Bool
blockMatchesHeader :: Header blk -> blk -> Bool
blockMatchesHeader = Header blk -> blk -> Bool
forall blk. GetHeader blk => Header blk -> blk -> Bool
Block.blockMatchesHeader
readCandidateChains :: STM m (Map peer (AnchoredFragment (Header blk)))
readCandidateChains :: STM m (Map peer (AnchoredFragment (Header blk)))
readCandidateChains = STM m (Map peer (AnchoredFragment (Header blk)))
getCandidates
readCurrentChain :: STM m (AnchoredFragment (Header blk))
readCurrentChain :: STM m (AnchoredFragment (Header blk))
readCurrentChain = ChainDbView m blk -> STM m (AnchoredFragment (Header blk))
forall (m :: * -> *) blk.
ChainDbView m blk -> STM m (AnchoredFragment (Header blk))
getCurrentChain ChainDbView m blk
chainDB
readFetchedBlocks :: STM m (Point blk -> Bool)
readFetchedBlocks :: STM m (Point blk -> Bool)
readFetchedBlocks = ChainDbView m blk -> STM m (Point blk -> Bool)
forall (m :: * -> *) blk.
ChainDbView m blk -> STM m (Point blk -> Bool)
getIsFetched ChainDbView m blk
chainDB
mkAddFetchedBlock ::
STM m (Point blk -> blk -> m ())
mkAddFetchedBlock :: STM m (Point blk -> blk -> m ())
mkAddFetchedBlock = do
BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m
pipeliningPunishment <- STM
m
(BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m)
forall (m :: * -> *) blk.
(IOLike m, BlockSupportsDiffusionPipelining blk) =>
STM
m
(BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m)
InvalidBlockPunishment.mkForDiffusionPipelining
(Point blk -> blk -> m ()) -> STM m (Point blk -> blk -> m ())
forall a. a -> STM m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ((Point blk -> blk -> m ()) -> STM m (Point blk -> blk -> m ()))
-> (Point blk -> blk -> m ()) -> STM m (Point blk -> blk -> m ())
forall a b. (a -> b) -> a -> b
$ (BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m)
-> DiffusionPipeliningSupport -> Point blk -> blk -> m ()
mkAddFetchedBlock_ BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m
pipeliningPunishment DiffusionPipeliningSupport
pipelining
mkAddFetchedBlock_ ::
( BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m
)
-> DiffusionPipeliningSupport
-> Point blk
-> blk
-> m ()
mkAddFetchedBlock_ :: (BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m)
-> DiffusionPipeliningSupport -> Point blk -> blk -> m ()
mkAddFetchedBlock_ BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m
pipeliningPunishment DiffusionPipeliningSupport
enabledPipelining Point blk
_pt blk
blk = m Bool -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m Bool -> m ()) -> m Bool -> m ()
forall a b. (a -> b) -> a -> b
$ do
InvalidBlockPunishment m
disconnect <- m (InvalidBlockPunishment m)
forall (m :: * -> *). IOLike m => m (InvalidBlockPunishment m)
InvalidBlockPunishment.mkPunishThisThread
let punishment :: InvalidBlockPunishment m
punishment = (Invalidity -> InvalidBlockPunishment m)
-> InvalidBlockPunishment m
forall (m :: * -> *).
(Invalidity -> InvalidBlockPunishment m)
-> InvalidBlockPunishment m
InvalidBlockPunishment.branch ((Invalidity -> InvalidBlockPunishment m)
-> InvalidBlockPunishment m)
-> (Invalidity -> InvalidBlockPunishment m)
-> InvalidBlockPunishment m
forall a b. (a -> b) -> a -> b
$ \case
Invalidity
InvalidBlockPunishment.BlockPrefix -> InvalidBlockPunishment m
disconnect
Invalidity
InvalidBlockPunishment.BlockItself -> case DiffusionPipeliningSupport
enabledPipelining of
DiffusionPipeliningSupport
DiffusionPipeliningOff -> InvalidBlockPunishment m
disconnect
DiffusionPipeliningSupport
DiffusionPipeliningOn ->
BlockConfig blk
-> Header blk
-> InvalidBlockPunishment m
-> InvalidBlockPunishment m
pipeliningPunishment BlockConfig blk
bcfg (blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader blk
blk) InvalidBlockPunishment m
disconnect
ChainDbView m blk -> InvalidBlockPunishment m -> blk -> m Bool
forall (m :: * -> *) blk.
ChainDbView m blk -> InvalidBlockPunishment m -> blk -> m Bool
addBlockWaitWrittenToDisk
ChainDbView m blk
chainDB
InvalidBlockPunishment m
punishment
blk
blk
readFetchedMaxSlotNo :: STM m MaxSlotNo
readFetchedMaxSlotNo :: STM m MaxSlotNo
readFetchedMaxSlotNo = ChainDbView m blk -> STM m MaxSlotNo
forall (m :: * -> *) blk. ChainDbView m blk -> STM m MaxSlotNo
getMaxSlotNo ChainDbView m blk
chainDB
plausibleCandidateChain :: HasCallStack
=> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
plausibleCandidateChain :: HasCallStack =>
AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Bool
plausibleCandidateChain AnchoredFragment (Header blk)
ours AnchoredFragment (Header blk)
cand
| AnchoredFragment (Header blk)
-> (WithOrigin BlockNo, WithOrigin SlotNo)
anchorBlockNoAndSlot AnchoredFragment (Header blk)
cand (WithOrigin BlockNo, WithOrigin SlotNo)
-> (WithOrigin BlockNo, WithOrigin SlotNo) -> Bool
forall a. Ord a => a -> a -> Bool
< AnchoredFragment (Header blk)
-> (WithOrigin BlockNo, WithOrigin SlotNo)
anchorBlockNoAndSlot AnchoredFragment (Header blk)
ours
= case (AnchoredFragment (Header blk) -> Bool
forall v a b. AnchoredSeq v a b -> Bool
AF.null AnchoredFragment (Header blk)
ours, AnchoredFragment (Header blk) -> Bool
forall v a b. AnchoredSeq v a b -> Bool
AF.null AnchoredFragment (Header blk)
cand) of
(Bool
False, Bool
False) -> BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
ours AnchoredFragment (Header blk)
cand
(Bool
_, Bool
True) -> Bool
False
(Bool
True, Bool
_) -> [Char] -> Bool
forall a. HasCallStack => [Char] -> a
error [Char]
"impossible"
| Bool
otherwise
= BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Bool
preferAnchoredCandidate BlockConfig blk
bcfg AnchoredFragment (Header blk)
ours AnchoredFragment (Header blk)
cand
where
anchorBlockNoAndSlot ::
AnchoredFragment (Header blk)
-> (WithOrigin BlockNo, WithOrigin SlotNo)
anchorBlockNoAndSlot :: AnchoredFragment (Header blk)
-> (WithOrigin BlockNo, WithOrigin SlotNo)
anchorBlockNoAndSlot AnchoredFragment (Header blk)
frag =
(Anchor (Header blk) -> WithOrigin BlockNo
forall block. Anchor block -> WithOrigin BlockNo
AF.anchorToBlockNo Anchor (Header blk)
a, Anchor (Header blk) -> WithOrigin SlotNo
forall block. Anchor block -> WithOrigin SlotNo
AF.anchorToSlotNo Anchor (Header blk)
a)
where
a :: Anchor (Header blk)
a = AnchoredFragment (Header blk) -> Anchor (Header blk)
forall v a b. AnchoredSeq v a b -> a
AF.anchor AnchoredFragment (Header blk)
frag
compareCandidateChains :: AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Ordering
compareCandidateChains :: AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk) -> Ordering
compareCandidateChains = BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Ordering
forall blk.
(BlockSupportsProtocol blk, HasCallStack) =>
BlockConfig blk
-> AnchoredFragment (Header blk)
-> AnchoredFragment (Header blk)
-> Ordering
compareAnchoredFragments BlockConfig blk
bcfg
headerForgeUTCTime :: FromConsensus (Header blk) -> STM m UTCTime
headerForgeUTCTime = SlotForgeTimeOracle m blk
slotForgeTime SlotForgeTimeOracle m blk
-> (FromConsensus (Header blk) -> RealPoint blk)
-> FromConsensus (Header blk)
-> STM m UTCTime
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Header blk -> RealPoint blk
forall blk.
(HasHeader (Header blk), HasHeader blk) =>
Header blk -> RealPoint blk
headerRealPoint (Header blk -> RealPoint blk)
-> (FromConsensus (Header blk) -> Header blk)
-> FromConsensus (Header blk)
-> RealPoint blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FromConsensus (Header blk) -> Header blk
forall a. FromConsensus a -> a
unFromConsensus
blockForgeUTCTime :: FromConsensus blk -> STM m UTCTime
blockForgeUTCTime = SlotForgeTimeOracle m blk
slotForgeTime SlotForgeTimeOracle m blk
-> (FromConsensus blk -> RealPoint blk)
-> FromConsensus blk
-> STM m UTCTime
forall b c a. (b -> c) -> (a -> b) -> a -> c
. blk -> RealPoint blk
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint (blk -> RealPoint blk)
-> (FromConsensus blk -> blk) -> FromConsensus blk -> RealPoint blk
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FromConsensus blk -> blk
forall a. FromConsensus a -> a
unFromConsensus