make consensus log verbose

pull/1970/head
chao 5 years ago
parent b7282a1ef3
commit 20feb3a2f7
  1. 239
      consensus/consensus_v2.go
  2. 104
      consensus/view_change.go

@ -19,7 +19,6 @@ import (
"github.com/harmony-one/harmony/internal/chain"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p/host"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/vdf/src/vdf_go"
@ -50,7 +49,7 @@ func (consensus *Consensus) handleMessageUpdate(payload []byte) {
if msg.Type == msg_pb.MessageType_VIEWCHANGE || msg.Type == msg_pb.MessageType_NEWVIEW {
if msg.GetViewchange() != nil && msg.GetViewchange().ShardId != consensus.ShardID {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint32("myShardId", consensus.ShardID).
Uint32("receivedShardId", msg.GetViewchange().ShardId).
Msg("Received view change message from different shard")
@ -58,7 +57,7 @@ func (consensus *Consensus) handleMessageUpdate(payload []byte) {
}
} else {
if msg.GetConsensus() != nil && msg.GetConsensus().ShardId != consensus.ShardID {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint32("myShardId", consensus.ShardID).
Uint32("receivedShardId", msg.GetConsensus().ShardId).
Msg("Received consensus message from different shard")
@ -100,12 +99,12 @@ func (consensus *Consensus) announce(block *types.Block) {
// prepare message and broadcast to validators
encodedBlock, err := rlp.EncodeToBytes(block)
if err != nil {
utils.Logger().Debug().Msg("[Announce] Failed encoding block")
consensus.getLogger().Debug().Msg("[Announce] Failed encoding block")
return
}
encodedBlockHeader, err := rlp.EncodeToBytes(block.Header())
if err != nil {
utils.Logger().Debug().Msg("[Announce] Failed encoding block header")
consensus.getLogger().Debug().Msg("[Announce] Failed encoding block header")
return
}
@ -120,13 +119,13 @@ func (consensus *Consensus) announce(block *types.Block) {
_ = protobuf.Unmarshal(msgPayload, msg)
FPBTMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Warn().Err(err).Msg("[Announce] Unable to parse FPBT message")
consensus.getLogger().Warn().Err(err).Msg("[Announce] Unable to parse FPBT message")
return
}
// TODO(chao): review FPBT log data structure
consensus.FBFTLog.AddMessage(FPBTMsg)
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("MsgBlockHash", FPBTMsg.BlockHash.Hex()).
Uint64("MsgViewID", FPBTMsg.ViewID).
Uint64("MsgBlockNum", FPBTMsg.BlockNum).
@ -138,7 +137,7 @@ func (consensus *Consensus) announce(block *types.Block) {
quorum.Prepare, consensus.PubKey, consensus.priKey.SignHash(consensus.blockHash[:]),
)
if err := consensus.prepareBitmap.SetKey(consensus.PubKey, true); err != nil {
utils.Logger().Warn().Err(err).Msg("[Announce] Leader prepareBitmap SetKey failed")
consensus.getLogger().Warn().Err(err).Msg("[Announce] Leader prepareBitmap SetKey failed")
return
}
@ -147,19 +146,19 @@ func (consensus *Consensus) announce(block *types.Block) {
consensus.blockNum, msg_pb.MessageType_ANNOUNCE, []nodeconfig.GroupID{
nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID)),
}, host.ConstructP2pMessage(byte(17), msgToSend)); err != nil {
utils.Logger().Warn().
consensus.getLogger().Warn().
Str("groupID", string(nodeconfig.NewGroupIDByShardID(
nodeconfig.ShardID(consensus.ShardID),
))).
Msg("[Announce] Cannot send announce message")
} else {
utils.Logger().Info().
consensus.getLogger().Info().
Str("blockHash", block.Hash().Hex()).
Uint64("blockNum", block.NumberU64()).
Msg("[Announce] Sent Announce Message!!")
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("From", consensus.phase.String()).
Str("To", FBFTPrepare.String()).
Msg("[Announce] Switching phase")
@ -167,32 +166,32 @@ func (consensus *Consensus) announce(block *types.Block) {
}
func (consensus *Consensus) onAnnounce(msg *msg_pb.Message) {
utils.Logger().Debug().Msg("[OnAnnounce] Receive announce message")
consensus.getLogger().Debug().Msg("[OnAnnounce] Receive announce message")
if consensus.IsLeader() && consensus.current.Mode() == Normal {
return
}
senderKey, err := consensus.verifySenderKey(msg)
if err != nil {
utils.Logger().Error().Err(err).Msg("[OnAnnounce] VerifySenderKey failed")
consensus.getLogger().Error().Err(err).Msg("[OnAnnounce] VerifySenderKey failed")
return
}
if !senderKey.IsEqual(consensus.LeaderPubKey) &&
consensus.current.Mode() == Normal && !consensus.ignoreViewIDCheck {
utils.Logger().Warn().
consensus.getLogger().Warn().
Str("senderKey", senderKey.SerializeToHexStr()).
Str("leaderKey", consensus.LeaderPubKey.SerializeToHexStr()).
Msg("[OnAnnounce] SenderKey does not match leader PubKey")
return
}
if err = verifyMessageSig(senderKey, msg); err != nil {
utils.Logger().Error().Err(err).Msg("[OnAnnounce] Failed to verify leader signature")
consensus.getLogger().Error().Err(err).Msg("[OnAnnounce] Failed to verify leader signature")
return
}
recvMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Error().
consensus.getLogger().Error().
Err(err).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnAnnounce] Unparseable leader message")
@ -205,7 +204,7 @@ func (consensus *Consensus) onAnnounce(msg *msg_pb.Message) {
header := new(block.Header)
err = rlp.DecodeBytes(encodedHeader, header)
if err != nil {
utils.Logger().Warn().
consensus.getLogger().Warn().
Err(err).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnAnnounce] Unparseable block header data")
@ -213,7 +212,7 @@ func (consensus *Consensus) onAnnounce(msg *msg_pb.Message) {
}
if recvMsg.BlockNum < consensus.blockNum || recvMsg.BlockNum != header.Number().Uint64() {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("hdrBlockNum", header.Number().Uint64()).
Uint64("consensuBlockNum", consensus.blockNum).
@ -222,7 +221,7 @@ func (consensus *Consensus) onAnnounce(msg *msg_pb.Message) {
}
if consensus.current.Mode() == Normal {
if err = chain.Engine.VerifyHeader(consensus.ChainReader, header, true); err != nil {
utils.Logger().Warn().
consensus.getLogger().Warn().
Err(err).
Str("inChain", consensus.ChainReader.CurrentHeader().Number().String()).
Str("MsgBlockNum", header.Number().String()).
@ -254,18 +253,24 @@ func (consensus *Consensus) onAnnounce(msg *msg_pb.Message) {
if len(logMsgs) > 0 {
if logMsgs[0].BlockHash != recvMsg.BlockHash &&
logMsgs[0].SenderPubkey.IsEqual(recvMsg.SenderPubkey) {
utils.Logger().Debug().
Str("leaderKey", consensus.LeaderPubKey.SerializeToHexStr()).
consensus.getLogger().Debug().
Str("logMsgSenderKey", logMsgs[0].SenderPubkey.SerializeToHexStr()).
Str("logMsgBlockHash", logMsgs[0].BlockHash.Hex()).
Str("recvMsg.SenderPubkey", recvMsg.SenderPubkey.SerializeToHexStr()).
Uint64("recvMsg.BlockNum", recvMsg.BlockNum).
Uint64("recvMsg.ViewID", recvMsg.ViewID).
Str("recvMsgBlockHash", recvMsg.BlockHash.Hex()).
Str("LeaderKey", consensus.LeaderPubKey.SerializeToHexStr()).
Msg("[OnAnnounce] Leader is malicious")
consensus.startViewChange(consensus.viewID + 1)
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("leaderKey", consensus.LeaderPubKey.SerializeToHexStr()).
Msg("[OnAnnounce] Announce message received again")
//return
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnAnnounce] Announce message Added")
@ -278,13 +283,13 @@ func (consensus *Consensus) onAnnounce(msg *msg_pb.Message) {
// we have already added message and block, skip check viewID and send prepare message if is in ViewChanging mode
if consensus.current.Mode() == ViewChanging {
utils.Logger().Debug().Msg("[OnAnnounce] Still in ViewChanging Mode, Exiting !!")
consensus.getLogger().Debug().Msg("[OnAnnounce] Still in ViewChanging Mode, Exiting !!")
return
}
if consensus.checkViewID(recvMsg) != nil {
if consensus.current.Mode() == Normal {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnAnnounce] ViewID check failed")
@ -303,13 +308,13 @@ func (consensus *Consensus) prepare() {
// TODO: this will not return immediatey, may block
if err := consensus.msgSender.SendWithoutRetry([]nodeconfig.GroupID{nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID))}, host.ConstructP2pMessage(byte(17), msgToSend)); err != nil {
utils.Logger().Warn().Err(err).Msg("[OnAnnounce] Cannot send prepare message")
consensus.getLogger().Warn().Err(err).Msg("[OnAnnounce] Cannot send prepare message")
} else {
utils.Logger().Info().
consensus.getLogger().Info().
Str("blockHash", hex.EncodeToString(consensus.blockHash[:])).
Msg("[OnAnnounce] Sent Prepare Message!!")
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("From", consensus.phase.String()).
Str("To", FBFTPrepare.String()).
Msg("[Announce] Switching Phase")
@ -324,22 +329,22 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
senderKey, err := consensus.verifySenderKey(msg)
if err != nil {
utils.Logger().Error().Err(err).Msg("[OnPrepare] VerifySenderKey failed")
consensus.getLogger().Error().Err(err).Msg("[OnPrepare] VerifySenderKey failed")
return
}
if err = verifyMessageSig(senderKey, msg); err != nil {
utils.Logger().Error().Err(err).Msg("[OnPrepare] Failed to verify sender's signature")
consensus.getLogger().Error().Err(err).Msg("[OnPrepare] Failed to verify sender's signature")
return
}
recvMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Error().Err(err).Msg("[OnPrepare] Unparseable validator message")
consensus.getLogger().Error().Err(err).Msg("[OnPrepare] Unparseable validator message")
return
}
if recvMsg.ViewID != consensus.viewID || recvMsg.BlockNum != consensus.blockNum {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("blockNum", consensus.blockNum).
@ -350,7 +355,7 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
if !consensus.FBFTLog.HasMatchingViewAnnounce(
consensus.blockNum, consensus.viewID, recvMsg.BlockHash,
) {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("blockNum", consensus.blockNum).
@ -364,7 +369,7 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
consensus.mutex.Lock()
defer consensus.mutex.Unlock()
logger := utils.Logger().With().
logger := consensus.getLogger().With().
Str("validatorPubKey", validatorPubKey.SerializeToHexStr()).Logger()
// proceed only when the message is not received before
@ -385,12 +390,12 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
var sign bls.Sign
err = sign.Deserialize(prepareSig)
if err != nil {
utils.Logger().Error().Err(err).
consensus.getLogger().Error().Err(err).
Msg("[OnPrepare] Failed to deserialize bls signature")
return
}
if !sign.VerifyHash(recvMsg.SenderPubkey, consensus.blockHash[:]) {
utils.Logger().Error().Msg("[OnPrepare] Received invalid BLS signature")
consensus.getLogger().Error().Msg("[OnPrepare] Received invalid BLS signature")
return
}
@ -401,7 +406,7 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
consensus.Decider.AddSignature(quorum.Prepare, validatorPubKey, &sign)
// Set the bitmap indicating that this validator signed.
if err := prepareBitmap.SetKey(recvMsg.SenderPubkey, true); err != nil {
utils.Logger().Warn().Err(err).Msg("[OnPrepare] prepareBitmap.SetKey failed")
consensus.getLogger().Warn().Err(err).Msg("[OnPrepare] prepareBitmap.SetKey failed")
return
}
@ -419,7 +424,7 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
FBFTMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Warn().Err(err).Msg("[OnPrepare] Unable to parse pbft message")
consensus.getLogger().Warn().Err(err).Msg("[OnPrepare] Unable to parse pbft message")
return
}
@ -433,7 +438,7 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
)
if err := consensus.commitBitmap.SetKey(consensus.PubKey, true); err != nil {
utils.Logger().Debug().Msg("[OnPrepare] Leader commit bitmap set failed")
consensus.getLogger().Debug().Msg("[OnPrepare] Leader commit bitmap set failed")
return
}
@ -444,9 +449,9 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
},
host.ConstructP2pMessage(byte(17), msgToSend),
); err != nil {
utils.Logger().Warn().Msg("[OnPrepare] Cannot send prepared message")
consensus.getLogger().Warn().Msg("[OnPrepare] Cannot send prepared message")
} else {
utils.Logger().Debug().
consensus.getLogger().Debug().
Hex("blockHash", consensus.blockHash[:]).
Uint64("blockNum", consensus.blockNum).
Msg("[OnPrepare] Sent Prepared Message!!")
@ -455,7 +460,7 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
// Stop retry committed msg of last consensus
consensus.msgSender.StopRetry(msg_pb.MessageType_COMMITTED)
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("From", consensus.phase.String()).
Str("To", FBFTCommit.String()).
Msg("[OnPrepare] Switching phase")
@ -465,38 +470,38 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
}
func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
utils.Logger().Debug().Msg("[OnPrepared] Received Prepared message")
consensus.getLogger().Debug().Msg("[OnPrepared] Received Prepared message")
if consensus.IsLeader() && consensus.current.Mode() == Normal {
return
}
senderKey, err := consensus.verifySenderKey(msg)
if err != nil {
utils.Logger().Debug().Err(err).Msg("[OnPrepared] VerifySenderKey failed")
consensus.getLogger().Debug().Err(err).Msg("[OnPrepared] VerifySenderKey failed")
return
}
if !senderKey.IsEqual(consensus.LeaderPubKey) &&
consensus.current.Mode() == Normal && !consensus.ignoreViewIDCheck {
utils.Logger().Warn().Msg("[OnPrepared] SenderKey not match leader PubKey")
consensus.getLogger().Warn().Msg("[OnPrepared] SenderKey not match leader PubKey")
return
}
if err := verifyMessageSig(senderKey, msg); err != nil {
utils.Logger().Debug().Err(err).Msg("[OnPrepared] Failed to verify sender's signature")
consensus.getLogger().Debug().Err(err).Msg("[OnPrepared] Failed to verify sender's signature")
return
}
recvMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Debug().Err(err).Msg("[OnPrepared] Unparseable validator message")
consensus.getLogger().Debug().Err(err).Msg("[OnPrepared] Unparseable validator message")
return
}
utils.Logger().Info().
consensus.getLogger().Info().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("MsgViewID", recvMsg.ViewID).
Msg("[OnPrepared] Received prepared message")
if recvMsg.BlockNum < consensus.blockNum {
utils.Logger().Debug().Uint64("MsgBlockNum", recvMsg.BlockNum).Msg("Old Block Received, ignoring!!")
consensus.getLogger().Debug().Uint64("MsgBlockNum", recvMsg.BlockNum).Msg("Old Block Received, ignoring!!")
return
}
@ -504,12 +509,12 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
blockHash := recvMsg.BlockHash
aggSig, mask, err := consensus.ReadSignatureBitmapPayload(recvMsg.Payload, 0)
if err != nil {
utils.Logger().Error().Err(err).Msg("ReadSignatureBitmapPayload failed!!")
consensus.getLogger().Error().Err(err).Msg("ReadSignatureBitmapPayload failed!!")
return
}
if !consensus.Decider.IsQuorumAchievedByMask(mask) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Msgf("[OnPrepared] Quorum Not achieved")
return
}
@ -517,7 +522,7 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
if !aggSig.VerifyHash(mask.AggregatePublic, blockHash[:]) {
myBlockHash := common.Hash{}
myBlockHash.SetBytes(consensus.blockHash[:])
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("MsgViewID", recvMsg.ViewID).
Msg("[OnPrepared] failed to verify multi signature for prepare phase")
@ -529,21 +534,21 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
var blockObj types.Block
err = rlp.DecodeBytes(block, &blockObj)
if err != nil {
utils.Logger().Warn().
consensus.getLogger().Warn().
Err(err).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnPrepared] Unparseable block header data")
return
}
if blockObj.NumberU64() != recvMsg.BlockNum || recvMsg.BlockNum < consensus.blockNum {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("blockNum", blockObj.NumberU64()).
Msg("[OnPrepared] BlockNum not match")
return
}
if blockObj.Header().Hash() != recvMsg.BlockHash {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Hex("MsgBlockHash", recvMsg.BlockHash[:]).
Str("blockObjHash", blockObj.Header().Hash().Hex()).
@ -553,7 +558,7 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
if consensus.current.Mode() == Normal {
err := chain.Engine.VerifyHeader(consensus.ChainReader, blockObj.Header(), true)
if err != nil {
utils.Logger().Error().
consensus.getLogger().Error().
Err(err).
Str("inChain", consensus.ChainReader.CurrentHeader().Number().String()).
Str("MsgBlockNum", blockObj.Header().Number().String()).
@ -563,7 +568,7 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
if consensus.BlockVerifier == nil {
// do nothing
} else if err := consensus.BlockVerifier(&blockObj); err != nil {
utils.Logger().Error().Err(err).Msg("[OnPrepared] Block verification failed")
consensus.getLogger().Error().Err(err).Msg("[OnPrepared] Block verification failed")
return
}
}
@ -571,7 +576,7 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
consensus.FBFTLog.AddBlock(&blockObj)
recvMsg.Block = []byte{} // save memory space
consensus.FBFTLog.AddMessage(recvMsg)
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Hex("blockHash", recvMsg.BlockHash[:]).
@ -582,13 +587,13 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
consensus.tryCatchup()
if consensus.current.Mode() == ViewChanging {
utils.Logger().Debug().Msg("[OnPrepared] Still in ViewChanging mode, Exiting!!")
consensus.getLogger().Debug().Msg("[OnPrepared] Still in ViewChanging mode, Exiting!!")
return
}
if consensus.checkViewID(recvMsg) != nil {
if consensus.current.Mode() == Normal {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnPrepared] ViewID check failed")
@ -596,7 +601,7 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
return
}
if recvMsg.BlockNum > consensus.blockNum {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("blockNum", consensus.blockNum).
Msg("[OnPrepared] Future Block Received, ignoring!!")
@ -631,15 +636,15 @@ func (consensus *Consensus) onPrepared(msg *msg_pb.Message) {
}
if err := consensus.msgSender.SendWithoutRetry([]nodeconfig.GroupID{nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID))}, host.ConstructP2pMessage(byte(17), msgToSend)); err != nil {
utils.Logger().Warn().Msg("[OnPrepared] Cannot send commit message!!")
consensus.getLogger().Warn().Msg("[OnPrepared] Cannot send commit message!!")
} else {
utils.Logger().Info().
consensus.getLogger().Info().
Uint64("blockNum", consensus.blockNum).
Hex("blockHash", consensus.blockHash[:]).
Msg("[OnPrepared] Sent Commit Message!!")
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("From", consensus.phase.String()).
Str("To", FBFTCommit.String()).
Msg("[OnPrepared] Switching phase")
@ -656,22 +661,22 @@ func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
senderKey, err := consensus.verifySenderKey(msg)
if err != nil {
utils.Logger().Debug().Err(err).Msg("[OnCommit] VerifySenderKey Failed")
consensus.getLogger().Debug().Err(err).Msg("[OnCommit] VerifySenderKey Failed")
return
}
if err = verifyMessageSig(senderKey, msg); err != nil {
utils.Logger().Debug().Err(err).Msg("[OnCommit] Failed to verify sender's signature")
consensus.getLogger().Debug().Err(err).Msg("[OnCommit] Failed to verify sender's signature")
return
}
recvMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Debug().Err(err).Msg("[OnCommit] Parse pbft message failed")
consensus.getLogger().Debug().Err(err).Msg("[OnCommit] Parse pbft message failed")
return
}
if recvMsg.ViewID != consensus.viewID || recvMsg.BlockNum != consensus.blockNum {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("blockNum", consensus.blockNum).
@ -681,7 +686,7 @@ func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
}
if !consensus.FBFTLog.HasMatchingAnnounce(consensus.blockNum, recvMsg.BlockHash) {
utils.Logger().Debug().
consensus.getLogger().Debug().
Hex("MsgBlockHash", recvMsg.BlockHash[:]).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("blockNum", consensus.blockNum).
@ -690,7 +695,7 @@ func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
}
if !consensus.FBFTLog.HasMatchingPrepared(consensus.blockNum, recvMsg.BlockHash) {
utils.Logger().Debug().
consensus.getLogger().Debug().
Hex("blockHash", recvMsg.BlockHash[:]).
Uint64("blockNum", consensus.blockNum).
Msg("[OnCommit] Cannot find matching prepared message")
@ -702,7 +707,7 @@ func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
consensus.mutex.Lock()
defer consensus.mutex.Unlock()
logger := utils.Logger().With().
logger := consensus.getLogger().With().
Str("validatorPubKey", validatorPubKey.SerializeToHexStr()).Logger()
if !consensus.IsValidatorInCommittee(recvMsg.SenderPubkey) {
logger.Error().Msg("[OnCommit] Invalid validator")
@ -744,7 +749,7 @@ func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
consensus.Decider.AddSignature(quorum.Commit, validatorPubKey, &sign)
// Set the bitmap indicating that this validator signed.
if err := commitBitmap.SetKey(recvMsg.SenderPubkey, true); err != nil {
utils.Logger().Warn().Err(err).Msg("[OnCommit] commitBitmap.SetKey failed")
consensus.getLogger().Warn().Err(err).Msg("[OnCommit] commitBitmap.SetKey failed")
return
}
@ -769,7 +774,7 @@ func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
}
func (consensus *Consensus) finalizeCommits() {
utils.Logger().Info().
consensus.getLogger().Info().
Int64("NumCommits", consensus.Decider.SignersCount(quorum.Commit)).
Msg("[Finalizing] Finalizing Block")
@ -784,7 +789,7 @@ func (consensus *Consensus) finalizeCommits() {
_ = protobuf.Unmarshal(msgPayload, msg)
pbftMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Warn().Err(err).Msg("[FinalizeCommits] Unable to parse pbft message")
consensus.getLogger().Warn().Err(err).Msg("[FinalizeCommits] Unable to parse pbft message")
return
}
consensus.FBFTLog.AddMessage(pbftMsg)
@ -793,14 +798,14 @@ func (consensus *Consensus) finalizeCommits() {
curBlockHash := consensus.blockHash
block := consensus.FBFTLog.GetBlockByHash(curBlockHash)
if block == nil {
utils.Logger().Warn().
consensus.getLogger().Warn().
Str("blockHash", hex.EncodeToString(curBlockHash[:])).
Msg("[FinalizeCommits] Cannot find block by hash")
return
}
consensus.tryCatchup()
if consensus.blockNum-beforeCatchupNum != 1 {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint64("beforeCatchupBlockNum", beforeCatchupNum).
Msg("[FinalizeCommits] Leader cannot provide the correct block for committed message")
return
@ -815,9 +820,9 @@ func (consensus *Consensus) finalizeCommits() {
nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID)),
},
host.ConstructP2pMessage(byte(17), msgToSend)); err != nil {
utils.Logger().Warn().Err(err).Msg("[Finalizing] Cannot send committed message")
consensus.getLogger().Warn().Err(err).Msg("[Finalizing] Cannot send committed message")
} else {
utils.Logger().Info().
consensus.getLogger().Info().
Hex("blockHash", curBlockHash[:]).
Uint64("blockNum", consensus.blockNum).
Msg("[Finalizing] Sent Committed Message")
@ -833,13 +838,13 @@ func (consensus *Consensus) finalizeCommits() {
if consensus.consensusTimeout[timeoutBootstrap].IsActive() {
consensus.consensusTimeout[timeoutBootstrap].Stop()
utils.Logger().Debug().Msg("[Finalizing] Start consensus timer; stop bootstrap timer only once")
consensus.getLogger().Debug().Msg("[Finalizing] Start consensus timer; stop bootstrap timer only once")
} else {
utils.Logger().Debug().Msg("[Finalizing] Start consensus timer")
consensus.getLogger().Debug().Msg("[Finalizing] Start consensus timer")
}
consensus.consensusTimeout[timeoutConsensus].Start()
utils.Logger().Info().
consensus.getLogger().Info().
Uint64("blockNum", block.NumberU64()).
Uint64("epochNum", block.Epoch().Uint64()).
Uint64("ViewId", block.Header().ViewID().Uint64()).
@ -853,7 +858,7 @@ func (consensus *Consensus) finalizeCommits() {
}
func (consensus *Consensus) onCommitted(msg *msg_pb.Message) {
utils.Logger().Debug().Msg("[OnCommitted] Receive committed message")
consensus.getLogger().Debug().Msg("[OnCommitted] Receive committed message")
if consensus.IsLeader() && consensus.current.Mode() == Normal {
return
@ -861,27 +866,27 @@ func (consensus *Consensus) onCommitted(msg *msg_pb.Message) {
senderKey, err := consensus.verifySenderKey(msg)
if err != nil {
utils.Logger().Warn().Err(err).Msg("[OnCommitted] verifySenderKey failed")
consensus.getLogger().Warn().Err(err).Msg("[OnCommitted] verifySenderKey failed")
return
}
if !senderKey.IsEqual(consensus.LeaderPubKey) &&
consensus.current.Mode() == Normal && !consensus.ignoreViewIDCheck {
utils.Logger().Warn().Msg("[OnCommitted] senderKey not match leader PubKey")
consensus.getLogger().Warn().Msg("[OnCommitted] senderKey not match leader PubKey")
return
}
if err = verifyMessageSig(senderKey, msg); err != nil {
utils.Logger().Warn().Err(err).Msg("[OnCommitted] Failed to verify sender's signature")
consensus.getLogger().Warn().Err(err).Msg("[OnCommitted] Failed to verify sender's signature")
return
}
recvMsg, err := ParseFBFTMessage(msg)
if err != nil {
utils.Logger().Warn().Msg("[OnCommitted] unable to parse msg")
consensus.getLogger().Warn().Msg("[OnCommitted] unable to parse msg")
return
}
if recvMsg.BlockNum < consensus.blockNum {
utils.Logger().Info().
consensus.getLogger().Info().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Uint64("blockNum", consensus.blockNum).
Msg("[OnCommitted] Received Old Blocks!!")
@ -890,12 +895,12 @@ func (consensus *Consensus) onCommitted(msg *msg_pb.Message) {
aggSig, mask, err := consensus.ReadSignatureBitmapPayload(recvMsg.Payload, 0)
if err != nil {
utils.Logger().Error().Err(err).Msg("[OnCommitted] readSignatureBitmapPayload failed")
consensus.getLogger().Error().Err(err).Msg("[OnCommitted] readSignatureBitmapPayload failed")
return
}
if !consensus.Decider.IsQuorumAchievedByMask(mask) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Msgf("[OnCommitted] Quorum Not achieved")
return
}
@ -904,7 +909,7 @@ func (consensus *Consensus) onCommitted(msg *msg_pb.Message) {
binary.LittleEndian.PutUint64(blockNumBytes, recvMsg.BlockNum)
commitPayload := append(blockNumBytes, recvMsg.BlockHash[:]...)
if !aggSig.VerifyHash(mask.AggregatePublic, commitPayload) {
utils.Logger().Error().
consensus.getLogger().Error().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnCommitted] Failed to verify the multi signature for commit phase")
return
@ -912,7 +917,7 @@ func (consensus *Consensus) onCommitted(msg *msg_pb.Message) {
consensus.FBFTLog.AddMessage(recvMsg)
consensus.ChainReader.WriteLastCommits(recvMsg.Payload)
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgViewID", recvMsg.ViewID).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[OnCommitted] Committed message added")
@ -924,7 +929,7 @@ func (consensus *Consensus) onCommitted(msg *msg_pb.Message) {
consensus.commitBitmap = mask
if recvMsg.BlockNum-consensus.blockNum > consensusBlockNumBuffer {
utils.Logger().Debug().Uint64("MsgBlockNum", recvMsg.BlockNum).Msg("[OnCommitted] out of sync")
consensus.getLogger().Debug().Uint64("MsgBlockNum", recvMsg.BlockNum).Msg("[OnCommitted] out of sync")
go func() {
select {
case consensus.blockNumLowChan <- struct{}{}:
@ -939,21 +944,21 @@ func (consensus *Consensus) onCommitted(msg *msg_pb.Message) {
}
// if consensus.checkViewID(recvMsg) != nil {
// utils.Logger().Debug("viewID check failed", "viewID", recvMsg.ViewID, "myViewID", consensus.viewID)
// consensus.getLogger().Debug("viewID check failed", "viewID", recvMsg.ViewID, "myViewID", consensus.viewID)
// return
// }
consensus.tryCatchup()
if consensus.current.Mode() == ViewChanging {
utils.Logger().Debug().Msg("[OnCommitted] Still in ViewChanging mode, Exiting!!")
consensus.getLogger().Debug().Msg("[OnCommitted] Still in ViewChanging mode, Exiting!!")
return
}
if consensus.consensusTimeout[timeoutBootstrap].IsActive() {
consensus.consensusTimeout[timeoutBootstrap].Stop()
utils.Logger().Debug().Msg("[OnCommitted] Start consensus timer; stop bootstrap timer only once")
consensus.getLogger().Debug().Msg("[OnCommitted] Start consensus timer; stop bootstrap timer only once")
} else {
utils.Logger().Debug().Msg("[OnCommitted] Start consensus timer")
consensus.getLogger().Debug().Msg("[OnCommitted] Start consensus timer")
}
consensus.consensusTimeout[timeoutConsensus].Start()
return
@ -968,7 +973,7 @@ func (consensus *Consensus) LastCommitSig() ([]byte, []byte, error) {
if err != nil || len(lastCommits) < 96 {
msgs := consensus.FBFTLog.GetMessagesByTypeSeq(msg_pb.MessageType_COMMITTED, consensus.blockNum-1)
if len(msgs) != 1 {
utils.Logger().Error().
consensus.getLogger().Error().
Int("numCommittedMsg", len(msgs)).
Msg("GetLastCommitSig failed with wrong number of committed message")
return nil, nil, ctxerror.New("GetLastCommitSig failed with wrong number of committed message", "numCommittedMsg", len(msgs))
@ -988,7 +993,7 @@ func (consensus *Consensus) LastCommitSig() ([]byte, []byte, error) {
// try to catch up if fall behind
func (consensus *Consensus) tryCatchup() {
utils.Logger().Info().Msg("[TryCatchup] commit new blocks")
consensus.getLogger().Info().Msg("[TryCatchup] commit new blocks")
// if consensus.phase != Commit && consensus.mode.Mode() == Normal {
// return
// }
@ -999,11 +1004,11 @@ func (consensus *Consensus) tryCatchup() {
break
}
if len(msgs) > 1 {
utils.Logger().Error().
consensus.getLogger().Error().
Int("numMsgs", len(msgs)).
Msg("[TryCatchup] DANGER!!! we should only get one committed message for a given blockNum")
}
utils.Logger().Info().Msg("[TryCatchup] committed message found")
consensus.getLogger().Info().Msg("[TryCatchup] committed message found")
block := consensus.FBFTLog.GetBlockByHash(msgs[0].BlockHash)
if block == nil {
@ -1013,22 +1018,22 @@ func (consensus *Consensus) tryCatchup() {
if consensus.BlockVerifier == nil {
// do nothing
} else if err := consensus.BlockVerifier(block); err != nil {
utils.Logger().Info().Msg("[TryCatchup] block verification failed")
consensus.getLogger().Info().Msg("[TryCatchup] block verification failed")
return
}
if block.ParentHash() != consensus.ChainReader.CurrentHeader().Hash() {
utils.Logger().Debug().Msg("[TryCatchup] parent block hash not match")
consensus.getLogger().Debug().Msg("[TryCatchup] parent block hash not match")
break
}
utils.Logger().Info().Msg("[TryCatchup] block found to commit")
consensus.getLogger().Info().Msg("[TryCatchup] block found to commit")
preparedMsgs := consensus.FBFTLog.GetMessagesByTypeSeqHash(msg_pb.MessageType_PREPARED, msgs[0].BlockNum, msgs[0].BlockHash)
msg := consensus.FBFTLog.FindMessageByMaxViewID(preparedMsgs)
if msg == nil {
break
}
utils.Logger().Info().Msg("[TryCatchup] prepared message found to commit")
consensus.getLogger().Info().Msg("[TryCatchup] prepared message found to commit")
// TODO(Chao): Explain the reasoning for these code
consensus.blockHash = [32]byte{}
@ -1036,14 +1041,14 @@ func (consensus *Consensus) tryCatchup() {
consensus.viewID = msgs[0].ViewID + 1
consensus.LeaderPubKey = msgs[0].SenderPubkey
utils.Logger().Info().Msg("[TryCatchup] Adding block to chain")
consensus.getLogger().Info().Msg("[TryCatchup] Adding block to chain")
consensus.OnConsensusDone(block, msgs[0].Payload)
consensus.ResetState()
select {
case consensus.VerifiedNewBlock <- block:
default:
utils.Logger().Info().
consensus.getLogger().Info().
Str("blockHash", block.Hash().String()).
Msg("[TryCatchup] consensus verified block send to chan failed")
continue
@ -1052,7 +1057,7 @@ func (consensus *Consensus) tryCatchup() {
break
}
if currentBlockNum < consensus.blockNum {
utils.Logger().Info().
consensus.getLogger().Info().
Uint64("From", currentBlockNum).
Uint64("To", consensus.blockNum).
Msg("[TryCatchup] Caught up!")
@ -1075,23 +1080,23 @@ func (consensus *Consensus) Start(blockChannel chan *types.Block, stopChan chan
toStart := false
isInitialLeader := consensus.IsLeader()
if isInitialLeader {
utils.Logger().Info().Time("time", time.Now()).Msg("[ConsensusMainLoop] Waiting for consensus start")
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 = true
utils.Logger().Info().Time("time", time.Now()).Msg("[ConsensusMainLoop] Send ReadySignal")
consensus.getLogger().Info().Time("time", time.Now()).Msg("[ConsensusMainLoop] Send ReadySignal")
consensus.ReadySignal <- struct{}{}
}()
}
utils.Logger().Info().Time("time", time.Now()).Msg("[ConsensusMainLoop] Consensus started")
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()
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("viewID", consensus.viewID).
Uint64("blockNum", consensus.blockNum).
Msg("[ConsensusMainLoop] Start bootstrap timeout (only once)")
@ -1112,11 +1117,11 @@ func (consensus *Consensus) Start(blockChannel chan *types.Block, stopChan chan
continue
}
if k != timeoutViewChange {
utils.Logger().Debug().Msg("[ConsensusMainLoop] Ops Consensus Timeout!!!")
consensus.getLogger().Debug().Msg("[ConsensusMainLoop] Ops Consensus Timeout!!!")
consensus.startViewChange(consensus.viewID + 1)
break
} else {
utils.Logger().Debug().Msg("[ConsensusMainLoop] Ops View Change Timeout!!!")
consensus.getLogger().Debug().Msg("[ConsensusMainLoop] Ops View Change Timeout!!!")
viewID := consensus.current.ViewID()
consensus.startViewChange(viewID + 1)
break
@ -1127,12 +1132,12 @@ func (consensus *Consensus) Start(blockChannel chan *types.Block, stopChan chan
consensus.SetViewID(consensus.ChainReader.CurrentHeader().ViewID().Uint64() + 1)
mode := consensus.UpdateConsensusInformation()
consensus.current.SetMode(mode)
utils.Logger().Info().Str("Mode", mode.String()).Msg("Node is in sync")
consensus.getLogger().Info().Str("Mode", mode.String()).Msg("Node is in sync")
case <-consensus.syncNotReadyChan:
consensus.SetBlockNum(consensus.ChainReader.CurrentHeader().Number().Uint64() + 1)
consensus.current.SetMode(Syncing)
utils.Logger().Info().Msg("Node is out of sync")
consensus.getLogger().Info().Msg("Node is out of sync")
case newBlock := <-blockChannel:
// Debug code to trigger leader change.
@ -1140,7 +1145,7 @@ func (consensus *Consensus) Start(blockChannel chan *types.Block, stopChan chan
// continue
//}
utils.Logger().Info().
consensus.getLogger().Info().
Uint64("MsgBlockNum", newBlock.NumberU64()).
Msg("[ConsensusMainLoop] Received Proposed New Block!")
@ -1223,7 +1228,7 @@ func (consensus *Consensus) Start(blockChannel chan *types.Block, stopChan chan
startTime = time.Now()
consensus.msgSender.Reset(newBlock.NumberU64())
utils.Logger().Debug().
consensus.getLogger().Debug().
Int("numTxs", len(newBlock.Transactions())).
Time("startTime", startTime).
Int64("publicKeys", consensus.Decider.ParticipantsCount()).

@ -77,7 +77,7 @@ func (consensus *Consensus) switchPhase(desired FBFTPhase, override bool) {
func (consensus *Consensus) GetNextLeaderKey() *bls.PublicKey {
wasFound, next := consensus.Decider.NextAfter(consensus.LeaderPubKey)
if !wasFound {
utils.Logger().Warn().
consensus.getLogger().Warn().
Str("key", consensus.LeaderPubKey.SerializeToHexStr()).
Msg("GetNextLeaderKey: currentLeaderKey not found")
}
@ -86,7 +86,7 @@ func (consensus *Consensus) GetNextLeaderKey() *bls.PublicKey {
// ResetViewChangeState reset the state for viewchange
func (consensus *Consensus) ResetViewChangeState() {
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("Phase", consensus.phase.String()).
Msg("[ResetViewChangeState] Resetting view change state")
consensus.current.SetMode(Normal)
@ -124,7 +124,7 @@ func (consensus *Consensus) startViewChange(viewID uint64) {
diff := viewID - consensus.viewID
duration := time.Duration(int64(diff) * int64(viewChangeDuration))
utils.Logger().Info().
consensus.getLogger().Info().
Uint64("ViewChangingID", viewID).
Dur("timeoutDuration", duration).
Str("NextLeader", consensus.LeaderPubKey.SerializeToHexStr()).
@ -139,7 +139,7 @@ func (consensus *Consensus) startViewChange(viewID uint64) {
consensus.consensusTimeout[timeoutViewChange].SetDuration(duration)
consensus.consensusTimeout[timeoutViewChange].Start()
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("ViewChangingID", consensus.current.ViewID()).
Msg("[startViewChange] start view change timer")
}
@ -147,7 +147,7 @@ func (consensus *Consensus) startViewChange(viewID uint64) {
func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
recvMsg, err := ParseViewChangeMessage(msg)
if err != nil {
utils.Logger().Warn().Msg("[onViewChange] Unable To Parse Viewchange Message")
consensus.getLogger().Warn().Msg("[onViewChange] Unable To Parse Viewchange Message")
return
}
newLeaderKey := recvMsg.LeaderPubkey
@ -156,7 +156,7 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
}
if consensus.Decider.IsQuorumAchieved(quorum.ViewChange) {
utils.Logger().Debug().
consensus.getLogger().Debug().
Int64("have", consensus.Decider.SignersCount(quorum.ViewChange)).
Int64("need", consensus.Decider.TwoThirdsSignersCount()).
Str("validatorPubKey", recvMsg.SenderPubkey.SerializeToHexStr()).
@ -166,21 +166,21 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
senderKey, err := consensus.verifyViewChangeSenderKey(msg)
if err != nil {
utils.Logger().Debug().Err(err).Msg("[onViewChange] VerifySenderKey Failed")
consensus.getLogger().Debug().Err(err).Msg("[onViewChange] VerifySenderKey Failed")
return
}
// TODO: if difference is only one, new leader can still propose the same committed block to avoid another view change
// TODO: new leader catchup without ignore view change message
if consensus.blockNum > recvMsg.BlockNum {
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[onViewChange] Message BlockNum Is Low")
return
}
if consensus.blockNum < recvMsg.BlockNum {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[onViewChange] New Leader Has Lower Blocknum")
return
@ -188,14 +188,14 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
if consensus.current.Mode() == ViewChanging &&
consensus.current.ViewID() > recvMsg.ViewID {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint64("MyViewChangingID", consensus.current.ViewID()).
Uint64("MsgViewChangingID", recvMsg.ViewID).
Msg("[onViewChange] ViewChanging ID Is Low")
return
}
if err = verifyMessageSig(senderKey, msg); err != nil {
utils.Logger().Debug().Err(err).Msg("[onViewChange] Failed To Verify Sender's Signature")
consensus.getLogger().Debug().Err(err).Msg("[onViewChange] Failed To Verify Sender's Signature")
return
}
@ -213,11 +213,11 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
)
preparedMsg := consensus.FBFTLog.FindMessageByMaxViewID(preparedMsgs)
if preparedMsg == nil {
utils.Logger().Debug().Msg("[onViewChange] add my M2(NIL) type messaage")
consensus.getLogger().Debug().Msg("[onViewChange] add my M2(NIL) type messaage")
consensus.nilSigs[consensus.PubKey.SerializeToHexStr()] = consensus.priKey.SignHash(NIL)
consensus.nilBitmap.SetKey(consensus.PubKey, true)
} else {
utils.Logger().Debug().Msg("[onViewChange] add my M1 type messaage")
consensus.getLogger().Debug().Msg("[onViewChange] add my M1 type messaage")
msgToSign := append(preparedMsg.BlockHash[:], preparedMsg.Payload...)
consensus.bhpSigs[consensus.PubKey.SerializeToHexStr()] = consensus.priKey.SignHash(msgToSign)
consensus.bhpBitmap.SetKey(consensus.PubKey, true)
@ -238,18 +238,18 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
if len(recvMsg.Payload) == 0 {
_, ok := consensus.nilSigs[senderKey.SerializeToHexStr()]
if ok {
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("validatorPubKey", senderKey.SerializeToHexStr()).
Msg("[onViewChange] Already Received M2 message from validator")
return
}
if !recvMsg.ViewchangeSig.VerifyHash(senderKey, NIL) {
utils.Logger().Warn().Msg("[onViewChange] Failed To Verify Signature For M2 Type Viewchange Message")
consensus.getLogger().Warn().Msg("[onViewChange] Failed To Verify Signature For M2 Type Viewchange Message")
return
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("validatorPubKey", senderKey.SerializeToHexStr()).
Msg("[onViewChange] Add M2 (NIL) type message")
consensus.nilSigs[senderKey.SerializeToHexStr()] = recvMsg.ViewchangeSig
@ -257,20 +257,20 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
} else { // m1 type message
_, ok := consensus.bhpSigs[senderKey.SerializeToHexStr()]
if ok {
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("validatorPubKey", senderKey.SerializeToHexStr()).
Msg("[onViewChange] Already Received M1 Message From the Validator")
return
}
if !recvMsg.ViewchangeSig.VerifyHash(recvMsg.SenderPubkey, recvMsg.Payload) {
utils.Logger().Warn().Msg("[onViewChange] Failed to Verify Signature for M1 Type Viewchange Message")
consensus.getLogger().Warn().Msg("[onViewChange] Failed to Verify Signature for M1 Type Viewchange Message")
return
}
// first time receive m1 type message, need verify validity of prepared message
if len(consensus.m1Payload) == 0 || !bytes.Equal(consensus.m1Payload, recvMsg.Payload) {
if len(recvMsg.Payload) <= 32 {
utils.Logger().Debug().
consensus.getLogger().Debug().
Int("len", len(recvMsg.Payload)).
Msg("[onViewChange] M1 RecvMsg Payload Not Enough Length")
return
@ -278,19 +278,19 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
blockHash := recvMsg.Payload[:32]
aggSig, mask, err := consensus.ReadSignatureBitmapPayload(recvMsg.Payload, 32)
if err != nil {
utils.Logger().Error().Err(err).Msg("[onViewChange] M1 RecvMsg Payload Read Error")
consensus.getLogger().Error().Err(err).Msg("[onViewChange] M1 RecvMsg Payload Read Error")
return
}
if !consensus.Decider.IsQuorumAchievedByMask(mask) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Msgf("[onViewChange] Quorum Not achieved")
return
}
// Verify the multi-sig for prepare phase
if !aggSig.VerifyHash(mask.AggregatePublic, blockHash[:]) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Hex("blockHash", blockHash).
Msg("[onViewChange] failed to verify multi signature for m1 prepared payload")
return
@ -310,11 +310,11 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
preparedMsg.Payload = make([]byte, len(recvMsg.Payload)-32)
copy(preparedMsg.Payload[:], recvMsg.Payload[32:])
preparedMsg.SenderPubkey = consensus.PubKey
utils.Logger().Info().Msg("[onViewChange] New Leader Prepared Message Added")
consensus.getLogger().Info().Msg("[onViewChange] New Leader Prepared Message Added")
consensus.FBFTLog.AddMessage(&preparedMsg)
}
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("validatorPubKey", senderKey.SerializeToHexStr()).
Msg("[onViewChange] Add M1 (prepared) type message")
consensus.bhpSigs[senderKey.SerializeToHexStr()] = recvMsg.ViewchangeSig
@ -324,7 +324,7 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
// check and add viewID (m3 type) message signature
sig := consensus.Decider.ReadSignature(quorum.ViewChange, senderKey)
if sig != nil {
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("validatorPubKey", senderKey.SerializeToHexStr()).
Msg("[onViewChange] Already Received M3(ViewID) message from the validator")
return
@ -332,18 +332,18 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
viewIDHash := make([]byte, 8)
binary.LittleEndian.PutUint64(viewIDHash, recvMsg.ViewID)
if !recvMsg.ViewidSig.VerifyHash(recvMsg.SenderPubkey, viewIDHash) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Uint64("MsgViewID", recvMsg.ViewID).
Msg("[onViewChange] Failed to Verify M3 Message Signature")
return
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("validatorPubKey", senderKey.SerializeToHexStr()).
Msg("[onViewChange] Add M3 (ViewID) type message")
consensus.Decider.AddSignature(quorum.ViewChange, senderKey, recvMsg.ViewidSig)
// Set the bitmap indicating that this validator signed.
consensus.viewIDBitmap.SetKey(recvMsg.SenderPubkey, true)
utils.Logger().Debug().
consensus.getLogger().Debug().
Int64("numSigs", consensus.Decider.SignersCount(quorum.ViewChange)).
Int64("needed", consensus.Decider.TwoThirdsSignersCount()).
Msg("[onViewChange]")
@ -359,7 +359,7 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
consensus.ReadySignal <- struct{}{}
}()
} else {
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("From", consensus.phase.String()).
Str("To", FBFTCommit.String()).
Msg("[OnViewChange] Switching phase")
@ -368,7 +368,7 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
aggSig, mask, err := consensus.ReadSignatureBitmapPayload(recvMsg.Payload, 32)
if err != nil {
utils.Logger().Error().Err(err).
consensus.getLogger().Error().Err(err).
Msg("[onViewChange] ReadSignatureBitmapPayload Fail")
return
}
@ -384,7 +384,7 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
)
if err = consensus.commitBitmap.SetKey(consensus.PubKey, true); err != nil {
utils.Logger().Debug().
consensus.getLogger().Debug().
Msg("[OnViewChange] New Leader commit bitmap set failed")
return
}
@ -393,7 +393,7 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
consensus.current.SetViewID(recvMsg.ViewID)
msgToSend := consensus.constructNewViewMessage()
utils.Logger().Warn().
consensus.getLogger().Warn().
Int("payloadSize", len(consensus.m1Payload)).
Hex("M1Payload", consensus.m1Payload).
Msg("[onViewChange] Sent NewView Message")
@ -403,10 +403,10 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
consensus.ResetViewChangeState()
consensus.consensusTimeout[timeoutViewChange].Stop()
consensus.consensusTimeout[timeoutConsensus].Start()
utils.Logger().Debug().
consensus.getLogger().Debug().
Uint64("viewChangingID", consensus.current.ViewID()).
Msg("[onViewChange] New Leader Start Consensus Timer and Stop View Change Timer")
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("myKey", consensus.PubKey.SerializeToHexStr()).
Uint64("viewID", consensus.viewID).
Uint64("block", consensus.blockNum).
@ -416,27 +416,27 @@ func (consensus *Consensus) onViewChange(msg *msg_pb.Message) {
// TODO: move to consensus_leader.go later
func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
utils.Logger().Debug().Msg("[onNewView] Received NewView Message")
consensus.getLogger().Debug().Msg("[onNewView] Received NewView Message")
senderKey, err := consensus.verifyViewChangeSenderKey(msg)
if err != nil {
utils.Logger().Warn().Err(err).Msg("[onNewView] VerifySenderKey Failed")
consensus.getLogger().Warn().Err(err).Msg("[onNewView] VerifySenderKey Failed")
return
}
recvMsg, err := consensus.ParseNewViewMessage(msg)
if err != nil {
utils.Logger().Warn().Err(err).Msg("[onNewView] Unable to Parse NewView Message")
consensus.getLogger().Warn().Err(err).Msg("[onNewView] Unable to Parse NewView Message")
return
}
if err = verifyMessageSig(senderKey, msg); err != nil {
utils.Logger().Error().Err(err).Msg("[onNewView] Failed to Verify New Leader's Signature")
consensus.getLogger().Error().Err(err).Msg("[onNewView] Failed to Verify New Leader's Signature")
return
}
consensus.vcLock.Lock()
defer consensus.vcLock.Unlock()
if recvMsg.M3AggSig == nil || recvMsg.M3Bitmap == nil {
utils.Logger().Error().Msg("[onNewView] M3AggSig or M3Bitmap is nil")
consensus.getLogger().Error().Msg("[onNewView] M3AggSig or M3Bitmap is nil")
return
}
m3Sig := recvMsg.M3AggSig
@ -446,13 +446,13 @@ func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
binary.LittleEndian.PutUint64(viewIDBytes, recvMsg.ViewID)
if !consensus.Decider.IsQuorumAchievedByMask(m3Mask) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Msgf("[onNewView] Quorum Not achieved")
return
}
if !m3Sig.VerifyHash(m3Mask.AggregatePublic, viewIDBytes) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Str("m3Sig", m3Sig.SerializeToHexStr()).
Hex("m3Mask", m3Mask.Bitmap).
Uint64("MsgViewID", recvMsg.ViewID).
@ -462,10 +462,10 @@ func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
m2Mask := recvMsg.M2Bitmap
if recvMsg.M2AggSig != nil {
utils.Logger().Debug().Msg("[onNewView] M2AggSig (NIL) is Not Empty")
consensus.getLogger().Debug().Msg("[onNewView] M2AggSig (NIL) is Not Empty")
m2Sig := recvMsg.M2AggSig
if !m2Sig.VerifyHash(m2Mask.AggregatePublic, NIL) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Msg("[onNewView] Unable to Verify Aggregated Signature of M2 (NIL) payload")
return
}
@ -476,7 +476,7 @@ func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
(m2Mask != nil && m2Mask.Bitmap != nil &&
utils.CountOneBits(m3Mask.Bitmap) > utils.CountOneBits(m2Mask.Bitmap)) {
if len(recvMsg.Payload) <= 32 {
utils.Logger().Debug().
consensus.getLogger().Debug().
Msg("[onNewView] M1 (prepared) Type Payload Not Have Enough Length")
return
}
@ -484,12 +484,12 @@ func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
blockHash := recvMsg.Payload[:32]
aggSig, mask, err := consensus.ReadSignatureBitmapPayload(recvMsg.Payload, 32)
if err != nil {
utils.Logger().Error().Err(err).
consensus.getLogger().Error().Err(err).
Msg("[onNewView] ReadSignatureBitmapPayload Failed")
return
}
if !aggSig.VerifyHash(mask.AggregatePublic, blockHash) {
utils.Logger().Warn().
consensus.getLogger().Warn().
Msg("[onNewView] Failed to Verify Signature for M1 (prepare) message")
return
}
@ -518,7 +518,7 @@ func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
// change view and leaderKey to keep in sync with network
if consensus.blockNum != recvMsg.BlockNum {
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("newLeaderKey", consensus.LeaderPubKey.SerializeToHexStr()).
Uint64("MsgBlockNum", recvMsg.BlockNum).
Msg("[onNewView] New Leader Changed")
@ -534,21 +534,21 @@ func (consensus *Consensus) onNewView(msg *msg_pb.Message) {
commitPayload := append(blockNumHash, consensus.blockHash[:]...)
msgToSend := consensus.constructCommitMessage(commitPayload)
utils.Logger().Info().Msg("onNewView === commit")
consensus.getLogger().Info().Msg("onNewView === commit")
consensus.host.SendMessageToGroups([]nodeconfig.GroupID{nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(consensus.ShardID))}, host.ConstructP2pMessage(byte(17), msgToSend))
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("From", consensus.phase.String()).
Str("To", FBFTCommit.String()).
Msg("[OnViewChange] Switching phase")
consensus.switchPhase(FBFTCommit, true)
} else {
consensus.ResetState()
utils.Logger().Info().Msg("onNewView === announce")
consensus.getLogger().Info().Msg("onNewView === announce")
}
utils.Logger().Debug().
consensus.getLogger().Debug().
Str("newLeaderKey", consensus.LeaderPubKey.SerializeToHexStr()).
Msg("new leader changed")
utils.Logger().Debug().
consensus.getLogger().Debug().
Msg("validator start consensus timer and stop view change timer")
consensus.consensusTimeout[timeoutConsensus].Start()
consensus.consensusTimeout[timeoutViewChange].Stop()

Loading…
Cancel
Save