diff --git a/cardano-db-sync/cardano-db-sync.cabal b/cardano-db-sync/cardano-db-sync.cabal index 0cf1c0e49..6c26e234b 100644 --- a/cardano-db-sync/cardano-db-sync.cabal +++ b/cardano-db-sync/cardano-db-sync.cabal @@ -45,6 +45,7 @@ library Cardano.DbSync.Api Cardano.DbSync.Api.Ledger Cardano.DbSync.Api.Types + Cardano.DbSync.Ledger.Async Cardano.DbSync.Config Cardano.DbSync.Config.Alonzo Cardano.DbSync.Config.Byron @@ -53,7 +54,6 @@ library Cardano.DbSync.Config.Node Cardano.DbSync.Config.Shelley Cardano.DbSync.Config.Types - Cardano.DbSync.Database Cardano.DbSync.DbAction Cardano.DbSync.Error @@ -109,12 +109,14 @@ library Cardano.DbSync.Metrics + Cardano.DbSync.Block Cardano.DbSync.Cache Cardano.DbSync.Cache.Epoch Cardano.DbSync.Cache.FIFO Cardano.DbSync.Cache.LRU + Cardano.DbSync.Cache.Stake Cardano.DbSync.Cache.Types - Cardano.DbSync.Default + Cardano.DbSync.Cache.Util Cardano.DbSync.Epoch Cardano.DbSync.Rollback @@ -130,6 +132,12 @@ library Cardano.DbSync.LocalStateQuery Cardano.DbSync.StateQuery Cardano.DbSync.Sync + Cardano.DbSync.Threads.Database + Cardano.DbSync.Threads.EpochStake + Cardano.DbSync.Threads.Ledger + Cardano.DbSync.Threads.Rewards + Cardano.DbSync.Threads.Stake + Cardano.DbSync.Threads.TxInResolve Cardano.DbSync.Tracing.ToObjectOrphans Cardano.DbSync.Types @@ -209,6 +217,7 @@ library , prometheus , psqueues , random-shuffle + , resource-pool , scientific , serialise , small-steps diff --git a/cardano-db-sync/src/Cardano/DbSync.hs b/cardano-db-sync/src/Cardano/DbSync.hs index 32fe21b1b..bd02cdac1 100644 --- a/cardano-db-sync/src/Cardano/DbSync.hs +++ b/cardano-db-sync/src/Cardano/DbSync.hs @@ -33,7 +33,6 @@ import Cardano.DbSync.Api.Types (InsertOptions (..), RunMigration, SyncEnv (..), import Cardano.DbSync.Config (configureLogging) import Cardano.DbSync.Config.Cardano import Cardano.DbSync.Config.Types -import Cardano.DbSync.Database import Cardano.DbSync.DbAction import Cardano.DbSync.Era import Cardano.DbSync.Error @@ -41,6 +40,10 @@ import Cardano.DbSync.Ledger.State import Cardano.DbSync.OffChain (runFetchOffChainPoolThread, runFetchOffChainVoteThread) import Cardano.DbSync.Rollback (unsafeRollback) import Cardano.DbSync.Sync (runSyncNodeClient) +import Cardano.DbSync.Threads.Database +import Cardano.DbSync.Threads.EpochStake +import Cardano.DbSync.Threads.Ledger +import Cardano.DbSync.Threads.Stake import Cardano.DbSync.Tracing.ToObjectOrphans () import Cardano.DbSync.Types import Cardano.DbSync.Util.Constraint (queryIsJsonbInSchema) @@ -163,8 +166,6 @@ runSyncNode metricsSetters trce iomgr dbConnString runMigrationFnc syncNodeConfi logInfo trce $ "Using shelley genesis file from: " <> (show . unGenesisFile $ dncShelleyGenesisFile syncNodeConfigFromFile) logInfo trce $ "Using alonzo genesis file from: " <> (show . unGenesisFile $ dncAlonzoGenesisFile syncNodeConfigFromFile) - let useLedger = shouldUseLedger (sioLedger $ dncInsertOptions syncNodeConfigFromFile) - Db.runIohkLogging trce $ withPostgresqlConn dbConnString $ \backend -> liftIO $ do @@ -206,6 +207,9 @@ runSyncNode metricsSetters trce iomgr dbConnString runMigrationFnc syncNodeConfi id [ runDbThread syncEnv metricsSetters threadChannels , runSyncNodeClient metricsSetters syncEnv iomgr trce threadChannels (enpSocketPath syncNodeParams) + , runLedgerThread syncEnv + , runEpochStakeThread syncEnv + , runStakeThread syncEnv , runFetchOffChainPoolThread syncEnv , runFetchOffChainVoteThread syncEnv , runLedgerStateWriteThread (getTrace syncEnv) (envLedgerEnv syncEnv) @@ -219,6 +223,8 @@ runSyncNode metricsSetters trce iomgr dbConnString runMigrationFnc syncNodeConfi removeJsonbFromSchemaConfig = ioRemoveJsonbFromSchema $ soptInsertOptions syncOptions maybeLedgerDir = enpMaybeLedgerStateDir syncNodeParams + useLedger = shouldUseLedger (sioLedger $ dncInsertOptions syncNodeConfigFromFile) + logProtocolMagicId :: Trace IO Text -> Crypto.ProtocolMagicId -> ExceptT SyncNodeError IO () logProtocolMagicId tracer pm = liftIO diff --git a/cardano-db-sync/src/Cardano/DbSync/Api.hs b/cardano-db-sync/src/Cardano/DbSync/Api.hs index a24f1baae..a4cacf93c 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Api.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Api.hs @@ -3,8 +3,6 @@ {-# LANGUAGE GADTs #-} {-# LANGUAGE OverloadedStrings #-} {-# LANGUAGE ScopedTypeVariables #-} -{-# LANGUAGE TupleSections #-} -{-# LANGUAGE TypeApplications #-} {-# LANGUAGE NoImplicitPrelude #-} module Cardano.DbSync.Api ( @@ -12,7 +10,6 @@ module Cardano.DbSync.Api ( setConsistentLevel, getConsistentLevel, isConsistent, - getIsConsumedFixed, getDisableInOutState, getRanIndexes, runIndexMigrations, @@ -30,20 +27,13 @@ module Cardano.DbSync.Api ( getSkipTxIn, getPrunes, mkSyncEnvFromConfig, - verifySnapshotPoint, getInsertOptions, getTrace, getTopLevelConfig, getNetwork, hasLedgerState, - getLatestPoints, - getSlotHash, - getDbLatestBlockInfo, - getDbTipBlockNo, - getCurrentTipBlockNo, - generateNewEpochEvents, - logDbState, - convertToPoint, + writePrefetch, + addNewEventsAndSort, ) where import Cardano.BM.Trace (Trace, logInfo, logWarning) @@ -51,19 +41,14 @@ import qualified Cardano.Chain.Genesis as Byron import Cardano.Crypto.ProtocolMagic (ProtocolMagicId (..)) import qualified Cardano.Db as DB import Cardano.DbSync.Api.Types -import Cardano.DbSync.Cache.Types (CacheCapacity (..), newEmptyCache, useNoCache) +import Cardano.DbSync.Cache.Types (CacheCapacity (..), newEmptyCache, newStakeChannels, useNoCache) import Cardano.DbSync.Config.Cardano import Cardano.DbSync.Config.Shelley import Cardano.DbSync.Config.Types import Cardano.DbSync.Error import Cardano.DbSync.Ledger.Event (LedgerEvent (..)) -import Cardano.DbSync.Ledger.State ( - getHeaderHash, - hashToAnnotation, - listKnownSnapshots, - mkHasLedgerEnv, - ) -import Cardano.DbSync.Ledger.Types (HasLedgerEnv (..), LedgerStateFile (..), SnapshotPoint (..)) +import Cardano.DbSync.Ledger.State (mkHasLedgerEnv) +import Cardano.DbSync.Ledger.Types import Cardano.DbSync.LocalStateQuery import Cardano.DbSync.Types import Cardano.DbSync.Util @@ -71,7 +56,7 @@ import Cardano.DbSync.Util.Constraint (dbConstraintNamesExists) import qualified Cardano.Ledger.BaseTypes as Ledger import qualified Cardano.Ledger.Shelley.Genesis as Shelley import Cardano.Prelude -import Cardano.Slotting.Slot (EpochNo (..), SlotNo (..), WithOrigin (..)) +import Cardano.Slotting.Slot (EpochNo (..)) import Control.Concurrent.Class.MonadSTM.Strict ( newTBQueueIO, newTVarIO, @@ -79,20 +64,18 @@ import Control.Concurrent.Class.MonadSTM.Strict ( readTVarIO, writeTVar, ) -import Control.Monad.Trans.Maybe (MaybeT (..)) +import qualified Control.Concurrent.Class.MonadSTM.Strict.TBQueue as TBQ import qualified Data.Strict.Maybe as Strict import Data.Time.Clock (getCurrentTime) -import Database.Persist.Postgresql (ConnectionString) +import Database.Persist.Postgresql (ConnectionString, createPostgresqlPool) import Database.Persist.Sql (SqlBackend) -import Ouroboros.Consensus.Block.Abstract (BlockProtocol, HeaderHash, Point (..), fromRawHash) +import Ouroboros.Consensus.Block.Abstract (BlockProtocol) import Ouroboros.Consensus.BlockchainTime.WallClock.Types (SystemStart (..)) import Ouroboros.Consensus.Config (SecurityParam (..), TopLevelConfig, configSecurityParam) import Ouroboros.Consensus.Node.ProtocolInfo (ProtocolInfo (pInfoConfig)) import qualified Ouroboros.Consensus.Node.ProtocolInfo as Consensus import Ouroboros.Consensus.Protocol.Abstract (ConsensusProtocol) -import Ouroboros.Network.Block (BlockNo (..), Point (..)) import Ouroboros.Network.Magic (NetworkMagic (..)) -import qualified Ouroboros.Network.Point as Point setConsistentLevel :: SyncEnv -> ConsistentLevel -> IO () setConsistentLevel env cst = do @@ -110,16 +93,6 @@ isConsistent env = do Consistent -> pure True _ -> pure False -getIsConsumedFixed :: SyncEnv -> IO (Maybe Word64) -getIsConsumedFixed env = - case (DB.pcmPruneTxOut pcm, DB.pcmConsumedTxOut pcm) of - (False, True) -> Just <$> DB.runDbIohkNoLogging backend (DB.queryWrongConsumedBy txOutTableType) - _ -> pure Nothing - where - txOutTableType = getTxOutTableType env - pcm = soptPruneConsumeMigration $ envOptions env - backend = envBackend env - getDisableInOutState :: SyncEnv -> IO Bool getDisableInOutState syncEnv = do bst <- readTVarIO $ envBootstrap syncEnv @@ -176,11 +149,12 @@ getSafeBlockNoDiff syncEnv = 2 * getSecurityParam syncEnv getPruneInterval :: SyncEnv -> Word64 getPruneInterval syncEnv = 10 * getSecurityParam syncEnv -whenConsumeOrPruneTxOut :: (MonadIO m) => SyncEnv -> m () -> m () -whenConsumeOrPruneTxOut env = - when (DB.pcmConsumedTxOut $ getPruneConsume env) +whenConsumeOrPruneTxOut :: MonadIO m => SyncEnv -> m () -> m () +whenConsumeOrPruneTxOut syncEnv action = do + disInOut <- liftIO $ getDisableInOutState syncEnv + when (not disInOut && DB.pcmConsumedTxOut (getPruneConsume syncEnv)) action -whenPruneTxOut :: (MonadIO m) => SyncEnv -> m () -> m () +whenPruneTxOut :: MonadIO m => SyncEnv -> m () -> m () whenPruneTxOut env = when (DB.pcmPruneTxOut $ getPruneConsume env) @@ -208,6 +182,13 @@ initCurrentEpochNo = { cenEpochNo = Strict.Nothing } +addNewEventsAndSort :: SyncEnv -> ApplyResult -> IO ApplyResult +addNewEventsAndSort env applyResult = do + epochEvents <- liftIO $ atomically $ generateNewEpochEvents env details + pure applyResult {apEvents = sort $ epochEvents <> apEvents applyResult} + where + details = apSlotDetails applyResult + generateNewEpochEvents :: SyncEnv -> SlotDetails -> STM [LedgerEvent] generateNewEpochEvents env details = do !lastEpochNo <- readTVar (envCurrentEpochNo env) @@ -253,59 +234,17 @@ getNetwork sEnv = getInsertOptions :: SyncEnv -> InsertOptions getInsertOptions = soptInsertOptions . envOptions -getSlotHash :: SqlBackend -> SlotNo -> IO [(SlotNo, ByteString)] -getSlotHash backend = DB.runDbIohkNoLogging backend . DB.querySlotHash - hasLedgerState :: SyncEnv -> Bool hasLedgerState syncEnv = case envLedgerEnv syncEnv of HasLedger _ -> True NoLedger _ -> False -getDbLatestBlockInfo :: SqlBackend -> IO (Maybe TipInfo) -getDbLatestBlockInfo backend = do - runMaybeT $ do - block <- MaybeT $ DB.runDbIohkNoLogging backend DB.queryLatestBlock - -- The EpochNo, SlotNo and BlockNo can only be zero for the Byron - -- era, but we need to make the types match, hence `fromMaybe`. - pure $ - TipInfo - { bHash = DB.blockHash block - , bEpochNo = EpochNo . fromMaybe 0 $ DB.blockEpochNo block - , bSlotNo = SlotNo . fromMaybe 0 $ DB.blockSlotNo block - , bBlockNo = BlockNo . fromMaybe 0 $ DB.blockBlockNo block - } - -getDbTipBlockNo :: SyncEnv -> IO (Point.WithOrigin BlockNo) -getDbTipBlockNo env = do - mblk <- getDbLatestBlockInfo (envBackend env) - pure $ maybe Point.Origin (Point.At . bBlockNo) mblk - -logDbState :: SyncEnv -> IO () -logDbState env = do - mblk <- getDbLatestBlockInfo (envBackend env) - case mblk of - Nothing -> logInfo tracer "Database is empty" - Just tip -> logInfo tracer $ mconcat ["Database tip is at ", showTip tip] - where - showTip :: TipInfo -> Text - showTip tipInfo = - mconcat - [ "slot " - , textShow (unSlotNo $ bSlotNo tipInfo) - , ", block " - , textShow (unBlockNo $ bBlockNo tipInfo) - ] - - tracer :: Trace IO Text - tracer = getTrace env - -getCurrentTipBlockNo :: SyncEnv -> IO (WithOrigin BlockNo) -getCurrentTipBlockNo env = do - maybeTip <- getDbLatestBlockInfo (envBackend env) - case maybeTip of - Just tip -> pure $ At (bBlockNo tip) - Nothing -> pure Origin +writePrefetch :: SyncEnv -> CardanoBlock -> IO () +writePrefetch syncEnv cblock = do + atomically $ + TBQ.writeTBQueue (pTxInQueue $ envPrefetch syncEnv) $ + PrefetchTxIdBlock cblock mkSyncEnv :: Trace IO Text -> @@ -334,17 +273,20 @@ mkSyncEnv trce backend connectionString syncOptions protoInfo nw nwMagic systemS , cacheCapacityTx = 100000 } else pure useNoCache + prefetch <- newPrefetch consistentLevelVar <- newTVarIO Unchecked indexesVar <- newTVarIO $ enpForceIndexes syncNP bts <- getBootstrapInProgress trce (isTxOutConsumedBootstrap' syncNodeConfigFromFile) backend bootstrapVar <- newTVarIO bts -- Offline Pool + Anchor queues + cChans <- newStakeChannels opwq <- newTBQueueIO 1000 oprq <- newTBQueueIO 1000 oawq <- newTBQueueIO 1000 oarq <- newTBQueueIO 1000 epochVar <- newTVarIO initCurrentEpochNo epochSyncTime <- newTVarIO =<< getCurrentTime + pool <- DB.runIohkLogging trce $ createPostgresqlPool connectionString 5 -- TODO make configurable ledgerEnvType <- case (enpMaybeLedgerStateDir syncNP, hasLedger' syncNodeConfigFromFile) of (Just dir, True) -> @@ -368,8 +310,10 @@ mkSyncEnv trce backend connectionString syncOptions protoInfo nw nwMagic systemS pure $ SyncEnv { envBackend = backend + , envPool = pool , envBootstrap = bootstrapVar , envCache = cache + , envPrefetch = prefetch , envConnectionString = connectionString , envConsistentLevel = consistentLevelVar , envDbConstraints = dbCNamesVar @@ -378,6 +322,7 @@ mkSyncEnv trce backend connectionString syncOptions protoInfo nw nwMagic systemS , envIndexes = indexesVar , envLedgerEnv = ledgerEnvType , envNetworkMagic = nwMagic + , envStakeChans = cChans , envOffChainPoolResultQueue = oprq , envOffChainPoolWorkQueue = opwq , envOffChainVoteResultQueue = oarq @@ -440,52 +385,6 @@ mkSyncEnvFromConfig trce backend connectionString syncOptions genCfg syncNodeCon syncNodeParams runMigrationFnc --- | 'True' is for in memory points and 'False' for on disk -getLatestPoints :: SyncEnv -> IO [(CardanoPoint, Bool)] -getLatestPoints env = do - case envLedgerEnv env of - HasLedger hasLedgerEnv -> do - snapshotPoints <- listKnownSnapshots hasLedgerEnv - verifySnapshotPoint env snapshotPoints - NoLedger _ -> do - -- Brings the 5 latest. - lastPoints <- DB.runDbIohkNoLogging (envBackend env) DB.queryLatestPoints - pure $ mapMaybe convert lastPoints - where - convert (Nothing, _) = Nothing - convert (Just slot, bs) = convertToDiskPoint (SlotNo slot) bs - -verifySnapshotPoint :: SyncEnv -> [SnapshotPoint] -> IO [(CardanoPoint, Bool)] -verifySnapshotPoint env snapPoints = - catMaybes <$> mapM validLedgerFileToPoint snapPoints - where - validLedgerFileToPoint :: SnapshotPoint -> IO (Maybe (CardanoPoint, Bool)) - validLedgerFileToPoint (OnDisk lsf) = do - hashes <- getSlotHash (envBackend env) (lsfSlotNo lsf) - let valid = find (\(_, h) -> lsfHash lsf == hashToAnnotation h) hashes - case valid of - Just (slot, hash) | slot == lsfSlotNo lsf -> pure $ convertToDiskPoint slot hash - _ -> pure Nothing - validLedgerFileToPoint (InMemory pnt) = do - case pnt of - GenesisPoint -> pure Nothing - BlockPoint slotNo hsh -> do - hashes <- getSlotHash (envBackend env) slotNo - let valid = find (\(_, dbHash) -> getHeaderHash hsh == dbHash) hashes - case valid of - Just (dbSlotNo, _) | slotNo == dbSlotNo -> pure $ Just (pnt, True) - _ -> pure Nothing - -convertToDiskPoint :: SlotNo -> ByteString -> Maybe (CardanoPoint, Bool) -convertToDiskPoint slot hashBlob = (,False) <$> convertToPoint slot hashBlob - -convertToPoint :: SlotNo -> ByteString -> Maybe CardanoPoint -convertToPoint slot hashBlob = - Point . Point.block slot <$> convertHashBlob hashBlob - where - convertHashBlob :: ByteString -> Maybe (HeaderHash CardanoBlock) - convertHashBlob = Just . fromRawHash (Proxy @CardanoBlock) - getSecurityParam :: SyncEnv -> Word64 getSecurityParam syncEnv = case envLedgerEnv syncEnv of diff --git a/cardano-db-sync/src/Cardano/DbSync/Api/Ledger.hs b/cardano-db-sync/src/Cardano/DbSync/Api/Ledger.hs index c0afff911..2ed4b4adb 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Api/Ledger.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Api/Ledger.hs @@ -15,8 +15,7 @@ import Cardano.DbSync.Era.Shelley.Generic.Tx.Babbage (fromTxOut) import Cardano.DbSync.Era.Shelley.Generic.Tx.Types (DBPlutusScript) import qualified Cardano.DbSync.Era.Shelley.Generic.Util as Generic import Cardano.DbSync.Era.Universal.Insert.Grouped -import Cardano.DbSync.Era.Universal.Insert.Tx (insertTxOut) -import Cardano.DbSync.Era.Util (liftLookupFail) +import qualified Cardano.DbSync.Era.Universal.Insert.Tx as Prepare import Cardano.DbSync.Error import Cardano.DbSync.Ledger.State import Cardano.DbSync.Types @@ -171,8 +170,7 @@ prepareTxOut syncEnv (TxIn txIntxId (TxIx index), txOut) = do let txHashByteString = Generic.safeHashToByteString $ unTxId txIntxId let genTxOut = fromTxOut index txOut txId <- liftLookupFail "prepareTxOut" $ queryTxIdWithCache cache txIntxId - insertTxOut trce cache iopts (txId, txHashByteString) genTxOut + Prepare.prepareTxOut syncEnv iopts (txId, txHashByteString) genTxOut where - trce = getTrace syncEnv cache = envCache syncEnv iopts = soptInsertOptions $ envOptions syncEnv diff --git a/cardano-db-sync/src/Cardano/DbSync/Api/Types.hs b/cardano-db-sync/src/Cardano/DbSync/Api/Types.hs index cb10af966..43062ccaf 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Api/Types.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Api/Types.hs @@ -1,6 +1,5 @@ {-# LANGUAGE GADTs #-} {-# LANGUAGE ScopedTypeVariables #-} -{-# LANGUAGE NoImplicitPrelude #-} module Cardano.DbSync.Api.Types ( SyncEnv (..), @@ -10,27 +9,32 @@ module Cardano.DbSync.Api.Types ( RunMigration, ConsistentLevel (..), CurrentEpochNo (..), + Prefetch (..), + PrefetchTxId (..), + newPrefetch, ) where import qualified Cardano.Db as DB -import Cardano.DbSync.Cache.Types (CacheStatus) +import Cardano.DbSync.Cache.Types (CacheStatus, StakeChannels) import Cardano.DbSync.Config.Types (SyncNodeConfig) +import qualified Cardano.DbSync.Era.Shelley.Generic.Tx.Types as Generic import Cardano.DbSync.Ledger.Types (HasLedgerEnv) import Cardano.DbSync.LocalStateQuery (NoLedgerEnv) import Cardano.DbSync.Types ( + CardanoBlock, OffChainPoolResult, OffChainPoolWorkQueue, OffChainVoteResult, OffChainVoteWorkQueue, ) -import Cardano.Prelude (Bool, Eq, IO, Show, Word64) import Cardano.Slotting.Slot (EpochNo (..)) -import Control.Concurrent.Class.MonadSTM.Strict ( - StrictTVar, - ) +import Control.Concurrent.Class.MonadSTM.Strict (StrictTVar, newTBQueueIO, newTVarIO) import Control.Concurrent.Class.MonadSTM.Strict.TBQueue (StrictTBQueue) +import Data.Map (Map) +import Data.Pool (Pool) import qualified Data.Strict.Maybe as Strict import Data.Time.Clock (UTCTime) +import Data.Word (Word64) import Database.Persist.Postgresql (ConnectionString) import Database.Persist.Sql (SqlBackend) import Ouroboros.Consensus.BlockchainTime.WallClock.Types (SystemStart (..)) @@ -38,6 +42,8 @@ import Ouroboros.Network.Magic (NetworkMagic (..)) data SyncEnv = SyncEnv { envBackend :: !SqlBackend + , envPool :: !(Pool SqlBackend) + , envPrefetch :: !Prefetch , envCache :: !CacheStatus , envConnectionString :: !ConnectionString , envConsistentLevel :: !(StrictTVar IO ConsistentLevel) @@ -48,6 +54,7 @@ data SyncEnv = SyncEnv , envBootstrap :: !(StrictTVar IO Bool) , envLedgerEnv :: !LedgerEnv , envNetworkMagic :: !NetworkMagic + , envStakeChans :: !StakeChannels , envOffChainPoolResultQueue :: !(StrictTBQueue IO OffChainPoolResult) , envOffChainPoolWorkQueue :: !(StrictTBQueue IO OffChainPoolWorkQueue) , envOffChainVoteResultQueue :: !(StrictTBQueue IO OffChainVoteResult) @@ -100,3 +107,16 @@ data ConsistentLevel = Consistent | DBAheadOfLedger | Unchecked newtype CurrentEpochNo = CurrentEpochNo { cenEpochNo :: Strict.Maybe EpochNo } + +data PrefetchTxId = PrefetchTxIdBlock CardanoBlock | PrefetchTxIdBlocks [CardanoBlock] + +data Prefetch = Prefetch + { pTxInQueue :: StrictTBQueue IO PrefetchTxId + , pTxIn :: StrictTVar IO (Map Generic.TxInKey (Maybe (DB.TxId, Either Generic.TxInKey DB.TxOutIdW, Maybe DB.DbLovelace))) + } + +newPrefetch :: IO Prefetch +newPrefetch = + Prefetch + <$> newTBQueueIO 1000 + <*> newTVarIO mempty diff --git a/cardano-db-sync/src/Cardano/DbSync/Block.hs b/cardano-db-sync/src/Cardano/DbSync/Block.hs new file mode 100644 index 000000000..099bfe7a6 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Block.hs @@ -0,0 +1,281 @@ +{-# LANGUAGE BangPatterns #-} +{-# LANGUAGE DataKinds #-} +{-# LANGUAGE ExplicitNamespaces #-} +{-# LANGUAGE FlexibleContexts #-} +{-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE RankNTypes #-} +{-# LANGUAGE NoImplicitPrelude #-} +{-# LANGUAGE LambdaCase #-} +{-# LANGUAGE ScopedTypeVariables #-} + +{-# OPTIONS_GHC -Wno-unused-matches #-} + +module Cardano.DbSync.Block ( + insertListBlocks, +) where + +import Cardano.BM.Trace (logError, logInfo) +import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Ledger +import Cardano.DbSync.Api.Types (ConsistentLevel (..), InsertOptions (..), SyncEnv (..)) +import Cardano.DbSync.Era.Byron.Insert (insertByronBlock) +import qualified Cardano.DbSync.Era.Shelley.Generic as Generic +import Cardano.DbSync.Era.Universal.Block (insertBlockUniversal, prepareBlock) +import Cardano.DbSync.Era.Universal.Epoch (hasEpochStartEvent) -- , hasNewEpochEvent) +import Cardano.DbSync.Era.Universal.Insert.LedgerEvent (insertNewEpochLedgerEvents) +import Cardano.DbSync.Error +import Cardano.DbSync.Ledger.Types +import Cardano.DbSync.Rollback +import Cardano.DbSync.Types +import Cardano.DbSync.Util +import Cardano.DbSync.Util.Constraint (addConstraintsIfNotExist) +import qualified Cardano.Ledger.Alonzo.Scripts as Ledger +import Cardano.Prelude +import Cardano.Slotting.Slot (EpochNo (..)) +import Control.Monad.Logger (LoggingT) +import qualified Data.ByteString.Short as SBS +import qualified Data.Strict.Maybe as Strict +import Database.Persist.SqlBackend.Internal +import Ouroboros.Consensus.Cardano.Block (HardForkBlock (..)) +import qualified Ouroboros.Consensus.HardFork.Combinator as Consensus +import Ouroboros.Network.Block (BlockNo, blockHash, blockNo, getHeaderFields, headerFieldBlockNo, unBlockNo) +import Ouroboros.Consensus.Byron.Ledger (ByronBlock (..)) +import Control.Concurrent.Async +import Cardano.DbSync.Era.Universal.Insert.Tx +import Cardano.DbSync.Era.Universal.Insert.Grouped (insertBlockGroupedData) +import Cardano.DbSync.Cache (queryPrevBlockWithCache) +import Control.Monad.Extra (whenJust) +import Database.Persist.Sql +import Cardano.DbSync.Threads.Ledger +import Control.Concurrent.Class.MonadSTM.Strict (readTMVar) + +insertListBlocks :: + SyncEnv -> + [CardanoBlock] -> + ExceptT SyncNodeError IO () +insertListBlocks syncEnv blocks = do + bl <- liftIO $ isConsistent syncEnv + if bl then + applyAndInsertBlocks syncEnv False blocks + else do + mrestBlocks <- applyAndInsertBlocksMaybe syncEnv blocks + whenJust mrestBlocks $ applyAndInsertBlocks syncEnv True + +applyAndInsertBlocksMaybe :: + SyncEnv -> + [CardanoBlock] -> + ExceptT SyncNodeError IO (Maybe [CardanoBlock]) +applyAndInsertBlocksMaybe syncEnv = go + where + go [] = pure Nothing + go ls@(cblk: rest) = do + eiBlockInDbAlreadyId <- lift $ DB.runDbLogging (envBackend syncEnv) tracer $ DB.queryBlockId (cardanoBlockHash cblk) + case eiBlockInDbAlreadyId of + Left _ -> do + liftIO + . logInfo tracer + $ mconcat + [ "Received block which is not in the db with " + , textShow (getHeaderFields cblk) + , ". Time to restore consistency." + ] + ExceptT $ DB.runDbIohkLogging (envBackend syncEnv) tracer $ runExceptT $ rollbackFromBlockNo syncEnv (blockNo cblk) + liftIO $ setConsistentLevel syncEnv Consistent + pure $ Just ls + Right _ -> do + applyRes <- fst <$> liftIO (mkApplyResult syncEnv cblk) + whenJust (getNewEpoch applyRes) $ \epochNo -> + liftIO $ logInfo tracer $ "Reached " <> textShow epochNo + go rest + + getNewEpoch :: ApplyResult -> Maybe EpochNo + getNewEpoch appRes = + Generic.neEpoch <$> maybeFromStrict (apNewEpoch appRes) + + tracer = getTrace syncEnv + +applyAndInsertBlocks :: + SyncEnv -> + Bool -> + [CardanoBlock] -> + ExceptT SyncNodeError IO () +applyAndInsertBlocks syncEnv firstAfterRollback = go + where + go [] = pure () + go ls@(blk : rest) = do + prevBlockId <- DB.runDbLoggingExceptT backend tracer $ queryPrevBlockWithCache "applyAndInsertBlocks" (envCache syncEnv) (cardanoBlockHash blk) + let newBlockId = 1 + DB.unBlockKey prevBlockId + let flagList = firstAfterRollback : replicate (length rest) False + let zippedArgs = zip (DB.BlockKey <$> [newBlockId..]) flagList + let (byronBlocks, blocks) = takeWhileByron $ zip zippedArgs (blk : rest) + DB.runDbIohkLoggingExceptT backend tracer $ mapM_ (applyAndInsertByronBlock syncEnv) byronBlocks + DB.runDbIohkLoggingExceptT backend tracer $ mapM_ (applyAndInsertBlock syncEnv) blocks -- we can use this split to parallelise even further within + + backend = envBackend syncEnv + tracer = getTrace syncEnv + +applyAndInsertByronBlock :: + SyncEnv -> + ((DB.BlockId, Bool), ByronBlock) -> + ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () +applyAndInsertByronBlock syncEnv ((_blockId, firstAfterRollback), blk) = do + (applyResult, tookSnapshot) <- liftIO (mkApplyResult syncEnv (BlockByron blk)) -- TODO use writeLedgerAction here as well for better performance + let isStartEventOrRollback = hasEpochStartEvent (apEvents applyResult) || firstAfterRollback + let details = apSlotDetails applyResult + insertNewEpochLedgerEvents syncEnv (sdEpochNo (apSlotDetails applyResult)) (apEvents applyResult) + ExceptT $ insertByronBlock syncEnv isStartEventOrRollback blk details + insertBlockRest syncEnv blkNo applyResult tookSnapshot + where + cblk :: CardanoBlock = BlockByron blk + blkNo = headerFieldBlockNo $ getHeaderFields cblk + +applyAndInsertBlock :: + SyncEnv -> + ((DB.BlockId, Bool), CardanoBlock) -> + ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () +applyAndInsertBlock syncEnv ((blockId, firstAfterRollback), cblock) = do + applyRessultVar <- liftIO (asyncApplyResult syncEnv cblock) + -- insertNewEpochLedgerEvents syncEnv (sdEpochNo (apSlotDetails applyResult)) (apEvents applyResult) + whenGeneric $ \blk -> + prepareInsertBlock syncEnv (blockId, blk) applyRessultVar firstAfterRollback + where + tracer = getTrace syncEnv + iopts = getInsertOptions syncEnv + whenGeneric action = + maybe (liftIO $ logError tracer "Found Byron Block after Shelley") action (toGenericBlock iopts cblock) + +prepareInsertBlock :: + SyncEnv -> + (DB.BlockId, Generic.Block) -> + LedgerResultResTMVar -> + Bool -> + ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () +prepareInsertBlock syncEnv (blockId, blk) applyRessultVar firstAfterRollback = do + (blockDB, preparedTxs) <- + liftIO $ concurrently + (runOrThrowIO $ runExceptT $ DB.runDbLoggingExceptT backend tracer $ prepareBlock syncEnv blk) + (mapConcurrently prepareTxWithPool (Generic.blkTxs blk)) + + _minIds <- insertBlockGroupedData syncEnv $ mconcat (snd <$> preparedTxs) + (applyResult, tookSnapshot) <- liftIO $ atomically $ readTMVar applyRessultVar + insertBlockWithLedger syncEnv blockId blockDB blk (fst <$> preparedTxs) applyResult firstAfterRollback tookSnapshot + where + prepareTxWithPool tx = runOrThrowIO $ runSqlPoolNoTransaction (prepTx tx) (envPool syncEnv) Nothing + prepTx = runExceptT . prepareTxGrouped syncEnv [] blockId + + backend = envBackend syncEnv + tracer = getTrace syncEnv + +insertBlockWithLedger :: + SyncEnv -> + DB.BlockId -> + DB.Block -> + Generic.Block -> + [(DB.TxId, DB.Tx, Generic.Tx)] -> + ApplyResult -> + Bool -> + Bool -> + ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () +insertBlockWithLedger syncEnv blockId blockDB blk txs applyResult firstAfterRollback tookSnapshot = do + mapM_ (uncurry3 $ insertTxRest syncEnv blockId epochNo slotNo applyResult) txs + insertBlockUniversal + syncEnv + blockId + blk + blockDB + applyResult + isStartEventOrRollback + insertBlockRest syncEnv blkNo applyResult tookSnapshot + where + details = apSlotDetails applyResult + isStartEventOrRollback = hasEpochStartEvent (apEvents applyResult) || firstAfterRollback + epochNo = sdEpochNo details + slotNo = sdSlotNo details + blkNo = Generic.blkBlockNo blk + +insertBlockRest :: + SyncEnv -> + BlockNo -> + ApplyResult -> + -- has snapshot been taken + Bool -> + ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () +insertBlockRest syncEnv blkNo applyResult tookSnapshot = do + + -- TODO update the epoch + -- updateEpoch + whenPruneTxOut syncEnv $ + when (unBlockNo blkNo `mod` getPruneInterval syncEnv == 0) $ + do + lift $ DB.deleteConsumedTxOut tracer txOutTableType (getSafeBlockNoDiff syncEnv) + commitOrIndexes + where + details = apSlotDetails applyResult + + {-} + isNewEpochEvent = hasNewEpochEvent (apEvents applyResult) + updateEpoch = + -- if have --dissable-epoch && --dissable-cache then no need to run this function + when (soptEpochAndCacheEnabled $ envOptions syncEnv) + . newExceptT + $ epochHandler + syncEnv + tracer + (envCache syncEnv) + isNewEpochEvent + (BlockDetails cblk details) + -} + _getPrices :: Maybe Ledger.Prices + _getPrices = case apPrices applyResult of + Strict.Just pr -> Just pr + Strict.Nothing | hasLedgerState syncEnv -> Just $ Ledger.Prices minBound minBound + Strict.Nothing -> Nothing + + commitOrIndexes :: ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () + commitOrIndexes = do + commited <- + if withinTwoMin || tookSnapshot + then do + lift DB.transactionCommit + pure True + else pure False + when withinHalfHour $ do + bootStrapMaybe syncEnv + ranIndexes <- liftIO $ getRanIndexes syncEnv + lift $ addConstraintsIfNotExist syncEnv tracer + unless ranIndexes $ do + lift $ unless commited DB.transactionCommit + liftIO $ runIndexMigrations syncEnv + + withinTwoMin = isSyncedWithinSeconds details 120 == SyncFollowing + withinHalfHour = isSyncedWithinSeconds details 1800 == SyncFollowing + + tracer = getTrace syncEnv + txOutTableType = getTxOutTableType syncEnv + +takeWhileByron :: [(a, CardanoBlock)] -> ([(a, ByronBlock)], [(a, CardanoBlock)]) +takeWhileByron = go [] + where + go accByron [] = (reverse accByron, []) + go accByron ls@(t : rest) = case toByron (snd t) of + Nothing -> (reverse accByron, ls) + Just byronBlock -> go ((fst t, byronBlock) : accByron) rest + + toByron :: CardanoBlock -> Maybe ByronBlock + toByron = \case + BlockByron blk -> Just blk + _ -> Nothing + +toGenericBlock :: InsertOptions -> CardanoBlock -> Maybe Generic.Block +toGenericBlock iopts = \case + BlockByron _ -> Nothing + BlockShelley blk -> Just $ Generic.fromShelleyBlock blk + BlockAllegra blk -> Just $ Generic.fromAllegraBlock blk + BlockMary blk -> Just $ Generic.fromMaryBlock blk + BlockAlonzo blk -> Just $ Generic.fromAlonzoBlock (ioPlutusExtra iopts) Nothing blk + BlockBabbage blk -> Just $ Generic.fromBabbageBlock (ioPlutusExtra iopts) Nothing blk + BlockConway blk -> Just $ Generic.fromConwayBlock (ioPlutusExtra iopts) Nothing blk + +cardanoBlockHash :: CardanoBlock -> ByteString +cardanoBlockHash = SBS.fromShort . Consensus.getOneEraHash . blockHash diff --git a/cardano-db-sync/src/Cardano/DbSync/Cache.hs b/cardano-db-sync/src/Cardano/DbSync/Cache.hs index 36c8315fd..21104bf4c 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Cache.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Cache.hs @@ -4,7 +4,6 @@ {-# LANGUAGE OverloadedStrings #-} {-# LANGUAGE RankNTypes #-} {-# LANGUAGE ScopedTypeVariables #-} -{-# LANGUAGE TupleSections #-} {-# LANGUAGE NoImplicitPrelude #-} module Cardano.DbSync.Cache ( @@ -16,11 +15,7 @@ module Cardano.DbSync.Cache ( queryPoolKeyOrInsert, queryPoolKeyWithCache, queryPrevBlockWithCache, - queryOrInsertStakeAddress, - queryOrInsertRewardAccount, insertAddressUsingCache, - insertStakeAddress, - queryStakeAddrWithCache, queryTxIdWithCache, rollbackCache, optimiseCaches, @@ -28,24 +23,24 @@ module Cardano.DbSync.Cache ( -- * CacheStatistics getCacheStatistics, + module X, ) where import Cardano.BM.Trace import qualified Cardano.Db as DB import qualified Cardano.Db.Schema.Variant.TxOut as V +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types import Cardano.DbSync.Cache.Epoch (rollbackMapEpochInCache) import qualified Cardano.DbSync.Cache.FIFO as FIFO import qualified Cardano.DbSync.Cache.LRU as LRU +import Cardano.DbSync.Cache.Stake as X import Cardano.DbSync.Cache.Types (CacheAction (..), CacheInternal (..), CacheStatistics (..), CacheStatus (..), StakeCache (..), initCacheStatistics, shouldCache) +import Cardano.DbSync.Cache.Util import qualified Cardano.DbSync.Era.Shelley.Generic.Util as Generic -import Cardano.DbSync.Era.Shelley.Query -import Cardano.DbSync.Era.Util import Cardano.DbSync.Error import Cardano.DbSync.Types -import qualified Cardano.Ledger.Address as Ledger -import Cardano.Ledger.BaseTypes (Network) import Cardano.Ledger.Mary.Value -import qualified Cardano.Ledger.TxIn as Ledger import Cardano.Prelude import Control.Concurrent.Class.MonadSTM.Strict ( StrictTVar, @@ -109,113 +104,6 @@ getCacheStatistics cs = NoCache -> pure initCacheStatistics ActiveCache ci -> readTVarIO (cStats ci) -queryOrInsertRewardAccount :: - (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> - CacheAction -> - Ledger.RewardAccount StandardCrypto -> - ReaderT SqlBackend m DB.StakeAddressId -queryOrInsertRewardAccount trce cache cacheUA rewardAddr = do - eiAddrId <- queryStakeAddrWithCacheRetBs trce cache cacheUA rewardAddr - case eiAddrId of - Left (_err, bs) -> insertStakeAddress rewardAddr (Just bs) - Right addrId -> pure addrId - -queryOrInsertStakeAddress :: - (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> - CacheAction -> - Network -> - StakeCred -> - ReaderT SqlBackend m DB.StakeAddressId -queryOrInsertStakeAddress trce cache cacheUA nw cred = - queryOrInsertRewardAccount trce cache cacheUA $ Ledger.RewardAccount nw cred - --- If the address already exists in the table, it will not be inserted again (due to --- the uniqueness constraint) but the function will return the 'StakeAddressId'. -insertStakeAddress :: - (MonadBaseControl IO m, MonadIO m) => - Ledger.RewardAccount StandardCrypto -> - Maybe ByteString -> - ReaderT SqlBackend m DB.StakeAddressId -insertStakeAddress rewardAddr stakeCredBs = do - DB.insertStakeAddress $ - DB.StakeAddress - { DB.stakeAddressHashRaw = addrBs - , DB.stakeAddressView = Generic.renderRewardAccount rewardAddr - , DB.stakeAddressScriptHash = Generic.getCredentialScriptHash $ Ledger.raCredential rewardAddr - } - where - addrBs = fromMaybe (Ledger.serialiseRewardAccount rewardAddr) stakeCredBs - -queryStakeAddrWithCache :: - forall m. - MonadIO m => - Trace IO Text -> - CacheStatus -> - CacheAction -> - Network -> - StakeCred -> - ReaderT SqlBackend m (Either DB.LookupFail DB.StakeAddressId) -queryStakeAddrWithCache trce cache cacheUA nw cred = - mapLeft fst <$> queryStakeAddrWithCacheRetBs trce cache cacheUA (Ledger.RewardAccount nw cred) - -queryStakeAddrWithCacheRetBs :: - forall m. - MonadIO m => - Trace IO Text -> - CacheStatus -> - CacheAction -> - Ledger.RewardAccount StandardCrypto -> - ReaderT SqlBackend m (Either (DB.LookupFail, ByteString) DB.StakeAddressId) -queryStakeAddrWithCacheRetBs _trce cache cacheUA ra@(Ledger.RewardAccount _ cred) = do - let bs = Ledger.serialiseRewardAccount ra - case cache of - NoCache -> rsStkAdrrs bs - ActiveCache ci -> do - withCacheOptimisationCheck ci (rsStkAdrrs bs) $ do - stakeCache <- liftIO $ readTVarIO (cStake ci) - case queryStakeCache cred stakeCache of - Just (addrId, stakeCache') -> do - liftIO $ hitCreds (cStats ci) - case cacheUA of - EvictAndUpdateCache -> do - liftIO $ atomically $ writeTVar (cStake ci) $ deleteStakeCache cred stakeCache' - pure $ Right addrId - _other -> do - liftIO $ atomically $ writeTVar (cStake ci) stakeCache' - pure $ Right addrId - Nothing -> do - queryRes <- mapLeft (,bs) <$> resolveStakeAddress bs - liftIO $ missCreds (cStats ci) - case queryRes of - Left _ -> pure queryRes - Right stakeAddrsId -> do - let !stakeCache' = case cacheUA of - UpdateCache -> stakeCache {scLruCache = LRU.insert cred stakeAddrsId (scLruCache stakeCache)} - UpdateCacheStrong -> stakeCache {scStableCache = Map.insert cred stakeAddrsId (scStableCache stakeCache)} - _otherwise -> stakeCache - liftIO $ - atomically $ - writeTVar (cStake ci) stakeCache' - pure $ Right stakeAddrsId - where - rsStkAdrrs bs = mapLeft (,bs) <$> resolveStakeAddress bs - --- | True if it was found in LRU -queryStakeCache :: StakeCred -> StakeCache -> Maybe (DB.StakeAddressId, StakeCache) -queryStakeCache scred scache = case Map.lookup scred (scStableCache scache) of - Just addrId -> Just (addrId, scache) - Nothing -> case LRU.lookup scred (scLruCache scache) of - Just (addrId, lru') -> Just (addrId, scache {scLruCache = lru'}) - Nothing -> Nothing - -deleteStakeCache :: StakeCred -> StakeCache -> StakeCache -deleteStakeCache scred scache = - scache {scStableCache = Map.delete scred (scStableCache scache)} - queryPoolKeyWithCache :: MonadIO m => CacheStatus -> @@ -352,14 +240,13 @@ insertPoolKeyWithCache cache cacheUA pHash = queryPoolKeyOrInsert :: (MonadBaseControl IO m, MonadIO m) => + SyncEnv -> Text -> - Trace IO Text -> - CacheStatus -> CacheAction -> Bool -> PoolKeyHash -> ReaderT SqlBackend m DB.PoolHashId -queryPoolKeyOrInsert txt trce cache cacheUA logsWarning hsh = do +queryPoolKeyOrInsert syncEnv txt cacheUA logsWarning hsh = do pk <- queryPoolKeyWithCache cache cacheUA hsh case pk of Right poolHashId -> pure poolHashId @@ -377,6 +264,9 @@ queryPoolKeyOrInsert txt trce cache cacheUA logsWarning hsh = do , ". We will assume that the pool exists and move on." ] insertPoolKeyWithCache cache cacheUA hsh + where + trce = getTrace syncEnv + cache = envCache syncEnv queryMAWithCache :: MonadIO m => @@ -442,7 +332,7 @@ queryPrevBlockWithCache msg cache hsh = queryTxIdWithCache :: MonadIO m => CacheStatus -> - Ledger.TxId StandardCrypto -> + TxIdLedger -> ReaderT SqlBackend m (Either DB.LookupFail DB.TxId) queryTxIdWithCache cache txIdLedger = do case cache of @@ -477,7 +367,7 @@ queryTxIdWithCache cache txIdLedger = do tryUpdateCacheTx :: MonadIO m => CacheStatus -> - Ledger.TxId StandardCrypto -> + TxIdLedger -> DB.TxId -> m () tryUpdateCacheTx (ActiveCache ci) ledgerTxId txId = @@ -487,20 +377,21 @@ tryUpdateCacheTx _ _ _ = pure () insertBlockAndCache :: (MonadIO m, MonadBaseControl IO m) => CacheStatus -> + DB.BlockId -> DB.Block -> - ReaderT SqlBackend m DB.BlockId -insertBlockAndCache cache block = + ReaderT SqlBackend m () +insertBlockAndCache cache k block = case cache of NoCache -> insBlck ActiveCache ci -> withCacheOptimisationCheck ci insBlck $ do - bid <- insBlck - liftIO $ do - missPrevBlock (cStats ci) - atomically $ writeTVar (cPrevBlock ci) $ Just (bid, DB.blockHash block) - pure bid + insBlck + liftIO $ + atomically $ + writeTVar (cPrevBlock ci) $ + Just (k, DB.blockHash block) where - insBlck = DB.insertBlock block + insBlck = DB.insertBlock k block queryDatum :: MonadIO m => @@ -544,27 +435,6 @@ insertDatumAndCache cache hsh dt = do LRU.insert hsh datumId pure datumId -withCacheOptimisationCheck :: - MonadIO m => - CacheInternal -> - m a -> -- Action to perform if cache is optimised - m a -> -- Action to perform if cache is not optimised - m a -withCacheOptimisationCheck ci ifOptimised ifNotOptimised = do - isCachedOptimised <- liftIO $ readTVarIO (cIsCacheOptimised ci) - if isCachedOptimised - then ifOptimised - else ifNotOptimised - --- Stakes -hitCreds :: StrictTVar IO CacheStatistics -> IO () -hitCreds ref = - atomically $ modifyTVar ref (\cs -> cs {credsHits = 1 + credsHits cs, credsQueries = 1 + credsQueries cs}) - -missCreds :: StrictTVar IO CacheStatistics -> IO () -missCreds ref = - atomically $ modifyTVar ref (\cs -> cs {credsQueries = 1 + credsQueries cs}) - -- Pools hitPools :: StrictTVar IO CacheStatistics -> IO () hitPools ref = diff --git a/cardano-db-sync/src/Cardano/DbSync/Cache/Stake.hs b/cardano-db-sync/src/Cardano/DbSync/Cache/Stake.hs new file mode 100644 index 000000000..adbc0166a --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Cache/Stake.hs @@ -0,0 +1,156 @@ +{-# LANGUAGE FlexibleContexts #-} +{-# LANGUAGE RankNTypes #-} +{-# LANGUAGE TupleSections #-} + +module Cardano.DbSync.Cache.Stake where + +import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types +import qualified Cardano.DbSync.Cache.LRU as LRU +import Cardano.DbSync.Cache.Types +import Cardano.DbSync.Cache.Util +import qualified Cardano.DbSync.Era.Shelley.Generic.Util as Generic +import Cardano.DbSync.Era.Shelley.Query +import Cardano.DbSync.Types +import qualified Cardano.Ledger.Address as Ledger +import Cardano.Prelude +import Control.Concurrent.Class.MonadSTM.Strict ( + StrictTVar, + modifyTVar, + newEmptyTMVarIO, + readTVarIO, + takeTMVar, + writeTVar, + ) +import qualified Control.Concurrent.STM.TBQueue as TBQ +import Control.Monad.Trans.Control (MonadBaseControl) +import Data.Either.Combinators +import qualified Data.Map.Strict as Map +import Database.Persist.Postgresql (SqlBackend) + +-- | TO be called only by the stake thread +resolveInsertRewardAccount :: + forall m. + (MonadBaseControl IO m, MonadIO m) => + SyncEnv -> + CacheAction -> + RewAccount -> + ReaderT SqlBackend m DB.StakeAddressId +resolveInsertRewardAccount syncEnv cacheUA ra = do + eiStakeId <- queryStakeAddrWithCacheRetBs syncEnv cacheUA False ra -- read only + case eiStakeId of + Right stakeId -> pure stakeId + Left (_, bs) -> insertStakeAddress ra (Just bs) + +-- | TO be called only by the stake thread +-- If the address already exists in the table, it will not be inserted again (due to +-- the uniqueness constraint) but the function will return the 'StakeAddressId'. +insertStakeAddress :: + (MonadBaseControl IO m, MonadIO m) => + RewAccount -> + Maybe ByteString -> + ReaderT SqlBackend m DB.StakeAddressId +insertStakeAddress rewardAddr stakeCredBs = do + DB.insertStakeAddress $ + DB.StakeAddress + { DB.stakeAddressHashRaw = addrBs + , DB.stakeAddressView = Generic.renderRewardAccount rewardAddr + , DB.stakeAddressScriptHash = Generic.getCredentialScriptHash $ Ledger.raCredential rewardAddr + } + where + addrBs = fromMaybe (Ledger.serialiseRewardAccount rewardAddr) stakeCredBs + +queryStakeAddrWithCacheRetBs :: + forall m. + MonadIO m => + SyncEnv -> + CacheAction -> + Bool -> + RewAccount -> + ReaderT SqlBackend m (Either (DB.LookupFail, ByteString) DB.StakeAddressId) +queryStakeAddrWithCacheRetBs syncEnv cacheUA readOnly ra@(Ledger.RewardAccount _ cred) = do + case cache of + NoCache -> queryStakeDB + ActiveCache ci -> do + withCacheOptimisationCheck ci queryStakeDB $ do + stakeCache <- liftIO $ readTVarIO (cStake ci) + case queryStakeCache cred stakeCache of + Just (addrId, stakeCache') -> do + unless readOnly $ liftIO $ hitCreds (cStats ci) + case cacheUA of + EvictAndUpdateCache -> do + unless readOnly $ liftIO $ atomically $ writeTVar (cStake ci) $ deleteStakeCache cred stakeCache' + pure $ Right addrId + _other -> do + unless readOnly $ liftIO $ atomically $ writeTVar (cStake ci) stakeCache' + pure $ Right addrId + Nothing -> do + queryRes <- queryStakeDB + unless readOnly $ liftIO $ missCreds (cStats ci) + case queryRes of + Left _ -> pure queryRes + Right stakeAddrsId -> do + let stakeCache' = case cacheUA of + UpdateCache -> stakeCache {scLruCache = LRU.insert cred stakeAddrsId (scLruCache stakeCache)} + UpdateCacheStrong -> stakeCache {scStableCache = Map.insert cred stakeAddrsId (scStableCache stakeCache)} + _otherwise -> stakeCache + unless readOnly $ + liftIO $ + atomically $ + writeTVar (cStake ci) stakeCache' + pure $ Right stakeAddrsId + where + bs = Ledger.serialiseRewardAccount ra + queryStakeDB = mapLeft (,bs) <$> resolveStakeAddress bs + _trce = getTrace syncEnv + cache = envCache syncEnv + +-- | Checks both caches +queryStakeCache :: StakeCred -> StakeCache -> Maybe (DB.StakeAddressId, StakeCache) +queryStakeCache scred scache = case Map.lookup scred (scStableCache scache) of + Just addrId -> Just (addrId, scache) + Nothing -> case LRU.lookup scred (scLruCache scache) of + Just (addrId, lru') -> Just (addrId, scache {scLruCache = lru'}) + Nothing -> Nothing + +deleteStakeCache :: StakeCred -> StakeCache -> StakeCache +deleteStakeCache scred scache = + scache {scStableCache = Map.delete scred (scStableCache scache)} + +hitCreds :: StrictTVar IO CacheStatistics -> IO () +hitCreds ref = + atomically $ modifyTVar ref (\cs -> cs {credsHits = 1 + credsHits cs, credsQueries = 1 + credsQueries cs}) + +missCreds :: StrictTVar IO CacheStatistics -> IO () +missCreds ref = + atomically $ modifyTVar ref (\cs -> cs {credsQueries = 1 + credsQueries cs}) + +queryOrInsertStakeAddress :: + forall m. + MonadIO m => + SyncEnv -> + CacheAction -> + StakeCred -> + ReaderT SqlBackend m DB.StakeAddressId +queryOrInsertStakeAddress syncEnv cacheUA cred = + queryOrInsertRewardAccount syncEnv cacheUA $ Ledger.RewardAccount (getNetwork syncEnv) cred + +queryOrInsertRewardAccount :: + forall m. + MonadIO m => + SyncEnv -> + CacheAction -> + RewAccount -> + ReaderT SqlBackend m DB.StakeAddressId +queryOrInsertRewardAccount syncEnv cacheUA ra = do + eiStakeId <- queryStakeAddrWithCacheRetBs syncEnv cacheUA True ra -- read only + case eiStakeId of + Right stakeId -> pure stakeId + Left _ -> liftIO $ do + resultVar <- newEmptyTMVarIO + atomically $ TBQ.writeTBQueue (scPriorityQueue stakeChan) $ QueryInsertStake ra cacheUA resultVar + atomically $ takeTMVar resultVar + where + _trce = getTrace syncEnv + stakeChan = envStakeChans syncEnv diff --git a/cardano-db-sync/src/Cardano/DbSync/Cache/Types.hs b/cardano-db-sync/src/Cardano/DbSync/Cache/Types.hs index 9c060f907..e45a1fe64 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Cache/Types.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Cache/Types.hs @@ -16,11 +16,14 @@ module Cardano.DbSync.Cache.Types ( EpochBlockDiff (..), StakeCache (..), StakePoolCache, + StakeDBAction (..), + StakeChannels (..), -- * Inits useNoCache, initCacheStatistics, newEmptyCache, + newStakeChannels, -- * Utils shouldCache, @@ -36,15 +39,17 @@ import Cardano.DbSync.Cache.FIFO (FIFOCache) import qualified Cardano.DbSync.Cache.FIFO as FIFO import Cardano.DbSync.Cache.LRU (LRUCache) import qualified Cardano.DbSync.Cache.LRU as LRU -import Cardano.DbSync.Types (DataHash, PoolKeyHash, StakeCred) +import Cardano.DbSync.Types (CardanoBlock, DataHash, PoolKeyHash, RewAccount, StakeCred, TxIdLedger) import Cardano.Ledger.Mary.Value (AssetName, PolicyID) -import qualified Cardano.Ledger.TxIn as Ledger import Cardano.Prelude import Control.Concurrent.Class.MonadSTM.Strict ( + StrictTMVar, StrictTVar, newTVarIO, readTVarIO, ) +import Control.Concurrent.STM.TBQueue (TBQueue) +import qualified Control.Concurrent.STM.TBQueue as TBQ import qualified Data.Map.Strict as Map import Data.Time.Clock (UTCTime) import Data.WideWord.Word128 (Word128) @@ -52,7 +57,7 @@ import Ouroboros.Consensus.Cardano.Block (StandardCrypto) type StakePoolCache = Map PoolKeyHash DB.PoolHashId --- | We use a stable cache for entries that are expected to be reused frequentyl. +-- | We use a stable cache for entries that are expected to be reused frequently. -- These are stake addresses that have rewards, delegations etc. -- They are never removed unless manually eg when it's deregistered -- The LRU cache is much smaller for the rest stake addresses. @@ -84,7 +89,7 @@ data CacheInternal = CacheInternal , cStats :: !(StrictTVar IO CacheStatistics) , cEpoch :: !(StrictTVar IO CacheEpoch) , cAddress :: !(StrictTVar IO (LRUCache ByteString V.AddressId)) - , cTxIds :: !(StrictTVar IO (FIFOCache (Ledger.TxId StandardCrypto) DB.TxId)) + , cTxIds :: !(StrictTVar IO (FIFOCache TxIdLedger DB.TxId)) } data CacheStatistics = CacheStatistics @@ -217,8 +222,8 @@ textShowStats (ActiveCache ic) = do useNoCache :: CacheStatus useNoCache = NoCache -newEmptyCache :: MonadIO m => CacheCapacity -> m CacheStatus -newEmptyCache CacheCapacity {..} = liftIO $ do +newEmptyCache :: CacheCapacity -> IO CacheStatus +newEmptyCache CacheCapacity {..} = do cIsCacheOptimised <- newTVarIO False cStake <- newTVarIO (StakeCache Map.empty (LRU.empty cacheCapacityStake)) cPools <- newTVarIO Map.empty @@ -255,3 +260,22 @@ shouldCache = \case UpdateCache -> True UpdateCacheStrong -> True _ -> False + +data StakeDBAction + = QueryInsertStake RewAccount CacheAction (StrictTMVar IO DB.StakeAddressId) + | CacheStake RewAccount DB.StakeAddressId Bool + | BulkPrefetch CardanoBlock + | CommitStake + +data StakeChannels = StakeChannels + { scPriorityQueue :: TBQueue StakeDBAction + , scSecondaryQueue :: TBQueue StakeDBAction + } + +newStakeChannels :: IO StakeChannels +newStakeChannels = + -- This may never be more than 1. But let's keep it a queue for extensibility shake. + -- This may allow us to parallelize the events workload even further + StakeChannels + <$> TBQ.newTBQueueIO 100 + <*> TBQ.newTBQueueIO 100 diff --git a/cardano-db-sync/src/Cardano/DbSync/Cache/Util.hs b/cardano-db-sync/src/Cardano/DbSync/Cache/Util.hs new file mode 100644 index 000000000..1287823c5 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Cache/Util.hs @@ -0,0 +1,17 @@ +module Cardano.DbSync.Cache.Util where + +import Cardano.DbSync.Cache.Types +import Cardano.Prelude +import Control.Concurrent.Class.MonadSTM.Strict (readTVarIO) + +withCacheOptimisationCheck :: + MonadIO m => + CacheInternal -> + m a -> -- Action to perform if cache is optimised + m a -> -- Action to perform if cache is not optimised + m a +withCacheOptimisationCheck ci ifOptimised ifNotOptimised = do + isCachedOptimised <- liftIO $ readTVarIO (cIsCacheOptimised ci) + if isCachedOptimised + then ifOptimised + else ifNotOptimised diff --git a/cardano-db-sync/src/Cardano/DbSync/Config.hs b/cardano-db-sync/src/Cardano/DbSync/Config.hs index f38e65307..cbe632a79 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Config.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Config.hs @@ -15,7 +15,6 @@ module Cardano.DbSync.Config ( SyncProtocol (..), SyncNodeConfig (..), SyncNodeParams (..), - cardanoLedgerConfig, genesisProtocolMagicId, readCardanoGenesisConfig, readSyncNodeConfig, diff --git a/cardano-db-sync/src/Cardano/DbSync/Config/Cardano.hs b/cardano-db-sync/src/Cardano/DbSync/Config/Cardano.hs index 060a095dd..9f8238744 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Config/Cardano.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Config/Cardano.hs @@ -7,7 +7,6 @@ module Cardano.DbSync.Config.Cardano ( GenesisConfig (..), - cardanoLedgerConfig, genesisProtocolMagicId, mkTopLevelConfig, mkProtocolInfoCardano, @@ -35,10 +34,10 @@ import Ouroboros.Consensus.Cardano (Nonce (..), ProtVer (ProtVer)) import qualified Ouroboros.Consensus.Cardano as Consensus import Ouroboros.Consensus.Cardano.Node import Ouroboros.Consensus.Config (TopLevelConfig (..), emptyCheckpointsMap) -import Ouroboros.Consensus.Ledger.Basics (LedgerConfig) import Ouroboros.Consensus.Node.ProtocolInfo (ProtocolInfo) import qualified Ouroboros.Consensus.Node.ProtocolInfo as Consensus import Ouroboros.Consensus.Shelley.Eras (StandardCrypto) +import Ouroboros.Consensus.Shelley.Ledger.SupportsProtocol () import Ouroboros.Consensus.Shelley.Node (ShelleyGenesis (..)) -- Usually only one constructor, but may have two when we are preparing for a HFC event. @@ -72,9 +71,6 @@ readCardanoGenesisConfig enc = -- ------------------------------------------------------------------------------------------------- -cardanoLedgerConfig :: GenesisConfig -> LedgerConfig CardanoBlock -cardanoLedgerConfig = topLevelConfigLedger . mkTopLevelConfig - mkTopLevelConfig :: GenesisConfig -> TopLevelConfig CardanoBlock mkTopLevelConfig cfg = Consensus.pInfoConfig $ fst $ mkProtocolInfoCardano cfg [] diff --git a/cardano-db-sync/src/Cardano/DbSync/Default.hs b/cardano-db-sync/src/Cardano/DbSync/Default.hs deleted file mode 100644 index 1703a584d..000000000 --- a/cardano-db-sync/src/Cardano/DbSync/Default.hs +++ /dev/null @@ -1,226 +0,0 @@ -{-# LANGUAGE BangPatterns #-} -{-# LANGUAGE DataKinds #-} -{-# LANGUAGE ExplicitNamespaces #-} -{-# LANGUAGE FlexibleContexts #-} -{-# LANGUAGE OverloadedStrings #-} -{-# LANGUAGE RankNTypes #-} -{-# LANGUAGE NoImplicitPrelude #-} -{-# OPTIONS_GHC -Wno-unused-matches #-} - -module Cardano.DbSync.Default ( - insertListBlocks, -) where - -import Cardano.BM.Trace (logInfo) -import qualified Cardano.Db as DB -import Cardano.DbSync.Api -import Cardano.DbSync.Api.Ledger -import Cardano.DbSync.Api.Types (ConsistentLevel (..), InsertOptions (..), LedgerEnv (..), SyncEnv (..), SyncOptions (..)) -import Cardano.DbSync.Epoch (epochHandler) -import Cardano.DbSync.Era.Byron.Insert (insertByronBlock) -import qualified Cardano.DbSync.Era.Shelley.Generic as Generic -import Cardano.DbSync.Era.Universal.Block (insertBlockUniversal) -import Cardano.DbSync.Era.Universal.Epoch (hasEpochStartEvent, hasNewEpochEvent) -import Cardano.DbSync.Era.Universal.Insert.Certificate (mkAdaPots) -import Cardano.DbSync.Era.Universal.Insert.LedgerEvent (insertNewEpochLedgerEvents) -import Cardano.DbSync.Error -import Cardano.DbSync.Ledger.State (applyBlockAndSnapshot, defaultApplyResult) -import Cardano.DbSync.Ledger.Types (ApplyResult (..)) -import Cardano.DbSync.LocalStateQuery -import Cardano.DbSync.Rollback -import Cardano.DbSync.Types -import Cardano.DbSync.Util -import Cardano.DbSync.Util.Constraint (addConstraintsIfNotExist) -import qualified Cardano.Ledger.Alonzo.Scripts as Ledger -import Cardano.Ledger.Shelley.AdaPots as Shelley -import Cardano.Node.Configuration.Logging (Trace) -import Cardano.Prelude -import Cardano.Slotting.Slot (EpochNo (..), SlotNo) -import Control.Monad.Logger (LoggingT) -import Control.Monad.Trans.Except.Extra (newExceptT) -import qualified Data.ByteString.Short as SBS -import qualified Data.Set as Set -import qualified Data.Strict.Maybe as Strict -import Database.Persist.SqlBackend.Internal -import Ouroboros.Consensus.Cardano.Block (HardForkBlock (..)) -import qualified Ouroboros.Consensus.HardFork.Combinator as Consensus -import Ouroboros.Network.Block (blockHash, blockNo, getHeaderFields, headerFieldBlockNo, unBlockNo) - -insertListBlocks :: - SyncEnv -> - [CardanoBlock] -> - IO (Either SyncNodeError ()) -insertListBlocks synEnv blocks = do - DB.runDbIohkLogging (envBackend synEnv) tracer - . runExceptT - $ traverse_ (applyAndInsertBlockMaybe synEnv tracer) blocks - where - tracer = getTrace synEnv - -applyAndInsertBlockMaybe :: - SyncEnv -> - Trace IO Text -> - CardanoBlock -> - ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () -applyAndInsertBlockMaybe syncEnv tracer cblk = do - bl <- liftIO $ isConsistent syncEnv - (!applyRes, !tookSnapshot) <- liftIO (mkApplyResult bl) - if bl - then -- In the usual case it will be consistent so we don't need to do any queries. Just insert the block - insertBlock syncEnv cblk applyRes False tookSnapshot - else do - eiBlockInDbAlreadyId <- lift (DB.queryBlockId (SBS.fromShort . Consensus.getOneEraHash $ blockHash cblk)) - -- If the block is already in db, do nothing. If not, delete all blocks with greater 'BlockNo' or - -- equal, insert the block and restore consistency between ledger and db. - case eiBlockInDbAlreadyId of - Left _ -> do - liftIO - . logInfo tracer - $ mconcat - [ "Received block which is not in the db with " - , textShow (getHeaderFields cblk) - , ". Time to restore consistency." - ] - rollbackFromBlockNo syncEnv (blockNo cblk) - insertBlock syncEnv cblk applyRes True tookSnapshot - liftIO $ setConsistentLevel syncEnv Consistent - Right blockId | Just (adaPots, slotNo, epochNo) <- getAdaPots applyRes -> do - replaced <- lift $ DB.replaceAdaPots blockId $ mkAdaPots blockId slotNo epochNo adaPots - if replaced - then liftIO $ logInfo tracer $ "Fixed AdaPots for " <> textShow epochNo - else liftIO $ logInfo tracer $ "Reached " <> textShow epochNo - Right _ - | Just epochNo <- getNewEpoch applyRes -> - liftIO $ logInfo tracer $ "Reached " <> textShow epochNo - _ -> pure () - where - mkApplyResult :: Bool -> IO (ApplyResult, Bool) - mkApplyResult isCons = do - case envLedgerEnv syncEnv of - HasLedger hle -> applyBlockAndSnapshot hle cblk isCons - NoLedger nle -> do - slotDetails <- getSlotDetailsNode nle (cardanoBlockSlotNo cblk) - pure (defaultApplyResult slotDetails, False) - - getAdaPots :: ApplyResult -> Maybe (Shelley.AdaPots, SlotNo, EpochNo) - getAdaPots appRes = do - newEpoch <- maybeFromStrict $ apNewEpoch appRes - adaPots <- maybeFromStrict $ Generic.neAdaPots newEpoch - pure (adaPots, sdSlotNo $ apSlotDetails appRes, sdEpochNo $ apSlotDetails appRes) - - getNewEpoch :: ApplyResult -> Maybe EpochNo - getNewEpoch appRes = - Generic.neEpoch <$> maybeFromStrict (apNewEpoch appRes) - -insertBlock :: - SyncEnv -> - CardanoBlock -> - ApplyResult -> - -- is first Block after rollback - Bool -> - -- has snapshot been taken - Bool -> - ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () -insertBlock syncEnv cblk applyRes firstAfterRollback tookSnapshot = do - !epochEvents <- liftIO $ atomically $ generateNewEpochEvents syncEnv (apSlotDetails applyRes) - let !applyResult = applyRes {apEvents = sort $ epochEvents <> apEvents applyRes} - let !details = apSlotDetails applyResult - let !withinTwoMin = isWithinTwoMin details - let !withinHalfHour = isWithinHalfHour details - insertNewEpochLedgerEvents syncEnv (sdEpochNo details) (apEvents applyResult) - let isNewEpochEvent = hasNewEpochEvent (apEvents applyResult) - let isStartEventOrRollback = hasEpochStartEvent (apEvents applyResult) || firstAfterRollback - let isMember poolId = Set.member poolId (apPoolsRegistered applyResult) - let insertBlockUniversal' blk = - insertBlockUniversal - syncEnv - isStartEventOrRollback - withinTwoMin - withinHalfHour - blk - details - isMember - applyResult - - -- Here we insert the block and it's txs, but in adition we also cache some values which we later - -- use when updating the Epoch, thus saving us having to recalulating them later. - case cblk of - BlockByron blk -> - newExceptT $ - insertByronBlock syncEnv isStartEventOrRollback blk details - BlockShelley blk -> - newExceptT $ - insertBlockUniversal' $ - Generic.fromShelleyBlock blk - BlockAllegra blk -> - newExceptT $ - insertBlockUniversal' $ - Generic.fromAllegraBlock blk - BlockMary blk -> - newExceptT $ - insertBlockUniversal' $ - Generic.fromMaryBlock blk - BlockAlonzo blk -> - newExceptT $ - insertBlockUniversal' $ - Generic.fromAlonzoBlock (ioPlutusExtra iopts) (getPrices applyResult) blk - BlockBabbage blk -> - newExceptT $ - insertBlockUniversal' $ - Generic.fromBabbageBlock (ioPlutusExtra iopts) (getPrices applyResult) blk - BlockConway blk -> - newExceptT $ - insertBlockUniversal' $ - Generic.fromConwayBlock (ioPlutusExtra iopts) (getPrices applyResult) blk - -- update the epoch - updateEpoch details isNewEpochEvent - whenPruneTxOut syncEnv $ - when (unBlockNo blkNo `mod` getPruneInterval syncEnv == 0) $ - do - lift $ DB.deleteConsumedTxOut tracer txOutTableType (getSafeBlockNoDiff syncEnv) - commitOrIndexes withinTwoMin withinHalfHour - where - tracer = getTrace syncEnv - txOutTableType = getTxOutTableType syncEnv - iopts = getInsertOptions syncEnv - - updateEpoch details isNewEpochEvent = - -- if have --dissable-epoch && --dissable-cache then no need to run this function - when (soptEpochAndCacheEnabled $ envOptions syncEnv) - . newExceptT - $ epochHandler - syncEnv - tracer - (envCache syncEnv) - isNewEpochEvent - (BlockDetails cblk details) - - getPrices :: ApplyResult -> Maybe Ledger.Prices - getPrices applyResult = case apPrices applyResult of - Strict.Just pr -> Just pr - Strict.Nothing | hasLedgerState syncEnv -> Just $ Ledger.Prices minBound minBound - Strict.Nothing -> Nothing - - commitOrIndexes :: Bool -> Bool -> ExceptT SyncNodeError (ReaderT SqlBackend (LoggingT IO)) () - commitOrIndexes withinTwoMin withinHalfHour = do - commited <- - if withinTwoMin || tookSnapshot - then do - lift DB.transactionCommit - pure True - else pure False - when withinHalfHour $ do - bootStrapMaybe syncEnv - ranIndexes <- liftIO $ getRanIndexes syncEnv - lift $ addConstraintsIfNotExist syncEnv tracer - unless ranIndexes $ do - lift $ unless commited DB.transactionCommit - liftIO $ runIndexMigrations syncEnv - - isWithinTwoMin :: SlotDetails -> Bool - isWithinTwoMin sd = isSyncedWithinSeconds sd 120 == SyncFollowing - - isWithinHalfHour :: SlotDetails -> Bool - isWithinHalfHour sd = isSyncedWithinSeconds sd 1800 == SyncFollowing - - blkNo = headerFieldBlockNo $ getHeaderFields cblk diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Genesis.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Genesis.hs index 493f5f4e5..b373f5506 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Genesis.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Genesis.hs @@ -25,7 +25,6 @@ import Cardano.DbSync.Cache (insertAddressUsingCache) import Cardano.DbSync.Cache.Types (CacheAction (..)) import Cardano.DbSync.Config.Types import qualified Cardano.DbSync.Era.Byron.Util as Byron -import Cardano.DbSync.Era.Util (liftLookupFail) import Cardano.DbSync.Error import Cardano.DbSync.Util import Cardano.Prelude @@ -88,8 +87,9 @@ insertValidateGenesisDist syncEnv (NetworkName networkName) cfg = do , DB.slotLeaderPoolHashId = Nothing , DB.slotLeaderDescription = "Genesis slot leader" } - bid <- - lift . DB.insertBlock $ + let bid = DB.BlockKey 0 + lift $ + DB.insertBlock bid $ DB.Block { DB.blockHash = configGenesisHash cfg , DB.blockEpochNo = Nothing @@ -187,27 +187,27 @@ insertTxOutsByron :: (Byron.Address, Byron.Lovelace) -> ExceptT SyncNodeError (ReaderT SqlBackend m) () insertTxOutsByron syncEnv disInOut blkId (address, value) = do + let txId = DB.TxKey $ DB.unBlockKey blkId case txHashOfAddress address of Left err -> throwError err Right val -> lift $ do -- Each address/value pair of the initial coin distribution comes from an artifical transaction -- with a hash generated by hashing the address. - txId <- do - DB.insertTx $ - DB.Tx - { DB.txHash = Byron.unTxHash val - , DB.txBlockId = blkId - , DB.txBlockIndex = 0 - , DB.txOutSum = DB.DbLovelace (Byron.unsafeGetLovelace value) - , DB.txFee = DB.DbLovelace 0 - , DB.txDeposit = Just 0 - , DB.txSize = 0 -- Genesis distribution address to not have a size. - , DB.txInvalidHereafter = Nothing - , DB.txInvalidBefore = Nothing - , DB.txValidContract = True - , DB.txScriptSize = 0 - , DB.txTreasuryDonation = DB.DbLovelace 0 - } + DB.insertTx txId $ + DB.Tx + { DB.txHash = Byron.unTxHash val + , DB.txBlockId = blkId + , DB.txBlockIndex = 0 + , DB.txOutSum = DB.DbLovelace (Byron.unsafeGetLovelace value) + , DB.txFee = DB.DbLovelace 0 + , DB.txDeposit = Just 0 + , DB.txSize = 0 -- Genesis distribution address to not have a size. + , DB.txInvalidHereafter = Nothing + , DB.txInvalidBefore = Nothing + , DB.txValidContract = True + , DB.txScriptSize = 0 + , DB.txTreasuryDonation = DB.DbLovelace 0 + } -- unless disInOut $ case getTxOutTableType syncEnv of diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Insert.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Insert.hs index e9934f6da..3ca395295 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Insert.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Byron/Insert.hs @@ -32,7 +32,6 @@ import Cardano.DbSync.Cache ( import Cardano.DbSync.Cache.Epoch (writeEpochBlockDiffToCache) import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..), EpochBlockDiff (..)) import qualified Cardano.DbSync.Era.Byron.Util as Byron -import Cardano.DbSync.Era.Util (liftLookupFail) import Cardano.DbSync.Error import Cardano.DbSync.Types import Cardano.DbSync.Util @@ -82,6 +81,7 @@ insertABOBBoundary syncEnv blk details = do cache = envCache syncEnv -- Will not get called in the OBFT part of the Byron era. pbid <- queryPrevBlockWithCache "insertABOBBoundary" cache (Byron.ebbPrevHash blk) + let blkId = DB.BlockKey $ 1 + DB.unBlockKey pbid let epochNo = unEpochNo $ sdEpochNo details slid <- lift . DB.insertSlotLeader $ @@ -90,8 +90,8 @@ insertABOBBoundary syncEnv blk details = do , DB.slotLeaderPoolHashId = Nothing , DB.slotLeaderDescription = "Epoch boundary slot leader" } - blkId <- - lift . insertBlockAndCache cache $ + lift $ + insertBlockAndCache cache blkId $ DB.Block { DB.blockHash = Byron.unHeaderHash $ Byron.boundaryHashAnnotated blk , DB.blockEpochNo = Just epochNo @@ -146,10 +146,11 @@ insertABlock :: ExceptT SyncNodeError (ReaderT SqlBackend m) () insertABlock syncEnv firstBlockOfEpoch blk details = do pbid <- queryPrevBlockWithCache "insertABlock" cache (Byron.blockPreviousHash blk) + let blkId = DB.BlockKey $ 1 + DB.unBlockKey pbid slid <- lift . DB.insertSlotLeader $ Byron.mkSlotLeader blk let txs = Byron.blockPayload blk - blkId <- - lift . insertBlockAndCache cache $ + lift $ + insertBlockAndCache cache blkId $ DB.Block { DB.blockHash = Byron.blockHash blk , DB.blockEpochNo = Just $ unEpochNo (sdEpochNo details) @@ -238,11 +239,12 @@ insertByronTx :: Word64 -> ExceptT SyncNodeError (ReaderT SqlBackend m) Word64 insertByronTx syncEnv blkId tx blockIndex = do + let txId = DB.TxKey $ DB.unBlockKey blkId disInOut <- liftIO $ getDisableInOutState syncEnv if disInOut then do - txId <- - lift . DB.insertTx $ + lift $ + DB.insertTx txId $ DB.Tx { DB.txHash = Byron.unTxHash $ Crypto.serializeCborHash (Byron.taTx tx) , DB.txBlockId = blkId @@ -282,10 +284,11 @@ insertByronTx' :: Word64 -> ExceptT SyncNodeError (ReaderT SqlBackend m) Word64 insertByronTx' syncEnv blkId tx blockIndex = do + let txId = DB.TxKey $ fromIntegral $ fromIntegral (DB.unBlockId blkId) * 1000 + blockIndex resolvedInputs <- mapM (resolveTxInputs txOutTableType) (toList $ Byron.txInputs (Byron.taTx tx)) valFee <- firstExceptT annotateTx $ ExceptT $ pure (calculateTxFee (Byron.taTx tx) resolvedInputs) - txId <- - lift . DB.insertTx $ + lift $ + DB.insertTx txId $ DB.Tx { DB.txHash = Byron.unTxHash $ Crypto.serializeCborHash (Byron.taTx tx) , DB.txBlockId = blkId diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Rewards.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Rewards.hs index 12410040b..1176bf098 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Rewards.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Rewards.hs @@ -10,6 +10,7 @@ module Cardano.DbSync.Era.Shelley.Generic.Rewards ( RewardRests (..), rewardsCount, rewardsTotalAda, + getRewardsUpdate, ) where import Cardano.Db (Ada, RewardSource (..), word64ToAda) @@ -19,6 +20,19 @@ import Cardano.Prelude import qualified Data.Map.Strict as Map import qualified Data.Set as Set import Ouroboros.Consensus.Cardano.CanHardFork () +import Cardano.Ledger.BaseTypes (strictMaybeToMaybe) +import Cardano.Ledger.Shelley.LedgerState hiding (LedgerState) +import Data.SOP.Strict.NP +import Ouroboros.Consensus.Shelley.Ledger.Ledger +import Ouroboros.Consensus.Shelley.Ledger (ShelleyBlock) +import Ouroboros.Consensus.Config (TopLevelConfig (..)) +import Cardano.Ledger.Crypto (StandardCrypto) +import Ouroboros.Consensus.Cardano.Block (LedgerState (..), EraCrypto) +import Ouroboros.Consensus.Ledger.Extended (ExtLedgerState (..)) +import Ouroboros.Consensus.HardFork.Combinator.PartialConfig +import Ouroboros.Consensus.Shelley.ShelleyHFC +import Ouroboros.Consensus.HardFork.Combinator.Basics +import Ouroboros.Consensus.HardFork.Combinator.AcrossEras data Reward = Reward { rewardSource :: !RewardSource @@ -53,3 +67,34 @@ rewardsTotalAda rwds = . sum . concatMap (map (unCoin . rewardAmount) . Set.toList) $ Map.elems (unRewards rwds) + +getRewardsUpdate :: TopLevelConfig CardanoBlock -> ExtLedgerState CardanoBlock -> Maybe (RewardUpdate StandardCrypto) +getRewardsUpdate cfg els = + case ledgerState els of + LedgerStateByron _ -> Nothing + LedgerStateShelley sls -> genericRewardUpdate cfg sls + LedgerStateAllegra als -> genericRewardUpdate cfg als + LedgerStateMary mls -> genericRewardUpdate cfg mls + LedgerStateAlonzo als -> genericRewardUpdate cfg als + LedgerStateBabbage bls -> genericRewardUpdate cfg bls + LedgerStateConway cls -> genericRewardUpdate cfg cls + +genericRewardUpdate :: + forall era p. + (EraCrypto era ~ StandardCrypto) => + TopLevelConfig CardanoBlock -> + LedgerState (ShelleyBlock p era) -> + Maybe (RewardUpdate StandardCrypto) +genericRewardUpdate cfg lstate = do + pulsing <- strictMaybeToMaybe mPulsing + case pulsing of + Complete _ -> Nothing + Pulsing _ _ -> do + let Identity (rewardUpdate, _) = runReaderT (completeRupd pulsing) globals + Just rewardUpdate + + where + mPulsing = nesRu $ shelleyLedgerState lstate + + globals = case getPerEraLedgerConfig $ hardForkLedgerConfigPerEra $ topLevelConfigLedger cfg of + _ :* wplc :* _ -> shelleyLedgerGlobals $ shelleyLedgerConfig $ unwrapPartialLedgerConfig wplc diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/StakeDist.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/StakeDist.hs index d6964a7cd..de68f6759 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/StakeDist.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/StakeDist.hs @@ -13,24 +13,22 @@ module Cardano.DbSync.Era.Shelley.Generic.StakeDist ( StakeSliceRes (..), StakeSlice (..), getSecurityParameter, - getStakeSlice, + snapShotToList, + getSnapShot, getPoolDistr, ) where import Cardano.DbSync.Types import Cardano.Ledger.Coin (Coin (..)) import qualified Cardano.Ledger.Compactible as Ledger -import Cardano.Ledger.Credential (Credential) import qualified Cardano.Ledger.EpochBoundary as Ledger import Cardano.Ledger.Era (EraCrypto) -import Cardano.Ledger.Keys (KeyHash (..), KeyRole (..)) import qualified Cardano.Ledger.Shelley.LedgerState as Shelley import Cardano.Ledger.Val ((<+>)) import Cardano.Prelude import qualified Data.Map.Strict as Map import Data.VMap (VB, VMap (..), VP) import qualified Data.VMap as VMap -import qualified Data.Vector.Generic as VG import Lens.Micro import Ouroboros.Consensus.Block import Ouroboros.Consensus.Cardano.Block (LedgerState (..), StandardCrypto) @@ -53,9 +51,6 @@ data StakeSlice = StakeSlice } deriving (Eq) -emptySlice :: EpochNo -> StakeSlice -emptySlice epoch = StakeSlice epoch Map.empty - getSecurityParameter :: ConsensusProtocol (BlockProtocol blk) => ProtocolInfo blk -> @@ -70,112 +65,50 @@ getSecurityParameter = maxRollbacks . configSecurityParam . pInfoConfig -- On mainnet, for a value minSliceSize = 2000, it will be used as the actual size of slices -- until the size of delegations grows up to 8.6M, in which case, the size of slices -- will be adjusted. -getStakeSlice :: - ConsensusProtocol (BlockProtocol blk) => - ProtocolInfo blk -> - Word64 -> +getSnapShot :: ExtLedgerState CardanoBlock -> - Bool -> - StakeSliceRes -getStakeSlice pInfo !epochBlockNo els isMigration = + Maybe (Ledger.SnapShot StandardCrypto, EpochNo) +getSnapShot els = case ledgerState els of - LedgerStateByron _ -> NoSlices - LedgerStateShelley sls -> genericStakeSlice pInfo epochBlockNo sls isMigration - LedgerStateAllegra als -> genericStakeSlice pInfo epochBlockNo als isMigration - LedgerStateMary mls -> genericStakeSlice pInfo epochBlockNo mls isMigration - LedgerStateAlonzo als -> genericStakeSlice pInfo epochBlockNo als isMigration - LedgerStateBabbage bls -> genericStakeSlice pInfo epochBlockNo bls isMigration - LedgerStateConway cls -> genericStakeSlice pInfo epochBlockNo cls isMigration - -genericStakeSlice :: - forall era c blk p. - (c ~ StandardCrypto, EraCrypto era ~ c, ConsensusProtocol (BlockProtocol blk)) => - ProtocolInfo blk -> - Word64 -> + LedgerStateByron _ -> Nothing + LedgerStateShelley sls -> Just $ genericSnapShot sls + LedgerStateAllegra als -> Just $ genericSnapShot als + LedgerStateMary mls -> Just $ genericSnapShot mls + LedgerStateAlonzo als -> Just $ genericSnapShot als + LedgerStateBabbage bls -> Just $ genericSnapShot bls + LedgerStateConway cls -> Just $ genericSnapShot cls + +genericSnapShot :: + forall era p. + (EraCrypto era ~ StandardCrypto) => LedgerState (ShelleyBlock p era) -> - Bool -> - StakeSliceRes -genericStakeSlice pInfo epochBlockNo lstate isMigration - | index > delegationsLen = NoSlices - | index == delegationsLen = Slice (emptySlice epoch) True - | index + size > delegationsLen = Slice (mkSlice (delegationsLen - index)) True - | otherwise = Slice (mkSlice size) False + (Ledger.SnapShot StandardCrypto, EpochNo) +genericSnapShot lstate = (stakeSnapshot, epoch) where - epoch :: EpochNo - epoch = EpochNo $ 1 + unEpochNo (Shelley.nesEL (Consensus.shelleyLedgerState lstate)) - - minSliceSize :: Word64 - minSliceSize = 2000 - - -- On mainnet this is 2160 - k :: Word64 - k = getSecurityParameter pInfo - -- We use 'ssStakeMark' here. That means that when these values -- are added to the database, the epoch number where they become active is the current -- epoch plus one. - stakeSnapshot :: Ledger.SnapShot c + stakeSnapshot :: Ledger.SnapShot StandardCrypto stakeSnapshot = Ledger.ssStakeMark . Shelley.esSnapshots . Shelley.nesEs $ Consensus.shelleyLedgerState lstate - delegations :: VMap.KVVector VB VB (Credential 'Staking c, KeyHash 'StakePool c) - delegations = VMap.unVMap $ Ledger.ssDelegations stakeSnapshot - - delegationsLen :: Word64 - delegationsLen = fromIntegral $ VG.length delegations + epoch = EpochNo $ 1 + unEpochNo (Shelley.nesEL (Consensus.shelleyLedgerState lstate)) - stakes :: VMap VB VP (Credential 'Staking c) (Ledger.CompactForm Coin) - stakes = Ledger.unStake $ Ledger.ssStake stakeSnapshot +snapShotToList :: + Ledger.SnapShot StandardCrypto -> + [(StakeCred, (Coin, PoolKeyHash))] +snapShotToList snapShot = + VMap.toList $ + VMap.mapMaybe id $ -- This line removes entries without stake. Should we assume 0 and insert it? + VMap.mapWithKey (\a p -> (,p) <$> lookupStake a) (Ledger.ssDelegations snapShot) + where + stakes :: VMap VB VP StakeCred (Ledger.CompactForm Coin) + stakes = Ledger.unStake $ Ledger.ssStake snapShot - lookupStake :: Credential 'Staking c -> Maybe Coin + lookupStake :: StakeCred -> Maybe Coin lookupStake cred = Ledger.fromCompact <$> VMap.lookup cred stakes - -- This is deterministic for the whole epoch and is the constant size of slices - -- until the data are over. This means the last slice could be of smaller size and slices - -- after that will be empty. - epochSliceSize :: Word64 - epochSliceSize = - max minSliceSize defaultEpochSliceSize - where - -- On mainnet this is 21600 - expectedBlocks :: Word64 - expectedBlocks = 10 * k - - -- This size of slices is enough to cover the whole list, even if only - -- the 20% of the expected blocks appear in an epoch. - defaultEpochSliceSize :: Word64 - defaultEpochSliceSize = 1 + div (delegationsLen * 5) expectedBlocks - - -- The starting index of the data in the delegation vector. - index :: Word64 - index - | isMigration = 0 - | epochBlockNo < k = delegationsLen + 1 -- so it creates the empty Slice. - | otherwise = (epochBlockNo - k) * epochSliceSize - - size :: Word64 - size - | isMigration, epochBlockNo + 1 < k = 0 - | isMigration = (epochBlockNo + 1 - k) * epochSliceSize - | otherwise = epochSliceSize - - mkSlice :: Word64 -> StakeSlice - mkSlice actualSize = - StakeSlice - { sliceEpochNo = epoch - , sliceDistr = distribution - } - where - delegationsSliced :: VMap VB VB (Credential 'Staking c) (KeyHash 'StakePool c) - delegationsSliced = VMap $ VG.slice (fromIntegral index) (fromIntegral actualSize) delegations - - distribution :: Map StakeCred (Coin, PoolKeyHash) - distribution = - VMap.toMap $ - VMap.mapMaybe id $ - VMap.mapWithKey (\a p -> (,p) <$> lookupStake a) delegationsSliced - getPoolDistr :: ExtLedgerState CardanoBlock -> Maybe (Map PoolKeyHash (Coin, Word64), Map PoolKeyHash Natural) diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Alonzo.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Alonzo.hs index 4b551fda4..3feb44bf5 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Alonzo.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Alonzo.hs @@ -35,7 +35,7 @@ import Cardano.DbSync.Era.Shelley.Generic.Tx.Shelley import Cardano.DbSync.Era.Shelley.Generic.Tx.Types import Cardano.DbSync.Era.Shelley.Generic.Util import Cardano.DbSync.Era.Shelley.Generic.Witness -import Cardano.DbSync.Types (DataHash) +import Cardano.DbSync.Types (DataHash, RewAccount) import qualified Cardano.Ledger.Address as Ledger import Cardano.Ledger.Allegra.Scripts (Timelock) import Cardano.Ledger.Alonzo.Scripts (AsIx (..), ExUnits (..), PlutusPurpose, txscriptfee, unPlutusBinary) @@ -199,7 +199,7 @@ resolveRedeemers ioExtraPlutus mprices tx toCert = txBody :: Core.TxBody era txBody = tx ^. Core.bodyTxL - withdrawalsNoRedeemers :: Map (Shelley.RewardAccount StandardCrypto) TxWithdrawal + withdrawalsNoRedeemers :: Map RewAccount TxWithdrawal withdrawalsNoRedeemers = Map.mapWithKey (curry mkTxWithdrawal) $ Shelley.unWithdrawals $ @@ -274,7 +274,7 @@ handleTxInPtr rdmrIx txIn mps = case Map.lookup txIn (rmInps mps) of let gtxIn' = gtxIn {txInRedeemerIndex = Just rdmrIx} in (mps {rmInps = Map.insert txIn gtxIn' (rmInps mps)}, Just (Left gtxIn')) -handleRewardPtr :: Word64 -> Shelley.RewardAccount StandardCrypto -> RedeemerMaps -> (RedeemerMaps, Maybe (Either TxIn ByteString)) +handleRewardPtr :: Word64 -> RewAccount -> RedeemerMaps -> (RedeemerMaps, Maybe (Either TxIn ByteString)) handleRewardPtr rdmrIx rwdAcnt mps = case Map.lookup rwdAcnt (rmWdrl mps) of Nothing -> (mps, Nothing) Just wdrl -> @@ -289,7 +289,7 @@ handleCertPtr rdmrIx dcert mps = f x = x data RedeemerMaps = RedeemerMaps - { rmWdrl :: Map (Shelley.RewardAccount StandardCrypto) TxWithdrawal + { rmWdrl :: Map RewAccount TxWithdrawal , rmCerts :: [(Cert, TxCertificate)] , rmInps :: Map (Ledger.TxIn StandardCrypto) TxIn } @@ -373,7 +373,7 @@ extraKeyWits txBody = Set.map (\(Ledger.KeyHash h) -> Crypto.hashToBytes h) $ txBody ^. Alonzo.reqSignerHashesTxBodyL -scriptHashAcnt :: Shelley.RewardAccount StandardCrypto -> Maybe ByteString +scriptHashAcnt :: RewAccount -> Maybe ByteString scriptHashAcnt rewardAddr = getCredentialScriptHash $ Ledger.raCredential rewardAddr scriptHashCert :: Cert -> Maybe ByteString diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Conway.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Conway.hs index a02e2ab46..a62517900 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Conway.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Conway.hs @@ -14,6 +14,7 @@ import Cardano.DbSync.Era.Shelley.Generic.Tx.Alonzo import qualified Cardano.DbSync.Era.Shelley.Generic.Tx.Babbage as Babbage import Cardano.DbSync.Era.Shelley.Generic.Tx.Shelley import Cardano.DbSync.Era.Shelley.Generic.Tx.Types +import Cardano.DbSync.Types import qualified Cardano.Ledger.Alonzo.Scripts as Alonzo import qualified Cardano.Ledger.Alonzo.Tx as Alonzo import Cardano.Ledger.Babbage.Core as Core hiding (Tx, TxOut) @@ -21,7 +22,6 @@ import Cardano.Ledger.BaseTypes import Cardano.Ledger.Conway.Governance import Cardano.Ledger.Conway.TxBody import qualified Cardano.Ledger.Core as Core -import Cardano.Ledger.TxIn import Cardano.Prelude import qualified Data.Map.Strict as Map import Lens.Micro @@ -77,7 +77,7 @@ fromConwayTx ioExtraPlutus mprices (blkIndex, tx) = txBody :: Core.TxBody StandardConway txBody = tx ^. Core.bodyTxL - txId :: TxId StandardCrypto + txId :: TxIdLedger txId = mkTxId tx outputs :: [TxOut] diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Shelley.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Shelley.hs index d0c8d5f84..098f35546 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Shelley.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Shelley.hs @@ -12,6 +12,7 @@ module Cardano.DbSync.Era.Shelley.Generic.Tx.Shelley ( getTxSize, mkTxIn, fromTxIn, + toTxInKey, mkTxOut, mkTxWithdrawals, mkTxWithdrawal, @@ -35,6 +36,7 @@ import Cardano.DbSync.Era.Shelley.Generic.Script (fromMultiSig) import Cardano.DbSync.Era.Shelley.Generic.Tx.Types import Cardano.DbSync.Era.Shelley.Generic.Util import Cardano.DbSync.Era.Shelley.Generic.Witness +import Cardano.DbSync.Types (RewAccount, TxIdLedger) import Cardano.Ledger.BaseTypes (TxIx (..), strictMaybeToMaybe) import Cardano.Ledger.Coin (Coin (..)) import qualified Cardano.Ledger.Core as Core @@ -122,12 +124,14 @@ mkTxOut txBody = zipWith fromTxOut [0 ..] $ toList (txBody ^. Core.outputsTxBody , txOutDatum = NoDatum -- Shelley does not support plutus data } +toTxInKey :: Ledger.TxIn StandardCrypto -> TxInKey +toTxInKey (Ledger.TxIn txId (TxIx w64)) = TxInKey txId w64 + fromTxIn :: Ledger.TxIn StandardCrypto -> TxIn -fromTxIn (Ledger.TxIn (Ledger.TxId txid) (TxIx w64)) = +fromTxIn txIn = TxIn - { txInIndex = w64 + { txInKey = toTxInKey txIn , txInRedeemerIndex = Nothing - , txInTxId = Ledger.TxId txid } txHashId :: (EraCrypto era ~ StandardCrypto, Core.EraTx era) => Core.Tx era -> ByteString @@ -136,7 +140,7 @@ txHashId = safeHashToByteString . txSafeHash txSafeHash :: (EraCrypto era ~ StandardCrypto, Core.EraTx era) => Core.Tx era -> Ledger.SafeHash StandardCrypto Core.EraIndependentTxBody txSafeHash tx = Ledger.hashAnnotated (tx ^. Core.bodyTxL) -mkTxId :: (EraCrypto era ~ StandardCrypto, Core.EraTx era) => Core.Tx era -> Ledger.TxId StandardCrypto +mkTxId :: (EraCrypto era ~ StandardCrypto, Core.EraTx era) => Core.Tx era -> TxIdLedger mkTxId = Ledger.TxId . txSafeHash txHashFromSafe :: Ledger.SafeHash StandardCrypto Core.EraIndependentTxBody -> ByteString @@ -171,7 +175,7 @@ mkTxWithdrawals :: mkTxWithdrawals bd = map mkTxWithdrawal $ Map.toList $ Shelley.unWithdrawals $ bd ^. Core.withdrawalsTxBodyL -mkTxWithdrawal :: (Shelley.RewardAccount StandardCrypto, Coin) -> TxWithdrawal +mkTxWithdrawal :: (RewAccount, Coin) -> TxWithdrawal mkTxWithdrawal (ra, c) = TxWithdrawal { txwRedeemerIndex = Nothing diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Types.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Types.hs index f4001498d..53d014076 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Types.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Tx/Types.hs @@ -12,6 +12,7 @@ module Cardano.DbSync.Era.Shelley.Generic.Tx.Types ( TxCertificate (..), TxWithdrawal (..), TxIn (..), + TxInKey (..), TxOut (..), TxRedeemer (..), TxScript (..), @@ -43,43 +44,41 @@ import Cardano.Ledger.Conway.Scripts import Cardano.Ledger.Conway.TxCert (ConwayTxCert) import Cardano.Ledger.Core (TxBody) import Cardano.Ledger.Mary.Value (AssetName, MultiAsset, PolicyID) -import qualified Cardano.Ledger.Shelley.TxBody as Shelley import Cardano.Ledger.Shelley.TxCert -import qualified Cardano.Ledger.TxIn as Ledger import Cardano.Prelude import Cardano.Slotting.Slot (SlotNo (..)) import Ouroboros.Consensus.Cardano.Block (StandardAlonzo, StandardBabbage, StandardConway, StandardCrypto, StandardShelley) data Tx = Tx - { txHash :: !ByteString - , txLedgerTxId :: !(Ledger.TxId StandardCrypto) - , txBlockIndex :: !Word64 + { txHash :: ByteString + , txLedgerTxId :: TxIdLedger + , txBlockIndex :: Word64 , txCBOR :: ByteString - , txSize :: !Word64 - , txValidContract :: !Bool - , txInputs :: ![TxIn] - , txCollateralInputs :: ![TxIn] - , txReferenceInputs :: ![TxIn] - , txOutputs :: ![TxOut] - , txCollateralOutputs :: ![TxOut] - , txFees :: !(Maybe Coin) -- Nothing means it needs to be computed by inSum - outSum and happens on phase 2 failures. - , txOutSum :: !Coin - , txInvalidBefore :: !(Maybe SlotNo) - , txInvalidHereafter :: !(Maybe SlotNo) - , txWithdrawalSum :: !Coin - , txMetadata :: !(Maybe (Map Word64 TxMetadataValue)) - , txCertificates :: ![TxCertificate] - , txWithdrawals :: ![TxWithdrawal] - , txParamProposal :: ![ParamProposal] - , txMint :: !(MultiAsset StandardCrypto) + , txSize :: Word64 + , txValidContract :: Bool + , txInputs :: [TxIn] + , txCollateralInputs :: [TxIn] + , txReferenceInputs :: [TxIn] + , txOutputs :: [TxOut] + , txCollateralOutputs :: [TxOut] + , txFees :: Maybe Coin -- Nothing means it needs to be computed by inSum - outSum and happens on phase 2 failures. + , txOutSum :: Coin + , txInvalidBefore :: Maybe SlotNo + , txInvalidHereafter :: Maybe SlotNo + , txWithdrawalSum :: Coin + , txMetadata :: Maybe (Map Word64 TxMetadataValue) + , txCertificates :: [TxCertificate] + , txWithdrawals :: [TxWithdrawal] + , txParamProposal :: [ParamProposal] + , txMint :: MultiAsset StandardCrypto , txRedeemer :: [(Word64, TxRedeemer)] , txData :: [PlutusData] , txScriptSizes :: [Word64] -- this contains only the sizes of plutus scripts in witnesses , txScripts :: [TxScript] - , txExtraKeyWitnesses :: ![ByteString] - , txVotingProcedure :: ![(Voter StandardCrypto, [(GovActionId StandardCrypto, VotingProcedure StandardConway)])] - , txProposalProcedure :: ![(GovActionId StandardCrypto, ProposalProcedure StandardConway)] - , txTreasuryDonation :: !Coin + , txExtraKeyWitnesses :: [ByteString] + , txVotingProcedure :: [(Voter StandardCrypto, [(GovActionId StandardCrypto, VotingProcedure StandardConway)])] + , txProposalProcedure :: [(GovActionId StandardCrypto, ProposalProcedure StandardConway)] + , txTreasuryDonation :: Coin } type ShelleyCert = ShelleyTxCert StandardShelley @@ -87,31 +86,36 @@ type ConwayCert = ConwayTxCert StandardConway type Cert = Either ShelleyCert ConwayCert data TxCertificate = TxCertificate - { txcRedeemerIndex :: !(Maybe Word64) - , txcIndex :: !Word16 - , txcCert :: !Cert + { txcRedeemerIndex :: Maybe Word64 + , txcIndex :: Word16 + , txcCert :: Cert } data TxWithdrawal = TxWithdrawal - { txwRedeemerIndex :: !(Maybe Word64) - , txwRewardAccount :: !(Shelley.RewardAccount StandardCrypto) - , txwAmount :: !Coin + { txwRedeemerIndex :: Maybe Word64 + , txwRewardAccount :: RewAccount + , txwAmount :: Coin } +data TxInKey = TxInKey + { txInTxId :: TxIdLedger + , txInIndex :: Word64 + } + deriving (Eq, Show, Ord) + data TxIn = TxIn - { txInIndex :: !Word64 - , txInTxId :: !(Ledger.TxId StandardCrypto) - , txInRedeemerIndex :: !(Maybe Word64) -- This only has a meaning for Alonzo. + { txInKey :: TxInKey + , txInRedeemerIndex :: Maybe Word64 -- This only has a meaning for Alonzo. } deriving (Show) data TxOut = TxOut - { txOutIndex :: !Word64 - , txOutAddress :: !(Ledger.Addr StandardCrypto) - , txOutAdaValue :: !Coin - , txOutMaValue :: !(Map (PolicyID StandardCrypto) (Map AssetName Integer)) + { txOutIndex :: Word64 + , txOutAddress :: Ledger.Addr StandardCrypto + , txOutAdaValue :: Coin + , txOutMaValue :: Map (PolicyID StandardCrypto) (Map AssetName Integer) , txOutScript :: Maybe TxScript - , txOutDatum :: !TxOutDatum + , txOutDatum :: TxOutDatum } data TxRedeemer = TxRedeemer @@ -127,7 +131,7 @@ data TxRedeemer = TxRedeemer -- Fields are intentionally left lazy, to avoid transformation if the entry already -- exists in db. data TxScript = TxScript - { txScriptHash :: !ByteString + { txScriptHash :: ByteString , txScriptType :: DB.ScriptType , txScriptPlutusSize :: Maybe Word64 , txScriptJson :: Maybe ByteString @@ -137,7 +141,7 @@ data TxScript = TxScript -- Fields are intentionally left lazy, to avoid transformation if the entry already -- exists in db. data PlutusData = PlutusData - { txDataHash :: !DataHash + { txDataHash :: DataHash , txDataValue :: ByteString -- we turn this into json later. , txDataBytes :: ByteString } @@ -175,7 +179,7 @@ getMaybeDatumHash (Just hsh) = DatumHash hsh sumTxOutCoin :: [TxOut] -> Coin sumTxOutCoin = Coin . sum . map (unCoin . txOutAdaValue) -toTxHash :: TxIn -> ByteString +toTxHash :: TxInKey -> ByteString toTxHash = unTxHash . txInTxId class AlonzoEraTxBody era => DBScriptPurpose era where diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Util.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Util.hs index cc12372e8..3658a5696 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Util.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Generic/Util.hs @@ -126,7 +126,7 @@ partitionMIRTargets = renderAddress :: Ledger.Addr StandardCrypto -> Text renderAddress = serialiseAddress -renderRewardAccount :: Ledger.RewardAccount StandardCrypto -> Text +renderRewardAccount :: RewAccount -> Text renderRewardAccount = serialiseRewardAccount stakingCredHash :: Ledger.Network -> Ledger.StakeCredential era -> ByteString diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Genesis.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Genesis.hs index 0cc49a38e..318fd069b 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Genesis.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Genesis.hs @@ -11,19 +11,18 @@ module Cardano.DbSync.Era.Shelley.Genesis ( insertValidateGenesisDist, ) where -import Cardano.BM.Trace (Trace, logError, logInfo) +import Cardano.BM.Trace (logError, logInfo) import qualified Cardano.Db as DB import qualified Cardano.Db.Schema.Core.TxOut as C import qualified Cardano.Db.Schema.Variant.TxOut as V import Cardano.DbSync.Api import Cardano.DbSync.Api.Types (InsertOptions (..), SyncEnv (..), SyncOptions (..)) import Cardano.DbSync.Cache (insertAddressUsingCache, tryUpdateCacheTx) -import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..), useNoCache) +import Cardano.DbSync.Cache.Types (CacheAction (..)) import qualified Cardano.DbSync.Era.Shelley.Generic.Util as Generic import Cardano.DbSync.Era.Universal.Insert.Certificate (insertDelegation, insertStakeRegistration) -import Cardano.DbSync.Era.Universal.Insert.Other (insertStakeAddressRefIfMissing) +import Cardano.DbSync.Era.Universal.Insert.Other (queryOrInsertStakeRef) import Cardano.DbSync.Era.Universal.Insert.Pool (insertPoolRegister) -import Cardano.DbSync.Era.Util (liftLookupFail) import Cardano.DbSync.Error import Cardano.DbSync.Util import Cardano.Ledger.Address (serialiseAddr) @@ -130,8 +129,9 @@ insertValidateGenesisDist syncEnv networkName cfg shelleyInitiation = do -- tree format, which is unavoidable. pid <- lift DB.queryLatestBlockId liftIO $ logInfo tracer $ textShow pid - bid <- - lift . DB.insertBlock $ + let bid = DB.BlockKey 1 + lift $ + DB.insertBlock bid $ DB.Block { DB.blockHash = configGenesisHash cfg , DB.blockEpochNo = Nothing @@ -153,12 +153,13 @@ insertValidateGenesisDist syncEnv networkName cfg shelleyInitiation = do } disInOut <- liftIO $ getDisableInOutState syncEnv unless disInOut $ do - lift $ mapM_ (insertTxOuts syncEnv tracer bid) $ genesisUtxOs cfg + lift $ mapM_ (insertTxOuts syncEnv bid) $ zip [0 ..] (genesisUtxOs cfg) liftIO . logInfo tracer $ "Initial genesis distribution populated. Hash " <> renderByteArray (configGenesisHash cfg) - when hasStakes $ - insertStaking tracer useNoCache bid cfg + when hasStakes $ do + let stakeTxId = DB.TxKey $ fromIntegral $ length (genesisUtxOs cfg) + insertStaking syncEnv bid stakeTxId cfg -- | Validate that the initial Genesis distribution in the DB matches the Genesis data. validateGenesisDistribution :: @@ -223,32 +224,30 @@ validateGenesisDistribution syncEnv prunes networkName cfg bid expectedTxCount = insertTxOuts :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> - Trace IO Text -> DB.BlockId -> - (TxIn StandardCrypto, ShelleyTxOut StandardShelley) -> + (Int, (TxIn StandardCrypto, ShelleyTxOut StandardShelley)) -> ReaderT SqlBackend m () -insertTxOuts syncEnv trce blkId (TxIn txInId _, txOut) = do +insertTxOuts syncEnv blkId (txIndex, (TxIn txInId _, txOut)) = do -- Each address/value pair of the initial coin distribution comes from an artifical transaction -- with a hash generated by hashing the address. - txId <- - DB.insertTx $ - DB.Tx - { DB.txHash = Generic.unTxHash txInId - , DB.txBlockId = blkId - , DB.txBlockIndex = 0 - , DB.txOutSum = Generic.coinToDbLovelace (txOut ^. Core.valueTxOutL) - , DB.txFee = DB.DbLovelace 0 - , DB.txDeposit = Just 0 - , DB.txSize = 0 -- Genesis distribution address to not have a size. - , DB.txInvalidHereafter = Nothing - , DB.txInvalidBefore = Nothing - , DB.txValidContract = True - , DB.txScriptSize = 0 - , DB.txTreasuryDonation = DB.DbLovelace 0 - } + DB.insertTx txId $ + DB.Tx + { DB.txHash = Generic.unTxHash txInId + , DB.txBlockId = blkId + , DB.txBlockIndex = 0 + , DB.txOutSum = Generic.coinToDbLovelace (txOut ^. Core.valueTxOutL) + , DB.txFee = DB.DbLovelace 0 + , DB.txDeposit = Just 0 + , DB.txSize = 0 -- Genesis distribution address to not have a size. + , DB.txInvalidHereafter = Nothing + , DB.txInvalidBefore = Nothing + , DB.txValidContract = True + , DB.txScriptSize = 0 + , DB.txTreasuryDonation = DB.DbLovelace 0 + } tryUpdateCacheTx (envCache syncEnv) txInId txId - _ <- insertStakeAddressRefIfMissing trce useNoCache (txOut ^. Core.addrTxOutL) + _ <- queryOrInsertStakeRef syncEnv (txOut ^. Core.addrTxOutL) case ioTxOutTableType . soptInsertOptions $ envOptions syncEnv of DB.TxOutCore -> void . DB.insertTxOut $ @@ -268,15 +267,16 @@ insertTxOuts syncEnv trce blkId (TxIn txInId _, txOut) = do } DB.TxOutVariantAddress -> do addrDetailId <- insertAddressUsingCache cache UpdateCache addrRaw vAddress - void . DB.insertTxOut $ DB.VTxOutW (makeVTxOut addrDetailId txId) Nothing + void . DB.insertTxOut $ DB.VTxOutW (makeVTxOut addrDetailId) Nothing where + txId = DB.TxKey $ fromIntegral txIndex + 1000 * DB.unBlockKey blkId addr = txOut ^. Core.addrTxOutL cache = envCache syncEnv hasScript = maybe False Generic.hasCredScript (Generic.getPaymentCred addr) addrRaw = serialiseAddr addr - makeVTxOut :: V.AddressId -> DB.TxId -> V.TxOut - makeVTxOut addrDetailId txId = + makeVTxOut :: V.AddressId -> V.TxOut + makeVTxOut addrDetailId = V.TxOut { V.txOutAddressId = addrDetailId , V.txOutConsumedByTxId = Nothing @@ -302,40 +302,39 @@ insertTxOuts syncEnv trce blkId (TxIn txInId _, txOut) = do -- Insert pools and delegations coming from Genesis. insertStaking :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> DB.BlockId -> + DB.TxId -> ShelleyGenesis StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertStaking tracer cache blkId genesis = do +insertStaking syncEnv blkId txId genesis = do -- All Genesis staking comes from an artifical transaction -- with a hash generated by hashing the address. - txId <- - lift $ - DB.insertTx $ - DB.Tx - { DB.txHash = configGenesisStakingHash - , DB.txBlockId = blkId - , DB.txBlockIndex = 0 - , DB.txOutSum = DB.DbLovelace 0 - , DB.txFee = DB.DbLovelace 0 - , DB.txDeposit = Just 0 - , DB.txSize = 0 - , DB.txInvalidHereafter = Nothing - , DB.txInvalidBefore = Nothing - , DB.txValidContract = True - , DB.txScriptSize = 0 - , DB.txTreasuryDonation = DB.DbLovelace 0 - } + lift $ + DB.insertTx txId $ + DB.Tx + { DB.txHash = configGenesisStakingHash + , DB.txBlockId = blkId + , DB.txBlockIndex = 0 + , DB.txOutSum = DB.DbLovelace 0 + , DB.txFee = DB.DbLovelace 0 + , DB.txDeposit = Just 0 + , DB.txSize = 0 + , DB.txInvalidHereafter = Nothing + , DB.txInvalidBefore = Nothing + , DB.txValidContract = True + , DB.txScriptSize = 0 + , DB.txTreasuryDonation = DB.DbLovelace 0 + } let params = zip [0 ..] $ ListMap.elems $ sgsPools $ sgStaking genesis let network = sgNetworkId genesis -- TODO: add initial deposits for genesis pools. - forM_ params $ uncurry (insertPoolRegister tracer useNoCache (const False) Nothing network (EpochNo 0) blkId txId) + forM_ params $ uncurry (insertPoolRegister syncEnv (const False) Nothing (EpochNo 0) blkId txId) let stakes = zip [0 ..] $ ListMap.toList (sgsStake $ sgStaking genesis) forM_ stakes $ \(n, (keyStaking, keyPool)) -> do -- TODO: add initial deposits for genesis stake keys. - insertStakeRegistration tracer cache (EpochNo 0) Nothing txId (2 * n) (Generic.annotateStakingCred network (KeyHashObj keyStaking)) - insertDelegation tracer cache network (EpochNo 0) 0 txId (2 * n + 1) Nothing (KeyHashObj keyStaking) keyPool + insertStakeRegistration syncEnv (EpochNo 0) Nothing txId (2 * n) (Generic.annotateStakingCred network (KeyHashObj keyStaking)) + insertDelegation syncEnv (EpochNo 0) 0 txId (2 * n + 1) Nothing (KeyHashObj keyStaking) keyPool -- ----------------------------------------------------------------------------- diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Query.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Query.hs index 51ad9952b..e5a902bd0 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Query.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Shelley/Query.hs @@ -6,7 +6,6 @@ module Cardano.DbSync.Era.Shelley.Query ( resolveStakeAddress, resolveInputTxOutId, - resolveInputValue, resolveInputTxOutIdValue, queryResolveInputCredentials, ) where @@ -26,18 +25,14 @@ import Database.Esqueleto.Experimental ( resolveStakeAddress :: MonadIO m => ByteString -> ReaderT SqlBackend m (Either LookupFail StakeAddressId) resolveStakeAddress addr = queryStakeAddress addr renderByteArray -resolveInputTxOutId :: MonadIO m => SyncEnv -> Generic.TxIn -> ReaderT SqlBackend m (Either LookupFail (TxId, TxOutIdW)) +resolveInputTxOutId :: MonadIO m => SyncEnv -> Generic.TxInKey -> ReaderT SqlBackend m (Either LookupFail (TxId, TxOutIdW)) resolveInputTxOutId syncEnv txIn = queryTxOutId (getTxOutTableType syncEnv) (Generic.toTxHash txIn, fromIntegral (Generic.txInIndex txIn)) -resolveInputValue :: MonadIO m => SyncEnv -> Generic.TxIn -> ReaderT SqlBackend m (Either LookupFail (TxId, DbLovelace)) -resolveInputValue syncEnv txIn = - queryTxOutValue (getTxOutTableType syncEnv) (Generic.toTxHash txIn, fromIntegral (Generic.txInIndex txIn)) - -resolveInputTxOutIdValue :: MonadIO m => SyncEnv -> Generic.TxIn -> ReaderT SqlBackend m (Either LookupFail (TxId, TxOutIdW, DbLovelace)) +resolveInputTxOutIdValue :: MonadIO m => SyncEnv -> Generic.TxInKey -> ReaderT SqlBackend m (Either LookupFail (TxId, TxOutIdW, DbLovelace)) resolveInputTxOutIdValue syncEnv txIn = queryTxOutIdValue (getTxOutTableType syncEnv) (Generic.toTxHash txIn, fromIntegral (Generic.txInIndex txIn)) -queryResolveInputCredentials :: MonadIO m => SyncEnv -> Generic.TxIn -> ReaderT SqlBackend m (Either LookupFail (Maybe ByteString, Bool)) +queryResolveInputCredentials :: MonadIO m => SyncEnv -> Generic.TxInKey -> ReaderT SqlBackend m (Either LookupFail (Maybe ByteString, Bool)) queryResolveInputCredentials syncEnv txIn = do queryTxOutCredentials (getTxOutTableType syncEnv) (Generic.toTxHash txIn, fromIntegral (Generic.txInIndex txIn)) diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Adjust.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Adjust.hs index 3c2dae95d..7feaf57db 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Adjust.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Adjust.hs @@ -7,16 +7,17 @@ module Cardano.DbSync.Era.Universal.Adjust ( adjustEpochRewards, ) where -import Cardano.BM.Trace (Trace, logInfo) +import Cardano.BM.Trace (logInfo) import qualified Cardano.Db as Db +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types import Cardano.DbSync.Cache ( + queryOrInsertStakeAddress, queryPoolKeyWithCache, - queryStakeAddrWithCache, ) -import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus) +import Cardano.DbSync.Cache.Types (CacheAction (..)) import qualified Cardano.DbSync.Era.Shelley.Generic.Rewards as Generic import Cardano.DbSync.Types (StakeCred) -import Cardano.Ledger.BaseTypes (Network) import Cardano.Prelude hiding (from, groupBy, on) import Cardano.Slotting.Slot (EpochNo (..)) import Control.Monad.Trans.Control (MonadBaseControl) @@ -48,16 +49,14 @@ import Database.Esqueleto.Experimental ( adjustEpochRewards :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - Network -> - CacheStatus -> + SyncEnv -> EpochNo -> Generic.Rewards -> Set StakeCred -> ReaderT SqlBackend m () -adjustEpochRewards trce nw cache epochNo rwds creds = do +adjustEpochRewards syncEnv epochNo rwds creds = do let eraIgnored = Map.toList $ Generic.unRewards rwds - liftIO . logInfo trce $ + liftIO . logInfo (getTrace syncEnv) $ mconcat [ "Removing " , if null eraIgnored then "" else textShow (length eraIgnored) <> " rewards and " @@ -66,23 +65,21 @@ adjustEpochRewards trce nw cache epochNo rwds creds = do ] forM_ eraIgnored $ \(cred, rewards) -> forM_ (Set.toList rewards) $ \rwd -> - deleteReward trce nw cache epochNo (cred, rwd) - crds <- rights <$> forM (Set.toList creds) (queryStakeAddrWithCache trce cache DoNotUpdateCache nw) + deleteReward syncEnv epochNo (cred, rwd) + crds <- forM (Set.toList creds) (queryOrInsertStakeAddress syncEnv DoNotUpdateCache) deleteOrphanedRewards epochNo crds deleteReward :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - Network -> - CacheStatus -> + SyncEnv -> EpochNo -> (StakeCred, Generic.Reward) -> ReaderT SqlBackend m () -deleteReward trce nw cache epochNo (cred, rwd) = do - mAddrId <- queryStakeAddrWithCache trce cache DoNotUpdateCache nw cred - eiPoolId <- queryPoolKeyWithCache cache DoNotUpdateCache (Generic.rewardPool rwd) - case (mAddrId, eiPoolId) of - (Right addrId, Right poolId) -> do +deleteReward syncEnv epochNo (cred, rwd) = do + addrId <- queryOrInsertStakeAddress syncEnv DoNotUpdateCache cred + eiPoolId <- queryPoolKeyWithCache (envCache syncEnv) DoNotUpdateCache (Generic.rewardPool rwd) + case eiPoolId of + Right poolId -> do delete $ do rwdDb <- from $ table @Db.Reward where_ (rwdDb ^. Db.RewardAddrId ==. val addrId) diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Block.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Block.hs index 0a30009e8..7ab5d8b4c 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Block.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Block.hs @@ -8,99 +8,99 @@ {-# LANGUAGE NoImplicitPrelude #-} module Cardano.DbSync.Era.Universal.Block ( + prepareBlock, insertBlockUniversal, ) where -import Cardano.BM.Trace (Trace, logDebug, logInfo) +import Cardano.BM.Trace (logDebug, logInfo) import qualified Cardano.Db as DB import Cardano.DbSync.Api -import Cardano.DbSync.Api.Types (InsertOptions (..), SyncEnv (..), SyncOptions (..)) +import Cardano.DbSync.Api.Types (InsertOptions (..), SyncEnv (..)) import Cardano.DbSync.Cache ( insertBlockAndCache, optimiseCaches, queryPoolKeyWithCache, queryPrevBlockWithCache, ) -import Cardano.DbSync.Cache.Epoch (writeEpochBlockDiffToCache) -import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..), EpochBlockDiff (..)) - +import Cardano.DbSync.Cache.Types (CacheAction (..)) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Universal.Epoch -import Cardano.DbSync.Era.Universal.Insert.Grouped -import Cardano.DbSync.Era.Universal.Insert.Tx (insertTx) -import Cardano.DbSync.Era.Util (liftLookupFail) import Cardano.DbSync.Error import Cardano.DbSync.Ledger.Types (ApplyResult (..)) import Cardano.DbSync.OffChain import Cardano.DbSync.Types import Cardano.DbSync.Util - -import Cardano.DbSync.Era.Universal.Insert.Pool (IsPoolMember) import Cardano.Ledger.BaseTypes import qualified Cardano.Ledger.BaseTypes as Ledger import Cardano.Ledger.Keys import Cardano.Prelude - import Control.Monad.Trans.Control (MonadBaseControl) -import Control.Monad.Trans.Except.Extra (newExceptT) import Data.Either.Extra (eitherToMaybe) +import Data.Time.Calendar (Day (..)) +import Data.Time.Clock (UTCTime (..)) import Database.Persist.Sql (SqlBackend) -------------------------------------------------------------------------------------------- -- Insert a universal Block. -- This is the entry point for inserting a block into the database, used for all eras appart from Byron. -------------------------------------------------------------------------------------------- + +prepareBlock :: + (MonadBaseControl IO m, MonadIO m) => + SyncEnv -> + Generic.Block -> + ExceptT SyncNodeError (ReaderT SqlBackend m) DB.Block +prepareBlock syncEnv blk = do + pbid <- case Generic.blkPreviousHash blk of + Nothing -> liftLookupFail (renderErrorMessage (Generic.blkEra blk)) DB.queryGenesis -- this is for networks that fork from Byron on epoch 0. + Just pHash -> queryPrevBlockWithCache (renderErrorMessage (Generic.blkEra blk)) cache pHash + mPhid <- lift $ queryPoolKeyWithCache cache UpdateCache $ coerceKeyRole $ Generic.blkSlotLeader blk + slid <- lift . DB.insertSlotLeader $ Generic.mkSlotLeader (ioShelley iopts) (Generic.unKeyHashRaw $ Generic.blkSlotLeader blk) (eitherToMaybe mPhid) + pure $ + DB.Block + { DB.blockHash = Generic.blkHash blk + , DB.blockEpochNo = Nothing + , DB.blockSlotNo = Just $ unSlotNo (Generic.blkSlotNo blk) + , DB.blockEpochSlotNo = Nothing + , DB.blockBlockNo = Just $ unBlockNo (Generic.blkBlockNo blk) + , DB.blockPreviousId = Just pbid + , DB.blockSlotLeaderId = slid + , DB.blockSize = Generic.blkSize blk + , DB.blockTime = dummyUTCTime + , DB.blockTxCount = fromIntegral $ length (Generic.blkTxs blk) + , DB.blockProtoMajor = getVersion $ Ledger.pvMajor (Generic.blkProto blk) + , DB.blockProtoMinor = fromIntegral $ Ledger.pvMinor (Generic.blkProto blk) + , -- Shelley specific + DB.blockVrfKey = Just $ Generic.blkVrfKey blk + , DB.blockOpCert = Just $ Generic.blkOpCert blk + , DB.blockOpCertCounter = Just $ Generic.blkOpCertCounter blk + } + where + dummyUTCTime = UTCTime (ModifiedJulianDay 0) 0 + cache = envCache syncEnv + iopts = getInsertOptions syncEnv + insertBlockUniversal :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> -- | Should log - Bool -> - -- | Within two minutes - Bool -> - -- | Within half hour - Bool -> + DB.BlockId -> Generic.Block -> - SlotDetails -> - IsPoolMember -> + DB.Block -> ApplyResult -> - ReaderT SqlBackend m (Either SyncNodeError ()) -insertBlockUniversal syncEnv shouldLog withinTwoMins withinHalfHour blk details isMember applyResult = do - -- if we're syncing within 2 mins of the tip, we optimise the caches. - when (isSyncedWithintwoMinutes details) $ optimiseCaches cache - runExceptT $ do - pbid <- case Generic.blkPreviousHash blk of - Nothing -> liftLookupFail (renderErrorMessage (Generic.blkEra blk)) DB.queryGenesis -- this is for networks that fork from Byron on epoch 0. - Just pHash -> queryPrevBlockWithCache (renderErrorMessage (Generic.blkEra blk)) cache pHash - mPhid <- lift $ queryPoolKeyWithCache cache UpdateCache $ coerceKeyRole $ Generic.blkSlotLeader blk - let epochNo = sdEpochNo details - - slid <- lift . DB.insertSlotLeader $ Generic.mkSlotLeader (ioShelley iopts) (Generic.unKeyHashRaw $ Generic.blkSlotLeader blk) (eitherToMaybe mPhid) - blkId <- - lift . insertBlockAndCache cache $ - DB.Block - { DB.blockHash = Generic.blkHash blk - , DB.blockEpochNo = Just $ unEpochNo epochNo - , DB.blockSlotNo = Just $ unSlotNo (Generic.blkSlotNo blk) - , DB.blockEpochSlotNo = Just $ unEpochSlot (sdEpochSlot details) - , DB.blockBlockNo = Just $ unBlockNo (Generic.blkBlockNo blk) - , DB.blockPreviousId = Just pbid - , DB.blockSlotLeaderId = slid - , DB.blockSize = Generic.blkSize blk - , DB.blockTime = sdSlotTime details - , DB.blockTxCount = fromIntegral $ length (Generic.blkTxs blk) - , DB.blockProtoMajor = getVersion $ Ledger.pvMajor (Generic.blkProto blk) - , DB.blockProtoMinor = fromIntegral $ Ledger.pvMinor (Generic.blkProto blk) - , -- Shelley specific - DB.blockVrfKey = Just $ Generic.blkVrfKey blk - , DB.blockOpCert = Just $ Generic.blkOpCert blk - , DB.blockOpCertCounter = Just $ Generic.blkOpCertCounter blk - } - - let zippedTx = zip [0 ..] (Generic.blkTxs blk) - let txInserter = insertTx syncEnv isMember blkId (sdEpochNo details) (Generic.blkSlotNo blk) applyResult - blockGroupedData <- foldM (\gp (idx, tx) -> txInserter idx tx gp) mempty zippedTx - minIds <- insertBlockGroupedData syncEnv blockGroupedData - + Bool -> + ExceptT SyncNodeError (ReaderT SqlBackend m) () +insertBlockUniversal syncEnv blkId genericBlock blk applyResult shouldLog = do + when (isSyncedWithintwoMinutes details) $ lift $ optimiseCaches $ envCache syncEnv + lift $ + insertBlockAndCache cache blkId $ + blk + { DB.blockEpochNo = Just $ unEpochNo epochNo + , DB.blockEpochSlotNo = Just $ unEpochSlot $ sdEpochSlot details + , DB.blockTime = sdSlotTime details + } + + {-} TODO -- now that we've inserted the Block and all it's txs lets cache what we'll need -- when we later update the epoch values. -- if have --dissable-epoch && --dissable-cache then no need to cache data. @@ -112,77 +112,81 @@ insertBlockUniversal syncEnv shouldLog withinTwoMins withinHalfHour blk details { ebdBlockId = blkId , ebdTime = sdSlotTime details , ebdFees = groupedTxFees blockGroupedData - , ebdEpochNo = unEpochNo (sdEpochNo details) + , ebdEpochNo = unEpochNo epochNo , ebdOutSum = fromIntegral $ groupedTxOutSum blockGroupedData - , ebdTxCount = fromIntegral $ length (Generic.blkTxs blk) + , ebdTxCount = fromIntegral $ length (Generic.blkTxs genericBlock) } when withinHalfHour $ insertReverseIndex blkId minIds + -} + liftIO (logBlockMaybe >> logEpochProgressMaybe) - liftIO $ do - let epoch = unEpochNo epochNo - slotWithinEpoch = unEpochSlot (sdEpochSlot details) - - when (withinTwoMins && slotWithinEpoch /= 0 && unBlockNo (Generic.blkBlockNo blk) `mod` 20 == 0) $ do - logInfo tracer $ - mconcat - [ renderInsertName (Generic.blkEra blk) - , ": continuing epoch " - , textShow epoch - , " (slot " - , textShow slotWithinEpoch - , "/" - , textShow (unEpochSize $ sdEpochSize details) - , ")" - ] - logger tracer $ - mconcat - [ renderInsertName (Generic.blkEra blk) - , ": epoch " - , textShow (unEpochNo epochNo) - , ", slot " - , textShow (unSlotNo $ Generic.blkSlotNo blk) - , ", block " - , textShow (unBlockNo $ Generic.blkBlockNo blk) - , ", hash " - , renderByteArray (Generic.blkHash blk) - ] - - whenStrictJust (apNewEpoch applyResult) $ \newEpoch -> do - insertOnNewEpoch syncEnv blkId (Generic.blkSlotNo blk) epochNo newEpoch - - insertStakeSlice syncEnv $ apStakeSlice applyResult - - when (ioGov iopts && (withinHalfHour || unBlockNo (Generic.blkBlockNo blk) `mod` 10000 == 0)) - . lift - $ insertOffChainVoteResults tracer (envOffChainVoteResultQueue syncEnv) - - when (ioOffChainPoolData iopts && (withinHalfHour || unBlockNo (Generic.blkBlockNo blk) `mod` 10000 == 0)) - . lift - $ insertOffChainPoolResults tracer (envOffChainPoolResultQueue syncEnv) + whenStrictJust (apNewEpoch applyResult) $ \newEpoch -> do + insertOnNewEpoch syncEnv blkId (Generic.blkSlotNo genericBlock) epochNo newEpoch + + when (ioGov iopts && (withinHalfHour || unBlockNo (Generic.blkBlockNo genericBlock) `mod` 10000 == 0)) + . lift + $ insertOffChainVoteResults tracer (envOffChainVoteResultQueue syncEnv) + + when (ioOffChainPoolData iopts && (withinHalfHour || unBlockNo (Generic.blkBlockNo genericBlock) `mod` 10000 == 0)) + . lift + $ insertOffChainPoolResults tracer (envOffChainPoolResultQueue syncEnv) where iopts = getInsertOptions syncEnv + details = apSlotDetails applyResult + epochNo = sdEpochNo details + epoch = unEpochNo epochNo + slotWithinEpoch = unEpochSlot (sdEpochSlot details) + withinTwoMins = isSyncedWithinSeconds details 120 == SyncFollowing + withinHalfHour = isSyncedWithinSeconds details 1800 == SyncFollowing - logger :: Trace IO a -> a -> IO () - logger - | shouldLog = logInfo - | withinTwoMins = logInfo - | unBlockNo (Generic.blkBlockNo blk) `mod` 5000 == 0 = logInfo - | otherwise = logDebug + tracer = getTrace syncEnv + cache = envCache syncEnv + + logBlockMaybe :: IO () + logBlockMaybe + | shouldLog = logInfo tracer logBlockMsg + | withinTwoMins = logInfo tracer logBlockMsg + | unBlockNo (Generic.blkBlockNo genericBlock) `mod` 5000 == 0 = logInfo tracer logBlockMsg + | otherwise = logDebug tracer logBlockMsg + + logEpochProgressMaybe :: IO () + logEpochProgressMaybe = + when (withinTwoMins && slotWithinEpoch /= 0 && unBlockNo (Generic.blkBlockNo genericBlock) `mod` 20 == 0) $ do + logInfo tracer logContinueMsg + + logBlockMsg = + mconcat + [ renderInsertName (Generic.blkEra genericBlock) + , ": epoch " + , textShow (unEpochNo epochNo) + , ", slot " + , textShow (unSlotNo $ Generic.blkSlotNo genericBlock) + , ", block " + , textShow (unBlockNo $ Generic.blkBlockNo genericBlock) + , ", hash " + , renderByteArray (Generic.blkHash genericBlock) + ] + + logContinueMsg = + mconcat + [ renderInsertName (Generic.blkEra genericBlock) + , ": continuing epoch " + , textShow epoch + , " (slot " + , textShow slotWithinEpoch + , "/" + , textShow (unEpochSize $ sdEpochSize details) + , ")" + ] renderInsertName :: Generic.BlockEra -> Text renderInsertName eraText = mconcat ["Insert ", textShow eraText, " Block"] - renderErrorMessage :: Generic.BlockEra -> Text - renderErrorMessage eraText = - case eraText of - Generic.Shelley -> "insertBlockForEra" - other -> mconcat ["insertBlockForEra(", textShow other, ")"] - - tracer :: Trace IO Text - tracer = getTrace syncEnv - - cache :: CacheStatus - cache = envCache syncEnv +renderErrorMessage :: Generic.BlockEra -> Text +renderErrorMessage eraText = + case eraText of + Generic.Shelley -> "insertBlockForEra" + other -> mconcat ["insertBlockForEra(", textShow other, ")"] diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Epoch.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Epoch.hs index cc1f86205..09c683843 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Epoch.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Epoch.hs @@ -15,6 +15,7 @@ module Cardano.DbSync.Era.Universal.Epoch ( insertRewards, hasNewEpochEvent, hasEpochStartEvent, + insertEpochStake, insertRewardRests, insertProposalRefunds, insertPoolDepositRefunds, @@ -27,7 +28,7 @@ import qualified Cardano.Db as DB import Cardano.DbSync.Api import Cardano.DbSync.Api.Types (InsertOptions (..), SyncEnv (..)) import Cardano.DbSync.Cache (queryOrInsertStakeAddress, queryPoolKeyOrInsert) -import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus) +import Cardano.DbSync.Cache.Types (CacheAction (..)) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Universal.Insert.Certificate (insertPots) import Cardano.DbSync.Era.Universal.Insert.GovAction (insertCostModel, insertDrepDistr, insertUpdateEnacted, updateExpired, updateRatified) @@ -38,7 +39,7 @@ import Cardano.DbSync.Types import Cardano.DbSync.Util (whenDefault, whenStrictJust, whenStrictJustDefault) import Cardano.DbSync.Util.Constraint (constraintNameEpochStake, constraintNameReward) import Cardano.Ledger.Address (RewardAccount (..)) -import Cardano.Ledger.BaseTypes (Network, unEpochInterval) +import Cardano.Ledger.BaseTypes (unEpochInterval) import qualified Cardano.Ledger.BaseTypes as Ledger import Cardano.Ledger.Binary.Version (getVersion) import qualified Cardano.Ledger.Coin as Shelley @@ -200,7 +201,7 @@ insertStakeSlice :: ExceptT SyncNodeError (ReaderT SqlBackend m) () insertStakeSlice _ Generic.NoSlices = pure () insertStakeSlice syncEnv (Generic.Slice slice finalSlice) = do - insertEpochStake syncEnv network (Generic.sliceEpochNo slice) (Map.toList $ Generic.sliceDistr slice) + insertEpochStake syncEnv (Generic.sliceEpochNo slice) (Map.toList $ Generic.sliceDistr slice) when finalSlice $ do lift $ DB.updateSetComplete $ unEpochNo $ Generic.sliceEpochNo slice size <- lift $ DB.queryEpochStakeCount (unEpochNo $ Generic.sliceEpochNo slice) @@ -211,32 +212,28 @@ insertStakeSlice syncEnv (Generic.Slice slice finalSlice) = do tracer :: Trace IO Text tracer = getTrace syncEnv - network :: Network - network = getNetwork syncEnv - +-- This is used by the epoch stake thread. insertEpochStake :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> - Network -> EpochNo -> [(StakeCred, (Shelley.Coin, PoolKeyHash))] -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertEpochStake syncEnv nw epochNo stakeChunk = do - let cache = envCache syncEnv +insertEpochStake syncEnv epochNo stakeChunk = do DB.ManualDbConstraints {..} <- liftIO $ readTVarIO $ envDbConstraints syncEnv - dbStakes <- mapM (mkStake cache) stakeChunk + dbStakes <- mapM mkStake stakeChunk let chunckDbStakes = splittRecordsEvery 100000 dbStakes -- minimising the bulk inserts into hundred thousand chunks to improve performance forM_ chunckDbStakes $ \dbs -> lift $ DB.insertManyEpochStakes dbConstraintEpochStake constraintNameEpochStake dbs where mkStake :: (MonadBaseControl IO m, MonadIO m) => - CacheStatus -> (StakeCred, (Shelley.Coin, PoolKeyHash)) -> ExceptT SyncNodeError (ReaderT SqlBackend m) DB.EpochStake - mkStake cache (saddr, (coin, pool)) = do - saId <- lift $ queryOrInsertStakeAddress trce cache UpdateCacheStrong nw saddr - poolId <- lift $ queryPoolKeyOrInsert "insertEpochStake" trce cache UpdateCache (ioShelley iopts) pool + mkStake (saddr, (coin, pool)) = do + -- TODO check that not updating the cache here is not an issue. + saId <- lift $ queryOrInsertStakeAddress syncEnv DoNotUpdateCache saddr + poolId <- lift $ queryPoolKeyOrInsert syncEnv "insertEpochStake" DoNotUpdateCache (ioShelley iopts) pool pure $ DB.EpochStake { DB.epochStakeAddrId = saId @@ -245,19 +242,16 @@ insertEpochStake syncEnv nw epochNo stakeChunk = do , DB.epochStakeEpochNo = unEpochNo epochNo -- The epoch where this delegation becomes valid. } - trce = getTrace syncEnv iopts = getInsertOptions syncEnv insertRewards :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> - Network -> EpochNo -> EpochNo -> - CacheStatus -> [(StakeCred, Set Generic.Reward)] -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertRewards syncEnv nw earnedEpoch spendableEpoch cache rewardsChunk = do +insertRewards syncEnv earnedEpoch spendableEpoch rewardsChunk = do DB.ManualDbConstraints {..} <- liftIO $ readTVarIO $ envDbConstraints syncEnv dbRewards <- concatMapM mkRewards rewardsChunk let chunckDbRewards = splittRecordsEvery 100000 dbRewards @@ -269,7 +263,7 @@ insertRewards syncEnv nw earnedEpoch spendableEpoch cache rewardsChunk = do (StakeCred, Set Generic.Reward) -> ExceptT SyncNodeError (ReaderT SqlBackend m) [DB.Reward] mkRewards (saddr, rset) = do - saId <- lift $ queryOrInsertStakeAddress trce cache UpdateCacheStrong nw saddr + saId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong saddr mapM (prepareReward saId) (Set.toList rset) prepareReward :: @@ -294,21 +288,18 @@ insertRewards syncEnv nw earnedEpoch spendableEpoch cache rewardsChunk = do PoolKeyHash -> ExceptT SyncNodeError (ReaderT SqlBackend m) DB.PoolHashId queryPool poolHash = - lift (queryPoolKeyOrInsert "insertRewards" trce cache UpdateCache (ioShelley iopts) poolHash) + lift (queryPoolKeyOrInsert syncEnv "insertRewards" UpdateCache (ioShelley iopts) poolHash) - trce = getTrace syncEnv iopts = getInsertOptions syncEnv insertRewardRests :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - Network -> + SyncEnv -> EpochNo -> EpochNo -> - CacheStatus -> [(StakeCred, Set Generic.RewardRest)] -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertRewardRests trce nw earnedEpoch spendableEpoch cache rewardsChunk = do +insertRewardRests syncEnv earnedEpoch spendableEpoch rewardsChunk = do dbRewards <- concatMapM mkRewards rewardsChunk let chunckDbRewards = splittRecordsEvery 100000 dbRewards -- minimising the bulk inserts into hundred thousand chunks to improve performance @@ -319,7 +310,7 @@ insertRewardRests trce nw earnedEpoch spendableEpoch cache rewardsChunk = do (StakeCred, Set Generic.RewardRest) -> ExceptT SyncNodeError (ReaderT SqlBackend m) [DB.RewardRest] mkRewards (saddr, rset) = do - saId <- lift $ queryOrInsertStakeAddress trce cache UpdateCacheStrong nw saddr + saId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong saddr pure $ map (prepareReward saId) (Set.toList rset) prepareReward :: @@ -337,14 +328,12 @@ insertRewardRests trce nw earnedEpoch spendableEpoch cache rewardsChunk = do insertProposalRefunds :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - Network -> + SyncEnv -> EpochNo -> EpochNo -> - CacheStatus -> [GovActionRefunded] -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertProposalRefunds trce nw earnedEpoch spendableEpoch cache refunds = do +insertProposalRefunds syncEnv earnedEpoch spendableEpoch refunds = do dbRewards <- mapM mkReward refunds lift $ DB.insertManyRewardRests dbRewards where @@ -353,7 +342,7 @@ insertProposalRefunds trce nw earnedEpoch spendableEpoch cache refunds = do GovActionRefunded -> ExceptT SyncNodeError (ReaderT SqlBackend m) DB.RewardRest mkReward refund = do - saId <- lift $ queryOrInsertStakeAddress trce cache UpdateCacheStrong nw (raCredential $ garReturnAddr refund) + saId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong (raCredential $ garReturnAddr refund) pure $ DB.RewardRest { DB.rewardRestAddrId = saId @@ -378,12 +367,11 @@ insertPoolDepositRefunds :: Generic.Rewards -> ExceptT SyncNodeError (ReaderT SqlBackend m) () insertPoolDepositRefunds syncEnv epochNo refunds = do - insertRewards syncEnv nw epochNo epochNo (envCache syncEnv) (Map.toList rwds) + insertRewards syncEnv epochNo epochNo (Map.toList rwds) liftIO . logInfo tracer $ "Inserted " <> show (Generic.rewardsCount refunds) <> " deposit refund rewards" where tracer = getTrace syncEnv rwds = Generic.unRewards refunds - nw = getNetwork syncEnv sumRewardTotal :: Map StakeCred (Set Generic.Reward) -> Shelley.Coin sumRewardTotal = @@ -406,7 +394,7 @@ insertPoolStats syncEnv epochNo mp = do where preparePoolStat :: (PoolKeyHash, Generic.PoolStats) -> ReaderT SqlBackend m DB.PoolStat preparePoolStat (pkh, ps) = do - poolId <- queryPoolKeyOrInsert "insertPoolStats" trce cache UpdateCache True pkh + poolId <- queryPoolKeyOrInsert syncEnv "insertPoolStats" UpdateCache True pkh pure DB.PoolStat { DB.poolStatPoolHashId = poolId @@ -416,6 +404,3 @@ insertPoolStats syncEnv epochNo mp = do , DB.poolStatStake = fromIntegral . Shelley.unCoin $ Generic.stake ps , DB.poolStatVotingPower = fromIntegral . Shelley.unCoin <$> Generic.votingPower ps } - - cache = envCache syncEnv - trce = getTrace syncEnv diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Certificate.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Certificate.hs index 46aac293a..2cc9aa6f0 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Certificate.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Certificate.hs @@ -22,7 +22,7 @@ module Cardano.DbSync.Era.Universal.Insert.Certificate ( mkAdaPots, ) where -import Cardano.BM.Trace (Trace, logWarning) +import Cardano.BM.Trace (logWarning) import qualified Cardano.Db as DB import Cardano.DbSync.Api import Cardano.DbSync.Api.Types (InsertOptions (..), SyncEnv (..)) @@ -31,7 +31,7 @@ import Cardano.DbSync.Cache ( queryOrInsertStakeAddress, queryPoolKeyOrInsert, ) -import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..)) +import Cardano.DbSync.Cache.Types (CacheAction (..)) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Universal.Insert.GovAction (insertCommitteeHash, insertCredDrepHash, insertDrep, insertVotingAnchor) import Cardano.DbSync.Era.Universal.Insert.Pool (IsPoolMember, insertPoolCert) @@ -48,7 +48,6 @@ import qualified Cardano.Ledger.Credential as Ledger import Cardano.Ledger.Keys import qualified Cardano.Ledger.Keys as Ledger import qualified Cardano.Ledger.Shelley.AdaPots as Shelley -import qualified Cardano.Ledger.Shelley.TxBody as Shelley import Cardano.Ledger.Shelley.TxCert import Cardano.Prelude import Control.Monad.Trans.Control (MonadBaseControl) @@ -72,11 +71,11 @@ insertCertificate :: insertCertificate syncEnv isMember mDeposits blkId txId epochNo slotNo redeemers (Generic.TxCertificate ridx idx cert) = case cert of Left (ShelleyTxCertDelegCert deleg) -> - when (ioShelley iopts) $ insertDelegCert tracer cache mDeposits network txId idx mRedeemerId epochNo slotNo deleg + when (ioShelley iopts) $ insertDelegCert syncEnv mDeposits network txId idx mRedeemerId epochNo slotNo deleg Left (ShelleyTxCertPool pool) -> - when (ioShelley iopts) $ insertPoolCert tracer cache isMember mDeposits network epochNo blkId txId idx pool + when (ioShelley iopts) $ insertPoolCert syncEnv isMember mDeposits epochNo blkId txId idx pool Left (ShelleyTxCertMir mir) -> - when (ioShelley iopts) $ insertMirCert tracer cache network txId idx mir + when (ioShelley iopts) $ insertMirCert syncEnv txId idx mir Left (ShelleyTxCertGenesisDeleg _gen) -> when (ioShelley iopts) $ liftIO $ @@ -84,7 +83,7 @@ insertCertificate syncEnv isMember mDeposits blkId txId epochNo slotNo redeemers Right (ConwayTxCertDeleg deleg) -> insertConwayDelegCert syncEnv mDeposits txId idx mRedeemerId epochNo slotNo deleg Right (ConwayTxCertPool pool) -> - when (ioShelley iopts) $ insertPoolCert tracer cache isMember mDeposits network epochNo blkId txId idx pool + when (ioShelley iopts) $ insertPoolCert syncEnv isMember mDeposits epochNo blkId txId idx pool Right (ConwayTxCertGov c) -> when (ioGov iopts) $ case c of ConwayRegDRep cred coin anchor -> @@ -99,15 +98,13 @@ insertCertificate syncEnv isMember mDeposits blkId txId epochNo slotNo redeemers lift $ insertDrepRegistration blkId txId idx cred Nothing (strictMaybeToMaybe anchor) where tracer = getTrace syncEnv - cache = envCache syncEnv iopts = getInsertOptions syncEnv network = getNetwork syncEnv mRedeemerId = mlookup ridx redeemers insertDelegCert :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> Maybe Generic.Deposits -> Ledger.Network -> DB.TxId -> @@ -117,11 +114,11 @@ insertDelegCert :: SlotNo -> ShelleyDelegCert StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertDelegCert tracer cache mDeposits network txId idx mRedeemerId epochNo slotNo dCert = +insertDelegCert syncEnv mDeposits network txId idx mRedeemerId epochNo slotNo dCert = case dCert of - ShelleyRegCert cred -> insertStakeRegistration tracer cache epochNo mDeposits txId idx $ Generic.annotateStakingCred network cred - ShelleyUnRegCert cred -> insertStakeDeregistration tracer cache network epochNo txId idx mRedeemerId cred - ShelleyDelegCert cred poolkh -> insertDelegation tracer cache network epochNo slotNo txId idx mRedeemerId cred poolkh + ShelleyRegCert cred -> insertStakeRegistration syncEnv epochNo mDeposits txId idx $ Generic.annotateStakingCred network cred + ShelleyUnRegCert cred -> insertStakeDeregistration syncEnv epochNo txId idx mRedeemerId cred + ShelleyDelegCert cred poolkh -> insertDelegation syncEnv epochNo slotNo txId idx mRedeemerId cred poolkh insertConwayDelegCert :: (MonadBaseControl IO m, MonadIO m) => @@ -138,46 +135,42 @@ insertConwayDelegCert syncEnv mDeposits txId idx mRedeemerId epochNo slotNo dCer case dCert of ConwayRegCert cred _dep -> when (ioShelley iopts) $ - insertStakeRegistration trce cache epochNo mDeposits txId idx $ + insertStakeRegistration syncEnv epochNo mDeposits txId idx $ Generic.annotateStakingCred network cred ConwayUnRegCert cred _dep -> when (ioShelley iopts) $ - insertStakeDeregistration trce cache network epochNo txId idx mRedeemerId cred + insertStakeDeregistration syncEnv epochNo txId idx mRedeemerId cred ConwayDelegCert cred delegatee -> insertDeleg cred delegatee ConwayRegDelegCert cred delegatee _dep -> do when (ioShelley iopts) $ - insertStakeRegistration trce cache epochNo mDeposits txId idx $ + insertStakeRegistration syncEnv epochNo mDeposits txId idx $ Generic.annotateStakingCred network cred insertDeleg cred delegatee where insertDeleg cred = \case DelegStake poolkh -> when (ioShelley iopts) $ - insertDelegation trce cache network epochNo slotNo txId idx mRedeemerId cred poolkh + insertDelegation syncEnv epochNo slotNo txId idx mRedeemerId cred poolkh DelegVote drep -> when (ioGov iopts) $ - insertDelegationVote trce cache network txId idx cred drep + insertDelegationVote syncEnv txId idx cred drep DelegStakeVote poolkh drep -> do when (ioShelley iopts) $ - insertDelegation trce cache network epochNo slotNo txId idx mRedeemerId cred poolkh + insertDelegation syncEnv epochNo slotNo txId idx mRedeemerId cred poolkh when (ioGov iopts) $ - insertDelegationVote trce cache network txId idx cred drep + insertDelegationVote syncEnv txId idx cred drep - trce = getTrace syncEnv - cache = envCache syncEnv iopts = getInsertOptions syncEnv network = getNetwork syncEnv insertMirCert :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> - Ledger.Network -> + SyncEnv -> DB.TxId -> Word16 -> MIRCert StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertMirCert tracer cache network txId idx mcert = do +insertMirCert syncEnv txId idx mcert = do case mirPot mcert of ReservesMIR -> case mirRewards mcert of @@ -193,7 +186,7 @@ insertMirCert tracer cache network txId idx mcert = do (StakeCred, Ledger.DeltaCoin) -> ExceptT SyncNodeError (ReaderT SqlBackend m) () insertMirReserves (cred, dcoin) = do - addrId <- lift $ queryOrInsertStakeAddress tracer cache UpdateCacheStrong network cred + addrId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong cred void . lift . DB.insertReserve $ DB.Reserve { DB.reserveAddrId = addrId @@ -207,7 +200,7 @@ insertMirCert tracer cache network txId idx mcert = do (StakeCred, Ledger.DeltaCoin) -> ExceptT SyncNodeError (ReaderT SqlBackend m) () insertMirTreasury (cred, dcoin) = do - addrId <- lift $ queryOrInsertStakeAddress tracer cache UpdateCacheStrong network cred + addrId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong cred void . lift . DB.insertTreasury $ DB.Treasury { DB.treasuryAddrId = addrId @@ -316,17 +309,15 @@ insertCommitteeDeRegistration blockId txId idx khCold mAnchor = do insertStakeDeregistration :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> - Ledger.Network -> + SyncEnv -> EpochNo -> DB.TxId -> Word16 -> Maybe DB.RedeemerId -> StakeCred -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertStakeDeregistration trce cache network epochNo txId idx mRedeemerId cred = do - scId <- lift $ queryOrInsertStakeAddress trce cache EvictAndUpdateCache network cred +insertStakeDeregistration syncEnv epochNo txId idx mRedeemerId cred = do + scId <- lift $ queryOrInsertStakeAddress syncEnv EvictAndUpdateCache cred void . lift . DB.insertStakeDeregistration $ DB.StakeDeregistration { DB.stakeDeregistrationAddrId = scId @@ -338,16 +329,15 @@ insertStakeDeregistration trce cache network epochNo txId idx mRedeemerId cred = insertStakeRegistration :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> EpochNo -> Maybe Generic.Deposits -> DB.TxId -> Word16 -> - Shelley.RewardAccount StandardCrypto -> + RewAccount -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertStakeRegistration tracer cache epochNo mDeposits txId idx rewardAccount = do - saId <- lift $ queryOrInsertRewardAccount tracer cache UpdateCache rewardAccount +insertStakeRegistration syncEnv epochNo mDeposits txId idx rewardAccount = do + saId <- lift $ queryOrInsertRewardAccount syncEnv UpdateCache rewardAccount void . lift . DB.insertStakeRegistration $ DB.StakeRegistration { DB.stakeRegistrationAddrId = saId @@ -401,9 +391,7 @@ mkAdaPots blockId slotNo epochNo pots = -------------------------------------------------------------------------------------------- insertDelegation :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> - Ledger.Network -> + SyncEnv -> EpochNo -> SlotNo -> DB.TxId -> @@ -412,9 +400,9 @@ insertDelegation :: StakeCred -> Ledger.KeyHash 'Ledger.StakePool StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertDelegation trce cache network (EpochNo epoch) slotNo txId idx mRedeemerId cred poolkh = do - addrId <- lift $ queryOrInsertStakeAddress trce cache UpdateCacheStrong network cred - poolHashId <- lift $ queryPoolKeyOrInsert "insertDelegation" trce cache UpdateCache True poolkh +insertDelegation syncEnv (EpochNo epoch) slotNo txId idx mRedeemerId cred poolkh = do + addrId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong cred + poolHashId <- lift $ queryPoolKeyOrInsert syncEnv "insertDelegation" UpdateCache True poolkh void . lift . DB.insertDelegation $ DB.Delegation { DB.delegationAddrId = addrId @@ -428,16 +416,14 @@ insertDelegation trce cache network (EpochNo epoch) slotNo txId idx mRedeemerId insertDelegationVote :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> - Ledger.Network -> + SyncEnv -> DB.TxId -> Word16 -> StakeCred -> DRep StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertDelegationVote trce cache network txId idx cred drep = do - addrId <- lift $ queryOrInsertStakeAddress trce cache UpdateCacheStrong network cred +insertDelegationVote syncEnv txId idx cred drep = do + addrId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong cred drepId <- lift $ insertDrep drep void . lift diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/GovAction.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/GovAction.hs index 6de4a5362..c72c00dc0 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/GovAction.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/GovAction.hs @@ -32,12 +32,13 @@ import Cardano.BM.Trace (Trace, logWarning) import qualified Cardano.Crypto as Crypto import Cardano.Db (DbWord64 (..)) import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types import Cardano.DbSync.Cache (queryOrInsertRewardAccount, queryPoolKeyOrInsert, queryTxIdWithCache) import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..)) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Shelley.Generic.ParamProposal import Cardano.DbSync.Era.Universal.Insert.Other (toDouble) -import Cardano.DbSync.Era.Util (liftLookupFail) import Cardano.DbSync.Error import Cardano.DbSync.Ledger.State import Cardano.DbSync.Util @@ -69,17 +70,16 @@ import Ouroboros.Consensus.Cardano.Block (StandardConway, StandardCrypto) insertGovActionProposal :: forall m. (MonadIO m, MonadBaseControl IO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> DB.BlockId -> DB.TxId -> Maybe EpochNo -> Maybe (ConwayGovState StandardConway) -> (Word64, (GovActionId StandardCrypto, ProposalProcedure StandardConway)) -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertGovActionProposal trce cache blkId txId govExpiresAt mcgs (index, (govId, pp)) = do +insertGovActionProposal syncEnv blkId txId govExpiresAt mcgs (index, (govId, pp)) = do addrId <- - lift $ queryOrInsertRewardAccount trce cache UpdateCache $ pProcReturnAddr pp + lift $ queryOrInsertRewardAccount syncEnv UpdateCache $ pProcReturnAddr pp votingAnchorId <- lift $ insertVotingAnchor blkId DB.GovActionAnchor $ pProcAnchor pp mParamProposalId <- lift $ case pProcGovAction pp of @@ -114,6 +114,9 @@ insertGovActionProposal trce cache blkId txId govExpiresAt mcgs (index, (govId, NewConstitution _ constitution -> lift $ void $ insertConstitution blkId (Just govActionProposalId) constitution _ -> pure () where + trce = getTrace syncEnv + cache = envCache syncEnv + mprevGovAction :: Maybe (GovActionId StandardCrypto) = case pProcGovAction pp of ParameterChange prv _ _ -> unGovPurposeId <$> strictMaybeToMaybe prv HardForkInitiation prv _ -> unGovPurposeId <$> strictMaybeToMaybe prv @@ -124,7 +127,7 @@ insertGovActionProposal trce cache blkId txId govExpiresAt mcgs (index, (govId, insertTreasuryWithdrawal gaId (rwdAcc, coin) = do addrId <- - queryOrInsertRewardAccount trce cache UpdateCache rwdAcc + queryOrInsertRewardAccount syncEnv UpdateCache rwdAcc DB.insertTreasuryWithdrawal $ DB.TreasuryWithdrawal { DB.treasuryWithdrawalGovActionProposalId = gaId @@ -264,25 +267,23 @@ insertConstitution blockId mgapId constitution = do -------------------------------------------------------------------------------------- insertVotingProcedures :: (MonadIO m, MonadBaseControl IO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> DB.BlockId -> DB.TxId -> (Voter StandardCrypto, [(GovActionId StandardCrypto, VotingProcedure StandardConway)]) -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertVotingProcedures trce cache blkId txId (voter, actions) = - mapM_ (insertVotingProcedure trce cache blkId txId voter) (zip [0 ..] actions) +insertVotingProcedures syncEnv blkId txId (voter, actions) = + mapM_ (insertVotingProcedure syncEnv blkId txId voter) (zip [0 ..] actions) insertVotingProcedure :: (MonadIO m, MonadBaseControl IO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> DB.BlockId -> DB.TxId -> Voter StandardCrypto -> (Word16, (GovActionId StandardCrypto, VotingProcedure StandardConway)) -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertVotingProcedure trce cache blkId txId voter (index, (gaId, vp)) = do +insertVotingProcedure syncEnv blkId txId voter (index, (gaId, vp)) = do govActionId <- resolveGovActionProposal cache gaId votingAnchorId <- whenMaybe (strictMaybeToMaybe $ vProcAnchor vp) $ lift . insertVotingAnchor blkId DB.VoteAnchor (mCommitteeVoterId, mDRepVoter, mStakePoolVoter) <- case voter of @@ -293,7 +294,7 @@ insertVotingProcedure trce cache blkId txId voter (index, (gaId, vp)) = do drep <- lift $ insertCredDrepHash cred pure (Nothing, Just drep, Nothing) StakePoolVoter poolkh -> do - poolHashId <- lift $ queryPoolKeyOrInsert "insertVotingProcedure" trce cache UpdateCache False poolkh + poolHashId <- lift $ queryPoolKeyOrInsert syncEnv "insertVotingProcedure" UpdateCache False poolkh pure (Nothing, Nothing, Just poolHashId) void . lift @@ -310,6 +311,8 @@ insertVotingProcedure trce cache blkId txId voter (index, (gaId, vp)) = do , DB.votingProcedureVotingAnchorId = votingAnchorId , DB.votingProcedureInvalid = Nothing } + where + cache = envCache syncEnv insertVotingAnchor :: (MonadIO m, MonadBaseControl IO m) => DB.BlockId -> DB.AnchorType -> Anchor StandardCrypto -> ReaderT SqlBackend m DB.VotingAnchorId insertVotingAnchor blockId anchorType anchor = diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Grouped.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Grouped.hs index dc6b61234..fc65dd770 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Grouped.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Grouped.hs @@ -1,5 +1,7 @@ {-# LANGUAGE FlexibleContexts #-} {-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE ScopedTypeVariables #-} +{-# LANGUAGE TupleSections #-} {-# LANGUAGE NoImplicitPrelude #-} module Cardano.DbSync.Era.Universal.Insert.Grouped ( @@ -9,7 +11,10 @@ module Cardano.DbSync.Era.Universal.Insert.Grouped ( ExtendedTxOut (..), insertBlockGroupedData, insertReverseIndex, + prepareResolveTxInputs, + resolveTxInputsMain, resolveTxInputs, + resolveTxInputsPrefetch, resolveScriptHash, mkmaTxOuts, ) where @@ -20,15 +25,18 @@ import qualified Cardano.Db as DB import qualified Cardano.Db.Schema.Core.TxOut as C import qualified Cardano.Db.Schema.Variant.TxOut as V import Cardano.DbSync.Api -import Cardano.DbSync.Api.Types (SyncEnv (..)) +import Cardano.DbSync.Api.Types (Prefetch (..), SyncEnv (..)) import Cardano.DbSync.Cache (queryTxIdWithCache) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Shelley.Query -import Cardano.DbSync.Era.Util import Cardano.DbSync.Error +import Cardano.DbSync.Types import Cardano.Prelude +import Control.Concurrent.Class.MonadSTM.Strict (modifyTVar, readTVarIO) import Control.Monad.Trans.Control (MonadBaseControl) +import Data.Either.Extra (eitherToMaybe) import qualified Data.List as List +import qualified Data.Map as Map import qualified Data.Text as Text import Database.Persist.Sql (SqlBackend) @@ -68,7 +76,7 @@ data ExtendedTxOut = ExtendedTxOut data ExtendedTxIn = ExtendedTxIn { etiTxIn :: !DB.TxIn - , etiTxOutId :: !(Either Generic.TxIn DB.TxOutIdW) + , etiTxOutId :: !(Either Generic.TxInKey DB.TxOutIdW) } deriving (Show) @@ -180,51 +188,97 @@ insertReverseIndex blockId minIdsWrapper = , DB.reverseIndexMinIds = minIdsVariantToText minIds } --- | If we can't resolve from the db, we fall back to the provided outputs --- This happens the input consumes an output introduced in the same block. -resolveTxInputs :: +resolveTxInputsPrefetch :: MonadIO m => SyncEnv -> Bool -> + Generic.TxInKey -> + ReaderT SqlBackend m () +resolveTxInputsPrefetch syncEnv needsValue txIn = do + eiRes <- resolveTxInputs syncEnv needsValue Nothing txIn + liftIO $ atomically $ modifyTVar (pTxIn $ envPrefetch syncEnv) $ Map.insert txIn (eitherToMaybe eiRes) + +resolveTxInputsMain :: + MonadIO m => + SyncEnv -> Bool -> - [ExtendedTxOut] -> + Maybe [ExtendedTxOut] -> Generic.TxIn -> - ExceptT SyncNodeError (ReaderT SqlBackend m) (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW, Maybe DbLovelace) -resolveTxInputs syncEnv hasConsumed needsValue groupedOutputs txIn = - liftLookupFail ("resolveTxInputs " <> textShow txIn <> " ") $ do - qres <- - case (hasConsumed, needsValue) of - (_, True) -> fmap convertFoundAll <$> resolveInputTxOutIdValue syncEnv txIn - (False, _) -> fmap convertnotFoundCache <$> queryTxIdWithCache (envCache syncEnv) (Generic.txInTxId txIn) - (True, False) -> fmap convertFoundTxOutId <$> resolveInputTxOutId syncEnv txIn - case qres of - Right ret -> pure $ Right ret - Left err -> - case (resolveInMemory txIn groupedOutputs, hasConsumed, needsValue) of - (Nothing, _, _) -> pure $ Left err - (Just eutxo, True, True) -> pure $ Right $ convertFoundValue (etoTxOut eutxo) - (Just eutxo, _, _) -> pure $ Right $ convertnotFound (etoTxOut eutxo) + ExceptT SyncNodeError (ReaderT SqlBackend m) (Generic.TxIn, DB.TxId, Either Generic.TxInKey DB.TxOutIdW, Maybe DbLovelace) +resolveTxInputsMain syncEnv needsValue mGroupedOutputs txIn = do + prefetch <- liftIO $ readTVarIO (pTxIn $ envPrefetch syncEnv) + case Map.lookup txIn' prefetch of + Just (Just ret) -> pure $ addTxIn ret + Nothing -> logWarn >> resolve + _ -> resolve + where + resolve = + addTxIn + <$> liftLookupFail + ("resolveTxInputs " <> textShow txIn' <> " ") + (resolveTxInputs syncEnv needsValue mGroupedOutputs txIn') + + -- A warning tells us that prefetcher didn't work as fast. This is purely for debugging and should + -- eventually be removed + logWarn = liftIO $ logWarning trce $ "Prefetcher missed " <> textShow txIn' + addTxIn (a, b, c) = (txIn, a, b, c) + txIn' = Generic.txInKey txIn + trce = getTrace syncEnv + +prepareResolveTxInputs :: + MonadIO m => + SyncEnv -> + [(TxIdLedger, DB.TxId)] -> + Generic.TxInKey -> + ExceptT SyncNodeError (ReaderT SqlBackend m) DB.TxId +prepareResolveTxInputs syncEnv txHashes txIn = liftLookupFail ("prepareResolveTxInputs: " <> textShow txIn) $ do + eiTxId <- queryTxIdWithCache (envCache syncEnv) (Generic.txInTxId txIn) + case eiTxId of + Right _ -> pure eiTxId + Left err -> case List.lookup (Generic.txInTxId txIn) txHashes of + Just txId -> pure $ Right txId + Nothing -> pure $ Left err + +-- | Concurrency Warning: This code may run by many threads concurrently. +-- If we can't resolve from the db, we fall back to the provided outputs +-- This happens when the input consumes an output introduced in the same block. +resolveTxInputs :: + MonadIO m => + SyncEnv -> + Bool -> + Maybe [ExtendedTxOut] -> + Generic.TxInKey -> + ReaderT SqlBackend m (Either DB.LookupFail (DB.TxId, Either Generic.TxInKey DB.TxOutIdW, Maybe DbLovelace)) +resolveTxInputs syncEnv needsValue mGroupedOutputs txIn = do + qres <- + case (hasConsumed, needsValue) of + (_, True) -> fmap convertFoundAll <$> resolveInputTxOutIdValue syncEnv txIn + (False, _) -> fmap convertnotFoundCache <$> queryTxIdWithCache (envCache syncEnv) (Generic.txInTxId txIn) + (True, False) -> fmap convertFoundTxOutId <$> resolveInputTxOutId syncEnv txIn + case (qres, mGroupedOutputs) of + (Right _, _) -> pure qres + (_, Nothing) -> pure qres + (Left err, Just groupedOutputs) -> + case (resolveInMemory txIn groupedOutputs, hasConsumed, needsValue) of + (Nothing, _, _) -> pure $ Left err + (Just eutxo, True, True) -> pure $ Right $ convertFoundValue (etoTxOut eutxo) + (Just eutxo, _, _) -> pure $ Right $ convertnotFound (etoTxOut eutxo) where - convertnotFoundCache :: DB.TxId -> (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW, Maybe DbLovelace) - convertnotFoundCache txId = (txIn, txId, Left txIn, Nothing) + convertnotFoundCache txId = (txId, Left txIn, Nothing) - convertnotFound :: DB.TxOutW -> (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW, Maybe DbLovelace) convertnotFound txOutWrapper = case txOutWrapper of - DB.CTxOutW cTxOut -> (txIn, C.txOutTxId cTxOut, Left txIn, Nothing) - DB.VTxOutW vTxOut _ -> (txIn, V.txOutTxId vTxOut, Left txIn, Nothing) + DB.CTxOutW cTxOut -> (C.txOutTxId cTxOut, Left txIn, Nothing) + DB.VTxOutW vTxOut _ -> (V.txOutTxId vTxOut, Left txIn, Nothing) - convertFoundTxOutId :: (DB.TxId, DB.TxOutIdW) -> (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW, Maybe DbLovelace) - convertFoundTxOutId (txId, txOutId) = (txIn, txId, Right txOutId, Nothing) + convertFoundTxOutId (txId, txOutId) = (txId, Right txOutId, Nothing) - -- convertFoundValue :: (DB.TxId, DbLovelace) -> (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW, Maybe DbLovelace) - convertFoundValue :: DB.TxOutW -> (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW, Maybe DbLovelace) convertFoundValue txOutWrapper = case txOutWrapper of - DB.CTxOutW cTxOut -> (txIn, C.txOutTxId cTxOut, Left txIn, Just $ C.txOutValue cTxOut) - DB.VTxOutW vTxOut _ -> (txIn, V.txOutTxId vTxOut, Left txIn, Just $ V.txOutValue vTxOut) - -- (txIn, txId, Left txIn, Just lovelace) + DB.CTxOutW cTxOut -> (C.txOutTxId cTxOut, Left txIn, Just $ C.txOutValue cTxOut) + DB.VTxOutW vTxOut _ -> (V.txOutTxId vTxOut, Left txIn, Just $ V.txOutValue vTxOut) - convertFoundAll :: (DB.TxId, DB.TxOutIdW, DbLovelace) -> (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW, Maybe DbLovelace) - convertFoundAll (txId, txOutId, lovelace) = (txIn, txId, Right txOutId, Just lovelace) + convertFoundAll (txId, txOutId, lovelace) = (txId, Right txOutId, Just lovelace) + + hasConsumed = getHasConsumedOrPruneTxOut syncEnv resolveRemainingInputs :: MonadIO m => @@ -245,7 +299,7 @@ resolveScriptHash :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> [ExtendedTxOut] -> - Generic.TxIn -> + Generic.TxInKey -> ExceptT SyncNodeError (ReaderT SqlBackend m) (Maybe ByteString) resolveScriptHash syncEnv groupedOutputs txIn = liftLookupFail "resolveScriptHash" $ do @@ -261,11 +315,11 @@ resolveScriptHash syncEnv groupedOutputs txIn = Nothing -> pure $ Left $ DB.DBTxOutVariant "resolveScriptHash: VTxOutW with Nothing address" Just vAddr -> pure $ Right $ V.addressPaymentCred vAddr -resolveInMemory :: Generic.TxIn -> [ExtendedTxOut] -> Maybe ExtendedTxOut +resolveInMemory :: Generic.TxInKey -> [ExtendedTxOut] -> Maybe ExtendedTxOut resolveInMemory txIn = List.find (matches txIn) -matches :: Generic.TxIn -> ExtendedTxOut -> Bool +matches :: Generic.TxInKey -> ExtendedTxOut -> Bool matches txIn eutxo = Generic.toTxHash txIn == etoTxHash eutxo && Generic.txInIndex txIn == getTxOutIndex (etoTxOut eutxo) diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/LedgerEvent.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/LedgerEvent.hs index c4938e8f6..b2624e16d 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/LedgerEvent.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/LedgerEvent.hs @@ -82,14 +82,14 @@ insertNewEpochLedgerEvents syncEnv currentEpochNo@(EpochNo curEpoch) = liftIO . logInfo tracer $ "Starting at epoch " <> textShow (unEpochNo en) LedgerDeltaRewards _e rwd -> do let rewards = Map.toList $ Generic.unRewards rwd - insertRewards syncEnv ntw (subFromCurrentEpoch 2) currentEpochNo cache (Map.toList $ Generic.unRewards rwd) + insertRewards syncEnv (subFromCurrentEpoch 2) currentEpochNo (Map.toList $ Generic.unRewards rwd) -- This event is only created when it's not empty, so we don't need to check for null here. liftIO . logInfo tracer $ "Inserted " <> show (length rewards) <> " Delta rewards" LedgerIncrementalRewards _ rwd -> do let rewards = Map.toList $ Generic.unRewards rwd - insertRewards syncEnv ntw (subFromCurrentEpoch 1) (EpochNo $ curEpoch + 1) cache rewards + insertRewards syncEnv (subFromCurrentEpoch 1) (EpochNo $ curEpoch + 1) rewards LedgerRestrainedRewards e rwd creds -> - lift $ adjustEpochRewards tracer ntw cache e rwd creds + lift $ adjustEpochRewards syncEnv e rwd creds LedgerTotalRewards _e rwd -> lift $ validateEpochRewards tracer ntw (subFromCurrentEpoch 2) currentEpochNo rwd LedgerAdaPots _ -> @@ -101,17 +101,17 @@ insertNewEpochLedgerEvents syncEnv currentEpochNo@(EpochNo curEpoch) = "Found " <> textShow (Set.size uncl) <> " unclaimed proposal refunds" updateDropped cache (EpochNo curEpoch) (garGovActionId <$> (dropped <> expired)) let refunded = filter (\e -> Set.notMember (garGovActionId e) uncl) (enacted <> dropped <> expired) - insertProposalRefunds tracer ntw (subFromCurrentEpoch 1) currentEpochNo cache refunded -- TODO: check if they are disjoint to avoid double entries. + insertProposalRefunds syncEnv (subFromCurrentEpoch 1) currentEpochNo refunded -- TODO: check if they are disjoint to avoid double entries. forM_ enacted $ \gar -> do gaId <- resolveGovActionProposal cache (garGovActionId gar) lift $ void $ DB.updateGovActionEnacted gaId (unEpochNo currentEpochNo) whenJust (garMTreasury gar) $ \treasuryMap -> do let rewards = Map.mapKeys Ledger.raCredential $ Map.map (Set.singleton . mkTreasuryReward) treasuryMap - insertRewardRests tracer ntw (subFromCurrentEpoch 1) currentEpochNo cache (Map.toList rewards) + insertRewardRests syncEnv (subFromCurrentEpoch 1) currentEpochNo (Map.toList rewards) LedgerMirDist rwd -> do unless (Map.null rwd) $ do let rewards = Map.toList rwd - insertRewardRests tracer ntw (subFromCurrentEpoch 1) currentEpochNo cache rewards + insertRewardRests syncEnv (subFromCurrentEpoch 1) currentEpochNo rewards liftIO . logInfo tracer $ "Inserted " <> show (length rewards) <> " Mir rewards" LedgerPoolReap en drs -> unless (Map.null $ Generic.unRewards drs) $ do diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Other.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Other.hs index 4099e8427..b1fce84c8 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Other.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Other.hs @@ -12,7 +12,7 @@ module Cardano.DbSync.Era.Universal.Insert.Other ( insertDatum, insertWithdrawals, insertRedeemerData, - insertStakeAddressRefIfMissing, + queryOrInsertStakeRef, insertMultiAsset, insertScript, insertExtraKeyWitness, @@ -22,11 +22,10 @@ import Cardano.BM.Trace (Trace) import qualified Cardano.Db as DB import Cardano.DbSync.Api (getTrace) import Cardano.DbSync.Api.Types (SyncEnv) -import Cardano.DbSync.Cache (insertDatumAndCache, queryDatum, queryMAWithCache, queryOrInsertRewardAccount, queryOrInsertStakeAddress) +import Cardano.DbSync.Cache (insertDatumAndCache, queryDatum, queryMAWithCache, queryOrInsertRewardAccount) import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..)) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Universal.Insert.Grouped -import Cardano.DbSync.Era.Util (safeDecodeToJson) import Cardano.DbSync.Error import Cardano.DbSync.Util import qualified Cardano.Ledger.Address as Ledger @@ -77,7 +76,7 @@ insertRedeemer syncEnv disInOut groupedOutputs txId (rix, redeemer) = do (True, _) -> pure Nothing (_, Nothing) -> pure Nothing (_, Just (Right bs)) -> pure $ Just bs - (_, Just (Left txIn)) -> resolveScriptHash syncEnv groupedOutputs txIn + (_, Just (Left txIn)) -> resolveScriptHash syncEnv groupedOutputs (Generic.txInKey txIn) insertRedeemerData :: (MonadBaseControl IO m, MonadIO m) => @@ -127,15 +126,14 @@ insertDatum tracer cache txId txd = do insertWithdrawals :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> DB.TxId -> Map Word64 DB.RedeemerId -> Generic.TxWithdrawal -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertWithdrawals tracer cache txId redeemers txWdrl = do +insertWithdrawals syncEnv txId redeemers txWdrl = do addrId <- - lift $ queryOrInsertRewardAccount tracer cache UpdateCache $ Generic.txwRewardAccount txWdrl + lift $ queryOrInsertRewardAccount syncEnv UpdateCache $ Generic.txwRewardAccount txWdrl void . lift . DB.insertWithdrawal $ DB.Withdrawal { DB.withdrawalAddrId = addrId @@ -146,19 +144,18 @@ insertWithdrawals tracer cache txId redeemers txWdrl = do -- | Insert a stake address if it is not already in the `stake_address` table. Regardless of -- whether it is newly inserted or it is already there, we retrun the `StakeAddressId`. -insertStakeAddressRefIfMissing :: +queryOrInsertStakeRef :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> Ledger.Addr StandardCrypto -> ReaderT SqlBackend m (Maybe DB.StakeAddressId) -insertStakeAddressRefIfMissing trce cache addr = +queryOrInsertStakeRef syncEnv addr = case addr of Ledger.AddrBootstrap {} -> pure Nothing Ledger.Addr nw _pcred sref -> case sref of Ledger.StakeRefBase cred -> do - Just <$> queryOrInsertStakeAddress trce cache UpdateCache nw cred + Just <$> queryOrInsertRewardAccount syncEnv UpdateCache (Ledger.RewardAccount nw cred) Ledger.StakeRefPtr ptr -> do DB.queryStakeRefPtr ptr Ledger.StakeRefNull -> pure Nothing diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Pool.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Pool.hs index 2631c8a6c..1bc8578cd 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Pool.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Pool.hs @@ -16,22 +16,20 @@ module Cardano.DbSync.Era.Universal.Insert.Pool ( insertPoolCert, ) where -import Cardano.BM.Trace (Trace) import Cardano.Crypto.Hash (hashToBytes) import Cardano.Db (PoolUrl (..)) import qualified Cardano.Db as DB +import Cardano.DbSync.Api.Types import Cardano.DbSync.Cache ( insertPoolKeyWithCache, - queryOrInsertRewardAccount, queryOrInsertStakeAddress, queryPoolKeyOrInsert, ) -import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..)) +import Cardano.DbSync.Cache.Types (CacheAction (..)) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Error import Cardano.DbSync.Types (PoolKeyHash) import Cardano.DbSync.Util -import qualified Cardano.Ledger.Address as Ledger import Cardano.Ledger.BaseTypes import qualified Cardano.Ledger.BaseTypes as Ledger import Cardano.Ledger.Core (PoolCert (..)) @@ -48,18 +46,16 @@ type IsPoolMember = PoolKeyHash -> Bool insertPoolRegister :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> IsPoolMember -> Maybe Generic.Deposits -> - Ledger.Network -> EpochNo -> DB.BlockId -> DB.TxId -> Word16 -> PoolP.PoolParams StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertPoolRegister trce cache isMember mdeposits network (EpochNo epoch) blkId txId idx params = do +insertPoolRegister syncEnv isMember mdeposits (EpochNo epoch) blkId txId idx params = do poolHashId <- lift $ insertPoolKeyWithCache cache UpdateCache (PoolP.ppId params) mdId <- case strictMaybeToMaybe $ PoolP.ppMetadata params of Just md -> Just <$> insertPoolMetaDataRef poolHashId txId md @@ -69,7 +65,7 @@ insertPoolRegister trce cache isMember mdeposits network (EpochNo epoch) blkId t let epochActivationDelay = if isRegistration then 2 else 3 deposit = if isRegistration then Generic.coinToDbLovelace . Generic.poolDeposit <$> mdeposits else Nothing - saId <- lift $ queryOrInsertRewardAccount trce cache UpdateCache (adjustNetworkTag $ PoolP.ppRewardAccount params) + saId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCache (ignoreNetworkTag $ PoolP.ppRewardAccount params) poolUpdateId <- lift . DB.insertPoolUpdate @@ -87,7 +83,7 @@ insertPoolRegister trce cache isMember mdeposits network (EpochNo epoch) blkId t , DB.poolUpdateRegisteredTxId = txId } - mapM_ (insertPoolOwner trce cache network poolUpdateId) $ toList (PoolP.ppOwners params) + mapM_ (insertPoolOwner syncEnv poolUpdateId) $ toList (PoolP.ppOwners params) mapM_ (insertPoolRelay poolUpdateId) $ toList (PoolP.ppRelays params) where isPoolRegistration :: MonadIO m => DB.PoolHashId -> ExceptT SyncNodeError (ReaderT SqlBackend m) Bool @@ -103,20 +99,20 @@ insertPoolRegister trce cache isMember mdeposits network (EpochNo epoch) blkId t -- Ignore the network in the `RewardAccount` and use the provided one instead. -- This is a workaround for https://github.com/IntersectMBO/cardano-db-sync/issues/546 - adjustNetworkTag :: Ledger.RewardAccount StandardCrypto -> Ledger.RewardAccount StandardCrypto - adjustNetworkTag (Shelley.RewardAccount _ cred) = Shelley.RewardAccount network cred + ignoreNetworkTag (Shelley.RewardAccount _ cred) = cred + + cache = envCache syncEnv insertPoolRetire :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> + SyncEnv -> DB.TxId -> - CacheStatus -> EpochNo -> Word16 -> Ledger.KeyHash 'Ledger.StakePool StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertPoolRetire trce txId cache epochNum idx keyHash = do - poolId <- lift $ queryPoolKeyOrInsert "insertPoolRetire" trce cache UpdateCache True keyHash +insertPoolRetire syncEnv txId epochNum idx keyHash = do + poolId <- lift $ queryPoolKeyOrInsert syncEnv "insertPoolRetire" UpdateCache True keyHash void . lift . DB.insertPoolRetire $ DB.PoolRetire { DB.poolRetireHashId = poolId @@ -143,14 +139,12 @@ insertPoolMetaDataRef poolId txId md = insertPoolOwner :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> - Ledger.Network -> + SyncEnv -> DB.PoolUpdateId -> Ledger.KeyHash 'Ledger.Staking StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertPoolOwner trce cache network poolUpdateId skh = do - saId <- lift $ queryOrInsertStakeAddress trce cache UpdateCacheStrong network (Ledger.KeyHashObj skh) +insertPoolOwner syncEnv poolUpdateId skh = do + saId <- lift $ queryOrInsertStakeAddress syncEnv UpdateCacheStrong (Ledger.KeyHashObj skh) void . lift . DB.insertPoolOwner $ DB.PoolOwner { DB.poolOwnerAddrId = saId @@ -197,18 +191,16 @@ insertPoolRelay updateId relay = insertPoolCert :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> IsPoolMember -> Maybe Generic.Deposits -> - Ledger.Network -> EpochNo -> DB.BlockId -> DB.TxId -> Word16 -> PoolCert StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertPoolCert tracer cache isMember mdeposits network epoch blkId txId idx pCert = +insertPoolCert syncEnv isMember mdeposits epoch blkId txId idx pCert = case pCert of - RegPool pParams -> insertPoolRegister tracer cache isMember mdeposits network epoch blkId txId idx pParams - RetirePool keyHash epochNum -> insertPoolRetire tracer txId cache epochNum idx keyHash + RegPool pParams -> insertPoolRegister syncEnv isMember mdeposits epoch blkId txId idx pParams + RetirePool keyHash epochNum -> insertPoolRetire syncEnv txId epochNum idx keyHash diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Tx.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Tx.hs index 4f12d03a9..c9a4caeca 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Tx.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Universal/Insert/Tx.hs @@ -9,12 +9,13 @@ {-# LANGUAGE NoImplicitPrelude #-} module Cardano.DbSync.Era.Universal.Insert.Tx ( - insertTx, - insertTxOut, + prepareTxGrouped, + insertTxRest, + prepareTxOut, ) where import Cardano.BM.Trace (Trace) -import Cardano.Db (DbLovelace (..), DbWord64 (..)) +import Cardano.Db (DbWord64 (..)) import qualified Cardano.Db as DB import qualified Cardano.Db.Schema.Core.TxOut as C import qualified Cardano.Db.Schema.Variant.TxOut as V @@ -24,7 +25,7 @@ import Cardano.DbSync.Cache (insertAddressUsingCache, queryTxIdWithCache, tryUpd import Cardano.DbSync.Cache.Types (CacheAction (..), CacheStatus (..)) import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Shelley.Generic.Metadata (TxMetadataValue (..), metadataValueToJsonNoSchema) -import Cardano.DbSync.Era.Shelley.Generic.Tx.Types (TxIn (..)) +import Cardano.DbSync.Era.Shelley.Generic.Tx.Types (TxInKey (..)) import Cardano.DbSync.Era.Universal.Insert.Certificate (insertCertificate) import Cardano.DbSync.Era.Universal.Insert.GovAction ( insertGovActionProposal, @@ -38,13 +39,12 @@ import Cardano.DbSync.Era.Universal.Insert.Other ( insertMultiAsset, insertRedeemer, insertScript, - insertStakeAddressRefIfMissing, insertWithdrawals, + queryOrInsertStakeRef, ) -import Cardano.DbSync.Era.Universal.Insert.Pool (IsPoolMember) -import Cardano.DbSync.Era.Util (liftLookupFail, safeDecodeToJson) import Cardano.DbSync.Error -import Cardano.DbSync.Ledger.Types (ApplyResult (..), getGovExpiresAt, lookupDepositsMap) +import Cardano.DbSync.Ledger.Types (ApplyResult (..), getGovExpiresAt) +import Cardano.DbSync.Types import Cardano.DbSync.Util import Cardano.DbSync.Util.Cbor (serialiseTxMetadataToCbor) import qualified Cardano.Ledger.Address as Ledger @@ -57,6 +57,7 @@ import Control.Monad.Trans.Control (MonadBaseControl) import qualified Data.Aeson as Aeson import qualified Data.ByteString.Lazy.Char8 as LBS import qualified Data.Map.Strict as Map +import qualified Data.Set as Set import qualified Data.Strict.Maybe as Strict import Database.Persist.Sql (SqlBackend) import Ouroboros.Consensus.Cardano.Block (StandardCrypto) @@ -64,68 +65,106 @@ import Ouroboros.Consensus.Cardano.Block (StandardCrypto) -------------------------------------------------------------------------------------- -- INSERT TX -------------------------------------------------------------------------------------- -insertTx :: + +prepareTxGrouped :: + (MonadIO m, MonadBaseControl IO m) => + SyncEnv -> + [(TxIdLedger, DB.TxId)] -> + DB.BlockId -> + Generic.Tx -> + ExceptT SyncNodeError (ReaderT SqlBackend m) ((DB.TxId, DB.Tx, Generic.Tx), BlockGroupedData) +prepareTxGrouped syncEnv txHashes blkId tx = do + disInOut <- liftIO $ getDisableInOutState syncEnv + txDb <- lift $ prepareTx syncEnv blkId tx + txIns <- whenTrueMempty disInOut $ mapM (prepareTxIn syncEnv txHashes txId) (Generic.txInputs tx) + txOuts <- whenTrueMempty disInOut $ mapM (prepareTxOut syncEnv iopts (txId, Generic.txHash tx)) (Generic.txOutputs tx) + txMetadata <- + whenFalseMempty (ioMetadata iopts && Generic.txValidContract tx) $ + prepareTxMetadata + tracer + txId + iopts + (Generic.txMetadata tx) + maTxMint <- + whenFalseMempty (ioMultiAssets iopts && Generic.txValidContract tx) $ + prepareMaTxMint tracer cache txId $ + Generic.txMint tx + pure ((txId, txDb, tx), BlockGroupedData txIns txOuts txMetadata maTxMint 0 outSum) + where + blockIndex = Generic.txBlockIndex tx + txId = DB.TxKey $ fromIntegral $ fromIntegral (DB.unBlockId blkId) * 1000 + blockIndex + tracer = getTrace syncEnv + cache = envCache syncEnv + iopts = getInsertOptions syncEnv + outSum = fromIntegral $ unCoin $ Generic.txOutSum tx + +prepareTx :: + MonadIO m => + SyncEnv -> + DB.BlockId -> + Generic.Tx -> + m DB.Tx +prepareTx syncEnv blkId tx = do + disInOut <- liftIO $ getDisableInOutState syncEnv + let fees = case (unCoin <$> Generic.txFees tx, disInOut) of + (_, True) -> 0 + (Nothing, _) -> 0 + (Just fees', _) -> fromIntegral fees' + pure + DB.Tx + { DB.txHash = txHash + , DB.txBlockId = blkId + , DB.txBlockIndex = Generic.txBlockIndex tx + , DB.txOutSum = DB.DbLovelace outSum + , DB.txFee = DB.DbLovelace fees -- may be wrong if txValidContract is False or outsputs are disabled + , DB.txDeposit = Nothing -- leaving this Nothing for now + , DB.txSize = Generic.txSize tx + , DB.txInvalidBefore = DbWord64 . unSlotNo <$> Generic.txInvalidBefore tx + , DB.txInvalidHereafter = DbWord64 . unSlotNo <$> Generic.txInvalidHereafter tx + , DB.txValidContract = Generic.txValidContract tx + , DB.txScriptSize = sum $ Generic.txScriptSizes tx + , DB.txTreasuryDonation = DB.DbLovelace (fromIntegral treasuryDonation) + } + where + txHash = Generic.txHash tx + outSum = fromIntegral $ unCoin $ Generic.txOutSum tx + treasuryDonation = unCoin $ Generic.txTreasuryDonation tx + +prepareTxIn :: + (MonadBaseControl IO m, MonadIO m) => + SyncEnv -> + [(TxIdLedger, DB.TxId)] -> + DB.TxId -> + Generic.TxIn -> + ExceptT SyncNodeError (ReaderT SqlBackend m) ExtendedTxIn +prepareTxIn syncEnv txHashes txId txIn = do + txOutTxId <- prepareResolveTxInputs syncEnv txHashes txInKey + pure $ ExtendedTxIn (txInDb txOutTxId) (Left txInKey) + where + txInKey = Generic.txInKey txIn + txInDb txOutTxId = + DB.TxIn + { DB.txInTxInId = txId + , DB.txInTxOutId = txOutTxId + , DB.txInTxOutIndex = fromIntegral $ Generic.txInIndex (Generic.txInKey txIn) + , DB.txInRedeemerId = Nothing -- Remove or fix later https://github.com/IntersectMBO/cardano-db-sync/issues/1746 + } + +-- let txId = DB.TxKey $ fromIntegral $ fromIntegral (DB.unBlockId blkId) * 1000 + blockIndex + +insertTxRest :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> - IsPoolMember -> DB.BlockId -> EpochNo -> SlotNo -> ApplyResult -> - Word64 -> + DB.TxId -> + DB.Tx -> Generic.Tx -> - BlockGroupedData -> - ExceptT SyncNodeError (ReaderT SqlBackend m) BlockGroupedData -insertTx syncEnv isMember blkId epochNo slotNo applyResult blockIndex tx grouped = do - let !txHash = Generic.txHash tx - let !mdeposits = if not (Generic.txValidContract tx) then Just (Coin 0) else lookupDepositsMap txHash (apDepositsMap applyResult) - let !outSum = fromIntegral $ unCoin $ Generic.txOutSum tx - !withdrawalSum = fromIntegral $ unCoin $ Generic.txWithdrawalSum tx - !treasuryDonation = unCoin $ Generic.txTreasuryDonation tx - hasConsumed = getHasConsumedOrPruneTxOut syncEnv - txIn = Generic.txInputs tx + ExceptT SyncNodeError (ReaderT SqlBackend m) () +insertTxRest syncEnv blkId epochNo slotNo applyResult txId _ tx = do disInOut <- liftIO $ getDisableInOutState syncEnv - -- In some txs and with specific configuration we may be able to find necessary data within the tx body. - -- In these cases we can avoid expensive queries. - (resolvedInputs, fees', deposits) <- case (disInOut, mdeposits, unCoin <$> Generic.txFees tx) of - (True, _, _) -> pure ([], 0, unCoin <$> mdeposits) - (_, Just deposits, Just fees) -> do - (resolvedInputs, _) <- splitLast <$> mapM (resolveTxInputs syncEnv hasConsumed False (fst <$> groupedTxOut grouped)) txIn - pure (resolvedInputs, fees, Just (unCoin deposits)) - (_, Nothing, Just fees) -> do - (resolvedInputs, amounts) <- splitLast <$> mapM (resolveTxInputs syncEnv hasConsumed False (fst <$> groupedTxOut grouped)) txIn - if any isNothing amounts - then pure (resolvedInputs, fees, Nothing) - else - let !inSum = sum $ map unDbLovelace $ catMaybes amounts - in pure (resolvedInputs, fees, Just $ fromIntegral (inSum + withdrawalSum) - fromIntegral outSum - fees - treasuryDonation) - (_, _, Nothing) -> do - -- Nothing in fees means a phase 2 failure - (resolvedInsFull, amounts) <- splitLast <$> mapM (resolveTxInputs syncEnv hasConsumed True (fst <$> groupedTxOut grouped)) txIn - let !inSum = sum $ map unDbLovelace $ catMaybes amounts - !diffSum = if inSum >= outSum then inSum - outSum else 0 - !fees = maybe diffSum (fromIntegral . unCoin) (Generic.txFees tx) - pure (resolvedInsFull, fromIntegral fees, Just 0) - let fees = fromIntegral fees' - -- Insert transaction and get txId from the DB. - !txId <- - lift - . DB.insertTx - $ DB.Tx - { DB.txHash = txHash - , DB.txBlockId = blkId - , DB.txBlockIndex = blockIndex - , DB.txOutSum = DB.DbLovelace outSum - , DB.txFee = DB.DbLovelace fees - , DB.txDeposit = fromIntegral <$> deposits - , DB.txSize = Generic.txSize tx - , DB.txInvalidBefore = DbWord64 . unSlotNo <$> Generic.txInvalidBefore tx - , DB.txInvalidHereafter = DbWord64 . unSlotNo <$> Generic.txInvalidHereafter tx - , DB.txValidContract = Generic.txValidContract tx - , DB.txScriptSize = sum $ Generic.txScriptSizes tx - , DB.txTreasuryDonation = DB.DbLovelace (fromIntegral treasuryDonation) - } - tryUpdateCacheTx cache (Generic.txLedgerTxId tx) txId when (ioTxCBOR iopts) $ do void @@ -136,86 +175,56 @@ insertTx syncEnv isMember blkId epochNo slotNo applyResult blockIndex tx grouped , DB.txCborBytes = Generic.txCBOR tx } - if not (Generic.txValidContract tx) - then do - !txOutsGrouped <- mapM (insertTxOut tracer cache iopts (txId, txHash)) (Generic.txOutputs tx) - - let !txIns = map (prepareTxIn txId Map.empty) resolvedInputs - -- There is a custom semigroup instance for BlockGroupedData which uses addition for the values `fees` and `outSum`. - -- Same happens bellow on last line of this function. - pure (grouped <> BlockGroupedData txIns txOutsGrouped [] [] fees outSum) - else do - -- The following operations only happen if the script passes stage 2 validation (or the tx has - -- no script). - !txOutsGrouped <- mapM (insertTxOut tracer cache iopts (txId, txHash)) (Generic.txOutputs tx) - - !redeemers <- - Map.fromList - <$> whenFalseMempty - (ioPlutusExtra iopts) - (mapM (insertRedeemer syncEnv disInOut (fst <$> groupedTxOut grouped) txId) (Generic.txRedeemer tx)) - - when (ioPlutusExtra iopts) $ do - mapM_ (insertDatum tracer cache txId) (Generic.txData tx) - mapM_ (insertCollateralTxIn syncEnv tracer txId) (Generic.txCollateralInputs tx) - mapM_ (insertReferenceTxIn syncEnv tracer txId) (Generic.txReferenceInputs tx) - mapM_ (insertCollateralTxOut tracer cache iopts (txId, txHash)) (Generic.txCollateralOutputs tx) - - txMetadata <- - whenFalseMempty (ioMetadata iopts) $ - insertTxMetadata - tracer - txId - iopts - (Generic.txMetadata tx) - mapM_ - (insertCertificate syncEnv isMember mDeposits blkId txId epochNo slotNo redeemers) - $ Generic.txCertificates tx - when (ioShelley iopts) $ - mapM_ (insertWithdrawals tracer cache txId redeemers) $ - Generic.txWithdrawals tx - when (ioShelley iopts) $ - mapM_ (lift . insertParamProposal blkId txId) $ - Generic.txParamProposal tx - - maTxMint <- - whenFalseMempty (ioMultiAssets iopts) $ - insertMaTxMint tracer cache txId $ - Generic.txMint tx - - when (ioPlutusExtra iopts) $ - mapM_ (lift . insertScript tracer txId) $ - Generic.txScripts tx - - when (ioPlutusExtra iopts) $ - mapM_ (insertExtraKeyWitness tracer txId) $ - Generic.txExtraKeyWitnesses tx - - when (ioGov iopts) $ do - mapM_ (insertGovActionProposal tracer cache blkId txId (getGovExpiresAt applyResult epochNo) (apGovActionState applyResult)) $ zip [0 ..] (Generic.txProposalProcedure tx) - mapM_ (insertVotingProcedures tracer cache blkId txId) (Generic.txVotingProcedure tx) - - let !txIns = map (prepareTxIn txId redeemers) resolvedInputs - pure (grouped <> BlockGroupedData txIns txOutsGrouped txMetadata maTxMint fees outSum) + when (Generic.txValidContract tx) $ do + -- The following operations only happen if the script passes stage 2 validation (or the tx has + -- no script). + !redeemers <- + Map.fromList + <$> whenFalseMempty + (ioPlutusExtra iopts) + (mapM (insertRedeemer syncEnv disInOut [] txId) (Generic.txRedeemer tx)) -- TODO leaving this empty for now + when (ioPlutusExtra iopts) $ do + mapM_ (insertDatum tracer cache txId) (Generic.txData tx) + mapM_ (insertCollateralTxIn syncEnv txId) (Generic.txInKey <$> Generic.txCollateralInputs tx) + mapM_ (insertReferenceTxIn syncEnv txId) (Generic.txInKey <$> Generic.txReferenceInputs tx) + mapM_ (insertCollateralTxOut syncEnv iopts (txId, Generic.txHash tx)) (Generic.txCollateralOutputs tx) + mapM_ + (insertCertificate syncEnv isMember mDeposits blkId txId epochNo slotNo redeemers) + $ Generic.txCertificates tx + when (ioShelley iopts) $ + mapM_ (insertWithdrawals syncEnv txId redeemers) $ + Generic.txWithdrawals tx + when (ioShelley iopts) $ + mapM_ (lift . insertParamProposal blkId txId) $ + Generic.txParamProposal tx + when (ioPlutusExtra iopts) $ + mapM_ (lift . insertScript tracer txId) $ + Generic.txScripts tx + when (ioPlutusExtra iopts) $ + mapM_ (insertExtraKeyWitness tracer txId) $ + Generic.txExtraKeyWitnesses tx + when (ioGov iopts) $ do + mapM_ (insertGovActionProposal syncEnv blkId txId (getGovExpiresAt applyResult epochNo) (apGovActionState applyResult)) $ zip [0 ..] (Generic.txProposalProcedure tx) + mapM_ (insertVotingProcedures syncEnv blkId txId) (Generic.txVotingProcedure tx) where tracer = getTrace syncEnv cache = envCache syncEnv iopts = getInsertOptions syncEnv mDeposits = maybeFromStrict $ apDeposits applyResult + isMember poolId = Set.member poolId (apPoolsRegistered applyResult) -------------------------------------------------------------------------------------- -- INSERT TXOUT -------------------------------------------------------------------------------------- -insertTxOut :: +prepareTxOut :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> InsertOptions -> (DB.TxId, ByteString) -> Generic.TxOut -> ExceptT SyncNodeError (ReaderT SqlBackend m) (ExtendedTxOut, [MissingMaTxOut]) -insertTxOut tracer cache iopts (txId, txHash) (Generic.TxOut index addr value maMap mScript dt) = do - mSaId <- lift $ insertStakeAddressRefIfMissing tracer cache addr +prepareTxOut syncEnv iopts (txId, txHash) (Generic.TxOut index addr value maMap mScript dt) = do + mSaId <- lift $ queryOrInsertStakeRef syncEnv addr mDatumId <- whenFalseEmpty (ioPlutusExtra iopts) Nothing $ Generic.whenInlineDatum dt $ @@ -284,22 +293,25 @@ insertTxOut tracer cache iopts (txId, txHash) (Generic.TxOut index addr value ma , V.txOutStakeAddressId = mSaId } -insertTxMetadata :: - (MonadBaseControl IO m, MonadIO m) => + tracer = getTrace syncEnv + cache = envCache syncEnv + +prepareTxMetadata :: + MonadIO m => Trace IO Text -> DB.TxId -> InsertOptions -> Maybe (Map Word64 TxMetadataValue) -> - ExceptT SyncNodeError (ReaderT SqlBackend m) [DB.TxMetadata] -insertTxMetadata tracer txId inOpts mmetadata = do + m [DB.TxMetadata] +prepareTxMetadata tracer txId inOpts mmetadata = do case mmetadata of Nothing -> pure [] Just metadata -> mapMaybeM prepare $ Map.toList metadata where prepare :: - (MonadBaseControl IO m, MonadIO m) => + MonadIO m => (Word64, TxMetadataValue) -> - ExceptT SyncNodeError (ReaderT SqlBackend m) (Maybe DB.TxMetadata) + m (Maybe DB.TxMetadata) prepare (key, md) = do case ioKeepMetadataNames inOpts of Strict.Just metadataNames -> do @@ -311,9 +323,9 @@ insertTxMetadata tracer txId inOpts mmetadata = do Strict.Nothing -> mkDbTxMetadata (key, md) mkDbTxMetadata :: - (MonadBaseControl IO m, MonadIO m) => + MonadIO m => (Word64, TxMetadataValue) -> - ExceptT SyncNodeError (ReaderT SqlBackend m) (Maybe DB.TxMetadata) + m (Maybe DB.TxMetadata) mkDbTxMetadata (key, md) = do let jsonbs = LBS.toStrict $ Aeson.encode (metadataValueToJsonNoSchema md) singleKeyCBORMetadata = serialiseTxMetadataToCbor $ Map.singleton key md @@ -330,14 +342,14 @@ insertTxMetadata tracer txId inOpts mmetadata = do -------------------------------------------------------------------------------------- -- INSERT MULTI ASSET -------------------------------------------------------------------------------------- -insertMaTxMint :: +prepareMaTxMint :: (MonadBaseControl IO m, MonadIO m) => Trace IO Text -> CacheStatus -> DB.TxId -> MultiAsset StandardCrypto -> ExceptT SyncNodeError (ReaderT SqlBackend m) [DB.MaTxMint] -insertMaTxMint _tracer cache txId (MultiAsset mintMap) = +prepareMaTxMint _tracer cache txId (MultiAsset mintMap) = concatMapM (lift . prepareOuter) $ Map.toList mintMap where prepareOuter :: @@ -395,14 +407,13 @@ insertMaTxOuts _tracer cache maMap = -------------------------------------------------------------------------------------- insertCollateralTxOut :: (MonadBaseControl IO m, MonadIO m) => - Trace IO Text -> - CacheStatus -> + SyncEnv -> InsertOptions -> (DB.TxId, ByteString) -> Generic.TxOut -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertCollateralTxOut tracer cache iopts (txId, _txHash) (Generic.TxOut index addr value maMap mScript dt) = do - mSaId <- lift $ insertStakeAddressRefIfMissing tracer cache addr +insertCollateralTxOut syncEnv iopts (txId, _txHash) (Generic.TxOut index addr value maMap mScript dt) = do + mSaId <- lift $ queryOrInsertStakeRef syncEnv addr mDatumId <- whenFalseEmpty (ioPlutusExtra iopts) Nothing $ Generic.whenInlineDatum dt $ @@ -460,14 +471,16 @@ insertCollateralTxOut tracer cache iopts (txId, _txHash) (Generic.TxOut index ad hasScript :: Bool hasScript = maybe False Generic.hasCredScript (Generic.getPaymentCred addr) + tracer = getTrace syncEnv + cache = envCache syncEnv + insertCollateralTxIn :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> - Trace IO Text -> DB.TxId -> - Generic.TxIn -> + Generic.TxInKey -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertCollateralTxIn syncEnv _tracer txInId txIn = do +insertCollateralTxIn syncEnv txInId txIn = do let txId = txInTxId txIn txOutId <- liftLookupFail "insertCollateralTxIn" $ queryTxIdWithCache (envCache syncEnv) txId void @@ -482,11 +495,10 @@ insertCollateralTxIn syncEnv _tracer txInId txIn = do insertReferenceTxIn :: (MonadBaseControl IO m, MonadIO m) => SyncEnv -> - Trace IO Text -> DB.TxId -> - Generic.TxIn -> + Generic.TxInKey -> ExceptT SyncNodeError (ReaderT SqlBackend m) () -insertReferenceTxIn syncEnv _tracer txInId txIn = do +insertReferenceTxIn syncEnv txInId txIn = do let txId = txInTxId txIn txOutId <- liftLookupFail "insertReferenceTxIn" $ queryTxIdWithCache (envCache syncEnv) txId void @@ -497,25 +509,3 @@ insertReferenceTxIn syncEnv _tracer txInId txIn = do , DB.referenceTxInTxOutId = txOutId , DB.referenceTxInTxOutIndex = fromIntegral (txInIndex txIn) } - --------------------------------------------------------------------------------------- --- Prepare TX-IN --------------------------------------------------------------------------------------- -prepareTxIn :: - DB.TxId -> - Map Word64 DB.RedeemerId -> - (Generic.TxIn, DB.TxId, Either Generic.TxIn DB.TxOutIdW) -> - ExtendedTxIn -prepareTxIn txInId redeemers (txIn, txOutId, mTxOutId) = - ExtendedTxIn - { etiTxIn = txInDB - , etiTxOutId = mTxOutId - } - where - txInDB = - DB.TxIn - { DB.txInTxInId = txInId - , DB.txInTxOutId = txOutId - , DB.txInTxOutIndex = fromIntegral $ Generic.txInIndex txIn - , DB.txInRedeemerId = mlookup (Generic.txInRedeemerIndex txIn) redeemers - } diff --git a/cardano-db-sync/src/Cardano/DbSync/Era/Util.hs b/cardano-db-sync/src/Cardano/DbSync/Era/Util.hs index e9a4a5430..1446c3d05 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Era/Util.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Era/Util.hs @@ -1,52 +1,4 @@ -{-# LANGUAGE OverloadedStrings #-} {-# LANGUAGE NoImplicitPrelude #-} module Cardano.DbSync.Era.Util ( - liftLookupFail, - containsUnicodeNul, - safeDecodeUtf8, - safeDecodeToJson, -) where - -import Cardano.BM.Trace (Trace, logWarning) -import qualified Cardano.Db as DB -import Cardano.DbSync.Error -import Cardano.Prelude -import Control.Monad.Trans.Except.Extra (firstExceptT, newExceptT) -import qualified Data.ByteString.Char8 as BS -import qualified Data.Text as Text -import qualified Data.Text.Encoding as Text -import qualified Data.Text.Encoding.Error as Text - -liftLookupFail :: Monad m => Text -> m (Either DB.LookupFail a) -> ExceptT SyncNodeError m a -liftLookupFail loc = - firstExceptT (\lf -> SNErrDefault $ mconcat [loc, " ", show lf]) . newExceptT - -safeDecodeUtf8 :: ByteString -> IO (Either Text.UnicodeException Text) -safeDecodeUtf8 bs - | BS.any isNullChar bs = pure $ Left (Text.DecodeError (BS.unpack bs) (Just 0)) - | otherwise = try $ evaluate (Text.decodeUtf8With Text.strictDecode bs) - where - isNullChar :: Char -> Bool - isNullChar ch = ord ch == 0 - -containsUnicodeNul :: Text -> Bool -containsUnicodeNul = Text.isInfixOf "\\u000" - -safeDecodeToJson :: MonadIO m => Trace IO Text -> Text -> ByteString -> m (Maybe Text) -safeDecodeToJson tracer tracePrefix jsonBs = do - ejson <- liftIO $ safeDecodeUtf8 jsonBs - case ejson of - Left err -> do - liftIO . logWarning tracer $ - mconcat - [tracePrefix, ": Could not decode to UTF8: ", textShow err] - -- We have to insert - pure Nothing - Right json -> - -- See https://github.com/IntersectMBO/cardano-db-sync/issues/297 - if containsUnicodeNul json - then do - liftIO $ logWarning tracer $ tracePrefix <> "was recorded as null, due to a Unicode NUL character found when trying to parse the json." - pure Nothing - else pure $ Just json + ) where diff --git a/cardano-db-sync/src/Cardano/DbSync/Error.hs b/cardano-db-sync/src/Cardano/DbSync/Error.hs index e01a3d3ba..1ee7fa199 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Error.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Error.hs @@ -10,10 +10,10 @@ module Cardano.DbSync.Error ( annotateInvariantTx, bsBase16Encode, dbSyncNodeError, - dbSyncInvariant, + liftLookupFail, renderSyncInvariant, runOrThrowIO, - fromEitherSTM, + throwLeftIO, logAndThrowIO, shouldAbortOnPanic, hasAbortOnPanicEnv, @@ -24,9 +24,12 @@ import qualified Cardano.Chain.Genesis as Byron import qualified Cardano.Chain.UTxO as Byron import qualified Cardano.Crypto as Crypto (serializeCborHash) import qualified Cardano.DbSync.Era.Byron.Util as Byron -import Cardano.DbSync.Util + +-- import Cardano.DbSync.Util + +import qualified Cardano.Db as DB import Cardano.Prelude -import Control.Monad.Trans.Except.Extra (left) +import Control.Monad.Trans.Except.Extra (firstExceptT, left, newExceptT) import qualified Data.ByteString.Base16 as Base16 import Data.String (String) import qualified Data.Text as Text @@ -154,8 +157,9 @@ annotateInvariantTx tx ei = dbSyncNodeError :: (Monad m) => Text -> ExceptT SyncNodeError m a dbSyncNodeError = left . SNErrDefault -dbSyncInvariant :: (Monad m) => Text -> SyncInvariant -> ExceptT SyncNodeError m a -dbSyncInvariant loc = left . SNErrInvariant loc +liftLookupFail :: Monad m => Text -> m (Either DB.LookupFail a) -> ExceptT SyncNodeError m a +liftLookupFail loc = + firstExceptT (\lf -> SNErrDefault $ mconcat [loc, " ", show lf]) . newExceptT renderSyncInvariant :: SyncInvariant -> Text renderSyncInvariant ei = @@ -174,8 +178,8 @@ renderSyncInvariant ei = , textShow tx ] -fromEitherSTM :: (Exception e) => Either e a -> STM a -fromEitherSTM = either throwSTM return +throwLeftIO :: Exception e => Either e a -> IO a +throwLeftIO = either throwIO pure bsBase16Encode :: ByteString -> Text bsBase16Encode bs = diff --git a/cardano-db-sync/src/Cardano/DbSync/Ledger/Async.hs b/cardano-db-sync/src/Cardano/DbSync/Ledger/Async.hs new file mode 100644 index 000000000..1537f9a77 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Ledger/Async.hs @@ -0,0 +1,91 @@ +{-# LANGUAGE OverloadedStrings #-} + +module Cardano.DbSync.Ledger.Async where + +import Cardano.DbSync.Types +import Data.Set (Set) +import Data.Map (Map) +import Cardano.DbSync.Ledger.Types +import Cardano.Ledger.BaseTypes (EpochNo) +import Cardano.Ledger.Crypto (StandardCrypto) +import qualified Cardano.Ledger.EpochBoundary as Ledger +import Control.Concurrent.Class.MonadSTM.Strict +import qualified Control.Concurrent.STM.TBQueue as TBQ +import qualified Cardano.Ledger.Rewards as Ledger + +-------------------------------------------------------------------------------- +-- EpochStake +-------------------------------------------------------------------------------- + +newEpochStakeChannels :: IO EpochStakeChannels +newEpochStakeChannels = + -- This may never be more than 1. But let's keep it a queue for extensibility shake. + -- This may allow us to parallelize the events workload even further + EpochStakeChannels + <$> TBQ.newTBQueueIO 1 + <*> newTVarIO Nothing + +-- To be used by the main thread +ensureStakeDone :: EpochStakeChannels -> EpochNo -> Ledger.SnapShot StandardCrypto -> IO () +ensureStakeDone sQueue epoch snapshot = atomically $ do + mLastEpochDone <- waitStakeFinished sQueue + case mLastEpochDone of + Just lastEpochDone | lastEpochDone == epoch -> pure () + _ -> do + -- If last is not already there, put it to list and wait again + writeEpochStakeAction sQueue epoch snapshot True + retry + +-- To be used by the main thread +waitStakeFinished :: EpochStakeChannels -> STM IO (Maybe EpochNo) +waitStakeFinished sQueue = do + stakeThreadState <- readTVar (epochResult sQueue) + case stakeThreadState of + Just (lastEpoch, Done) -> pure $ Just lastEpoch -- Normal case + Just (_, Running) -> retry -- Wait to finish current work. + Nothing -> pure Nothing -- This will happen after a restart + +-- To be used by the main thread +writeEpochStakeAction :: EpochStakeChannels -> EpochNo -> Ledger.SnapShot StandardCrypto -> Bool -> STM IO () +writeEpochStakeAction sQueue epoch snapShot checkFirst = do + TBQ.writeTBQueue (estakeQueue sQueue) $ EpochStakeDBAction epoch snapShot checkFirst + writeTVar (epochResult sQueue) $ Just (epoch, Running) + + +-------------------------------------------------------------------------------- +-- Rewards +-------------------------------------------------------------------------------- + +newRewardsChannels :: IO RewardsChannels +newRewardsChannels = + -- This may never be more than 1. But let's keep it a queue for extensibility shake. + -- This may allow us to parallelize the events workload even further + RewardsChannels + <$> TBQ.newTBQueueIO 1 + <*> newTVarIO Nothing + +-- To be used by the main thread +ensureRewardsDone :: RewardsChannels -> EpochNo -> EpochNo -> Map StakeCred (Set (Ledger.Reward StandardCrypto)) -> IO () +ensureRewardsDone sQueue epoch epoch' mp = atomically $ do + mLastEpochDone <- waitRewardsFinished sQueue + case mLastEpochDone of + Just lastEpochDone | lastEpochDone == epoch -> pure () + _ -> do + -- If last is not already there, put it to list and wait again + writeRewardsAction sQueue epoch epoch' mp True + retry + +-- To be used by the main thread +waitRewardsFinished :: RewardsChannels -> STM IO (Maybe EpochNo) +waitRewardsFinished sQueue = do + rewardsThreadState <- readTVar (rewardsResult sQueue) + case rewardsThreadState of + Just (lastEpoch, Done) -> pure $ Just lastEpoch -- Normal case + Just (_, Running) -> retry -- Wait to finish current work. + Nothing -> pure Nothing -- This will happen after a restart + +-- To be used by the main thread +writeRewardsAction :: RewardsChannels -> EpochNo -> EpochNo -> Map StakeCred (Set (Ledger.Reward StandardCrypto)) -> Bool -> STM IO () +writeRewardsAction sQueue epoch epoch' mp checkFirst = do + TBQ.writeTBQueue (rQueue sQueue) $ RewardsDBAction epoch epoch' mp checkFirst + writeTVar (rewardsResult sQueue) $ Just (epoch, Running) diff --git a/cardano-db-sync/src/Cardano/DbSync/Ledger/Event.hs b/cardano-db-sync/src/Cardano/DbSync/Ledger/Event.hs index 24c0e8617..ee37fe593 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Ledger/Event.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Ledger/Event.hs @@ -26,7 +26,6 @@ import qualified Cardano.DbSync.Era.Shelley.Generic as Generic import Cardano.DbSync.Era.Shelley.Generic.Tx.Shelley import Cardano.DbSync.Types import Cardano.DbSync.Util -import Cardano.Ledger.Address (RewardAccount) import qualified Cardano.Ledger.Allegra.Rules as Allegra import Cardano.Ledger.Alonzo.Rules (AlonzoBbodyEvent (..), AlonzoUtxoEvent (..), AlonzoUtxowEvent (..)) import qualified Cardano.Ledger.Alonzo.Rules as Alonzo @@ -83,8 +82,8 @@ data LedgerEvent data GovActionRefunded = GovActionRefunded { garGovActionId :: GovActionId StandardCrypto , garDeposit :: Coin - , garReturnAddr :: RewardAccount StandardCrypto - , garMTreasury :: Maybe (Map (RewardAccount StandardCrypto) Coin) + , garReturnAddr :: RewAccount + , garMTreasury :: Maybe (Map RewAccount Coin) } deriving (Eq) diff --git a/cardano-db-sync/src/Cardano/DbSync/Ledger/State.hs b/cardano-db-sync/src/Cardano/DbSync/Ledger/State.hs index 60bb24f12..7403092ae 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Ledger/State.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Ledger/State.hs @@ -32,7 +32,6 @@ module Cardano.DbSync.Ledger.State ( hashToAnnotation, getHeaderHash, runLedgerStateWriteThread, - getStakeSlice, getSliceMeta, findProposedCommittee, ) where @@ -75,7 +74,8 @@ import qualified Control.Exception as Exception import qualified Data.ByteString.Base16 as Base16 import Cardano.DbSync.Api.Types (InsertOptions (..), LedgerEnv (..), SyncOptions (..)) -import Cardano.DbSync.Error (SyncNodeError (..), fromEitherSTM) +import Cardano.DbSync.Error (SyncNodeError (..), throwLeftIO) +import Cardano.DbSync.Ledger.Async import Cardano.Ledger.BaseTypes (StrictMaybe) import Cardano.Ledger.Conway.Core as Shelley import Cardano.Ledger.Conway.Governance @@ -133,6 +133,8 @@ import System.Directory (doesFileExist, listDirectory, removeFile) import System.FilePath (dropExtension, takeExtension, ()) import System.Mem (performMajorGC) import Prelude (String, id) +import Cardano.Ledger.Shelley.RewardUpdate +import Control.Monad.Extra (whenJust) -- Note: The decision on whether a ledger-state is written to disk is based on the block number -- rather than the slot number because while the block number is fully populated (for every block @@ -175,6 +177,9 @@ mkHasLedgerEnv trce protoInfo dir nw systemStart syncOptions = do svar <- newTVarIO Strict.Nothing intervar <- newTVarIO Strict.Nothing swQueue <- newTBQueueIO 5 -- Should be relatively shallow. + stakeChans <- newEpochStakeChannels + rewardsChans <- newRewardsChannels + applyQueue <- newTBQueueIO 10 pure HasLedgerEnv { leTrace = trce @@ -190,6 +195,9 @@ mkHasLedgerEnv trce protoInfo dir nw systemStart syncOptions = do , leInterpreter = intervar , leStateVar = svar , leStateWriteQueue = swQueue + , leApplyQueue = applyQueue + , leEpochStakeChans = stakeChans + , leRewardsChans = rewardsChans } initCardanoLedgerState :: Consensus.ProtocolInfo CardanoBlock -> CardanoLedgerState @@ -226,37 +234,37 @@ applyBlockAndSnapshot ledgerEnv blk isCons = do applyBlock :: HasLedgerEnv -> CardanoBlock -> IO (CardanoLedgerState, ApplyResult) applyBlock env blk = do time <- getCurrentTime - atomically $ do - !ledgerDB <- readStateUnsafe env - let oldState = ledgerDbCurrent ledgerDB - !result <- fromEitherSTM $ tickThenReapplyCheckHash (ExtLedgerCfg (getTopLevelconfigHasLedger env)) blk (clsState oldState) - let ledgerEventsFull = mapMaybe (convertAuxLedgerEvent (leHasRewards env)) (lrEvents result) - let (ledgerEvents, deposits) = splitDeposits ledgerEventsFull - let !newLedgerState = finaliseDrepDistr (lrResult result) - !details <- getSlotDetails env (ledgerState newLedgerState) time (cardanoBlockSlotNo blk) - !newEpoch <- fromEitherSTM $ mkOnNewEpoch (clsState oldState) newLedgerState (findAdaPots ledgerEvents) - let !newEpochBlockNo = applyToEpochBlockNo (isJust $ blockIsEBB blk) (isJust newEpoch) (clsEpochBlockNo oldState) - let !newState = CardanoLedgerState newLedgerState newEpochBlockNo - let !ledgerDB' = pushLedgerDB ledgerDB newState - writeTVar (leStateVar env) (Strict.Just ledgerDB') - let !appResult = - if leUseLedger env - then - ApplyResult - { apPrices = getPrices newState - , apGovExpiresAfter = getGovExpiration newState - , apPoolsRegistered = getRegisteredPools oldState - , apNewEpoch = maybeToStrict newEpoch - , apOldLedger = Strict.Just oldState - , apDeposits = maybeToStrict $ Generic.getDeposits newLedgerState - , apSlotDetails = details - , apStakeSlice = getStakeSlice env newState False - , apEvents = ledgerEvents - , apGovActionState = getGovState newLedgerState - , apDepositsMap = DepositsMap deposits - } - else defaultApplyResult details - pure (oldState, appResult) + !ledgerDB <- atomically $ readStateUnsafe env + let oldState = ledgerDbCurrent ledgerDB + !result <- throwLeftIO $ tickThenReapplyCheckHash (ExtLedgerCfg (getTopLevelconfigHasLedger env)) blk (clsState oldState) + let ledgerEventsFull = mapMaybe (convertAuxLedgerEvent (leHasRewards env)) (lrEvents result) + let (ledgerEvents, deposits) = splitDeposits ledgerEventsFull + let !newLedgerState = finaliseDrepDistr (lrResult result) + !details <- atomically $ getSlotDetails env (ledgerState newLedgerState) time (cardanoBlockSlotNo blk) + !newEpoch <- throwLeftIO $ mkOnNewEpoch (clsState oldState) newLedgerState (findAdaPots ledgerEvents) + let !newEpochBlockNo = applyToEpochBlockNo (isJust $ blockIsEBB blk) (isJust newEpoch) (clsEpochBlockNo oldState) + let !newState = CardanoLedgerState newLedgerState newEpochBlockNo + asyncWriteStakeSnapShot env oldState newState + asyncWriteRewards env newState (sdEpochNo details) + let !ledgerDB' = pushLedgerDB ledgerDB newState + atomically $ writeTVar (leStateVar env) (Strict.Just ledgerDB') + let !appResult = + if leUseLedger env + then + ApplyResult + { apPrices = getPrices newState + , apGovExpiresAfter = getGovExpiration newState + , apPoolsRegistered = getRegisteredPools oldState + , apNewEpoch = maybeToStrict newEpoch + , apOldLedger = Strict.Just oldState + , apDeposits = maybeToStrict $ Generic.getDeposits newLedgerState + , apSlotDetails = details + , apEvents = ledgerEvents + , apGovActionState = getGovState newLedgerState + , apDepositsMap = DepositsMap deposits + } + else defaultApplyResult details + pure (oldState, appResult) where mkOnNewEpoch :: ExtLedgerState CardanoBlock -> ExtLedgerState CardanoBlock -> Maybe AdaPots -> Either SyncNodeError (Maybe Generic.NewEpoch) mkOnNewEpoch oldState newState mPots = do @@ -305,16 +313,26 @@ getGovState ls = case ledgerState ls of Just $ Consensus.shelleyLedgerState cls ^. Shelley.newEpochStateGovStateL _ -> Nothing -getStakeSlice :: HasLedgerEnv -> CardanoLedgerState -> Bool -> Generic.StakeSliceRes -getStakeSlice env cls isMigration = - case clsEpochBlockNo cls of - EpochBlockNo n -> - Generic.getStakeSlice - (leProtocolInfo env) - n - (clsState cls) - isMigration - _ -> Generic.NoSlices +asyncWriteStakeSnapShot :: HasLedgerEnv -> CardanoLedgerState -> CardanoLedgerState -> IO () +asyncWriteStakeSnapShot env oldState newState = + case clsEpochBlockNo newState of + -- start inserting after the 3rd block. This limits the chances that due to an epoch boundary + -- rollback we will have to delete the snapshot. + EpochBlockNo n + | n == 2 + , Just (snapshot, epoch) <- Generic.getSnapShot (clsState newState) -> do + atomically $ writeEpochStakeAction (leEpochStakeChans env) epoch snapshot False + -- on the next epoch boundary makes sure that the epoch stake thread has finished. + EpochBlockNo n + | n == 0 + , Just (snapshot, epoch) <- Generic.getSnapShot (clsState oldState) -> do + ensureStakeDone (leEpochStakeChans env) epoch snapshot + _ -> pure () + +asyncWriteRewards :: HasLedgerEnv -> CardanoLedgerState -> EpochNo -> IO () +asyncWriteRewards env newState e = + whenJust (Generic.getRewardsUpdate (getTopLevelconfigHasLedger env) (clsState newState)) $ \ru -> do + atomically $ writeRewardsAction (leRewardsChans env) e e (rs ru) True -- (e-1) (e+1) getSliceMeta :: Generic.StakeSliceRes -> Maybe (Bool, EpochNo) getSliceMeta (Generic.Slice (Generic.StakeSlice epochNo _) isFinal) = Just (isFinal, epochNo) diff --git a/cardano-db-sync/src/Cardano/DbSync/Ledger/Types.hs b/cardano-db-sync/src/Cardano/DbSync/Ledger/Types.hs index 282c833ef..9addb4134 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Ledger/Types.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Ledger/Types.hs @@ -21,12 +21,14 @@ import Cardano.DbSync.Types ( CardanoPoint, PoolKeyHash, SlotDetails, + StakeCred, ) import Cardano.Ledger.Alonzo.Scripts (Prices) import qualified Cardano.Ledger.BaseTypes as Ledger import Cardano.Ledger.Coin (Coin) import Cardano.Ledger.Conway.Governance import Cardano.Ledger.Credential (Credential (..)) +import qualified Cardano.Ledger.EpochBoundary as Ledger import Cardano.Ledger.Keys (KeyRole (..)) import Cardano.Ledger.Shelley.LedgerState (NewEpochState ()) import Cardano.Prelude hiding (atomically) @@ -36,6 +38,7 @@ import Cardano.Slotting.Slot ( WithOrigin (..), ) import Control.Concurrent.Class.MonadSTM.Strict ( + StrictTMVar, StrictTVar, ) import Control.Concurrent.STM.TBQueue (TBQueue) @@ -53,6 +56,7 @@ import qualified Ouroboros.Consensus.Node.ProtocolInfo as Consensus import Ouroboros.Consensus.Shelley.Ledger (LedgerState (..), ShelleyBlock) import Ouroboros.Network.AnchoredSeq (Anchorable (..), AnchoredSeq (..)) import Prelude (fail, id) +import qualified Cardano.Ledger.Rewards as Ledger -------------------------------------------------------------------------- -- Ledger Types @@ -72,6 +76,9 @@ data HasLedgerEnv = HasLedgerEnv , leInterpreter :: !(StrictTVar IO (Strict.Maybe CardanoInterpreter)) , leStateVar :: !(StrictTVar IO (Strict.Maybe LedgerDB)) , leStateWriteQueue :: !(TBQueue (FilePath, CardanoLedgerState)) + , leApplyQueue :: TBQueue LedgerAction + , leEpochStakeChans :: EpochStakeChannels + , leRewardsChans :: RewardsChannels } data CardanoLedgerState = CardanoLedgerState @@ -140,7 +147,6 @@ data ApplyResult = ApplyResult , apOldLedger :: !(Strict.Maybe CardanoLedgerState) , apDeposits :: !(Strict.Maybe Generic.Deposits) -- The current required deposits , apSlotDetails :: !SlotDetails - , apStakeSlice :: !Generic.StakeSliceRes , apEvents :: ![LedgerEvent] , apGovActionState :: !(Maybe (ConwayGovState StandardConway)) , apDepositsMap :: !DepositsMap @@ -156,7 +162,6 @@ defaultApplyResult slotDetails = , apOldLedger = Strict.Nothing , apDeposits = Strict.Nothing , apSlotDetails = slotDetails - , apStakeSlice = Generic.NoSlices , apEvents = [] , apGovActionState = Nothing , apDepositsMap = emptyDepositsMap @@ -196,6 +201,34 @@ instance Anchorable (WithOrigin SlotNo) CardanoLedgerState CardanoLedgerState wh data SnapshotPoint = OnDisk LedgerStateFile | InMemory CardanoPoint +data LedgerAction = LedgerAction CardanoBlock LedgerResultResTMVar +type LedgerResultResTMVar = StrictTMVar IO (ApplyResult, Bool) + +data EpochStakeDBAction = EpochStakeDBAction + { esaEpochNo :: EpochNo + , esaSnapShot :: Ledger.SnapShot StandardCrypto + , esaCheckFirst :: Bool -- Check if the data is already there before inserting + } + +data EpochState = Running | Done +data EpochStakeChannels = EpochStakeChannels + { estakeQueue :: TBQueue EpochStakeDBAction + , epochResult :: StrictTVar IO (Maybe (EpochNo, EpochState)) + } + +data EpochRewardState = RewRunning | Inserted | Cleaned +data RewardsDBAction = RewardsDBAction + { raEpochNo :: EpochNo + , raEpochNo' :: EpochNo + , raSnapShot :: Map StakeCred (Set (Ledger.Reward StandardCrypto)) + , raCheckFirst :: Bool -- Check if the data is already there before inserting + } + +data RewardsChannels = RewardsChannels + { rQueue :: TBQueue RewardsDBAction + , rewardsResult :: StrictTVar IO (Maybe (EpochNo, EpochState)) + } + -- | Per-era pure getters and setters on @NewEpochState@. Note this is a bit of an abuse -- of the cardano-ledger/ouroboros-consensus public APIs, because ledger state is not -- designed to be updated this way. We are only replaying the chain, so this should be diff --git a/cardano-db-sync/src/Cardano/DbSync/Prepare/Tx/Types.hs b/cardano-db-sync/src/Cardano/DbSync/Prepare/Tx/Types.hs new file mode 100644 index 000000000..17741c072 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Prepare/Tx/Types.hs @@ -0,0 +1,38 @@ +module Cardano.DbSync.Prepare.Tx.Types where + +data Tx = Tx + { tx :: DB.Tx + , txCbor :: Maybe DB.TxCbor + , txGrouped :: Grouped + } + +data Grouped = Grouped + { txInputs :: [DB.TxIn] + , txOutputs :: [DB.TxOutW] + , txMetadata :: [DB.TxMetadata] + , txMint :: [DB.MaTxMint] + , txRedeemer :: [DB.Redeemer] + , txCollateralInputs :: [DB.CollateralTxIn] + , txReferenceInputs :: [DB.ReferenceTxIn] + , txCollateralOutputs :: [DB.CollateralTxOut] + , txCertificates :: [DBTxCertificate] + , txWithdrawals :: [DB.Withdrawal] + , txParamProposal :: [DB.ParamProposal] + , txExtraKeyWitness :: [DB.ExtraKeyWitness] + } + +data DBTxCertificate = + StakeRegistration DB.StakeRegistration + | StakeDeregistration DB.StakeDeregistration + | Delegation DB.Delegation + | PoolRegister DB.PoolMetadataRef DB.PoolUpdate [DB.PoolOwner] [DB.PoolRelay] + | PoolRetire DB.PoolRetire + | MirCert MirCert + | DrepRegistration (Maybe DB.VotingAnchor) DB.DrepRegistration + | DrepRegistration DB.DrepRegistration + | CommitteeRegistration DB.CommitteeRegistration + | CommitteeDeRegistration (Maybe DB.VotingAnchor) DB.CommitteeDeRegistration + +data MirCert = Reserve DB.Reserve | Treasury DB.Treasury | PotTransfer DB.PotTransfer + +data GovProposal = GovProposal DB.VotingAnchor DB.ParamProposal diff --git a/cardano-db-sync/src/Cardano/DbSync/Rollback.hs b/cardano-db-sync/src/Cardano/DbSync/Rollback.hs index 9124bae6d..ff0ec6223 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Rollback.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Rollback.hs @@ -14,7 +14,6 @@ import qualified Cardano.Db as DB import Cardano.DbSync.Api import Cardano.DbSync.Api.Types (SyncEnv (..)) import Cardano.DbSync.Cache -import Cardano.DbSync.Era.Util import Cardano.DbSync.Error import Cardano.DbSync.Types import Cardano.DbSync.Util diff --git a/cardano-db-sync/src/Cardano/DbSync/Sync.hs b/cardano-db-sync/src/Cardano/DbSync/Sync.hs index e8724185d..433338873 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Sync.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Sync.hs @@ -28,9 +28,8 @@ import Cardano.BM.Trace (Trace, appendName, logInfo) import qualified Cardano.BM.Trace as Logging import Cardano.Client.Subscription (subscribe) import Cardano.DbSync.Api -import Cardano.DbSync.Api.Types (ConsistentLevel (..), LedgerEnv (..), SyncEnv (..), envLedgerEnv, envNetworkMagic, envOptions) +import Cardano.DbSync.Api.Types (LedgerEnv (..), SyncEnv (..), envLedgerEnv, envNetworkMagic, envOptions) import Cardano.DbSync.Config -import Cardano.DbSync.Database import Cardano.DbSync.DbAction import Cardano.DbSync.LocalStateQuery import Cardano.DbSync.Metrics @@ -202,7 +201,6 @@ dbSyncProtocols syncEnv metricsSetters tc codecConfig version bversion = MiniProtocolCb $ \_ctx channel -> liftIO . logException tracer "ChainSyncWithBlocksPtcl: " $ do logInfo tracer "Starting ChainSync client" - setConsistentLevel syncEnv Unchecked (latestPoints, currentTip) <- waitRestartState tc let (inMemory, onDisk) = List.span snd latestPoints diff --git a/cardano-db-sync/src/Cardano/DbSync/Database.hs b/cardano-db-sync/src/Cardano/DbSync/Threads/Database.hs similarity index 60% rename from cardano-db-sync/src/Cardano/DbSync/Database.hs rename to cardano-db-sync/src/Cardano/DbSync/Threads/Database.hs index 4583b8204..70fad0045 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Database.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Threads/Database.hs @@ -1,9 +1,11 @@ {-# LANGUAGE FlexibleInstances #-} {-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE TupleSections #-} +{-# LANGUAGE TypeApplications #-} {-# LANGUAGE TypeFamilies #-} {-# LANGUAGE NoImplicitPrelude #-} -module Cardano.DbSync.Database ( +module Cardano.DbSync.Threads.Database ( DbAction (..), ThreadChannels, lengthDbActionQueue, @@ -11,27 +13,32 @@ module Cardano.DbSync.Database ( runDbThread, ) where -import Cardano.BM.Trace (logDebug, logError, logInfo) +import Cardano.BM.Trace (logError, logInfo) +import qualified Cardano.Db as DB import Cardano.DbSync.Api import Cardano.DbSync.Api.Types (ConsistentLevel (..), LedgerEnv (..), SyncEnv (..)) +import Cardano.DbSync.Block import Cardano.DbSync.DbAction -import Cardano.DbSync.Default import Cardano.DbSync.Error import Cardano.DbSync.Ledger.State -import Cardano.DbSync.Ledger.Types (CardanoLedgerState (..), SnapshotPoint (..)) +import Cardano.DbSync.Ledger.Types (CardanoLedgerState (..), LedgerStateFile (..), SnapshotPoint (..)) import Cardano.DbSync.Metrics import Cardano.DbSync.Rollback import Cardano.DbSync.Types import Cardano.DbSync.Util import Cardano.Prelude hiding (atomically) -import Cardano.Slotting.Slot (WithOrigin (..)) +import Cardano.Slotting.Slot (EpochNo (..), SlotNo (..), WithOrigin (..)) import Control.Concurrent.Class.MonadSTM.Strict import Control.Monad.Extra (whenJust) import Control.Monad.Trans.Except.Extra (newExceptT) +import Control.Monad.Trans.Maybe (MaybeT (..)) +import Database.Persist.Sql (SqlBackend) +import Ouroboros.Consensus.Block.Abstract (HeaderHash, Point (..), fromRawHash) import Ouroboros.Consensus.HeaderValidation hiding (TipInfo) import Ouroboros.Consensus.Ledger.Extended -import Ouroboros.Network.Block (BlockNo, Point (..)) +import Ouroboros.Network.Block (BlockNo (..), Point (..)) import Ouroboros.Network.Point (blockPointHash, blockPointSlot) +import qualified Ouroboros.Network.Point as Point data NextState = Continue @@ -52,9 +59,6 @@ runDbThread syncEnv metricsSetters queue = do loop = do xs <- blockingFlushDbActionQueue queue - when (length xs > 1) $ do - logDebug trce $ "runDbThread: " <> textShow (length xs) <> " blocks" - case hasRestart xs of Nothing -> do eNextState <- runExceptT $ runActions syncEnv xs @@ -115,7 +119,7 @@ runActions syncEnv actions = do lift $ atomically $ putTMVar resultVar (points, blockNo) dbAction Continue ys (ys, zs) -> do - newExceptT $ insertListBlocks syncEnv ys + insertListBlocks syncEnv ys if null zs then pure Continue else dbAction Continue zs @@ -153,7 +157,7 @@ validateConsistentLevel syncEnv stPoint = do compareTips stPoint dbTipInfo cLevel where compareTips _ dbTip Unchecked = - logAndThrowIO tracer $ + logAndThrowIO trce $ SNErrDatabaseValConstLevel $ "Found Unchecked Consistent Level. " <> showContext dbTip Unchecked compareTips (Point Origin) Nothing Consistent = pure () @@ -165,11 +169,11 @@ validateConsistentLevel syncEnv stPoint = do compareTips (Point (At blk)) (Just tip) DBAheadOfLedger | blockPointSlot blk <= bSlotNo tip = pure () compareTips _ dbTip cLevel = - logAndThrowIO tracer $ + logAndThrowIO trce $ SNErrDatabaseValConstLevel $ "Unexpected Consistent Level. " <> showContext dbTip cLevel - tracer = getTrace syncEnv + trce = getTrace syncEnv showContext dbTip cLevel = mconcat [ "Ledger state point is " @@ -193,3 +197,96 @@ hasRestart = go go [] = Nothing go (DbRestartState mvar : _) = Just mvar go (_ : rest) = go rest + +-- | 'True' is for in memory points and 'False' for on disk +getLatestPoints :: SyncEnv -> IO [(CardanoPoint, Bool)] +getLatestPoints env = do + case envLedgerEnv env of + HasLedger hasLedgerEnv -> do + snapshotPoints <- listKnownSnapshots hasLedgerEnv + verifySnapshotPoint env snapshotPoints + NoLedger _ -> do + -- Brings the 5 latest. + lastPoints <- DB.runDbIohkNoLogging (envBackend env) DB.queryLatestPoints + pure $ mapMaybe convert lastPoints + where + convert (Nothing, _) = Nothing + convert (Just slot, bs) = convertToDiskPoint (SlotNo slot) bs + +verifySnapshotPoint :: SyncEnv -> [SnapshotPoint] -> IO [(CardanoPoint, Bool)] +verifySnapshotPoint env snapPoints = + catMaybes <$> mapM validLedgerFileToPoint snapPoints + where + validLedgerFileToPoint :: SnapshotPoint -> IO (Maybe (CardanoPoint, Bool)) + validLedgerFileToPoint (OnDisk lsf) = do + hashes <- getSlotHash (envBackend env) (lsfSlotNo lsf) + let valid = find (\(_, h) -> lsfHash lsf == hashToAnnotation h) hashes + case valid of + Just (slot, hash) | slot == lsfSlotNo lsf -> pure $ convertToDiskPoint slot hash + _ -> pure Nothing + validLedgerFileToPoint (InMemory pnt) = do + case pnt of + GenesisPoint -> pure Nothing + BlockPoint slotNo hsh -> do + hashes <- getSlotHash (envBackend env) slotNo + let valid = find (\(_, dbHash) -> getHeaderHash hsh == dbHash) hashes + case valid of + Just (dbSlotNo, _) | slotNo == dbSlotNo -> pure $ Just (pnt, True) + _ -> pure Nothing + +convertToDiskPoint :: SlotNo -> ByteString -> Maybe (CardanoPoint, Bool) +convertToDiskPoint slot hashBlob = (,False) <$> convertToPoint slot hashBlob + +convertToPoint :: SlotNo -> ByteString -> Maybe CardanoPoint +convertToPoint slot hashBlob = + Point . Point.block slot <$> convertHashBlob hashBlob + where + convertHashBlob :: ByteString -> Maybe (HeaderHash CardanoBlock) + convertHashBlob = Just . fromRawHash (Proxy @CardanoBlock) + +getSlotHash :: SqlBackend -> SlotNo -> IO [(SlotNo, ByteString)] +getSlotHash backend = DB.runDbIohkNoLogging backend . DB.querySlotHash + +getDbLatestBlockInfo :: SqlBackend -> IO (Maybe TipInfo) +getDbLatestBlockInfo backend = do + runMaybeT $ do + block <- MaybeT $ DB.runDbIohkNoLogging backend DB.queryLatestBlock + -- The EpochNo, SlotNo and BlockNo can only be zero for the Byron + -- era, but we need to make the types match, hence `fromMaybe`. + pure $ + TipInfo + { bHash = DB.blockHash block + , bEpochNo = EpochNo . fromMaybe 0 $ DB.blockEpochNo block + , bSlotNo = SlotNo . fromMaybe 0 $ DB.blockSlotNo block + , bBlockNo = BlockNo . fromMaybe 0 $ DB.blockBlockNo block + } + +getDbTipBlockNo :: SyncEnv -> IO (Point.WithOrigin BlockNo) +getDbTipBlockNo env = do + mblk <- getDbLatestBlockInfo (envBackend env) + pure $ maybe Point.Origin (Point.At . bBlockNo) mblk + +logDbState :: SyncEnv -> IO () +logDbState env = do + mblk <- getDbLatestBlockInfo (envBackend env) + case mblk of + Nothing -> logInfo trce "Database is empty" + Just tip -> logInfo trce $ mconcat ["Database tip is at ", showTip tip] + where + showTip :: TipInfo -> Text + showTip tipInfo = + mconcat + [ "slot " + , textShow (unSlotNo $ bSlotNo tipInfo) + , ", block " + , textShow (unBlockNo $ bBlockNo tipInfo) + ] + + trce = getTrace env + +getCurrentTipBlockNo :: SyncEnv -> IO (WithOrigin BlockNo) +getCurrentTipBlockNo env = do + maybeTip <- getDbLatestBlockInfo (envBackend env) + case maybeTip of + Just tip -> pure $ At (bBlockNo tip) + Nothing -> pure Origin diff --git a/cardano-db-sync/src/Cardano/DbSync/Threads/EpochStake.hs b/cardano-db-sync/src/Cardano/DbSync/Threads/EpochStake.hs new file mode 100644 index 000000000..4b51977d7 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Threads/EpochStake.hs @@ -0,0 +1,56 @@ +{-# LANGUAGE FlexibleContexts #-} +{-# LANGUAGE OverloadedStrings #-} + +module Cardano.DbSync.Threads.EpochStake where + +import Cardano.BM.Trace (logInfo) +import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types +import Cardano.DbSync.Era.Shelley.Generic.StakeDist +import Cardano.DbSync.Era.Universal.Epoch +import Cardano.DbSync.Error +import Cardano.DbSync.Ledger.Types +import Cardano.DbSync.Util +import Cardano.Slotting.Slot (EpochNo (..)) +import Control.Concurrent.Class.MonadSTM.Strict +import qualified Control.Concurrent.STM.TBQueue as TBQ +import Control.Monad +import Control.Monad.IO.Class (liftIO) +import Control.Monad.Trans.Class +import Control.Monad.Trans.Except.Extra (runExceptT) +import Database.Persist.Postgresql (IsolationLevel (..), runSqlConnWithIsolation, withPostgresqlConn) + +runEpochStakeThread :: + SyncEnv -> + IO () +runEpochStakeThread syncEnv = + case envLedgerEnv syncEnv of + NoLedger _ -> pure () + HasLedger le -> do + logInfo trce "Running Event thread" + logException trce "runEpochStakeThread: " (runESLoop syncEnv le) + logInfo trce "Shutting Event thread" + where + trce = getTrace syncEnv + +runESLoop :: SyncEnv -> HasLedgerEnv -> IO () +runESLoop syncEnv lenv = + DB.runIohkLogging trce $ + withPostgresqlConn (envConnectionString syncEnv) loop + where + loop backend = do + runOrThrowIO $ runSqlConnWithIsolation (runExceptT loopAction) backend Serializable + loop backend + + loopAction = do + EpochStakeDBAction epoch snapShot shouldCheck <- liftIO $ atomically $ TBQ.readTBQueue (estakeQueue estakeChan) + if shouldCheck + then do + stakeExists <- lift $ DB.queryEpochStakeExists (unEpochNo epoch) + unless stakeExists $ insertEpochStake syncEnv epoch (snapShotToList snapShot) + else insertEpochStake syncEnv epoch (snapShotToList snapShot) + liftIO $ atomically $ writeTVar (epochResult estakeChan) $ Just (epoch, Done) + + estakeChan = leEpochStakeChans lenv + trce = getTrace syncEnv diff --git a/cardano-db-sync/src/Cardano/DbSync/Threads/Ledger.hs b/cardano-db-sync/src/Cardano/DbSync/Threads/Ledger.hs new file mode 100644 index 000000000..7fd04b3a7 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Threads/Ledger.hs @@ -0,0 +1,78 @@ +{-# LANGUAGE FlexibleContexts #-} +{-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE TupleSections #-} + +module Cardano.DbSync.Threads.Ledger where + +import Cardano.BM.Trace (logInfo) +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types +import Cardano.DbSync.Ledger.State +import Cardano.DbSync.Ledger.Types +import Cardano.DbSync.LocalStateQuery +import Cardano.DbSync.Types +import Cardano.DbSync.Util +import Control.Concurrent.Class.MonadSTM.Strict ( + atomically, + newEmptyTMVarIO, + newTMVarIO, + writeTMVar, + ) +import qualified Control.Concurrent.STM.TBQueue as TBQ +import Control.Monad (forever) +import Control.Monad.IO.Class (liftIO) + +runLedgerThread :: + SyncEnv -> + IO () +runLedgerThread syncEnv = + case envLedgerEnv syncEnv of + NoLedger _ -> pure () + HasLedger le -> do + logInfo trce "Running Event thread" + logException trce "runEpochStakeThread: " (runLedgerLoop syncEnv le) + logInfo trce "Shutting Event thread" + where + trce = getTrace syncEnv + +runLedgerLoop :: SyncEnv -> HasLedgerEnv -> IO () +runLedgerLoop syncEnv lenv = forever $ do + LedgerAction cblk resVar <- liftIO $ atomically $ TBQ.readTBQueue (leApplyQueue lenv) + res <- applyBlockAction syncEnv lenv cblk True + atomically $ writeTMVar resVar res + +-- May be used by 2 different thread. Not at the same time. +applyBlockAction :: SyncEnv -> HasLedgerEnv -> CardanoBlock -> Bool -> IO (ApplyResult, Bool) +applyBlockAction syncEnv lenv cblk isCons = do + (applyRes, tookSnapshot) <- applyBlockAndSnapshot lenv cblk isCons + applyRes' <- addNewEventsAndSort syncEnv applyRes + pure (applyRes', tookSnapshot) + +-- Not used by the Ledger thread +noLedgerAction :: SyncEnv -> NoLedgerEnv -> CardanoBlock -> IO ApplyResult +noLedgerAction syncEnv nle cblk = do + slotDetails <- getSlotDetailsNode nle (cardanoBlockSlotNo cblk) + addNewEventsAndSort syncEnv $ defaultApplyResult slotDetails + +-- Not used by the Ledger thread +writeLedgerAction :: HasLedgerEnv -> CardanoBlock -> IO LedgerResultResTMVar +writeLedgerAction lenv cblock = do + resVar <- newEmptyTMVarIO + atomically $ TBQ.writeTBQueue (leApplyQueue lenv) $ LedgerAction cblock resVar + pure resVar + +-- Not used by the Ledger thread +asyncApplyResult :: SyncEnv -> CardanoBlock -> IO LedgerResultResTMVar +asyncApplyResult syncEnv cblk = + case envLedgerEnv syncEnv of + HasLedger hle -> writeLedgerAction hle cblk + NoLedger nle -> do + applyRes <- noLedgerAction syncEnv nle cblk + newTMVarIO (applyRes, False) + +-- Not used by the Ledger thread. This doesn't even send something to the thread. +mkApplyResult :: SyncEnv -> CardanoBlock -> IO (ApplyResult, Bool) +mkApplyResult syncEnv cblk = do + case envLedgerEnv syncEnv of + HasLedger hle -> applyBlockAction syncEnv hle cblk False + NoLedger nle -> (,False) <$> noLedgerAction syncEnv nle cblk diff --git a/cardano-db-sync/src/Cardano/DbSync/Threads/Promise.hs b/cardano-db-sync/src/Cardano/DbSync/Threads/Promise.hs new file mode 100644 index 000000000..0ed9842c6 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Threads/Promise.hs @@ -0,0 +1,16 @@ +module Cardano.DbSync.Threads.Promise where + +import Cardano.BM.Trace (logInfo) +import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types +import Cardano.DbSync.Cache.Stake +import Cardano.DbSync.Cache.Types +import Cardano.DbSync.Util +import Control.Concurrent.Class.MonadSTM.Strict +import qualified Control.Concurrent.STM.TBQueue as TBQ +import Control.Monad +import Control.Monad.IO.Class (liftIO) +import Database.Persist.Postgresql (IsolationLevel (..), runSqlConnWithIsolation, withPostgresqlConn) + +data Promise = \ No newline at end of file diff --git a/cardano-db-sync/src/Cardano/DbSync/Threads/Rewards.hs b/cardano-db-sync/src/Cardano/DbSync/Threads/Rewards.hs new file mode 100644 index 000000000..5744e1fb8 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Threads/Rewards.hs @@ -0,0 +1,54 @@ +{-# LANGUAGE FlexibleContexts #-} +{-# LANGUAGE OverloadedStrings #-} + +module Cardano.DbSync.Threads.Rewards where + +import Cardano.BM.Trace (logInfo) +import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types +import Cardano.DbSync.Era.Shelley.Generic.StakeDist +import Cardano.DbSync.Era.Universal.Epoch +import Cardano.DbSync.Error +import Cardano.DbSync.Ledger.Types +import Cardano.DbSync.Util +import Cardano.Slotting.Slot (EpochNo (..)) +import Control.Concurrent.Class.MonadSTM.Strict +import qualified Control.Concurrent.STM.TBQueue as TBQ +import Control.Monad +import Control.Monad.IO.Class (liftIO) +import Control.Monad.Trans.Class +import Control.Monad.Trans.Except.Extra (runExceptT) +import Database.Persist.Postgresql (IsolationLevel (..), runSqlConnWithIsolation, withPostgresqlConn) + +runRewardsThread :: + SyncEnv -> + IO () +runRewardsThread syncEnv = + case envLedgerEnv syncEnv of + NoLedger _ -> pure () + HasLedger le -> do + logInfo trce "Running Rewards thread" + logException trce "runRewardsThread: " (runRewLoop syncEnv le) + logInfo trce "Shutting Rewards thread" + where + trce = getTrace syncEnv + +runRewLoop :: SyncEnv -> HasLedgerEnv -> IO () +runRewLoop syncEnv lenv = + DB.runIohkLogging trce $ + withPostgresqlConn (envConnectionString syncEnv) loop + where + loop backend = do + runOrThrowIO $ runSqlConnWithIsolation (runExceptT loopAction) backend Serializable + loop backend + + loopAction = do + rAction <- liftIO $ atomically $ TBQ.readTBQueue (rQueue rewardsChan) + case rAction of + RewardsDBAction epoch epoch' mp shouldCheck -> do + insertRewards syncEnv epoch epoch' (toList $ convertPoolRewards mp) + liftIO $ atomically $ writeTVar (rewardsResult rewardsChan) $ Just (epoch', Done) + + rewardsChan = leRewardsChans lenv + trce = getTrace syncEnv diff --git a/cardano-db-sync/src/Cardano/DbSync/Threads/Stake.hs b/cardano-db-sync/src/Cardano/DbSync/Threads/Stake.hs new file mode 100644 index 000000000..a13044f55 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Threads/Stake.hs @@ -0,0 +1,45 @@ +{-# LANGUAGE FlexibleContexts #-} +{-# LANGUAGE OverloadedStrings #-} + +module Cardano.DbSync.Threads.Stake where + +import Cardano.BM.Trace (logInfo) +import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types +import Cardano.DbSync.Cache.Stake +import Cardano.DbSync.Cache.Types +import Cardano.DbSync.Util +import Control.Concurrent.Class.MonadSTM.Strict +import qualified Control.Concurrent.STM.TBQueue as TBQ +import Control.Monad +import Control.Monad.IO.Class (liftIO) +import Database.Persist.Postgresql (IsolationLevel (..), runSqlConnWithIsolation, withPostgresqlConn) + +runStakeThread :: SyncEnv -> IO () +runStakeThread syncEnv = do + logInfo trce "Running Event thread" + logException trce "runStakeThread: " (runStakeLoop syncEnv) + logInfo trce "Shutting Event thread" + where + trce = getTrace syncEnv + +runStakeLoop :: SyncEnv -> IO () +runStakeLoop syncEnv = + DB.runIohkLogging trce $ + withPostgresqlConn (envConnectionString syncEnv) actionDB + where + actionDB backend = runSqlConnWithIsolation (forever loopAction) backend Serializable + + loopAction = do + action <- liftIO $ atomically $ TBQ.readTBQueue (scPriorityQueue stakeChan) + case action of + QueryInsertStake rewardAcc ca resVar -> do + stakeId <- resolveInsertRewardAccount syncEnv ca rewardAcc + liftIO $ atomically $ writeTMVar resVar stakeId + CacheStake _ _ _ -> pure () + BulkPrefetch _ -> pure () + CommitStake -> DB.transactionCommit + + stakeChan = envStakeChans syncEnv + trce = getTrace syncEnv diff --git a/cardano-db-sync/src/Cardano/DbSync/Threads/TxInResolve.hs b/cardano-db-sync/src/Cardano/DbSync/Threads/TxInResolve.hs new file mode 100644 index 000000000..144167dc8 --- /dev/null +++ b/cardano-db-sync/src/Cardano/DbSync/Threads/TxInResolve.hs @@ -0,0 +1,94 @@ +{-# LANGUAGE FlexibleInstances #-} +{-# LANGUAGE OverloadedStrings #-} +{-# LANGUAGE TupleSections #-} +{-# LANGUAGE TypeApplications #-} +{-# LANGUAGE TypeFamilies #-} +{-# LANGUAGE NoImplicitPrelude #-} + +module Cardano.DbSync.Threads.TxInResolve where + +import Cardano.BM.Trace (logInfo) +import qualified Cardano.Db as DB +import Cardano.DbSync.Api +import Cardano.DbSync.Api.Types +import Cardano.DbSync.Era.Shelley.Generic.Block +import Cardano.DbSync.Era.Shelley.Generic.Tx.Shelley +import Cardano.DbSync.Era.Shelley.Generic.Tx.Types +import Cardano.DbSync.Era.Universal.Insert.Grouped +import Cardano.DbSync.Types +import Cardano.DbSync.Util +import qualified Cardano.Ledger.Alonzo.Tx as Alonzo +import qualified Cardano.Ledger.Alonzo.TxBody as Alonzo +import qualified Cardano.Ledger.Core as Core +import Cardano.Ledger.Crypto (StandardCrypto) +import Cardano.Prelude hiding (atomically) +import Control.Concurrent.Async +import Control.Concurrent.Class.MonadSTM.Strict +import Control.Concurrent.Class.MonadSTM.Strict.TBQueue as TBQ +import qualified Data.Set as Set +import Database.Persist.Postgresql +import Lens.Micro +import Ouroboros.Consensus.Cardano.Block (HardForkBlock (..)) +import Ouroboros.Consensus.Shelley.Ledger.Block (ShelleyBlock) + +-- | This is actually more than one thread. +runTxInResolve :: SyncEnv -> IO () +runTxInResolve syncEnv = do + logInfo trce "Running TxIn thread" + _ <- logException trce "runStakeThread: " loopWithPool + logInfo trce "Shutting TxIn thread" + where + nPool = 5 + trce = getTrace syncEnv + loopWithPool = + DB.runIohkLogging trce $ + withPostgresqlPool (envConnectionString syncEnv) nPool (forever . loopAction) + + loopAction pool = do + action <- liftIO $ atomically $ TBQ.readTBQueue (pTxInQueue (envPrefetch syncEnv)) + case action of + PrefetchTxIdBlock cblock -> prefetchBlock cblock + PrefetchTxIdBlocks cblocks -> mapM_ prefetchBlock cblocks + where + prefetchBlock cblock = + liftIO $ mapConcurrently_ resolver (groupByModulo nPool $ concat (getTxIns cblock)) + + -- There are nPool connections in pool and nPool threads, so each thread can keep a single + -- pool for all queries it performs. + resolver ls = runSqlPoolNoTransaction (queries ls) pool Nothing + queries = mapM_ $ \(isValid, txIn) -> + resolveTxInputsPrefetch syncEnv (not isValid) txIn + +getTxIns :: CardanoBlock -> [[(Bool, TxInKey)]] +getTxIns cblock = case cblock of + BlockByron _ -> [] -- Impossible + BlockShelley blk -> mkTxInsShelley blk + BlockAllegra blk -> mkTxInsShelley blk + BlockMary blk -> mkTxInsShelley blk + BlockAlonzo blk -> mkTxInsAlonzo blk + BlockBabbage blk -> mkTxInsAlonzo blk + BlockConway blk -> mkTxInsAlonzo blk + +mkTxInsShelley :: + (Core.EraSegWits era, Core.EraCrypto era ~ StandardCrypto) => + ShelleyBlock p era -> + [[(Bool, TxInKey)]] +mkTxInsShelley blk = map (map ((True,) . txInKey) . mkTxIn . getTxBody . snd) (getTxs blk) + where + getTxBody tx = tx ^. Core.bodyTxL + +mkTxInsAlonzo :: + (Core.EraSegWits era, Core.EraCrypto era ~ StandardCrypto, Core.Tx era ~ Alonzo.AlonzoTx era, Alonzo.AlonzoEraTxBody era) => + ShelleyBlock p era -> + [[(Bool, TxInKey)]] +mkTxInsAlonzo blk = map (pairWithAll . getTxInsAlonzo . snd) (getTxs blk) + where + pairWithAll (bl, s) = map ((bl,) . toTxInKey) (Set.toList s) + getTxInsAlonzo tx = case Alonzo.isValid tx of + Alonzo.IsValid True -> (True, tx ^. (Core.bodyTxL . Core.inputsTxBodyL)) + Alonzo.IsValid False -> (False, tx ^. (Core.bodyTxL . Alonzo.collateralInputsTxBodyL)) + +groupByModulo :: Int -> [a] -> [[a]] +groupByModulo n xs = [[x | (x, i) <- indexed, i `mod` n == m] | m <- [0 .. n - 1]] + where + indexed = zip xs [0 ..] diff --git a/cardano-db-sync/src/Cardano/DbSync/Types.hs b/cardano-db-sync/src/Cardano/DbSync/Types.hs index 3460bfb36..00dc1bd4e 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Types.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Types.hs @@ -9,7 +9,9 @@ module Cardano.DbSync.Types ( BlockEra (..), CardanoBlock, CardanoPoint, + TxIdLedger, StakeCred, + RewAccount, PoolKeyHash, DataHash, CardanoInterpreter, @@ -56,6 +58,8 @@ import Cardano.Ledger.Crypto (StandardCrypto) import qualified Cardano.Ledger.Hashes as Ledger import Cardano.Ledger.Keys +import qualified Cardano.Ledger.Address as Ledger +import qualified Cardano.Ledger.TxIn as Ledger import Cardano.Prelude hiding (Meta, show) import Cardano.Slotting.Slot (EpochNo (..), EpochSize (..), SlotNo (..)) import qualified Data.Text as Text @@ -80,8 +84,12 @@ type CardanoInterpreter = type CardanoPoint = Point CardanoBlock +type TxIdLedger = Ledger.TxId StandardCrypto + type StakeCred = Ledger.StakeCredential StandardCrypto +type RewAccount = Ledger.RewardAccount StandardCrypto + type PoolKeyHash = KeyHash 'StakePool StandardCrypto type DataHash = Ledger.DataHash StandardCrypto diff --git a/cardano-db-sync/src/Cardano/DbSync/Util.hs b/cardano-db-sync/src/Cardano/DbSync/Util.hs index ef0523828..148bdce49 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Util.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Util.hs @@ -31,6 +31,7 @@ module Cardano.DbSync.Util ( third, thrd3, forth4, + uncurry3, splitLast, traverseMEither, whenStrictJust, @@ -41,9 +42,13 @@ module Cardano.DbSync.Util ( whenRight, whenFalseEmpty, whenFalseMempty, + whenTrueMempty, + containsUnicodeNul, + safeDecodeUtf8, + safeDecodeToJson, ) where -import Cardano.BM.Trace (Trace, logError, logInfo) +import Cardano.BM.Trace (Trace, logError, logInfo, logWarning) import Cardano.Db (RewardSource (..)) import Cardano.DbSync.Config.Types () import Cardano.DbSync.Types @@ -56,11 +61,13 @@ import Control.Monad.Trans.Control (MonadBaseControl) import Data.ByteArray (ByteArrayAccess) import qualified Data.ByteArray import qualified Data.ByteString.Base16 as Base16 +import qualified Data.ByteString.Char8 as BS import qualified Data.List as List import qualified Data.Map.Strict as Map import qualified Data.Strict.Maybe as Strict import qualified Data.Text as Text import qualified Data.Text.Encoding as Text +import qualified Data.Text.Encoding.Error as Text import qualified Data.Time.Clock as Time import Ouroboros.Consensus.Block.Abstract (ConvertRawHash (..)) import Ouroboros.Consensus.Protocol.Praos () @@ -137,7 +144,7 @@ logException tracer txt action = logError tracer $ txt <> textShow e throwIO e --- | Eequired for testing or when disabling the metrics. +-- | Required for testing or when disabling the metrics. nullMetricSetters :: MetricSetters nullMetricSetters = MetricSetters @@ -214,6 +221,9 @@ thrd3 (_, _, c, _) = c forth4 :: (a, b, c, d) -> d forth4 (_, _, _, d) = d +uncurry3 :: (a -> b -> c -> d) -> (a, b, c) -> d +uncurry3 f (a, b, c) = f a b c + splitLast :: [(a, b, c, d)] -> ([(a, b, c)], [d]) splitLast = unzip . fmap (\(a, b, c, d) -> ((a, b, c), d)) @@ -233,3 +243,36 @@ whenFalseEmpty flag a mkAs = whenFalseMempty :: (Monoid a, Applicative m) => Bool -> m a -> m a whenFalseMempty flag mkAs = if flag then mkAs else pure mempty + +whenTrueMempty :: (Monoid a, Applicative m) => Bool -> m a -> m a +whenTrueMempty flag mkAs = + if flag then pure mempty else mkAs + +safeDecodeUtf8 :: ByteString -> IO (Either Text.UnicodeException Text) +safeDecodeUtf8 bs + | BS.any isNullChar bs = pure $ Left (Text.DecodeError (BS.unpack bs) (Just 0)) + | otherwise = try $ evaluate (Text.decodeUtf8With Text.strictDecode bs) + where + isNullChar :: Char -> Bool + isNullChar ch = ord ch == 0 + +containsUnicodeNul :: Text -> Bool +containsUnicodeNul = Text.isInfixOf "\\u000" + +safeDecodeToJson :: MonadIO m => Trace IO Text -> Text -> ByteString -> m (Maybe Text) +safeDecodeToJson tracer tracePrefix jsonBs = do + ejson <- liftIO $ safeDecodeUtf8 jsonBs + case ejson of + Left err -> do + liftIO . logWarning tracer $ + mconcat + [tracePrefix, ": Could not decode to UTF8: ", textShow err] + -- We have to insert + pure Nothing + Right json -> + -- See https://github.com/IntersectMBO/cardano-db-sync/issues/297 + if containsUnicodeNul json + then do + liftIO $ logWarning tracer $ tracePrefix <> "was recorded as null, due to a Unicode NUL character found when trying to parse the json." + pure Nothing + else pure $ Just json diff --git a/cardano-db-sync/src/Cardano/DbSync/Util/Address.hs b/cardano-db-sync/src/Cardano/DbSync/Util/Address.hs index dbcf0d2ed..29fdd4c94 100644 --- a/cardano-db-sync/src/Cardano/DbSync/Util/Address.hs +++ b/cardano-db-sync/src/Cardano/DbSync/Util/Address.hs @@ -8,6 +8,7 @@ module Cardano.DbSync.Util.Address ( deserialiseRewardAccount, ) where +import Cardano.DbSync.Types import Cardano.DbSync.Util.Bech32 (deserialiseFromBech32, serialiseToBech32) import qualified Cardano.Ledger.Address as Address import Cardano.Ledger.Api.Tx.Address (decodeAddrLenient) @@ -39,7 +40,7 @@ deserialiseShelleyAddress bech32 = decodeAddrLenient =<< rawBytes rawBytes = rightToMaybe $ deserialiseFromBech32 bech32 -- | Serialise a Shelley era stake address to bech32 -serialiseRewardAccount :: Address.RewardAccount StandardCrypto -> Text +serialiseRewardAccount :: RewAccount -> Text serialiseRewardAccount acnt@(Address.RewardAccount net _) = serialiseToBech32 (prefix net) (Address.serialiseRewardAccount acnt) where diff --git a/cardano-db-sync/test/Cardano/DbSync/Util/AddressTest.hs b/cardano-db-sync/test/Cardano/DbSync/Util/AddressTest.hs index 16448b4f7..bf3e3a784 100644 --- a/cardano-db-sync/test/Cardano/DbSync/Util/AddressTest.hs +++ b/cardano-db-sync/test/Cardano/DbSync/Util/AddressTest.hs @@ -4,6 +4,7 @@ module Cardano.DbSync.Util.AddressTest (tests) where import Cardano.Binary (FromCBOR (..), unsafeDeserialize') +import Cardano.DbSync.Types import Cardano.DbSync.Util.Address import Cardano.Ledger.Address (Addr (..), BootstrapAddress (..), RewardAccount (..)) import Cardano.Ledger.BaseTypes (Network (..)) @@ -218,7 +219,7 @@ genByronAddress = arbitrary genShelleyAddress :: Gen (Addr StandardCrypto) genShelleyAddress = Addr <$> arbitrary <*> arbitrary <*> arbitrary -genRewardAccount :: Gen (RewardAccount StandardCrypto) +genRewardAccount :: Gen RewAccount genRewardAccount = arbitrary deserialiseBase16 :: FromCBOR a => Text -> a diff --git a/cardano-db-tool/cardano-db-tool.cabal b/cardano-db-tool/cardano-db-tool.cabal index 3dc10238b..d775dfe0e 100644 --- a/cardano-db-tool/cardano-db-tool.cabal +++ b/cardano-db-tool/cardano-db-tool.cabal @@ -74,6 +74,7 @@ library , ouroboros-consensus-cardano , ouroboros-network , ouroboros-network-api + , persistent , random , random-shuffle , text diff --git a/cardano-db-tool/src/Cardano/DbTool/Validate/TxAccounting.hs b/cardano-db-tool/src/Cardano/DbTool/Validate/TxAccounting.hs index d229f045e..faec0e02c 100644 --- a/cardano-db-tool/src/Cardano/DbTool/Validate/TxAccounting.hs +++ b/cardano-db-tool/src/Cardano/DbTool/Validate/TxAccounting.hs @@ -33,7 +33,6 @@ import Database.Esqueleto.Experimental ( on, select, table, - toSqlKey, val, where_, (==.), @@ -151,7 +150,7 @@ queryTestTxIds = do lower <- select $ from (table @Tx) >>= \tx -> do - where_ (tx ^. TxBlockId >. val (toSqlKey 1)) + where_ (tx ^. TxBlockId >. val (BlockKey 1)) pure (tx ^. TxId) upper <- select $ from (table @Tx) >> pure countRows pure (maybe 0 (unTxId . unValue) (listToMaybe lower), maybe 0 unValue (listToMaybe upper)) @@ -160,7 +159,7 @@ queryTxFeeDeposit :: MonadIO m => Word64 -> ReaderT SqlBackend m (Ada, Int64) queryTxFeeDeposit txId = do res <- select $ do tx <- from $ table @Tx - where_ (tx ^. TxId ==. val (toSqlKey $ fromIntegral txId)) + where_ (tx ^. TxId ==. val (TxKey $ fromIntegral txId)) pure (tx ^. TxFee, tx ^. TxDeposit) pure $ maybe (0, 0) convert (listToMaybe res) where @@ -182,7 +181,7 @@ queryInputsBody txId = do `on` (\(tx :& txin) -> tx ^. TxId ==. txin ^. TxInTxInId) `innerJoin` table @(TxOutTable a) `on` (\(_tx :& txin :& txout) -> txin ^. TxInTxOutId ==. txout ^. txOutTxIdField @a) - where_ (tx ^. TxId ==. val (toSqlKey $ fromIntegral txId)) + where_ (tx ^. TxId ==. val (TxKey $ fromIntegral txId)) where_ (txout ^. txOutIndexField @a ==. txin ^. TxInTxOutIndex) pure txout pure $ entityVal <$> res @@ -200,7 +199,7 @@ queryTxOutputsBody txId = do $ table @Tx `innerJoin` table @(TxOutTable a) `on` (\(tx :& txout) -> tx ^. TxId ==. txout ^. txOutTxIdField @a) - where_ (tx ^. TxId ==. val (toSqlKey $ fromIntegral txId)) + where_ (tx ^. TxId ==. val (TxKey $ fromIntegral txId)) pure txout pure $ entityVal <$> res @@ -208,7 +207,7 @@ queryTxWithdrawal :: MonadIO m => Word64 -> ReaderT SqlBackend m Ada queryTxWithdrawal txId = do res <- select $ do withdraw <- from $ table @Withdrawal - where_ (withdraw ^. WithdrawalTxId ==. val (toSqlKey $ fromIntegral txId)) + where_ (withdraw ^. WithdrawalTxId ==. val (TxKey $ fromIntegral txId)) pure (withdraw ^. WithdrawalAmount) -- It is probably not possible to have two withdrawals in a single Tx. -- If it is possible then there will be an accounting error. diff --git a/cardano-db/cardano-db.cabal b/cardano-db/cardano-db.cabal index d8afa1a33..5fea0ab3d 100644 --- a/cardano-db/cardano-db.cabal +++ b/cardano-db/cardano-db.cabal @@ -57,6 +57,7 @@ library Cardano.Db.Schema.Orphans Cardano.Db.Schema.Types Cardano.Db.Types + Cardano.Db.Util build-depends: aeson , base >= 4.14 && < 5 @@ -142,7 +143,6 @@ test-suite test-db other-modules: Test.IO.Cardano.Db.Insert Test.IO.Cardano.Db.Migration - Test.IO.Cardano.Db.Rollback Test.IO.Cardano.Db.TotalSupply Test.IO.Cardano.Db.Util Test.IO.Cardano.Db.PGConfig @@ -160,11 +160,9 @@ test-suite test-db build-depends: base , bytestring , cardano-db - , cardano-slotting , directory , extra , filepath - , monad-control , persistent , tasty , tasty-hunit diff --git a/cardano-db/src/Cardano/Db.hs b/cardano-db/src/Cardano/Db.hs index 630df6f2a..72e13aa4f 100644 --- a/cardano-db/src/Cardano/Db.hs +++ b/cardano-db/src/Cardano/Db.hs @@ -29,3 +29,4 @@ import Cardano.Db.Run as X import Cardano.Db.Schema.BaseSchema as X import Cardano.Db.Schema.Types as X import Cardano.Db.Types as X +import Cardano.Db.Util as X diff --git a/cardano-db/src/Cardano/Db/Operations/Insert.hs b/cardano-db/src/Cardano/Db/Operations/Insert.hs index f498ae285..f504daa2e 100644 --- a/cardano-db/src/Cardano/Db/Operations/Insert.hs +++ b/cardano-db/src/Cardano/Db/Operations/Insert.hs @@ -72,7 +72,6 @@ module Cardano.Db.Operations.Insert ( setNullRatified, setNullExpired, setNullDropped, - replaceAdaPots, insertAnchor, insertConstitution, insertGovActionProposal, @@ -92,8 +91,6 @@ module Cardano.Db.Operations.Insert ( insertAlwaysNoConfidence, insertUnchecked, insertMany', - -- Export mainly for testing. - insertBlockChecked, ) where import Cardano.Db.Operations.Query @@ -121,6 +118,7 @@ import Database.Persist.Class ( checkUnique, insert, insertBy, + insertKey, replaceUnique, ) import Database.Persist.EntityDef.Internal (entityDB, entityUniques) @@ -134,7 +132,6 @@ import Database.Persist.Sql ( insertMany, rawExecute, rawSql, - replace, toPersistFields, toPersistValue, uniqueDBName, @@ -171,8 +168,8 @@ import Database.PostgreSQL.Simple (SqlError) insertAdaPots :: (MonadBaseControl IO m, MonadIO m) => AdaPots -> ReaderT SqlBackend m AdaPotsId insertAdaPots = insertUnchecked "AdaPots" -insertBlock :: (MonadBaseControl IO m, MonadIO m) => Block -> ReaderT SqlBackend m BlockId -insertBlock = insertUnchecked "Block" +insertBlock :: (MonadBaseControl IO m, MonadIO m) => BlockId -> Block -> ReaderT SqlBackend m () +insertBlock = insertKeyUnchecked "Block" insertCollateralTxIn :: (MonadBaseControl IO m, MonadIO m) => CollateralTxIn -> ReaderT SqlBackend m CollateralTxInId insertCollateralTxIn = insertUnchecked "CollateralTxIn" @@ -282,8 +279,8 @@ insertStakeRegistration = insertUnchecked "StakeRegistration" insertTreasury :: (MonadBaseControl IO m, MonadIO m) => Treasury -> ReaderT SqlBackend m TreasuryId insertTreasury = insertUnchecked "Treasury" -insertTx :: (MonadBaseControl IO m, MonadIO m) => Tx -> ReaderT SqlBackend m TxId -insertTx tx = insertUnchecked ("Tx: " ++ show (BS.length (txHash tx))) tx +insertTx :: (MonadBaseControl IO m, MonadIO m) => TxId -> Tx -> ReaderT SqlBackend m () +insertTx k tx = insertKeyUnchecked ("Tx: " ++ show (BS.length (txHash tx))) k tx insertTxIn :: (MonadBaseControl IO m, MonadIO m) => TxIn -> ReaderT SqlBackend m TxInId insertTxIn = insertUnchecked "TxIn" @@ -407,18 +404,6 @@ setNullDropped :: MonadIO m => Word64 -> ReaderT SqlBackend m Int64 setNullDropped eNo = updateWhereCount [GovActionProposalDroppedEpoch !=. Nothing, GovActionProposalDroppedEpoch >. Just eNo] [GovActionProposalDroppedEpoch =. Nothing] -replaceAdaPots :: (MonadBaseControl IO m, MonadIO m) => BlockId -> AdaPots -> ReaderT SqlBackend m Bool -replaceAdaPots blockId adapots = do - mAdaPotsId <- queryAdaPotsId blockId - case mAdaPotsId of - Nothing -> pure False - Just adaPotsDB - | entityVal adaPotsDB == adapots -> - pure False - Just adaPotsDB -> do - replace (entityKey adaPotsDB) adapots - pure True - insertAnchor :: (MonadBaseControl IO m, MonadIO m) => VotingAnchor -> ReaderT SqlBackend m VotingAnchorId insertAnchor = insertCheckUnique "VotingAnchor" @@ -699,6 +684,26 @@ insertUnchecked vtype = exceptHandler e = liftIO $ throwIO (DbInsertException vtype e) +-- Insert without checking uniqueness constraints. This should be safe for most tables +-- even tables with uniqueness constraints, especially block, tx and many others, where +-- uniqueness is enforced by the ledger. +insertKeyUnchecked :: + ( MonadIO m + , MonadBaseControl IO m + , PersistEntityBackend record ~ SqlBackend + , PersistEntity record + ) => + String -> + Key record -> + record -> + ReaderT SqlBackend m () +insertKeyUnchecked vtype k v = + handle exceptHandler $ insertKey k v + where + exceptHandler :: MonadIO m => SqlError -> ReaderT SqlBackend m a + exceptHandler e = + liftIO $ throwIO (DbInsertException vtype e) + -- This is cargo culted from Persistent because it is not exported. escapeFieldName :: FieldNameDB -> Text escapeFieldName (FieldNameDB s) = @@ -715,8 +720,3 @@ onlyOneUniqueDef prxy = case entityUniques (entityDef prxy) of [uniq] -> uniq _ -> error "impossible due to OnlyOneUniqueKey constraint" - --- Used in tests - -insertBlockChecked :: (MonadBaseControl IO m, MonadIO m) => Block -> ReaderT SqlBackend m BlockId -insertBlockChecked = insertCheckUnique "Block" diff --git a/cardano-db/src/Cardano/Db/Operations/Other/ConsumedTxOut.hs b/cardano-db/src/Cardano/Db/Operations/Other/ConsumedTxOut.hs index 47f68e513..7322e327b 100644 --- a/cardano-db/src/Cardano/Db/Operations/Other/ConsumedTxOut.hs +++ b/cardano-db/src/Cardano/Db/Operations/Other/ConsumedTxOut.hs @@ -239,6 +239,9 @@ updateListTxOutConsumedByTxId ls = do CTxOutIdW txOutId' -> update txOutId' [C.TxOutConsumedByTxId =. Just txId] VTxOutIdW txOutId' -> update txOutId' [V.TxOutConsumedByTxId =. Just txId] +updateListTxOutConsumedByTxId' :: [((TxId, Word64), TxId)] -> ReaderT SqlBackend m () +updateListTxOutConsumedByTxId' _ls = undefined + migrateTxOut :: ( MonadBaseControl IO m , MonadIO m diff --git a/cardano-db/src/Cardano/Db/Operations/Query.hs b/cardano-db/src/Cardano/Db/Operations/Query.hs index 904ed1646..122c90f00 100644 --- a/cardano-db/src/Cardano/Db/Operations/Query.hs +++ b/cardano-db/src/Cardano/Db/Operations/Query.hs @@ -39,6 +39,7 @@ module Cardano.Db.Operations.Query ( queryTxId, queryEpochFromNum, queryEpochStakeCount, + queryEpochStakeExists, queryForEpochId, queryLatestEpoch, queryMinRefId, @@ -574,6 +575,15 @@ queryEpochStakeCount epoch = do pure countRows pure $ maybe 0 unValue (listToMaybe res) +queryEpochStakeExists :: MonadIO m => Word64 -> ReaderT SqlBackend m Bool +queryEpochStakeExists epoch = do + res <- select $ do + epochStake <- from $ table @EpochStake + where_ (epochStake ^. EpochStakeEpochNo ==. val epoch) + limit 1 + pure (epochStake ^. EpochStakeId) + pure $ not (null res) + queryMinRefId :: forall m field record. (MonadIO m, PersistEntity record, PersistField field) => diff --git a/cardano-db/src/Cardano/Db/Operations/QueryHelper.hs b/cardano-db/src/Cardano/Db/Operations/QueryHelper.hs index 64da0a70f..8543b6ccc 100644 --- a/cardano-db/src/Cardano/Db/Operations/QueryHelper.hs +++ b/cardano-db/src/Cardano/Db/Operations/QueryHelper.hs @@ -66,10 +66,10 @@ entityPair e = (entityKey e, entityVal e) unBlockId :: BlockId -> Word64 -unBlockId = fromIntegral . unSqlBackendKey . unBlockKey +unBlockId = fromIntegral . unBlockKey unTxId :: TxId -> Word64 -unTxId = fromIntegral . unSqlBackendKey . unTxKey +unTxId = fromIntegral . unTxKey unTxInId :: TxInId -> Word64 unTxInId = fromIntegral . unSqlBackendKey . unTxInKey diff --git a/cardano-db/src/Cardano/Db/Run.hs b/cardano-db/src/Cardano/Db/Run.hs index 0aabb07d0..e07163460 100644 --- a/cardano-db/src/Cardano/Db/Run.hs +++ b/cardano-db/src/Cardano/Db/Run.hs @@ -7,7 +7,10 @@ module Cardano.Db.Run ( getBackendGhci, ghciDebugQuery, runDbHandleLogger, + runDbLogging, + runDbLoggingExceptT, runDbIohkLogging, + runDbIohkLoggingExceptT, runDbIohkNoLogging, runDbNoLogging, runDbNoLoggingEnv, @@ -44,7 +47,8 @@ import Control.Monad.Logger ( runStdoutLoggingT, ) import Control.Monad.Trans.Control (MonadBaseControl) -import Control.Monad.Trans.Reader (ReaderT) +import Control.Monad.Trans.Except +import Control.Monad.Trans.Reader (ReaderT, runReaderT) import Control.Monad.Trans.Resource (MonadUnliftIO) import Control.Tracer (traceWith) import qualified Data.ByteString.Char8 as BS @@ -113,11 +117,26 @@ runWithConnectionNoLogging source dbAction = do $ \backend -> runSqlConnWithIsolation dbAction backend Serializable +-- | Use an existing connection to run queries without using a transactions +runDbLogging :: SqlBackend -> Trace IO Text -> ReaderT SqlBackend (LoggingT m) b -> m b +runDbLogging backend tracer dbAction = do + runIohkLogging tracer $ runReaderT dbAction backend + +-- | Use an existing connection to run queries without using a transactions +runDbLoggingExceptT :: SqlBackend -> Trace IO Text -> ExceptT e (ReaderT SqlBackend (LoggingT m)) b -> ExceptT e m b +runDbLoggingExceptT backend tracer dbAction = do + ExceptT $ runIohkLogging tracer $ runReaderT (runExceptT dbAction) backend + -- | Run a DB action logging via iohk-monitoring-framework. runDbIohkLogging :: MonadUnliftIO m => SqlBackend -> Trace IO Text -> ReaderT SqlBackend (LoggingT m) b -> m b runDbIohkLogging backend tracer dbAction = do runIohkLogging tracer $ runSqlConnWithIsolation dbAction backend Serializable +-- | Run a DB action logging via iohk-monitoring-framework. +runDbIohkLoggingExceptT :: MonadUnliftIO m => SqlBackend -> Trace IO Text -> ExceptT e (ReaderT SqlBackend (LoggingT m)) b -> ExceptT e m b +runDbIohkLoggingExceptT backend tracer dbAction = do + ExceptT $ runIohkLogging tracer $ runSqlConnWithIsolation (runExceptT dbAction) backend Serializable + -- | Run a DB action using a Pool via iohk-monitoring-framework. runPoolDbIohkLogging :: MonadUnliftIO m => Pool SqlBackend -> Trace IO Text -> ReaderT SqlBackend (LoggingT m) b -> m b runPoolDbIohkLogging backend tracer dbAction = do diff --git a/cardano-db/src/Cardano/Db/Schema/BaseSchema.hs b/cardano-db/src/Cardano/Db/Schema/BaseSchema.hs index 51b939650..7c2c14edd 100644 --- a/cardano-db/src/Cardano/Db/Schema/BaseSchema.hs +++ b/cardano-db/src/Cardano/Db/Schema/BaseSchema.hs @@ -95,6 +95,7 @@ share -- All NULL-able fields other than 'epochNo' are NULL for EBBs, whereas 'epochNo' is -- only NULL for the genesis block. Block + Id Int64 sqltype=bigint Primary hash ByteString sqltype=hash32type epochNo Word64 Maybe sqltype=word31type slotNo Word64 Maybe sqltype=word63type @@ -114,6 +115,7 @@ share UniqueBlock hash Tx + Id Int64 sqltype=bigint Primary -- generated="block_id * 1000 + block_index" hash ByteString sqltype=hash32type blockId BlockId noreference -- This type is the primary key for the 'block' table. blockIndex Word64 sqltype=word31type -- The index of this transaction within the block. diff --git a/cardano-db/src/Cardano/Db/Util.hs b/cardano-db/src/Cardano/Db/Util.hs new file mode 100644 index 000000000..87475dde9 --- /dev/null +++ b/cardano-db/src/Cardano/Db/Util.hs @@ -0,0 +1,7 @@ +module Cardano.Db.Util where + +import Cardano.Db.Schema.BaseSchema +import Data.Word (Word64) + +toTxId :: BlockId -> Word64 -> TxId +toTxId blockId blockIndex = TxKey $ 1000 * unBlockKey blockId + fromIntegral blockIndex diff --git a/cardano-db/test/Test/IO/Cardano/Db/Insert.hs b/cardano-db/test/Test/IO/Cardano/Db/Insert.hs index 233a4400c..d08357d5c 100644 --- a/cardano-db/test/Test/IO/Cardano/Db/Insert.hs +++ b/cardano-db/test/Test/IO/Cardano/Db/Insert.hs @@ -6,7 +6,6 @@ module Test.IO.Cardano.Db.Insert ( ) where import Cardano.Db -import Control.Monad (void) import Data.ByteString.Char8 (ByteString) import qualified Data.ByteString.Char8 as BS import Data.Time.Clock @@ -19,44 +18,30 @@ tests :: TestTree tests = testGroup "Insert" - [ testCase "Insert zeroth block" insertZeroTest - , testCase "Insert first block" insertFirstTest + [ testCase "Insert 2 blocks" insertTwoBlocks , testCase "Insert twice" insertTwice , testCase "Insert foreign key missing" insertForeignKeyMissing ] -insertZeroTest :: IO () -insertZeroTest = - runDbNoLoggingEnv $ do - deleteAllBlocks - -- Delete the blocks if they exist. - slid <- insertSlotLeader testSlotLeader - void $ deleteBlock TxOutCore (blockOne slid) - void $ deleteBlock TxOutCore (blockZero slid) - -- Insert the same block twice. The first should be successful (resulting - -- in a 'Right') and the second should return the same value in a 'Left'. - bid0 <- insertBlockChecked (blockZero slid) - bid1 <- insertBlockChecked (blockZero slid) - assertBool (show bid0 ++ " /= " ++ show bid1) (bid0 == bid1) - -insertFirstTest :: IO () -insertFirstTest = +insertTwoBlocks :: IO () +insertTwoBlocks = runDbNoLoggingEnv $ do deleteAllBlocks -- Delete the block if it exists. slid <- insertSlotLeader testSlotLeader - void $ deleteBlock TxOutCore (blockOne slid) -- Insert the same block twice. - bid0 <- insertBlockChecked (blockZero slid) - bid1 <- insertBlockChecked $ (\b -> b {blockPreviousId = Just bid0}) (blockOne slid) - assertBool (show bid0 ++ " == " ++ show bid1) (bid0 /= bid1) + let bid0 = BlockKey 0 + insertBlock bid0 (blockZero slid) + let bid1 = BlockKey 1 + insertBlock bid1 $ (\b -> b {blockPreviousId = Just bid0}) (blockOne slid) insertTwice :: IO () insertTwice = runDbNoLoggingEnv $ do deleteAllBlocks slid <- insertSlotLeader testSlotLeader - bid <- insertBlockChecked (blockZero slid) + let bid = BlockKey 0 + insertBlock bid (blockZero slid) let adaPots = adaPotsZero bid _ <- insertAdaPots adaPots Just pots0 <- queryAdaPots bid @@ -73,8 +58,10 @@ insertForeignKeyMissing = do runDbNoLoggingEnv $ do deleteAllBlocks slid <- insertSlotLeader testSlotLeader - bid <- insertBlockChecked (blockZero slid) - txid <- insertTx (txZero bid) + let bid = BlockKey 0 + insertBlock bid (blockZero slid) + let txid = toTxId bid 0 + insertTx txid (txZero bid) phid <- insertPoolHash poolHash0 pmrid <- insertPoolMetadataRef $ poolMetadataRef txid phid let fe = offChainPoolFetchError phid pmrid time diff --git a/cardano-db/test/Test/IO/Cardano/Db/Rollback.hs b/cardano-db/test/Test/IO/Cardano/Db/Rollback.hs deleted file mode 100644 index b4133bd92..000000000 --- a/cardano-db/test/Test/IO/Cardano/Db/Rollback.hs +++ /dev/null @@ -1,146 +0,0 @@ -{-# LANGUAGE CPP #-} -{-# LANGUAGE FlexibleContexts #-} -{-# LANGUAGE ScopedTypeVariables #-} - -#if __GLASGOW_HASKELL__ >= 908 -{-# OPTIONS_GHC -Wno-x-partial #-} -{-# LANGUAGE TypeApplications #-} -{-# LANGUAGE DataKinds #-} -#endif - -module Test.IO.Cardano.Db.Rollback ( - tests, -) where - -import Cardano.Db -import Cardano.Slotting.Slot (SlotNo (..)) -import Control.Monad (void) -import Control.Monad.IO.Class (MonadIO) -import Control.Monad.Trans.Control (MonadBaseControl) -import Control.Monad.Trans.Reader (ReaderT) -import Data.Word (Word64) -import Database.Persist.Sql (SqlBackend) -import Test.IO.Cardano.Db.Util -import Test.Tasty (TestTree, testGroup) - -tests :: TestTree -tests = - testGroup - "Rollback" - [] - --- testCase "Can rollback" rollbackTest TODO: fix - -_rollbackTest :: IO () -_rollbackTest = - runDbNoLoggingEnv $ do - -- Delete the blocks if they exist. - deleteAllBlocks - setupBlockCount <- queryBlockCount - assertBool ("Block on setup is " ++ show setupBlockCount ++ " but should be 0.") $ setupBlockCount == 0 - -- Set up state before rollback and assert expected counts. - createAndInsertBlocks 10 - beforeBlocks <- queryBlockCount - assertBool ("Block count before rollback is " ++ show beforeBlocks ++ " but should be 10.") $ beforeBlocks == 10 - beforeTxCount <- queryTxCount - assertBool ("Tx count before rollback is " ++ show beforeTxCount ++ " but should be 9.") $ beforeTxCount == 9 - beforeTxOutCount <- queryTxOutCount TxOutCore - assertBool ("TxOut count before rollback is " ++ show beforeTxOutCount ++ " but should be 2.") $ beforeTxOutCount == 2 - beforeTxInCount <- queryTxInCount - assertBool ("TxIn count before rollback is " ++ show beforeTxInCount ++ " but should be 1.") $ beforeTxInCount == 1 - -- Rollback a set of blocks. - latestSlotNo <- queryLatestSlotNo - Just pSlotNo <- queryWalkChain 5 latestSlotNo - void $ deleteBlocksSlotNoNoTrace TxOutCore (SlotNo pSlotNo) - -- Assert the expected final state. - afterBlocks <- queryBlockCount - assertBool ("Block count after rollback is " ++ show afterBlocks ++ " but should be 10") $ afterBlocks == 4 - afterTxCount <- queryTxCount - assertBool ("Tx count after rollback is " ++ show afterTxCount ++ " but should be 10") $ afterTxCount == 1 - afterTxOutCount <- queryTxOutCount TxOutCore - assertBool ("TxOut count after rollback is " ++ show afterTxOutCount ++ " but should be 1.") $ afterTxOutCount == 1 - afterTxInCount <- queryTxInCount - assertBool ("TxIn count after rollback is " ++ show afterTxInCount ++ " but should be 0.") $ afterTxInCount == 0 - --- ----------------------------------------------------------------------------- - -queryWalkChain :: (MonadBaseControl IO m, MonadIO m) => Int -> Word64 -> ReaderT SqlBackend m (Maybe Word64) -queryWalkChain count blkNo - | count <= 0 = pure $ Just blkNo - | otherwise = do - mpBlkNo <- queryPreviousSlotNo blkNo - case mpBlkNo of - Nothing -> pure Nothing - Just pBlkNo -> queryWalkChain (count - 1) pBlkNo - -createAndInsertBlocks :: (MonadBaseControl IO m, MonadIO m) => Word64 -> ReaderT SqlBackend m () -createAndInsertBlocks blockCount = - void $ loop (0, Nothing, Nothing) - where - loop :: - (MonadBaseControl IO m, MonadIO m) => - (Word64, Maybe BlockId, Maybe TxId) -> - ReaderT SqlBackend m (Word64, Maybe BlockId, Maybe TxId) - loop (indx, mPrevId, mOutId) = - if indx < blockCount - then loop =<< createAndInsert (indx, mPrevId, mOutId) - else pure (0, Nothing, Nothing) - - createAndInsert :: - (MonadBaseControl IO m, MonadIO m) => - (Word64, Maybe BlockId, Maybe TxId) -> - ReaderT SqlBackend m (Word64, Maybe BlockId, Maybe TxId) - createAndInsert (indx, mPrevId, mTxOutId) = do - slid <- insertSlotLeader testSlotLeader - let newBlock = - Block - { blockHash = mkBlockHash indx - , blockEpochNo = Just 0 - , blockSlotNo = Just indx - , blockEpochSlotNo = Just indx - , blockBlockNo = Just indx - , blockPreviousId = mPrevId - , blockSlotLeaderId = slid - , blockSize = 42 - , blockTime = dummyUTCTime - , blockTxCount = 0 - , blockProtoMajor = 1 - , blockProtoMinor = 0 - , blockVrfKey = Nothing - , blockOpCert = Nothing - , blockOpCertCounter = Nothing - } - - blkId <- insertBlock newBlock - newMTxOutId <- - if indx /= 0 - then pure mTxOutId - else do - txId <- - insertTx $ - Tx - (mkTxHash blkId 0) - blkId - 0 - (DbLovelace 0) - (DbLovelace 0) - (Just 0) - 12 - Nothing - Nothing - True - 0 - (DbLovelace 0) - - void $ insertTxOut (mkTxOutCore blkId txId) - pure $ Just txId - case (indx, mTxOutId) of - (8, Just txOutId) -> do - -- Insert Txs here to test that they are cascade deleted when the blocks - -- they are associcated with are deleted. - - txId <- head <$> mapM insertTx (mkTxs blkId 8) - void $ insertTxIn (TxIn txId txOutId 0 Nothing) - void $ insertTxOut (mkTxOutCore blkId txId) - _otherwise -> pure () - pure (indx + 1, Just blkId, newMTxOutId) diff --git a/cardano-db/test/Test/IO/Cardano/Db/TotalSupply.hs b/cardano-db/test/Test/IO/Cardano/Db/TotalSupply.hs index 0a7ac3dc4..66309886d 100644 --- a/cardano-db/test/Test/IO/Cardano/Db/TotalSupply.hs +++ b/cardano-db/test/Test/IO/Cardano/Db/TotalSupply.hs @@ -33,33 +33,36 @@ initialSupplyTest = -- Set up initial supply. slid <- insertSlotLeader testSlotLeader - bid0 <- insertBlock (mkBlock 0 slid) - (tx0Ids :: [TxId]) <- mapM insertTx $ mkTxs bid0 4 - mapM_ (insertTxOut . mkTxOutCore bid0) tx0Ids + let bid0 = BlockKey 0 + insertBlock bid0 (mkBlock 0 slid) + let tx0s = mkTxs bid0 4 + mapM_ (uncurry insertTx) tx0s + mapM_ (insertTxOut . mkTxOutCore bid0) (fst <$> tx0s) count <- queryBlockCount assertBool ("Block count should be 1, got " ++ show count) (count == 1) supply0 <- queryTotalSupply TxOutCore assertBool "Total supply should not be > 0" (supply0 > Ada 0) -- Spend from the Utxo set. - bid1 <- insertBlock (mkBlock 1 slid) - tx1Id <- - insertTx $ - Tx - { txHash = mkTxHash bid1 1 - , txBlockId = bid1 - , txBlockIndex = 0 - , txOutSum = DbLovelace 500000000 - , txFee = DbLovelace 100 - , txDeposit = Just 0 - , txSize = 123 - , txInvalidHereafter = Nothing - , txInvalidBefore = Nothing - , txValidContract = True - , txScriptSize = 0 - , txTreasuryDonation = DbLovelace 0 - } - _ <- insertTxIn (TxIn tx1Id (head tx0Ids) 0 Nothing) + let bid1 = BlockKey 1 + insertBlock bid1 (mkBlock 1 slid) + let tx1Id = toTxId bid1 0 + insertTx tx1Id $ + Tx + { txHash = mkTxHash bid1 1 + , txBlockId = bid1 + , txBlockIndex = 0 + , txOutSum = DbLovelace 500000000 + , txFee = DbLovelace 100 + , txDeposit = Just 0 + , txSize = 123 + , txInvalidHereafter = Nothing + , txInvalidBefore = Nothing + , txValidContract = True + , txScriptSize = 0 + , txTreasuryDonation = DbLovelace 0 + } + _ <- insertTxIn (TxIn tx1Id (fst $ head tx0s) 0 Nothing) let addr = mkAddressHash bid1 tx1Id _ <- insertTxOut $ diff --git a/cardano-db/test/Test/IO/Cardano/Db/Util.hs b/cardano-db/test/Test/IO/Cardano/Db/Util.hs index 1bf6cece7..edfbf90e1 100644 --- a/cardano-db/test/Test/IO/Cardano/Db/Util.hs +++ b/cardano-db/test/Test/IO/Cardano/Db/Util.hs @@ -73,25 +73,27 @@ mkTxHash :: BlockId -> Word64 -> ByteString mkTxHash blk tx = BS.pack (take 32 $ printf "block #%d, tx #%d" (unBlockId blk) tx ++ replicate 32 ' ') -mkTxs :: BlockId -> Word -> [Tx] +mkTxs :: BlockId -> Word -> [(TxId, Tx)] mkTxs blkId count = take (fromIntegral count) $ map create [0 ..] where create w = - Tx - { txHash = mkTxHash blkId w - , txBlockId = blkId - , txBlockIndex = 0 - , txOutSum = DbLovelace 2 - , txFee = DbLovelace 1 - , txDeposit = Just 0 - , txSize = 12 - , txInvalidHereafter = Nothing - , txInvalidBefore = Nothing - , txValidContract = True - , txScriptSize = 0 - , txTreasuryDonation = DbLovelace 0 - } + ( toTxId blkId w + , Tx + { txHash = mkTxHash blkId w + , txBlockId = blkId + , txBlockIndex = 0 + , txOutSum = DbLovelace 2 + , txFee = DbLovelace 1 + , txDeposit = Just 0 + , txSize = 12 + , txInvalidHereafter = Nothing + , txInvalidBefore = Nothing + , txValidContract = True + , txScriptSize = 0 + , txTreasuryDonation = DbLovelace 0 + } + ) testSlotLeader :: SlotLeader testSlotLeader = diff --git a/cardano-db/test/test-db.hs b/cardano-db/test/test-db.hs index 4ddfa9fcf..01341274f 100644 --- a/cardano-db/test/test-db.hs +++ b/cardano-db/test/test-db.hs @@ -8,7 +8,6 @@ import System.FilePath (()) import qualified Test.IO.Cardano.Db.Insert import qualified Test.IO.Cardano.Db.Migration import qualified Test.IO.Cardano.Db.PGConfig -import qualified Test.IO.Cardano.Db.Rollback import qualified Test.IO.Cardano.Db.TotalSupply import Test.Tasty (defaultMain, testGroup) import Prelude @@ -27,6 +26,5 @@ main = do [ Test.IO.Cardano.Db.Migration.tests , Test.IO.Cardano.Db.Insert.tests , Test.IO.Cardano.Db.TotalSupply.tests - , Test.IO.Cardano.Db.Rollback.tests , Test.IO.Cardano.Db.PGConfig.tests ]