{-# LANGUAGE GADTs #-}
{-# LANGUAGE NamedFieldPuns #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}

module Test.ThreadNet.BFT (tests) where

import           Data.Constraint
import           Ouroboros.Consensus.Block
import           Ouroboros.Consensus.BlockchainTime
import           Ouroboros.Consensus.Config.SecurityParam
import qualified Ouroboros.Consensus.HardFork.History as HardFork
import           Ouroboros.Consensus.Mock.Ledger
import           Ouroboros.Consensus.Mock.Node ()
import           Ouroboros.Consensus.Mock.Node.BFT
import           Ouroboros.Consensus.Mock.Node.Serialisation
import           Test.Consensus.Ledger.Mock.Generators ()
import           Test.QuickCheck
import           Test.Tasty
import           Test.Tasty.QuickCheck
import           Test.ThreadNet.General
import           Test.ThreadNet.TxGen.Mock ()
import           Test.ThreadNet.Util
import           Test.ThreadNet.Util.NodeJoinPlan
import           Test.ThreadNet.Util.NodeRestarts
import           Test.ThreadNet.Util.NodeToNodeVersion
import           Test.ThreadNet.Util.SimpleBlock
import           Test.Util.HardFork.Future (singleEraFuture)
import           Test.Util.Orphans.Arbitrary ()
import           Test.Util.Serialisation.Roundtrip
import           Test.Util.Serialisation.SomeResult ()

data TestSetup = TestSetup
  { TestSetup -> SecurityParam
setupK            :: SecurityParam
  , TestSetup -> TestConfig
setupTestConfig   :: TestConfig
  , TestSetup -> NodeJoinPlan
setupNodeJoinPlan :: NodeJoinPlan
  }
  deriving (Int -> TestSetup -> ShowS
[TestSetup] -> ShowS
TestSetup -> String
(Int -> TestSetup -> ShowS)
-> (TestSetup -> String)
-> ([TestSetup] -> ShowS)
-> Show TestSetup
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> TestSetup -> ShowS
showsPrec :: Int -> TestSetup -> ShowS
$cshow :: TestSetup -> String
show :: TestSetup -> String
$cshowList :: [TestSetup] -> ShowS
showList :: [TestSetup] -> ShowS
Show)

instance Arbitrary TestSetup where
  arbitrary :: Gen TestSetup
arbitrary = do
      -- TODO k > 1 as a workaround for Issue #1511.
      SecurityParam
k <- 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
2, Word64
10)

      TestConfig
testConfig <- Gen TestConfig
forall a. Arbitrary a => Gen a
arbitrary
      let TestConfig{NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes :: TestConfig -> NumCoreNodes
numCoreNodes, NumSlots
numSlots :: NumSlots
numSlots :: TestConfig -> NumSlots
numSlots} = TestConfig
testConfig

      NodeJoinPlan
nodeJoinPlan <- NumCoreNodes -> NumSlots -> Gen NodeJoinPlan
genNodeJoinPlan NumCoreNodes
numCoreNodes NumSlots
numSlots

      TestSetup -> Gen TestSetup
forall a. a -> Gen a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (TestSetup -> Gen TestSetup) -> TestSetup -> Gen TestSetup
forall a b. (a -> b) -> a -> b
$ SecurityParam -> TestConfig -> NodeJoinPlan -> TestSetup
TestSetup SecurityParam
k TestConfig
testConfig NodeJoinPlan
nodeJoinPlan

  -- TODO shrink

tests :: TestTree
tests :: TestTree
tests = String -> [TestTree] -> TestTree
testGroup String
"BFT" ([TestTree] -> TestTree) -> [TestTree] -> TestTree
forall a b. (a -> b) -> a -> b
$
    [ CodecConfig MockBftBlock
-> (forall a.
    NestedCtxt_ MockBftBlock Header a -> Dict (Eq a, Show a))
-> TestTree
forall blk.
(SerialiseDiskConstraints blk, SerialiseNodeToNodeConstraints blk,
 SerialiseNodeToClientConstraints blk,
 Show (BlockNodeToNodeVersion blk),
 Show (BlockNodeToClientVersion blk), StandardHash blk,
 GetHeader blk, Arbitrary' blk, Arbitrary' (Header blk),
 Arbitrary' (HeaderHash blk), Arbitrary' (LedgerState blk),
 Arbitrary' (AnnTip blk),
 Arbitrary' (ChainDepState (BlockProtocol blk)),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Coherent blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (Header blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTx blk),
 ArbitraryWithVersion (BlockNodeToNodeVersion blk) (GenTxId blk),
 ArbitraryWithVersion
   (BlockNodeToNodeVersion blk) (SomeSecond (NestedCtxt Header) blk),
 ArbitraryWithVersion (BlockNodeToClientVersion blk) blk,
 ArbitraryWithVersion (BlockNodeToClientVersion blk) (GenTx blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (ApplyTxErr blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeSecond BlockQuery blk),
 ArbitraryWithVersion
   (BlockNodeToClientVersion blk) (SomeResult blk),
 ArbitraryWithVersion
   (QueryVersion, BlockNodeToClientVersion blk)
   (SomeSecond Query blk)) =>
CodecConfig blk
-> (forall a. NestedCtxt_ blk Header a -> Dict (Eq a, Show a))
-> TestTree
roundtrip_all CodecConfig MockBftBlock
forall c ext. CodecConfig (SimpleBlock c ext)
SimpleCodecConfig NestedCtxt_ MockBftBlock Header a -> Dict (Eq a, Show a)
forall a. NestedCtxt_ MockBftBlock Header a -> Dict (Eq a, Show a)
dictNestedHdr
    , String -> (TestSetup -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"simple convergence" ((TestSetup -> Property) -> TestTree)
-> (TestSetup -> Property) -> TestTree
forall a b. (a -> b) -> a -> b
$ \TestSetup
setup ->
        TestSetup -> Property
prop_simple_bft_convergence TestSetup
setup
    ]
  where
    dictNestedHdr :: forall a. NestedCtxt_ MockBftBlock Header a -> Dict (Eq a, Show a)
    dictNestedHdr :: forall a. NestedCtxt_ MockBftBlock Header a -> Dict (Eq a, Show a)
dictNestedHdr NestedCtxt_ MockBftBlock Header a
R:NestedCtxt_SimpleBlock'fa
  SimpleMockCrypto
  (SimpleBftExt SimpleMockCrypto BftMockCrypto)
  Header
  a
CtxtMock = Dict (Eq a, Show a)
forall (a :: Constraint). a => Dict a
Dict

prop_simple_bft_convergence :: TestSetup -> Property
prop_simple_bft_convergence :: TestSetup -> Property
prop_simple_bft_convergence TestSetup
  { setupK :: TestSetup -> SecurityParam
setupK            = SecurityParam
k
  , setupTestConfig :: TestSetup -> TestConfig
setupTestConfig   = TestConfig
testConfig
  , setupNodeJoinPlan :: TestSetup -> NodeJoinPlan
setupNodeJoinPlan = NodeJoinPlan
nodeJoinPlan
  } =
    PropGeneralArgs MockBftBlock -> TestOutput MockBftBlock -> Property
forall blk.
(Condense blk, Condense (HeaderHash blk), Eq blk, RunNode blk) =>
PropGeneralArgs blk -> TestOutput blk -> Property
prop_general PropGeneralArgs
      { pgaBlockProperty :: MockBftBlock -> Property
pgaBlockProperty       = MockBftBlock -> Property
forall c ext ext'.
(SimpleCrypto c, Typeable ext, Typeable ext') =>
SimpleBlock' c ext ext' -> Property
prop_validSimpleBlock
      , pgaCountTxs :: MockBftBlock -> Word64
pgaCountTxs            = MockBftBlock -> Word64
forall c ext. SimpleBlock c ext -> Word64
countSimpleGenTxs
      , pgaExpectedCannotForge :: SlotNo -> NodeId -> WrapCannotForge MockBftBlock -> Bool
pgaExpectedCannotForge = SlotNo -> NodeId -> WrapCannotForge MockBftBlock -> Bool
forall blk. SlotNo -> NodeId -> WrapCannotForge blk -> Bool
noExpectedCannotForges
      , pgaFirstBlockNo :: BlockNo
pgaFirstBlockNo        = BlockNo
0
      , pgaFixedMaxForkLength :: Maybe NumBlocks
pgaFixedMaxForkLength  = Maybe NumBlocks
forall a. Maybe a
Nothing
      , 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
k
      , pgaTestConfig :: TestConfig
pgaTestConfig          = TestConfig
testConfig
      , pgaTestConfigB :: TestConfigB MockBftBlock
pgaTestConfigB         = TestConfigB MockBftBlock
testConfigB
      }
      TestOutput MockBftBlock
testOutput
  where
    TestConfig{NumCoreNodes
numCoreNodes :: TestConfig -> NumCoreNodes
numCoreNodes :: NumCoreNodes
numCoreNodes, NumSlots
numSlots :: TestConfig -> NumSlots
numSlots :: NumSlots
numSlots} = TestConfig
testConfig
    slotLength :: SlotLength
slotLength = Integer -> SlotLength
slotLengthFromSec Integer
20
    testConfigB :: TestConfigB MockBftBlock
testConfigB = TestConfigB
      { forgeEbbEnv :: Maybe (ForgeEbbEnv MockBftBlock)
forgeEbbEnv = Maybe (ForgeEbbEnv MockBftBlock)
forall a. Maybe a
Nothing
      , future :: Future
future      = SlotLength -> EpochSize -> Future
singleEraFuture
          SlotLength
slotLength
          (Word64 -> EpochSize
EpochSize (Word64 -> EpochSize) -> Word64 -> EpochSize
forall a b. (a -> b) -> a -> b
$ SecurityParam -> Word64
maxRollbacks SecurityParam
k Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
* Word64
10)
          -- The mock ledger doesn't really care, and neither does BFT. We
          -- stick with the common @k * 10@ size for now.
      , messageDelay :: CalcMessageDelay MockBftBlock
messageDelay = CalcMessageDelay MockBftBlock
forall blk. CalcMessageDelay blk
noCalcMessageDelay
      , NodeJoinPlan
nodeJoinPlan :: NodeJoinPlan
nodeJoinPlan :: NodeJoinPlan
nodeJoinPlan
      , nodeRestarts :: NodeRestarts
nodeRestarts = NodeRestarts
noRestarts
      , txGenExtra :: TxGenExtra MockBftBlock
txGenExtra   = ()
      , version :: (NodeToNodeVersion, BlockNodeToNodeVersion MockBftBlock)
version      = Proxy MockBftBlock
-> (NodeToNodeVersion, BlockNodeToNodeVersion MockBftBlock)
forall blk.
SupportedNetworkProtocolVersion blk =>
Proxy blk -> (NodeToNodeVersion, BlockNodeToNodeVersion blk)
newestVersion (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @MockBftBlock)
      }

    testOutput :: TestOutput MockBftBlock
testOutput =
        TestConfig
-> TestConfigB MockBftBlock
-> (forall (m :: * -> *). IOLike m => TestConfigMB m MockBftBlock)
-> TestOutput MockBftBlock
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 MockBftBlock
testConfigB TestConfigMB
            { nodeInfo :: CoreNodeId -> TestNodeInitialization m MockBftBlock
nodeInfo = \CoreNodeId
nid ->
                ProtocolInfo MockBftBlock
-> m [BlockForging m MockBftBlock]
-> TestNodeInitialization m MockBftBlock
forall blk (m :: * -> *).
ProtocolInfo blk
-> m [BlockForging m blk] -> TestNodeInitialization m blk
plainTestNodeInitialization
                  (NumCoreNodes
-> CoreNodeId
-> SecurityParam
-> EraParams
-> ProtocolInfo MockBftBlock
protocolInfoBft
                    NumCoreNodes
numCoreNodes
                    CoreNodeId
nid
                    SecurityParam
k
                    (SecurityParam -> SlotLength -> EraParams
HardFork.defaultEraParams SecurityParam
k SlotLength
slotLength))
                  ([BlockForging m MockBftBlock] -> m [BlockForging m MockBftBlock]
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ([BlockForging m MockBftBlock] -> m [BlockForging m MockBftBlock])
-> [BlockForging m MockBftBlock] -> m [BlockForging m MockBftBlock]
forall a b. (a -> b) -> a -> b
$ CoreNodeId -> [BlockForging m MockBftBlock]
forall (m :: * -> *).
Monad m =>
CoreNodeId -> [BlockForging m MockBftBlock]
blockForgingBft CoreNodeId
nid)
            , mkRekeyM :: Maybe (m (RekeyM m MockBftBlock))
mkRekeyM = Maybe (m (RekeyM m MockBftBlock))
forall a. Maybe a
Nothing
            }