{-# 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           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.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
setupGenesis :: SetupDualByron -> ByronSpecGenesis
setupByron :: SetupDualByron -> TestSetup
..}  = 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)
setupEBBs :: TestSetup -> ProduceEBBs
setupK :: TestSetup -> SecurityParam
setupTestConfig :: TestSetup -> TestConfig
setupNodeJoinPlan :: TestSetup -> NodeJoinPlan
setupNodeRestarts :: TestSetup -> NodeRestarts
setupSlotLength :: TestSetup -> SlotLength
setupVersion :: TestSetup -> (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
..} = TestSetup
setupByron
    TestConfig{NumCoreNodes
NumSlots
NodeTopology
Seed
numCoreNodes :: NumCoreNodes
numSlots :: NumSlots
initSeed :: Seed
nodeTopology :: NodeTopology
initSeed :: TestConfig -> Seed
nodeTopology :: TestConfig -> NodeTopology
numCoreNodes :: TestConfig -> NumCoreNodes
numSlots :: TestConfig -> NumSlots
..}      = 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
setupGenesis :: SetupDualByron -> ByronSpecGenesis
setupByron :: SetupDualByron -> TestSetup
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
setupEBBs :: TestSetup -> ProduceEBBs
setupK :: TestSetup -> SecurityParam
setupTestConfig :: TestSetup -> TestConfig
setupNodeJoinPlan :: TestSetup -> NodeJoinPlan
setupNodeRestarts :: TestSetup -> NodeRestarts
setupSlotLength :: TestSetup -> SlotLength
setupVersion :: TestSetup -> (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
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
setupGenesis :: SetupDualByron -> ByronSpecGenesis
setupByron :: SetupDualByron -> TestSetup
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
setupGenesis :: SetupDualByron -> ByronSpecGenesis
setupByron :: SetupDualByron -> TestSetup
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
setupEBBs :: TestSetup -> ProduceEBBs
setupK :: TestSetup -> SecurityParam
setupTestConfig :: TestSetup -> TestConfig
setupNodeJoinPlan :: TestSetup -> NodeJoinPlan
setupNodeRestarts :: TestSetup -> NodeRestarts
setupSlotLength :: TestSetup -> SlotLength
setupVersion :: TestSetup -> (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
setupK :: SecurityParam
setupNodeRestarts :: NodeRestarts
setupEBBs :: ProduceEBBs
setupTestConfig :: TestConfig
setupNodeJoinPlan :: NodeJoinPlan
setupSlotLength :: SlotLength
setupVersion :: (NodeToNodeVersion, BlockNodeToNodeVersion ByronBlock)
..} = TestSetup
setupByron
    TestConfig{NumCoreNodes
NumSlots
NodeTopology
Seed
initSeed :: TestConfig -> Seed
nodeTopology :: TestConfig -> NodeTopology
numCoreNodes :: TestConfig -> NumCoreNodes
numSlots :: TestConfig -> NumSlots
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
numSlots <- Gen NumSlots
forall a. Arbitrary a => Gen a
arbitrary
      SlotLength
slotLen  <- Gen SlotLength
forall a. Arbitrary a => Gen a
arbitrary

      ByronSpecGenesis
genesis0                 <- SlotLength -> NumSlots -> Gen ByronSpecGenesis
genSpecGenesis SlotLength
slotLen NumSlots
numSlots
      let params :: PBftParams
params@PBftParams{SecurityParam
NumCoreNodes
PBftSignatureThreshold
pbftSecurityParam :: SecurityParam
pbftNumNodes :: NumCoreNodes
pbftSignatureThreshold :: PBftSignatureThreshold
pbftSecurityParam :: PBftParams -> SecurityParam
pbftNumNodes :: PBftParams -> NumCoreNodes
pbftSignatureThreshold :: PBftParams -> PBftSignatureThreshold
..} = ByronSpecGenesis -> PBftParams
byronPBftParams ByronSpecGenesis
genesis0
          setupGenesis :: ByronSpecGenesis
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..).
      TestSetup
setupByron <-
        (\TestSetup
x -> TestSetup
x{Byron.setupNodeRestarts = noRestarts})
        (TestSetup -> TestSetup) -> Gen TestSetup -> Gen TestSetup
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> SecurityParam
-> NumCoreNodes -> NumSlots -> SlotLength -> Gen TestSetup
Byron.genTestSetup
              SecurityParam
pbftSecurityParam
              NumCoreNodes
pbftNumNodes
              NumSlots
numSlots
              SlotLength
slotLen

      SetupDualByron -> Gen SetupDualByron
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return SetupDualByron{ByronSpecGenesis
TestSetup
setupGenesis :: ByronSpecGenesis
setupByron :: TestSetup
setupGenesis :: ByronSpecGenesis
setupByron :: TestSetup
..}
    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
byronSpecGenesisDelegators :: ByronSpecGenesis -> Set VKeyGenesis
byronSpecGenesisInitUtxo :: ByronSpecGenesis -> UTxO
byronSpecGenesisInitPParams :: ByronSpecGenesis -> PParams
byronSpecGenesisSecurityParam :: ByronSpecGenesis -> BlockCount
byronSpecGenesisSlotLength :: ByronSpecGenesis -> Natural
..} =
    SecurityParam -> NumCoreNodes -> PBftParams
Byron.byronPBftParams (Word64 -> SecurityParam
SecurityParam Word64
k) NumCoreNodes
numCoreNodes
  where
    Spec.BlockCount Word64
k = BlockCount
byronSpecGenesisSecurityParam

    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
-> Gen [GenTx DualByronBlock]
testGenTxs CoreNodeId
_coreNodeId NumCoreNodes
_numCoreNodes SlotNo
curSlotNo TopLevelConfig DualByronBlock
cfg () = \LedgerState DualByronBlock
st -> do
      Integer
n <- (Integer, Integer) -> Gen Integer
forall a. Random a => (a, a) -> Gen a
choose (Integer
0, Integer
20)
      [GenTx DualByronBlock]
-> Integer
-> TickedLedgerState DualByronBlock
-> Gen [GenTx DualByronBlock]
go [] Integer
n (TickedLedgerState DualByronBlock -> Gen [GenTx DualByronBlock])
-> TickedLedgerState DualByronBlock -> Gen [GenTx DualByronBlock]
forall a b. (a -> b) -> a -> b
$ LedgerCfg (LedgerState DualByronBlock)
-> SlotNo
-> LedgerState DualByronBlock
-> TickedLedgerState DualByronBlock
forall l. IsLedger l => LedgerCfg l -> SlotNo -> l -> Ticked l
applyChainTick (TopLevelConfig DualByronBlock
-> LedgerCfg (LedgerState DualByronBlock)
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig DualByronBlock
cfg) SlotNo
curSlotNo LedgerState DualByronBlock
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
         -> Gen [GenTx DualByronBlock]
      go :: [GenTx DualByronBlock]
-> Integer
-> TickedLedgerState DualByronBlock
-> Gen [GenTx DualByronBlock]
go [GenTx DualByronBlock]
acc Integer
0 TickedLedgerState DualByronBlock
_  = [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
st = do
          GenTx DualByronBlock
tx <- TopLevelConfig DualByronBlock
-> TickedLedgerState DualByronBlock -> Gen (GenTx DualByronBlock)
genTx TopLevelConfig DualByronBlock
cfg TickedLedgerState DualByronBlock
st
          case Except
  (DualGenTxErr ByronBlock ByronSpecBlock)
  (TickedLedgerState DualByronBlock,
   Validated (GenTx DualByronBlock))
-> Either
     (DualGenTxErr ByronBlock ByronSpecBlock)
     (TickedLedgerState DualByronBlock,
      Validated (GenTx DualByronBlock))
forall e a. Except e a -> Either e a
runExcept (Except
   (DualGenTxErr ByronBlock ByronSpecBlock)
   (TickedLedgerState DualByronBlock,
    Validated (GenTx DualByronBlock))
 -> Either
      (DualGenTxErr ByronBlock ByronSpecBlock)
      (TickedLedgerState DualByronBlock,
       Validated (GenTx DualByronBlock)))
-> Except
     (DualGenTxErr ByronBlock ByronSpecBlock)
     (TickedLedgerState DualByronBlock,
      Validated (GenTx DualByronBlock))
-> Either
     (DualGenTxErr ByronBlock ByronSpecBlock)
     (TickedLedgerState DualByronBlock,
      Validated (GenTx DualByronBlock))
forall a b. (a -> b) -> a -> b
$ LedgerCfg (LedgerState DualByronBlock)
-> WhetherToIntervene
-> SlotNo
-> GenTx DualByronBlock
-> TickedLedgerState DualByronBlock
-> Except
     (ApplyTxErr DualByronBlock)
     (TickedLedgerState DualByronBlock,
      Validated (GenTx DualByronBlock))
forall blk.
LedgerSupportsMempool blk =>
LedgerConfig blk
-> WhetherToIntervene
-> SlotNo
-> GenTx blk
-> TickedLedgerState blk
-> Except
     (ApplyTxErr blk) (TickedLedgerState blk, Validated (GenTx blk))
applyTx
                             (TopLevelConfig DualByronBlock
-> LedgerCfg (LedgerState DualByronBlock)
forall blk. TopLevelConfig blk -> LedgerConfig blk
configLedger TopLevelConfig DualByronBlock
cfg)
                             WhetherToIntervene
DoNotIntervene
                             SlotNo
curSlotNo
                             GenTx DualByronBlock
tx
                             TickedLedgerState DualByronBlock
st of
            Right (TickedLedgerState DualByronBlock
st', Validated (GenTx DualByronBlock)
_vtx) -> [GenTx DualByronBlock]
-> Integer
-> TickedLedgerState DualByronBlock
-> Gen [GenTx DualByronBlock]
go (GenTx DualByronBlock
txGenTx 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
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
      -> Ticked (LedgerState DualByronBlock)
      -> Gen (GenTx DualByronBlock)
genTx :: TopLevelConfig DualByronBlock
-> TickedLedgerState DualByronBlock -> Gen (GenTx DualByronBlock)
genTx TopLevelConfig DualByronBlock
cfg TickedLedgerState DualByronBlock
st = do
    Tx
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 :: ATxAux ByteString
main = (TxId -> TxId) -> Tx -> ATxAux ByteString
Spec.Test.elaborateTxBS
                 TxId -> TxId
elaborateTxId
                 Tx
aux

    GenTx DualByronBlock -> Gen (GenTx DualByronBlock)
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return (GenTx DualByronBlock -> Gen (GenTx DualByronBlock))
-> GenTx DualByronBlock -> Gen (GenTx DualByronBlock)
forall a b. (a -> b) -> a -> b
$ DualGenTx {
        dualGenTxMain :: GenTx ByronBlock
dualGenTxMain   = TxId -> ATxAux ByteString -> GenTx ByronBlock
ByronTx (ATxAux ByteString -> TxId
byronIdTx ATxAux ByteString
main) ATxAux ByteString
main
      , dualGenTxAux :: GenTx ByronSpecBlock
dualGenTxAux    = ByronSpecGenTx -> GenTx ByronSpecBlock
ByronSpecGenTx (ByronSpecGenTx -> GenTx ByronSpecBlock)
-> ByronSpecGenTx -> GenTx ByronSpecBlock
forall a b. (a -> b) -> a -> b
$ Tx -> ByronSpecGenTx
ByronSpecGenTxTx Tx
aux
      , dualGenTxBridge :: BridgeTx ByronBlock ByronSpecBlock
dualGenTxBridge = Tx -> ATxAux ByteString -> SpecToImplIds
specToImplTx Tx
aux ATxAux ByteString
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) -> State CHAIN
tickedByronSpecLedgerState (Ticked (LedgerState ByronSpecBlock) -> State CHAIN)
-> Ticked (LedgerState ByronSpecBlock) -> State CHAIN
forall a b. (a -> b) -> a -> b
$ TickedLedgerState DualByronBlock
-> Ticked (LedgerState ByronSpecBlock)
forall m a.
Ticked (LedgerState (DualBlock m a)) -> Ticked (LedgerState a)
tickedDualLedgerStateAux TickedLedgerState DualByronBlock
st

    bridge :: ByronSpecBridge
    bridge :: ByronSpecBridge
bridge = TickedLedgerState DualByronBlock
-> BridgeLedger ByronBlock ByronSpecBlock
forall m a.
Ticked (LedgerState (DualBlock m a)) -> BridgeLedger m a
tickedDualLedgerStateBridge TickedLedgerState DualByronBlock
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
getRuleState :: forall sts. RuleContext sts -> GetChainState (State sts)
modRuleState :: forall sts. RuleContext sts -> ModChainState (State sts)
liftFailure :: forall sts.
RuleContext sts -> PredicateFailure sts -> PredicateFailure CHAIN
getRuleEnv :: forall sts. RuleContext sts -> State CHAIN -> Environment 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)