{-# LANGUAGE CPP #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE TypeApplications #-}
{-# OPTIONS_GHC -Wno-incomplete-uni-patterns #-}
{-# OPTIONS_GHC -Wno-orphans #-}
#if __GLASGOW_HASKELL__ >= 908
{-# OPTIONS_GHC -Wno-x-partial #-}
#endif

-- | Primary index tests.
--
-- This is a sequence of relatively simple property tests:
--
-- * Writing a primary index to disk and then reading it again is an identity
--   operation (@prop_write_load@)
-- * We can create new primary indices by appending new entries to them
--   (@prop_open_appendOffsets_load@)
-- * We can truncate primary indices to particular slot.
-- * Finding and reporting "filled slots" (not all slots in a chunk file, and
--   hence in a primary index, need to contain a block) works as expected.
-- * Reconstructing a primary index from the same data results in the same
--   primary index.
--
-- These property tests are QuickCheck based, which means they generate random
-- indices, random slot numbers, etc., and come with a proper shrinker.
module Test.Ouroboros.Storage.ImmutableDB.Primary (tests) where

import Data.Functor ((<&>))
import Data.Maybe (fromJust)
import Data.Proxy (Proxy (..))
import Data.Vector.Unboxed (Vector)
import qualified Data.Vector.Unboxed as V
import Ouroboros.Consensus.Block (ConvertRawHash (..))
import Ouroboros.Consensus.Storage.ImmutableDB.Chunks
import Ouroboros.Consensus.Storage.ImmutableDB.Chunks.Internal
  ( RelativeSlot (..)
  , maxRelativeIndex
  )
import Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Primary
  ( PrimaryIndex
  , SecondaryOffset
  )
import qualified Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Primary as Primary
import qualified Ouroboros.Consensus.Storage.ImmutableDB.Impl.Index.Secondary as Secondary
import Ouroboros.Consensus.Storage.ImmutableDB.Impl.Types (BlockOrEBB)
import Ouroboros.Consensus.Storage.ImmutableDB.Impl.Validation
  ( ShouldBeFinalised (..)
  , reconstructPrimaryIndex
  )
import Ouroboros.Consensus.Util.IOLike (try)
import System.FS.API hiding (allowExisting)
import System.FS.Sim.MockFS (HandleMock)
import qualified System.FS.Sim.MockFS as Mock
import qualified System.FS.Sim.STM as Sim
import Test.Ouroboros.Storage.TestBlock (TestBlock)
import Test.QuickCheck
import Test.QuickCheck.Monadic (monadicIO, run)
import Test.Tasty (TestTree, testGroup)
import Test.Tasty.QuickCheck (testProperty)
import Test.Util.Orphans.Arbitrary ()

{------------------------------------------------------------------------------
  The tests
------------------------------------------------------------------------------}

tests :: TestTree
tests :: TestTree
tests =
  String -> [TestTree] -> TestTree
testGroup
    String
"PrimaryIndex"
    [ String -> (TestPrimaryIndex -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"isFilledSlot iff in filledSlots" TestPrimaryIndex -> Property
prop_filledSlots_isFilledSlot
    , String -> (TestPrimaryIndex -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"write/load roundtrip" TestPrimaryIndex -> Property
prop_write_load
    , String -> (TestPrimaryIndex -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"open+appendOffsets/load roundtrip" TestPrimaryIndex -> Property
prop_open_appendOffsets_load
    , String -> (TestPrimaryIndex -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"truncateToSlotFS/truncateToSlot" TestPrimaryIndex -> Property
prop_truncateToSlotFS_truncateToSlot
    , String -> (TestPrimaryIndex -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty
        String
"readFirstFilledSlot/load+firstFilledSlot"
        TestPrimaryIndex -> Property
prop_readFirstFilledSlot_load_firstFilledSlot
    , String -> (TestPrimaryIndex -> Property) -> TestTree
forall a. Testable a => String -> a -> TestTree
testProperty String
"reconstructPrimaryIndex" TestPrimaryIndex -> Property
prop_reconstructPrimaryIndex
    ]

{------------------------------------------------------------------------------
  The properties
------------------------------------------------------------------------------}

prop_filledSlots_isFilledSlot :: TestPrimaryIndex -> Property
prop_filledSlots_isFilledSlot :: TestPrimaryIndex -> Property
prop_filledSlots_isFilledSlot (TestPrimaryIndex ChunkInfo
chunkInfo ChunkNo
chunk PrimaryIndex
idx RelativeSlot
_slot) =
  [Property] -> Property
forall prop. Testable prop => [prop] -> Property
conjoin
    [ HasCallStack => PrimaryIndex -> RelativeSlot -> Bool
PrimaryIndex -> RelativeSlot -> Bool
Primary.isFilledSlot PrimaryIndex
idx RelativeSlot
slot
        Bool -> Bool -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== (RelativeSlot
slot RelativeSlot -> [RelativeSlot] -> Bool
forall a. Eq a => a -> [a] -> Bool
forall (t :: * -> *) a. (Foldable t, Eq a) => a -> t a -> Bool
`elem` ChunkInfo -> PrimaryIndex -> [RelativeSlot]
Primary.filledSlots ChunkInfo
chunkInfo PrimaryIndex
idx)
    | RelativeSlot
slot <- [RelativeSlot]
slots
    ]
 where
  slots :: [RelativeSlot]
  slots :: [RelativeSlot]
slots
    | Word64
totalSlots Word64 -> Word64 -> Bool
forall a. Eq a => a -> a -> Bool
== Word64
0 = []
    | Bool
otherwise = (Word64 -> RelativeSlot) -> [Word64] -> [RelativeSlot]
forall a b. (a -> b) -> [a] -> [b]
map (HasCallStack => ChunkInfo -> ChunkNo -> Word64 -> RelativeSlot
ChunkInfo -> ChunkNo -> Word64 -> RelativeSlot
mkRelativeSlot ChunkInfo
chunkInfo ChunkNo
chunk) [Word64
0 .. Word64
totalSlots Word64 -> Word64 -> Word64
forall a. Num a => a -> a -> a
- Word64
1]
  totalSlots :: Word64
totalSlots = PrimaryIndex -> Word64
Primary.slots PrimaryIndex
idx

prop_write_load :: TestPrimaryIndex -> Property
prop_write_load :: TestPrimaryIndex -> Property
prop_write_load (TestPrimaryIndex ChunkInfo
_chunkInfo ChunkNo
chunk PrimaryIndex
index RelativeSlot
_slot) =
  PropertyM IO Property -> Property
forall a. Testable a => PropertyM IO a -> Property
monadicIO (PropertyM IO Property -> Property)
-> PropertyM IO Property -> Property
forall a b. (a -> b) -> a -> b
$ IO Property -> PropertyM IO Property
forall (m :: * -> *) a. Monad m => m a -> PropertyM m a
run (IO Property -> PropertyM IO Property)
-> IO Property -> PropertyM IO Property
forall a b. (a -> b) -> a -> b
$ (HasFS IO HandleMock -> IO Property) -> IO Property
runFS HasFS IO HandleMock -> IO Property
forall h. HasFS IO h -> IO Property
prop
 where
  prop :: HasFS IO h -> IO Property
  prop :: forall h. HasFS IO h -> IO Property
prop HasFS IO h
hasFS = do
    HasFS IO h -> ChunkNo -> PrimaryIndex -> IO ()
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> ChunkNo -> PrimaryIndex -> m ()
Primary.write HasFS IO h
hasFS ChunkNo
chunk PrimaryIndex
index
    index' <- Proxy TestBlock -> HasFS IO h -> ChunkNo -> IO PrimaryIndex
forall blk (m :: * -> *) h.
(HasCallStack, MonadThrow m, StandardHash blk, Typeable blk) =>
Proxy blk -> HasFS m h -> ChunkNo -> m PrimaryIndex
Primary.load (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @TestBlock) HasFS IO h
hasFS ChunkNo
chunk
    return $ index === index'

prop_open_appendOffsets_load :: TestPrimaryIndex -> Property
prop_open_appendOffsets_load :: TestPrimaryIndex -> Property
prop_open_appendOffsets_load (TestPrimaryIndex ChunkInfo
_chunkInfo ChunkNo
chunk PrimaryIndex
index RelativeSlot
_slot) =
  PropertyM IO Property -> Property
forall a. Testable a => PropertyM IO a -> Property
monadicIO (PropertyM IO Property -> Property)
-> PropertyM IO Property -> Property
forall a b. (a -> b) -> a -> b
$ IO Property -> PropertyM IO Property
forall (m :: * -> *) a. Monad m => m a -> PropertyM m a
run (IO Property -> PropertyM IO Property)
-> IO Property -> PropertyM IO Property
forall a b. (a -> b) -> a -> b
$ (HasFS IO HandleMock -> IO Property) -> IO Property
runFS HasFS IO HandleMock -> IO Property
forall h. HasFS IO h -> IO Property
prop
 where
  prop :: HasFS IO h -> IO Property
  prop :: forall h. HasFS IO h -> IO Property
prop HasFS IO h
hasFS = do
    pHnd <- HasFS IO h -> ChunkNo -> AllowExisting -> IO (Handle h)
forall (m :: * -> *) h.
(HasCallStack, MonadCatch m) =>
HasFS m h -> ChunkNo -> AllowExisting -> m (Handle h)
Primary.open HasFS IO h
hasFS ChunkNo
chunk AllowExisting
MustBeNew
    -- Don't write the first offset, which is always 0; it is written by
    -- 'Primary.open'.
    Primary.appendOffsets hasFS pHnd (drop 1 (Primary.toSecondaryOffsets index))
    index' <- Primary.load (Proxy @TestBlock) hasFS chunk
    return $ index === index'

prop_truncateToSlotFS_truncateToSlot :: TestPrimaryIndex -> Property
prop_truncateToSlotFS_truncateToSlot :: TestPrimaryIndex -> Property
prop_truncateToSlotFS_truncateToSlot (TestPrimaryIndex ChunkInfo
chunkInfo ChunkNo
chunk PrimaryIndex
index RelativeSlot
slot) =
  PropertyM IO Property -> Property
forall a. Testable a => PropertyM IO a -> Property
monadicIO (PropertyM IO Property -> Property)
-> PropertyM IO Property -> Property
forall a b. (a -> b) -> a -> b
$ IO Property -> PropertyM IO Property
forall (m :: * -> *) a. Monad m => m a -> PropertyM m a
run (IO Property -> PropertyM IO Property)
-> IO Property -> PropertyM IO Property
forall a b. (a -> b) -> a -> b
$ (HasFS IO HandleMock -> IO Property) -> IO Property
runFS HasFS IO HandleMock -> IO Property
forall h. HasFS IO h -> IO Property
prop
 where
  prop :: HasFS IO h -> IO Property
  prop :: forall h. HasFS IO h -> IO Property
prop HasFS IO h
hasFS = do
    HasFS IO h -> ChunkNo -> PrimaryIndex -> IO ()
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> ChunkNo -> PrimaryIndex -> m ()
Primary.write HasFS IO h
hasFS ChunkNo
chunk PrimaryIndex
index
    HasFS IO h -> ChunkNo -> RelativeSlot -> IO ()
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> ChunkNo -> RelativeSlot -> m ()
Primary.truncateToSlotFS HasFS IO h
hasFS ChunkNo
chunk RelativeSlot
slot
    index' <- Proxy TestBlock -> HasFS IO h -> ChunkNo -> IO PrimaryIndex
forall blk (m :: * -> *) h.
(HasCallStack, MonadThrow m, StandardHash blk, Typeable blk) =>
Proxy blk -> HasFS m h -> ChunkNo -> m PrimaryIndex
Primary.load (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @TestBlock) HasFS IO h
hasFS ChunkNo
chunk
    return $ Primary.truncateToSlot chunkInfo slot index === index'

prop_readFirstFilledSlot_load_firstFilledSlot :: TestPrimaryIndex -> Property
prop_readFirstFilledSlot_load_firstFilledSlot :: TestPrimaryIndex -> Property
prop_readFirstFilledSlot_load_firstFilledSlot (TestPrimaryIndex ChunkInfo
chunkInfo ChunkNo
chunk PrimaryIndex
index RelativeSlot
_slot) =
  PropertyM IO Property -> Property
forall a. Testable a => PropertyM IO a -> Property
monadicIO (PropertyM IO Property -> Property)
-> PropertyM IO Property -> Property
forall a b. (a -> b) -> a -> b
$ IO Property -> PropertyM IO Property
forall (m :: * -> *) a. Monad m => m a -> PropertyM m a
run (IO Property -> PropertyM IO Property)
-> IO Property -> PropertyM IO Property
forall a b. (a -> b) -> a -> b
$ (HasFS IO HandleMock -> IO Property) -> IO Property
runFS HasFS IO HandleMock -> IO Property
forall h. HasFS IO h -> IO Property
prop
 where
  prop :: HasFS IO h -> IO Property
  prop :: forall h. HasFS IO h -> IO Property
prop HasFS IO h
hasFS = do
    HasFS IO h -> ChunkNo -> PrimaryIndex -> IO ()
forall (m :: * -> *) h.
(HasCallStack, MonadThrow m) =>
HasFS m h -> ChunkNo -> PrimaryIndex -> m ()
Primary.write HasFS IO h
hasFS ChunkNo
chunk PrimaryIndex
index
    mbFirstFilledsLot <-
      Proxy TestBlock
-> HasFS IO h -> ChunkInfo -> ChunkNo -> IO (Maybe RelativeSlot)
forall blk (m :: * -> *) h.
(HasCallStack, MonadThrow m, StandardHash blk, Typeable blk) =>
Proxy blk
-> HasFS m h -> ChunkInfo -> ChunkNo -> m (Maybe RelativeSlot)
Primary.readFirstFilledSlot (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @TestBlock) HasFS IO h
hasFS ChunkInfo
chunkInfo ChunkNo
chunk
    return $ mbFirstFilledsLot === Primary.firstFilledSlot chunkInfo index

{------------------------------------------------------------------------------
  reconstructPrimaryIndex
------------------------------------------------------------------------------}

-- | DummyBlock to define an instance of 'ConvertRawHash' with a fixed hash
-- size.
data DummyBlock

-- | Only 'hashSize' is used.
instance ConvertRawHash DummyBlock where
  hashSize :: forall (proxy :: * -> *). proxy DummyBlock -> SecondaryOffset
hashSize proxy DummyBlock
_ = SecondaryOffset
32
  toRawHash :: forall (proxy :: * -> *).
proxy DummyBlock -> HeaderHash DummyBlock -> ByteString
toRawHash proxy DummyBlock
_ = String -> HeaderHash DummyBlock -> ByteString
forall a. HasCallStack => String -> a
error String
"not used in the tests"
  fromRawHash :: forall (proxy :: * -> *).
proxy DummyBlock -> ByteString -> HeaderHash DummyBlock
fromRawHash proxy DummyBlock
_ = String -> ByteString -> HeaderHash DummyBlock
forall a. HasCallStack => String -> a
error String
"not used in the tests"

prop_reconstructPrimaryIndex :: TestPrimaryIndex -> Property
prop_reconstructPrimaryIndex :: TestPrimaryIndex -> Property
prop_reconstructPrimaryIndex (TestPrimaryIndex ChunkInfo
chunkInfo ChunkNo
chunk PrimaryIndex
primaryIndex RelativeSlot
_slot) =
  String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"filledSlots  : " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> [RelativeSlot] -> String
forall a. Show a => a -> String
show [RelativeSlot]
filledSlots) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
    String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"blocksOrEBBs : " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> [BlockOrEBB] -> String
forall a. Show a => a -> String
show [BlockOrEBB]
blockOrEBBs) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
      String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"relativeSlots: " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> [RelativeSlot] -> String
forall a. Show a => a -> String
show ((BlockOrEBB -> RelativeSlot) -> [BlockOrEBB] -> [RelativeSlot]
forall a b. (a -> b) -> [a] -> [b]
map BlockOrEBB -> RelativeSlot
toRelativeSlot [BlockOrEBB]
blockOrEBBs)) (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
        String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (String
"primaryIndex': " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> PrimaryIndex -> String
forall a. Show a => a -> String
show PrimaryIndex
primaryIndex') (Property -> Property) -> Property -> Property
forall a b. (a -> b) -> a -> b
$
          PrimaryIndex
reconstructedPrimaryIndex PrimaryIndex -> PrimaryIndex -> Property
forall a. (Eq a, Show a) => a -> a -> Property
=== PrimaryIndex
primaryIndex'
 where
  reconstructedPrimaryIndex :: PrimaryIndex
  reconstructedPrimaryIndex :: PrimaryIndex
reconstructedPrimaryIndex =
    Proxy DummyBlock
-> ChunkInfo
-> ShouldBeFinalised
-> ChunkNo
-> [BlockOrEBB]
-> PrimaryIndex
forall blk.
(ConvertRawHash blk, HasCallStack) =>
Proxy blk
-> ChunkInfo
-> ShouldBeFinalised
-> ChunkNo
-> [BlockOrEBB]
-> PrimaryIndex
reconstructPrimaryIndex
      (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @DummyBlock)
      ChunkInfo
chunkInfo
      ShouldBeFinalised
ShouldNotBeFinalised
      ChunkNo
chunk
      [BlockOrEBB]
blockOrEBBs

  -- Remove empty trailing slots because we don't reconstruct them
  primaryIndex' :: PrimaryIndex
  primaryIndex' :: PrimaryIndex
primaryIndex' = case HasCallStack => ChunkInfo -> PrimaryIndex -> Maybe RelativeSlot
ChunkInfo -> PrimaryIndex -> Maybe RelativeSlot
Primary.lastFilledSlot ChunkInfo
chunkInfo PrimaryIndex
primaryIndex of
    Just RelativeSlot
slot -> ChunkInfo -> RelativeSlot -> PrimaryIndex -> PrimaryIndex
Primary.truncateToSlot ChunkInfo
chunkInfo RelativeSlot
slot PrimaryIndex
primaryIndex
    -- Index is empty, use the minimal empty index without any trailing
    -- slots
    Maybe RelativeSlot
Nothing -> Maybe PrimaryIndex -> PrimaryIndex
forall a. HasCallStack => Maybe a -> a
fromJust (Maybe PrimaryIndex -> PrimaryIndex)
-> Maybe PrimaryIndex -> PrimaryIndex
forall a b. (a -> b) -> a -> b
$ ChunkNo -> [SecondaryOffset] -> Maybe PrimaryIndex
Primary.mk ChunkNo
chunk [SecondaryOffset
0]

  filledSlots :: [RelativeSlot]
  filledSlots :: [RelativeSlot]
filledSlots = ChunkInfo -> PrimaryIndex -> [RelativeSlot]
Primary.filledSlots ChunkInfo
chunkInfo PrimaryIndex
primaryIndex

  blockOrEBBs :: [BlockOrEBB]
  blockOrEBBs :: [BlockOrEBB]
blockOrEBBs =
    [ ChunkInfo -> ChunkSlot -> BlockOrEBB
chunkSlotToBlockOrEBB ChunkInfo
chunkInfo (ChunkNo -> RelativeSlot -> ChunkSlot
UnsafeChunkSlot ChunkNo
chunk RelativeSlot
relSlot)
    | RelativeSlot
relSlot <- [RelativeSlot]
filledSlots
    ]

  -- This emulates what 'reconstructPrimaryIndex' does internally
  toRelativeSlot :: BlockOrEBB -> RelativeSlot
  toRelativeSlot :: BlockOrEBB -> RelativeSlot
toRelativeSlot = ChunkSlot -> RelativeSlot
chunkRelative (ChunkSlot -> RelativeSlot)
-> (BlockOrEBB -> ChunkSlot) -> BlockOrEBB -> RelativeSlot
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ChunkInfo -> BlockOrEBB -> ChunkSlot
chunkSlotForBlockOrEBB ChunkInfo
chunkInfo

{------------------------------------------------------------------------------
  Helpers
------------------------------------------------------------------------------}

runFS :: (HasFS IO HandleMock -> IO Property) -> IO Property
runFS :: (HasFS IO HandleMock -> IO Property) -> IO Property
runFS HasFS IO HandleMock -> IO Property
m =
  IO (Property, MockFS) -> IO (Either FsError (Property, MockFS))
forall e a. Exception e => IO a -> IO (Either e a)
forall (m :: * -> *) e a.
(MonadCatch m, Exception e) =>
m a -> m (Either e a)
try (MockFS
-> (HasFS IO HandleMock -> IO Property) -> IO (Property, MockFS)
forall (m :: * -> *) a.
(MonadSTM m, MonadThrow m, PrimMonad m) =>
MockFS -> (HasFS m HandleMock -> m a) -> m (a, MockFS)
Sim.runSimFS MockFS
Mock.empty HasFS IO HandleMock -> IO Property
m) IO (Either FsError (Property, MockFS))
-> (Either FsError (Property, MockFS) -> IO Property)
-> IO Property
forall a b. IO a -> (a -> IO b) -> IO b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \case
    Left FsError
e -> String -> IO Property
forall a. String -> IO a
forall (m :: * -> *) a. MonadFail m => String -> m a
fail (FsError -> String
prettyFsError FsError
e)
    Right (Property
p, MockFS
mockFS) -> Property -> IO Property
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Property -> IO Property) -> Property -> IO Property
forall a b. (a -> b) -> a -> b
$ String -> Property -> Property
forall prop. Testable prop => String -> prop -> Property
counterexample (MockFS -> String
Mock.pretty MockFS
mockFS) Property
p

{------------------------------------------------------------------------------
  Generators
------------------------------------------------------------------------------}

data TestPrimaryIndex = TestPrimaryIndex
  { TestPrimaryIndex -> ChunkInfo
testChunkInfo :: ChunkInfo
  , TestPrimaryIndex -> ChunkNo
testChunkNo :: ChunkNo
  , TestPrimaryIndex -> PrimaryIndex
testPrimaryIndex :: PrimaryIndex
  , TestPrimaryIndex -> RelativeSlot
testRandomSlot :: RelativeSlot
  -- ^ For tests that need it, a random slot for this thunk
  --
  -- We guarantee that the 'relativeChunkNo' matches 'testChunkNo' and is
  -- within the bounds set by the 'ChunkSize' of the chunk (although it
  -- may still be past the actual number of entries in the index).
  }
  deriving Int -> TestPrimaryIndex -> String -> String
[TestPrimaryIndex] -> String -> String
TestPrimaryIndex -> String
(Int -> TestPrimaryIndex -> String -> String)
-> (TestPrimaryIndex -> String)
-> ([TestPrimaryIndex] -> String -> String)
-> Show TestPrimaryIndex
forall a.
(Int -> a -> String -> String)
-> (a -> String) -> ([a] -> String -> String) -> Show a
$cshowsPrec :: Int -> TestPrimaryIndex -> String -> String
showsPrec :: Int -> TestPrimaryIndex -> String -> String
$cshow :: TestPrimaryIndex -> String
show :: TestPrimaryIndex -> String
$cshowList :: [TestPrimaryIndex] -> String -> String
showList :: [TestPrimaryIndex] -> String -> String
Show

instance Arbitrary TestPrimaryIndex where
  arbitrary :: Gen TestPrimaryIndex
arbitrary = do
    chunkSize <- Gen ChunkSize
forall a. Arbitrary a => Gen a
arbitrary
    let chunkInfo = ChunkSize -> ChunkInfo
singleChunkInfo ChunkSize
chunkSize
    -- The chunk number is not very relevant here; a single primary index
    -- is anyway for a single chunk and hence a single chunk size.
    chunk <- arbitrary
    nbOffsets <- fromIntegral <$> choose (1, maxRelativeIndex chunkSize)
    offsets <-
      go nbOffsets 0 [] <&> \[SecondaryOffset]
offsets ->
        case ChunkNo -> [SecondaryOffset] -> Maybe PrimaryIndex
Primary.mk ChunkNo
chunk [SecondaryOffset]
offsets of
          Maybe PrimaryIndex
Nothing -> String -> PrimaryIndex
forall a. HasCallStack => String -> a
error (String -> PrimaryIndex) -> String -> PrimaryIndex
forall a b. (a -> b) -> a -> b
$ String
"invalid offsets: " String -> String -> String
forall a. Semigroup a => a -> a -> a
<> [SecondaryOffset] -> String
forall a. Show a => a -> String
show [SecondaryOffset]
offsets
          Just PrimaryIndex
index -> PrimaryIndex
index
    slot <-
      mkRelativeSlot chunkInfo chunk
        <$> choose (0, maxRelativeIndex chunkSize)
    return $ TestPrimaryIndex chunkInfo chunk offsets slot
   where
    -- All entries in the secondary index will have the same size
    offsetSize :: SecondaryOffset
offsetSize = Proxy DummyBlock -> SecondaryOffset
forall blk. ConvertRawHash blk => Proxy blk -> SecondaryOffset
Secondary.entrySize (forall t. Proxy t
forall {k} (t :: k). Proxy t
Proxy @DummyBlock)

    go ::
      Int ->
      SecondaryOffset ->
      [SecondaryOffset] ->
      Gen [SecondaryOffset]
    go :: Int
-> SecondaryOffset -> [SecondaryOffset] -> Gen [SecondaryOffset]
go Int
0 SecondaryOffset
prev [SecondaryOffset]
acc = [SecondaryOffset] -> Gen [SecondaryOffset]
forall a. a -> Gen a
forall (m :: * -> *) a. Monad m => a -> m a
return ([SecondaryOffset] -> Gen [SecondaryOffset])
-> [SecondaryOffset] -> Gen [SecondaryOffset]
forall a b. (a -> b) -> a -> b
$ [SecondaryOffset] -> [SecondaryOffset]
forall a. [a] -> [a]
reverse (SecondaryOffset
prev SecondaryOffset -> [SecondaryOffset] -> [SecondaryOffset]
forall a. a -> [a] -> [a]
: [SecondaryOffset]
acc)
    go Int
n SecondaryOffset
prev [SecondaryOffset]
acc =
      Gen Bool
forall a. Arbitrary a => Gen a
arbitrary Gen Bool
-> (Bool -> Gen [SecondaryOffset]) -> Gen [SecondaryOffset]
forall a b. Gen a -> (a -> Gen b) -> Gen b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \Bool
repeatLast ->
        if Bool
repeatLast
          then
            Int
-> SecondaryOffset -> [SecondaryOffset] -> Gen [SecondaryOffset]
go (Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) SecondaryOffset
prev (SecondaryOffset
prev SecondaryOffset -> [SecondaryOffset] -> [SecondaryOffset]
forall a. a -> [a] -> [a]
: [SecondaryOffset]
acc)
          else do
            Int
-> SecondaryOffset -> [SecondaryOffset] -> Gen [SecondaryOffset]
go (Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) (SecondaryOffset
prev SecondaryOffset -> SecondaryOffset -> SecondaryOffset
forall a. Num a => a -> a -> a
+ SecondaryOffset
offsetSize) (SecondaryOffset
prev SecondaryOffset -> [SecondaryOffset] -> [SecondaryOffset]
forall a. a -> [a] -> [a]
: [SecondaryOffset]
acc)

  -- Shrinking will
  --
  -- \* Remove entries from the end of index
  -- \* Shrink the 'ChunkNo'
  shrink :: TestPrimaryIndex -> [TestPrimaryIndex]
shrink TestPrimaryIndex
test =
    [[TestPrimaryIndex]] -> [TestPrimaryIndex]
forall (t :: * -> *) a. Foldable t => t [a] -> [a]
concat
      [ ((PrimaryIndex -> TestPrimaryIndex)
 -> [PrimaryIndex] -> [TestPrimaryIndex])
-> [PrimaryIndex]
-> (PrimaryIndex -> TestPrimaryIndex)
-> [TestPrimaryIndex]
forall a b c. (a -> b -> c) -> b -> a -> c
flip (PrimaryIndex -> TestPrimaryIndex)
-> [PrimaryIndex] -> [TestPrimaryIndex]
forall a b. (a -> b) -> [a] -> [b]
map (PrimaryIndex -> [PrimaryIndex]
shrinkIndex (PrimaryIndex -> [PrimaryIndex]) -> PrimaryIndex -> [PrimaryIndex]
forall a b. (a -> b) -> a -> b
$ TestPrimaryIndex -> PrimaryIndex
testPrimaryIndex TestPrimaryIndex
test) ((PrimaryIndex -> TestPrimaryIndex) -> [TestPrimaryIndex])
-> (PrimaryIndex -> TestPrimaryIndex) -> [TestPrimaryIndex]
forall a b. (a -> b) -> a -> b
$ \PrimaryIndex
i ->
          TestPrimaryIndex
test
            { testPrimaryIndex = i
            }
      , ((ChunkNo -> TestPrimaryIndex) -> [ChunkNo] -> [TestPrimaryIndex])
-> [ChunkNo] -> (ChunkNo -> TestPrimaryIndex) -> [TestPrimaryIndex]
forall a b c. (a -> b -> c) -> b -> a -> c
flip (ChunkNo -> TestPrimaryIndex) -> [ChunkNo] -> [TestPrimaryIndex]
forall a b. (a -> b) -> [a] -> [b]
map (ChunkNo -> [ChunkNo]
forall a. Arbitrary a => a -> [a]
shrink (ChunkNo -> [ChunkNo]) -> ChunkNo -> [ChunkNo]
forall a b. (a -> b) -> a -> b
$ TestPrimaryIndex -> ChunkNo
testChunkNo TestPrimaryIndex
test) ((ChunkNo -> TestPrimaryIndex) -> [TestPrimaryIndex])
-> (ChunkNo -> TestPrimaryIndex) -> [TestPrimaryIndex]
forall a b. (a -> b) -> a -> b
$ \ChunkNo
chunkNo' ->
          TestPrimaryIndex
test
            { testChunkNo = chunkNo'
            , testPrimaryIndex =
                (testPrimaryIndex test)
                  { Primary.primaryIndexChunkNo = chunkNo'
                  }
            , testRandomSlot =
                (testRandomSlot test)
                  { relativeSlotChunkNo = chunkNo'
                  }
            }
      ]
   where
    shrinkIndex :: PrimaryIndex -> [PrimaryIndex]
    shrinkIndex :: PrimaryIndex -> [PrimaryIndex]
shrinkIndex PrimaryIndex
i =
      (Vector SecondaryOffset -> PrimaryIndex)
-> [Vector SecondaryOffset] -> [PrimaryIndex]
forall a b. (a -> b) -> [a] -> [b]
map (\Vector SecondaryOffset
os -> PrimaryIndex
i{Primary.primaryIndexOffsets = os}) ([Vector SecondaryOffset] -> [PrimaryIndex])
-> [Vector SecondaryOffset] -> [PrimaryIndex]
forall a b. (a -> b) -> a -> b
$
        Vector SecondaryOffset -> [Vector SecondaryOffset]
shrinkOffsets (PrimaryIndex -> Vector SecondaryOffset
Primary.primaryIndexOffsets PrimaryIndex
i)

    -- The generator is careful to insert values of 'offsetSize'
    -- We should maintain this invariant here, so we just drop elements
    -- from the end of the list.
    shrinkOffsets :: Vector SecondaryOffset -> [Vector SecondaryOffset]
    shrinkOffsets :: Vector SecondaryOffset -> [Vector SecondaryOffset]
shrinkOffsets =
      ([SecondaryOffset] -> Vector SecondaryOffset)
-> [[SecondaryOffset]] -> [Vector SecondaryOffset]
forall a b. (a -> b) -> [a] -> [b]
map [SecondaryOffset] -> Vector SecondaryOffset
forall a. Unbox a => [a] -> Vector a
V.fromList
        ([[SecondaryOffset]] -> [Vector SecondaryOffset])
-> (Vector SecondaryOffset -> [[SecondaryOffset]])
-> Vector SecondaryOffset
-> [Vector SecondaryOffset]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (\(SecondaryOffset
0 : [SecondaryOffset]
xs) -> ([SecondaryOffset] -> [SecondaryOffset])
-> [[SecondaryOffset]] -> [[SecondaryOffset]]
forall a b. (a -> b) -> [a] -> [b]
map (SecondaryOffset
0 SecondaryOffset -> [SecondaryOffset] -> [SecondaryOffset]
forall a. a -> [a] -> [a]
:) ([[SecondaryOffset]] -> [[SecondaryOffset]])
-> [[SecondaryOffset]] -> [[SecondaryOffset]]
forall a b. (a -> b) -> a -> b
$ [SecondaryOffset] -> [[SecondaryOffset]]
forall a. [a] -> [[a]]
dropOne [SecondaryOffset]
xs)
        ([SecondaryOffset] -> [[SecondaryOffset]])
-> (Vector SecondaryOffset -> [SecondaryOffset])
-> Vector SecondaryOffset
-> [[SecondaryOffset]]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Vector SecondaryOffset -> [SecondaryOffset]
forall a. Unbox a => Vector a -> [a]
V.toList

    dropOne :: [a] -> [[a]]
    dropOne :: forall a. [a] -> [[a]]
dropOne [] = []
    dropOne [a]
xs = [[a] -> [a]
forall a. [a] -> [a]
reverse ([a] -> [a]) -> ([a] -> [a]) -> [a] -> [a]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [a] -> [a]
forall a. HasCallStack => [a] -> [a]
tail ([a] -> [a]) -> ([a] -> [a]) -> [a] -> [a]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [a] -> [a]
forall a. [a] -> [a]
reverse ([a] -> [a]) -> [a] -> [a]
forall a b. (a -> b) -> a -> b
$ [a]
xs]