aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--core/agreement-state_test.go2
-rw-r--r--core/agreement_test.go2
-rw-r--r--core/configuration-chain.go43
-rw-r--r--core/configuration-chain_test.go4
-rw-r--r--core/consensus.go87
-rw-r--r--core/interfaces.go2
-rw-r--r--core/leader-selector.go6
-rw-r--r--core/leader-selector_test.go2
-rw-r--r--core/nodeset-cache_test.go3
-rw-r--r--core/test/governance.go10
-rw-r--r--core/ticker.go9
-rw-r--r--core/types/dkg.go1
-rw-r--r--core/types/nodeset.go13
-rw-r--r--core/types/nodeset_test.go2
-rw-r--r--simulation/governance.go13
15 files changed, 142 insertions, 57 deletions
diff --git a/core/agreement-state_test.go b/core/agreement-state_test.go
index 6164264..b4619d6 100644
--- a/core/agreement-state_test.go
+++ b/core/agreement-state_test.go
@@ -94,7 +94,7 @@ func (s *AgreementStateTestSuite) SetupTest() {
}
func (s *AgreementStateTestSuite) newAgreement(numNode int) *agreement {
- leader := newGenesisLeaderSelector([]byte("I ❤️ DEXON"))
+ leader := newLeaderSelector(common.NewRandomHash())
notarySet := make(map[types.NodeID]struct{})
for i := 0; i < numNode-1; i++ {
prvKey, err := ecdsa.NewPrivateKey()
diff --git a/core/agreement_test.go b/core/agreement_test.go
index 124ca48..5a14cda 100644
--- a/core/agreement_test.go
+++ b/core/agreement_test.go
@@ -82,7 +82,7 @@ func (s *AgreementTestSuite) SetupTest() {
}
func (s *AgreementTestSuite) newAgreement(numNotarySet int) *agreement {
- leader := newGenesisLeaderSelector([]byte("🖖👽"))
+ leader := newLeaderSelector(common.NewRandomHash())
agreementIdx := len(s.agreement)
notarySet := make(map[types.NodeID]struct{})
for i := 0; i < numNotarySet-1; i++ {
diff --git a/core/configuration-chain.go b/core/configuration-chain.go
index d179b61..300b080 100644
--- a/core/configuration-chain.go
+++ b/core/configuration-chain.go
@@ -66,6 +66,8 @@ func newConfigurationChain(
}
func (cc *configurationChain) registerDKG(round uint64, threshold int) {
+ cc.dkgLock.Lock()
+ defer cc.dkgLock.Unlock()
cc.dkg = newDKGProtocol(
cc.ID,
cc.recv,
@@ -88,7 +90,7 @@ func (cc *configurationChain) runDKG(round uint64) error {
return nil
}
- ticker := newTicker(cc.gov, TickerDKG)
+ ticker := newTicker(cc.gov, round, TickerDKG)
cc.dkgLock.Unlock()
<-ticker.Tick()
cc.dkgLock.Lock()
@@ -136,8 +138,8 @@ func (cc *configurationChain) runDKG(round uint64) error {
for nID := range gpk.qualifyNodeIDs {
qualifies += fmt.Sprintf("%s ", nID.String()[:6])
}
- log.Printf("[%s] Qualify Nodes(%d): %s\n",
- cc.ID, len(gpk.qualifyIDs), qualifies)
+ log.Printf("[%s] Qualify Nodes(%d): (%d) %s\n",
+ cc.ID, round, len(gpk.qualifyIDs), qualifies)
cc.dkgResult.Lock()
defer cc.dkgResult.Unlock()
cc.dkgSigner[round] = signer
@@ -146,7 +148,8 @@ func (cc *configurationChain) runDKG(round uint64) error {
}
func (cc *configurationChain) preparePartialSignature(
- round uint64, hash common.Hash) (*types.DKGPartialSignature, error) {
+ round uint64, hash common.Hash, psigType types.DKGPartialSignatureType) (
+ *types.DKGPartialSignature, error) {
signer, exist := func() (*dkgShareSecret, bool) {
cc.dkgResult.RLock()
defer cc.dkgResult.RUnlock()
@@ -159,12 +162,14 @@ func (cc *configurationChain) preparePartialSignature(
return &types.DKGPartialSignature{
ProposerID: cc.ID,
Round: round,
+ Type: psigType,
PartialSignature: signer.sign(hash),
}, nil
}
-func (cc *configurationChain) runBlockTSig(
- round uint64, hash common.Hash) (crypto.Signature, error) {
+func (cc *configurationChain) runTSig(
+ round uint64, hash common.Hash, psigType types.DKGPartialSignatureType) (
+ crypto.Signature, error) {
gpk, exist := func() (*dkgGroupPublicKey, bool) {
cc.dkgResult.RLock()
defer cc.dkgResult.RUnlock()
@@ -176,7 +181,10 @@ func (cc *configurationChain) runBlockTSig(
}
cc.tsigReady.L.Lock()
defer cc.tsigReady.L.Unlock()
- cc.tsig = newTSigProtocol(gpk, hash, types.TSigConfigurationBlock)
+ for cc.tsig != nil {
+ cc.tsigReady.Wait()
+ }
+ cc.tsig = newTSigProtocol(gpk, hash, psigType)
pendingPsig := cc.pendingPsig
cc.pendingPsig = []*types.DKGPartialSignature{}
go func() {
@@ -195,13 +203,30 @@ func (cc *configurationChain) runBlockTSig(
cc.tsigReady.Wait()
}
cc.tsig = nil
+ cc.tsigReady.Broadcast()
if err != nil {
return crypto.Signature{}, err
}
- log.Printf("[%s] TSIG: %s\n", cc.ID, signature)
return signature, nil
}
+func (cc *configurationChain) runBlockTSig(
+ round uint64, hash common.Hash) (crypto.Signature, error) {
+ sig, err := cc.runTSig(round, hash, types.TSigConfigurationBlock)
+ if err != nil {
+ return crypto.Signature{}, err
+ }
+ log.Printf("[%s] Block TSIG(%d): %s\n", cc.ID, round, sig)
+ return sig, nil
+}
+
+func (cc *configurationChain) runCRSTSig(
+ round uint64, crs common.Hash) ([]byte, error) {
+ sig, err := cc.runTSig(round, crs, types.TSigCRS)
+ log.Printf("[%s] CRS(%d): %s\n", cc.ID, round+1, sig)
+ return sig.Signature[:], err
+}
+
func (cc *configurationChain) processPrivateShare(
prvShare *types.DKGPrivateShare) error {
cc.dkgLock.Lock()
@@ -216,7 +241,7 @@ func (cc *configurationChain) processPartialSignature(
psig *types.DKGPartialSignature) error {
cc.tsigReady.L.Lock()
defer cc.tsigReady.L.Unlock()
- if cc.tsig == nil {
+ if cc.tsig == nil || psig.Type != cc.tsig.psigType {
ok, err := verifyDKGPartialSignatureSignature(psig)
if err != nil {
return err
diff --git a/core/configuration-chain_test.go b/core/configuration-chain_test.go
index a9615ec..7ebc502 100644
--- a/core/configuration-chain_test.go
+++ b/core/configuration-chain_test.go
@@ -190,7 +190,7 @@ func (s *ConfigurationChainTestSuite) TestConfigurationChain() {
if _, exist := cc.gpk[round].qualifyNodeIDs[nID]; !exist {
continue
}
- psig, err := cc.preparePartialSignature(round, hash)
+ psig, err := cc.preparePartialSignature(round, hash, types.TSigCRS)
s.Require().NoError(err)
prvKey, exist := s.prvKeys[cc.ID]
s.Require().True(exist)
@@ -207,7 +207,7 @@ func (s *ConfigurationChainTestSuite) TestConfigurationChain() {
continue
}
go func(cc *configurationChain) {
- tsig, err := cc.runBlockTSig(round, hash)
+ tsig, err := cc.runTSig(round, hash, types.TSigCRS)
// Prevent racing by collecting errors and check in main thread.
errs <- err
tsigChan <- tsig
diff --git a/core/consensus.go b/core/consensus.go
index d6ab930..a1642df 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -63,10 +63,11 @@ var (
// consensusBAReceiver implements agreementReceiver.
type consensusBAReceiver struct {
// TODO(mission): consensus would be replaced by shard and network.
- consensus *Consensus
- agreementModule *agreement
- chainID uint32
- restartNotary chan bool
+ consensus *Consensus
+ agreementModule *agreement
+ chainID uint32
+ changeNotaryTime time.Time
+ restartNotary chan bool
}
func (recv *consensusBAReceiver) ProposeVote(vote *types.Vote) {
@@ -103,7 +104,7 @@ func (recv *consensusBAReceiver) ConfirmBlock(hash common.Hash) {
log.Println(err)
return
}
- recv.restartNotary <- false
+ recv.restartNotary <- block.Timestamp.After(recv.changeNotaryTime)
}
// consensusDKGReceiver implements dkgReceiver.
@@ -250,7 +251,7 @@ func NewConsensus(
gov)
// Register DKG for the initial round. This is a temporary function call for
// simulation.
- cfgModule.registerDKG(0, len(nodes.IDs)/3)
+ cfgModule.registerDKG(0, config.NumDKGSet/3)
// Check if the application implement Debug interface.
debug, _ := app.(Debug)
@@ -265,7 +266,7 @@ func NewConsensus(
gov: gov,
db: db,
network: network,
- tickerObj: newTicker(gov, TickerBA),
+ tickerObj: newTicker(gov, 0, TickerBA),
dkgReady: sync.NewCond(&sync.Mutex{}),
cfgModule: cfgModule,
nodeSetCache: nodeSetCache,
@@ -287,7 +288,7 @@ func NewConsensus(
con.ID,
recv,
nodes.IDs,
- newGenesisLeaderSelector(crs),
+ newLeaderSelector(crs),
con.authModule,
)
// Hacky way to make agreement module self contained.
@@ -302,17 +303,20 @@ func NewConsensus(
func (con *Consensus) Run() {
go con.processMsg(con.network.ReceiveChan())
con.runDKGTSIG()
- con.dkgReady.L.Lock()
- defer con.dkgReady.L.Unlock()
- for con.dkgRunning != 2 {
- con.dkgReady.Wait()
- }
+ func() {
+ con.dkgReady.L.Lock()
+ defer con.dkgReady.L.Unlock()
+ for con.dkgRunning != 2 {
+ con.dkgReady.Wait()
+ }
+ }()
ticks := make([]chan struct{}, 0, con.currentConfig.NumChains)
for i := uint32(0); i < con.currentConfig.NumChains; i++ {
tick := make(chan struct{})
ticks = append(ticks, tick)
go con.runBA(i, tick)
}
+ go con.runCRS()
// Reset ticker.
<-con.tickerObj.Tick()
@@ -329,6 +333,7 @@ func (con *Consensus) runBA(chainID uint32, tick <-chan struct{}) {
// TODO(jimmy-dexon): move this function inside agreement.
agreement := con.baModules[chainID]
recv := con.receivers[chainID]
+ recv.changeNotaryTime = time.Now().UTC()
recv.restartNotary <- true
nIDs := make(map[types.NodeID]struct{})
// Reset ticker
@@ -346,12 +351,13 @@ BALoop:
select {
case newNotary := <-recv.restartNotary:
if newNotary {
+ recv.changeNotaryTime.Add(con.currentConfig.RoundInterval)
nodes, err := con.nodeSetCache.GetNodeSet(con.round)
if err != nil {
panic(err)
}
nIDs = nodes.GetSubSet(con.gov.GetConfiguration(con.round).NumNotarySet,
- types.NewNotarySetTarget(con.gov.GetCRS(con.round), 0, chainID))
+ types.NewNotarySetTarget(con.gov.GetCRS(con.round), chainID))
}
aID := types.Position{
ChainID: chainID,
@@ -377,11 +383,18 @@ func (con *Consensus) runDKGTSIG() {
}
con.dkgRunning = 1
go func() {
+ startTime := time.Now().UTC()
defer func() {
con.dkgReady.L.Lock()
defer con.dkgReady.L.Unlock()
con.dkgReady.Broadcast()
con.dkgRunning = 2
+ DKGTime := time.Now().Sub(startTime)
+ if DKGTime.Nanoseconds() >=
+ con.currentConfig.RoundInterval.Nanoseconds()/2 {
+ log.Printf("[%s] WARNING!!! Your computer cannot finish DKG on time!\n",
+ con.ID)
+ }
}()
round := con.round
if err := con.cfgModule.runDKG(round); err != nil {
@@ -396,7 +409,8 @@ func (con *Consensus) runDKGTSIG() {
con.gov.GetConfiguration(round),
common.Hash{},
con.cfgModule.prevHash)
- psig, err := con.cfgModule.preparePartialSignature(round, hash)
+ psig, err := con.cfgModule.preparePartialSignature(
+ round, hash, types.TSigConfigurationBlock)
if err != nil {
panic(err)
}
@@ -413,6 +427,47 @@ func (con *Consensus) runDKGTSIG() {
}()
}
+func (con *Consensus) runCRS() {
+ for {
+ ticker := newTicker(con.gov, con.round, TickerCRS)
+ select {
+ case <-con.ctx.Done():
+ return
+ default:
+ }
+ <-ticker.Tick()
+ // Start running next round CRS.
+ psig, err := con.cfgModule.preparePartialSignature(
+ con.round, con.gov.GetCRS(con.round), types.TSigCRS)
+ if err != nil {
+ log.Println(err)
+ } else if err = con.authModule.SignDKGPartialSignature(psig); err != nil {
+ log.Println(err)
+ } else if err = con.cfgModule.processPartialSignature(psig); err != nil {
+ log.Println(err)
+ } else {
+ con.network.BroadcastDKGPartialSignature(psig)
+ crs, err := con.cfgModule.runCRSTSig(con.round, con.gov.GetCRS(con.round))
+ if err != nil {
+ log.Println(err)
+ } else {
+ con.gov.ProposeCRS(con.round+1, crs)
+ }
+ }
+ con.cfgModule.registerDKG(con.round+1, con.currentConfig.NumDKGSet/3)
+ <-ticker.Tick()
+ // Change round.
+ con.round++
+ con.currentConfig = con.gov.GetConfiguration(con.round)
+ func() {
+ con.dkgReady.L.Lock()
+ defer con.dkgReady.L.Unlock()
+ con.dkgRunning = 0
+ }()
+ con.runDKGTSIG()
+ }
+}
+
// Stop the Consensus core.
func (con *Consensus) Stop() {
con.ctxCancel()
@@ -464,7 +519,7 @@ func (con *Consensus) proposeBlock(chainID uint32) *types.Block {
// TODO(mission): decide CRS by block's round, which could be determined by
// block's info (ex. position, timestamp).
if err := con.authModule.SignCRS(
- block, crypto.Keccak256Hash(con.gov.GetCRS(0))); err != nil {
+ block, con.gov.GetCRS(0)); err != nil {
log.Println(err)
return nil
}
diff --git a/core/interfaces.go b/core/interfaces.go
index 465a3ce..dcaa0c4 100644
--- a/core/interfaces.go
+++ b/core/interfaces.go
@@ -85,7 +85,7 @@ type Governance interface {
// GetCRS returns the CRS for a given round.
// Return the genesis CRS if round == 0.
- GetCRS(round uint64) []byte
+ GetCRS(round uint64) common.Hash
// Propose a CRS of round.
ProposeCRS(round uint64, crs []byte)
diff --git a/core/leader-selector.go b/core/leader-selector.go
index 1a1885b..de519b0 100644
--- a/core/leader-selector.go
+++ b/core/leader-selector.go
@@ -53,12 +53,6 @@ type leaderSelector struct {
minBlockHash common.Hash
}
-func newGenesisLeaderSelector(
- crs []byte) *leaderSelector {
- hash := crypto.Keccak256Hash(crs)
- return newLeaderSelector(hash)
-}
-
func newLeaderSelector(
crs common.Hash) *leaderSelector {
numCRS := big.NewInt(0)
diff --git a/core/leader-selector_test.go b/core/leader-selector_test.go
index b5dfe91..e2f7070 100644
--- a/core/leader-selector_test.go
+++ b/core/leader-selector_test.go
@@ -32,7 +32,7 @@ type LeaderSelectorTestSuite struct {
}
func (s *LeaderSelectorTestSuite) newLeader() *leaderSelector {
- return newGenesisLeaderSelector([]byte("DEXON 🚀"))
+ return newLeaderSelector(common.NewRandomHash())
}
func (s *LeaderSelectorTestSuite) TestDistance() {
diff --git a/core/nodeset-cache_test.go b/core/nodeset-cache_test.go
index 947e239..def8450 100644
--- a/core/nodeset-cache_test.go
+++ b/core/nodeset-cache_test.go
@@ -20,6 +20,7 @@ package core
import (
"testing"
+ "github.com/dexon-foundation/dexon-consensus-core/common"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto/ecdsa"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
@@ -32,7 +33,7 @@ type testGov struct {
}
func (g *testGov) GetConfiguration(round uint64) (cfg *types.Config) { return }
-func (g *testGov) GetCRS(round uint64) (b []byte) { return }
+func (g *testGov) GetCRS(round uint64) (b common.Hash) { return }
func (g *testGov) ProposeCRS(uint64, []byte) {}
func (g *testGov) GetNodeSet(round uint64) []crypto.PublicKey {
// Randomly generating keys, and check them for verification.
diff --git a/core/test/governance.go b/core/test/governance.go
index 0fc962b..62ebef0 100644
--- a/core/test/governance.go
+++ b/core/test/governance.go
@@ -22,6 +22,7 @@ import (
"sync"
"time"
+ "github.com/dexon-foundation/dexon-consensus-core/common"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto/ecdsa"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
@@ -38,7 +39,7 @@ type Governance struct {
lambdaBA time.Duration
lambdaDKG time.Duration
privateKeys map[types.NodeID]crypto.PrivateKey
- crs map[uint64][]byte
+ crs map[uint64]common.Hash
tsig map[uint64]crypto.Signature
DKGComplaint map[uint64][]*types.DKGComplaint
DKGMasterPublicKey map[uint64][]*types.DKGMasterPublicKey
@@ -51,11 +52,12 @@ type Governance struct {
// NewGovernance constructs a Governance instance.
func NewGovernance(nodeCount int, lambda time.Duration) (
g *Governance, err error) {
+ hashCRS := crypto.Keccak256Hash([]byte("__ DEXON"))
g = &Governance{
lambdaBA: lambda,
lambdaDKG: lambda * 10,
privateKeys: make(map[types.NodeID]crypto.PrivateKey),
- crs: map[uint64][]byte{0: []byte("__ DEXON")},
+ crs: map[uint64]common.Hash{0: hashCRS},
tsig: make(map[uint64]crypto.Signature),
DKGComplaint: make(map[uint64][]*types.DKGComplaint),
DKGMasterPublicKey: make(map[uint64][]*types.DKGMasterPublicKey),
@@ -103,13 +105,13 @@ func (g *Governance) GetConfiguration(_ uint64) *types.Config {
}
// GetCRS returns the CRS for a given round.
-func (g *Governance) GetCRS(round uint64) []byte {
+func (g *Governance) GetCRS(round uint64) common.Hash {
return g.crs[round]
}
// ProposeCRS propose a CRS.
func (g *Governance) ProposeCRS(round uint64, crs []byte) {
- g.crs[round] = crs
+ g.crs[round] = crypto.Keccak256Hash(crs)
}
// GetPrivateKeys return the private key for that node, this function
diff --git a/core/ticker.go b/core/ticker.go
index 5dbbc2a..eac80de 100644
--- a/core/ticker.go
+++ b/core/ticker.go
@@ -26,6 +26,7 @@ type TickerType int
const (
TickerBA TickerType = iota
TickerDKG
+ TickerCRS
)
// defaultTicker is a wrapper to implement ticker interface based on
@@ -52,7 +53,7 @@ func (t *defaultTicker) Stop() {
// newTicker is a helper to setup a ticker by giving an Governance. If
// the governace object implements a ticker generator, a ticker from that
// generator would be returned, else constructs a default one.
-func newTicker(gov Governance, tickerType TickerType) (t Ticker) {
+func newTicker(gov Governance, round uint64, tickerType TickerType) (t Ticker) {
type tickerGenerator interface {
NewTicker(TickerType) Ticker
}
@@ -64,9 +65,11 @@ func newTicker(gov Governance, tickerType TickerType) (t Ticker) {
var duration time.Duration
switch tickerType {
case TickerBA:
- duration = gov.GetConfiguration(0).LambdaBA
+ duration = gov.GetConfiguration(round).LambdaBA
case TickerDKG:
- duration = gov.GetConfiguration(0).LambdaDKG
+ duration = gov.GetConfiguration(round).LambdaDKG
+ case TickerCRS:
+ duration = gov.GetConfiguration(round).RoundInterval / 2
}
t = newDefaultTicker(duration)
}
diff --git a/core/types/dkg.go b/core/types/dkg.go
index 7738783..df9d8e3 100644
--- a/core/types/dkg.go
+++ b/core/types/dkg.go
@@ -73,6 +73,7 @@ type DKGPartialSignatureType uint32
const (
TSigConfigurationBlock DKGPartialSignatureType = iota
TSigNotaryAck
+ TSigCRS
)
// DKGPartialSignature describe a partial signature in DKG protocol.
diff --git a/core/types/nodeset.go b/core/types/nodeset.go
index 6b78cab..83c64a7 100644
--- a/core/types/nodeset.go
+++ b/core/types/nodeset.go
@@ -22,6 +22,7 @@ import (
"encoding/binary"
"math/big"
+ "github.com/dexon-foundation/dexon-consensus-core/common"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto"
)
@@ -71,27 +72,27 @@ func NewNodeSet() *NodeSet {
}
// NewNotarySetTarget is the target for getting Notary Set.
-func NewNotarySetTarget(crs []byte, shardID, chainID uint32) SubSetTarget {
+func NewNotarySetTarget(crs common.Hash, chainID uint32) SubSetTarget {
binaryChainID := make([]byte, 4)
binary.LittleEndian.PutUint32(binaryChainID, chainID)
- return newTarget(targetNotarySet, crs, binaryChainID)
+ return newTarget(targetNotarySet, crs[:], binaryChainID)
}
// NewWitnessSetTarget is the target for getting DKG Set.
-func NewWitnessSetTarget(crs []byte, round uint64) SubSetTarget {
+func NewWitnessSetTarget(crs common.Hash, round uint64) SubSetTarget {
binaryRound := make([]byte, 8)
binary.LittleEndian.PutUint64(binaryRound, round)
- return newTarget(targetWitnessSet, crs, binaryRound)
+ return newTarget(targetWitnessSet, crs[:], binaryRound)
}
// NewDKGSetTarget is the target for getting DKG Set.
-func NewDKGSetTarget(crs []byte, round uint64) SubSetTarget {
+func NewDKGSetTarget(crs common.Hash, round uint64) SubSetTarget {
binaryRound := make([]byte, 8)
binary.LittleEndian.PutUint64(binaryRound, round)
- return newTarget(targetDKGSet, crs, binaryRound)
+ return newTarget(targetDKGSet, crs[:], binaryRound)
}
// Add a NodeID to the set.
diff --git a/core/types/nodeset_test.go b/core/types/nodeset_test.go
index 021a0a4..d6f55c2 100644
--- a/core/types/nodeset_test.go
+++ b/core/types/nodeset_test.go
@@ -35,7 +35,7 @@ func (s *NodeSetTestSuite) TestGetSubSet() {
for len(nodes.IDs) < total {
nodes.IDs[NodeID{common.NewRandomHash()}] = struct{}{}
}
- target := NewNotarySetTarget(crs[:], 0, 0)
+ target := NewNotarySetTarget(crs, 0)
ranks := make(map[NodeID]*nodeRank, len(nodes.IDs))
for nID := range nodes.IDs {
ranks[nID] = newNodeRank(nID, target)
diff --git a/simulation/governance.go b/simulation/governance.go
index 9e7cbaf..21bf35a 100644
--- a/simulation/governance.go
+++ b/simulation/governance.go
@@ -22,6 +22,7 @@ import (
"sync"
"time"
+ "github.com/dexon-foundation/dexon-consensus-core/common"
"github.com/dexon-foundation/dexon-consensus-core/core/crypto"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
"github.com/dexon-foundation/dexon-consensus-core/simulation/config"
@@ -37,7 +38,7 @@ type simGovernance struct {
k int
phiRatio float32
chainNum uint32
- crs map[uint64][]byte
+ crs map[uint64]common.Hash
tsig map[uint64]crypto.Signature
dkgComplaint map[uint64][]*types.DKGComplaint
dkgMasterPublicKey map[uint64][]*types.DKGMasterPublicKey
@@ -51,6 +52,7 @@ type simGovernance struct {
func newSimGovernance(
id types.NodeID,
numNodes int, consensusConfig config.Consensus) *simGovernance {
+ hashCRS := crypto.Keccak256Hash([]byte(consensusConfig.GenesisCRS))
return &simGovernance{
id: id,
nodeSet: make(map[types.NodeID]crypto.PublicKey),
@@ -58,8 +60,8 @@ func newSimGovernance(
k: consensusConfig.K,
phiRatio: consensusConfig.PhiRatio,
chainNum: consensusConfig.ChainNum,
- crs: map[uint64][]byte{
- 0: []byte(consensusConfig.GenesisCRS)},
+ crs: map[uint64]common.Hash{
+ 0: hashCRS},
tsig: make(map[uint64]crypto.Signature),
dkgComplaint: make(map[uint64][]*types.DKGComplaint),
dkgMasterPublicKey: make(map[uint64][]*types.DKGMasterPublicKey),
@@ -101,17 +103,18 @@ func (g *simGovernance) GetConfiguration(round uint64) *types.Config {
NumDKGSet: len(g.nodeSet),
MinBlockInterval: g.lambdaBA * 3,
MaxBlockInterval: g.lambdaBA * 8,
+ RoundInterval: g.roundInterval,
}
}
// GetCRS returns the CRS for a given round.
-func (g *simGovernance) GetCRS(round uint64) []byte {
+func (g *simGovernance) GetCRS(round uint64) common.Hash {
return g.crs[round]
}
// ProposeCRS proposes a CRS of round.
func (g *simGovernance) ProposeCRS(round uint64, crs []byte) {
- g.crs[round] = crs
+ g.crs[round] = crypto.Keccak256Hash(crs)
}
// addNode add a new node into the simulated governance contract.