diff options
author | Mission Liao <mission.liao@dexon.org> | 2018-11-08 15:58:51 +0800 |
---|---|---|
committer | GitHub <noreply@github.com> | 2018-11-08 15:58:51 +0800 |
commit | 1ee5863fd4a295d34c3a2d602d5603e8746e3f7b (patch) | |
tree | 044308b22000bb0c9f5a8c3c21f465159418db24 | |
parent | dbe83ea4a324941417d6ff09230e5874d5ba5df5 (diff) | |
download | tangerine-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar.gz tangerine-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar.zst tangerine-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.zip |
simulation: use test.Governacne in simulation (#311)
* Move simulation.Network to test package
* Use test.Governance in simulation
* Pack/Apply state request in blocks payload
* Add Governance.SwitchToRemoteMode
This would trigger governance to broadcast
pending state change requests when changes.
* Allow to marshal/unmarshal packedStateChanges
* Attach test.Network and test.State
-rw-r--r-- | core/consensus.go | 3 | ||||
-rw-r--r-- | core/lattice.go | 2 | ||||
-rw-r--r-- | core/test/governance.go | 75 | ||||
-rw-r--r-- | core/test/marshaller.go | 9 | ||||
-rw-r--r-- | core/test/network.go | 67 | ||||
-rw-r--r-- | core/test/state.go | 8 | ||||
-rw-r--r-- | core/test/state_test.go | 3 | ||||
-rw-r--r-- | simulation/app.go | 39 | ||||
-rw-r--r-- | simulation/app_test.go | 2 | ||||
-rw-r--r-- | simulation/governance.go | 227 | ||||
-rw-r--r-- | simulation/node.go | 62 | ||||
-rw-r--r-- | test_config/test.toml | 8 |
12 files changed, 185 insertions, 320 deletions
diff --git a/core/consensus.go b/core/consensus.go index b9da4f0..a449701 100644 --- a/core/consensus.go +++ b/core/consensus.go @@ -660,7 +660,8 @@ func (con *Consensus) initialRound( // unexpected network fluctuation and ensure the robustness. for (con.gov.CRS(nextRound) == common.Hash{}) { con.logger.Info("CRS is not ready yet. Try again later...", - "nodeID", con.ID) + "nodeID", con.ID, + "round", nextRound) time.Sleep(500 * time.Millisecond) } nextDkgSet, err := con.nodeSetCache.GetDKGSet(nextRound) diff --git a/core/lattice.go b/core/lattice.go index 20e16f2..dcb3368 100644 --- a/core/lattice.go +++ b/core/lattice.go @@ -81,7 +81,7 @@ func (l *Lattice) PrepareBlock( if err = l.data.prepareBlock(b); err != nil { return } - l.logger.Debug("Calling Application.PreparePayload", "position", b.Position) + l.logger.Debug("Calling Application.PreparePayload", "position", &b.Position) if b.Payload, err = l.app.PreparePayload(b.Position); err != nil { return } diff --git a/core/test/governance.go b/core/test/governance.go index 42e0da1..e6b6232 100644 --- a/core/test/governance.go +++ b/core/test/governance.go @@ -37,7 +37,8 @@ import ( type Governance struct { configs []*types.Config nodeSets [][]crypto.PublicKey - state *State + stateModule *State + networkModule *Network pendingConfigChanges map[uint64]map[StateChangeType]interface{} lock sync.RWMutex } @@ -51,7 +52,7 @@ func NewGovernance(genesisNodes []crypto.PublicKey, // modification smaller. g = &Governance{ pendingConfigChanges: make(map[uint64]map[StateChangeType]interface{}), - state: NewState(genesisNodes, lambda, true), + stateModule: NewState(genesisNodes, lambda, true), } return } @@ -89,7 +90,7 @@ func (g *Governance) Configuration(round uint64) *types.Config { // CRS returns the CRS for a given round. func (g *Governance) CRS(round uint64) common.Hash { - return g.state.CRS(round) + return g.stateModule.CRS(round) } // NotifyRoundHeight notifies governace contract to snapshot config. @@ -100,11 +101,12 @@ func (g *Governance) NotifyRoundHeight(round, height uint64) { g.lock.Lock() defer g.lock.Unlock() for t, v := range g.pendingConfigChanges[round+1] { - if err := g.state.RequestChange(t, v); err != nil { + if err := g.stateModule.RequestChange(t, v); err != nil { panic(err) } } delete(g.pendingConfigChanges, round+1) + g.broadcastPendingStateChanges() }() } @@ -113,13 +115,15 @@ func (g *Governance) ProposeCRS(round uint64, signedCRS []byte) { g.lock.Lock() defer g.lock.Unlock() crs := crypto.Keccak256Hash(signedCRS) - if err := g.state.ProposeCRS(round, crs); err != nil { + if err := g.stateModule.ProposeCRS(round, crs); err != nil { // CRS can be proposed multiple times, other errors are not // accepted. if err != ErrDuplicatedChange { panic(err) } + return } + g.broadcastPendingStateChanges() } // AddDKGComplaint add a DKGComplaint. @@ -131,12 +135,16 @@ func (g *Governance) AddDKGComplaint( if g.IsDKGFinal(complaint.Round) { return } - g.state.RequestChange(StateAddDKGComplaint, complaint) + if err := g.stateModule.RequestChange( + StateAddDKGComplaint, complaint); err != nil { + panic(err) + } + g.broadcastPendingStateChanges() } // DKGComplaints returns the DKGComplaints of round. func (g *Governance) DKGComplaints(round uint64) []*typesDKG.Complaint { - return g.state.DKGComplaints(round) + return g.stateModule.DKGComplaints(round) } // AddDKGMasterPublicKey adds a DKGMasterPublicKey. @@ -145,13 +153,17 @@ func (g *Governance) AddDKGMasterPublicKey( if round != masterPublicKey.Round { return } - g.state.RequestChange(StateAddDKGMasterPublicKey, masterPublicKey) + if err := g.stateModule.RequestChange( + StateAddDKGMasterPublicKey, masterPublicKey); err != nil { + panic(err) + } + g.broadcastPendingStateChanges() } // DKGMasterPublicKeys returns the DKGMasterPublicKeys of round. func (g *Governance) DKGMasterPublicKeys( round uint64) []*typesDKG.MasterPublicKey { - return g.state.DKGMasterPublicKeys(round) + return g.stateModule.DKGMasterPublicKeys(round) } // AddDKGFinalize adds a DKG finalize message. @@ -159,7 +171,10 @@ func (g *Governance) AddDKGFinalize(round uint64, final *typesDKG.Finalize) { if round != final.Round { return } - g.state.RequestChange(StateAddDKGFinal, final) + if err := g.stateModule.RequestChange(StateAddDKGFinal, final); err != nil { + panic(err) + } + g.broadcastPendingStateChanges() } // IsDKGFinal checks if DKG is final. @@ -174,16 +189,31 @@ func (g *Governance) IsDKGFinal(round uint64) bool { if round >= uint64(len(g.configs)) { return false } - return g.state.IsDKGFinal(round, int(g.configs[round].DKGSetSize)/3*2) + return g.stateModule.IsDKGFinal(round, int(g.configs[round].DKGSetSize)/3*2) } // // Test Utilities // +type packedStateChanges []byte + +// This method broadcasts pending state change requests in the underlying +// State instance, this behavior is to simulate tx-gossiping in full nodes. +func (g *Governance) broadcastPendingStateChanges() { + if g.networkModule == nil { + return + } + packed, err := g.stateModule.PackOwnRequests() + if err != nil { + panic(err) + } + g.networkModule.Broadcast(packedStateChanges(packed)) +} + // State allows to access embed State instance. func (g *Governance) State() *State { - return g.state + return g.stateModule } // CatchUpWithRound attempts to perform state snapshot to @@ -199,7 +229,7 @@ func (g *Governance) CatchUpWithRound(round uint64) { g.lock.Lock() defer g.lock.Unlock() for uint64(len(g.configs)) <= round { - config, nodeSet := g.state.Snapshot() + config, nodeSet := g.stateModule.Snapshot() g.configs = append(g.configs, config) g.nodeSets = append(g.nodeSets, nodeSet) } @@ -210,7 +240,7 @@ func (g *Governance) Clone() *Governance { g.lock.RLock() defer g.lock.RUnlock() // Clone state. - copiedState := g.state.Clone() + copiedState := g.stateModule.Clone() // Clone configs. copiedConfigs := []*types.Config{} for _, c := range g.configs { @@ -239,10 +269,9 @@ func (g *Governance) Clone() *Governance { copiedNodeSets = append(copiedNodeSets, copiedNodeSet) } // Clone pending changes. - return &Governance{ configs: copiedConfigs, - state: copiedState, + stateModule: copiedState, nodeSets: copiedNodeSets, pendingConfigChanges: copiedPendingChanges, } @@ -286,7 +315,7 @@ func (g *Governance) Equal(other *Governance, checkState bool) bool { // different state, only the snapshots (configs and node sets) are // essentially equal. if checkState { - return g.state.Equal(other.state) == nil + return g.stateModule.Equal(other.stateModule) == nil } return true } @@ -319,3 +348,15 @@ func (g *Governance) RegisterConfigChange( pendingChangesForRound[t] = v return nil } + +// SwitchToRemoteMode would switch this governance instance to remote mode, +// which means: it will broadcast all changes from its underlying state +// instance. +func (g *Governance) SwitchToRemoteMode(n *Network) { + if g.networkModule != nil { + panic(errors.New("not in local mode before switching")) + } + g.stateModule.SwitchToRemoteMode() + g.networkModule = n + n.addStateModule(g.stateModule) +} diff --git a/core/test/marshaller.go b/core/test/marshaller.go index fc42639..a1b15b6 100644 --- a/core/test/marshaller.go +++ b/core/test/marshaller.go @@ -95,6 +95,12 @@ func (m *DefaultMarshaller) Unmarshal( break } msg = final + case "packed-state-changes": + packed := &packedStateChanges{} + if err = json.Unmarshal(payload, packed); err != nil { + break + } + msg = *packed default: if m.fallback == nil { err = fmt.Errorf("unknown msg type: %v", msgType) @@ -136,6 +142,9 @@ func (m *DefaultMarshaller) Marshal( case *typesDKG.Finalize: msgType = "dkg-finalize" payload, err = json.Marshal(msg) + case packedStateChanges: + msgType = "packed-state-changes" + payload, err = json.Marshal(msg) default: if m.fallback == nil { err = fmt.Errorf("unknwon message type: %v", msg) diff --git a/core/test/network.go b/core/test/network.go index 00c60d9..0d92af0 100644 --- a/core/test/network.go +++ b/core/test/network.go @@ -19,9 +19,11 @@ package test import ( "context" + "errors" "fmt" "net" "strconv" + "sync" "github.com/dexon-foundation/dexon-consensus/common" "github.com/dexon-foundation/dexon-consensus/core/crypto" @@ -48,16 +50,20 @@ type NetworkConfig struct { // Network implements core.Network interface based on TransportClient. type Network struct { - config NetworkConfig - ctx context.Context - ctxCancel context.CancelFunc - trans TransportClient - fromTransport <-chan *TransportEnvelope - toConsensus chan interface{} - toNode chan interface{} - sentRandomness map[common.Hash]struct{} - sentAgreement map[common.Hash]struct{} - blockCache map[common.Hash]*types.Block + config NetworkConfig + ctx context.Context + ctxCancel context.CancelFunc + trans TransportClient + 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 + stateModule *State } // NewNetwork setup network stuffs for nodes, which provides an @@ -91,6 +97,8 @@ func NewNetwork(pubKey crypto.PublicKey, latency LatencyModel, // PullBlocks implements core.Network interface. func (n *Network) PullBlocks(hashes common.Hashes) { go func() { + n.blockCacheLock.RLock() + defer n.blockCacheLock.RUnlock() for _, hash := range hashes { // TODO(jimmy-dexon): request block from network instead of cache. if block, exist := n.blockCache[hash]; exist { @@ -124,6 +132,8 @@ func (n *Network) BroadcastBlock(block *types.Block) { // BroadcastAgreementResult implements core.Network interface. func (n *Network) BroadcastAgreementResult( randRequest *types.AgreementResult) { + n.sentAgreementLock.Lock() + defer n.sentAgreementLock.Unlock() if _, exist := n.sentAgreement[randRequest.BlockHash]; exist { return } @@ -143,6 +153,8 @@ func (n *Network) BroadcastAgreementResult( // BroadcastRandomnessResult implements core.Network interface. func (n *Network) BroadcastRandomnessResult( randResult *types.BlockRandomnessResult) { + n.sentRandomnessLock.Lock() + defer n.sentRandomnessLock.Unlock() if _, exist := n.sentRandomness[randResult.BlockHash]; exist { return } @@ -214,21 +226,33 @@ func (n *Network) Setup(serverEndpoint interface{}) (err error) { return } -func (n *Network) msgHandler(e *TransportEnvelope) { +func (n *Network) dispatchMsg(e *TransportEnvelope) { switch v := e.Msg.(type) { case *types.Block: - if len(n.blockCache) > 500 { - // Randomly purge one block from cache. - for k := range n.blockCache { - delete(n.blockCache, k) - break + func() { + n.blockCacheLock.Lock() + defer n.blockCacheLock.Unlock() + if len(n.blockCache) > 500 { + // Randomly purge one block from cache. + for k := range n.blockCache { + delete(n.blockCache, k) + break + } } - } - n.blockCache[v.Hash] = v + n.blockCache[v.Hash] = v + }() n.toConsensus <- e.Msg case *types.Vote, *types.AgreementResult, *types.BlockRandomnessResult, *typesDKG.PrivateShare, *typesDKG.PartialSignature: n.toConsensus <- e.Msg + case packedStateChanges: + if n.stateModule == nil { + panic(errors.New( + "receive packed state change request without state attached")) + } + if err := n.stateModule.AddRequestsFromOthers([]byte(v)); err != nil { + panic(err) + } default: n.toNode <- e.Msg } @@ -250,7 +274,7 @@ Loop: if !ok { break Loop } - n.msgHandler(e) + n.dispatchMsg(e) } } } @@ -291,3 +315,8 @@ func (n *Network) Broadcast(msg interface{}) { func (n *Network) ReceiveChanForNode() <-chan interface{} { return n.toNode } + +// addStateModule attaches a State instance to this network. +func (n *Network) addStateModule(s *State) { + n.stateModule = s +} diff --git a/core/test/state.go b/core/test/state.go index 2bb04e5..630c43f 100644 --- a/core/test/state.go +++ b/core/test/state.go @@ -518,9 +518,11 @@ func (s *State) AddRequestsFromOthers(reqsAsBytes []byte) (err error) { // PackRequests pack all current pending requests, include those from others. func (s *State) PackRequests() (b []byte, err error) { - // Convert own requests to global one for packing. - if _, err = s.PackOwnRequests(); err != nil { - return + if s.local { + // Convert own requests to global one for packing. + if _, err = s.PackOwnRequests(); err != nil { + return + } } // Pack requests in global pool. packed := []*StateChangeRequest{} diff --git a/core/test/state_test.go b/core/test/state_test.go index 6c5f882..864b5be 100644 --- a/core/test/state_test.go +++ b/core/test/state_test.go @@ -315,6 +315,9 @@ func (s *StateTestSuite) TestPacking() { req.Empty(st.DKGMasterPublicKeys(2)) req.Empty(st.DKGComplaints(2)) req.False(st.IsDKGFinal(2, 0)) + // In remote mode, we need to manually convert own requests to global ones. + _, err = st.PackOwnRequests() + req.NoError(err) // Pack changes into bytes. b, err := st.PackRequests() req.NoError(err) diff --git a/simulation/app.go b/simulation/app.go index ea45cc9..9195619 100644 --- a/simulation/app.go +++ b/simulation/app.go @@ -46,11 +46,12 @@ type timestampMessage struct { // simApp is an DEXON app for simulation. type simApp struct { - NodeID types.NodeID - Outputs []*types.Block - Early bool - netModule *test.Network - DeliverID int + NodeID types.NodeID + Outputs []*types.Block + Early bool + netModule *test.Network + stateModule *test.State + DeliverID int // blockSeen stores the time when block is delivered by Total Ordering. blockSeen map[common.Hash]time.Time // uncofirmBlocks stores the blocks whose timestamps are not ready. @@ -62,10 +63,12 @@ type simApp struct { } // newSimApp returns point to a new instance of simApp. -func newSimApp(id types.NodeID, netModule *test.Network) *simApp { +func newSimApp( + id types.NodeID, netModule *test.Network, stateModule *test.State) *simApp { return &simApp{ NodeID: id, netModule: netModule, + stateModule: stateModule, DeliverID: 0, blockSeen: make(map[common.Hash]time.Time), unconfirmedBlocks: make(map[types.NodeID]common.Hashes), @@ -116,7 +119,7 @@ func (a *simApp) getAckedBlocks(ackHash common.Hash) (output common.Hashes) { // PreparePayload implements core.Application. func (a *simApp) PreparePayload(position types.Position) ([]byte, error) { - return []byte{}, nil + return a.stateModule.PackRequests() } // PrepareWitness implements core.Application. @@ -149,17 +152,23 @@ func (a *simApp) TotalOrderingDelivered( // BlockDelivered is called when a block in compaction chain is delivered. func (a *simApp) BlockDelivered( - blockHash common.Hash, _ types.Position, result types.FinalizationResult) { - if len(result.Randomness) == 0 && func() bool { + blockHash common.Hash, pos types.Position, result types.FinalizationResult) { + if len(result.Randomness) == 0 && pos.Round > 0 { + panic(fmt.Errorf("Block %s randomness is empty", blockHash)) + } + func() { + a.blockByHashMutex.Lock() + defer a.blockByHashMutex.Unlock() if block, exist := a.blockByHash[blockHash]; exist { - if block.Position.Round == 0 { - return false + if err := a.stateModule.Apply(block.Payload); err != nil { + if err != test.ErrDuplicatedChange { + panic(err) + } } + } else { + panic(fmt.Errorf("Block is not confirmed yet: %s", blockHash)) } - return true - }() { - panic(fmt.Errorf("Block %s randomness is empty", blockHash)) - } + }() func() { a.latestWitnessReady.L.Lock() defer a.latestWitnessReady.L.Unlock() diff --git a/simulation/app_test.go b/simulation/app_test.go index efd27b5..02b0f48 100644 --- a/simulation/app_test.go +++ b/simulation/app_test.go @@ -32,7 +32,7 @@ type SimAppSuite struct { func (s *SimAppSuite) TestAppInterface() { var app core.Application - app = newSimApp(types.NodeID{}, nil) + app = newSimApp(types.NodeID{}, nil, nil) s.NotPanics(func() { _ = app.(core.Debug) }) diff --git a/simulation/governance.go b/simulation/governance.go deleted file mode 100644 index 4fdaadb..0000000 --- a/simulation/governance.go +++ /dev/null @@ -1,227 +0,0 @@ -// Copyright 2018 The dexon-consensus Authors -// This file is part of the dexon-consensus library. -// -// The dexon-consensus 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 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 library. If not, see -// <http://www.gnu.org/licenses/>. - -package simulation - -import ( - "fmt" - "sync" - "time" - - "github.com/dexon-foundation/dexon-consensus/common" - "github.com/dexon-foundation/dexon-consensus/core/crypto" - "github.com/dexon-foundation/dexon-consensus/core/test" - "github.com/dexon-foundation/dexon-consensus/core/types" - typesDKG "github.com/dexon-foundation/dexon-consensus/core/types/dkg" - "github.com/dexon-foundation/dexon-consensus/simulation/config" -) - -// simGovernance is a simulated governance contract implementing the -// core.Governance interface. -type simGovernance struct { - id types.NodeID - lock sync.RWMutex - nodeSet map[types.NodeID]crypto.PublicKey - expectedNumNodes uint32 - notarySetSize uint32 - dkgSetSize uint32 - k int - phiRatio float32 - chainNum uint32 - crs []common.Hash - tsig map[uint64]crypto.Signature - dkgComplaint map[uint64][]*typesDKG.Complaint - dkgMasterPublicKey map[uint64][]*typesDKG.MasterPublicKey - dkgFinal map[uint64]map[types.NodeID]struct{} - lambdaBA time.Duration - lambdaDKG time.Duration - roundInterval time.Duration - network *test.Network -} - -// newSimGovernance returns a new simGovernance instance. -func newSimGovernance( - id types.NodeID, - numNodes uint32, - notarySetSize uint32, - dkgSetSize uint32, - consensusConfig config.Consensus) *simGovernance { - hashCRS := crypto.Keccak256Hash([]byte(consensusConfig.GenesisCRS)) - return &simGovernance{ - id: id, - nodeSet: make(map[types.NodeID]crypto.PublicKey), - expectedNumNodes: numNodes, - notarySetSize: notarySetSize, - dkgSetSize: dkgSetSize, - k: consensusConfig.K, - phiRatio: consensusConfig.PhiRatio, - chainNum: consensusConfig.ChainNum, - crs: []common.Hash{hashCRS}, - tsig: make(map[uint64]crypto.Signature), - dkgComplaint: make(map[uint64][]*typesDKG.Complaint), - dkgMasterPublicKey: make(map[uint64][]*typesDKG.MasterPublicKey), - dkgFinal: make(map[uint64]map[types.NodeID]struct{}), - lambdaBA: time.Duration(consensusConfig.LambdaBA) * - time.Millisecond, - lambdaDKG: time.Duration(consensusConfig.LambdaDKG) * - time.Millisecond, - roundInterval: time.Duration(consensusConfig.RoundInterval) * - time.Millisecond, - } -} - -func (g *simGovernance) setNetwork(network *test.Network) { - g.network = network -} - -// NodeSet returns the current notary set. -func (g *simGovernance) NodeSet(round uint64) (ret []crypto.PublicKey) { - g.lock.RLock() - defer g.lock.RUnlock() - - for _, pubKey := range g.nodeSet { - ret = append(ret, pubKey) - } - return -} - -// Configuration returns the configuration at a given round. -func (g *simGovernance) Configuration(round uint64) *types.Config { - return &types.Config{ - NumChains: g.chainNum, - LambdaBA: g.lambdaBA, - LambdaDKG: g.lambdaDKG, - K: g.k, - PhiRatio: g.phiRatio, - NotarySetSize: g.notarySetSize, - DKGSetSize: g.dkgSetSize, - MinBlockInterval: g.lambdaBA * 3, - RoundInterval: g.roundInterval, - } -} - -// CRS returns the CRS for a given round. -func (g *simGovernance) CRS(round uint64) common.Hash { - if round >= uint64(len(g.crs)) { - return common.Hash{} - } - return g.crs[round] -} - -// NotifyRoundHeight notifies governance contract to snapshot configuration -// for that round with the block on that consensus height. -func (g *simGovernance) NotifyRoundHeight(round, height uint64) { -} - -// ProposeCRS proposes a CRS of round. -func (g *simGovernance) ProposeCRS(round uint64, signedCRS []byte) { - crs := crypto.Keccak256Hash(signedCRS) - if g.crs[len(g.crs)-1].Equal(crs) { - return - } - g.crs = append(g.crs, crs) -} - -// addNode add a new node into the simulated governance contract. -func (g *simGovernance) addNode(pubKey crypto.PublicKey) { - nID := types.NewNodeID(pubKey) - - g.lock.Lock() - defer g.lock.Unlock() - - if _, exists := g.nodeSet[nID]; exists { - return - } - if uint32(len(g.nodeSet)) == g.expectedNumNodes { - panic(fmt.Errorf("attempt to add node when ready")) - } - g.nodeSet[nID] = pubKey -} - -// AddDKGComplaint adds a DKGComplaint. -func (g *simGovernance) AddDKGComplaint( - round uint64, complaint *typesDKG.Complaint) { - if round != complaint.Round { - return - } - if g.IsDKGFinal(complaint.Round) { - return - } - if _, exist := g.dkgFinal[complaint.Round][complaint.ProposerID]; exist { - return - } - // TODO(jimmy-dexon): check if the input is valid. - g.dkgComplaint[complaint.Round] = append( - g.dkgComplaint[complaint.Round], complaint) - if complaint.ProposerID == g.id { - g.network.Broadcast(complaint) - } -} - -// DKGComplaints returns the DKGComplaints of round. -func (g *simGovernance) DKGComplaints(round uint64) []*typesDKG.Complaint { - complaints, exist := g.dkgComplaint[round] - if !exist { - return []*typesDKG.Complaint{} - } - return complaints -} - -// AddDKGMasterPublicKey adds a DKGMasterPublicKey. -func (g *simGovernance) AddDKGMasterPublicKey( - round uint64, masterPublicKey *typesDKG.MasterPublicKey) { - if round != masterPublicKey.Round { - return - } - // TODO(jimmy-dexon): check if the input is valid. - g.dkgMasterPublicKey[masterPublicKey.Round] = append( - g.dkgMasterPublicKey[masterPublicKey.Round], masterPublicKey) - if masterPublicKey.ProposerID == g.id { - g.network.Broadcast(masterPublicKey) - } -} - -// DKGMasterPublicKeys returns the DKGMasterPublicKeys of round. -func (g *simGovernance) DKGMasterPublicKeys( - round uint64) []*typesDKG.MasterPublicKey { - masterPublicKeys, exist := g.dkgMasterPublicKey[round] - if !exist { - return []*typesDKG.MasterPublicKey{} - } - return masterPublicKeys -} - -// AddDKGFinalize adds a DKG finalize message. -func (g *simGovernance) AddDKGFinalize( - round uint64, final *typesDKG.Finalize) { - if round != final.Round { - return - } - // TODO(jimmy-dexon): check if the input is valid. - if _, exist := g.dkgFinal[final.Round]; !exist { - g.dkgFinal[final.Round] = make(map[types.NodeID]struct{}) - } - g.dkgFinal[final.Round][final.ProposerID] = struct{}{} - if final.ProposerID == g.id { - g.network.Broadcast(final) - } -} - -// IsDKGFinal checks if DKG is final. -func (g *simGovernance) IsDKGFinal(round uint64) bool { - return len(g.dkgFinal[round]) > int(g.Configuration(round).DKGSetSize)/3*2 -} diff --git a/simulation/node.go b/simulation/node.go index 56c5832..1127f5a 100644 --- a/simulation/node.go +++ b/simulation/node.go @@ -20,7 +20,6 @@ package simulation import ( "encoding/json" "fmt" - "sort" "time" "github.com/dexon-foundation/dexon-consensus/common" @@ -29,7 +28,6 @@ import ( "github.com/dexon-foundation/dexon-consensus/core/crypto" "github.com/dexon-foundation/dexon-consensus/core/test" "github.com/dexon-foundation/dexon-consensus/core/types" - typesDKG "github.com/dexon-foundation/dexon-consensus/core/types/dkg" "github.com/dexon-foundation/dexon-consensus/simulation/config" ) @@ -56,15 +54,11 @@ type message struct { // node represents a node in DexCon. type node struct { - app core.Application - gov *simGovernance - db blockdb.BlockDatabase - - config config.Node + app core.Application + db blockdb.BlockDatabase + gov *test.Governance netModule *test.Network - ID types.NodeID - chainID uint64 prvKey crypto.PrivateKey consensus *core.Consensus } @@ -91,17 +85,32 @@ func newNode( if err != nil { panic(err) } - gov := newSimGovernance( - id, - config.Node.Num, - config.Node.Consensus.NotarySetSize, - config.Node.Consensus.DKGSetSize, - config.Node.Consensus) + // Sync config to state in governance. + cConfig := config.Node.Consensus + gov, err := test.NewGovernance([]crypto.PublicKey{pubKey}, time.Millisecond) + if err != nil { + panic(err) + } + gov.State().RequestChange(test.StateChangeK, cConfig.K) + gov.State().RequestChange(test.StateChangePhiRatio, cConfig.PhiRatio) + gov.State().RequestChange(test.StateChangeNumChains, cConfig.ChainNum) + gov.State().RequestChange( + test.StateChangeNotarySetSize, cConfig.NotarySetSize) + gov.State().RequestChange(test.StateChangeDKGSetSize, cConfig.DKGSetSize) + gov.State().RequestChange(test.StateChangeLambdaBA, time.Duration( + cConfig.LambdaBA)*time.Millisecond) + gov.State().RequestChange(test.StateChangeLambdaDKG, time.Duration( + cConfig.LambdaDKG)*time.Millisecond) + gov.State().RequestChange(test.StateChangeRoundInterval, time.Duration( + cConfig.RoundInterval)*time.Millisecond) + gov.State().RequestChange( + test.StateChangeMinBlockInterval, + 3*time.Duration(cConfig.LambdaBA)*time.Millisecond) + gov.State().ProposeCRS(0, crypto.Keccak256Hash([]byte(cConfig.GenesisCRS))) return &node{ ID: id, prvKey: prvKey, - config: config.Node, - app: newSimApp(id, netModule), + app: newSimApp(id, netModule, gov.State()), gov: gov, db: db, netModule: netModule, @@ -122,21 +131,16 @@ func (n *node) run(serverEndpoint interface{}, dMoment time.Time) { msgChannel := n.netModule.ReceiveChanForNode() peers := n.netModule.Peers() go n.netModule.Run() - n.gov.setNetwork(n.netModule) // Run consensus. hashes := make(common.Hashes, 0, len(peers)) for _, pubKey := range peers { nID := types.NewNodeID(pubKey) - n.gov.addNode(pubKey) + n.gov.State().RequestChange(test.StateAddNode, pubKey) hashes = append(hashes, nID.Hash) } - sort.Sort(hashes) - for i, hash := range hashes { - if hash == n.ID.Hash { - n.chainID = uint64(i) - break - } - } + // Setup of governance is ready, can be switched to remote mode. + n.gov.SwitchToRemoteMode(n.netModule) + // Setup Consensus. n.consensus = core.NewConsensus( dMoment, n.app, @@ -156,12 +160,6 @@ MainLoop: if val == statusShutdown { break MainLoop } - case *typesDKG.Complaint: - n.gov.AddDKGComplaint(val.Round, val) - case *typesDKG.MasterPublicKey: - n.gov.AddDKGMasterPublicKey(val.Round, val) - case *typesDKG.Finalize: - n.gov.AddDKGFinalize(val.Round, val) default: panic(fmt.Errorf("unexpected message from server: %v", val)) } diff --git a/test_config/test.toml b/test_config/test.toml index 37ac26a..4c4d979 100644 --- a/test_config/test.toml +++ b/test_config/test.toml @@ -10,8 +10,8 @@ k = 1 chain_num = 7 genesis_crs = "In DEXON we trust." lambda_ba = 250 -lambda_dkg = 1000 -round_interval = 300000 +lambda_dkg = 4000 +round_interval = 200000 notary_set_size = 7 dkg_set_size = 7 @@ -20,9 +20,9 @@ propose_interval_mean = 5e+02 propose_interval_sigma = 5e+01 [networking] -type = "tcp-local" +type = "fake" peer_server = "127.0.0.1" -mean = 1e+02 +mean = 1e+01 sigma = 1e+01 loss_rate_value = 0e+00 |