hotstuff

package
v0.33.17 Latest Latest
Warning

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

Go to latest
Published: Apr 17, 2024 License: AGPL-3.0 Imports: 13 Imported by: 30

README

Flow's HotStuff

We use a BFT consensus algorithm with deterministic finality in Flow for

  • Consensus Nodes: decide on the content of blocks by including collections of transactions,
  • Cluster of Collector Nodes: batch transactions into collections.

Conceptually, Flow uses a derivative of HotStuff called Jolteon. It was originally described in the paper 'Jolteon and Ditto: Network-Adaptive Efficient Consensus with Asynchronous Fallback', published June 2021 by Meta’s blockchain research team Novi and academic collaborators. Meta’s team (then called 'Diem') implemented Jolteon with marginal modifications and named it DiemBFT v4, which was subsequently rebranded as AptosBFT. Conceptually, Jolteon (DiemBFT v4, AptosBFT) belongs to the family of HotStuff consensus protocols, but adds two significant improvements over the original HotStuff: (i) Jolteon incorporates a PaceMaker with active message exchange for view synchronization and (ii) utilizes the most efficient 2-chain commit rule.

The foundational innovation in the original HotStuff was its pipelining of block production and finalization. It utilizes leaders for information collection and to drive consensus, which makes it highly message-efficient. In HotStuff, the consensus mechanics are cleverly arranged such that the protocol runs as fast as network conditions permit. (Not be limited by fixed, minimal wait times for messages.) This property is called "responsiveness" and is very important in practise.
HotStuff is a round-based consensus algorithm, which requires a supermajority of nodes to be in the same view to make progress. It is the role of the pacemaker to guarantee that eventually a supermajority of nodes will be in the same view. In the original HotStuff, the pacemaker was essentially left as a black box. The only requirement was that the pacemaker had to get the nodes eventually into the same view. Vanilla HotStuff requires 3 subsequent children to finalize a block on the happy path (aka '3-chain rule'). In the original HotStuff paper, the authors discuss the more efficient 2-chain rule. They explain a timing-related edge-case, where the protocol could theoretically get stuck in a timeout loop without progress. To guarantee liveness despite this edge case, the event-driven HotStuff variant in the original paper employs the 3-chain rule.

As this discussion illustrates, HotStuff is more a family of algorithms: the pacemaker is conceptually separated and can be implemented in may ways. The finality rule is easily swapped out. In addition, there are various other degrees of freedom left open in the family of HotStuff protocols. The Jolteon protocol extends HotStuff by specifying one particular pacemaker, which utilizes dedicated messages to synchronize views and provides very strong guarantees. Thereby, the Jolteon pacemaker closes the timing edge case forcing the original HotStuff to use the 3-chain rule for liveness. As a consequence, the Jolteon protocol can utilize the most efficient 2-chain rule. While Jolteon's close integration of the pacemaker into the consensus meachanics changes the correctness and liveness proofs significantly, the protocol's runtime behaviour matches the HotStuff framework. Therefore, we categorize Jolteon, DiemBFT v4, and AptosBFT as members of the HotStuff family.

Flow's consensus is largely an implementation of Jolteon with some elements from DiemBFT v4. While these consensus protocols are identical on a conceptual level, they subtly differ in the information included into the different consensus messages. For Flow's implementation, we combined nuances from Jolteon and DiemBFT v4 as follows to improve runtime efficiency, reduce code complexity, and minimize the surface for byzantine attacks:

  • Flow's TimeoutObject implements the timeout message in Jolteon.

    • In the Jolteon protocol, the timeout message contains the view V, which the sender wishes to abandon and the latest Quorum Certificate [QC] known to the sender. Due to successive leader failures, the QC might not be from the previous view, i.e. QC.View + 1 < V is possible. When receiving a timeout message, it is possible for the recipient to advance to round QC.View + 1, but not necessarily to view V, as a malicious sender might set V to an erroneously large value. On the one hand, a recipient that has fallen behind cannot catch up to view V immediately. On the other hand, the recipient must cache the timeout to guarantee liveness, making it vulnerable to memory exhaustion attacks.
    • DiemBFT v4 introduced the additional rule that the timeout must additionally include the Timeout Certificate [TC] for the previous view, if and only if the contained QC is not from the previous round (i.e. QC.View + 1 < V). Conceptually, this means that the sender of the timeout message must prove that they entered round V according to protocol rules. In other words, malicious nodes cannot send timeouts for future views that they should not have entered.

    For Flow, we follow the convention from DiemBFT v4. This modification simplifies byzantine-resilient processing of TimeoutObjects, avoiding subtle spamming and memory exhaustion attacks. Furthermore, it speeds up the recovery of crashed nodes.

  • For consensus votes, we stick with the original Jolteon format, i.e. we do not include the highest QC known to the voter, which is the case in DiemBFT v4. The QC is useful only on the unhappy path, where a node has missed some recent blocks. However, including a QC in every vote adds consistent overhead to the happy path. In Jolteon as well as DiemBFT v4, the timeout messages already contain the highest known QCs. Therefore, the highest QC is already shared among the network in the unhappy path even without including it in the votes.

  • In Jolteon, the TC contains the full QCs from a supermajority of nodes, which have some overhead in size. DiemBFT v4 improves this by only including the QC's respective views in the TC. Flow utilizes this optimization from DiemBFT v4.

In the following, we will use the terms Jolteon and HotStuff interchangeably to refer to Flow's consensus implementation. Beyond the Realm of HotStuff and Jolteon, we have added the following advancement to Flow's consensus system:

  • Flow contains a decentralized random beacon (based on Dfinity's proposal). The random beacon is run by Flow's consensus nodes and integrated into the consensus voting process. The random beacon provides a nearly unbiasable source of entropy natively within the protocol that is verifiable and deterministic. The random beacon can be used to generate pseudo random numbers, which we use within Flow protocol in various places. We plan to also use the random beacon to implement secure pseudo random number generators in Candence.

Architecture

Concepts and Terminology

In Flow, there are multiple HotStuff instances running in parallel. Specifically, the consensus nodes form a HotStuff committee and each collector cluster is its own committee. In the following, we refer to an authorized set of nodes, who run a particular HotStuff instance as a (HotStuff) committee.

  • Flow allows nodes to have different weights, reflecting how much they are trusted by the protocol. The weight of a node can change over time due to stake changes or discovering protocol violations. A super-majority of nodes is defined as a subset of the consensus committee, where the nodes have more than 2/3 of the entire committee's accumulated weight.
  • Conceptually, Flow allows that the random beacon is run only by a subset of consensus nodes, aka the "random beacon committee".
  • The messages from zero-weighted nodes are ignored by all committee members.
Determining block validity

In addition to Jolteon's requirements on block validity, the Flow protocol adds additional requirements. For example, it is illegal to repeatedly include the same payload entities (e.g. collections, challenges, etc) in the same fork. Generally, payload entities expire. However, within the expiry horizon, all ancestors of a block need to be known to verify that payload entities are not repeated.

We exclude the entire logic for determining payload validity from the HotStuff core implementation. This functionality is encapsulated in the Chain Compliance Layer (CCL) which precedes HotStuff. The CCL is designed to forward only fully validated blocks to the HotStuff core logic. The CCL forwards a block to the HotStuff core logic only if

  • the block's header is valid (including QC and optional TC),
  • the block's payload is valid,
  • the block is connected to the most recently finalized block, and
  • all ancestors have previously been forwarded to HotStuff.

If ancestors of a block are missing, the CCL caches the respective block and (iteratively) requests missing ancestors.

Payload generation

Payloads are generated outside the HotStuff core logic. HotStuff only incorporates the payload root hash into the block header.

Structure of votes

In Flow's HotStuff implementation, votes are used for two purposes:

  1. Prove that a super-majority of committee nodes consider the respective block a valid extension of the chain. Therefore, nodes include a StakingSignature (BLS with curve BLS12-381) in their vote.
  2. Construct a Source of Randomness as described in Dfinity's Random Beacon. Therefore, consensus nodes include a RandomBeaconSignature (also BLS with curve BLS12-381, used in a threshold signature scheme) in their vote.

When the primary collects the votes, it verifies the content of SigData, which can contain only a StakingSignature or a pair StakingSignature + RandomBeaconSignature. A StakingSignature must be present in all votes. (There is an optimization already implemented in the code, making the StakingSignature optional, but it is not enabled.) If either signature is invalid, the entire vote is discarded. From all valid votes, the StakingSignatures and the RandomBeaconSignatures are aggregated separately.

For purely consensus-theoretical purposes, it would be sufficient to use a threshold signature scheme. However, thresholds signatures have the following two important limitations, for which reason Flow uses aggregated signatures in addition:

  • The threshold signature carries no information about who signed. Meaning with the threshold signature alone, we have to way to distinguish the nodes are contributing from the ones being offline. The mature flow protocol will reward nodes based on their contributions to QCs, which requires a conventional aggregated signature.
  • Furthermore, the distributed key generation [DKG] for threshold keys currently limits the number of nodes. By including a signature aggregate, we can scale the consensus committee somewhat beyond the limitations of the [DKG]. Then, the nodes contributing to the random beacon would only be a subset of the entire consensus committee.
Communication topology
  • Following version 6 of the HotStuff paper, replicas forward their votes for block b to the leader of the next view, i.e. the primary for view b.View + 1.
  • A proposer will attach its own vote for its proposal in the block proposal message (instead of signing the block proposal for authenticity and separately sending a vote).
Primary section

For primary section, we use a randomized, weight-proportional selection.

Implementation Components

HotStuff's core logic is broken down into multiple components. The figure below illustrates the dependencies of the core components and information flow between these components.

  • MessageHub is responsible for relaying HotStuff messages. Incoming messages are relayed to the respective modules depending on their message type. Outgoing messages are relayed to the committee though the networking layer via epidemic gossip ('broadcast') or one-to-one communication ('unicast').
  • compliance.Engine is responsible for processing incoming blocks, caching if needed, validating, extending state and forwarding them to HotStuff for further processing. Note: The embedded compliance.Core component is responsible for business logic and maintaining state; compliance.Engine schedules work and manages worker threads for the Core.
  • EventLoop buffers all incoming events. It manages a single worker routine executing the EventHandler`'s logic.
  • EventHandler orchestrates all HotStuff components and implements the HotStuff's state machine. The event handler is designed to be executed single-threaded.
  • SafetyRules tracks the latest vote, the latest timeout and determines whether to vote for a block and if it's safe to timeout current round.
  • Pacemaker implements Jolteon's PaceMaker. It manages and updates a replica's local view and synchronizes it with other replicas. The Pacemaker ensures liveness by keeping a supermajority of the committee in the same view.
  • Forks maintains an in-memory representation of all blocks b, whose view is larger or equal to the view of the latest finalized block (known to this specific replica). As blocks with missing ancestors are cached outside HotStuff (by the Chain Compliance Layer), all blocks stored in Forks are guaranteed to be connected to the genesis block (or the trusted checkpoint from which the replica started). Forks tracks the finalized blocks and triggers finalization events whenever it observes a valid extension to the chain of finalized blocks. Forks is implemented using LevelledForest:
    • Conceptually, a blockchain constructs a tree of blocks. When removing all blocks with views strictly smaller than the last finalized block, this graph decomposes into multiple disconnected trees (referred to as a forest in graph theory). LevelledForest is an in-memory data structure to store and maintain a levelled forest. It provides functions to add vertices, query vertices by their ID (block's hash), query vertices by level (block's view), query the children of a vertex, and prune vertices by level (remove them from memory). To separate general graph-theoretical concepts from the concrete blockchain application, LevelledForest refers to blocks as graph vertices and to a block's view number as level.
  • Validator validates the HotStuff-relevant aspects of
    • QC: total weight of all signers is more than 2/3 of committee weight, validity of signatures, view number is strictly monotonously increasing;
    • TC: total weight of all signers is more than 2/3 of committee weight, validity of signatures, proof for entering view;
    • block proposal: from designated primary for the block's respective view, contains proposer's vote for its own block, QC in block is valid, a valid TC for the previous view is included if and only if the QC is not for the previous view;
    • vote: validity of signature, voter is has positive weight.
  • VoteAggregator caches votes on a per-block basis and builds QC if enough votes have been accumulated.
  • TimeoutAggregator caches timeouts on a per-view basis and builds TC if enough timeouts have been accumulated. Performs validation and verification of timeouts.
  • Replicas maintains the list of all authorized network members and their respective weight, queryable by view. It maintains a static list, which changes only between epochs. Furthermore, Replicas knows the primary for each view.
  • DynamicCommittee maintains the list of all authorized network members and their respective weight on a per-block basis. It extends Replicas allowing for committee changes mid epoch, e.g. due to slashing or node ejection.
  • BlockProducer constructs the payload of a block, after the HotStuff core logic has decided which fork to extend

Implementation

We have translated the HotStuff protocol into the state machine shown below. The state machine is implemented in EventHandler.

PaceMaker

The HotStuff state machine interacts with the PaceMaker, which triggers view changes. The PaceMaker keeps track of liveness data (newest QC, current view, TC for last view), and updates it when supplied with new data from EventHandler. Conceptually, the PaceMaker interfaces with the EventHandler in two different modes:

  • [asynchronous] On timeouts, the PaceMaker will emit a timeout event, which is processed as any other event (such as incoming blocks or votes) through the EventLoop.
  • [synchronous] When progress is made following the core business logic, the EventHandler will inform the PaceMaker about discovering new QCs or TCs via a direct method call (see PaceMaker interface). If the PaceMaker changed the view in response, it returns a NewViewEvent which will be synchronously processed by the EventHandler.

Flow's PaceMaker utilizes dedicated messages for synchronizing the consensus participant's local views. It broadcasts a TimeoutObject, whenever no progress is made during the current round. After collecting timeouts from a supermajority of participants, the replica constructs a TC which can be used to enter the next round V = TC.View + 1. For calculating round timeouts we use a truncated exponential backoff. We will increase round duration exponentially if no progress is made and exponentially decrease timeouts on happy path. During normal operation with some benign crash failures, a small number of k subsequent leader failures is expected. Therefore, our PaceMaker tolerates a few failures (k=6) before starting to increase timeouts, which is valuable for quickly skipping over the offline replicase. However, the probability of k subsequent leader failures decreases exponentially with k (due to Flow's randomized leader selection). Therefore, beyond k=6, we start increasing timeouts. The timeout values are limited by lower and upper-bounds to ensure that the PaceMaker can change from large to small timeouts in a reasonable number of views. The specific values for lower and upper timeout bounds are protocol-specified; we envision the bounds to be on the order of 1sec (lower bound) and one minute (upper bound).

Progress, from the perspective of the PaceMaker, is defined as entering view V for which the replica knows a QC or a TC with V = QC.view + 1 or V = TC.view + 1. In other words, we transition into the next view when observing a quorum from the last view. In contrast to HotStuff, Jolteon only allows a transition into view V+1 after observing a valid quorum for view V. There is no other, passive method for honest nodes to change views.

A central, non-trivial functionality of the PaceMaker is to skip views. Specifically, given a QC or TC with view V, the Pacemaker will skip ahead to view V + 1 if currentView ≤ V.

Code structure

All relevant code implementing the core HotStuff business logic is contained in /consensus/hotstuff/ (folder containing this README). When starting to look into the code, we suggest starting with /consensus/hotstuff/event_loop.go and /consensus/hotstuff/event_handler.go.

Folder structure

All files in the /consensus/hotstuff/ folder, except for follower_loop.go, are interfaces for HotStuff-related components. The concrete implementations for all HotStuff-relevant components are in corresponding sub-folders. For completeness, we list the component implemented in each sub-folder below:

  • /consensus/hotstuff/blockproducer builds a block proposal for a specified QC, interfaces with the logic for assembling a block payload, combines all relevant fields into a new block proposal.
  • /consensus/hotstuff/committees maintains the list of all authorized network members and their respective weight on a per-block and per-view basis depending on implementation; contains the primary selection algorithm.
  • /consensus/hotstuff/eventloop buffers all incoming events, so EventHandler can process one event at a time in a single thread.
  • /consensus/hotstuff/eventhandler orchestrates all HotStuff components and implements the HotStuff state machine. The event handler is designed to be executed single-threaded.
  • /consensus/hotstuff/follower This component is only used by nodes that are not participating in the HotStuff committee. As Flow has dedicated node roles with specialized network functions, only a subset of nodes run the full HotStuff protocol. Nevertheless, all nodes need to be able to act on blocks being finalized. The approach we have taken for Flow is that block proposals are broadcast to all nodes (including non-committee nodes). Non-committee nodes locally determine block finality by applying HotStuff's finality rules. The HotStuff Follower contains the functionality to consume block proposals and trigger downstream processing of finalized blocks. The Follower does not actively participate in HotStuff.
  • /consensus/hotstuff/forks maintains an in-memory representation of blocks, whose view is larger or equal to the view of the latest finalized block (known to this specific HotStuff replica). Per convention, all blocks stored in forks passed validation and their ancestry is fully known. forks tracks the last finalized block and implements the 2-chain finalization rule. Specifically, we finalize block B, if a certified child B' is known that was produced in the view B.View +1.
  • /consensus/hotstuff/helper contains broadly-used helper functions for testing
  • /consensus/hotstuff/integration integration tests for verifying correct interaction of multiple HotStuff replicas
  • /consensus/hotstuff/model contains the HotStuff data models, including block proposal, vote, timeout, etc. Many HotStuff data models are built on top of basic data models defined in /model/flow/.
  • /consensus/hotstuff/notifications: All relevant events within the HotStuff logic are exported though a notification system. Notifications are used by some HotStuff components internally to drive core logic (e.g. events from VoteAggregator and TimeoutAggregator can trigger progress in the EventHandler). Furthermore, notifications inform other components within the same node of relevant progress and are used for collecting HotStuff metrics. Per convention, notifications are idempotent.
  • /consensus/hotstuff/pacemaker contains the implementation of Flow's Active PaceMaker, as described above. Is responsible for protocol liveness.
  • /consensus/hotstuff/persister stores the latest safety and liveness data synchronously on disk. The persister only covers the minimal amount of data that is absolutely necessary to avoid equivocation after a crash. This data must be stored on disk immediately whenever updated, before the node can progress with its consensus logic. In comparison, the majority of the consensus state is held in-memory for performance reasons and updated in an eventually consistent manner. After a crash, some of this data might be lost (but can be re-requested) without risk of protocol violations.
  • /consensus/hotstuff/safetyrules tracks the latest vote and the latest timeout. It determines whether to vote for a block and if it's safe to construct a timeout for the current round.
  • /consensus/hotstuff/signature contains the implementation for threshold signature aggregation for all types of signatures that are used in HotStuff protocol.
  • /consensus/hotstuff/timeoutcollector encapsulates the logic for validating timeouts for one particular view and aggregating them to a TC.
  • /consensus/hotstuff/timeoutaggregator orchestrates the TimeoutCollectors for different views. It distributes timeouts to the respective TimeoutCollector and prunes collectors that are no longer needed.
  • /consensus/hotstuff/tracker implements utility code for tracking the newest QC and TC in a multithreaded environment.
  • /consensus/hotstuff/validator holds the logic for validating the HotStuff-relevant aspects of blocks, QCs, TC, and votes
  • /consensus/hotstuff/verification contains integration of Flow's cryptographic primitives (signing and signature verification)
  • /consensus/hotstuff/votecollector encapsulates the logic for caching, validating, and aggregating votes for one particular view. It tracks, whether a valid proposal for view is known and when enough votes have been collected, it builds a QC.
  • /consensus/hotstuff/voteaggregator orchestrates the VoteCollectors for different views. It distributes votes to the respective VoteCollector, notifies the VoteCollector about the arrival of their respective block, and prunes collectors that are no longer needed.

Telemetry

The HotStuff state machine exposes some details about its internal progress as notification through the hotstuff.Consumer. The following figure depicts at which points notifications are emitted.

We have implemented a telemetry system (hotstuff.notifications.TelemetryConsumer) which implements the Consumer interface. The TelemetryConsumer tracks all events as belonging together that were emitted during a path through the state machine as well as events from components that perform asynchronous processing (VoteAggregator, TimeoutAggregator). Each path through the state machine is identified by a unique id. Generally, the TelemetryConsumer could export the collected data to a variety of backends. For now, we export the data to a logger.

Documentation

Index

Constants

This section is empty.

Variables

This section is empty.

Functions

This section is empty.

Types

type BlockProducer

type BlockProducer interface {
	// MakeBlockProposal builds a new HotStuff block proposal using the given view,
	// the given quorum certificate for its parent and [optionally] a timeout certificate for last view(could be nil).
	// No errors are expected during normal operation.
	MakeBlockProposal(view uint64, qc *flow.QuorumCertificate, lastViewTC *flow.TimeoutCertificate) (*flow.Header, error)
}

BlockProducer builds a new block proposal by building a new block payload with the builder module, and uses VoteCollectorFactory to create a disposable VoteCollector for producing the proposal vote. BlockProducer assembles the new block proposal using the block payload, block header and the proposal vote.

type BlockSignatureData added in v0.23.9

type BlockSignatureData struct {
	StakingSigners               flow.IdentifierList
	RandomBeaconSigners          flow.IdentifierList
	AggregatedStakingSig         []byte // if BLS is used, this is equivalent to crypto.Signature
	AggregatedRandomBeaconSig    []byte // if BLS is used, this is equivalent to crypto.Signature
	ReconstructedRandomBeaconSig crypto.Signature
}

BlockSignatureData is an intermediate struct for Packer to pack the aggregated signature data into raw bytes or unpack from raw bytes.

type BlockSignerDecoder added in v0.26.17

type BlockSignerDecoder interface {
	// DecodeSignerIDs decodes the signer indices from the given block header into full node IDs.
	// Note: A block header contains a quorum certificate for its parent, which proves that the
	// consensus committee has reached agreement on validity of parent block. Consequently, the
	// returned IdentifierList contains the consensus participants that signed the parent block.
	// Expected Error returns during normal operations:
	//  - signature.InvalidSignerIndicesError if signer indices included in the header do
	//    not encode a valid subset of the consensus committee
	DecodeSignerIDs(header *flow.Header) (flow.IdentifierList, error)
}

BlockSignerDecoder defines how to convert the ParentSignerIndices field within a particular block header to the identifiers of the nodes which signed the block.

type CommunicatorConsumer added in v0.29.0

type CommunicatorConsumer interface {
	// OnOwnVote notifies about intent to send a vote for the given parameters to the specified recipient.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnOwnVote(blockID flow.Identifier, view uint64, sigData []byte, recipientID flow.Identifier)

	// OnOwnTimeout notifies about intent to broadcast the given timeout object(TO) to all actors of the consensus process.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnOwnTimeout(timeout *model.TimeoutObject)

	// OnOwnProposal notifies about intent to broadcast the given block proposal to all actors of
	// the consensus process.
	// delay is to hold the proposal before broadcasting it. Useful to control the block production rate.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnOwnProposal(proposal *flow.Header, targetPublicationTime time.Time)
}

CommunicatorConsumer consumes outbound notifications produced by HotStuff and it's components. Notifications allow the HotStuff core algorithm to communicate with the other actors of the consensus process. Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type Consumer

Consumer consumes outbound notifications produced by consensus participants. Notifications are consensus-internal state changes which are potentially relevant to the larger node in which HotStuff is running. The notifications are emitted in the order in which the HotStuff algorithm makes the respective steps.

Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type DKG

type DKG interface {
	protocol.DKG
}

type DynamicCommittee added in v0.29.0

type DynamicCommittee interface {
	Replicas

	// IdentitiesByBlock returns a list of the legitimate HotStuff participants for the given block.
	// The returned list of HotStuff participants:
	//   - contains nodes that are allowed to submit proposals, votes, and timeouts
	//     (un-ejected, non-zero weight at current block)
	//   - is ordered in the canonical order
	//   - contains no duplicates.
	//
	// ERROR conditions:
	//  - state.ErrUnknownSnapshotReference if the blockID is for an unknown block
	IdentitiesByBlock(blockID flow.Identifier) (flow.IdentityList, error)

	// IdentityByBlock returns the full Identity for specified HotStuff participant.
	// The node must be a legitimate HotStuff participant with NON-ZERO WEIGHT at the specified block.
	// ERROR conditions:
	//  - model.InvalidSignerError if participantID does NOT correspond to an authorized HotStuff participant at the specified block.
	//  - state.ErrUnknownSnapshotReference if the blockID is for an unknown block
	IdentityByBlock(blockID flow.Identifier, participantID flow.Identifier) (*flow.Identity, error)
}

DynamicCommittee extends Replicas to provide the consensus committee for the purposes of validating proposals. The proposer committee reflects block-to-block changes in the identity table to support immediately rejecting proposals from nodes after they are ejected. For validating proposals, we use *ByBlock methods.

Since the proposer committee can change at any block:

  • we query by block ID
  • we must have incorporated the full block ancestry prior to validating messages

type EventHandler

type EventHandler interface {

	// OnReceiveQc processes a valid qc constructed by internal vote aggregator or discovered in TimeoutObject.
	// All inputs should be validated before feeding into this function. Assuming trusted data.
	// No errors are expected during normal operation.
	OnReceiveQc(qc *flow.QuorumCertificate) error

	// OnReceiveTc processes a valid tc constructed by internal timeout aggregator, discovered in TimeoutObject or
	// broadcast over the network.
	// All inputs should be validated before feeding into this function. Assuming trusted data.
	// No errors are expected during normal operation.
	OnReceiveTc(tc *flow.TimeoutCertificate) error

	// OnReceiveProposal processes a block proposal received from another HotStuff
	// consensus participant.
	// All inputs should be validated before feeding into this function. Assuming trusted data.
	// No errors are expected during normal operation.
	OnReceiveProposal(proposal *model.Proposal) error

	// OnLocalTimeout handles a local timeout event by creating a model.TimeoutObject and broadcasting it.
	// No errors are expected during normal operation.
	OnLocalTimeout() error

	// OnPartialTcCreated handles notification produces by the internal timeout aggregator. If the notification is for the current view,
	// a corresponding model.TimeoutObject is broadcast to the consensus committee.
	// No errors are expected during normal operation.
	OnPartialTcCreated(partialTC *PartialTcCreated) error

	// TimeoutChannel returns a channel that sends a signal on timeout.
	TimeoutChannel() <-chan time.Time

	// Start starts the event handler.
	// No errors are expected during normal operation.
	// CAUTION: EventHandler is not concurrency safe. The Start method must
	// be executed by the same goroutine that also calls the other business logic
	// methods, or concurrency safety has to be implemented externally.
	Start(ctx context.Context) error
}

EventHandler runs a state machine to process proposals, QC and local timeouts. Not concurrency safe.

type EventLoop

EventLoop performs buffer and processing of incoming proposals and QCs.

type FinalityProof added in v0.31.0

type FinalityProof struct {
	Block          *model.Block
	CertifiedChild model.CertifiedBlock
}

FinalityProof represents a finality proof for a Block. By convention, a FinalityProof is immutable. Finality in Jolteon/HotStuff is determined by the 2-chain rule:

There exists a _certified_ block C, such that Block.View + 1 = C.View

type FinalizationConsumer

type FinalizationConsumer interface {
	// OnBlockIncorporated notifications are produced by the Finalization Logic
	// whenever a block is incorporated into the consensus state.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnBlockIncorporated(*model.Block)

	// OnFinalizedBlock notifications are produced by the Finalization Logic whenever
	// a block has been finalized. They are emitted in the order the blocks are finalized.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnFinalizedBlock(*model.Block)
}

FinalizationConsumer consumes outbound notifications produced by the logic tracking forks and finalization. Such notifications are produced by the active consensus participants, and generally potentially relevant to the larger node. The notifications are emitted in the order in which the finalization algorithm makes the respective steps.

Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type FollowerConsumer added in v0.31.0

type FollowerConsumer interface {
	ProposalViolationConsumer
	FinalizationConsumer
}

FollowerConsumer consumes outbound notifications produced by consensus followers. It is a subset of the notifications produced by consensus participants. Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type FollowerLoop

type FollowerLoop struct {
	*component.ComponentManager
	// contains filtered or unexported fields
}

FollowerLoop implements interface module.HotStuffFollower. FollowerLoop buffers all incoming events to the hotstuff FollowerLogic, and feeds FollowerLogic one event at a time using a worker thread. Concurrency safe.

func NewFollowerLoop

func NewFollowerLoop(log zerolog.Logger, mempoolMetrics module.MempoolMetrics, forks Forks) (*FollowerLoop, error)

NewFollowerLoop creates an instance of HotStuffFollower

func (*FollowerLoop) AddCertifiedBlock added in v0.31.0

func (fl *FollowerLoop) AddCertifiedBlock(certifiedBlock *model.CertifiedBlock)

AddCertifiedBlock appends the given certified block to the tree of pending blocks and updates the latest finalized block (if finalization progressed). Unless the parent is below the pruning threshold (latest finalized view), we require that the parent has previously been added.

Notes:

  • Under normal operations, this method is non-blocking. The follower internally queues incoming blocks and processes them in its own worker routine. However, when the inbound queue is, we block until there is space in the queue. This behavior is intentional, because we cannot drop blocks (otherwise, we would cause disconnected blocks). Instead, we simply block the compliance layer to avoid any pathological edge cases.
  • Blocks whose views are below the latest finalized view are dropped.
  • Inputs are idempotent (repetitions are no-ops).

type Forks

type Forks interface {

	// GetBlocksForView returns all known blocks for the given view
	GetBlocksForView(view uint64) []*model.Block

	// GetBlock returns (BlockProposal, true) if the block with the specified
	// id was found and (nil, false) otherwise.
	GetBlock(blockID flow.Identifier) (*model.Block, bool)

	// FinalizedView returns the largest view number where a finalized block is known
	FinalizedView() uint64

	// FinalizedBlock returns the finalized block with the largest view number
	FinalizedBlock() *model.Block

	// FinalityProof returns the latest finalized block and a certified child from
	// the subsequent view, which proves finality.
	// CAUTION: method returns (nil, false), when Forks has not yet finalized any
	// blocks beyond the finalized root block it was initialized with.
	FinalityProof() (*FinalityProof, bool)

	// AddValidatedBlock appends the validated block to the tree of pending
	// blocks and updates the latest finalized block (if applicable). Unless the parent is
	// below the pruning threshold (latest finalized view), we require that the parent is
	// already stored in Forks. Calling this method with previously processed blocks
	// leaves the consensus state invariant (though, it will potentially cause some
	// duplicate processing).
	// Notes:
	//   - Method `AddCertifiedBlock(..)` should be used preferably, if a QC certifying
	//     `block` is already known. This is generally the case for the consensus follower.
	//     Method `AddValidatedBlock` is intended for active consensus participants, which fully
	//     validate blocks (incl. payload), i.e. QCs are processed as part of validated proposals.
	//
	// Possible error returns:
	//   - model.MissingBlockError if the parent does not exist in the forest (but is above
	//     the pruned view). From the perspective of Forks, this error is benign (no-op).
	//   - model.InvalidBlockError if the block is invalid (see `Forks.EnsureBlockIsValidExtension`
	//     for details). From the perspective of Forks, this error is benign (no-op). However, we
	//     assume all blocks are fully verified, i.e. they should satisfy all consistency
	//     requirements. Hence, this error is likely an indicator of a bug in the compliance layer.
	//   - model.ByzantineThresholdExceededError if conflicting QCs or conflicting finalized
	//     blocks have been detected (violating a foundational consensus guarantees). This
	//     indicates that there are 1/3+ Byzantine nodes (weighted by stake) in the network,
	//     breaking the safety guarantees of HotStuff (or there is a critical bug / data
	//     corruption). Forks cannot recover from this exception.
	//   - All other errors are potential symptoms of bugs or state corruption.
	AddValidatedBlock(proposal *model.Block) error

	// AddCertifiedBlock appends the given certified block to the tree of pending
	// blocks and updates the latest finalized block (if finalization progressed).
	// Unless the parent is below the pruning threshold (latest finalized view), we
	// require that the parent is already stored in Forks. Calling this method with
	// previously processed blocks leaves the consensus state invariant (though,
	// it will potentially cause some duplicate processing).
	//
	// Possible error returns:
	//   - model.MissingBlockError if the parent does not exist in the forest (but is above
	//     the pruned view). From the perspective of Forks, this error is benign (no-op).
	//   - model.InvalidBlockError if the block is invalid (see `Forks.EnsureBlockIsValidExtension`
	//     for details). From the perspective of Forks, this error is benign (no-op). However, we
	//     assume all blocks are fully verified, i.e. they should satisfy all consistency
	//     requirements. Hence, this error is likely an indicator of a bug in the compliance layer.
	//   - model.ByzantineThresholdExceededError if conflicting QCs or conflicting finalized
	//     blocks have been detected (violating a foundational consensus guarantees). This
	//     indicates that there are 1/3+ Byzantine nodes (weighted by stake) in the network,
	//     breaking the safety guarantees of HotStuff (or there is a critical bug / data
	//     corruption). Forks cannot recover from this exception.
	//   - All other errors are potential symptoms of bugs or state corruption.
	AddCertifiedBlock(certifiedBlock *model.CertifiedBlock) error
}

Forks maintains an in-memory data-structure of all blocks whose view-number is larger or equal to the latest finalized block. The latest finalized block is defined as the finalized block with the largest view number. When adding blocks, Forks automatically updates its internal state (including finalized blocks). Furthermore, blocks whose view number is smaller than the latest finalized block are pruned automatically.

PREREQUISITES: Forks expects that only blocks are added that can be connected to its latest finalized block (without missing interim ancestors). If this condition is violated, Forks will raise an error and ignore the block.

type LivenessData added in v0.29.0

type LivenessData struct {
	// CurrentView is the currently active view tracked by the PaceMaker. It is updated
	// whenever the PaceMaker sees evidence (QC or TC) for advancing to next view.
	CurrentView uint64
	// NewestQC is the newest QC (by view) observed by the PaceMaker. The QC can be observed on its own or as a part of TC.
	NewestQC *flow.QuorumCertificate
	// LastViewTC is the TC for the prior view (CurrentView-1), if this view timed out. If the previous round
	// ended with a QC, this QC is stored in NewestQC and LastViewTC is nil.
	LastViewTC *flow.TimeoutCertificate
}

type OnQCCreated added in v0.23.9

type OnQCCreated func(*flow.QuorumCertificate)

OnQCCreated is a callback which will be used by VoteCollector to submit a QC when it's able to create it

type PaceMaker

type PaceMaker interface {
	ProposalDurationProvider

	// CurView returns the current view.
	CurView() uint64

	// NewestQC returns QC with the highest view discovered by PaceMaker.
	NewestQC() *flow.QuorumCertificate

	// LastViewTC returns TC for last view, this could be nil if previous round
	// has entered with a QC.
	LastViewTC() *flow.TimeoutCertificate

	// ProcessQC will check if the given QC will allow PaceMaker to fast-forward to QC.view+1.
	// If PaceMaker incremented the current View, a NewViewEvent will be returned.
	// No errors are expected during normal operation.
	ProcessQC(qc *flow.QuorumCertificate) (*model.NewViewEvent, error)

	// ProcessTC will check if the given TC will allow PaceMaker to fast-forward to TC.view+1.
	// If PaceMaker incremented the current View, a NewViewEvent will be returned.
	// A nil TC is an expected valid input.
	// No errors are expected during normal operation.
	ProcessTC(tc *flow.TimeoutCertificate) (*model.NewViewEvent, error)

	// TimeoutChannel returns the timeout channel for the CURRENTLY ACTIVE timeout.
	// Each time the pacemaker starts a new timeout, this channel is replaced.
	TimeoutChannel() <-chan time.Time

	// Start starts the PaceMaker (i.e. the timeout for the configured starting value for view).
	// CAUTION: EventHandler is not concurrency safe. The Start method must
	// be executed by the same goroutine that also calls the other business logic
	// methods, or concurrency safety has to be implemented externally.
	Start(ctx context.Context)
}

PaceMaker for HotStuff. The component is passive in that it only reacts to method calls. The PaceMaker does not perform state transitions on its own. Timeouts are emitted through channels. Each timeout has its own dedicated channel, which is garbage collected after the respective state has been passed. It is the EventHandler's responsibility to pick up timeouts from the currently active TimeoutChannel process them first and subsequently inform the PaceMaker about processing the timeout. Specifically, the intended usage pattern for the TimeoutChannels is as follows:

• Each time the PaceMaker starts a new timeout, it created a new TimeoutChannel

• The channel for the CURRENTLY ACTIVE timeout is returned by PaceMaker.TimeoutChannel()

  • Each time the EventHandler processes an event, the EventHandler might call into PaceMaker potentially resulting in a state transition and the PaceMaker starting a new timeout

  • Hence, after processing any event, EventHandler should retrieve the current TimeoutChannel from the PaceMaker.

For Example:

for {
	timeoutChannel := el.eventHandler.TimeoutChannel()
	select {
	   case <-timeoutChannel:
	    	el.eventHandler.OnLocalTimeout()
	   case <other events>
	}
}

Not concurrency safe.

type Packer added in v0.23.9

type Packer interface {
	// Pack serializes the provided BlockSignatureData into a precursor format of a QC.
	// view is the view of the block that the aggregated signature is for.
	// sig is the aggregated signature data.
	// Expected error returns during normal operations:
	//  * none; all errors are symptoms of inconsistent input data or corrupted internal state.
	Pack(view uint64, sig *BlockSignatureData) (signerIndices []byte, sigData []byte, err error)

	// Unpack de-serializes the provided signature data.
	// sig is the aggregated signature data
	// It returns:
	//  - (sigData, nil) if successfully unpacked the signature data
	//  - (nil, model.InvalidFormatError) if failed to unpack the signature data
	Unpack(signerIdentities flow.IdentityList, sigData []byte) (*BlockSignatureData, error)
}

Packer packs aggregated signature data into raw bytes to be used in block header.

type PartialTcCreated added in v0.29.0

type PartialTcCreated struct {
	View       uint64
	NewestQC   *flow.QuorumCertificate
	LastViewTC *flow.TimeoutCertificate
}

PartialTcCreated represents a notification emitted by the TimeoutProcessor component, whenever it has collected TimeoutObjects from a superminority of consensus participants for a specific view. Along with the view, it reports the newest QC and TC (for previous view) discovered during timeout collection. Per convention, the newest QC is never nil, while the TC for the previous view might be nil.

type ParticipantConsumer added in v0.31.0

type ParticipantConsumer interface {
	// OnEventProcessed notifications are produced by the EventHandler when it is done processing
	// and hands control back to the EventLoop to wait for the next event.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnEventProcessed()

	// OnStart notifications are produced by the EventHandler when it starts blocks recovery and
	// prepares for handling incoming events from EventLoop.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnStart(currentView uint64)

	// OnReceiveProposal notifications are produced by the EventHandler when it starts processing a block.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnReceiveProposal(currentView uint64, proposal *model.Proposal)

	// OnReceiveQc notifications are produced by the EventHandler when it starts processing a
	// QuorumCertificate [QC] constructed by the node's internal vote aggregator.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnReceiveQc(currentView uint64, qc *flow.QuorumCertificate)

	// OnReceiveTc notifications are produced by the EventHandler when it starts processing a
	// TimeoutCertificate [TC]  constructed by the node's internal timeout aggregator.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnReceiveTc(currentView uint64, tc *flow.TimeoutCertificate)

	// OnPartialTc notifications are produced by the EventHandler when it starts processing partial TC
	// constructed by local timeout aggregator.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnPartialTc(currentView uint64, partialTc *PartialTcCreated)

	// OnLocalTimeout notifications are produced by the EventHandler when it reacts to expiry of round duration timer.
	// Such a notification indicates that the PaceMaker's timeout was processed by the system.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnLocalTimeout(currentView uint64)

	// OnViewChange notifications are produced by PaceMaker when it transitions to a new view
	// based on processing a QC or TC. The arguments specify the oldView (first argument),
	// and the newView to which the PaceMaker transitioned (second argument).
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnViewChange(oldView, newView uint64)

	// OnQcTriggeredViewChange notifications are produced by PaceMaker when it moves to a new view
	// based on processing a QC. The arguments specify the qc (first argument), which triggered
	// the view change, and the newView to which the PaceMaker transitioned (second argument).
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnQcTriggeredViewChange(oldView uint64, newView uint64, qc *flow.QuorumCertificate)

	// OnTcTriggeredViewChange notifications are produced by PaceMaker when it moves to a new view
	// based on processing a TC. The arguments specify the tc (first argument), which triggered
	// the view change, and the newView to which the PaceMaker transitioned (second argument).
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnTcTriggeredViewChange(oldView uint64, newView uint64, tc *flow.TimeoutCertificate)

	// OnStartingTimeout notifications are produced by PaceMaker. Such a notification indicates that the
	// PaceMaker is now waiting for the system to (receive and) process blocks or votes.
	// The specific timeout type is contained in the TimerInfo.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnStartingTimeout(model.TimerInfo)

	// OnCurrentViewDetails notifications are produced by the EventHandler during the course of a view with auxiliary information.
	// These notifications are generally not produced for all views (for example skipped views).
	// These notifications are guaranteed to be produced for all views we enter after fully processing a message.
	// Example 1:
	//   - We are in view 8. We process a QC with view 10, causing us to enter view 11.
	//   - Then this notification will be produced for view 11.
	// Example 2:
	//   - We are in view 8. We process a proposal with view 10, which contains a TC for view 9 and TC.NewestQC for view 8.
	//   - The QC would allow us to enter view 9 and the TC would allow us to enter view 10,
	//     so after fully processing the message we are in view 10.
	//   - Then this notification will be produced for view 10, but not view 9
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnCurrentViewDetails(currentView, finalizedView uint64, currentLeader flow.Identifier)
}

ParticipantConsumer consumes outbound notifications produced by consensus participants actively proposing blocks, voting, collecting & aggregating votes to QCs, and participating in the pacemaker (sending timeouts, collecting & aggregating timeouts to TCs). Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type Persister

type Persister interface {
	// GetSafetyData will retrieve last persisted safety data.
	// During normal operations, no errors are expected.
	GetSafetyData() (*SafetyData, error)

	// PutSafetyData persists the last safety data.
	// This method blocks until `safetyData` was successfully persisted.
	// During normal operations, no errors are expected.
	PutSafetyData(safetyData *SafetyData) error

	// GetLivenessData will retrieve last persisted liveness data.
	// During normal operations, no errors are expected.
	GetLivenessData() (*LivenessData, error)

	// PutLivenessData persists the last liveness data.
	// This method blocks until `safetyData` was successfully persisted.
	// During normal operations, no errors are expected.
	PutLivenessData(livenessData *LivenessData) error
}

Persister is responsible for persisting state we need to bootstrap after a restart or crash.

type ProposalDurationProvider added in v0.31.0

type ProposalDurationProvider interface {
	// TargetPublicationTime is intended to be called by the EventHandler, whenever it
	// wants to publish a new proposal. The event handler inputs
	//  - proposalView: the view it is proposing for,
	//  - timeViewEntered: the time when the EventHandler entered this view
	//  - parentBlockId: the ID of the parent block , which the EventHandler is building on
	// TargetPublicationTime returns the time stamp when the new proposal should be broadcasted.
	// For a given view where we are the primary, suppose the actual time we are done building our proposal is P:
	//   - if P < TargetPublicationTime(..), then the EventHandler should wait until
	//     `TargetPublicationTime` to broadcast the proposal
	//   - if P >= TargetPublicationTime(..), then the EventHandler should immediately broadcast the proposal
	// Concurrency safe.
	TargetPublicationTime(proposalView uint64, timeViewEntered time.Time, parentBlockId flow.Identifier) time.Time
}

ProposalDurationProvider generates the target publication time for block proposals.

type ProposalViolationConsumer added in v0.31.0

type ProposalViolationConsumer interface {
	// OnInvalidBlockDetected notifications are produced by components that have detected
	// that a block proposal is invalid and need to report it.
	// Most of the time such block can be detected by calling Validator.ValidateProposal.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnInvalidBlockDetected(err flow.Slashable[model.InvalidProposalError])

	// OnDoubleProposeDetected notifications are produced by the Finalization Logic
	// whenever a double block proposal (equivocation) was detected.
	// Equivocation occurs when the same leader proposes two different blocks for the same view.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnDoubleProposeDetected(*model.Block, *model.Block)
}

ProposalViolationConsumer consumes outbound notifications about HotStuff-protocol violations. Such notifications are produced by the active consensus participants and consensus follower.

Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type RandomBeaconInspector added in v0.23.9

type RandomBeaconInspector interface {
	// Verify verifies the signature share under the signer's public key and the message agreed upon.
	// The function is thread-safe and wait-free (i.e. allowing arbitrary many routines to
	// execute the business logic, without interfering with each other).
	// It allows concurrent verification of the given signature.
	// Returns :
	//  - model.InvalidSignerError if signerIndex is invalid
	//  - model.ErrInvalidSignature if signerIndex is valid but signature is cryptographically invalid
	//  - other error if there is an unexpected exception.
	Verify(signerIndex int, share crypto.Signature) error

	// TrustedAdd adds a share to the internal signature shares store.
	// There is no pre-check of the signature's validity _before_ adding it.
	// It is the caller's responsibility to make sure the signature was previously verified.
	// Nevertheless, the implementation guarantees safety (only correct threshold signatures
	// are returned) through a post-check (verifying the threshold signature
	// _after_ reconstruction before returning it).
	// The function is thread-safe but locks its internal state, thereby permitting only
	// one routine at a time to add a signature.
	// Returns:
	//  - (true, nil) if the signature has been added, and enough shares have been collected.
	//  - (false, nil) if the signature has been added, but not enough shares were collected.
	//  - (false, error) if there is any exception adding the signature share.
	//      - model.InvalidSignerError if signerIndex is invalid (out of the valid range)
	//  	- model.DuplicatedSignerError if the signer has been already added
	//      - other error if there is an unexpected exception.
	TrustedAdd(signerIndex int, share crypto.Signature) (enoughshares bool, exception error)

	// EnoughShares indicates whether enough shares have been accumulated in order to reconstruct
	// a group signature. The function is thread-safe.
	EnoughShares() bool

	// Reconstruct reconstructs the group signature. The function is thread-safe but locks
	// its internal state, thereby permitting only one routine at a time.
	//
	// Returns:
	// - (signature, nil) if no error occurred
	// - (nil, model.InsufficientSignaturesError) if not enough shares were collected
	// - (nil, model.InvalidSignatureIncluded) if at least one collected share does not serialize to a valid BLS signature,
	//    or if the constructed signature failed to verify against the group public key and stored message. This post-verification
	//    is required  for safety, as `TrustedAdd` allows adding invalid signatures.
	// - (nil, error) for any other unexpected error.
	Reconstruct() (crypto.Signature, error)
}

RandomBeaconInspector encapsulates all methods needed by a Hotstuff leader to validate the beacon votes and reconstruct a beacon signature. The random beacon methods are based on a threshold signature scheme.

type RandomBeaconReconstructor added in v0.23.9

type RandomBeaconReconstructor interface {
	// Verify verifies the signature share under the signer's public key and the message agreed upon.
	// The function is thread-safe and wait-free (i.e. allowing arbitrary many routines to
	// execute the business logic, without interfering with each other).
	// It allows concurrent verification of the given signature.
	// Returns :
	//  - model.InvalidSignerError if signerIndex is invalid
	//  - model.ErrInvalidSignature if signerID is valid but signature is cryptographically invalid
	//  - other error if there is an unexpected exception.
	Verify(signerID flow.Identifier, sig crypto.Signature) error

	// TrustedAdd adds a share to the internal signature shares store.
	// There is no pre-check of the signature's validity _before_ adding it.
	// It is the caller's responsibility to make sure the signature was previously verified.
	// Nevertheless, the implementation guarantees safety (only correct threshold signatures
	// are returned) through a post-check (verifying the threshold signature
	// _after_ reconstruction before returning it).
	// The function is thread-safe but locks its internal state, thereby permitting only
	// one routine at a time to add a signature.
	// Returns:
	//  - (true, nil) if the signature has been added, and enough shares have been collected.
	//  - (false, nil) if the signature has been added, but not enough shares were collected.
	//  - (false, error) if there is any exception adding the signature share.
	//      - model.InvalidSignerError if signerIndex is invalid (out of the valid range)
	//  	- model.DuplicatedSignerError if the signer has been already added
	//      - other error if there is an unexpected exception.
	TrustedAdd(signerID flow.Identifier, sig crypto.Signature) (EnoughShares bool, err error)

	// EnoughShares indicates whether enough shares have been accumulated in order to reconstruct
	// a group signature. The function is thread-safe.
	EnoughShares() bool

	// Reconstruct reconstructs the group signature. The function is thread-safe but locks
	// its internal state, thereby permitting only one routine at a time.
	//
	// Returns:
	// - (signature, nil) if no error occurred
	// - (nil, model.InsufficientSignaturesError) if not enough shares were collected
	// - (nil, model.InvalidSignatureIncluded) if at least one collected share does not serialize to a valid BLS signature,
	//    or if the constructed signature failed to verify against the group public key and stored message. This post-verification
	//    is required  for safety, as `TrustedAdd` allows adding invalid signatures.
	// - (nil, error) for any other unexpected error.
	Reconstruct() (crypto.Signature, error)
}

RandomBeaconReconstructor encapsulates all methods needed by a Hotstuff leader to validate the beacon votes and reconstruct a beacon signature. The random beacon methods are based on a threshold signature scheme.

type Replicas added in v0.29.0

type Replicas interface {

	// LeaderForView returns the identity of the leader for a given view.
	// CAUTION: per liveness requirement of HotStuff, the leader must be fork-independent.
	//          Therefore, a node retains its proposer view slots even if it is slashed.
	//          Its proposal is simply considered invalid, as it is not from a legitimate participant.
	// Returns the following expected errors for invalid inputs:
	//   - model.ErrViewForUnknownEpoch if no epoch containing the given view is known
	LeaderForView(view uint64) (flow.Identifier, error)

	// QuorumThresholdForView returns the minimum total weight for a supermajority
	// at the given view. This weight threshold is computed using the total weight
	// of the initial committee and is static over the course of an epoch.
	// Returns the following expected errors for invalid inputs:
	//   - model.ErrViewForUnknownEpoch if no epoch containing the given view is known
	QuorumThresholdForView(view uint64) (uint64, error)

	// TimeoutThresholdForView returns the minimum total weight of observed timeout objects
	// required to safely timeout for the given view. This weight threshold is computed
	// using the total weight of the initial committee and is static over the course of
	// an epoch.
	// Returns the following expected errors for invalid inputs:
	//   - model.ErrViewForUnknownEpoch if no epoch containing the given view is known
	TimeoutThresholdForView(view uint64) (uint64, error)

	// Self returns our own node identifier.
	// TODO: ultimately, the own identity of the node is necessary for signing.
	//       Ideally, we would move the method for checking whether an Identifier refers to this node to the signer.
	//       This would require some refactoring of EventHandler (postponed to later)
	Self() flow.Identifier

	// DKG returns the DKG info for epoch given by the input view.
	// Returns the following expected errors for invalid inputs:
	//   - model.ErrViewForUnknownEpoch if no epoch containing the given view is known
	DKG(view uint64) (DKG, error)

	// IdentitiesByEpoch returns a list of the legitimate HotStuff participants for the epoch
	// given by the input view.
	// The returned list of HotStuff participants:
	//   - contains nodes that are allowed to submit votes or timeouts within the given epoch
	//     (un-ejected, non-zero weight at the beginning of the epoch)
	//   - is ordered in the canonical order
	//   - contains no duplicates.
	//
	// CAUTION: DO NOT use this method for validating block proposals.
	// CAUTION: This method considers epochs outside of Previous, Current, Next, w.r.t. the
	// finalized block, to be unknown. https://github.com/onflow/flow-go/issues/4085
	//
	// Returns the following expected errors for invalid inputs:
	//   - model.ErrViewForUnknownEpoch if no epoch containing the given view is known
	//
	// TODO: should return identity skeleton https://github.com/dapperlabs/flow-go/issues/6232
	IdentitiesByEpoch(view uint64) (flow.IdentityList, error)

	// IdentityByEpoch returns the full Identity for specified HotStuff participant.
	// The node must be a legitimate HotStuff participant with NON-ZERO WEIGHT at the specified block.
	// CAUTION: This method considers epochs outside of Previous, Current, Next, w.r.t. the
	// finalized block, to be unknown. https://github.com/onflow/flow-go/issues/4085
	//
	// ERROR conditions:
	//  - model.InvalidSignerError if participantID does NOT correspond to an authorized HotStuff participant at the specified block.
	//
	// Returns the following expected errors for invalid inputs:
	//   - model.ErrViewForUnknownEpoch if no epoch containing the given view is known
	//
	// TODO: should return identity skeleton https://github.com/dapperlabs/flow-go/issues/6232
	IdentityByEpoch(view uint64, participantID flow.Identifier) (*flow.Identity, error)
}

Replicas defines the consensus committee for the purposes of validating votes, timeouts, quorum certificates, and timeout certificates. Any consensus committee member who was authorized to contribute to consensus AT THE BEGINNING of the epoch may produce valid votes and timeouts for the entire epoch, even if they are later ejected. So for validating votes/timeouts we use *ByEpoch methods.

Since the voter committee is considered static over an epoch:

  • we can query identities by view
  • we don't need the full block ancestry prior to validating messages

type SafetyData added in v0.29.0

type SafetyData struct {
	// LockedOneChainView is the head block's view of the newest 1-chain this replica has voted for.
	// The 1-chain can be indirect.
	//     <·· <QC>[B0] <- <QC_B0>[B1] <- [my vote for B1]
	// In the depicted scenario, the replica voted for block B1, which forms a (potentially indirect)
	// 1-chain on top of B0. The replica updated LockedOneChainView to the max of the current value and
	// QC_B0.View = B0.View. Thereby, the safety module guarantees that the replica will not sign
	// a TimeoutObject that would allow a malicious leader to fork below the latest finalized block.
	LockedOneChainView uint64
	// HighestAcknowledgedView is the highest view where we have voted or triggered a timeout
	HighestAcknowledgedView uint64
	// LastTimeout is the last timeout that was produced by this node (may be nil if no timeout occurred yet)
	LastTimeout *model.TimeoutObject
}

type SafetyRules added in v0.29.0

type SafetyRules interface {
	// ProduceVote takes a block proposal and current view, and decides whether to vote for the block.
	// Voting is deterministic meaning voting for same proposal will always result in the same vote.
	// Returns:
	//  * (vote, nil): On the _first_ block for the current view that is safe to vote for.
	//    Subsequently, voter does _not_ vote for any _other_  block with the same (or lower) view.
	//    SafetyRules internally caches and persists its latest vote. As long as the SafetyRules' internal
	//    state remains unchanged, ProduceVote will return its cached for identical inputs.
	//  * (nil, model.NoVoteError): If the safety module decides that it is not safe to vote for the given block.
	//    This is a sentinel error and _expected_ during normal operation.
	// All other errors are unexpected and potential symptoms of uncovered edge cases or corrupted internal state (fatal).
	ProduceVote(proposal *model.Proposal, curView uint64) (*model.Vote, error)
	// ProduceTimeout takes current view, highest locally known QC and TC (optional, must be nil if and
	// only if QC is for previous view) and decides whether to produce timeout for current view.
	// Returns:
	//  * (timeout, nil): It is safe to timeout for current view using newestQC and lastViewTC.
	//  * (nil, model.NoTimeoutError): If replica is not part of the authorized consensus committee (anymore) and
	//    therefore is not authorized to produce a valid timeout object. This sentinel error is _expected_ during
	//    normal operation, e.g. during the grace-period after Epoch switchover or after the replica self-ejected.
	// All other errors are unexpected and potential symptoms of uncovered edge cases or corrupted internal state (fatal).
	ProduceTimeout(curView uint64, newestQC *flow.QuorumCertificate, lastViewTC *flow.TimeoutCertificate) (*model.TimeoutObject, error)
}

SafetyRules enforces all consensus rules that guarantee safety. It produces votes for the given blocks or TimeoutObject for the given views, only if all safety rules are satisfied.

type Signer

type Signer interface {
	// CreateProposal creates a proposal for the given block. No error returns
	// are expected during normal operations (incl. presence of byz. actors).
	CreateProposal(block *model.Block) (*model.Proposal, error)

	// CreateVote creates a vote for the given block. No error returns are
	// expected during normal operations (incl. presence of byz. actors).
	CreateVote(block *model.Block) (*model.Vote, error)

	// CreateTimeout creates a timeout for given view. No errors return are
	// expected during normal operations(incl presence of byz. actors).
	CreateTimeout(curView uint64, newestQC *flow.QuorumCertificate, lastViewTC *flow.TimeoutCertificate) (*model.TimeoutObject, error)
}

Signer is responsible for creating votes, proposals for a given block.

type TimeoutAggregationConsumer added in v0.31.0

type TimeoutAggregationConsumer interface {
	TimeoutAggregationViolationConsumer
	TimeoutCollectorConsumer
}

TimeoutAggregationConsumer consumes outbound notifications produced by Vote Aggregation logic. It is a subset of the notifications produced by consensus participants. Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type TimeoutAggregationViolationConsumer added in v0.31.0

type TimeoutAggregationViolationConsumer interface {
	// OnDoubleTimeoutDetected notifications are produced by the Timeout Aggregation logic
	// whenever a double timeout (same replica producing two different timeouts at the same view) was detected.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnDoubleTimeoutDetected(*model.TimeoutObject, *model.TimeoutObject)

	// OnInvalidTimeoutDetected notifications are produced by the Timeout Aggregation logic
	// whenever an invalid timeout was detected.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnInvalidTimeoutDetected(err model.InvalidTimeoutError)
}

TimeoutAggregationViolationConsumer consumes outbound notifications about Active Pacemaker violations specifically invalid timeouts during processing. Such notifications are produced by the Timeout Aggregation logic.

Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type TimeoutAggregator added in v0.29.0

type TimeoutAggregator interface {
	module.ReadyDoneAware
	module.Startable

	// AddTimeout verifies and aggregates a timeout object.
	// This method can be called concurrently, timeouts will be queued and processed asynchronously.
	AddTimeout(timeoutObject *model.TimeoutObject)

	// PruneUpToView deletes all `TimeoutCollector`s _below_ to the given view, as well as
	// related indices. We only retain and process `TimeoutCollector`s, whose view is equal or larger
	// than `lowestRetainedView`. If `lowestRetainedView` is smaller than the
	// previous value, the previous value is kept and the method call is a NoOp.
	// This value should be set to the latest active view maintained by `Pacemaker`.
	PruneUpToView(lowestRetainedView uint64)
}

TimeoutAggregator verifies and aggregates timeout objects to build timeout certificates [TCs]. When enough timeout objects are collected, it builds a TC and sends it to the EventLoop TimeoutAggregator also detects protocol violation, including invalid timeouts, double timeout, etc and notifies a HotStuff consumer for slashing.

type TimeoutCollector added in v0.29.0

type TimeoutCollector interface {
	// AddTimeout adds a Timeout Object [TO] to the collector.
	// When TOs from strictly more than 1/3 of consensus participants (measured by weight)
	// were collected, the callback for partial TC will be triggered.
	// After collecting TOs from a supermajority, a TC will be created and passed to the EventLoop.
	// Expected error returns during normal operations:
	// * timeoutcollector.ErrTimeoutForIncompatibleView - submitted timeout for incompatible view
	// All other exceptions are symptoms of potential state corruption.
	AddTimeout(timeoutObject *model.TimeoutObject) error

	// View returns the view that this instance is collecting timeouts for.
	// This method is useful when adding the newly created timeout collector to timeout collectors map.
	View() uint64
}

TimeoutCollector collects all timeout objects for a specified view. On the happy path, it generates a TimeoutCertificate when enough timeouts have been collected. The TimeoutCollector is a higher-level structure that orchestrates deduplication, caching and processing of timeouts, delegating those tasks to underlying modules (such as TimeoutProcessor). Implementations of TimeoutCollector must be concurrency safe.

type TimeoutCollectorConsumer added in v0.29.0

type TimeoutCollectorConsumer interface {
	// OnTcConstructedFromTimeouts notifications are produced by the TimeoutProcessor
	// component, whenever it constructs a TC based on TimeoutObjects from a
	// supermajority of consensus participants.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnTcConstructedFromTimeouts(certificate *flow.TimeoutCertificate)

	// OnPartialTcCreated notifications are produced by the TimeoutProcessor
	// component, whenever it collected TimeoutObjects from a superminority
	// of consensus participants for a specific view. Along with the view, it
	// reports the newest QC and TC (for previous view) discovered in process of
	// timeout collection. Per convention, the newest QC is never nil, while
	// the TC for the previous view might be nil.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnPartialTcCreated(view uint64, newestQC *flow.QuorumCertificate, lastViewTC *flow.TimeoutCertificate)

	// OnNewQcDiscovered notifications are produced by the TimeoutCollector
	// component, whenever it discovers new QC included in timeout object.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnNewQcDiscovered(certificate *flow.QuorumCertificate)

	// OnNewTcDiscovered notifications are produced by the TimeoutCollector
	// component, whenever it discovers new TC included in timeout object.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnNewTcDiscovered(certificate *flow.TimeoutCertificate)

	// OnTimeoutProcessed notifications are produced by the Timeout Aggregation logic,
	// each time we successfully ingest a valid timeout.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnTimeoutProcessed(timeout *model.TimeoutObject)
}

TimeoutCollectorConsumer consumes outbound notifications produced by HotStuff's timeout aggregation component. These events are primarily intended for the HotStuff-internal state machine (EventHandler), but might also be relevant to the larger node in which HotStuff is running.

Caution: the events are not strictly ordered by increasing views! The notifications are emitted by concurrent processing logic. Over larger time scales, the emitted events are for statistically increasing views. However, on short time scales there are _no_ monotonicity guarantees w.r.t. the events' views.

Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type TimeoutCollectorFactory added in v0.29.0

type TimeoutCollectorFactory interface {
	// Create is a factory method to generate a TimeoutCollector for a given view
	// Expected error returns during normal operations:
	//  * model.ErrViewForUnknownEpoch no epoch containing the given view is known
	// All other errors should be treated as exceptions.
	Create(view uint64) (TimeoutCollector, error)
}

TimeoutCollectorFactory performs creation of TimeoutCollector for a given view

type TimeoutCollectors added in v0.29.0

type TimeoutCollectors interface {
	// GetOrCreateCollector retrieves the TimeoutCollector for the specified
	// view or creates one if none exists.  When creating a timeout collector,
	// the view is used to query the consensus committee for the respective
	// Epoch the view belongs to.
	// It returns:
	//  -  (collector, true, nil) if no collector can be found by the view, and a new collector was created.
	//  -  (collector, false, nil) if the collector can be found by the view.
	//  -  (nil, false, error) if running into any exception creating the timeout collector.
	// Expected error returns during normal operations:
	//  * mempool.BelowPrunedThresholdError if view is below the pruning threshold
	//  * model.ErrViewForUnknownEpoch if view is not yet pruned but no epoch containing the given view is known
	GetOrCreateCollector(view uint64) (collector TimeoutCollector, created bool, err error)

	// PruneUpToView prunes the timeout collectors with views _below_ the given value, i.e.
	// we only retain and process timeout collectors, whose views are equal or larger than `lowestRetainedView`.
	// If `lowestRetainedView` is smaller than the previous value, the previous value is
	// kept and the method call is a NoOp.
	PruneUpToView(lowestRetainedView uint64)
}

TimeoutCollectors encapsulates the functionality to generate, store and prune `TimeoutCollector` instances (one per view). Its main purpose is to provide a higher-level API to `TimeoutAggregator` for managing and interacting with the view-specific `TimeoutCollector` instances. Implementations are concurrency safe.

type TimeoutProcessor added in v0.29.0

type TimeoutProcessor interface {
	// Process performs processing of single timeout object. This function is safe to call from multiple goroutines.
	// Expected error returns during normal operations:
	// * timeoutcollector.ErrTimeoutForIncompatibleView - submitted timeout for incompatible view
	// * model.InvalidTimeoutError - submitted invalid timeout(invalid structure or invalid signature)
	// * model.DuplicatedSignerError if a timeout from the same signer was previously already added
	//   It does _not necessarily_ imply that the timeout is invalid or the sender is equivocating.
	// All other errors should be treated as exceptions.
	Process(timeout *model.TimeoutObject) error
}

TimeoutProcessor ingests Timeout Objects [TO] for a particular view. It implements the algorithms for validating TOs, orchestrates their low-level aggregation and emits `OnPartialTcCreated` and `OnTcConstructedFromTimeouts` notifications. TimeoutProcessor cannot deduplicate TOs (this should be handled by the higher-level TimeoutCollector) and errors instead. Depending on their implementation, a TimeoutProcessor might drop timeouts or attempt to construct a TC.

type TimeoutProcessorFactory added in v0.29.0

type TimeoutProcessorFactory interface {
	// Create is a factory method to generate a TimeoutProcessor for a given view
	// Expected error returns during normal operations:
	//  * model.ErrViewForUnknownEpoch no epoch containing the given view is known
	// All other errors should be treated as exceptions.
	Create(view uint64) (TimeoutProcessor, error)
}

TimeoutProcessorFactory performs creation of TimeoutProcessor for a given view

type TimeoutSignatureAggregator added in v0.29.0

type TimeoutSignatureAggregator interface {
	// VerifyAndAdd verifies the signature under the stored public keys and adds the signature and the corresponding
	// highest QC to the internal set. Internal set and collected weight is modified iff signature _is_ valid.
	// The total weight of all collected signatures (excluding duplicates) is returned regardless
	// of any returned error.
	// Expected errors during normal operations:
	//  - model.InvalidSignerError if signerID is invalid (not a consensus participant)
	//  - model.DuplicatedSignerError if the signer has been already added
	//  - model.ErrInvalidSignature if signerID is valid but signature is cryptographically invalid
	VerifyAndAdd(signerID flow.Identifier, sig crypto.Signature, newestQCView uint64) (totalWeight uint64, exception error)

	// TotalWeight returns the total weight presented by the collected signatures.
	TotalWeight() uint64

	// View returns the view that this instance is aggregating signatures for.
	View() uint64

	// Aggregate aggregates the signatures and returns with additional data.
	// Aggregated signature will be returned as SigData of timeout certificate.
	// Caller can be sure that resulting signature is valid.
	// Expected errors during normal operations:
	//  - model.InsufficientSignaturesError if no signatures have been added yet
	Aggregate() (signersInfo []TimeoutSignerInfo, aggregatedSig crypto.Signature, exception error)
}

TimeoutSignatureAggregator aggregates timeout signatures for one particular view. When instantiating a TimeoutSignatureAggregator, the following information is supplied:

  • The view for which the aggregator collects timeouts.
  • For each replicas that is authorized to send a timeout at this particular view: the node ID, public staking keys, and weight

Timeouts for other views or from non-authorized replicas are rejected. In their TimeoutObjects, replicas include a signature over the pair (view, newestQCView), where `view` is the view number the timeout is for and `newestQCView` is the view of the newest QC known to the replica. TimeoutSignatureAggregator collects these signatures, internally tracks the total weight of all collected signatures. Note that in general the signed messages are different, which makes the aggregation a comparatively expensive operation. Upon calling `Aggregate`, the TimeoutSignatureAggregator aggregates all valid signatures collected up to this point. The aggregate signature is guaranteed to be correct, as only valid signatures are excepted as inputs. TimeoutSignatureAggregator internally tracks the total weight of all collected signatures. Implementations must be concurrency safe.

type TimeoutSignerInfo added in v0.29.0

type TimeoutSignerInfo struct {
	NewestQCView uint64
	Signer       flow.Identifier
}

TimeoutSignerInfo is a helper structure that stores the QC views that each signer contributed to a TC. Used as result of TimeoutSignatureAggregator.Aggregate()

type Validator

type Validator interface {

	// ValidateQC checks the validity of a QC.
	// During normal operations, the following error returns are expected:
	//  * model.InvalidQCError if the QC is invalid
	//  * model.ErrViewForUnknownEpoch if the QC refers unknown epoch
	ValidateQC(qc *flow.QuorumCertificate) error

	// ValidateTC checks the validity of a TC.
	// During normal operations, the following error returns are expected:
	//  * model.InvalidTCError if the TC is invalid
	//  * model.ErrViewForUnknownEpoch if the TC refers unknown epoch
	ValidateTC(tc *flow.TimeoutCertificate) error

	// ValidateProposal checks the validity of a proposal.
	// During normal operations, the following error returns are expected:
	//  * model.InvalidProposalError if the block is invalid
	//  * model.ErrViewForUnknownEpoch if the proposal refers unknown epoch
	ValidateProposal(proposal *model.Proposal) error

	// ValidateVote checks the validity of a vote.
	// Returns the full entity for the voter. During normal operations,
	// the following errors are expected:
	//  * model.InvalidVoteError for invalid votes
	//  * model.ErrViewForUnknownEpoch if the vote refers unknown epoch
	ValidateVote(vote *model.Vote) (*flow.Identity, error)
}

Validator provides functions to validate QC, proposals and votes.

type Verifier

type Verifier interface {

	// VerifyVote checks the cryptographic validity of a vote's `SigData` w.r.t.
	// the view and blockID. It is the responsibility of the calling code to ensure
	// that `voter` is authorized to vote.
	// Return values:
	//  * nil if `sigData` is cryptographically valid
	//  * model.InvalidFormatError if the signature has an incompatible format.
	//  * model.ErrInvalidSignature is the signature is invalid
	//  * model.InvalidSignerError is only relevant for extended signature schemes,
	//    where special signing authority is only given to a _subset_ of consensus
	//    participants (e.g. random beacon). In case a participant signed despite not
	//    being authorized, an InvalidSignerError is returned.
	//  * model.ErrViewForUnknownEpoch is only relevant for extended signature schemes,
	//    where querying of DKG might fail if no epoch containing the given view is known.
	//  * unexpected errors should be treated as symptoms of bugs or uncovered
	//    edge cases in the logic (i.e. as fatal)
	VerifyVote(voter *flow.Identity, sigData []byte, view uint64, blockID flow.Identifier) error

	// VerifyQC checks the cryptographic validity of a QC's `SigData` w.r.t. the
	// given view and blockID. It is the responsibility of the calling code to ensure that
	// all `signers` are authorized, without duplicates.
	// Return values:
	//  * nil if `sigData` is cryptographically valid
	//  * model.InvalidFormatError if `sigData` has an incompatible format
	//  * model.InsufficientSignaturesError if `signers is empty.
	//    Depending on the order of checks in the higher-level logic this error might
	//    be an indicator of a external byzantine input or an internal bug.
	//  * model.ErrInvalidSignature if a signature is invalid
	//  * model.InvalidSignerError is only relevant for extended signature schemes,
	//    where special signing authority is only given to a _subset_ of consensus
	//    participants (e.g. random beacon). In case a participant signed despite not
	//    being authorized, an InvalidSignerError is returned.
	//  * model.ErrViewForUnknownEpoch is only relevant for extended signature schemes,
	//    where querying of DKG might fail if no epoch containing the given view is known.
	//  * unexpected errors should be treated as symptoms of bugs or uncovered
	//	  edge cases in the logic (i.e. as fatal)
	VerifyQC(signers flow.IdentityList, sigData []byte, view uint64, blockID flow.Identifier) error

	// VerifyTC checks cryptographic validity of the TC's `sigData` w.r.t. the
	// given view. It is the responsibility of the calling code to ensure
	// that all `signers` are authorized, without duplicates. Return values:
	//  * nil if `sigData` is cryptographically valid
	//  * model.InsufficientSignaturesError if `signers is empty.
	//  * model.InvalidFormatError if `signers`/`highQCViews` have differing lengths
	//  * model.ErrInvalidSignature if a signature is invalid
	//  * unexpected errors should be treated as symptoms of bugs or uncovered
	//	  edge cases in the logic (i.e. as fatal)
	VerifyTC(signers flow.IdentityList, sigData []byte, view uint64, highQCViews []uint64) error
}

Verifier is the component responsible for the cryptographic integrity of votes, proposals and QC's against the block they are signing. Overall, there are two criteria for the validity of a vote and QC:

(1) the signer ID(s) must correspond to authorized consensus participants
(2) the signature must be cryptographically valid.

Note that Verifier only implements (2). This API design allows to decouple

 (i)  the common logic for checking that a super-majority of the consensus
      committee voted
 (ii) the handling of combined staking+RandomBeacon votes (consensus nodes)
	  vs only staking votes (collector nodes)

On the one hand, this API design makes code less concise, as the two checks are now distributed over API boundaries. On the other hand, we can avoid repeated Identity lookups in the implementation, which increases performance.

type VerifyingVoteProcessor added in v0.23.9

type VerifyingVoteProcessor interface {
	VoteProcessor

	// Block returns which block that will be used to collector votes for. Transition to VerifyingVoteCollector can occur only
	// when we have received block proposal so this information has to be available.
	Block() *model.Block
}

VerifyingVoteProcessor is a VoteProcessor that attempts to construct a QC for the given block.

type VoteAggregationConsumer added in v0.31.0

type VoteAggregationConsumer interface {
	VoteAggregationViolationConsumer
	VoteCollectorConsumer
}

VoteAggregationConsumer consumes outbound notifications produced by Vote Aggregation logic. It is a subset of the notifications produced by consensus participants. Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type VoteAggregationViolationConsumer added in v0.31.0

type VoteAggregationViolationConsumer interface {
	// OnDoubleVotingDetected notifications are produced by the Vote Aggregation logic
	// whenever a double voting (same voter voting for different blocks at the same view) was detected.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnDoubleVotingDetected(*model.Vote, *model.Vote)

	// OnInvalidVoteDetected notifications are produced by the Vote Aggregation logic
	// whenever an invalid vote was detected.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnInvalidVoteDetected(err model.InvalidVoteError)

	// OnVoteForInvalidBlockDetected notifications are produced by the Vote Aggregation logic
	// whenever vote for invalid proposal was detected.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnVoteForInvalidBlockDetected(vote *model.Vote, invalidProposal *model.Proposal)
}

VoteAggregationViolationConsumer consumes outbound notifications about HotStuff-protocol violations specifically invalid votes during processing. Such notifications are produced by the Vote Aggregation logic.

Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type VoteAggregator

type VoteAggregator interface {
	module.ReadyDoneAware
	module.Startable

	// AddVote verifies and aggregates a vote.
	// The voting block could either be known or unknown.
	// If the voting block is unknown, the vote won't be processed until AddBlock is called with the block.
	// This method can be called concurrently, votes will be queued and processed asynchronously.
	AddVote(vote *model.Vote)

	// AddBlock notifies the VoteAggregator that it should start processing votes for the given block.
	// The input block is queued internally within the `VoteAggregator` and processed _asynchronously_
	// by the VoteAggregator's internal worker routines.
	// CAUTION: we expect that the input block's validity has been confirmed prior to calling AddBlock,
	// including the proposer's signature. Otherwise, VoteAggregator might crash or exhibit undefined
	// behaviour.
	AddBlock(block *model.Proposal)

	// InvalidBlock notifies the VoteAggregator about an invalid proposal, so that it
	// can process votes for the invalid block and slash the voters.
	// No errors are expected during normal operations
	InvalidBlock(block *model.Proposal) error

	// PruneUpToView deletes all votes _below_ to the given view, as well as
	// related indices. We only retain and process whose view is equal or larger
	// than `lowestRetainedView`. If `lowestRetainedView` is smaller than the
	// previous value, the previous value is kept and the method call is a NoOp.
	PruneUpToView(view uint64)
}

VoteAggregator verifies and aggregates votes to build QC. When enough votes have been collected, it builds a QC and send it to the EventLoop VoteAggregator also detects protocol violation, including invalid votes, double voting etc, and notifies a HotStuff consumer for slashing.

type VoteCollector added in v0.23.9

type VoteCollector interface {
	// ProcessBlock performs validation of block signature and processes block with respected collector.
	// Calling this function will mark conflicting collector as stale and change state of valid collectors
	// It returns nil if the block is valid.
	// It returns model.InvalidProposalError if block is invalid.
	// It returns other error if there is exception processing the block.
	ProcessBlock(block *model.Proposal) error

	// AddVote adds a vote to the collector
	// When enough votes have been added to produce a QC, the QC will be created asynchronously, and
	// passed to EventLoop through a callback.
	// No errors are expected during normal operations.
	AddVote(vote *model.Vote) error

	// RegisterVoteConsumer registers a VoteConsumer. Upon registration, the collector
	// feeds all cached votes into the consumer in the order they arrived.
	// CAUTION, VoteConsumer implementations must be
	//  * NON-BLOCKING and consume the votes without noteworthy delay, and
	//  * CONCURRENCY SAFE
	RegisterVoteConsumer(consumer VoteConsumer)

	// View returns the view that this instance is collecting votes for.
	// This method is useful when adding the newly created vote collector to vote collectors map.
	View() uint64

	// Status returns the status of the vote collector
	Status() VoteCollectorStatus
}

VoteCollector collects all votes for a specified view. On the happy path, it generates a QC when enough votes have been collected. The VoteCollector internally delegates the vote-format specific processing to the VoteProcessor.

type VoteCollectorConsumer added in v0.31.0

type VoteCollectorConsumer interface {
	// OnQcConstructedFromVotes notifications are produced by the VoteAggregator
	// component, whenever it constructs a QC from votes.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnQcConstructedFromVotes(*flow.QuorumCertificate)

	// OnVoteProcessed notifications are produced by the Vote Aggregation logic, each time
	// we successfully ingest a valid vote.
	// Prerequisites:
	// Implementation must be concurrency safe; Non-blocking;
	// and must handle repetition of the same events (with some processing overhead).
	OnVoteProcessed(vote *model.Vote)
}

VoteCollectorConsumer consumes outbound notifications produced by HotStuff's vote aggregation component. These events are primarily intended for the HotStuff-internal state machine (EventHandler), but might also be relevant to the larger node in which HotStuff is running.

Implementations must:

  • be concurrency safe
  • be non-blocking
  • handle repetition of the same events (with some processing overhead).

type VoteCollectorStatus added in v0.23.9

type VoteCollectorStatus int

VoteCollectorStatus indicates the VoteCollector's status It has three different status.

const (
	// VoteCollectorStatusCaching is for the status when the block has not been received.
	// The vote collector in this status will cache all the votes without verifying them
	VoteCollectorStatusCaching VoteCollectorStatus = iota

	// VoteCollectorStatusVerifying is for the status when the block has been received,
	// and is able to process all votes for it.
	VoteCollectorStatusVerifying

	// VoteCollectorStatusInvalid is for the status when the block has been verified and
	// is invalid. All votes to this block will be collected to slash the voter.
	VoteCollectorStatusInvalid
)

func (VoteCollectorStatus) String added in v0.23.9

func (ps VoteCollectorStatus) String() string

type VoteCollectors added in v0.23.9

type VoteCollectors interface {
	module.ReadyDoneAware
	module.Startable

	// GetOrCreateCollector retrieves the hotstuff.VoteCollector for the specified
	// view or creates one if none exists.
	// When creating a vote collector, the view will be used to get epoch by view, then create the random beacon
	// signer object by epoch, because epoch determines DKG, which determines random beacon committee.
	// It returns:
	//  -  (collector, true, nil) if no collector can be found by the view, and a new collector was created.
	//  -  (collector, false, nil) if the collector can be found by the view
	//  -  (nil, false, error) if running into any exception creating the vote collector state machine
	// Expected error returns during normal operations:
	//  * mempool.BelowPrunedThresholdError - in case view is lower than last pruned view
	GetOrCreateCollector(view uint64) (collector VoteCollector, created bool, err error)

	// PruneUpToView prunes the vote collectors with views _below_ the given value, i.e.
	// we only retain and process whose view is equal or larger than `lowestRetainedView`.
	// If `lowestRetainedView` is smaller than the previous value, the previous value is
	// kept and the method call is a NoOp.
	PruneUpToView(lowestRetainedView uint64)
}

VoteCollectors is an interface which allows VoteAggregator to interact with collectors structured by view. Implementations of this interface are responsible for state transitions of `VoteCollector`s and pruning of stale and outdated collectors by view.

type VoteConsumer added in v0.23.9

type VoteConsumer func(vote *model.Vote)

VoteConsumer consumes all votes for one specific view. It is registered with the `VoteCollector` for the respective view. Upon registration, the `VoteCollector` feeds votes into the consumer in the order they are received (already cached votes as well as votes received in the future). Only votes that pass de-duplication and equivocation detection are passed on. CAUTION, VoteConsumer implementations must be

  • NON-BLOCKING and consume the votes without noteworthy delay, and
  • CONCURRENCY SAFE

type VoteProcessor added in v0.23.9

type VoteProcessor interface {
	// Process performs processing of single vote. This function is safe to call from multiple goroutines.
	// Expected error returns during normal operations:
	// * VoteForIncompatibleBlockError - submitted vote for incompatible block
	// * VoteForIncompatibleViewError - submitted vote for incompatible view
	// * model.InvalidVoteError - submitted vote with invalid signature
	// * model.DuplicatedSignerError - vote from a signer whose vote was previously already processed
	// All other errors should be treated as exceptions.
	Process(vote *model.Vote) error

	// Status returns the status of the vote processor
	Status() VoteCollectorStatus
}

VoteProcessor processes votes. It implements the vote-format specific processing logic. Depending on their implementation, a VoteProcessor might drop votes or attempt to construct a QC.

type VoteProcessorFactory added in v0.23.9

type VoteProcessorFactory interface {
	// Create instantiates a VerifyingVoteProcessor for processing votes for a specific proposal.
	// Caller can be sure that proposal vote was successfully verified and processed.
	// Expected error returns during normal operations:
	// * model.InvalidProposalError - proposal has invalid proposer vote
	Create(log zerolog.Logger, proposal *model.Proposal) (VerifyingVoteProcessor, error)
}

VoteProcessorFactory is a factory that can be used to create a verifying vote processors for a specific proposal. Depending on factory implementation it will return processors for consensus or collection clusters

type WeightedSignatureAggregator added in v0.23.9

type WeightedSignatureAggregator interface {
	// Verify verifies the signature under the stored public keys and message.
	// Expected errors during normal operations:
	//  - model.InvalidSignerError if signerID is invalid (not a consensus participant)
	//  - model.ErrInvalidSignature if signerID is valid but signature is cryptographically invalid
	Verify(signerID flow.Identifier, sig crypto.Signature) error

	// TrustedAdd adds a signature to the internal set of signatures and adds the signer's
	// weight to the total collected weight, iff the signature is _not_ a duplicate. The
	// total weight of all collected signatures (excluding duplicates) is returned regardless
	// of any returned error.
	// Expected errors during normal operations:
	//  - model.InvalidSignerError if signerID is invalid (not a consensus participant)
	//  - model.DuplicatedSignerError if the signer has been already added
	TrustedAdd(signerID flow.Identifier, sig crypto.Signature) (totalWeight uint64, exception error)

	// TotalWeight returns the total weight presented by the collected signatures.
	TotalWeight() uint64

	// Aggregate aggregates the signatures and returns the aggregated signature.
	// The function performs a final verification and errors if the aggregated signature is invalid. This is
	// required for the function safety since `TrustedAdd` allows adding invalid signatures.
	// The function errors with:
	//   - model.InsufficientSignaturesError if no signatures have been added yet
	//   - model.InvalidSignatureIncludedError if:
	//     -- some signature(s), included via TrustedAdd, fail to deserialize (regardless of the aggregated public key)
	//     -- or all signatures deserialize correctly but some signature(s), included via TrustedAdd, are
	//       invalid (while aggregated public key is valid)
	//   - model.InvalidAggregatedKeyError if all signatures deserialize correctly but the signer's
	//     staking public keys sum up to an invalid key (BLS identity public key).
	//     Any aggregated signature would fail the cryptographic verification under the identity public
	//     key and therefore such signature is considered invalid. Such scenario can only happen if
	//     staking public keys of signers were forged to add up to the identity public key.
	//     Under the assumption that all staking key PoPs are valid, this error case can only
	//     happen if all signers are malicious and colluding. If there is at least one honest signer,
	//     there is a negligible probability that the aggregated key is identity.
	//
	// The function is thread-safe.
	Aggregate() (flow.IdentifierList, []byte, error)
}

WeightedSignatureAggregator aggregates signatures of the same signature scheme and the same message from different signers. The public keys and message are agreed upon upfront. It is also recommended to only aggregate signatures generated with keys representing equivalent security-bit level. Furthermore, a weight [unsigned int64] is assigned to each signer ID. The WeightedSignatureAggregator internally tracks the total weight of all collected signatures. Implementations must be concurrency safe.

type Workerpool added in v0.23.9

type Workerpool interface {
	Workers

	// StopWait stops the worker pool and waits for all queued tasks to
	// complete.  No additional tasks may be submitted, but all pending tasks are
	// executed by workers before this function returns.
	StopWait()
}

Workerpool adds the functionality to terminate the workers to the Workers interface.

type Workers added in v0.23.9

type Workers interface {
	// Submit enqueues a function for a worker to execute. Submit will not block
	// regardless of the number of tasks submitted. Each task is immediately
	// given to an available worker or queued otherwise. Tasks are processed in
	// FiFO order.
	Submit(task func())
}

Workers queues and processes submitted tasks. We explicitly do not expose any functionality to terminate the worker pool.

Directories

Path Synopsis
(c) 2020 Dapper Labs - ALL RIGHTS RESERVED
(c) 2020 Dapper Labs - ALL RIGHTS RESERVED
Package cruisectl implements a "cruise control" system for Flow by adjusting nodes' latest ProposalTiming in response to changes in the measured view rate and target epoch switchover time.
Package cruisectl implements a "cruise control" system for Flow by adjusting nodes' latest ProposalTiming in response to changes in the measured view rate and target epoch switchover time.

Jump to

Keyboard shortcuts

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