{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# OPTIONS_GHC -Wno-orphans #-}
module Test.Consensus.MiniProtocol.LocalStateQuery.Server (tests) where
import Cardano.Crypto.DSIGN.Mock
import Cardano.Ledger.BaseTypes (nonZero, unNonZero)
import Control.Concurrent.Class.MonadSTM.Strict.TMVar
import Control.Monad.IOSim (runSimOrThrow)
import Control.ResourceRegistry
import Control.Tracer
import qualified Data.Map.Strict as Map
import Data.Maybe (fromMaybe)
import Network.TypedProtocol.Stateful.Proofs (connect)
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.BlockchainTime
import Ouroboros.Consensus.Config
import qualified Ouroboros.Consensus.HardFork.History as HardFork
import Ouroboros.Consensus.Ledger.Basics
import Ouroboros.Consensus.Ledger.Extended
import Ouroboros.Consensus.Ledger.Query (Query (..))
import Ouroboros.Consensus.MiniProtocol.LocalStateQuery.Server
import Ouroboros.Consensus.Node.ProtocolInfo (NumCoreNodes (..))
import Ouroboros.Consensus.NodeId
import Ouroboros.Consensus.Protocol.BFT
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.BlockCache as BlockCache
import Ouroboros.Consensus.Storage.ImmutableDB.Stream hiding
(streamAPI)
import Ouroboros.Consensus.Storage.LedgerDB
import qualified Ouroboros.Consensus.Storage.LedgerDB as LedgerDB
import Ouroboros.Consensus.Storage.LedgerDB.Snapshots
import Ouroboros.Consensus.Storage.LedgerDB.V1.Args
import Ouroboros.Consensus.Util.IOLike hiding (newTVarIO)
import Ouroboros.Network.Mock.Chain (Chain (..))
import qualified Ouroboros.Network.Mock.Chain as Chain
import Ouroboros.Network.Protocol.LocalStateQuery.Client
import Ouroboros.Network.Protocol.LocalStateQuery.Examples
(localStateQueryClient)
import Ouroboros.Network.Protocol.LocalStateQuery.Server
import Ouroboros.Network.Protocol.LocalStateQuery.Type
(AcquireFailure (..), State (..), Target (..))
import System.FS.API (SomeHasFS (..))
import qualified System.FS.Sim.MockFS as MockFS
import System.FS.Sim.STM
import Test.Cardano.Ledger.Core.Arbitrary ()
import Test.QuickCheck hiding (Result)
import Test.Tasty
import Test.Tasty.QuickCheck
import Test.Util.Orphans.IOLike ()
import Test.Util.TestBlock
tests :: TestTree
tests :: TestTree
tests = String -> [TestTree] -> TestTree
testGroup String
"LocalStateQueryServer"
[ String
-> (SecurityParam
-> BlockTree -> Permutation -> Positive (Small Int) -> Property)
-> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"localStateQueryServer" SecurityParam
-> BlockTree -> Permutation -> Positive (Small Int) -> Property
prop_localStateQueryServer
]
prop_localStateQueryServer
:: SecurityParam
-> BlockTree
-> Permutation
-> Positive (Small Int)
-> Property
prop_localStateQueryServer :: SecurityParam
-> BlockTree -> Permutation -> Positive (Small Int) -> Property
prop_localStateQueryServer SecurityParam
k BlockTree
bt Permutation
p (Positive (Small Int
n)) = SecurityParam
-> Chain TestBlock
-> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
-> Property
checkOutcome SecurityParam
k Chain TestBlock
chain [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
actualOutcome
where
chain :: Chain TestBlock
chain :: Chain TestBlock
chain = BlockTree -> Chain TestBlock
treePreferredChain BlockTree
bt
points :: [Target (Point TestBlock)]
points :: [Target (Point TestBlock)]
points = Permutation
-> [Target (Point TestBlock)] -> [Target (Point TestBlock)]
forall a. Permutation -> [a] -> [a]
permute Permutation
p ([Target (Point TestBlock)] -> [Target (Point TestBlock)])
-> [Target (Point TestBlock)] -> [Target (Point TestBlock)]
forall a b. (a -> b) -> a -> b
$
Int -> Target (Point TestBlock) -> [Target (Point TestBlock)]
forall a. Int -> a -> [a]
replicate Int
n Target (Point TestBlock)
forall point. Target point
VolatileTip
[Target (Point TestBlock)]
-> [Target (Point TestBlock)] -> [Target (Point TestBlock)]
forall a. [a] -> [a] -> [a]
++ (Point TestBlock -> Target (Point TestBlock)
forall point. point -> Target point
SpecificPoint (Point TestBlock -> Target (Point TestBlock))
-> (TestBlock -> Point TestBlock)
-> TestBlock
-> Target (Point TestBlock)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TestBlock -> Point TestBlock
forall block. HasHeader block => block -> Point block
blockPoint (TestBlock -> Target (Point TestBlock))
-> [TestBlock] -> [Target (Point TestBlock)]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> BlockTree -> [TestBlock]
treeToBlocks BlockTree
bt)
actualOutcome :: [(Target (Point TestBlock), Either AcquireFailure (Point TestBlock))]
actualOutcome :: [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
actualOutcome = (forall s.
IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
forall a. (forall s. IOSim s a) -> a
runSimOrThrow ((forall s.
IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> (forall s.
IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
forall a b. (a -> b) -> a -> b
$ (ResourceRegistry (IOSim s)
-> IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
forall (m :: * -> *) a.
(MonadSTM m, MonadMask m, MonadThread m, HasCallStack) =>
(ResourceRegistry m -> m a) -> m a
withRegistry ((ResourceRegistry (IOSim s)
-> IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> (ResourceRegistry (IOSim s)
-> IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))])
-> IOSim
s
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
forall a b. (a -> b) -> a -> b
$ \ResourceRegistry (IOSim s)
rr ->do
let client :: LocalStateQueryClient
TestBlock
(Point TestBlock)
(Query TestBlock)
(IOSim s)
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
client = [Target (Point TestBlock)]
-> LocalStateQueryClient
TestBlock
(Point TestBlock)
(Query TestBlock)
(IOSim s)
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
forall (m :: * -> *).
Monad m =>
[Target (Point TestBlock)]
-> LocalStateQueryClient
TestBlock
(Point TestBlock)
(Query TestBlock)
m
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
mkClient [Target (Point TestBlock)]
points
server <- ResourceRegistry (IOSim s)
-> SecurityParam
-> Chain TestBlock
-> IOSim
s
(LocalStateQueryServer
TestBlock (Point TestBlock) (Query TestBlock) (IOSim s) ())
forall (m :: * -> *).
IOLike m =>
ResourceRegistry m
-> SecurityParam
-> Chain TestBlock
-> m (LocalStateQueryServer
TestBlock (Point TestBlock) (Query TestBlock) m ())
mkServer ResourceRegistry (IOSim s)
rr SecurityParam
k Chain TestBlock
chain
(\([(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
a, ()
_, TerminalStates
(LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock))
_) -> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
a) <$>
connect
StateIdle
(localStateQueryClientPeer client)
(localStateQueryServerPeer server)
checkOutcome ::
SecurityParam
-> Chain TestBlock
-> [(Target (Point TestBlock), Either AcquireFailure (Point TestBlock))]
-> Property
checkOutcome :: SecurityParam
-> Chain TestBlock
-> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
-> Property
checkOutcome SecurityParam
k Chain TestBlock
chain = [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin ([Property] -> Property)
-> ([(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
-> [Property])
-> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
-> Property
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))
-> Property)
-> [(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
-> [Property]
forall a b. (a -> b) -> [a] -> [b]
map ((Target (Point TestBlock)
-> Either AcquireFailure (Point TestBlock) -> Property)
-> (Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))
-> Property
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry Target (Point TestBlock)
-> Either AcquireFailure (Point TestBlock) -> Property
checkResult)
where
immutableSlot :: WithOrigin SlotNo
immutableSlot :: WithOrigin SlotNo
immutableSlot = Chain TestBlock -> WithOrigin SlotNo
forall block. HasHeader block => Chain block -> WithOrigin SlotNo
Chain.headSlot (Chain TestBlock -> WithOrigin SlotNo)
-> Chain TestBlock -> WithOrigin SlotNo
forall a b. (a -> b) -> a -> b
$
Int -> Chain TestBlock -> Chain TestBlock
forall block. Int -> Chain block -> Chain block
Chain.drop (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 (SecurityParam -> NonZero Word64
maxRollbacks SecurityParam
k)) Chain TestBlock
chain
checkResult
:: Target (Point TestBlock)
-> Either AcquireFailure (Point TestBlock)
-> Property
checkResult :: Target (Point TestBlock)
-> Either AcquireFailure (Point TestBlock) -> Property
checkResult (SpecificPoint Point TestBlock
pt) = \case
Right Point TestBlock
result
-> String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"Acquired" [String
"Success"] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ Point TestBlock
result Point TestBlock -> Point TestBlock -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== Point TestBlock
pt
Left AcquireFailure
AcquireFailurePointNotOnChain
| Point TestBlock -> Chain TestBlock -> Bool
forall block. HasHeader block => Point block -> Chain block -> Bool
Chain.pointOnChain Point TestBlock
pt Chain TestBlock
chain
-> String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
(String
"Point " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> Point TestBlock -> String
forall a. Show a => a -> String
show Point TestBlock
pt String -> String -> String
forall a. Semigroup a => a -> a -> a
<>
String
" on chain, but got AcquireFailurePointNotOnChain")
(Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
False)
| Bool
otherwise
-> String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"Acquired" [String
"AcquireFailurePointNotOnChain"] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True
Left AcquireFailure
AcquireFailurePointTooOld
| Point TestBlock -> WithOrigin SlotNo
forall {k} (block :: k). Point block -> WithOrigin SlotNo
pointSlot Point TestBlock
pt WithOrigin SlotNo -> WithOrigin SlotNo -> Bool
forall a. Ord a => a -> a -> Bool
>= WithOrigin SlotNo
immutableSlot
-> String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample
(String
"Point " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> Point TestBlock -> String
forall a. Show a => a -> String
show Point TestBlock
pt String -> String -> String
forall a. Semigroup a => a -> a -> a
<>
String
" newer or equal than the immutable tip " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> WithOrigin SlotNo -> String
forall a. Show a => a -> String
show WithOrigin SlotNo
immutableSlot String -> String -> String
forall a. Semigroup a => a -> a -> a
<>String
", but got AcquireFailurePointTooOld")
(Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
False)
| Bool
otherwise
-> String -> [String] -> Property -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"Acquired" [String
"AcquireFailurePointTooOld"] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$ Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True
checkResult Target (Point TestBlock)
VolatileTip = \case
Right Point TestBlock
_result -> String -> [String] -> Bool -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"Acquired" [String
"Success"] Bool
True
Left AcquireFailure
failure -> String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"acquire tip point resulted in " String -> String -> String
forall a. [a] -> [a] -> [a]
++ AcquireFailure -> String
forall a. Show a => a -> String
show AcquireFailure
failure) Bool
False
checkResult Target (Point TestBlock)
ImmutableTip = \case
Right Point TestBlock
_result -> String -> [String] -> Bool -> Property
forall prop.
Testable prop =>
String -> [String] -> prop -> Property
tabulate String
"Acquired" [String
"Success"] Bool
True
Left AcquireFailure
failure -> String -> Bool -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"acquire tip point resulted in " String -> String -> String
forall a. [a] -> [a] -> [a]
++ AcquireFailure -> String
forall a. Show a => a -> String
show AcquireFailure
failure) Bool
False
mkClient ::
Monad m
=> [Target (Point TestBlock)]
-> LocalStateQueryClient
TestBlock
(Point TestBlock)
(Query TestBlock)
m
[(Target (Point TestBlock), Either AcquireFailure (Point TestBlock))]
mkClient :: forall (m :: * -> *).
Monad m =>
[Target (Point TestBlock)]
-> LocalStateQueryClient
TestBlock
(Point TestBlock)
(Query TestBlock)
m
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
mkClient [Target (Point TestBlock)]
points = [(Target (Point TestBlock), Query TestBlock (Point TestBlock))]
-> LocalStateQueryClient
TestBlock
(Point TestBlock)
(Query TestBlock)
m
[(Target (Point TestBlock),
Either AcquireFailure (Point TestBlock))]
forall block point (query :: * -> *) result (m :: * -> *).
Applicative m =>
[(Target point, query result)]
-> LocalStateQueryClient
block point query m [(Target point, Either AcquireFailure result)]
localStateQueryClient [(Target (Point TestBlock)
pt, BlockQuery TestBlock 'QFNoTables (Point TestBlock)
-> Query TestBlock (Point TestBlock)
forall (footprint :: QueryFootprint) blk result.
SingI footprint =>
BlockQuery blk footprint result -> Query blk result
BlockQuery BlockQuery TestBlock 'QFNoTables (Point TestBlock)
QueryLedgerTip) | Target (Point TestBlock)
pt <- [Target (Point TestBlock)]
points]
mkServer ::
IOLike m
=> ResourceRegistry m
-> SecurityParam
-> Chain TestBlock
-> m (LocalStateQueryServer TestBlock (Point TestBlock) (Query TestBlock) m ())
mkServer :: forall (m :: * -> *).
IOLike m =>
ResourceRegistry m
-> SecurityParam
-> Chain TestBlock
-> m (LocalStateQueryServer
TestBlock (Point TestBlock) (Query TestBlock) m ())
mkServer ResourceRegistry m
rr SecurityParam
k Chain TestBlock
chain = do
lgrDB <- SecurityParam -> Chain TestBlock -> m (LedgerDB' m TestBlock)
forall (m :: * -> *).
IOLike m =>
SecurityParam -> Chain TestBlock -> m (LedgerDB' m TestBlock)
initLedgerDB SecurityParam
k Chain TestBlock
chain
return $
localStateQueryServer
cfg
(LedgerDB.getReadOnlyForker lgrDB rr)
where
cfg :: ExtLedgerCfg TestBlock
cfg = TopLevelConfig TestBlock -> ExtLedgerCfg TestBlock
forall blk. TopLevelConfig blk -> ExtLedgerCfg blk
ExtLedgerCfg (TopLevelConfig TestBlock -> ExtLedgerCfg TestBlock)
-> TopLevelConfig TestBlock -> ExtLedgerCfg TestBlock
forall a b. (a -> b) -> a -> b
$ SecurityParam -> TopLevelConfig TestBlock
testCfg SecurityParam
k
streamAPI :: forall m. IOLike m => StreamAPI m TestBlock TestBlock
streamAPI :: forall (m :: * -> *). IOLike m => StreamAPI m TestBlock TestBlock
streamAPI = StreamAPI {Point TestBlock
-> (Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m b)
-> m b
forall b.
HasCallStack =>
Point TestBlock
-> (Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m b)
-> m b
forall a.
Point TestBlock
-> (Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m a)
-> m a
streamAfter :: forall a.
Point TestBlock
-> (Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m a)
-> m a
streamAfter :: forall b.
HasCallStack =>
Point TestBlock
-> (Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m b)
-> m b
streamAfter}
where
streamAfter ::
Point TestBlock
-> (Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m a)
-> m a
streamAfter :: forall a.
Point TestBlock
-> (Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m a)
-> m a
streamAfter Point TestBlock
_ Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m a
k = do
Either (RealPoint TestBlock) (m (NextItem TestBlock)) -> m a
k (m (NextItem TestBlock)
-> Either (RealPoint TestBlock) (m (NextItem TestBlock))
forall a b. b -> Either a b
Right (NextItem TestBlock -> m (NextItem TestBlock)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure NextItem TestBlock
forall blk. NextItem blk
NoMoreItems))
initLedgerDB ::
IOLike m
=> SecurityParam
-> Chain TestBlock
-> m (LedgerDB' m TestBlock)
initLedgerDB :: forall (m :: * -> *).
IOLike m =>
SecurityParam -> Chain TestBlock -> m (LedgerDB' m TestBlock)
initLedgerDB SecurityParam
s Chain TestBlock
c = do
reg <- m (ResourceRegistry m)
forall (m :: * -> *).
(MonadSTM m, MonadThread m, HasCallStack) =>
m (ResourceRegistry m)
unsafeNewRegistry
fs <- newTMVarIO MockFS.empty
let args = LedgerDbArgs
{ lgrSnapshotPolicyArgs :: SnapshotPolicyArgs
lgrSnapshotPolicyArgs = SnapshotPolicyArgs
defaultSnapshotPolicyArgs
, lgrHasFS :: HKD Identity (SomeHasFS m)
lgrHasFS = HasFS m HandleMock -> SomeHasFS m
forall h (m :: * -> *). Eq h => HasFS m h -> SomeHasFS m
SomeHasFS (HasFS m HandleMock -> SomeHasFS m)
-> HasFS m HandleMock -> SomeHasFS m
forall a b. (a -> b) -> a -> b
$ StrictTMVar m MockFS -> HasFS m HandleMock
forall (m :: * -> *).
(MonadSTM m, MonadThrow m, PrimMonad m) =>
StrictTMVar m MockFS -> HasFS m HandleMock
simHasFS StrictTMVar m MockFS
fs
, lgrGenesis :: HKD Identity (m (ExtLedgerState TestBlock ValuesMK))
lgrGenesis = ExtLedgerState TestBlock ValuesMK
-> m (ExtLedgerState TestBlock ValuesMK)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ExtLedgerState TestBlock ValuesMK
testInitExtLedger
, lgrTracer :: Tracer m (TraceEvent TestBlock)
lgrTracer = Tracer m (TraceEvent TestBlock)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
, lgrFlavorArgs :: LedgerDbFlavorArgs Identity m
lgrFlavorArgs = LedgerDbFlavorArgs Identity m -> LedgerDbFlavorArgs Identity m
forall (f :: * -> *) (m :: * -> *).
LedgerDbFlavorArgs f m -> LedgerDbFlavorArgs f m
LedgerDbFlavorArgsV1 (LedgerDbFlavorArgs Identity m -> LedgerDbFlavorArgs Identity m)
-> LedgerDbFlavorArgs Identity m -> LedgerDbFlavorArgs Identity m
forall a b. (a -> b) -> a -> b
$ FlushFrequency
-> BackingStoreArgs Identity m -> LedgerDbFlavorArgs Identity m
forall (f :: * -> *) (m :: * -> *).
FlushFrequency -> BackingStoreArgs f m -> LedgerDbFlavorArgs f m
V1Args FlushFrequency
DefaultFlushFrequency BackingStoreArgs Identity m
forall (f :: * -> *) (m :: * -> *). BackingStoreArgs f m
InMemoryBackingStoreArgs
, lgrConfig :: LedgerDbCfgF Identity (ExtLedgerState TestBlock)
lgrConfig = TopLevelConfig TestBlock
-> ComputeLedgerEvents
-> LedgerDbCfgF Identity (ExtLedgerState TestBlock)
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk
-> ComputeLedgerEvents -> LedgerDbCfg (ExtLedgerState blk)
LedgerDB.configLedgerDb (SecurityParam -> TopLevelConfig TestBlock
testCfg SecurityParam
s) ComputeLedgerEvents
OmitLedgerEvents
, lgrQueryBatchSize :: QueryBatchSize
lgrQueryBatchSize = QueryBatchSize
DefaultQueryBatchSize
, lgrRegistry :: HKD Identity (ResourceRegistry m)
lgrRegistry = HKD Identity (ResourceRegistry m)
ResourceRegistry m
reg
, lgrStartSnapshot :: Maybe DiskSnapshot
lgrStartSnapshot = Maybe DiskSnapshot
forall a. Maybe a
Nothing
}
ldb <- fst <$> LedgerDB.openDB
args
streamAPI
(Chain.headPoint c)
(\RealPoint TestBlock
rpt -> TestBlock -> m TestBlock
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (TestBlock -> m TestBlock) -> TestBlock -> m TestBlock
forall a b. (a -> b) -> a -> b
$ TestBlock -> Maybe TestBlock -> TestBlock
forall a. a -> Maybe a -> a
fromMaybe (String -> TestBlock
forall a. HasCallStack => String -> a
error String
"impossible") (Maybe TestBlock -> TestBlock) -> Maybe TestBlock -> TestBlock
forall a b. (a -> b) -> a -> b
$ (TestBlock -> Bool) -> Chain TestBlock -> Maybe TestBlock
forall block. (block -> Bool) -> Chain block -> Maybe block
Chain.findBlock ((RealPoint TestBlock
rpt RealPoint TestBlock -> RealPoint TestBlock -> Bool
forall a. Eq a => a -> a -> Bool
==) (RealPoint TestBlock -> Bool)
-> (TestBlock -> RealPoint TestBlock) -> TestBlock -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TestBlock -> RealPoint TestBlock
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint) Chain TestBlock
c)
result <- LedgerDB.validateFork ldb reg (const $ pure ()) BlockCache.empty 0 (map getHeader $ Chain.toOldestFirst c)
case result of
LedgerDB.ValidateSuccessful Forker m (ExtLedgerState TestBlock) TestBlock
forker -> 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
$ Forker m (ExtLedgerState TestBlock) TestBlock -> STM m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> STM m ()
LedgerDB.forkerCommit Forker m (ExtLedgerState TestBlock) TestBlock
forker
Forker m (ExtLedgerState TestBlock) TestBlock -> m ()
forall (m :: * -> *) (l :: LedgerStateKind) blk.
Forker m l blk -> m ()
LedgerDB.forkerClose Forker m (ExtLedgerState TestBlock) TestBlock
forker
LedgerDB.ValidateExceededRollBack ExceededRollback
_ ->
String -> m ()
forall a. HasCallStack => String -> a
error String
"impossible: rollback was 0"
LedgerDB.ValidateLedgerError AnnLedgerError m (ExtLedgerState TestBlock) TestBlock
_ ->
String -> m ()
forall a. HasCallStack => String -> a
error String
"impossible: there were no invalid blocks"
pure ldb
testCfg :: SecurityParam -> TopLevelConfig TestBlock
testCfg :: SecurityParam -> TopLevelConfig TestBlock
testCfg SecurityParam
securityParam = TopLevelConfig {
topLevelConfigProtocol :: ConsensusConfig (BlockProtocol TestBlock)
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. k -> a -> Map k a
Map.singleton (CoreNodeId -> NodeId
CoreId (Word64 -> CoreNodeId
CoreNodeId Word64
0)) (Word64 -> VerKeyDSIGN MockDSIGN
VerKeyMockDSIGN Word64
0)
}
, topLevelConfigLedger :: LedgerConfig TestBlock
topLevelConfigLedger = EraParams -> TestBlockLedgerConfig
testBlockLedgerConfigFrom EraParams
eraParams
, topLevelConfigBlock :: BlockConfig TestBlock
topLevelConfigBlock = NumCoreNodes -> BlockConfig TestBlock
forall ptype. NumCoreNodes -> BlockConfig (TestBlockWith ptype)
TestBlockConfig NumCoreNodes
numCoreNodes
, topLevelConfigCodec :: CodecConfig TestBlock
topLevelConfigCodec = CodecConfig TestBlock
TestBlockCodecConfig
, topLevelConfigStorage :: StorageConfig TestBlock
topLevelConfigStorage = StorageConfig TestBlock
TestBlockStorageConfig
, topLevelConfigCheckpoints :: CheckpointsMap TestBlock
topLevelConfigCheckpoints = CheckpointsMap TestBlock
forall blk. CheckpointsMap blk
emptyCheckpointsMap
}
where
slotLength :: SlotLength
slotLength :: SlotLength
slotLength = Integer -> SlotLength
slotLengthFromSec Integer
20
numCoreNodes :: NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes = Word64 -> NumCoreNodes
NumCoreNodes Word64
1
eraParams :: HardFork.EraParams
eraParams :: EraParams
eraParams = SecurityParam -> SlotLength -> EraParams
HardFork.defaultEraParams SecurityParam
securityParam SlotLength
slotLength
instance Arbitrary SecurityParam where
arbitrary :: Gen SecurityParam
arbitrary = NonZero Word64 -> SecurityParam
SecurityParam (NonZero Word64 -> SecurityParam)
-> Gen (NonZero Word64) -> Gen SecurityParam
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (Word64, Word64) -> Gen Word64
forall a. Random a => (a, a) -> Gen a
choose (Word64
1, Word64
100) Gen Word64
-> (Word64 -> Maybe (NonZero Word64)) -> Gen (NonZero Word64)
forall a b. Gen a -> (a -> Maybe b) -> Gen b
`suchThatMap` Word64 -> Maybe (NonZero Word64)
forall a. HasZero a => a -> Maybe (NonZero a)
nonZero
shrink :: SecurityParam -> [SecurityParam]
shrink (SecurityParam NonZero Word64
k) = [NonZero Word64 -> SecurityParam
SecurityParam NonZero Word64
k' | NonZero Word64
k' <- NonZero Word64 -> [NonZero Word64]
forall a. Arbitrary a => a -> [a]
shrink NonZero Word64
k, NonZero Word64 -> Word64
forall a. NonZero a -> a
unNonZero NonZero Word64
k' Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
> Word64
0]