The core protocol of WoopChain
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
woop/consensus/consensus_v2.go

661 lines
23 KiB

package consensus
import (
"bytes"
"context"
"encoding/hex"
"sync/atomic"
"time"
"github.com/harmony-one/harmony/crypto/bls"
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/block"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/types"
vrf_bls "github.com/harmony-one/harmony/crypto/vrf/bls"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/vdf/src/vdf_go"
"github.com/pkg/errors"
)
var (
errSenderPubKeyNotLeader = errors.New("sender pubkey doesn't match leader")
errVerifyMessageSignature = errors.New("verify message signature failed")
)
// IsViewChangingMode return true if curernt mode is viewchanging
func (consensus *Consensus) IsViewChangingMode() bool {
return consensus.current.Mode() == ViewChanging
}
// HandleMessageUpdate will update the consensus state according to received message
func (consensus *Consensus) HandleMessageUpdate(ctx context.Context, msg *msg_pb.Message, senderKey *bls.SerializedPublicKey) error {
// when node is in ViewChanging mode, it still accepts normal messages into FBFTLog
// in order to avoid possible trap forever but drop PREPARE and COMMIT
// which are message types specifically for a node acting as leader
// so we just ignore those messages
if consensus.IsViewChangingMode() &&
[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
(msg.Type == msg_pb.MessageType_PREPARE ||
msg.Type == msg_pb.MessageType_COMMIT) {
return nil
}
[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
intendedForValidator, intendedForLeader :=
!consensus.IsLeader(),
[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.IsLeader()
switch t := msg.Type; true {
// Handle validator intended messages first
case t == msg_pb.MessageType_ANNOUNCE && intendedForValidator:
if !bytes.Equal(senderKey[:], consensus.LeaderPubKey.Bytes[:]) &&
consensus.current.Mode() == Normal && !consensus.IgnoreViewIDCheck.IsSet() {
return errSenderPubKeyNotLeader
}
if !consensus.senderKeySanityChecks(msg, senderKey) {
return errVerifyMessageSignature
}
consensus.onAnnounce(msg)
case t == msg_pb.MessageType_PREPARED && intendedForValidator:
if !bytes.Equal(senderKey[:], consensus.LeaderPubKey.Bytes[:]) &&
consensus.current.Mode() == Normal && !consensus.IgnoreViewIDCheck.IsSet() {
return errSenderPubKeyNotLeader
}
if !consensus.senderKeySanityChecks(msg, senderKey) {
return errVerifyMessageSignature
}
consensus.onPrepared(msg)
case t == msg_pb.MessageType_COMMITTED && intendedForValidator:
if !bytes.Equal(senderKey[:], consensus.LeaderPubKey.Bytes[:]) &&
consensus.current.Mode() == Normal && !consensus.IgnoreViewIDCheck.IsSet() {
return errSenderPubKeyNotLeader
}
if !consensus.senderKeySanityChecks(msg, senderKey) {
return errVerifyMessageSignature
}
consensus.onCommitted(msg)
[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
// Handle leader intended messages now
case t == msg_pb.MessageType_PREPARE && intendedForLeader:
[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.onPrepare(msg)
case t == msg_pb.MessageType_COMMIT && intendedForLeader:
[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.onCommit(msg)
// Handle view change messages
case t == msg_pb.MessageType_VIEWCHANGE:
if !consensus.senderKeySanityChecks(msg, senderKey) {
return errVerifyMessageSignature
}
consensus.onViewChange(msg)
case t == msg_pb.MessageType_NEWVIEW:
if !consensus.senderKeySanityChecks(msg, senderKey) {
return errVerifyMessageSignature
}
consensus.onNewView(msg)
}
return nil
}
func (consensus *Consensus) finalizeCommits() {
consensus.getLogger().Info().
Int64("NumCommits", consensus.Decider.SignersCount(quorum.Commit)).
Msg("[finalizeCommits] Finalizing Block")
beforeCatchupNum := consensus.blockNum
leaderPriKey, err := consensus.GetConsensusLeaderPrivateKey()
if err != nil {
consensus.getLogger().Error().Err(err).Msg("[FinalizeCommits] leader not found")
return
}
// Construct committed message
network, err := consensus.construct(msg_pb.MessageType_COMMITTED, nil, []*bls.PrivateKeyWrapper{leaderPriKey})
if err != nil {
[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.getLogger().Warn().Err(err).
Msg("[FinalizeCommits] Unable to construct Committed message")
return
}
[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
msgToSend, aggSig, FBFTMsg :=
network.Bytes,
network.OptionalAggregateSignature,
network.FBFTMsg
consensus.aggregatedCommitSig = aggSig // this may not needed
consensus.FBFTLog.AddMessage(FBFTMsg)
// find correct block content
curBlockHash := consensus.blockHash
block := consensus.FBFTLog.GetBlockByHash(curBlockHash)
if block == nil {
consensus.getLogger().Warn().
Str("blockHash", hex.EncodeToString(curBlockHash[:])).
Msg("[FinalizeCommits] Cannot find block by hash")
return
}
consensus.tryCatchup()
if consensus.blockNum-beforeCatchupNum != 1 {
consensus.getLogger().Warn().
Uint64("beforeCatchupBlockNum", beforeCatchupNum).
Msg("[FinalizeCommits] Leader cannot provide the correct block for committed message")
return
}
5 years ago
// if leader success finalize the block, send committed message to validators
if err := consensus.msgSender.SendWithRetry(
block.NumberU64(),
msg_pb.MessageType_COMMITTED, []nodeconfig.GroupID{
nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID)),
},
p2p.ConstructMessage(msgToSend)); err != nil {
consensus.getLogger().Warn().Err(err).Msg("[finalizeCommits] Cannot send committed message")
} else {
consensus.getLogger().Info().
Hex("blockHash", curBlockHash[:]).
Uint64("blockNum", consensus.blockNum).
Msg("[finalizeCommits] Sent Committed Message")
}
// Dump new block into level db
// In current code, we add signatures in block in tryCatchup, the block dump to explorer does not contains signatures
// but since explorer doesn't need signatures, it should be fine
// in future, we will move signatures to next block
//explorer.GetStorageInstance(consensus.leader.IP, consensus.leader.Port, true).Dump(block, beforeCatchupNum)
if consensus.consensusTimeout[timeoutBootstrap].IsActive() {
consensus.consensusTimeout[timeoutBootstrap].Stop()
consensus.getLogger().Info().Msg("[finalizeCommits] Start consensus timer; stop bootstrap timer only once")
} else {
consensus.getLogger().Info().Msg("[finalizeCommits] Start consensus timer")
}
consensus.consensusTimeout[timeoutConsensus].Start()
consensus.getLogger().Info().
Uint64("blockNum", block.NumberU64()).
Uint64("epochNum", block.Epoch().Uint64()).
Uint64("ViewId", block.Header().ViewID().Uint64()).
Str("blockHash", block.Hash().String()).
5 years ago
Int("numTxns", len(block.Transactions())).
Int("numStakingTxns", len(block.StakingTransactions())).
Msg("HOORAY!!!!!!! CONSENSUS REACHED!!!!!!!")
// Sleep to wait for the full block time
consensus.getLogger().Info().Msg("[finalizeCommits] Waiting for Block Time")
<-time.After(time.Until(consensus.NextBlockDue))
// Send signal to Node to propose the new block for consensus
consensus.ReadySignal <- struct{}{}
// Update time due for next block
consensus.NextBlockDue = time.Now().Add(consensus.BlockPeriod)
}
// BlockCommitSig returns the byte array of aggregated
// commit signature and bitmap signed on the block
func (consensus *Consensus) BlockCommitSig(blockNum uint64) ([]byte, []byte, error) {
if consensus.blockNum <= 1 {
return nil, nil, nil
}
lastCommits, err := consensus.ChainReader.ReadCommitSig(blockNum)
[double-sign] Provide proof of double sign in slash record sent to beaconchain (#2253) * [double-sign] Commit changes in consensus needed for double-sign * [double-sign] Leader captures when valdator double signs, broadcasts to beaconchain * [slash] Add quick iteration tool for testing double-signing * [slash] Add webhook example * [slash] Add http server for hook to trigger double sign behavior * [double-sign] Use bin/trigger-double-sign to cause a double-sign * [double-sign] Full feedback loop working * [slash] Thread through the slash records in the block proposal step * [slash] Compute the slashing rate * [double-sign] Generalize yaml malicious for many keys * [double-sign][slash] Modify data structures, verify via webhook handler * [slash][double-sign] Find one address of bls public key signer, seemingly settle on data structures * [slash] Apply to state slashing for double signing * [slash][double-sign] Checkpoint for working code that slashes on beaconchain * [slash] Keep track of the total slash and total reporters reward * [slash] Dump account state before and after the slash * [slash] Satisfy Travis * [slash][state] Apply slash to the snapshot at beginning of epoch, now need to capture also the new delegates * [slash] Capture the unique new delegations since snapshot as well * [slash] Filter undelegation by epoch of double sign * [slash] Add TODO of correctness needed in slash needs on off-chain data * [rpc] Fix closure issue on shardID * [slash] Add delegator to double-sign testing script * [slash] Expand crt-validator.sh with commenting printfs and make delegation * [slash] Finish track payment of leftover slash debt after undelegation runs out * [slash] Now be explicit about error wrt delegatorSlashApply * [slash] Capture specific sanity check on slash paidoff * [slash] Track slash from undelegation piecemeal * [slash][delegation] Named slice types, .String() * [slash] Do no RLP encode twice, once is enough * [slash] Remove special case of validators own delegation * [slash] Refactor approach to slash state application * [slash] Begin expanding out Verify * [slash] Slash on snapshot delegations, not current * [slash] Fix Epoch Cmp * [slash] Third iteration on slash logic * [slash] Use full slash amount * [slash] More log, whitespace * [slash] Remove Println, add log * [slash] Remove debug Println * [slash] Add record in unit test * [slash] Build Validator snapshot, current. Fill out slash record * [slash] Need to get RLP dump of a header to use in test * [slash] Factor out double sign test constants * [slash] Factor out common for validator, stub out slash application, finish out deserialization setup * [slash] Factor out data structure creation because of var lexical scoping * [slash] Seem to have pipeline of unit test e2e executing * [slash] Add expected snitch, slash amounts * [slash] Checkpoint * [slash] Unit test correctly checks case of validator own stake which could drop below 1 ONE in slashing * [config] add double-sign testnet config (#1) Signed-off-by: Leo Chen <leo@harmony.one> * [slash] Commit for as is code & data of current dump.json * [slash] Order of state operation not correct in test, hence bad results, thank you dlv * [slash] Add snapshot state dump * [slash] Pay off slash of validator own delegation correctly * [slash] Pay off slash debt with special case for min-self * [slash] Pass first scenario conclusively * [slash] 2% slash passes unit test for own delegation and external * [slash] Parameterize unit test to easily test .02 vs .80 slash * [slash] Handle own delegation correctly at 80% slash * [slash] Have 80% slash working with external delegator * [slash] Remove debug code from slash * [slash] Adjust Apply signature, test again for 2% slash * [slash] Factor out scenario in testing so can test 2% and 80% at same time * [slash] Correct balance deduction on plan delegation * [slash] Mock out ChainReader for TestVerify * [slash] Small surface area interface, now feedback loop for verify * [slash] Remove development json * [slash] trigger-double-sign consumes yaml * [slash] Remove dead code * [slash][test] Factor ValidatorWrapper into scenario * [slash][test] Add example from local-testing dump - caution might be off * [slash] Factor out mutation of slashDebt * [slash][test] Factor out tests so can easily load test-case from bytes * [slash] Fix payment mistake in validator own delegation wrt min-self-delgation respected * [slash] Satisfy Travis * [slash] Begin cleanup of PR * [slash] Apply slash from header to Finalize via state processor * [slash] Productionize code, Println => logs; adjust slash picked in newblock * [slash] Need pointer for rlp.Decode * [slash] ValidatorInformation use full wrapper * Fix median stake * [staking] Adjust MarshalJSON for Validator, Wrapper * Refactor offchain data commit; Make block onchain/offchain commit atomic (#2279) * Refactor offchain data; Add epoch to ValidatorSnapshot * Make block onchain/offchain data commit atomically * [slash][committee] Set .Active to false on double sign, do not consider banned or inactive for committee assignment * [effective] VC eligible.go * [consensus] Redundant field in printf * [docker] import-ks for a dev account * [slash] Create BLS key for dockerfile and crt-validator.sh * [slash][docker] Easy deployment of double-sign testing * [docker] Have slash work as single docker command * [rpc] Fix median-stake RPC * [slash] Update webhook with default docker BLS key * [docker][slash] Fresh yaml copy for docker build, remove dev code in main.go * [slash] Remove helper binary, commented out code, change to local config * [params] Factor out test genesis value * Add shard checking to Tx-Pool & correct blacklist (#2301) * [core] Fix blacklist & add shardID check * [staking + node + cmd] Fix blacklist & add shardID check * [slash] Adjust to PR comments part 1 * [docker] Use different throw away funded account * [docker] Create easier testing for delegation with private keys * [docker] Update yaml * [slash] Remove special case for slashing validator own delegation wrt min-self-delegate * [docker] Install nano as well * [slash] Early error if banned * [quorum] Expose earning account in decider marshal json * Revert "Refactor offchain data commit; Make block onchain/offchain commit atomic (#2279)" This reverts commit 9ffbf682c075b49188923c65a0bbf39ac188be00. * [slash] Add non-sanity check way to update validator * [reward] Increase percision on percentage in schedule * [slash] Adjust logs * [committee] Check eligibility of validator before doing sanity check * [slash] Update docker * [slash] Move create validator script to test * [slash] More log * [param] Make things faster * [slash][off-chain] Clear out slashes from pending in writeblockwithstate * [cross-link] Log is not error, just info * [blockchain] Not necessary to guard DeletePendingSlashingCandidates * [slash][consensus] Use plain []byte for signature b/c bls.Sign has private impl fields, rlp does not encode that * [slash][test] Use faucet as sender, assume user imported * [slash] Test setup * [slash] reserve error for real error in logs * [slash][availability] Apply availability correct, bump signing count each block * [slash][staking] Consider banned field in sanity check, pay snitch only half of what was actually slashed * [slash] Pay as much as can * [slash] use right nowAmt * [slash] Take away from rewards as well * [slash] iterate faster * [slash] Remove dev based timing * [slash] Add more log, sanity check incoming slash records, only count external for slash rate * [availability][state] Adjust signature of ValidatorWrapper wrt state, filter out for staked validators, correct availaibility measure on running counters * [availability] More log * [slash] Simply pre slash erra slashing * [slash] Remove development code * [slash] Use height from recvMsg, todo on epoch * [staking] Not necessary to touch LastEpochInCommittee in staking_verifier * [slash] Undo ds in endpoint pattern config * [slash] Add TODO and log when delegation becomes 0 b/c slash debt payment * [slash] Abstract staked validators from shard.State into type, set slash rate based BLSKey count Co-authored-by: Leo Chen <leo@harmony.one> Co-authored-by: flicker-harmony <52401354+flicker-harmony@users.noreply.github.com> Co-authored-by: Rongjian Lan <rongjian@harmony.one> Co-authored-by: Daniel Van Der Maden <daniel@harmony.one>
5 years ago
if err != nil ||
len(lastCommits) < bls.BLSSignatureSizeInBytes {
msgs := consensus.FBFTLog.GetMessagesByTypeSeq(
msg_pb.MessageType_COMMITTED, blockNum,
)
if len(msgs) != 1 {
consensus.getLogger().Error().
Int("numCommittedMsg", len(msgs)).
Msg("GetLastCommitSig failed with wrong number of committed message")
return nil, nil, errors.Errorf(
"GetLastCommitSig failed with wrong number of committed message %d", len(msgs),
)
}
lastCommits = msgs[0].Payload
}
//#### Read payload data from committed msg
aggSig := make([]byte, bls.BLSSignatureSizeInBytes)
bitmap := make([]byte, len(lastCommits)-bls.BLSSignatureSizeInBytes)
offset := 0
copy(aggSig[:], lastCommits[offset:offset+bls.BLSSignatureSizeInBytes])
offset += bls.BLSSignatureSizeInBytes
copy(bitmap[:], lastCommits[offset:])
//#### END Read payload data from committed msg
return aggSig, bitmap, nil
}
// try to catch up if fall behind
func (consensus *Consensus) tryCatchup() {
consensus.getLogger().Info().Msg("[TryCatchup] commit new blocks")
currentBlockNum := consensus.blockNum
for {
[rpc][availability][apr] Richer validator information, implement APR, unify EPoS computation, remove fall 2019 tech debt (#2484) * [rpc][validator] Extend hmy blockchain validator information * [availability] Optimize bump count * [staking][validator][rpc] Remove validator stats rpc, fold into validator information, make existing pattern default behavior * [slash] Reimplement SetDifference * [reward][engine][network] Remove bad API from fall, begin setup for Per validator awards * [header] Custom Marshal header for downstream, remove dev code * [effective][committee] Factor out EPoS round of computation thereby unification in codebase of EPoS * [unit-test] Fix semantically wrong validator unit tests, punt on maxBLS key wrt tx-pool test * [reward] Use excellent singleflight package for caching lookup of subcommittees * [apr][reward] Begin APR package itself, iterate on iterface signatures * [reward] Handle possible error from singleflight * [rpc][validator][reward] Adjust RPC committees, singleflight on votingPower, foldStats into Validator Information * [apr] Stub out computation of APR * [effective][committee] Upgrade SlotPurchase with named fields, provide marshal * [effective] Update Tests * [blockchain] TODO Remove the validators no longer in committee * [validator][effective] More expressive string representation of eligibilty, ValidatorRPC explicit say if in committee now * [rpc] Median-stake more semantic meaningful * [validator] Iterate on semantic meaning of JSON representation * [offchain] Make validator stats return explicit error * [availability] Small typo * [rpc] Quick visual hack until fix delete out kicked out validators * [offchain] Delete validator from offchain that lost their slot * [apr] Forgot to update interface signature * [apr] Mul instead of Div * [protocol][validator] Fold block reward accum per vaidator into validator-wrapper, off-chain => on-chain * [votepower] Refactor votepower Roster, simplify aggregation of network wide rosters * [votepower][shard] Adjust roster, optimize usage of BLSPublicKey as key, use MarshalText trick * [shard] Granular errors * [votepower][validator] Unify votepower data structure with off-chain usage * [votepower][consensus][validator] Further simplify and unify votepower with off-chain, validator stats * [votepower] Use RJs naming convention group,overall * [votepower] Remove Println, do keep enforcing order * [effective][reward] Expand semantics of eligibility as it was overloaded and confusing, evict old voting power computations * [apr] Adjust json field name * [votepower] Only aggregate on external validator * [votepower] Mistake on aggregation, custom presentation network-wide * [rpc][validator][availability] Remove parameter, take into account empty snapshot * [apr] Use snapshots from two, one epochs ago. Still have question on header * [apr] Use GetHeaderByNumber for the header needed for time stamp * [chain] Evict > 3 epoch old voting power * [blockchain] Leave Delete Validator snapshot as TODO * [validator][rpc][effective] Undo changes to Protocol field, use virtual construct at RPC layer for meaning * [project] Address PR comments * [committee][rpc] Move +1 to computation of epos round rather than hack mutation * [reward] Remove entire unnecessary loop, hook on AddReward. Remove unnecessary new big int * [votepower][rpc][validator] Stick with numeric.Dec for token involved with computation, expose accumulate block-reward in RPC * [effective][committee] Track the candidates for the EPoS auction, RPC median-stake benefits * [node] Add hack way to get real error reason of why cannot load shardchain * [consensus] Expand log on current issue on nil block * [apr] Do the actual call to compute for validator's APR * [committee] Wrap SlotOrder with validator address, manifests in median-stake RPC * [apr] Incorrect error handle order * [quorum] Remove incorrect compare on bls Key, (typo), remove redundant error check * [shard] Add log if stakedSlots is 0 * [apr] More sanity check on div by zero, more lenient on error when dont have historical data yet * [committee] Remove + 1 on seat count * [apr] Use int64() directly * [apr] Log when odd empty nil header * [apr] Do not crash on empty header, figure out later
5 years ago
msgs := consensus.FBFTLog.GetMessagesByTypeSeq(
msg_pb.MessageType_COMMITTED, consensus.blockNum,
)
if len(msgs) == 0 {
break
}
if len(msgs) > 1 {
consensus.getLogger().Error().
Int("numMsgs", len(msgs)).
Msg("[TryCatchup] DANGER!!! we should only get one committed message for a given blockNum")
}
var committedMsg *FBFTMessage
var block *types.Block
for i := range msgs {
tmpBlock := consensus.FBFTLog.GetBlockByHash(msgs[i].BlockHash)
if tmpBlock == nil {
blksRepr, msgsRepr, incomingMsg :=
consensus.FBFTLog.Blocks().String(),
consensus.FBFTLog.Messages().String(),
msgs[i].String()
consensus.getLogger().Debug().
Str("FBFT-log-blocks", blksRepr).
Str("FBFT-log-messages", msgsRepr).
Str("incoming-message", incomingMsg).
Uint64("blockNum", msgs[i].BlockNum).
Uint64("viewID", msgs[i].ViewID).
Str("blockHash", msgs[i].BlockHash.Hex()).
Msg("[TryCatchup] Failed finding a matching block for committed message")
continue
}
committedMsg = msgs[i]
block = tmpBlock
break
}
if block == nil || committedMsg == nil {
consensus.getLogger().Error().Msg("[TryCatchup] Failed finding a valid committed message.")
break
}
if block.ParentHash() != consensus.ChainReader.CurrentHeader().Hash() {
consensus.getLogger().Debug().Msg("[TryCatchup] parent block hash not match")
break
}
if len(committedMsg.SenderPubkeys) != 1 {
consensus.getLogger().Error().Msg("[TryCatchup] Leader message can not have multiple sender keys")
break
}
consensus.getLogger().Info().Msg("[TryCatchup] block found to commit")
atomic.AddUint64(&consensus.blockNum, 1)
consensus.SetCurViewID(committedMsg.ViewID + 1)
consensus.LeaderPubKey = committedMsg.SenderPubkeys[0]
consensus.getLogger().Info().Msg("[TryCatchup] Adding block to chain")
// Fill in the commit signatures
block.SetCurrentCommitSig(committedMsg.Payload)
consensus.OnConsensusDone(block)
consensus.ResetState()
select {
case consensus.VerifiedNewBlock <- block:
default:
consensus.getLogger().Info().
Str("blockHash", block.Hash().String()).
Msg("[TryCatchup] consensus verified block send to chan failed")
continue
}
break
}
if currentBlockNum < consensus.blockNum {
consensus.getLogger().Info().
Uint64("From", currentBlockNum).
Uint64("To", consensus.blockNum).
Msg("[TryCatchup] Caught up!")
consensus.switchPhase(FBFTAnnounce, true)
}
// catup up and skip from view change trap
if currentBlockNum < consensus.blockNum &&
consensus.IsViewChangingMode() {
consensus.current.SetMode(Normal)
consensus.consensusTimeout[timeoutViewChange].Stop()
}
// clean up old log
consensus.FBFTLog.DeleteBlocksLessThan(consensus.blockNum - 1)
consensus.FBFTLog.DeleteMessagesLessThan(consensus.blockNum - 1)
}
// Start waits for the next new block and run consensus
func (consensus *Consensus) Start(
blockChannel chan *types.Block, stopChan, stoppedChan, startChannel chan struct{},
) {
go func() {
toStart := make(chan struct{}, 1)
isInitialLeader := consensus.IsLeader()
if isInitialLeader {
consensus.getLogger().Info().Time("time", time.Now()).Msg("[ConsensusMainLoop] Waiting for consensus start")
// send a signal to indicate it's ready to run consensus
// this signal is consumed by node object to create a new block and in turn trigger a new consensus on it
go func() {
<-startChannel
toStart <- struct{}{}
consensus.getLogger().Info().Time("time", time.Now()).Msg("[ConsensusMainLoop] Send ReadySignal")
consensus.ReadySignal <- struct{}{}
}()
}
consensus.getLogger().Info().Time("time", time.Now()).Msg("[ConsensusMainLoop] Consensus started")
defer close(stoppedChan)
ticker := time.NewTicker(3 * time.Second)
defer ticker.Stop()
consensus.consensusTimeout[timeoutBootstrap].Start()
consensus.getLogger().Info().Msg("[ConsensusMainLoop] Start bootstrap timeout (only once)")
vdfInProgress := false
// Set up next block due time.
consensus.NextBlockDue = time.Now().Add(consensus.BlockPeriod)
start := false
for {
select {
case <-toStart:
start = true
case <-ticker.C:
consensus.getLogger().Debug().Msg("[ConsensusMainLoop] Ticker")
if !start && isInitialLeader {
continue
}
for k, v := range consensus.consensusTimeout {
if consensus.current.Mode() == Syncing ||
consensus.current.Mode() == Listening {
v.Stop()
}
if !v.CheckExpire() {
continue
}
if k != timeoutViewChange {
consensus.getLogger().Warn().Msg("[ConsensusMainLoop] Ops Consensus Timeout!!!")
viewID := consensus.GetCurViewID()
consensus.startViewChange(viewID + 1)
break
} else {
consensus.getLogger().Warn().Msg("[ConsensusMainLoop] Ops View Change Timeout!!!")
viewID := consensus.GetViewChangingID()
consensus.startViewChange(viewID + 1)
break
}
}
case <-consensus.syncReadyChan:
consensus.getLogger().Info().Msg("[ConsensusMainLoop] syncReadyChan")
consensus.SetBlockNum(consensus.ChainReader.CurrentHeader().Number().Uint64() + 1)
consensus.SetViewIDs(consensus.ChainReader.CurrentHeader().ViewID().Uint64() + 1)
mode := consensus.UpdateConsensusInformation()
consensus.current.SetMode(mode)
consensus.getLogger().Info().Str("Mode", mode.String()).Msg("Node is IN SYNC")
case <-consensus.syncNotReadyChan:
consensus.getLogger().Info().Msg("[ConsensusMainLoop] syncNotReadyChan")
consensus.SetBlockNum(consensus.ChainReader.CurrentHeader().Number().Uint64() + 1)
consensus.current.SetMode(Syncing)
consensus.getLogger().Info().Msg("[ConsensusMainLoop] Node is OUT OF SYNC")
case newBlock := <-blockChannel:
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Msg("[ConsensusMainLoop] Received Proposed New Block!")
//VRF/VDF is only generated in the beacon chain
if consensus.NeedsRandomNumberGeneration(newBlock.Header().Epoch()) {
// generate VRF if the current block has a new leader
if !consensus.ChainReader.IsSameLeaderAsPreviousBlock(newBlock) {
vrfBlockNumbers, err := consensus.ChainReader.ReadEpochVrfBlockNums(newBlock.Header().Epoch())
if err != nil {
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Uint64("Epoch", newBlock.Header().Epoch().Uint64()).
Msg("[ConsensusMainLoop] no VRF block number from local db")
}
//check if VRF is already generated for the current block
vrfAlreadyGenerated := false
for _, v := range vrfBlockNumbers {
if v == newBlock.NumberU64() {
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Uint64("Epoch", newBlock.Header().Epoch().Uint64()).
Msg("[ConsensusMainLoop] VRF is already generated for this block")
vrfAlreadyGenerated = true
break
}
}
if !vrfAlreadyGenerated {
//generate a new VRF for the current block
vrfBlockNumbers := consensus.GenerateVrfAndProof(newBlock, vrfBlockNumbers)
//generate a new VDF for the current epoch if there are enough VRFs in the current epoch
//note that >= instead of == is used, because it is possible the current leader
//can commit this block, go offline without finishing VDF
if (!vdfInProgress) && len(vrfBlockNumbers) >= consensus.VdfSeedSize() {
//check local database to see if there's a VDF generated for this epoch
//generate a VDF if no blocknum is available
_, err := consensus.ChainReader.ReadEpochVdfBlockNum(newBlock.Header().Epoch())
if err != nil {
consensus.GenerateVdfAndProof(newBlock, vrfBlockNumbers)
vdfInProgress = true
}
}
}
}
vdfOutput, seed, err := consensus.GetNextRnd()
if err == nil {
vdfInProgress = false
// Verify the randomness
vdfObject := vdf_go.New(shard.Schedule.VdfDifficulty(), seed)
if !vdfObject.Verify(vdfOutput) {
consensus.getLogger().Warn().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Uint64("Epoch", newBlock.Header().Epoch().Uint64()).
Msg("[ConsensusMainLoop] failed to verify the VDF output")
} else {
//write the VDF only if VDF has not been generated
_, err := consensus.ChainReader.ReadEpochVdfBlockNum(newBlock.Header().Epoch())
if err == nil {
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Uint64("Epoch", newBlock.Header().Epoch().Uint64()).
Msg("[ConsensusMainLoop] VDF has already been generated previously")
} else {
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Uint64("Epoch", newBlock.Header().Epoch().Uint64()).
Msg("[ConsensusMainLoop] Generated a new VDF")
newBlock.AddVdf(vdfOutput[:])
}
}
}
}
startTime = time.Now()
consensus.msgSender.Reset(newBlock.NumberU64())
5 years ago
consensus.getLogger().Info().
Int("numTxs", len(newBlock.Transactions())).
Int("numStakingTxs", len(newBlock.StakingTransactions())).
Time("startTime", startTime).
Int64("publicKeys", consensus.Decider.ParticipantsCount()).
Msg("[ConsensusMainLoop] STARTING CONSENSUS")
consensus.announce(newBlock)
case viewID := <-consensus.commitFinishChan:
consensus.getLogger().Info().Msg("[ConsensusMainLoop] commitFinishChan")
// Only Leader execute this condition
func() {
consensus.mutex.Lock()
defer consensus.mutex.Unlock()
if viewID == consensus.GetCurViewID() {
consensus.finalizeCommits()
}
}()
case <-stopChan:
consensus.getLogger().Info().Msg("[ConsensusMainLoop] stopChan")
return
}
}
consensus.getLogger().Info().Msg("[ConsensusMainLoop] Ended.")
}()
}
// GenerateVrfAndProof generates new VRF/Proof from hash of previous block
func (consensus *Consensus) GenerateVrfAndProof(newBlock *types.Block, vrfBlockNumbers []uint64) []uint64 {
key, err := consensus.GetConsensusLeaderPrivateKey()
if err != nil {
consensus.getLogger().Error().
Err(err).
Msg("[GenerateVrfAndProof] VRF generation error")
return vrfBlockNumbers
}
sk := vrf_bls.NewVRFSigner(key.Pri)
blockHash := [32]byte{}
[double-sign] Provide proof of double sign in slash record sent to beaconchain (#2253) * [double-sign] Commit changes in consensus needed for double-sign * [double-sign] Leader captures when valdator double signs, broadcasts to beaconchain * [slash] Add quick iteration tool for testing double-signing * [slash] Add webhook example * [slash] Add http server for hook to trigger double sign behavior * [double-sign] Use bin/trigger-double-sign to cause a double-sign * [double-sign] Full feedback loop working * [slash] Thread through the slash records in the block proposal step * [slash] Compute the slashing rate * [double-sign] Generalize yaml malicious for many keys * [double-sign][slash] Modify data structures, verify via webhook handler * [slash][double-sign] Find one address of bls public key signer, seemingly settle on data structures * [slash] Apply to state slashing for double signing * [slash][double-sign] Checkpoint for working code that slashes on beaconchain * [slash] Keep track of the total slash and total reporters reward * [slash] Dump account state before and after the slash * [slash] Satisfy Travis * [slash][state] Apply slash to the snapshot at beginning of epoch, now need to capture also the new delegates * [slash] Capture the unique new delegations since snapshot as well * [slash] Filter undelegation by epoch of double sign * [slash] Add TODO of correctness needed in slash needs on off-chain data * [rpc] Fix closure issue on shardID * [slash] Add delegator to double-sign testing script * [slash] Expand crt-validator.sh with commenting printfs and make delegation * [slash] Finish track payment of leftover slash debt after undelegation runs out * [slash] Now be explicit about error wrt delegatorSlashApply * [slash] Capture specific sanity check on slash paidoff * [slash] Track slash from undelegation piecemeal * [slash][delegation] Named slice types, .String() * [slash] Do no RLP encode twice, once is enough * [slash] Remove special case of validators own delegation * [slash] Refactor approach to slash state application * [slash] Begin expanding out Verify * [slash] Slash on snapshot delegations, not current * [slash] Fix Epoch Cmp * [slash] Third iteration on slash logic * [slash] Use full slash amount * [slash] More log, whitespace * [slash] Remove Println, add log * [slash] Remove debug Println * [slash] Add record in unit test * [slash] Build Validator snapshot, current. Fill out slash record * [slash] Need to get RLP dump of a header to use in test * [slash] Factor out double sign test constants * [slash] Factor out common for validator, stub out slash application, finish out deserialization setup * [slash] Factor out data structure creation because of var lexical scoping * [slash] Seem to have pipeline of unit test e2e executing * [slash] Add expected snitch, slash amounts * [slash] Checkpoint * [slash] Unit test correctly checks case of validator own stake which could drop below 1 ONE in slashing * [config] add double-sign testnet config (#1) Signed-off-by: Leo Chen <leo@harmony.one> * [slash] Commit for as is code & data of current dump.json * [slash] Order of state operation not correct in test, hence bad results, thank you dlv * [slash] Add snapshot state dump * [slash] Pay off slash of validator own delegation correctly * [slash] Pay off slash debt with special case for min-self * [slash] Pass first scenario conclusively * [slash] 2% slash passes unit test for own delegation and external * [slash] Parameterize unit test to easily test .02 vs .80 slash * [slash] Handle own delegation correctly at 80% slash * [slash] Have 80% slash working with external delegator * [slash] Remove debug code from slash * [slash] Adjust Apply signature, test again for 2% slash * [slash] Factor out scenario in testing so can test 2% and 80% at same time * [slash] Correct balance deduction on plan delegation * [slash] Mock out ChainReader for TestVerify * [slash] Small surface area interface, now feedback loop for verify * [slash] Remove development json * [slash] trigger-double-sign consumes yaml * [slash] Remove dead code * [slash][test] Factor ValidatorWrapper into scenario * [slash][test] Add example from local-testing dump - caution might be off * [slash] Factor out mutation of slashDebt * [slash][test] Factor out tests so can easily load test-case from bytes * [slash] Fix payment mistake in validator own delegation wrt min-self-delgation respected * [slash] Satisfy Travis * [slash] Begin cleanup of PR * [slash] Apply slash from header to Finalize via state processor * [slash] Productionize code, Println => logs; adjust slash picked in newblock * [slash] Need pointer for rlp.Decode * [slash] ValidatorInformation use full wrapper * Fix median stake * [staking] Adjust MarshalJSON for Validator, Wrapper * Refactor offchain data commit; Make block onchain/offchain commit atomic (#2279) * Refactor offchain data; Add epoch to ValidatorSnapshot * Make block onchain/offchain data commit atomically * [slash][committee] Set .Active to false on double sign, do not consider banned or inactive for committee assignment * [effective] VC eligible.go * [consensus] Redundant field in printf * [docker] import-ks for a dev account * [slash] Create BLS key for dockerfile and crt-validator.sh * [slash][docker] Easy deployment of double-sign testing * [docker] Have slash work as single docker command * [rpc] Fix median-stake RPC * [slash] Update webhook with default docker BLS key * [docker][slash] Fresh yaml copy for docker build, remove dev code in main.go * [slash] Remove helper binary, commented out code, change to local config * [params] Factor out test genesis value * Add shard checking to Tx-Pool & correct blacklist (#2301) * [core] Fix blacklist & add shardID check * [staking + node + cmd] Fix blacklist & add shardID check * [slash] Adjust to PR comments part 1 * [docker] Use different throw away funded account * [docker] Create easier testing for delegation with private keys * [docker] Update yaml * [slash] Remove special case for slashing validator own delegation wrt min-self-delegate * [docker] Install nano as well * [slash] Early error if banned * [quorum] Expose earning account in decider marshal json * Revert "Refactor offchain data commit; Make block onchain/offchain commit atomic (#2279)" This reverts commit 9ffbf682c075b49188923c65a0bbf39ac188be00. * [slash] Add non-sanity check way to update validator * [reward] Increase percision on percentage in schedule * [slash] Adjust logs * [committee] Check eligibility of validator before doing sanity check * [slash] Update docker * [slash] Move create validator script to test * [slash] More log * [param] Make things faster * [slash][off-chain] Clear out slashes from pending in writeblockwithstate * [cross-link] Log is not error, just info * [blockchain] Not necessary to guard DeletePendingSlashingCandidates * [slash][consensus] Use plain []byte for signature b/c bls.Sign has private impl fields, rlp does not encode that * [slash][test] Use faucet as sender, assume user imported * [slash] Test setup * [slash] reserve error for real error in logs * [slash][availability] Apply availability correct, bump signing count each block * [slash][staking] Consider banned field in sanity check, pay snitch only half of what was actually slashed * [slash] Pay as much as can * [slash] use right nowAmt * [slash] Take away from rewards as well * [slash] iterate faster * [slash] Remove dev based timing * [slash] Add more log, sanity check incoming slash records, only count external for slash rate * [availability][state] Adjust signature of ValidatorWrapper wrt state, filter out for staked validators, correct availaibility measure on running counters * [availability] More log * [slash] Simply pre slash erra slashing * [slash] Remove development code * [slash] Use height from recvMsg, todo on epoch * [staking] Not necessary to touch LastEpochInCommittee in staking_verifier * [slash] Undo ds in endpoint pattern config * [slash] Add TODO and log when delegation becomes 0 b/c slash debt payment * [slash] Abstract staked validators from shard.State into type, set slash rate based BLSKey count Co-authored-by: Leo Chen <leo@harmony.one> Co-authored-by: flicker-harmony <52401354+flicker-harmony@users.noreply.github.com> Co-authored-by: Rongjian Lan <rongjian@harmony.one> Co-authored-by: Daniel Van Der Maden <daniel@harmony.one>
5 years ago
previousHeader := consensus.ChainReader.GetHeaderByNumber(
newBlock.NumberU64() - 1,
)
if previousHeader == nil {
return vrfBlockNumbers
}
previousHash := previousHeader.Hash()
copy(blockHash[:], previousHash[:])
vrf, proof := sk.Evaluate(blockHash[:])
newBlock.AddVrf(append(vrf[:], proof...))
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Uint64("Epoch", newBlock.Header().Epoch().Uint64()).
Int("Num of VRF", len(vrfBlockNumbers)).
Msg("[ConsensusMainLoop] Leader generated a VRF")
return vrfBlockNumbers
}
// ValidateVrfAndProof validates a VRF/Proof from hash of previous block
func (consensus *Consensus) ValidateVrfAndProof(headerObj *block.Header) bool {
vrfPk := vrf_bls.NewVRFVerifier(consensus.LeaderPubKey.Object)
var blockHash [32]byte
previousHeader := consensus.ChainReader.GetHeaderByNumber(
headerObj.Number().Uint64() - 1,
)
if previousHeader == nil {
return false
}
previousHash := previousHeader.Hash()
copy(blockHash[:], previousHash[:])
vrfProof := [96]byte{}
copy(vrfProof[:], headerObj.Vrf()[32:])
hash, err := vrfPk.ProofToHash(blockHash[:], vrfProof[:])
if err != nil {
consensus.getLogger().Warn().
Err(err).
Str("MsgBlockNum", headerObj.Number().String()).
Msg("[OnAnnounce] VRF verification error")
return false
}
if !bytes.Equal(hash[:], headerObj.Vrf()[:32]) {
consensus.getLogger().Warn().
Str("MsgBlockNum", headerObj.Number().String()).
Msg("[OnAnnounce] VRF proof is not valid")
return false
}
vrfBlockNumbers, _ := consensus.ChainReader.ReadEpochVrfBlockNums(
headerObj.Epoch(),
)
consensus.getLogger().Info().
Str("MsgBlockNum", headerObj.Number().String()).
Int("Number of VRF", len(vrfBlockNumbers)).
Msg("[OnAnnounce] validated a new VRF")
return true
}
// GenerateVdfAndProof generates new VDF/Proof from VRFs in the current epoch
func (consensus *Consensus) GenerateVdfAndProof(newBlock *types.Block, vrfBlockNumbers []uint64) {
//derive VDF seed from VRFs generated in the current epoch
seed := [32]byte{}
for i := 0; i < consensus.VdfSeedSize(); i++ {
previousVrf := consensus.ChainReader.GetVrfByNumber(vrfBlockNumbers[i])
for j := 0; j < len(seed); j++ {
seed[j] = seed[j] ^ previousVrf[j]
}
}
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Uint64("Epoch", newBlock.Header().Epoch().Uint64()).
Int("Num of VRF", len(vrfBlockNumbers)).
Msg("[ConsensusMainLoop] VDF computation started")
// TODO ek – limit concurrency
go func() {
vdf := vdf_go.New(shard.Schedule.VdfDifficulty(), seed)
outputChannel := vdf.GetOutputChannel()
start := time.Now()
vdf.Execute()
duration := time.Since(start)
consensus.getLogger().Info().
Dur("duration", duration).
Msg("[ConsensusMainLoop] VDF computation finished")
output := <-outputChannel
// The first 516 bytes are the VDF+proof and the last 32 bytes are XORed VRF as seed
rndBytes := [548]byte{}
copy(rndBytes[:516], output[:])
copy(rndBytes[516:], seed[:])
consensus.RndChannel <- rndBytes
}()
}
// ValidateVdfAndProof validates the VDF/proof in the current epoch
func (consensus *Consensus) ValidateVdfAndProof(headerObj *block.Header) bool {
vrfBlockNumbers, err := consensus.ChainReader.ReadEpochVrfBlockNums(headerObj.Epoch())
if err != nil {
consensus.getLogger().Error().Err(err).
Str("MsgBlockNum", headerObj.Number().String()).
Msg("[OnAnnounce] failed to read VRF block numbers for VDF computation")
}
//extra check to make sure there's no index out of range error
//it can happen if epoch is messed up, i.e. VDF ouput is generated in the next epoch
if consensus.VdfSeedSize() > len(vrfBlockNumbers) {
return false
}
seed := [32]byte{}
for i := 0; i < consensus.VdfSeedSize(); i++ {
previousVrf := consensus.ChainReader.GetVrfByNumber(vrfBlockNumbers[i])
for j := 0; j < len(seed); j++ {
seed[j] = seed[j] ^ previousVrf[j]
}
}
vdfObject := vdf_go.New(shard.Schedule.VdfDifficulty(), seed)
vdfOutput := [516]byte{}
copy(vdfOutput[:], headerObj.Vdf())
if vdfObject.Verify(vdfOutput) {
consensus.getLogger().Info().
Str("MsgBlockNum", headerObj.Number().String()).
Int("Num of VRF", consensus.VdfSeedSize()).
Msg("[OnAnnounce] validated a new VDF")
} else {
consensus.getLogger().Warn().
Str("MsgBlockNum", headerObj.Number().String()).
Uint64("Epoch", headerObj.Epoch().Uint64()).
Int("Num of VRF", consensus.VdfSeedSize()).
Msg("[OnAnnounce] VDF proof is not valid")
return false
}
return true
}