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/view_change.go

428 lines
14 KiB

package consensus
import (
"sync"
"time"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/ethereum/go-ethereum/common"
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/consensus/quorum"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/pkg/errors"
)
// MaxViewIDDiff limits the received view ID to only 249 further from the current view ID
const MaxViewIDDiff = 249
// State contains current mode and current viewID
type State struct {
mode Mode
modeMux sync.RWMutex
// current view id in normal mode
// it changes per successful consensus
blockViewID uint64
cViewMux sync.RWMutex
// view changing id is used during view change mode
// it is the next view id
viewChangingID uint64
viewMux sync.RWMutex
}
// Mode return the current node mode
func (pm *State) Mode() Mode {
pm.modeMux.RLock()
defer pm.modeMux.RUnlock()
return pm.mode
}
// SetMode set the node mode as required
func (pm *State) SetMode(s Mode) {
pm.modeMux.Lock()
defer pm.modeMux.Unlock()
pm.mode = s
}
// GetCurBlockViewID return the current view id
func (pm *State) GetCurBlockViewID() uint64 {
pm.cViewMux.RLock()
defer pm.cViewMux.RUnlock()
return pm.blockViewID
}
// SetCurBlockViewID sets the current view id
func (pm *State) SetCurBlockViewID(viewID uint64) {
pm.cViewMux.Lock()
defer pm.cViewMux.Unlock()
pm.blockViewID = viewID
}
// GetViewChangingID return the current view changing id
// It is meaningful during view change mode
func (pm *State) GetViewChangingID() uint64 {
pm.viewMux.RLock()
defer pm.viewMux.RUnlock()
return pm.viewChangingID
}
// SetViewChangingID set the current view changing id
// It is meaningful during view change mode
func (pm *State) SetViewChangingID(id uint64) {
pm.viewMux.Lock()
defer pm.viewMux.Unlock()
pm.viewChangingID = id
}
// GetViewChangeDuraion return the duration of the current view change
// It increase in the power of difference betweeen view changing ID and current view ID
func (pm *State) GetViewChangeDuraion() time.Duration {
pm.viewMux.RLock()
pm.cViewMux.RLock()
defer pm.viewMux.RUnlock()
defer pm.cViewMux.RUnlock()
diff := int64(pm.viewChangingID - pm.blockViewID)
return time.Duration(diff * diff * int64(viewChangeDuration))
}
// GetNextLeaderKey uniquely determine who is the leader for given viewID
func (consensus *Consensus) GetNextLeaderKey(viewID uint64) *bls.PublicKeyWrapper {
gap := 1
consensus.getLogger().Info().
Str("leaderPubKey", consensus.LeaderPubKey.Bytes.Hex()).
Uint64("newViewID", viewID).
Uint64("myCurBlockViewID", consensus.GetCurBlockViewID()).
Msg("[GetNextLeaderKey] got leaderPubKey from coinbase")
wasFound, next := consensus.Decider.NthNext(consensus.LeaderPubKey, gap)
if !wasFound {
consensus.getLogger().Warn().
Str("key", consensus.LeaderPubKey.Bytes.Hex()).
Msg("GetNextLeaderKey: currentLeaderKey not found")
}
return next
}
func createTimeout() map[TimeoutType]*utils.Timeout {
timeouts := make(map[TimeoutType]*utils.Timeout)
timeouts[timeoutConsensus] = utils.NewTimeout(phaseDuration)
timeouts[timeoutViewChange] = utils.NewTimeout(viewChangeDuration)
timeouts[timeoutBootstrap] = utils.NewTimeout(bootstrapDuration)
return timeouts
}
// startViewChange send a new view change
// the viewID is the current viewID
func (consensus *Consensus) startViewChange(viewID uint64) {
if consensus.disableViewChange {
return
}
consensus.consensusTimeout[timeoutConsensus].Stop()
consensus.consensusTimeout[timeoutBootstrap].Stop()
consensus.current.SetMode(ViewChanging)
consensus.SetViewChangingID(viewID)
consensus.LeaderPubKey = consensus.GetNextLeaderKey(viewID)
duration := consensus.current.GetViewChangeDuraion()
consensus.getLogger().Warn().
Uint64("viewID", viewID).
Uint64("viewChangingID", consensus.GetViewChangingID()).
Dur("timeoutDuration", duration).
Str("NextLeader", consensus.LeaderPubKey.Bytes.Hex()).
Msg("[startViewChange]")
consensus.consensusTimeout[timeoutViewChange].SetDuration(duration)
defer consensus.consensusTimeout[timeoutViewChange].Start()
// update the dictionary key if the viewID is first time received
consensus.vc.AddViewIDKeyIfNotExist(viewID, consensus.Decider.Participants())
// init my own payload
if err := consensus.vc.InitPayload(
consensus.FBFTLog,
viewID,
consensus.blockNum,
consensus.priKey); err != nil {
consensus.getLogger().Error().Err(err).Msg("Init Payload Error")
}
// for view change, send separate view change per public key
// do not do multi-sign of view change message
for _, key := range consensus.priKey {
if !consensus.IsValidatorInCommittee(key.Pub.Bytes) {
continue
}
msgToSend := consensus.constructViewChangeMessage(&key)
if err := consensus.msgSender.SendWithRetry(
consensus.blockNum,
msg_pb.MessageType_VIEWCHANGE,
[]nodeconfig.GroupID{
nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID))},
p2p.ConstructMessage(msgToSend),
); err != nil {
consensus.getLogger().Err(err).
Msg("could not send out the ViewChange message")
}
}
}
// startNewView stops the current view change
func (consensus *Consensus) startNewView(viewID uint64, newLeaderPriKey *bls.PrivateKeyWrapper) error {
consensus.mutex.Lock()
defer consensus.mutex.Unlock()
msgToSend := consensus.constructNewViewMessage(
viewID, newLeaderPriKey,
)
if err := consensus.msgSender.SendWithRetry(
consensus.blockNum,
msg_pb.MessageType_NEWVIEW,
[]nodeconfig.GroupID{
nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID))},
p2p.ConstructMessage(msgToSend),
); err != nil {
return errors.New("failed to send out the NewView message")
}
consensus.getLogger().Info().
Str("myKey", newLeaderPriKey.Pub.Bytes.Hex()).
Hex("M1Payload", consensus.vc.GetM1Payload()).
Msg("[startNewView] Sent NewView Messge")
consensus.current.SetMode(Normal)
consensus.consensusTimeout[timeoutViewChange].Stop()
consensus.SetViewIDs(viewID)
consensus.ResetViewChangeState()
consensus.consensusTimeout[timeoutConsensus].Start()
consensus.getLogger().Info().
Uint64("viewID", viewID).
Str("myKey", newLeaderPriKey.Pub.Bytes.Hex()).
Msg("[startNewView] viewChange stopped. I am the New Leader")
consensus.ResetState()
consensus.LeaderPubKey = newLeaderPriKey.Pub
return nil
}
// onViewChange is called when the view change message is received.
func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
consensus.getLogger().Info().Msg("[onViewChange] Received ViewChange Message")
recvMsg, err := ParseViewChangeMessage(msg)
if err != nil {
consensus.getLogger().Warn().Err(err).Msg("[onViewChange] Unable To Parse Viewchange Message")
return
}
// if not leader, noop
newLeaderKey := recvMsg.LeaderPubkey
newLeaderPriKey, err := consensus.GetLeaderPrivateKey(newLeaderKey.Object)
if err != nil {
consensus.getLogger().Info().
Err(err).
Interface("SenderPubkeys", recvMsg.SenderPubkeys).
Str("NextLeader", recvMsg.LeaderPubkey.Bytes.Hex()).
Str("myBLSPubKey", consensus.priKey.GetPublicKeys().SerializeToHexStr()).
Msg("[onViewChange] I am not the Leader")
return
}
if consensus.Decider.IsQuorumAchieved(quorum.ViewChange) {
consensus.getLogger().Info().
Int64("have", consensus.Decider.SignersCount(quorum.ViewChange)).
Int64("need", consensus.Decider.TwoThirdsSignersCount()).
Interface("SenderPubkeys", recvMsg.SenderPubkeys).
Str("newLeaderKey", newLeaderKey.Bytes.Hex()).
Msg("[onViewChange] Received Enough View Change Messages")
return
}
if !consensus.onViewChangeSanityCheck(recvMsg) {
return
}
// already checked the length of SenderPubkeys in onViewChangeSanityCheck
senderKey := recvMsg.SenderPubkeys[0]
5 years ago
// update the dictionary key if the viewID is first time received
members := consensus.Decider.Participants()
consensus.vc.AddViewIDKeyIfNotExist(recvMsg.ViewID, members)
// do it once only per viewID/Leader
if err := consensus.vc.InitPayload(consensus.FBFTLog,
recvMsg.ViewID,
recvMsg.BlockNum,
consensus.priKey); err != nil {
consensus.getLogger().Error().Err(err).Msg("[onViewChange] Init Payload Error")
return
}
err = consensus.vc.ProcessViewChangeMsg(consensus.FBFTLog, consensus.Decider, recvMsg)
if err != nil {
consensus.getLogger().Error().Err(err).
Uint64("viewID", recvMsg.ViewID).
Uint64("blockNum", recvMsg.BlockNum).
Str("msgSender", senderKey.Bytes.Hex()).
Msg("[onViewChange] process View Change message error")
return
}
// received enough view change messages, change state to normal consensus
if consensus.Decider.IsQuorumAchievedByMask(consensus.vc.GetViewIDBitmap(recvMsg.ViewID)) && consensus.IsViewChangingMode() {
// no previous prepared message, go straight to normal mode
// and start proposing new block
if consensus.vc.IsM1PayloadEmpty() {
if err := consensus.startNewView(recvMsg.ViewID, newLeaderPriKey); err != nil {
consensus.getLogger().Error().Err(err).Msg("[onViewChange] startNewView failed")
return
}
go func() {
consensus.ReadySignal <- struct{}{}
}()
return
}
payload := consensus.vc.GetM1Payload()
if err := consensus.selfCommit(payload); err != nil {
consensus.getLogger().Error().Err(err).Msg("[onViewChange] self commit failed")
return
[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 := consensus.startNewView(recvMsg.ViewID, newLeaderPriKey); err != nil {
consensus.getLogger().Error().Err(err).Msg("[onViewChange] startNewView failed")
return
[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
}
}
}
// onNewView is called when validators received newView message from the new leader
// the validator needs to check the m3bitmap to see if the quorum is reached
// If the new view message contains payload (block), and at least one m1 message was
// collected by the new leader (m3count > m2count), the validator will create a new
// prepared message from the payload and commit it to the block
// Or the validator will enter announce phase to wait for the new block proposed
// from the new leader
func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
consensus.getLogger().Info().Msg("[onNewView] Received NewView Message")
members := consensus.Decider.Participants()
recvMsg, err := ParseNewViewMessage(msg, members)
if err != nil {
consensus.getLogger().Warn().Err(err).Msg("[onNewView] Unable to Parse NewView Message")
return
}
// change view and leaderKey to keep in sync with network
if consensus.blockNum != recvMsg.BlockNum {
consensus.getLogger().Warn().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("myBlockNum", consensus.blockNum).
Msg("[onNewView] Invalid block number")
return
}
if len(recvMsg.SenderPubkeys) != 1 {
consensus.getLogger().Error().Msg("[onNewView] multiple signers in view change message.")
return
}
senderKey := recvMsg.SenderPubkeys[0]
if !consensus.onNewViewSanityCheck(recvMsg) {
return
}
preparedBlock, err := consensus.vc.VerifyNewViewMsg(recvMsg)
if err != nil {
consensus.getLogger().Warn().Err(err).Msg("[onNewView] Verify New View Msg Failed")
return
}
m3Mask := recvMsg.M3Bitmap
if !consensus.Decider.IsQuorumAchievedByMask(m3Mask) {
consensus.getLogger().Warn().
Msgf("[onNewView] Quorum Not achieved")
return
}
m2Mask := recvMsg.M2Bitmap
if m2Mask == nil || m2Mask.Bitmap == nil ||
(m2Mask != nil && m2Mask.Bitmap != nil &&
utils.CountOneBits(m3Mask.Bitmap) > utils.CountOneBits(m2Mask.Bitmap)) {
// m1 is not empty, check it's valid
blockHash := recvMsg.Payload[:32]
aggSig, mask, err := consensus.ReadSignatureBitmapPayload(recvMsg.Payload, 32)
if err != nil {
consensus.getLogger().Error().Err(err).
Msg("[onNewView] ReadSignatureBitmapPayload Failed")
return
}
if !aggSig.VerifyHash(mask.AggregatePublic, blockHash) {
consensus.getLogger().Warn().
Msg("[onNewView] Failed to Verify Signature for M1 (prepare) message")
return
}
consensus.mutex.Lock()
copy(consensus.blockHash[:], blockHash)
consensus.aggregatedPrepareSig = aggSig
consensus.prepareBitmap = mask
consensus.mutex.Unlock()
// create prepared message from newview
preparedMsg := FBFTMessage{
MessageType: msg_pb.MessageType_PREPARED,
ViewID: recvMsg.ViewID,
BlockNum: recvMsg.BlockNum,
}
preparedMsg.BlockHash = common.Hash{}
copy(preparedMsg.BlockHash[:], blockHash[:])
preparedMsg.Payload = make([]byte, len(recvMsg.Payload)-32)
copy(preparedMsg.Payload[:], recvMsg.Payload[32:])
preparedMsg.SenderPubkeys = []*bls.PublicKeyWrapper{senderKey}
consensus.FBFTLog.AddMessage(&preparedMsg)
if preparedBlock != nil {
consensus.FBFTLog.AddBlock(preparedBlock)
}
}
if !consensus.IsViewChangingMode() {
consensus.getLogger().Info().Msg("Not in ViewChanging Mode.")
return
}
consensus.mutex.Lock()
defer consensus.mutex.Unlock()
consensus.consensusTimeout[timeoutViewChange].Stop()
// newView message verified success, override my state
consensus.SetViewIDs(recvMsg.ViewID)
consensus.LeaderPubKey = senderKey
consensus.ResetViewChangeState()
// NewView message is verified, change state to normal consensus
if preparedBlock != nil {
consensus.sendCommitMessages(preparedBlock)
consensus.switchPhase("onNewView", FBFTCommit)
} else {
consensus.ResetState()
consensus.getLogger().Info().Msg("onNewView === announce")
}
consensus.getLogger().Info().
Str("newLeaderKey", consensus.LeaderPubKey.Bytes.Hex()).
Msg("new leader changed")
consensus.consensusTimeout[timeoutConsensus].Start()
}
// ResetViewChangeState resets the view change structure
func (consensus *Consensus) ResetViewChangeState() {
consensus.getLogger().Info().
Str("Phase", consensus.phase.String()).
Msg("[ResetViewChangeState] Resetting view change state")
consensus.current.SetMode(Normal)
consensus.vc.Reset()
consensus.Decider.ResetViewChangeVotes()
}