Documentation ¶
Index ¶
- Constants
- Variables
- type Consensus
- func (consensus *Consensus) BlockCommitSig(blockNum uint64) ([]byte, []byte, error)
- func (consensus *Consensus) BlocksNotSynchronized()
- func (consensus *Consensus) BlocksSynchronized()
- func (consensus *Consensus) GenerateVdfAndProof(newBlock *types.Block, vrfBlockNumbers []uint64)
- func (consensus *Consensus) GenerateVrfAndProof(newBlock *types.Block, vrfBlockNumbers []uint64) []uint64
- func (consensus *Consensus) GetConsensusLeaderPrivateKey() (*bls.SecretKey, error)
- func (consensus *Consensus) GetLeaderPrivateKey(leaderKey *bls.PublicKey) (*bls.SecretKey, error)
- func (consensus *Consensus) GetNextLeaderKey() *bls.PublicKey
- func (consensus *Consensus) GetNextRnd() ([vdFAndProofSize]byte, [32]byte, error)
- func (consensus *Consensus) GetNilSigsArray(viewID uint64) []*bls.Sign
- func (consensus *Consensus) GetNodeIDs() []libp2p_peer.ID
- func (consensus *Consensus) GetValidatorPeers() []p2p.Peer
- func (consensus *Consensus) GetViewID() uint64
- func (consensus *Consensus) GetViewIDSigsArray(viewID uint64) []*bls.Sign
- func (consensus *Consensus) IsLeader() bool
- func (consensus *Consensus) IsValidatorInCommittee(pubKey *bls.PublicKey) bool
- func (consensus *Consensus) Mode() Mode
- func (consensus *Consensus) NeedsRandomNumberGeneration(epoch *big.Int) bool
- func (consensus *Consensus) ParseNewViewMessage(msg *msg_pb.Message) (*FBFTMessage, error)
- func (consensus *Consensus) ReadSignatureBitmapPayload(recvPayload []byte, offset int) (*bls.Sign, *bls_cosi.Mask, error)
- func (consensus *Consensus) RegisterPRndChannel(pRndChannel chan []byte)
- func (consensus *Consensus) RegisterRndChannel(rndChannel chan [548]byte)
- func (consensus *Consensus) ResetState()
- func (consensus *Consensus) ResetViewChangeState()
- func (consensus *Consensus) SetBlockNum(blockNum uint64)
- func (consensus *Consensus) SetCommitDelay(delay time.Duration)
- func (consensus *Consensus) SetEpochNum(epoch uint64)
- func (consensus *Consensus) SetMode(m Mode)
- func (consensus *Consensus) SetViewID(height uint64)
- func (consensus *Consensus) Start(blockChannel chan *types.Block, ...)
- func (consensus *Consensus) String() string
- func (consensus *Consensus) ToggleConsensusCheck()
- func (consensus *Consensus) UpdateConsensusInformation() Mode
- func (consensus *Consensus) UpdatePublicKeys(pubKeys []*bls.PublicKey) int64
- func (consensus *Consensus) ValidateVdfAndProof(headerObj *block.Header) bool
- func (consensus *Consensus) ValidateVrfAndProof(headerObj *block.Header) bool
- func (consensus *Consensus) VdfSeedSize() int
- func (consensus *Consensus) WaitForNewRandomness()
- type FBFTLog
- func (log *FBFTLog) AddBlock(block *types.Block)
- func (log *FBFTLog) AddMessage(msg *FBFTMessage)
- func (log *FBFTLog) Blocks() mapset.Set
- func (log *FBFTLog) DeleteBlockByNumber(number uint64)
- func (log *FBFTLog) DeleteBlocksLessThan(number uint64)
- func (log *FBFTLog) DeleteMessagesLessThan(number uint64)
- func (log *FBFTLog) FindMessageByMaxViewID(msgs []*FBFTMessage) *FBFTMessage
- func (log *FBFTLog) GetBlockByHash(hash common.Hash) *types.Block
- func (log *FBFTLog) GetBlocksByNumber(number uint64) []*types.Block
- func (log *FBFTLog) GetMessagesByTypeSeq(typ msg_pb.MessageType, blockNum uint64) []*FBFTMessage
- func (log *FBFTLog) GetMessagesByTypeSeqHash(typ msg_pb.MessageType, blockNum uint64, blockHash common.Hash) []*FBFTMessage
- func (log *FBFTLog) GetMessagesByTypeSeqView(typ msg_pb.MessageType, blockNum uint64, viewID uint64) []*FBFTMessage
- func (log *FBFTLog) GetMessagesByTypeSeqViewHash(typ msg_pb.MessageType, blockNum uint64, viewID uint64, blockHash common.Hash) []*FBFTMessage
- func (log *FBFTLog) HasMatchingAnnounce(blockNum uint64, blockHash common.Hash) bool
- func (log *FBFTLog) HasMatchingPrepared(blockNum uint64, blockHash common.Hash) bool
- func (log *FBFTLog) HasMatchingViewAnnounce(blockNum uint64, viewID uint64, blockHash common.Hash) bool
- func (log *FBFTLog) HasMatchingViewPrepared(blockNum uint64, viewID uint64, blockHash common.Hash) bool
- func (log *FBFTLog) Messages() mapset.Set
- type FBFTMessage
- type FBFTPhase
- type MessageRetry
- type MessageSender
- func (sender *MessageSender) Reset(blockNum uint64)
- func (sender *MessageSender) Retry(msgRetry *MessageRetry)
- func (sender *MessageSender) SendWithRetry(blockNum uint64, msgType msg_pb.MessageType, groups []nodeconfig.GroupID, ...) error
- func (sender *MessageSender) SendWithoutRetry(groups []nodeconfig.GroupID, p2pMsg []byte) error
- func (sender *MessageSender) StopAllRetriesExceptCommitted()
- func (sender *MessageSender) StopRetry(msgType msg_pb.MessageType)
- type Mode
- type NetworkMessage
- type State
- type TimeoutType
Constants ¶
const MaxBlockNumDiff = 100
MaxBlockNumDiff limits the received block number to only 100 further from the current block number
const MaxViewIDDiff = 100
MaxViewIDDiff limits the received view ID to only 100 further from the current view ID
const (
// RetryIntervalInSec is the interval for message retry
RetryIntervalInSec = 10
)
Variables ¶
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 Consensus ¶
type Consensus struct { Decider quorum.Decider // FBFTLog stores the pbft messages and blocks during FBFT process FBFTLog *FBFTLog // channel to receive consensus message MsgChan chan []byte // The chain reader for the blockchain this consensus is working on ChainReader *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 PubKey *multibls.PublicKey // the publickey of leader LeaderPubKey *bls.PublicKey // Shard Id which this node belongs to ShardID uint32 // Signal channel for starting a new consensus process ReadySignal chan struct{} // The post-consensus processing func passed from Node object // Called when consensus on a new block is done OnConsensusDone func(*types.Block) // The verifier func passed from Node object BlockVerifier func(*types.Block) error // 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 // 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.PrivateKey, Decider quorum.Decider, ) (*Consensus, error)
New create a new Consensus record
func (*Consensus) BlockCommitSig ¶ added in v1.3.8
BlockCommitSig 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) GenerateVdfAndProof ¶
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) GetConsensusLeaderPrivateKey ¶ added in v1.2.4
GetConsensusLeaderPrivateKey returns consensus leader private key if node is the leader
func (*Consensus) GetLeaderPrivateKey ¶ added in v1.2.4
GetLeaderPrivateKey returns leader private key if node is the leader
func (*Consensus) GetNextLeaderKey ¶
GetNextLeaderKey uniquely determine who is the leader for given viewID
func (*Consensus) GetNextRnd ¶
GetNextRnd returns the oldest available randomness along with the hash of the block there randomness preimage is committed.
func (*Consensus) GetNilSigsArray ¶
GetNilSigsArray returns the signatures for nil prepared message in viewchange
func (*Consensus) GetNodeIDs ¶
func (consensus *Consensus) GetNodeIDs() []libp2p_peer.ID
GetNodeIDs returns Node IDs of all nodes in the same shard
func (*Consensus) GetValidatorPeers ¶
GetValidatorPeers returns list of validator peers.
func (*Consensus) GetViewIDSigsArray ¶
GetViewIDSigsArray returns the signatures for viewID in viewchange
func (*Consensus) IsLeader ¶
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 ¶
IsValidatorInCommittee returns whether the given validator BLS address is part of my committee
func (*Consensus) NeedsRandomNumberGeneration ¶
NeedsRandomNumberGeneration returns true if the current epoch needs random number generation
func (*Consensus) ParseNewViewMessage ¶
func (consensus *Consensus) ParseNewViewMessage(msg *msg_pb.Message) (*FBFTMessage, error)
ParseNewViewMessage parses new view message into FBFTMessage structure
func (*Consensus) ReadSignatureBitmapPayload ¶
func (consensus *Consensus) ReadSignatureBitmapPayload( recvPayload []byte, offset int, ) (*bls.Sign, *bls_cosi.Mask, error)
ReadSignatureBitmapPayload read the payload for signature and bitmap; offset is the beginning position of reading
func (*Consensus) RegisterPRndChannel ¶
RegisterPRndChannel registers the channel for receiving randomness preimage from DRG protocol
func (*Consensus) RegisterRndChannel ¶
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 reset the state for viewchange
func (*Consensus) SetBlockNum ¶
SetBlockNum sets the blockNum in consensus object, called at node bootstrap
func (*Consensus) SetCommitDelay ¶
SetCommitDelay sets the commit message delay. If set to non-zero, validator delays commit message by the amount.
func (*Consensus) SetEpochNum ¶
SetEpochNum sets the epoch in consensus object
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) ToggleConsensusCheck ¶
func (consensus *Consensus) ToggleConsensusCheck()
ToggleConsensusCheck flip the flag of whether ignore viewID check during consensus process
func (*Consensus) UpdateConsensusInformation ¶
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) UpdatePublicKeys ¶
UpdatePublicKeys updates the PublicKeys for quorum on current subcommittee, protected by a mutex
func (*Consensus) ValidateVdfAndProof ¶
ValidateVdfAndProof validates the VDF/proof in the current epoch
func (*Consensus) ValidateVrfAndProof ¶
ValidateVrfAndProof validates a VRF/Proof from hash of previous block
func (*Consensus) VdfSeedSize ¶
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 (*FBFTLog) AddMessage ¶
func (log *FBFTLog) AddMessage(msg *FBFTMessage)
AddMessage adds a pbft message into the log
func (*FBFTLog) DeleteBlockByNumber ¶
DeleteBlockByNumber deletes block of specific number
func (*FBFTLog) DeleteBlocksLessThan ¶
DeleteBlocksLessThan deletes blocks less than given block number
func (*FBFTLog) DeleteMessagesLessThan ¶
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 ¶
GetBlockByHash returns the block matches the given block hash
func (*FBFTLog) GetBlocksByNumber ¶
GetBlocksByNumber returns the blocks match the given block number
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) HasMatchingAnnounce ¶
HasMatchingAnnounce returns whether the log contains announce type message with given blockNum, blockHash
func (*FBFTLog) HasMatchingPrepared ¶
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
type FBFTMessage ¶
type FBFTMessage struct { MessageType msg_pb.MessageType ViewID uint64 BlockNum uint64 BlockHash common.Hash Block []byte SenderPubkey *bls.PublicKey LeaderPubkey *bls.PublicKey Payload []byte ViewchangeSig *bls.Sign ViewidSig *bls.Sign M2AggSig *bls.Sign M2Bitmap *bls_cosi.Mask M3AggSig *bls.Sign M3Bitmap *bls_cosi.Mask }
FBFTMessage is the record of pbft messages received by a node during FBFT process
func ParseFBFTMessage ¶
func ParseFBFTMessage(msg *msg_pb.Message) (*FBFTMessage, error)
ParseFBFTMessage parses FBFT message into FBFTMessage structure
func ParseViewChangeMessage ¶
func ParseViewChangeMessage(msg *msg_pb.Message) (*FBFTMessage, error)
ParseViewChangeMessage parses view change message into FBFTMessage structure
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) 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 NetworkMessage ¶ added in v1.3.1
type NetworkMessage struct { Phase msg_pb.MessageType Bytes []byte FBFTMsg *FBFTMessage OptionalAggregateSignature *bls.Sign }
NetworkMessage is a message intended to be created only for distribution to all the other quorum members.