diff --git a/CHANGELOG.md b/CHANGELOG.md index 5353f2c9f9..dea8409347 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,8 @@ # Changelog ## Unreleased changes +- The account map is now kept solely on disk in a separate lmdb database and it is no longer part of the internal block state database. + This change results in less memory usage per account and a decrease in the growth of the database. - Remove V1 GRPC API from the node. This removes configuration options `CONCORDIUM_NODE_RPC_SERVER_PORT`, `CONCORDIUM_NODE_RPC_SERVER_ADDRESS`, diff --git a/concordium-consensus/src/Concordium/External.hs b/concordium-consensus/src/Concordium/External.hs index 30f2bfa663..7e11c6c94c 100644 --- a/concordium-consensus/src/Concordium/External.hs +++ b/concordium-consensus/src/Concordium/External.hs @@ -265,6 +265,7 @@ toStartResult = GenesisBlockIncorrect _ -> 9 DatabaseInvariantViolation _ -> 10 IncorrectDatabaseVersion _ -> 11 + AccountMapPermissionError -> 12 -- | Catch exceptions which may occur at start up and return an appropriate exit code. handleStartExceptions :: LogMethod IO -> IO StartResult -> IO Int64 diff --git a/concordium-consensus/src/Concordium/GlobalState.hs b/concordium-consensus/src/Concordium/GlobalState.hs index 3f0c21d5dd..a2e8144f04 100644 --- a/concordium-consensus/src/Concordium/GlobalState.hs +++ b/concordium-consensus/src/Concordium/GlobalState.hs @@ -15,6 +15,7 @@ import Control.Monad.Reader.Class import Control.Monad.Trans.Reader hiding (ask) import Data.Proxy +import Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap import Concordium.GlobalState.BlockState import Concordium.GlobalState.Parameters import Concordium.GlobalState.Persistent.Account (newAccountCache) @@ -29,9 +30,14 @@ import Concordium.Types.ProtocolVersion -- | Configuration that uses the disk implementation for both the tree state -- and the block state data GlobalStateConfig = GlobalStateConfig - { dtdbRuntimeParameters :: !RuntimeParameters, + { -- | Runtime parameters. + dtdbRuntimeParameters :: !RuntimeParameters, + -- | Path to the tree state database. dtdbTreeStateDirectory :: !FilePath, - dtdbBlockStateFile :: !FilePath + -- | Path to the block state database. + dtdbBlockStateFile :: !FilePath, + -- | Path to the account map database. + dtdAccountMapDirectory :: !FilePath } -- | Exceptions that can occur when initialising the global state. @@ -65,7 +71,7 @@ type GSState pv = SkovPersistentData pv initialiseExistingGlobalState :: forall pv. (IsProtocolVersion pv) => SProtocolVersion pv -> GlobalStateConfig -> LogIO (Maybe (GSContext pv, GSState pv)) initialiseExistingGlobalState _ GlobalStateConfig{..} = do -- check if all the necessary database files exist - existingDB <- checkExistingDatabase dtdbTreeStateDirectory dtdbBlockStateFile + existingDB <- checkExistingDatabase dtdbTreeStateDirectory dtdbBlockStateFile dtdAccountMapDirectory if existingDB then do logm <- ask @@ -73,6 +79,7 @@ initialiseExistingGlobalState _ GlobalStateConfig{..} = do pbscAccountCache <- newAccountCache (rpAccountsCacheSize dtdbRuntimeParameters) pbscModuleCache <- Modules.newModuleCache (rpModulesCacheSize dtdbRuntimeParameters) pbscBlobStore <- loadBlobStore dtdbBlockStateFile + pbscAccountMap <- liftIO $ LMDBAccountMap.openDatabase dtdAccountMapDirectory let pbsc = PersistentBlockStateContext{..} skovData <- runLoggerT (loadSkovPersistentData dtdbRuntimeParameters dtdbTreeStateDirectory pbsc) logm @@ -80,6 +87,17 @@ initialiseExistingGlobalState _ GlobalStateConfig{..} = do return (Just (pbsc, skovData)) else return Nothing +-- | Initialize a 'PersistentBlockStateContext' via the provided +-- 'GlobalStateConfig'. +-- This function attempts to create a new blob store. +initializePersistentBlockStateContext :: GlobalStateConfig -> IO (PersistentBlockStateContext pv) +initializePersistentBlockStateContext GlobalStateConfig{..} = do + pbscBlobStore <- createBlobStore dtdbBlockStateFile + pbscAccountCache <- newAccountCache (rpAccountsCacheSize dtdbRuntimeParameters) + pbscModuleCache <- Modules.newModuleCache (rpModulesCacheSize dtdbRuntimeParameters) + pbscAccountMap <- LMDBAccountMap.openDatabase dtdAccountMapDirectory + return PersistentBlockStateContext{..} + -- | Migrate an existing global state. This is only intended to be used on a -- protocol update and requires that the initial state for the new protocol -- version is prepared (see @TreeState.storeFinalState@). This function will @@ -109,11 +127,8 @@ migrateExistingState :: Regenesis pv -> -- | The return value is the context and state for the new chain. LogIO (GSContext pv, GSState pv) -migrateExistingState GlobalStateConfig{..} oldPbsc oldState migration genData = do - pbscBlobStore <- liftIO $ createBlobStore dtdbBlockStateFile - pbscAccountCache <- liftIO $ newAccountCache (rpAccountsCacheSize dtdbRuntimeParameters) - pbscModuleCache <- liftIO $ Modules.newModuleCache (rpModulesCacheSize dtdbRuntimeParameters) - let pbsc = PersistentBlockStateContext{..} +migrateExistingState gsc@GlobalStateConfig{..} oldPbsc oldState migration genData = do + pbsc <- liftIO $ initializePersistentBlockStateContext gsc newInitialBlockState <- flip runBlobStoreT oldPbsc . flip runBlobStoreT pbsc $ do case _nextGenesisInitialState oldState of Nothing -> error "Precondition violation. Migration called in state without initial block state." @@ -132,18 +147,15 @@ migrateExistingState GlobalStateConfig{..} oldPbsc oldState migration genData = (Just (_pendingTransactions oldState)) isd <- runReaderT (runPersistentBlockStateMonad initGS) pbsc - `onException` liftIO (destroyBlobStore pbscBlobStore) + `onException` liftIO (destroyBlobStore (pbscBlobStore pbsc)) return (pbsc, isd) -- | Initialise new global state with the given genesis. If the state already -- exists this will raise an exception. It is not necessary to call 'activateGlobalState' -- on the generated state, as this will establish the necessary invariants. initialiseNewGlobalState :: (IsProtocolVersion pv, IsConsensusV0 pv) => GenesisData pv -> GlobalStateConfig -> LogIO (GSContext pv, GSState pv) -initialiseNewGlobalState genData GlobalStateConfig{..} = do - pbscBlobStore <- liftIO $ createBlobStore dtdbBlockStateFile - pbscAccountCache <- liftIO $ newAccountCache (rpAccountsCacheSize dtdbRuntimeParameters) - pbscModuleCache <- liftIO $ Modules.newModuleCache (rpModulesCacheSize dtdbRuntimeParameters) - let pbsc = PersistentBlockStateContext{..} +initialiseNewGlobalState genData gsc@GlobalStateConfig{..} = do + pbsc@PersistentBlockStateContext{..} <- liftIO $ initializePersistentBlockStateContext gsc let initGS = do logEvent GlobalState LLTrace "Creating persistent global state" result <- genesisState genData diff --git a/concordium-consensus/src/Concordium/GlobalState/AccountMap/DifferenceMap.hs b/concordium-consensus/src/Concordium/GlobalState/AccountMap/DifferenceMap.hs new file mode 100644 index 0000000000..bc2197448b --- /dev/null +++ b/concordium-consensus/src/Concordium/GlobalState/AccountMap/DifferenceMap.hs @@ -0,0 +1,161 @@ +{-# LANGUAGE BangPatterns #-} + +-- | The 'DifferenceMap' stores accounts that have been created in a non-finalized block. +-- When a block is finalized then the associated 'DifferenceMap' must be written +-- to disk via 'Concordium.GlobalState.AccountMap.LMDB.insertAccounts'. +module Concordium.GlobalState.AccountMap.DifferenceMap ( + -- * Definitions + + -- The difference map definition. + DifferenceMap (..), + -- A mutable reference to a 'DifferenceMap'. + DifferenceMapReference, + + -- * Auxiliary functions + newEmptyReference, + flatten, + empty, + fromList, + insert, + lookupViaEquivalenceClass, + lookupExact, + clearReferences, +) where + +import Control.Monad.IO.Class +import qualified Data.HashMap.Strict as HM +import Data.IORef +import Data.Tuple (swap) +import Prelude hiding (lookup) + +import Concordium.Types +import Concordium.Types.Option (Option (..)) + +-- | A mutable reference to a 'DiffMap.DifferenceMap'. +-- This is an 'IORef' since the parent map may belong +-- to multiple blocks if they have not yet been persisted. +-- +-- The 'IORef' enables us to clear any child difference maps +-- when a block is finalized. +type DifferenceMapReference = IORef (Option DifferenceMap) + +-- | Create a new empty reference. +newEmptyReference :: (MonadIO m) => m DifferenceMapReference +newEmptyReference = liftIO $ newIORef Absent + +-- | A difference map that indicates newly added accounts for +-- a block identified by a 'BlockHash' and its associated 'BlockHeight'. +-- The difference map only contains accounts that were added since the '_dmParentMapRef'. +data DifferenceMap = DifferenceMap + { -- | Accounts added in a block keyed by their equivalence class and + -- the @AccountIndex@ and canonical account adddress as values. + dmAccounts :: !(HM.HashMap AccountAddressEq (AccountIndex, AccountAddress)), + -- | Parent map of non-finalized blocks. + -- In other words, if the parent block is finalized, + -- then the parent map is @Absent@ as the LMDB account map + -- should be consulted instead. + dmParentMapRef :: !DifferenceMapReference + } + deriving (Eq) + +-- | Gather all accounts from the provided 'DifferenceMap' and its parent maps. +-- Accounts are returned in ascending order of their 'AccountAddress'. +-- +-- Note. This function does not guarantee the order of the returned pairs. +flatten :: (MonadIO m) => DifferenceMap -> m [(AccountAddress, AccountIndex)] +flatten dmap = go dmap [] + where + go diffMap !accum = do + mParentMap <- liftIO $ readIORef (dmParentMapRef diffMap) + case mParentMap of + Absent -> return collectedAccounts + Present parentMap -> go parentMap collectedAccounts + where + collectedAccounts = map swap (HM.elems $ dmAccounts diffMap) <> accum + +-- | Create a new empty 'DifferenceMap' potentially based on the difference map of +-- the parent. +empty :: DifferenceMapReference -> DifferenceMap +empty mParentDifferenceMap = + DifferenceMap + { dmAccounts = HM.empty, + dmParentMapRef = mParentDifferenceMap + } + +-- | Internal helper function for looking up an entry in @dmAccounts@. +-- Returns @Right AccountIndex AccountAddress Word64@ if the account could be looked up, +-- and otherwise @Left Word64@, where the number indicates how many accounts are present in the difference map +-- and potentially any parent difference maps. +lookupViaEquivalenceClass' :: (MonadIO m) => AccountAddressEq -> DifferenceMap -> m (Either Int (AccountIndex, AccountAddress)) +lookupViaEquivalenceClass' addr = check 0 + where + check !accum diffMap = case HM.lookup addr (dmAccounts diffMap) of + Nothing -> do + mParentMap <- liftIO $ readIORef (dmParentMapRef diffMap) + let !accum' = accum + HM.size (dmAccounts diffMap) + case mParentMap of + Absent -> return $ Left accum' + Present parentMap -> check accum' parentMap + Just res -> return $ Right res + +-- | Lookup an account in the difference map or any of the parent +-- difference maps using the account address equivalence class. +-- Returns @Just AccountIndex@ if the account is present and +-- otherwise @Left Word64@ indicating how many accounts there are present in the +-- difference map(s). +-- Precondition: As this implementation uses the 'AccountAddressEq' equivalence +-- class for looking up an 'AccountIndex', then it MUST only be used +-- when account aliases are supported. +lookupViaEquivalenceClass :: (MonadIO m) => AccountAddressEq -> DifferenceMap -> m (Either Int AccountIndex) +lookupViaEquivalenceClass addr dm = fmap fst <$> lookupViaEquivalenceClass' addr dm + +-- | Lookup an account in the difference map or any of the parent +-- difference maps via an exactness check. +-- Returns @Just AccountIndex@ if the account is present and +-- otherwise @Left Word64@ indicating how many accounts there are present in the +-- difference map(s). +-- Note that this function also returns @Nothing@ if the provided 'AccountAddress' +-- is an alias but not the canonical address. +lookupExact :: (MonadIO m) => AccountAddress -> DifferenceMap -> m (Either Int AccountIndex) +lookupExact addr diffMap = + lookupViaEquivalenceClass' (accountAddressEmbed addr) diffMap >>= \case + Left noAccounts -> return $ Left noAccounts + Right (accIdx, actualAddr) -> + if actualAddr == addr + then return $ Right accIdx + else do + -- This extra flatten is really not ideal, but it should also really never happen, + -- hence the extra flatten here justifies the simpler implementation and optimization + -- towards the normal use case. + size <- length <$> flatten diffMap + return $ Left size + +-- | Insert an account into the difference map. +-- Note that it is up to the caller to ensure only the canonical 'AccountAddress' is inserted. +insert :: AccountAddress -> AccountIndex -> DifferenceMap -> DifferenceMap +insert addr accIndex m = m{dmAccounts = HM.insert (accountAddressEmbed addr) (accIndex, addr) $ dmAccounts m} + +-- | Create a 'DifferenceMap' with the provided parent and list of account addresses and account indices. +fromList :: IORef (Option DifferenceMap) -> [(AccountAddress, AccountIndex)] -> DifferenceMap +fromList parentRef listOfAccountsAndIndices = + DifferenceMap + { dmAccounts = HM.fromList $ map mkKeyVal listOfAccountsAndIndices, + dmParentMapRef = parentRef + } + where + -- Make a key value pair to put in the @dmAccounts@. + mkKeyVal (accAddr, accIdx) = (accountAddressEmbed accAddr, (accIdx, accAddr)) + +-- | Clear the reference to the parent difference map (if any). +-- Note that if there is a parent map then this function clears the remaining parent references +-- recursively. +clearReferences :: (MonadIO m) => DifferenceMap -> m () +clearReferences DifferenceMap{..} = do + oParentDiffMap <- liftIO $ readIORef dmParentMapRef + case oParentDiffMap of + Absent -> return () + Present diffMap -> do + -- Clear this parent reference. + liftIO $ atomicWriteIORef dmParentMapRef Absent + -- Continue and check if the parent should have cleared it parent(s). + clearReferences diffMap diff --git a/concordium-consensus/src/Concordium/GlobalState/AccountMap/LMDB.hs b/concordium-consensus/src/Concordium/GlobalState/AccountMap/LMDB.hs new file mode 100644 index 0000000000..174fe3ce85 --- /dev/null +++ b/concordium-consensus/src/Concordium/GlobalState/AccountMap/LMDB.hs @@ -0,0 +1,288 @@ +{-# LANGUAGE BangPatterns #-} +{-# LANGUAGE DataKinds #-} +{-# LANGUAGE DeriveDataTypeable #-} +{-# LANGUAGE DerivingVia #-} +{-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE ScopedTypeVariables #-} +{-# LANGUAGE StandaloneDeriving #-} +{-# LANGUAGE TemplateHaskell #-} +{-# LANGUAGE TypeApplications #-} +{-# LANGUAGE TypeFamilies #-} +{-# LANGUAGE UndecidableInstances #-} + +-- | This module exposes an account map backed by a LMDB database. +-- The account map is a simple key/value store where the keys consists of the +-- canonical 'AccountAddress' and the values are the assoicated 'AccountIndex'. +-- +-- The LMDB account map only stores accounts that are finalized. +-- Non finalized accounts are being kept in a 'DifferenceMap' which +-- is written to this LMDB account map when a block is finalized. +-- +-- This also means that accounts once put in the account map, then +-- they can never be deleted again (hence they're finalized). +-- +-- As opposed to the account table of the block state this database does not +-- include historical data i.e., the state of this database is from the perspective +-- of the last finalized block always. +-- +-- The account map is integrated with the block state “on-the-fly” meaning that +-- whenever the node starts up and the account map is not populated, then it will be +-- initialized on startup via the existing ‘PersistentAccountMap’. +-- +-- Invariants: +-- * Only accounts that are in finalized blocks are present in the ‘AccountMap’ +module Concordium.GlobalState.AccountMap.LMDB where + +import Control.Concurrent +import Control.Monad.Catch +import Control.Monad.IO.Class +import Control.Monad.Identity +import Control.Monad.Reader +import Control.Monad.State.Strict +import Control.Monad.Trans.Except +import Control.Monad.Trans.Writer.Strict +import qualified Data.ByteString as BS +import Data.Data (Typeable) +import qualified Data.FixedByteString as FBS +import Data.Kind (Type) +import Data.Word +import Database.LMDB.Raw +import Lens.Micro.Platform +import System.Directory + +import Concordium.GlobalState.Classes +import Concordium.GlobalState.LMDB.Helpers +import Concordium.Logger +import Concordium.TimeMonad +import Concordium.Types +import Concordium.Utils.Serialization.Put + +-- * Exceptions + +-- | Exception occurring from a violation of database invariants in the LMDB database. +newtype DatabaseInvariantViolation = DatabaseInvariantViolation String + deriving (Eq, Show, Typeable) + +instance Exception DatabaseInvariantViolation where + displayException (DatabaseInvariantViolation reason) = + "Database invariant violation: " + ++ show reason + +-- | Monad for inserting and looking up accounts in the account map +-- backed by an LMDB database. +-- For more information, refer to the module documentation. +-- +-- An implementation should ensure atomicity of operations. +-- +-- Invariants: +-- * All accounts in the store are finalized. +-- * The store should only retain canonical account addresses. +class (Monad m) => MonadAccountMapStore m where + -- | Inserts the accounts to the underlying store. + -- Only canonical addresses should be added. + insertAccounts :: [(AccountAddress, AccountIndex)] -> m () + + -- | Looks up the ‘AccountIndex’ for the provided ‘AccountAddressEq’. + -- Returns @Just AccountIndex@ if the account is present in the ‘AccountMap’ + -- and returns @Nothing@ if the account was not present. + lookupAccountIndexViaEquivalence :: AccountAddressEq -> m (Maybe AccountIndex) + + -- | Looks up the ‘AccountIndex’ for the provided ‘AccountAddress'. + -- Returns @Just AccountIndex@ if the account is present in the ‘AccountMap’ + -- and returns @Nothing@ if the account was not present. + -- Note that this only returns a result for the canonical account address. + lookupAccountIndexViaExactness :: AccountAddress -> m (Maybe AccountIndex) + + -- | Return all the canonical addresses and their associated account indices of accounts present + -- in the store where their @AccountIndex@ is less or equal to the provided @AccountIndex@. + getAllAccounts :: AccountIndex -> m [(AccountAddress, AccountIndex)] + + -- | Get number of entries in the account map. + getNumberOfAccounts :: m Word64 + + -- | Clear and set the accounts to the ones provided. + reconstruct :: [(AccountAddress, AccountIndex)] -> m () + +instance (Monad (t m), MonadTrans t, MonadAccountMapStore m) => MonadAccountMapStore (MGSTrans t m) where + insertAccounts accs = lift $ insertAccounts accs + lookupAccountIndexViaEquivalence = lift . lookupAccountIndexViaEquivalence + lookupAccountIndexViaExactness = lift . lookupAccountIndexViaExactness + getAllAccounts = lift . getAllAccounts + getNumberOfAccounts = lift getNumberOfAccounts + reconstruct = lift . reconstruct + {-# INLINE insertAccounts #-} + {-# INLINE lookupAccountIndexViaEquivalence #-} + {-# INLINE lookupAccountIndexViaExactness #-} + {-# INLINE getAllAccounts #-} + {-# INLINE getNumberOfAccounts #-} + {-# INLINE reconstruct #-} + +deriving via (MGSTrans (StateT s) m) instance (MonadAccountMapStore m) => MonadAccountMapStore (StateT s m) +deriving via (MGSTrans (ExceptT e) m) instance (MonadAccountMapStore m) => MonadAccountMapStore (ExceptT e m) +deriving via (MGSTrans (WriterT w) m) instance (Monoid w, MonadAccountMapStore m) => MonadAccountMapStore (WriterT w m) + +instance (MonadAccountMapStore m) => MonadAccountMapStore (PutT m) where + insertAccounts accs = lift $ insertAccounts accs + lookupAccountIndexViaEquivalence = lift . lookupAccountIndexViaEquivalence + lookupAccountIndexViaExactness = lift . lookupAccountIndexViaExactness + getAllAccounts = lift . getAllAccounts + getNumberOfAccounts = lift getNumberOfAccounts + reconstruct = lift . reconstruct + {-# INLINE insertAccounts #-} + {-# INLINE lookupAccountIndexViaEquivalence #-} + {-# INLINE lookupAccountIndexViaExactness #-} + {-# INLINE getAllAccounts #-} + {-# INLINE getNumberOfAccounts #-} + {-# INLINE reconstruct #-} + +-- * Database stores + +-- | Store that retains the account address -> account index mappings. +newtype AccountMapStore = AccountMapStore MDB_dbi' + +-- | Name of the table used for storing the map from account addresses to account indices. +accountMapStoreName :: String +accountMapStoreName = "accounts" + +instance MDBDatabase AccountMapStore where + type DBKey AccountMapStore = AccountAddress + type DBValue AccountMapStore = AccountIndex + +-- | Datbase handlers to interact with the account map lmdb +-- database. Create via 'makeDatabasehandlers'. +data DatabaseHandlers = DatabaseHandlers + { -- | The underlying lmdb store environment. + _dbhStoreEnv :: !StoreEnv, + -- | The only store for this lmdb database. + -- The account map functions as a persistent @AccountAddress -> Maybe AccountIndex@ mapping. + _dbhAccountMapStore :: !AccountMapStore + } + +makeClassy ''DatabaseHandlers + +-- | The number of stores in the LMDB environment for 'DatabaseHandlers'. +databaseCount :: Int +databaseCount = 1 + +-- | Database growth size increment. +-- This is currently set at 4MB, and must be a multiple of the page size. +-- For reference: ~ 90k accounts takes up around 7MB, so this should ensure not much resizing required. +dbStepSize :: Int +dbStepSize = 2 ^ (22 :: Int) -- 4MB + +-- ** Initialization + +-- | Initialize database handlers. +-- The size will be rounded up to a multiple of 'dbStepSize'. +-- (This ensures in particular that the size is a multiple of the page size, which is required by +-- LMDB.) +makeDatabaseHandlers :: + -- | Path of database + FilePath -> + -- | Open read only + Bool -> + IO DatabaseHandlers +makeDatabaseHandlers accountMapDir readOnly = do + _dbhStoreEnv <- makeStoreEnv' dbStepSize defaultMaxStepSize + -- here nobody else has access to the environment, so we need not lock + let env = _dbhStoreEnv ^. seEnv + mdb_env_set_mapsize env dbStepSize + mdb_env_set_maxdbs env databaseCount + mdb_env_set_maxreaders env 126 + mdb_env_open env accountMapDir [MDB_RDONLY | readOnly] + transaction _dbhStoreEnv readOnly $ \txn -> do + _dbhAccountMapStore <- + AccountMapStore + <$> mdb_dbi_open' + txn + (Just accountMapStoreName) + [MDB_CREATE | not readOnly] + return DatabaseHandlers{..} + +-- | Create the lmdb stores and return back database handlers for interacting with it. +-- This simply loads the references and does not initialize the databases. +-- The initial environment size is set to 'dbStepSize' (4MB). +-- Note that this function creates the directory for the database if not already present at the provided +-- path and any missing parent directories. +openDatabase :: FilePath -> IO DatabaseHandlers +openDatabase accountMapDir = do + createDirectoryIfMissing True accountMapDir + makeDatabaseHandlers accountMapDir False + +-- | Close the database. The database should not be used after it is closed. +closeDatabase :: DatabaseHandlers -> IO () +closeDatabase dbHandlers = runInBoundThread $ mdb_env_close $ dbHandlers ^. dbhStoreEnv . seEnv + +-- ** Monad implementation + +-- | The 'AccountMapStoreMonad' for interacting with the LMDB database. +newtype AccountMapStoreMonad (m :: Type -> Type) (a :: Type) = AccountMapStoreMonad {runAccountMapStoreMonad :: m a} + deriving (Functor, Applicative, Monad, MonadIO, MonadThrow, MonadCatch, MonadLogger, MonadReader r, MonadState s, TimeMonad) + deriving (MonadTrans) via IdentityT + +deriving instance (MonadProtocolVersion m) => MonadProtocolVersion (AccountMapStoreMonad m) + +instance + ( MonadReader r m, + HasDatabaseHandlers r, + MonadIO m, + MonadLogger m + ) => + MonadAccountMapStore (AccountMapStoreMonad m) + where + insertAccounts accounts = do + dbh <- ask + asWriteTransaction (dbh ^. dbhStoreEnv) $ \txn -> do + forM_ accounts $ \(accAddr, accIndex) -> do + storeRecord txn (dbh ^. dbhAccountMapStore) accAddr accIndex + + lookupAccountIndexViaEquivalence a@(AccountAddressEq (AccountAddress accAddr)) = do + dbh <- ask + asReadTransaction (dbh ^. dbhStoreEnv) $ \txn -> + withCursor txn (dbh ^. dbhAccountMapStore) $ \cursor -> do + withMDB_val accLookupKey $ \k -> do + getCursor (CursorMoveTo k) cursor >>= \case + Nothing -> return Nothing + Just (Left err) -> throwM $ DatabaseInvariantViolation err + Just (Right (foundAccAddr, accIdx)) -> + -- we need to check equivalence here as we are performing + -- prefix lookup in the lmdb database, so if the account does not exist + -- then the lmdb query would return the "next" account address + -- by lexicographic order of account address. + if a == accountAddressEmbed foundAccAddr + then return $ Just accIdx + else return Nothing + where + -- The key to use for looking up an account. + -- We do a prefix lookup on the first 29 bytes of the account address as + -- the last 3 bytes are reserved for aliases. + accLookupKey = BS.take accountAddressPrefixSize $ FBS.toByteString accAddr + + lookupAccountIndexViaExactness addr = do + dbh <- ask + asReadTransaction (dbh ^. dbhStoreEnv) $ \txn -> loadRecord txn (dbh ^. dbhAccountMapStore) addr + + getAllAccounts maxAccountIndex = do + dbh <- ask + asReadTransaction (dbh ^. dbhStoreEnv) $ \txn -> + withCursor txn (dbh ^. dbhAccountMapStore) $ \cursor -> + let go !accum Nothing = return accum + go !accum (Just (Right acc@(_, accIdx))) = do + -- We only accumulate accounts which have an @AccountIndex@ at most + -- the provided one. + if accIdx <= maxAccountIndex + then go (acc : accum) =<< getCursor CursorNext cursor + else go accum =<< getCursor CursorNext cursor + go _ (Just (Left err)) = throwM $ DatabaseInvariantViolation err + in go [] =<< getCursor CursorFirst cursor + + getNumberOfAccounts = do + dbh <- ask + asReadTransaction (dbh ^. dbhStoreEnv) $ \txn -> databaseSize txn (dbh ^. dbhAccountMapStore) + + reconstruct accounts = do + dbh <- ask + asWriteTransaction (dbh ^. dbhStoreEnv) $ \txn -> do + deleteAll txn (dbh ^. dbhAccountMapStore) + forM_ accounts $ \(accAddr, accIndex) -> do + storeRecord txn (dbh ^. dbhAccountMapStore) accAddr accIndex diff --git a/concordium-consensus/src/Concordium/GlobalState/BlockState.hs b/concordium-consensus/src/Concordium/GlobalState/BlockState.hs index 0878270817..65c9038329 100644 --- a/concordium-consensus/src/Concordium/GlobalState/BlockState.hs +++ b/concordium-consensus/src/Concordium/GlobalState/BlockState.hs @@ -62,6 +62,7 @@ import Data.Word import qualified Concordium.Crypto.SHA256 as H import Concordium.GlobalState.Account +import qualified Concordium.GlobalState.AccountMap.DifferenceMap as DiffMap import Concordium.GlobalState.Classes import Concordium.GlobalState.Persistent.BlobStore import qualified Concordium.GlobalState.Wasm as GSWasm @@ -1385,6 +1386,36 @@ class (BlockStateOperations m, FixedSizeSerialization (BlockStateRef m)) => Bloc -- | Ensure that a block state is stored and return a reference to it. saveBlockState :: BlockState m -> m (BlockStateRef m) + -- | Ensure that any accounts created in a block are persisted. + -- This should be called when a block is being finalized. + -- + -- Precondition: The block state must be in memory and it must not have been archived. + saveAccounts :: BlockState m -> m () + + -- | Reconstructs the account difference map and return it. + -- This function is used for blocks that are stored but are not finalized (in particular, certified blocks) + -- since only the accounts for finalized blocks are stored in the LMDB store. + -- + -- Preconditions: + -- * This function MUST only be called on a certified block. + -- * This function MUST only be called on a block state that does not already + -- have a difference map. + -- * The provided list of accounts MUST correspond to the accounts created in the block, + -- and the account addresses in the list MUST be by order of creation. + -- * The provided difference map reference MUST be the one of the parent block. + -- + -- This function should only be used when starting from an already initialized state, and hence + -- we need to reconstruct the difference map since the accounts are not yet finalized. + reconstructAccountDifferenceMap :: + -- | The block state to reconstruct the difference map for. + BlockState m -> + -- | The difference map reference from the parent block's state. + DiffMap.DifferenceMapReference -> + -- | The account addresses created in the block, in order of creation. + [AccountAddress] -> + -- | Reference to the new difference map for the block. + m DiffMap.DifferenceMapReference + -- | Load a block state from a reference, given its state hash if provided, -- otherwise calculate the state hash upon loading. -- In particular the 'StateHash' should be supplied if loading a non-genesis block state. @@ -1410,6 +1441,12 @@ class (BlockStateOperations m, FixedSizeSerialization (BlockStateRef m)) => Bloc -- and update sequence numbers populated. cacheBlockStateAndGetTransactionTable :: BlockState m -> m TransactionTable + -- | Populate the LMDB account map if it has not already been initialized. + -- If the lmdb store has already been initialized, then this function does nothing. + -- Otherwise this function populates the lmdb backed account map with the accounts + -- present in the account table of the block state. + tryPopulateAccountMap :: BlockState m -> m () + instance (Monad (t m), MonadTrans t, ModuleQuery m) => ModuleQuery (MGSTrans t m) where getModuleArtifact = lift . getModuleArtifact {-# INLINE getModuleArtifact #-} @@ -1671,24 +1708,29 @@ instance (Monad (t m), MonadTrans t, BlockStateStorage m) => BlockStateStorage ( purgeBlockState = lift . purgeBlockState archiveBlockState = lift . archiveBlockState saveBlockState = lift . saveBlockState + saveAccounts = lift . saveAccounts + reconstructAccountDifferenceMap bs parentMap = lift . reconstructAccountDifferenceMap bs parentMap loadBlockState hsh = lift . loadBlockState hsh serializeBlockState = lift . serializeBlockState blockStateLoadCallback = lift blockStateLoadCallback collapseCaches = lift collapseCaches cacheBlockState = lift . cacheBlockState cacheBlockStateAndGetTransactionTable = lift . cacheBlockStateAndGetTransactionTable + tryPopulateAccountMap = lift . tryPopulateAccountMap {-# INLINE thawBlockState #-} {-# INLINE freezeBlockState #-} {-# INLINE dropUpdatableBlockState #-} {-# INLINE purgeBlockState #-} {-# INLINE archiveBlockState #-} {-# INLINE saveBlockState #-} + {-# INLINE reconstructAccountDifferenceMap #-} {-# INLINE loadBlockState #-} {-# INLINE serializeBlockState #-} {-# INLINE blockStateLoadCallback #-} {-# INLINE collapseCaches #-} {-# INLINE cacheBlockState #-} {-# INLINE cacheBlockStateAndGetTransactionTable #-} + {-# INLINE tryPopulateAccountMap #-} deriving via (MGSTrans MaybeT m) instance (BlockStateQuery m) => BlockStateQuery (MaybeT m) deriving via (MGSTrans MaybeT m) instance (AccountOperations m) => AccountOperations (MaybeT m) diff --git a/concordium-consensus/src/Concordium/GlobalState/LMDB/Helpers.hs b/concordium-consensus/src/Concordium/GlobalState/LMDB/Helpers.hs index d96530778f..45302a5426 100644 --- a/concordium-consensus/src/Concordium/GlobalState/LMDB/Helpers.hs +++ b/concordium-consensus/src/Concordium/GlobalState/LMDB/Helpers.hs @@ -11,10 +11,18 @@ -- provide a type-safe abstraction over cursors. module Concordium.GlobalState.LMDB.Helpers ( -- * Database environment. - StoreEnv, + StoreEnv (..), makeStoreEnv, + makeStoreEnv', withWriteStoreEnv, seEnv, + seStepSize, + seMaxStepSize, + defaultEnvSize, + defaultStepSize, + defaultMaxStepSize, + resizeDatabaseHandlers, + resizeOnResized, -- * Database queries and updates. MDBDatabase (..), @@ -43,13 +51,21 @@ module Concordium.GlobalState.LMDB.Helpers ( -- * Low level operations. byteStringFromMDB_val, unsafeByteStringFromMDB_val, + withMDB_val, + + -- * Helpers for reading and writing to a lmdb store. + asReadTransaction, + asWriteTransaction, ) where +import Concordium.Logger import Control.Concurrent (runInBoundThread, yield) import Control.Concurrent.MVar -import Control.Exception +import Control.Exception (assert) import Control.Monad +import Control.Monad.Catch +import Control.Monad.IO.Class import Control.Monad.Trans.Except import Data.ByteString import qualified Data.ByteString.Lazy as LBS @@ -271,18 +287,53 @@ data StoreEnv = StoreEnv _seEnv :: !MDB_env, -- | Lock to quard access to the environment. When resizing the environment -- we must ensure that there are no outstanding transactions. - _seEnvLock :: !RWLock + _seEnvLock :: !RWLock, + -- | Database growth size increment. + -- Must be a multiple of the page size. + _seStepSize :: !Int, + -- | Maximum step to increment the database size. + _seMaxStepSize :: !Int } makeLenses ''StoreEnv +-- | Default start environment size. +-- Tree state database sizes for historical protocol versions have been between 7-60 times +-- the 'defaultEnvSize'. +defaultEnvSize :: Int +defaultEnvSize = 2 ^ (27 :: Int) -- 128MB + +-- | Database growth size increment. +-- This is currently set at 64MB, and must be a multiple of the page size. +defaultStepSize :: Int +defaultStepSize = 2 ^ (26 :: Int) -- 64MB + +-- | Maximum step to increment the database size. +-- A ceiling that supposedly never gets hit. +-- We need some bound as we're growing the environment exponentially when +-- transactions fail and we resize recursively. +defaultMaxStepSize :: Int +defaultMaxStepSize = 2 ^ (30 :: Int) -- 1GB + -- | Construct a new LMDB environment with associated locks that protect its use. -makeStoreEnv :: IO StoreEnv -makeStoreEnv = do +makeStoreEnv' :: + -- | Initial database growth when resizing the environment. + -- Precondition: Must be a multiple of the OS page size. + Int -> + -- | Maximum database growth size. + Int -> + -- | The resulting environment 'StoreEnv'. + IO StoreEnv +makeStoreEnv' _seStepSize _seMaxStepSize = do _seEnv <- mdb_env_create _seEnvLock <- initializeLock return StoreEnv{..} +-- | Construct a new LMDB environment with assoicated locks that protects it, +-- with default environment paremeters. +makeStoreEnv :: IO StoreEnv +makeStoreEnv = makeStoreEnv' defaultStepSize defaultMaxStepSize + -- | Acquire exclusive access to the LMDB environment and perform the given action. -- The IO action should not leak the 'MDB_env'. withWriteStoreEnv :: StoreEnv -> (MDB_env -> IO a) -> IO a @@ -415,10 +466,13 @@ data CursorMove CursorNext | -- | Move to the previous key CursorPrevious + | -- | Move to key greater than or equal to provided key. + CursorMoveTo MDB_val -- | Move a cursor and read the key and value at the new location. getPrimitiveCursor :: CursorMove -> PrimitiveCursor -> IO (Maybe (MDB_val, MDB_val)) getPrimitiveCursor movement PrimitiveCursor{..} = do + mapM_ (poke pcKeyPtr) mKey res <- mdb_cursor_get' moveOp pcCursor pcKeyPtr pcValPtr if res then do @@ -427,12 +481,13 @@ getPrimitiveCursor movement PrimitiveCursor{..} = do return $ Just (key, val) else return Nothing where - moveOp = case movement of - CursorCurrent -> MDB_GET_CURRENT - CursorFirst -> MDB_FIRST - CursorLast -> MDB_LAST - CursorNext -> MDB_NEXT - CursorPrevious -> MDB_PREV + (moveOp, mKey) = case movement of + CursorCurrent -> (MDB_GET_CURRENT, Nothing) + CursorFirst -> (MDB_FIRST, Nothing) + CursorLast -> (MDB_LAST, Nothing) + CursorNext -> (MDB_NEXT, Nothing) + CursorPrevious -> (MDB_PREV, Nothing) + CursorMoveTo k -> (MDB_SET_RANGE, Just k) -- | Move a cursor to a specified key. movePrimitiveCursor :: MDB_val -> PrimitiveCursor -> IO (Maybe MDB_val) @@ -650,3 +705,52 @@ databaseSize :: db -> IO Word64 databaseSize txn dbi = fromIntegral . ms_entries <$> mdb_stat' txn (mdbDatabase dbi) + +-- | Increase the database size by at least the supplied size. +-- The provided size will be rounded up to a multiple of 'seStepSize'. +-- This ensures that the new size is a multiple of the page size, which is required by lmdb. +resizeDatabaseHandlers :: (MonadIO m, MonadLogger m) => StoreEnv -> Int -> m () +resizeDatabaseHandlers env delta = do + envInfo <- liftIO $ mdb_env_info (env ^. seEnv) + let oldMapSize = fromIntegral $ me_mapsize envInfo + stepSize = env ^. seStepSize + newMapSize = oldMapSize + (delta + stepSize - delta `mod` stepSize) + _storeEnv = env + logEvent LMDB LLDebug $ "Resizing database from " ++ show oldMapSize ++ " to " ++ show newMapSize + liftIO . withWriteStoreEnv env $ flip mdb_env_set_mapsize newMapSize + +-- | Perform a database action and resize the LMDB map if the file size has changed. +resizeOnResized :: (MonadIO m, MonadCatch m) => StoreEnv -> m a -> m a +resizeOnResized se a = inner + where + inner = handleJust checkResized onResized a + checkResized LMDB_Error{..} = guard (e_code == Right MDB_MAP_RESIZED) + onResized _ = do + liftIO (withWriteStoreEnv se $ flip mdb_env_set_mapsize 0) + inner + +-- | Run a read-only transaction. +asReadTransaction :: (MonadIO m) => StoreEnv -> (MDB_txn -> IO a) -> m a +asReadTransaction env t = do + liftIO $ transaction env True t + +-- | Run a write transaction. If the transaction fails due to the database being full, this resizes +-- the database and retries the transaction. +asWriteTransaction :: (MonadIO m, MonadLogger m) => StoreEnv -> (MDB_txn -> IO a) -> m a +asWriteTransaction env t = do + let doTransaction = transaction env False t + inner step = do + r <- liftIO $ tryJust selectDBFullError doTransaction + case r of + Left _ -> do + -- We resize by the step size initially, and by double for each successive + -- failure. + resizeDatabaseHandlers env step + inner (min (step * 2) (env ^. seMaxStepSize)) + Right res -> return res + inner $ env ^. seStepSize + where + -- only handle the db full error and propagate other exceptions. + selectDBFullError = \case + (LMDB_Error _ _ (Right MDB_MAP_FULL)) -> Just () + _ -> Nothing diff --git a/concordium-consensus/src/Concordium/GlobalState/Persistent/Accounts.hs b/concordium-consensus/src/Concordium/GlobalState/Persistent/Accounts.hs index 609f9102fe..f33c6d1540 100644 --- a/concordium-consensus/src/Concordium/GlobalState/Persistent/Accounts.hs +++ b/concordium-consensus/src/Concordium/GlobalState/Persistent/Accounts.hs @@ -10,31 +10,81 @@ -- for pattern matching. (See: https://gitlab.haskell.org/ghc/ghc/-/issues/20896) {-# OPTIONS_GHC -Wno-redundant-constraints #-} +-- | +-- * Adding accounts +-- When an account is added (via 'putNewAccount') then it is first added to the ‘DifferenceMap', +-- it is kept in memory for the block until the block either gets finalized or pruned. +-- If a block is pruned then the retaining pointers are dropped and thus the block and associated ‘DifferenceMap' is evicted from memory. +-- +-- When the accounts structure is finalized, then 'writeAccountsCreated' must be invoked in order to store the newly created accounts +-- in the LMDB backed account map. +-- When thawing from a non-persisted block then the difference map is being inherited by the new thawed updatable block, +-- thus the difference map potentially forms a chain of difference map "down" until the highest persisted block. +-- +-- * Startup flow +-- When a consensus runner starts up it can either be via an existing state or +-- from a fresh state (i.e. via a provided genesis configuration) +-- +-- In the latter case then when starting up it is checked whether the lmdb backed account map is populated or not. +-- If the map is not populated then it is being populated by traversing the account table +-- and writing all @AccountAddress -> AccountIndex@ mappings into +-- the lmdb store in one transaction and then it proceeds as normal. +-- On the other hand, if the lmdb backed account map is already populated then the startup procedure will skip the populating step ('tryPopulateLMDBStore'). +-- +-- For consensus version 1, then the associated 'DiffMap.DifferenceMap' is reconstructed via 'reconstructDifferenceMap' for certified blocks. +-- +-- When starting up from a fresh genesis configuration then as part of creating the genesis state, +-- then the difference map is being built containing all accounts present in the genesis configuration. +-- When the genesis block is being written to disk, then so is the ‘DifferenceMap' +-- via the ‘storeUpdate' implementation of the accounts structure. +-- +-- General flow +-- The account map resides in its own lmdb database and functions across protocol versions. +-- For non-persisted blocks, then the ‘DifferenceMap' is 'DiffMap.DifferenceMapReference', +-- i.e. either @IORef Nothing@ or @IORef (Just DifferenceMap)@ depending on whether the block is written to disk. +-- When a block state is thawed (made ready for modification), then a new 'DiffMap.DifferenceMap' is created for the @Accounts pv@ structure +-- of the 'UpdatableBlockState' which has a parent pointer on the 'DiffMap.DifferenceMap' of the block state that was thawed. +-- +-- The 'putNewAccount' function creates a new 'DifferenceMap' on demand, hence a new 'Accounts' is initialized with a @accountDiffMap@ set to @IORef Nothing@. +-- Subsequent accounts created are then being added to the difference map created by the first invocation of 'putNewAccount'. +-- Blocks that are persisted always have a @IORef Nothing@ 'accountDiffMapRef'. +-- +-- The lmdb backed account map consists of a single lmdb store indexed by AccountAddresses and values are the associated ‘AccountIndex' for each account. +-- +-- (The ‘DifferenceMap' consists of a @Map AccountAddressEq (AccountIndex, AccountAddress)@ which retains the accounts that have been added to the chain for the associated block.) +-- The equivalence class 'AccountAddressEq' is used for looking up accounts in the 'DifferenceMap'. The values are pairs @(AccountIndex, AccountAddress)@ where the +-- 'AccountIndex' determines the location of the associated account in the account table. The second component (the 'AccountAddress) is the canonical account address of +-- the account i.e. the account address that is derived from the reg id. +-- Note that the ‘DifferenceMap' can potentially retain a pointer to a parent ‘DifferenceMap', i.e. @Maybe DifferenceMap@. +-- If this is @Nothing@ then it means that the parent block is finalized or no accounts have been added. +-- If the parent map yields a ‘DifferenceMap' then the parent block is not finalized yet, and so the ‘DifferenceMap' uses this parent map +-- for keeping track of non persisted accounts for supporting e.g. queries via the ‘AccountAddress'. module Concordium.GlobalState.Persistent.Accounts where -import Control.Monad -import Data.Foldable (foldlM) -import qualified Data.Map.Strict as Map -import Data.Maybe -import Data.Serialize -import Lens.Micro.Platform - +import qualified Concordium.Crypto.SHA256 as H +import qualified Concordium.GlobalState.AccountMap as OldMap +import qualified Concordium.GlobalState.AccountMap.DifferenceMap as DiffMap +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap +import Concordium.GlobalState.Parameters import Concordium.GlobalState.Persistent.Account import Concordium.GlobalState.Persistent.BlobStore import Concordium.GlobalState.Persistent.Cache import Concordium.GlobalState.Persistent.CachedRef -import qualified Concordium.GlobalState.Persistent.Trie as Trie -import qualified Concordium.ID.Types as ID -import Concordium.Types -import Concordium.Utils.Serialization.Put - -import qualified Concordium.Crypto.SHA256 as H -import qualified Concordium.GlobalState.AccountMap as AccountMap -import Concordium.GlobalState.Parameters import Concordium.GlobalState.Persistent.LFMBTree (LFMBTree') import qualified Concordium.GlobalState.Persistent.LFMBTree as L +import qualified Concordium.GlobalState.Persistent.Trie as Trie import Concordium.ID.Parameters +import qualified Concordium.ID.Types as ID +import Concordium.Types import Concordium.Types.HashableTo +import Concordium.Types.Option (Option (..)) +import Concordium.Utils.Serialization.Put +import Control.Monad.Reader +import Data.Foldable (foldlM) +import Data.IORef +import qualified Data.Map.Strict as Map +import Data.Maybe +import Data.Serialize -- | Representation of the set of accounts on the chain. -- Each account has an 'AccountIndex' which is the order @@ -42,7 +92,7 @@ import Concordium.Types.HashableTo -- -- The operations on 'Accounts', when used correctly, maintain the following invariants: -- --- * Every @(address, index)@ pair in 'accountMap' has a corresponding account +-- * Every @(address, index)@ pair in the LMDB account map and difference map has a corresponding account -- in 'accountTable' with the given index and address. -- * Every @(index, account)@ pair in 'accountTable' has a corresponding entry -- in 'accountMap', which maps the account address to @index@. @@ -69,96 +119,171 @@ import Concordium.Types.HashableTo -- hence the current solution was chosen. Caching by account index (probably with an LRU strategy) -- would likely be a more effective strategy over all. data Accounts (pv :: ProtocolVersion) = Accounts - { -- | Unique index of accounts by 'AccountAddress' - accountMap :: !(AccountMap.PersistentAccountMap pv), - -- | Hashed Merkle-tree of the accounts + { -- | Hashed Merkle-tree of the accounts accountTable :: !(LFMBTree' AccountIndex HashedBufferedRef (AccountRef (AccountVersionFor pv))), -- | Persisted representation of the map from registration ids to account indices. - accountRegIdHistory :: !(Trie.TrieN UnbufferedFix ID.RawCredentialRegistrationID AccountIndex) + accountRegIdHistory :: !(Trie.TrieN UnbufferedFix ID.RawCredentialRegistrationID AccountIndex), + -- | An in-memory difference map used for keeping track of accounts that are + -- added in blocks which are not yet finalized. + -- In particular the difference map retains accounts created, but not + -- yet stored in the LMDB backed account map. + accountDiffMapRef :: !DiffMap.DifferenceMapReference } +instance (IsProtocolVersion pv) => Show (Accounts pv) where + show accts = "Accounts: " <> show (accountTable accts) + -- | A constraint that ensures a monad @m@ supports the persistent account operations. -- This essentially requires that the monad support 'MonadBlobStore', and 'MonadCache' for --- the account cache. +-- the account cache and 'MonadAccountMapStore' for the persistent account map. type SupportsPersistentAccount pv m = ( IsProtocolVersion pv, MonadBlobStore m, - MonadCache (AccountCache (AccountVersionFor pv)) m + MonadCache (AccountCache (AccountVersionFor pv)) m, + LMDBAccountMap.MonadAccountMapStore m ) -instance (IsProtocolVersion pv) => Show (Accounts pv) where - show a = show (accountTable a) - instance (SupportsPersistentAccount pv m) => MHashableTo m H.Hash (Accounts pv) where getHashM Accounts{..} = getHashM accountTable +-- | Write accounts created for this block or any non-persisted parent block. +-- Note that this also empties the difference map for this block. +-- This function MUST be called whenever a block is finalized. +writeAccountsCreated :: (SupportsPersistentAccount pv m) => Accounts pv -> m () +writeAccountsCreated Accounts{..} = do + mAccountsCreated <- liftIO $ readIORef accountDiffMapRef + forM_ mAccountsCreated $ \accountsCreated -> do + listOfAccountsCreated <- liftIO $ DiffMap.flatten accountsCreated + -- Write all accounts from the difference map to the lmdb backed account map. + LMDBAccountMap.insertAccounts listOfAccountsCreated + -- Finally, clear the difference map for this block state and all parent block states. + liftIO $ do + DiffMap.clearReferences accountsCreated + atomicWriteIORef accountDiffMapRef Absent + +-- | Create a new @Accounts pv@ structure from the provided one. +-- This function creates a new 'DiffMap.DifferenceMap' for the resulting @Accounts pv@ which +-- has a reference to the difference map of the provided @Accounts pv@. +mkNewChildDifferenceMap :: (SupportsPersistentAccount pv m) => Accounts pv -> m (Accounts pv) +mkNewChildDifferenceMap accts@Accounts{..} = do + newDiffMapRef <- liftIO $ newIORef $ Present $ DiffMap.empty accountDiffMapRef + return accts{accountDiffMapRef = newDiffMapRef} + +-- | Create and set the 'DiffMap.DifferenceMap' for the provided @Accounts pv@. +-- This function constructs the difference map for the block such that the assoicated difference map +-- and lmdb backed account map are consistent with the account table. +-- +-- The function is highly unsafe and can cause state invariant failures if not all of the +-- below preconditions are respected. +-- Precondition: +-- * The function assumes that the account table already contains every account added for the block state. +-- * The provided @DiffMap.DifferenceMapReference@ MUST correspond to the parent map. +-- * The provided list of accounts MUST be in ascending order of account index, hence the list of accounts +-- MUST be provided in the order of which the corresponding credential deployment transactions were executed. +reconstructDifferenceMap :: + (SupportsPersistentAccount pv m) => + -- | Reference to the parent difference map. + DiffMap.DifferenceMapReference -> + -- | Account addresses to add to the difference map. + -- The list MUST be in ascending order of 'AccountIndex'. + [AccountAddress] -> + -- | The accounts to write difference map to. + Accounts pv -> + -- | Reference to the newly created difference map. + m DiffMap.DifferenceMapReference +reconstructDifferenceMap parentRef listOfAccounts Accounts{..} = do + -- As it is presumed that the account table already yields any accounts added, then + -- in order to the obtain the account indices we subtract the number of accounts missing + -- missing in the lmdb account map from the total number of accounts, hence obtaining the first @AccountIndex@ + -- to use for adding new accounts to the lmdb backed account map. + let diffMap' = DiffMap.fromList parentRef $ zip listOfAccounts [AccountIndex (L.size accountTable - fromIntegral (length listOfAccounts)) ..] + liftIO $ atomicWriteIORef accountDiffMapRef $ Present diffMap' + return accountDiffMapRef + +-- | Determine whether the given protocol version requires an account map entry +-- in the storage of 'Accounts'. We require this for all protocol versions that exist +-- prior to the account map storage revision (i.e. 'P6' and earlier) for compatibility +-- with databases created by versions of the node that store the account map in +-- the block state. +storeRequiresAccountMap :: SProtocolVersion pv -> Bool +storeRequiresAccountMap spv = demoteProtocolVersion spv <= P6 + instance (SupportsPersistentAccount pv m) => BlobStorable m (Accounts pv) where storeUpdate Accounts{..} = do - (pMap, accountMap') <- storeUpdate accountMap + -- put an empty 'OldMap.PersistentAccountMap'. + -- In earlier versions of the node the above mentioned account map was used, + -- but this is now superseded by the 'LMDBAccountMap.MonadAccountMapStore'. + -- We put this empty map here if the protocol version requires it in order to remain backwards compatible. + pAccountMap <- + if storeRequiresAccountMap (protocolVersion @pv) + then fst <$> storeUpdate (OldMap.empty @pv @BufferedFix) + else return (return ()) (pTable, accountTable') <- storeUpdate accountTable (pRegIdHistory, regIdHistory') <- storeUpdate accountRegIdHistory let newAccounts = Accounts - { accountMap = accountMap', - accountTable = accountTable', - accountRegIdHistory = regIdHistory' + { accountTable = accountTable', + accountRegIdHistory = regIdHistory', + .. } - return (pMap >> pTable >> pRegIdHistory, newAccounts) + return (pAccountMap >> pTable >> pRegIdHistory, newAccounts) load = do - maccountMap <- load + -- If we're on protocol version 6 or older, then load the persistent account map and throw it away as + -- the 'OldMap.PersistentAccountMap' is now superseded by the LMDBAccountMap.MonadAccountMapStore. + when (storeRequiresAccountMap (protocolVersion @pv)) $ do + void (load :: Get (m (OldMap.PersistentAccountMap pv))) maccountTable <- load mrRIH <- load return $ do - accountMap <- maccountMap accountTable <- maccountTable accountRegIdHistory <- mrRIH + accountDiffMapRef <- DiffMap.newEmptyReference return $ Accounts{..} instance (SupportsPersistentAccount pv m, av ~ AccountVersionFor pv) => Cacheable1 m (Accounts pv) (PersistentAccount av) where liftCache cch accts@Accounts{..} = do - acctMap <- cache accountMap acctTable <- liftCache (liftCache @_ @(HashedCachedRef (AccountCache av) (PersistentAccount av)) cch) accountTable return - accts - { accountMap = acctMap, - accountTable = acctTable - } + accts{accountTable = acctTable} --- | An 'Accounts' with no accounts. -emptyAccounts :: Accounts pv -emptyAccounts = Accounts AccountMap.empty L.empty Trie.empty +-- | Create a new empty 'Accounts' structure. +emptyAccounts :: (MonadIO m) => m (Accounts pv) +emptyAccounts = do + accountDiffMapRef <- liftIO DiffMap.newEmptyReference + return $ Accounts L.empty Trie.empty accountDiffMapRef -- | Add a new account. Returns @Just idx@ if the new account is fresh, i.e., the address does not exist, -- or @Nothing@ in case the account already exists. In the latter case there is no change to the accounts structure. putNewAccount :: (SupportsPersistentAccount pv m) => PersistentAccount (AccountVersionFor pv) -> Accounts pv -> m (Maybe AccountIndex, Accounts pv) -putNewAccount !acct accts0 = do +putNewAccount !acct a0@Accounts{..} = do addr <- accountCanonicalAddress acct - (existingAccountId, newAccountMap) <- AccountMap.maybeInsert addr acctIndex (accountMap accts0) - if isNothing existingAccountId - then do - (_, newAccountTable) <- L.append acct (accountTable accts0) - return (Just acctIndex, accts0{accountMap = newAccountMap, accountTable = newAccountTable}) - else return (Nothing, accts0) - where - acctIndex = fromIntegral $ L.size (accountTable accts0) + exists addr a0 >>= \case + True -> return (Nothing, a0) + False -> do + (accIdx, newAccountTable) <- L.append acct accountTable + mAccountDiffMap <- liftIO $ readIORef accountDiffMapRef + accountDiffMap' <- case mAccountDiffMap of + Absent -> do + -- create a difference map for this block state with an @Absent@ as the parent. + freshDifferenceMap <- liftIO DiffMap.newEmptyReference + return $ DiffMap.insert addr accIdx $ DiffMap.empty freshDifferenceMap + Present accDiffMap -> do + -- reuse the already existing difference map for this block state. + return $ DiffMap.insert addr accIdx accDiffMap + liftIO $ atomicWriteIORef accountDiffMapRef (Present accountDiffMap') + return (Just accIdx, a0{accountTable = newAccountTable}) -- | Construct an 'Accounts' from a list of accounts. Inserted in the order of the list. fromList :: (SupportsPersistentAccount pv m) => [PersistentAccount (AccountVersionFor pv)] -> m (Accounts pv) -fromList = foldlM insert emptyAccounts +fromList accs = do + accum <- emptyAccounts + foldlM insert accum accs where insert accounts account = snd <$> putNewAccount account accounts -- | Determine if an account with the given address exists. -exists :: (IsProtocolVersion pv, MonadBlobStore m) => AccountAddress -> Accounts pv -> m Bool -exists addr Accounts{..} = AccountMap.isAddressAssigned addr accountMap - --- | Retrieve an account with the given address. --- Returns @Nothing@ if no such account exists. -getAccount :: (SupportsPersistentAccount pv m) => AccountAddress -> Accounts pv -> m (Maybe (PersistentAccount (AccountVersionFor pv))) -getAccount addr Accounts{..} = - AccountMap.lookup addr accountMap >>= \case - Nothing -> return Nothing - Just ai -> L.lookup ai accountTable +exists :: (SupportsPersistentAccount pv m) => AccountAddress -> Accounts pv -> m Bool +exists addr accts = isJust <$> getAccountIndex addr accts -- | Retrieve an account associated with the given credential registration ID. -- Returns @Nothing@ if no such account exists. @@ -168,35 +293,64 @@ getAccountByCredId cid accs@Accounts{..} = Nothing -> return Nothing Just ai -> fmap (ai,) <$> indexedAccount ai accs --- | Get the account at a given index (if any). -getAccountIndex :: (IsProtocolVersion pv, MonadBlobStore m) => AccountAddress -> Accounts pv -> m (Maybe AccountIndex) -getAccountIndex addr Accounts{..} = AccountMap.lookup addr accountMap +-- | Get the 'AccountIndex' for the provided 'AccountAddress' (if any). +-- First try lookup in the in-memory difference map associated with the the provided 'Accounts pv', +-- if no account could be looked up, then we fall back to the lmdb backed account map. +-- +-- If account alises are supported then the equivalence class 'AccountAddressEq' is used for determining +-- whether the provided @AccountAddress@ is in the map, otherwise we check for exactness. +getAccountIndex :: forall pv m. (SupportsPersistentAccount pv m) => AccountAddress -> Accounts pv -> m (Maybe AccountIndex) +getAccountIndex addr Accounts{..} = do + mAccountDiffMap <- liftIO $ readIORef accountDiffMapRef + case mAccountDiffMap of + Absent -> lookupDisk 0 + Present accDiffMap -> + if supportsAccountAliases (protocolVersion @pv) + then + DiffMap.lookupViaEquivalenceClass (accountAddressEmbed addr) accDiffMap >>= \case + Right accIdx -> return $ Just accIdx + Left diffMapSize -> lookupDisk $ fromIntegral diffMapSize + else + DiffMap.lookupExact addr accDiffMap >>= \case + Right accIdx -> return $ Just accIdx + Left diffMapSize -> lookupDisk $ fromIntegral diffMapSize + where + -- Lookup the 'AccountIndex' in the lmdb backed account map, + -- and make sure it's within the bounds of the account table. + -- We do the bounds check as it could be that the lmdb backed account map + -- yields accounts which are not yet present in the @accountTable@. + -- In particular this can be the case if finalized blocks have been rolled + -- back as part of database recovery. + -- + -- If an account is not present in the difference map, then it must have an account index where + -- @account index < number of accounts in account table - number of accounts in the difference map(s).@ + lookupDisk diffMapSize = + if supportsAccountAliases (protocolVersion @pv) + then withSafeBounds <$> LMDBAccountMap.lookupAccountIndexViaEquivalence (accountAddressEmbed addr) + else withSafeBounds <$> LMDBAccountMap.lookupAccountIndexViaExactness addr + where + withSafeBounds Nothing = Nothing + withSafeBounds (Just accIdx@(AccountIndex k)) = if k <= fromIntegral (L.size accountTable) - (1 + diffMapSize) then Just accIdx else Nothing + +-- | Retrieve an account with the given address. +-- Returns @Nothing@ if no such account exists. +getAccount :: (SupportsPersistentAccount pv m) => AccountAddress -> Accounts pv -> m (Maybe (PersistentAccount (AccountVersionFor pv))) +getAccount addr accts = fmap snd <$> getAccountWithIndex addr accts -- | Retrieve an account and its index from a given address. -- Returns @Nothing@ if no such account exists. -getAccountWithIndex :: (SupportsPersistentAccount pv m) => AccountAddress -> Accounts pv -> m (Maybe (AccountIndex, PersistentAccount (AccountVersionFor pv))) -getAccountWithIndex addr Accounts{..} = - AccountMap.lookup addr accountMap >>= \case +getAccountWithIndex :: forall pv m. (SupportsPersistentAccount pv m) => AccountAddress -> Accounts pv -> m (Maybe (AccountIndex, PersistentAccount (AccountVersionFor pv))) +getAccountWithIndex addr accts = + getAccountIndex addr accts >>= \case Nothing -> return Nothing - Just ai -> fmap (ai,) <$> L.lookup ai accountTable + Just ai -> do + mAcc <- L.lookup ai $ accountTable accts + return $ (ai,) <$> mAcc -- | Retrieve the account at a given index. indexedAccount :: (SupportsPersistentAccount pv m) => AccountIndex -> Accounts pv -> m (Maybe (PersistentAccount (AccountVersionFor pv))) indexedAccount ai Accounts{..} = L.lookup ai accountTable --- | Retrieve an account with the given address. --- An account with the address is required to exist. -unsafeGetAccount :: (SupportsPersistentAccount pv m) => AccountAddress -> Accounts pv -> m (PersistentAccount (AccountVersionFor pv)) -unsafeGetAccount addr accts = - getAccount addr accts <&> \case - Just acct -> acct - Nothing -> error $ "unsafeGetAccount: Account " ++ show addr ++ " does not exist." - --- | Check whether the given account address would clash with any existing account address. --- The meaning of "clash" depends on the protocol version. -addressWouldClash :: (IsProtocolVersion pv, MonadBlobStore m) => AccountAddress -> Accounts pv -> m Bool -addressWouldClash addr Accounts{..} = AccountMap.addressWouldClash addr accountMap - -- | Check that an account registration ID is not already on the chain. -- See the foundation (Section 4.2) for why this is necessary. -- Return @Just ai@ if the registration ID already exists, and @ai@ is the index of the account it is or was associated with. @@ -207,10 +361,7 @@ regIdExists rid accts = Trie.lookup (ID.toRawCredRegId rid) (accountRegIdHistory recordRegId :: (MonadBlobStore m) => ID.CredentialRegistrationID -> AccountIndex -> Accounts pv -> m (Accounts pv) recordRegId rid idx accts0 = do accountRegIdHistory' <- Trie.insert (ID.toRawCredRegId rid) idx (accountRegIdHistory accts0) - return $! - accts0 - { accountRegIdHistory = accountRegIdHistory' - } + return $! accts0{accountRegIdHistory = accountRegIdHistory'} recordRegIds :: (MonadBlobStore m) => [(ID.CredentialRegistrationID, AccountIndex)] -> Accounts pv -> m (Accounts pv) recordRegIds rids accts0 = foldM (\accts (cid, idx) -> recordRegId cid idx accts) accts0 rids @@ -228,9 +379,16 @@ loadRegIds accts = Trie.toMap (accountRegIdHistory accts) -- -- This should not be used to alter the address of an account (which is -- disallowed). -updateAccounts :: (SupportsPersistentAccount pv m) => (PersistentAccount (AccountVersionFor pv) -> m (a, PersistentAccount (AccountVersionFor pv))) -> AccountAddress -> Accounts pv -> m (Maybe (AccountIndex, a), Accounts pv) -updateAccounts fupd addr a0@Accounts{..} = - AccountMap.lookup addr accountMap >>= \case +updateAccounts :: + (SupportsPersistentAccount pv m) => + ( PersistentAccount (AccountVersionFor pv) -> + m (a, PersistentAccount (AccountVersionFor pv)) + ) -> + AccountAddress -> + Accounts pv -> + m (Maybe (AccountIndex, a), Accounts pv) +updateAccounts fupd addr a0@Accounts{..} = do + getAccountIndex addr a0 >>= \case Nothing -> return (Nothing, a0) Just ai -> L.update fupd ai accountTable >>= \case @@ -256,24 +414,67 @@ updateAccountsAtIndex' fupd ai = fmap snd . updateAccountsAtIndex fupd' ai where fupd' = fmap ((),) . fupd +-- | Get a list of all account addresses and their associated account indices. +-- There are no guarantees of the order of the list. +allAccounts :: (SupportsPersistentAccount pv m) => Accounts pv -> m [(AccountAddress, AccountIndex)] +allAccounts accounts = do + mDiffMap <- liftIO $ readIORef (accountDiffMapRef accounts) + case mDiffMap of + Absent -> LMDBAccountMap.getAllAccounts $ (AccountIndex . L.size) (accountTable accounts) - 1 + Present accDiffMap -> do + -- Get all persisted accounts from the account map up to and including the last account of the account table minus what we found the in the difference map. + flattenedDiffMapAccounts <- DiffMap.flatten accDiffMap + persistedAccs <- LMDBAccountMap.getAllAccounts . AccountIndex $ L.size (accountTable accounts) - (1 + fromIntegral (length flattenedDiffMapAccounts)) + return $! persistedAccs <> flattenedDiffMapAccounts + -- | Get a list of all account addresses. -accountAddresses :: (MonadBlobStore m) => Accounts pv -> m [AccountAddress] -accountAddresses = AccountMap.addresses . accountMap +-- There are no guarantees of the order of the list. This is because the resulting list is potentially +-- a concatenation of two lists of account addresses. +accountAddresses :: (SupportsPersistentAccount pv m) => Accounts pv -> m [AccountAddress] +accountAddresses accounts = map fst <$> allAccounts accounts -- | Serialize accounts in V0 format. serializeAccounts :: (SupportsPersistentAccount pv m, MonadPut m) => GlobalContext -> Accounts pv -> m () -serializeAccounts cryptoParams accts = do - liftPut $ putWord64be $ L.size (accountTable accts) - L.mmap_ (serializeAccount cryptoParams) (accountTable accts) +serializeAccounts cryptoParams Accounts{..} = do + liftPut $ putWord64be $ L.size accountTable + L.mmap_ (serializeAccount cryptoParams) accountTable -- | Fold over the account table in ascending order of account index. foldAccounts :: (SupportsPersistentAccount pv m) => (a -> PersistentAccount (AccountVersionFor pv) -> m a) -> a -> Accounts pv -> m a foldAccounts f a accts = L.mfold f a (accountTable accts) --- | Fold over the account table in ascending order of account index. +-- | Fold over the account table in descending order of account index. foldAccountsDesc :: (SupportsPersistentAccount pv m) => (a -> PersistentAccount (AccountVersionFor pv) -> m a) -> a -> Accounts pv -> m a foldAccountsDesc f a accts = L.mfoldDesc f a (accountTable accts) +-- | Initialize the LMDB account map if it is not already. +-- If the account map has fewer accounts than the provided account table, the account map is +-- wiped and repopulated from the account table. Otherwise, the account map is unchanged. +tryPopulateLMDBStore :: (SupportsPersistentAccount pv m) => Accounts pv -> m () +tryPopulateLMDBStore accts = do + noLMDBAccounts <- LMDBAccountMap.getNumberOfAccounts + let expectedSize = L.size $ accountTable accts + -- If the number of accounts in the lmdb backed account map is + -- less than the size of the account table then we rebuild it here. + -- This should really never happen as we're writing to the lmdb backed account map + -- BEFORE the block (and assoicated block state) is written to disk. + when (noLMDBAccounts < expectedSize) $ do + -- The number of accounts in the lmdb backed account map does not match + -- the number of accounts in the account table. + -- Clear the map and reconstruct it from the accounts table. + -- This ensures consistency across restarts between the last finalized block and the account map. + LMDBAccountMap.reconstruct =<< allAccountsViaTable + where + allAccountsViaTable = + fst + <$> foldAccounts + ( \(!accum, !nextix) pacc -> do + !addr <- accountCanonicalAddress pacc + return ((addr, nextix) : accum, nextix + 1) + ) + ([], 0) + accts + -- | See documentation of @migratePersistentBlockState@. migrateAccounts :: forall oldpv pv t m. @@ -287,14 +488,13 @@ migrateAccounts :: Accounts oldpv -> t m (Accounts pv) migrateAccounts migration Accounts{..} = do - newAccountMap <- AccountMap.migratePersistentAccountMap accountMap newAccountTable <- L.migrateLFMBTree (migrateHashedCachedRef' (migratePersistentAccount migration)) accountTable -- The account registration IDs are not cached. There is a separate cache -- that is purely in-memory and just copied over. newAccountRegIds <- Trie.migrateUnbufferedTrieN return accountRegIdHistory return $! Accounts - { accountMap = newAccountMap, - accountTable = newAccountTable, - accountRegIdHistory = newAccountRegIds + { accountTable = newAccountTable, + accountRegIdHistory = newAccountRegIds, + accountDiffMapRef = accountDiffMapRef } diff --git a/concordium-consensus/src/Concordium/GlobalState/Persistent/BlobStore.hs b/concordium-consensus/src/Concordium/GlobalState/Persistent/BlobStore.hs index 1043385db5..953b4e216f 100644 --- a/concordium-consensus/src/Concordium/GlobalState/Persistent/BlobStore.hs +++ b/concordium-consensus/src/Concordium/GlobalState/Persistent/BlobStore.hs @@ -10,7 +10,6 @@ {-# LANGUAGE RankNTypes #-} {-# LANGUAGE ScopedTypeVariables #-} {-# LANGUAGE StandaloneDeriving #-} -{-# LANGUAGE TypeApplications #-} {-# LANGUAGE TypeFamilies #-} {-# LANGUAGE UndecidableInstances #-} -- FIXME: GHC 9.2.5 reports that `MonadBlobStore m` is a redundant constraint in a @@ -183,6 +182,7 @@ import Concordium.Types.Updates import Concordium.Wasm import qualified Concordium.Crypto.SHA256 as H +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap import Concordium.Types.HashableTo -- | A @BlobRef a@ represents an offset on a file, at @@ -309,24 +309,30 @@ destroyBlobStore bs@BlobStore{..} = do -- The given FilePath is a directory where the temporary blob -- store will be created. -- The blob store file is deleted afterwards. -runBlobStoreTemp :: FilePath -> BlobStoreM a -> IO a -runBlobStoreTemp dir a = bracket openf closef usef +runBlobStoreTemp :: forall m a. (MonadIO m, MonadCatch.MonadMask m) => FilePath -> BlobStoreT BlobStore m a -> m a +runBlobStoreTemp dir a = MonadCatch.bracket openf closef usef where - openf = openBinaryTempFile dir "blb.dat" - closef (tempFP, h) = do + openf = liftIO $ openBinaryTempFile dir "blb.dat" + closef (tempFP, h) = liftIO $ do hClose h performGC removeFile tempFP `catch` (\(_ :: IOException) -> return ()) + usef :: (FilePath, Handle) -> m a usef (fp, h) = do - mv <- newMVar (BlobHandle h True 0) - mmap <- newIORef BS.empty - let bscBlobStore = BlobStoreAccess mv fp mmap - (bscLoadCallback, bscStoreCallback) <- mkCallbacksFromBlobStore bscBlobStore - res <- runBlobStoreM a BlobStore{..} - _ <- takeMVar mv - writeIORef mmap BS.empty - freeCallbacks bscLoadCallback bscStoreCallback - return res + bs <- liftIO $ do + mv <- newMVar (BlobHandle h True 0) + mmap <- newIORef BS.empty + let bscBlobStore = BlobStoreAccess mv fp mmap + (bscLoadCallback, bscStoreCallback) <- mkCallbacksFromBlobStore bscBlobStore + return BlobStore{..} + res <- runBlobStoreT a bs + liftIO $ do + let BlobStore{..} = bs + BlobStoreAccess mv _ mmap = bscBlobStore + _ <- takeMVar mv + writeIORef mmap BS.empty + freeCallbacks bscLoadCallback bscStoreCallback + return res -- | Truncate the blob store after the blob stored at the given offset. The blob should not be -- corrupted (i.e., its size header should be readable, and its size should match the size header). @@ -587,7 +593,7 @@ type SupportMigration m t = (MonadBlobStore m, MonadTrans t, MonadBlobStore (t m -- based on the context (rather than lifting). newtype BlobStoreT (r :: Type) (m :: Type -> Type) (a :: Type) = BlobStoreT {runBlobStoreT :: r -> m a} deriving - (Functor, Applicative, Monad, MonadReader r, MonadIO, MonadFail, MonadLogger, MonadCatch.MonadThrow, MonadCatch.MonadCatch) + (Functor, Applicative, Monad, MonadReader r, MonadIO, MonadFail, MonadLogger, MonadCatch.MonadThrow, MonadCatch.MonadCatch, MonadCatch.MonadMask) via (ReaderT r m) deriving (MonadTrans) @@ -595,6 +601,12 @@ newtype BlobStoreT (r :: Type) (m :: Type -> Type) (a :: Type) = BlobStoreT {run instance (HasBlobStore r, MonadIO m) => MonadBlobStore (BlobStoreT r m) +deriving via + (LMDBAccountMap.AccountMapStoreMonad (BlobStoreT r m)) + instance + (MonadIO m, MonadLogger m, LMDBAccountMap.HasDatabaseHandlers r) => + LMDBAccountMap.MonadAccountMapStore (BlobStoreT r m) + -- | Apply a given function to modify the context of a 'BlobStoreT' operation. alterBlobStoreT :: (r1 -> r2) -> BlobStoreT r2 m a -> BlobStoreT r1 m a alterBlobStoreT f (BlobStoreT a) = BlobStoreT (a . f) diff --git a/concordium-consensus/src/Concordium/GlobalState/Persistent/BlockState.hs b/concordium-consensus/src/Concordium/GlobalState/Persistent/BlockState.hs index d8e7e141ac..5905b4b68f 100644 --- a/concordium-consensus/src/Concordium/GlobalState/Persistent/BlockState.hs +++ b/concordium-consensus/src/Concordium/GlobalState/Persistent/BlockState.hs @@ -1,6 +1,7 @@ {-# LANGUAGE BangPatterns #-} {-# LANGUAGE ConstraintKinds #-} {-# LANGUAGE DataKinds #-} +{-# LANGUAGE DerivingVia #-} {-# LANGUAGE EmptyCase #-} {-# LANGUAGE GADTs #-} {-# LANGUAGE MultiWayIf #-} @@ -30,7 +31,7 @@ module Concordium.GlobalState.Persistent.BlockState ( PersistentState, BlockRewardDetails (..), PersistentBlockStateMonad (..), - withNewAccountCache, + withNewAccountCacheAndLMDBAccountMap, cacheState, cacheStateAndGetTransactionTable, migratePersistentBlockState, @@ -40,6 +41,7 @@ module Concordium.GlobalState.Persistent.BlockState ( import qualified Concordium.Crypto.SHA256 as H import qualified Concordium.Genesis.Data.P6 as P6 import Concordium.GlobalState.Account hiding (addIncomingEncryptedAmount, addToSelfEncryptedAmount) +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap import Concordium.GlobalState.BakerInfo import Concordium.GlobalState.BlockState import Concordium.GlobalState.CapitalDistribution @@ -48,6 +50,7 @@ import Concordium.GlobalState.Parameters import Concordium.GlobalState.Persistent.Account import Concordium.GlobalState.Persistent.Accounts (SupportsPersistentAccount) import qualified Concordium.GlobalState.Persistent.Accounts as Accounts +import qualified Concordium.GlobalState.Persistent.Accounts as LMDBAccountMap import Concordium.GlobalState.Persistent.Bakers import Concordium.GlobalState.Persistent.BlobStore import qualified Concordium.GlobalState.Persistent.BlockState.Modules as Modules @@ -86,7 +89,8 @@ import Concordium.Utils.BinarySearch import Concordium.Utils.Serialization import Concordium.Utils.Serialization.Put import qualified Concordium.Wasm as Wasm - +import Control.Exception +import qualified Control.Monad.Catch as MonadCatch import qualified Control.Monad.Except as MTL import Control.Monad.Reader import qualified Control.Monad.State.Strict as MTL @@ -102,6 +106,7 @@ import qualified Data.Set as Set import qualified Data.Vector as Vec import Data.Word import Lens.Micro.Platform +import System.Directory (removeDirectoryRecursive) -- * Birk parameters @@ -867,6 +872,7 @@ bspPoolRewards bsp = case bspRewardDetails bsp of BlockRewardDetailsV1 pr -> pr -- | An initial 'HashedPersistentBlockState', which may be used for testing purposes. +{-# WARNING initialPersistentState "should only be used for testing" #-} initialPersistentState :: (SupportsPersistentState pv m) => SeedState (SeedStateVersionFor pv) -> @@ -888,7 +894,6 @@ initialPersistentState seedState cryptoParams accounts ips ars keysCollection ch updates <- refMake =<< initialUpdates keysCollection chainParams releaseSchedule <- emptyReleaseSchedule red <- emptyBlockRewardDetails - bsp <- makeBufferedRef $ BlockStatePointers @@ -927,10 +932,11 @@ emptyBlockState bspBirkParameters cryptParams keysCollection chainParams = do bspUpdates <- refMake =<< initialUpdates keysCollection chainParams bspReleaseSchedule <- emptyReleaseSchedule bspRewardDetails <- emptyBlockRewardDetails + bspAccounts <- Accounts.emptyAccounts bsp <- makeBufferedRef $ BlockStatePointers - { bspAccounts = Accounts.emptyAccounts, + { bspAccounts = bspAccounts, bspInstances = Instances.emptyInstances, bspModules = modules, bspBank = makeHashed Rewards.emptyBankStatus, @@ -2245,11 +2251,6 @@ doAccountList pbs = do bsp <- loadPBS pbs Accounts.accountAddresses (bspAccounts bsp) -doAddressWouldClash :: (SupportsPersistentState pv m) => PersistentBlockState pv -> AccountAddress -> m Bool -doAddressWouldClash pbs addr = do - bsp <- loadPBS pbs - Accounts.addressWouldClash addr (bspAccounts bsp) - doRegIdExists :: (SupportsPersistentState pv m) => PersistentBlockState pv -> ID.CredentialRegistrationID -> m Bool doRegIdExists pbs regid = do bsp <- loadPBS pbs @@ -3308,9 +3309,14 @@ data PersistentBlockStateContext pv = PersistentBlockStateContext -- | Cache used for caching accounts. pbscAccountCache :: !(AccountCache (AccountVersionFor pv)), -- | Cache used for caching modules. - pbscModuleCache :: !Modules.ModuleCache + pbscModuleCache :: !Modules.ModuleCache, + -- | LMDB account map + pbscAccountMap :: !LMDBAccountMap.DatabaseHandlers } +instance LMDBAccountMap.HasDatabaseHandlers (PersistentBlockStateContext pv) where + databaseHandlers = lens pbscAccountMap (\s v -> s{pbscAccountMap = v}) + instance HasBlobStore (PersistentBlockStateContext av) where blobStore = bscBlobStore . pbscBlobStore blobLoadCallback = bscLoadCallback . pbscBlobStore @@ -3325,16 +3331,23 @@ instance Cache.HasCache Modules.ModuleCache (PersistentBlockStateContext pv) whe instance (IsProtocolVersion pv) => MonadProtocolVersion (BlobStoreT (PersistentBlockStateContext pv) m) where type MPV (BlobStoreT (PersistentBlockStateContext pv) m) = pv --- | Create a new account cache of the specified size for running the given monadic operation by +-- | Create a new account cache of the specified size and a temporary 'LMDBAccountMap' for running the given monadic operation by -- extending the 'BlobStore' context to a 'PersistentBlockStateContext'. -withNewAccountCache :: (MonadIO m) => Int -> BlobStoreT (PersistentBlockStateContext pv) m a -> BlobStoreT BlobStore m a -withNewAccountCache size bsm = do - ac <- liftIO $ newAccountCache size - mc <- liftIO $ Modules.newModuleCache 100 - alterBlobStoreT (\bs -> PersistentBlockStateContext bs ac mc) bsm +-- Note. this function should only be used for tests. +withNewAccountCacheAndLMDBAccountMap :: (MonadIO m, MonadCatch.MonadMask m) => Int -> FilePath -> BlobStoreT (PersistentBlockStateContext pv) m a -> BlobStoreT BlobStore m a +withNewAccountCacheAndLMDBAccountMap size lmdbAccountMapDir bsm = MonadCatch.bracket openLmdbAccMap closeLmdbAccMap runAction + where + openLmdbAccMap = liftIO $ LMDBAccountMap.openDatabase lmdbAccountMapDir + closeLmdbAccMap handlers = liftIO $ do + LMDBAccountMap.closeDatabase handlers + removeDirectoryRecursive lmdbAccountMapDir `catch` (\(e :: IOException) -> liftIO $ void $ print e) + runAction lmdbAccMap = do + ac <- liftIO $ newAccountCache size + mc <- liftIO $ Modules.newModuleCache 100 + alterBlobStoreT (\bs -> PersistentBlockStateContext bs ac mc lmdbAccMap) bsm newtype PersistentBlockStateMonad (pv :: ProtocolVersion) (r :: Type) (m :: Type -> Type) (a :: Type) = PersistentBlockStateMonad {runPersistentBlockStateMonad :: m a} - deriving (Functor, Applicative, Monad, MonadIO, MonadReader r, MonadLogger, TimeMonad, MTL.MonadState s) + deriving (Functor, Applicative, Monad, MonadIO, MonadReader r, MonadLogger, TimeMonad, MTL.MonadState s, MonadCatch.MonadCatch, MonadCatch.MonadThrow) type PersistentState av pv r m = ( MonadIO m, @@ -3342,7 +3355,9 @@ type PersistentState av pv r m = HasBlobStore r, AccountVersionFor pv ~ av, Cache.HasCache (AccountCache av) r, - Cache.HasCache Modules.ModuleCache r + Cache.HasCache Modules.ModuleCache r, + LMDBAccountMap.HasDatabaseHandlers r, + MonadLogger m ) instance MonadTrans (PersistentBlockStateMonad pv r) where @@ -3355,6 +3370,8 @@ instance (PersistentState av pv r m) => MonadBlobStore (PutH (PersistentBlockSta instance (PersistentState av pv r m) => Cache.MonadCache (AccountCache av) (PersistentBlockStateMonad pv r m) instance (PersistentState av pv r m) => Cache.MonadCache Modules.ModuleCache (PersistentBlockStateMonad pv r m) +deriving via (LMDBAccountMap.AccountMapStoreMonad m) instance (MonadIO m, MonadLogger m, MonadReader r m, LMDBAccountMap.HasDatabaseHandlers r) => LMDBAccountMap.MonadAccountMapStore (PersistentBlockStateMonad pv r m) + type instance BlockStatePointer (PersistentBlockState pv) = BlobRef (BlockStatePointers pv) type instance BlockStatePointer (HashedPersistentBlockState pv) = BlobRef (BlockStatePointers pv) @@ -3475,7 +3492,7 @@ instance (IsProtocolVersion pv, PersistentState av pv r m) => BlockStateOperatio bsoGetAccountIndex = doGetAccountIndex bsoGetAccountByIndex = doGetAccountByIndex bsoGetInstance = doGetInstance - bsoAddressWouldClash = doAddressWouldClash + bsoAddressWouldClash = doGetAccountExists bsoRegIdExists = doRegIdExists bsoCreateAccount = doCreateAccount bsoPutNewInstance = doPutNewInstance @@ -3546,10 +3563,9 @@ instance (IsProtocolVersion pv, PersistentState av pv r m) => BlockStateOperatio bsoIsProtocolUpdateEffective = doIsProtocolUpdateEffective instance (IsProtocolVersion pv, PersistentState av pv r m) => BlockStateStorage (PersistentBlockStateMonad pv r m) where - thawBlockState HashedPersistentBlockState{..} = - liftIO $ newIORef =<< readIORef hpbsPointers + thawBlockState = doThawBlockState - freezeBlockState pbs = hashBlockState pbs + freezeBlockState = hashBlockState dropUpdatableBlockState pbs = liftIO $ writeIORef pbs (error "Block state dropped") @@ -3568,6 +3584,19 @@ instance (IsProtocolVersion pv, PersistentState av pv r m) => BlockStateStorage flushStore return ref + saveAccounts HashedPersistentBlockState{..} = do + -- this load should be cheap as the blockstate is in memory. + accs <- bspAccounts <$> loadPBS hpbsPointers + -- write the accounts that were created in the block and + -- potentially non-finalized parent blocks. + -- Note that this also empties the difference map for the + -- block. + void $ Accounts.writeAccountsCreated accs + + reconstructAccountDifferenceMap HashedPersistentBlockState{..} parentDifferenceMap listOfAccounts = do + accs <- bspAccounts <$> loadPBS hpbsPointers + Accounts.reconstructDifferenceMap parentDifferenceMap listOfAccounts accs + loadBlockState hpbsHashM ref = do hpbsPointers <- liftIO $ newIORef $ blobRefToBufferedRef ref case hpbsHashM of @@ -3588,6 +3617,11 @@ instance (IsProtocolVersion pv, PersistentState av pv r m) => BlockStateStorage cacheBlockState = cacheState cacheBlockStateAndGetTransactionTable = cacheStateAndGetTransactionTable + tryPopulateAccountMap HashedPersistentBlockState{..} = do + -- load the top level references and write the accounts to the LMDB backed + -- account map (if this has not already been done). + BlockStatePointers{..} <- loadPBS hpbsPointers + LMDBAccountMap.tryPopulateLMDBStore bspAccounts -- | Migrate the block state from the representation used by protocol version -- @oldpv@ to the one used by protocol version @pv@. The migration is done gradually, @@ -3688,6 +3722,23 @@ migrateBlockPointers migration BlockStatePointers{..} = do bspRewardDetails = newRewardDetails } +-- | Thaw the block state, making it ready for modification. +-- This function wraps the underlying 'PersistentBlockState' of the provided 'HashedPersistentBlockState' in a new 'IORef' +-- such that changes to the thawed block state does not propagate into the parent state. +-- +-- Further the 'DiffMap.DifferenceMap' of the accounts structure in the provided block state is +-- "bumped" in the sense that a new one is created for the new thawed block with a pointer to the parent difference map. +-- The parent difference map is empty if the parent is persisted otherwise it may contain new accounts created in that block. +doThawBlockState :: + (SupportsPersistentState pv m) => + HashedPersistentBlockState pv -> + m (PersistentBlockState pv) +doThawBlockState HashedPersistentBlockState{..} = do + bsp@BlockStatePointers{..} <- loadPBS hpbsPointers + bspAccounts' <- Accounts.mkNewChildDifferenceMap bspAccounts + let bsp' = bsp{bspAccounts = bspAccounts'} + liftIO $ newIORef =<< makeBufferedRef bsp' + -- | Cache the block state. cacheState :: forall pv m. diff --git a/concordium-consensus/src/Concordium/GlobalState/Persistent/Genesis.hs b/concordium-consensus/src/Concordium/GlobalState/Persistent/Genesis.hs index 73a6f7bc20..053fcf5ca6 100644 --- a/concordium-consensus/src/Concordium/GlobalState/Persistent/Genesis.hs +++ b/concordium-consensus/src/Concordium/GlobalState/Persistent/Genesis.hs @@ -110,18 +110,20 @@ data AccumGenesisState pv = AccumGenesisState --------- Helper functions ---------- -- | The initial value for accumulating data from genesis data accounts. -initialAccumGenesisState :: AccumGenesisState pv -initialAccumGenesisState = - AccumGenesisState - { agsAllAccounts = Accounts.emptyAccounts, - agsBakerIds = Trie.empty, - agsBakerKeys = Trie.empty, - agsTotal = 0, - agsStakedTotal = 0, - agsBakerInfoRefs = Vec.empty, - agsBakerStakes = Vec.empty, - agsBakerCapitals = Vec.empty - } +initialAccumGenesisState :: (MTL.MonadIO m) => m (AccumGenesisState pv) +initialAccumGenesisState = do + emptyAccs <- Accounts.emptyAccounts + return $ + AccumGenesisState + { agsAllAccounts = emptyAccs, + agsBakerIds = Trie.empty, + agsBakerKeys = Trie.empty, + agsTotal = 0, + agsStakedTotal = 0, + agsBakerInfoRefs = Vec.empty, + agsBakerStakes = Vec.empty, + agsBakerCapitals = Vec.empty + } -- | Construct a hashed persistent block state from the data in genesis. -- The result is immediately flushed to disc and cached. @@ -132,8 +134,9 @@ buildGenesisBlockState :: GenesisData.GenesisState pv -> MTL.ExceptT String m (BS.HashedPersistentBlockState pv, TransactionTable.TransactionTable) buildGenesisBlockState vcgp GenesisData.GenesisState{..} = do + initState <- initialAccumGenesisState -- Iterate the accounts in genesis once and accumulate all relevant information. - AccumGenesisState{..} <- Vec.ifoldM' accumStateFromGenesisAccounts initialAccumGenesisState genesisAccounts + AccumGenesisState{..} <- Vec.ifoldM' accumStateFromGenesisAccounts initState genesisAccounts -- Birk parameters persistentBirkParameters :: BS.PersistentBirkParameters pv <- do @@ -258,7 +261,7 @@ buildGenesisBlockState vcgp GenesisData.GenesisState{..} = do genesisChainParameters genesisAccount -- Insert the account - (maybeIndex, nextAccounts0) <- Accounts.putNewAccount persistentAccount $ agsAllAccounts state + (maybeIndex, nextAccounts0) <- Accounts.putNewAccount persistentAccount (agsAllAccounts state) nextAccounts <- case maybeIndex of Nothing -> MTL.throwError "Duplicate account address in genesis accounts." Just ai -> diff --git a/concordium-consensus/src/Concordium/GlobalState/Persistent/LMDB.hs b/concordium-consensus/src/Concordium/GlobalState/Persistent/LMDB.hs index 388da62fc7..15dfe2221b 100644 --- a/concordium-consensus/src/Concordium/GlobalState/Persistent/LMDB.hs +++ b/concordium-consensus/src/Concordium/GlobalState/Persistent/LMDB.hs @@ -25,7 +25,6 @@ module Concordium.GlobalState.Persistent.LMDB ( closeDatabase, addDatabaseVersion, checkDatabaseVersion, - resizeOnResized, finalizedByHeightStore, StoredBlock (..), StoredBlockWithStateHash (..), @@ -66,7 +65,7 @@ import Concordium.Types.Transactions import Control.Arrow ((&&&)) import Control.Concurrent (runInBoundThread) import Control.Monad -import Control.Monad.Catch (MonadCatch, handleJust, tryJust) +import Control.Monad.Catch (tryJust) import Control.Monad.IO.Class import Control.Monad.State import qualified Data.ByteString as BS @@ -317,24 +316,13 @@ metadataStoreName = "metadata" databaseCount :: Int databaseCount = 5 --- | Database growth size increment. --- This is currently set at 64MB, and must be a multiple of the page size. -dbStepSize :: Int -dbStepSize = 2 ^ (26 :: Int) -- 64MB - --- | Initial database size. --- This is currently set to be the same as 'dbStepSize'. -dbInitSize :: Int -dbInitSize = dbStepSize - -- | Initialize database handlers in ReadWrite mode. -- This simply loads the references and does not initialize the databases. --- The initial size is set to 64MB. databaseHandlers :: FilePath -> IO (DatabaseHandlers pv st) -databaseHandlers treeStateDir = makeDatabaseHandlers treeStateDir False dbInitSize +databaseHandlers treeStateDir = makeDatabaseHandlers treeStateDir False defaultEnvSize -- | Initialize database handlers. --- The size will be rounded up to a multiple of 'dbStepSize'. +-- The size will be rounded up to a multiple of 'seStepSize'. -- (This ensures in particular that the size is a multiple of the page size, which is required by -- LMDB.) makeDatabaseHandlers :: @@ -349,7 +337,8 @@ makeDatabaseHandlers treeStateDir readOnly initSize = do _storeEnv <- makeStoreEnv -- here nobody else has access to the environment, so we need not lock let env = _storeEnv ^. seEnv - mdb_env_set_mapsize env (initSize + dbStepSize - initSize `mod` dbStepSize) + stepSize = _storeEnv ^. seStepSize + mdb_env_set_mapsize env (initSize + stepSize - initSize `mod` stepSize) mdb_env_set_maxdbs env databaseCount mdb_env_set_maxreaders env 126 -- TODO: Consider MDB_NOLOCK @@ -382,12 +371,12 @@ openReadOnlyDatabase :: openReadOnlyDatabase treeStateDir = do _storeEnv <- makeStoreEnv let env = _storeEnv ^. seEnv - mdb_env_set_mapsize env dbInitSize + mdb_env_set_mapsize env defaultEnvSize mdb_env_set_maxdbs env databaseCount mdb_env_set_maxreaders env 126 -- TODO: Consider MDB_NOLOCK mdb_env_open env treeStateDir [MDB_RDONLY] - (_metadataStore, mversion) <- resizeOnResizedInternal _storeEnv $ transaction _storeEnv True $ \txn -> do + (_metadataStore, mversion) <- resizeOnResized _storeEnv $ transaction _storeEnv True $ \txn -> do _metadataStore <- MetadataStore <$> mdb_dbi_open' txn (Just metadataStoreName) [] mversion <- loadRecord txn _metadataStore versionMetadata return (_metadataStore, mversion) @@ -401,7 +390,7 @@ openReadOnlyDatabase treeStateDir = do -- version. case promoteProtocolVersion vmProtocolVersion of SomeProtocolVersion (_ :: SProtocolVersion pv) -> - resizeOnResizedInternal _storeEnv $ transaction _storeEnv True $ \txn -> do + resizeOnResized _storeEnv $ transaction _storeEnv True $ \txn -> do _blockStore <- BlockStore <$> mdb_dbi_open' txn (Just blockStoreName) [] _finalizationRecordStore <- FinalizationRecordStore <$> mdb_dbi_open' txn (Just finalizationRecordStoreName) [] _finalizedByHeightStore <- FinalizedByHeightStore <$> mdb_dbi_open' txn (Just finalizedByHeightStoreName) [] @@ -458,7 +447,7 @@ initializeDatabase gb stRef gbStateHash treeStateDir = do -- migrating a database from an earlier version. addDatabaseVersion :: (MonadLogger m, MonadIO m) => FilePath -> m () addDatabaseVersion treeStateDir = do - handlers :: DatabaseHandlers 'P1 () <- liftIO $ makeDatabaseHandlers treeStateDir False dbInitSize + handlers :: DatabaseHandlers 'P1 () <- liftIO $ makeDatabaseHandlers treeStateDir False defaultEnvSize handlers' <- execStateT ( resizeOnFull 4096 $ -- This size is mostly arbitrary, but should be enough to store the serialized metadata @@ -500,43 +489,10 @@ checkDatabaseVersion db = _ -> Left $ "expected " ++ show expectedVersion ++ " but the version could not be deserialized" -- | Close down the database, freeing the file handles. +-- The use of withWriteStoreEnv ensures that there are no outstanding transactions and cursors are closed. closeDatabase :: DatabaseHandlers pv st -> IO () closeDatabase db = runInBoundThread $ withWriteStoreEnv (db ^. storeEnv) mdb_env_close --- The use of withWriteStoreEnv ensures that there are no outstanding transactions and cursors are closed. - --- | Resize the LMDB map if the file size has changed. --- This is used to allow a secondary process that is reading the database --- to handle resizes to the database that are made by the writer. --- The supplied action will be executed. If it fails with an 'MDB_MAP_RESIZED' --- error, then the map will be resized and the action retried. -resizeOnResized :: (MonadIO m, MonadState s m, HasDatabaseHandlers pv st s, MonadCatch m) => m a -> m a -resizeOnResized a = do - dbh <- use dbHandlers - resizeOnResizedInternal (dbh ^. storeEnv) a - --- | Perform a database action and resize the LMDB map if the file size has changed. The difference --- with `resizeOnResized` is that this function takes database handlers as an argument, instead of --- reading their value from `HasDatabaseHandlers`. -resizeOnResizedInternal :: (MonadIO m, MonadCatch m) => StoreEnv -> m a -> m a -resizeOnResizedInternal se a = inner - where - inner = handleJust checkResized onResized a - checkResized LMDB_Error{..} = guard (e_code == Right MDB_MAP_RESIZED) - onResized _ = do - liftIO (withWriteStoreEnv se $ flip mdb_env_set_mapsize 0) - inner - -resizeDatabaseHandlers :: (MonadIO m, MonadLogger m) => DatabaseHandlers pv st -> Int -> m () -resizeDatabaseHandlers dbh size = do - envInfo <- liftIO $ mdb_env_info (dbh ^. storeEnv . seEnv) - let delta = size + (dbStepSize - size `mod` dbStepSize) - oldMapSize = fromIntegral $ me_mapsize envInfo - newMapSize = oldMapSize + delta - _storeEnv = dbh ^. storeEnv - logEvent LMDB LLDebug $ "Resizing database from " ++ show oldMapSize ++ " to " ++ show newMapSize - liftIO . withWriteStoreEnv (dbh ^. storeEnv) $ flip mdb_env_set_mapsize newMapSize - -- | Load a block and its state hash (if available). -- Normal blocks already contain their state hash. For genesis blocks, the state hash is loaded -- from the metadata table if it is present there. @@ -717,7 +673,7 @@ resizeOnFullInternal addSize dbh a = inner Left _ -> do -- Resize the database handlers, and try to add again in case the size estimate -- given by lmdbStoreTypeSize is off. - resizeDatabaseHandlers dbh addSize + resizeDatabaseHandlers (dbh ^. storeEnv) addSize inner Right res -> return res -- only handle the db full error and propagate other exceptions. diff --git a/concordium-consensus/src/Concordium/GlobalState/Persistent/TreeState.hs b/concordium-consensus/src/Concordium/GlobalState/Persistent/TreeState.hs index cdb03df5d5..2f0c035069 100644 --- a/concordium-consensus/src/Concordium/GlobalState/Persistent/TreeState.hs +++ b/concordium-consensus/src/Concordium/GlobalState/Persistent/TreeState.hs @@ -77,6 +77,8 @@ data InitException | DatabaseInvariantViolation !String | -- | The database version is not correct. IncorrectDatabaseVersion !String + | -- | Cannot get read/write permissions for the account map file. + AccountMapPermissionError deriving (Show, Typeable) instance Exception InitException where @@ -90,6 +92,7 @@ instance Exception InitException where displayException (DatabaseInvariantViolation err) = "Database invariant violation: " ++ err displayException (IncorrectDatabaseVersion err) = "Incorrect database version: " ++ err + displayException AccountMapPermissionError = "Cannot get read and write permissions for the account map file." logExceptionAndThrowTS :: (MonadLogger m, MonadIO m, Exception e) => e -> m a logExceptionAndThrowTS = logExceptionAndThrow TreeState @@ -248,6 +251,7 @@ instance -- | Initial skov data with default runtime parameters (block size = 10MB). initialSkovPersistentDataDefault :: (IsProtocolVersion pv, MonadIO m) => + -- | Tree state directory FilePath -> GenesisConfiguration -> PBS.HashedPersistentBlockState pv -> @@ -324,11 +328,15 @@ checkExistingDatabase :: FilePath -> -- | Block state file FilePath -> + -- | Account map path + FilePath -> m Bool -checkExistingDatabase treeStateDir blockStateFile = do +checkExistingDatabase treeStateDir blockStateFile accountMapDir = do let treeStateFile = treeStateDir "data.mdb" + let accountMapFile = accountMapDir "data.mdb" bsPathEx <- liftIO $ doesPathExist blockStateFile tsPathEx <- liftIO $ doesPathExist treeStateFile + amPathEx <- liftIO $ doesPathExist accountMapFile -- Check whether a path is a normal file that is readable and writable let checkRWFile :: FilePath -> InitException -> m () @@ -347,14 +355,18 @@ checkExistingDatabase treeStateDir blockStateFile = do unless (readable perms && writable perms) $ do logExceptionAndThrowTS exc - -- if both files exist we check whether they are both readable and writable. + -- if all files exist we check whether they are both readable and writable. -- In case only one of them exists we raise an appropriate exception. We don't want to delete any data. if | bsPathEx && tsPathEx -> do -- check whether it is a normal file and whether we have the right permissions checkRWFile blockStateFile BlockStatePermissionError checkRWFile treeStateFile TreeStatePermissionError - mapM_ (logEvent TreeState LLTrace) ["Existing database found.", "TreeState filepath: " ++ show blockStateFile, "BlockState filepath: " ++ show treeStateFile] + when amPathEx $ checkRWFile accountMapFile AccountMapPermissionError + logEvent TreeState LLTrace "Existing database found." + logEvent TreeState LLTrace $ "TreeState filepath: " ++ show treeStateFile + logEvent TreeState LLTrace $ "BlockState filepath: " ++ show blockStateFile + logEvent TreeState LLTrace $ if amPathEx then "AccountMap filepath: " ++ show accountMapFile else "AccountMap not found" return True | bsPathEx -> do logEvent GlobalState LLWarning "Block state file exists, but tree state database does not. Deleting the block state file." @@ -410,7 +422,7 @@ loadSkovPersistentData rp _treeStateDirectory pbsc = do -- Unroll the treestate if the last finalized blockstate is corrupted. If the last finalized -- blockstate is not corrupted, the treestate is unchanged. - unrollTreeStateWhile _db (liftIO . isBlockStateCorrupted) >>= \case + unrollTreeStateWhile _db isBlockStateCorrupted >>= \case Left e -> logExceptionAndThrowTS . DatabaseInvariantViolation $ "The block state database is corrupt. Recovery attempt failed: " <> e @@ -424,7 +436,8 @@ loadSkovPersistentData rp _treeStateDirectory pbsc = do genStoredBlock <- maybe (logExceptionAndThrowTS GenesisBlockNotInDataBaseError) return =<< liftIO (getFirstBlock _db) - _genesisBlockPointer <- liftIO $ makeBlockPointer genStoredBlock + + _genesisBlockPointer <- makeBlockPointer genStoredBlock when (isNothing (sbshStateHash genStoredBlock)) $ do -- This should only occur when updating from a node version prior to 6.1.6 -- the first time the database is loaded. @@ -436,7 +449,7 @@ loadSkovPersistentData rp _treeStateDirectory pbsc = do GenesisBlock gd' -> return gd' _ -> logExceptionAndThrowTS (DatabaseInvariantViolation "Block at height 0 is not a genesis block.") -- Get the last finalized block. - _lastFinalized <- liftIO (makeBlockPointer lfStoredBlock) + _lastFinalized <- makeBlockPointer lfStoredBlock return SkovPersistentData { _possiblyPendingTable = HM.empty, @@ -458,13 +471,13 @@ loadSkovPersistentData rp _treeStateDirectory pbsc = do where makeBlockPointer :: StoredBlockWithStateHash pv (TS.BlockStatePointer (PBS.PersistentBlockState pv)) -> - IO (PersistentBlockPointer pv (PBS.HashedPersistentBlockState pv)) + LogIO (PersistentBlockPointer pv (PBS.HashedPersistentBlockState pv)) makeBlockPointer StoredBlockWithStateHash{sbshStoredBlock = StoredBlock{..}, ..} = do bstate <- runReaderT (PBS.runPersistentBlockStateMonad (loadBlockState sbshStateHash sbState)) pbsc makeBlockPointerFromPersistentBlock sbBlock bstate sbInfo - isBlockStateCorrupted :: StoredBlock pv (TS.BlockStatePointer (PBS.PersistentBlockState pv)) -> IO Bool + isBlockStateCorrupted :: StoredBlock pv (TS.BlockStatePointer (PBS.PersistentBlockState pv)) -> LogIO Bool isBlockStateCorrupted block = - not <$> runBlobStoreT (isValidBlobRef (sbState block)) pbsc + not <$> runReaderT (PBS.runPersistentBlockStateMonad (isValidBlobRef (sbState block))) pbsc -- | Activate the state and make it usable for use by consensus. This concretely -- means that the block state for the last finalized block is cached, and that @@ -486,6 +499,10 @@ activateSkovPersistentData pbsc uninitState = let bps = _bpState $ _lastFinalized uninitState tt <- cacheBlockStateAndGetTransactionTable bps logEvent GlobalState LLTrace "Done caching last finalized block" + -- initialize the account map if it has not already been so. + logEvent GlobalState LLDebug "Initializing LMDB account map" + void $ tryPopulateAccountMap bps + logEvent GlobalState LLDebug "Finished initializing LMDB account map" return $! uninitState{_transactionTable = tt} where runBlockState a = runReaderT (PBS.runPersistentBlockStateMonad @pv a) pbsc @@ -696,7 +713,9 @@ instance markFinalized bh fr = use (skovPersistentData . blockTable . liveMap . at' bh) >>= \case Just (BlockAlive bp) -> do + -- Save the block state and write the accounts out to disk. st <- saveBlockState (_bpState bp) + void $ saveAccounts (_bpState bp) -- NB: Removing the block from the in-memory cache only makes -- sense if no block lookups are done between the call to this -- function and 'wrapUpFinalization'. This is currently the case, diff --git a/concordium-consensus/src/Concordium/ImportExport.hs b/concordium-consensus/src/Concordium/ImportExport.hs index d19ead9dde..26b7f30988 100644 --- a/concordium-consensus/src/Concordium/ImportExport.hs +++ b/concordium-consensus/src/Concordium/ImportExport.hs @@ -44,7 +44,7 @@ import Control.Monad import Control.Monad.Catch import Control.Monad.IO.Class import Control.Monad.Reader -import Control.Monad.State (MonadState, evalStateT) +import Control.Monad.State (MonadState, evalStateT, gets) import Control.Monad.Trans.Except import qualified Data.Attoparsec.Text as AP import Data.Bits @@ -69,6 +69,7 @@ import Concordium.Common.Version import Concordium.GlobalState.Block import Concordium.GlobalState.BlockPointer import Concordium.GlobalState.Finalization +import qualified Concordium.GlobalState.LMDB.Helpers as LMDBHelpers import Concordium.GlobalState.Persistent.LMDB import qualified Concordium.KonsensusV1.TreeState.LowLevel as KonsensusV1 import qualified Concordium.KonsensusV1.TreeState.LowLevel.LMDB as KonsensusV1 @@ -445,7 +446,8 @@ exportConsensusV0Blocks :: -- and the resulting 'BlockIndex' (the entries that have been added). m (Bool, BlockIndex) exportConsensusV0Blocks firstBlock outDir chunkSize genIndex startHeight blockIndex lastWrittenChunkM = do - mgenFinRec <- resizeOnResized $ readFinalizationRecord 0 + env <- _storeEnv <$> gets _dbsHandlers + mgenFinRec <- LMDBHelpers.resizeOnResized env $ readFinalizationRecord 0 case mgenFinRec of Nothing -> do logEvent External LLError "No finalization record found in database for finalization index 0." @@ -471,7 +473,7 @@ exportConsensusV0Blocks firstBlock outDir chunkSize genIndex startHeight blockIn return (True, Empty) else do let getBlockAt height = - resizeOnResized (readFinalizedBlockAtHeight height) >>= \case + LMDBHelpers.resizeOnResized env (readFinalizedBlockAtHeight height) >>= \case Nothing -> return Nothing Just StoredBlockWithStateHash{..} | NormalBlock normalBlock <- sbBlock sbshStoredBlock -> do let serializedBlock = runPut $ putVersionedBlock (protocolVersion @pv) normalBlock @@ -485,7 +487,7 @@ exportConsensusV0Blocks firstBlock outDir chunkSize genIndex startHeight blockIn getFinalizationAt mFinIndex = case mFinIndex of Nothing -> return Nothing Just finIndex -> - resizeOnResized (readFinalizationRecord finIndex) >>= \case + LMDBHelpers.resizeOnResized env (readFinalizationRecord finIndex) >>= \case Nothing -> return Nothing Just fr -> return . Just $ runPut $ putVersionedFinalizationRecordV0 fr chunks <- @@ -545,7 +547,8 @@ exportConsensusV1Blocks :: -- and the resulting 'BlockIndex' (the entries that have been added). m (Bool, BlockIndex) exportConsensusV1Blocks outDir chunkSize genIndex startHeight blockIndex lastWrittenChunkM = do - KonsensusV1.resizeOnResized KonsensusV1.lookupFirstBlock >>= \case + env <- view KonsensusV1.storeEnv + LMDBHelpers.resizeOnResized env KonsensusV1.lookupFirstBlock >>= \case Nothing -> do logEvent External LLError "Could not read from database." return (True, Empty) @@ -557,14 +560,15 @@ exportConsensusV1Blocks outDir chunkSize genIndex startHeight blockIndex lastWri logEvent External LLError "Genesis hash does not match the recently exported block index." return (True, Empty) else do - KonsensusV1.resizeOnResized KonsensusV1.lookupLastFinalizedBlock >>= \case + LMDBHelpers.resizeOnResized env KonsensusV1.lookupLastFinalizedBlock >>= \case Nothing -> do logEvent External LLError "Cannot read last block of the database." return (True, Empty) Just sb -> do let getBlockAt :: BlockHeight -> m (Maybe (BS.ByteString, BlockHash)) getBlockAt height = - KonsensusV1.resizeOnResized + LMDBHelpers.resizeOnResized + env (KonsensusV1.lookupBlockByHeight height) >>= \case Nothing -> return Nothing @@ -577,7 +581,8 @@ exportConsensusV1Blocks outDir chunkSize genIndex startHeight blockIndex lastWri -- the finalization entry. getFinalizationAt :: BlockHash -> m (Maybe BS.ByteString) getFinalizationAt bh = - KonsensusV1.resizeOnResized + LMDBHelpers.resizeOnResized + env KonsensusV1.lookupLatestFinalizationEntry >>= \case Nothing -> return Nothing diff --git a/concordium-consensus/src/Concordium/KonsensusV1.hs b/concordium-consensus/src/Concordium/KonsensusV1.hs index 6e4c0d0efd..c35e3d66d3 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1.hs @@ -31,6 +31,7 @@ import Concordium.Skov.Monad (UpdateResult (..), transactionVerificationResultTo import Concordium.TimeMonad import Concordium.TimerMonad import Concordium.Types +import Concordium.Types.Option import Concordium.Types.Parameters -- | Handle receiving a finalization message (either a 'QuorumMessage' or a 'TimeoutMessage'). diff --git a/concordium-consensus/src/Concordium/KonsensusV1/Consensus.hs b/concordium-consensus/src/Concordium/KonsensusV1/Consensus.hs index 5c5a9a00c7..e4e605dd80 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/Consensus.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/Consensus.hs @@ -34,6 +34,7 @@ import qualified Concordium.KonsensusV1.TreeState.LowLevel as LowLevel import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types import Concordium.Logger +import Concordium.Types.Option import Concordium.Types.SeedState (currentLeadershipElectionNonce, triggerBlockTime) import Concordium.Types.UpdateQueues import Concordium.Types.Updates diff --git a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Blocks.hs b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Blocks.hs index 4485d80cc7..54accf05a8 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Blocks.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Blocks.hs @@ -52,6 +52,7 @@ import Concordium.KonsensusV1.Types import Concordium.Scheduler (FilteredTransactions (..)) import Concordium.TimerMonad import Concordium.Types.BakerIdentity +import Concordium.Types.Option -- | A block that has passed initial verification, but must still be executed, added to the state, -- and (potentially) signed as a finalizer. diff --git a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp.hs b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp.hs index 65463f0a30..cea82c00f2 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp.hs @@ -64,6 +64,7 @@ import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types import Concordium.TimeMonad import Concordium.TimerMonad +import Concordium.Types.Option import Concordium.Types.Parameters -- | 'CatchUpPartialResponse' represents a stream of blocks to send as a response to a catch-up diff --git a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp/Types.hs b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp/Types.hs index 5a347c1e64..9c8f6b3265 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp/Types.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/CatchUp/Types.hs @@ -6,6 +6,7 @@ import Data.Serialize hiding (getListOf, putListOf) import Concordium.KonsensusV1.Types import Concordium.Types +import Concordium.Types.Option import Concordium.Utils.Serialization import Control.Monad diff --git a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Finality.hs b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Finality.hs index b80c07c056..6ba984f089 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Finality.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Finality.hs @@ -32,6 +32,7 @@ import Concordium.KonsensusV1.TreeState.Implementation import qualified Concordium.KonsensusV1.TreeState.LowLevel as LowLevel import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types +import Concordium.Types.Option -- | Ensure that the given certified block is written to the low-level database. -- Check if the certified block causes its parent to become finalized. @@ -70,7 +71,7 @@ processCertifiedBlock cb@CertifiedBlock{..} then do -- We do not need to update the last finalized block, but we do need to store this -- as a certified block. - storedBlock <- makeStoredBlock cbQuorumBlock + storedBlock <- makeStoredBlock False cbQuorumBlock LowLevel.writeCertifiedBlock storedBlock cbQuorumCertificate else do let !newFinalizedPtr = parentOfLive sd cbQuorumBlock @@ -83,7 +84,7 @@ processCertifiedBlock cb@CertifiedBlock{..} processFinalizationHelper newFinalizedPtr newFinalizationEntry (Just cb) shrinkTimeout cbQuorumBlock | otherwise = unlessStored $ do - storedBlock <- makeStoredBlock cbQuorumBlock + storedBlock <- makeStoredBlock False cbQuorumBlock LowLevel.writeCertifiedBlock storedBlock cbQuorumCertificate where unlessStored a = do @@ -189,14 +190,20 @@ processFinalizationEntry newFinalizedPtr newFinalizationEntry = -- | Write a block's state out to the block state database and construct a 'LowLevel.StoredBlock' -- that can be written to the tree state database. +-- +-- If the provided block is finalized then also any accounts created for the block +-- will be persisted. makeStoredBlock :: ( GSTypes.BlockState m ~ PBS.HashedPersistentBlockState (MPV m), BlockStateStorage m ) => + -- | @True@ if the block is finalized, @False@ if it is certified. + Bool -> BlockPointer (MPV m) -> m (LowLevel.StoredBlock (MPV m)) -makeStoredBlock blockPtr = do +makeStoredBlock finalized blockPtr = do statePointer <- saveBlockState (bpState blockPtr) + when finalized $ saveAccounts (bpState blockPtr) return LowLevel.StoredBlock { stbInfo = blockMetadata blockPtr, @@ -301,11 +308,11 @@ processFinalizationHelper newFinalizedBlock newFinalizationEntry mCertifiedBlock -- Store the blocks and finalization entry in the low-level tree state database, including -- indexing the finalized transactions. -- Store the finalized blocks in the low-level tree state database. - finalizedBlocks <- mapM makeStoredBlock prFinalized + finalizedBlocks <- mapM (makeStoredBlock True) prFinalized case mCertifiedBlock of Nothing -> LowLevel.writeFinalizedBlocks finalizedBlocks newFinalizationEntry Just certifiedBlock -> do - storedCertifiedBlock <- makeStoredBlock (cbQuorumBlock certifiedBlock) + storedCertifiedBlock <- makeStoredBlock False (cbQuorumBlock certifiedBlock) LowLevel.writeCertifiedBlockWithFinalization finalizedBlocks storedCertifiedBlock diff --git a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Timeout.hs b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Timeout.hs index 1b519d19f8..28f68485c7 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Timeout.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/Consensus/Timeout.hs @@ -36,6 +36,7 @@ import qualified Concordium.KonsensusV1.TreeState.LowLevel as LowLevel import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types import Concordium.TimerMonad +import Concordium.Types.Option -- | Reasons that a 'TimeoutMessage' can be rejected. data ReceiveTimeoutMessageRejectReason diff --git a/concordium-consensus/src/Concordium/KonsensusV1/SkovMonad.hs b/concordium-consensus/src/Concordium/KonsensusV1/SkovMonad.hs index c9d871395e..d25efb6d15 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/SkovMonad.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/SkovMonad.hs @@ -31,6 +31,8 @@ import Concordium.GlobalState (GlobalStateInitException (..)) import Concordium.GlobalState.BlockMonads import Concordium.GlobalState.BlockState +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap +import qualified Concordium.GlobalState.BlockState as PBS import Concordium.GlobalState.Parameters hiding (getChainParameters) import Concordium.GlobalState.Persistent.Account import Concordium.GlobalState.Persistent.BlobStore @@ -72,8 +74,8 @@ type PersistentBlockStateMonadHelper pv m = -- -- * @MonadReader (SkovV1Context pv m)@, where the 'SkovV1Context' implements 'HasBlobStore', -- @'Cache.HasCache' ('AccountCache' (AccountVersionFor pv))@, --- @'Cache.HasCache' 'Modules.ModuleCache'@, 'HasHandlerContext', 'HasBakerContext', and --- 'HasDatabaseHandlers'. +-- @'Cache.HasCache' 'Modules.ModuleCache'@, 'HasHandlerContext', 'HasBakerContext', +-- 'HasDatabaseHandlers' and 'LMDBAccountMap.HasDatabaseHandlers. -- -- * @MonadState ('SkovData' pv)@. -- @@ -167,7 +169,7 @@ data SkovV1Context (pv :: ProtocolVersion) m = SkovV1Context _vcBakerContext :: !BakerContext, -- | Blob store and caches used by the block state storage. _vcPersistentBlockStateContext :: !(PersistentBlockStateContext pv), - -- | In-memory low-level tree state database. + -- | low-level tree state database. _vcDisk :: !(DatabaseHandlers pv), -- | Handler functions. _vcHandlers :: !(HandlerContext pv m), @@ -187,6 +189,9 @@ instance (AccountVersionFor pv ~ av) => Cache.HasCache (AccountCache av) (SkovV1 instance Cache.HasCache Modules.ModuleCache (SkovV1Context pv m) where projectCache = Cache.projectCache . _vcPersistentBlockStateContext +instance LMDBAccountMap.HasDatabaseHandlers (SkovV1Context pv m) where + databaseHandlers = lens _vcPersistentBlockStateContext (\s v -> s{_vcPersistentBlockStateContext = v}) . LMDBAccountMap.databaseHandlers + -- Note, these template haskell splices go here because of staging restrictions. -- '_skovV1TUnliftIO' creates a cyclic dependency between 'SkovV1Context' and 'SkovV1T'. -- The above instances are required by deriving via instances attached to the 'SkovV1T' definition. @@ -230,27 +235,27 @@ instance (Monad m) => MonadState (SkovData pv) (SkovV1T pv m) where deriving via (PersistentBlockStateMonadHelper pv m) instance - (IsProtocolVersion pv) => MonadProtocolVersion (SkovV1T pv m) + (IsProtocolVersion pv, MonadLogger m) => MonadProtocolVersion (SkovV1T pv m) deriving via (PersistentBlockStateMonadHelper pv m) instance - (IsProtocolVersion pv, MonadIO m) => AccountOperations (SkovV1T pv m) + (IsProtocolVersion pv, MonadIO m, MonadLogger m) => AccountOperations (SkovV1T pv m) deriving via (PersistentBlockStateMonadHelper pv m) instance - (IsProtocolVersion pv, MonadIO m) => BlockStateQuery (SkovV1T pv m) + (IsProtocolVersion pv, MonadIO m, MonadLogger m) => BlockStateQuery (SkovV1T pv m) deriving via (PersistentBlockStateMonadHelper pv m) instance - (IsProtocolVersion pv, MonadIO m) => BlockStateOperations (SkovV1T pv m) + (IsProtocolVersion pv, MonadIO m, MonadLogger m) => BlockStateOperations (SkovV1T pv m) deriving via (PersistentBlockStateMonadHelper pv m) instance - (IsProtocolVersion pv, MonadIO m) => BlockStateStorage (SkovV1T pv m) + (IsProtocolVersion pv, MonadIO m, MonadLogger m) => BlockStateStorage (SkovV1T pv m) deriving via (DiskLLDBM pv (InnerSkovV1T pv m)) @@ -262,6 +267,16 @@ deriving via ) => LowLevel.MonadTreeStateStore (SkovV1T pv m) +deriving via + (LMDBAccountMap.AccountMapStoreMonad (InnerSkovV1T pv m)) + instance + ( IsProtocolVersion pv, + MonadIO m, + MonadCatch m, + MonadLogger m + ) => + LMDBAccountMap.MonadAccountMapStore (SkovV1T pv m) + instance (Monad m) => MonadBroadcast (SkovV1T pv m) where sendTimeoutMessage tm = do handler <- view sendTimeoutHandler @@ -328,14 +343,16 @@ data GlobalStateConfig = GlobalStateConfig -- | Path to the tree state directory. gscTreeStateDirectory :: !FilePath, -- | Path to the block state file. - gscBlockStateFile :: !FilePath + gscBlockStateFile :: !FilePath, + -- | Path to the account map directory + gscAccountMapDirectory :: !FilePath } -- | Context used by the 'InitMonad'. data InitContext pv = InitContext { -- | Blob store and caches used by the block state storage. _icPersistentBlockStateContext :: !(PersistentBlockStateContext pv), - -- | In-memory low-level tree state database. + -- | low-level tree state database. _icDatabaseHandlers :: !(DatabaseHandlers pv) } @@ -355,6 +372,9 @@ instance Cache.HasCache Modules.ModuleCache (InitContext pv) where instance HasDatabaseHandlers (InitContext pv) pv where databaseHandlers = icDatabaseHandlers +instance LMDBAccountMap.HasDatabaseHandlers (InitContext pv) where + databaseHandlers = icPersistentBlockStateContext . LMDBAccountMap.databaseHandlers + -- | Inner type of 'InitMonad'. type InnerInitMonad pv = ReaderT (InitContext pv) LogIO @@ -393,7 +413,8 @@ newtype InitMonad pv a = InitMonad {runInitMonad' :: InnerInitMonad pv a} ContractStateOperations, ModuleQuery, MonadBlobStore, - Cache.MonadCache Modules.ModuleCache + Cache.MonadCache Modules.ModuleCache, + LMDBAccountMap.MonadAccountMapStore ) via (PersistentBlockStateMonad pv (InitContext pv) (InnerInitMonad pv)) @@ -446,6 +467,23 @@ data ExistingSkov pv m = ExistingSkov esProtocolUpdate :: !(Maybe ProtocolUpdate) } +-- | Internal type used for deriving 'HasDatabaseHandlers' and 'LMDBAccountMap.HasDatabaseHandlers' +-- used for computations where both lmdb databases are required. +data LMDBDatabases pv = LMDBDatabases + { -- | the skov lmdb database + _lmdbSkov :: !(DatabaseHandlers pv), + -- | the account map lmdb database + _lmdbAccountMap :: !LMDBAccountMap.DatabaseHandlers + } + +makeLenses ''LMDBDatabases + +instance HasDatabaseHandlers (LMDBDatabases pv) pv where + databaseHandlers = lmdbSkov + +instance LMDBAccountMap.HasDatabaseHandlers (LMDBDatabases pv) where + databaseHandlers = lmdbAccountMap + -- | Load an existing SkovV1 state. -- Returns 'Nothing' if there is no database to load. -- May throw a 'TreeStateInvariantViolation' if a database invariant violation occurs when @@ -458,32 +496,28 @@ initialiseExistingSkovV1 :: (forall a. SkovV1T pv m a -> IO a) -> GlobalStateConfig -> LogIO (Maybe (ExistingSkov pv m)) -initialiseExistingSkovV1 bakerCtx handlerCtx unliftSkov GlobalStateConfig{..} = do +initialiseExistingSkovV1 bakerCtx handlerCtx unliftSkov gsc@GlobalStateConfig{..} = do logEvent Skov LLDebug "Attempting to use existing global state." - existingDB <- checkExistingDatabase gscTreeStateDirectory gscBlockStateFile + existingDB <- checkExistingDatabase gscTreeStateDirectory gscBlockStateFile gscAccountMapDirectory if existingDB then do - pbscAccountCache <- liftIO $ newAccountCache (rpAccountsCacheSize gscRuntimeParameters) - pbscModuleCache <- liftIO $ Modules.newModuleCache (rpModulesCacheSize gscRuntimeParameters) - pbscBlobStore <- liftIO $ loadBlobStore gscBlockStateFile - let pbsc = PersistentBlockStateContext{..} - let initWithLLDB lldb = do - checkDatabaseVersion lldb - let checkBlockState bs = runBlobStoreT (isValidBlobRef bs) pbsc - (rollCount, bestState) <- - runReaderT - (runDiskLLDBM $ rollBackBlocksUntil checkBlockState) - lldb - when (rollCount > 0) $ do + pbsc <- newPersistentBlockStateContext False gsc + let initWithLLDB skovLldb = do + checkDatabaseVersion skovLldb + let checkBlockState bs = runReaderT (PBS.runPersistentBlockStateMonad (isValidBlobRef bs)) pbsc + RollbackResult{..} <- + flip runReaderT (LMDBDatabases skovLldb $ pbscAccountMap pbsc) $ + runDiskLLDBM (rollBackBlocksUntil checkBlockState) + when (rbrCount > 0) $ do logEvent Skov LLWarning $ "Could not load state for " - ++ show rollCount + ++ show rbrCount ++ " blocks. Truncating block state database." - liftIO $ truncateBlobStore (bscBlobStore . PBS.pbscBlobStore $ pbsc) bestState - let initContext = InitContext pbsc lldb + liftIO $ truncateBlobStore (bscBlobStore . PBS.pbscBlobStore $ pbsc) rbrBestState + let initContext = InitContext pbsc skovLldb (initialSkovData, effectiveProtocolUpdate) <- runInitMonad - (loadSkovData gscRuntimeParameters (rollCount > 0)) + (loadSkovData gscRuntimeParameters (rbrCount > 0)) initContext let !es = ExistingSkov @@ -491,7 +525,7 @@ initialiseExistingSkovV1 bakerCtx handlerCtx unliftSkov GlobalStateConfig{..} = SkovV1Context { _vcBakerContext = bakerCtx, _vcPersistentBlockStateContext = pbsc, - _vcDisk = lldb, + _vcDisk = skovLldb, _vcHandlers = handlerCtx, _skovV1TUnliftIO = unliftSkov }, @@ -510,7 +544,7 @@ initialiseExistingSkovV1 bakerCtx handlerCtx unliftSkov GlobalStateConfig{..} = let initWithBlockState = do (lldb :: DatabaseHandlers pv) <- liftIO $ openDatabase gscTreeStateDirectory initWithLLDB lldb `onException` liftIO (closeDatabase lldb) - initWithBlockState `onException` liftIO (closeBlobStore pbscBlobStore) + initWithBlockState `onException` liftIO (closeBlobStore $ pbscBlobStore pbsc) else do logEvent Skov LLDebug "No existing global state." return Nothing @@ -527,7 +561,7 @@ initialiseNewSkovV1 :: LogIO (SkovV1Context pv m, SkovV1State pv) initialiseNewSkovV1 genData bakerCtx handlerCtx unliftSkov gsConfig@GlobalStateConfig{..} = do logEvent Skov LLDebug "Creating new global state." - pbsc@PersistentBlockStateContext{..} <- newPersistentBlockStateContext gsConfig + pbsc@PersistentBlockStateContext{..} <- newPersistentBlockStateContext True gsConfig let initGS :: InitMonad pv (SkovData pv) initGS = do @@ -538,6 +572,7 @@ initialiseNewSkovV1 genData bakerCtx handlerCtx unliftSkov gsConfig@GlobalStateC Right genState -> return genState logEvent GlobalState LLTrace "Writing persistent global state" stateRef <- saveBlockState pbs + saveAccounts pbs logEvent GlobalState LLTrace "Creating persistent global state context" let genHash = genesisBlockHash genData let genMeta = @@ -605,6 +640,9 @@ activateSkovV1State = do bps <- use $ lastFinalized . to bpState !tt <- cacheBlockStateAndGetTransactionTable bps transactionTable .= tt + logEvent GlobalState LLDebug "Initializing LMDB account map" + void $ PBS.tryPopulateAccountMap bps + logEvent GlobalState LLDebug "Finished initializing LMDB account map" logEvent GlobalState LLTrace "Loading certified blocks" loadCertifiedBlocks logEvent GlobalState LLTrace "Done activating global state" @@ -659,22 +697,24 @@ migrateSkovV1 :: -- | Return back the 'SkovV1Context' and the migrated 'SkovV1State' LogIO (SkovV1Context pv m, SkovV1State pv) migrateSkovV1 regenesis migration gsConfig@GlobalStateConfig{..} oldPbsc oldBlockState bakerCtx handlerCtx unliftSkov migrateTT migratePTT = do - pbsc@PersistentBlockStateContext{..} <- newPersistentBlockStateContext gsConfig + pbsc@PersistentBlockStateContext{..} <- newPersistentBlockStateContext True gsConfig logEvent GlobalState LLDebug "Migrating existing global state." - newInitialBlockState <- flip runBlobStoreT oldPbsc . flip runBlobStoreT pbsc $ do - newState <- migratePersistentBlockState migration $ hpbsPointers oldBlockState - hashBlockState newState + let newInitialBlockState :: InitMonad pv (HashedPersistentBlockState pv) + newInitialBlockState = flip runBlobStoreT oldPbsc . flip runBlobStoreT pbsc $ do + newState <- migratePersistentBlockState migration $ hpbsPointers oldBlockState + hashBlockState newState let initGS :: InitMonad pv (SkovData pv) initGS = do - stateRef <- saveBlockState newInitialBlockState - chainParams <- getChainParameters newInitialBlockState - genEpochBakers <- genesisEpochBakers newInitialBlockState - let genMeta = regenesisMetadata (getHash newInitialBlockState) regenesis + newState <- newInitialBlockState + stateRef <- saveBlockState newState + chainParams <- getChainParameters newState + genEpochBakers <- genesisEpochBakers newState + let genMeta = regenesisMetadata (getHash newState) regenesis let genTimeoutDuration = chainParams ^. cpConsensusParameters . cpTimeoutParameters . tpTimeoutBase let !initSkovData = - mkInitialSkovData gscRuntimeParameters genMeta newInitialBlockState genTimeoutDuration genEpochBakers migrateTT migratePTT + mkInitialSkovData gscRuntimeParameters genMeta newState genTimeoutDuration genEpochBakers migrateTT migratePTT let storedGenesis = LowLevel.StoredBlock { stbStatePointer = stateRef, @@ -706,17 +746,21 @@ migrateSkovV1 regenesis migration gsConfig@GlobalStateConfig{..} oldPbsc oldBloc -- | Make a new 'PersistentBlockStateContext' based on the -- 'GlobalStateConfig' passed into this function. --- This function creates the block state file i.e. the blob store, --- the account cache and the module cache. +-- This function creates the block state file (the blob store) if @True@ is passed in, +-- otherwise it tries to reuse an existing blob store. +-- New account cache and the module cache are created. newPersistentBlockStateContext :: (IsProtocolVersion pv, MonadIO m) => + -- | Whether a new blobstore should be created or a current one should be reused. + Bool -> -- | The global state config to use -- for constructing the persistent block state context. GlobalStateConfig -> -- | The the persistent block state context. m (PersistentBlockStateContext pv) -newPersistentBlockStateContext GlobalStateConfig{..} = liftIO $ do - pbscBlobStore <- createBlobStore gscBlockStateFile +newPersistentBlockStateContext initialize GlobalStateConfig{..} = liftIO $ do + pbscBlobStore <- if initialize then createBlobStore gscBlockStateFile else loadBlobStore gscBlockStateFile pbscAccountCache <- newAccountCache $ rpAccountsCacheSize gscRuntimeParameters pbscModuleCache <- Modules.newModuleCache $ rpModulesCacheSize gscRuntimeParameters + pbscAccountMap <- LMDBAccountMap.openDatabase gscAccountMapDirectory return PersistentBlockStateContext{..} diff --git a/concordium-consensus/src/Concordium/KonsensusV1/TestMonad.hs b/concordium-consensus/src/Concordium/KonsensusV1/TestMonad.hs index 3bd15d6f7f..c0cfb1e73b 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/TestMonad.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/TestMonad.hs @@ -25,6 +25,7 @@ import Concordium.Types import qualified Concordium.Genesis.Data.BaseV1 as BaseV1 import qualified Concordium.Genesis.Data.P6 as P6 +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap import Concordium.GlobalState.BlockState import Concordium.GlobalState.Parameters ( GenesisData (GDP6), @@ -58,9 +59,7 @@ data TestContext (pv :: ProtocolVersion) = TestContext -- | In-memory low-level tree state database. _tcMemoryLLDB :: !(IORef (LowLevelDB pv)), -- | The current time (reported by 'currentTime'). - _tcCurrentTime :: !UTCTime, - -- | Callback to use for logging. - _tcLogger :: !(LogMethod (TestMonad pv)) + _tcCurrentTime :: !UTCTime } instance HasBlobStore (TestContext pv) where @@ -77,6 +76,11 @@ instance Cache.HasCache Module.ModuleCache (TestContext pv) where instance HasMemoryLLDB pv (TestContext pv) where theMemoryLLDB = _tcMemoryLLDB +instance LMDBAccountMap.HasDatabaseHandlers (TestContext pv) where + databaseHandlers = + lens _tcPersistentBlockStateContext (\s v -> s{_tcPersistentBlockStateContext = v}) + . LMDBAccountMap.databaseHandlers + -- | State used for running the 'TestMonad'. data TestState pv = TestState { -- | The 'SkovData'. @@ -110,7 +114,7 @@ type TestWrite pv = [TestEvent pv] -- Hence the 'PersistentBlockStateMonadHelper' transformer is using this monad -- as is the 'TestMonad' -- This makes it possible to easily derive the required instances via the 'PersistentBlockStateMonad'. -type InnerTestMonad (pv :: ProtocolVersion) = RWST (TestContext pv) (TestWrite pv) (TestState pv) IO +type InnerTestMonad (pv :: ProtocolVersion) = RWST (TestContext pv) (TestWrite pv) (TestState pv) LogIO -- | This type is used to derive instances of various block state classes for 'TestMonad'. type PersistentBlockStateMonadHelper pv = @@ -126,7 +130,7 @@ type PersistentBlockStateMonadHelper pv = -- 'MonadConsensusEvent'. -- The state is 'TestState', which includes the 'SkovData' and a map of the pending timer events. newtype TestMonad (pv :: ProtocolVersion) a = TestMonad {runTestMonad' :: (InnerTestMonad pv) a} - deriving newtype (Functor, Applicative, Monad, MonadReader (TestContext pv), MonadIO, MonadThrow, MonadWriter (TestWrite pv)) + deriving newtype (Functor, Applicative, Monad, MonadReader (TestContext pv), MonadIO, MonadThrow, MonadWriter (TestWrite pv), MonadLogger) deriving (BlockStateTypes, ContractStateOperations, ModuleQuery) via (PersistentBlockStateMonadHelper pv) @@ -146,7 +150,7 @@ genesisCore = case protocolVersion @pv of -- This sets up a temporary blob store for the block state that is deleted after use. runTestMonad :: (IsConsensusV1 pv, IsProtocolVersion pv) => BakerContext -> UTCTime -> GenesisData pv -> TestMonad pv a -> IO a runTestMonad _tcBakerContext _tcCurrentTime genData (TestMonad a) = - runBlobStoreTemp "." $ withNewAccountCache 1000 $ do + runLog $ runBlobStoreTemp "." $ withNewAccountCacheAndLMDBAccountMap 1000 "accountmap" $ do (genState, genStateRef, initTT, genTimeoutBase, genEpochBakers) <- runPersistentBlockStateMonad $ do genesisState genData >>= \case Left e -> error e @@ -168,6 +172,7 @@ runTestMonad _tcBakerContext _tcCurrentTime genData (TestMonad a) = _nextPayday = payday } genStateRef <- saveBlockState genState + void $ saveAccounts genState return (genState, genStateRef, initTT, genTimeoutBase, genEpochBakers) let genMetadata = GenesisMetadata @@ -196,11 +201,13 @@ runTestMonad _tcBakerContext _tcCurrentTime genData (TestMonad a) = liftIO . newIORef $! initialLowLevelDB genStoredBlock (_tsSkovData ^. persistentRoundStatus) _tcPersistentBlockStateContext <- ask - let _tcLogger src lvl msg = liftIO $ putStrLn $ "[" ++ show lvl ++ "] " ++ show src ++ ": " ++ msg let ctx = TestContext{..} let _tsPendingTimers = Map.empty let st = TestState{..} - fst <$> liftIO (evalRWST a ctx st) + fst <$> lift (evalRWST a ctx st) + where + logger src lvl msg = putStrLn $ "[" ++ show lvl ++ "] " ++ show src ++ ": " ++ msg + runLog = flip runLoggerT logger -- Instances that are required for the 'TestMonad'. deriving via @@ -264,11 +271,6 @@ instance MonadConsensusEvent (TestMonad pv) where onBlock = tell . (: []) . OnBlock . bpBlock onFinalize fe _ = tell [OnFinalize fe] -instance MonadLogger (TestMonad pv) where - logEvent src lvl msg = do - logger <- view tcLogger - logger src lvl msg - -- | Get the currently-pending timers. getPendingTimers :: TestMonad pv (Map.Map Integer (Timeout, TestMonad pv ())) getPendingTimers = TestMonad (gets _tsPendingTimers) diff --git a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Implementation.hs b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Implementation.hs index 77f682f699..e38f3db0fa 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Implementation.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Implementation.hs @@ -43,6 +43,7 @@ import qualified Concordium.Genesis.Data.BaseV1 as Base import Concordium.Types import Concordium.Types.Execution import Concordium.Types.HashableTo +import Concordium.Types.Option import Concordium.Types.Transactions import Concordium.Types.Updates import Concordium.Utils diff --git a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/LowLevel/LMDB.hs b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/LowLevel/LMDB.hs index e2173d135f..4b1b271100 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/LowLevel/LMDB.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/LowLevel/LMDB.hs @@ -36,6 +36,7 @@ import qualified Concordium.Crypto.SHA256 as Hash import Concordium.Logger import Concordium.Types import Concordium.Types.HashableTo +import Concordium.Types.Option import Concordium.GlobalState.LMDB.Helpers import Concordium.KonsensusV1.TreeState.LowLevel @@ -246,60 +247,10 @@ metadataStoreName = "metadata" databaseCount :: Int databaseCount = 6 --- | Database growth size increment. --- This is currently set at 64MB, and must be a multiple of the page size. -dbStepSize :: Int -dbStepSize = 2 ^ (26 :: Int) -- 64MB - --- | Maximum step to increment the database size. -dbMaxStepSize :: Int -dbMaxStepSize = 2 ^ (30 :: Int) -- 1GB - --- | Initial database size. --- This is currently set to be the same as 'dbStepSize'. -dbInitSize :: Int -dbInitSize = dbStepSize - --- ** Helpers - --- | Resize the LMDB map if the file size has changed. --- This is used to allow a secondary process that is reading the database --- to handle resizes to the database that are made by the writer. --- The supplied action will be executed. If it fails with an 'MDB_MAP_RESIZED' --- error, then the map will be resized and the action retried. -resizeOnResized :: (MonadIO m, MonadReader r m, HasDatabaseHandlers r pv, MonadCatch m) => m a -> m a -resizeOnResized a = do - dbh <- view databaseHandlers - resizeOnResizedInternal (dbh ^. storeEnv) a - --- | Perform a database action and resize the LMDB map if the file size has changed. The difference --- with `resizeOnResized` is that this function takes database handlers as an argument, instead of --- reading their value from `HasDatabaseHandlers`. -resizeOnResizedInternal :: (MonadIO m, MonadCatch m) => StoreEnv -> m a -> m a -resizeOnResizedInternal se a = inner - where - inner = handleJust checkResized onResized a - checkResized LMDB_Error{..} = guard (e_code == Right MDB_MAP_RESIZED) - onResized _ = do - liftIO (withWriteStoreEnv se $ flip mdb_env_set_mapsize 0) - inner - --- | Increase the database size by at least the supplied size. --- The size SHOULD be a multiple of 'dbStepSize', and MUST be a multiple of the page size. -resizeDatabaseHandlers :: (MonadIO m, MonadLogger m) => DatabaseHandlers pv -> Int -> m () -resizeDatabaseHandlers dbh delta = do - envInfo <- liftIO $ mdb_env_info (dbh ^. storeEnv . seEnv) - let oldMapSize = fromIntegral $ me_mapsize envInfo - newMapSize = oldMapSize + delta - _storeEnv = dbh ^. storeEnv - logEvent LMDB LLDebug $ "Resizing database from " ++ show oldMapSize ++ " to " ++ show newMapSize - liftIO . withWriteStoreEnv (dbh ^. storeEnv) $ flip mdb_env_set_mapsize newMapSize - -- ** Initialization -- | Initialize database handlers. --- The size will be rounded up to a multiple of 'dbStepSize'. --- (This ensures in particular that the size is a multiple of the page size, which is required by +-- (The provided @initSize@ must be a multiple of the page size, which is required by -- LMDB.) makeDatabaseHandlers :: -- | Path of database @@ -313,7 +264,7 @@ makeDatabaseHandlers treeStateDir readOnly initSize = do _storeEnv <- makeStoreEnv -- here nobody else has access to the environment, so we need not lock let env = _storeEnv ^. seEnv - mdb_env_set_mapsize env (initSize + dbStepSize - initSize `mod` dbStepSize) + mdb_env_set_mapsize env initSize mdb_env_set_maxdbs env databaseCount mdb_env_set_maxreaders env 126 mdb_env_open env treeStateDir [MDB_RDONLY | readOnly] @@ -359,11 +310,10 @@ makeDatabaseHandlers treeStateDir readOnly initSize = do -- | Initialize database handlers in ReadWrite mode. -- This simply loads the references and does not initialize the databases. --- The initial size is set to 64MB. openDatabase :: FilePath -> IO (DatabaseHandlers pv) openDatabase treeStateDir = do createDirectoryIfMissing False treeStateDir - makeDatabaseHandlers treeStateDir False dbInitSize + makeDatabaseHandlers treeStateDir False defaultEnvSize -- | Close the database. The database should not be used after it is closed. closeDatabase :: DatabaseHandlers pv -> IO () @@ -416,11 +366,11 @@ openReadOnlyDatabase :: openReadOnlyDatabase treeStateDir = do _storeEnv <- makeStoreEnv let env = _storeEnv ^. seEnv - mdb_env_set_mapsize env dbInitSize + mdb_env_set_mapsize env defaultEnvSize mdb_env_set_maxdbs env databaseCount mdb_env_set_maxreaders env 126 mdb_env_open env treeStateDir [MDB_RDONLY] - (_metadataStore, mversion) <- resizeOnResizedInternal _storeEnv $ transaction _storeEnv True $ \txn -> do + (_metadataStore, mversion) <- resizeOnResized _storeEnv $ transaction _storeEnv True $ \txn -> do _metadataStore <- MetadataStore <$> mdb_dbi_open' txn (Just metadataStoreName) [] mversion <- loadRecord txn _metadataStore versionMetadata return (_metadataStore, mversion) @@ -434,7 +384,7 @@ openReadOnlyDatabase treeStateDir = do -- version. case promoteProtocolVersion vmProtocolVersion of SomeProtocolVersion (_ :: SProtocolVersion pv) -> - resizeOnResizedInternal _storeEnv $ transaction _storeEnv True $ \txn -> do + resizeOnResized _storeEnv $ transaction _storeEnv True $ \txn -> do _blockStore <- BlockStore <$> mdb_dbi_open' @@ -483,34 +433,6 @@ deriving instance (MonadReader r m) => MonadReader r (DiskLLDBM pv m) instance (IsProtocolVersion pv) => MonadProtocolVersion (DiskLLDBM pv m) where type MPV (DiskLLDBM pv m) = pv --- | Run a read-only transaction. -asReadTransaction :: (MonadIO m, MonadReader r m, HasDatabaseHandlers r pv) => (DatabaseHandlers pv -> MDB_txn -> IO a) -> DiskLLDBM pv m a -asReadTransaction t = do - dbh <- view databaseHandlers - liftIO $ transaction (dbh ^. storeEnv) True $ t dbh - --- | Run a write transaction. If the transaction fails due to the database being full, this resizes --- the database and retries the transaction. -asWriteTransaction :: (MonadIO m, MonadReader r m, HasDatabaseHandlers r pv, MonadLogger m) => (DatabaseHandlers pv -> MDB_txn -> IO a) -> DiskLLDBM pv m a -asWriteTransaction t = do - dbh <- view databaseHandlers - let doTransaction = transaction (dbh ^. storeEnv) False $ t dbh - inner step = do - r <- liftIO $ tryJust selectDBFullError doTransaction - case r of - Left _ -> do - -- We resize by the step size initially, and by double for each successive - -- failure. - resizeDatabaseHandlers dbh step - inner (min (step * 2) dbMaxStepSize) - Right res -> return res - inner dbStepSize - where - -- only handle the db full error and propagate other exceptions. - selectDBFullError = \case - (LMDB_Error _ _ (Right MDB_MAP_FULL)) -> Just () - _ -> Nothing - -- | Helper function for implementing 'writeFinalizedBlocks'. writeFinalizedBlocksHelper :: (HasDatabaseHandlers dbh pv, IsProtocolVersion pv) => @@ -594,27 +516,40 @@ instance ) => MonadTreeStateStore (DiskLLDBM pv m) where - lookupBlock bh = asReadTransaction $ \dbh txn -> - loadRecord txn (dbh ^. blockStore) bh - memberBlock bh = asReadTransaction $ \dbh txn -> - isRecordPresent txn (dbh ^. blockStore) bh + lookupBlock bh = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + loadRecord txn (dbh ^. blockStore) bh + memberBlock bh = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + isRecordPresent txn (dbh ^. blockStore) bh lookupFirstBlock = lookupBlockByHeight 0 - lookupLastFinalizedBlock = asReadTransaction $ \dbh txn -> - withCursor txn (dbh ^. finalizedBlockIndex) (getCursor CursorLast) >>= \case - Just (Right (_, bh)) -> - loadRecord txn (dbh ^. blockStore) bh - _ -> return Nothing - lookupBlockByHeight height = asReadTransaction $ \dbh txn -> - loadRecord txn (dbh ^. finalizedBlockIndex) height >>= \case - Just bh -> loadRecord txn (dbh ^. blockStore) bh - _ -> return Nothing - lookupTransaction txHash = asReadTransaction $ \dbh txn -> - loadRecord txn (dbh ^. transactionStatusStore) txHash - memberTransaction txHash = asReadTransaction $ \dbh txn -> - isRecordPresent txn (dbh ^. transactionStatusStore) txHash + lookupLastFinalizedBlock = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + withCursor txn (dbh ^. finalizedBlockIndex) (getCursor CursorLast) >>= \case + Just (Right (_, bh)) -> + loadRecord txn (dbh ^. blockStore) bh + _ -> return Nothing + lookupBlockByHeight height = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + loadRecord txn (dbh ^. finalizedBlockIndex) height >>= \case + Just bh -> loadRecord txn (dbh ^. blockStore) bh + _ -> return Nothing + lookupTransaction txHash = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + loadRecord txn (dbh ^. transactionStatusStore) txHash + memberTransaction txHash = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + isRecordPresent txn (dbh ^. transactionStatusStore) txHash writeFinalizedBlocks finBlocks finEntry = do - delBlocks <- asWriteTransaction $ writeFinalizedBlocksHelper finBlocks finEntry + dbh <- ask + delBlocks <- asWriteTransaction (dbh ^. storeEnv) $ writeFinalizedBlocksHelper finBlocks finEntry dbh logEvent LMDB LLTrace $ "Finalized blocks: " ++ show (getHash @BlockHash <$> finBlocks) @@ -623,7 +558,8 @@ instance return () writeCertifiedBlock certBlock qc = do - asWriteTransaction $ writeCertifiedBlockHelper certBlock qc + dbh <- ask + asWriteTransaction (dbh ^. storeEnv) $ writeCertifiedBlockHelper certBlock qc dbh logEvent LMDB LLTrace $ "Certified block: " ++ show (qcBlock qc) @@ -632,7 +568,8 @@ instance ++ ")" writeCertifiedBlockWithFinalization finBlocks certBlock finEntry = do - delBlocks <- asWriteTransaction $ \dbh txn -> do + dbh <- ask + delBlocks <- asWriteTransaction (dbh ^. storeEnv) $ \txn -> do delBlocks <- writeFinalizedBlocksHelper finBlocks finEntry dbh txn writeCertifiedBlockHelper certBlock (feSuccessorQuorumCertificate finEntry) dbh txn return delBlocks @@ -648,32 +585,40 @@ instance ++ show (blockHeight certBlock) ++ ")" - lookupLatestFinalizationEntry = asReadTransaction $ \dbh txn -> - loadRecord txn (dbh ^. latestFinalizationEntryStore) CSKLatestFinalizationEntry + lookupLatestFinalizationEntry = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + loadRecord txn (dbh ^. latestFinalizationEntryStore) CSKLatestFinalizationEntry - lookupCertifiedBlocks = asReadTransaction $ \dbh txn -> do - withCursor txn (dbh ^. nonFinalizedQuorumCertificateStore) $ \cursor -> do - let loop l Nothing = return l - loop _ (Just (Left e)) = throwM . DatabaseInvariantViolation $ e - loop l (Just (Right (_, qc))) = do - loadRecord txn (dbh ^. blockStore) (qcBlock qc) >>= \case - Nothing -> - throwM . DatabaseInvariantViolation $ - "Missing block for QC " - <> show (qcBlock qc) - <> " in round " - <> show (qcRound qc) - Just block -> - loop ((block, qc) : l) =<< getCursor CursorPrevious cursor - loop [] =<< getCursor CursorLast cursor - - lookupCurrentRoundStatus = asReadTransaction $ \dbh txn -> - loadRecord txn (dbh ^. roundStatusStore) CSKRoundStatus >>= \case - Just rs -> return rs - _ -> throwM (DatabaseInvariantViolation "Missing current round status") - - writeCurrentRoundStatus rs = asWriteTransaction $ \dbh txn -> - storeReplaceRecord txn (dbh ^. roundStatusStore) CSKRoundStatus rs + lookupCertifiedBlocks = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> do + withCursor txn (dbh ^. nonFinalizedQuorumCertificateStore) $ \cursor -> do + let loop l Nothing = return l + loop _ (Just (Left e)) = throwM . DatabaseInvariantViolation $ e + loop l (Just (Right (_, qc))) = do + loadRecord txn (dbh ^. blockStore) (qcBlock qc) >>= \case + Nothing -> + throwM . DatabaseInvariantViolation $ + "Missing block for QC " + <> show (qcBlock qc) + <> " in round " + <> show (qcRound qc) + Just block -> + loop ((block, qc) : l) =<< getCursor CursorPrevious cursor + loop [] =<< getCursor CursorLast cursor + + lookupCurrentRoundStatus = do + dbh <- ask + asReadTransaction (dbh ^. storeEnv) $ \txn -> + loadRecord txn (dbh ^. roundStatusStore) CSKRoundStatus >>= \case + Just rs -> return rs + _ -> throwM (DatabaseInvariantViolation "Missing current round status") + + writeCurrentRoundStatus rs = do + dbh <- ask + asWriteTransaction (dbh ^. storeEnv) $ \txn -> + storeReplaceRecord txn (dbh ^. roundStatusStore) CSKRoundStatus rs -- | Initialise the low-level database by writing out the genesis block, initial round status and -- version metadata. @@ -685,16 +630,27 @@ initialiseLowLevelDB :: -- | Initial persistent round status. PersistentRoundStatus -> DiskLLDBM pv m () -initialiseLowLevelDB genesisBlock roundStatus = asWriteTransaction $ \dbh txn -> do - storeReplaceRecord txn (dbh ^. blockStore) (getHash genesisBlock) genesisBlock - storeReplaceRecord txn (dbh ^. finalizedBlockIndex) 0 (getHash genesisBlock) - storeReplaceRecord txn (dbh ^. roundStatusStore) CSKRoundStatus roundStatus - let metadata = - VersionMetadata - { vmDatabaseVersion = 1, - vmProtocolVersion = demoteProtocolVersion (protocolVersion @pv) - } - storeReplaceRecord txn (dbh ^. metadataStore) versionMetadata $ S.encode metadata +initialiseLowLevelDB genesisBlock roundStatus = do + dbh <- ask + asWriteTransaction (dbh ^. storeEnv) $ \txn -> do + storeReplaceRecord txn (dbh ^. blockStore) (getHash genesisBlock) genesisBlock + storeReplaceRecord txn (dbh ^. finalizedBlockIndex) 0 (getHash genesisBlock) + storeReplaceRecord txn (dbh ^. roundStatusStore) CSKRoundStatus roundStatus + let metadata = + VersionMetadata + { vmDatabaseVersion = 1, + vmProtocolVersion = demoteProtocolVersion (protocolVersion @pv) + } + storeReplaceRecord txn (dbh ^. metadataStore) versionMetadata $ S.encode metadata + +-- | A result of a roll back. +data RollbackResult = forall (pv :: ProtocolVersion). + RollbackResult + { -- | Number of blocks rolled back. + rbrCount :: !Int, + -- | Reference to the best block after the rollback. + rbrBestState :: !(BlockStateRef pv) + } -- | Remove certified and finalized blocks from the database whose states cannot be loaded. -- This can throw an exception if the database recovery was not possible. @@ -722,7 +678,8 @@ rollBackBlocksUntil :: ) => -- | Callback for checking if the state at a given reference is valid. (BlockStateRef pv -> DiskLLDBM pv m Bool) -> - DiskLLDBM pv m (Int, BlockStateRef pv) + -- | Returns the number of blocks rolled back and the best state after the roll back. + DiskLLDBM pv m RollbackResult rollBackBlocksUntil checkState = do lookupLastFinalizedBlock >>= \case Nothing -> throwM . DatabaseRecoveryFailure $ "No last finalized block." @@ -736,7 +693,8 @@ rollBackBlocksUntil checkState = do -- The last finalized block is not intact, so roll back all of the -- certified blocks, then roll back finalized blocks. count <- purgeCertified - rollFinalized count lastFin + (count', bstState) <- rollFinalized count lastFin + return $ RollbackResult count' bstState where -- Check the non-finalized certified blocks, from the highest round backwards. checkCertified :: @@ -744,16 +702,17 @@ rollBackBlocksUntil checkState = do Round -> -- highest surviving block state so far (from last finalized block) BlockStateRef pv -> - -- returns the number of blocks rolled back and the highest surviving block state - DiskLLDBM pv m (Int, BlockStateRef pv) + -- returns the @RollbackResult@. + DiskLLDBM pv m RollbackResult checkCertified lastFinRound bestState = do - mHighestQC <- asReadTransaction $ \dbh txn -> + dbh <- ask + mHighestQC <- asReadTransaction (dbh ^. storeEnv) $ \txn -> withCursor txn (dbh ^. nonFinalizedQuorumCertificateStore) (getCursor CursorLast) case mHighestQC of - Nothing -> return (0, bestState) + Nothing -> return $ RollbackResult 0 bestState Just (Left e) -> throwM . DatabaseRecoveryFailure $ e Just (Right (_, qc)) -> checkCertifiedWithQC lastFinRound bestState 0 qc -- Given the round and QC for a certified block, check that the block's state can be @@ -767,10 +726,11 @@ rollBackBlocksUntil checkState = do Int -> -- QC for certified block to check QuorumCertificate -> - -- returns the number of blocks rolled back and the highest surviving block state - DiskLLDBM pv m (Int, BlockStateRef pv) + -- returns the @RollbackResult@. + DiskLLDBM pv m RollbackResult checkCertifiedWithQC lastFinRound bestState !count qc = do - mBlock <- asReadTransaction $ \dbh txn -> + dbh <- ask + mBlock <- asReadTransaction (dbh ^. storeEnv) $ \txn -> loadRecord txn (dbh ^. blockStore) (qcBlock qc) case mBlock of Nothing -> @@ -787,7 +747,7 @@ rollBackBlocksUntil checkState = do (qcRound qc - 1) else do -- Delete the block and the QC - asWriteTransaction $ \dbh txn -> do + asWriteTransaction (dbh ^. storeEnv) $ \txn -> do void $ deleteRecord txn @@ -814,12 +774,13 @@ rollBackBlocksUntil checkState = do Int -> -- round to check for Round -> - -- returns the number of blocks rolled back and the highest surviving block state - DiskLLDBM pv m (Int, BlockStateRef pv) + -- returns the @RollbackResult@. + DiskLLDBM pv m RollbackResult checkCertifiedPreviousRound lastFinRound bestState count currentRound - | currentRound <= lastFinRound = return (count, bestState) + | currentRound <= lastFinRound = return $ RollbackResult count bestState | otherwise = do - mNextQC <- asReadTransaction $ \dbh txn -> + dbh <- ask + mNextQC <- asReadTransaction (dbh ^. storeEnv) $ \txn -> loadRecord txn (dbh ^. nonFinalizedQuorumCertificateStore) currentRound case mNextQC of Nothing -> @@ -828,7 +789,8 @@ rollBackBlocksUntil checkState = do checkCertifiedWithQC lastFinRound bestState count qc -- Purge all of the certified blocks. Returns the number of blocks rolled back. purgeCertified = do - (count, hashes) <- asWriteTransaction $ \dbh txn -> do + dbh <- ask + (count, hashes) <- asWriteTransaction (dbh ^. storeEnv) $ \txn -> do withCursor txn (dbh ^. nonFinalizedQuorumCertificateStore) $ \cursor -> do let loop !count hashes Nothing = return (count, hashes) loop _ _ (Just (Left e)) = throwM . DatabaseRecoveryFailure $ e @@ -850,7 +812,8 @@ rollBackBlocksUntil checkState = do when (blockRound lastFin == 0) $ throwM . DatabaseRecoveryFailure $ "Genesis block state could not be recovered." - (count', hashes, newLastFin) <- asWriteTransaction $ \dbh txn -> do + dbh <- ask + (count', hashes, newLastFin) <- asWriteTransaction (dbh ^. storeEnv) $ \txn -> do let loop :: -- Current count of blocks rolled back Int -> diff --git a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/StartUp.hs b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/StartUp.hs index 10a24208ee..e398417c9f 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/StartUp.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/StartUp.hs @@ -9,6 +9,8 @@ import Control.Monad.Catch import Control.Monad.IO.Class import Control.Monad.State.Strict import qualified Data.Map.Strict as Map + +import Data.Maybe import qualified Data.Sequence as Seq import Lens.Micro.Platform @@ -21,12 +23,14 @@ import Concordium.Types.UpdateQueues import Concordium.Types.Updates import Concordium.Utils +import qualified Concordium.GlobalState.AccountMap.DifferenceMap as DiffMap import Concordium.GlobalState.BlockState import Concordium.GlobalState.Parameters hiding (getChainParameters) import qualified Concordium.GlobalState.Persistent.BlockState as PBS import qualified Concordium.GlobalState.Statistics as Stats import qualified Concordium.GlobalState.TransactionTable as TT import qualified Concordium.GlobalState.Types as GSTypes +import Concordium.ID.Types import Concordium.KonsensusV1.Consensus import Concordium.KonsensusV1.Consensus.Timeout import Concordium.KonsensusV1.Transactions @@ -37,6 +41,9 @@ import Concordium.KonsensusV1.Types import Concordium.Logger import Concordium.TimeMonad import Concordium.TransactionVerification as TVer +import Concordium.Types.Option +import Concordium.Types.Transactions +import qualified Data.HashMap.Strict as HM -- | Generate the 'EpochBakers' for a genesis block. genesisEpochBakers :: @@ -293,8 +300,9 @@ loadSkovData _runtimeParameters didRollback = do -- | Load the certified blocks from the low-level database into the tree state. -- This caches their block states, adds them to the block table and branches, -- adds their transactions to the transaction table and pending transaction table, --- updates the highest certified block, and records block signature witnesses and --- checked quorum certificates for the blocks. +-- updates the highest certified block, records block signature witnesses and +-- checked quorum certificates for the blocks, and inserts any created accounts into +-- the account difference maps for the certified blocks. -- -- This also sets the previous round timeout if the low level state records that it timed out. -- It also puts the latest timeout message in the set of timeout messages for the current round @@ -316,7 +324,10 @@ loadCertifiedBlocks :: m () loadCertifiedBlocks = do certBlocks <- LowLevel.lookupCertifiedBlocks - mapM_ loadCertBlock certBlocks + -- Load all certified blocks + -- This sets the skov state, puts transactions in the transaction table, + -- and reconstructs the account map difference maps for the certified blocks. + foldM_ (flip loadCertBlock) (HM.empty :: HM.HashMap BlockHash DiffMap.DifferenceMapReference) certBlocks oLastTimeout <- use $ persistentRoundStatus . prsLatestTimeout forM_ oLastTimeout $ \lastTimeout -> do curRound <- use $ roundStatus . rsCurrentRound @@ -417,8 +428,32 @@ loadCertifiedBlocks = do when (tmRound (tmBody tm) == rs ^. rsCurrentRound) $ do forM_ (updateTimeoutMessages Absent tm) $ \tms -> currentTimeoutMessages .= Present tms where - loadCertBlock (storedBlock, qc) = do + -- Get the account address from a credential deployment. + getAccountAddressFromDeployment bi = case bi of + WithMetadata{wmdData = CredentialDeployment{biCred = AccountCreation{..}}} -> (Just . addressFromRegId . credId) credential + _ -> Nothing + loadCertBlock (storedBlock, qc) loadedBlocks = do blockPointer <- mkBlockPointer storedBlock + -- As only finalized accounts are stored in the account map, then + -- we need to reconstruct the 'DiffMap.DifferenceMap' here for the certified block we're loading. + let accountsToInsert = mapMaybe getAccountAddressFromDeployment (blockTransactions storedBlock) + -- If a parent cannot be looked up in the @loadedBlocks@ it must mean that parent block is finalized, + -- and as a result we simply set the parent reference for the difference map to be empty. + -- This is alright as the certified blocks we're folding over are in order of ascending round number. + parentDiffMapReference <- case blockBakedData storedBlock of + -- If the parent is a genesis block then there is no difference map for it. + Absent -> liftIO DiffMap.newEmptyReference + Present b -> do + let parentHash = qcBlock $ bbQuorumCertificate $ sbBlock b + -- If the parent cannot be looked up, then it must be finalized and hence no + -- difference map exists. + case HM.lookup parentHash loadedBlocks of + Nothing -> liftIO DiffMap.newEmptyReference + Just diffMapReference -> return diffMapReference + newDifferenceMap <- reconstructAccountDifferenceMap (bpState blockPointer) parentDiffMapReference accountsToInsert + -- append to the accummulator with this new difference map reference + let loadedBlocks' = HM.insert (getHash storedBlock) newDifferenceMap loadedBlocks + cacheBlockState (bpState blockPointer) blockTable . liveMap . at' (getHash blockPointer) ?=! blockPointer addToBranches blockPointer @@ -449,6 +484,7 @@ loadCertifiedBlocks = do roundBakerExistingBlock (blockRound signedBlock) (blockBaker signedBlock) ?= toBlockSignatureWitness signedBlock recordCheckedQuorumCertificate qc + return loadedBlocks' -- Set the previous round timeout. setLastTimeout lastTimeout certBlock = do diff --git a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Types.hs b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Types.hs index 0a2e6ef693..7746b43cd4 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Types.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/TreeState/Types.hs @@ -26,6 +26,7 @@ import Concordium.GlobalState.BakerInfo import qualified Concordium.GlobalState.Persistent.BlockState as PBS import Concordium.GlobalState.TransactionTable import Concordium.KonsensusV1.Types +import Concordium.Types.Option (Option (..), ofOption) -- | Status information for a finalized transaction. data FinalizedTransactionStatus = FinalizedTransactionStatus diff --git a/concordium-consensus/src/Concordium/KonsensusV1/Types.hs b/concordium-consensus/src/Concordium/KonsensusV1/Types.hs index dd4bb7806d..881c806f3f 100644 --- a/concordium-consensus/src/Concordium/KonsensusV1/Types.hs +++ b/concordium-consensus/src/Concordium/KonsensusV1/Types.hs @@ -1,7 +1,6 @@ {-# LANGUAGE BangPatterns #-} {-# LANGUAGE BinaryLiterals #-} {-# LANGUAGE DataKinds #-} -{-# LANGUAGE DeriveTraversable #-} {-# LANGUAGE OverloadedStrings #-} {-# LANGUAGE ScopedTypeVariables #-} {-# LANGUAGE TypeApplications #-} @@ -30,56 +29,12 @@ import Concordium.Genesis.Data.BaseV1 import qualified Concordium.GlobalState.Basic.BlockState.LFMBTree as LFMBT import Concordium.Types import Concordium.Types.HashableTo +import Concordium.Types.Option import Concordium.Types.Parameters (IsConsensusV1) import Concordium.Types.Transactions import Concordium.Utils.BinarySearch import Concordium.Utils.Serialization --- | A strict version of 'Maybe'. -data Option a - = Absent - | Present !a - deriving (Eq, Ord, Show, Functor, Foldable, Traversable) - --- | Putter for an @Option a@. -putOptionOf :: Putter a -> Putter (Option a) -putOptionOf _ Absent = putWord8 0 -putOptionOf pa (Present a) = putWord8 1 >> pa a - --- | Getter for an @Option a@. -getOptionOf :: Get a -> Get (Option a) -getOptionOf ma = do - getWord8 >>= \case - 0 -> return Absent - 1 -> Present <$> ma - _ -> fail "invalid tag for Option" - --- | 'Serialize' instance for an @Option a@. -instance (Serialize a) => Serialize (Option a) where - put = putOptionOf put - get = getOptionOf get - --- | Returns 'True' if and only if the value is 'Present'. -isPresent :: Option a -> Bool -isPresent Absent = False -isPresent (Present _) = True - --- | Returns 'True' if and only if the value is 'Absent'. -isAbsent :: Option a -> Bool -isAbsent Absent = True -isAbsent (Present _) = False - --- | Get the contents of an 'Option' or the supplied default value if it is 'Absent'. -fromOption :: a -> Option a -> a -fromOption def Absent = def -fromOption _ (Present v) = v - --- | Deconstruct an 'Option', returning the first argument if it is 'Absent', and otherwise --- applying the second argument to the value if it is 'Present'. (Analogous to 'maybe'.) -ofOption :: b -> (a -> b) -> Option a -> b -ofOption ab _ Absent = ab -ofOption _ pr (Present v) = pr v - -- | The message that is signed by a finalizer to certify a block. data QuorumSignatureMessage = QuorumSignatureMessage { -- | Hash of the genesis block. diff --git a/concordium-consensus/src/Concordium/MultiVersion.hs b/concordium-consensus/src/Concordium/MultiVersion.hs index d7ec452d45..45ee9d3ceb 100644 --- a/concordium-consensus/src/Concordium/MultiVersion.hs +++ b/concordium-consensus/src/Concordium/MultiVersion.hs @@ -65,7 +65,6 @@ import qualified Concordium.KonsensusV1.SkovMonad as SkovV1 import qualified Concordium.KonsensusV1.Transactions as SkovV1 import qualified Concordium.KonsensusV1.TreeState.LowLevel.LMDB as LowLevelDB import qualified Concordium.KonsensusV1.TreeState.Types as SkovV1 -import Concordium.KonsensusV1.Types (Option (..)) import qualified Concordium.KonsensusV1.Types as KonsensusV1 import qualified Concordium.ProtocolUpdate.V0 as ProtocolUpdateV0 import qualified Concordium.ProtocolUpdate.V1 as ProtocolUpdateV1 @@ -74,6 +73,7 @@ import Concordium.TimeMonad import Concordium.TimerMonad import qualified Concordium.TransactionVerification as TVer import Concordium.Types.CatchUp +import Concordium.Types.Option -- | Handler configuration for supporting protocol updates. -- This handler defines an instance of 'HandlerConfigHandlers' that responds to finalization events @@ -195,7 +195,8 @@ globalStateConfig DiskStateConfig{..} rtp gi _ = ( GlobalStateConfig { dtdbRuntimeParameters = rtp, dtdbTreeStateDirectory = stateBasePath ("treestate-" ++ show gi), - dtdbBlockStateFile = stateBasePath ("blockstate-" ++ show gi) <.> "dat" + dtdbBlockStateFile = stateBasePath ("blockstate-" ++ show gi) <.> "dat", + dtdAccountMapDirectory = stateBasePath "accountmap" } ) @@ -209,7 +210,8 @@ globalStateConfigV1 DiskStateConfig{..} rtp gi = ( SkovV1.GlobalStateConfig { gscRuntimeParameters = rtp, gscTreeStateDirectory = stateBasePath ("treestate-" ++ show gi), - gscBlockStateFile = stateBasePath ("blockstate-" ++ show gi) <.> "dat" + gscBlockStateFile = stateBasePath ("blockstate-" ++ show gi) <.> "dat", + gscAccountMapDirectory = stateBasePath "accountmap" } ) @@ -289,7 +291,7 @@ skovV1Handlers gi genHeight = SkovV1.HandlerContext{..} let isHomeBaked = case nodeBakerIdMaybe of Nothing -> False Just nodeBakerId -> - KonsensusV1.Present nodeBakerId + Present nodeBakerId == (KonsensusV1.blockBaker <$> KonsensusV1.blockBakedData block) liftIO (notifyCallback (getHash block) height isHomeBaked) @@ -304,7 +306,7 @@ skovV1Handlers gi genHeight = SkovV1.HandlerContext{..} let isHomeBaked = case nodeBakerIdMaybe of Nothing -> False Just nodeBakerId -> - KonsensusV1.Present nodeBakerId + Present nodeBakerId == (KonsensusV1.blockBaker <$> KonsensusV1.blockBakedData bp) liftIO (notifyCallback (getHash bp) height isHomeBaked) checkForProtocolUpdateV1 diff --git a/concordium-consensus/src/Concordium/Queries.hs b/concordium-consensus/src/Concordium/Queries.hs index 99f9b9d708..144ca513e9 100644 --- a/concordium-consensus/src/Concordium/Queries.hs +++ b/concordium-consensus/src/Concordium/Queries.hs @@ -75,6 +75,7 @@ import Concordium.Skov as Skov ( SkovQueryMonad (getBlocksAtHeight), evalSkovT, ) +import Concordium.Types.Option import Control.Monad.State.Class import Data.Time @@ -731,7 +732,7 @@ getBlockInfo = let biBlockArriveTime = SkovV1.blockArriveTime bp let biBlockSlot = Nothing -- no slots in consensus version 1 let biBlockSlotTime = timestampToUTCTime $ SkovV1.blockTimestamp bp - let biBlockBaker = SkovV1.ofOption Nothing (Just . SkovV1.blockBaker) $ SkovV1.blockBakedData bp + let biBlockBaker = ofOption Nothing (Just . SkovV1.blockBaker) $ SkovV1.blockBakedData bp let biTransactionCount = SkovV1.blockTransactionCount bp let biTransactionEnergyCost = SkovV1.blockEnergyCost bp let biTransactionsSize = fromIntegral $ SkovV1.blockTransactionsSize bp @@ -1610,9 +1611,9 @@ getBlockCertificates = liftSkovQueryBHI (\_ -> return $ Left BlockCertificatesIn qcAggregateSignature = QueriesKonsensusV1.QuorumCertificateSignature . (SkovV1.theQuorumSignature . SkovV1.qcAggregateSignature) $ qc, qcSignatories = finalizerSetToBakerIds committee (SkovV1.qcSignatories qc) } - mkTimeoutCertificateOut :: SkovV1.FinalizationCommittee -> SkovV1.Option SkovV1.TimeoutCertificate -> Maybe QueriesKonsensusV1.TimeoutCertificate - mkTimeoutCertificateOut _ SkovV1.Absent = Nothing - mkTimeoutCertificateOut committee (SkovV1.Present tc) = + mkTimeoutCertificateOut :: SkovV1.FinalizationCommittee -> Option SkovV1.TimeoutCertificate -> Maybe QueriesKonsensusV1.TimeoutCertificate + mkTimeoutCertificateOut _ Absent = Nothing + mkTimeoutCertificateOut committee (Present tc) = Just $ QueriesKonsensusV1.TimeoutCertificate { tcRound = SkovV1.tcRound tc, @@ -1621,9 +1622,9 @@ getBlockCertificates = liftSkovQueryBHI (\_ -> return $ Left BlockCertificatesIn tcFinalizerQCRoundsSecondEpoch = finalizerRound committee $ SkovV1.tcFinalizerQCRoundsSecondEpoch tc, tcAggregateSignature = QueriesKonsensusV1.TimeoutCertificateSignature . (SkovV1.theTimeoutSignature . SkovV1.tcAggregateSignature) $ tc } - mkEpochFinalizationEntryOut :: SkovV1.FinalizationCommittee -> SkovV1.Option SkovV1.FinalizationEntry -> Maybe QueriesKonsensusV1.EpochFinalizationEntry - mkEpochFinalizationEntryOut _ SkovV1.Absent = Nothing - mkEpochFinalizationEntryOut committee (SkovV1.Present SkovV1.FinalizationEntry{..}) = + mkEpochFinalizationEntryOut :: SkovV1.FinalizationCommittee -> Option SkovV1.FinalizationEntry -> Maybe QueriesKonsensusV1.EpochFinalizationEntry + mkEpochFinalizationEntryOut _ Absent = Nothing + mkEpochFinalizationEntryOut committee (Present SkovV1.FinalizationEntry{..}) = Just $ QueriesKonsensusV1.EpochFinalizationEntry { efeFinalizedQC = mkQuorumCertificateOut committee feFinalizedQuorumCertificate, diff --git a/concordium-consensus/src/Concordium/Skov/MonadImplementations.hs b/concordium-consensus/src/Concordium/Skov/MonadImplementations.hs index 926560d606..c52abccab8 100644 --- a/concordium-consensus/src/Concordium/Skov/MonadImplementations.hs +++ b/concordium-consensus/src/Concordium/Skov/MonadImplementations.hs @@ -31,6 +31,7 @@ import Concordium.Afgjort.Finalize import Concordium.Afgjort.Finalize.Types import Concordium.Afgjort.Monad import Concordium.GlobalState +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap import Concordium.GlobalState.BlockMonads import Concordium.GlobalState.BlockState import Concordium.GlobalState.Finalization @@ -524,9 +525,16 @@ instance (c ~ SkovConfig pv finconfig handlerconfig, AccountVersionFor pv ~ av) instance (c ~ SkovConfig pv finconfig handlerconfig) => HasCache ModuleCache (SkovTContext h (SkovContext c)) where projectCache = projectCache . srContext +instance (c ~ SkovConfig pv finconfig handlerconfig) => LMDBAccountMap.HasDatabaseHandlers (SkovContext c) where + databaseHandlers = lens scGSContext (\s v -> s{scGSContext = v}) . LMDBAccountMap.databaseHandlers + +instance (c ~ SkovConfig pv finconfig handlerconfig) => LMDBAccountMap.HasDatabaseHandlers (SkovTContext h (SkovContext c)) where + databaseHandlers = lens srContext (\s v -> s{srContext = v}) . LMDBAccountMap.databaseHandlers + deriving instance ( IsProtocolVersion pv, MonadIO m, + MonadLogger m, c ~ SkovConfig pv finconfig handlerconfig ) => BlockStateQuery (SkovT pv h c m) @@ -534,6 +542,7 @@ deriving instance deriving instance ( MonadIO m, IsProtocolVersion pv, + MonadLogger m, c ~ SkovConfig pv finconfig handlerconfig ) => AccountOperations (SkovT pv h c m) @@ -541,6 +550,7 @@ deriving instance deriving instance ( MonadIO m, IsProtocolVersion pv, + MonadLogger m, c ~ SkovConfig pv finconfig handlerconfig ) => ContractStateOperations (SkovT pv h c m) @@ -548,6 +558,7 @@ deriving instance deriving instance ( MonadIO m, IsProtocolVersion pv, + MonadLogger m, c ~ SkovConfig pv finconfig handlerconfig ) => ModuleQuery (SkovT pv h c m) @@ -555,6 +566,7 @@ deriving instance deriving instance ( IsProtocolVersion pv, MonadIO m, + MonadLogger m, c ~ SkovConfig pv finconfig handlerconfig ) => BlockStateOperations (SkovT pv h c m) @@ -562,6 +574,7 @@ deriving instance deriving instance ( IsProtocolVersion pv, MonadIO m, + MonadLogger m, c ~ SkovConfig pv finconfig handlerconfig ) => BlockStateStorage (SkovT pv h c m) diff --git a/concordium-consensus/src/Concordium/Types/Option.hs b/concordium-consensus/src/Concordium/Types/Option.hs new file mode 100644 index 0000000000..de28932584 --- /dev/null +++ b/concordium-consensus/src/Concordium/Types/Option.hs @@ -0,0 +1,62 @@ +{-# LANGUAGE DeriveTraversable #-} + +-- | This module provides a strict version of 'Maybe'. +module Concordium.Types.Option ( + -- * Strict version of 'Maybe'. + Option (..), + + -- * Auxiliary functions + putOptionOf, + getOptionOf, + isPresent, + isAbsent, + fromOption, + ofOption, +) where + +import Data.Serialize + +-- | A strict version of 'Maybe'. +data Option a + = Absent + | Present !a + deriving (Eq, Ord, Show, Functor, Foldable, Traversable) + +-- | Putter for an @Option a@. +putOptionOf :: Putter a -> Putter (Option a) +putOptionOf _ Absent = putWord8 0 +putOptionOf pa (Present a) = putWord8 1 >> pa a + +-- | Getter for an @Option a@. +getOptionOf :: Get a -> Get (Option a) +getOptionOf ma = do + getWord8 >>= \case + 0 -> return Absent + 1 -> Present <$> ma + _ -> fail "invalid tag for Option" + +-- | 'Serialize' instance for an @Option a@. +instance (Serialize a) => Serialize (Option a) where + put = putOptionOf put + get = getOptionOf get + +-- | Returns 'True' if and only if the value is 'Present'. +isPresent :: Option a -> Bool +isPresent Absent = False +isPresent (Present _) = True + +-- | Returns 'True' if and only if the value is 'Absent'. +isAbsent :: Option a -> Bool +isAbsent Absent = True +isAbsent (Present _) = False + +-- | Get the contents of an 'Option' or the supplied default value if it is 'Absent'. +fromOption :: a -> Option a -> a +fromOption def Absent = def +fromOption _ (Present v) = v + +-- | Deconstruct an 'Option', returning the first argument if it is 'Absent', and otherwise +-- applying the second argument to the value if it is 'Present'. (Analogous to 'maybe'.) +ofOption :: b -> (a -> b) -> Option a -> b +ofOption ab _ Absent = ab +ofOption _ pr (Present v) = pr v diff --git a/concordium-consensus/test-runners/deterministic/Main.hs b/concordium-consensus/test-runners/deterministic/Main.hs index cff7496f6b..728995b8d7 100644 --- a/concordium-consensus/test-runners/deterministic/Main.hs +++ b/concordium-consensus/test-runners/deterministic/Main.hs @@ -68,8 +68,8 @@ type PV = 'P5 -- | Construct the global state configuration. -- Can be customised if changing the configuration. -makeGlobalStateConfig :: RuntimeParameters -> FilePath -> FilePath -> IO GlobalStateConfig -makeGlobalStateConfig rt treeStateDir blockStateFile = return $ GlobalStateConfig rt treeStateDir blockStateFile +makeGlobalStateConfig :: RuntimeParameters -> FilePath -> FilePath -> FilePath -> IO GlobalStateConfig +makeGlobalStateConfig rt treeStateDir blockStateFile accMapDirectory = return $ GlobalStateConfig rt treeStateDir blockStateFile accMapDirectory {- type TreeConfig = PairGSConfig MemoryTreeMemoryBlockConfig DiskTreeDiskBlockConfig @@ -316,6 +316,7 @@ initialState numAccts = do defaultRuntimeParameters{rpAccountsCacheSize = 5000} ("data/treestate-" ++ show now ++ "-" ++ show bakerId) ("data/blockstate-" ++ show now ++ "-" ++ show bakerId ++ ".dat") + ("data/accountmap-" ++ show now ++ "-" ++ show bakerId ++ ".dat") let finconfig = ActiveFinalization (FinalizationInstance (bakerSignKey _bsIdentity) (bakerElectionKey _bsIdentity) (bakerAggregationKey _bsIdentity)) hconfig = NoHandler diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/EndToEnd/CredentialDeploymentTests.hs b/concordium-consensus/tests/consensus/ConcordiumTests/EndToEnd/CredentialDeploymentTests.hs new file mode 100644 index 0000000000..22acf3b5e9 --- /dev/null +++ b/concordium-consensus/tests/consensus/ConcordiumTests/EndToEnd/CredentialDeploymentTests.hs @@ -0,0 +1,342 @@ +{-# LANGUAGE NumericUnderscores #-} +{-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE TemplateHaskell #-} + +-- | End to end tests for credential deployments. +-- For future maintainers: Note that the blocks below have hardcoded transaction outcome and state hashes. +-- These can be obtained by running the test and observe the program output. +-- (The monad we're running the tests within has a logger) +-- It is not expected that the hardcoded hashes change unless the protocol version changes (AND the underlying hashing scheme). +module ConcordiumTests.EndToEnd.CredentialDeploymentTests (tests) where + +import Concordium.Utils +import Control.Monad.IO.Class +import Control.Monad.State +import qualified Data.Aeson as AE +import qualified Data.ByteString.Lazy as BSL +import Data.FileEmbed +import qualified Data.Vector as Vec +import Lens.Micro.Platform +import Test.HUnit +import Test.Hspec + +import Concordium.Common.Version +import Concordium.GlobalState.BlockState +import Concordium.ID.Types +import Concordium.KonsensusV1.TestMonad +import Concordium.KonsensusV1.TreeState.Implementation +import Concordium.KonsensusV1.TreeState.Types +import Concordium.KonsensusV1.Types +import Concordium.Types +import Concordium.Types.HashableTo +import Concordium.Types.Option +import Concordium.Types.Transactions + +import ConcordiumTests.KonsensusV1.Consensus.Blocks hiding (testBB1, testBB2, testBB2', testBB3, testBB3', tests) + +-- | Helper for reading an 'AccountCreation' from a 'ByteString'. +readAccountCreation :: BSL.ByteString -> AccountCreation +readAccountCreation bs = + case AE.eitherDecode bs of + Left err -> error $ "Cannot read account creation " ++ err + Right d -> if vVersion d == 0 then vValue d else error "Incorrect account creation version." + +-- 3 valid credentials +{-# WARNING cred1 "Do not use in production." #-} +cred1 :: AccountCreation +cred1 = readAccountCreation . BSL.fromStrict $ $(makeRelativeToProject "testdata/initial-credential-1.json" >>= embedFile) + +{-# WARNING cred2 "Do not use in production." #-} +cred2 :: AccountCreation +cred2 = readAccountCreation . BSL.fromStrict $ $(makeRelativeToProject "testdata/initial-credential-2.json" >>= embedFile) + +{-# WARNING cred3 "Do not use in production." #-} +cred3 :: AccountCreation +cred3 = readAccountCreation . BSL.fromStrict $ $(makeRelativeToProject "testdata/credential-1.json" >>= embedFile) + +-- | A credential deployment transaction yielding cred1. +credBi1 :: BlockItem +credBi1 = + credentialDeployment $ addMetadata (\x -> CredentialDeployment{biCred = x}) (tt + 1) cred1 + where + tt = utcTimeToTransactionTime testTime + +-- | A credential deployment transaction yielding cred2. +credBi2 :: BlockItem +credBi2 = + credentialDeployment $ addMetadata (\x -> CredentialDeployment{biCred = x}) (tt + 1) cred2 + where + tt = utcTimeToTransactionTime testTime + +-- | A credential deployment transaction yielding cred3 +credBi3 :: BlockItem +credBi3 = + credentialDeployment $ addMetadata (\x -> CredentialDeployment{biCred = x}) (tt + 1) cred3 + where + tt = utcTimeToTransactionTime testTime + +-- | Valid block for round 1 with 1 credential deployment. +testBB1 :: BakedBlock +testBB1 = + BakedBlock + { bbRound = 1, + bbEpoch = 0, + bbTimestamp = 1_000, + bbBaker = bakerId, + bbQuorumCertificate = genesisQuorumCertificate genesisHash, + bbTimeoutCertificate = Absent, + bbEpochFinalizationEntry = Absent, + bbNonce = computeBlockNonce genesisLEN 1 (bakerVRFKey bakerId), + bbTransactions = Vec.fromList [credBi1], + bbTransactionOutcomesHash = read "b9444648bf759471276fdba1930af0c543847d22de89c27939791898d757516d", + bbStateHash = read "b8bc96ec5f162db36784ea96ec29e3e8ad92abff341a6847e3bf524fdada28ff" + } + where + bakerId = 2 + +-- | Valid block for round 2. +-- This block carries a QC for 'testBB1' thus certifying it. +testBB2 :: BakedBlock +testBB2 = + BakedBlock + { bbRound = 2, + bbEpoch = 0, + bbTimestamp = 3_000, + bbBaker = bakerId, + bbQuorumCertificate = validQCFor testBB1, + bbTimeoutCertificate = Absent, + bbEpochFinalizationEntry = Absent, + bbNonce = computeBlockNonce genesisLEN 2 (bakerVRFKey bakerId), + bbTransactions = Vec.empty, + bbTransactionOutcomesHash = read "375fef64a251f353d608171d283d00fe00aa0bd77596ba7703c810f48056ef89", + bbStateHash = read "798d5089818bcc7b8873e2585fb4fbf3d4dceffca32531259f466e7c435c8817" + } + where + bakerId = 4 + +-- | Valid block for round 3, finalizes 'testBB1' as this block +-- carries a QC for 'testBB2'. +testBB3 :: BakedBlock +testBB3 = + BakedBlock + { bbRound = 3, + bbEpoch = 0, + bbTimestamp = 5_000, + bbBaker = bakerId, + bbQuorumCertificate = validQCFor testBB2, + bbTimeoutCertificate = Absent, + bbEpochFinalizationEntry = Absent, + bbNonce = computeBlockNonce genesisLEN 3 (bakerVRFKey bakerId), + bbTransactions = Vec.empty, + bbTransactionOutcomesHash = read "375fef64a251f353d608171d283d00fe00aa0bd77596ba7703c810f48056ef89", + bbStateHash = read "4da0deab5b564cd77c617a2ac7dc8a6064f87e99b09e58c87b5f9e687db2197a" + } + where + bakerId = 4 + +-- | A test that deploys a single credential, and it ends up in the last finalized block. +testDeployCredential :: Assertion +testDeployCredential = runTestMonad noBaker testTime genesisData $ do + lfbState0 <- use (lastFinalized . to bpState) + noAccs0 <- length <$> getAccountList lfbState0 + let b1 = signedPB testBB1 + succeedReceiveBlock b1 + let b2 = signedPB testBB2 + succeedReceiveBlock b2 + -- b3 finalizes b1 as it carries a qc for b2 (which carries a qc for b1). + let b3 = signedPB testBB3 + succeedReceiveBlock b3 + -- check that the account is now present in the last finalized block. + lfbState1 <- use (lastFinalized . to bpState) + noAccs1 <- length <$> getAccountList lfbState1 + liftIO $ assertEqual "there should be one extra account in lfb" (noAccs0 + 1) noAccs1 + +-- | Valid block for round 2. +-- This block has one credential deployment. +-- This block carries a QC for 'testBB1' thus certifying it. +testBB2' :: BakedBlock +testBB2' = + BakedBlock + { bbRound = 2, + bbEpoch = 0, + bbTimestamp = 3_000, + bbBaker = bakerId, + bbQuorumCertificate = validQCFor testBB1, + bbTimeoutCertificate = Absent, + bbEpochFinalizationEntry = Absent, + bbNonce = computeBlockNonce genesisLEN 2 (bakerVRFKey bakerId), + bbTransactions = Vec.fromList [credBi2], + bbTransactionOutcomesHash = read "abc4628869bb526115226dd01ad54bf33f54609fa770d50a9242aaf009f42fa1", + bbStateHash = read "e3cf3b280159bc20645738fb1343486d16104989a524fb5feb59ac1b0b7af9ad" + } + where + bakerId = 4 + +-- | Valid block for round 3, carries a TC for round 2. +-- This block has one credential deployment. +testBB3' :: BakedBlock +testBB3' = + BakedBlock + { bbRound = 3, + bbEpoch = 0, + bbTimestamp = 5_000, + bbBaker = bakerId, + bbQuorumCertificate = validQCFor testBB1, + bbTimeoutCertificate = Present (validTimeoutFor (validQCFor testBB1) 2), + bbEpochFinalizationEntry = Absent, + bbNonce = computeBlockNonce genesisLEN 3 (bakerVRFKey bakerId), + bbTransactions = Vec.fromList [credBi3], + bbTransactionOutcomesHash = read "3af8504795a03353248be256f66366263f7484c814c5a26760210bbdfd609003", + bbStateHash = read "67eb8f778a4a43efa80c73a954110154ae417e21d43c33b857b962af36913e29" + } + where + bakerId = 4 + +testBB4 :: BakedBlock +testBB4 = + BakedBlock + { bbRound = 4, + bbEpoch = 0, + bbTimestamp = 7_000, + bbBaker = bakerId, + bbQuorumCertificate = validQCFor testBB3', + bbTimeoutCertificate = Absent, + bbEpochFinalizationEntry = Absent, + bbNonce = computeBlockNonce genesisLEN 4 (bakerVRFKey bakerId), + bbTransactions = Vec.empty, + bbTransactionOutcomesHash = read "b0972dd7af05ed6feaa40099fffa9c5c5e0ba9741938166cdb57584780688743", + bbStateHash = read "9e698b9c6425b382d8fda5584f530688c237ad013e8aaf848fea274e50244111" + } + where + bakerId = 3 + +testBB5 :: BakedBlock +testBB5 = + BakedBlock + { bbRound = 5, + bbEpoch = 0, + bbTimestamp = 9_000, + bbBaker = bakerId, + bbQuorumCertificate = validQCFor testBB4, + bbTimeoutCertificate = Absent, + bbEpochFinalizationEntry = Absent, + bbNonce = computeBlockNonce genesisLEN 5 (bakerVRFKey bakerId), + bbTransactions = Vec.empty, + bbTransactionOutcomesHash = read "b0972dd7af05ed6feaa40099fffa9c5c5e0ba9741938166cdb57584780688743", + bbStateHash = read "d9dd62c227d1cbc0d42da0d90bfc11d61533d058cc54b0745d6a597039dbe0ec" + } + where + bakerId = 3 + +-- | Compute the 'AccountCreation' from the provided 'AccountCreation'. +getAccAddress :: AccountCreation -> AccountAddress +getAccAddress accCreation = case credential accCreation of + InitialACWP x -> initialCredentialAccountAddress $ icdiValues x + NormalACWP x -> credentialAccountAddress $ cdiValues x + +-- | Test that two credential deployments (each on their own branch and with same block height) does not: +-- * Alter the state of the parent block (a new child difference map and associated reference is created). +testDeployCredentialBranching :: Assertion +testDeployCredentialBranching = runTestMonad noBaker testTime genesisData $ do + genesisState <- use (lastFinalized . to bpState) + noGenesisAccs <- length <$> getAccountList genesisState + let b1 = signedPB testBB1 + succeedReceiveBlock b1 + -- Branch + let b2 = signedPB testBB2' + succeedReceiveBlock b2 + -- Another branch. + let b3 = signedPB testBB3' + succeedReceiveBlock b3 + + sd <- get + + -- Check that only the first credential deployed is present in block b1. + case sd ^. blockTable . liveMap . at' (getHash b1) of + Nothing -> liftIO $ assertFailure "failed getting bp1" + Just bp1 -> do + noAccountsBp1 <- length <$> getAccountList (bpState bp1) + liftIO $ assertEqual "check that there is one extra account" (noGenesisAccs + 1) noAccountsBp1 + getAccount (bpState bp1) (getAccAddress cred1) >>= \case + Nothing -> liftIO $ assertFailure "Should yield cred1" + Just (accIndex, _) -> liftIO $ assertEqual "incorrect account index" noGenesisAccs (fromIntegral accIndex) + + getAccount (bpState bp1) (getAccAddress cred2) >>= \case + Nothing -> return () + Just _ -> liftIO $ assertFailure "cred2 should not be present" + + getAccount (bpState bp1) (getAccAddress cred3) >>= \case + Nothing -> return () + Just _ -> liftIO $ assertFailure "cred3 should not be present" + + -- Check that cred1 and cred2 is present in b2 (but not cred3) + case sd ^. blockTable . liveMap . at' (getHash b2) of + Nothing -> liftIO $ assertFailure "failed getting bp1" + Just bp2 -> do + noAccountsBp2 <- length <$> getAccountList (bpState bp2) + liftIO $ assertEqual "check that there is one extra account" (noGenesisAccs + 2) noAccountsBp2 + getAccount (bpState bp2) (getAccAddress cred1) >>= \case + Nothing -> liftIO $ assertFailure "Should yield cred1" + Just (accIndex, _) -> liftIO $ assertEqual "incorrect account index" noGenesisAccs (fromIntegral accIndex) + + getAccount (bpState bp2) (getAccAddress cred2) >>= \case + Nothing -> liftIO $ assertFailure "Should yield cred2" + Just (accIndex, _) -> liftIO $ assertEqual "incorrect account index" (noGenesisAccs + 1) (fromIntegral accIndex) + + getAccount (bpState bp2) (getAccAddress cred3) >>= \case + Nothing -> return () + Just _ -> liftIO $ assertFailure $ "cred3 should not be present: " <> show (getAccAddress cred3) + + -- Check that cred1 and cred3 is present in b3 (but not cred2) + case sd ^. blockTable . liveMap . at' (getHash b3) of + Nothing -> liftIO $ assertFailure "failed getting bp1" + Just bp3 -> do + noAccountsBp3 <- length <$> getAccountList (bpState bp3) + liftIO $ assertEqual "check that there is one extra account" (noGenesisAccs + 2) noAccountsBp3 + getAccount (bpState bp3) (getAccAddress cred1) >>= \case + Nothing -> liftIO $ assertFailure "Should yield cred1" + Just (accIndex, _) -> liftIO $ assertEqual "incorrect account index" noGenesisAccs (fromIntegral accIndex) + + getAccount (bpState bp3) (getAccAddress cred3) >>= \case + Nothing -> liftIO $ assertFailure "Should yield cred3" + Just (accIndex, _) -> liftIO $ assertEqual "incorrect account index" (noGenesisAccs + 1) (fromIntegral accIndex) + + getAccount (bpState bp3) (getAccAddress cred2) >>= \case + Nothing -> return () + Just _ -> liftIO $ assertFailure $ "cred2 should not be present: " <> show (getAccAddress cred3) + + -- finalize bp3 and make sure that the state of the lfb matches b3. + let b4 = signedPB testBB4 + succeedReceiveBlock b4 + let b5 = signedPB testBB5 + succeedReceiveBlock b5 + + lfbState <- use (lastFinalized . to bpState) + noAccountsLfb <- length <$> getAccountList lfbState + liftIO $ assertEqual "check that there aer two extra accounts (cred 1 and 3)" (noGenesisAccs + 2) noAccountsLfb + + getAccount lfbState (getAccAddress cred1) >>= \case + Nothing -> liftIO $ assertFailure "Should yield cred1" + Just (accIndex, _) -> liftIO $ assertEqual "incorrect account index" noGenesisAccs (fromIntegral accIndex) + + getAccount lfbState (getAccAddress cred3) >>= \case + Nothing -> liftIO $ assertFailure "Should yield cred3" + Just (accIndex, _) -> liftIO $ assertEqual "incorrect account index" (noGenesisAccs + 1) (fromIntegral accIndex) + + getAccount lfbState (getAccAddress cred2) >>= \case + Nothing -> return () + Just _ -> liftIO $ assertFailure $ "cred2 should not be present: " <> show (getAccAddress cred2) + + -- Check that querying the old bs is not affected by the updated lmdb backed account map. + noFinal <- length <$> getAccountList genesisState + liftIO $ assertEqual "There should be the same number of accounts present" noGenesisAccs noFinal + -- We thaw here so we can use @bsoGetAccountIndex@ for querying account index directly. + updatableBlockState <- thawBlockState genesisState + bsoGetAccountIndex updatableBlockState (getAccAddress cred1) >>= \case + Nothing -> return () + Just _ -> liftIO $ assertFailure "cred 1 should not be present." + +tests :: Word -> Spec +tests _ = describe "EndToEndTests.CredentialDeployments" $ do + it "deploy and finalize one credential" testDeployCredential + it "deploy two credentials in two branches" testDeployCredentialBranching diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/FinalizationRecover.hs b/concordium-consensus/tests/consensus/ConcordiumTests/FinalizationRecover.hs index eb45e74cbd..eb52875627 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/FinalizationRecover.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/FinalizationRecover.hs @@ -37,7 +37,7 @@ dummyArs :: AnonymityRevokers dummyArs = emptyAnonymityRevokers makeGlobalStateConfig :: FilePath -> RuntimeParameters -> GlobalStateConfig -makeGlobalStateConfig tempDir rt = GlobalStateConfig rt tempDir (tempDir "data" <.> "blob") +makeGlobalStateConfig tempDir rt = GlobalStateConfig rt tempDir (tempDir "data" <.> "blob") (tempDir "accountmap") genesis :: Word -> (GenesisData PV, [(BakerIdentity, FullBakerInfo)], Amount) genesis nBakers = diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/CatchUp.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/CatchUp.hs index 0472be2014..c984b8ccb6 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/CatchUp.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/CatchUp.hs @@ -28,6 +28,7 @@ import Concordium.KonsensusV1.TestMonad import Concordium.KonsensusV1.TreeState.Implementation import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types +import Concordium.Types.Option import ConcordiumTests.KonsensusV1.Consensus.Blocks import qualified ConcordiumTests.KonsensusV1.Consensus.Blocks as TestBlocks diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Common.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Common.hs index b1772abb1e..3dedffdb00 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Common.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Common.hs @@ -13,6 +13,7 @@ import qualified Concordium.Crypto.SHA256 as Hash import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types import Concordium.Types +import Concordium.Types.Option import Concordium.Types.Transactions import ConcordiumTests.KonsensusV1.TreeStateTest hiding (tests) diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus.hs index d9a713d02c..831701f4b4 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus.hs @@ -26,6 +26,7 @@ import Concordium.KonsensusV1.Types import Concordium.Startup import Concordium.Types import qualified Concordium.Types.DummyData as Dummy +import Concordium.Types.Option import ConcordiumTests.KonsensusV1.TreeStateTest (dummyBlock) genesisData :: GenesisData 'P6 diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus/Blocks.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus/Blocks.hs index 245d3464c1..fab6774f13 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus/Blocks.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Consensus/Blocks.hs @@ -46,6 +46,7 @@ import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types import Concordium.Startup import Concordium.TimerMonad +import Concordium.Types.Option maxBaker :: (Integral a) => a maxBaker = 5 diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/LMDB.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/LMDB.hs index 940ca460ed..b7fc4d2912 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/LMDB.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/LMDB.hs @@ -31,6 +31,7 @@ import Concordium.KonsensusV1.Types import Concordium.Logger import Concordium.Types import Concordium.Types.HashableTo +import Concordium.Types.Option import Concordium.Types.Transactions -- | A dummy UTCTime used for tests where the actual value is not significant. @@ -194,7 +195,7 @@ runLLMDBTest :: IO a runLLMDBTest name action = withTempDirectory "" name $ \path -> bracket - (makeDatabaseHandlers path False 1000 :: IO (DatabaseHandlers 'P6)) + (openDatabase path :: IO (DatabaseHandlers 'P6)) closeDatabase (\dbhandlers -> runSilentLogger $ runReaderT (runDiskLLDBM action) dbhandlers) diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Timeout.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Timeout.hs index efb4723ad1..df41f647ee 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Timeout.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Timeout.hs @@ -40,6 +40,7 @@ import Concordium.Startup import Concordium.Types import Concordium.Types.BakerIdentity import qualified Concordium.Types.DummyData as Dummy +import Concordium.Types.Option import ConcordiumTests.KonsensusV1.Common import ConcordiumTests.KonsensusV1.TreeStateTest hiding (tests) import ConcordiumTests.KonsensusV1.Types hiding (tests) diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TransactionProcessingTest.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TransactionProcessingTest.hs index 444bf35ac1..0d2a3adc77 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TransactionProcessingTest.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TransactionProcessingTest.hs @@ -51,6 +51,7 @@ import Concordium.GlobalState.Persistent.BlockState import Concordium.GlobalState.Persistent.Genesis (genesisState) import Concordium.GlobalState.TransactionTable import Concordium.ID.Types (randomAccountAddress) +import Concordium.Logger import Concordium.Scheduler.DummyData import Concordium.TimeMonad import qualified Concordium.TransactionVerification as TVer @@ -59,6 +60,7 @@ import Concordium.Types.AnonymityRevokers import Concordium.Types.Execution import Concordium.Types.HashableTo import Concordium.Types.IdentityProviders +import Concordium.Types.Option import Concordium.Types.Parameters import Concordium.Types.Transactions @@ -128,6 +130,12 @@ instance (MonadReader r m) => MonadReader r (FixedTimeT m) where ask = lift ask local f (FixedTime k) = FixedTime $ local f . k +newtype NoLoggerT m a = NoLoggerT {runNoLoggerT :: m a} + deriving (Functor, Applicative, Monad, MonadIO, MonadReader r, MonadFail, TimeMonad, MonadState s) + +instance (Monad m) => MonadLogger (NoLoggerT m) where + logEvent _ _ _ = return () + -- | A test monad that is suitable for testing transaction processing -- as it derives the required capabilities. -- I.e. 'BlockStateQuery' is supported via the 'PersistentBlockStateMonad and a 'MonadState' over the 'SkovData pv'. @@ -137,9 +145,11 @@ type MyTestMonad = ( PersistentBlockStateMonad 'P6 (PersistentBlockStateContext 'P6) - ( StateT - (SkovData 'P6) - (FixedTimeT (BlobStoreM' (PersistentBlockStateContext 'P6))) + ( NoLoggerT + ( StateT + (SkovData 'P6) + (FixedTimeT (BlobStoreM' (PersistentBlockStateContext 'P6))) + ) ) ) @@ -152,15 +162,19 @@ type MyTestMonad = runMyTestMonad :: IdentityProviders -> UTCTime -> MyTestMonad a -> IO (a, SkovData 'P6) runMyTestMonad idps time action = do runBlobStoreTemp "." $ - withNewAccountCache 1_000 $ do - initState <- runPersistentBlockStateMonad initialData - runDeterministic (runStateT (runPersistentBlockStateMonad (runAccountNonceQueryT action)) initState) time + withNewAccountCacheAndLMDBAccountMap 1_000 "accountmap" $ do + initState <- runNoLoggerT $ runPersistentBlockStateMonad initialData + flip runDeterministic time $ + flip runStateT initState $ + runNoLoggerT $ + runPersistentBlockStateMonad $ + runAccountNonceQueryT action where initialData :: PersistentBlockStateMonad 'P6 (PersistentBlockStateContext 'P6) - (BlobStoreM' (PersistentBlockStateContext 'P6)) + (NoLoggerT (BlobStoreM' (PersistentBlockStateContext 'P6))) (SkovData 'P6) initialData = do (bs, _) <- diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TreeStateTest.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TreeStateTest.hs index 61a86e0660..d84a32e079 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TreeStateTest.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/TreeStateTest.hs @@ -103,6 +103,7 @@ import Concordium.KonsensusV1.TreeState.LowLevel.Memory import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types import qualified Concordium.TransactionVerification as TVer +import Concordium.Types.Option import Concordium.Types.Updates -- We derive these instances here so we don't accidentally end up using them in production. diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Types.hs b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Types.hs index 0e6d6beb8e..69486ceb5f 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Types.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/KonsensusV1/Types.hs @@ -25,6 +25,7 @@ import qualified Data.FixedByteString as FBS import Concordium.KonsensusV1.TreeState.Types import Concordium.KonsensusV1.Types +import Concordium.Types.Option -- | Generate a 'FinalizerSet'. The size parameter determines the size of the committee that -- the finalizers are (nominally) sampled from. diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/PassiveFinalization.hs b/concordium-consensus/tests/consensus/ConcordiumTests/PassiveFinalization.hs index 11f50d6b03..4eb0f56d60 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/PassiveFinalization.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/PassiveFinalization.hs @@ -370,7 +370,7 @@ createInitStates additionalFinMembers = do let fininst = FinalizationInstance (bakerSignKey bid) (bakerElectionKey bid) (bakerAggregationKey bid) config = SkovConfig - (GlobalStateConfig defaultRuntimeParameters tempDir (tempDir "data" <.> "blob")) + (GlobalStateConfig defaultRuntimeParameters tempDir (tempDir "data" <.> "blob") (tempDir "accountmap")) (ActiveFinalization fininst) NoHandler (initCtx, initState) <- runSilentLogger (initialiseSkov gen config) diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/ReceiveTransactionsTest.hs b/concordium-consensus/tests/consensus/ConcordiumTests/ReceiveTransactionsTest.hs index 0092b94de1..679f5fba64 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/ReceiveTransactionsTest.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/ReceiveTransactionsTest.hs @@ -243,8 +243,8 @@ runTestSkovQueryMonad' :: TestSkovQueryMonad a -> UTCTime -> GenesisData PV -> I runTestSkovQueryMonad' act time gd = do withTempDirectory "." "treestate" $ \tsDir -> do Blob.runBlobStoreTemp "." $ - BS.withNewAccountCache 1000 $ do - initState <- runPersistentBlockStateMonad $ initialData tsDir + BS.withNewAccountCacheAndLMDBAccountMap 1000 "accountmap" $ do + initState <- runNoLoggerT $ runPersistentBlockStateMonad $ initialData tsDir runDeterministic (runNoLoggerT (runStateT (runPersistentBlockStateMonad . runPersistentTreeStateMonad . runSkovQueryMonad $ act) initState)) time where initialData :: @@ -252,7 +252,7 @@ runTestSkovQueryMonad' act time gd = do BS.PersistentBlockStateMonad PV (BS.PersistentBlockStateContext PV) - (Blob.BlobStoreM' (BS.PersistentBlockStateContext PV)) + (NoLoggerT (Blob.BlobStoreM' (BS.PersistentBlockStateContext PV))) TestSkovState initialData tsDir = do (bs, genTT) <- diff --git a/concordium-consensus/tests/consensus/ConcordiumTests/Update.hs b/concordium-consensus/tests/consensus/ConcordiumTests/Update.hs index 92a2effa1b..5289ccf5b7 100644 --- a/concordium-consensus/tests/consensus/ConcordiumTests/Update.hs +++ b/concordium-consensus/tests/consensus/ConcordiumTests/Update.hs @@ -135,7 +135,7 @@ createInitStates dir = do let fininst = FinalizationInstance (bakerSignKey bid) (bakerElectionKey bid) (bakerAggregationKey bid) config = SkovConfig - (GlobalStateConfig defaultRuntimeParameters (dir uni) (dir uni <.> "dat")) + (GlobalStateConfig defaultRuntimeParameters (dir uni) (dir uni <.> "dat") (dir uni <.> "accountmap")) (ActiveFinalization fininst) NoHandler (initCtx, initState) <- runSilentLogger (initialiseSkov gen config) diff --git a/concordium-consensus/tests/consensus/Spec.hs b/concordium-consensus/tests/consensus/Spec.hs index 03297f22ae..fb31005530 100644 --- a/concordium-consensus/tests/consensus/Spec.hs +++ b/concordium-consensus/tests/consensus/Spec.hs @@ -7,6 +7,7 @@ import qualified ConcordiumTests.Afgjort.Freeze (tests) import qualified ConcordiumTests.Afgjort.Lottery (tests) import qualified ConcordiumTests.Afgjort.Types (tests) import qualified ConcordiumTests.Afgjort.WMVBA (tests) +import qualified ConcordiumTests.EndToEnd.CredentialDeploymentTests (tests) import qualified ConcordiumTests.FinalizationRecover (test) import qualified ConcordiumTests.KonsensusV1.CatchUp (tests) import qualified ConcordiumTests.KonsensusV1.Consensus (tests) @@ -63,3 +64,4 @@ main = atLevel $ \lvl -> hspec $ do ConcordiumTests.KonsensusV1.Timeout.tests ConcordiumTests.KonsensusV1.Consensus.Blocks.tests ConcordiumTests.KonsensusV1.CatchUp.tests + ConcordiumTests.EndToEnd.CredentialDeploymentTests.tests lvl diff --git a/concordium-consensus/tests/globalstate/GlobalStateTests/AccountReleaseScheduleTest.hs b/concordium-consensus/tests/globalstate/GlobalStateTests/AccountReleaseScheduleTest.hs index 4f727481ee..cad3eb7160 100644 --- a/concordium-consensus/tests/globalstate/GlobalStateTests/AccountReleaseScheduleTest.hs +++ b/concordium-consensus/tests/globalstate/GlobalStateTests/AccountReleaseScheduleTest.hs @@ -14,6 +14,7 @@ import Concordium.GlobalState.BlockState import Concordium.GlobalState.DummyData import Concordium.GlobalState.Persistent.BlobStore import qualified Concordium.GlobalState.Persistent.BlockState as PBS +import Concordium.Logger import Concordium.Scheduler.DummyData import Concordium.Types import Control.Monad @@ -37,11 +38,17 @@ import Test.QuickCheck -- | Protocol version. type PV = 'P5 +newtype NoLoggerT m a = NoLoggerT {runNoLoggerT :: m a} + deriving (Functor, Applicative, Monad, MonadIO, MonadReader r) + +instance (Monad m) => MonadLogger (NoLoggerT m) where + logEvent _ _ _ = return () + type ThisMonadConcrete pv = PBS.PersistentBlockStateMonad pv (PBS.PersistentBlockStateContext pv) - (BlobStoreM' (PBS.PersistentBlockStateContext pv)) + (NoLoggerT (BlobStoreM' (PBS.PersistentBlockStateContext pv))) --------------------------------- Test values ---------------------------------- @@ -70,18 +77,19 @@ createGS = do acc0 <- makeTestAccountFromSeed 1_000_000 0 acc1 <- makeTestAccountFromSeed 1_000_000 1 initState <- - PBS.hpbsPointers - <$> PBS.initialPersistentState - (initialSeedStateV0 (Hash.hash "") 1_000) - dummyCryptographicParameters - [acc0, acc1] - dummyIdentityProviders - dummyArs - dummyKeyCollection - dummyChainParameters + PBS.initialPersistentState + (initialSeedStateV0 (Hash.hash "") 1_000) + dummyCryptographicParameters + [acc0, acc1] + dummyIdentityProviders + dummyArs + dummyKeyCollection + dummyChainParameters + -- save the block state so accounts are written to the lmdb database. + void $ saveBlockState initState addr0 <- BS.accountCanonicalAddress acc0 addr1 <- BS.accountCanonicalAddress acc1 - return (addr0, 0, addr1, 1, initState) + return (addr0, 0, addr1, 1, PBS.hpbsPointers initState) ------------------------------------- Test ------------------------------------- @@ -122,8 +130,9 @@ tests = do describe "GlobalState.AccountReleaseScheduleTest" $ specify "correct releases" $ runBlobStoreTemp "." $ - PBS.withNewAccountCache 1_000 $ - PBS.runPersistentBlockStateMonad testing + PBS.withNewAccountCacheAndLMDBAccountMap 1_000 "accountmap" $ + runNoLoggerT $ + PBS.runPersistentBlockStateMonad testing ------------------------------------ Checks ------------------------------------ diff --git a/concordium-consensus/tests/globalstate/GlobalStateTests/Accounts.hs b/concordium-consensus/tests/globalstate/GlobalStateTests/Accounts.hs index 9ba4ff3407..489773cf49 100644 --- a/concordium-consensus/tests/globalstate/GlobalStateTests/Accounts.hs +++ b/concordium-consensus/tests/globalstate/GlobalStateTests/Accounts.hs @@ -5,11 +5,15 @@ module GlobalStateTests.Accounts where +import qualified Basic.AccountTable as BAT +import qualified Basic.Accounts as B import Concordium.Crypto.DummyData import Concordium.Crypto.FFIDataTypes import qualified Concordium.Crypto.SHA256 as H import qualified Concordium.Crypto.SignatureScheme as Sig import qualified Concordium.GlobalState.AccountMap as AccountMap +import qualified Concordium.GlobalState.AccountMap.DifferenceMap as DiffMap +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap import Concordium.GlobalState.Basic.BlockState.Account as BA import Concordium.GlobalState.DummyData import qualified Concordium.GlobalState.Persistent.Account as PA @@ -17,18 +21,21 @@ import qualified Concordium.GlobalState.Persistent.Accounts as P import Concordium.GlobalState.Persistent.BlobStore import Concordium.GlobalState.Persistent.BlockState (PersistentBlockStateContext (..)) import qualified Concordium.GlobalState.Persistent.BlockState.Modules as M -import Concordium.GlobalState.Persistent.Cache (MonadCache) import qualified Concordium.GlobalState.Persistent.LFMBTree as L import Concordium.ID.DummyData import qualified Concordium.ID.Types as ID +import Concordium.Logger import Concordium.Types import Concordium.Types.HashableTo +import Concordium.Types.Option (Option (..)) import Control.Exception (bracket) -import Control.Monad hiding (fail) +import Control.Monad.Reader import Data.Either import qualified Data.FixedByteString as FBS +import qualified Data.HashMap.Strict as HM +import Data.IORef +import Data.List (sortOn) import qualified Data.Map.Strict as Map -import qualified Data.Map.Strict as OrdMap import Data.Serialize as S import qualified Data.Set as Set import Lens.Micro.Platform @@ -39,12 +46,13 @@ import Test.Hspec import Test.QuickCheck import Prelude hiding (fail) -import Control.Monad.IO.Class +type PV = 'P5 -import qualified Basic.AccountTable as BAT -import qualified Basic.Accounts as B +newtype NoLoggerT m a = NoLoggerT {runNoLoggerT :: m a} + deriving (Functor, Applicative, Monad, MonadIO, MonadReader r, MonadFail) -type PV = 'P5 +instance (Monad m) => MonadLogger (NoLoggerT m) where + logEvent _ _ _ = return () assertRight :: Either String a -> Assertion assertRight (Left e) = assertFailure e @@ -61,10 +69,10 @@ checkBinaryM bop x y sbop sx sy = do -- | Check that a 'B.Accounts' and a 'P.Accounts' are equivalent. -- That is, they have the same account map, account table, and set of -- use registration ids. -checkEquivalent :: (MonadBlobStore m, MonadFail m, MonadCache (PA.AccountCache (AccountVersionFor PV)) m) => B.Accounts PV -> P.Accounts PV -> m () +checkEquivalent :: (P.SupportsPersistentAccount PV m, av ~ AccountVersionFor PV) => B.Accounts PV -> P.Accounts PV -> m () checkEquivalent ba pa = do - pam <- AccountMap.toMap (P.accountMap pa) - checkBinary (==) (AccountMap.toMapPure (B.accountMap ba)) pam "==" "Basic account map" "Persistent account map" + addrsAndIndices <- P.allAccounts pa + checkBinary (==) (AccountMap.toMapPure (B.accountMap ba)) (Map.fromList addrsAndIndices) "==" "Basic account map" "Persistent account map" let bat = BAT.toList (B.accountTable ba) pat <- L.toAscPairList (P.accountTable pa) bpat <- mapM (_2 PA.toTransientAccount) pat @@ -80,15 +88,15 @@ data AccountAction | Exists AccountAddress | GetAccount AccountAddress | UpdateAccount AccountAddress (Account (AccountVersionFor PV) -> Account (AccountVersionFor PV)) - | UnsafeGetAccount AccountAddress | RegIdExists ID.CredentialRegistrationID | RecordRegId ID.CredentialRegistrationID AccountIndex | FlushPersistent | ArchivePersistent + | Reconstruct randomizeAccount :: AccountAddress -> ID.CredentialPublicKeys -> Gen (Account (AccountVersionFor PV)) randomizeAccount _accountAddress _accountVerificationKeys = do - let vfKey = snd . head $ OrdMap.toAscList (ID.credKeys _accountVerificationKeys) + let vfKey = snd . head $ Map.toAscList (ID.credKeys _accountVerificationKeys) let cred = dummyCredential dummyCryptographicParameters _accountAddress vfKey dummyMaxValidTo dummyCreatedAt let a0 = newAccount dummyCryptographicParameters _accountAddress cred nonce <- Nonce <$> arbitrary @@ -104,7 +112,7 @@ randomActions = sized (ra Set.empty Map.empty) randAccount = do address <- ID.AccountAddress . FBS.pack <$> vector ID.accountAddressSize n <- choose (1, 255) - credKeys <- OrdMap.fromList . zip [0 ..] . map Sig.correspondingVerifyKey <$> replicateM n genSigSchemeKeyPair + credKeys <- Map.fromList . zip [0 ..] . map Sig.correspondingVerifyKey <$> replicateM n genSigSchemeKeyPair credThreshold <- fromIntegral <$> choose (1, n) return (ID.CredentialPublicKeys{..}, address) ra _ _ 0 = return [] @@ -117,12 +125,13 @@ randomActions = sized (ra Set.empty Map.empty) (ArchivePersistent :) <$> ra s rids (n - 1), exRandReg, recRandReg, - updateRandAcc + updateRandAcc, + (Reconstruct :) <$> ra s rids (n - 1) ] ++ if null s then [] else - [exExAcc, getExAcc, unsafeGetExAcc, updateExAcc] + [exExAcc, getExAcc, updateExAcc] ++ if null rids then [] else [exExReg, recExReg] where fresh x @@ -160,9 +169,6 @@ randomActions = sized (ra Set.empty Map.empty) if (vk, addr) `Set.member` s then ra s rids n else (UpdateAccount addr upd :) <$> ra s rids (n - 1) - unsafeGetExAcc = do - (_, addr) <- elements (Set.toList s) - (UnsafeGetAccount addr :) <$> ra s rids (n - 1) exRandReg = do rid <- randomCredential (RegIdExists rid :) <$> ra s rids (n - 1) @@ -178,12 +184,13 @@ randomActions = sized (ra Set.empty Map.empty) (rid, ai) <- elements (Map.toList rids) (RecordRegId rid ai :) <$> ra s rids (n - 1) -runAccountAction :: (MonadBlobStore m, MonadIO m, MonadCache (PA.AccountCache (AccountVersionFor PV)) m) => AccountAction -> (B.Accounts PV, P.Accounts PV) -> m (B.Accounts PV, P.Accounts PV) +runAccountAction :: (P.SupportsPersistentAccount PV m, av ~ AccountVersionFor PV) => AccountAction -> (B.Accounts PV, P.Accounts PV) -> m (B.Accounts PV, P.Accounts PV) runAccountAction (PutAccount acct) (ba, pa) = do let ba' = B.putNewAccount acct ba pAcct <- PA.makePersistentAccount acct - pa' <- P.putNewAccount pAcct pa - return (snd ba', snd pa') + pa' <- P.mkNewChildDifferenceMap pa + pa'' <- P.putNewAccount pAcct pa' + return (snd ba', snd pa'') runAccountAction (Exists addr) (ba, pa) = do let be = B.exists addr ba pe <- P.exists addr pa @@ -204,17 +211,13 @@ runAccountAction (UpdateAccount addr upd) (ba, pa) = do PA.makePersistentAccount $ f bAcc (_, pa') <- P.updateAccounts (fmap ((),) . liftP upd) addr pa return (ba', pa') -runAccountAction (UnsafeGetAccount addr) (ba, pa) = do - let bacct = B.unsafeGetAccount addr ba - pacct <- P.unsafeGetAccount addr pa - bpacct <- PA.toTransientAccount pacct - checkBinary (==) bacct bpacct "==" "account in basic" "account in persistent" - return (ba, pa) runAccountAction FlushPersistent (ba, pa) = do (_, pa') <- storeUpdate pa + void $ P.writeAccountsCreated pa' return (ba, pa') runAccountAction ArchivePersistent (ba, pa) = do ppa <- fst <$> storeUpdate pa + void $ P.writeAccountsCreated pa pa' <- fromRight (error "couldn't deserialize archived persistent") $ S.runGet load (S.runPut ppa) return (ba, pa') runAccountAction (RegIdExists rid) (ba, pa) = do @@ -226,18 +229,40 @@ runAccountAction (RecordRegId rid ai) (ba, pa) = do let ba' = B.recordRegId (ID.toRawCredRegId rid) ai ba pa' <- P.recordRegId rid ai pa return (ba', pa') +runAccountAction Reconstruct (ba, pa) = do + oPaDiffMap <- liftIO $ readIORef $ P.accountDiffMapRef pa + -- Get the parent difference map reference and a list of accounts of the current difference map. + (parentDiffMapRef, diffMapAccs) <- case oPaDiffMap of + Absent -> do + ref <- liftIO DiffMap.newEmptyReference + return (ref, []) + Present paDiffMap -> do + let ref = DiffMap.dmParentMapRef paDiffMap + -- Note that we sort them by ascending account index such that the order + -- matches the insertion order. + accs = map snd $ sortOn fst $ HM.elems $ DiffMap.dmAccounts paDiffMap + return (ref, accs) + -- create pa' which is the same as pa, but with an empty difference map. + emptyRef <- liftIO DiffMap.newEmptyReference + let pa' = pa{P.accountDiffMapRef = emptyRef} + -- reconstruct pa into pa'. + void $ P.reconstructDifferenceMap parentDiffMapRef diffMapAccs pa' + return (ba, pa') emptyTest :: SpecWith (PersistentBlockStateContext PV) emptyTest = - it "empty" $ - runBlobStoreM - (checkEquivalent B.emptyAccounts P.emptyAccounts :: BlobStoreM' (PersistentBlockStateContext PV) ()) + it "empty" $ \bs -> + runNoLoggerT $ + flip runBlobStoreT bs $ do + emptyPersistentAccs <- P.emptyAccounts + (checkEquivalent B.emptyAccounts emptyPersistentAccs :: BlobStoreT (PersistentBlockStateContext PV) (NoLoggerT IO) ()) actionTest :: Word -> SpecWith (PersistentBlockStateContext PV) actionTest lvl = it "account actions" $ \bs -> withMaxSuccess (100 * fromIntegral lvl) $ property $ do acts <- randomActions - return $ ioProperty $ flip runBlobStoreM bs $ do - (ba, pa) <- foldM (flip runAccountAction) (B.emptyAccounts, P.emptyAccounts) acts + return $ ioProperty $ runNoLoggerT $ flip runBlobStoreT bs $ do + emptyPersistentAccs <- P.emptyAccounts + (ba, pa) <- foldM (flip runAccountAction) (B.emptyAccounts, emptyPersistentAccs) acts checkEquivalent ba pa tests :: Word -> Spec @@ -250,6 +275,7 @@ tests lvl = describe "GlobalStateTests.Accounts" pbscBlobStore <- createBlobStore (dir "blockstate.dat") pbscAccountCache <- PA.newAccountCache 100 pbscModuleCache <- M.newModuleCache 100 + pbscAccountMap <- LMDBAccountMap.openDatabase (dir "accountmap") return PersistentBlockStateContext{..} ) (closeBlobStore . pbscBlobStore) diff --git a/concordium-consensus/tests/globalstate/GlobalStateTests/DifferenceMap.hs b/concordium-consensus/tests/globalstate/GlobalStateTests/DifferenceMap.hs new file mode 100644 index 0000000000..a3f4c240e1 --- /dev/null +++ b/concordium-consensus/tests/globalstate/GlobalStateTests/DifferenceMap.hs @@ -0,0 +1,189 @@ +{-# LANGUAGE BangPatterns #-} + +-- | A module that tests the functionality of the 'DiffMap.DifferenceMap'. +-- * Insert and lookup operations. +-- * Flattening the 'DiffMap.DifferenceMap'. +module GlobalStateTests.DifferenceMap where + +import Concordium.ID.Types (accountAddressSize, randomAccountAddress) +import Concordium.Types +import Control.Monad +import Control.Monad.IO.Class +import qualified Data.FixedByteString as FBS +import qualified Data.HashMap.Strict as HM +import Data.IORef +import System.Random +import Test.HUnit +import Test.Hspec +import Test.QuickCheck + +import qualified Concordium.GlobalState.AccountMap.DifferenceMap as DiffMap +import Concordium.Types.Option + +-- | Create a pair consisting of an account address and an account index based on the provided seed. +dummyPair :: Int -> (AccountAddress, AccountIndex) +dummyPair seed = (fst $ randomAccountAddress (mkStdGen seed), AccountIndex $ fromIntegral seed) + +-- | Test for looking up both via equivalence class and by exactness. +-- Precondition: The provided @AccountAddress@ MUST be the canonical address, +-- and it should be present in the underlying store. +-- The equivalence lookup always looks up by an alias. +testDoLookup :: (MonadIO m) => AccountAddress -> DiffMap.DifferenceMap -> m (Either Int AccountIndex) +testDoLookup accAddr diffMap = do + res1 <- DiffMap.lookupViaEquivalenceClass (accountAddressEmbed $ createAlias accAddr 42) diffMap + res2 <- DiffMap.lookupExact accAddr diffMap + liftIO $ assertEqual "results should be the same" res1 res2 + return res1 + +-- | Test that an account can be inserted and looked up in the 'DiffMap.DifferenceMap'. +testInsertLookupAccount :: Assertion +testInsertLookupAccount = do + emptyParentMap <- liftIO DiffMap.newEmptyReference + let diffMap = uncurry DiffMap.insert acc $ DiffMap.empty emptyParentMap + testDoLookup (fst acc) diffMap >>= \case + Left _ -> assertFailure "account should be present in diff map" + Right accIdx -> assertEqual "account should be there" (snd acc) accIdx + where + acc = dummyPair 1 + +-- | Create a parent pointer for the provided difference map. +mkParentPointer :: Option DiffMap.DifferenceMap -> IO (IORef (Option DiffMap.DifferenceMap)) +mkParentPointer diffMap = newIORef diffMap >>= return + +-- | Testing lookups in flat and nested difference maps. +testLookups :: Assertion +testLookups = do + emptyParentMap <- liftIO DiffMap.newEmptyReference + let diffMap1 = uncurry DiffMap.insert (dummyPair 1) $ DiffMap.empty emptyParentMap + diffMap1Pointer <- mkParentPointer $ Present diffMap1 + let diffMap2 = uncurry DiffMap.insert (dummyPair 2) (DiffMap.empty diffMap1Pointer) + diffMap2Pointer <- mkParentPointer $ Present diffMap2 + let diffMap3 = uncurry DiffMap.insert (dummyPair 3) (DiffMap.empty diffMap2Pointer) + checkExists (dummyPair 1) diffMap1 + checkExists (dummyPair 1) diffMap2 + checkExists (dummyPair 2) diffMap2 + checkExists (dummyPair 1) diffMap3 + checkExists (dummyPair 2) diffMap3 + checkExists (dummyPair 3) diffMap3 + where + checkExists pair diffMap = + testDoLookup (fst pair) diffMap >>= \case + Left _ -> assertFailure "account should be present" + Right accIdx -> assertEqual "wrong account index" (snd pair) accIdx + +-- | Test flattening a difference map i.e. return all accounts as one flat map. +testFlatten :: Assertion +testFlatten = do + emptyParentMap <- liftIO DiffMap.newEmptyReference + let diffMap1 = uncurry DiffMap.insert (dummyPair 1) $ DiffMap.empty emptyParentMap + diffMap1Pointer <- mkParentPointer $ Present diffMap1 + let diffMap2 = uncurry DiffMap.insert (dummyPair 2) (DiffMap.empty diffMap1Pointer) + diffMap2Pointer <- mkParentPointer $ Present diffMap2 + let diffMap3 = uncurry DiffMap.insert (dummyPair 3) (DiffMap.empty diffMap2Pointer) + assertEqual "accounts should be the same" (map dummyPair [1 .. 3]) =<< DiffMap.flatten diffMap3 + +-- | Make the reference map for comparing lookups. +makeReference :: [(AccountAddress, AccountIndex)] -> HM.HashMap AccountAddress AccountIndex +makeReference = HM.fromList + +-- | Generate an 'AccountAddress' +genAccountAddress :: Gen AccountAddress +genAccountAddress = AccountAddress . FBS.pack <$> vector accountAddressSize + +-- | Generate account addresses, account indices and depth of the difference map. +genInputs :: Gen ([(AccountAddress, AccountIndex)], Int) +genInputs = sized $ \n -> do + let maxAccs = min n 10000 + len <- choose (0, maxAccs) + accs <- replicateM len ((,) <$> genAccountAddress <*> (AccountIndex <$> arbitrary)) + noDifferenceMaps <- choose (0, len) + return (accs, noDifferenceMaps) + +genInputs2 :: Gen [(AccountAddress, AccountIndex)] +genInputs2 = sized $ \n -> do + let maxAccs = min n 10000 + len <- choose (0, maxAccs) + replicateM len ((,) <$> genAccountAddress <*> (AccountIndex <$> arbitrary)) + +-- | Test insertions and lookups on the difference map. +insertionsAndLookups :: Spec +insertionsAndLookups = it "insertions and lookups" $ + withMaxSuccess 10000 $ + forAll genInputs $ \(inputs, noDifferenceMaps) -> do + let reference = HM.fromList inputs + emptyRef <- liftIO DiffMap.newEmptyReference + diffMap <- populateDiffMap inputs noDifferenceMaps $ DiffMap.empty emptyRef + checkAll reference diffMap + let nonExistantAcc = fst (dummyPair (-1)) + testDoLookup nonExistantAcc diffMap >>= \case + Right _ -> liftIO $ assertFailure "account should not be present" + Left size -> do + expectedSize <- length <$> DiffMap.flatten diffMap + liftIO $ assertEqual "Sizes should match" expectedSize size + where + checkAll ref diffMap = forM_ (HM.toList ref) (check diffMap) + check diffMap (accAddr, accIdx) = do + testDoLookup accAddr diffMap >>= \case + Left _ -> liftIO $ assertFailure "account address should be present" + Right actualAccIdx -> liftIO $ assertEqual "account index should be equal" accIdx actualAccIdx + -- return the generated difference map(s) + populateDiffMap [] _ !accum = return accum + -- dump any remaining accounts at the top most difference map. + populateDiffMap ((accAddr, accIdx) : rest) 0 !accum = populateDiffMap rest 0 $ DiffMap.insert accAddr accIdx accum + -- create a new layer and insert an account. + populateDiffMap ((accAddr, accIdx) : rest) remaining !accum = do + pRef <- mkParentPointer (Present accum) + let accumDiffMap'' = DiffMap.insert accAddr accIdx $ DiffMap.empty pRef + populateDiffMap rest (remaining - 1) accumDiffMap'' + +-- | A test that makes sure if multiple difference maps are +-- derivied via a common parent, then additions in one branch +-- are not propagating to other branches. +testMultipleChildrenDifferenceMaps :: Assertion +testMultipleChildrenDifferenceMaps = do + emptyRoot <- liftIO DiffMap.newEmptyReference + -- The common parent + let parent = uncurry DiffMap.insert (dummyPair 1) $ DiffMap.empty emptyRoot + parentReference <- mkParentPointer $ Present parent + -- First branch + let branch0 = uncurry DiffMap.insert (dummyPair 2) $ DiffMap.empty parentReference + -- Second branch + let branch1 = uncurry DiffMap.insert (dummyPair 3) $ DiffMap.empty parentReference + + -- Account from common parent should exist in both branches. + checkExists (fst $ dummyPair 1) (snd $ dummyPair 1) branch0 + checkExists (fst $ dummyPair 1) (snd $ dummyPair 1) branch1 + -- Check that we cannot lookup elements from a different branch. + checkNotExists (fst $ dummyPair 2) branch1 + checkNotExists (fst $ dummyPair 3) branch0 + where + checkExists addr expectedAccIdx diffMap = + testDoLookup addr diffMap >>= \case + Right accIdx -> liftIO $ assertEqual "Account index should match" expectedAccIdx accIdx + Left _ -> liftIO $ assertFailure "Expected an entry" + checkNotExists addr diffMap = + testDoLookup addr diffMap >>= \case + Right _ -> liftIO $ assertFailure "Did not expect an entry" + Left size -> do + expectedSize <- length <$> DiffMap.flatten diffMap + liftIO $ assertEqual "Size reported back should match flattened size" expectedSize size + +-- | Test the 'fromList' function. +testFromList :: Assertion +testFromList = do + emptyRoot <- liftIO DiffMap.newEmptyReference + -- check creating from empty list + let emptyDiffMap = DiffMap.empty emptyRoot + liftIO $ assertBool "fromList on empty list should yield the empty difference map" (emptyDiffMap == DiffMap.fromList emptyRoot []) + -- check for a difference map with 1 element. + let nonEmptyDiffMap = uncurry DiffMap.insert (dummyPair 1) $ DiffMap.empty emptyRoot + liftIO $ assertBool "fromList on empty list should yield the empty difference map" (nonEmptyDiffMap == DiffMap.fromList emptyRoot [dummyPair 1]) + +tests :: Spec +tests = describe "AccountMap.DifferenceMap" $ do + it "Test insert and lookup account" testInsertLookupAccount + it "test lookups" testLookups + it "Test flatten" testFlatten + it "test lookups on branches" testMultipleChildrenDifferenceMaps + it "test fromList" testFromList + insertionsAndLookups diff --git a/concordium-consensus/tests/globalstate/GlobalStateTests/LMDBAccountMap.hs b/concordium-consensus/tests/globalstate/GlobalStateTests/LMDBAccountMap.hs new file mode 100644 index 0000000000..697670a624 --- /dev/null +++ b/concordium-consensus/tests/globalstate/GlobalStateTests/LMDBAccountMap.hs @@ -0,0 +1,117 @@ +{-# LANGUAGE OverloadedStrings #-} + +-- Tests of the LMDB account map related operations. + +-- | Tests for the LMDB account map +-- This module tests the following: +-- * Accounts can be inserted. +-- * Accounts can be looked up. +-- * Accounts can be rolled back. +module GlobalStateTests.LMDBAccountMap where + +import Control.Exception (bracket) +import Control.Monad.Reader +import qualified Data.HashMap.Strict as HM +import Data.Maybe (isJust, isNothing) +import System.IO.Temp +import System.Random +import Test.HUnit +import Test.Hspec + +import Concordium.ID.Types (randomAccountAddress) +import Concordium.Logger +import Concordium.Types + +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap + +-- | Create a pair consisting of an account address and an account index based on the provided seed. +dummyPair :: Int -> (AccountAddress, AccountIndex) +dummyPair seed = (fst $ randomAccountAddress (mkStdGen seed), AccountIndex $ fromIntegral seed) + +-- | Helper function for running a test in a context which has access to a temporary lmdb store. +runTest :: + String -> + LMDBAccountMap.AccountMapStoreMonad (ReaderT LMDBAccountMap.DatabaseHandlers LogIO) a -> + IO a +runTest dirName action = withTempDirectory "" dirName $ \path -> + bracket + (LMDBAccountMap.makeDatabaseHandlers path False :: IO LMDBAccountMap.DatabaseHandlers) + LMDBAccountMap.closeDatabase + (\dbh -> runSilentLogger $ runReaderT (LMDBAccountMap.runAccountMapStoreMonad action) dbh) + +-- | Test for looking up both via equivalence class and by exactness. +-- Precondition: The provided @AccountAddress@ MUST be the canonical address, +-- and it should be present in the underlying store. +-- The equivalence lookup always goes through an alias. +testDoLookup :: (MonadIO m, LMDBAccountMap.MonadAccountMapStore m) => AccountAddress -> m (Maybe AccountIndex) +testDoLookup accAddr = do + res1 <- LMDBAccountMap.lookupAccountIndexViaExactness accAddr + res2 <- LMDBAccountMap.lookupAccountIndexViaEquivalence (accountAddressEmbed $ createAlias accAddr 42) + liftIO $ assertEqual "Results should be the same" res1 res2 + return res1 + +-- | Test that a database is initialized. +testCheckDBNoOfAccounts :: Assertion +testCheckDBNoOfAccounts = runTest "initialized" $ do + -- initialize the database + void $ LMDBAccountMap.insertAccounts [dummyPair 1] + noOfAccounts <- LMDBAccountMap.getNumberOfAccounts + liftIO $ assertEqual "database should have been initialized" 1 noOfAccounts + +-- | Test that inserts a set of accounts and afterwards asserts that they are present. +testInsertAndLookupAccounts :: Assertion +testInsertAndLookupAccounts = runTest "insertandlookups" $ do + let accounts = dummyPair <$> [1 .. 42] + void $ LMDBAccountMap.insertAccounts accounts + + forM_ accounts $ \(accAddr, accIndex) -> do + testDoLookup accAddr >>= \case + Nothing -> liftIO $ assertFailure $ "account was not present " <> show accAddr <> " account index " <> show accIndex + Just foundAccountIndex -> liftIO $ assertEqual "account indices should be the same" accIndex foundAccountIndex + +-- | Test for looking up an account via an alias +testLookupAccountViaAlias :: Assertion +testLookupAccountViaAlias = runTest "lookupviaalias" $ do + -- initialize the database + void $ LMDBAccountMap.insertAccounts [acc] + let alias = createAlias (fst acc) 42 + exactLookup <- isNothing <$> LMDBAccountMap.lookupAccountIndexViaExactness alias + liftIO $ assertBool "Alias lookup should've failed" exactLookup + LMDBAccountMap.lookupAccountIndexViaEquivalence (accountAddressEmbed alias) >>= \case + Nothing -> liftIO $ assertFailure "account could not be looked up via alias" + Just accIndex -> liftIO $ assertEqual "account indices should match" (snd acc) accIndex + where + acc = dummyPair 1 + +-- | Test for retrieving all accounts present in the LMDB store. +testGetAllAccounts :: Assertion +testGetAllAccounts = runTest "allaccounts" $ do + -- initialize the database + void $ LMDBAccountMap.insertAccounts $ dummyPair <$> [0 .. 42] + void $ LMDBAccountMap.insertAccounts $ dummyPair <$> [42 .. 84] + allAccounts <- LMDBAccountMap.getAllAccounts (AccountIndex 85) + when (length allAccounts /= 85) $ + liftIO $ + assertFailure $ + "unexpected number of accounts: " <> (show . length) allAccounts <> " should be " <> show (85 :: Int) + forM_ (dummyPair <$> [0 .. 84]) $ \(accAddr, _) -> do + isPresent <- isJust <$> testDoLookup accAddr + liftIO $ assertBool "account should be present" isPresent + +-- | Test reconstructing a lmdb evicts all former values and puts in the new ones. +testReconstruction :: Assertion +testReconstruction = runTest "reconstruction" $ do + void $ LMDBAccountMap.insertAccounts $ dummyPair <$> [0 .. 42] + accs0 <- LMDBAccountMap.getAllAccounts (AccountIndex 43) + liftIO $ assertEqual "accounts before reconstruction should be equal" (HM.fromList (dummyPair <$> [0 .. 42])) (HM.fromList accs0) + void $ LMDBAccountMap.reconstruct $ dummyPair <$> [100 .. 142] + accs1 <- LMDBAccountMap.getAllAccounts (AccountIndex 143) + liftIO $ assertEqual "accounts after reconstruction should match" (HM.fromList (dummyPair <$> [100 .. 142])) (HM.fromList accs1) + +tests :: Spec +tests = describe "AccountMap.LMDB" $ do + it "Test checking db is initialized" testCheckDBNoOfAccounts + it "Test inserts and lookups" testInsertAndLookupAccounts + it "Test getting all accounts" testGetAllAccounts + it "Test looking up account via alias" testLookupAccountViaAlias + it "Test reconstruction" testReconstruction diff --git a/concordium-consensus/tests/globalstate/GlobalStateTests/PersistentTreeState.hs b/concordium-consensus/tests/globalstate/GlobalStateTests/PersistentTreeState.hs index 1656bd8ad7..b3e7f8a714 100644 --- a/concordium-consensus/tests/globalstate/GlobalStateTests/PersistentTreeState.hs +++ b/concordium-consensus/tests/globalstate/GlobalStateTests/PersistentTreeState.hs @@ -76,7 +76,7 @@ createGlobalState dbDir = do let n = 3 genesis = makeTestingGenesisDataP5 now n 1 1 dummyFinalizationCommitteeMaxSize dummyCryptographicParameters emptyIdentityProviders emptyAnonymityRevokers maxBound dummyKeyCollection dummyChainParameters - config = GlobalStateConfig defaultRuntimeParameters dbDir (dbDir "blockstate" <.> "dat") + config = GlobalStateConfig defaultRuntimeParameters dbDir (dbDir "blockstate" <.> "dat") (dbDir "accountmap") (x, y) <- runSilentLogger $ initialiseGlobalState genesis config return (x, y) diff --git a/concordium-consensus/tests/globalstate/GlobalStateTests/Trie.hs b/concordium-consensus/tests/globalstate/GlobalStateTests/Trie.hs index e21fb2be77..b0535210ea 100644 --- a/concordium-consensus/tests/globalstate/GlobalStateTests/Trie.hs +++ b/concordium-consensus/tests/globalstate/GlobalStateTests/Trie.hs @@ -4,7 +4,6 @@ module GlobalStateTests.Trie where -import Control.Monad.IO.Class import Data.Serialize import Data.Word @@ -50,8 +49,8 @@ testUpdateBranchNull = forAll genBranchList $ \v i -> tests :: Spec tests = describe "GlobalStateTests.Trie" $ do - it "simple test" $ - runBlobStoreTemp "." $ do + it "simple test" $ do + r <- runBlobStoreTemp "." $ do let e = Trie.empty :: Trie.TrieN BufferedFix Word64 (SerializeStorable String) e0 <- Trie.insert 27 (SerStore "Hello") e e1 <- Trie.insert 13 (SerStore "World") e0 @@ -64,8 +63,9 @@ tests = describe "GlobalStateTests.Trie" $ do -- the bytestring it was serialized to. Left _ -> error "loadRes should be Right" (e2' :: Trie.TrieN BufferedFix Word64 (SerializeStorable String)) <- me2' - r <- Trie.lookup 27 e2' - liftIO $ r `shouldBe` Just (SerStore "Hello") + res <- Trie.lookup 27 e2' + return res + r `shouldBe` Just (SerStore "Hello") it "branchesFromToList" $ withMaxSuccess 10000 testBranchesFromToList it "branchAtFromList" $ withMaxSuccess 10000 testBranchAtFromList it "updateBranchSome" $ withMaxSuccess 10000 testUpdateBranchSome diff --git a/concordium-consensus/tests/globalstate/GlobalStateTests/Updates.hs b/concordium-consensus/tests/globalstate/GlobalStateTests/Updates.hs index 1f45b7e961..3749a68876 100644 --- a/concordium-consensus/tests/globalstate/GlobalStateTests/Updates.hs +++ b/concordium-consensus/tests/globalstate/GlobalStateTests/Updates.hs @@ -42,6 +42,7 @@ import Concordium.GlobalState.BakerInfo import qualified Concordium.GlobalState.Persistent.BlockState.Updates as PU import Concordium.ID.DummyData import Concordium.ID.Parameters +import Concordium.Logger import Concordium.Types.DummyData import Concordium.Types.SeedState (initialSeedStateV0) import Test.HUnit (assertEqual) @@ -52,13 +53,20 @@ import Test.HUnit (assertEqual) -- -- -------------------------------------------------------------------------------- +newtype NoLoggerT m a = NoLoggerT {runNoLoggerT :: m a} + deriving (Functor, Applicative, Monad, MonadIO, MonadReader r, MonadFail) + +instance (Monad m) => MonadLogger (NoLoggerT m) where + logEvent _ _ _ = return () + type PV = 'P5 type ThisMonadConcrete pv = - PBS.PersistentBlockStateMonad + ( PBS.PersistentBlockStateMonad pv (PBS.PersistentBlockStateContext pv) - (BlobStoreM' (PBS.PersistentBlockStateContext pv)) + (NoLoggerT (BlobStoreM' (PBS.PersistentBlockStateContext pv))) + ) -------------------------------------------------------------------------------- -- -- @@ -333,8 +341,9 @@ tests = do wtdgs s t = specify s $ runBlobStoreTemp "." $ - PBS.withNewAccountCache 1_000 $ - PBS.runPersistentBlockStateMonad t + PBS.withNewAccountCacheAndLMDBAccountMap 1_000 "accountmap" $ + runNoLoggerT $ + PBS.runPersistentBlockStateMonad t describe "GlobalState.Updates - BakerStakeThreshold" $ do wtdgs "not enough stake - must fail" testing1 wtdgs "enough stake >decrease> not enough - must fail" testing2'1 diff --git a/concordium-consensus/tests/globalstate/Spec.hs b/concordium-consensus/tests/globalstate/Spec.hs index 99d997f2ad..9b83df6211 100644 --- a/concordium-consensus/tests/globalstate/Spec.hs +++ b/concordium-consensus/tests/globalstate/Spec.hs @@ -9,10 +9,12 @@ import qualified GlobalStateTests.Accounts (tests) import qualified GlobalStateTests.BlobStore (tests) import qualified GlobalStateTests.BlockHash (tests) import qualified GlobalStateTests.Cache (tests) +import qualified GlobalStateTests.DifferenceMap (tests) import qualified GlobalStateTests.EnduringDataFlags (tests) import qualified GlobalStateTests.FinalizationSerializationSpec (tests) import qualified GlobalStateTests.Instances (tests) import qualified GlobalStateTests.LFMBTree (tests) +import qualified GlobalStateTests.LMDBAccountMap (tests) import qualified GlobalStateTests.PersistentTreeState (tests) import qualified GlobalStateTests.Trie (tests) import qualified GlobalStateTests.UpdateQueues (tests) @@ -47,3 +49,5 @@ main = atLevel $ \lvl -> hspec $ do GlobalStateTests.EnduringDataFlags.tests GlobalStateTests.BlobStore.tests GlobalStateTests.UpdateQueues.tests + GlobalStateTests.LMDBAccountMap.tests + GlobalStateTests.DifferenceMap.tests diff --git a/concordium-consensus/tests/scheduler/SchedulerTests/Helpers.hs b/concordium-consensus/tests/scheduler/SchedulerTests/Helpers.hs index 5ca5d2fc28..8314414815 100644 --- a/concordium-consensus/tests/scheduler/SchedulerTests/Helpers.hs +++ b/concordium-consensus/tests/scheduler/SchedulerTests/Helpers.hs @@ -1,4 +1,5 @@ {-# LANGUAGE DataKinds #-} +{-# LANGUAGE DerivingVia #-} {-# LANGUAGE GADTs #-} {-# LANGUAGE NumericUnderscores #-} {-# LANGUAGE OverloadedStrings #-} @@ -38,6 +39,7 @@ import qualified Concordium.Wasm as Wasm import qualified Concordium.Common.Time as Time import qualified Concordium.Cost as Cost +import qualified Concordium.GlobalState.AccountMap.LMDB as LMDBAccountMap import qualified Concordium.GlobalState.BlockState as BS import qualified Concordium.GlobalState.DummyData as DummyData import qualified Concordium.GlobalState.Persistent.Account as BS @@ -84,19 +86,15 @@ newtype PersistentBSM pv a = PersistentBSM BS.PersistentBlockStateMonad pv (BS.PersistentBlockStateContext pv) - (Blob.BlobStoreM' (BS.PersistentBlockStateContext pv)) + (Blob.BlobStoreT (BS.PersistentBlockStateContext pv) LogIO) a } deriving ( Applicative, Functor, Monad, - BS.ContractStateOperations, - BS.ModuleQuery, BlockStateTypes, - Blob.MonadBlobStore, - MonadIO, - MonadCache BS.ModuleCache + MonadIO ) deriving instance (Types.IsProtocolVersion pv) => BS.AccountOperations (PersistentBSM pv) @@ -104,6 +102,11 @@ deriving instance (Types.IsProtocolVersion pv) => BS.BlockStateOperations (Persi deriving instance (Types.IsProtocolVersion pv) => BS.BlockStateQuery (PersistentBSM pv) deriving instance (Types.IsProtocolVersion pv) => Types.MonadProtocolVersion (PersistentBSM pv) deriving instance (Types.IsProtocolVersion pv) => BS.BlockStateStorage (PersistentBSM pv) +deriving instance (Types.IsProtocolVersion pv) => BS.ModuleQuery (PersistentBSM pv) +deriving instance (Types.IsProtocolVersion pv) => BS.ContractStateOperations (PersistentBSM pv) +deriving instance (Types.IsProtocolVersion pv) => Blob.MonadBlobStore (PersistentBSM pv) +deriving instance (Types.IsProtocolVersion pv) => MonadCache BS.ModuleCache (PersistentBSM pv) +deriving instance (Types.IsProtocolVersion pv) => LMDBAccountMap.MonadAccountMapStore (PersistentBSM pv) deriving instance (Types.AccountVersionFor pv ~ av) => @@ -137,15 +140,20 @@ createTestBlockStateWithAccounts :: (Types.IsProtocolVersion pv) => [BS.PersistentAccount (Types.AccountVersionFor pv)] -> PersistentBSM pv (BS.HashedPersistentBlockState pv) -createTestBlockStateWithAccounts accounts = - BS.initialPersistentState - seedState - DummyData.dummyCryptographicParameters - accounts - DummyData.dummyIdentityProviders - DummyData.dummyArs - keys - DummyData.dummyChainParameters +createTestBlockStateWithAccounts accounts = do + bs <- + BS.initialPersistentState + seedState + DummyData.dummyCryptographicParameters + accounts + DummyData.dummyIdentityProviders + DummyData.dummyArs + keys + DummyData.dummyChainParameters + -- save block state and accounts. + void $ BS.saveBlockState bs + void $ BS.saveAccounts bs + return bs where keys = Types.withIsAuthorizationsVersionForPV (Types.protocolVersion @pv) $ DummyData.dummyKeyCollection seedState = case Types.consensusVersionFor (Types.protocolVersion @pv) of @@ -163,19 +171,20 @@ createTestBlockStateWithAccountsM accounts = -- | Run test block state computation provided an account cache size. -- The module cache size is 100. -- --- This function creates a temporary file for the blobstore, which is removed right after the +-- This function creates temporary files for the blobstore and account map. These are removed right after the -- running the computation, meaning the result of the computation should not retain any references -- and should be fully evaluated. runTestBlockStateWithCacheSize :: Int -> PersistentBSM pv a -> IO a runTestBlockStateWithCacheSize cacheSize computation = - Blob.runBlobStoreTemp "." $ - BS.withNewAccountCache cacheSize $ - BS.runPersistentBlockStateMonad $ - _runPersistentBSM computation + runSilentLogger $ + Blob.runBlobStoreTemp "." $ + BS.withNewAccountCacheAndLMDBAccountMap cacheSize "accountmap" $ + BS.runPersistentBlockStateMonad $ + _runPersistentBSM computation -- | Run test block state computation with a account cache size and module cache size of 100. -- --- This function creates a temporary file for the blobstore, which is removed right after the +-- This function creates a temporary files for the blobstore and account map. These are removed right after the -- running the computation, meaning the result of the computation should not retain any references -- and should be fully evaluated. runTestBlockState :: PersistentBSM pv a -> IO a @@ -376,6 +385,7 @@ reloadBlockState :: reloadBlockState persistentState = do frozen <- BS.freezeBlockState persistentState br <- BS.saveBlockState frozen + void $ BS.saveAccounts frozen BS.thawBlockState =<< BS.loadBlockState ((Just . BS.hpbsHash) frozen) br -- | Takes a function for checking the block state, which is then run on the block state, the block diff --git a/concordium-consensus/tests/scheduler/SchedulerTests/Payday.hs b/concordium-consensus/tests/scheduler/SchedulerTests/Payday.hs index bf130b1050..e2062e786f 100644 --- a/concordium-consensus/tests/scheduler/SchedulerTests/Payday.hs +++ b/concordium-consensus/tests/scheduler/SchedulerTests/Payday.hs @@ -338,7 +338,7 @@ createGlobalState :: (IsProtocolVersion pv, IsConsensusV0 pv) => FilePath -> IO createGlobalState dbDir = do let n = 5 - config = GlobalStateConfig defaultRuntimeParameters dbDir (dbDir "blockstate" <.> "dat") + config = GlobalStateConfig defaultRuntimeParameters dbDir (dbDir "blockstate" <.> "dat") (dbDir "accountmap") (x, y) <- runSilentLogger $ initialiseGlobalState (genesis n ^. _1) config return (x, y)