aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorMission Liao <mission.liao@dexon.org>2018-11-08 15:58:51 +0800
committerGitHub <noreply@github.com>2018-11-08 15:58:51 +0800
commit1ee5863fd4a295d34c3a2d602d5603e8746e3f7b (patch)
tree044308b22000bb0c9f5a8c3c21f465159418db24
parentdbe83ea4a324941417d6ff09230e5874d5ba5df5 (diff)
downloaddexon-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar
dexon-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar.gz
dexon-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar.bz2
dexon-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar.lz
dexon-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar.xz
dexon-consensus-1ee5863fd4a295d34c3a2d602d5603e8746e3f7b.tar.zst
dexon-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.go3
-rw-r--r--core/lattice.go2
-rw-r--r--core/test/governance.go75
-rw-r--r--core/test/marshaller.go9
-rw-r--r--core/test/network.go67
-rw-r--r--core/test/state.go8
-rw-r--r--core/test/state_test.go3
-rw-r--r--simulation/app.go39
-rw-r--r--simulation/app_test.go2
-rw-r--r--simulation/governance.go227
-rw-r--r--simulation/node.go62
-rw-r--r--test_config/test.toml8
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