From 1ee5863fd4a295d34c3a2d602d5603e8746e3f7b Mon Sep 17 00:00:00 2001 From: Mission Liao Date: Thu, 8 Nov 2018 15:58:51 +0800 Subject: 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 --- simulation/app.go | 39 ++++---- simulation/app_test.go | 2 +- simulation/governance.go | 227 ----------------------------------------------- simulation/node.go | 62 +++++++------ 4 files changed, 55 insertions(+), 275 deletions(-) delete mode 100644 simulation/governance.go (limited to 'simulation') 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 -// . - -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)) } -- cgit v1.2.3