ouroboros-consensus-0.1.0.1: Consensus layer for the Ouroboros blockchain protocol
Safe Haskell None
Language Haskell2010

Ouroboros.Consensus.Storage.ChainDB.API

Synopsis

Main ChainDB API

data ChainDB m blk Source #

The chain database

The chain database provides a unified interface on top of:

  • The ImmutableDB, storing the part of the chain that can't roll back.
  • The VolatileDB, storing the blocks near the tip of the chain, possibly in multiple competing forks.
  • The LedgerDB, storing snapshots of the ledger state for blocks in the ImmutableDB (and in-memory snapshots for the rest).

In addition to providing a unifying interface on top of these disparate components, the main responsibilities that the ChainDB itself has are:

  • Chain selection (on initialization and whenever a block is added)
  • Trigger full recovery whenever we detect disk failure in any component
  • Provide iterators across fixed fragments of the current chain
  • Provide followers that track the status of the current chain

The ChainDB instantiates all the various type parameters of these databases to conform to the unified interface we provide here.

Constructors

ChainDB

Fields

  • addBlockAsync :: InvalidBlockPunishment m -> blk -> m ( AddBlockPromise m blk)

    Add a block to the heap of blocks

    We do not assume that the block is valid (under the legder rules); it is the responsibility of the Chain DB itself to only select chains that are valid.

    Conversely, the caller cannot assume that the new block will be added to the current chain; even if the block is valid, it will not become part of the chain if there are other chains available that are preferred by the consensus algorithm (typically, longer chains).

    This function typically returns immediately, yielding a AddBlockPromise which can be used to wait for the result. You can use addBlock to add the block synchronously.

    NOTE: back pressure can be applied when overloaded.

  • getCurrentChain :: STM m ( AnchoredFragment ( Header blk))

    Get the current chain fragment

    Suppose the current chain is

    a -> b -> c -> d -> e -> f

    and suppose k = 2 ; this means that the most distant fork we can switch to is something like

    a -> b -> c -> d -> e' -> f'

    The fragment we return will be [e, f] , anchored at d . In other words, the length of the fragment will under normal circumstances be exactly k blocks long. It may be shorter if

    • We are near genesis The anchor will be the genesis point (which does not correspond to an actual block)
    • The volatile DB suffered some data loss Typically (but not necessarily) the immutable DB will not be empty and the anchor will be pointing to the tip of the immutable DB.

    POSTCONDITION: The Chain DB will be able to switch to any fork starting from the anchor of the returned fragment or any subsequent block (provided the new fork is at least of the same length as the old).

    NOTE: A direct consequence of this guarantee is that the anchor of the fragment will move as the chain grows.

  • getLedgerDB :: STM m ( LedgerDB ( ExtLedgerState blk))

    Return the LedgerDB containing the last k ledger states.

  • getTipBlock :: m ( Maybe blk)

    Get block at the tip of the chain, if one exists

    Returns Nothing if the database is empty.

  • getTipHeader :: m ( Maybe ( Header blk))

    Get header at the tip of the chain

    NOTE: Calling getTipHeader is cheaper than getTipBlock and then extracting the header: most of the time the header at the tip is actually in memory, whereas the block never is.

    Returns Nothing if the database is empty.

  • getTipPoint :: STM m ( Point blk)

    Get point of the tip of the chain

    Will return genesisPoint if the database is empty; if the current chain fragment is empty due to data loss in the volatile DB, getTipPoint will return the tip of the immutable DB.

  • getBlockComponent :: forall b. BlockComponent blk b -> RealPoint blk -> m ( Maybe b)

    Get the given component(s) of the block at the specified point. If there is no block at the given point, Nothing is returned.

  • getIsFetched :: STM m ( Point blk -> Bool )

    Return membership check function for recent blocks

    This check is only reliable for blocks up to k away from the tip. For blocks older than that the results should be regarded as non-deterministic.

  • getIsValid :: STM m ( RealPoint blk -> Maybe Bool )

    Return a function that tells whether a block is known to be valid or invalid.

    The function will return:

    • Just True : for blocks in the volatile DB that have been validated and were found to be valid. All blocks in the current chain fragment (i.e., getCurrentChain ) are valid.
    • Just False : for blocks in the volatile DB that have been validated and were found to be invalid.
    • Nothing : for blocks not or no longer in the volatile DB, whether they are valid or not, including blocks in the immutable DB. Also for blocks in the volatile DB that haven't been validated (yet), e.g., because they are disconnected from the current chain or they are part of a shorter fork.
  • getMaxSlotNo :: STM m MaxSlotNo

    Get the highest slot number stored in the ChainDB.

    Note that the corresponding block doesn't have to be part of the current chain, it could be part of some fork, or even be a disconnected block.

  • stream :: forall b. ResourceRegistry m -> BlockComponent blk b -> StreamFrom blk -> StreamTo blk -> m ( Either ( UnknownRange blk) ( Iterator m blk b))

    Stream blocks

    Streaming is not restricted to the current fork, but there must be an unbroken path from the starting point to the end point /at the time of initialization/ of the iterator. Once the iterator has been initialized, it will not be affected by subsequent calls to addBlock . To track the current chain, use a Follower instead.

    Streaming blocks older than k is permitted, but only when they are part of the current fork (at the time of initialization). Streaming a fork that forks off more than k blocks in the past is not permitted and an UnknownRange error will be returned in that case.

    The iterator does have a limited lifetime, however. The chain DB internally partitions the chain into an " immutable " part and a " volatile " part, moving blocks from the volatile DB to the immutable DB when they become more than k deep into the chain. When a block with slot number n is added to the immutble DB, a time delay t kicks in; after that time delay expires, all blocks older than n may be removed from the volatile DB, /including any blocks that happen to live on other forks/ (since those forks must now, by definition, be too distant). This time delay t also provides a worst-case bound for the lifetime of the iterator: if the iterator traverses a chain that forks off from our current chain at the tip of the immutable DB, then the first block on that fork will become unavailable as soon as another block is pushed to the current chain and the subsequent time delay expires.

    Note: although blocks are moved from the volatile DB to the immutable DB after they have become k deep into the chain, due to data corruption the suffix of the chain in the volatile DB might be shorter than k . The immutable DB always determines the maximum rollback, which may therefore be shorter than k under such circumstances. In addition, streaming blocks which aren't on the current fork is permitted, but the oldest volatile block must fit on to the tip of the immutable DB.

    When the given bounds are nonsensical, an InvalidIteratorRange is thrown.

    When the given bounds are not part of the chain DB, an UnknownRange error is returned.

    To stream all blocks from the current chain, use streamAll , as it correctly handles an empty ChainDB.

  • newFollower :: forall b. ResourceRegistry m -> ChainType -> BlockComponent blk b -> m ( Follower m blk b)

    Chain follower

    A chain follower is an iterator that tracks the state of the current chain: calling next on the iterator will either give you the next block header, or (if we have switched to a fork) the instruction to rollback.

    The tracking iterator starts at genesis (see also trackForward ).

    This is intended for use by chain consumers to reliably follow a chain, desipite the chain being volatile.

    Examples of users: * The server side of the chain sync mini-protocol for the node-to-node protocol using headers and the block size. * The server side of the chain sync mini-protocol for the node-to-client protocol using blocks.

  • getIsInvalidBlock :: STM m ( WithFingerprint ( HeaderHash blk -> Maybe ( InvalidBlockReason blk)))

    Function to check whether a block is known to be invalid.

    Blocks unknown to the ChainDB will result in False .

    If the hash corresponds to a block that is known to be invalid, but is now older than k , this function may return False .

    Whenever a new invalid block is added, the Fingerprint will be changed. This is useful when "watching" this function in a transaction.

    Note that when invalid blocks are garbage collected and thus no longer detected by this function, the Fingerprint doesn't have to change, since the function will not detect new invalid blocks.

  • closeDB :: m ()

    Close the ChainDB

    Idempotent.

    Should only be called on shutdown.

  • isOpen :: STM m Bool

    Return True when the database is open.

    False when the database is closed.

getHeaderStateHistory :: Monad ( STM m) => ChainDB m blk -> STM m ( HeaderStateHistory blk) Source #

Get a HeaderStateHistory populated with the HeaderState s of the last k blocks of the current chain.

getImmutableLedger :: Monad ( STM m) => ChainDB m blk -> STM m ( ExtLedgerState blk) Source #

Get the immutable ledger, i.e., typically k blocks back.

getPastLedger :: ( Monad ( STM m), LedgerSupportsProtocol blk) => ChainDB m blk -> Point blk -> STM m ( Maybe ( ExtLedgerState blk)) Source #

Get the ledger for the given point.

When the given point is not among the last k blocks of the current chain (i.e., older than k or not on the current chain), Nothing is returned.

Adding a block

data AddBlockPromise m blk Source #

Constructors

AddBlockPromise

Fields

  • blockWrittenToDisk :: STM m Bool

    Use this STM transaction to wait until the block has been written to disk.

    Returns True when the block was written to disk or False when it was ignored, e.g., because it was older than k .

    If the STM transaction has returned True then getIsFetched will return True for the added block.

    NOTE: Even when the result is False , getIsFetched might still return True , e.g., the block was older than k , but it has been downloaded and stored on disk before.

  • blockProcessed :: STM m ( Point blk)

    Use this STM transaction to wait until the block has been processed: the block has been written to disk and chain selection has been performed for the block, unless the block is from the future.

    The ChainDB's tip after chain selection is returned. When this tip doesn't match the added block, it doesn't necessarily mean the block wasn't adopted. We might have adopted a longer chain of which the added block is a part, but not the tip.

    NOTE: When the block is from the future, chain selection for the block won't be performed until the block is no longer in the future, which might take some time. For that reason, this transaction will not wait for chain selection of a block from the future. It will return the current tip of the ChainDB after writing the block to disk.

addBlock :: IOLike m => ChainDB m blk -> InvalidBlockPunishment m -> blk -> m ( Point blk) Source #

Add a block synchronously: wait until the block has been processed (see blockProcessed ). The new tip of the ChainDB is returned.

addBlockWaitWrittenToDisk :: IOLike m => ChainDB m blk -> InvalidBlockPunishment m -> blk -> m Bool Source #

Add a block synchronously: wait until the block has been written to disk (see blockWrittenToDisk ).

addBlock_ :: IOLike m => ChainDB m blk -> InvalidBlockPunishment m -> blk -> m () Source #

Add a block synchronously. Variant of addBlock that doesn't return the new tip of the ChainDB.

Serialised block/header with its point

data WithPoint blk b Source #

A b together with its Point .

The Point is needed because we often need to know the hash, slot, or point itself of the block or header in question, and we don't want to deserialise the block to obtain it.

Constructors

WithPoint

Fields

BlockComponent

data BlockComponent blk a where Source #

Which component of the block to read from a database: the whole block, its header, its hash, the block size, ..., or combinations thereof.

NOTE: when requesting multiple components, we will not optimise/cache them.

Support for tests

fromChain :: forall m blk. IOLike m => m ( ChainDB m blk) -> Chain blk -> m ( ChainDB m blk) Source #

Iterator API

data Iterator m blk b Source #

Constructors

Iterator

Fields

Instances

Instances details
Functor m => Functor ( Iterator m blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

Methods

fmap :: (a -> b) -> Iterator m blk a -> Iterator m blk b Source #

(<$) :: a -> Iterator m blk b -> Iterator m blk a Source #

Foldable m => Foldable ( Iterator m blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

Methods

fold :: Monoid m0 => Iterator m blk m0 -> m0 Source #

foldMap :: Monoid m0 => (a -> m0) -> Iterator m blk a -> m0 Source #

foldMap' :: Monoid m0 => (a -> m0) -> Iterator m blk a -> m0 Source #

foldr :: (a -> b -> b) -> b -> Iterator m blk a -> b Source #

foldr' :: (a -> b -> b) -> b -> Iterator m blk a -> b Source #

foldl :: (b -> a -> b) -> b -> Iterator m blk a -> b Source #

foldl' :: (b -> a -> b) -> b -> Iterator m blk a -> b Source #

foldr1 :: (a -> a -> a) -> Iterator m blk a -> a Source #

foldl1 :: (a -> a -> a) -> Iterator m blk a -> a Source #

toList :: Iterator m blk a -> [a] Source #

null :: Iterator m blk a -> Bool Source #

length :: Iterator m blk a -> Int Source #

elem :: Eq a => a -> Iterator m blk a -> Bool Source #

maximum :: Ord a => Iterator m blk a -> a Source #

minimum :: Ord a => Iterator m blk a -> a Source #

sum :: Num a => Iterator m blk a -> a Source #

product :: Num a => Iterator m blk a -> a Source #

Traversable m => Traversable ( Iterator m blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

Methods

traverse :: Applicative f => (a -> f b) -> Iterator m blk a -> f ( Iterator m blk b) Source #

sequenceA :: Applicative f => Iterator m blk (f a) -> f ( Iterator m blk a) Source #

mapM :: Monad m0 => (a -> m0 b) -> Iterator m blk a -> m0 ( Iterator m blk b) Source #

sequence :: Monad m0 => Iterator m blk (m0 a) -> m0 ( Iterator m blk a) Source #

data IteratorResult blk b Source #

Constructors

IteratorExhausted
IteratorResult b
IteratorBlockGCed ( RealPoint blk)

The block that was supposed to be streamed was garbage-collected from the VolatileDB, but not added to the ImmutableDB.

This will only happen when streaming very old forks very slowly.

Instances

Instances details
Functor ( IteratorResult blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

Foldable ( IteratorResult blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

Traversable ( IteratorResult blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

( Eq blk, Eq b, StandardHash blk) => Eq ( IteratorResult blk b) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

( Show blk, Show b, StandardHash blk) => Show ( IteratorResult blk b) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

data StreamFrom blk Source #

The lower bound for an iterator

Hint: use StreamFromExclusive genesisPoint to start streaming from Genesis.

Instances

Instances details
StandardHash blk => Eq ( StreamFrom blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

StandardHash blk => Show ( StreamFrom blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

Generic ( StreamFrom blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

Associated Types

type Rep ( StreamFrom blk) :: Type -> Type Source #

( StandardHash blk, Typeable blk) => NoThunks ( StreamFrom blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

type Rep ( StreamFrom blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

type Rep ( StreamFrom blk) = D1 (' MetaData "StreamFrom" "Ouroboros.Consensus.Storage.Common" "ouroboros-consensus-0.1.0.1-DT4Cvwf63DZKctsEvaJqCU" ' False ) ( C1 (' MetaCons "StreamFromInclusive" ' PrefixI ' False ) ( S1 (' MetaSel (' Nothing :: Maybe Symbol ) ' NoSourceUnpackedness ' SourceStrict ' DecidedStrict ) ( Rec0 ( RealPoint blk))) :+: C1 (' MetaCons "StreamFromExclusive" ' PrefixI ' False ) ( S1 (' MetaSel (' Nothing :: Maybe Symbol ) ' NoSourceUnpackedness ' SourceStrict ' DecidedStrict ) ( Rec0 ( Point blk))))

newtype StreamTo blk Source #

Instances

Instances details
StandardHash blk => Eq ( StreamTo blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

StandardHash blk => Show ( StreamTo blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

Generic ( StreamTo blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

Associated Types

type Rep ( StreamTo blk) :: Type -> Type Source #

( StandardHash blk, Typeable blk) => NoThunks ( StreamTo blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

type Rep ( StreamTo blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.Common

type Rep ( StreamTo blk) = D1 (' MetaData "StreamTo" "Ouroboros.Consensus.Storage.Common" "ouroboros-consensus-0.1.0.1-DT4Cvwf63DZKctsEvaJqCU" ' True ) ( C1 (' MetaCons "StreamToInclusive" ' PrefixI ' False ) ( S1 (' MetaSel (' Nothing :: Maybe Symbol ) ' NoSourceUnpackedness ' NoSourceStrictness ' DecidedLazy ) ( Rec0 ( RealPoint blk))))

data UnknownRange blk Source #

Constructors

MissingBlock ( RealPoint blk)

The block at the given point was not found in the ChainDB.

ForkTooOld ( StreamFrom blk)

The requested range forks off too far in the past, i.e. it doesn't fit on the tip of the ImmutableDB.

emptyIterator :: Monad m => Iterator m blk b Source #

An iterator that is immediately exhausted.

streamAll :: ( MonadSTM m, HasHeader blk, HasCallStack ) => ChainDB m blk -> ResourceRegistry m -> BlockComponent blk b -> m ( Iterator m blk b) Source #

Stream all blocks from the current chain.

streamFrom :: ( MonadSTM m, HasHeader blk, HasCallStack ) => StreamFrom blk -> ChainDB m blk -> ResourceRegistry m -> BlockComponent blk b -> m ( Iterator m blk b) Source #

Stream blocks from the given point up to the tip from the current chain.

To stream all blocks from the current chain from the ChainDB, one would use StreamFromExclusive genesisPoint as the lower bound and StreamToInclusive tip as the upper bound where tip is retrieved with getTipPoint .

However, when the ChainDB is empty, tip will be genesisPoint too, in which case the bounds don't make sense. This function correctly handles this case.

Note that this is not a Follower , so the stream will not include blocks that are added to the current chain after starting the stream.

traverseIterator :: Monad m => (b -> m b') -> Iterator m blk b -> Iterator m blk b' Source #

Variant of traverse instantiated to Iterator m blk that executes the monadic function when calling iteratorNext .

validBounds :: StandardHash blk => StreamFrom blk -> StreamTo blk -> Bool Source #

Check whether the bounds make sense

An example of bounds that don't make sense:

StreamFromExclusive (BlockPoint 3 ..)
StreamToInclusive   (RealPoint  3 ..)

This function does not check whether the bounds correspond to existing blocks.

Invalid block reason

data InvalidBlockReason blk Source #

The reason why a block is invalid.

Constructors

ValidationError !( ExtValidationError blk)

The ledger found the block to be invalid.

InFutureExceedsClockSkew !( RealPoint blk)

The block's slot is in the future, exceeding the allowed clock skew.

Possible causes, order by decreasing likelihood:

  1. Our clock is behind (significantly more likely than the others)
  2. Their clock is ahead
  3. It's intentional, i.e., an attack

Instances

Instances details
LedgerSupportsProtocol blk => Eq ( InvalidBlockReason blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

LedgerSupportsProtocol blk => Show ( InvalidBlockReason blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

Generic ( InvalidBlockReason blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

LedgerSupportsProtocol blk => NoThunks ( InvalidBlockReason blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

type Rep ( InvalidBlockReason blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

type Rep ( InvalidBlockReason blk) = D1 (' MetaData "InvalidBlockReason" "Ouroboros.Consensus.Storage.ChainDB.API" "ouroboros-consensus-0.1.0.1-DT4Cvwf63DZKctsEvaJqCU" ' False ) ( C1 (' MetaCons "ValidationError" ' PrefixI ' False ) ( S1 (' MetaSel (' Nothing :: Maybe Symbol ) ' NoSourceUnpackedness ' SourceStrict ' DecidedStrict ) ( Rec0 ( ExtValidationError blk))) :+: C1 (' MetaCons "InFutureExceedsClockSkew" ' PrefixI ' False ) ( S1 (' MetaSel (' Nothing :: Maybe Symbol ) ' NoSourceUnpackedness ' SourceStrict ' DecidedStrict ) ( Rec0 ( RealPoint blk))))

Followers

data Follower m blk a Source #

Follower

See newFollower for more info.

The type parameter a will be instantiated with blk or Header blk@.

Constructors

Follower

Fields

Instances

Instances details
Functor m => Functor ( Follower m blk) Source #
Instance details

Defined in Ouroboros.Consensus.Storage.ChainDB.API

Methods

fmap :: (a -> b) -> Follower m blk a -> Follower m blk b Source #

(<$) :: a -> Follower m blk b -> Follower m blk a Source #

traverseFollower :: Monad m => (b -> m b') -> Follower m blk b -> Follower m blk b' Source #

Variant of traverse instantiated to Follower m blk that executes the monadic function when calling followerInstruction and followerInstructionBlocking .

Recovery

data ChainDbFailure blk Source #

Database failure

This exception wraps any kind of unexpected problem with the on-disk storage of the chain.

The various constructors only serve to give more detailed information about what went wrong, in case sysadmins want to investigate the disk failure. The Chain DB itself does not differentiate; all disk failures are treated equal and all trigger the same recovery procedure.

Constructors

LgrDbFailure FsError

The ledger DB threw a file-system error

ChainDbMissingBlock ( RealPoint blk)

Block missing from the chain DB

Thrown when we are not sure in which DB the block should have been.

data IsEBB Source #

Whether a block is an Epoch Boundary Block (EBB)

See Ouroboros.Storage.ImmutableDB.API for a discussion of EBBs. Key idiosyncracies:

  • An EBB carries no unique information.
  • An EBB has the same BlockNo as its predecessor.
  • EBBs are vestigial. As of Shelley, nodes no longer forge EBBs: they are only a legacy/backwards-compatibility concern.

Constructors

IsEBB
IsNotEBB

Instances

Instances details
Eq IsEBB Source #
Instance details

Defined in Ouroboros.Consensus.Block.EBB

Show IsEBB Source #
Instance details

Defined in Ouroboros.Consensus.Block.EBB

Generic IsEBB Source #
Instance details

Defined in Ouroboros.Consensus.Block.EBB

NoThunks IsEBB Source #
Instance details

Defined in Ouroboros.Consensus.Block.EBB

Serialise IsEBB Source #
Instance details

Defined in Ouroboros.Consensus.Block.EBB

Condense IsEBB Source #
Instance details

Defined in Ouroboros.Consensus.Block.EBB

type Rep IsEBB Source #
Instance details

Defined in Ouroboros.Consensus.Block.EBB

type Rep IsEBB = D1 (' MetaData "IsEBB" "Ouroboros.Consensus.Block.EBB" "ouroboros-consensus-0.1.0.1-DT4Cvwf63DZKctsEvaJqCU" ' False ) ( C1 (' MetaCons "IsEBB" ' PrefixI ' False ) ( U1 :: Type -> Type ) :+: C1 (' MetaCons "IsNotEBB" ' PrefixI ' False ) ( U1 :: Type -> Type ))

Exceptions

data ChainDbError blk Source #

Database error

Thrown upon incorrect use: invalid input.

Constructors

ClosedDBError PrettyCallStack

The ChainDB is closed.

This will be thrown when performing any operation on the ChainDB except for isOpen and closeDB . The CallStack of the operation on the ChainDB is included in the error.

ClosedFollowerError

The follower is closed.

This will be thrown when performing any operation on a closed followers, except for followerClose .

InvalidIteratorRange ( StreamFrom blk) ( StreamTo blk)

When there is no chain/fork that satisfies the bounds passed to streamBlocks .

  • The lower and upper bound are not on the same chain.
  • The bounds don't make sense, e.g., the lower bound starts after the upper bound, or the lower bound starts from genesis, inclusive .