|
|
|
package consensus
|
|
|
|
|
|
|
|
import (
|
|
|
|
"fmt"
|
|
|
|
"sync"
|
|
|
|
"time"
|
|
|
|
|
|
|
|
"github.com/harmony-one/harmony/crypto/bls"
|
|
|
|
|
|
|
|
"github.com/harmony-one/abool"
|
|
|
|
bls_core "github.com/harmony-one/bls/ffi/go/bls"
|
|
|
|
"github.com/harmony-one/harmony/consensus/quorum"
|
|
|
|
"github.com/harmony-one/harmony/core"
|
|
|
|
"github.com/harmony-one/harmony/core/types"
|
|
|
|
bls_cosi "github.com/harmony-one/harmony/crypto/bls"
|
|
|
|
"github.com/harmony-one/harmony/internal/utils"
|
|
|
|
"github.com/harmony-one/harmony/multibls"
|
|
|
|
"github.com/harmony-one/harmony/p2p"
|
[slash][consensus] Notice double sign & broadcast, factor out tech debt of consensus (#2152)
* [slash] Remove dead interface, associated piping
* [slash] Expand out structs
* [consensus] Write to a chan when find a case of double-signing, remove dead code
* [slash] Broadcast the noticing of a double signing
* [rawdb] CRUD for slashing candidates
* [slashing][node][proto] Broadcast the slash record after receive from consensus, handle received proto message, persist in off-chain db while pending
* [slash][node][propose-block] Add verified slashes proposed into the header in block proposal
* [slash][shard] Factor out external validator as method on shard state, add double-signature field
* [slash][engine] Apply slash, name boolean expression for sorts, use stable sort
* [slash] Abstract Ballot results so keep track of both pre and post double sign event
* [slash] Fix type errors on test code
* [slash] Read from correct rawdb
* [slash] Add epoch based guards in CRUD of slashing
* [slash] Write to correct cache for slashing candidates
* [shard] Use explicit named type of BLS Signature, use convention
* [slash] Fix mistake done in refactor, improper header used. Factor out fromSlice to set
* [slash][node] Restore newblock to master, try again minimial change
* [cx-receipts] Break up one-liner, use SliceStable, not Slice
* [network] Finish refactor that makes network message headers once
* [network] Simplify creation further of headers write
* [slash] Adjust data structure of slash after offline discussion with RJ, Chao
* [slash] Still did need signature of the double signature
* [consensus] Prepare message does not have block header
* [consensus] Soft reset three files to 968517d~1
* [consensus] Begin factor consensus network intended message out with prepare first
* [consensus] Factor out Prepared message
* [consensus] Factor out announce message creation
* [consensus] Committed Message, branch on verify sender key for clearer log
* [consensus] Committed Message Factor out
* [consensus] Do jenkins MVP of signatures adjustment
* [main][slash] Provide YAML config as webhook config for double sign event
* [consensus] Adjust signatures, whitespace, lessen GC pressure
* [consensus] Remove dead code
* [consensus] Factor out commit overloaded message, give commit payload override in construct
* [consensus] Fix travis tests
* [consensus] Provide block bytes in SubmitVote(quorum.Commit)
* [consensus] Factor out noisy sanity checks in BFT, move existing commit check earlier as was before
* [quorum] Adjust signatures in quorum
* [staking] Adjust after merge from master
* [consensus] Finish refactor of consensus
* [node] Fix import
* [consensus] Fix travis
* [consensus] Use origin/master copy of block, fix mistake of pointer to empty byte
* [consensus] Less verbose bools
* [consensus] Remove unused trailing mutation hook in message construct
* [consensus] Address some TODOs on err, comment out double sign
5 years ago
|
|
|
"github.com/harmony-one/harmony/staking/slash"
|
|
|
|
"github.com/pkg/errors"
|
|
|
|
)
|
|
|
|
|
|
|
|
const (
|
|
|
|
vdFAndProofSize = 516 // size of VDF and Proof
|
|
|
|
vdfAndSeedSize = 548 // size of VDF/Proof and Seed
|
|
|
|
)
|
|
|
|
|
|
|
|
var errLeaderPriKeyNotFound = errors.New("getting leader private key from consensus public keys failed")
|
|
|
|
|
|
|
|
// Consensus is the main struct with all states and data related to consensus process.
|
|
|
|
type Consensus struct {
|
|
|
|
Decider quorum.Decider
|
|
|
|
// FBFTLog stores the pbft messages and blocks during FBFT process
|
|
|
|
FBFTLog *FBFTLog
|
|
|
|
// phase: different phase of FBFT protocol: pre-prepare, prepare, commit, finish etc
|
|
|
|
phase FBFTPhase
|
|
|
|
// current indicates what state a node is in
|
|
|
|
current State
|
|
|
|
// How long to delay sending commit messages.
|
|
|
|
delayCommit time.Duration
|
|
|
|
// Consensus rounds whose commit phase finished
|
|
|
|
commitFinishChan chan uint64
|
|
|
|
// 2 types of timeouts: normal and viewchange
|
|
|
|
consensusTimeout map[TimeoutType]*utils.Timeout
|
|
|
|
// Commits collected from validators.
|
|
|
|
aggregatedPrepareSig *bls_core.Sign
|
|
|
|
aggregatedCommitSig *bls_core.Sign
|
|
|
|
prepareBitmap *bls_cosi.Mask
|
|
|
|
commitBitmap *bls_cosi.Mask
|
|
|
|
// Commits collected from view change
|
|
|
|
// for each viewID, we need keep track of corresponding sigs and bitmap
|
|
|
|
// until one of the viewID has enough votes (>=2f+1)
|
|
|
|
// after one of viewID has enough votes, we can reset and clean the map
|
|
|
|
// honest nodes will never double votes on different viewID
|
|
|
|
// bhpSigs: blockHashPreparedSigs is the signature on m1 type message
|
|
|
|
bhpSigs map[uint64]map[string]*bls_core.Sign
|
|
|
|
// nilSigs: there is no prepared message when view change,
|
|
|
|
// it's signature on m2 type (i.e. nil) messages
|
|
|
|
nilSigs map[uint64]map[string]*bls_core.Sign
|
|
|
|
viewIDSigs map[uint64]map[string]*bls_core.Sign
|
|
|
|
bhpBitmap map[uint64]*bls_cosi.Mask
|
|
|
|
nilBitmap map[uint64]*bls_cosi.Mask
|
|
|
|
viewIDBitmap map[uint64]*bls_cosi.Mask
|
|
|
|
m1Payload []byte // message payload for type m1 := |vcBlockHash|prepared_agg_sigs|prepared_bitmap|, new leader only need one
|
|
|
|
// 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
|
|
|
|
pubKeyLock sync.Mutex
|
|
|
|
// private/public keys of current node
|
|
|
|
priKey multibls.PrivateKeys
|
|
|
|
// the publickey of leader
|
|
|
|
LeaderPubKey *bls.PublicKeyWrapper
|
|
|
|
// blockNum: the next blockNumber that FBFT is going to agree on,
|
|
|
|
// should be equal to the blockNumber of next block
|
|
|
|
blockNum uint64
|
|
|
|
// Blockhash - 32 byte
|
|
|
|
blockHash [32]byte
|
|
|
|
// Block to run consensus on
|
|
|
|
block []byte
|
|
|
|
// Shard Id which this node belongs to
|
|
|
|
ShardID uint32
|
|
|
|
// IgnoreViewIDCheck determines whether to ignore viewID check
|
|
|
|
IgnoreViewIDCheck *abool.AtomicBool
|
|
|
|
// consensus mutex
|
|
|
|
mutex sync.Mutex
|
|
|
|
// mutex for view change
|
|
|
|
vcLock sync.Mutex
|
|
|
|
// 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
|
|
|
|
pendingRnds [][vdfAndSeedSize]byte // A list of pending randomness
|
|
|
|
// The p2p host used to send/receive p2p messages
|
|
|
|
host p2p.Host
|
|
|
|
// MessageSender takes are of sending consensus message and the corresponding retry logic.
|
|
|
|
msgSender *MessageSender
|
|
|
|
// Used to convey to the consensus main loop that block syncing has finished.
|
|
|
|
syncReadyChan chan struct{}
|
|
|
|
// Used to convey to the consensus main loop that node is out of sync
|
|
|
|
syncNotReadyChan chan struct{}
|
|
|
|
// If true, this consensus will not propose view change.
|
|
|
|
disableViewChange bool
|
[slash][consensus] Notice double sign & broadcast, factor out tech debt of consensus (#2152)
* [slash] Remove dead interface, associated piping
* [slash] Expand out structs
* [consensus] Write to a chan when find a case of double-signing, remove dead code
* [slash] Broadcast the noticing of a double signing
* [rawdb] CRUD for slashing candidates
* [slashing][node][proto] Broadcast the slash record after receive from consensus, handle received proto message, persist in off-chain db while pending
* [slash][node][propose-block] Add verified slashes proposed into the header in block proposal
* [slash][shard] Factor out external validator as method on shard state, add double-signature field
* [slash][engine] Apply slash, name boolean expression for sorts, use stable sort
* [slash] Abstract Ballot results so keep track of both pre and post double sign event
* [slash] Fix type errors on test code
* [slash] Read from correct rawdb
* [slash] Add epoch based guards in CRUD of slashing
* [slash] Write to correct cache for slashing candidates
* [shard] Use explicit named type of BLS Signature, use convention
* [slash] Fix mistake done in refactor, improper header used. Factor out fromSlice to set
* [slash][node] Restore newblock to master, try again minimial change
* [cx-receipts] Break up one-liner, use SliceStable, not Slice
* [network] Finish refactor that makes network message headers once
* [network] Simplify creation further of headers write
* [slash] Adjust data structure of slash after offline discussion with RJ, Chao
* [slash] Still did need signature of the double signature
* [consensus] Prepare message does not have block header
* [consensus] Soft reset three files to 968517d~1
* [consensus] Begin factor consensus network intended message out with prepare first
* [consensus] Factor out Prepared message
* [consensus] Factor out announce message creation
* [consensus] Committed Message, branch on verify sender key for clearer log
* [consensus] Committed Message Factor out
* [consensus] Do jenkins MVP of signatures adjustment
* [main][slash] Provide YAML config as webhook config for double sign event
* [consensus] Adjust signatures, whitespace, lessen GC pressure
* [consensus] Remove dead code
* [consensus] Factor out commit overloaded message, give commit payload override in construct
* [consensus] Fix travis tests
* [consensus] Provide block bytes in SubmitVote(quorum.Commit)
* [consensus] Factor out noisy sanity checks in BFT, move existing commit check earlier as was before
* [quorum] Adjust signatures in quorum
* [staking] Adjust after merge from master
* [consensus] Finish refactor of consensus
* [node] Fix import
* [consensus] Fix travis
* [consensus] Use origin/master copy of block, fix mistake of pointer to empty byte
* [consensus] Less verbose bools
* [consensus] Remove unused trailing mutation hook in message construct
* [consensus] Address some TODOs on err, comment out double sign
5 years ago
|
|
|
// 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
|
|
|
|
}
|
|
|
|
|
|
|
|
// SetCommitDelay sets the commit message delay. If set to non-zero,
|
|
|
|
// validator delays commit message by the amount.
|
|
|
|
func (consensus *Consensus) SetCommitDelay(delay time.Duration) {
|
|
|
|
consensus.delayCommit = delay
|
|
|
|
}
|
|
|
|
|
|
|
|
// BlocksSynchronized lets the main loop know that block synchronization finished
|
|
|
|
// thus the blockchain is likely to be up to date.
|
|
|
|
func (consensus *Consensus) BlocksSynchronized() {
|
|
|
|
consensus.syncReadyChan <- struct{}{}
|
|
|
|
}
|
|
|
|
|
|
|
|
// BlocksNotSynchronized lets the main loop know that block is not synchronized
|
|
|
|
func (consensus *Consensus) BlocksNotSynchronized() {
|
|
|
|
consensus.syncNotReadyChan <- struct{}{}
|
|
|
|
}
|
|
|
|
|
|
|
|
// VdfSeedSize returns the number of VRFs for VDF computation
|
|
|
|
func (consensus *Consensus) VdfSeedSize() int {
|
|
|
|
return int(consensus.Decider.ParticipantsCount()) * 2 / 3
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetPublicKeys returns the public keys
|
|
|
|
func (consensus *Consensus) GetPublicKeys() multibls.PublicKeys {
|
|
|
|
return consensus.priKey.GetPublicKeys()
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetLeaderPrivateKey returns leader private key if node is the leader
|
|
|
|
func (consensus *Consensus) GetLeaderPrivateKey(leaderKey *bls_core.PublicKey) (*bls.PrivateKeyWrapper, error) {
|
|
|
|
for i, key := range consensus.priKey {
|
|
|
|
if key.Pub.Object.IsEqual(leaderKey) {
|
|
|
|
return &consensus.priKey[i], nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil, errors.Wrapf(errLeaderPriKeyNotFound, leaderKey.SerializeToHexStr())
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetConsensusLeaderPrivateKey returns consensus leader private key if node is the leader
|
|
|
|
func (consensus *Consensus) GetConsensusLeaderPrivateKey() (*bls.PrivateKeyWrapper, error) {
|
|
|
|
return consensus.GetLeaderPrivateKey(consensus.LeaderPubKey.Object)
|
|
|
|
}
|
|
|
|
|
|
|
|
// New create a new Consensus record
|
|
|
|
func New(
|
|
|
|
host p2p.Host, shard uint32, leader p2p.Peer, multiBLSPriKey multibls.PrivateKeys,
|
|
|
|
Decider quorum.Decider,
|
|
|
|
) (*Consensus, error) {
|
|
|
|
consensus := Consensus{}
|
|
|
|
consensus.Decider = Decider
|
|
|
|
consensus.host = host
|
|
|
|
consensus.msgSender = NewMessageSender(host)
|
|
|
|
consensus.BlockNumLowChan = make(chan struct{})
|
|
|
|
// FBFT related
|
|
|
|
consensus.FBFTLog = NewFBFTLog()
|
|
|
|
consensus.phase = FBFTAnnounce
|
|
|
|
// TODO Refactor consensus.block* into State?
|
|
|
|
consensus.current = State{mode: Normal}
|
|
|
|
// FBFT timeout
|
|
|
|
consensus.consensusTimeout = createTimeout()
|
|
|
|
|
|
|
|
if multiBLSPriKey != nil {
|
|
|
|
consensus.priKey = multiBLSPriKey
|
|
|
|
utils.Logger().Info().
|
|
|
|
Str("publicKey", consensus.GetPublicKeys().SerializeToHexStr()).Msg("My Public Key")
|
|
|
|
} else {
|
|
|
|
utils.Logger().Error().Msg("the bls key is nil")
|
|
|
|
return nil, fmt.Errorf("nil bls key, aborting")
|
|
|
|
}
|
|
|
|
|
|
|
|
// viewID has to be initialized as the height of
|
|
|
|
// the blockchain during initialization as it was
|
|
|
|
// displayed on explorer as Height right now
|
|
|
|
consensus.SetCurViewID(0)
|
|
|
|
consensus.ShardID = shard
|
|
|
|
consensus.syncReadyChan = make(chan struct{})
|
|
|
|
consensus.syncNotReadyChan = make(chan struct{})
|
[slash][consensus] Notice double sign & broadcast, factor out tech debt of consensus (#2152)
* [slash] Remove dead interface, associated piping
* [slash] Expand out structs
* [consensus] Write to a chan when find a case of double-signing, remove dead code
* [slash] Broadcast the noticing of a double signing
* [rawdb] CRUD for slashing candidates
* [slashing][node][proto] Broadcast the slash record after receive from consensus, handle received proto message, persist in off-chain db while pending
* [slash][node][propose-block] Add verified slashes proposed into the header in block proposal
* [slash][shard] Factor out external validator as method on shard state, add double-signature field
* [slash][engine] Apply slash, name boolean expression for sorts, use stable sort
* [slash] Abstract Ballot results so keep track of both pre and post double sign event
* [slash] Fix type errors on test code
* [slash] Read from correct rawdb
* [slash] Add epoch based guards in CRUD of slashing
* [slash] Write to correct cache for slashing candidates
* [shard] Use explicit named type of BLS Signature, use convention
* [slash] Fix mistake done in refactor, improper header used. Factor out fromSlice to set
* [slash][node] Restore newblock to master, try again minimial change
* [cx-receipts] Break up one-liner, use SliceStable, not Slice
* [network] Finish refactor that makes network message headers once
* [network] Simplify creation further of headers write
* [slash] Adjust data structure of slash after offline discussion with RJ, Chao
* [slash] Still did need signature of the double signature
* [consensus] Prepare message does not have block header
* [consensus] Soft reset three files to 968517d~1
* [consensus] Begin factor consensus network intended message out with prepare first
* [consensus] Factor out Prepared message
* [consensus] Factor out announce message creation
* [consensus] Committed Message, branch on verify sender key for clearer log
* [consensus] Committed Message Factor out
* [consensus] Do jenkins MVP of signatures adjustment
* [main][slash] Provide YAML config as webhook config for double sign event
* [consensus] Adjust signatures, whitespace, lessen GC pressure
* [consensus] Remove dead code
* [consensus] Factor out commit overloaded message, give commit payload override in construct
* [consensus] Fix travis tests
* [consensus] Provide block bytes in SubmitVote(quorum.Commit)
* [consensus] Factor out noisy sanity checks in BFT, move existing commit check earlier as was before
* [quorum] Adjust signatures in quorum
* [staking] Adjust after merge from master
* [consensus] Finish refactor of consensus
* [node] Fix import
* [consensus] Fix travis
* [consensus] Use origin/master copy of block, fix mistake of pointer to empty byte
* [consensus] Less verbose bools
* [consensus] Remove unused trailing mutation hook in message construct
* [consensus] Address some TODOs on err, comment out double sign
5 years ago
|
|
|
consensus.SlashChan = make(chan slash.Record)
|
|
|
|
consensus.commitFinishChan = make(chan uint64)
|
|
|
|
consensus.ReadySignal = make(chan struct{})
|
|
|
|
// channel for receiving newly generated VDF
|
|
|
|
consensus.RndChannel = make(chan [vdfAndSeedSize]byte)
|
|
|
|
consensus.IgnoreViewIDCheck = abool.NewBool(false)
|
|
|
|
return &consensus, nil
|
|
|
|
}
|