{-# LANGUAGE GADTs #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE ScopedTypeVariables #-}

{-# OPTIONS_GHC -Wno-orphans #-}

-- | Tests for the local state query server.
--
-- The local state query protocol allows clients such as wallets to query the
-- state of the ledger at any point within @k@ blocks from the tip. The test for
-- this is quite minimal at present: it prepopulates a ledger DB with a bunch of
-- blocks, and then verifies that requesting the ledger tip corresponding to the
-- these blocks gives the right answers, and that asking for blocks not on the
-- chain results in the right error message.
--
-- Note that the query protocol is abstract in the ledger, and the query
-- /language/ we offer (the kinds of queries that can be asked) of course
-- depends on the ledger. The tests use a mock ledger for this purpose.
--
module Test.Consensus.MiniProtocol.LocalStateQuery.Server (tests) where

import           Cardano.Crypto.DSIGN.Mock
import           Control.Monad.IOSim (runSimOrThrow)
import           Control.Tracer (nullTracer)
import           Data.Map.Strict (Map)
import qualified Data.Map.Strict as Map
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.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.ChainDB.Impl.LgrDB (LgrDB,
                     LgrDbArgs (..), mkLgrDB)
import qualified Ouroboros.Consensus.Storage.ChainDB.Impl.LgrDB as LgrDB
import           Ouroboros.Consensus.Storage.LedgerDB (configLedgerDb,
                     defaultDiskPolicyArgs)
import qualified Ouroboros.Consensus.Storage.LedgerDB as LgrDB (ledgerDbPast,
                     ledgerDbTip, ledgerDbWithAnchor)
import           Ouroboros.Consensus.Util.IOLike
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 (HasFS, SomeHasFS (..))
import           Test.QuickCheck hiding (Result)
import           Test.Tasty
import           Test.Tasty.QuickCheck
import           Test.Util.Orphans.IOLike ()
import           Test.Util.TestBlock

{-------------------------------------------------------------------------------
  Top-level tests
-------------------------------------------------------------------------------}

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
    ]

{-------------------------------------------------------------------------------
  Main property
-------------------------------------------------------------------------------}

-- | Plan:
-- * Preseed the LgrDB of the server with the preferred chain of the
--  'BlockTree'.
-- * Acquire for each block in the 'BlockTree', including the ones not on the
--   chain, a state and send the 'QueryLedgerTip'. Collect these results.
-- * Check that when acquiring failed, it rightfully failed. Otherwise, check
--   whether the returned tip matches the block.
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
$ 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
      LocalStateQueryServer
  TestBlock (Point TestBlock) (Query TestBlock) (IOSim s) ()
server <- SecurityParam
-> Chain TestBlock
-> IOSim
     s
     (LocalStateQueryServer
        TestBlock (Point TestBlock) (Query TestBlock) (IOSim s) ())
forall (m :: * -> *).
IOLike m =>
SecurityParam
-> Chain TestBlock
-> m (LocalStateQueryServer
        TestBlock (Point TestBlock) (Query TestBlock) m ())
mkServer 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) (([(Target (Point TestBlock),
    Either AcquireFailure (Point TestBlock))],
  (),
  TerminalStates
    (LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock)))
 -> [(Target (Point TestBlock),
      Either AcquireFailure (Point TestBlock))])
-> IOSim
     s
     ([(Target (Point TestBlock),
        Either AcquireFailure (Point TestBlock))],
      (),
      TerminalStates
        (LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock)))
-> IOSim
     s
     [(Target (Point TestBlock),
       Either AcquireFailure (Point TestBlock))]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
        State 'StIdle
-> Peer
     (LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock))
     'AsClient
     'StIdle
     State
     (IOSim s)
     [(Target (Point TestBlock),
       Either AcquireFailure (Point TestBlock))]
-> Peer
     (LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock))
     (FlipAgency 'AsClient)
     'StIdle
     State
     (IOSim s)
     ()
-> IOSim
     s
     ([(Target (Point TestBlock),
        Either AcquireFailure (Point TestBlock))],
      (),
      TerminalStates
        (LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock)))
forall ps (pr :: PeerRole) (st :: ps) (f :: ps -> *) (m :: * -> *)
       a b.
(MonadSTM m, SingI pr) =>
f st
-> Peer ps pr st f m a
-> Peer ps (FlipAgency pr) st f m b
-> m (a, b, TerminalStates ps)
connect
          State 'StIdle
forall {block} {point} {query :: * -> *}. State 'StIdle
StateIdle
          (LocalStateQueryClient
  TestBlock
  (Point TestBlock)
  (Query TestBlock)
  (IOSim s)
  [(Target (Point TestBlock),
    Either AcquireFailure (Point TestBlock))]
-> Peer
     (LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock))
     'AsClient
     'StIdle
     State
     (IOSim s)
     [(Target (Point TestBlock),
       Either AcquireFailure (Point TestBlock))]
forall block point (query :: * -> *) (m :: * -> *) a.
Monad m =>
LocalStateQueryClient block point query m a
-> Client (LocalStateQuery block point query) 'StIdle State m a
localStateQueryClientPeer LocalStateQueryClient
  TestBlock
  (Point TestBlock)
  (Query TestBlock)
  (IOSim s)
  [(Target (Point TestBlock),
    Either AcquireFailure (Point TestBlock))]
client)
          (LocalStateQueryServer
  TestBlock (Point TestBlock) (Query TestBlock) (IOSim s) ()
-> Server
     (LocalStateQuery TestBlock (Point TestBlock) (Query TestBlock))
     'StIdle
     State
     (IOSim s)
     ()
forall block point (query :: * -> *) (m :: * -> *) a.
Monad m =>
LocalStateQueryServer block point query m a
-> Server (LocalStateQuery block point query) 'StIdle State m a
localStateQueryServerPeer LocalStateQueryServer
  TestBlock (Point TestBlock) (Query TestBlock) (IOSim s) ()
server)

{-------------------------------------------------------------------------------
  Test setup
-------------------------------------------------------------------------------}

-- | Checks whether the given outcome is correct: in case of an
-- 'AcquireFailure', we check whether it was warranted. Otherwise we check
-- whether the results are correct.
--
-- NOTE: when we don't get an 'AcquireFailure', even though we expected it, we
-- accept it. This is because the LgrDB may contain snapshots for blocks on
-- the current chain older than @k@, but we do not want to imitate such
-- implementation details.
--
-- Additionally, this function labels the test results.
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 (SecurityParam -> 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 than the immutable tip, 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 (Point TestBlock)
-> Query TestBlock (Point TestBlock)
forall blk result. BlockQuery blk result -> Query blk result
BlockQuery BlockQuery TestBlock (Point TestBlock)
QueryLedgerTip) | Target (Point TestBlock)
pt <- [Target (Point TestBlock)]
points]

mkServer ::
     IOLike m
  => SecurityParam
  -> Chain TestBlock
  -> m (LocalStateQueryServer TestBlock (Point TestBlock) (Query TestBlock) m ())
mkServer :: forall (m :: * -> *).
IOLike m =>
SecurityParam
-> Chain TestBlock
-> m (LocalStateQueryServer
        TestBlock (Point TestBlock) (Query TestBlock) m ())
mkServer SecurityParam
k Chain TestBlock
chain = do
    LgrDB m TestBlock
lgrDB <- SecurityParam -> Chain TestBlock -> m (LgrDB m TestBlock)
forall (m :: * -> *).
IOLike m =>
SecurityParam -> Chain TestBlock -> m (LgrDB m TestBlock)
initLgrDB SecurityParam
k Chain TestBlock
chain
    LocalStateQueryServer
  TestBlock (Point TestBlock) (Query TestBlock) m ()
-> m (LocalStateQueryServer
        TestBlock (Point TestBlock) (Query TestBlock) m ())
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (LocalStateQueryServer
   TestBlock (Point TestBlock) (Query TestBlock) m ()
 -> m (LocalStateQueryServer
         TestBlock (Point TestBlock) (Query TestBlock) m ()))
-> LocalStateQueryServer
     TestBlock (Point TestBlock) (Query TestBlock) m ()
-> m (LocalStateQueryServer
        TestBlock (Point TestBlock) (Query TestBlock) m ())
forall a b. (a -> b) -> a -> b
$
      ExtLedgerCfg TestBlock
-> STM m (Point TestBlock)
-> (Point TestBlock -> STM m (Maybe (ExtLedgerState TestBlock)))
-> STM m (Point TestBlock)
-> LocalStateQueryServer
     TestBlock (Point TestBlock) (Query TestBlock) m ()
forall (m :: * -> *) blk.
(IOLike m, BlockSupportsLedgerQuery blk, ConfigSupportsNode blk,
 HasAnnTip blk) =>
ExtLedgerCfg blk
-> STM m (Point blk)
-> (Point blk -> STM m (Maybe (ExtLedgerState blk)))
-> STM m (Point blk)
-> LocalStateQueryServer blk (Point blk) (Query blk) m ()
localStateQueryServer
        ExtLedgerCfg TestBlock
cfg
        (Point (ExtLedgerState TestBlock) -> Point TestBlock
forall {k1} {k2} (b :: k1) (b' :: k2).
Coercible (HeaderHash b) (HeaderHash b') =>
Point b -> Point b'
castPoint (Point (ExtLedgerState TestBlock) -> Point TestBlock)
-> (LedgerDB (ExtLedgerState TestBlock)
    -> Point (ExtLedgerState TestBlock))
-> LedgerDB (ExtLedgerState TestBlock)
-> Point TestBlock
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LedgerDB (ExtLedgerState TestBlock)
-> Point (ExtLedgerState TestBlock)
forall l. GetTip l => LedgerDB l -> Point l
LgrDB.ledgerDbTip (LedgerDB (ExtLedgerState TestBlock) -> Point TestBlock)
-> STM m (LedgerDB (ExtLedgerState TestBlock))
-> STM m (Point TestBlock)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> LgrDB m TestBlock -> STM m (LedgerDB (ExtLedgerState TestBlock))
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (LedgerDB' blk)
LgrDB.getCurrent LgrDB m TestBlock
lgrDB)
        (\Point TestBlock
pt -> Point TestBlock
-> LedgerDB (ExtLedgerState TestBlock)
-> Maybe (ExtLedgerState TestBlock)
forall blk l.
(HasHeader blk, IsLedger l, HeaderHash l ~ HeaderHash blk) =>
Point blk -> LedgerDB l -> Maybe l
LgrDB.ledgerDbPast Point TestBlock
pt (LedgerDB (ExtLedgerState TestBlock)
 -> Maybe (ExtLedgerState TestBlock))
-> STM m (LedgerDB (ExtLedgerState TestBlock))
-> STM m (Maybe (ExtLedgerState TestBlock))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> LgrDB m TestBlock -> STM m (LedgerDB (ExtLedgerState TestBlock))
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> STM m (LedgerDB' blk)
LgrDB.getCurrent LgrDB m TestBlock
lgrDB)
        STM m (Point TestBlock)
getImmutablePoint
  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
    getImmutablePoint :: STM m (Point TestBlock)
getImmutablePoint = Point TestBlock -> STM m (Point TestBlock)
forall a. a -> STM m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Point TestBlock -> STM m (Point TestBlock))
-> Point TestBlock -> STM m (Point TestBlock)
forall a b. (a -> b) -> a -> b
$ Chain TestBlock -> Point TestBlock
forall block. HasHeader block => Chain block -> Point block
Chain.headPoint (Chain TestBlock -> Point TestBlock)
-> Chain TestBlock -> Point TestBlock
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 (SecurityParam -> Word64
maxRollbacks SecurityParam
k)) Chain TestBlock
chain

-- | Initialise a 'LgrDB' with the given chain.
initLgrDB ::
     forall m. IOLike m
  => SecurityParam
  -> Chain TestBlock
  -> m (LgrDB m TestBlock)
initLgrDB :: forall (m :: * -> *).
IOLike m =>
SecurityParam -> Chain TestBlock -> m (LgrDB m TestBlock)
initLgrDB SecurityParam
k Chain TestBlock
chain = do
    StrictTVar m (LedgerDB (ExtLedgerState TestBlock))
varDB          <- LedgerDB (ExtLedgerState TestBlock)
-> m (StrictTVar m (LedgerDB (ExtLedgerState TestBlock)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO LedgerDB (ExtLedgerState TestBlock)
genesisLedgerDB
    StrictTVar m (Set (RealPoint TestBlock))
varPrevApplied <- Set (RealPoint TestBlock)
-> m (StrictTVar m (Set (RealPoint TestBlock)))
forall (m :: * -> *) a.
(HasCallStack, MonadSTM m, NoThunks a) =>
a -> m (StrictTVar m a)
newTVarIO Set (RealPoint TestBlock)
forall a. Monoid a => a
mempty
    let lgrDB :: LgrDB m TestBlock
lgrDB = StrictTVar m (LedgerDB (ExtLedgerState TestBlock))
-> StrictTVar m (Set (RealPoint TestBlock))
-> (RealPoint TestBlock -> m TestBlock)
-> Complete LgrDbArgs m TestBlock
-> SecurityParam
-> LgrDB m TestBlock
forall (m :: * -> *) blk.
StrictTVar m (LedgerDB' blk)
-> StrictTVar m (Set (RealPoint blk))
-> (RealPoint blk -> m blk)
-> Complete LgrDbArgs m blk
-> SecurityParam
-> LgrDB m blk
mkLgrDB StrictTVar m (LedgerDB (ExtLedgerState TestBlock))
varDB StrictTVar m (Set (RealPoint TestBlock))
varPrevApplied RealPoint TestBlock -> m TestBlock
resolve Complete LgrDbArgs m TestBlock
args SecurityParam
k
    LgrDB m TestBlock
-> LedgerDB (ExtLedgerState TestBlock)
-> BlockCache TestBlock
-> Word64
-> (UpdateLedgerDbTraceEvent TestBlock -> m ())
-> [Header TestBlock]
-> m (ValidateResult TestBlock)
forall (m :: * -> *) blk.
(IOLike m, LedgerSupportsProtocol blk, HasCallStack) =>
LgrDB m blk
-> LedgerDB' blk
-> BlockCache blk
-> Word64
-> (UpdateLedgerDbTraceEvent blk -> m ())
-> [Header blk]
-> m (ValidateResult blk)
LgrDB.validate LgrDB m TestBlock
lgrDB LedgerDB (ExtLedgerState TestBlock)
genesisLedgerDB BlockCache TestBlock
forall blk. BlockCache blk
BlockCache.empty Word64
0 UpdateLedgerDbTraceEvent TestBlock -> m ()
forall blk. blk -> m ()
noopTrace
      ((TestBlock -> Header TestBlock)
-> [TestBlock] -> [Header TestBlock]
forall a b. (a -> b) -> [a] -> [b]
map TestBlock -> Header TestBlock
forall blk. GetHeader blk => blk -> Header blk
getHeader (Chain TestBlock -> [TestBlock]
forall block. Chain block -> [block]
Chain.toOldestFirst Chain TestBlock
chain)) m (ValidateResult TestBlock)
-> (ValidateResult TestBlock -> m (LgrDB m TestBlock))
-> m (LgrDB m TestBlock)
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
        LgrDB.ValidateExceededRollBack ExceededRollback
_ ->
          String -> m (LgrDB m TestBlock)
forall a. HasCallStack => String -> a
error String
"impossible: rollback was 0"
        LgrDB.ValidateLedgerError AnnLedgerError' TestBlock
_ ->
          String -> m (LgrDB m TestBlock)
forall a. HasCallStack => String -> a
error String
"impossible: there were no invalid blocks"
        LgrDB.ValidateSuccessful LedgerDB (ExtLedgerState TestBlock)
ledgerDB' -> 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
$ LgrDB m TestBlock
-> LedgerDB (ExtLedgerState TestBlock) -> STM m ()
forall (m :: * -> *) blk.
IOLike m =>
LgrDB m blk -> LedgerDB' blk -> STM m ()
LgrDB.setCurrent LgrDB m TestBlock
lgrDB LedgerDB (ExtLedgerState TestBlock)
ledgerDB'
          LgrDB m TestBlock -> m (LgrDB m TestBlock)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return LgrDB m TestBlock
lgrDB
  where
    resolve :: RealPoint TestBlock -> m TestBlock
    resolve :: RealPoint TestBlock -> m TestBlock
resolve = TestBlock -> m TestBlock
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (TestBlock -> m TestBlock)
-> (RealPoint TestBlock -> TestBlock)
-> RealPoint TestBlock
-> m TestBlock
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Map (RealPoint TestBlock) TestBlock
blockMapping Map (RealPoint TestBlock) TestBlock
-> RealPoint TestBlock -> TestBlock
forall k a. Ord k => Map k a -> k -> a
Map.!)

    blockMapping :: Map (RealPoint TestBlock) TestBlock
    blockMapping :: Map (RealPoint TestBlock) TestBlock
blockMapping = [(RealPoint TestBlock, TestBlock)]
-> Map (RealPoint TestBlock) TestBlock
forall k a. Ord k => [(k, a)] -> Map k a
Map.fromList
      [(TestBlock -> RealPoint TestBlock
forall blk. HasHeader blk => blk -> RealPoint blk
blockRealPoint TestBlock
b, TestBlock
b) | TestBlock
b <- Chain TestBlock -> [TestBlock]
forall block. Chain block -> [block]
Chain.toOldestFirst Chain TestBlock
chain]

    cfg :: LedgerDbCfg (ExtLedgerState TestBlock)
cfg = TopLevelConfig TestBlock -> LedgerDbCfg (ExtLedgerState TestBlock)
forall blk.
ConsensusProtocol (BlockProtocol blk) =>
TopLevelConfig blk -> LedgerDbCfg (ExtLedgerState blk)
configLedgerDb (TopLevelConfig TestBlock
 -> LedgerDbCfg (ExtLedgerState TestBlock))
-> TopLevelConfig TestBlock
-> LedgerDbCfg (ExtLedgerState TestBlock)
forall a b. (a -> b) -> a -> b
$ SecurityParam -> TopLevelConfig TestBlock
testCfg SecurityParam
k

    genesisLedgerDB :: LedgerDB (ExtLedgerState TestBlock)
genesisLedgerDB = ExtLedgerState TestBlock -> LedgerDB (ExtLedgerState TestBlock)
forall l. GetTip l => l -> LedgerDB l
LgrDB.ledgerDbWithAnchor ExtLedgerState TestBlock
testInitExtLedger

    noopTrace :: blk -> m ()
    noopTrace :: forall blk. blk -> m ()
noopTrace = m () -> blk -> m ()
forall a b. a -> b -> a
const (m () -> blk -> m ()) -> m () -> blk -> m ()
forall a b. (a -> b) -> a -> b
$ () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

    args :: Complete LgrDbArgs m TestBlock
args = LgrDbArgs
      { lgrConfig :: HKD Identity (LedgerDbCfg (ExtLedgerState TestBlock))
lgrConfig               = HKD Identity (LedgerDbCfg (ExtLedgerState TestBlock))
LedgerDbCfg (ExtLedgerState TestBlock)
cfg
      , lgrHasFS :: HKD Identity (SomeHasFS m)
lgrHasFS                = HasFS m () -> SomeHasFS m
forall h (m :: * -> *). Eq h => HasFS m h -> SomeHasFS m
SomeHasFS (String -> HasFS m ()
forall a. HasCallStack => String -> a
error String
"lgrHasFS" :: HasFS m ())
      , lgrDiskPolicyArgs :: DiskPolicyArgs
lgrDiskPolicyArgs       = DiskPolicyArgs
defaultDiskPolicyArgs
      , lgrGenesis :: HKD Identity (m (ExtLedgerState TestBlock))
lgrGenesis              = ExtLedgerState TestBlock -> m (ExtLedgerState TestBlock)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ExtLedgerState TestBlock
testInitExtLedger
      , lgrTracer :: Tracer m (TraceSnapshotEvent TestBlock)
lgrTracer               = Tracer m (TraceSnapshotEvent TestBlock)
forall (m :: * -> *) a. Applicative m => Tracer m a
nullTracer
      }

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

{-------------------------------------------------------------------------------
  Orphans
-------------------------------------------------------------------------------}

instance Arbitrary SecurityParam where
  arbitrary :: Gen SecurityParam
arbitrary = Word64 -> SecurityParam
SecurityParam (Word64 -> SecurityParam) -> Gen 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)
  shrink :: SecurityParam -> [SecurityParam]
shrink (SecurityParam Word64
k) = [Word64 -> SecurityParam
SecurityParam Word64
k' |  Word64
k' <- Word64 -> [Word64]
forall a. Arbitrary a => a -> [a]
shrink Word64
k, Word64
k' Word64 -> Word64 -> Bool
forall a. Ord a => a -> a -> Bool
> Word64
0]