Merge pull request #348 from harmony-one/rj_branch

Basic multisig checking for consensus
pull/351/head
Rongjian Lan 6 years ago committed by GitHub
commit 210e1aeeaa
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 87
      consensus/consensus_leader.go
  2. 7
      consensus/consensus_leader_test.go
  3. 26
      consensus/consensus_validator.go
  4. 5
      consensus/consensus_validator_msg.go
  5. 4
      consensus/consensus_validator_msg_test.go
  6. 2
      consensus/consensus_validator_test.go

@ -12,6 +12,7 @@ import (
consensus_proto "github.com/harmony-one/harmony/api/consensus"
"github.com/harmony-one/harmony/api/services/explorer"
"github.com/harmony-one/harmony/core/types"
bls_cosi "github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/internal/profiler"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
@ -94,7 +95,7 @@ func (consensus *Consensus) startConsensus(newBlock *types.Block) {
// Set state to AnnounceDone
consensus.state = AnnounceDone
// Leader sign the multi-sig itself
// Leader sign the block hash itself
(*consensus.prepareSigs)[consensus.nodeID] = consensus.priKey.SignHash(consensus.blockHash[:])
host.BroadcastMessageFromLeader(consensus.host, consensus.GetValidatorPeers(), msgToSend, consensus.OfflinePeers)
@ -144,41 +145,46 @@ func (consensus *Consensus) processPrepareMessage(message consensus_proto.Messag
// proceed only when the message is not received before
_, ok = (*prepareSigs)[validatorID]
shouldProcess := !ok
if len((*prepareSigs)) >= ((len(consensus.PublicKeys)*2)/3 + 1) {
shouldProcess = false
if ok {
utils.GetLogInstance().Debug("Already received prepare message from the validator", "validatorID", validatorID)
return
}
if shouldProcess {
var sign bls.Sign
err := sign.Deserialize(prepareSig)
if err != nil {
utils.GetLogInstance().Error("Failed to deserialize bls signature", "validatorID", validatorID)
}
// TODO: check bls signature
(*prepareSigs)[validatorID] = &sign
utils.GetLogInstance().Debug("Received new prepare signature", "numReceivedSoFar", len(*prepareSigs), "validatorID", validatorID, "PublicKeys", len(consensus.PublicKeys))
if len((*prepareSigs)) >= ((len(consensus.PublicKeys)*2)/3 + 1) {
utils.GetLogInstance().Debug("Received additional new prepare message", "validatorID", validatorID)
return
}
// Set the bitmap indicate this validate signed.
prepareBitmap.SetKey(value.PubKey, true)
// Check BLS signature for the multi-sig
var sign bls.Sign
err = sign.Deserialize(prepareSig)
if err != nil {
utils.GetLogInstance().Error("Failed to deserialize bls signature", "validatorID", validatorID)
return
}
if !shouldProcess {
utils.GetLogInstance().Debug("Received additional new commit message", "validatorID", validatorID)
if !sign.VerifyHash(value.PubKey, consensus.blockHash[:]) {
utils.GetLogInstance().Error("Received invalid BLS signature", "validatorID", validatorID)
return
}
(*prepareSigs)[validatorID] = &sign
utils.GetLogInstance().Debug("Received new prepare signature", "numReceivedSoFar", len(*prepareSigs), "validatorID", validatorID, "PublicKeys", len(consensus.PublicKeys))
// Set the bitmap indicate this validate signed.
prepareBitmap.SetKey(value.PubKey, true)
targetState := PreparedDone
if len((*prepareSigs)) >= ((len(consensus.PublicKeys)*2)/3+1) && consensus.state < targetState {
utils.GetLogInstance().Debug("Enough commitments received with signatures", "num", len(*prepareSigs), "state", consensus.state)
utils.GetLogInstance().Debug("Enough prepares received with signatures", "num", len(*prepareSigs), "state", consensus.state)
// Construct prepared message
msgToSend, aggSig := consensus.constructPreparedMessage()
consensus.aggregatedPrepareSig = aggSig
// Leader sign the multi-sig itself
// TODO: sign on the prepared multi-sig, rather than the block hash
(*consensus.commitSigs)[consensus.nodeID] = consensus.priKey.SignHash(consensus.blockHash[:])
// Leader sign the multi-sig and bitmap
multiSigAndBitmap := append(aggSig.Serialize(), prepareBitmap.Bitmap...)
(*consensus.commitSigs)[consensus.nodeID] = consensus.priKey.SignHash(multiSigAndBitmap)
// Broadcast prepared message
host.BroadcastMessageFromLeader(consensus.host, consensus.GetValidatorPeers(), msgToSend, consensus.OfflinePeers)
@ -195,14 +201,13 @@ func (consensus *Consensus) processCommitMessage(message consensus_proto.Message
validatorID := message.SenderId
commitSig := message.Payload
shouldProcess := true
consensus.mutex.Lock()
defer consensus.mutex.Unlock()
// check consensus Id
if consensusID != consensus.consensusID {
shouldProcess = false
utils.GetLogInstance().Warn("Received Commit with wrong consensus Id", "myConsensusId", consensus.consensusID, "theirConsensusId", consensusID, "consensus", consensus)
return
}
if !bytes.Equal(blockHash, consensus.blockHash[:]) {
@ -234,30 +239,34 @@ func (consensus *Consensus) processCommitMessage(message consensus_proto.Message
// proceed only when the message is not received before
_, ok = (*commitSigs)[validatorID]
shouldProcess = shouldProcess && !ok
if ok {
utils.GetLogInstance().Debug("Already received commit message from the validator", "validatorID", validatorID)
return
}
if len((*commitSigs)) >= ((len(consensus.PublicKeys)*2)/3 + 1) {
shouldProcess = false
utils.GetLogInstance().Debug("Received additional new commit message", "validatorID", strconv.Itoa(int(validatorID)))
return
}
if shouldProcess {
var sign bls.Sign
err := sign.Deserialize(commitSig)
if err != nil {
utils.GetLogInstance().Debug("Failed to deserialize bls signature", "validatorID", validatorID)
}
// TODO: check bls signature
(*commitSigs)[validatorID] = &sign
utils.GetLogInstance().Debug("Received new commit message", "numReceivedSoFar", len(*commitSigs), "validatorID", strconv.Itoa(int(validatorID)))
// Set the bitmap indicate this validate signed.
commitBitmap.SetKey(value.PubKey, true)
var sign bls.Sign
err = sign.Deserialize(commitSig)
if err != nil {
utils.GetLogInstance().Debug("Failed to deserialize bls signature", "validatorID", validatorID)
return
}
if !shouldProcess {
utils.GetLogInstance().Debug("Received additional new commit message", "validatorID", strconv.Itoa(int(validatorID)))
// Verify the signature on prepare multi-sig and bitmap is correct
aggSig := bls_cosi.AggregateSig(consensus.GetPrepareSigsArray())
if !sign.VerifyHash(value.PubKey, append(aggSig.Serialize(), consensus.prepareBitmap.Bitmap...)) {
utils.GetLogInstance().Error("Received invalid BLS signature", "validatorID", validatorID)
return
}
(*commitSigs)[validatorID] = &sign
utils.GetLogInstance().Debug("Received new commit message", "numReceivedSoFar", len(*commitSigs), "validatorID", strconv.Itoa(int(validatorID)))
// Set the bitmap indicate this validate signed.
commitBitmap.SetKey(value.PubKey, true)
targetState := CommittedDone
if len(*commitSigs) >= ((len(consensus.PublicKeys)*2)/3+1) && consensus.state != targetState {
utils.GetLogInstance().Info("Enough commits received!", "num", len(*commitSigs), "state", consensus.state)

@ -13,6 +13,7 @@ import (
protobuf "github.com/golang/protobuf/proto"
consensus_proto "github.com/harmony-one/harmony/api/consensus"
"github.com/harmony-one/harmony/core/types"
bls_cosi "github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/internal/utils"
mock_host "github.com/harmony-one/harmony/p2p/host/mock"
"github.com/stretchr/testify/assert"
@ -159,6 +160,10 @@ func TestProcessMessageLeaderCommit(test *testing.T) {
consensusLeader.blockHash = blockHash
consensusLeader.OnConsensusDone = func(newBlock *types.Block) {}
consensusLeader.block, _ = rlp.EncodeToBytes(types.NewBlock(&types.Header{}, nil, nil))
(*consensusLeader.prepareSigs)[consensusLeader.nodeID] = consensusLeader.priKey.SignHash(consensusLeader.blockHash[:])
aggSig := bls_cosi.AggregateSig(consensusLeader.GetPrepareSigsArray())
multiSigAndBitmap := append(aggSig.Serialize(), consensusLeader.prepareBitmap.Bitmap...)
consensusValidators := make([]*Consensus, 3)
@ -169,7 +174,7 @@ func TestProcessMessageLeaderCommit(test *testing.T) {
for i := 0; i < 3; i++ {
consensusValidators[i] = New(hosts[i], "0", validators, leader)
consensusValidators[i].blockHash = blockHash
msg := consensusValidators[i].constructCommitMessage()
msg := consensusValidators[i].constructCommitMessage(multiSigAndBitmap)
consensusLeader.ProcessMessageLeader(msg[1:])
}

@ -3,6 +3,9 @@ package consensus
import (
"bytes"
"github.com/harmony-one/bls/ffi/go/bls"
bls_cosi "github.com/harmony-one/harmony/crypto/bls"
"github.com/ethereum/go-ethereum/rlp"
protobuf "github.com/golang/protobuf/proto"
consensus_proto "github.com/harmony-one/harmony/api/consensus"
@ -152,12 +155,22 @@ func (consensus *Consensus) processPreparedMessage(message consensus_proto.Messa
return
}
_ = multiSig
_ = bitmap
// TODO: verify multi signature
deserializedMultiSig := bls.Sign{}
err = deserializedMultiSig.Deserialize(multiSig)
if err != nil {
utils.GetLogInstance().Warn("Failed to deserialize the multi signature", "Error", err, "leader ID", leaderID)
return
}
mask, err := bls_cosi.NewMask(consensus.PublicKeys, nil)
mask.SetMask(bitmap)
if !deserializedMultiSig.VerifyHash(mask.AggregatePublic, blockHash) || err != nil {
utils.GetLogInstance().Warn("Failed to verify the multi signature", "Error", err, "leader ID", leaderID)
return
}
multiSigAndBitmap := append(multiSig, bitmap...)
// Construct and send the commit message
msgToSend := consensus.constructCommitMessage()
msgToSend := consensus.constructCommitMessage(multiSigAndBitmap)
consensus.SendMessage(consensus.leader, msgToSend)
@ -226,11 +239,6 @@ func (consensus *Consensus) processCommittedMessage(message consensus_proto.Mess
_ = bitmap
// TODO: verify multi signature
// Construct and send the prepare message
msgToSend := consensus.constructPrepareMessage()
consensus.SendMessage(consensus.leader, msgToSend)
consensus.state = CommittedDone
// TODO: the block catch up logic is a temporary workaround for full failure node catchup. Implement the full node catchup logic

@ -44,7 +44,7 @@ func (consensus *Consensus) constructPrepareMessage() []byte {
}
// Construct the commit message to send to leader (assumption the consensus data is already verified)
func (consensus *Consensus) constructCommitMessage() []byte {
func (consensus *Consensus) constructCommitMessage(multiSigAndBitmap []byte) []byte {
message := consensus_proto.Message{}
message.Type = consensus_proto.MessageType_COMMIT
@ -58,8 +58,7 @@ func (consensus *Consensus) constructCommitMessage() []byte {
message.SenderId = uint32(consensus.nodeID)
// 48 byte of bls signature
// TODO: sign on the prepared message hash, rather than the block hash
sign := consensus.priKey.SignHash(message.BlockHash)
sign := consensus.priKey.SignHash(multiSigAndBitmap)
if sign != nil {
message.Payload = sign.Serialize()
}

@ -36,9 +36,9 @@ func TestConstructCommitMessage(test *testing.T) {
}
consensus := New(host, "0", []p2p.Peer{leader, validator}, leader)
consensus.blockHash = [32]byte{}
msg := consensus.constructCommitMessage()
msg := consensus.constructCommitMessage([]byte("random string"))
if len(msg) != 93 {
if len(msg) != 143 {
test.Errorf("Commit message is not constructed in the correct size: %d", len(msg))
}
}

@ -104,6 +104,8 @@ func TestProcessMessageValidatorPrepared(test *testing.T) {
copy(consensusLeader.blockHash[:], hashBytes[:])
announceMsg := consensusLeader.constructAnnounceMessage()
(*consensusLeader.prepareSigs)[consensusLeader.nodeID] = consensusLeader.priKey.SignHash(consensusLeader.blockHash[:])
preparedMsg, _ := consensusLeader.constructPreparedMessage()
if err != nil {

Loading…
Cancel
Save