{-# LANGUAGE DataKinds #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
{-# OPTIONS_GHC -Wno-orphans #-}

-- | This runs the Byron ledger and the Byron specification in lockstep,
--   verifying that they agree at every point.
module Test.ThreadNet.DualByron (tests) where

import qualified Byron.Spec.Chain.STS.Rule.Chain as Spec
import qualified Byron.Spec.Ledger.Core as Spec
import qualified Byron.Spec.Ledger.UTxO as Spec
import qualified Cardano.Chain.ProtocolConstants as Impl
import qualified Cardano.Chain.UTxO as Impl
import Cardano.Ledger.BaseTypes (nonZeroOr)
import Control.Monad.Except
import qualified Control.State.Transition.Extended as Spec
import Data.ByteString (ByteString)
import qualified Data.Map.Strict as Map
import Data.Proxy
import qualified Data.Set as Set
import Ouroboros.Consensus.Block
import Ouroboros.Consensus.BlockchainTime
import Ouroboros.Consensus.Byron.Ledger
import Ouroboros.Consensus.Byron.Ledger.Conversions
import Ouroboros.Consensus.ByronDual.Ledger
import Ouroboros.Consensus.ByronDual.Node
import Ouroboros.Consensus.ByronSpec.Ledger
import qualified Ouroboros.Consensus.ByronSpec.Ledger.Genesis as Genesis
import qualified Ouroboros.Consensus.ByronSpec.Ledger.Rules as Rules
import Ouroboros.Consensus.Config
import Ouroboros.Consensus.Ledger.Abstract
import Ouroboros.Consensus.Ledger.Dual
import Ouroboros.Consensus.Ledger.SupportsMempool
import Ouroboros.Consensus.Ledger.Tables.Utils
import Ouroboros.Consensus.Node.ProtocolInfo
import Ouroboros.Consensus.NodeId
import Ouroboros.Consensus.Protocol.PBFT
import Ouroboros.Consensus.TypeFamilyWrappers
import qualified Test.Cardano.Chain.Elaboration.UTxO as Spec.Test
import qualified Test.Control.State.Transition.Generator as Spec.QC
import Test.QuickCheck
import Test.QuickCheck.Hedgehog (hedgehog)
import Test.Tasty
import Test.Tasty.QuickCheck
import qualified Test.ThreadNet.Byron as Byron
import Test.ThreadNet.General
import qualified Test.ThreadNet.Ref.PBFT as Ref
import Test.ThreadNet.TxGen
import Test.ThreadNet.Util
import Test.ThreadNet.Util.NodeRestarts (noRestarts)
import Test.ThreadNet.Util.NodeToNodeVersion (newestVersion)
import Test.Util.HardFork.Future (singleEraFuture)
import Test.Util.Slots (NumSlots (..))
import Test.Util.TestEnv (adjustQuickCheckTests)

tests :: TestTree
tests :: TestTree
tests =
  TestName -> [TestTree] -> TestTree
testGroup
    TestName
"DualByron"
    [ (Int -> Int) -> TestTree -> TestTree
adjustQuickCheckTests (Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
10) (TestTree -> TestTree) -> TestTree -> TestTree
forall a b. (a -> b) -> a -> b
$ TestName -> (SetupDualByron -> Property) -> TestTree
forall a. Testable a => TestName -> a -> TestTree
testProperty TestName
"convergence" ((SetupDualByron -> Property) -> TestTree)
-> (SetupDualByron -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$ SetupDualByron -> Property
prop_convergence
    ]

-- These tests are very expensive, due to the Byron generators
-- (100 tests take about 20 minutes)
-- We limit it to 10 tests for now.
prop_convergence :: SetupDualByron -> Property
prop_convergence :: SetupDualByron -> Property
prop_convergence SetupDualByron
setup =
  (\Property
prop -> if Bool
mightForgeInSlot0 then Property
forall a. a
discard else Property
prop) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    TestName -> [TestName] -> Property -> Property
forall prop.
Testable prop =>
TestName -> [TestName] -> prop -> Property
tabulate TestName
"Ref.PBFT result" [Result -> TestName
Ref.resultConstrName Result
refResult] (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
      PropGeneralArgs DualByronBlock
-> TestOutput DualByronBlock -> Property
forall blk.
(Condense blk, Condense (HeaderHash blk), Eq blk, RunNode blk) =>
PropGeneralArgs blk -> TestOutput blk -> Property
prop_general
        PropGeneralArgs
          { pgaBlockProperty :: DualByronBlock -> Property
pgaBlockProperty = Property -> DualByronBlock -> Property
forall a b. a -> b -> a
const (Property -> DualByronBlock -> Property)
-> Property -> DualByronBlock -> Property
forall a b. (a -> b) -> a -> b
$ Bool -> Property
forall prop. Testable prop => prop -> Property
property Bool
True
          , pgaCountTxs :: DualByronBlock -> Word64
pgaCountTxs = ByronBlock -> Word64
countByronGenTxs (ByronBlock -> Word64)
-> (DualByronBlock -> ByronBlock) -> DualByronBlock -> Word64
forall b c a. (b -> c) -> (a -> b) -> a -> c
. DualByronBlock -> ByronBlock
forall m a. DualBlock m a -> m
dualBlockMain
          , pgaExpectedCannotForge :: SlotNo -> NodeId -> WrapCannotForge DualByronBlock -> Bool
pgaExpectedCannotForge = SetupDualByron
-> SlotNo -> NodeId -> WrapCannotForge DualByronBlock -> Bool
setupExpectedCannotForge SetupDualByron
setup
          , pgaFirstBlockNo :: BlockNo
pgaFirstBlockNo = BlockNo
1
          , pgaFixedMaxForkLength :: Maybe NumBlocks
pgaFixedMaxForkLength =
              NumBlocks -> Maybe NumBlocks
forall a. a -> Maybe a
Just (NumBlocks -> Maybe NumBlocks) -> NumBlocks -> Maybe NumBlocks
forall a b. (a -> b) -> a -> b
$ Word64 -> NumBlocks
NumBlocks (Word64 -> NumBlocks) -> Word64 -> NumBlocks
forall a b. (a -> b) -> a -> b
$ case Result
refResult of
                Ref.Forked{} -> Word64
1
                Result
_ -> Word64
0
          , pgaFixedSchedule :: Maybe LeaderSchedule
pgaFixedSchedule =
              LeaderSchedule -> Maybe LeaderSchedule
forall a. a -> Maybe a
Just (LeaderSchedule -> Maybe LeaderSchedule)
-> LeaderSchedule -> Maybe LeaderSchedule
forall a b. (a -> b) -> a -> b
$ NumCoreNodes -> NumSlots -> LeaderSchedule
roundRobinLeaderSchedule NumCoreNodes
numCoreNodes NumSlots
numSlots
          , pgaSecurityParam :: SecurityParam
pgaSecurityParam = SecurityParam
setupK
          , pgaTestConfig :: TestConfig
pgaTestConfig = TestConfig
setupTestConfig
          , pgaTestConfigB :: TestConfigB DualByronBlock
pgaTestConfigB = SetupDualByron -> TestConfigB DualByronBlock
setupTestConfigB SetupDualByron
setup
          }
        (SetupDualByron -> TestOutput DualByronBlock
setupTestOutput SetupDualByron
setup)
 where
  SetupDualByron{ByronSpecGenesis
TestSetup
setupGenesis :: ByronSpecGenesis
setupByron :: TestSetup
setupByron :: SetupDualByron -> TestSetup
setupGenesis :: SetupDualByron -> ByronSpecGenesis
..} = SetupDualByron
setup
  Byron.TestSetup{(NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
SlotLength
SecurityParam
NodeJoinPlan
NodeRestarts
TestConfig
ProduceEBBs
setupK :: SecurityParam
setupTestConfig :: TestConfig
setupEBBs :: ProduceEBBs
setupNodeJoinPlan :: NodeJoinPlan
setupNodeRestarts :: NodeRestarts
setupSlotLength :: SlotLength
setupVersion :: (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
setupVersion :: TestSetup -> (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
setupSlotLength :: TestSetup -> SlotLength
setupNodeRestarts :: TestSetup -> NodeRestarts
setupNodeJoinPlan :: TestSetup -> NodeJoinPlan
setupTestConfig :: TestSetup -> TestConfig
setupK :: TestSetup -> SecurityParam
setupEBBs :: TestSetup -> ProduceEBBs
..} = TestSetup
setupByron
  TestConfig{NumCoreNodes
NumSlots
NodeTopology
Seed
numCoreNodes :: NumCoreNodes
numSlots :: NumSlots
initSeed :: Seed
nodeTopology :: NodeTopology
numSlots :: TestConfig -> NumSlots
numCoreNodes :: TestConfig -> NumCoreNodes
nodeTopology :: TestConfig -> NodeTopology
initSeed :: TestConfig -> Seed
..} = TestConfig
setupTestConfig

  refResult :: Ref.Result
  refResult :: Result
refResult =
    HasCallStack => PBftParams -> NodeJoinPlan -> NumSlots -> Result
PBftParams -> NodeJoinPlan -> NumSlots -> Result
Ref.simulate (SetupDualByron -> PBftParams
setupParams SetupDualByron
setup) NodeJoinPlan
setupNodeJoinPlan NumSlots
numSlots

  -- The test infrastructure allows nodes to forge in slot 0; however, the
  -- cardano-ledger-specs code causes @PBFTFailure (SlotNotAfterLastBlock
  -- (Slot 0) (Slot 0))@ in that case. So we discard such tests.
  --
  -- This is ultimately due to the spec not modeling EBBs, while Byron
  -- requires that successor of the genesis block is always the epoch 0 EBB.
  -- As a result, the PBFT implementation tests the slot progression with
  -- @<=@ to accomodate EBBs whereas the executable STS spec uses @<@.
  mightForgeInSlot0 :: Bool
  mightForgeInSlot0 :: Bool
mightForgeInSlot0 = case Result
refResult of
    Ref.Forked NumSlots
_ Map CoreNodeId (Set SlotNo)
m -> (Set SlotNo -> Bool) -> Map CoreNodeId (Set SlotNo) -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
any (SlotNo
0 SlotNo -> Set SlotNo -> Bool
forall a. Ord a => a -> Set a -> Bool
`Set.member`) Map CoreNodeId (Set SlotNo)
m
    Result
Ref.Nondeterministic -> Bool
True
    Ref.Outcomes [Outcome]
outcomes -> case [Outcome]
outcomes of
      [] -> Bool
False
      Outcome
o : [Outcome]
_ -> case Outcome
o of
        Outcome
Ref.Absent -> Bool
False
        Outcome
Ref.Nominal -> Bool
True
        Outcome
Ref.Unable -> Bool
True
        Outcome
Ref.Wasted -> Bool
True

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

data SetupDualByron = SetupDualByron
  { SetupDualByron -> ByronSpecGenesis
setupGenesis :: ByronSpecGenesis
  , SetupDualByron -> TestSetup
setupByron :: Byron.TestSetup
  }
  deriving Int -> SetupDualByron -> ShowS
[SetupDualByron] -> ShowS
SetupDualByron -> TestName
(Int -> SetupDualByron -> ShowS)
-> (SetupDualByron -> TestName)
-> ([SetupDualByron] -> ShowS)
-> Show SetupDualByron
forall a.
(Int -> a -> ShowS) -> (a -> TestName) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> SetupDualByron -> ShowS
showsPrec :: Int -> SetupDualByron -> ShowS
$cshow :: SetupDualByron -> TestName
show :: SetupDualByron -> TestName
$cshowList :: [SetupDualByron] -> ShowS
showList :: [SetupDualByron] -> ShowS
Show

setupParams :: SetupDualByron -> PBftParams
setupParams :: SetupDualByron -> PBftParams
setupParams = ByronSpecGenesis -> PBftParams
byronPBftParams (ByronSpecGenesis -> PBftParams)
-> (SetupDualByron -> ByronSpecGenesis)
-> SetupDualByron
-> PBftParams
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SetupDualByron -> ByronSpecGenesis
setupGenesis

setupTestConfigB :: SetupDualByron -> TestConfigB DualByronBlock
setupTestConfigB :: SetupDualByron -> TestConfigB DualByronBlock
setupTestConfigB SetupDualByron{ByronSpecGenesis
TestSetup
setupByron :: SetupDualByron -> TestSetup
setupGenesis :: SetupDualByron -> ByronSpecGenesis
setupGenesis :: ByronSpecGenesis
setupByron :: TestSetup
..} =
  TestConfigB
    { forgeEbbEnv :: Maybe (ForgeEbbEnv DualByronBlock)
forgeEbbEnv = Maybe (ForgeEbbEnv DualByronBlock)
forall a. Maybe a
Nothing -- spec does not model EBBs
    , future :: Future
future = SlotLength -> EpochSize -> Future
singleEraFuture SlotLength
setupSlotLength EpochSize
epochSize
    , messageDelay :: CalcMessageDelay DualByronBlock
messageDelay = CalcMessageDelay DualByronBlock
forall blk. CalcMessageDelay blk
noCalcMessageDelay
    , nodeJoinPlan :: NodeJoinPlan
nodeJoinPlan = NodeJoinPlan
setupNodeJoinPlan
    , nodeRestarts :: NodeRestarts
nodeRestarts = NodeRestarts
setupNodeRestarts
    , txGenExtra :: TxGenExtra DualByronBlock
txGenExtra = ()
    , version :: (NodeToNodeVersion, BlockNodeToNodeVersion DualByronBlock)
version = Proxy DualByronBlock
-> (NodeToNodeVersion, BlockNodeToNodeVersion DualByronBlock)
forall blk.
SupportedNetworkProtocolVersion blk =>
Proxy blk -> (NodeToNodeVersion, BlockNodeToNodeVersion blk)
newestVersion (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @DualByronBlock)
    }
 where
  Byron.TestSetup{(NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
SlotLength
SecurityParam
NodeJoinPlan
NodeRestarts
TestConfig
ProduceEBBs
setupVersion :: TestSetup -> (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
setupSlotLength :: TestSetup -> SlotLength
setupNodeRestarts :: TestSetup -> NodeRestarts
setupNodeJoinPlan :: TestSetup -> NodeJoinPlan
setupTestConfig :: TestSetup -> TestConfig
setupK :: TestSetup -> SecurityParam
setupEBBs :: TestSetup -> ProduceEBBs
setupSlotLength :: SlotLength
setupNodeJoinPlan :: NodeJoinPlan
setupNodeRestarts :: NodeRestarts
setupEBBs :: ProduceEBBs
setupK :: SecurityParam
setupTestConfig :: TestConfig
setupVersion :: (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
..} = TestSetup
setupByron

  epochSize :: EpochSize
  epochSize :: EpochSize
epochSize =
    EpochSlots -> EpochSize
fromByronEpochSlots (EpochSlots -> EpochSize) -> EpochSlots -> EpochSize
forall a b. (a -> b) -> a -> b
$ BlockCount -> EpochSlots
Impl.kEpochSlots (SecurityParam -> BlockCount
toByronBlockCount SecurityParam
setupK)

setupTestOutput :: SetupDualByron -> TestOutput DualByronBlock
setupTestOutput :: SetupDualByron -> TestOutput DualByronBlock
setupTestOutput setup :: SetupDualByron
setup@SetupDualByron{ByronSpecGenesis
TestSetup
setupByron :: SetupDualByron -> TestSetup
setupGenesis :: SetupDualByron -> ByronSpecGenesis
setupGenesis :: ByronSpecGenesis
setupByron :: TestSetup
..} =
  TestConfig
-> TestConfigB DualByronBlock
-> (forall (m :: * -> *).
    IOLike m =>
    TestConfigMB m DualByronBlock)
-> TestOutput DualByronBlock
forall blk.
(RunNode blk, TxGen blk, TracingConstraints blk, HasCallStack) =>
TestConfig
-> TestConfigB blk
-> (forall (m :: * -> *). IOLike m => TestConfigMB m blk)
-> TestOutput blk
runTestNetwork
    TestConfig
testConfig
    TestConfigB DualByronBlock
testConfigB
    TestConfigMB
      { nodeInfo :: CoreNodeId -> TestNodeInitialization m DualByronBlock
nodeInfo = \CoreNodeId
coreNodeId ->
          (ProtocolInfo DualByronBlock
 -> m [BlockForging m DualByronBlock]
 -> TestNodeInitialization m DualByronBlock)
-> (ProtocolInfo DualByronBlock, m [BlockForging m DualByronBlock])
-> TestNodeInitialization m DualByronBlock
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry
            ProtocolInfo DualByronBlock
-> m [BlockForging m DualByronBlock]
-> TestNodeInitialization m DualByronBlock
forall blk (m :: * -> *).
ProtocolInfo blk
-> m [BlockForging m blk] -> TestNodeInitialization m blk
plainTestNodeInitialization
            ( ByronSpecGenesis
-> PBftParams
-> [CoreNodeId]
-> (ProtocolInfo DualByronBlock, m [BlockForging m DualByronBlock])
forall (m :: * -> *).
Monad m =>
ByronSpecGenesis
-> PBftParams
-> [CoreNodeId]
-> (ProtocolInfo DualByronBlock, m [BlockForging m DualByronBlock])
protocolInfoDualByron
                ByronSpecGenesis
setupGenesis
                (SetupDualByron -> PBftParams
setupParams SetupDualByron
setup)
                [CoreNodeId
coreNodeId]
            )
      , mkRekeyM :: Maybe (m (RekeyM m DualByronBlock))
mkRekeyM = Maybe (m (RekeyM m DualByronBlock))
forall a. Maybe a
Nothing -- TODO
      }
 where
  testConfig :: TestConfig
testConfig = TestSetup -> TestConfig
Byron.setupTestConfig TestSetup
setupByron
  testConfigB :: TestConfigB DualByronBlock
testConfigB = SetupDualByron -> TestConfigB DualByronBlock
setupTestConfigB SetupDualByron
setup

setupExpectedCannotForge ::
  SetupDualByron ->
  SlotNo ->
  NodeId ->
  WrapCannotForge DualByronBlock ->
  Bool
setupExpectedCannotForge :: SetupDualByron
-> SlotNo -> NodeId -> WrapCannotForge DualByronBlock -> Bool
setupExpectedCannotForge SetupDualByron{ByronSpecGenesis
TestSetup
setupByron :: SetupDualByron -> TestSetup
setupGenesis :: SetupDualByron -> ByronSpecGenesis
setupGenesis :: ByronSpecGenesis
setupByron :: TestSetup
..} SlotNo
s NodeId
nid (WrapCannotForge CannotForge DualByronBlock
cl) =
  SecurityParam
-> NumCoreNodes
-> NodeRestarts
-> SlotNo
-> NodeId
-> WrapCannotForge ByronBlock
-> Bool
Byron.expectedCannotForge
    SecurityParam
setupK
    NumCoreNodes
numCoreNodes
    NodeRestarts
setupNodeRestarts
    SlotNo
s
    NodeId
nid
    (CannotForge ByronBlock -> WrapCannotForge ByronBlock
forall blk. CannotForge blk -> WrapCannotForge blk
WrapCannotForge CannotForge DualByronBlock
CannotForge ByronBlock
cl)
 where
  Byron.TestSetup{(NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
SlotLength
SecurityParam
NodeJoinPlan
NodeRestarts
TestConfig
ProduceEBBs
setupVersion :: TestSetup -> (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
setupSlotLength :: TestSetup -> SlotLength
setupNodeRestarts :: TestSetup -> NodeRestarts
setupNodeJoinPlan :: TestSetup -> NodeJoinPlan
setupTestConfig :: TestSetup -> TestConfig
setupK :: TestSetup -> SecurityParam
setupEBBs :: TestSetup -> ProduceEBBs
setupK :: SecurityParam
setupNodeRestarts :: NodeRestarts
setupEBBs :: ProduceEBBs
setupTestConfig :: TestConfig
setupNodeJoinPlan :: NodeJoinPlan
setupSlotLength :: SlotLength
setupVersion :: (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
..} = TestSetup
setupByron
  TestConfig{NumCoreNodes
NumSlots
NodeTopology
Seed
numSlots :: TestConfig -> NumSlots
numCoreNodes :: TestConfig -> NumCoreNodes
nodeTopology :: TestConfig -> NodeTopology
initSeed :: TestConfig -> Seed
numCoreNodes :: NumCoreNodes
initSeed :: Seed
nodeTopology :: NodeTopology
numSlots :: NumSlots
..} = TestConfig
setupTestConfig

{-------------------------------------------------------------------------------
  Generator for 'SetupDualByron'
-------------------------------------------------------------------------------}

-- | We do an awkward dance in this generator. We want to reuse
-- 'Byron.TestSetup' as much as possible. However, 'genSpecGenesis' needs values
-- provided by 'Byron.TestSetup' (ie @numSlots@ and @slotLen@) but also sets a
-- value provided by 'Byron.TestSetup' (eg @k@).
instance Arbitrary SetupDualByron where
  arbitrary :: Gen SetupDualByron
arbitrary = do
    numSlots <- Gen NumSlots
forall a. Arbitrary a => Gen a
arbitrary
    slotLen <- arbitrary

    genesis0 <- genSpecGenesis slotLen numSlots
    let params@PBftParams{..} = byronPBftParams genesis0
        setupGenesis = PBftParams -> ByronSpecGenesis -> ByronSpecGenesis
adjustGenesis PBftParams
params ByronSpecGenesis
genesis0

    -- TODO: Once we produce all kinds of transactions, we will need to
    -- rethink rekeys/restarts (but might not be trivial, as we do not
    -- generate the blocks upfront..).
    setupByron <-
      (\TestSetup
x -> TestSetup
x{Byron.setupNodeRestarts = noRestarts})
        <$> Byron.genTestSetup
          pbftSecurityParam
          pbftNumNodes
          numSlots
          slotLen

    return SetupDualByron{..}
   where
    -- The spec tests and the Byron tests compute a different test value for
    -- the PBFT threshold. For now we ignore the value computed by the spec
    -- and override it with the value computed in the Byron tests.
    --
    -- TODO: It would be interesting to see if we can bring these two in line,
    -- but if we do, we probably need to adjust 'expectedBlockRejection'.
    adjustGenesis ::
      PBftParams ->
      ByronSpecGenesis ->
      ByronSpecGenesis
    adjustGenesis :: PBftParams -> ByronSpecGenesis -> ByronSpecGenesis
adjustGenesis =
      (Double -> Double) -> ByronSpecGenesis -> ByronSpecGenesis
Genesis.modPBftThreshold
        ((Double -> Double) -> ByronSpecGenesis -> ByronSpecGenesis)
-> (PBftParams -> Double -> Double)
-> PBftParams
-> ByronSpecGenesis
-> ByronSpecGenesis
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Double -> Double -> Double
forall a b. a -> b -> a
const
        (Double -> Double -> Double)
-> (PBftParams -> Double) -> PBftParams -> Double -> Double
forall b c a. (b -> c) -> (a -> b) -> a -> c
. PBftSignatureThreshold -> Double
getPBftSignatureThreshold
        (PBftSignatureThreshold -> Double)
-> (PBftParams -> PBftSignatureThreshold) -> PBftParams -> Double
forall b c a. (b -> c) -> (a -> b) -> a -> c
. PBftParams -> PBftSignatureThreshold
pbftSignatureThreshold

-- TODO shrink

-- | Generate abstract genesis config (environment for the CHAIN rule)
--
-- The generator for the environment tries to pick a @k@ that ensures the
-- trace (independent of its length) contains multiple epochs, which is why
-- this wants to know the chain length; we don't know that a-priority, but we
-- do know the number of slots, and will use that as a stand-in.
genSpecGenesis :: SlotLength -> NumSlots -> Gen ByronSpecGenesis
genSpecGenesis :: SlotLength -> NumSlots -> Gen ByronSpecGenesis
genSpecGenesis SlotLength
slotLen (NumSlots Word64
numSlots) =
  ((Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
 -> ByronSpecGenesis)
-> Gen (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
-> Gen ByronSpecGenesis
forall a b. (a -> b) -> Gen a -> Gen b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
-> ByronSpecGenesis
Environment CHAIN -> ByronSpecGenesis
fromEnv (Gen (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
 -> Gen ByronSpecGenesis)
-> (Gen (Environment CHAIN)
    -> Gen (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount))
-> Gen (Environment CHAIN)
-> Gen ByronSpecGenesis
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Gen (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
-> Gen (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
Gen (Environment CHAIN)
-> Gen (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
forall a. Gen a -> Gen a
hedgehog (Gen (Environment CHAIN) -> Gen ByronSpecGenesis)
-> Gen (Environment CHAIN) -> Gen ByronSpecGenesis
forall a b. (a -> b) -> a -> b
$
    -- Convert Hedgehog generator to QuickCheck one
    -- Unfortunately, this does mean we lose any shrinking.
    forall s. HasTrace s => Word64 -> Gen (Environment s)
Spec.QC.envGen @Spec.CHAIN Word64
numSlots
 where
  -- Start with a larger initial UTxO. This is important, because the Byron
  -- spec TX generator is wasteful, and with every transaction the UTxO
  -- shrinks. By starting with a larger initial UTxO we avoid the depleting
  -- the UTxO too early (at which point we'd not be able to generate further
  -- transactions, and produce empty blocks only).
  fromEnv :: Spec.Environment Spec.CHAIN -> ByronSpecGenesis
  fromEnv :: Environment CHAIN -> ByronSpecGenesis
fromEnv =
    (Integer -> Integer) -> ByronSpecGenesis -> ByronSpecGenesis
Genesis.modUtxoValues (Integer -> Integer -> Integer
forall a. Num a => a -> a -> a
* Integer
10000)
      (ByronSpecGenesis -> ByronSpecGenesis)
-> ((Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
    -> ByronSpecGenesis)
-> (Slot, UTxO, Set VKeyGenesis, PParams, BlockCount)
-> ByronSpecGenesis
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Natural -> Environment CHAIN -> ByronSpecGenesis
Genesis.fromChainEnv (SlotLength -> Natural
toByronSlotLength SlotLength
slotLen)

byronPBftParams :: ByronSpecGenesis -> PBftParams
byronPBftParams :: ByronSpecGenesis -> PBftParams
byronPBftParams ByronSpecGenesis{Natural
Set VKeyGenesis
BlockCount
PParams
UTxO
byronSpecGenesisDelegators :: Set VKeyGenesis
byronSpecGenesisInitUtxo :: UTxO
byronSpecGenesisInitPParams :: PParams
byronSpecGenesisSecurityParam :: BlockCount
byronSpecGenesisSlotLength :: Natural
byronSpecGenesisSlotLength :: ByronSpecGenesis -> Natural
byronSpecGenesisSecurityParam :: ByronSpecGenesis -> BlockCount
byronSpecGenesisInitPParams :: ByronSpecGenesis -> PParams
byronSpecGenesisInitUtxo :: ByronSpecGenesis -> UTxO
byronSpecGenesisDelegators :: ByronSpecGenesis -> Set VKeyGenesis
..} =
  SecurityParam -> NumCoreNodes -> PBftParams
Byron.byronPBftParams (NonZero Word64 -> SecurityParam
SecurityParam NonZero Word64
k') NumCoreNodes
numCoreNodes
 where
  Spec.BlockCount Word64
k = BlockCount
byronSpecGenesisSecurityParam
  k' :: NonZero Word64
k' = Word64 -> NonZero Word64 -> NonZero Word64
forall a. HasZero a => a -> NonZero a -> NonZero a
nonZeroOr Word64
k (NonZero Word64 -> NonZero Word64)
-> NonZero Word64 -> NonZero Word64
forall a b. (a -> b) -> a -> b
$ TestName -> NonZero Word64
forall a. HasCallStack => TestName -> a
error TestName
"Got zero. Expected nonzero."

  numCoreNodes :: NumCoreNodes
  numCoreNodes :: NumCoreNodes
numCoreNodes =
    Word64 -> NumCoreNodes
NumCoreNodes (Word64 -> NumCoreNodes) -> Word64 -> NumCoreNodes
forall a b. (a -> b) -> a -> b
$
      Int -> Word64
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Set VKeyGenesis -> Int
forall a. Set a -> Int
Set.size Set VKeyGenesis
byronSpecGenesisDelegators)

{-------------------------------------------------------------------------------
  Generate transactions
-------------------------------------------------------------------------------}

instance TxGen DualByronBlock where
  testGenTxs :: CoreNodeId
-> NumCoreNodes
-> SlotNo
-> TopLevelConfig DualByronBlock
-> TxGenExtra DualByronBlock
-> LedgerState DualByronBlock ValuesMK
-> Gen [GenTx DualByronBlock]
testGenTxs CoreNodeId
_coreNodeId NumCoreNodes
_numCoreNodes SlotNo
curSlotNo TopLevelConfig DualByronBlock
cfg () = \LedgerState DualByronBlock ValuesMK
st -> do
    n <- (Integer, Integer) -> Gen Integer
forall a. Random a => (a, a) -> Gen a
choose (Integer
0, Integer
20)
    go [] n $
      applyDiffs st $
        applyChainTick OmitLedgerEvents (configLedger cfg) curSlotNo $
          forgetLedgerTables st
   where
    -- Attempt to produce @n@ transactions
    -- Stops when the transaction generator cannot produce more txs
    go ::
      [GenTx DualByronBlock] -> -- Accumulator
      Integer -> -- Number of txs to still produce
      TickedLedgerState DualByronBlock ValuesMK ->
      Gen [GenTx DualByronBlock]
    go :: [GenTx DualByronBlock]
-> Integer
-> TickedLedgerState DualByronBlock ValuesMK
-> Gen [GenTx DualByronBlock]
go [GenTx DualByronBlock]
acc Integer
0 TickedLedgerState DualByronBlock ValuesMK
_ = [GenTx DualByronBlock] -> Gen [GenTx DualByronBlock]
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return ([GenTx DualByronBlock] -> [GenTx DualByronBlock]
forall a. [a] -> [a]
reverse [GenTx DualByronBlock]
acc)
    go [GenTx DualByronBlock]
acc Integer
n TickedLedgerState DualByronBlock ValuesMK
st = do
      tx <- TopLevelConfig DualByronBlock
-> TickedLedgerState DualByronBlock ValuesMK
-> Gen (GenTx DualByronBlock)
genTx TopLevelConfig DualByronBlock
cfg TickedLedgerState DualByronBlock ValuesMK
st
      case runExcept $
        applyTx
          (configLedger cfg)
          DoNotIntervene
          curSlotNo
          tx
          st of
        Right (Ticked (LedgerState DualByronBlock) DiffMK
st', Validated (GenTx DualByronBlock)
_vtx) ->
          [GenTx DualByronBlock]
-> Integer
-> TickedLedgerState DualByronBlock ValuesMK
-> Gen [GenTx DualByronBlock]
go (GenTx DualByronBlock
tx GenTx DualByronBlock
-> [GenTx DualByronBlock] -> [GenTx DualByronBlock]
forall a. a -> [a] -> [a]
: [GenTx DualByronBlock]
acc) (Integer
n Integer -> Integer -> Integer
forall a. Num a => a -> a -> a
- Integer
1) (TickedLedgerState DualByronBlock ValuesMK
-> Ticked (LedgerState DualByronBlock) DiffMK
-> TickedLedgerState DualByronBlock ValuesMK
forall (l :: MapKind -> *) (l' :: MapKind -> *).
(SameUtxoTypes l l', HasLedgerTables l, HasLedgerTables l') =>
l ValuesMK -> l' DiffMK -> l' ValuesMK
applyDiffs TickedLedgerState DualByronBlock ValuesMK
st Ticked (LedgerState DualByronBlock) DiffMK
st')
        Left DualGenTxErr ByronBlock ByronSpecBlock
_ -> TestName -> Gen [GenTx DualByronBlock]
forall a. HasCallStack => TestName -> a
error TestName
"testGenTxs: unexpected invalid tx"

-- | Generate transaction
--
-- For now we only generate regular transactions. Generating delegation
-- certificates and update proposals/votes is out of the scope of this test,
-- for now. Extending the scope will require integration with the restart/rekey
-- infrastructure of the Byron tests.
genTx ::
  TopLevelConfig DualByronBlock ->
  TickedLedgerState DualByronBlock ValuesMK ->
  Gen (GenTx DualByronBlock)
genTx :: TopLevelConfig DualByronBlock
-> TickedLedgerState DualByronBlock ValuesMK
-> Gen (GenTx DualByronBlock)
genTx TopLevelConfig DualByronBlock
cfg TickedLedgerState DualByronBlock ValuesMK
st = do
  aux <- RuleContext UTXOW -> State CHAIN -> Gen (Signal UTXOW)
forall sts.
HasTrace sts =>
RuleContext sts -> State CHAIN -> Gen (Signal sts)
sigGen (ByronSpecGenesis -> RuleContext UTXOW
Rules.ctxtUTXOW ByronSpecGenesis
cfg') State CHAIN
st'
  let main :: Impl.ATxAux ByteString
      main =
        (TxId -> TxId) -> Tx -> ATxAux ByteString
Spec.Test.elaborateTxBS
          TxId -> TxId
elaborateTxId
          Tx
aux

  return $
    DualGenTx
      { dualGenTxMain = ByronTx (byronIdTx main) main
      , dualGenTxAux = ByronSpecGenTx $ ByronSpecGenTxTx aux
      , dualGenTxBridge = specToImplTx aux main
      }
 where
  cfg' :: ByronSpecGenesis
  st' :: Spec.State Spec.CHAIN

  cfg' :: ByronSpecGenesis
cfg' = DualLedgerConfig ByronBlock ByronSpecBlock
-> LedgerConfig ByronSpecBlock
forall m a. DualLedgerConfig m a -> LedgerConfig a
dualLedgerConfigAux (TopLevelConfig DualByronBlock
-> LedgerCfg (LedgerState DualByronBlock)
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig DualByronBlock
cfg)
  st' :: State CHAIN
st' = Ticked (LedgerState ByronSpecBlock) ValuesMK -> State CHAIN
forall (mk :: MapKind).
Ticked (LedgerState ByronSpecBlock) mk -> State CHAIN
tickedByronSpecLedgerState (Ticked (LedgerState ByronSpecBlock) ValuesMK -> State CHAIN)
-> Ticked (LedgerState ByronSpecBlock) ValuesMK -> State CHAIN
forall a b. (a -> b) -> a -> b
$ TickedLedgerState DualByronBlock ValuesMK
-> Ticked (LedgerState ByronSpecBlock) ValuesMK
forall m a (mk :: MapKind).
Ticked (LedgerState (DualBlock m a)) mk
-> Ticked (LedgerState a) ValuesMK
tickedDualLedgerStateAux TickedLedgerState DualByronBlock ValuesMK
st

  bridge :: ByronSpecBridge
  bridge :: ByronSpecBridge
bridge = TickedLedgerState DualByronBlock ValuesMK
-> BridgeLedger ByronBlock ByronSpecBlock
forall m a (mk :: MapKind).
Ticked (LedgerState (DualBlock m a)) mk -> BridgeLedger m a
tickedDualLedgerStateBridge TickedLedgerState DualByronBlock ValuesMK
st

  elaborateTxId :: Spec.TxId -> Impl.TxId
  elaborateTxId :: TxId -> TxId
elaborateTxId TxId
tid =
    case TxId -> Map TxId TxId -> Maybe TxId
forall k a. Ord k => k -> Map k a -> Maybe a
Map.lookup TxId
tid (BridgeLedger ByronBlock ByronSpecBlock -> Map TxId TxId
bridgeTransactionIds BridgeLedger ByronBlock ByronSpecBlock
ByronSpecBridge
bridge) of
      Maybe TxId
Nothing -> TestName -> TxId
forall a. HasCallStack => TestName -> a
error (TestName -> TxId) -> TestName -> TxId
forall a b. (a -> b) -> a -> b
$ TestName
"elaborateTxId: unknown tx ID " TestName -> ShowS
forall a. [a] -> [a] -> [a]
++ TxId -> TestName
forall a. Show a => a -> TestName
show TxId
tid
      Just TxId
tid' -> TxId
tid'

sigGen ::
  forall sts.
  Spec.QC.HasTrace sts =>
  Rules.RuleContext sts ->
  Spec.State Spec.CHAIN ->
  Gen (Spec.Signal sts)
sigGen :: forall sts.
HasTrace sts =>
RuleContext sts -> State CHAIN -> Gen (Signal sts)
sigGen Rules.RuleContext{PredicateFailure sts -> PredicateFailure CHAIN
GetChainState (State sts)
State CHAIN -> Environment sts
ModChainState (State sts)
getRuleState :: GetChainState (State sts)
modRuleState :: ModChainState (State sts)
liftFailure :: PredicateFailure sts -> PredicateFailure CHAIN
getRuleEnv :: State CHAIN -> Environment sts
getRuleEnv :: forall sts. RuleContext sts -> State CHAIN -> Environment sts
liftFailure :: forall sts.
RuleContext sts -> PredicateFailure sts -> PredicateFailure CHAIN
modRuleState :: forall sts. RuleContext sts -> ModChainState (State sts)
getRuleState :: forall sts. RuleContext sts -> GetChainState (State sts)
..} State CHAIN
st =
  Gen (Signal sts) -> Gen (Signal sts)
forall a. Gen a -> Gen a
hedgehog (Gen (Signal sts) -> Gen (Signal sts))
-> Gen (Signal sts) -> Gen (Signal sts)
forall a b. (a -> b) -> a -> b
$
    -- Convert Hedgehog generator to QuickCheck one
    -- Unfortunately, this does mean we lose any shrinking.
    forall s. HasTrace s => SignalGenerator s
Spec.QC.sigGen @sts (State CHAIN -> Environment sts
getRuleEnv State CHAIN
st) (GetChainState (State sts)
getRuleState State CHAIN
st)