tmconsensus

package
v0.0.0-...-8215cf4 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Jan 29, 2025 License: Apache-2.0 Imports: 15 Imported by: 4

Documentation

Overview

Package tmconsensus contains low-level consensus primitives.

Index

Constants

This section is empty.

Variables

View Source
var ErrProposedBlockChoiceNotReady = errors.New("not ready to choose proposed block")

ErrProposedBlockChoiceNotReady is a sentinel error the ConsensusStrategy must return from its ConsiderProposedBlocks method, if it is not ready to choose a proposed block.

Functions

func ByzantineMajority

func ByzantineMajority(n uint64) uint64

ByzantineMajority returns the minimum value to exceed 2/3 of n. Use should always involve >= comparison, not >. For example, 2/3 of 12 is 8, so ByzantineMajority(12) = 9. Similarly, 2/3 of 10 is 6+2/3, so ByzantineMajority(10) = 7.

ByzantineMajority(0) panics.

func ByzantineMinority

func ByzantineMinority(n uint64) uint64

ByzantineMinority returns the minimum value to reach 1/3 of n. Use should always involve >= comparison, not >.

For votes that are simply in favor or against a binary decision, reaching 1/3 means it is possible to reach a majority decision; unless both the for and against votes have reached the minority, in which case it is impossible for one vote to reach the majority.

ByzantineMinority(0) panics.

func CanTrustValidators

func CanTrustValidators(vs []Validator, pubKeys []gcrypto.PubKey) bool

CanTrustValidators reports whether the validator set vs contains at least 1/3 voting power represented by the passed-in set of trusted public keys.

func PrecommitSignBytes

func PrecommitSignBytes(vt VoteTarget, s SignatureScheme) ([]byte, error)

PrecommitSignBytes returns a new byte slice containing the precommit sign bytes for v, as defined by s.

Use this function for one-off calls, but prefer to maintain a local bytes.Buffer in loops involving signatures.

func PrevoteSignBytes

func PrevoteSignBytes(vt VoteTarget, s SignatureScheme) ([]byte, error)

PrevoteSignBytes returns a new byte slice containing the prevote sign bytes for v, as defined by s.

Use this function for one-off calls, but prefer to maintain a local bytes.Buffer in loops involving signatures.

func ProposalSignBytes

func ProposalSignBytes(h Header, round uint32, pbAnnotations Annotations, s SignatureScheme) ([]byte, error)

ProposalSignBytes returns a new byte slice containing the proposal sign bytes for h, as defined by s.

Use this function for one-off calls, but prefer to maintain a local bytes.Buffer in loops involving signatures.

func SortValidators

func SortValidators(vs []Validator)

SortValidators sorts vs in-place, by power descending, and then by public key ascending.

func ValidatorSlicesEqual

func ValidatorSlicesEqual(vs1, vs2 []Validator) bool

ValidatorSlicesEqual reports whether the slices vs1 and vs2 are equivalent.

func ValidatorsToPubKeys

func ValidatorsToPubKeys(vs []Validator) []gcrypto.PubKey

ValidatorsToPubKeys returns a slice of just the public keys of vs.

func ValidatorsToVotePowers

func ValidatorsToVotePowers(vs []Validator) []uint64

ValidatorsToVotePowers returns a slice of just the vote powers of vs.

Types

type AcceptAllValidFeedbackMapper

type AcceptAllValidFeedbackMapper struct {
	Handler FineGrainedConsensusHandler
}

AcceptAllValidFeedbackMapper converts a FineGrainedConsensusHandler to a ConsensusHandler, accepting any valid input, even if the input was already known.

func (AcceptAllValidFeedbackMapper) HandlePrecommitProofs

func (AcceptAllValidFeedbackMapper) HandlePrevoteProofs

func (AcceptAllValidFeedbackMapper) HandleProposedHeader

func (m AcceptAllValidFeedbackMapper) HandleProposedHeader(
	ctx context.Context, ph ProposedHeader,
) gexchange.Feedback

type Annotations

type Annotations struct {
	User, Driver []byte
}

Annotations are arbitrary data to associate with a Header or ProposedHeader.

The Driver annotations are set by the driver (that is, the low-level code providing the ConsensusStrategy). The User annotations are provided by the higher-level application.

type AppDataHashMismatchError

type AppDataHashMismatchError struct {
	Want, Got []byte
}

AppDataHashMismatchError indicates an input that has the wrong app data hash.

func (AppDataHashMismatchError) Error

func (e AppDataHashMismatchError) Error() string

type AppStateHashMismatchError

type AppStateHashMismatchError struct {
	Want, Got []byte
}

AppStateHashMismatchError indicates an input that has the wrong app state hash.

func (AppStateHashMismatchError) Error

type CommitProof

type CommitProof struct {
	// Necessary to verify signature content.
	Round uint32

	// The hash of the ordered collection of public keys
	// of the validators at the height where the commit proof occurred.
	// Derived through a [HashScheme] method.
	PubKeyHash string

	// Keyed by block hash, or an empty string for nil block.
	Proofs map[string][]gcrypto.SparseSignature
}

CommitProof is the commit proof for a block.

func (CommitProof) Clone

func (p CommitProof) Clone() CommitProof

Clone returns a new copy of CommitProof with identical values but without any references to p.

type CommittedHeader

type CommittedHeader struct {
	Header Header
	Proof  CommitProof
}

CommittedHeader is a header and the proof that it was committed.

type ConsensusHandler

type ConsensusHandler interface {
	HandleProposedHeader(context.Context, ProposedHeader) gexchange.Feedback
	HandlePrevoteProofs(context.Context, PrevoteSparseProof) gexchange.Feedback
	HandlePrecommitProofs(context.Context, PrecommitSparseProof) gexchange.Feedback
}

ConsensusHandler is the interface to handle the set of consensus messages.

In production this will be a [tmgossip.Strategy] value.

type ConsensusStrategy

type ConsensusStrategy interface {
	// The state machine calls this method synchronously when it begins a round.
	// It is possible that the state machine is lagging behind the network,
	// in which case there may be existing proposed blocks or votes in rv.
	//
	// The state machine does not call EnterRound if it is catching up
	// on blocks that are already committed to the chain.
	//
	// If the application is going to propose a block for this round,
	// it must publish the proposal information to the proposalOut channel;
	// the state machine will compose that information into a proposed block.
	EnterRound(ctx context.Context, rv RoundView, proposalOut chan<- Proposal) error

	// ConsiderProposedBlocks is called when new proposed headers arrive,
	// or when new block data has arrived,
	// so long as the proposed block timeout has not yet elapsed.
	//
	// While most of the rest of the tm package refers to proposed headers,
	// this method includes the words "proposed blocks" because it is assumed that
	// the consensus strategy cannot decide whether to vote for the header
	// until it has the actual block data to go with it.
	//
	// The reason argument is a hint to the consensus strategy
	// about which information is new in this call,
	// compared to the previous call.
	//
	// If the returned error is nil, the returned string is assumed to be the block hash to prevote.
	// The empty string indicates a prevote for nil.
	//
	// If the consensus strategy wants to wait longer before making a selection,
	// it must return [ErrProposedBlockChoiceNotReady].
	// Any other error is fatal.
	ConsiderProposedBlocks(
		ctx context.Context,
		phs []ProposedHeader,
		reason ConsiderProposedBlocksReason,
	) (string, error)

	// ChooseProposedBlock is called when the state machine's proposal delay has elapsed.
	// The phs slice may be empty.
	//
	// ChooseProposedBlock must return the hash of the block to vote for.
	// Under certain circumstances (like Proof of Lock),
	// the returned hash may not be present in the slice of proposed blocks.
	//
	// The state machine calls this in a background goroutine,
	// so the method may block as long as necessary.
	// Nonetheless, the method must respect context cancellation.
	ChooseProposedBlock(ctx context.Context, phs []ProposedHeader) (string, error)

	// DecidePrecommit is called when prevoting is finished
	// and the state machine needs to submit a precommit.
	//
	// The returned string value is the block hash to precommit.
	// The empty string indicates a precommit for nil.
	// A returned error is fatal.
	//
	// NOTE: this method will likely change with vote extensions.
	DecidePrecommit(ctx context.Context, vs VoteSummary) (string, error)
}

ConsensusStrategy determines how a state machine proposes blocks and what blocks to prevote or precommit.

type ConsiderProposedBlocksReason

type ConsiderProposedBlocksReason struct {
	// The hashes of the new blocks since the previous call to ConsiderProposedBlocks.
	NewProposedBlocks []string

	// Any data IDs that have been marked as updated
	// since the previous call to ConsensusStrategy.
	// The data IDs are provided in an arbitrary order.
	UpdatedBlockDataIDs []string

	// Indicates whether >2/3 of voting power is present,
	// but does not necessarily indicate that the voting power
	// is for a single block or nil; voting may be split.
	MajorityVotingPowerPresent bool
}

ConsiderProposedBlocksReason is an argument in [ConsensusStrategy.ConsiderProposedBlocks]. It is a hint to the ConsensusStrategy about anything new to check during this call, compared to the previous call. The state of what has been sent previously is already available in the engine, so the consensus strategy does not need to track this state on its own.

type DoublePrecommitError

type DoublePrecommitError struct {
	PubKeys []gcrypto.PubKey
}

DoublePrecommitError indicates a precommit message contained both an active and a nil precommit from one or more validators.

func (DoublePrecommitError) Error

func (e DoublePrecommitError) Error() string

type DoublePrevoteError

type DoublePrevoteError struct {
	PubKeys []gcrypto.PubKey
}

DoublePrevoteError indicates a prevote message contained both an active and a nil prevote from one or more validators.

func (DoublePrevoteError) Error

func (e DoublePrevoteError) Error() string

type DoubleVoteByIndexError

type DoubleVoteByIndexError struct {
	ValidatorIdxs []int
}

DoublePrevoteByIndexError indicates a prevote contained both an active and a nil prevote from one or more validators.

DoublePrevoteError should be preferred over this type, but in some circumstances the validators are not available, so all we can do is use their indices.

func (DoubleVoteByIndexError) Error

func (e DoubleVoteByIndexError) Error() string

func (DoubleVoteByIndexError) ToDoublePrecommitError

func (e DoubleVoteByIndexError) ToDoublePrecommitError(vals []Validator) DoublePrecommitError

func (DoubleVoteByIndexError) ToDoublePrevoteError

func (e DoubleVoteByIndexError) ToDoublePrevoteError(vals []Validator) DoublePrevoteError

type DropDuplicateFeedbackMapper

type DropDuplicateFeedbackMapper struct {
	Handler FineGrainedConsensusHandler
}

DropDuplicateFeedbackMapper is a [Handler] that wraps a FineGrainedConsensusHandler that ignores proposed block messages if we already have the proposed block and ignores vote messages if they do not increase existing vote knowledge.

func (DropDuplicateFeedbackMapper) HandlePrecommitProofs

func (DropDuplicateFeedbackMapper) HandlePrevoteProofs

func (DropDuplicateFeedbackMapper) HandleProposedHeader

func (m DropDuplicateFeedbackMapper) HandleProposedHeader(
	ctx context.Context, ph ProposedHeader,
) gexchange.Feedback

type EmptyVoteError

type EmptyVoteError struct{}

func (EmptyVoteError) Error

func (EmptyVoteError) Error() string

type ExternalGenesis

type ExternalGenesis struct {
	ChainID string

	// Height to use for the first proposed block.
	InitialHeight uint64

	// Initial application state as specified by the external genesis description.
	// Format is determined by the application; it is opaque to the consensus engine.
	//
	// This is a Reader, not a byte slice, so that the consensus engine
	// isn't forced to load the entire state into memory.
	InitialAppState io.Reader

	// Validators according to the consensus engine's view.
	// Can be overridden in the [tmdriver.InitChainResponse].
	GenesisValidatorSet ValidatorSet
}

ExternalGenesis is a view of the externally defined genesis data, sent to the app as part of [tmdriver.InitChainRequest].

type FineGrainedConsensusHandler

type FineGrainedConsensusHandler interface {
	HandleProposedHeader(context.Context, ProposedHeader) HandleProposedHeaderResult
	HandlePrevoteProofs(context.Context, PrevoteSparseProof) HandleVoteProofsResult
	HandlePrecommitProofs(context.Context, PrecommitSparseProof) HandleVoteProofsResult
}

FineGrainedConsensusHandler is the preferred interface (over ConsensusHandler) for handling incoming p2p consensus messages.

Use one of the FeedbackMapper types to convert a FineGrainedConsensusHandler into a ConsensusHandler.

type Genesis

type Genesis struct {
	ChainID string

	// The height of the first block to be proposed.
	InitialHeight uint64

	// This determines PrevAppStateHash for the first proposed block.
	CurrentAppStateHash []byte

	// The set of validators to propose and vote on the first block.
	ValidatorSet ValidatorSet
}

Genesis is the value used to initialize a consensus store.

In normal use this is derived from InitChain, but in tests is is constructed by hand, usually through a [tmconsensustest.StandardFixture].

func (Genesis) Header

func (g Genesis) Header(hs HashScheme) (Header, error)

Header returns the genesis Header corresponding to g. It will have only its Height, NextValidators, and Hash set. If there is an error retrieving the hash, that error is returned.

type HandleProposedHeaderResult

type HandleProposedHeaderResult uint8

HandleProposedHeaderResult is a set of constants to be returned from a FineGrainedConsensusHandler's HandleProposedHeader method.

const (

	// Everything checked out -- this was a new proposed block added to our store.
	HandleProposedHeaderAccepted HandleProposedHeaderResult

	// We already stored a copy of this proposed block.
	HandleProposedHeaderAlreadyStored

	// The signer of the proposed block did not match a validator in the current round.
	HandleProposedHeaderSignerUnrecognized

	// Our calculation of the block hash was different from what the block reported.
	HandleProposedHeaderBadBlockHash

	// Signature verification on the proposed header failed.
	HandleProposedHeaderBadSignature

	// ProposerPubKey field was not populated.
	HandleProposedHeaderMissingProposerPubKey

	// Something was wrong with the header's PrevCommitProof.
	// It could be a validator pub key mismatch,
	// an invalid signature in the proof,
	// or an incorrect amount of votes (e.g. less than majority for the committing block).
	HandleProposedHeaderBadPrevCommitProofPubKeyHash
	HandleProposedHeaderBadPrevCommitProofSignature
	HandleProposedHeaderBadPrevCommitProofDoubleSigned
	HandleProposedHeaderBadPrevCommitVoteCount

	// Proposed block had older height or round than our current view of the world.
	HandleProposedHeaderRoundTooOld

	// Proposed block is beyond our NextHeight and/or NextRound handlers.
	HandleProposedHeaderRoundTooFarInFuture

	// Internal error not necessarily correlated with the actual proposed block.
	HandleProposedHeaderInternalError
)

func (HandleProposedHeaderResult) String

type HandleVoteProofsResult

type HandleVoteProofsResult uint8

HandleVoteProofsResult is a set of constants to be returned from a FineGrainedConsensusHandler's HandlePrevoteProofs and HandlePrecommitProofs methods.

const (

	// Proofs were added to the store.
	HandleVoteProofsAccepted HandleVoteProofsResult

	// We already had all the signatures in the given proof.
	HandleVoteProofsNoNewSignatures

	// There were no proofs in the message.
	// (This should only happen on messages from a misbehaving peer.)
	HandleVoteProofsEmpty

	// The public key hash did not match what we expected for the given height and round.
	HandleVoteProofsBadPubKeyHash

	// Votes had older height or round than our current view of the world.
	HandleVoteProofsRoundTooOld

	// Vote is beyond our NextHeight and/or NextRound handlers.
	HandleVoteProofsTooFarInFuture

	// Internal error not necessarily correlated with the actual prevote proof.
	HandleVoteProofsInternalError
)

func (HandleVoteProofsResult) String

func (i HandleVoteProofsResult) String() string

type HashAlreadyExistsError

type HashAlreadyExistsError struct {
	Hash []byte
}

func (HashAlreadyExistsError) Error

func (e HashAlreadyExistsError) Error() string

type HashScheme

type HashScheme interface {
	// Block calculates and returns the block hash given a header,
	// without consulting or modifying existing Hash field on the header.
	Block(Header) ([]byte, error)

	// PubKeys calculates and returns the hash of the ordered set of public keys.
	PubKeys([]gcrypto.PubKey) ([]byte, error)

	// VotePowers calculates and returns the hash of the ordered set of voting power,
	// mapped 1:1 with the ordered set of public keys.
	VotePowers([]uint64) ([]byte, error)
}

HashScheme defines ways to determine various hashes in a consensus engine.

type HashUnknownError

type HashUnknownError struct {
	Got []byte
}

HashUnknownError indicates a reference to an unknown or unrecognized hash.

func (HashUnknownError) Error

func (e HashUnknownError) Error() string
type Header struct {
	// Determined based on all the other fields.
	// Derived through a [HashScheme] method.
	Hash []byte

	// Hash of the previous block.
	PrevBlockHash []byte

	// Height of this block.
	Height uint64

	// PrevCommitProof is the proof for the previous committed block,
	// where there may be precommits for other blocks
	// besides the committed one and nil.
	//
	// The consensus engine determines the format of this proof.
	// Non-aggregating signatures will typically be a discrete set of signatures,
	// but aggregating signatures will usually undergo a final aggregation,
	// which is incompatible with the form used for consensus gossip.
	PrevCommitProof CommitProof

	// The validators for this block and the next block.
	ValidatorSet, NextValidatorSet ValidatorSet

	// ID of the data for this block.
	// The user-defined consensus strategy provides this ID,
	// and the driver is responsible for retrieving the raw data belonging to the ID.
	//
	// The ID is typically, but not necessarily,
	// a cryptographic hash of the application data for the block.
	DataID []byte

	// The hash of the app state as a result of executing the previous block.
	// Deriving this hash is an application-level concern.
	PrevAppStateHash []byte

	// Arbitrary data to associate with the block.
	// Unlike the annotations on a proposed block, these values are persisted to chain.
	// The values must be respected in the block's hash.
	//
	// Low-level driver code may set the Annotations.Driver field,
	// while on-chain code may set the Annotations.User field.
	//
	// One example of a use case for a block annotation from the driver
	// could be to include a timestamp with the block.
	// One contrived example for the user annotation could be
	// including the version of the application,
	// in order to reject blocks that don't match the version.
	Annotations Annotations
}

Header is the logical representation of a block header. The header may go through transformations, such as storing only hashes of validator sets rather than the longhand raw validator data, before writing to disk or sending across the network.

type HeightMismatchError

type HeightMismatchError struct {
	Want, Got uint64
}

HeightMismatchError indicates an input that has the wrong height.

func (HeightMismatchError) Error

func (e HeightMismatchError) Error() string

type HeightUnknownError

type HeightUnknownError struct {
	Want uint64
}

HeightUnknownError indicates a request for a height that is not known.

func (HeightUnknownError) Error

func (e HeightUnknownError) Error() string

type PassthroughSigner

type PassthroughSigner struct {
	Signer          gcrypto.Signer
	SignatureScheme SignatureScheme
}

PassthroughSigner is a Signer that directly generates signatures with the given signer and scheme.

func (PassthroughSigner) Precommit

func (s PassthroughSigner) Precommit(ctx context.Context, vt VoteTarget) (
	signContent, signature []byte, err error,
)

func (PassthroughSigner) Prevote

func (s PassthroughSigner) Prevote(ctx context.Context, vt VoteTarget) (
	signContent, signature []byte, err error,
)

func (PassthroughSigner) PubKey

func (s PassthroughSigner) PubKey() gcrypto.PubKey

func (PassthroughSigner) SignProposedHeader

func (s PassthroughSigner) SignProposedHeader(ctx context.Context, ph *ProposedHeader) error

type PrecommitProof

type PrecommitProof struct {
	Height uint64
	Round  uint32

	Proofs map[string]gcrypto.CommonMessageSignatureProof
}

func (PrecommitProof) AsSparse

func (p PrecommitProof) AsSparse() (PrecommitSparseProof, error)

type PrecommitSparseProof

type PrecommitSparseProof struct {
	Height uint64
	Round  uint32

	PubKeyHash string

	Proofs map[string][]gcrypto.SparseSignature
}

PrecommitSparseProof is the representation of sparse proofs for precommits arriving across the network. It is currently identical to PrevoteSparseProof, but that may change with vote extensions.

func PrecommitSparseProofFromFullProof

func PrecommitSparseProofFromFullProof(height uint64, round uint32, fullProof map[string]gcrypto.CommonMessageSignatureProof) (PrecommitSparseProof, error)

func (PrecommitSparseProof) Clone

func (PrecommitSparseProof) ToFull

func (p PrecommitSparseProof) ToFull(
	cmsps gcrypto.CommonMessageSignatureProofScheme,
	sigScheme SignatureScheme,
	hashScheme HashScheme,
	trustedVals []Validator,
) (PrecommitProof, error)

type PreviousHashMismatchError

type PreviousHashMismatchError struct {
	Want, Got []byte
}

PreviousHashMismatchError indicates an input that has the wrong previous block hash.

func (PreviousHashMismatchError) Error

type PrevoteProof

type PrevoteProof struct {
	Height uint64
	Round  uint32

	Proofs map[string]gcrypto.CommonMessageSignatureProof
}

func (PrevoteProof) AsSparse

func (p PrevoteProof) AsSparse() (PrevoteSparseProof, error)

type PrevoteSparseProof

type PrevoteSparseProof struct {
	Height uint64
	Round  uint32

	PubKeyHash string

	Proofs map[string][]gcrypto.SparseSignature
}

PrevoteSparseProof is the representation of sparse proofs for prevotes arriving across the network.

func PrevoteSparseProofFromFullProof

func PrevoteSparseProofFromFullProof(height uint64, round uint32, fullProof map[string]gcrypto.CommonMessageSignatureProof) (PrevoteSparseProof, error)

func (PrevoteSparseProof) Clone

func (PrevoteSparseProof) ToFull

func (p PrevoteSparseProof) ToFull(
	cmsps gcrypto.CommonMessageSignatureProofScheme,
	sigScheme SignatureScheme,
	hashScheme HashScheme,
	trustedVals []Validator,
) (PrevoteProof, error)

type Proposal

type Proposal struct {
	// The ID of the data inside the block.
	// This value will be used to set [Block.DataID].
	DataID string // TODO: this should switch back to []byte.

	// Respectively sets [ProposedBlock.Annotations] and [Block.Annotations].
	ProposalAnnotations, BlockAnnotations Annotations
}

Proposal is the data an application needs to provide, for the engine to compose a ProposedHeader.

type ProposedHeader

type ProposedHeader struct {
	// The header of the block to consider committing.
	Header Header

	// The round in which this block was proposed.
	Round uint32

	// The public key of the proposer.
	// Used to verify the signature.
	ProposerPubKey gcrypto.PubKey

	// Arbitrary data to associate with the proposed block.
	// The annotations are considered when producing the proposed block's signature,
	// but they are otherwise not persisted to chain.
	// (Of course, off-chain utilities like indexers may persist the data off-chain.)
	//
	// Low-level driver code may set the Annotations.Driver field,
	// while on-chain code may set the Annotations.User field.
	//
	// The [ConsensusStrategy] may directly set either of those fields
	// when it provides a proposed block to the engine.
	//
	// One example of a use case for a proposed block annotation
	// would be for the engine to include its network address or p2p ID
	// as proof that the particular validator is associated with a particular peer.
	// There would be no need for that to be stored on chain,
	// but it would be potentially relevant to other live validators on the network.
	Annotations Annotations

	// Signature of the proposer.
	// The signing content is determined by the engine's [SignatureScheme].
	Signature []byte
}

ProposedHeader is the data sent by a proposer at the beginning of a round. This is the logical representation within the engine, not necessarily an exact representation of the data sent across the network.

type RoundUnknownError

type RoundUnknownError struct {
	WantHeight uint64
	WantRound  uint32
}

RoundUnknownError indicates a request for a height and round with no corresponding record.

func (RoundUnknownError) Error

func (e RoundUnknownError) Error() string

type RoundView

type RoundView struct {
	Height uint64
	Round  uint32

	ValidatorSet ValidatorSet

	PrevCommitProof CommitProof

	ProposedHeaders []ProposedHeader

	PrevoteProofs, PrecommitProofs map[string]gcrypto.CommonMessageSignatureProof

	VoteSummary VoteSummary
}

RoundView is the engine's observed view of the state of a particular round.

The RoundView may be on a later height and round, or with different validators, compared to the local state machine.

func (*RoundView) Clone

func (v *RoundView) Clone() RoundView

Clone returns a RoundView, with values identical to v, and underlying slices and maps copied from v.

func (RoundView) LogValue

func (v RoundView) LogValue() slog.Value

LogValue converts v into an slog.Value. This provides a highly detailed log, so it is only appropriate for infrequent log events, such as responding to a watchdog termination signal.

func (*RoundView) Reset

func (v *RoundView) Reset()

Reset zeros out all the fields of the RoundView, retaining any allocated capacity for its slices and maps. This is helpful for reusing RoundView values to avoid unnecessary memory allocations.

func (*RoundView) ResetForSameHeight

func (v *RoundView) ResetForSameHeight()

ResetForSameHeight clears the round, proposed blocks, and vote information on v. It does not modify the height, validators, or validator hashes.

This is intended to be used when it is known that a view is going to be reused in the same height, where it should be safe to keep the validator slice and validator hashes.

type SignatureScheme

type SignatureScheme interface {
	WriteProposalSigningContent(w io.Writer, h Header, round uint32, pbAnnotations Annotations) (int, error)

	WritePrevoteSigningContent(io.Writer, VoteTarget) (int, error)

	WritePrecommitSigningContent(io.Writer, VoteTarget) (int, error)
}

SignatureScheme determines the content to be signed, for consensus messages.

Rather than returning a slice of bytes, its methods write to an io.Writer. This enables the caller to potentially reduce allocations, for example by reusing a bytes.Buffer:

var scheme SignatureScheme = getSignatureScheme()
var buf bytes.Buffer
for i, pv := range prevotes {
  buf.Reset()
  _, err := scheme.WritePrevoteSigningContent(&buf, pv)
  if err != nil {
    panic(err)
  }
  checkSignature(buf.Bytes(), pv.Signature, publicKeys[i])
}

type Signer

type Signer interface {
	// Prevote and Precommit return the byte slices containing
	// the signing content and signature for a prevote or precommit
	// for a block or nil, as specified by the VoteTarget.
	//
	// The signing content is necessary as part of the return signature,
	// in order to reduce duplicative work elsewhere internal to the consensus engine.
	Prevote(ctx context.Context, vt VoteTarget) (signContent, signature []byte, err error)
	Precommit(ctx context.Context, vt VoteTarget) (signContent, signature []byte, err error)

	// SignProposedHeader sets the Signature field on the proposed block header.
	// All other fields on ph must already be populated.
	SignProposedHeader(ctx context.Context, ph *ProposedHeader) error

	// PubKey returns the public key of the signer.
	PubKey() gcrypto.PubKey
}

Signer is the tm-aware signer. While gcrypto.Signer offers a low level interface to sign raw bytes, this Signer is aware of tmconsensus types, in case the underlying signer needs any additional context on what exactly is being signed.

type SparseSignatureCollection

type SparseSignatureCollection struct {
	// There is exactly one public key hash
	// the keys and signatures in the entire collection.
	// Consumers must assume this slice is shared across many goroutines,
	// and therefore must never modify the slice.
	PubKeyHash []byte

	// Mapping of block hash to sparse signatures.
	// The empty string represents a vote for nil.
	BlockSignatures map[string][]gcrypto.SparseSignature
}

SparseSignatureCollection is a tm-specific, network- and store-optimized collection of gcrypto.SparseSignature.

A gcrypto.SparseSignatureProof is one public key hash with a collection of signatures, without a clear relationship to any particular block hash. Using a map[string]gcrypto.SparseSignatureProof requires the caller to verify every public key hash in every proof. So instead, with the SparseSignatureCollection, we raise the public key hash up a level so it only exists once.

This type lives in tmconsensus instead of gcrypto due to its larger awareness of block hashes, which are outside the scope of the gcrypto package.

func (SparseSignatureCollection) ToFullPrecommitProofMap

func (c SparseSignatureCollection) ToFullPrecommitProofMap(
	height uint64,
	round uint32,
	valSet ValidatorSet,
	sigScheme SignatureScheme,
	cmspScheme gcrypto.CommonMessageSignatureProofScheme,
) (map[string]gcrypto.CommonMessageSignatureProof, error)

ToFullPrecommitProofMap converts c to a map of block hashes to full proofs.

func (SparseSignatureCollection) ToFullPrevoteProofMap

ToFullPrevoteProofMap converts c to a map of block hashes to full proofs.

type Validator

type Validator struct {
	PubKey gcrypto.PubKey
	Power  uint64
}

Validator is the simple representation of a validator, just a public key and a voting power.

type ValidatorSet

type ValidatorSet struct {
	// Consumers must assume that the Validators slice is shared in many places
	// and therefore cannot be modified.
	Validators []Validator

	// Hashes generated via a [HashScheme].
	PubKeyHash, VotePowerHash []byte
}

ValidatorSet is a fixed, ordered collection of validators.

func NewValidatorSet

func NewValidatorSet(vs []Validator, hs HashScheme) (ValidatorSet, error)

NewValidatorSet returns a ValidatorSet based on vs, with hashes calculated using hs.

NewValidatorSet assumes ownership over the validator slice, so that slice should not be modified after passing it to NewValidatorSet.

func (ValidatorSet) Equal

func (v ValidatorSet) Equal(other ValidatorSet) bool

ValidatorSet reports whether the collection of validators and the calculated hashes are the same in v and other.

type VersionedRoundView

type VersionedRoundView struct {
	// Embedded round view for ease of access.
	RoundView

	// Overall version that gets incremented with each atomic change.
	// It is possible that the overall version is incremented once
	// while sub-versions are incremented multiple times.
	// It seems very unlikely that a view of a single height/round
	// would get anywhere close to 2^32 versions.
	Version uint32

	// The overall version of the particular vote.
	// It seems very unlikely that a view of a single height/round
	// would get anywhere close to 2^32 versions.
	PrevoteVersion, PrecommitVersion uint32

	// The version of the votes we have seen for particular blocks.
	// This is independent of the overall vote version in the previous field.
	// If we see a vote for block A, then the map will contain A=>1,
	// and if that was the first update, the overall vote version may be 2
	// (because initial state is version 1).
	//
	// Then if another update occurs where we see an additional vote for A
	// and a new vote for B, this map may contain A=>2 and B=>1,
	// whereas the overall version may have been incremented from 2 to 3.
	PrevoteBlockVersions, PrecommitBlockVersions map[string]uint32
}

VersionedRoundView is a superset of RoundView that contains version information, for use cases where a RoundView may be receiving live updates and a consumer may care to identify what has changed from one update to another.

This type is used internally to the engine and exposed to the gossip strategy.

func (*VersionedRoundView) Clone

Clone returns a VersionedRoundView, with values identical to v, and underlying slices and maps copied from v.

func (VersionedRoundView) LogValue

func (v VersionedRoundView) LogValue() slog.Value

LogValue converts v into an slog.Value. This provides a highly detailed log, so it is only appropriate for infrequent log events, such as responding to a watchdog termination signal.

func (*VersionedRoundView) Reset

func (v *VersionedRoundView) Reset()

Reset zeros out all the fields of the VersionedRoundView, retaining any allocated capacity for its slices and maps. This is helpful for reusing RoundView values to avoid unnecessary memory allocations.

func (*VersionedRoundView) ResetForSameHeight

func (v *VersionedRoundView) ResetForSameHeight()

ResetForSameHeight resets the version information on the VersionedRoundView and calls v.RoundView.ResetForSameHeight.

This is particularly useful when there is existing validator information on v.RoundView that should not be discarded.

type VoteSummary

type VoteSummary struct {
	// The total voting power of the validators.
	AvailablePower uint64

	// The cumulative voting power present for prevotes or precommits.
	TotalPrevotePower, TotalPrecommitPower uint64

	// The voting power by block hash for prevotes or precommits.
	PrevoteBlockPower, PrecommitBlockPower map[string]uint64

	// Which prevote or precommit currently has the most votes.
	// If nothing has any votes, or if nil has the most votes, this is the empty string.
	// If there is a tie in voting power,
	// the value will be the lexicographically earlier hash, for consistency purposes.
	MostVotedPrevoteHash, MostVotedPrecommitHash string
}

VoteSummary summarizes the known votes in a round.

func NewVoteSummary

func NewVoteSummary() VoteSummary

func (VoteSummary) Clone

func (vs VoteSummary) Clone() VoteSummary

func (VoteSummary) LogValue

func (vs VoteSummary) LogValue() slog.Value

func (*VoteSummary) Reset

func (vs *VoteSummary) Reset()

func (*VoteSummary) ResetForSameHeight

func (vs *VoteSummary) ResetForSameHeight()

func (*VoteSummary) SetAvailablePower

func (vs *VoteSummary) SetAvailablePower(vals []Validator)

func (*VoteSummary) SetPrecommitPowers

func (vs *VoteSummary) SetPrecommitPowers(vals []Validator, precommits map[string]gcrypto.CommonMessageSignatureProof)

func (*VoteSummary) SetPrevotePowers

func (vs *VoteSummary) SetPrevotePowers(vals []Validator, prevotes map[string]gcrypto.CommonMessageSignatureProof)

func (*VoteSummary) SetVotePowers

func (vs *VoteSummary) SetVotePowers(vals []Validator, prevotes, precommits map[string]gcrypto.CommonMessageSignatureProof)

type VoteTarget

type VoteTarget struct {
	Height uint64
	Round  uint32

	// While the block hash is conventionally []byte,
	// we use a string here for simpler map keys
	// and because the hash is intended to be immutable after creation.
	// Note that an empty string indicates a nil vote.
	BlockHash string
}

VoteTarget is the reference of the block targeted for a prevote or precommit.

type VoteTargetMismatchError

type VoteTargetMismatchError struct {
	Want, Got VoteTarget
}

VoteTargetMismatchError indicates two prevote or precommit values disagree on their vote target.

func (VoteTargetMismatchError) Error

func (e VoteTargetMismatchError) Error() string

Directories

Path Synopsis
Package tmconsensustest contains utilities helpful for interacting with tmconsensus types in tests.
Package tmconsensustest contains utilities helpful for interacting with tmconsensus types in tests.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL