Safe Haskell | Safe-Inferred |
---|---|
Language | Haskell2010 |
Types used throughout the implementation: handle, state, environment, types, trace types, etc.
Synopsis
- data ChainDbEnv m blk = CDB {
- cdbImmutableDB ∷ !(ImmutableDB m blk)
- cdbVolatileDB ∷ !(VolatileDB m blk)
- cdbLgrDB ∷ !(LgrDB m blk)
- cdbChain ∷ !(StrictTVar m (AnchoredFragment (Header blk)))
- cdbTentativeState ∷ !(StrictTVar m (TentativeHeaderState blk))
- cdbTentativeHeader ∷ !(StrictTVar m (StrictMaybe (Header blk)))
- cdbIterators ∷ !(StrictTVar m (Map IteratorKey (m ())))
- cdbFollowers ∷ !(StrictTVar m (Map FollowerKey (FollowerHandle m blk)))
- cdbTopLevelConfig ∷ !(TopLevelConfig blk)
- cdbInvalid ∷ !(StrictTVar m (WithFingerprint (InvalidBlocks blk)))
- cdbNextIteratorKey ∷ !(StrictTVar m IteratorKey)
- cdbNextFollowerKey ∷ !(StrictTVar m FollowerKey)
- cdbCopyFuse ∷ !(Fuse m)
- cdbChainSelFuse ∷ !(Fuse m)
- cdbTracer ∷ !(Tracer m (TraceEvent blk))
- cdbRegistry ∷ !(ResourceRegistry m)
- cdbGcDelay ∷ !DiffTime
- cdbGcInterval ∷ !DiffTime
- cdbKillBgThreads ∷ !(StrictTVar m (m ()))
- cdbChainSelQueue ∷ !(ChainSelQueue m blk)
- cdbLoE ∷ !(m (LoE (AnchoredFragment (Header blk))))
- newtype ChainDbHandle m blk = CDBHandle (StrictTVar m (ChainDbState m blk))
- data ChainDbState m blk
- = ChainDbOpen !(ChainDbEnv m blk)
- | ChainDbClosed
- newtype ChainSelectionPromise m = ChainSelectionPromise {
- waitChainSelectionPromise ∷ m ()
- class (ImmutableDbSerialiseConstraints blk, LgrDbSerialiseConstraints blk, VolatileDbSerialiseConstraints blk, EncodeDiskDep (NestedCtxt Header) blk) ⇒ SerialiseDiskConstraints blk
- getEnv ∷ ∀ m blk r. (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → m r) → m r
- getEnv1 ∷ (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → a → m r) → a → m r
- getEnv2 ∷ (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → a → b → m r) → a → b → m r
- getEnvSTM ∷ ∀ m blk r. (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → STM m r) → STM m r
- getEnvSTM1 ∷ ∀ m blk a r. (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → a → STM m r) → a → STM m r
- data Internal m blk = Internal {
- intCopyToImmutableDB ∷ m (WithOrigin SlotNo)
- intGarbageCollect ∷ SlotNo → m ()
- intUpdateLedgerSnapshots ∷ m ()
- intAddBlockRunner ∷ m Void
- intKillBgThreads ∷ StrictTVar m (m ())
- newtype IteratorKey = IteratorKey Word
- data FollowerHandle m blk = FollowerHandle {
- fhChainType ∷ ChainType
- fhSwitchFork ∷ Point blk → Set (Point blk) → STM m ()
- fhClose ∷ m ()
- newtype FollowerKey = FollowerKey Word
- data FollowerRollState blk
- = RollBackTo !(Point blk)
- | RollForwardFrom !(Point blk)
- data FollowerState m blk b
- = FollowerInit
- | FollowerInImmutableDB !(FollowerRollState blk) !(Iterator m blk (Point blk, b))
- | FollowerInMem !(FollowerRollState blk)
- followerRollStatePoint ∷ FollowerRollState blk → Point blk
- data InvalidBlockInfo blk = InvalidBlockInfo {}
- type InvalidBlocks blk = Map (HeaderHash blk) (InvalidBlockInfo blk)
- data BlockToAdd m blk = BlockToAdd {
- blockPunish ∷ !(InvalidBlockPunishment m)
- blockToAdd ∷ !blk
- varBlockWrittenToDisk ∷ !(StrictTMVar m Bool)
- varBlockProcessed ∷ !(StrictTMVar m (AddBlockResult blk))
- data ChainSelMessage m blk
- = ChainSelAddBlock !(BlockToAdd m blk)
- | ChainSelReprocessLoEBlocks !(StrictTMVar m ())
- data ChainSelQueue m blk
- addBlockToAdd ∷ (IOLike m, HasHeader blk) ⇒ Tracer m (TraceAddBlockEvent blk) → ChainSelQueue m blk → InvalidBlockPunishment m → blk → m (AddBlockPromise m blk)
- addReprocessLoEBlocks ∷ IOLike m ⇒ Tracer m (TraceAddBlockEvent blk) → ChainSelQueue m blk → m (ChainSelectionPromise m)
- closeChainSelQueue ∷ IOLike m ⇒ ChainSelQueue m blk → STM m ()
- getChainSelMessage ∷ IOLike m ⇒ ChainSelQueue m blk → m (ChainSelMessage m blk)
- newChainSelQueue ∷ IOLike m ⇒ Word → m (ChainSelQueue m blk)
- data SelectionChangedInfo blk = SelectionChangedInfo {
- newTipPoint ∷ RealPoint blk
- newTipEpoch ∷ EpochNo
- newTipSlotInEpoch ∷ Word64
- newTipTrigger ∷ RealPoint blk
- newTipSelectView ∷ SelectView (BlockProtocol blk)
- oldTipSelectView ∷ Maybe (SelectView (BlockProtocol blk))
- data TraceAddBlockEvent blk
- = IgnoreBlockOlderThanK (RealPoint blk)
- | IgnoreBlockAlreadyInVolatileDB (RealPoint blk)
- | IgnoreInvalidBlock (RealPoint blk) (ExtValidationError blk)
- | AddedBlockToQueue (RealPoint blk) (Enclosing' Word)
- | PoppedBlockFromQueue (Enclosing' (RealPoint blk))
- | AddedReprocessLoEBlocksToQueue
- | PoppedReprocessLoEBlocksFromQueue
- | AddedBlockToVolatileDB (RealPoint blk) BlockNo IsEBB Enclosing
- | TryAddToCurrentChain (RealPoint blk)
- | TrySwitchToAFork (RealPoint blk) (ChainDiff (HeaderFields blk))
- | StoreButDontChange (RealPoint blk)
- | ChainSelectionLoEDebug (AnchoredFragment (Header blk)) (LoE (AnchoredFragment (Header blk)))
- | AddedToCurrentChain [LedgerEvent blk] (SelectionChangedInfo blk) (AnchoredFragment (Header blk)) (AnchoredFragment (Header blk))
- | SwitchedToAFork [LedgerEvent blk] (SelectionChangedInfo blk) (AnchoredFragment (Header blk)) (AnchoredFragment (Header blk))
- | AddBlockValidation (TraceValidationEvent blk)
- | PipeliningEvent (TracePipeliningEvent blk)
- | ChangingSelection (Point blk)
- data TraceCopyToImmutableDBEvent blk
- data TraceEvent blk
- = TraceAddBlockEvent (TraceAddBlockEvent blk)
- | TraceFollowerEvent (TraceFollowerEvent blk)
- | TraceCopyToImmutableDBEvent (TraceCopyToImmutableDBEvent blk)
- | TraceGCEvent (TraceGCEvent blk)
- | TraceInitChainSelEvent (TraceInitChainSelEvent blk)
- | TraceOpenEvent (TraceOpenEvent blk)
- | TraceIteratorEvent (TraceIteratorEvent blk)
- | TraceSnapshotEvent (TraceSnapshotEvent blk)
- | TraceLedgerReplayEvent (TraceReplayEvent blk)
- | TraceImmutableDBEvent (TraceEvent blk)
- | TraceVolatileDBEvent (TraceEvent blk)
- | TraceLastShutdownUnclean
- data TraceFollowerEvent blk
- = NewFollower
- | FollowerNoLongerInMem (FollowerRollState blk)
- | FollowerSwitchToMem (Point blk) (WithOrigin SlotNo)
- | FollowerNewImmIterator (Point blk) (WithOrigin SlotNo)
- data TraceGCEvent blk
- data TraceInitChainSelEvent blk
- data TraceIteratorEvent blk
- = UnknownRangeRequested (UnknownRange blk)
- | StreamFromVolatileDB (StreamFrom blk) (StreamTo blk) [RealPoint blk]
- | StreamFromImmutableDB (StreamFrom blk) (StreamTo blk)
- | StreamFromBoth (StreamFrom blk) (StreamTo blk) [RealPoint blk]
- | BlockMissingFromVolatileDB (RealPoint blk)
- | BlockWasCopiedToImmutableDB (RealPoint blk)
- | BlockGCedFromVolatileDB (RealPoint blk)
- | SwitchBackToVolatileDB
- data TraceOpenEvent blk
- = StartedOpeningDB
- | OpenedDB (Point blk) (Point blk)
- | ClosedDB (Point blk) (Point blk)
- | StartedOpeningImmutableDB
- | OpenedImmutableDB (Point blk) ChunkNo
- | StartedOpeningVolatileDB
- | OpenedVolatileDB MaxSlotNo
- | StartedOpeningLgrDB
- | OpenedLgrDB
- data TracePipeliningEvent blk
- = SetTentativeHeader (Header blk) Enclosing
- | TrapTentativeHeader (Header blk)
- | OutdatedTentativeHeader (Header blk)
- data TraceValidationEvent blk
- = InvalidBlock (ExtValidationError blk) (RealPoint blk)
- | ValidCandidate (AnchoredFragment (Header blk))
- | UpdateLedgerDbTraceEvent (UpdateLedgerDbTraceEvent blk)
Documentation
data ChainDbEnv m blk Source #
CDB | |
|
Instances
newtype ChainDbHandle m blk Source #
A handle to the internal ChainDB state
CDBHandle (StrictTVar m (ChainDbState m blk)) |
data ChainDbState m blk Source #
ChainDbOpen !(ChainDbEnv m blk) | |
ChainDbClosed |
Instances
Generic (ChainDbState m blk) Source # | |
Defined in Ouroboros.Consensus.Storage.ChainDB.Impl.Types type Rep (ChainDbState m blk) ∷ Type → Type # from ∷ ChainDbState m blk → Rep (ChainDbState m blk) x # to ∷ Rep (ChainDbState m blk) x → ChainDbState m blk # | |
(IOLike m, LedgerSupportsProtocol blk, BlockSupportsDiffusionPipelining blk) ⇒ NoThunks (ChainDbState m blk) Source # | |
type Rep (ChainDbState m blk) Source # | |
Defined in Ouroboros.Consensus.Storage.ChainDB.Impl.Types type Rep (ChainDbState m blk) = D1 ('MetaData "ChainDbState" "Ouroboros.Consensus.Storage.ChainDB.Impl.Types" "ouroboros-consensus-0.21.0.0-inplace" 'False) (C1 ('MetaCons "ChainDbOpen" 'PrefixI 'False) (S1 ('MetaSel ('Nothing ∷ Maybe Symbol) 'NoSourceUnpackedness 'SourceStrict 'DecidedStrict) (Rec0 (ChainDbEnv m blk))) :+: C1 ('MetaCons "ChainDbClosed" 'PrefixI 'False) (U1 ∷ Type → Type)) |
newtype ChainSelectionPromise m Source #
A promise that the chain selection will be performed. It is returned by
triggerChainSelectionAsync
and contains a monadic action that waits until
the corresponding run of Chain Selection is done.
class (ImmutableDbSerialiseConstraints blk, LgrDbSerialiseConstraints blk, VolatileDbSerialiseConstraints blk, EncodeDiskDep (NestedCtxt Header) blk) ⇒ SerialiseDiskConstraints blk Source #
All the serialisation related constraints needed by the ChainDB.
Instances
getEnv ∷ ∀ m blk r. (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → m r) → m r Source #
Check if the ChainDB is open, if so, executing the given function on the
ChainDbEnv
, otherwise, throw a CloseDBError
.
getEnv1 ∷ (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → a → m r) → a → m r Source #
Variant 'of getEnv
for functions taking one argument.
getEnv2 ∷ (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → a → b → m r) → a → b → m r Source #
Variant 'of getEnv
for functions taking two arguments.
getEnvSTM ∷ ∀ m blk r. (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → STM m r) → STM m r Source #
getEnvSTM1 ∷ ∀ m blk a r. (IOLike m, HasCallStack, HasHeader blk) ⇒ ChainDbHandle m blk → (ChainDbEnv m blk → a → STM m r) → a → STM m r Source #
Exposed internals for testing purposes
Internal | |
|
Iterator-related
newtype IteratorKey Source #
We use this internally to track iterators in a map (cdbIterators
) in
the ChainDB state so that we can remove them from the map when the iterator
is closed.
We store them in the map so that the ChainDB can close all open iterators when it is closed itself.
Instances
Follower-related
data FollowerHandle m blk Source #
Internal handle to a Follower
without an explicit b
(blk
,
, etc.) parameter so Header
blkFollower
s with different' b
s can be stored
together in cdbFollowers
.
FollowerHandle | |
|
Instances
NoThunks (FollowerHandle m blk) Source # | |
newtype FollowerKey Source #
We use this internally to track follower in a map (cdbFollowers
) in the
ChainDB state so that we can remove them from the map when the follower is
closed.
We store them in the map so that the ChainDB can close all open followers when it is closed itself and to update the followers in case we switch to a different chain.
Instances
data FollowerRollState blk Source #
Similar to FollowerState
.
RollBackTo !(Point blk) | We don't know at which point the user is, but the next message we'll send is to roll back to this point. |
RollForwardFrom !(Point blk) | We know that the follower is at this point and the next message we'll send is to roll forward to the point after this point on our chain. |
Instances
data FollowerState m blk b Source #
b
corresponds to the BlockComponent
that is being read.
FollowerInit | The This is equivalent to having a Therefore, we have this extra initial state, that avoids this cost. When the user doesn't move the Follower forward, an iterator is opened. |
FollowerInImmutableDB !(FollowerRollState blk) !(Iterator m blk (Point blk, b)) | The Note that the iterator includes 'Point blk' in addition to INVARIANT: for all |
FollowerInMem !(FollowerRollState blk) | The |
Instances
followerRollStatePoint ∷ FollowerRollState blk → Point blk Source #
Get the point the FollowerRollState
should roll back to or roll forward
from.
Invalid blocks
data InvalidBlockInfo blk Source #
In addition to the reason why a block is invalid, the slot number of the
block is stored, so that whenever a garbage collection is performed on the
VolatileDB for some slot s
, the hashes older or equal to s
can be
removed from this map.
Instances
type InvalidBlocks blk = Map (HeaderHash blk) (InvalidBlockInfo blk) Source #
Hashes corresponding to invalid blocks. This is used to ignore these blocks during chain selection.
Blocks to add
data BlockToAdd m blk Source #
Entry in the ChainSelQueue
queue: a block together with the TMVar
s used
to implement AddBlockPromise
.
BlockToAdd | |
|
data ChainSelMessage m blk Source #
Different async tasks for triggering ChainSel
ChainSelAddBlock !(BlockToAdd m blk) | Add a new block |
ChainSelReprocessLoEBlocks | Reprocess blocks that have been postponed by the LoE. |
|
data ChainSelQueue m blk Source #
FIFO queue used to add blocks asynchronously to the ChainDB. Blocks are read from this queue by a background thread, which processes the blocks synchronously.
Instances
NoThunks (ChainSelQueue m blk) Source # | |
addBlockToAdd ∷ (IOLike m, HasHeader blk) ⇒ Tracer m (TraceAddBlockEvent blk) → ChainSelQueue m blk → InvalidBlockPunishment m → blk → m (AddBlockPromise m blk) Source #
Add a block to the ChainSelQueue
queue. Can block when the queue is full.
addReprocessLoEBlocks ∷ IOLike m ⇒ Tracer m (TraceAddBlockEvent blk) → ChainSelQueue m blk → m (ChainSelectionPromise m) Source #
Try to add blocks again that were postponed due to the LoE.
closeChainSelQueue ∷ IOLike m ⇒ ChainSelQueue m blk → STM m () Source #
Flush the ChainSelQueue
queue and notify the waiting threads.
getChainSelMessage ∷ IOLike m ⇒ ChainSelQueue m blk → m (ChainSelMessage m blk) Source #
Get the oldest message from the ChainSelQueue
queue. Can block when the
queue is empty.
newChainSelQueue ∷ IOLike m ⇒ Word → m (ChainSelQueue m blk) Source #
Create a new ChainSelQueue
with the given size.
Trace types
data SelectionChangedInfo blk Source #
Information on having changed our selection to a chain with a (necessarily) new tip.
NOTE: the fields of this record are intentionally lazy to prevent the forcing of this information in case it doesn't have to be traced. However, this means that the tracer processing this message must not hold on to it, otherwise it leaks memory.
SelectionChangedInfo | |
|
Instances
data TraceAddBlockEvent blk Source #
Trace type for the various events that occur when adding a block.
IgnoreBlockOlderThanK (RealPoint blk) | A block with a |
IgnoreBlockAlreadyInVolatileDB (RealPoint blk) | A block that is already in the Volatile DB was ignored. |
IgnoreInvalidBlock (RealPoint blk) (ExtValidationError blk) | A block that is know to be invalid was ignored. |
AddedBlockToQueue (RealPoint blk) (Enclosing' Word) | The block was added to the queue and will be added to the ChainDB by the background thread. The size of the queue is included. |
PoppedBlockFromQueue (Enclosing' (RealPoint blk)) | The block popped from the queue and will imminently be added to the ChainDB. |
AddedReprocessLoEBlocksToQueue | A message was added to the queue that requests that ChainSel reprocess blocks that were postponed by the LoE. |
PoppedReprocessLoEBlocksFromQueue | ChainSel will reprocess blocks that were postponed by the LoE. |
AddedBlockToVolatileDB (RealPoint blk) BlockNo IsEBB Enclosing | A block was added to the Volatile DB |
TryAddToCurrentChain (RealPoint blk) | The block fits onto the current chain, we'll try to use it to extend our chain. |
TrySwitchToAFork (RealPoint blk) (ChainDiff (HeaderFields blk)) | The block fits onto some fork, we'll try to switch to that fork (if it is preferable to our chain). |
StoreButDontChange (RealPoint blk) | The block doesn't fit onto any other block, so we store it and ignore it. |
ChainSelectionLoEDebug (AnchoredFragment (Header blk)) (LoE (AnchoredFragment (Header blk))) | Debugging information about chain selection and LoE |
AddedToCurrentChain [LedgerEvent blk] (SelectionChangedInfo blk) (AnchoredFragment (Header blk)) (AnchoredFragment (Header blk)) | The new block fits onto the current chain (first fragment) and we have successfully used it to extend our (new) current chain (second fragment). |
SwitchedToAFork [LedgerEvent blk] (SelectionChangedInfo blk) (AnchoredFragment (Header blk)) (AnchoredFragment (Header blk)) | The new block fits onto some fork and we have switched to that fork (second fragment), as it is preferable to our (previous) current chain (first fragment). |
AddBlockValidation (TraceValidationEvent blk) | An event traced during validating performed while adding a block. |
PipeliningEvent (TracePipeliningEvent blk) | The tentative header (in the context of diffusion pipelining) has been updated. |
ChangingSelection (Point blk) | Herald of |
Instances
data TraceCopyToImmutableDBEvent blk Source #
CopiedBlockToImmutableDB (Point blk) | A block was successfully copied to the ImmutableDB. |
NoBlocksToCopyToImmutableDB | There are no block to copy to the ImmutableDB. |
Instances
data TraceEvent blk Source #
Trace type for the various events of the ChainDB.
Instances
data TraceFollowerEvent blk Source #
NewFollower | A new follower was created. |
FollowerNoLongerInMem (FollowerRollState blk) | The follower was in the |
FollowerSwitchToMem | The follower was in the |
| |
FollowerNewImmIterator | The follower is in the |
|
Instances
data TraceGCEvent blk Source #
ScheduledGC SlotNo Time | A garbage collection for the given |
PerformedGC SlotNo | A garbage collection for the given |
Instances
data TraceInitChainSelEvent blk Source #
StartedInitChainSelection | An event traced when inital chain selection has started during the initialization of ChainDB |
InitialChainSelected | An event traced when inital chain has been selected |
InitChainSelValidation (TraceValidationEvent blk) | An event traced during validation performed while performing initial chain selection. |
Instances
data TraceIteratorEvent blk Source #
UnknownRangeRequested (UnknownRange blk) | An unknown range was requested, see |
StreamFromVolatileDB (StreamFrom blk) (StreamTo blk) [RealPoint blk] | Stream only from the VolatileDB. |
StreamFromImmutableDB (StreamFrom blk) (StreamTo blk) | Stream only from the ImmutableDB. |
StreamFromBoth (StreamFrom blk) (StreamTo blk) [RealPoint blk] | Stream from both the VolatileDB and the ImmutableDB. |
BlockMissingFromVolatileDB (RealPoint blk) | A block is no longer in the VolatileDB because it has been garbage collected. It might now be in the ImmutableDB if it was part of the current chain. |
BlockWasCopiedToImmutableDB (RealPoint blk) | A block that has been garbage collected from the VolatileDB is now found and streamed from the ImmutableDB. |
BlockGCedFromVolatileDB (RealPoint blk) | A block is no longer in the VolatileDB and isn't in the ImmutableDB either; it wasn't part of the current chain. |
SwitchBackToVolatileDB | We have streamed one or more blocks from the ImmutableDB that were part of the VolatileDB when initialising the iterator. Now, we have to look back in the VolatileDB again because the ImmutableDB doesn't have the next block we're looking for. |
Instances
data TraceOpenEvent blk Source #
StartedOpeningDB | The ChainDB started the process of opening. |
OpenedDB | The ChainDB was opened. |
ClosedDB | The ChainDB was closed. |
StartedOpeningImmutableDB | The ImmutableDB started the process of opening. |
OpenedImmutableDB | The ImmutableDB was opened. |
StartedOpeningVolatileDB | The VolatileDB started opening. |
OpenedVolatileDB MaxSlotNo | The VolatileDB was opened, with the highest seen slot number for any block currently in the DB. |
StartedOpeningLgrDB | The LedgerDB started opening. |
OpenedLgrDB | The LedgerDB was opened. |
Instances
data TracePipeliningEvent blk Source #
SetTentativeHeader (Header blk) Enclosing | A new tentative header got set. |
TrapTentativeHeader (Header blk) | The body of tentative block turned out to be invalid. |
OutdatedTentativeHeader (Header blk) | We selected a new (better) chain, which cleared the previous tentative header. |
Instances
Show (Header blk) ⇒ Show (TracePipeliningEvent blk) Source # | |
Defined in Ouroboros.Consensus.Storage.ChainDB.Impl.Types showsPrec ∷ Int → TracePipeliningEvent blk → ShowS # show ∷ TracePipeliningEvent blk → String # showList ∷ [TracePipeliningEvent blk] → ShowS # | |
Eq (Header blk) ⇒ Eq (TracePipeliningEvent blk) Source # | |
Defined in Ouroboros.Consensus.Storage.ChainDB.Impl.Types (==) ∷ TracePipeliningEvent blk → TracePipeliningEvent blk → Bool # (/=) ∷ TracePipeliningEvent blk → TracePipeliningEvent blk → Bool # |
data TraceValidationEvent blk Source #
InvalidBlock (ExtValidationError blk) (RealPoint blk) | A point was found to be invalid. |
ValidCandidate (AnchoredFragment (Header blk)) | A candidate chain was valid. |
UpdateLedgerDbTraceEvent (UpdateLedgerDbTraceEvent blk) |