diff options
Diffstat (limited to 'core/test')
-rw-r--r-- | core/test/governance.go | 6 | ||||
-rw-r--r-- | core/test/governance_test.go | 16 | ||||
-rw-r--r-- | core/test/marshaller.go | 15 | ||||
-rw-r--r-- | core/test/network.go | 272 | ||||
-rw-r--r-- | core/test/network_test.go | 20 | ||||
-rw-r--r-- | core/test/state-change-request.go | 5 | ||||
-rw-r--r-- | core/test/state-change-request_test.go | 2 | ||||
-rw-r--r-- | core/test/state.go | 11 | ||||
-rw-r--r-- | core/test/state_test.go | 3 | ||||
-rw-r--r-- | core/test/tcp-transport.go | 2 | ||||
-rw-r--r-- | core/test/utils.go | 8 |
11 files changed, 85 insertions, 275 deletions
diff --git a/core/test/governance.go b/core/test/governance.go index e256504..538d064 100644 --- a/core/test/governance.go +++ b/core/test/governance.go @@ -231,7 +231,7 @@ func (g *Governance) IsDKGMPKReady(round uint64) bool { if round >= uint64(len(g.configs)) { return false } - return g.stateModule.IsDKGMPKReady(round, int(g.configs[round].DKGSetSize)/3*2) + return g.stateModule.IsDKGMPKReady(round, int(g.configs[round].NotarySetSize)/3*2) } // AddDKGFinalize adds a DKG finalize message. @@ -259,7 +259,7 @@ func (g *Governance) IsDKGFinal(round uint64) bool { if round >= uint64(len(g.configs)) { return false } - return g.stateModule.IsDKGFinal(round, int(g.configs[round].DKGSetSize)/3*2) + return g.stateModule.IsDKGFinal(round, int(g.configs[round].NotarySetSize)/3*2) } // ReportForkVote reports a node for forking votes. @@ -449,7 +449,7 @@ func (g *Governance) Equal(other *Governance, checkState bool) bool { // NOTE: this function should be called before running. func (g *Governance) RegisterConfigChange( round uint64, t StateChangeType, v interface{}) (err error) { - if t < StateAddCRS || t > StateChangeDKGSetSize { + if t < StateAddCRS || t > StateChangeNotarySetSize { return fmt.Errorf("state changes to register is not supported: %v", t) } if round < 2 { diff --git a/core/test/governance_test.go b/core/test/governance_test.go index b64f785..474ec80 100644 --- a/core/test/governance_test.go +++ b/core/test/governance_test.go @@ -92,16 +92,16 @@ func (s *GovernanceTestSuite) TestRegisterChange() { req.NoError(g.State().RequestChange(StateChangeRoundLength, uint64(roundLength))) // Unable to register change for genesis round. - req.Error(g.RegisterConfigChange(0, StateChangeDKGSetSize, uint32(32))) + req.Error(g.RegisterConfigChange(0, StateChangeNotarySetSize, uint32(32))) // Make some round prepared. g.CatchUpWithRound(4) - req.Equal(g.Configuration(4).DKGSetSize, uint32(20)) + req.Equal(g.Configuration(4).NotarySetSize, uint32(20)) // Unable to register change for prepared round. - req.Error(g.RegisterConfigChange(4, StateChangeDKGSetSize, uint32(32))) + req.Error(g.RegisterConfigChange(4, StateChangeNotarySetSize, uint32(32))) // It's ok to make some change when condition is met. - req.NoError(g.RegisterConfigChange(5, StateChangeDKGSetSize, uint32(32))) - req.NoError(g.RegisterConfigChange(6, StateChangeDKGSetSize, uint32(32))) - req.NoError(g.RegisterConfigChange(7, StateChangeDKGSetSize, uint32(40))) + req.NoError(g.RegisterConfigChange(5, StateChangeNotarySetSize, uint32(32))) + req.NoError(g.RegisterConfigChange(6, StateChangeNotarySetSize, uint32(32))) + req.NoError(g.RegisterConfigChange(7, StateChangeNotarySetSize, uint32(40))) // In local mode, state for round 6 would be ready after notified with // round 2. g.NotifyRound(2, roundLength*2) @@ -111,8 +111,8 @@ func (s *GovernanceTestSuite) TestRegisterChange() { g.NotifyRound(4, roundLength*4) // Notify governance to take a snapshot for round 7's configuration. g.NotifyRound(5, roundLength*5) - req.Equal(g.Configuration(6).DKGSetSize, uint32(32)) - req.Equal(g.Configuration(7).DKGSetSize, uint32(40)) + req.Equal(g.Configuration(6).NotarySetSize, uint32(32)) + req.Equal(g.Configuration(7).NotarySetSize, uint32(40)) } func (s *GovernanceTestSuite) TestProhibit() { diff --git a/core/test/marshaller.go b/core/test/marshaller.go index 5f15e11..91a3057 100644 --- a/core/test/marshaller.go +++ b/core/test/marshaller.go @@ -53,14 +53,8 @@ func (m *DefaultMarshaller) Unmarshal( break } msg = vote - case "block-randomness-request": - request := &types.AgreementResult{} - if err = json.Unmarshal(payload, request); err != nil { - break - } - msg = request - case "block-randomness-result": - result := &types.BlockRandomnessResult{} + case "agreement-result": + result := &types.AgreementResult{} if err = json.Unmarshal(payload, result); err != nil { break } @@ -128,10 +122,7 @@ func (m *DefaultMarshaller) Marshal( msgType = "vote" payload, err = json.Marshal(msg) case *types.AgreementResult: - msgType = "block-randomness-request" - payload, err = json.Marshal(msg) - case *types.BlockRandomnessResult: - msgType = "block-randomness-result" + msgType = "agreement-result" payload, err = json.Marshal(msg) case *typesDKG.PrivateShare: msgType = "dkg-private-share" diff --git a/core/test/network.go b/core/test/network.go index 443a26c..6034fa6 100644 --- a/core/test/network.go +++ b/core/test/network.go @@ -39,6 +39,9 @@ const ( maxPullingPeerCount = 3 maxBlockCache = 1000 maxVoteCache = 128 + + // Gossiping parameter. + gossipAgreementResultPercent = 33 ) // NetworkType is the simulation network type. @@ -77,8 +80,6 @@ func (req *PullRequest) MarshalJSON() (b []byte, err error) { idAsBytes, err = json.Marshal(req.Identity.(common.Hashes)) case "vote": idAsBytes, err = json.Marshal(req.Identity.(types.Position)) - case "randomness": - idAsBytes, err = json.Marshal(req.Identity.(common.Hashes)) default: err = fmt.Errorf("unknown ID type for pull request: %v", req.Type) } @@ -117,12 +118,6 @@ func (req *PullRequest) UnmarshalJSON(data []byte) (err error) { break } ID = pos - case "randomness": - hashes := common.Hashes{} - if err = json.Unmarshal(rawReq.Identity, &hashes); err != nil { - break - } - ID = hashes default: err = fmt.Errorf("unknown pull request type: %v", rawReq.Type) } @@ -137,38 +132,30 @@ func (req *PullRequest) UnmarshalJSON(data []byte) (err error) { // Network implements core.Network interface based on TransportClient. type Network struct { - ID types.NodeID - config NetworkConfig - ctx context.Context - ctxCancel context.CancelFunc - trans TransportClient - dMoment time.Time - fromTransport <-chan *TransportEnvelope - toConsensus chan interface{} - toNode chan interface{} - sentRandomnessLock sync.Mutex - sentRandomness map[common.Hash]struct{} - sentAgreementLock sync.Mutex - sentAgreement map[common.Hash]struct{} - blockCacheLock sync.RWMutex - blockCache map[common.Hash]*types.Block - voteCacheLock sync.RWMutex - voteCache map[types.Position]map[types.VoteHeader]*types.Vote - voteCacheSize int - votePositions []types.Position - randomnessCacheLock sync.RWMutex - randomnessCache map[common.Hash]*types.BlockRandomnessResult - stateModule *State - peers map[types.NodeID]struct{} - unreceivedBlocksLock sync.RWMutex - unreceivedBlocks map[common.Hash]chan<- common.Hash - unreceivedRandomnessLock sync.RWMutex - unreceivedRandomness map[common.Hash]chan<- common.Hash - cache *utils.NodeSetCache - notarySetCachesLock sync.Mutex - notarySetCaches map[uint64]map[types.NodeID]struct{} - dkgSetCachesLock sync.Mutex - dkgSetCaches map[uint64]map[types.NodeID]struct{} + ID types.NodeID + config NetworkConfig + ctx context.Context + ctxCancel context.CancelFunc + trans TransportClient + dMoment time.Time + fromTransport <-chan *TransportEnvelope + toConsensus chan interface{} + toNode chan interface{} + sentAgreementLock sync.Mutex + sentAgreement map[common.Hash]struct{} + blockCacheLock sync.RWMutex + blockCache map[common.Hash]*types.Block + voteCacheLock sync.RWMutex + voteCache map[types.Position]map[types.VoteHeader]*types.Vote + voteCacheSize int + votePositions []types.Position + stateModule *State + peers map[types.NodeID]struct{} + unreceivedBlocksLock sync.RWMutex + unreceivedBlocks map[common.Hash]chan<- common.Hash + cache *utils.NodeSetCache + notarySetCachesLock sync.Mutex + notarySetCaches map[uint64]map[types.NodeID]struct{} } // NewNetwork setup network stuffs for nodes, which provides an @@ -177,19 +164,15 @@ func NewNetwork(pubKey crypto.PublicKey, config NetworkConfig) ( n *Network) { // Construct basic network instance. n = &Network{ - ID: types.NewNodeID(pubKey), - config: config, - toConsensus: make(chan interface{}, 1000), - toNode: make(chan interface{}, 1000), - sentRandomness: make(map[common.Hash]struct{}), - sentAgreement: make(map[common.Hash]struct{}), - blockCache: make(map[common.Hash]*types.Block, maxBlockCache), - randomnessCache: make(map[common.Hash]*types.BlockRandomnessResult), - unreceivedBlocks: make(map[common.Hash]chan<- common.Hash), - unreceivedRandomness: make(map[common.Hash]chan<- common.Hash), - peers: make(map[types.NodeID]struct{}), - notarySetCaches: make(map[uint64]map[types.NodeID]struct{}), - dkgSetCaches: make(map[uint64]map[types.NodeID]struct{}), + ID: types.NewNodeID(pubKey), + config: config, + toConsensus: make(chan interface{}, 1000), + toNode: make(chan interface{}, 1000), + sentAgreement: make(map[common.Hash]struct{}), + blockCache: make(map[common.Hash]*types.Block, maxBlockCache), + unreceivedBlocks: make(map[common.Hash]chan<- common.Hash), + peers: make(map[types.NodeID]struct{}), + notarySetCaches: make(map[uint64]map[types.NodeID]struct{}), voteCache: make( map[types.Position]map[types.VoteHeader]*types.Vote), } @@ -218,11 +201,6 @@ func (n *Network) PullVotes(pos types.Position) { go n.pullVotesAsync(pos) } -// PullRandomness implememnts core.Network interface. -func (n *Network) PullRandomness(hashes common.Hashes) { - go n.pullRandomnessAsync(hashes) -} - // BroadcastVote implements core.Network interface. func (n *Network) BroadcastVote(vote *types.Vote) { if err := n.trans.Broadcast(n.getNotarySet(vote.Position.Round), @@ -254,37 +232,22 @@ func (n *Network) BroadcastAgreementResult( if !n.markAgreementResultAsSent(result.BlockHash) { return } - // Send to DKG set first. - dkgSet := n.getDKGSet(result.Position.Round) - if err := n.trans.Broadcast( - dkgSet, n.config.DirectLatency, result); err != nil { - panic(err) - } - // Gossip to other nodes. - if err := n.trans.Broadcast(getComplementSet(n.peers, dkgSet), - n.config.GossipLatency, result); err != nil { - panic(err) - } -} - -// BroadcastRandomnessResult implements core.Network interface. -func (n *Network) BroadcastRandomnessResult( - randResult *types.BlockRandomnessResult) { - if !n.markRandomnessResultAsSent(randResult.BlockHash) { - return - } - // Send to notary set first. - notarySet := n.getNotarySet(randResult.Position.Round) - if err := n.trans.Broadcast( - notarySet, n.config.DirectLatency, randResult); err != nil { - panic(err) + n.addBlockRandomnessToCache(result.BlockHash, result.Randomness) + notarySet := n.getNotarySet(result.Position.Round) + count := len(notarySet) * gossipAgreementResultPercent / 100 + for nID := range notarySet { + if count--; count < 0 { + break + } + if err := n.trans.Send(nID, result); err != nil { + panic(err) + } } // Gossip to other nodes. if err := n.trans.Broadcast(getComplementSet(n.peers, notarySet), - n.config.GossipLatency, randResult); err != nil { + n.config.GossipLatency, result); err != nil { panic(err) } - n.addRandomnessToCache(randResult) } // SendDKGPrivateShare implements core.Network interface. @@ -296,7 +259,7 @@ func (n *Network) SendDKGPrivateShare( // BroadcastDKGPrivateShare implements core.Network interface. func (n *Network) BroadcastDKGPrivateShare( prvShare *typesDKG.PrivateShare) { - if err := n.trans.Broadcast(n.getDKGSet(prvShare.Round), + if err := n.trans.Broadcast(n.getNotarySet(prvShare.Round), n.config.DirectLatency, prvShare); err != nil { panic(err) } @@ -306,7 +269,7 @@ func (n *Network) BroadcastDKGPrivateShare( func (n *Network) BroadcastDKGPartialSignature( psig *typesDKG.PartialSignature) { if err := n.trans.Broadcast( - n.getDKGSet(psig.Round), n.config.DirectLatency, psig); err != nil { + n.getNotarySet(psig.Round), n.config.DirectLatency, psig); err != nil { panic(err) } } @@ -358,7 +321,7 @@ func (n *Network) dispatchMsg(e *TransportEnvelope) { // Add this vote to cache. n.addVoteToCache(v) n.toConsensus <- v - case *types.AgreementResult, *types.BlockRandomnessResult, + case *types.AgreementResult, *typesDKG.PrivateShare, *typesDKG.PartialSignature: n.toConsensus <- v case packedStateChanges: @@ -408,25 +371,6 @@ func (n *Network) handlePullRequest(req *PullRequest) { } } }() - case "randomness": - hashes := req.Identity.(common.Hashes) - func() { - n.randomnessCacheLock.Lock() - defer n.randomnessCacheLock.Unlock() - All: - for _, h := range hashes { - r, exists := n.randomnessCache[h] - if !exists { - continue - } - select { - case <-n.ctx.Done(): - break All - default: - } - n.send(req.Requester, r) - } - }() default: panic(fmt.Errorf("unknown type of pull request: %v", req.Type)) } @@ -582,57 +526,6 @@ func (n *Network) pullVotesAsync(pos types.Position) { } } -func (n *Network) pullRandomnessAsync(hashes common.Hashes) { - // Setup notification channels for each block hash. - notYetReceived := make(map[common.Hash]struct{}) - ch := make(chan common.Hash, len(hashes)) - func() { - n.unreceivedRandomnessLock.Lock() - defer n.unreceivedRandomnessLock.Unlock() - for _, h := range hashes { - if _, exists := n.unreceivedRandomness[h]; exists { - continue - } - n.unreceivedRandomness[h] = ch - notYetReceived[h] = struct{}{} - } - }() - req := &PullRequest{ - Requester: n.ID, - Type: "randomness", - Identity: hashes, - } - // Randomly pick peers to send pull requests. -Loop: - for nID := range n.peers { - if nID == n.ID { - continue - } - n.send(nID, req) - select { - case <-n.ctx.Done(): - break Loop - case <-time.After(2 * n.config.DirectLatency.Delay()): - // Consume everything in the notification channel. - for { - select { - case h, ok := <-ch: - if !ok { - // This network module is closed. - break Loop - } - delete(notYetReceived, h) - if len(notYetReceived) == 0 { - break Loop - } - default: - continue Loop - } - } - } - } -} - func (n *Network) addBlockToCache(b *types.Block) { n.blockCacheLock.Lock() defer n.blockCacheLock.Unlock() @@ -646,6 +539,16 @@ func (n *Network) addBlockToCache(b *types.Block) { n.blockCache[b.Hash] = b.Clone() } +func (n *Network) addBlockRandomnessToCache(hash common.Hash, rand []byte) { + n.blockCacheLock.Lock() + defer n.blockCacheLock.Unlock() + block, exist := n.blockCache[hash] + if !exist { + return + } + block.Finalization.Randomness = rand +} + func (n *Network) addVoteToCache(v *types.Vote) { n.voteCacheLock.Lock() defer n.voteCacheLock.Unlock() @@ -667,19 +570,6 @@ func (n *Network) addVoteToCache(v *types.Vote) { n.voteCacheSize++ } -func (n *Network) addRandomnessToCache(rand *types.BlockRandomnessResult) { - n.randomnessCacheLock.Lock() - defer n.randomnessCacheLock.Unlock() - if len(n.randomnessCache) > 1000 { - // Randomly purge one randomness from cache. - for k := range n.randomnessCache { - delete(n.randomnessCache, k) - break - } - } - n.randomnessCache[rand.BlockHash] = rand -} - func (n *Network) markAgreementResultAsSent(blockHash common.Hash) bool { n.sentAgreementLock.Lock() defer n.sentAgreementLock.Unlock() @@ -697,23 +587,6 @@ func (n *Network) markAgreementResultAsSent(blockHash common.Hash) bool { return true } -func (n *Network) markRandomnessResultAsSent(blockHash common.Hash) bool { - n.sentRandomnessLock.Lock() - defer n.sentRandomnessLock.Unlock() - if _, exist := n.sentRandomness[blockHash]; exist { - return false - } - if len(n.sentRandomness) > 1000 { - // Randomly drop one entry. - for k := range n.sentRandomness { - delete(n.sentRandomness, k) - break - } - } - n.sentRandomness[blockHash] = struct{}{} - return true -} - func (n *Network) cloneForFake(v interface{}) interface{} { if n.config.Type != NetworkTypeFake { return v @@ -721,9 +594,9 @@ func (n *Network) cloneForFake(v interface{}) interface{} { switch val := v.(type) { case *types.Block: return val.Clone() - case *types.BlockRandomnessResult: + case *types.AgreementResult: // Perform deep copy for randomness result. - return cloneBlockRandomnessResult(val) + return cloneAgreementResult(val) } return v } @@ -749,27 +622,6 @@ func (n *Network) getNotarySet(round uint64) map[types.NodeID]struct{} { return set } -// getDKGSet gets DKG set for that round from cache. -func (n *Network) getDKGSet(round uint64) map[types.NodeID]struct{} { - if n.cache == nil { - // Default behavior is to broadcast to all peers, which makes it easier - // to be used in simple test cases. - return n.peers - } - n.dkgSetCachesLock.Lock() - defer n.dkgSetCachesLock.Unlock() - set, exists := n.dkgSetCaches[round] - if !exists { - var err error - set, err = n.cache.GetDKGSet(round) - if err != nil { - panic(err) - } - n.dkgSetCaches[round] = set - } - return set -} - func (n *Network) send(endpoint types.NodeID, msg interface{}) { go func() { time.Sleep(n.config.DirectLatency.Delay()) diff --git a/core/test/network_test.go b/core/test/network_test.go index 63407d6..993ae70 100644 --- a/core/test/network_test.go +++ b/core/test/network_test.go @@ -251,43 +251,31 @@ func (s *NetworkTestSuite) TestBroadcastToSet() { gov, err := NewGovernance(NewState( 1, pubKeys, time.Second, &common.NullLogger{}, true), 2) req.NoError(err) - req.NoError(gov.State().RequestChange(StateChangeDKGSetSize, uint32(1))) req.NoError(gov.State().RequestChange(StateChangeNotarySetSize, uint32(1))) gov.NotifyRound(round, gov.Configuration(0).RoundLength) networks := s.setupNetworks(pubKeys) cache := utils.NewNodeSetCache(gov) // Cache required set of nodeIDs. - dkgSet, err := cache.GetDKGSet(round) - req.NoError(err) - req.Len(dkgSet, 1) notarySet, err := cache.GetNotarySet(round) req.NoError(err) req.Len(notarySet, 1) var ( // Some node don't belong to any set. - nerd *Network - dkgNode, notaryNode *Network + nerd *Network + notaryNode *Network ) for nID, n := range networks { - if _, exists := dkgSet[nID]; exists { - continue - } if _, exists := notarySet[nID]; exists { continue } nerd = n break } - for nID := range dkgSet { - dkgNode = networks[nID] - break - } for nID := range notarySet { notaryNode = networks[nID] break } req.NotNil(nerd) - req.NotNil(dkgNode) req.NotNil(notaryNode) nerd.AttachNodeSetCache(cache) // Try broadcasting with datum from round 0, and make sure only node belongs @@ -296,9 +284,9 @@ func (s *NetworkTestSuite) TestBroadcastToSet() { Position: types.Position{Round: round}}}) req.IsType(&types.Vote{}, <-notaryNode.ReceiveChan()) nerd.BroadcastDKGPrivateShare(&typesDKG.PrivateShare{Round: round}) - req.IsType(&typesDKG.PrivateShare{}, <-dkgNode.ReceiveChan()) + req.IsType(&typesDKG.PrivateShare{}, <-notaryNode.ReceiveChan()) nerd.BroadcastDKGPartialSignature(&typesDKG.PartialSignature{Round: round}) - req.IsType(&typesDKG.PartialSignature{}, <-dkgNode.ReceiveChan()) + req.IsType(&typesDKG.PartialSignature{}, <-notaryNode.ReceiveChan()) nerd.BroadcastBlock(&types.Block{Position: types.Position{Round: round}}) req.IsType(&types.Block{}, <-notaryNode.ReceiveChan()) } diff --git a/core/test/state-change-request.go b/core/test/state-change-request.go index b7c7bac..fe55d6e 100644 --- a/core/test/state-change-request.go +++ b/core/test/state-change-request.go @@ -47,7 +47,6 @@ const ( StateChangeRoundLength StateChangeMinBlockInterval StateChangeNotarySetSize - StateChangeDKGSetSize // Node set related. StateAddNode ) @@ -76,8 +75,6 @@ func (t StateChangeType) String() string { return "ChangeMinBlockInterval" case StateChangeNotarySetSize: return "ChangeNotarySetSize" - case StateChangeDKGSetSize: - return "ChangeDKGSetSize" case StateAddNode: return "AddNode" } @@ -195,8 +192,6 @@ func (req *StateChangeRequest) String() (ret string) { ret += fmt.Sprintf("%v", time.Duration(req.Payload.(uint64))) case StateChangeNotarySetSize: ret += fmt.Sprintf("%v", req.Payload.(uint32)) - case StateChangeDKGSetSize: - ret += fmt.Sprintf("%v", req.Payload.(uint32)) case StateAddNode: ret += fmt.Sprintf( "%s", types.NewNodeID(req.Payload.(crypto.PublicKey)).String()[:6]) diff --git a/core/test/state-change-request_test.go b/core/test/state-change-request_test.go index eeba4c4..517a929 100644 --- a/core/test/state-change-request_test.go +++ b/core/test/state-change-request_test.go @@ -42,7 +42,7 @@ func (s *StateChangeRequestTestSuite) TestEqual() { func (s *StateChangeRequestTestSuite) TestClone() { // The cloned one should be no error when compared with 'Equal' method. - st00 := NewStateChangeRequest(StateChangeDKGSetSize, uint32(7)) + st00 := NewStateChangeRequest(StateChangeNotarySetSize, uint32(7)) s.NoError(st00.Equal(st00.Clone())) st10 := NewStateChangeRequest( StateAddDKGMasterPublicKey, typesDKG.NewMasterPublicKey()) diff --git a/core/test/state.go b/core/test/state.go index ce906ae..41c6b38 100644 --- a/core/test/state.go +++ b/core/test/state.go @@ -94,7 +94,6 @@ type State struct { lambdaBA time.Duration lambdaDKG time.Duration notarySetSize uint32 - dkgSetSize uint32 roundInterval uint64 minBlockInterval time.Duration // Nodes @@ -145,7 +144,6 @@ func NewState( crs: crs, nodes: nodes, notarySetSize: uint32(len(nodes)), - dkgSetSize: uint32(len(nodes)), ownRequests: make(map[common.Hash]*StateChangeRequest), globalRequests: make(map[common.Hash]*StateChangeRequest), dkgReadys: make( @@ -183,7 +181,6 @@ func (s *State) Snapshot() (*types.Config, []crypto.PublicKey) { LambdaBA: s.lambdaBA, LambdaDKG: s.lambdaDKG, NotarySetSize: s.notarySetSize, - DKGSetSize: s.dkgSetSize, RoundLength: s.roundInterval, MinBlockInterval: s.minBlockInterval, } @@ -238,10 +235,6 @@ func (s *State) unpackPayload( var tmp uint32 err = rlp.DecodeBytes(raw.Payload, &tmp) v = tmp - case StateChangeDKGSetSize: - var tmp uint32 - err = rlp.DecodeBytes(raw.Payload, &tmp) - v = tmp case StateAddNode: var tmp []byte err = rlp.DecodeBytes(raw.Payload, &tmp) @@ -283,7 +276,6 @@ func (s *State) Equal(other *State) error { configEqual := s.lambdaBA == other.lambdaBA && s.lambdaDKG == other.lambdaDKG && s.notarySetSize == other.notarySetSize && - s.dkgSetSize == other.dkgSetSize && s.roundInterval == other.roundInterval && s.minBlockInterval == other.minBlockInterval if !configEqual { @@ -452,7 +444,6 @@ func (s *State) Clone() (copied *State) { lambdaBA: s.lambdaBA, lambdaDKG: s.lambdaDKG, notarySetSize: s.notarySetSize, - dkgSetSize: s.dkgSetSize, roundInterval: s.roundInterval, minBlockInterval: s.minBlockInterval, local: s.local, @@ -774,8 +765,6 @@ func (s *State) applyRequest(req *StateChangeRequest) error { s.minBlockInterval = time.Duration(req.Payload.(uint64)) case StateChangeNotarySetSize: s.notarySetSize = req.Payload.(uint32) - case StateChangeDKGSetSize: - s.dkgSetSize = req.Payload.(uint32) default: return errors.New("you are definitely kidding me") } diff --git a/core/test/state_test.go b/core/test/state_test.go index 0ec90a4..5ad5a3e 100644 --- a/core/test/state_test.go +++ b/core/test/state_test.go @@ -147,7 +147,6 @@ func (s *StateTestSuite) makeConfigChanges(st *State) { st.RequestChange(StateChangeRoundLength, uint64(1001)) st.RequestChange(StateChangeMinBlockInterval, time.Second) st.RequestChange(StateChangeNotarySetSize, uint32(5)) - st.RequestChange(StateChangeDKGSetSize, uint32(6)) } func (s *StateTestSuite) checkConfigChanges(config *types.Config) { @@ -157,7 +156,6 @@ func (s *StateTestSuite) checkConfigChanges(config *types.Config) { req.Equal(config.RoundLength, uint64(1001)) req.Equal(config.MinBlockInterval, time.Second) req.Equal(config.NotarySetSize, uint32(5)) - req.Equal(config.DKGSetSize, uint32(6)) } func (s *StateTestSuite) TestEqual() { @@ -265,7 +263,6 @@ func (s *StateTestSuite) TestLocalMode() { req.Equal(config1.LambdaDKG, lambda*10) req.Equal(config1.RoundLength, uint64(1000)) req.Equal(config1.NotarySetSize, uint32(len(genesisNodes))) - req.Equal(config1.DKGSetSize, uint32(len(genesisNodes))) // Request some changes, every fields for config should be affected. s.makeConfigChanges(st) // Add new node. diff --git a/core/test/tcp-transport.go b/core/test/tcp-transport.go index 19d9bfa..bbc4d56 100644 --- a/core/test/tcp-transport.go +++ b/core/test/tcp-transport.go @@ -896,8 +896,6 @@ func (t *TCPTransport) handleThroughputData(msg interface{}, payload []byte) { recordType = "agreement_result" case *dkg.PartialSignature: recordType = "partial_sig" - case *types.BlockRandomnessResult: - recordType = "block_random" } if len(recordType) > 0 { t.throughputRecords = append(t.throughputRecords, ThroughputRecord{ diff --git a/core/test/utils.go b/core/test/utils.go index a2819ae..74cde45 100644 --- a/core/test/utils.go +++ b/core/test/utils.go @@ -184,13 +184,13 @@ func CloneDKGPrivateShare(prvShare *typesDKG.PrivateShare) ( return } -func cloneBlockRandomnessResult(rand *types.BlockRandomnessResult) ( - copied *types.BlockRandomnessResult) { - b, err := rlp.EncodeToBytes(rand) +func cloneAgreementResult(result *types.AgreementResult) ( + copied *types.AgreementResult) { + b, err := rlp.EncodeToBytes(result) if err != nil { panic(err) } - copied = &types.BlockRandomnessResult{} + copied = &types.AgreementResult{} if err = rlp.DecodeBytes(b, copied); err != nil { panic(err) } |