From 421d72b2d796195178104a0eb1dedf319ba8664c Mon Sep 17 00:00:00 2001 From: Mission Liao Date: Thu, 20 Sep 2018 09:09:37 +0800 Subject: Rename validator* to node* (#120) --- core/agreement-state.go | 2 +- core/agreement-state_test.go | 76 ++++++++-------- core/agreement.go | 48 +++++----- core/agreement_test.go | 32 +++---- core/blockdb/level-db_test.go | 8 +- core/blockdb/memory_test.go | 4 +- core/compaction-chain.go | 12 +-- core/compaction-chain_test.go | 12 +-- core/consensus-timestamp_test.go | 8 +- core/consensus.go | 48 +++++----- core/consensus_test.go | 110 +++++++++++----------- core/crypto.go | 12 +-- core/crypto_test.go | 20 ++-- core/dkg-tsig-protocol.go | 56 ++++++------ core/dkg-tsig-protocol_test.go | 116 +++++++++++------------ core/interfaces.go | 6 +- core/leader-selector_test.go | 4 +- core/negative-ack.go | 146 ++++++++++++++--------------- core/negative-ack_test.go | 126 ++++++++++++------------- core/reliable-broadcast.go | 46 +++++----- core/reliable-broadcast_test.go | 82 ++++++++--------- core/test/blocks-generator.go | 157 ++++++++++++++++---------------- core/test/blocks-generator_test.go | 44 ++++----- core/test/fake-transport.go | 28 +++--- core/test/governance.go | 28 +++--- core/test/interface.go | 12 +-- core/test/revealer.go | 36 ++++---- core/test/revealer_test.go | 12 +-- core/test/scheduler-event.go | 8 +- core/test/scheduler.go | 20 ++-- core/test/scheduler_test.go | 60 ++++++------ core/test/stopper.go | 42 ++++----- core/test/stopper_test.go | 34 +++---- core/test/tcp-transport.go | 86 +++++++++--------- core/test/transport_test.go | 64 ++++++------- core/test/utils.go | 10 +- core/total-ordering_test.go | 182 ++++++++++++++++++------------------- core/types/block.go | 2 +- core/types/dkg.go | 8 +- core/types/node.go | 56 ++++++++++++ core/types/validator.go | 56 ------------ core/types/vote.go | 2 +- core/types/witness.go | 2 +- 43 files changed, 963 insertions(+), 960 deletions(-) create mode 100644 core/types/node.go delete mode 100644 core/types/validator.go (limited to 'core') diff --git a/core/agreement-state.go b/core/agreement-state.go index d7225d6..fbee21a 100644 --- a/core/agreement-state.go +++ b/core/agreement-state.go @@ -242,7 +242,7 @@ func (s *pass2State) nextState() (agreementState, error) { s.a.defaultBlock = hash s.a.period++ oldBlock := s.a.blocks[s.a.ID] - s.a.blocks = map[types.ValidatorID]*types.Block{ + s.a.blocks = map[types.NodeID]*types.Block{ s.a.ID: oldBlock, } } diff --git a/core/agreement-state_test.go b/core/agreement-state_test.go index bc8dc8e..79e4102 100644 --- a/core/agreement-state_test.go +++ b/core/agreement-state_test.go @@ -31,8 +31,8 @@ import ( type AgreementStateTestSuite struct { suite.Suite - ID types.ValidatorID - prvKey map[types.ValidatorID]crypto.PrivateKey + ID types.NodeID + prvKey map[types.NodeID]crypto.PrivateKey voteChan chan *types.Vote blockChan chan common.Hash confirmChan chan common.Hash @@ -67,13 +67,13 @@ func (s *AgreementStateTestSuite) proposeBlock( } func (s *AgreementStateTestSuite) prepareVote( - vID types.ValidatorID, voteType types.VoteType, blockHash common.Hash, + nID types.NodeID, voteType types.VoteType, blockHash common.Hash, period uint64) ( vote *types.Vote) { - prvKey, exist := s.prvKey[vID] + prvKey, exist := s.prvKey[nID] s.Require().True(exist) vote = &types.Vote{ - ProposerID: vID, + ProposerID: nID, Type: voteType, BlockHash: blockHash, Period: period, @@ -87,8 +87,8 @@ func (s *AgreementStateTestSuite) prepareVote( func (s *AgreementStateTestSuite) SetupTest() { prvKey, err := eth.NewPrivateKey() s.Require().Nil(err) - s.ID = types.NewValidatorID(prvKey.PublicKey()) - s.prvKey = map[types.ValidatorID]crypto.PrivateKey{ + s.ID = types.NewNodeID(prvKey.PublicKey()) + s.prvKey = map[types.NodeID]crypto.PrivateKey{ s.ID: prvKey, } s.voteChan = make(chan *types.Vote, 100) @@ -97,24 +97,24 @@ func (s *AgreementStateTestSuite) SetupTest() { s.block = make(map[common.Hash]*types.Block) } -func (s *AgreementStateTestSuite) newAgreement(numValidator int) *agreement { +func (s *AgreementStateTestSuite) newAgreement(numNode int) *agreement { leader := newGenesisLeaderSelector("I ❤️ DEXON", eth.SigToPub) blockProposer := func() *types.Block { return s.proposeBlock(leader) } - validators := make(types.ValidatorIDs, numValidator-1) - for i := range validators { + notarySet := make(types.NodeIDs, numNode-1) + for i := range notarySet { prvKey, err := eth.NewPrivateKey() s.Require().Nil(err) - validators[i] = types.NewValidatorID(prvKey.PublicKey()) - s.prvKey[validators[i]] = prvKey + notarySet[i] = types.NewNodeID(prvKey.PublicKey()) + s.prvKey[notarySet[i]] = prvKey } - validators = append(validators, s.ID) + notarySet = append(notarySet, s.ID) agreement := newAgreement( s.ID, &agreementStateTestReceiver{s}, - validators, + notarySet, leader, eth.SigToPub, blockProposer, @@ -144,8 +144,8 @@ func (s *AgreementStateTestSuite) TestPrepareState() { _, err = state.nextState() s.Equal(ErrNoEnoughVoteInPrepareState, err) - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, common.Hash{}, 1) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, common.Hash{}, 1) s.Require().Nil(a.processVote(vote)) } @@ -159,11 +159,11 @@ func (s *AgreementStateTestSuite) TestPrepareState() { block := s.proposeBlock(a.data.leader) prv, err := eth.NewPrivateKey() s.Require().Nil(err) - block.ProposerID = types.NewValidatorID(prv.PublicKey()) + block.ProposerID = types.NewNodeID(prv.PublicKey()) s.Require().Nil(a.data.leader.prepareBlock(block, prv)) s.Require().Nil(a.processBlock(block)) - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, block.Hash, 2) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, block.Hash, 2) s.Require().Nil(a.processVote(vote)) } @@ -183,7 +183,7 @@ func (s *AgreementStateTestSuite) TestAckState() { blocks[i] = s.proposeBlock(a.data.leader) prv, err := eth.NewPrivateKey() s.Require().Nil(err) - blocks[i].ProposerID = types.NewValidatorID(prv.PublicKey()) + blocks[i].ProposerID = types.NewNodeID(prv.PublicKey()) s.Require().Nil(a.data.leader.prepareBlock(blocks[i], prv)) s.Require().Nil(a.processBlock(blocks[i])) } @@ -201,8 +201,8 @@ func (s *AgreementStateTestSuite) TestAckState() { // For period >= 2, if block v equal to {} has more than 2f+1 pass-vote // in period 1, propose ack-vote for the block having largest potential. a.data.period = 2 - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, common.Hash{}, 1) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, common.Hash{}, 1) s.Require().Nil(a.processVote(vote)) } newState, err = state.nextState() @@ -217,8 +217,8 @@ func (s *AgreementStateTestSuite) TestAckState() { // in period 1, propose ack-vote for block v. hash := blocks[0].Hash a.data.period = 3 - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, hash, 2) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, hash, 2) s.Require().Nil(a.processVote(vote)) } newState, err = state.nextState() @@ -241,8 +241,8 @@ func (s *AgreementStateTestSuite) TestConfirmState() { a.data.period = 1 block := s.proposeBlock(a.data.leader) s.Require().Nil(a.processBlock(block)) - for vID := range a.validators { - vote := s.prepareVote(vID, types.VoteAck, block.Hash, 1) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VoteAck, block.Hash, 1) s.Require().Nil(a.processVote(vote)) } s.Require().Nil(state.receiveVote()) @@ -265,8 +265,8 @@ func (s *AgreementStateTestSuite) TestConfirmState() { // If there are 2f+1 ack-vote for block v equal to {}, // no vote should be proposed. a.data.period = 3 - for vID := range a.validators { - vote := s.prepareVote(vID, types.VoteAck, common.Hash{}, 3) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VoteAck, common.Hash{}, 3) s.Require().Nil(a.processVote(vote)) } s.Require().Nil(state.receiveVote()) @@ -299,8 +299,8 @@ func (s *AgreementStateTestSuite) TestPass1State() { // Else if period >= 2 and has 2f+1 pass-vote in period-1 for block {}, // propose pass-vote for block {}. a.data.period = 2 - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, common.Hash{}, 1) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, common.Hash{}, 1) s.Require().Nil(a.processVote(vote)) } vote = s.prepareVote(s.ID, types.VoteAck, common.Hash{}, 2) @@ -318,8 +318,8 @@ func (s *AgreementStateTestSuite) TestPass1State() { block := s.proposeBlock(a.data.leader) a.data.defaultBlock = block.Hash hash = common.NewRandomHash() - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, hash, 2) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, hash, 2) s.Require().Nil(a.processVote(vote)) } vote = s.prepareVote(s.ID, types.VoteAck, common.Hash{}, 3) @@ -371,8 +371,8 @@ func (s *AgreementStateTestSuite) TestPass2State() { // propose pass-vote for v. block := s.proposeBlock(a.data.leader) s.Require().Nil(a.processBlock(block)) - for vID := range a.validators { - vote := s.prepareVote(vID, types.VoteAck, block.Hash, 1) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VoteAck, block.Hash, 1) s.Require().Nil(a.processVote(vote)) } s.Require().Nil(state.receiveVote()) @@ -386,8 +386,8 @@ func (s *AgreementStateTestSuite) TestPass2State() { a = s.newAgreement(4) state = newPass2State(a.data) a.data.period = 2 - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, common.Hash{}, 1) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, common.Hash{}, 1) s.Require().Nil(a.processVote(vote)) } vote := s.prepareVote(s.ID, types.VoteAck, common.Hash{}, 2) @@ -412,8 +412,8 @@ func (s *AgreementStateTestSuite) TestPass2State() { a = s.newAgreement(4) state = newPass2State(a.data) a.data.period = 1 - for vID := range a.validators { - vote := s.prepareVote(vID, types.VotePass, common.Hash{}, 1) + for nID := range a.notarySet { + vote := s.prepareVote(nID, types.VotePass, common.Hash{}, 1) s.Require().Nil(a.processVote(vote)) } s.Require().Nil(state.receiveVote()) diff --git a/core/agreement.go b/core/agreement.go index 86cb6fc..ffc4ba8 100644 --- a/core/agreement.go +++ b/core/agreement.go @@ -30,38 +30,38 @@ import ( // Errors for agreement module. var ( - ErrNotValidator = fmt.Errorf("not a validaotr") + ErrNotInNotarySet = fmt.Errorf("not in notary set") ErrIncorrectVoteSignature = fmt.Errorf("incorrect vote signature") ) // ErrFork for fork error in agreement. type ErrFork struct { - vID types.ValidatorID + nID types.NodeID old, new common.Hash } func (e *ErrFork) Error() string { return fmt.Sprintf("fork is found for %s, old %s, new %s", - e.vID.String(), e.old, e.new) + e.nID.String(), e.old, e.new) } // ErrForkVote for fork vote error in agreement. type ErrForkVote struct { - vID types.ValidatorID + nID types.NodeID old, new *types.Vote } func (e *ErrForkVote) Error() string { return fmt.Sprintf("fork vote is found for %s, old %s, new %s", - e.vID.String(), e.old, e.new) + e.nID.String(), e.old, e.new) } type blockProposerFn func() *types.Block -func newVoteListMap() []map[types.ValidatorID]*types.Vote { - listMap := make([]map[types.ValidatorID]*types.Vote, types.MaxVoteType) +func newVoteListMap() []map[types.NodeID]*types.Vote { + listMap := make([]map[types.NodeID]*types.Vote, types.MaxVoteType) for idx := range listMap { - listMap[idx] = make(map[types.ValidatorID]*types.Vote) + listMap[idx] = make(map[types.NodeID]*types.Vote) } return listMap } @@ -87,14 +87,14 @@ type pendingVote struct { type agreementData struct { recv agreementReceiver - ID types.ValidatorID + ID types.NodeID leader *leaderSelector defaultBlock common.Hash period uint64 requiredVote int - votes map[uint64][]map[types.ValidatorID]*types.Vote + votes map[uint64][]map[types.NodeID]*types.Vote votesLock sync.RWMutex - blocks map[types.ValidatorID]*types.Block + blocks map[types.NodeID]*types.Block blocksLock sync.Mutex blockProposer blockProposerFn } @@ -104,7 +104,7 @@ type agreement struct { state agreementState data *agreementData aID *atomic.Value - validators map[types.ValidatorID]struct{} + notarySet map[types.NodeID]struct{} sigToPub SigToPubFn hasOutput bool lock sync.RWMutex @@ -115,9 +115,9 @@ type agreement struct { // newAgreement creates a agreement instance. func newAgreement( - ID types.ValidatorID, + ID types.NodeID, recv agreementReceiver, - validators types.ValidatorIDs, + notarySet types.NodeIDs, leader *leaderSelector, sigToPub SigToPubFn, blockProposer blockProposerFn) *agreement { @@ -132,7 +132,7 @@ func newAgreement( sigToPub: sigToPub, candidateBlock: make(map[common.Hash]*types.Block), } - agreement.restart(validators, types.Position{}) + agreement.restart(notarySet, types.Position{}) return agreement } @@ -144,7 +144,7 @@ func (a *agreement) terminate() { } // restart the agreement -func (a *agreement) restart(validators types.ValidatorIDs, aID types.Position) { +func (a *agreement) restart(notarySet types.NodeIDs, aID types.Position) { func() { a.lock.Lock() defer a.lock.Unlock() @@ -152,18 +152,18 @@ func (a *agreement) restart(validators types.ValidatorIDs, aID types.Position) { defer a.data.votesLock.Unlock() a.data.blocksLock.Lock() defer a.data.blocksLock.Unlock() - a.data.votes = make(map[uint64][]map[types.ValidatorID]*types.Vote) + a.data.votes = make(map[uint64][]map[types.NodeID]*types.Vote) a.data.votes[1] = newVoteListMap() a.data.period = 1 - a.data.blocks = make(map[types.ValidatorID]*types.Block) - a.data.requiredVote = len(validators)/3*2 + 1 + a.data.blocks = make(map[types.NodeID]*types.Block) + a.data.requiredVote = len(notarySet)/3*2 + 1 a.data.leader.restart() a.data.defaultBlock = common.Hash{} a.hasOutput = false a.state = newPrepareState(a.data) - a.validators = make(map[types.ValidatorID]struct{}) - for _, v := range validators { - a.validators[v] = struct{}{} + a.notarySet = make(map[types.NodeID]struct{}) + for _, v := range notarySet { + a.notarySet[v] = struct{}{} } a.candidateBlock = make(map[common.Hash]*types.Block) a.aID.Store(aID) @@ -232,10 +232,10 @@ func (a *agreement) sanityCheck(vote *types.Vote) error { if exist := func() bool { a.lock.RLock() defer a.lock.RUnlock() - _, exist := a.validators[vote.ProposerID] + _, exist := a.notarySet[vote.ProposerID] return exist }(); !exist { - return ErrNotValidator + return ErrNotInNotarySet } ok, err := verifyVoteSignature(vote, a.sigToPub) if err != nil { diff --git a/core/agreement_test.go b/core/agreement_test.go index 4f6ec5b..b344dd9 100644 --- a/core/agreement_test.go +++ b/core/agreement_test.go @@ -57,8 +57,8 @@ func (s *AgreementTestSuite) proposeBlock( type AgreementTestSuite struct { suite.Suite - ID types.ValidatorID - prvKey map[types.ValidatorID]crypto.PrivateKey + ID types.NodeID + prvKey map[types.NodeID]crypto.PrivateKey voteChan chan *types.Vote blockChan chan common.Hash confirmChan chan common.Hash @@ -69,8 +69,8 @@ type AgreementTestSuite struct { func (s *AgreementTestSuite) SetupTest() { prvKey, err := eth.NewPrivateKey() s.Require().Nil(err) - s.ID = types.NewValidatorID(prvKey.PublicKey()) - s.prvKey = map[types.ValidatorID]crypto.PrivateKey{ + s.ID = types.NewNodeID(prvKey.PublicKey()) + s.prvKey = map[types.NodeID]crypto.PrivateKey{ s.ID: prvKey, } s.voteChan = make(chan *types.Vote, 100) @@ -79,25 +79,25 @@ func (s *AgreementTestSuite) SetupTest() { s.block = make(map[common.Hash]*types.Block) } -func (s *AgreementTestSuite) newAgreement(numValidator int) *agreement { +func (s *AgreementTestSuite) newAgreement(numNotarySet int) *agreement { leader := newGenesisLeaderSelector("🖖👽", eth.SigToPub) agreementIdx := len(s.agreement) blockProposer := func() *types.Block { return s.proposeBlock(agreementIdx) } - validators := make(types.ValidatorIDs, numValidator-1) - for i := range validators { + notarySet := make(types.NodeIDs, numNotarySet-1) + for i := range notarySet { prvKey, err := eth.NewPrivateKey() s.Require().Nil(err) - validators[i] = types.NewValidatorID(prvKey.PublicKey()) - s.prvKey[validators[i]] = prvKey + notarySet[i] = types.NewNodeID(prvKey.PublicKey()) + s.prvKey[notarySet[i]] = prvKey } - validators = append(validators, s.ID) + notarySet = append(notarySet, s.ID) agreement := newAgreement( s.ID, &agreementTestReceiver{s}, - validators, + notarySet, leader, eth.SigToPub, blockProposer, @@ -116,7 +116,7 @@ func (s *AgreementTestSuite) prepareVote(vote *types.Vote) { } func (s *AgreementTestSuite) copyVote( - vote *types.Vote, proposer types.ValidatorID) *types.Vote { + vote *types.Vote, proposer types.NodeID) *types.Vote { v := vote.Clone() v.ProposerID = proposer s.prepareVote(v) @@ -138,8 +138,8 @@ func (s *AgreementTestSuite) TestSimpleConfirm() { s.Require().Len(s.voteChan, 1) vote := <-s.voteChan s.Equal(types.VoteAck, vote.Type) - for vID := range s.prvKey { - v := s.copyVote(vote, vID) + for nID := range s.prvKey { + v := s.copyVote(vote, nID) s.Require().NoError(a.processVote(v)) } a.nextState() @@ -147,8 +147,8 @@ func (s *AgreementTestSuite) TestSimpleConfirm() { s.Require().Len(s.voteChan, 1) vote = <-s.voteChan s.Equal(types.VoteConfirm, vote.Type) - for vID := range s.prvKey { - v := s.copyVote(vote, vID) + for nID := range s.prvKey { + v := s.copyVote(vote, nID) s.Require().NoError(a.processVote(v)) } // We have enough of Confirm-Votes. diff --git a/core/blockdb/level-db_test.go b/core/blockdb/level-db_test.go index 06829f0..9cabc4a 100644 --- a/core/blockdb/level-db_test.go +++ b/core/blockdb/level-db_test.go @@ -51,9 +51,9 @@ func (s *LevelDBTestSuite) TestBasicUsage() { s.Equal(ErrBlockDoesNotExist, err) // Update on an empty database should not success. - validator1 := types.ValidatorID{Hash: common.NewRandomHash()} + node1 := types.NodeID{Hash: common.NewRandomHash()} block1 := types.Block{ - ProposerID: validator1, + ProposerID: node1, Hash: hash1, Position: types.Position{ Height: 1, @@ -78,7 +78,7 @@ func (s *LevelDBTestSuite) TestBasicUsage() { err = db.Update(queried) s.Nil(err) - // Try to get it back via ValidatorID and height. + // Try to get it back via NodeID and height. queried, err = db.Get(block1.Hash) s.Nil(err) @@ -100,7 +100,7 @@ func (s *LevelDBTestSuite) TestSyncIndex() { blocks := [10]types.Block{} for i := range blocks { block := types.Block{ - ProposerID: types.ValidatorID{Hash: common.NewRandomHash()}, + ProposerID: types.NodeID{Hash: common.NewRandomHash()}, Hash: common.NewRandomHash(), Position: types.Position{ Height: uint64(i), diff --git a/core/blockdb/memory_test.go b/core/blockdb/memory_test.go index 9a3cfa2..a77d355 100644 --- a/core/blockdb/memory_test.go +++ b/core/blockdb/memory_test.go @@ -29,12 +29,12 @@ import ( type MemBackedBlockDBTestSuite struct { suite.Suite - v0 types.ValidatorID + v0 types.NodeID b00, b01, b02 *types.Block } func (s *MemBackedBlockDBTestSuite) SetupSuite() { - s.v0 = types.ValidatorID{Hash: common.NewRandomHash()} + s.v0 = types.NodeID{Hash: common.NewRandomHash()} genesisHash := common.NewRandomHash() s.b00 = &types.Block{ diff --git a/core/compaction-chain.go b/core/compaction-chain.go index c4e0f2f..c72cd7b 100644 --- a/core/compaction-chain.go +++ b/core/compaction-chain.go @@ -52,7 +52,7 @@ type compactionChain struct { prevBlockLock sync.RWMutex prevBlock *types.Block witnessAcksLock sync.RWMutex - latestWitnessAcks map[types.ValidatorID]*types.WitnessAck + latestWitnessAcks map[types.NodeID]*types.WitnessAck sigToPub SigToPubFn } @@ -63,7 +63,7 @@ func newCompactionChain( return &compactionChain{ db: db, pendingAck: make(map[common.Hash]*pendingAck), - latestWitnessAcks: make(map[types.ValidatorID]*types.WitnessAck), + latestWitnessAcks: make(map[types.NodeID]*types.WitnessAck), sigToPub: sigToPub, } } @@ -83,7 +83,7 @@ func (cc *compactionChain) sanityCheck( if err != nil { return err } - if witnessAck.ProposerID != types.NewValidatorID(pubKey) { + if witnessAck.ProposerID != types.NewNodeID(pubKey) { return ErrIncorrectWitnessSignature } return nil @@ -123,7 +123,7 @@ func (cc *compactionChain) prepareWitnessAck(prvKey crypto.PrivateKey) ( return } witnessAck = &types.WitnessAck{ - ProposerID: types.NewValidatorID(prvKey.PublicKey()), + ProposerID: types.NewNodeID(prvKey.PublicKey()), WitnessBlockHash: lastBlock.Hash, Signature: sig, Hash: hash, @@ -230,10 +230,10 @@ func (cc *compactionChain) processPendingWitnessAcks() { cc.pendingAck = pendingAck } -func (cc *compactionChain) witnessAcks() map[types.ValidatorID]*types.WitnessAck { +func (cc *compactionChain) witnessAcks() map[types.NodeID]*types.WitnessAck { cc.witnessAcksLock.RLock() defer cc.witnessAcksLock.RUnlock() - acks := make(map[types.ValidatorID]*types.WitnessAck) + acks := make(map[types.NodeID]*types.WitnessAck) for k, v := range cc.latestWitnessAcks { acks[k] = v.Clone() } diff --git a/core/compaction-chain_test.go b/core/compaction-chain_test.go index 70a368d..5c08798 100644 --- a/core/compaction-chain_test.go +++ b/core/compaction-chain_test.go @@ -117,8 +117,8 @@ func (s *CompactionChainTestSuite) TestProcessWitnessAck() { s.Require().Nil(err) prv2, err := eth.NewPrivateKey() s.Require().Nil(err) - vID1 := types.NewValidatorID(prv1.PublicKey()) - vID2 := types.NewValidatorID(prv2.PublicKey()) + nID1 := types.NewNodeID(prv1.PublicKey()) + nID2 := types.NewNodeID(prv2.PublicKey()) witnessAcks1 := []*types.WitnessAck{} witnessAcks2 := []*types.WitnessAck{} for _, block := range blocks { @@ -150,8 +150,8 @@ func (s *CompactionChainTestSuite) TestProcessWitnessAck() { s.Nil(cc.processWitnessAck(witnessAcks1[3])) acks := cc.witnessAcks() - s.Equal(blocks[3].Hash, acks[vID1].WitnessBlockHash) - s.Equal(blocks[1].Hash, acks[vID2].WitnessBlockHash) + s.Equal(blocks[3].Hash, acks[nID1].WitnessBlockHash) + s.Equal(blocks[1].Hash, acks[nID2].WitnessBlockHash) // Test that witnessAck on less Witness.Height should be ignored. s.Require().Nil(s.db.Put(*blocks[4])) @@ -162,8 +162,8 @@ func (s *CompactionChainTestSuite) TestProcessWitnessAck() { s.Nil(cc.processWitnessAck(witnessAcks2[4])) acks = cc.witnessAcks() - s.Equal(blocks[5].Hash, acks[vID1].WitnessBlockHash) - s.Equal(blocks[5].Hash, acks[vID2].WitnessBlockHash) + s.Equal(blocks[5].Hash, acks[nID1].WitnessBlockHash) + s.Equal(blocks[5].Hash, acks[nID2].WitnessBlockHash) } func TestCompactionChain(t *testing.T) { diff --git a/core/consensus-timestamp_test.go b/core/consensus-timestamp_test.go index a5f9fb7..be90f41 100644 --- a/core/consensus-timestamp_test.go +++ b/core/consensus-timestamp_test.go @@ -91,7 +91,7 @@ func (s *ConsensusTimestampTest) extractTimestamps( // should have the same result as processing the whole chain at once. func (s *ConsensusTimestampTest) TestTimestampPartition() { blockNums := []int{50, 100, 30} - validatorNum := 19 + chainNum := 19 sigma := 100 * time.Millisecond totalTimestamps := make([]time.Time, 0) ct := newConsensusTimestamp() @@ -100,7 +100,7 @@ func (s *ConsensusTimestampTest) TestTimestampPartition() { totalBlockNum += blockNum } totalChain := s.generateBlocksWithTimestamp( - totalBlockNum, validatorNum, time.Second, sigma) + totalBlockNum, chainNum, time.Second, sigma) for _, blockNum := range blockNums { var chain []*types.Block chain, totalChain = totalChain[:blockNum], totalChain[blockNum:] @@ -118,10 +118,10 @@ func (s *ConsensusTimestampTest) TestTimestampPartition() { } func (s *ConsensusTimestampTest) TestTimestampIncrease() { - validatorNum := 19 + chainNum := 19 sigma := 100 * time.Millisecond ct := newConsensusTimestamp() - chain := s.generateBlocksWithTimestamp(1000, validatorNum, time.Second, sigma) + chain := s.generateBlocksWithTimestamp(1000, chainNum, time.Second, sigma) err := ct.processBlocks(chain) s.Require().NoError(err) timestamps := s.extractTimestamps(chain) diff --git a/core/consensus.go b/core/consensus.go index 7398628..1af66b3 100644 --- a/core/consensus.go +++ b/core/consensus.go @@ -47,8 +47,8 @@ func (e *ErrMissingBlockInfo) Error() string { // Errors for consensus core. var ( - ErrProposerNotValidator = fmt.Errorf( - "proposer is not a validator") + ErrProposerNotInNotarySet = fmt.Errorf( + "proposer is not in notary set") ErrIncorrectHash = fmt.Errorf( "hash of block is incorrect") ErrIncorrectSignature = fmt.Errorf( @@ -113,7 +113,7 @@ func (recv *consensusReceiver) ConfirmBlock(hash common.Hash) { // Consensus implements DEXON Consensus algorithm. type Consensus struct { - ID types.ValidatorID + ID types.NodeID app Application gov Governance config *types.Config @@ -143,29 +143,29 @@ func NewConsensus( sigToPub SigToPubFn) *Consensus { config := gov.GetConfiguration(0) - validatorSet := gov.GetNotarySet() + notarySet := gov.GetNotarySet() // Setup acking by information returned from Governace. rb := newReliableBroadcast() rb.setChainNum(config.NumChains) - for vID := range validatorSet { - rb.addValidator(vID) + for nID := range notarySet { + rb.addNode(nID) } // Setup context. ctx, ctxCancel := context.WithCancel(context.Background()) // Setup sequencer by information returned from Governace. - var validators types.ValidatorIDs - for vID := range validatorSet { - validators = append(validators, vID) + var nodes types.NodeIDs + for nID := range notarySet { + nodes = append(nodes, nID) } to := newTotalOrdering( uint64(config.K), - uint64(float32(len(validatorSet)-1)*config.PhiRatio+1), + uint64(float32(len(notarySet)-1)*config.PhiRatio+1), config.NumChains) con := &Consensus{ - ID: types.NewValidatorID(prv.PublicKey()), + ID: types.NewNodeID(prv.PublicKey()), rbModule: rb, toModule: to, ctModule: newConsensusTimestamp(), @@ -199,7 +199,7 @@ func NewConsensus( con.baModules[chainID] = newAgreement( con.ID, con.receivers[chainID], - validators, + nodes, newGenesisLeaderSelector(con.config.GenesisCRS, con.sigToPub), con.sigToPub, blockProposer, @@ -230,10 +230,10 @@ func (con *Consensus) Run() { func (con *Consensus) runBA(chainID uint32, tick <-chan struct{}) { // TODO(jimmy-dexon): move this function inside agreement. - validatorSet := con.gov.GetNotarySet() - validators := make(types.ValidatorIDs, 0, len(validatorSet)) - for vID := range validatorSet { - validators = append(validators, vID) + notarySet := con.gov.GetNotarySet() + nodes := make(types.NodeIDs, 0, len(notarySet)) + for nID := range notarySet { + nodes = append(nodes, nID) } agreement := con.baModules[chainID] recv := con.receivers[chainID] @@ -252,13 +252,13 @@ BALoop: } select { case <-recv.restart: - // TODO(jimmy-dexon): handling change of validator set. + // TODO(jimmy-dexon): handling change of notary set. aID := types.Position{ ShardID: 0, ChainID: chainID, Height: con.rbModule.nextHeight(chainID), } - agreement.restart(validators, aID) + agreement.restart(nodes, aID) default: } err := agreement.nextState() @@ -275,8 +275,8 @@ func (con *Consensus) RunLegacy() { chainID := uint32(0) hashes := make(common.Hashes, 0, len(con.gov.GetNotarySet())) - for vID := range con.gov.GetNotarySet() { - hashes = append(hashes, vID.Hash) + for nID := range con.gov.GetNotarySet() { + hashes = append(hashes, nID.Hash) } sort.Sort(hashes) for i, hash := range hashes { @@ -498,7 +498,7 @@ func (con *Consensus) ProcessBlock(block *types.Block) (err error) { func (con *Consensus) checkPrepareBlock( b *types.Block, proposeTime time.Time) (err error) { - if (b.ProposerID == types.ValidatorID{}) { + if (b.ProposerID == types.NodeID{}) { err = &ErrMissingBlockInfo{MissingField: "ProposerID"} return } @@ -556,14 +556,14 @@ func (con *Consensus) PrepareGenesisBlock(b *types.Block, func (con *Consensus) ProcessWitnessAck(witnessAck *types.WitnessAck) (err error) { witnessAck = witnessAck.Clone() if _, exists := con.gov.GetNotarySet()[witnessAck.ProposerID]; !exists { - err = ErrProposerNotValidator + err = ErrProposerNotInNotarySet return } err = con.ccModule.processWitnessAck(witnessAck) return } -// WitnessAcks returns the latest WitnessAck received from all other validators. -func (con *Consensus) WitnessAcks() map[types.ValidatorID]*types.WitnessAck { +// WitnessAcks returns the latest WitnessAck received from all other nodes. +func (con *Consensus) WitnessAcks() map[types.NodeID]*types.WitnessAck { return con.ccModule.witnessAcks() } diff --git a/core/consensus_test.go b/core/consensus_test.go index 32f9986..c3f33fa 100644 --- a/core/consensus_test.go +++ b/core/consensus_test.go @@ -47,7 +47,7 @@ func (n *network) BroadcastWitnessAck(witnessAck *types.WitnessAck) { // SendDKGPrivateShare sends PrivateShare to a DKG participant. func (n *network) SendDKGPrivateShare( - recv types.ValidatorID, prvShare *types.DKGPrivateShare) { + recv types.NodeID, prvShare *types.DKGPrivateShare) { } // ReceiveChan returns a channel to receive messages from DEXON network. @@ -60,7 +60,7 @@ type ConsensusTestSuite struct { } func (s *ConsensusTestSuite) prepareGenesisBlock( - proposerID types.ValidatorID, + proposerID types.NodeID, chainID uint32, con *Consensus) *types.Block { @@ -76,12 +76,12 @@ func (s *ConsensusTestSuite) prepareGenesisBlock( } func (s *ConsensusTestSuite) prepareConsensus( - gov *test.Governance, vID types.ValidatorID) (*Application, *Consensus) { + gov *test.Governance, nID types.NodeID) (*Application, *Consensus) { app := test.NewApp() db, err := blockdb.NewMemBackedBlockDB() s.Require().Nil(err) - prv, exist := gov.GetPrivateKey(vID) + prv, exist := gov.GetPrivateKey(nID) s.Require().Nil(exist) con := NewConsensus(app, gov, db, &network{}, prv, eth.SigToPub) return &con.app, con @@ -96,29 +96,29 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { // | o | | <- the only block which is acked by all other blocks // |/|\|\| at the same height. // o o o o <- genesis blocks - // 0 1 2 3 <- index of validator ID + // 0 1 2 3 <- index of node ID // // This test case only works for Total Ordering with K=0. var ( minInterval = 50 * time.Millisecond gov, err = test.NewGovernance(4, time.Second) req = s.Require() - validators []types.ValidatorID + nodes []types.NodeID ) s.Require().Nil(err) - for vID := range gov.GetNotarySet() { - validators = append(validators, vID) + for nID := range gov.GetNotarySet() { + nodes = append(nodes, nID) } // Setup core.Consensus and test.App. - objs := map[types.ValidatorID]*struct { + objs := map[types.NodeID]*struct { app *Application con *Consensus }{} - for _, vID := range validators { - app, con := s.prepareConsensus(gov, vID) - objs[vID] = &struct { + for _, nID := range nodes { + app, con := s.prepareConsensus(gov, nID) + objs[nID] = &struct { app *Application con *Consensus }{app, con} @@ -131,13 +131,13 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { } } // Genesis blocks - b00 := s.prepareGenesisBlock(validators[0], 0, objs[validators[0]].con) + b00 := s.prepareGenesisBlock(nodes[0], 0, objs[nodes[0]].con) time.Sleep(minInterval) - b10 := s.prepareGenesisBlock(validators[1], 1, objs[validators[1]].con) + b10 := s.prepareGenesisBlock(nodes[1], 1, objs[nodes[1]].con) time.Sleep(minInterval) - b20 := s.prepareGenesisBlock(validators[2], 2, objs[validators[2]].con) + b20 := s.prepareGenesisBlock(nodes[2], 2, objs[nodes[2]].con) time.Sleep(minInterval) - b30 := s.prepareGenesisBlock(validators[3], 3, objs[validators[3]].con) + b30 := s.prepareGenesisBlock(nodes[3], 3, objs[nodes[3]].con) broadcast(b00) broadcast(b10) broadcast(b20) @@ -145,14 +145,14 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { // Setup b11. time.Sleep(minInterval) b11 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], Position: types.Position{ ChainID: 1, }, } b11.Hash, err = hashBlock(b11) s.Require().Nil(err) - req.Nil(objs[validators[1]].con.PrepareBlock(b11, time.Now().UTC())) + req.Nil(objs[nodes[1]].con.PrepareBlock(b11, time.Now().UTC())) req.Len(b11.Acks, 4) req.Contains(b11.Acks, b00.Hash) req.Contains(b11.Acks, b10.Hash) @@ -162,37 +162,37 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { // Setup b01. time.Sleep(minInterval) b01 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], Position: types.Position{ ChainID: 0, }, Hash: common.NewRandomHash(), } - req.Nil(objs[validators[0]].con.PrepareBlock(b01, time.Now().UTC())) + req.Nil(objs[nodes[0]].con.PrepareBlock(b01, time.Now().UTC())) req.Len(b01.Acks, 4) req.Contains(b01.Acks, b11.Hash) // Setup b21. time.Sleep(minInterval) b21 := &types.Block{ - ProposerID: validators[2], + ProposerID: nodes[2], Position: types.Position{ ChainID: 2, }, Hash: common.NewRandomHash(), } - req.Nil(objs[validators[2]].con.PrepareBlock(b21, time.Now().UTC())) + req.Nil(objs[nodes[2]].con.PrepareBlock(b21, time.Now().UTC())) req.Len(b21.Acks, 4) req.Contains(b21.Acks, b11.Hash) // Setup b31. time.Sleep(minInterval) b31 := &types.Block{ - ProposerID: validators[3], + ProposerID: nodes[3], Position: types.Position{ ChainID: 3, }, Hash: common.NewRandomHash(), } - req.Nil(objs[validators[3]].con.PrepareBlock(b31, time.Now().UTC())) + req.Nil(objs[nodes[3]].con.PrepareBlock(b31, time.Now().UTC())) req.Len(b31.Acks, 4) req.Contains(b31.Acks, b11.Hash) // Broadcast other height=1 blocks. @@ -203,13 +203,13 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { // Setup b02. time.Sleep(minInterval) b02 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], Position: types.Position{ ChainID: 0, }, Hash: common.NewRandomHash(), } - req.Nil(objs[validators[0]].con.PrepareBlock(b02, time.Now().UTC())) + req.Nil(objs[nodes[0]].con.PrepareBlock(b02, time.Now().UTC())) req.Len(b02.Acks, 3) req.Contains(b02.Acks, b01.Hash) req.Contains(b02.Acks, b21.Hash) @@ -217,13 +217,13 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { // Setup b12. time.Sleep(minInterval) b12 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], Position: types.Position{ ChainID: 1, }, Hash: common.NewRandomHash(), } - req.Nil(objs[validators[1]].con.PrepareBlock(b12, time.Now().UTC())) + req.Nil(objs[nodes[1]].con.PrepareBlock(b12, time.Now().UTC())) req.Len(b12.Acks, 4) req.Contains(b12.Acks, b01.Hash) req.Contains(b12.Acks, b11.Hash) @@ -232,13 +232,13 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { // Setup b22. time.Sleep(minInterval) b22 := &types.Block{ - ProposerID: validators[2], + ProposerID: nodes[2], Position: types.Position{ ChainID: 2, }, Hash: common.NewRandomHash(), } - req.Nil(objs[validators[2]].con.PrepareBlock(b22, time.Now().UTC())) + req.Nil(objs[nodes[2]].con.PrepareBlock(b22, time.Now().UTC())) req.Len(b22.Acks, 3) req.Contains(b22.Acks, b01.Hash) req.Contains(b22.Acks, b21.Hash) @@ -246,13 +246,13 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { // Setup b32. time.Sleep(minInterval) b32 := &types.Block{ - ProposerID: validators[3], + ProposerID: nodes[3], Position: types.Position{ ChainID: 3, }, Hash: common.NewRandomHash(), } - req.Nil(objs[validators[3]].con.PrepareBlock(b32, time.Now().UTC())) + req.Nil(objs[nodes[3]].con.PrepareBlock(b32, time.Now().UTC())) req.Len(b32.Acks, 3) req.Contains(b32.Acks, b01.Hash) req.Contains(b32.Acks, b21.Hash) @@ -324,30 +324,30 @@ func (s *ConsensusTestSuite) TestPrepareBlock() { // - Make sure Consensus.PrepareBlock would only attempt to // ack the prepared block. var ( - gov, err = test.NewGovernance(4, time.Second) - req = s.Require() - validators []types.ValidatorID + gov, err = test.NewGovernance(4, time.Second) + req = s.Require() + nodes []types.NodeID ) s.Require().Nil(err) - for vID := range gov.GetNotarySet() { - validators = append(validators, vID) + for nID := range gov.GetNotarySet() { + nodes = append(nodes, nID) } // Setup core.Consensus and test.App. - objs := map[types.ValidatorID]*struct { + objs := map[types.NodeID]*struct { app *Application con *Consensus }{} - for _, vID := range validators { - app, con := s.prepareConsensus(gov, vID) - objs[vID] = &struct { + for _, nID := range nodes { + app, con := s.prepareConsensus(gov, nID) + objs[nID] = &struct { app *Application con *Consensus }{app, con} } - b00 := s.prepareGenesisBlock(validators[0], 0, objs[validators[0]].con) - b10 := s.prepareGenesisBlock(validators[1], 1, objs[validators[1]].con) - b20 := s.prepareGenesisBlock(validators[2], 2, objs[validators[2]].con) - b30 := s.prepareGenesisBlock(validators[3], 3, objs[validators[3]].con) + b00 := s.prepareGenesisBlock(nodes[0], 0, objs[nodes[0]].con) + b10 := s.prepareGenesisBlock(nodes[1], 1, objs[nodes[1]].con) + b20 := s.prepareGenesisBlock(nodes[2], 2, objs[nodes[2]].con) + b30 := s.prepareGenesisBlock(nodes[3], 3, objs[nodes[3]].con) for _, obj := range objs { con := obj.con req.Nil(con.ProcessBlock(b00)) @@ -356,11 +356,11 @@ func (s *ConsensusTestSuite) TestPrepareBlock() { req.Nil(con.ProcessBlock(b30)) } b11 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], } // Sleep to make sure 'now' is slower than b10's timestamp. time.Sleep(100 * time.Millisecond) - req.Nil(objs[validators[1]].con.PrepareBlock(b11, time.Now().UTC())) + req.Nil(objs[nodes[1]].con.PrepareBlock(b11, time.Now().UTC())) // Make sure we would assign 'now' to the timestamp belongs to // the proposer. req.True( @@ -371,25 +371,25 @@ func (s *ConsensusTestSuite) TestPrepareBlock() { req.Nil(con.ProcessBlock(b11)) } b12 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], } - req.Nil(objs[validators[1]].con.PrepareBlock(b12, time.Now().UTC())) + req.Nil(objs[nodes[1]].con.PrepareBlock(b12, time.Now().UTC())) req.Len(b12.Acks, 1) req.Contains(b12.Acks, b11.Hash) } func (s *ConsensusTestSuite) TestPrepareGenesisBlock() { var ( - gov, err = test.NewGovernance(4, time.Second) - validators []types.ValidatorID + gov, err = test.NewGovernance(4, time.Second) + nodes []types.NodeID ) s.Require().Nil(err) - for vID := range gov.GetNotarySet() { - validators = append(validators, vID) + for nID := range gov.GetNotarySet() { + nodes = append(nodes, nID) } - _, con := s.prepareConsensus(gov, validators[0]) + _, con := s.prepareConsensus(gov, nodes[0]) block := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], } con.PrepareGenesisBlock(block, time.Now().UTC()) s.True(block.IsGenesis()) diff --git a/core/crypto.go b/core/crypto.go index 707a2b9..e68d7cc 100644 --- a/core/crypto.go +++ b/core/crypto.go @@ -103,7 +103,7 @@ func verifyVoteSignature(vote *types.Vote, sigToPub SigToPubFn) (bool, error) { if err != nil { return false, err } - if vote.ProposerID != types.NewValidatorID(pubKey) { + if vote.ProposerID != types.NewNodeID(pubKey) { return false, nil } return true, nil @@ -121,7 +121,7 @@ func verifyCRSSignature(block *types.Block, crs common.Hash, sigToPub SigToPubFn if err != nil { return false, err } - if block.ProposerID != types.NewValidatorID(pubKey) { + if block.ProposerID != types.NewNodeID(pubKey) { return false, nil } return true, nil @@ -162,7 +162,7 @@ func verifyDKGPrivateShareSignature( if err != nil { return false, err } - if prvShare.ProposerID != types.NewValidatorID(pubKey) { + if prvShare.ProposerID != types.NewNodeID(pubKey) { return false, nil } return true, nil @@ -187,7 +187,7 @@ func verifyDKGMasterPublicKeySignature( if err != nil { return false, err } - if mpk.ProposerID != types.NewValidatorID(pubKey) { + if mpk.ProposerID != types.NewNodeID(pubKey) { return false, nil } return true, nil @@ -213,7 +213,7 @@ func verifyDKGComplaintSignature( if err != nil { return false, err } - if complaint.ProposerID != types.NewValidatorID(pubKey) { + if complaint.ProposerID != types.NewNodeID(pubKey) { return false, nil } return true, nil @@ -237,7 +237,7 @@ func verifyDKGPartialSignatureSignature( if err != nil { return false, err } - if psig.ProposerID != types.NewValidatorID(pubKey) { + if psig.ProposerID != types.NewNodeID(pubKey) { return false, nil } return true, nil diff --git a/core/crypto_test.go b/core/crypto_test.go index fe6bbc1..6c807da 100644 --- a/core/crypto_test.go +++ b/core/crypto_test.go @@ -33,7 +33,7 @@ type CryptoTestSuite struct { suite.Suite } -var myVID = types.ValidatorID{Hash: common.NewRandomHash()} +var myNID = types.NodeID{Hash: common.NewRandomHash()} func (s *CryptoTestSuite) prepareBlock(prevBlock *types.Block) *types.Block { acks := common.Hashes{} @@ -181,9 +181,9 @@ func (s *CryptoTestSuite) TestVoteSignature() { prv, err := eth.NewPrivateKey() s.Require().Nil(err) pub := prv.PublicKey() - vID := types.NewValidatorID(pub) + nID := types.NewNodeID(pub) vote := &types.Vote{ - ProposerID: vID, + ProposerID: nID, Type: types.VoteAck, BlockHash: common.NewRandomHash(), Period: 1, @@ -200,9 +200,9 @@ func (s *CryptoTestSuite) TestCRSSignature() { prv, err := eth.NewPrivateKey() s.Require().Nil(err) pub := prv.PublicKey() - vID := types.NewValidatorID(pub) + nID := types.NewNodeID(pub) block := &types.Block{ - ProposerID: vID, + ProposerID: nID, } block.CRSSignature, err = prv.Sign(hashCRS(block, crs)) s.Require().Nil(err) @@ -214,9 +214,9 @@ func (s *CryptoTestSuite) TestCRSSignature() { func (s *CryptoTestSuite) TestDKGSignature() { prv, err := eth.NewPrivateKey() s.Require().Nil(err) - vID := types.NewValidatorID(prv.PublicKey()) + nID := types.NewNodeID(prv.PublicKey()) prvShare := &types.DKGPrivateShare{ - ProposerID: vID, + ProposerID: nID, Round: 5, PrivateShare: *dkg.NewPrivateKey(), } @@ -229,7 +229,7 @@ func (s *CryptoTestSuite) TestDKGSignature() { id := dkg.NewID([]byte{13}) _, pkShare := dkg.NewPrivateKeyShares(1) mpk := &types.DKGMasterPublicKey{ - ProposerID: vID, + ProposerID: nID, Round: 5, DKGID: id, PublicKeyShares: *pkShare, @@ -241,7 +241,7 @@ func (s *CryptoTestSuite) TestDKGSignature() { s.False(verifyDKGMasterPublicKeySignature(mpk, eth.SigToPub)) complaint := &types.DKGComplaint{ - ProposerID: vID, + ProposerID: nID, Round: 5, PrivateShare: *prvShare, } @@ -252,7 +252,7 @@ func (s *CryptoTestSuite) TestDKGSignature() { s.False(verifyDKGComplaintSignature(complaint, eth.SigToPub)) sig := &types.DKGPartialSignature{ - ProposerID: vID, + ProposerID: nID, Round: 5, PartialSignature: dkg.PartialSignature{}, } diff --git a/core/dkg-tsig-protocol.go b/core/dkg-tsig-protocol.go index 2db18f1..e1a0635 100644 --- a/core/dkg-tsig-protocol.go +++ b/core/dkg-tsig-protocol.go @@ -54,20 +54,20 @@ type dkgComplaintReceiver interface { ProposeDKGMasterPublicKey(mpk *types.DKGMasterPublicKey) // ProposeDKGPrivateShare propose a DKGPrivateShare. - ProposeDKGPrivateShare(to types.ValidatorID, prv *types.DKGPrivateShare) + ProposeDKGPrivateShare(to types.NodeID, prv *types.DKGPrivateShare) } type dkgProtocol struct { - ID types.ValidatorID + ID types.NodeID recv dkgComplaintReceiver round uint64 threshold int sigToPub SigToPubFn - idMap map[types.ValidatorID]dkg.ID - mpkMap map[types.ValidatorID]*dkg.PublicKeyShares + idMap map[types.NodeID]dkg.ID + mpkMap map[types.NodeID]*dkg.PublicKeyShares masterPrivateShare *dkg.PrivateKeyShares prvShares *dkg.PrivateKeyShares - prvSharesReceived map[types.ValidatorID]struct{} + prvSharesReceived map[types.NodeID]struct{} } type dkgShareSecret struct { @@ -77,8 +77,8 @@ type dkgShareSecret struct { type dkgGroupPublicKey struct { round uint64 qualifyIDs dkg.IDs - idMap map[types.ValidatorID]dkg.ID - publicKeys map[types.ValidatorID]*dkg.PublicKey + idMap map[types.NodeID]dkg.ID + publicKeys map[types.NodeID]*dkg.PublicKey groupPublicKey *dkg.PublicKey threshold int sigToPub SigToPubFn @@ -90,12 +90,12 @@ type tsigProtocol struct { threshold int } -func newDKGID(ID types.ValidatorID) dkg.ID { +func newDKGID(ID types.NodeID) dkg.ID { return dkg.NewID(ID.Hash[:]) } func newDKGProtocol( - ID types.ValidatorID, + ID types.NodeID, recv dkgComplaintReceiver, round uint64, threshold int, @@ -116,24 +116,24 @@ func newDKGProtocol( round: round, threshold: threshold, sigToPub: sigToPub, - idMap: make(map[types.ValidatorID]dkg.ID), - mpkMap: make(map[types.ValidatorID]*dkg.PublicKeyShares), + idMap: make(map[types.NodeID]dkg.ID), + mpkMap: make(map[types.NodeID]*dkg.PublicKeyShares), masterPrivateShare: prvShare, prvShares: dkg.NewEmptyPrivateKeyShares(), - prvSharesReceived: make(map[types.ValidatorID]struct{}), + prvSharesReceived: make(map[types.NodeID]struct{}), } } func (d *dkgProtocol) processMasterPublicKeys( mpks []*types.DKGMasterPublicKey) error { - d.idMap = make(map[types.ValidatorID]dkg.ID, len(mpks)) - d.mpkMap = make(map[types.ValidatorID]*dkg.PublicKeyShares, len(mpks)) - d.prvSharesReceived = make(map[types.ValidatorID]struct{}, len(mpks)) + d.idMap = make(map[types.NodeID]dkg.ID, len(mpks)) + d.mpkMap = make(map[types.NodeID]*dkg.PublicKeyShares, len(mpks)) + d.prvSharesReceived = make(map[types.NodeID]struct{}, len(mpks)) ids := make(dkg.IDs, len(mpks)) for i := range mpks { - vID := mpks[i].ProposerID - d.idMap[vID] = mpks[i].DKGID - d.mpkMap[vID] = &mpks[i].PublicKeyShares + nID := mpks[i].ProposerID + d.idMap[nID] = mpks[i].DKGID + d.mpkMap[nID] = &mpks[i].PublicKeyShares ids[i] = mpks[i].DKGID } d.masterPrivateShare.SetParticipants(ids) @@ -152,15 +152,15 @@ func (d *dkgProtocol) processMasterPublicKeys( } func (d *dkgProtocol) proposeNackComplaints() { - for vID := range d.mpkMap { - if _, exist := d.prvSharesReceived[vID]; exist { + for nID := range d.mpkMap { + if _, exist := d.prvSharesReceived[nID]; exist { continue } d.recv.ProposeDKGComplaint(&types.DKGComplaint{ ProposerID: d.ID, Round: d.round, PrivateShare: types.DKGPrivateShare{ - ProposerID: vID, + ProposerID: nID, Round: d.round, }, }) @@ -187,7 +187,7 @@ func (d *dkgProtocol) processPrivateShare( return nil } self, exist := d.idMap[d.ID] - // This validator is not a DKG participant, ignore the private share. + // This node is not a DKG participant, ignore the private share. if !exist { return nil } @@ -242,19 +242,19 @@ func newDKGGroupPublicKey( threshold int, sigToPub SigToPubFn) ( *dkgGroupPublicKey, error) { // Calculate qualify members. - complaintsByID := map[types.ValidatorID]int{} + complaintsByID := map[types.NodeID]int{} for _, complaint := range complaints { complaintsByID[complaint.PrivateShare.ProposerID]++ } - disqualifyIDs := map[types.ValidatorID]struct{}{} - for vID, num := range complaintsByID { + disqualifyIDs := map[types.NodeID]struct{}{} + for nID, num := range complaintsByID { if num > threshold { - disqualifyIDs[vID] = struct{}{} + disqualifyIDs[nID] = struct{}{} } } qualifyIDs := make(dkg.IDs, 0, len(mpks)-len(disqualifyIDs)) mpkMap := make(map[dkg.ID]*types.DKGMasterPublicKey, cap(qualifyIDs)) - idMap := make(map[types.ValidatorID]dkg.ID) + idMap := make(map[types.NodeID]dkg.ID) for _, mpk := range mpks { if _, exist := disqualifyIDs[mpk.ProposerID]; exist { continue @@ -264,7 +264,7 @@ func newDKGGroupPublicKey( qualifyIDs = append(qualifyIDs, mpk.DKGID) } // Recover qualify members' public key. - pubKeys := make(map[types.ValidatorID]*dkg.PublicKey, len(qualifyIDs)) + pubKeys := make(map[types.NodeID]*dkg.PublicKey, len(qualifyIDs)) for _, recvID := range qualifyIDs { pubShares := dkg.NewEmptyPublicKeyShares() for _, id := range qualifyIDs { diff --git a/core/dkg-tsig-protocol_test.go b/core/dkg-tsig-protocol_test.go index 7da8866..fc64788 100644 --- a/core/dkg-tsig-protocol_test.go +++ b/core/dkg-tsig-protocol_test.go @@ -33,18 +33,18 @@ import ( type DKGTSIGProtocolTestSuite struct { suite.Suite - vIDs types.ValidatorIDs - dkgIDs map[types.ValidatorID]dkg.ID - prvKeys map[types.ValidatorID]crypto.PrivateKey + nIDs types.NodeIDs + dkgIDs map[types.NodeID]dkg.ID + prvKeys map[types.NodeID]crypto.PrivateKey } type testDKGReceiver struct { s *DKGTSIGProtocolTestSuite prvKey crypto.PrivateKey - complaints map[types.ValidatorID]*types.DKGComplaint + complaints map[types.NodeID]*types.DKGComplaint mpk *types.DKGMasterPublicKey - prvShare map[types.ValidatorID]*types.DKGPrivateShare + prvShare map[types.NodeID]*types.DKGPrivateShare } func newTestDKGReceiver( @@ -52,8 +52,8 @@ func newTestDKGReceiver( return &testDKGReceiver{ s: s, prvKey: prvKey, - complaints: make(map[types.ValidatorID]*types.DKGComplaint), - prvShare: make(map[types.ValidatorID]*types.DKGPrivateShare), + complaints: make(map[types.NodeID]*types.DKGComplaint), + prvShare: make(map[types.NodeID]*types.DKGPrivateShare), } } @@ -72,7 +72,7 @@ func (r *testDKGReceiver) ProposeDKGMasterPublicKey( r.mpk = mpk } func (r *testDKGReceiver) ProposeDKGPrivateShare( - to types.ValidatorID, prv *types.DKGPrivateShare) { + to types.NodeID, prv *types.DKGPrivateShare) { var err error prv.Signature, err = r.prvKey.Sign(hashDKGPrivateShare(prv)) r.s.Require().NoError(err) @@ -80,38 +80,38 @@ func (r *testDKGReceiver) ProposeDKGPrivateShare( } func (s *DKGTSIGProtocolTestSuite) setupDKGParticipants(n int) { - s.vIDs = make(types.ValidatorIDs, 0, n) - s.prvKeys = make(map[types.ValidatorID]crypto.PrivateKey, n) - s.dkgIDs = make(map[types.ValidatorID]dkg.ID) + s.nIDs = make(types.NodeIDs, 0, n) + s.prvKeys = make(map[types.NodeID]crypto.PrivateKey, n) + s.dkgIDs = make(map[types.NodeID]dkg.ID) ids := make(dkg.IDs, 0, n) for i := 0; i < n; i++ { prvKey, err := eth.NewPrivateKey() s.Require().NoError(err) - vID := types.NewValidatorID(prvKey.PublicKey()) - s.vIDs = append(s.vIDs, vID) - s.prvKeys[vID] = prvKey - id := dkg.NewID(vID.Hash[:]) + nID := types.NewNodeID(prvKey.PublicKey()) + s.nIDs = append(s.nIDs, nID) + s.prvKeys[nID] = prvKey + id := dkg.NewID(nID.Hash[:]) ids = append(ids, id) - s.dkgIDs[vID] = id + s.dkgIDs[nID] = id } } func (s *DKGTSIGProtocolTestSuite) newProtocols(k, n int, round uint64) ( - map[types.ValidatorID]*testDKGReceiver, map[types.ValidatorID]*dkgProtocol) { + map[types.NodeID]*testDKGReceiver, map[types.NodeID]*dkgProtocol) { s.setupDKGParticipants(n) - receivers := make(map[types.ValidatorID]*testDKGReceiver, n) - protocols := make(map[types.ValidatorID]*dkgProtocol, n) - for _, vID := range s.vIDs { - receivers[vID] = newTestDKGReceiver(s, s.prvKeys[vID]) - protocols[vID] = newDKGProtocol( - vID, - receivers[vID], + receivers := make(map[types.NodeID]*testDKGReceiver, n) + protocols := make(map[types.NodeID]*dkgProtocol, n) + for _, nID := range s.nIDs { + receivers[nID] = newTestDKGReceiver(s, s.prvKeys[nID]) + protocols[nID] = newDKGProtocol( + nID, + receivers[nID], round, k, eth.SigToPub, ) - s.Require().NotNil(receivers[vID].mpk) + s.Require().NotNil(receivers[nID].mpk) } return receivers, protocols } @@ -140,8 +140,8 @@ func (s *DKGTSIGProtocolTestSuite) TestDKGTSIGProtocol() { for _, receiver := range receivers { s.Require().Len(receiver.prvShare, n) - for vID, prvShare := range receiver.prvShare { - s.Require().NoError(protocols[vID].processPrivateShare(prvShare)) + for nID, prvShare := range receiver.prvShare { + s.Require().NoError(protocols[nID].processPrivateShare(prvShare)) } } @@ -166,26 +166,26 @@ func (s *DKGTSIGProtocolTestSuite) TestDKGTSIGProtocol() { } shareSecrets := make( - map[types.ValidatorID]*dkgShareSecret, len(qualifyIDs)) + map[types.NodeID]*dkgShareSecret, len(qualifyIDs)) - for vID, protocol := range protocols { - _, exist := qualifyIDs[s.dkgIDs[vID]] + for nID, protocol := range protocols { + _, exist := qualifyIDs[s.dkgIDs[nID]] s.Require().True(exist) var err error - shareSecrets[vID], err = protocol.recoverShareSecret(gpk.qualifyIDs) + shareSecrets[nID], err = protocol.recoverShareSecret(gpk.qualifyIDs) s.Require().NoError(err) } tsig := newTSigProtocol(gpk) msgHash := crypto.Keccak256Hash([]byte("🏖🍹")) - for vID, shareSecret := range shareSecrets { + for nID, shareSecret := range shareSecrets { psig := &types.DKGPartialSignature{ - ProposerID: vID, + ProposerID: nID, Round: round, PartialSignature: shareSecret.sign(msgHash), } var err error - psig.Signature, err = s.prvKeys[vID].Sign(hashDKGPartialSignature(psig)) + psig.Signature, err = s.prvKeys[nID].Sign(hashDKGPartialSignature(psig)) s.Require().NoError(err) s.Require().NoError(tsig.processPartialSignature(msgHash, psig)) if len(tsig.sigs) > k { @@ -207,7 +207,7 @@ func (s *DKGTSIGProtocolTestSuite) TestNackComplaint() { receivers, protocols := s.newProtocols(k, n, round) - byzantineID := s.vIDs[0] + byzantineID := s.nIDs[0] for _, receiver := range receivers { gov.AddDKGMasterPublicKey(receiver.mpk) @@ -223,8 +223,8 @@ func (s *DKGTSIGProtocolTestSuite) TestNackComplaint() { if senderID == byzantineID { continue } - for vID, prvShare := range receiver.prvShare { - s.Require().NoError(protocols[vID].processPrivateShare(prvShare)) + for nID, prvShare := range receiver.prvShare { + s.Require().NoError(protocols[nID].processPrivateShare(prvShare)) } } @@ -250,8 +250,8 @@ func (s *DKGTSIGProtocolTestSuite) TestComplaint() { receivers, protocols := s.newProtocols(k, n, round) - byzantineID := s.vIDs[0] - targetID := s.vIDs[1] + byzantineID := s.nIDs[0] + targetID := s.nIDs[1] receiver := receivers[targetID] protocol := protocols[targetID] @@ -266,7 +266,7 @@ func (s *DKGTSIGProtocolTestSuite) TestComplaint() { // These messages are not valid. err = protocol.processPrivateShare(&types.DKGPrivateShare{ - ProposerID: types.ValidatorID{Hash: common.NewRandomHash()}, + ProposerID: types.NodeID{Hash: common.NewRandomHash()}, Round: round, }) s.Error(ErrNotDKGParticipant, err) @@ -296,7 +296,7 @@ func (s *DKGTSIGProtocolTestSuite) TestQualifyIDs() { receivers, _ := s.newProtocols(k, n, round) - byzantineID := s.vIDs[0] + byzantineID := s.nIDs[0] for _, receiver := range receivers { gov.AddDKGMasterPublicKey(receiver.mpk) @@ -304,9 +304,9 @@ func (s *DKGTSIGProtocolTestSuite) TestQualifyIDs() { complaints := make([]*types.DKGComplaint, k+1) for i := range complaints { - vID := s.vIDs[i] + nID := s.nIDs[i] complaints[i] = &types.DKGComplaint{ - ProposerID: vID, + ProposerID: nID, Round: round, PrivateShare: types.DKGPrivateShare{ ProposerID: byzantineID, @@ -344,7 +344,7 @@ func (s *DKGTSIGProtocolTestSuite) TestPartialSignature() { receivers, protocols := s.newProtocols(k, n, round) - byzantineID := s.vIDs[0] + byzantineID := s.nIDs[0] for _, receiver := range receivers { gov.AddDKGMasterPublicKey(receiver.mpk) @@ -360,8 +360,8 @@ func (s *DKGTSIGProtocolTestSuite) TestPartialSignature() { if senderID == byzantineID { continue } - for vID, prvShare := range receiver.prvShare { - s.Require().NoError(protocols[vID].processPrivateShare(prvShare)) + for nID, prvShare := range receiver.prvShare { + s.Require().NoError(protocols[nID].processPrivateShare(prvShare)) } } @@ -389,38 +389,38 @@ func (s *DKGTSIGProtocolTestSuite) TestPartialSignature() { } shareSecrets := make( - map[types.ValidatorID]*dkgShareSecret, len(qualifyIDs)) + map[types.NodeID]*dkgShareSecret, len(qualifyIDs)) - for vID, protocol := range protocols { - _, exist := qualifyIDs[s.dkgIDs[vID]] - if vID == byzantineID { + for nID, protocol := range protocols { + _, exist := qualifyIDs[s.dkgIDs[nID]] + if nID == byzantineID { exist = !exist } s.Require().True(exist) var err error - shareSecrets[vID], err = protocol.recoverShareSecret(gpk.qualifyIDs) + shareSecrets[nID], err = protocol.recoverShareSecret(gpk.qualifyIDs) s.Require().NoError(err) } tsig := newTSigProtocol(gpk) msgHash := crypto.Keccak256Hash([]byte("🏖🍹")) - byzantineID2 := s.vIDs[1] - for vID, shareSecret := range shareSecrets { + byzantineID2 := s.nIDs[1] + for nID, shareSecret := range shareSecrets { psig := &types.DKGPartialSignature{ - ProposerID: vID, + ProposerID: nID, Round: round, PartialSignature: shareSecret.sign(msgHash), } - if vID == byzantineID2 { + if nID == byzantineID2 { psig.PartialSignature[0]++ } var err error - psig.Signature, err = s.prvKeys[vID].Sign(hashDKGPartialSignature(psig)) + psig.Signature, err = s.prvKeys[nID].Sign(hashDKGPartialSignature(psig)) s.Require().NoError(err) err = tsig.processPartialSignature(msgHash, psig) - if vID == byzantineID { + if nID == byzantineID { s.Require().Error(ErrNotQualifyDKGParticipant, err) - } else if vID == byzantineID2 { + } else if nID == byzantineID2 { s.Require().Error(ErrIncorrectPartialSignature, err) } else { s.Require().NoError(err) diff --git a/core/interfaces.go b/core/interfaces.go index 3fae590..4f67e1e 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -63,7 +63,7 @@ type Network interface { BroadcastWitnessAck(witnessAck *types.WitnessAck) // SendDKGPrivateShare sends PrivateShare to a DKG participant. - SendDKGPrivateShare(recv types.ValidatorID, prvShare *types.DKGPrivateShare) + SendDKGPrivateShare(recv types.NodeID, prvShare *types.DKGPrivateShare) // ReceiveChan returns a channel to receive messages from DEXON network. ReceiveChan() <-chan interface{} @@ -76,8 +76,8 @@ type Governance interface { // GetConfiguration returns the configuration at a given block height. GetConfiguration(blockHeight uint64) *types.Config - // Get the current validator set. - GetNotarySet() map[types.ValidatorID]struct{} + // Get the current notary set. + GetNotarySet() map[types.NodeID]struct{} //// DKG-related methods. diff --git a/core/leader-selector_test.go b/core/leader-selector_test.go index e18dd49..7eb9b3c 100644 --- a/core/leader-selector_test.go +++ b/core/leader-selector_test.go @@ -85,7 +85,7 @@ func (s *LeaderSelectorTestSuite) TestLeaderBlockHash() { prv, err := eth.NewPrivateKey() s.Require().Nil(err) block := &types.Block{ - ProposerID: types.NewValidatorID(prv.PublicKey()), + ProposerID: types.NewNodeID(prv.PublicKey()), Hash: common.NewRandomHash(), } s.Require().Nil(leader.prepareBlock(block, prv)) @@ -110,7 +110,7 @@ func (s *LeaderSelectorTestSuite) TestPrepareBlock() { prv, err := eth.NewPrivateKey() s.Require().Nil(err) block := &types.Block{ - ProposerID: types.NewValidatorID(prv.PublicKey()), + ProposerID: types.NewNodeID(prv.PublicKey()), } s.Require().Nil(leader.prepareBlock(block, prv)) s.Nil(leader.processBlock(block)) diff --git a/core/negative-ack.go b/core/negative-ack.go index 13a4832..8957152 100644 --- a/core/negative-ack.go +++ b/core/negative-ack.go @@ -25,67 +25,67 @@ import ( type negativeAck struct { // owner is the ID of proposer itself, this is used when deciding - // a validator to be restricted or not. - owner types.ValidatorID + // a node to be restricted or not. + owner types.NodeID - numOfValidators int + numOfNodes int // timeDelay and timeExpire are for nack timeout. timeDelay time.Duration timeExpire time.Duration - // restricteds stores validators which has been restricted and the time it's + // restricteds stores nodes which has been restricted and the time it's // restricted. - restricteds map[types.ValidatorID]time.Time + restricteds map[types.NodeID]time.Time - // lastVotes and lockedVotes store the votes for nack. lastVotes[vid1][vid2] - // and lockedVotes[vid1][vid2] both mean that vid2 votes vid1. The difference - // is lockedVotes works only when vid1 is restricted, so that the votes are + // lastVotes and lockedVotes store the votes for nack. lastVotes[nid1][nid2] + // and lockedVotes[nid1][nid2] both mean that nid2 votes nid1. The difference + // is lockedVotes works only when nid1 is restricted, so that the votes are // needed to be locked. - lastVotes map[types.ValidatorID]map[types.ValidatorID]struct{} - lockedVotes map[types.ValidatorID]map[types.ValidatorID]struct{} + lastVotes map[types.NodeID]map[types.NodeID]struct{} + lockedVotes map[types.NodeID]map[types.NodeID]struct{} - // timeDiffs is the cache for last time stamps. timeDiffs[vid1][vid2] means - // the last updated timestamps vid1 sees vid2. - timeDiffs map[types.ValidatorID]map[types.ValidatorID]map[types.ValidatorID]time.Time + // timeDiffs is the cache for last time stamps. timeDiffs[nid1][nid2] means + // the last updated timestamps nid1 sees nid2. + timeDiffs map[types.NodeID]map[types.NodeID]map[types.NodeID]time.Time } // newNegativeAck creates a new negaticeAck instance. -func newNegativeAck(vid types.ValidatorID) *negativeAck { +func newNegativeAck(nid types.NodeID) *negativeAck { n := &negativeAck{ - owner: vid, - numOfValidators: 0, - restricteds: make(map[types.ValidatorID]time.Time), - lastVotes: make(map[types.ValidatorID]map[types.ValidatorID]struct{}), - lockedVotes: make(map[types.ValidatorID]map[types.ValidatorID]struct{}), - timeDiffs: make(map[types.ValidatorID]map[types.ValidatorID]map[types.ValidatorID]time.Time), + owner: nid, + numOfNodes: 0, + restricteds: make(map[types.NodeID]time.Time), + lastVotes: make(map[types.NodeID]map[types.NodeID]struct{}), + lockedVotes: make(map[types.NodeID]map[types.NodeID]struct{}), + timeDiffs: make(map[types.NodeID]map[types.NodeID]map[types.NodeID]time.Time), } - n.addValidator(vid) + n.addNode(nid) return n } -// processNewVote is called when a new "vote" occurs, that is, a validator -// sees that other 2f + 1 validators think a validator is slow. "vid" is the -// validator which propesed the block which the timestamps votes and "h" is -// the validator been voted to be nacked. +// processNewVote is called when a new "vote" occurs, that is, a node +// sees that other 2f + 1 nodes think a node is slow. "nid" is the +// node which propesed the block which the timestamps votes and "h" is +// the node been voted to be nacked. func (n *negativeAck) processNewVote( - vid types.ValidatorID, - h types.ValidatorID, -) []types.ValidatorID { + nid types.NodeID, + h types.NodeID, +) []types.NodeID { - nackeds := []types.ValidatorID{} + nackeds := []types.NodeID{} if _, exist := n.restricteds[h]; exist { - n.lockedVotes[h][vid] = struct{}{} - if len(n.lockedVotes[h]) > 2*(n.numOfValidators-1)/3 { + n.lockedVotes[h][nid] = struct{}{} + if len(n.lockedVotes[h]) > 2*(n.numOfNodes-1)/3 { nackeds = append(nackeds, h) delete(n.restricteds, h) } } else { - if n.owner == vid { + if n.owner == nid { n.restrict(h) } else { - n.lastVotes[h][vid] = struct{}{} - if len(n.lastVotes[h]) > (n.numOfValidators-1)/3 { + n.lastVotes[h][nid] = struct{}{} + if len(n.lastVotes[h]) > (n.numOfNodes-1)/3 { n.restrict(h) } } @@ -94,44 +94,44 @@ func (n *negativeAck) processNewVote( } // processTimestamps process new timestamps of a block which is proposed by -// validator vid, and returns the validators being nacked. +// node nid, and returns the nodes being nacked. func (n *negativeAck) processTimestamps( - vid types.ValidatorID, - ts map[types.ValidatorID]time.Time, -) []types.ValidatorID { + nid types.NodeID, + ts map[types.NodeID]time.Time, +) []types.NodeID { n.checkRestrictExpire() - nackeds := []types.ValidatorID{} + nackeds := []types.NodeID{} for h := range n.timeDiffs { - if n.timeDiffs[vid][h][h].Equal(ts[h]) { + if n.timeDiffs[nid][h][h].Equal(ts[h]) { votes := 0 for hh := range n.timeDiffs { - if ts[hh].Sub(n.timeDiffs[vid][h][hh]) >= n.timeDelay { + if ts[hh].Sub(n.timeDiffs[nid][h][hh]) >= n.timeDelay { votes++ } } - if votes > 2*((n.numOfValidators-1)/3) { - n.lastVotes[h][vid] = struct{}{} - nack := n.processNewVote(vid, h) + if votes > 2*((n.numOfNodes-1)/3) { + n.lastVotes[h][nid] = struct{}{} + nack := n.processNewVote(nid, h) for _, i := range nack { nackeds = append(nackeds, i) } } else { - delete(n.lastVotes[h], vid) + delete(n.lastVotes[h], nid) } } else { for hh := range n.timeDiffs { - n.timeDiffs[vid][h][hh] = ts[hh] + n.timeDiffs[nid][h][hh] = ts[hh] } - delete(n.lastVotes[h], vid) + delete(n.lastVotes[h], nid) } } return nackeds } func (n *negativeAck) checkRestrictExpire() { - expired := []types.ValidatorID{} + expired := []types.NodeID{} now := time.Now() for h, t := range n.restricteds { if now.Sub(t) >= n.timeExpire { @@ -143,19 +143,19 @@ func (n *negativeAck) checkRestrictExpire() { } } -func (n *negativeAck) restrict(vid types.ValidatorID) { - if _, exist := n.restricteds[vid]; !exist { - n.restricteds[vid] = time.Now().UTC() - n.lockedVotes[vid] = map[types.ValidatorID]struct{}{} - for h := range n.lastVotes[vid] { - n.lockedVotes[vid][h] = struct{}{} +func (n *negativeAck) restrict(nid types.NodeID) { + if _, exist := n.restricteds[nid]; !exist { + n.restricteds[nid] = time.Now().UTC() + n.lockedVotes[nid] = map[types.NodeID]struct{}{} + for h := range n.lastVotes[nid] { + n.lockedVotes[nid][h] = struct{}{} } } } -func (n *negativeAck) getRestrictedValidators() map[types.ValidatorID]struct{} { +func (n *negativeAck) getRestrictedNodes() map[types.NodeID]struct{} { n.checkRestrictExpire() - ret := map[types.ValidatorID]struct{}{} + ret := map[types.NodeID]struct{}{} for h := range n.restricteds { ret[h] = struct{}{} } @@ -170,42 +170,42 @@ func (n *negativeAck) setTimeExpire(t time.Duration) { n.timeExpire = t } -func (n *negativeAck) addValidator(vid types.ValidatorID) { - n.numOfValidators++ - n.lastVotes[vid] = make(map[types.ValidatorID]struct{}) - n.lockedVotes[vid] = make(map[types.ValidatorID]struct{}) +func (n *negativeAck) addNode(nid types.NodeID) { + n.numOfNodes++ + n.lastVotes[nid] = make(map[types.NodeID]struct{}) + n.lockedVotes[nid] = make(map[types.NodeID]struct{}) - newTimeDiff := make(map[types.ValidatorID]map[types.ValidatorID]time.Time) + newTimeDiff := make(map[types.NodeID]map[types.NodeID]time.Time) for h := range n.timeDiffs { - newTimeDiff2 := make(map[types.ValidatorID]time.Time) + newTimeDiff2 := make(map[types.NodeID]time.Time) for hh := range n.timeDiffs { newTimeDiff2[hh] = time.Time{} } newTimeDiff[h] = newTimeDiff2 } - n.timeDiffs[vid] = newTimeDiff + n.timeDiffs[nid] = newTimeDiff for h := range n.timeDiffs { - n.timeDiffs[h][vid] = make(map[types.ValidatorID]time.Time) + n.timeDiffs[h][nid] = make(map[types.NodeID]time.Time) } } -func (n *negativeAck) deleteValidator(vid types.ValidatorID) { - n.numOfValidators-- +func (n *negativeAck) deleteNode(nid types.NodeID) { + n.numOfNodes-- - delete(n.timeDiffs, vid) + delete(n.timeDiffs, nid) for h := range n.lastVotes { - delete(n.lastVotes[h], vid) + delete(n.lastVotes[h], nid) } - delete(n.lastVotes, vid) - delete(n.lockedVotes, vid) + delete(n.lastVotes, nid) + delete(n.lockedVotes, nid) for h := range n.timeDiffs { - delete(n.timeDiffs[h], vid) + delete(n.timeDiffs[h], nid) for hh := range n.timeDiffs[h] { - delete(n.timeDiffs[h][hh], vid) + delete(n.timeDiffs[h][hh], nid) } } - delete(n.restricteds, vid) + delete(n.restricteds, nid) } diff --git a/core/negative-ack_test.go b/core/negative-ack_test.go index 990871e..22280da 100644 --- a/core/negative-ack_test.go +++ b/core/negative-ack_test.go @@ -46,93 +46,93 @@ func (s *NegativeAckTest) SetupTest() { } func (s *NegativeAckTest) checkLastVotes( - vids []types.ValidatorID, - vs map[types.ValidatorID]map[types.ValidatorID]struct{}, + nids []types.NodeID, + vs map[types.NodeID]map[types.NodeID]struct{}, a [][]bool, ) { - for i := 0; i < len(vids); i++ { - for j := 0; j < len(vids); j++ { - _, exist := vs[vids[i]][vids[j]] + for i := 0; i < len(nids); i++ { + for j := 0; j < len(nids); j++ { + _, exist := vs[nids[i]][nids[j]] s.Require().Equal(a[i][j], exist) } } } func (s *NegativeAckTest) checkTimeDiff( - vids []types.ValidatorID, - ts map[types.ValidatorID]map[types.ValidatorID]time.Time, + nids []types.NodeID, + ts map[types.NodeID]map[types.NodeID]time.Time, a [][]int, ) { - for i := 0; i < len(vids); i++ { - for j := 0; j < len(vids); j++ { + for i := 0; i < len(nids); i++ { + for j := 0; j < len(nids); j++ { s.Require().Equal( time.Duration(a[i][j])*timeDelay, - ts[vids[i]][vids[j]].Sub(baseTime), + ts[nids[i]][nids[j]].Sub(baseTime), ) } } } -func genTimestamp(vids []types.ValidatorID, a []int) map[types.ValidatorID]time.Time { - ts := map[types.ValidatorID]time.Time{} - for i := 0; i < len(vids); i++ { - ts[vids[i]] = baseTime.Add(time.Duration(a[i]) * timeDelay) +func genTimestamp(nids []types.NodeID, a []int) map[types.NodeID]time.Time { + ts := map[types.NodeID]time.Time{} + for i := 0; i < len(nids); i++ { + ts[nids[i]] = baseTime.Add(time.Duration(a[i]) * timeDelay) } return ts } -func genTestNegativeAck(num int) (*negativeAck, []types.ValidatorID) { - vids := test.GenerateRandomValidatorIDs(num) - n := newNegativeAck(vids[0]) +func genTestNegativeAck(num int) (*negativeAck, []types.NodeID) { + nids := test.GenerateRandomNodeIDs(num) + n := newNegativeAck(nids[0]) for i := 1; i < num; i++ { - n.addValidator(vids[i]) + n.addNode(nids[i]) } - return n, vids + return n, nids } func (s *NegativeAckTest) TestProcessTimestamps() { - n, vids := genTestNegativeAck(4) + n, nids := genTestNegativeAck(4) n.setTimeDelay(timeDelay) n.setTimeExpire(timeExpire) - n.processTimestamps(vids[0], genTimestamp(vids, []int{1, 1, 1, 0})) - s.checkTimeDiff(vids, n.timeDiffs[vids[0]], [][]int{ + n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 0})) + s.checkTimeDiff(nids, n.timeDiffs[nids[0]], [][]int{ {1, 1, 1, 0}, {1, 1, 1, 0}, {1, 1, 1, 0}, {1, 1, 1, 0}, }) - s.checkLastVotes(vids, n.lastVotes, [][]bool{ + s.checkLastVotes(nids, n.lastVotes, [][]bool{ {false, false, false, false}, {false, false, false, false}, {false, false, false, false}, {false, false, false, false}, }) - n.processTimestamps(vids[0], genTimestamp(vids, []int{3, 1, 2, 1})) - s.checkTimeDiff(vids, n.timeDiffs[vids[0]], [][]int{ + n.processTimestamps(nids[0], genTimestamp(nids, []int{3, 1, 2, 1})) + s.checkTimeDiff(nids, n.timeDiffs[nids[0]], [][]int{ {3, 1, 2, 1}, {1, 1, 1, 0}, {3, 1, 2, 1}, {3, 1, 2, 1}, }) - s.checkLastVotes(vids, n.lastVotes, [][]bool{ + s.checkLastVotes(nids, n.lastVotes, [][]bool{ {false, false, false, false}, {true, false, false, false}, {false, false, false, false}, {false, false, false, false}, }) - n.processTimestamps(vids[0], genTimestamp(vids, []int{5, 1, 2, 2})) - s.checkTimeDiff(vids, n.timeDiffs[vids[0]], [][]int{ + n.processTimestamps(nids[0], genTimestamp(nids, []int{5, 1, 2, 2})) + s.checkTimeDiff(nids, n.timeDiffs[nids[0]], [][]int{ {5, 1, 2, 2}, {1, 1, 1, 0}, {3, 1, 2, 1}, {5, 1, 2, 2}, }) - s.checkLastVotes(vids, n.lastVotes, [][]bool{ + s.checkLastVotes(nids, n.lastVotes, [][]bool{ {false, false, false, false}, {true, false, false, false}, {false, false, false, false}, @@ -142,83 +142,83 @@ func (s *NegativeAckTest) TestProcessTimestamps() { func (s *NegativeAckTest) TestRestrictBySelf() { var exist bool - n, vids := genTestNegativeAck(4) + n, nids := genTestNegativeAck(4) n.setTimeDelay(timeDelay) n.setTimeExpire(timeExpire) - n.processTimestamps(vids[0], genTimestamp(vids, []int{1, 1, 1, 0})) - _, exist = n.getRestrictedValidators()[vids[1]] + n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 0})) + _, exist = n.getRestrictedNodes()[nids[1]] s.Require().False(exist) - n.processTimestamps(vids[0], genTimestamp(vids, []int{3, 1, 2, 1})) - _, exist = n.getRestrictedValidators()[vids[1]] + n.processTimestamps(nids[0], genTimestamp(nids, []int{3, 1, 2, 1})) + _, exist = n.getRestrictedNodes()[nids[1]] s.Require().True(exist) } func (s *NegativeAckTest) TestRestrictByVoting() { - var nackeds []types.ValidatorID + var nackeds []types.NodeID var exist bool - n, vids := genTestNegativeAck(4) + n, nids := genTestNegativeAck(4) n.setTimeDelay(timeDelay) n.setTimeExpire(timeExpire) - n.processTimestamps(vids[0], genTimestamp(vids, []int{1, 1, 1, 1})) - n.processTimestamps(vids[0], genTimestamp(vids, []int{2, 2, 2, 2})) + n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[0], genTimestamp(nids, []int{2, 2, 2, 2})) - n.processTimestamps(vids[1], genTimestamp(vids, []int{1, 1, 1, 1})) - n.processTimestamps(vids[2], genTimestamp(vids, []int{1, 1, 1, 1})) - n.processTimestamps(vids[3], genTimestamp(vids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[3], genTimestamp(nids, []int{1, 1, 1, 1})) - nackeds = n.processTimestamps(vids[1], genTimestamp(vids, []int{1, 3, 3, 3})) - _, exist = n.getRestrictedValidators()[vids[0]] + nackeds = n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 3, 3, 3})) + _, exist = n.getRestrictedNodes()[nids[0]] s.Require().False(exist) s.Require().Equal(0, len(nackeds)) - nackeds = n.processTimestamps(vids[2], genTimestamp(vids, []int{1, 3, 3, 3})) - _, exist = n.getRestrictedValidators()[vids[0]] + nackeds = n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 3, 3, 3})) + _, exist = n.getRestrictedNodes()[nids[0]] s.Require().True(exist) s.Require().Equal(0, len(nackeds)) - nackeds = n.processTimestamps(vids[3], genTimestamp(vids, []int{1, 3, 3, 3})) - _, exist = n.getRestrictedValidators()[vids[0]] + nackeds = n.processTimestamps(nids[3], genTimestamp(nids, []int{1, 3, 3, 3})) + _, exist = n.getRestrictedNodes()[nids[0]] s.Require().False(exist) s.Require().Equal(1, len(nackeds)) - s.Require().Equal(vids[0], nackeds[0]) + s.Require().Equal(nids[0], nackeds[0]) } func (s *NegativeAckTest) TestExpire() { var exist bool - n, vids := genTestNegativeAck(4) + n, nids := genTestNegativeAck(4) n.setTimeDelay(timeDelay) n.setTimeExpire(timeExpire) - n.processTimestamps(vids[0], genTimestamp(vids, []int{1, 1, 1, 1})) - n.processTimestamps(vids[1], genTimestamp(vids, []int{1, 1, 1, 1})) - n.processTimestamps(vids[2], genTimestamp(vids, []int{1, 1, 1, 1})) - n.processTimestamps(vids[3], genTimestamp(vids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 1, 1, 1})) + n.processTimestamps(nids[3], genTimestamp(nids, []int{1, 1, 1, 1})) - n.processTimestamps(vids[1], genTimestamp(vids, []int{1, 3, 3, 3})) - n.processTimestamps(vids[2], genTimestamp(vids, []int{1, 3, 3, 3})) - _, exist = n.getRestrictedValidators()[vids[0]] + n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 3, 3, 3})) + n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 3, 3, 3})) + _, exist = n.getRestrictedNodes()[nids[0]] s.Require().True(exist) time.Sleep(2 * timeExpire) - n.processTimestamps(vids[0], genTimestamp(vids, []int{2, 2, 2, 2})) + n.processTimestamps(nids[0], genTimestamp(nids, []int{2, 2, 2, 2})) - _, exist = n.getRestrictedValidators()[vids[0]] + _, exist = n.getRestrictedNodes()[nids[0]] s.Require().False(exist) } -func (s *NegativeAckTest) TestAddDeleteValidator() { - n, vids := genTestNegativeAck(10) +func (s *NegativeAckTest) TestAddDeleteNode() { + n, nids := genTestNegativeAck(10) s.Require().Equal(10, len(n.timeDiffs)) - s.Require().Equal(10, len(n.timeDiffs[vids[0]])) + s.Require().Equal(10, len(n.timeDiffs[nids[0]])) - for _, vid := range vids { - n.deleteValidator(vid) + for _, nid := range nids { + n.deleteNode(nid) } s.Require().Equal(0, len(n.timeDiffs)) } diff --git a/core/reliable-broadcast.go b/core/reliable-broadcast.go index 1681270..3c290ff 100644 --- a/core/reliable-broadcast.go +++ b/core/reliable-broadcast.go @@ -38,8 +38,8 @@ const ( // reliableBroadcast is a module for reliable broadcast. type reliableBroadcast struct { - // lattice stores validator's blocks and other info. - lattice []*rbcValidatorStatus + // lattice stores node's blocks and other info. + lattice []*rbcNodeStatus // blockInfos stores block infos. blockInfos map[common.Hash]*rbcBlockInfo @@ -48,12 +48,12 @@ type reliableBroadcast struct { // in lattice. receivedBlocks map[common.Hash]*types.Block - // validators stores validator set. - validators map[types.ValidatorID]struct{} + // nodes stores node set. + nodes map[types.NodeID]struct{} } -type rbcValidatorStatus struct { - // blocks stores blocks proposed by specified validator in map which key is +type rbcNodeStatus struct { + // blocks stores blocks proposed by specified node in map which key is // the height of the block. blocks map[uint64]*types.Block @@ -94,7 +94,7 @@ func newReliableBroadcast() *reliableBroadcast { return &reliableBroadcast{ blockInfos: make(map[common.Hash]*rbcBlockInfo), receivedBlocks: make(map[common.Hash]*types.Block), - validators: make(map[types.ValidatorID]struct{}), + nodes: make(map[types.NodeID]struct{}), } } @@ -104,8 +104,8 @@ func (rb *reliableBroadcast) sanityCheck(b *types.Block) error { return ErrInvalidChainID } - // Check if its proposer is in validator set. - if _, exist := rb.validators[b.ProposerID]; !exist { + // Check if its proposer is in node set. + if _, exist := rb.nodes[b.ProposerID]; !exist { return ErrInvalidProposerID } @@ -224,7 +224,7 @@ func (rb *reliableBroadcast) processBlock(block *types.Block) (err error) { for _, h := range b.Acks { bAckStat := rb.blockInfos[h] // Update nextAck only when bAckStat.block.Position.Height + 1 - // is greater. A block might ack blocks proposed by same validator with + // is greater. A block might ack blocks proposed by same node with // different height. if rb.lattice[chainID].nextAck[bAckStat.block.Position.ChainID] < bAckStat.block.Position.Height+1 { @@ -241,7 +241,7 @@ func (rb *reliableBroadcast) processBlock(block *types.Block) (err error) { } bAckStat.ackedChain[chainID] = struct{}{} // A block is strongly acked if it is acked by more than - // 2 * (maximum number of byzatine validators) unique validators. + // 2 * (maximum number of byzatine nodes) unique nodes. if len(bAckStat.ackedChain) > 2*((len(rb.lattice)-1)/3) { blocksToAcked[bAckStat.block.Hash] = bAckStat.block } @@ -272,8 +272,8 @@ func (rb *reliableBroadcast) processBlock(block *types.Block) (err error) { // Delete old blocks in "lattice" and "blocks" for release memory space. // First, find the height that blocks below it can be deleted. This height - // is defined by finding minimum of validator's nextOutput and last acking - // heights from other validators, i.e. rb.lattice[v_other].nextAck[this_vid]. + // is defined by finding minimum of node's nextOutput and last acking + // heights from other nodes, i.e. rb.lattice[v_other].nextAck[this_vid]. // This works because blocks of height below this minimum are not going to be // acked anymore, the ackings of these blocks are illegal. for vid := range rb.lattice { @@ -351,7 +351,7 @@ func (rb *reliableBroadcast) extractBlocks() []*types.Block { // prepareBlock helps to setup fields of block based on its ProposerID, // including: -// - Set 'Acks' and 'Timestamps' for the highest block of each validator not +// - Set 'Acks' and 'Timestamps' for the highest block of each node not // acked by this proposer before. // - Set 'ParentHash' and 'Height' from parent block, if we can't find a // parent, these fields would be setup like a genesis block. @@ -361,7 +361,7 @@ func (rb *reliableBroadcast) prepareBlock(block *types.Block) { block.ParentHash = common.Hash{} acks := common.Hashes{} for chainID := range rb.lattice { - // find height of the latest block for that validator. + // find height of the latest block for that node. var ( curBlock *types.Block nextHeight = rb.lattice[block.Position.ChainID].nextAck[chainID] @@ -394,21 +394,21 @@ func (rb *reliableBroadcast) prepareBlock(block *types.Block) { return } -// addValidator adds validator in the validator set. -func (rb *reliableBroadcast) addValidator(h types.ValidatorID) { - rb.validators[h] = struct{}{} +// addNode adds node in the node set. +func (rb *reliableBroadcast) addNode(h types.NodeID) { + rb.nodes[h] = struct{}{} } -// deleteValidator deletes validator in validator set. -func (rb *reliableBroadcast) deleteValidator(h types.ValidatorID) { - delete(rb.validators, h) +// deleteNode deletes node in node set. +func (rb *reliableBroadcast) deleteNode(h types.NodeID) { + delete(rb.nodes, h) } // setChainNum set the number of chains. func (rb *reliableBroadcast) setChainNum(num uint32) { - rb.lattice = make([]*rbcValidatorStatus, num) + rb.lattice = make([]*rbcNodeStatus, num) for i := range rb.lattice { - rb.lattice[i] = &rbcValidatorStatus{ + rb.lattice[i] = &rbcNodeStatus{ blocks: make(map[uint64]*types.Block), nextAck: make([]uint64, num), nextOutput: 0, diff --git a/core/reliable-broadcast_test.go b/core/reliable-broadcast_test.go index b1fc672..cf82949 100644 --- a/core/reliable-broadcast_test.go +++ b/core/reliable-broadcast_test.go @@ -46,8 +46,8 @@ func (s *ReliableBroadcastTest) SetupTest() { } func (s *ReliableBroadcastTest) prepareGenesisBlock( - proposerID types.ValidatorID, - validatorIDs []types.ValidatorID) (b *types.Block) { + proposerID types.NodeID, + nodeIDs []types.NodeID) (b *types.Block) { b = &types.Block{ ProposerID: proposerID, @@ -58,7 +58,7 @@ func (s *ReliableBroadcastTest) prepareGenesisBlock( Acks: common.NewSortedHashes(common.Hashes{}), Timestamp: time.Now().UTC(), } - for i, vID := range validatorIDs { + for i, vID := range nodeIDs { if proposerID == vID { b.Position.ChainID = uint32(i) break @@ -79,16 +79,16 @@ func (s *ReliableBroadcastTest) prepareGenesisBlock( // 1 | 1 // | | | // 0 0 0 0 (block height) -// 0 1 2 3 (validator) -func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.ValidatorID { - // Create new reliableBroadcast instance with 4 validators +// 0 1 2 3 (node) +func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.NodeID { + // Create new reliableBroadcast instance with 4 nodes var b *types.Block var h common.Hash - vids := []types.ValidatorID{} + vids := []types.NodeID{} for i := 0; i < 4; i++ { - vid := types.ValidatorID{Hash: common.NewRandomHash()} - rb.addValidator(vid) + vid := types.NodeID{Hash: common.NewRandomHash()} + rb.addNode(vid) vids = append(vids, vid) } rb.setChainNum(uint32(len(vids))) @@ -176,20 +176,20 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid return vids } -func (s *ReliableBroadcastTest) TestAddValidator() { +func (s *ReliableBroadcastTest) TestAddNode() { rb := newReliableBroadcast() s.Require().Equal(len(rb.lattice), 0) vids := genTestCase1(s, rb) s.Require().Equal(len(rb.lattice), 4) for _, vid := range vids { - rb.deleteValidator(vid) + rb.deleteNode(vid) } } func (s *ReliableBroadcastTest) TestSanityCheck() { var b *types.Block var h common.Hash - var vids []types.ValidatorID + var vids []types.NodeID var err error rb := newReliableBroadcast() vids = genTestCase1(s, rb) @@ -247,7 +247,7 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { // Invalid proposer ID. h = rb.lattice[1].blocks[0].Hash b = &types.Block{ - ProposerID: types.ValidatorID{Hash: common.NewRandomHash()}, + ProposerID: types.NodeID{Hash: common.NewRandomHash()}, ParentHash: h, Position: types.Position{ Height: 1, @@ -365,7 +365,7 @@ func (s *ReliableBroadcastTest) TestAreAllAcksInLattice() { func (s *ReliableBroadcastTest) TestStrongAck() { var b *types.Block - var vids []types.ValidatorID + var vids []types.NodeID rb := newReliableBroadcast() vids = genTestCase1(s, rb) @@ -491,13 +491,13 @@ func (s *ReliableBroadcastTest) TestExtractBlocks() { func (s *ReliableBroadcastTest) TestRandomIntensiveAcking() { rb := newReliableBroadcast() - vids := test.GenerateRandomValidatorIDs(4) - heights := map[types.ValidatorID]uint64{} + vids := test.GenerateRandomNodeIDs(4) + heights := map[types.NodeID]uint64{} extractedBlocks := []*types.Block{} - // Generate validators. + // Generate nodes. for _, vid := range vids { - rb.addValidator(vid) + rb.addNode(vid) } rb.setChainNum(uint32(len(vids))) // Generate genesis blocks. @@ -545,9 +545,9 @@ func (s *ReliableBroadcastTest) TestRandomIntensiveAcking() { func (s *ReliableBroadcastTest) TestRandomlyGeneratedBlocks() { var ( - validatorCount = 19 - blockCount = 50 - repeat = 20 + nodeCount = 19 + blockCount = 50 + repeat = 20 ) // Prepare a randomly generated blocks. @@ -560,7 +560,7 @@ func (s *ReliableBroadcastTest) TestRandomlyGeneratedBlocks() { } }() gen := test.NewBlocksGenerator(nil, hashBlock) - _, err = gen.Generate(validatorCount, blockCount, nil, db) + _, err = gen.Generate(nodeCount, blockCount, nil, db) s.Require().Nil(err) iter, err := db.GetAll() s.Require().Nil(err) @@ -570,9 +570,9 @@ func (s *ReliableBroadcastTest) TestRandomlyGeneratedBlocks() { stronglyAckedHashesAsString := map[string]struct{}{} for i := 0; i < repeat; i++ { - validators := map[types.ValidatorID]struct{}{} + nodes := map[types.NodeID]struct{}{} rb := newReliableBroadcast() - rb.setChainNum(uint32(validatorCount)) + rb.setChainNum(uint32(nodeCount)) stronglyAckedHashes := common.Hashes{} revealer.Reset() @@ -587,10 +587,10 @@ func (s *ReliableBroadcastTest) TestRandomlyGeneratedBlocks() { } s.Require().Nil(err) - // It's a hack to add validator to reliableBroadcast module. - if _, added := validators[b.ProposerID]; !added { - rb.addValidator(b.ProposerID) - validators[b.ProposerID] = struct{}{} + // It's a hack to add node to reliableBroadcast module. + if _, added := nodes[b.ProposerID]; !added { + rb.addNode(b.ProposerID) + nodes[b.ProposerID] = struct{}{} } // Perform reliable broadcast process. s.Require().Nil(rb.processBlock(&b)) @@ -621,21 +621,21 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { req = s.Require() rb = newReliableBroadcast() minInterval = 50 * time.Millisecond - validators = test.GenerateRandomValidatorIDs(4) + nodes = test.GenerateRandomNodeIDs(4) ) - // Prepare validator IDs. - for _, vID := range validators { - rb.addValidator(vID) + // Prepare node IDs. + for _, vID := range nodes { + rb.addNode(vID) } - rb.setChainNum(uint32(len(validators))) + rb.setChainNum(uint32(len(nodes))) // Setup genesis blocks. - b00 := s.prepareGenesisBlock(validators[0], validators) + b00 := s.prepareGenesisBlock(nodes[0], nodes) time.Sleep(minInterval) - b10 := s.prepareGenesisBlock(validators[1], validators) + b10 := s.prepareGenesisBlock(nodes[1], nodes) time.Sleep(minInterval) - b20 := s.prepareGenesisBlock(validators[2], validators) + b20 := s.prepareGenesisBlock(nodes[2], nodes) time.Sleep(minInterval) - b30 := s.prepareGenesisBlock(validators[3], validators) + b30 := s.prepareGenesisBlock(nodes[3], nodes) // Submit these blocks to reliableBroadcast instance. s.Require().Nil(rb.processBlock(b00)) s.Require().Nil(rb.processBlock(b10)) @@ -644,7 +644,7 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { // We should be able to collect all 4 genesis blocks by calling // prepareBlock. b11 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], Position: types.Position{ ChainID: 1, }, @@ -664,7 +664,7 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { s.Require().Nil(rb.processBlock(b11)) // Propose/Process a block based on collected info. b12 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], Position: types.Position{ ChainID: 1, }, @@ -677,10 +677,10 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { req.Contains(b12.Acks, b11.Hash) req.Equal(b12.ParentHash, b11.Hash) req.Equal(b12.Position.Height, uint64(2)) - // When calling with other validator ID, we should be able to + // When calling with other node ID, we should be able to // get 4 blocks to ack. b01 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], Position: types.Position{ ChainID: 0, }, diff --git a/core/test/blocks-generator.go b/core/test/blocks-generator.go index 93867f1..759e3a6 100644 --- a/core/test/blocks-generator.go +++ b/core/test/blocks-generator.go @@ -28,94 +28,97 @@ import ( "github.com/dexon-foundation/dexon-consensus-core/core/types" ) +// TODO(mission): blocks generator should generate blocks based on chain, +// not nodes. + // ErrParentNotAcked would be raised when some block doesn't // ack its parent block. var ErrParentNotAcked = errors.New("parent is not acked") -// validatorStatus is a state holder for each validator +// nodeStatus is a state holder for each node // during generating blocks. -type validatorStatus struct { +type nodeStatus struct { blocks []*types.Block - lastAckingHeight map[types.ValidatorID]uint64 + lastAckingHeight map[types.NodeID]uint64 } type hashBlockFn func(*types.Block) (common.Hash, error) // getAckedBlockHash would randomly pick one block between // last acked one to current head. -func (vs *validatorStatus) getAckedBlockHash( - ackedVID types.ValidatorID, - ackedValidator *validatorStatus, +func (vs *nodeStatus) getAckedBlockHash( + ackedNID types.NodeID, + ackedNode *nodeStatus, randGen *rand.Rand) ( hash common.Hash, ok bool) { - baseAckingHeight, exists := vs.lastAckingHeight[ackedVID] + baseAckingHeight, exists := vs.lastAckingHeight[ackedNID] if exists { // Do not ack the same block(height) twice. baseAckingHeight++ } - totalBlockCount := uint64(len(ackedValidator.blocks)) + totalBlockCount := uint64(len(ackedNode.blocks)) if totalBlockCount <= baseAckingHeight { // There is no new block to ack. return } ackableRange := totalBlockCount - baseAckingHeight height := uint64((randGen.Uint64() % ackableRange) + baseAckingHeight) - vs.lastAckingHeight[ackedVID] = height - hash = ackedValidator.blocks[height].Hash + vs.lastAckingHeight[ackedNID] = height + hash = ackedNode.blocks[height].Hash ok = true return } -// validatorSetStatus is a state holder for all validators +// nodeSetStatus is a state holder for all nodes // during generating blocks. -type validatorSetStatus struct { - status map[types.ValidatorID]*validatorStatus - proposerChain map[types.ValidatorID]uint32 +type nodeSetStatus struct { + status map[types.NodeID]*nodeStatus + proposerChain map[types.NodeID]uint32 timestamps []time.Time - validatorIDs []types.ValidatorID + nodeIDs []types.NodeID randGen *rand.Rand hashBlock hashBlockFn } -func newValidatorSetStatus(vIDs []types.ValidatorID, hashBlock hashBlockFn) *validatorSetStatus { - status := make(map[types.ValidatorID]*validatorStatus) - timestamps := make([]time.Time, 0, len(vIDs)) - proposerChain := make(map[types.ValidatorID]uint32) - for i, vID := range vIDs { - status[vID] = &validatorStatus{ +func newNodeSetStatus(nIDs []types.NodeID, hashBlock hashBlockFn) *nodeSetStatus { + status := make(map[types.NodeID]*nodeStatus) + timestamps := make([]time.Time, 0, len(nIDs)) + proposerChain := make(map[types.NodeID]uint32) + for i, nID := range nIDs { + status[nID] = &nodeStatus{ blocks: []*types.Block{}, - lastAckingHeight: make(map[types.ValidatorID]uint64), + lastAckingHeight: make(map[types.NodeID]uint64), } timestamps = append(timestamps, time.Now().UTC()) - proposerChain[vID] = uint32(i) + proposerChain[nID] = uint32(i) } - return &validatorSetStatus{ + return &nodeSetStatus{ status: status, proposerChain: proposerChain, timestamps: timestamps, - validatorIDs: vIDs, + nodeIDs: nIDs, randGen: rand.New(rand.NewSource(time.Now().UnixNano())), hashBlock: hashBlock, } } -// findIncompleteValidators is a helper to check which validator +// findIncompleteNodes is a helper to check which node // doesn't generate enough blocks. -func (vs *validatorSetStatus) findIncompleteValidators( - blockCount int) (vIDs []types.ValidatorID) { +func (vs *nodeSetStatus) findIncompleteNodes( + blockCount int) (nIDs []types.NodeID) { - for vID, status := range vs.status { + for nID, status := range vs.status { if len(status.blocks) < blockCount { - vIDs = append(vIDs, vID) + nIDs = append(nIDs, nID) } } return } // prepareAcksForNewBlock collects acks for one block. -func (vs *validatorSetStatus) prepareAcksForNewBlock( - proposerID types.ValidatorID, ackingCount int) ( +func (vs *nodeSetStatus) prepareAcksForNewBlock( + proposerID types.NodeID, ackingCount int) ( acks common.Hashes, err error) { acks = common.Hashes{} @@ -123,22 +126,22 @@ func (vs *validatorSetStatus) prepareAcksForNewBlock( // The 'Acks' filed of genesis blocks would always be empty. return } - // Pick validatorIDs to be acked. - ackingVIDs := map[types.ValidatorID]struct{}{ + // Pick nodeIDs to be acked. + ackingNIDs := map[types.NodeID]struct{}{ proposerID: struct{}{}, // Acking parent block is always required. } if ackingCount > 0 { ackingCount-- // We would always include ack to parent block. } - for _, i := range vs.randGen.Perm(len(vs.validatorIDs))[:ackingCount] { - ackingVIDs[vs.validatorIDs[i]] = struct{}{} + for _, i := range vs.randGen.Perm(len(vs.nodeIDs))[:ackingCount] { + ackingNIDs[vs.nodeIDs[i]] = struct{}{} } // Generate acks. - for vID := range ackingVIDs { + for nID := range ackingNIDs { ack, ok := vs.status[proposerID].getAckedBlockHash( - vID, vs.status[vID], vs.randGen) + nID, vs.status[nID], vs.randGen) if !ok { - if vID == proposerID { + if nID == proposerID { err = ErrParentNotAcked } continue @@ -148,9 +151,9 @@ func (vs *validatorSetStatus) prepareAcksForNewBlock( return } -// proposeBlock propose new block and update validator status. -func (vs *validatorSetStatus) proposeBlock( - proposerID types.ValidatorID, +// proposeBlock propose new block and update node status. +func (vs *nodeSetStatus) proposeBlock( + proposerID types.NodeID, acks common.Hashes) (*types.Block, error) { status := vs.status[proposerID] @@ -171,8 +174,8 @@ func (vs *validatorSetStatus) proposeBlock( Acks: common.NewSortedHashes(acks), Timestamp: vs.timestamps[chainID], } - for i, vID := range vs.validatorIDs { - if vID == proposerID { + for i, nID := range vs.nodeIDs { + if nID == proposerID { newBlock.Position.ChainID = uint32(i) } } @@ -188,13 +191,13 @@ func (vs *validatorSetStatus) proposeBlock( // normalAckingCountGenerator would randomly pick acking count // by a normal distribution. func normalAckingCountGenerator( - validatorCount int, mean, deviation float64) func() int { + nodeCount int, mean, deviation float64) func() int { return func() int { var expected float64 for { expected = rand.NormFloat64()*deviation + mean - if expected >= 0 && expected <= float64(validatorCount) { + if expected >= 0 && expected <= float64(nodeCount) { break } } @@ -208,32 +211,32 @@ func MaxAckingCountGenerator(count int) func() int { return func() int { return count } } -// generateValidatorPicker is a function generator, which would generate -// a function to randomly pick one validator ID from a slice of validator ID. -func generateValidatorPicker() func([]types.ValidatorID) types.ValidatorID { +// generateNodePicker is a function generator, which would generate +// a function to randomly pick one node ID from a slice of node ID. +func generateNodePicker() func([]types.NodeID) types.NodeID { privateRand := rand.New(rand.NewSource(time.Now().UnixNano())) - return func(vIDs []types.ValidatorID) types.ValidatorID { - return vIDs[privateRand.Intn(len(vIDs))] + return func(nIDs []types.NodeID) types.NodeID { + return nIDs[privateRand.Intn(len(nIDs))] } } // BlocksGenerator could generate blocks forming valid DAGs. type BlocksGenerator struct { - validatorPicker func([]types.ValidatorID) types.ValidatorID - hashBlock hashBlockFn + nodePicker func([]types.NodeID) types.NodeID + hashBlock hashBlockFn } // NewBlocksGenerator constructs BlockGenerator. -func NewBlocksGenerator(validatorPicker func( - []types.ValidatorID) types.ValidatorID, +func NewBlocksGenerator(nodePicker func( + []types.NodeID) types.NodeID, hashBlock hashBlockFn) *BlocksGenerator { - if validatorPicker == nil { - validatorPicker = generateValidatorPicker() + if nodePicker == nil { + nodePicker = generateNodePicker() } return &BlocksGenerator{ - validatorPicker: validatorPicker, - hashBlock: hashBlock, + nodePicker: nodePicker, + hashBlock: hashBlock, } } @@ -244,45 +247,45 @@ func NewBlocksGenerator(validatorPicker func( // has maximum 2 acks. // func () int { return 2 } // The default ackingCountGenerator would randomly pick a number based on -// the validatorCount you provided with a normal distribution. +// the nodeCount you provided with a normal distribution. func (gen *BlocksGenerator) Generate( - validatorCount int, + nodeCount int, blockCount int, ackingCountGenerator func() int, writer blockdb.Writer) ( - validators types.ValidatorIDs, err error) { + nodes types.NodeIDs, err error) { if ackingCountGenerator == nil { ackingCountGenerator = normalAckingCountGenerator( - validatorCount, - float64(validatorCount/2), - float64(validatorCount/4+1)) + nodeCount, + float64(nodeCount/2), + float64(nodeCount/4+1)) } - validators = types.ValidatorIDs{} - for i := 0; i < validatorCount; i++ { - validators = append( - validators, types.ValidatorID{Hash: common.NewRandomHash()}) + nodes = types.NodeIDs{} + for i := 0; i < nodeCount; i++ { + nodes = append( + nodes, types.NodeID{Hash: common.NewRandomHash()}) } - status := newValidatorSetStatus(validators, gen.hashBlock) + status := newNodeSetStatus(nodes, gen.hashBlock) // We would record the smallest height of block that could be acked - // from each validator's point-of-view. - toAck := make(map[types.ValidatorID]map[types.ValidatorID]uint64) - for _, vID := range validators { - toAck[vID] = make(map[types.ValidatorID]uint64) + // from each node's point-of-view. + toAck := make(map[types.NodeID]map[types.NodeID]uint64) + for _, nID := range nodes { + toAck[nID] = make(map[types.NodeID]uint64) } for { - // Find validators that doesn't propose enough blocks and + // Find nodes that doesn't propose enough blocks and // pick one from them randomly. - notYet := status.findIncompleteValidators(blockCount) + notYet := status.findIncompleteNodes(blockCount) if len(notYet) == 0 { break } // Propose a new block. var ( - proposerID = gen.validatorPicker(notYet) + proposerID = gen.nodePicker(notYet) acks common.Hashes ) acks, err = status.prepareAcksForNewBlock( diff --git a/core/test/blocks-generator_test.go b/core/test/blocks-generator_test.go index e607796..f8aa325 100644 --- a/core/test/blocks-generator_test.go +++ b/core/test/blocks-generator_test.go @@ -33,21 +33,21 @@ type BlocksGeneratorTestCase struct { func (s *BlocksGeneratorTestCase) TestGenerate() { // This test case is to make sure the generated blocks are legimate. - validatorCount := 19 + nodeCount := 19 blockCount := 50 gen := NewBlocksGenerator(nil, stableRandomHash) db, err := blockdb.NewMemBackedBlockDB() s.Require().Nil(err) - validators, err := gen.Generate( - validatorCount, blockCount, nil, db) + nodes, err := gen.Generate( + nodeCount, blockCount, nil, db) s.Require().Nil(err) - s.Require().Len(validators, validatorCount) + s.Require().Len(nodes, nodeCount) // Load all blocks in that database for further checking. iter, err := db.GetAll() s.Require().Nil(err) - blocksByValidator := make(map[types.ValidatorID][]*types.Block) + blocksByNode := make(map[types.NodeID][]*types.Block) blocksByHash := make(map[common.Hash]*types.Block) for { block, err := iter.Next() @@ -56,9 +56,9 @@ func (s *BlocksGeneratorTestCase) TestGenerate() { } s.Nil(err) - blocksByValidator[block.ProposerID] = - append(blocksByValidator[block.ProposerID], &block) - sort.Sort(types.ByHeight(blocksByValidator[block.ProposerID])) + blocksByNode[block.ProposerID] = + append(blocksByNode[block.ProposerID], &block) + sort.Sort(types.ByHeight(blocksByNode[block.ProposerID])) blocksByHash[block.Hash] = &block } @@ -67,8 +67,8 @@ func (s *BlocksGeneratorTestCase) TestGenerate() { // compared to its parent block. // - Parent Ack: always ack its parent block. // - No Acks in genesis bloc - for _, blocks := range blocksByValidator { - lastAckingHeights := map[types.ValidatorID]uint64{} + for _, blocks := range blocksByNode { + lastAckingHeights := map[types.NodeID]uint64{} s.Require().NotEmpty(blocks) // Check genesis block. @@ -106,19 +106,19 @@ func (s *BlocksGeneratorTestCase) TestGenerate() { func (s *BlocksGeneratorTestCase) TestGenerateWithMaxAckCount() { var ( - validatorCount = 13 - blockCount = 50 - gen = NewBlocksGenerator(nil, stableRandomHash) - req = s.Require() + nodeCount = 13 + blockCount = 50 + gen = NewBlocksGenerator(nil, stableRandomHash) + req = s.Require() ) // Generate with 0 acks. db, err := blockdb.NewMemBackedBlockDB() req.Nil(err) - validators, err := gen.Generate( - validatorCount, blockCount, MaxAckingCountGenerator(0), db) + nodes, err := gen.Generate( + nodeCount, blockCount, MaxAckingCountGenerator(0), db) req.Nil(err) - req.Len(validators, validatorCount) + req.Len(nodes, nodeCount) // Load blocks to check their acking count. iter, err := db.GetAll() req.Nil(err) @@ -137,11 +137,11 @@ func (s *BlocksGeneratorTestCase) TestGenerateWithMaxAckCount() { // Generate with acks as many as possible. db, err = blockdb.NewMemBackedBlockDB() req.Nil(err) - validators, err = gen.Generate( - validatorCount, blockCount, MaxAckingCountGenerator( - validatorCount), db) + nodes, err = gen.Generate( + nodeCount, blockCount, MaxAckingCountGenerator( + nodeCount), db) req.Nil(err) - req.Len(validators, validatorCount) + req.Len(nodes, nodeCount) // Load blocks to verify the average acking count. totalAckingCount := 0 totalBlockCount := 0 @@ -160,7 +160,7 @@ func (s *BlocksGeneratorTestCase) TestGenerateWithMaxAckCount() { totalBlockCount++ } req.NotZero(totalBlockCount) - req.True((totalAckingCount / totalBlockCount) >= (validatorCount / 2)) + req.True((totalAckingCount / totalBlockCount) >= (nodeCount / 2)) } func TestBlocksGenerator(t *testing.T) { diff --git a/core/test/fake-transport.go b/core/test/fake-transport.go index 2615bd4..2f1686e 100644 --- a/core/test/fake-transport.go +++ b/core/test/fake-transport.go @@ -28,10 +28,10 @@ import ( // by using golang channel. type FakeTransport struct { peerType TransportPeerType - vID types.ValidatorID + nID types.NodeID recvChannel chan *TransportEnvelope serverChannel chan<- *TransportEnvelope - peers map[types.ValidatorID]chan<- *TransportEnvelope + peers map[types.NodeID]chan<- *TransportEnvelope latency LatencyModel } @@ -45,19 +45,19 @@ func NewFakeTransportServer() TransportServer { // NewFakeTransportClient constructs FakeTransport instance for peer. func NewFakeTransportClient( - vID types.ValidatorID, latency LatencyModel) TransportClient { + nID types.NodeID, latency LatencyModel) TransportClient { return &FakeTransport{ peerType: TransportPeer, recvChannel: make(chan *TransportEnvelope, 1000), - vID: vID, + nID: nID, latency: latency, } } // Send implements Transport.Send method. func (t *FakeTransport) Send( - endpoint types.ValidatorID, msg interface{}) (err error) { + endpoint types.NodeID, msg interface{}) (err error) { ch, exists := t.peers[endpoint] if !exists { @@ -70,7 +70,7 @@ func (t *FakeTransport) Send( } ch <- &TransportEnvelope{ PeerType: t.peerType, - From: t.vID, + From: t.nID, Msg: msg, } }(ch) @@ -82,7 +82,7 @@ func (t *FakeTransport) Report(msg interface{}) (err error) { go func() { t.serverChannel <- &TransportEnvelope{ PeerType: TransportPeer, - From: t.vID, + From: t.nID, Msg: msg, } }() @@ -92,7 +92,7 @@ func (t *FakeTransport) Report(msg interface{}) (err error) { // Broadcast implements Transport.Broadcast method. func (t *FakeTransport) Broadcast(msg interface{}) (err error) { for k := range t.peers { - if k == t.vID { + if k == t.nID { continue } t.Send(k, msg) @@ -107,10 +107,10 @@ func (t *FakeTransport) Close() (err error) { } // Peers implements Transport.Peers method. -func (t *FakeTransport) Peers() (peers map[types.ValidatorID]struct{}) { - peers = make(map[types.ValidatorID]struct{}) - for vID := range t.peers { - peers[vID] = struct{}{} +func (t *FakeTransport) Peers() (peers map[types.NodeID]struct{}) { + peers = make(map[types.NodeID]struct{}) + for nID := range t.peers { + peers[nID] = struct{}{} } return } @@ -135,7 +135,7 @@ func (t *FakeTransport) Join( continue } if t.peers, ok = - envelope.Msg.(map[types.ValidatorID]chan<- *TransportEnvelope); !ok { + envelope.Msg.(map[types.NodeID]chan<- *TransportEnvelope); !ok { envelopes = append(envelopes, envelope) continue @@ -155,7 +155,7 @@ func (t *FakeTransport) Host() (chan *TransportEnvelope, error) { // WaitForPeers implements TransportServer.WaitForPeers method. func (t *FakeTransport) WaitForPeers(numPeers int) (err error) { - t.peers = make(map[types.ValidatorID]chan<- *TransportEnvelope) + t.peers = make(map[types.NodeID]chan<- *TransportEnvelope) for { envelope := <-t.recvChannel // Panic here if some peer send other stuffs before diff --git a/core/test/governance.go b/core/test/governance.go index c5746cb..a2e6f69 100644 --- a/core/test/governance.go +++ b/core/test/governance.go @@ -28,44 +28,44 @@ import ( var ( // ErrPrivateKeyNotExists means caller request private key for an - // unknown validator ID. + // unknown node ID. ErrPrivateKeyNotExists = fmt.Errorf("private key not exists") ) // Governance is an implementation of Goverance for testing purpose. type Governance struct { lambda time.Duration - notarySet map[types.ValidatorID]struct{} - privateKeys map[types.ValidatorID]crypto.PrivateKey + notarySet map[types.NodeID]struct{} + privateKeys map[types.NodeID]crypto.PrivateKey DKGComplaint map[uint64][]*types.DKGComplaint DKGMasterPublicKey map[uint64][]*types.DKGMasterPublicKey } // NewGovernance constructs a Governance instance. -func NewGovernance(validatorCount int, lambda time.Duration) ( +func NewGovernance(nodeCount int, lambda time.Duration) ( g *Governance, err error) { g = &Governance{ lambda: lambda, - notarySet: make(map[types.ValidatorID]struct{}), - privateKeys: make(map[types.ValidatorID]crypto.PrivateKey), + notarySet: make(map[types.NodeID]struct{}), + privateKeys: make(map[types.NodeID]crypto.PrivateKey), DKGComplaint: make(map[uint64][]*types.DKGComplaint), DKGMasterPublicKey: make(map[uint64][]*types.DKGMasterPublicKey), } - for i := 0; i < validatorCount; i++ { + for i := 0; i < nodeCount; i++ { prv, err := eth.NewPrivateKey() if err != nil { return nil, err } - vID := types.NewValidatorID(prv.PublicKey()) - g.notarySet[vID] = struct{}{} - g.privateKeys[vID] = prv + nID := types.NewNodeID(prv.PublicKey()) + g.notarySet[nID] = struct{}{} + g.privateKeys[nID] = prv } return } // GetNotarySet implements Governance interface to return current // notary set. -func (g *Governance) GetNotarySet() map[types.ValidatorID]struct{} { +func (g *Governance) GetNotarySet() map[types.NodeID]struct{} { return g.notarySet } @@ -81,12 +81,12 @@ func (g *Governance) GetConfiguration(blockHeight uint64) *types.Config { } } -// GetPrivateKey return the private key for that validator, this function +// GetPrivateKey return the private key for that node, this function // is a test utility and not a general Governance interface. func (g *Governance) GetPrivateKey( - vID types.ValidatorID) (key crypto.PrivateKey, err error) { + nID types.NodeID) (key crypto.PrivateKey, err error) { - key, exists := g.privateKeys[vID] + key, exists := g.privateKeys[nID] if !exists { err = ErrPrivateKeyNotExists return diff --git a/core/test/interface.go b/core/test/interface.go index 0dc2382..a422ee7 100644 --- a/core/test/interface.go +++ b/core/test/interface.go @@ -38,7 +38,7 @@ type Stopper interface { // moment. // The Stopper should check state of that handler and return 'true' // if the execution could be stopped. - ShouldStop(vID types.ValidatorID) bool + ShouldStop(nID types.NodeID) bool } // EventHandler defines an interface to handle a Scheduler event. @@ -62,8 +62,8 @@ type TransportEnvelope struct { // PeerType defines the type of source peer, could be either "peer" or // "server". PeerType TransportPeerType - // From defines the validatorID of the source peer. - From types.ValidatorID + // From defines the nodeID of the source peer. + From types.NodeID // Msg is the actual payload of this message. Msg interface{} } @@ -92,14 +92,14 @@ type Transport interface { // Broadcast a message to all peers in network. Broadcast(msg interface{}) error // Send one message to a peer. - Send(endpoint types.ValidatorID, msg interface{}) error + Send(endpoint types.NodeID, msg interface{}) error // Close would cleanup allocated resources. Close() error - // Peers return IDs of all connected validators in p2p favor. + // Peers return IDs of all connected nodes in p2p favor. // This method should be accessed after ether 'Join' or 'WaitForPeers' // returned. - Peers() map[types.ValidatorID]struct{} + Peers() map[types.NodeID]struct{} } // Marshaller defines an interface to convert between interface{} and []byte. diff --git a/core/test/revealer.go b/core/test/revealer.go index b8eb9b4..b3af4d7 100644 --- a/core/test/revealer.go +++ b/core/test/revealer.go @@ -63,12 +63,12 @@ func loadAllBlocks(iter blockdb.BlockIterator) ( // all blocks from blockdb, and randomly pick one block to reveal if // it still forms a valid DAG in revealed blocks. type RandomDAGRevealer struct { - // blocksByValidator group all blocks by validators and sorting + // blocksByNode group all blocks by nodes and sorting // them by height. - blocksByValidator map[types.ValidatorID][]*types.Block - // tipIndexes store the height of next block from one validator + blocksByNode map[types.NodeID][]*types.Block + // tipIndexes store the height of next block from one node // to check if is candidate. - tipIndexes map[types.ValidatorID]int + tipIndexes map[types.NodeID]int // candidate are blocks that forms valid DAG with // current revealed blocks. candidates []*types.Block @@ -86,19 +86,19 @@ func NewRandomDAGRevealer( return } - // Rearrange blocks by validators and height. - blocksByValidator := make(map[types.ValidatorID][]*types.Block) + // Rearrange blocks by nodes and height. + blocksByNode := make(map[types.NodeID][]*types.Block) for _, block := range blocks { - blocksByValidator[block.ProposerID] = - append(blocksByValidator[block.ProposerID], block) + blocksByNode[block.ProposerID] = + append(blocksByNode[block.ProposerID], block) } // Make sure blocks are sorted by block heights, from lower to higher. - for vID := range blocksByValidator { - sort.Sort(types.ByHeight(blocksByValidator[vID])) + for nID := range blocksByNode { + sort.Sort(types.ByHeight(blocksByNode[nID])) } r = &RandomDAGRevealer{ - blocksByValidator: blocksByValidator, - randGen: rand.New(rand.NewSource(time.Now().UnixNano())), + blocksByNode: blocksByNode, + randGen: rand.New(rand.NewSource(time.Now().UnixNano())), } // Make sure this revealer is ready to use. r.Reset() @@ -107,8 +107,8 @@ func NewRandomDAGRevealer( // pickCandidates is a helper function to pick candidates from current tips. func (r *RandomDAGRevealer) pickCandidates() { - for vID, tip := range r.tipIndexes { - blocks, exists := r.blocksByValidator[vID] + for nID, tip := range r.tipIndexes { + blocks, exists := r.blocksByNode[nID] if !exists { continue } @@ -117,7 +117,7 @@ func (r *RandomDAGRevealer) pickCandidates() { } block := blocks[tip] if isAllAckingBlockRevealed(block, r.revealed) { - r.tipIndexes[vID]++ + r.tipIndexes[nID]++ r.candidates = append(r.candidates, block) } } @@ -145,9 +145,9 @@ func (r *RandomDAGRevealer) Next() (types.Block, error) { // Reset implement Revealer.Reset method, which would reset the revealing. func (r *RandomDAGRevealer) Reset() { - r.tipIndexes = make(map[types.ValidatorID]int) - for vID := range r.blocksByValidator { - r.tipIndexes[vID] = 0 + r.tipIndexes = make(map[types.NodeID]int) + for nID := range r.blocksByNode { + r.tipIndexes[nID] = 0 } r.revealed = make(map[common.Hash]struct{}) r.candidates = []*types.Block{} diff --git a/core/test/revealer_test.go b/core/test/revealer_test.go index 16d3b18..0c20520 100644 --- a/core/test/revealer_test.go +++ b/core/test/revealer_test.go @@ -35,9 +35,9 @@ type RevealerTestSuite struct { func (s *RevealerTestSuite) SetupSuite() { var ( - err error - validatorCount = 19 - blockCount = 50 + err error + nodeCount = 19 + blockCount = 50 ) // Setup block database. s.db, err = blockdb.NewMemBackedBlockDB() @@ -45,10 +45,10 @@ func (s *RevealerTestSuite) SetupSuite() { // Randomly generate blocks. gen := NewBlocksGenerator(nil, stableRandomHash) - validators, err := gen.Generate( - validatorCount, blockCount, nil, s.db) + nodes, err := gen.Generate( + nodeCount, blockCount, nil, s.db) s.Require().Nil(err) - s.Require().Len(validators, validatorCount) + s.Require().Len(nodes, nodeCount) // Cache the count of total generated block. iter, err := s.db.GetAll() diff --git a/core/test/scheduler-event.go b/core/test/scheduler-event.go index 85968c5..2863b54 100644 --- a/core/test/scheduler-event.go +++ b/core/test/scheduler-event.go @@ -27,8 +27,8 @@ import ( type Event struct { // HistoryIndex is the index of this event in history. HistoryIndex int - // ValidatorID is the ID of handler that this event deginated to. - ValidatorID types.ValidatorID + // NodeID is the ID of handler that this event deginated to. + NodeID types.NodeID // Time is the expected execution time of this event. Time time.Time // ExecError record the error when handling this event. @@ -67,12 +67,12 @@ func (eq *eventQueue) Pop() interface{} { // NewEvent is the constructor for Event. func NewEvent( - vID types.ValidatorID, when time.Time, payload interface{}) *Event { + nID types.NodeID, when time.Time, payload interface{}) *Event { return &Event{ HistoryIndex: -1, ParentHistoryIndex: -1, - ValidatorID: vID, + NodeID: nID, Time: when, Payload: payload, } diff --git a/core/test/scheduler.go b/core/test/scheduler.go index 6a3a40a..7c5bbde 100644 --- a/core/test/scheduler.go +++ b/core/test/scheduler.go @@ -48,7 +48,7 @@ type Scheduler struct { history []*Event historyLock sync.RWMutex isStarted bool - handlers map[types.ValidatorID]*schedulerHandlerRecord + handlers map[types.NodeID]*schedulerHandlerRecord handlersLock sync.RWMutex eventNotification chan struct{} ctx context.Context @@ -62,7 +62,7 @@ func NewScheduler(stopper Stopper) *Scheduler { return &Scheduler{ events: eventQueue{}, history: []*Event{}, - handlers: make(map[types.ValidatorID]*schedulerHandlerRecord), + handlers: make(map[types.NodeID]*schedulerHandlerRecord), eventNotification: make(chan struct{}, 100000), ctx: ctx, cancelFunc: cancel, @@ -98,15 +98,15 @@ func (sch *Scheduler) Seed(e *Event) error { } // RegisterEventHandler register an event handler by providing ID of -// corresponding validator. +// corresponding node. func (sch *Scheduler) RegisterEventHandler( - vID types.ValidatorID, + nID types.NodeID, handler EventHandler) { sch.handlersLock.Lock() defer sch.handlersLock.Unlock() - sch.handlers[vID] = &schedulerHandlerRecord{handler: handler} + sch.handlers[nID] = &schedulerHandlerRecord{handler: handler} } // nextTick would pick the oldest event from eventQueue. @@ -144,12 +144,12 @@ func (sch *Scheduler) workerRoutine(wg *sync.WaitGroup) { handleEvent := func(e *Event) { // Find correspond handler record. - hRec := func(vID types.ValidatorID) *schedulerHandlerRecord { + hRec := func(nID types.NodeID) *schedulerHandlerRecord { sch.handlersLock.RLock() defer sch.handlersLock.RUnlock() - return sch.handlers[vID] - }(e.ValidatorID) + return sch.handlers[nID] + }(e.NodeID) newEvents := func() []*Event { // This lock makes sure there would be no concurrent access @@ -161,8 +161,8 @@ func (sch *Scheduler) workerRoutine(wg *sync.WaitGroup) { beforeExecution := time.Now().UTC() newEvents := hRec.handler.Handle(e) e.ExecInterval = time.Now().UTC().Sub(beforeExecution) - // It's safe to check status of that validator under 'hRec.lock'. - if sch.stopper.ShouldStop(e.ValidatorID) { + // It's safe to check status of that node under 'hRec.lock'. + if sch.stopper.ShouldStop(e.NodeID) { sch.cancelFunc() } return newEvents diff --git a/core/test/scheduler_test.go b/core/test/scheduler_test.go index 5aef36e..1e6d52f 100644 --- a/core/test/scheduler_test.go +++ b/core/test/scheduler_test.go @@ -33,16 +33,16 @@ type SchedulerTestSuite struct { type simpleStopper struct { lock sync.Mutex - touched map[types.ValidatorID]int + touched map[types.NodeID]int touchedCount int } func newSimpleStopper( - validators []types.ValidatorID, touchedCount int) *simpleStopper { + nodes []types.NodeID, touchedCount int) *simpleStopper { - touched := make(map[types.ValidatorID]int) - for _, vID := range validators { - touched[vID] = 0 + touched := make(map[types.NodeID]int) + for _, nID := range nodes { + touched[nID] = 0 } return &simpleStopper{ touched: touched, @@ -50,11 +50,11 @@ func newSimpleStopper( } } -func (stopper *simpleStopper) ShouldStop(vID types.ValidatorID) bool { +func (stopper *simpleStopper) ShouldStop(nID types.NodeID) bool { stopper.lock.Lock() defer stopper.lock.Unlock() - stopper.touched[vID] = stopper.touched[vID] + 1 + stopper.touched[nID] = stopper.touched[nID] + 1 for _, count := range stopper.touched { if count < stopper.touchedCount { return false @@ -65,26 +65,26 @@ func (stopper *simpleStopper) ShouldStop(vID types.ValidatorID) bool { type simpleHandler struct { count int - vID types.ValidatorID + nID types.NodeID } func (handler *simpleHandler) Handle(e *Event) (events []*Event) { - if e.ValidatorID == handler.vID { + if e.NodeID == handler.nID { handler.count++ } return } type fixedLatencyHandler struct { - vID types.ValidatorID + nID types.NodeID } func (handler *fixedLatencyHandler) Handle(e *Event) (events []*Event) { // Simulate execution time. time.Sleep(500 * time.Millisecond) return []*Event{&Event{ - ValidatorID: handler.vID, - Time: e.Time.Add(800 * time.Millisecond), + NodeID: handler.nID, + Time: e.Time.Add(800 * time.Millisecond), }} } @@ -114,22 +114,22 @@ func (s *SchedulerTestSuite) TestEventSequence() { func (s *SchedulerTestSuite) TestBasicRound() { // This test case makes sure these facts: - // - event is dispatched by validatorID attached to each handler. + // - event is dispatched by NodeID attached to each handler. // - stopper can stop the execution when condition is met. var ( - req = s.Require() - validators = GenerateRandomValidatorIDs(3) - stopper = newSimpleStopper(validators, 2) - sch = NewScheduler(stopper) - handlers = make(map[types.ValidatorID]*simpleHandler) + req = s.Require() + nodes = GenerateRandomNodeIDs(3) + stopper = newSimpleStopper(nodes, 2) + sch = NewScheduler(stopper) + handlers = make(map[types.NodeID]*simpleHandler) ) - for _, vID := range validators { - handler := &simpleHandler{vID: vID} - handlers[vID] = handler - sch.RegisterEventHandler(vID, handler) - req.Nil(sch.Seed(&Event{ValidatorID: vID})) - req.Nil(sch.Seed(&Event{ValidatorID: vID})) + for _, nID := range nodes { + handler := &simpleHandler{nID: nID} + handlers[nID] = handler + sch.RegisterEventHandler(nID, handler) + req.Nil(sch.Seed(&Event{NodeID: nID})) + req.Nil(sch.Seed(&Event{NodeID: nID})) } sch.Run(10) // Verify result. @@ -143,16 +143,16 @@ func (s *SchedulerTestSuite) TestChildEvent() { // assigned correctly. var ( req = s.Require() - vID = types.ValidatorID{Hash: common.NewRandomHash()} - stopper = newSimpleStopper(types.ValidatorIDs{vID}, 3) - handler = &fixedLatencyHandler{vID: vID} + nID = types.NodeID{Hash: common.NewRandomHash()} + stopper = newSimpleStopper(types.NodeIDs{nID}, 3) + handler = &fixedLatencyHandler{nID: nID} sch = NewScheduler(stopper) ) - sch.RegisterEventHandler(vID, handler) + sch.RegisterEventHandler(nID, handler) req.Nil(sch.Seed(&Event{ - ValidatorID: vID, - Time: time.Now().UTC(), + NodeID: nID, + Time: time.Now().UTC(), })) sch.Run(1) // Verify result. diff --git a/core/test/stopper.go b/core/test/stopper.go index 7c75958..9fe5592 100644 --- a/core/test/stopper.go +++ b/core/test/stopper.go @@ -24,13 +24,13 @@ import ( "github.com/dexon-foundation/dexon-consensus-core/core/types" ) -// StopByConfirmedBlocks would make sure each validators confirms +// StopByConfirmedBlocks would make sure each nodes confirms // at least X blocks proposed by itself. type StopByConfirmedBlocks struct { - apps map[types.ValidatorID]*App - dbs map[types.ValidatorID]blockdb.BlockDatabase - lastCheckDelivered map[types.ValidatorID]int - confirmedBlocks map[types.ValidatorID]int + apps map[types.NodeID]*App + dbs map[types.NodeID]blockdb.BlockDatabase + lastCheckDelivered map[types.NodeID]int + confirmedBlocks map[types.NodeID]int blockCount int lock sync.Mutex } @@ -38,45 +38,45 @@ type StopByConfirmedBlocks struct { // NewStopByConfirmedBlocks construct an StopByConfirmedBlocks instance. func NewStopByConfirmedBlocks( blockCount int, - apps map[types.ValidatorID]*App, - dbs map[types.ValidatorID]blockdb.BlockDatabase) *StopByConfirmedBlocks { + apps map[types.NodeID]*App, + dbs map[types.NodeID]blockdb.BlockDatabase) *StopByConfirmedBlocks { - confirmedBlocks := make(map[types.ValidatorID]int) - for vID := range apps { - confirmedBlocks[vID] = 0 + confirmedBlocks := make(map[types.NodeID]int) + for nID := range apps { + confirmedBlocks[nID] = 0 } return &StopByConfirmedBlocks{ apps: apps, dbs: dbs, - lastCheckDelivered: make(map[types.ValidatorID]int), + lastCheckDelivered: make(map[types.NodeID]int), confirmedBlocks: confirmedBlocks, blockCount: blockCount, } } // ShouldStop implements Stopper interface. -func (s *StopByConfirmedBlocks) ShouldStop(vID types.ValidatorID) bool { +func (s *StopByConfirmedBlocks) ShouldStop(nID types.NodeID) bool { s.lock.Lock() defer s.lock.Unlock() - // Accumulate confirmed blocks proposed by this validator in this round. - lastChecked := s.lastCheckDelivered[vID] - currentConfirmedBlocks := s.confirmedBlocks[vID] - db := s.dbs[vID] - s.apps[vID].Check(func(app *App) { + // Accumulate confirmed blocks proposed by this node in this round. + lastChecked := s.lastCheckDelivered[nID] + currentConfirmedBlocks := s.confirmedBlocks[nID] + db := s.dbs[nID] + s.apps[nID].Check(func(app *App) { for _, h := range app.DeliverSequence[lastChecked:] { b, err := db.Get(h) if err != nil { panic(err) } - if b.ProposerID == vID { + if b.ProposerID == nID { currentConfirmedBlocks++ } } - s.lastCheckDelivered[vID] = len(app.DeliverSequence) + s.lastCheckDelivered[nID] = len(app.DeliverSequence) }) - s.confirmedBlocks[vID] = currentConfirmedBlocks - // Check if all validators confirmed at least 'blockCount' blocks. + s.confirmedBlocks[nID] = currentConfirmedBlocks + // Check if all nodes confirmed at least 'blockCount' blocks. for _, v := range s.confirmedBlocks { if v < s.blockCount { return false diff --git a/core/test/stopper_test.go b/core/test/stopper_test.go index 9a0e430..262e178 100644 --- a/core/test/stopper_test.go +++ b/core/test/stopper_test.go @@ -33,20 +33,20 @@ type StopperTestSuite struct { func (s *StopperTestSuite) TestStopByConfirmedBlocks() { // This test case makes sure this stopper would stop when - // all validators confirmed at least 'x' count of blocks produced + // all nodes confirmed at least 'x' count of blocks produced // by themselves. var ( req = s.Require() ) - apps := make(map[types.ValidatorID]*App) - dbs := make(map[types.ValidatorID]blockdb.BlockDatabase) - validators := GenerateRandomValidatorIDs(2) + apps := make(map[types.NodeID]*App) + dbs := make(map[types.NodeID]blockdb.BlockDatabase) + nodes := GenerateRandomNodeIDs(2) db, err := blockdb.NewMemBackedBlockDB() req.Nil(err) - for _, vID := range validators { - apps[vID] = NewApp() - dbs[vID] = db + for _, nID := range nodes { + apps[nID] = NewApp() + dbs[nID] = db } deliver := func(blocks []*types.Block) { hashes := common.Hashes{} @@ -54,8 +54,8 @@ func (s *StopperTestSuite) TestStopByConfirmedBlocks() { hashes = append(hashes, b.Hash) req.Nil(db.Put(*b)) } - for _, vID := range validators { - app := apps[vID] + for _, nID := range nodes { + app := apps[nID] for _, h := range hashes { app.StronglyAcked(h) } @@ -67,35 +67,35 @@ func (s *StopperTestSuite) TestStopByConfirmedBlocks() { } stopper := NewStopByConfirmedBlocks(2, apps, dbs) b00 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], Hash: common.NewRandomHash(), } deliver([]*types.Block{b00}) b10 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], Hash: common.NewRandomHash(), } b11 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], ParentHash: b10.Hash, Hash: common.NewRandomHash(), } deliver([]*types.Block{b10, b11}) - req.False(stopper.ShouldStop(validators[1])) + req.False(stopper.ShouldStop(nodes[1])) b12 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], ParentHash: b11.Hash, Hash: common.NewRandomHash(), } deliver([]*types.Block{b12}) - req.False(stopper.ShouldStop(validators[1])) + req.False(stopper.ShouldStop(nodes[1])) b01 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b00.Hash, Hash: common.NewRandomHash(), } deliver([]*types.Block{b01}) - req.True(stopper.ShouldStop(validators[0])) + req.True(stopper.ShouldStop(nodes[0])) } func TestStopper(t *testing.T) { diff --git a/core/test/tcp-transport.go b/core/test/tcp-transport.go index 2afea14..8bbaf9c 100644 --- a/core/test/tcp-transport.go +++ b/core/test/tcp-transport.go @@ -37,18 +37,18 @@ import ( // tcpMessage is the general message between peers and server. type tcpMessage struct { - ValidatorID types.ValidatorID `json:"vid"` - Type string `json:"type"` - Info string `json:"conn"` + NodeID types.NodeID `json:"nid"` + Type string `json:"type"` + Info string `json:"conn"` } // TCPTransport implements Transport interface via TCP connection. type TCPTransport struct { peerType TransportPeerType - vID types.ValidatorID + nID types.NodeID localPort int - peersInfo map[types.ValidatorID]string - peers map[types.ValidatorID]chan<- []byte + peersInfo map[types.NodeID]string + peers map[types.NodeID]chan<- []byte peersLock sync.RWMutex recvChannel chan *TransportEnvelope ctx context.Context @@ -60,7 +60,7 @@ type TCPTransport struct { // NewTCPTransport constructs an TCPTransport instance. func NewTCPTransport( peerType TransportPeerType, - vID types.ValidatorID, + nID types.NodeID, latency LatencyModel, marshaller Marshaller, localPort int) *TCPTransport { @@ -68,9 +68,9 @@ func NewTCPTransport( ctx, cancel := context.WithCancel(context.Background()) return &TCPTransport{ peerType: peerType, - vID: vID, - peersInfo: make(map[types.ValidatorID]string), - peers: make(map[types.ValidatorID]chan<- []byte), + nID: nID, + peersInfo: make(map[types.NodeID]string), + peers: make(map[types.NodeID]chan<- []byte), recvChannel: make(chan *TransportEnvelope, 1000), ctx: ctx, cancel: cancel, @@ -82,7 +82,7 @@ func NewTCPTransport( // Send implements Transport.Send method. func (t *TCPTransport) Send( - endpoint types.ValidatorID, msg interface{}) (err error) { + endpoint types.NodeID, msg interface{}) (err error) { payload, err := t.marshalMessage(msg) if err != nil { @@ -110,8 +110,8 @@ func (t *TCPTransport) Broadcast(msg interface{}) (err error) { t.peersLock.RLock() defer t.peersLock.RUnlock() - for vID, ch := range t.peers { - if vID == t.vID { + for nID, ch := range t.peers { + if nID == t.nID { continue } go func(ch chan<- []byte) { @@ -131,7 +131,7 @@ func (t *TCPTransport) Close() (err error) { // Reset peers. t.peersLock.Lock() defer t.peersLock.Unlock() - t.peers = make(map[types.ValidatorID]chan<- []byte) + t.peers = make(map[types.NodeID]chan<- []byte) // Tell our user that this channel is closed. close(t.recvChannel) t.recvChannel = nil @@ -139,10 +139,10 @@ func (t *TCPTransport) Close() (err error) { } // Peers implements Transport.Peers method. -func (t *TCPTransport) Peers() (peers map[types.ValidatorID]struct{}) { - peers = make(map[types.ValidatorID]struct{}) - for vID := range t.peersInfo { - peers[vID] = struct{}{} +func (t *TCPTransport) Peers() (peers map[types.NodeID]struct{}) { + peers = make(map[types.NodeID]struct{}) + for nID := range t.peersInfo { + peers[nID] = struct{}{} } return } @@ -152,16 +152,16 @@ func (t *TCPTransport) marshalMessage( msgCarrier := struct { PeerType TransportPeerType `json:"peer_type"` - From types.ValidatorID `json:"from"` + From types.NodeID `json:"from"` Type string `json:"type"` Payload interface{} `json:"payload"` }{ PeerType: t.peerType, - From: t.vID, + From: t.nID, Payload: msg, } switch msg.(type) { - case map[types.ValidatorID]string: + case map[types.NodeID]string: msgCarrier.Type = "peerlist" case *tcpMessage: msgCarrier.Type = "trans-msg" @@ -188,13 +188,13 @@ func (t *TCPTransport) marshalMessage( func (t *TCPTransport) unmarshalMessage( payload []byte) ( peerType TransportPeerType, - from types.ValidatorID, + from types.NodeID, msg interface{}, err error) { msgCarrier := struct { PeerType TransportPeerType `json:"peer_type"` - From types.ValidatorID `json:"from"` + From types.NodeID `json:"from"` Type string `json:"type"` Payload json.RawMessage `json:"payload"` }{} @@ -205,7 +205,7 @@ func (t *TCPTransport) unmarshalMessage( from = msgCarrier.From switch msgCarrier.Type { case "peerlist": - var peers map[types.ValidatorID]string + var peers map[types.NodeID]string if err = json.Unmarshal(msgCarrier.Payload, &peers); err != nil { return } @@ -376,12 +376,12 @@ func (t *TCPTransport) listenerRoutine(listener *net.TCPListener) { // we only utilize the write part for simplicity. func (t *TCPTransport) buildConnectionsToPeers() (err error) { var wg sync.WaitGroup - for vID, addr := range t.peersInfo { - if vID == t.vID { + for nID, addr := range t.peersInfo { + if nID == t.nID { continue } wg.Add(1) - go func(vID types.ValidatorID, addr string) { + go func(nID types.NodeID, addr string) { defer wg.Done() conn, localErr := net.Dial("tcp", addr) @@ -394,8 +394,8 @@ func (t *TCPTransport) buildConnectionsToPeers() (err error) { t.peersLock.Lock() defer t.peersLock.Unlock() - t.peers[vID] = t.connWriter(conn) - }(vID, addr) + t.peers[nID] = t.connWriter(conn) + }(nID, addr) } wg.Wait() return @@ -410,13 +410,13 @@ type TCPTransportClient struct { // NewTCPTransportClient constructs a TCPTransportClient instance. func NewTCPTransportClient( - vID types.ValidatorID, + nID types.NodeID, latency LatencyModel, marshaller Marshaller, local bool) *TCPTransportClient { return &TCPTransportClient{ - TCPTransport: *NewTCPTransport(TransportPeer, vID, latency, marshaller, 8080), + TCPTransport: *NewTCPTransport(TransportPeer, nID, latency, marshaller, 8080), local: local, } } @@ -492,15 +492,15 @@ func (t *TCPTransportClient) Join( conn = net.JoinHostPort(ip, strconv.Itoa(t.localPort)) } if err = t.Report(&tcpMessage{ - Type: "conn", - ValidatorID: t.vID, - Info: conn, + Type: "conn", + NodeID: t.nID, + Info: conn, }); err != nil { return } // Wait for peers list sent by server. e := <-t.recvChannel - if t.peersInfo, ok = e.Msg.(map[types.ValidatorID]string); !ok { + if t.peersInfo, ok = e.Msg.(map[types.NodeID]string); !ok { panic(fmt.Errorf("expect peer list, not %v", e)) } // Setup connections to other peers. @@ -509,8 +509,8 @@ func (t *TCPTransportClient) Join( } // Report to server that the connections to other peers are ready. if err = t.Report(&tcpMessage{ - Type: "conn-ready", - ValidatorID: t.vID, + Type: "conn-ready", + NodeID: t.nID, }); err != nil { return } @@ -547,11 +547,11 @@ func NewTCPTransportServer( serverPort int) *TCPTransportServer { return &TCPTransportServer{ - // NOTE: the assumption here is the validator ID of peers + // NOTE: the assumption here is the node ID of peers // won't be zero. TCPTransport: *NewTCPTransport( TransportPeerServer, - types.ValidatorID{}, + types.NodeID{}, nil, marshaller, serverPort), @@ -586,7 +586,7 @@ func (t *TCPTransportServer) WaitForPeers(numPeers int) (err error) { if msg.Type != "conn" { panic(fmt.Errorf("expect connection report, not %v", e)) } - t.peersInfo[msg.ValidatorID] = msg.Info + t.peersInfo[msg.NodeID] = msg.Info // Check if we already collect enought peers. if len(t.peersInfo) == numPeers { break @@ -600,7 +600,7 @@ func (t *TCPTransportServer) WaitForPeers(numPeers int) (err error) { return } // Wait for peers to send 'ready' report. - readies := make(map[types.ValidatorID]struct{}) + readies := make(map[types.NodeID]struct{}) for { e := <-t.recvChannel msg, ok := e.Msg.(*tcpMessage) @@ -610,10 +610,10 @@ func (t *TCPTransportServer) WaitForPeers(numPeers int) (err error) { if msg.Type != "conn-ready" { panic(fmt.Errorf("expect connection ready, not %v", e)) } - if _, reported := readies[msg.ValidatorID]; reported { + if _, reported := readies[msg.NodeID]; reported { panic(fmt.Errorf("already report conn-ready message: %v", e)) } - readies[msg.ValidatorID] = struct{}{} + readies[msg.NodeID] = struct{}{} if len(readies) == numPeers { break } diff --git a/core/test/transport_test.go b/core/test/transport_test.go index 9bfc12b..1d4b53d 100644 --- a/core/test/transport_test.go +++ b/core/test/transport_test.go @@ -32,21 +32,21 @@ import ( ) type testPeer struct { - vID types.ValidatorID + nID types.NodeID trans TransportClient recv <-chan *TransportEnvelope expectedEchoHash common.Hash echoBlock *types.Block myBlock *types.Block myBlockSentTime time.Time - blocks map[types.ValidatorID]*types.Block + blocks map[types.NodeID]*types.Block blocksReceiveTime map[common.Hash]time.Time } type testPeerServer struct { trans TransportServer recv chan *TransportEnvelope - peerBlocks map[types.ValidatorID]*types.Block + peerBlocks map[types.NodeID]*types.Block } type testMarshaller struct{} @@ -88,7 +88,7 @@ type TransportTestSuite struct { func (s *TransportTestSuite) baseTest( server *testPeerServer, - peers map[types.ValidatorID]*testPeer, + peers map[types.NodeID]*testPeer, delay time.Duration) { var ( @@ -98,11 +98,11 @@ func (s *TransportTestSuite) baseTest( // For each peers, do following stuffs: // - broadcast 1 block. - // - report one random block to server, along with its validator ID. + // - report one random block to server, along with its node ID. // Server would echo the random block back to the peer. handleServer := func(server *testPeerServer) { defer wg.Done() - server.peerBlocks = make(map[types.ValidatorID]*types.Block) + server.peerBlocks = make(map[types.NodeID]*types.Block) for { select { case e := <-server.recv: @@ -123,14 +123,14 @@ func (s *TransportTestSuite) baseTest( } handlePeer := func(peer *testPeer) { defer wg.Done() - peer.blocks = make(map[types.ValidatorID]*types.Block) + peer.blocks = make(map[types.NodeID]*types.Block) peer.blocksReceiveTime = make(map[common.Hash]time.Time) for { select { case e := <-peer.recv: switch v := e.Msg.(type) { case *types.Block: - if v.ProposerID == peer.vID { + if v.ProposerID == peer.nID { req.Equal(e.PeerType, TransportPeerServer) peer.echoBlock = v } else { @@ -150,11 +150,11 @@ func (s *TransportTestSuite) baseTest( } wg.Add(len(peers) + 1) go handleServer(server) - for vID, peer := range peers { + for nID, peer := range peers { go handlePeer(peer) // Broadcast a block. peer.myBlock = &types.Block{ - ProposerID: vID, + ProposerID: nID, Hash: common.NewRandomHash(), } peer.myBlockSentTime = time.Now() @@ -162,28 +162,28 @@ func (s *TransportTestSuite) baseTest( // Report a block to server. peer.expectedEchoHash = common.NewRandomHash() peer.trans.Report(&types.Block{ - ProposerID: vID, + ProposerID: nID, Hash: peer.expectedEchoHash, }) } wg.Wait() // Make sure each sent block is received. - for vID, peer := range peers { + for nID, peer := range peers { req.NotNil(peer.echoBlock) req.Equal(peer.echoBlock.Hash, peer.expectedEchoHash) - for otherVID, otherPeer := range peers { - if vID == otherVID { + for othernID, otherPeer := range peers { + if nID == othernID { continue } req.Equal( peer.myBlock.Hash, - otherPeer.blocks[peer.vID].Hash) + otherPeer.blocks[peer.nID].Hash) } } // Make sure the latency is expected. - for vID, peer := range peers { - for otherVID, otherPeer := range peers { - if otherVID == vID { + for nID, peer := range peers { + for othernID, otherPeer := range peers { + if othernID == nID { continue } req.True(otherPeer.blocksReceiveTime[peer.myBlock.Hash].Sub( @@ -196,8 +196,8 @@ func (s *TransportTestSuite) TestFake() { var ( peerCount = 13 req = s.Require() - peers = make(map[types.ValidatorID]*testPeer) - vIDs = GenerateRandomValidatorIDs(peerCount) + peers = make(map[types.NodeID]*testPeer) + nIDs = GenerateRandomNodeIDs(peerCount) err error wg sync.WaitGroup latency = &FixedLatencyModel{Latency: 300} @@ -207,13 +207,13 @@ func (s *TransportTestSuite) TestFake() { server.recv, err = server.trans.Host() req.Nil(err) // Setup Peers - wg.Add(len(vIDs)) - for _, vID := range vIDs { + wg.Add(len(nIDs)) + for _, nID := range nIDs { peer := &testPeer{ - vID: vID, - trans: NewFakeTransportClient(vID, latency), + nID: nID, + trans: NewFakeTransportClient(nID, latency), } - peers[vID] = peer + peers[nID] = peer go func() { defer wg.Done() recv, err := peer.trans.Join(server.recv) @@ -236,8 +236,8 @@ func (s *TransportTestSuite) TestTCPLocal() { var ( peerCount = 25 req = s.Require() - peers = make(map[types.ValidatorID]*testPeer) - vIDs = GenerateRandomValidatorIDs(peerCount) + peers = make(map[types.NodeID]*testPeer) + nIDs = GenerateRandomNodeIDs(peerCount) err error wg sync.WaitGroup latency = &FixedLatencyModel{Latency: 300} @@ -250,13 +250,13 @@ func (s *TransportTestSuite) TestTCPLocal() { server.recv, err = server.trans.Host() req.Nil(err) // Setup Peers - wg.Add(len(vIDs)) - for _, vID := range vIDs { + wg.Add(len(nIDs)) + for _, nID := range nIDs { peer := &testPeer{ - vID: vID, - trans: NewTCPTransportClient(vID, latency, &testMarshaller{}, true), + nID: nID, + trans: NewTCPTransportClient(nID, latency, &testMarshaller{}, true), } - peers[vID] = peer + peers[nID] = peer go func() { defer wg.Done() diff --git a/core/test/utils.go b/core/test/utils.go index 138e8a1..887ef14 100644 --- a/core/test/utils.go +++ b/core/test/utils.go @@ -34,11 +34,11 @@ func stableRandomHash(block *types.Block) (common.Hash, error) { return common.NewRandomHash(), nil } -// GenerateRandomValidatorIDs generates randomly a slices of types.ValidatorID. -func GenerateRandomValidatorIDs(validatorCount int) (vIDs types.ValidatorIDs) { - vIDs = types.ValidatorIDs{} - for i := 0; i < validatorCount; i++ { - vIDs = append(vIDs, types.ValidatorID{Hash: common.NewRandomHash()}) +// GenerateRandomNodeIDs generates randomly a slices of types.NodeID. +func GenerateRandomNodeIDs(nodeCount int) (nIDs types.NodeIDs) { + nIDs = types.NodeIDs{} + for i := 0; i < nodeCount; i++ { + nIDs = append(nIDs, types.NodeID{Hash: common.NewRandomHash()}) } return } diff --git a/core/total-ordering_test.go b/core/total-ordering_test.go index d6805b8..c9dd5a5 100644 --- a/core/total-ordering_test.go +++ b/core/total-ordering_test.go @@ -34,7 +34,7 @@ type TotalOrderingTestSuite struct { } func (s *TotalOrderingTestSuite) genGenesisBlock( - vIDs types.ValidatorIDs, + vIDs types.NodeIDs, chainID uint32, acks common.Hashes) *types.Block { @@ -77,9 +77,9 @@ func (s *TotalOrderingTestSuite) TestBlockRelation() { // // The DAG used below is: // A <- B <- C - validators := test.GenerateRandomValidatorIDs(5) - vID := validators[0] - blockA := s.genGenesisBlock(validators, 0, common.Hashes{}) + nodes := test.GenerateRandomNodeIDs(5) + vID := nodes[0] + blockA := s.genGenesisBlock(nodes, 0, common.Hashes{}) blockB := &types.Block{ ProposerID: vID, ParentHash: blockA.Hash, @@ -101,7 +101,7 @@ func (s *TotalOrderingTestSuite) TestBlockRelation() { Acks: common.NewSortedHashes(common.Hashes{blockB.Hash}), } - to := newTotalOrdering(1, 3, uint32(len(validators))) + to := newTotalOrdering(1, 3, uint32(len(nodes))) s.checkNotDeliver(to, blockA) s.checkNotDeliver(to, blockB) s.checkNotDeliver(to, blockC) @@ -204,15 +204,15 @@ func (s *TotalOrderingTestSuite) TestGrade() { // This test case just fake some internal structure used // when performing total ordering. var ( - validators = test.GenerateRandomValidatorIDs(5) - cache = newTotalOrderingObjectCache(5) - dirtyValidators = []int{0, 1, 2, 3, 4} + nodes = test.GenerateRandomNodeIDs(5) + cache = newTotalOrderingObjectCache(5) + dirtyNodes = []int{0, 1, 2, 3, 4} ) - ansLength := uint64(len(map[types.ValidatorID]struct{}{ - validators[0]: struct{}{}, - validators[1]: struct{}{}, - validators[2]: struct{}{}, - validators[3]: struct{}{}, + ansLength := uint64(len(map[types.NodeID]struct{}{ + nodes[0]: struct{}{}, + nodes[1]: struct{}{}, + nodes[2]: struct{}{}, + nodes[3]: struct{}{}, })) candidate1 := newTotalOrderingCandidateInfo(common.Hash{}, cache) candidate1.cachedHeightVector = []uint64{ @@ -225,29 +225,29 @@ func (s *TotalOrderingTestSuite) TestGrade() { 1, 1, infinity, infinity, infinity} candidate2.updateWinRecord( - 0, candidate1, dirtyValidators, cache) + 0, candidate1, dirtyNodes, cache) s.Equal(candidate2.winRecords[0].grade(5, 3, ansLength), 1) candidate1.updateWinRecord( - 1, candidate2, dirtyValidators, cache) + 1, candidate2, dirtyNodes, cache) s.Equal(candidate1.winRecords[1].grade(5, 3, ansLength), 0) candidate2.updateWinRecord( - 2, candidate3, dirtyValidators, cache) + 2, candidate3, dirtyNodes, cache) s.Equal(candidate2.winRecords[2].grade(5, 3, ansLength), -1) candidate3.updateWinRecord( - 1, candidate2, dirtyValidators, cache) + 1, candidate2, dirtyNodes, cache) s.Equal(candidate3.winRecords[1].grade(5, 3, ansLength), 0) } func (s *TotalOrderingTestSuite) TestCycleDetection() { // Make sure we don't get hang by cycle from // block's acks. - validators := test.GenerateRandomValidatorIDs(5) + nodes := test.GenerateRandomNodeIDs(5) // create blocks with cycles in acking relation. cycledHash := common.NewRandomHash() - b00 := s.genGenesisBlock(validators, 0, common.Hashes{cycledHash}) + b00 := s.genGenesisBlock(nodes, 0, common.Hashes{cycledHash}) b01 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b00.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -257,7 +257,7 @@ func (s *TotalOrderingTestSuite) TestCycleDetection() { Acks: common.NewSortedHashes(common.Hashes{b00.Hash}), } b02 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b01.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -267,7 +267,7 @@ func (s *TotalOrderingTestSuite) TestCycleDetection() { Acks: common.NewSortedHashes(common.Hashes{b01.Hash}), } b03 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b02.Hash, Hash: cycledHash, Position: types.Position{ @@ -278,11 +278,11 @@ func (s *TotalOrderingTestSuite) TestCycleDetection() { } // Create a block acks self. - b10 := s.genGenesisBlock(validators, 1, common.Hashes{}) + b10 := s.genGenesisBlock(nodes, 1, common.Hashes{}) b10.Acks = append(b10.Acks, b10.Hash) // Make sure we won't hang when cycle exists. - to := newTotalOrdering(1, 3, uint32(len(validators))) + to := newTotalOrdering(1, 3, uint32(len(nodes))) s.checkNotDeliver(to, b00) s.checkNotDeliver(to, b01) s.checkNotDeliver(to, b02) @@ -294,12 +294,12 @@ func (s *TotalOrderingTestSuite) TestCycleDetection() { } func (s *TotalOrderingTestSuite) TestNotValidDAGDetection() { - validators := test.GenerateRandomValidatorIDs(5) - to := newTotalOrdering(1, 3, uint32(len(validators))) + nodes := test.GenerateRandomNodeIDs(5) + to := newTotalOrdering(1, 3, uint32(len(nodes))) - b00 := s.genGenesisBlock(validators, 0, common.Hashes{}) + b00 := s.genGenesisBlock(nodes, 0, common.Hashes{}) b01 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b00.Hash, Position: types.Position{ Height: 1, @@ -326,8 +326,8 @@ func (s *TotalOrderingTestSuite) TestEarlyDeliver() { // A B // Even when B is not received, A should // be able to be delivered. - validators := test.GenerateRandomValidatorIDs(5) - to := newTotalOrdering(2, 3, uint32(len(validators))) + nodes := test.GenerateRandomNodeIDs(5) + to := newTotalOrdering(2, 3, uint32(len(nodes))) genNextBlock := func(b *types.Block) *types.Block { return &types.Block{ ProposerID: b.ProposerID, @@ -341,19 +341,19 @@ func (s *TotalOrderingTestSuite) TestEarlyDeliver() { } } - b00 := s.genGenesisBlock(validators, 0, common.Hashes{}) + b00 := s.genGenesisBlock(nodes, 0, common.Hashes{}) b01 := genNextBlock(b00) b02 := genNextBlock(b01) - b10 := s.genGenesisBlock(validators, 1, common.Hashes{b00.Hash}) + b10 := s.genGenesisBlock(nodes, 1, common.Hashes{b00.Hash}) b11 := genNextBlock(b10) b12 := genNextBlock(b11) - b20 := s.genGenesisBlock(validators, 2, common.Hashes{b00.Hash}) + b20 := s.genGenesisBlock(nodes, 2, common.Hashes{b00.Hash}) b21 := genNextBlock(b20) b22 := genNextBlock(b21) - b30 := s.genGenesisBlock(validators, 3, common.Hashes{b00.Hash}) + b30 := s.genGenesisBlock(nodes, 3, common.Hashes{b00.Hash}) b31 := genNextBlock(b30) b32 := genNextBlock(b31) @@ -431,16 +431,16 @@ func (s *TotalOrderingTestSuite) TestEarlyDeliver() { func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { // It's a handcrafted test case. - validators := test.GenerateRandomValidatorIDs(5) - to := newTotalOrdering(2, 3, uint32(len(validators))) + nodes := test.GenerateRandomNodeIDs(5) + to := newTotalOrdering(2, 3, uint32(len(nodes))) // Setup blocks. - b00 := s.genGenesisBlock(validators, 0, common.Hashes{}) - b10 := s.genGenesisBlock(validators, 1, common.Hashes{}) - b20 := s.genGenesisBlock(validators, 2, common.Hashes{b10.Hash}) - b30 := s.genGenesisBlock(validators, 3, common.Hashes{b20.Hash}) - b40 := s.genGenesisBlock(validators, 4, common.Hashes{}) + b00 := s.genGenesisBlock(nodes, 0, common.Hashes{}) + b10 := s.genGenesisBlock(nodes, 1, common.Hashes{}) + b20 := s.genGenesisBlock(nodes, 2, common.Hashes{b10.Hash}) + b30 := s.genGenesisBlock(nodes, 3, common.Hashes{b20.Hash}) + b40 := s.genGenesisBlock(nodes, 4, common.Hashes{}) b11 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], ParentHash: b10.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -450,7 +450,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b10.Hash, b00.Hash}), } b01 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b00.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -460,7 +460,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b00.Hash, b11.Hash}), } b21 := &types.Block{ - ProposerID: validators[2], + ProposerID: nodes[2], ParentHash: b20.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -470,7 +470,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b20.Hash, b01.Hash}), } b31 := &types.Block{ - ProposerID: validators[3], + ProposerID: nodes[3], ParentHash: b30.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -480,7 +480,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b30.Hash, b21.Hash}), } b02 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b01.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -490,7 +490,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b01.Hash, b21.Hash}), } b12 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], ParentHash: b11.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -500,7 +500,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b11.Hash, b21.Hash}), } b32 := &types.Block{ - ProposerID: validators[3], + ProposerID: nodes[3], ParentHash: b31.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -510,7 +510,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b31.Hash}), } b22 := &types.Block{ - ProposerID: validators[2], + ProposerID: nodes[2], ParentHash: b21.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -520,7 +520,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b21.Hash, b32.Hash}), } b23 := &types.Block{ - ProposerID: validators[2], + ProposerID: nodes[2], ParentHash: b22.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -530,7 +530,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b22.Hash}), } b03 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b02.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -540,7 +540,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b02.Hash, b22.Hash}), } b13 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], ParentHash: b12.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -550,7 +550,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b12.Hash, b22.Hash}), } b14 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], ParentHash: b13.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -560,7 +560,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b13.Hash}), } b41 := &types.Block{ - ProposerID: validators[4], + ProposerID: nodes[4], ParentHash: b40.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -570,7 +570,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { Acks: common.NewSortedHashes(common.Hashes{b40.Hash}), } b42 := &types.Block{ - ProposerID: validators[4], + ProposerID: nodes[4], ParentHash: b41.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -766,17 +766,17 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { // v v v v // o o o <- o Height: 0 var ( - req = s.Require() - validators = test.GenerateRandomValidatorIDs(5) - to = newTotalOrdering(0, 3, uint32(len(validators))) + req = s.Require() + nodes = test.GenerateRandomNodeIDs(5) + to = newTotalOrdering(0, 3, uint32(len(nodes))) ) // Setup blocks. - b00 := s.genGenesisBlock(validators, 0, common.Hashes{}) - b10 := s.genGenesisBlock(validators, 1, common.Hashes{}) - b20 := s.genGenesisBlock(validators, 2, common.Hashes{}) - b30 := s.genGenesisBlock(validators, 3, common.Hashes{b20.Hash}) + b00 := s.genGenesisBlock(nodes, 0, common.Hashes{}) + b10 := s.genGenesisBlock(nodes, 1, common.Hashes{}) + b20 := s.genGenesisBlock(nodes, 2, common.Hashes{}) + b30 := s.genGenesisBlock(nodes, 3, common.Hashes{b20.Hash}) b01 := &types.Block{ - ProposerID: validators[0], + ProposerID: nodes[0], ParentHash: b00.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -786,7 +786,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { Acks: common.NewSortedHashes(common.Hashes{b00.Hash, b10.Hash}), } b11 := &types.Block{ - ProposerID: validators[1], + ProposerID: nodes[1], ParentHash: b10.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -796,7 +796,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { Acks: common.NewSortedHashes(common.Hashes{b10.Hash, b20.Hash}), } b21 := &types.Block{ - ProposerID: validators[2], + ProposerID: nodes[2], ParentHash: b20.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -806,7 +806,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { Acks: common.NewSortedHashes(common.Hashes{b20.Hash}), } b31 := &types.Block{ - ProposerID: validators[3], + ProposerID: nodes[3], ParentHash: b30.Hash, Hash: common.NewRandomHash(), Position: types.Position{ @@ -815,7 +815,7 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { }, Acks: common.NewSortedHashes(common.Hashes{b21.Hash, b30.Hash}), } - b40 := s.genGenesisBlock(validators, 4, common.Hashes{b31.Hash}) + b40 := s.genGenesisBlock(nodes, 4, common.Hashes{b31.Hash}) s.checkNotDeliver(to, b00) s.checkNotDeliver(to, b10) @@ -864,8 +864,8 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { } func (s *TotalOrderingTestSuite) baseTestRandomlyGeneratedBlocks( - totalOrderingConstructor func(types.ValidatorIDs) *totalOrdering, - validatorCount, blockCount int, + totalOrderingConstructor func(types.NodeIDs) *totalOrdering, + nodeCount, blockCount int, ackingCountGenerator func() int, repeat int) { @@ -878,10 +878,10 @@ func (s *TotalOrderingTestSuite) baseTestRandomlyGeneratedBlocks( db, err := blockdb.NewMemBackedBlockDB() req.Nil(err) - validators, err := gen.Generate( - validatorCount, blockCount, ackingCountGenerator, db) + nodes, err := gen.Generate( + nodeCount, blockCount, ackingCountGenerator, db) req.Nil(err) - req.Len(validators, validatorCount) + req.Len(nodes, nodeCount) iter, err := db.GetAll() req.Nil(err) // Setup a revealer that would reveal blocks forming @@ -894,7 +894,7 @@ func (s *TotalOrderingTestSuite) baseTestRandomlyGeneratedBlocks( revealed := "" ordered := "" revealer.Reset() - to := totalOrderingConstructor(validators) + to := totalOrderingConstructor(nodes) for { // Reveal next block. b, err := revealer.Next() @@ -937,44 +937,44 @@ func (s *TotalOrderingTestSuite) baseTestRandomlyGeneratedBlocks( func (s *TotalOrderingTestSuite) TestRandomlyGeneratedBlocks() { var ( - validatorCount = 13 - blockCount = 50 - phi uint64 = 10 - repeat = 8 + nodeCount = 13 + blockCount = 50 + phi uint64 = 10 + repeat = 8 ) ackingCountGenerators := []func() int{ nil, // Acking frequency with normal distribution. - test.MaxAckingCountGenerator(0), // Low acking frequency. - test.MaxAckingCountGenerator(validatorCount), // High acking frequency. + test.MaxAckingCountGenerator(0), // Low acking frequency. + test.MaxAckingCountGenerator(nodeCount), // High acking frequency. } // Test based on different acking frequency. for _, gen := range ackingCountGenerators { // Test for K=0. - constructor := func(validators types.ValidatorIDs) *totalOrdering { - return newTotalOrdering(0, phi, uint32(len(validators))) + constructor := func(nodes types.NodeIDs) *totalOrdering { + return newTotalOrdering(0, phi, uint32(len(nodes))) } s.baseTestRandomlyGeneratedBlocks( - constructor, validatorCount, blockCount, gen, repeat) + constructor, nodeCount, blockCount, gen, repeat) // Test for K=1, - constructor = func(validators types.ValidatorIDs) *totalOrdering { - return newTotalOrdering(1, phi, uint32(len(validators))) + constructor = func(nodes types.NodeIDs) *totalOrdering { + return newTotalOrdering(1, phi, uint32(len(nodes))) } s.baseTestRandomlyGeneratedBlocks( - constructor, validatorCount, blockCount, gen, repeat) + constructor, nodeCount, blockCount, gen, repeat) // Test for K=2, - constructor = func(validators types.ValidatorIDs) *totalOrdering { - return newTotalOrdering(2, phi, uint32(len(validators))) + constructor = func(nodes types.NodeIDs) *totalOrdering { + return newTotalOrdering(2, phi, uint32(len(nodes))) } s.baseTestRandomlyGeneratedBlocks( - constructor, validatorCount, blockCount, gen, repeat) + constructor, nodeCount, blockCount, gen, repeat) // Test for K=3, - constructor = func(validators types.ValidatorIDs) *totalOrdering { - return newTotalOrdering(3, phi, uint32(len(validators))) + constructor = func(nodes types.NodeIDs) *totalOrdering { + return newTotalOrdering(3, phi, uint32(len(nodes))) } s.baseTestRandomlyGeneratedBlocks( - constructor, validatorCount, blockCount, gen, repeat) + constructor, nodeCount, blockCount, gen, repeat) } } diff --git a/core/types/block.go b/core/types/block.go index 61cb458..949876f 100644 --- a/core/types/block.go +++ b/core/types/block.go @@ -54,7 +54,7 @@ func NewBlock() (b *Block) { // Block represents a single event broadcasted on the network. type Block struct { - ProposerID ValidatorID `json:"proposer_id"` + ProposerID NodeID `json:"proposer_id"` ParentHash common.Hash `json:"parent_hash"` Hash common.Hash `json:"hash"` Position Position `json:"position"` diff --git a/core/types/dkg.go b/core/types/dkg.go index 7fb686c..78f4da6 100644 --- a/core/types/dkg.go +++ b/core/types/dkg.go @@ -24,7 +24,7 @@ import ( // DKGPrivateShare describe a secret share in DKG protocol. type DKGPrivateShare struct { - ProposerID ValidatorID `json:"proposer_id"` + ProposerID NodeID `json:"proposer_id"` Round uint64 `json:"round"` PrivateShare dkg.PrivateKey `json:"private_share"` Signature crypto.Signature `json:"signature"` @@ -32,7 +32,7 @@ type DKGPrivateShare struct { // DKGMasterPublicKey decrtibe a master public key in DKG protocol. type DKGMasterPublicKey struct { - ProposerID ValidatorID `json:"proposer_id"` + ProposerID NodeID `json:"proposer_id"` Round uint64 `json:"round"` DKGID dkg.ID `json:"dkg_id"` PublicKeyShares dkg.PublicKeyShares `json:"public_key_shares"` @@ -41,7 +41,7 @@ type DKGMasterPublicKey struct { // DKGComplaint describe a complaint in DKG protocol. type DKGComplaint struct { - ProposerID ValidatorID `json:"proposer_id"` + ProposerID NodeID `json:"proposer_id"` Round uint64 `json:"round"` PrivateShare DKGPrivateShare `json:"private_share"` Signature crypto.Signature `json:"signature"` @@ -49,7 +49,7 @@ type DKGComplaint struct { // DKGPartialSignature describe a partial signature in DKG protocol. type DKGPartialSignature struct { - ProposerID ValidatorID `json:"proposerID"` + ProposerID NodeID `json:"proposerID"` Round uint64 `json:"round"` PartialSignature dkg.PartialSignature `json:"partial_signature"` Signature crypto.Signature `json:"signature"` diff --git a/core/types/node.go b/core/types/node.go new file mode 100644 index 0000000..8a856de --- /dev/null +++ b/core/types/node.go @@ -0,0 +1,56 @@ +// Copyright 2018 The dexon-consensus-core Authors +// This file is part of the dexon-consensus-core library. +// +// The dexon-consensus-core library is free software: you can redistribute it +// and/or modify it under the terms of the GNU Lesser General Public License as +// published by the Free Software Foundation, either version 3 of the License, +// or (at your option) any later version. +// +// The dexon-consensus-core library is distributed in the hope that it will be +// useful, but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU Lesser +// General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the dexon-consensus-core library. If not, see +// . + +package types + +import ( + "bytes" + + "github.com/dexon-foundation/dexon-consensus-core/common" + "github.com/dexon-foundation/dexon-consensus-core/crypto" +) + +// NodeID is the ID type for nodes. +type NodeID struct { + common.Hash +} + +// NewNodeID returns a NodeID with Hash set to the hash value of +// public key. +func NewNodeID(pubKey crypto.PublicKey) NodeID { + return NodeID{Hash: crypto.Keccak256Hash(pubKey.Bytes())} +} + +// Equal checks if the hash representation is the same NodeID. +func (v NodeID) Equal(hash common.Hash) bool { + return v.Hash == hash +} + +// NodeIDs implements sort.Interface for NodeID. +type NodeIDs []NodeID + +func (v NodeIDs) Len() int { + return len(v) +} + +func (v NodeIDs) Less(i int, j int) bool { + return bytes.Compare([]byte(v[i].Hash[:]), []byte(v[j].Hash[:])) == -1 +} + +func (v NodeIDs) Swap(i int, j int) { + v[i], v[j] = v[j], v[i] +} diff --git a/core/types/validator.go b/core/types/validator.go deleted file mode 100644 index 5151a6d..0000000 --- a/core/types/validator.go +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright 2018 The dexon-consensus-core Authors -// This file is part of the dexon-consensus-core library. -// -// The dexon-consensus-core library is free software: you can redistribute it -// and/or modify it under the terms of the GNU Lesser General Public License as -// published by the Free Software Foundation, either version 3 of the License, -// or (at your option) any later version. -// -// The dexon-consensus-core library is distributed in the hope that it will be -// useful, but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU Lesser -// General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the dexon-consensus-core library. If not, see -// . - -package types - -import ( - "bytes" - - "github.com/dexon-foundation/dexon-consensus-core/common" - "github.com/dexon-foundation/dexon-consensus-core/crypto" -) - -// ValidatorID is the ID type for validators. -type ValidatorID struct { - common.Hash -} - -// NewValidatorID returns a ValidatorID with Hash set to the hash value of -// public key. -func NewValidatorID(pubKey crypto.PublicKey) ValidatorID { - return ValidatorID{Hash: crypto.Keccak256Hash(pubKey.Bytes())} -} - -// Equal checks if the hash representation is the same ValidatorID. -func (v ValidatorID) Equal(hash common.Hash) bool { - return v.Hash == hash -} - -// ValidatorIDs implements sort.Interface for ValidatorID. -type ValidatorIDs []ValidatorID - -func (v ValidatorIDs) Len() int { - return len(v) -} - -func (v ValidatorIDs) Less(i int, j int) bool { - return bytes.Compare([]byte(v[i].Hash[:]), []byte(v[j].Hash[:])) == -1 -} - -func (v ValidatorIDs) Swap(i int, j int) { - v[i], v[j] = v[j], v[i] -} diff --git a/core/types/vote.go b/core/types/vote.go index bae8f7d..e92aa67 100644 --- a/core/types/vote.go +++ b/core/types/vote.go @@ -38,7 +38,7 @@ const ( // Vote is the vote structure defined in Crypto Shuffle Algorithm. type Vote struct { - ProposerID ValidatorID `json:"proposer_id"` + ProposerID NodeID `json:"proposer_id"` Type VoteType `json:"type"` BlockHash common.Hash `json:"block_hash"` Period uint64 `json:"period"` diff --git a/core/types/witness.go b/core/types/witness.go index 1e432f6..349c1ab 100644 --- a/core/types/witness.go +++ b/core/types/witness.go @@ -28,7 +28,7 @@ import ( // WitnessAck represents the acking to the compaction chain. type WitnessAck struct { - ProposerID ValidatorID `json:"proposer_id"` + ProposerID NodeID `json:"proposer_id"` WitnessBlockHash common.Hash `json:"witness_block_hash"` Hash common.Hash `json:"hash"` // WitnessSignature is the signature of the hash value of BlockWitness. -- cgit v1.2.3