Documentation ¶
Index ¶
- Constants
- Variables
- func BuildExtendedCommitInfo(ec *types.ExtendedCommit, valSet *types.ValidatorSet, initialHeight int64, ...) abci.ExtendedCommitInfo
- func BuildLastCommitInfo(block *types.Block, lastValSet *types.ValidatorSet, initialHeight int64) abci.CommitInfo
- func ExecCommitBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block, logger log.Logger, ...) ([]byte, error)
- func IsEmpty(store dbStore) (bool, error)
- func MakeGenesisDocFromFile(genDocFile string) (*types.GenesisDoc, error)
- func MedianTime(commit *types.Commit, validators *types.ValidatorSet) time.Time
- func Rollback(bs BlockStore, ss Store, removeBlock bool) (int64, []byte, error)
- func TxPostCheck(state State) mempl.PostCheckFunc
- func TxPreCheck(state State) mempl.PreCheckFunc
- func TxResultsHash(txResults []*abci.ExecTxResult) []byte
- type ABCIResponsesPrunedInfo
- type BlockExecutor
- func (blockExec *BlockExecutor) ApplyBlock(state State, blockID types.BlockID, block *types.Block) (State, error)
- func (blockExec *BlockExecutor) Commit(state State, block *types.Block, abciResponse *abci.ResponseFinalizeBlock) (int64, error)
- func (blockExec *BlockExecutor) CreateProposalBlock(ctx context.Context, height int64, state State, ...) (*types.Block, error)
- func (blockExec *BlockExecutor) ExtendVote(ctx context.Context, vote *types.Vote, block *types.Block, state State) ([]byte, error)
- func (blockExec *BlockExecutor) ProcessProposal(block *types.Block, state State) (bool, error)
- func (blockExec *BlockExecutor) SetEventBus(eventBus types.BlockEventPublisher)
- func (blockExec *BlockExecutor) Store() Store
- func (blockExec *BlockExecutor) ValidateBlock(state State, block *types.Block) error
- func (blockExec *BlockExecutor) VerifyVoteExtension(ctx context.Context, vote *types.Vote) error
- type BlockExecutorOption
- type BlockStore
- type BlocksPrunedInfo
- type EmptyEvidencePool
- func (EmptyEvidencePool) AddEvidence(types.Evidence) error
- func (EmptyEvidencePool) CheckEvidence(types.EvidenceList) error
- func (EmptyEvidencePool) PendingEvidence(int64) (ev []types.Evidence, size int64)
- func (EmptyEvidencePool) ReportConflictingVotes(*types.Vote, *types.Vote)
- func (EmptyEvidencePool) Update(State, types.EvidenceList)
- type ErrAppBlockHeightTooHigh
- type ErrAppBlockHeightTooLow
- type ErrBlockHashMismatch
- type ErrCannotLoadState
- type ErrFailedToPruneBlocks
- type ErrFailedToPruneStates
- type ErrInvalidBlock
- type ErrLastStateMismatch
- type ErrNoABCIResponsesForHeight
- type ErrNoConsensusParamsForHeight
- type ErrNoValSetForHeight
- type ErrProxyAppConn
- type ErrPrunerFailedToGetRetainHeight
- type ErrPrunerFailedToLoadState
- type ErrStateMismatch
- type ErrUnknownBlock
- type EvidencePool
- type Metrics
- type NoopPrunerObserver
- type Pruner
- func (p *Pruner) GetABCIResRetainHeight() (int64, error)
- func (p *Pruner) GetApplicationRetainHeight() (int64, error)
- func (p *Pruner) GetBlockIndexerRetainHeight() (int64, error)
- func (p *Pruner) GetCompanionBlockRetainHeight() (int64, error)
- func (p *Pruner) GetTxIndexerRetainHeight() (int64, error)
- func (p *Pruner) OnStart() error
- func (p *Pruner) SetABCIResRetainHeight(height int64) error
- func (p *Pruner) SetApplicationBlockRetainHeight(height int64) error
- func (p *Pruner) SetBlockIndexerRetainHeight(height int64) error
- func (p *Pruner) SetCompanionBlockRetainHeight(height int64) error
- func (p *Pruner) SetObserver(obs PrunerObserver)
- func (p *Pruner) SetTxIndexerRetainHeight(height int64) error
- type PrunerObserver
- type PrunerOption
- type State
- func (state State) Bytes() []byte
- func (state State) Copy() State
- func (state State) Equals(state2 State) bool
- func (state State) IsEmpty() bool
- func (state State) MakeBlock(height int64, txs []types.Tx, lastCommit *types.Commit, ...) *types.Block
- func (state *State) ToProto() (*cmtstate.State, error)
- type Store
- type StoreOptions
Constants ¶
const ( // MetricsSubsystem is a subsystem shared by all metrics exposed by this // package. MetricsSubsystem = "state" )
Variables ¶
var ( ErrFinalizeBlockResponsesNotPersisted = errors.New("node is not persisting finalize block responses") ErrPrunerCannotLowerRetainHeight = errors.New("cannot set a height lower than previously requested - heights might have already been pruned") ErrInvalidRetainHeight = errors.New("retain height cannot be less or equal than 0") )
var ( AppRetainHeightKey = []byte("AppRetainHeightKey") CompanionBlockRetainHeightKey = []byte("DCBlockRetainHeightKey") ABCIResultsRetainHeightKey = []byte("ABCIResRetainHeightKey") )
var ( ErrKeyNotFound = errors.New("key not found") ErrInvalidHeightValue = errors.New("invalid height value") )
var InitStateVersion = cmtstate.Version{ Consensus: cmtversion.Consensus{ Block: version.BlockProtocol, App: 0, }, Software: version.TMCoreSemVer, }
InitStateVersion sets the Consensus.Block and Software versions, but leaves the Consensus.App version blank. The Consensus.App version will be set during the Handshake, once we hear from the app what protocol version it is running.
Functions ¶
func BuildExtendedCommitInfo ¶
func BuildExtendedCommitInfo(ec *types.ExtendedCommit, valSet *types.ValidatorSet, initialHeight int64, ap types.ABCIParams) abci.ExtendedCommitInfo
BuildExtendedCommitInfo builds an ExtendedCommitInfo from the given block and validator set. If you want to load the validator set from the store instead of providing it, use buildExtendedCommitInfoFromStore.
func BuildLastCommitInfo ¶
func BuildLastCommitInfo(block *types.Block, lastValSet *types.ValidatorSet, initialHeight int64) abci.CommitInfo
BuildLastCommitInfo builds a CommitInfo from the given block and validator set. If you want to load the validator set from the store instead of providing it, use buildLastCommitInfoFromStore.
func ExecCommitBlock ¶
func ExecCommitBlock( appConnConsensus proxy.AppConnConsensus, block *types.Block, logger log.Logger, store Store, initialHeight int64, ) ([]byte, error)
ExecCommitBlock executes and commits a block on the proxyApp without validating or mutating the state. It returns the application root hash (result of abci.Commit).
func MakeGenesisDocFromFile ¶
func MakeGenesisDocFromFile(genDocFile string) (*types.GenesisDoc, error)
MakeGenesisDocFromFile reads and unmarshals genesis doc from the given file.
func MedianTime ¶
MedianTime computes a median time for a given Commit (based on Timestamp field of votes messages) and the corresponding validator set. The computed time is always between timestamps of the votes sent by honest processes, i.e., a faulty processes can not arbitrarily increase or decrease the computed value.
func Rollback ¶
Rollback overwrites the current CometBFT state (height n) with the most recent previous state (height n - 1). Note that this function does not affect application state.
func TxPostCheck ¶
func TxPostCheck(state State) mempl.PostCheckFunc
TxPostCheck returns a function to filter transactions after processing. The function limits the gas wanted by a transaction to the block's maximum total gas.
func TxPreCheck ¶
func TxPreCheck(state State) mempl.PreCheckFunc
TxPreCheck returns a function to filter transactions before processing. The function limits the size of a transaction to the block's maximum data size.
func TxResultsHash ¶
func TxResultsHash(txResults []*abci.ExecTxResult) []byte
TxResultsHash returns the root hash of a Merkle tree of ExecTxResulst responses (see ABCIResults.Hash)
See merkle.SimpleHashFromByteSlices
Types ¶
type ABCIResponsesPrunedInfo ¶
type ABCIResponsesPrunedInfo struct { FromHeight int64 // The height from which ABCI responses were pruned (inclusive). ToHeight int64 // The height to which ABCI responses were pruned (inclusive). }
ABCIResponsesPrunedInfo provides information about ABCI responses pruned during a single run of the pruner.
type BlockExecutor ¶
type BlockExecutor struct {
// contains filtered or unexported fields
}
BlockExecutor provides the context and accessories for properly executing a block.
func NewBlockExecutor ¶
func NewBlockExecutor( stateStore Store, logger log.Logger, proxyApp proxy.AppConnConsensus, mempool mempool.Mempool, evpool EvidencePool, blockStore BlockStore, options ...BlockExecutorOption, ) *BlockExecutor
NewBlockExecutor returns a new BlockExecutor with a NopEventBus. Call SetEventBus to provide one.
func (*BlockExecutor) ApplyBlock ¶
func (blockExec *BlockExecutor) ApplyBlock( state State, blockID types.BlockID, block *types.Block, ) (State, error)
ApplyBlock validates the block against the state, executes it against the app, fires the relevant events, commits the app, and saves the new state and responses. It returns the new state. It's the only function that needs to be called from outside this package to process and commit an entire block. It takes a blockID to avoid recomputing the parts hash.
func (*BlockExecutor) Commit ¶
func (blockExec *BlockExecutor) Commit( state State, block *types.Block, abciResponse *abci.ResponseFinalizeBlock, ) (int64, error)
Commit locks the mempool, runs the ABCI Commit message, and updates the mempool. It returns the result of calling abci.Commit which is the height to retain (if any)). The application is expected to have persisted its state (if any) before returning from the ABCI Commit call. This is the only place where the application should persist its state. The Mempool must be locked during commit and update because state is typically reset on Commit and old txs must be replayed against committed state before new txs are run in the mempool, lest they be invalid.
func (*BlockExecutor) CreateProposalBlock ¶
func (blockExec *BlockExecutor) CreateProposalBlock( ctx context.Context, height int64, state State, lastExtCommit *types.ExtendedCommit, proposerAddr []byte, ) (*types.Block, error)
CreateProposalBlock calls state.MakeBlock with evidence from the evpool and txs from the mempool. The max bytes must be big enough to fit the commit. The block space is first allocated to outstanding evidence. The rest is given to txs, up to the max gas.
Contract: application will not return more bytes than are sent over the wire.
func (*BlockExecutor) ExtendVote ¶
func (*BlockExecutor) ProcessProposal ¶
func (*BlockExecutor) SetEventBus ¶
func (blockExec *BlockExecutor) SetEventBus(eventBus types.BlockEventPublisher)
SetEventBus - sets the event bus for publishing block related events. If not called, it defaults to types.NopEventBus.
func (*BlockExecutor) Store ¶
func (blockExec *BlockExecutor) Store() Store
func (*BlockExecutor) ValidateBlock ¶
func (blockExec *BlockExecutor) ValidateBlock(state State, block *types.Block) error
ValidateBlock validates the given block against the given state. If the block is invalid, it returns an error. Validation does not mutate state, but does require historical information from the stateDB, ie. to verify evidence from a validator at an old height.
func (*BlockExecutor) VerifyVoteExtension ¶
type BlockExecutorOption ¶
type BlockExecutorOption func(executor *BlockExecutor)
func BlockExecutorWithMetrics ¶
func BlockExecutorWithMetrics(metrics *Metrics) BlockExecutorOption
func BlockExecutorWithPruner ¶
func BlockExecutorWithPruner(pruner *Pruner) BlockExecutorOption
type BlockStore ¶
type BlockStore interface { Base() int64 Height() int64 Size() int64 LoadBaseMeta() *types.BlockMeta LoadBlockMeta(height int64) *types.BlockMeta LoadBlock(height int64) (*types.Block, *types.BlockMeta) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) SaveBlockWithExtendedCommit(block *types.Block, blockParts *types.PartSet, seenCommit *types.ExtendedCommit) PruneBlocks(height int64, state State) (uint64, int64, error) LoadBlockByHash(hash []byte) (*types.Block, *types.BlockMeta) LoadBlockMetaByHash(hash []byte) *types.BlockMeta LoadBlockPart(height int64, index int) *types.Part LoadBlockCommit(height int64) *types.Commit LoadSeenCommit(height int64) *types.Commit LoadBlockExtendedCommit(height int64) *types.ExtendedCommit DeleteLatestBlock() error Close() error }
BlockStore defines the interface used by the ConsensusState.
type BlocksPrunedInfo ¶
type BlocksPrunedInfo struct { FromHeight int64 // The height from which blocks were pruned (inclusive). ToHeight int64 // The height to which blocks were pruned (inclusive). }
BlocksPrunedInfo provides information about blocks pruned during a single run of the pruner.
type EmptyEvidencePool ¶
type EmptyEvidencePool struct{}
EmptyEvidencePool is an empty implementation of EvidencePool, useful for testing. It also complies to the consensus evidence pool interface
func (EmptyEvidencePool) AddEvidence ¶
func (EmptyEvidencePool) AddEvidence(types.Evidence) error
func (EmptyEvidencePool) CheckEvidence ¶
func (EmptyEvidencePool) CheckEvidence(types.EvidenceList) error
func (EmptyEvidencePool) PendingEvidence ¶
func (EmptyEvidencePool) PendingEvidence(int64) (ev []types.Evidence, size int64)
func (EmptyEvidencePool) ReportConflictingVotes ¶
func (EmptyEvidencePool) ReportConflictingVotes(*types.Vote, *types.Vote)
func (EmptyEvidencePool) Update ¶
func (EmptyEvidencePool) Update(State, types.EvidenceList)
type ErrAppBlockHeightTooHigh ¶
func (ErrAppBlockHeightTooHigh) Error ¶
func (e ErrAppBlockHeightTooHigh) Error() string
type ErrAppBlockHeightTooLow ¶
func (ErrAppBlockHeightTooLow) Error ¶
func (e ErrAppBlockHeightTooLow) Error() string
type ErrBlockHashMismatch ¶
func (ErrBlockHashMismatch) Error ¶
func (e ErrBlockHashMismatch) Error() string
type ErrCannotLoadState ¶
type ErrCannotLoadState struct {
Err error
}
func (ErrCannotLoadState) Error ¶
func (e ErrCannotLoadState) Error() string
func (ErrCannotLoadState) Unwrap ¶
func (e ErrCannotLoadState) Unwrap() error
type ErrFailedToPruneBlocks ¶
func (ErrFailedToPruneBlocks) Error ¶
func (e ErrFailedToPruneBlocks) Error() string
func (ErrFailedToPruneBlocks) Unwrap ¶
func (e ErrFailedToPruneBlocks) Unwrap() error
type ErrFailedToPruneStates ¶
func (ErrFailedToPruneStates) Error ¶
func (e ErrFailedToPruneStates) Error() string
func (ErrFailedToPruneStates) Unwrap ¶
func (e ErrFailedToPruneStates) Unwrap() error
type ErrInvalidBlock ¶
type ErrInvalidBlock error
type ErrLastStateMismatch ¶
func (ErrLastStateMismatch) Error ¶
func (e ErrLastStateMismatch) Error() string
type ErrNoABCIResponsesForHeight ¶
type ErrNoABCIResponsesForHeight struct {
Height int64
}
func (ErrNoABCIResponsesForHeight) Error ¶
func (e ErrNoABCIResponsesForHeight) Error() string
type ErrNoConsensusParamsForHeight ¶
type ErrNoConsensusParamsForHeight struct {
Height int64
}
func (ErrNoConsensusParamsForHeight) Error ¶
func (e ErrNoConsensusParamsForHeight) Error() string
type ErrNoValSetForHeight ¶
type ErrNoValSetForHeight struct {
Height int64
}
func (ErrNoValSetForHeight) Error ¶
func (e ErrNoValSetForHeight) Error() string
type ErrProxyAppConn ¶
type ErrProxyAppConn error
type ErrPrunerFailedToGetRetainHeight ¶
func (ErrPrunerFailedToGetRetainHeight) Error ¶
func (e ErrPrunerFailedToGetRetainHeight) Error() string
func (ErrPrunerFailedToGetRetainHeight) Unwrap ¶
func (e ErrPrunerFailedToGetRetainHeight) Unwrap() error
type ErrPrunerFailedToLoadState ¶
type ErrPrunerFailedToLoadState struct {
Err error
}
func (ErrPrunerFailedToLoadState) Error ¶
func (e ErrPrunerFailedToLoadState) Error() string
func (ErrPrunerFailedToLoadState) Unwrap ¶
func (e ErrPrunerFailedToLoadState) Unwrap() error
type ErrStateMismatch ¶
func (ErrStateMismatch) Error ¶
func (e ErrStateMismatch) Error() string
type ErrUnknownBlock ¶
type ErrUnknownBlock struct {
Height int64
}
func (ErrUnknownBlock) Error ¶
func (e ErrUnknownBlock) Error() string
type EvidencePool ¶
type EvidencePool interface { PendingEvidence(maxBytes int64) (ev []types.Evidence, size int64) AddEvidence(types.Evidence) error Update(State, types.EvidenceList) CheckEvidence(types.EvidenceList) error }
EvidencePool defines the EvidencePool interface used by State.
type Metrics ¶
type Metrics struct { // Time spent processing FinalizeBlock BlockProcessingTime metrics.Histogram `metrics_buckettype:"lin" metrics_bucketsizes:"1, 10, 10"` // ConsensusParamUpdates is the total number of times the application has // updated the consensus params since process start. //metrics:Number of consensus parameter updates returned by the application since process start. ConsensusParamUpdates metrics.Counter // ValidatorSetUpdates is the total number of times the application has // updated the validator set since process start. //metrics:Number of validator set updates returned by the application since process start. ValidatorSetUpdates metrics.Counter // PruningServiceBlockRetainHeight is the accepted block // retain height set by the data companion PruningServiceBlockRetainHeight metrics.Gauge // PruningServiceBlockResultsRetainHeight is the accepted block results // retain height set by the data companion PruningServiceBlockResultsRetainHeight metrics.Gauge // PruningServiceTxIndexerRetainHeight is the accepted transactions indices // retain height set by the data companion PruningServiceTxIndexerRetainHeight metrics.Gauge // PruningServiceBlockIndexerRetainHeight is the accepted blocks indices // retain height set by the data companion PruningServiceBlockIndexerRetainHeight metrics.Gauge // ApplicationBlockRetainHeight is the accepted block // retain height set by the application ApplicationBlockRetainHeight metrics.Gauge // BlockStoreBaseHeight shows the first height at which // a block is available BlockStoreBaseHeight metrics.Gauge // ABCIResultsBaseHeight shows the first height at which // abci results are available ABCIResultsBaseHeight metrics.Gauge // TxIndexerBaseHeight shows the first height at which // tx indices are available TxIndexerBaseHeight metrics.Gauge // BlockIndexerBaseHeight shows the first height at which // block indices are available BlockIndexerBaseHeight metrics.Gauge }
Metrics contains metrics exposed by this package.
func NopMetrics ¶
func NopMetrics() *Metrics
func PrometheusMetrics ¶
type NoopPrunerObserver ¶
type NoopPrunerObserver struct{}
NoopPrunerObserver does nothing.
func (NoopPrunerObserver) PrunerPrunedABCIRes ¶
func (NoopPrunerObserver) PrunerPrunedABCIRes(*ABCIResponsesPrunedInfo)
PrunerPrunedABCIRes implements PrunerObserver.
func (NoopPrunerObserver) PrunerPrunedBlocks ¶
func (NoopPrunerObserver) PrunerPrunedBlocks(*BlocksPrunedInfo)
PrunerPrunedBlocks implements PrunerObserver.
func (NoopPrunerObserver) PrunerStarted ¶
func (NoopPrunerObserver) PrunerStarted(time.Duration)
PrunerStarted implements PrunerObserver.
type Pruner ¶
type Pruner struct { service.BaseService // contains filtered or unexported fields }
Pruner is a service that reads the retain heights for blocks, state and ABCI results from the database and prunes the corresponding data based on the minimum retain height set. The service sleeps between each run based on the configured pruner interval, and re-evaluates the retain height.
func NewPruner ¶
func NewPruner( stateStore Store, bs BlockStore, blockIndexer indexer.BlockIndexer, txIndexer txindex.TxIndexer, logger log.Logger, options ...PrunerOption, ) *Pruner
NewPruner creates a service that controls background pruning of node data.
Assumes that the initial application and data companion retain heights have already been configured in the state store.
func (*Pruner) GetABCIResRetainHeight ¶
GetABCIResRetainHeight is a convenience method for accessing the GetABCIResRetainHeight method of the underlying state store.
func (*Pruner) GetApplicationRetainHeight ¶
GetApplicationRetainHeight is a convenience method for accessing the GetApplicationRetainHeight method of the underlying state store.
func (*Pruner) GetBlockIndexerRetainHeight ¶
GetBlockIndexerRetainHeight is a convenience method for accessing the GetBlockIndexerRetainHeight method of the underlying state store.
func (*Pruner) GetCompanionBlockRetainHeight ¶
GetCompanionBlockRetainHeight is a convenience method for accessing the GetCompanionBlockRetainHeight method of the underlying state store.
func (*Pruner) GetTxIndexerRetainHeight ¶
GetTxIndexerRetainHeight is a convenience method for accessing the GetTxIndexerRetainHeight method of the underlying indexer
func (*Pruner) SetABCIResRetainHeight ¶
SetABCIResRetainHeight sets the retain height for ABCI responses.
If the application has set the DiscardABCIResponses flag to true, nothing will be pruned.
func (*Pruner) SetApplicationBlockRetainHeight ¶
SetApplicationBlockRetainHeight sets the application block retain height with some basic checks on the requested height.
If a higher retain height is already set, we cannot accept the requested height because the blocks might have been pruned.
func (*Pruner) SetBlockIndexerRetainHeight ¶
func (*Pruner) SetCompanionBlockRetainHeight ¶
SetCompanionBlockRetainHeight sets the application block retain height with some basic checks on the requested height.
If a higher retain height is already set, we cannot accept the requested height because the blocks might have been pruned.
func (*Pruner) SetObserver ¶
func (p *Pruner) SetObserver(obs PrunerObserver)
func (*Pruner) SetTxIndexerRetainHeight ¶
type PrunerObserver ¶
type PrunerObserver interface { // PrunerStarted is called when the pruner's background pruning routine has // been started. PrunerStarted(interval time.Duration) // PrunerPrunedABCIRes is called after each successful pruning of ABCI results. PrunerPrunedABCIRes(prunedInfo *ABCIResponsesPrunedInfo) // PrunerPrunedBlocks is called after each successful pruning of blocks. PrunerPrunedBlocks(prunedInfo *BlocksPrunedInfo) }
PrunerObserver allows an external observer to be notified of certain events generated by the Pruner.
type PrunerOption ¶
type PrunerOption func(*prunerConfig)
func WithPrunerCompanionEnabled ¶
func WithPrunerCompanionEnabled() PrunerOption
WithPrunerCompanionEnabled indicates to the pruner that it must respect the retain heights set by the data companion. By default, if this option is not supplied, the pruner will ignore any retain heights set by the data companion.
func WithPrunerInterval ¶
func WithPrunerInterval(t time.Duration) PrunerOption
WithPrunerInterval allows control over the interval between each run of the pruner.
func WithPrunerMetrics ¶
func WithPrunerMetrics(metrics *Metrics) PrunerOption
func WithPrunerObserver ¶
func WithPrunerObserver(obs PrunerObserver) PrunerOption
type State ¶
type State struct { Version cmtstate.Version // immutable ChainID string InitialHeight int64 // should be 1, not 0, when starting from height 1 // LastBlockHeight=0 at genesis (ie. block(H=0) does not exist) LastBlockHeight int64 LastBlockID types.BlockID LastBlockTime time.Time // LastValidators is used to validate block.LastCommit. // Validators are persisted to the database separately every time they change, // so we can query for historical validator sets. // Note that if s.LastBlockHeight causes a valset change, // we set s.LastHeightValidatorsChanged = s.LastBlockHeight + 1 + 1 // Extra +1 due to nextValSet delay. NextValidators *types.ValidatorSet Validators *types.ValidatorSet LastValidators *types.ValidatorSet LastHeightValidatorsChanged int64 // Consensus parameters used for validating blocks. // Changes returned by FinalizeBlock and updated after Commit. ConsensusParams types.ConsensusParams LastHeightConsensusParamsChanged int64 // Merkle root of the results from executing prev block LastResultsHash []byte // the latest AppHash we've received from calling abci.Commit() AppHash []byte }
State is a short description of the latest committed block of the consensus protocol. It keeps all information necessary to validate new blocks, including the last validator set and the consensus params. All fields are exposed so the struct can be easily serialized, but none of them should be mutated directly. Instead, use state.Copy() or state.NextState(...). NOTE: not goroutine-safe.
func MakeGenesisState ¶
func MakeGenesisState(genDoc *types.GenesisDoc) (State, error)
MakeGenesisState creates state from types.GenesisDoc.
func MakeGenesisStateFromFile ¶
MakeGenesisStateFromFile reads and unmarshals state from the given file.
Used during replay and in tests.
func (State) Bytes ¶
Bytes serializes the State using protobuf. It panics if either casting to protobuf or serialization fails.
func (State) MakeBlock ¶
func (state State) MakeBlock( height int64, txs []types.Tx, lastCommit *types.Commit, evidence []types.Evidence, proposerAddress []byte, ) *types.Block
MakeBlock builds a block from the current state with the given txs, commit, and evidence. Note it also takes a proposerAddress because the state does not track rounds, and hence does not know the correct proposer. TODO: fix this!
type Store ¶
type Store interface { // LoadFromDBOrGenesisFile loads the most recent state. // If the chain is new it will use the genesis file from the provided genesis file path as the current state. LoadFromDBOrGenesisFile(string) (State, error) // LoadFromDBOrGenesisDoc loads the most recent state. // If the chain is new it will use the genesis doc as the current state. LoadFromDBOrGenesisDoc(*types.GenesisDoc) (State, error) // Load loads the current state of the blockchain Load() (State, error) // LoadValidators loads the validator set at a given height LoadValidators(height int64) (*types.ValidatorSet, error) // LoadFinalizeBlockResponse loads the abciResponse for a given height LoadFinalizeBlockResponse(height int64) (*abci.ResponseFinalizeBlock, error) // LoadLastABCIResponse loads the last abciResponse for a given height LoadLastFinalizeBlockResponse(height int64) (*abci.ResponseFinalizeBlock, error) // LoadConsensusParams loads the consensus params for a given height LoadConsensusParams(height int64) (types.ConsensusParams, error) // Save overwrites the previous state with the updated one Save(state State) error // SaveFinalizeBlockResponse saves ABCIResponses for a given height SaveFinalizeBlockResponse(height int64, res *abci.ResponseFinalizeBlock) error // Bootstrap is used for bootstrapping state when not starting from a initial height. Bootstrap(state State) error // PruneStates takes the height from which to start pruning and which height stop at PruneStates(fromHeight, toHeight, evidenceThresholdHeight int64) error // PruneABCIResponses will prune all ABCI responses below the given height. PruneABCIResponses(targetRetainHeight int64) (int64, int64, error) // SaveApplicationRetainHeight persists the application retain height from the application SaveApplicationRetainHeight(height int64) error // GetApplicationRetainHeight returns the retain height set by the application GetApplicationRetainHeight() (int64, error) // SaveCompanionBlockRetainHeight saves the retain height set by the data companion SaveCompanionBlockRetainHeight(height int64) error // GetCompanionBlockRetainHeight returns the retain height set by the data companion GetCompanionBlockRetainHeight() (int64, error) // SaveABCIResRetainHeight persists the retain height for ABCI results set by the data companion SaveABCIResRetainHeight(height int64) error // GetABCIResRetainHeight returns the last saved retain height for ABCI results set by the data companion GetABCIResRetainHeight() (int64, error) // Saves the height at which the store is bootstrapped after out of band statesync SetOfflineStateSyncHeight(height int64) error // Gets the height at which the store is bootstrapped after out of band statesync GetOfflineStateSyncHeight() (int64, error) // Close closes the connection with the database Close() error }
Store defines the state store interface
It is used to retrieve current state and save and load ABCI responses, validators and consensus parameters
type StoreOptions ¶
type StoreOptions struct { // DiscardABCIResponses determines whether or not the store // retains all ABCIResponses. If DiscardABCIResponses is enabled, // the store will maintain only the response object from the latest // height. DiscardABCIResponses bool }