consensus

package
v1.10.2 Latest Latest
Warning

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

Go to latest
Published: Jan 29, 2021 License: MIT Imports: 40 Imported by: 0

README

Consensus package includes the Harmony BFT consensus protocol code, which uses BLS-based multi-signature to cosign the new block. The details are in Harmony's new consensus protocol design.

Introduction to Harmony BFT with BLS signatures

Harmony BFT consensus protocol consist of normal mode and view changing mode which is same as the PBFT(practical byzantine fault tolerance) protocol. The difference is we use the BLS aggregated signature to reduce O(N^2) communications to O(N), which is more efficient and scalable to traditional PBFT. For brevity, we will still call the whole process as PBFT.

Normal mode

To reach the consensus of the next block, there are 3 phases: announce(i.e. pre-prepare in PBFT), prepare and commit.

  • Announce(leader): The leader broadcasts ANNOUNCE message along with candidate of the next block.
  • Prepare(validator): The validator will validate the block sent by leader and send PREPARE message; if the block is invalid, the validator will propose view change. If the prepare timeout, the validator will also propose view change.
  • Prepared(leader): The leader will collect 2f+1 PREPARE message including itself and broadcast PREPARED message with the aggregated signature
  • Commit(validator): The validator will check the validity of aggregated signature (# of signatures >= 2f+1) and send COMMIT message; if the commit timeout, the validator will also propose view change.
  • Committed(leader): The leader will collect 2f+1 COMMIT message including itself and broadcast COMMITTED message with the aggregated signature
  • Finalize(leader and validators): Both the leader and validators will finalize the block into blockchain together with 2f+1 aggregated signatures.

View changing mode

  • ViewChange(validator): whenever a validator receives invalid block/signature from the leader, it should send VIEWCHANGE message with view v+1 together with its own prepared message(>=2f+1 aggregated prepare signatures) from previous views.
  • NewView(new leader): when the new leader (uniquely determined) collect enough (2f+1) view change messages, it broadcasts the NEWVIEW message with aggregated VIEWCHANGE signatures.
  • During the view changing process, if the new leader not send NEWVIEW message on time, the validator will propose ViewChange for the next view v+2 and so on...

State Machine

The whole process of PBFT can be described as a state machine. We don't separate the roles of leader and validators, instead we use PBFTState structure to describe the role and phase of a given node who is joining the consensus process. When a node receives a new message from its peer, its state will be updated. i.e. pbft_state --(upon receive new PBFTMessage)--> new_pbft_state. Thus the most nature and clear way is to describe the whole process as state machine.

// PBFTState holds the state of a node in PBFT process
type PBFTState struct {
   IsLeader bool
   phase PBFTPhase // Announce, Prepare(d), Commit(ted)
   ...

}

// PBFTLog stores the data in PBFT process, it will be used in different phases in order to determine whether a new PBFTMessage is valid or not.
type PBFTLog struct {
    blocks []*types.Block
    messages []*PBFTMessage
}

// entry point and main loop;
// in each loop, the node will receive PBFT message from peers with timeout,
// then update its state accordingly. handleMessageUpdate function handles various kinds of messages and update its state
// it will also send new PBFT message (if not null) to its peers.
// in the same loop, the node will also check whether it should send view changing message to new leader
// finally, it will periodically try to publish new block into blockchain
func (consensus *Consensus) Start(stopChan chan struct{}, stoppedChan chan struct{}) {
    defer close(stoppedChan)
    tick := time.NewTicker(blockDuration)
    consensus.idleTimeout.Start()
    for {
        select {
        default:
            msg := consensus.recvWithTimeout(receiveTimeout)
            consensus.handleMessageUpdate(msg)
            if consensus.idleTimeout.CheckExpire() {
                consensus.startViewChange(consensus.viewID + 1)
            }
            if consensus.commitTimeout.CheckExpire() {
                consensus.startViewChange(consensus.viewID + 1)
            }
            if consensus.viewChangeTimeout.CheckExpire() {
                if consensus.mode.Mode() == Normal {
                    continue
                }
                viewID := consensus.mode.ViewID()
                consensus.startViewChange(viewID + 1)
            }
        case <-tick.C:
            consensus.tryPublishBlock()
        case <-stopChan:
            return
        }
    }

}

Documentation

Index

Constants

View Source
const (
	// CommitSigSenderTimeout is the timeout for sending the commit sig to finish block proposal
	CommitSigSenderTimeout = 10 * time.Second
	// CommitSigReceiverTimeout is the timeout for the receiving side of the commit sig
	// if timeout, the receiver should instead ready directly from db for the commit sig
	CommitSigReceiverTimeout = 8 * time.Second
)

timeout constant

View Source
const MaxBlockNumDiff = 100

MaxBlockNumDiff limits the received block number to only 100 further from the current block number

View Source
const MaxViewIDDiff = 249

MaxViewIDDiff limits the received view ID to only 249 further from the current view ID

View Source
const (
	// RetryIntervalInSec is the interval for message retry
	RetryIntervalInSec = 7
)
View Source
const (
	// ValidPayloadLength is the valid length for viewchange payload
	ValidPayloadLength = 32 + bls.BLSSignatureSizeInBytes
)

Variables

View Source
var (
	// NIL is the m2 type message, which suppose to be nil/empty, however
	// we cannot sign on empty message, instead we sign on some default "nil" message
	// to indicate there is no prepared message received when we start view change
	NIL = []byte{0x01}
)

Functions

This section is empty.

Types

type BlockVerifierFunc added in v1.10.0

type BlockVerifierFunc func(*types.Block) error

BlockVerifierFunc is a function used to verify the block

type Consensus

type Consensus struct {
	Decider quorum.Decider
	// FBFTLog stores the pbft messages and blocks during FBFT process
	FBFTLog *FBFTLog

	// The blockchain this consensus is working on
	Blockchain *core.BlockChain
	// Minimal number of peers in the shard
	// If the number of validators is less than minPeers, the consensus won't start
	MinPeers int

	// the publickey of leader
	LeaderPubKey *bls.PublicKeyWrapper

	// Shard Id which this node belongs to
	ShardID uint32
	// IgnoreViewIDCheck determines whether to ignore viewID check
	IgnoreViewIDCheck *abool.AtomicBool

	// Signal channel for proposing a new block and start new consensus
	ReadySignal chan ProposalType
	// Channel to send full commit signatures to finish new block proposal
	CommitSigChannel chan []byte
	// The post-consensus job func passed from Node object
	// Called when consensus on a new block is done
	PostConsensusJob func(*types.Block) error
	// The verifier func passed from Node object
	BlockVerifier BlockVerifierFunc
	// verified block to state sync broadcast
	VerifiedNewBlock chan *types.Block
	// will trigger state syncing when blockNum is low
	BlockNumLowChan chan struct{}
	// Channel for DRG protocol to send pRnd (preimage of randomness resulting from combined vrf
	// randomnesses) to consensus. The first 32 bytes are randomness, the rest is for bitmap.
	PRndChannel chan []byte
	// Channel for DRG protocol to send VDF. The first 516 bytes are the VDF/Proof and the last 32
	// bytes are the seed for deriving VDF
	RndChannel chan [vdfAndSeedSize]byte

	// Have a dedicated reader thread pull from this chan, like in node
	SlashChan chan slash.Record
	// How long in second the leader needs to wait to propose a new block.
	BlockPeriod time.Duration
	// The time due for next block proposal
	NextBlockDue time.Time
	// Temporary flag to control whether aggregate signature signing is enabled
	AggregateSig bool
	// contains filtered or unexported fields
}

Consensus is the main struct with all states and data related to consensus process.

func New

func New(
	host p2p.Host, shard uint32, leader p2p.Peer, multiBLSPriKey multibls.PrivateKeys,
	Decider quorum.Decider,
) (*Consensus, error)

New create a new Consensus record

func NewFaker

func NewFaker() *Consensus

NewFaker returns a faker consensus.

func (*Consensus) AddPubkeyMetrics added in v1.10.0

func (consensus *Consensus) AddPubkeyMetrics()

AddPubkeyMetrics add the list of blskeys to prometheus metrics

func (*Consensus) BlockCommitSigs added in v1.10.0

func (consensus *Consensus) BlockCommitSigs(blockNum uint64) ([]byte, error)

BlockCommitSigs returns the byte array of aggregated commit signature and bitmap signed on the block

func (*Consensus) BlocksNotSynchronized

func (consensus *Consensus) BlocksNotSynchronized()

BlocksNotSynchronized lets the main loop know that block is not synchronized

func (*Consensus) BlocksSynchronized

func (consensus *Consensus) BlocksSynchronized()

BlocksSynchronized lets the main loop know that block synchronization finished thus the blockchain is likely to be up to date.

func (*Consensus) FinishFinalityCount added in v1.10.0

func (consensus *Consensus) FinishFinalityCount()

FinishFinalityCount calculate the current finality

func (*Consensus) GenerateVdfAndProof

func (consensus *Consensus) GenerateVdfAndProof(newBlock *types.Block, vrfBlockNumbers []uint64)

GenerateVdfAndProof generates new VDF/Proof from VRFs in the current epoch

func (*Consensus) GenerateVrfAndProof

func (consensus *Consensus) GenerateVrfAndProof(newBlock *types.Block, vrfBlockNumbers []uint64) []uint64

GenerateVrfAndProof generates new VRF/Proof from hash of previous block

func (*Consensus) GetBlockNum added in v1.10.0

func (c *Consensus) GetBlockNum() uint64

GetBlockNum return the current blockNum of the consensus struct

func (*Consensus) GetConsensusLeaderPrivateKey added in v1.2.4

func (consensus *Consensus) GetConsensusLeaderPrivateKey() (*bls.PrivateKeyWrapper, error)

GetConsensusLeaderPrivateKey returns consensus leader private key if node is the leader

func (*Consensus) GetConsensusMode added in v1.10.0

func (c *Consensus) GetConsensusMode() string

GetConsensusMode returns the current mode of the consensus

func (*Consensus) GetConsensusPhase added in v1.10.0

func (c *Consensus) GetConsensusPhase() string

GetConsensusPhase returns the current phase of the consensus

func (*Consensus) GetCurBlockViewID added in v1.10.0

func (c *Consensus) GetCurBlockViewID() uint64

GetCurBlockViewID returns the current view ID of the consensus

func (*Consensus) GetFinality added in v1.10.0

func (consensus *Consensus) GetFinality() int64

GetFinality returns the finality time in milliseconds of previous consensus

func (*Consensus) GetLastMileBlockIter added in v1.10.0

func (consensus *Consensus) GetLastMileBlockIter(bnStart uint64) (*LastMileBlockIter, error)

GetLastMileBlockIter get the iterator of the last mile blocks starting from number bnStart

func (*Consensus) GetLeaderPrivateKey added in v1.2.4

func (consensus *Consensus) GetLeaderPrivateKey(leaderKey *bls_core.PublicKey) (*bls.PrivateKeyWrapper, error)

GetLeaderPrivateKey returns leader private key if node is the leader

func (*Consensus) GetNextRnd

func (consensus *Consensus) GetNextRnd() ([vdFAndProofSize]byte, [32]byte, error)

GetNextRnd returns the oldest available randomness along with the hash of the block there randomness preimage is committed.

func (*Consensus) GetPublicKeys added in v1.10.0

func (consensus *Consensus) GetPublicKeys() multibls.PublicKeys

GetPublicKeys returns the public keys

func (*Consensus) GetViewChangingID added in v1.10.0

func (c *Consensus) GetViewChangingID() uint64

GetViewChangingID returns the current view changing ID of the consensus

func (*Consensus) HandleMessageUpdate added in v1.10.0

func (consensus *Consensus) HandleMessageUpdate(ctx context.Context, msg *msg_pb.Message, senderKey *bls.SerializedPublicKey) error

HandleMessageUpdate will update the consensus state according to received message

func (*Consensus) IsLeader

func (consensus *Consensus) IsLeader() bool

IsLeader check if the node is a leader or not by comparing the public key of the node with the leader public key

func (*Consensus) IsValidatorInCommittee

func (consensus *Consensus) IsValidatorInCommittee(pubKey bls.SerializedPublicKey) bool

IsValidatorInCommittee returns whether the given validator BLS address is part of my committee

func (*Consensus) IsViewChangingMode added in v1.10.0

func (consensus *Consensus) IsViewChangingMode() bool

IsViewChangingMode return true if curernt mode is viewchanging

func (*Consensus) Mode

func (consensus *Consensus) Mode() Mode

Mode returns the mode of consensus

func (*Consensus) NeedsRandomNumberGeneration

func (consensus *Consensus) NeedsRandomNumberGeneration(epoch *big.Int) bool

NeedsRandomNumberGeneration returns true if the current epoch needs random number generation

func (*Consensus) NumSignaturesIncludedInBlock added in v1.10.0

func (consensus *Consensus) NumSignaturesIncludedInBlock(block *types.Block) uint32

NumSignaturesIncludedInBlock returns the number of signatures included in the block

func (*Consensus) ParseFBFTMessage added in v1.10.0

func (consensus *Consensus) ParseFBFTMessage(msg *msg_pb.Message) (*FBFTMessage, error)

ParseFBFTMessage parses FBFT message into FBFTMessage structure

func (*Consensus) ReadSignatureBitmapPayload

func (consensus *Consensus) ReadSignatureBitmapPayload(
	recvPayload []byte, offset int,
) (*bls_core.Sign, *bls_cosi.Mask, error)

ReadSignatureBitmapPayload read the payload for signature and bitmap; offset is the beginning position of reading

func (*Consensus) RegisterPRndChannel

func (consensus *Consensus) RegisterPRndChannel(pRndChannel chan []byte)

RegisterPRndChannel registers the channel for receiving randomness preimage from DRG protocol

func (*Consensus) RegisterRndChannel

func (consensus *Consensus) RegisterRndChannel(rndChannel chan [548]byte)

RegisterRndChannel registers the channel for receiving final randomness from DRG protocol

func (*Consensus) ResetState

func (consensus *Consensus) ResetState()

ResetState resets the state of the consensus

func (*Consensus) ResetViewChangeState

func (consensus *Consensus) ResetViewChangeState()

ResetViewChangeState resets the view change structure

func (*Consensus) SetBlockNum

func (consensus *Consensus) SetBlockNum(blockNum uint64)

SetBlockNum sets the blockNum in consensus object, called at node bootstrap

func (*Consensus) SetBlockVerifier added in v1.10.0

func (consensus *Consensus) SetBlockVerifier(verifier BlockVerifierFunc)

SetBlockVerifier sets the block verifier

func (*Consensus) SetCommitDelay

func (consensus *Consensus) SetCommitDelay(delay time.Duration)

SetCommitDelay sets the commit message delay. If set to non-zero, validator delays commit message by the amount.

func (*Consensus) SetCurBlockViewID added in v1.10.0

func (consensus *Consensus) SetCurBlockViewID(viewID uint64)

SetCurBlockViewID set the current view ID

func (*Consensus) SetMode

func (consensus *Consensus) SetMode(m Mode)

SetMode sets the mode of consensus

func (*Consensus) SetViewChangingID added in v1.10.0

func (consensus *Consensus) SetViewChangingID(viewID uint64)

SetViewChangingID set the current view change ID

func (*Consensus) SetViewIDs added in v1.10.0

func (consensus *Consensus) SetViewIDs(height uint64)

SetViewIDs set both current view ID and view changing ID to the height of the blockchain. It is used during client startup to recover the state

func (*Consensus) SetupForNewConsensus added in v1.10.0

func (consensus *Consensus) SetupForNewConsensus(blk *types.Block, committedMsg *FBFTMessage)

SetupForNewConsensus sets the state for new consensus

func (*Consensus) Start

func (consensus *Consensus) Start(
	blockChannel chan *types.Block, stopChan, stoppedChan, startChannel chan struct{},
)

Start waits for the next new block and run consensus

func (*Consensus) StartFinalityCount added in v1.10.0

func (consensus *Consensus) StartFinalityCount()

StartFinalityCount set the finality counter to current time

func (*Consensus) UpdateBitmaps added in v1.10.0

func (consensus *Consensus) UpdateBitmaps()

UpdateBitmaps update the bitmaps for prepare and commit phase

func (*Consensus) UpdateConsensusInformation

func (consensus *Consensus) UpdateConsensusInformation() Mode

UpdateConsensusInformation will update shard information (epoch, publicKeys, blockNum, viewID) based on the local blockchain. It is called in two cases for now: 1. consensus object initialization. because of current dependency where chainreader is only available after node is initialized; node is only available after consensus is initialized we need call this function separately after create consensus object 2. after state syncing is finished It will return the mode: (a) node not in committed: Listening mode (b) node in committed but has any err during processing: Syncing mode (c) node in committed and everything looks good: Normal mode

func (*Consensus) UpdateLeaderMetrics added in v1.10.0

func (consensus *Consensus) UpdateLeaderMetrics(numCommits float64, blockNum float64)

UpdateLeaderMetrics will udpate leader metrics

func (*Consensus) UpdatePublicKeys

func (consensus *Consensus) UpdatePublicKeys(pubKeys []bls_cosi.PublicKeyWrapper) int64

UpdatePublicKeys updates the PublicKeys for quorum on current subcommittee, protected by a mutex

func (*Consensus) UpdateValidatorMetrics added in v1.10.0

func (consensus *Consensus) UpdateValidatorMetrics(numSig float64, blockNum float64)

UpdateValidatorMetrics will udpate validator metrics

func (*Consensus) ValidateVdfAndProof

func (consensus *Consensus) ValidateVdfAndProof(headerObj *block.Header) bool

ValidateVdfAndProof validates the VDF/proof in the current epoch

func (*Consensus) ValidateVrfAndProof

func (consensus *Consensus) ValidateVrfAndProof(headerObj *block.Header) bool

ValidateVrfAndProof validates a VRF/Proof from hash of previous block

func (*Consensus) VdfSeedSize

func (consensus *Consensus) VdfSeedSize() int

VdfSeedSize returns the number of VRFs for VDF computation

func (*Consensus) WaitForNewRandomness

func (consensus *Consensus) WaitForNewRandomness()

WaitForNewRandomness listens to the RndChannel to receive new VDF randomness.

type FBFTLog

type FBFTLog struct {
	// contains filtered or unexported fields
}

FBFTLog represents the log stored by a node during FBFT process

func NewFBFTLog

func NewFBFTLog() *FBFTLog

NewFBFTLog returns new instance of FBFTLog

func (*FBFTLog) AddBlock

func (log *FBFTLog) AddBlock(block *types.Block)

AddBlock add a new block into the log

func (*FBFTLog) AddNotVerifiedMessage added in v1.10.0

func (log *FBFTLog) AddNotVerifiedMessage(msg *FBFTMessage)

AddNotVerifiedMessage adds a not signature verified pbft message into the log

func (*FBFTLog) AddVerifiedMessage added in v1.10.0

func (log *FBFTLog) AddVerifiedMessage(msg *FBFTMessage)

AddVerifiedMessage adds a signature verified pbft message into the log

func (*FBFTLog) DeleteBlockByNumber

func (log *FBFTLog) DeleteBlockByNumber(number uint64)

DeleteBlockByNumber deletes block of specific number

func (*FBFTLog) DeleteBlocksLessThan

func (log *FBFTLog) DeleteBlocksLessThan(number uint64)

DeleteBlocksLessThan deletes blocks less than given block number

func (*FBFTLog) DeleteMessagesLessThan

func (log *FBFTLog) DeleteMessagesLessThan(number uint64)

DeleteMessagesLessThan deletes messages less than given block number

func (*FBFTLog) FindMessageByMaxViewID

func (log *FBFTLog) FindMessageByMaxViewID(msgs []*FBFTMessage) *FBFTMessage

FindMessageByMaxViewID returns the message that has maximum ViewID

func (*FBFTLog) GetBlockByHash

func (log *FBFTLog) GetBlockByHash(hash common.Hash) *types.Block

GetBlockByHash returns the block matches the given block hash

func (*FBFTLog) GetBlocksByNumber

func (log *FBFTLog) GetBlocksByNumber(number uint64) []*types.Block

GetBlocksByNumber returns the blocks match the given block number

func (*FBFTLog) GetCommittedBlockAndMsgsFromNumber added in v1.10.0

func (log *FBFTLog) GetCommittedBlockAndMsgsFromNumber(bn uint64, logger *zerolog.Logger) (*types.Block, *FBFTMessage, error)

GetCommittedBlockAndMsgsFromNumber get committed block and message starting from block number bn.

func (*FBFTLog) GetMessagesByTypeSeq

func (log *FBFTLog) GetMessagesByTypeSeq(typ msg_pb.MessageType, blockNum uint64) []*FBFTMessage

GetMessagesByTypeSeq returns pbft messages with matching type, blockNum

func (*FBFTLog) GetMessagesByTypeSeqHash

func (log *FBFTLog) GetMessagesByTypeSeqHash(typ msg_pb.MessageType, blockNum uint64, blockHash common.Hash) []*FBFTMessage

GetMessagesByTypeSeqHash returns pbft messages with matching type, blockNum

func (*FBFTLog) GetMessagesByTypeSeqView

func (log *FBFTLog) GetMessagesByTypeSeqView(typ msg_pb.MessageType, blockNum uint64, viewID uint64) []*FBFTMessage

GetMessagesByTypeSeqView returns pbft messages with matching type, blockNum and viewID

func (*FBFTLog) GetMessagesByTypeSeqViewHash

func (log *FBFTLog) GetMessagesByTypeSeqViewHash(typ msg_pb.MessageType, blockNum uint64, viewID uint64, blockHash common.Hash) []*FBFTMessage

GetMessagesByTypeSeqViewHash returns pbft messages with matching type, blockNum, viewID and blockHash

func (*FBFTLog) GetNotVerifiedCommittedMessages added in v1.10.0

func (log *FBFTLog) GetNotVerifiedCommittedMessages(blockNum uint64, viewID uint64, blockHash common.Hash) []*FBFTMessage

GetNotVerifiedCommittedMessages returns not verified committed pbft messages with matching blockNum, viewID and blockHash

func (*FBFTLog) HasMatchingAnnounce

func (log *FBFTLog) HasMatchingAnnounce(blockNum uint64, blockHash common.Hash) bool

HasMatchingAnnounce returns whether the log contains announce type message with given blockNum, blockHash

func (*FBFTLog) HasMatchingPrepared

func (log *FBFTLog) HasMatchingPrepared(blockNum uint64, blockHash common.Hash) bool

HasMatchingPrepared returns whether the log contains prepared message with given blockNum, viewID and blockHash

func (*FBFTLog) HasMatchingViewAnnounce

func (log *FBFTLog) HasMatchingViewAnnounce(blockNum uint64, viewID uint64, blockHash common.Hash) bool

HasMatchingViewAnnounce returns whether the log contains announce type message with given blockNum, viewID and blockHash

func (*FBFTLog) HasMatchingViewPrepared

func (log *FBFTLog) HasMatchingViewPrepared(blockNum uint64, viewID uint64, blockHash common.Hash) bool

HasMatchingViewPrepared returns whether the log contains prepared message with given blockNum, viewID and blockHash

func (*FBFTLog) IsBlockVerified added in v1.10.0

func (log *FBFTLog) IsBlockVerified(block *types.Block) bool

IsBlockVerified checks whether the block is verified

func (*FBFTLog) MarkBlockVerified added in v1.10.0

func (log *FBFTLog) MarkBlockVerified(block *types.Block)

MarkBlockVerified marks the block as verified

func (*FBFTLog) PruneCacheBeforeBlock added in v1.10.0

func (log *FBFTLog) PruneCacheBeforeBlock(bn uint64)

PruneCacheBeforeBlock prune all blocks before bn

type FBFTMessage

type FBFTMessage struct {
	MessageType        msg_pb.MessageType
	ViewID             uint64
	BlockNum           uint64
	BlockHash          common.Hash
	Block              []byte
	SenderPubkeys      []*bls.PublicKeyWrapper
	SenderPubkeyBitmap []byte
	LeaderPubkey       *bls.PublicKeyWrapper
	Payload            []byte
	ViewchangeSig      *bls_core.Sign
	ViewidSig          *bls_core.Sign
	M2AggSig           *bls_core.Sign
	M2Bitmap           *bls_cosi.Mask
	M3AggSig           *bls_core.Sign
	M3Bitmap           *bls_cosi.Mask
	Verified           bool
}

FBFTMessage is the record of pbft messages received by a node during FBFT process

func ParseNewViewMessage added in v1.10.0

func ParseNewViewMessage(msg *msg_pb.Message, members multibls.PublicKeys) (*FBFTMessage, error)

ParseNewViewMessage parses new view message into FBFTMessage structure

func ParseViewChangeMessage

func ParseViewChangeMessage(msg *msg_pb.Message) (*FBFTMessage, error)

ParseViewChangeMessage parses view change message into FBFTMessage structure

func (*FBFTMessage) HasSingleSender added in v1.10.0

func (m *FBFTMessage) HasSingleSender() bool

HasSingleSender returns whether the message has only a single sender

func (*FBFTMessage) String added in v1.3.3

func (m *FBFTMessage) String() string

String ..

type FBFTPhase

type FBFTPhase byte

FBFTPhase : different phases of consensus

const (
	FBFTAnnounce FBFTPhase = iota
	FBFTPrepare
	FBFTCommit
)

Enum for FBFTPhase

func (FBFTPhase) String

func (p FBFTPhase) String() string

type LastMileBlockIter added in v1.10.0

type LastMileBlockIter struct {
	// contains filtered or unexported fields
}

LastMileBlockIter is the iterator to iterate over the last mile blocks in consensus cache. All blocks returned are guaranteed to pass the verification.

func (*LastMileBlockIter) Next added in v1.10.0

func (iter *LastMileBlockIter) Next() *types.Block

Next iterate to the next last mile block

type MessageRetry

type MessageRetry struct {
	// contains filtered or unexported fields
}

MessageRetry controls the message that can be retried

type MessageSender

type MessageSender struct {
	// contains filtered or unexported fields
}

MessageSender is the wrapper object that controls how a consensus message is sent

func NewMessageSender

func NewMessageSender(host p2p.Host) *MessageSender

NewMessageSender initializes the consensus message sender.

func (*MessageSender) DelayedSendWithRetry added in v1.10.0

func (sender *MessageSender) DelayedSendWithRetry(blockNum uint64, msgType msg_pb.MessageType, groups []nodeconfig.GroupID, p2pMsg []byte)

DelayedSendWithRetry is similar to SendWithRetry but without the initial message sending but only retries.

func (*MessageSender) Reset

func (sender *MessageSender) Reset(blockNum uint64)

Reset resets the sender's state for new block

func (*MessageSender) Retry

func (sender *MessageSender) Retry(msgRetry *MessageRetry)

Retry will retry the consensus message for <RetryTimes> times.

func (*MessageSender) SendWithRetry

func (sender *MessageSender) SendWithRetry(blockNum uint64, msgType msg_pb.MessageType, groups []nodeconfig.GroupID, p2pMsg []byte) error

SendWithRetry sends message with retry logic.

func (*MessageSender) SendWithoutRetry

func (sender *MessageSender) SendWithoutRetry(groups []nodeconfig.GroupID, p2pMsg []byte) error

SendWithoutRetry sends message without retry logic.

func (*MessageSender) StopAllRetriesExceptCommitted

func (sender *MessageSender) StopAllRetriesExceptCommitted()

StopAllRetriesExceptCommitted stops all the existing retries except committed message (which lives across consensus).

func (*MessageSender) StopRetry

func (sender *MessageSender) StopRetry(msgType msg_pb.MessageType)

StopRetry stops the retry.

type Mode

type Mode byte

Mode is the current

const (
	// Normal ..
	Normal Mode = iota
	// ViewChanging ..
	ViewChanging
	// Syncing ..
	Syncing
	// Listening ..
	Listening
)

func (Mode) String

func (m Mode) String() string

type NetworkMessage added in v1.3.1

type NetworkMessage struct {
	MessageType                msg_pb.MessageType
	Bytes                      []byte
	FBFTMsg                    *FBFTMessage
	OptionalAggregateSignature *bls_core.Sign
}

NetworkMessage is a message intended to be created only for distribution to all the other quorum members.

type ProposalType added in v1.10.0

type ProposalType byte

ProposalType is to indicate the type of signal for new block proposal

const (
	SyncProposal ProposalType = iota
	AsyncProposal
)

Constant of the type of new block proposal

type State

type State struct {
	// contains filtered or unexported fields
}

State contains current mode and current viewID

func (*State) GetCurBlockViewID added in v1.10.0

func (pm *State) GetCurBlockViewID() uint64

GetCurBlockViewID return the current view id

func (*State) GetViewChangeDuraion added in v1.10.0

func (pm *State) GetViewChangeDuraion() time.Duration

GetViewChangeDuraion return the duration of the current view change It increase in the power of difference betweeen view changing ID and current view ID

func (*State) GetViewChangingID added in v1.10.0

func (pm *State) GetViewChangingID() uint64

GetViewChangingID return the current view changing id It is meaningful during view change mode

func (*State) Mode

func (pm *State) Mode() Mode

Mode return the current node mode

func (*State) SetCurBlockViewID added in v1.10.0

func (pm *State) SetCurBlockViewID(viewID uint64)

SetCurBlockViewID sets the current view id

func (*State) SetMode

func (pm *State) SetMode(s Mode)

SetMode set the node mode as required

func (*State) SetViewChangingID added in v1.10.0

func (pm *State) SetViewChangingID(id uint64)

SetViewChangingID set the current view changing id It is meaningful during view change mode

type TimeoutType

type TimeoutType int

TimeoutType is the type of timeout in view change protocol

Directories

Path Synopsis

Jump to

Keyboard shortcuts

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