{-# LANGUAGE DataKinds #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE TypeApplications #-}
module Main (main) where
import Bench.Consensus.ChainSyncClient.Driver (mainWith)
import Cardano.Crypto.DSIGN.Mock
import Cardano.Ledger.BaseTypes (knownNonZeroBounded)
import Control.Monad (void)
import Control.ResourceRegistry
import Control.Tracer (contramap, debugTracer, nullTracer)
import Data.IORef (newIORef, readIORef, writeIORef)
import qualified Data.List.NonEmpty as NE
import qualified Data.Map.Strict as Map
import Main.Utf8 (withStdTerminalHandles)
import Network.TypedProtocol.Channel
import Network.TypedProtocol.Driver.Simple
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.BlockchainTime
import Ouroboros.Consensus.Config
import qualified Ouroboros.Consensus.HardFork.History as HardFork
import qualified Ouroboros.Consensus.HeaderStateHistory as HeaderStateHistory
import qualified Ouroboros.Consensus.HeaderValidation as HV
import qualified Ouroboros.Consensus.Ledger.Extended as Extended
import qualified Ouroboros.Consensus.MiniProtocol.ChainSync.Client as CSClient
import qualified Ouroboros.Consensus.MiniProtocol.ChainSync.Client.HistoricityCheck as HistoricityCheck
import Ouroboros.Consensus.MiniProtocol.ChainSync.Client.InFutureCheck
(clockSkewInSeconds)
import qualified Ouroboros.Consensus.MiniProtocol.ChainSync.Client.InFutureCheck as InFutureCheck
import Ouroboros.Consensus.MiniProtocol.ChainSync.Server
(chainSyncServerForFollower)
import Ouroboros.Consensus.Node.NetworkProtocolVersion
(NodeToNodeVersion)
import Ouroboros.Consensus.Node.ProtocolInfo
import Ouroboros.Consensus.NodeId
import Ouroboros.Consensus.Protocol.BFT
import qualified Ouroboros.Consensus.Storage.ChainDB.API as ChainDB
import Ouroboros.Consensus.Util.IOLike
import Ouroboros.Consensus.Util.STM (Fingerprint (..),
WithFingerprint (..))
import Ouroboros.Consensus.Util.Time (secondsToNominalDiffTime)
import Ouroboros.Network.AnchoredFragment (AnchoredFragment)
import qualified Ouroboros.Network.AnchoredFragment as AF
import Ouroboros.Network.Block (ChainUpdate (AddBlock, RollBack),
Tip (TipGenesis), tipFromHeader)
import Ouroboros.Network.ControlMessage (ControlMessage (Continue))
import qualified Ouroboros.Network.Mock.Chain as Chain
import Ouroboros.Network.Protocol.ChainSync.ClientPipelined
import Ouroboros.Network.Protocol.ChainSync.Codec (codecChainSyncId)
import Ouroboros.Network.Protocol.ChainSync.PipelineDecision
(pipelineDecisionLowHighMark)
import Ouroboros.Network.Protocol.ChainSync.Server
import Test.Util.Orphans.Arbitrary ()
import Test.Util.Orphans.IOLike ()
import qualified Test.Util.TestBlock as TB
type B = TB.TestBlock
type H = Header B
main :: IO ()
IO ()
main = IO () -> IO ()
forall (m :: * -> *) r. (MonadIO m, MonadMask m) => m r -> m r
withStdTerminalHandles (IO () -> IO ()) -> IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ (Int64 -> IO ()) -> IO ()
mainWith ((Int64 -> IO ()) -> IO ()) -> (Int64 -> IO ()) -> IO ()
forall a b. (a -> b) -> a -> b
$ \Int64
n -> do
varCandidate <- AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B)
-> IO
(StrictTVar
IO
(AnchoredSeq
(WithOrigin SlotNo)
(Anchor (HeaderWithTime B))
(HeaderWithTime B)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO (AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B)
-> IO
(StrictTVar
IO
(AnchoredSeq
(WithOrigin SlotNo)
(Anchor (HeaderWithTime B))
(HeaderWithTime B))))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B)
-> IO
(StrictTVar
IO
(AnchoredSeq
(WithOrigin SlotNo)
(Anchor (HeaderWithTime B))
(HeaderWithTime B)))
forall a b. (a -> b) -> a -> b
$ Anchor (HeaderWithTime B)
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty Anchor (HeaderWithTime B)
forall block. Anchor block
AF.AnchorGenesis
varServerTip <- newTVarIO TipGenesis
follower <- mkFollower varServerTip
oneBenchRun
varCandidate
varServerTip
follower
(fromIntegral n)
{-# INLINE oneBenchRun #-}
oneBenchRun ::
StrictTVar IO (AnchoredFragment (HV.HeaderWithTime B))
-> StrictTVar IO (Tip B)
-> ChainDB.Follower IO B (ChainDB.WithPoint B H)
-> Int
-> IO ()
oneBenchRun :: StrictTVar
IO
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B))
-> StrictTVar IO (Tip B)
-> Follower IO B (WithPoint B (Header B))
-> Int
-> IO ()
oneBenchRun
StrictTVar
IO
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B))
varCandidate
StrictTVar IO (Tip B)
varServerTip
Follower IO B (WithPoint B (Header B))
follower
Int
n
=
(ResourceRegistry IO -> IO ()) -> IO ()
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
(ResourceRegistry m -> m a) -> m a
withRegistry ((ResourceRegistry IO -> IO ()) -> IO ())
-> (ResourceRegistry IO -> IO ()) -> IO ()
forall a b. (a -> b) -> a -> b
$ \ResourceRegistry IO
registry -> do
(clientChannel, serverChannel) <- IO
(Channel IO (AnyMessage (ChainSync (Header B) (Point B) (Tip B))),
Channel IO (AnyMessage (ChainSync (Header B) (Point B) (Tip B))))
forall (m :: * -> *) a.
(MonadLabelledSTM m, MonadTraceSTM m, Show a) =>
m (Channel m a, Channel m a)
createConnectedChannels
void
$ forkLinkedThread registry "ChainSyncServer"
$ runPeer nullTracer codecChainSyncId serverChannel
$ chainSyncServerPeer server
void
$ forkLinkedThread registry "ChainSyncClient"
$ void
$ runPipelinedPeer nullTracer codecChainSyncId clientChannel
$ chainSyncClientPeerPipelined client
atomically $ do
candidate <- readTVar varCandidate
check $ case pointHash $ AF.headPoint candidate of
BlockHash (TB.TestHash NonEmpty Word64
ne) -> Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral Int
n Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
< NonEmpty Word64 -> Word64
forall a. NonEmpty a -> a
NE.head NonEmpty Word64
ne
ChainHash (HeaderWithTime B)
_ -> Bool
False
where
chainDbView :: CSClient.ChainDbView IO B
chainDbView :: ChainDbView IO B
chainDbView = CSClient.ChainDbView {
getCurrentChain :: STM IO (AnchoredFragment (Header B))
CSClient.getCurrentChain = AnchoredFragment (Header B) -> STM IO (AnchoredFragment (Header B))
forall a. a -> STM IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (AnchoredFragment (Header B)
-> STM IO (AnchoredFragment (Header B)))
-> AnchoredFragment (Header B)
-> STM IO (AnchoredFragment (Header B))
forall a b. (a -> b) -> a -> b
$ Anchor (Header B) -> AnchoredFragment (Header B)
forall v a b. Anchorable v a b => a -> AnchoredSeq v a b
AF.Empty Anchor (Header B)
forall block. Anchor block
AF.AnchorGenesis
, getHeaderStateHistory :: STM IO (HeaderStateHistory B)
CSClient.getHeaderStateHistory =
HeaderStateHistory B -> STM IO (HeaderStateHistory B)
forall a. a -> STM IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
(HeaderStateHistory B -> STM IO (HeaderStateHistory B))
-> HeaderStateHistory B -> STM IO (HeaderStateHistory B)
forall a b. (a -> b) -> a -> b
$ TopLevelConfig B
-> ExtLedgerState B ValuesMK -> Chain B -> HeaderStateHistory B
forall blk.
(ApplyBlock (ExtLedgerState blk) blk, HasHardForkHistory blk,
HasAnnTip blk) =>
TopLevelConfig blk
-> ExtLedgerState blk ValuesMK
-> Chain blk
-> HeaderStateHistory blk
HeaderStateHistory.fromChain
TopLevelConfig B
topConfig
(Point B -> ExtLedgerState B ValuesMK
forall (mk :: MapKind). Point B -> ExtLedgerState B mk
oracularLedgerDB Point B
forall {k} (block :: k). Point block
GenesisPoint)
Chain B
forall block. Chain block
Chain.Genesis
, getIsInvalidBlock :: STM
IO (WithFingerprint (HeaderHash B -> Maybe (ExtValidationError B)))
CSClient.getIsInvalidBlock = WithFingerprint (TestHash -> Maybe (ExtValidationError B))
-> STM (WithFingerprint (TestHash -> Maybe (ExtValidationError B)))
forall a. a -> STM a
forall (f :: * -> *) a. Applicative f => a -> f a
pure WithFingerprint (HeaderHash B -> Maybe (ExtValidationError B))
WithFingerprint (TestHash -> Maybe (ExtValidationError B))
forall blk.
WithFingerprint (HeaderHash blk -> Maybe (ExtValidationError blk))
invalidBlock
, getPastLedger :: Point B -> STM IO (Maybe (ExtLedgerState B EmptyMK))
CSClient.getPastLedger = Maybe (ExtLedgerState B EmptyMK)
-> STM (Maybe (ExtLedgerState B EmptyMK))
forall a. a -> STM a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (ExtLedgerState B EmptyMK)
-> STM (Maybe (ExtLedgerState B EmptyMK)))
-> (Point B -> Maybe (ExtLedgerState B EmptyMK))
-> Point B
-> STM (Maybe (ExtLedgerState B EmptyMK))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ExtLedgerState B EmptyMK -> Maybe (ExtLedgerState B EmptyMK)
forall a. a -> Maybe a
Just (ExtLedgerState B EmptyMK -> Maybe (ExtLedgerState B EmptyMK))
-> (Point B -> ExtLedgerState B EmptyMK)
-> Point B
-> Maybe (ExtLedgerState B EmptyMK)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Point B -> ExtLedgerState B EmptyMK
forall (mk :: MapKind). Point B -> ExtLedgerState B mk
oracularLedgerDB
}
headerInFutureCheck ::
InFutureCheck.SomeHeaderInFutureCheck IO B
headerInFutureCheck :: SomeHeaderInFutureCheck IO B
headerInFutureCheck =
ClockSkew -> SystemTime IO -> SomeHeaderInFutureCheck IO B
forall blk (m :: * -> *).
(HasHeader blk, HasHeader (Header blk), HasHardForkHistory blk,
MonadDelay m) =>
ClockSkew -> SystemTime m -> SomeHeaderInFutureCheck m blk
InFutureCheck.realHeaderInFutureCheck
(Double -> ClockSkew
clockSkewInSeconds Double
0)
SystemTime IO
inTheYearOneBillion
client :: CSClient.Consensus ChainSyncClientPipelined B IO
client :: ChainSyncClientPipelined
(Header B) (Point B) (Tip B) IO ChainSyncClientResult
client =
ConfigEnv IO B
-> DynamicEnv IO B
-> ChainSyncClientPipelined
(Header B) (Point B) (Tip B) IO ChainSyncClientResult
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk) =>
ConfigEnv m blk
-> DynamicEnv m blk -> Consensus ChainSyncClientPipelined blk m
CSClient.chainSyncClient
CSClient.ConfigEnv {
ChainDbView IO B
chainDbView :: ChainDbView IO B
chainDbView :: ChainDbView IO B
CSClient.chainDbView
, cfg :: TopLevelConfig B
CSClient.cfg = TopLevelConfig B
topConfig
, tracer :: Tracer IO (TraceChainSyncClientEvent B)
CSClient.tracer = Tracer IO (TraceChainSyncClientEvent B)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer Tracer IO (TraceChainSyncClientEvent B)
-> Tracer IO (TraceChainSyncClientEvent B)
-> Tracer IO (TraceChainSyncClientEvent B)
forall a. a -> a -> a
`asTypeOf` (TraceChainSyncClientEvent B -> [Char])
-> Tracer IO [Char] -> Tracer IO (TraceChainSyncClientEvent B)
forall a' a. (a' -> a) -> Tracer IO a -> Tracer IO a'
forall (f :: * -> *) a' a.
Contravariant f =>
(a' -> a) -> f a -> f a'
contramap TraceChainSyncClientEvent B -> [Char]
forall a. Show a => a -> [Char]
show Tracer IO [Char]
forall (m :: * -> *). Applicative m => Tracer m [Char]
debugTracer
, someHeaderInFutureCheck :: SomeHeaderInFutureCheck IO B
CSClient.someHeaderInFutureCheck = SomeHeaderInFutureCheck IO B
headerInFutureCheck
, historicityCheck :: HistoricityCheck IO B
CSClient.historicityCheck = HistoricityCheck IO B
forall (m :: * -> *) blk. Applicative m => HistoricityCheck m blk
HistoricityCheck.noCheck
, mkPipelineDecision0 :: MkPipelineDecision
CSClient.mkPipelineDecision0 =
Word16 -> Word16 -> MkPipelineDecision
pipelineDecisionLowHighMark Word16
10 Word16
20
, getDiffusionPipeliningSupport :: DiffusionPipeliningSupport
CSClient.getDiffusionPipeliningSupport =
DiffusionPipeliningSupport
DiffusionPipeliningOn
}
CSClient.DynamicEnv {
version :: NodeToNodeVersion
CSClient.version = NodeToNodeVersion
forall a. Bounded a => a
maxBound :: NodeToNodeVersion
, controlMessageSTM :: ControlMessageSTM IO
CSClient.controlMessageSTM = ControlMessage -> STM ControlMessage
forall a. a -> STM a
forall (m :: * -> *) a. Monad m => a -> m a
return ControlMessage
Continue
, headerMetricsTracer :: HeaderMetricsTracer IO
CSClient.headerMetricsTracer = Tracer STM (SlotNo, Time)
HeaderMetricsTracer IO
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
, setCandidate :: AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B)
-> STM IO ()
CSClient.setCandidate = StrictTVar
IO
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B))
-> AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B)
-> STM IO ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar
IO
(AnchoredSeq
(WithOrigin SlotNo) (Anchor (HeaderWithTime B)) (HeaderWithTime B))
varCandidate
, setLatestSlot :: WithOrigin SlotNo -> STM IO ()
CSClient.setLatestSlot = \WithOrigin SlotNo
_ -> () -> STM ()
forall a. a -> STM a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
, idling :: Idling IO
CSClient.idling = Idling IO
forall (m :: * -> *). Applicative m => Idling m
CSClient.noIdling
, loPBucket :: LoPBucket IO
CSClient.loPBucket = LoPBucket IO
forall (m :: * -> *). Applicative m => LoPBucket m
CSClient.noLoPBucket
, jumping :: Jumping IO B
CSClient.jumping = Jumping IO B
forall (m :: * -> *) blk. MonadSTM m => Jumping m blk
CSClient.noJumping
}
server :: ChainSyncServer H (Point B) (Tip B) IO ()
server :: ChainSyncServer (Header B) (Point B) (Tip B) IO ()
server =
Tracer IO (TraceChainSyncServerEvent B)
-> STM IO (Tip B)
-> Follower IO B (WithPoint B (Header B))
-> ChainSyncServer (Header B) (Point B) (Tip B) IO ()
forall (m :: * -> *) blk b.
IOLike m =>
Tracer m (TraceChainSyncServerEvent blk)
-> STM m (Tip blk)
-> Follower m blk (WithPoint blk b)
-> ChainSyncServer b (Point blk) (Tip blk) m ()
chainSyncServerForFollower
Tracer IO (TraceChainSyncServerEvent B)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
(StrictTVar IO (Tip B) -> STM IO (Tip B)
forall (m :: * -> *) a. MonadSTM m => StrictTVar m a -> STM m a
readTVar StrictTVar IO (Tip B)
varServerTip)
Follower IO B (WithPoint B (Header B))
follower
invalidBlock ::
WithFingerprint
(HeaderHash blk -> Maybe (Extended.ExtValidationError blk))
invalidBlock :: forall blk.
WithFingerprint (HeaderHash blk -> Maybe (ExtValidationError blk))
invalidBlock =
(HeaderHash blk -> Maybe (ExtValidationError blk))
-> Fingerprint
-> WithFingerprint
(HeaderHash blk -> Maybe (ExtValidationError blk))
forall a. a -> Fingerprint -> WithFingerprint a
WithFingerprint HeaderHash blk -> Maybe (ExtValidationError blk)
forall {p} {a}. p -> Maybe a
isInvalidBlock Fingerprint
fp
where
isInvalidBlock :: p -> Maybe a
isInvalidBlock p
_hash = Maybe a
forall a. Maybe a
Nothing
fp :: Fingerprint
fp = Word64 -> Fingerprint
Fingerprint (Word64 -> Fingerprint) -> Word64 -> Fingerprint
forall a b. (a -> b) -> a -> b
$ Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Int
0 :: Int)
inTheYearOneBillion :: SystemTime IO
inTheYearOneBillion :: SystemTime IO
inTheYearOneBillion = SystemTime {
systemTimeWait :: IO ()
systemTimeWait = () -> IO ()
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
, systemTimeCurrent :: IO RelativeTime
systemTimeCurrent = RelativeTime -> IO RelativeTime
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (RelativeTime -> IO RelativeTime)
-> RelativeTime -> IO RelativeTime
forall a b. (a -> b) -> a -> b
$ NominalDiffTime -> RelativeTime
RelativeTime (NominalDiffTime -> RelativeTime)
-> NominalDiffTime -> RelativeTime
forall a b. (a -> b) -> a -> b
$
Double -> NominalDiffTime
secondsToNominalDiffTime (Double -> NominalDiffTime) -> Double -> NominalDiffTime
forall a b. (a -> b) -> a -> b
$
Double
86400
Double -> Double -> Double
forall a. Num a => a -> a -> a
* Double
365
Double -> Double -> Double
forall a. Num a => a -> a -> a
* Double
1e9
}
oracularLedgerDB :: Point B -> Extended.ExtLedgerState B mk
oracularLedgerDB :: forall (mk :: MapKind). Point B -> ExtLedgerState B mk
oracularLedgerDB Point B
p =
Extended.ExtLedgerState {
headerState :: HeaderState B
Extended.headerState = HV.HeaderState {
headerStateTip :: WithOrigin (AnnTip B)
HV.headerStateTip = case Point B -> WithOrigin (RealPoint B)
forall blk. Point blk -> WithOrigin (RealPoint blk)
pointToWithOriginRealPoint Point B
p of
WithOrigin (RealPoint B)
Origin -> WithOrigin (AnnTip B)
forall t. WithOrigin t
Origin
NotOrigin RealPoint B
rp -> AnnTip B -> WithOrigin (AnnTip B)
forall t. t -> WithOrigin t
NotOrigin (AnnTip B -> WithOrigin (AnnTip B))
-> AnnTip B -> WithOrigin (AnnTip B)
forall a b. (a -> b) -> a -> b
$ HV.AnnTip {
annTipSlotNo :: SlotNo
HV.annTipSlotNo = RealPoint B -> SlotNo
forall blk. RealPoint blk -> SlotNo
realPointSlot RealPoint B
rp
, annTipInfo :: TipInfo B
HV.annTipInfo = RealPoint B -> HeaderHash B
forall blk. RealPoint blk -> HeaderHash blk
realPointHash RealPoint B
rp
, annTipBlockNo :: BlockNo
HV.annTipBlockNo =
TestHash -> BlockNo
testBlockHashBlockNo (RealPoint B -> HeaderHash B
forall blk. RealPoint blk -> HeaderHash blk
realPointHash RealPoint B
rp)
}
, headerStateChainDep :: ChainDepState (BlockProtocol B)
HV.headerStateChainDep = ()
}
, ledgerState :: LedgerState B mk
Extended.ledgerState = TB.TestLedger {
lastAppliedPoint :: Point B
TB.lastAppliedPoint = Point B
p
, payloadDependentState :: PayloadDependentState () mk
TB.payloadDependentState = PayloadDependentState () mk
forall (mk :: MapKind). PayloadDependentState () mk
TB.EmptyPLDS
}
}
testBlockHashBlockNo :: TB.TestHash -> BlockNo
testBlockHashBlockNo :: TestHash -> BlockNo
testBlockHashBlockNo (TB.TestHash NonEmpty Word64
ne) = Word64 -> BlockNo
BlockNo (Word64 -> BlockNo) -> Word64 -> BlockNo
forall a b. (a -> b) -> a -> b
$ Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Int -> Word64) -> Int -> Word64
forall a b. (a -> b) -> a -> b
$ NonEmpty Word64 -> Int
forall a. NonEmpty a -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
length NonEmpty Word64
ne
kInt :: Int
kInt :: Int
kInt = Int
5
securityParam :: SecurityParam
securityParam :: SecurityParam
securityParam = NonZero Word64 -> SecurityParam
SecurityParam (NonZero Word64 -> SecurityParam)
-> NonZero Word64 -> SecurityParam
forall a b. (a -> b) -> a -> b
$ forall (n :: Natural) a.
(KnownNat n, 1 <= n, WithinBounds n a, Num a) =>
NonZero a
knownNonZeroBounded @5
initialChain :: NE.NonEmpty B
initialChain :: NonEmpty B
initialChain =
[B] -> NonEmpty B
forall a. HasCallStack => [a] -> NonEmpty a
NE.fromList
([B] -> NonEmpty B) -> [B] -> NonEmpty B
forall a b. (a -> b) -> a -> b
$ Int -> [B] -> [B]
forall a. Int -> [a] -> [a]
take Int
kInt
([B] -> [B]) -> [B] -> [B]
forall a b. (a -> b) -> a -> b
$ (B -> B) -> B -> [B]
forall a. (a -> a) -> a -> [a]
iterate B -> B
TB.successorBlock
(B -> [B]) -> B -> [B]
forall a b. (a -> b) -> a -> b
$ Word64 -> B
TB.firstBlock Word64
0
slotLengthInSeconds :: Int
slotLengthInSeconds :: Int
slotLengthInSeconds = Int
1
slotLength :: SlotLength
slotLength :: SlotLength
slotLength = Integer -> SlotLength
slotLengthFromSec (Integer -> SlotLength) -> Integer -> SlotLength
forall a b. (a -> b) -> a -> b
$ Int -> Integer
forall a. Enum a => Int -> a
toEnum Int
slotLengthInSeconds
numCoreNodes :: NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes = Word64 -> NumCoreNodes
NumCoreNodes Word64
2
topConfig :: TopLevelConfig B
topConfig :: TopLevelConfig B
topConfig = TopLevelConfig {
topLevelConfigProtocol :: ConsensusConfig (BlockProtocol B)
topLevelConfigProtocol = BftConfig {
bftParams :: BftParams
bftParams = BftParams {
bftSecurityParam :: SecurityParam
bftSecurityParam = SecurityParam
securityParam
, bftNumNodes :: NumCoreNodes
bftNumNodes = NumCoreNodes
numCoreNodes
}
, bftSignKey :: SignKeyDSIGN (BftDSIGN BftMockCrypto)
bftSignKey = Word64 -> SignKeyDSIGN MockDSIGN
SignKeyMockDSIGN Word64
0
, bftVerKeys :: Map NodeId (VerKeyDSIGN (BftDSIGN BftMockCrypto))
bftVerKeys = [(NodeId, VerKeyDSIGN MockDSIGN)]
-> Map NodeId (VerKeyDSIGN MockDSIGN)
forall k a. Ord k => [(k, a)] -> Map k a
Map.fromList [
(CoreNodeId -> NodeId
CoreId (Word64 -> CoreNodeId
CoreNodeId Word64
0), Word64 -> VerKeyDSIGN MockDSIGN
VerKeyMockDSIGN Word64
0)
, (CoreNodeId -> NodeId
CoreId (Word64 -> CoreNodeId
CoreNodeId Word64
1), Word64 -> VerKeyDSIGN MockDSIGN
VerKeyMockDSIGN Word64
1)
]
}
, topLevelConfigLedger :: LedgerConfig B
topLevelConfigLedger = EraParams -> TestBlockLedgerConfig
TB.testBlockLedgerConfigFrom EraParams
eraParams
, topLevelConfigBlock :: BlockConfig B
topLevelConfigBlock = NumCoreNodes -> BlockConfig B
forall ptype. NumCoreNodes -> BlockConfig (TestBlockWith ptype)
TB.TestBlockConfig NumCoreNodes
numCoreNodes
, topLevelConfigCodec :: CodecConfig B
topLevelConfigCodec = CodecConfig B
TB.TestBlockCodecConfig
, topLevelConfigStorage :: StorageConfig B
topLevelConfigStorage = StorageConfig B
TB.TestBlockStorageConfig
, topLevelConfigCheckpoints :: CheckpointsMap B
topLevelConfigCheckpoints = CheckpointsMap B
forall blk. CheckpointsMap blk
emptyCheckpointsMap
}
where
eraParams :: HardFork.EraParams
eraParams :: EraParams
eraParams = SecurityParam -> SlotLength -> EraParams
HardFork.defaultEraParams SecurityParam
securityParam SlotLength
slotLength
data FollowerState =
Resting !(RealPoint B)
| Switching !(Point B) !(NE.NonEmpty B)
| Switched !(NE.NonEmpty B)
mkFollower ::
StrictTVar IO (Tip B)
-> IO (ChainDB.Follower IO B (ChainDB.WithPoint B H))
mkFollower :: StrictTVar IO (Tip B)
-> IO (Follower IO B (WithPoint B (Header B)))
mkFollower StrictTVar IO (Tip B)
varTip = do
varState <- FollowerState -> IO (IORef FollowerState)
forall a. a -> IO (IORef a)
newIORef (FollowerState -> IO (IORef FollowerState))
-> FollowerState -> IO (IORef FollowerState)
forall a b. (a -> b) -> a -> b
$ RealPoint B -> FollowerState
Resting (RealPoint B -> FollowerState) -> RealPoint B -> FollowerState
forall a b. (a -> b) -> a -> b
$ B -> RealPoint B
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint (B -> RealPoint B) -> B -> RealPoint B
forall a b. (a -> b) -> a -> b
$ NonEmpty B -> B
forall a. NonEmpty a -> a
NE.last NonEmpty B
initialChain
let wrap blk
blk = Header blk -> Point blk -> WithPoint blk (Header blk)
forall blk b. b -> Point blk -> WithPoint blk b
ChainDB.WithPoint (blk -> Header blk
forall blk. GetHeader blk => blk -> Header blk
getHeader blk
blk) (blk -> Point blk
forall block. HasHeader block => block -> Point block
blockPoint blk
blk)
let next = IORef FollowerState -> IO FollowerState
forall a. IORef a -> IO a
readIORef IORef FollowerState
varState IO FollowerState
-> (FollowerState -> IO (ChainUpdate B (WithPoint B (Header B))))
-> IO (ChainUpdate B (WithPoint B (Header B)))
forall a b. IO a -> (a -> IO b) -> IO b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
Switching Point B
ipoint NonEmpty B
blks -> do
IORef FollowerState -> FollowerState -> IO ()
forall a. IORef a -> a -> IO ()
writeIORef IORef FollowerState
varState (FollowerState -> IO ()) -> FollowerState -> IO ()
forall a b. (a -> b) -> a -> b
$ NonEmpty B -> FollowerState
Switched NonEmpty B
blks
STM IO () -> IO ()
forall a. HasCallStack => STM IO a -> IO a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM IO () -> IO ()) -> STM IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ StrictTVar IO (Tip B) -> Tip B -> STM IO ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar IO (Tip B)
varTip (Tip B -> STM IO ()) -> Tip B -> STM IO ()
forall a b. (a -> b) -> a -> b
$ B -> Tip B
forall a. HasHeader a => a -> Tip a
tipFromHeader (B -> Tip B) -> B -> Tip B
forall a b. (a -> b) -> a -> b
$ NonEmpty B -> B
forall a. NonEmpty a -> a
NE.last NonEmpty B
blks
ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B)))
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B))))
-> ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B)))
forall a b. (a -> b) -> a -> b
$ Point B -> ChainUpdate B (WithPoint B (Header B))
forall {k} (block :: k) a. Point block -> ChainUpdate block a
RollBack Point B
ipoint
Switched NonEmpty B
blks -> do
let blk :: B
blk = NonEmpty B -> B
forall a. NonEmpty a -> a
NE.head NonEmpty B
blks
IORef FollowerState -> FollowerState -> IO ()
forall a. IORef a -> a -> IO ()
writeIORef IORef FollowerState
varState (FollowerState -> IO ()) -> FollowerState -> IO ()
forall a b. (a -> b) -> a -> b
$ case [B] -> Maybe (NonEmpty B)
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty (NonEmpty B -> [B]
forall a. NonEmpty a -> [a]
NE.tail NonEmpty B
blks) of
Maybe (NonEmpty B)
Nothing -> RealPoint B -> FollowerState
Resting (RealPoint B -> FollowerState) -> RealPoint B -> FollowerState
forall a b. (a -> b) -> a -> b
$ B -> RealPoint B
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint B
blk
Just NonEmpty B
blks' -> NonEmpty B -> FollowerState
Switched NonEmpty B
blks'
STM IO () -> IO ()
forall a. HasCallStack => STM IO a -> IO a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM IO () -> IO ()) -> STM IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ StrictTVar IO (Tip B) -> Tip B -> STM IO ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar IO (Tip B)
varTip (Tip B -> STM IO ()) -> Tip B -> STM IO ()
forall a b. (a -> b) -> a -> b
$ B -> Tip B
forall a. HasHeader a => a -> Tip a
tipFromHeader (B -> Tip B) -> B -> Tip B
forall a b. (a -> b) -> a -> b
$ NonEmpty B -> B
forall a. NonEmpty a -> a
NE.last NonEmpty B
blks
ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B)))
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B))))
-> ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B)))
forall a b. (a -> b) -> a -> b
$ WithPoint B (Header B) -> ChainUpdate B (WithPoint B (Header B))
forall {k} (block :: k) a. a -> ChainUpdate block a
AddBlock (WithPoint B (Header B) -> ChainUpdate B (WithPoint B (Header B)))
-> WithPoint B (Header B) -> ChainUpdate B (WithPoint B (Header B))
forall a b. (a -> b) -> a -> b
$ B -> WithPoint B (Header B)
forall {blk}.
(GetHeader blk, HasHeader blk) =>
blk -> WithPoint blk (Header blk)
wrap B
blk
Resting RealPoint B
rp -> do
let (Point B
ipoint, NonEmpty B
blks) = RealPoint B -> (Point B, NonEmpty B)
TB.updateToNextNumeral RealPoint B
rp
IORef FollowerState -> FollowerState -> IO ()
forall a. IORef a -> a -> IO ()
writeIORef IORef FollowerState
varState (FollowerState -> IO ()) -> FollowerState -> IO ()
forall a b. (a -> b) -> a -> b
$ NonEmpty B -> FollowerState
Switched NonEmpty B
blks
STM IO () -> IO ()
forall a. HasCallStack => STM IO a -> IO a
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
STM m a -> m a
atomically (STM IO () -> IO ()) -> STM IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ StrictTVar IO (Tip B) -> Tip B -> STM IO ()
forall (m :: * -> *) a.
(MonadSTM m, HasCallStack) =>
StrictTVar m a -> a -> STM m ()
writeTVar StrictTVar IO (Tip B)
varTip (Tip B -> STM IO ()) -> Tip B -> STM IO ()
forall a b. (a -> b) -> a -> b
$ B -> Tip B
forall a. HasHeader a => a -> Tip a
tipFromHeader (B -> Tip B) -> B -> Tip B
forall a b. (a -> b) -> a -> b
$ NonEmpty B -> B
forall a. NonEmpty a -> a
NE.last NonEmpty B
blks
ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B)))
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B))))
-> ChainUpdate B (WithPoint B (Header B))
-> IO (ChainUpdate B (WithPoint B (Header B)))
forall a b. (a -> b) -> a -> b
$ Point B -> ChainUpdate B (WithPoint B (Header B))
forall {k} (block :: k) a. Point block -> ChainUpdate block a
RollBack Point B
ipoint
pure ChainDB.Follower {
ChainDB.followerClose = pure ()
, ChainDB.followerInstruction = Just <$> next
, ChainDB.followerInstructionBlocking = next
, ChainDB.followerForward = \case
Point B
GenesisPoint : [Point B]
_ -> do
IORef FollowerState -> FollowerState -> IO ()
forall a. IORef a -> a -> IO ()
writeIORef IORef FollowerState
varState (FollowerState -> IO ()) -> FollowerState -> IO ()
forall a b. (a -> b) -> a -> b
$ Point B -> NonEmpty B -> FollowerState
Switching Point B
forall {k} (block :: k). Point block
GenesisPoint NonEmpty B
initialChain
Maybe (Point B) -> IO (Maybe (Point B))
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (Point B) -> IO (Maybe (Point B)))
-> Maybe (Point B) -> IO (Maybe (Point B))
forall a b. (a -> b) -> a -> b
$ Point B -> Maybe (Point B)
forall a. a -> Maybe a
Just Point B
forall {k} (block :: k). Point block
GenesisPoint
[Point B]
ps -> [Char] -> IO (Maybe (Point B))
forall a. HasCallStack => [Char] -> a
error ([Char] -> IO (Maybe (Point B))) -> [Char] -> IO (Maybe (Point B))
forall a b. (a -> b) -> a -> b
$ [Char]
"impossible! " [Char] -> [Char] -> [Char]
forall a. Semigroup a => a -> a -> a
<> [[Char]] -> [Char]
unlines ((Point B -> [Char]) -> [Point B] -> [[Char]]
forall a b. (a -> b) -> [a] -> [b]
map Point B -> [Char]
forall a. Show a => a -> [Char]
show [Point B]
ps)
}