aboutsummaryrefslogtreecommitdiffstats
path: root/core/test
diff options
context:
space:
mode:
authorMission Liao <mission.liao@dexon.org>2018-10-26 13:26:18 +0800
committerGitHub <noreply@github.com>2018-10-26 13:26:18 +0800
commitef9575062f5ec6a36b30efb5064c0e3a442075fe (patch)
tree703c21ed27bed375ed1584a5114ec6b28c4965f0 /core/test
parent721d36f2720b0cbf648cbffe40fd05c9a60061e4 (diff)
downloaddexon-consensus-ef9575062f5ec6a36b30efb5064c0e3a442075fe.tar
dexon-consensus-ef9575062f5ec6a36b30efb5064c0e3a442075fe.tar.gz
dexon-consensus-ef9575062f5ec6a36b30efb5064c0e3a442075fe.tar.bz2
dexon-consensus-ef9575062f5ec6a36b30efb5064c0e3a442075fe.tar.lz
dexon-consensus-ef9575062f5ec6a36b30efb5064c0e3a442075fe.tar.xz
dexon-consensus-ef9575062f5ec6a36b30efb5064c0e3a442075fe.tar.zst
dexon-consensus-ef9575062f5ec6a36b30efb5064c0e3a442075fe.zip
test: clone types.Config, test.Governance, test.State (#262)
Diffstat (limited to 'core/test')
-rw-r--r--core/test/governance.go91
-rw-r--r--core/test/governance_test.go64
-rw-r--r--core/test/state.go327
-rw-r--r--core/test/state_test.go104
4 files changed, 544 insertions, 42 deletions
diff --git a/core/test/governance.go b/core/test/governance.go
index a3b9168..4a2b61d 100644
--- a/core/test/governance.go
+++ b/core/test/governance.go
@@ -18,29 +18,25 @@
package test
import (
- "fmt"
+ "encoding/hex"
+ "reflect"
+ "sort"
"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"
typesDKG "github.com/dexon-foundation/dexon-consensus-core/core/types/dkg"
)
-var (
- // ErrPrivateKeyNotExists means caller request private key for an
- // unknown node ID.
- ErrPrivateKeyNotExists = fmt.Errorf("private key not exists")
-)
-
// Governance is an implementation of Goverance for testing purpose.
type Governance struct {
- privateKeys map[types.NodeID]crypto.PrivateKey
- configs []*types.Config
- nodeSets [][]crypto.PublicKey
- state *State
- lock sync.RWMutex
+ configs []*types.Config
+ nodeSets [][]crypto.PublicKey
+ state *State
+ lock sync.RWMutex
}
// NewGovernance constructs a Governance instance.
@@ -193,3 +189,74 @@ func (g *Governance) CatchUpWithRound(round uint64) {
g.nodeSets = append(g.nodeSets, nodeSet)
}
}
+
+// Clone a governance instance with replicate internal state.
+func (g *Governance) Clone() *Governance {
+ g.lock.RLock()
+ defer g.lock.RUnlock()
+ // Clone state.
+ copiedState := g.state.Clone()
+ // Clone configs.
+ copiedConfigs := []*types.Config{}
+ for _, c := range g.configs {
+ copiedConfigs = append(copiedConfigs, c.Clone())
+ }
+ // Clone node sets.
+ // NOTE: here I assume the key is from ecdsa.
+ copiedNodeSets := [][]crypto.PublicKey{}
+ for _, nodeSetForRound := range g.nodeSets {
+ copiedNodeSet := []crypto.PublicKey{}
+ for _, node := range nodeSetForRound {
+ pubKey, err := ecdsa.NewPublicKeyFromByteSlice(node.Bytes())
+ if err != nil {
+ panic(err)
+ }
+ copiedNodeSet = append(copiedNodeSet, pubKey)
+ }
+ copiedNodeSets = append(copiedNodeSets, copiedNodeSet)
+ }
+ return &Governance{
+ configs: copiedConfigs,
+ state: copiedState,
+ nodeSets: copiedNodeSets,
+ }
+}
+
+// Equal checks equality between two Governance instances.
+func (g *Governance) Equal(other *Governance, checkState bool) bool {
+ // Check configs.
+ if !reflect.DeepEqual(g.configs, other.configs) {
+ return false
+ }
+ // Check node sets.
+ if len(g.nodeSets) != len(other.nodeSets) {
+ return false
+ }
+ getSortedKeys := func(keys []crypto.PublicKey) (encoded []string) {
+ for _, key := range keys {
+ encoded = append(encoded, hex.EncodeToString(key.Bytes()))
+ }
+ sort.Strings(encoded)
+ return
+ }
+ for round, nodeSetsForRound := range g.nodeSets {
+ otherNodeSetsForRound := other.nodeSets[round]
+ if len(nodeSetsForRound) != len(otherNodeSetsForRound) {
+ return false
+ }
+ if !reflect.DeepEqual(
+ getSortedKeys(nodeSetsForRound),
+ getSortedKeys(otherNodeSetsForRound)) {
+ return false
+ }
+ }
+ // Check state if needed.
+ //
+ // While testing, it's expected that two governance instances contain
+ // different state, only the snapshots (configs and node sets) are
+ // essentially equal.
+ if checkState {
+ return g.state.Equal(other.state) == nil
+ }
+ return true
+}
diff --git a/core/test/governance_test.go b/core/test/governance_test.go
new file mode 100644
index 0000000..16de2a1
--- /dev/null
+++ b/core/test/governance_test.go
@@ -0,0 +1,64 @@
+// Copyright 2018 The dexon-consensus-core Authors
+// This file is part of the dexon-consensus-core library.
+//
+// The dexon-consensus-core 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-core 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-core library. If not, see
+// <http://www.gnu.org/licenses/>.
+
+package test
+
+import (
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/suite"
+)
+
+type GovernanceTestSuite struct {
+ suite.Suite
+}
+
+func (s *GovernanceTestSuite) TestEqual() {
+ var req = s.Require()
+ // Setup a base governance.
+ _, genesisNodes, err := NewKeys(20)
+ req.NoError(err)
+ g1, err := NewGovernance(genesisNodes, 100*time.Millisecond)
+ req.NoError(err)
+ // Create a governance with different lambda.
+ g2, err := NewGovernance(genesisNodes, 50*time.Millisecond)
+ req.NoError(err)
+ req.False(g1.Equal(g2, true))
+ // Create configs for 3 rounds for g1.
+ g1.CatchUpWithRound(3)
+ // Make a clone.
+ g3 := g1.Clone()
+ req.True(g1.Equal(g3, true))
+ // Create a new round for g1.
+ g1.CatchUpWithRound(4)
+ req.False(g1.Equal(g3, true))
+ // Make another clone.
+ g4 := g1.Clone()
+ req.True(g1.Equal(g4, true))
+ // Add a node to g4.
+ _, newNodes, err := NewKeys(1)
+ req.NoError(err)
+ g4.State().RequestChange(StateAddNode, newNodes[0])
+ g1.CatchUpWithRound(5)
+ g4.CatchUpWithRound(5)
+ req.False(g1.Equal(g4, true))
+}
+
+func TestGovernance(t *testing.T) {
+ suite.Run(t, new(GovernanceTestSuite))
+}
diff --git a/core/test/state.go b/core/test/state.go
index f8f9ebe..34c38bc 100644
--- a/core/test/state.go
+++ b/core/test/state.go
@@ -18,8 +18,10 @@
package test
import (
+ "bytes"
"errors"
"math"
+ "reflect"
"sync"
"time"
@@ -46,6 +48,28 @@ var (
ErrUnknownStateChangeType = errors.New("unknown state change type")
// ErrProposerIsFinal means a proposer of one complaint is finalized.
ErrProposerIsFinal = errors.New("proposer is final")
+ // ErrStateConfigNotEqual means configuration part of two states is not
+ // equal.
+ ErrStateConfigNotEqual = errors.New("config not equal")
+ // ErrStateLocalFlagNotEqual means local flag of two states is not equal.
+ ErrStateLocalFlagNotEqual = errors.New("local flag not equal")
+ // ErrStateNodeSetNotEqual means node sets of two states are not equal.
+ ErrStateNodeSetNotEqual = errors.New("node set not equal")
+ // ErrStateDKGComplaintsNotEqual means DKG complaints for two states are not
+ // equal.
+ ErrStateDKGComplaintsNotEqual = errors.New("dkg complaints not equal")
+ // ErrStateDKGMasterPublicKeysNotEqual means DKG master public keys of two
+ // states are not equal.
+ ErrStateDKGMasterPublicKeysNotEqual = errors.New(
+ "dkg master public keys not equal")
+ // ErrStateDKGFinalsNotEqual means DKG finalizations of two states are not
+ // equal.
+ ErrStateDKGFinalsNotEqual = errors.New("dkg finalizations not equal")
+ // ErrStateCRSsNotEqual means CRSs of two states are not equal.
+ ErrStateCRSsNotEqual = errors.New("crs not equal")
+ // ErrStatePendingChangesNotEqual means pending change requests of two
+ // states are not equal.
+ ErrStatePendingChangesNotEqual = errors.New("pending changes not equal")
)
// Types of state change.
@@ -247,6 +271,288 @@ func (s *State) unpackPayload(
return
}
+func (s *State) cloneDKGComplaint(
+ comp *typesDKG.Complaint) (copied *typesDKG.Complaint) {
+ b, err := rlp.EncodeToBytes(comp)
+ if err != nil {
+ panic(err)
+ }
+ copied = &typesDKG.Complaint{}
+ if err = rlp.DecodeBytes(b, copied); err != nil {
+ panic(err)
+ }
+ return
+}
+
+func (s *State) cloneDKGMasterPublicKey(mpk *typesDKG.MasterPublicKey) (
+ copied *typesDKG.MasterPublicKey) {
+ b, err := rlp.EncodeToBytes(mpk)
+ if err != nil {
+ panic(err)
+ }
+ copied = typesDKG.NewMasterPublicKey()
+ if err = rlp.DecodeBytes(b, copied); err != nil {
+ panic(err)
+ }
+ return
+}
+
+func (s *State) cloneDKGFinalize(final *typesDKG.Finalize) (
+ copied *typesDKG.Finalize) {
+ b, err := rlp.EncodeToBytes(final)
+ if err != nil {
+ panic(err)
+ }
+ copied = &typesDKG.Finalize{}
+ if err = rlp.DecodeBytes(b, copied); err != nil {
+ panic(err)
+ }
+ return
+}
+
+// Equal checks equality between State instance.
+func (s *State) Equal(other *State) error {
+ // Check configuration part.
+ configEqual := s.numChains == other.numChains &&
+ s.lambdaBA == other.lambdaBA &&
+ s.lambdaDKG == other.lambdaDKG &&
+ s.k == other.k &&
+ s.phiRatio == other.phiRatio &&
+ s.notarySetSize == other.notarySetSize &&
+ s.dkgSetSize == other.dkgSetSize &&
+ s.roundInterval == other.roundInterval &&
+ s.minBlockInterval == other.minBlockInterval &&
+ s.maxBlockInterval == other.maxBlockInterval
+ if !configEqual {
+ return ErrStateConfigNotEqual
+ }
+ // Check local flag.
+ if s.local != other.local {
+ return ErrStateLocalFlagNotEqual
+ }
+ // Check node set.
+ if len(s.nodes) != len(other.nodes) {
+ return ErrStateNodeSetNotEqual
+ }
+ for nID, key := range s.nodes {
+ otherKey, exists := other.nodes[nID]
+ if !exists {
+ return ErrStateNodeSetNotEqual
+ }
+ if bytes.Compare(key.Bytes(), otherKey.Bytes()) != 0 {
+ return ErrStateNodeSetNotEqual
+ }
+ }
+ // Check DKG Complaints, here I assume the addition sequence of complaints
+ // proposed by one node would be identical on each node (this should be true
+ // when state change requests are carried by blocks and executed in order).
+ if len(s.dkgComplaints) != len(other.dkgComplaints) {
+ return ErrStateDKGComplaintsNotEqual
+ }
+ for round, compsForRound := range s.dkgComplaints {
+ otherCompsForRound, exists := other.dkgComplaints[round]
+ if !exists {
+ return ErrStateDKGComplaintsNotEqual
+ }
+ if len(compsForRound) != len(otherCompsForRound) {
+ return ErrStateDKGComplaintsNotEqual
+ }
+ for nID, comps := range compsForRound {
+ otherComps, exists := otherCompsForRound[nID]
+ if !exists {
+ return ErrStateDKGComplaintsNotEqual
+ }
+ if len(comps) != len(otherComps) {
+ return ErrStateDKGComplaintsNotEqual
+ }
+ for idx, comp := range comps {
+ if !comp.Equal(otherComps[idx]) {
+ return ErrStateDKGComplaintsNotEqual
+ }
+ }
+ }
+ }
+ // Check DKG master public keys.
+ if len(s.dkgMasterPublicKeys) != len(other.dkgMasterPublicKeys) {
+ return ErrStateDKGMasterPublicKeysNotEqual
+ }
+ for round, mKeysForRound := range s.dkgMasterPublicKeys {
+ otherMKeysForRound, exists := other.dkgMasterPublicKeys[round]
+ if !exists {
+ return ErrStateDKGMasterPublicKeysNotEqual
+ }
+ if len(mKeysForRound) != len(otherMKeysForRound) {
+ return ErrStateDKGMasterPublicKeysNotEqual
+ }
+ for nID, mKey := range mKeysForRound {
+ otherMKey, exists := otherMKeysForRound[nID]
+ if !exists {
+ return ErrStateDKGMasterPublicKeysNotEqual
+ }
+ if !mKey.Equal(otherMKey) {
+ return ErrStateDKGMasterPublicKeysNotEqual
+ }
+ }
+ }
+ // Check DKG finals.
+ if len(s.dkgFinals) != len(other.dkgFinals) {
+ return ErrStateDKGFinalsNotEqual
+ }
+ for round, finalsForRound := range s.dkgFinals {
+ otherFinalsForRound, exists := other.dkgFinals[round]
+ if !exists {
+ return ErrStateDKGFinalsNotEqual
+ }
+ if len(finalsForRound) != len(otherFinalsForRound) {
+ return ErrStateDKGFinalsNotEqual
+ }
+ for nID, final := range finalsForRound {
+ otherFinal, exists := otherFinalsForRound[nID]
+ if !exists {
+ return ErrStateDKGFinalsNotEqual
+ }
+ if !final.Equal(otherFinal) {
+ return ErrStateDKGFinalsNotEqual
+ }
+ }
+ }
+ // Check CRS part.
+ if len(s.crs) != len(other.crs) {
+ return ErrStateCRSsNotEqual
+ }
+ for idx, crs := range s.crs {
+ if crs != other.crs[idx] {
+ return ErrStateCRSsNotEqual
+ }
+ }
+ // Check pending changes.
+ if !reflect.DeepEqual(
+ s.pendingChangedConfigs, other.pendingChangedConfigs) {
+ return ErrStatePendingChangesNotEqual
+ }
+ if !reflect.DeepEqual(s.pendingCRS, other.pendingCRS) {
+ return ErrStatePendingChangesNotEqual
+ }
+ if !reflect.DeepEqual(s.pendingNodes, other.pendingNodes) {
+ return ErrStatePendingChangesNotEqual
+ }
+ // Check pending DKG complaints.
+ if len(s.pendingDKGComplaints) != len(other.pendingDKGComplaints) {
+ return ErrStatePendingChangesNotEqual
+ }
+ for idx, comp := range s.pendingDKGComplaints {
+ if !comp.Equal(other.pendingDKGComplaints[idx]) {
+ return ErrStatePendingChangesNotEqual
+ }
+ }
+ // Check pending DKG finals.
+ if len(s.pendingDKGFinals) != len(other.pendingDKGFinals) {
+ return ErrStatePendingChangesNotEqual
+ }
+ for idx, final := range s.pendingDKGFinals {
+ if !final.Equal(other.pendingDKGFinals[idx]) {
+ return ErrStatePendingChangesNotEqual
+ }
+ }
+ // Check pending DKG Master public keys.
+ if len(s.pendingDKGMasterPublicKeys) !=
+ len(other.pendingDKGMasterPublicKeys) {
+ return ErrStatePendingChangesNotEqual
+ }
+ for idx, mKey := range s.pendingDKGMasterPublicKeys {
+ if !mKey.Equal(other.pendingDKGMasterPublicKeys[idx]) {
+ return ErrStatePendingChangesNotEqual
+ }
+ }
+ return nil
+}
+
+// Clone returns a copied State instance.
+func (s *State) Clone() (copied *State) {
+ // Clone configuration parts.
+ copied = &State{
+ numChains: s.numChains,
+ lambdaBA: s.lambdaBA,
+ lambdaDKG: s.lambdaDKG,
+ k: s.k,
+ phiRatio: s.phiRatio,
+ notarySetSize: s.notarySetSize,
+ dkgSetSize: s.dkgSetSize,
+ roundInterval: s.roundInterval,
+ minBlockInterval: s.minBlockInterval,
+ maxBlockInterval: s.maxBlockInterval,
+ local: s.local,
+ nodes: make(map[types.NodeID]crypto.PublicKey),
+ dkgComplaints: make(
+ map[uint64]map[types.NodeID][]*typesDKG.Complaint),
+ dkgMasterPublicKeys: make(
+ map[uint64]map[types.NodeID]*typesDKG.MasterPublicKey),
+ dkgFinals: make(map[uint64]map[types.NodeID]*typesDKG.Finalize),
+ pendingChangedConfigs: make(map[StateChangeType]interface{}),
+ }
+ // Nodes
+ for nID, key := range s.nodes {
+ copied.nodes[nID] = key
+ }
+ // DKG & CRS
+ for round, complaintsForRound := range s.dkgComplaints {
+ copied.dkgComplaints[round] =
+ make(map[types.NodeID][]*typesDKG.Complaint)
+ for nID, comps := range complaintsForRound {
+ tmpComps := []*typesDKG.Complaint{}
+ for _, comp := range comps {
+ tmpComps = append(tmpComps, s.cloneDKGComplaint(comp))
+ }
+ copied.dkgComplaints[round][nID] = tmpComps
+ }
+ }
+ for round, mKeysForRound := range s.dkgMasterPublicKeys {
+ copied.dkgMasterPublicKeys[round] =
+ make(map[types.NodeID]*typesDKG.MasterPublicKey)
+ for nID, mKey := range mKeysForRound {
+ copied.dkgMasterPublicKeys[round][nID] =
+ s.cloneDKGMasterPublicKey(mKey)
+ }
+ }
+ for round, finalsForRound := range s.dkgFinals {
+ copied.dkgFinals[round] = make(map[types.NodeID]*typesDKG.Finalize)
+ for nID, final := range finalsForRound {
+ copied.dkgFinals[round][nID] = s.cloneDKGFinalize(final)
+ }
+ }
+ for _, crs := range s.crs {
+ copied.crs = append(copied.crs, crs)
+ }
+ // Pending Changes
+ for t, v := range s.pendingChangedConfigs {
+ copied.pendingChangedConfigs[t] = v
+ }
+ for _, bs := range s.pendingNodes {
+ tmpBytes := make([]byte, len(bs))
+ copy(tmpBytes, bs)
+ copied.pendingNodes = append(copied.pendingNodes, tmpBytes)
+ }
+ for _, comp := range s.pendingDKGComplaints {
+ copied.pendingDKGComplaints = append(
+ copied.pendingDKGComplaints, s.cloneDKGComplaint(comp))
+ }
+ for _, final := range s.pendingDKGFinals {
+ copied.pendingDKGFinals = append(
+ copied.pendingDKGFinals, s.cloneDKGFinalize(final))
+ }
+ for _, mKey := range s.pendingDKGMasterPublicKeys {
+ copied.pendingDKGMasterPublicKeys = append(
+ copied.pendingDKGMasterPublicKeys, s.cloneDKGMasterPublicKey(mKey))
+ }
+ for _, req := range s.pendingCRS {
+ copied.pendingCRS = append(copied.pendingCRS, &crsAdditionRequest{
+ Round: req.Round,
+ CRS: req.CRS,
+ })
+ }
+ return
+}
+
// Apply change requests, this function would also
// be called when we extract these request from delivered blocks.
func (s *State) Apply(reqsAsBytes []byte) (err error) {
@@ -527,15 +833,7 @@ func (s *State) DKGComplaints(round uint64) []*typesDKG.Complaint {
tmpComps := make([]*typesDKG.Complaint, 0, len(comps))
for _, compProp := range comps {
for _, comp := range compProp {
- bytes, err := rlp.EncodeToBytes(comp)
- if err != nil {
- panic(err)
- }
- compCopy := &typesDKG.Complaint{}
- if err = rlp.DecodeBytes(bytes, compCopy); err != nil {
- panic(err)
- }
- tmpComps = append(tmpComps, compCopy)
+ tmpComps = append(tmpComps, s.cloneDKGComplaint(comp))
}
}
return tmpComps
@@ -553,16 +851,7 @@ func (s *State) DKGMasterPublicKeys(round uint64) []*typesDKG.MasterPublicKey {
}
mpks := make([]*typesDKG.MasterPublicKey, 0, len(masterPublicKeys))
for _, mpk := range masterPublicKeys {
- // Return a deep copied master public keys.
- b, err := rlp.EncodeToBytes(mpk)
- if err != nil {
- panic(err)
- }
- mpkCopy := typesDKG.NewMasterPublicKey()
- if err = rlp.DecodeBytes(b, mpkCopy); err != nil {
- panic(err)
- }
- mpks = append(mpks, mpkCopy)
+ mpks = append(mpks, s.cloneDKGMasterPublicKey(mpk))
}
return mpks
}
diff --git a/core/test/state_test.go b/core/test/state_test.go
index ebb5a25..59b8f63 100644
--- a/core/test/state_test.go
+++ b/core/test/state_test.go
@@ -82,15 +82,6 @@ func (s *StateTestSuite) newDKGFinal(round uint64) *typesDKG.Finalize {
}
}
-func (s *StateTestSuite) genNodes(count int) (nodes []crypto.PublicKey) {
- for i := 0; i < count; i++ {
- prv, err := ecdsa.NewPrivateKey()
- s.Require().NoError(err)
- nodes = append(nodes, prv.PublicKey())
- }
- return
-}
-
func (s *StateTestSuite) compareNodes(node1, node2 []crypto.PublicKey) bool {
id1 := common.Hashes{}
for _, n := range node1 {
@@ -162,13 +153,103 @@ func (s *StateTestSuite) checkConfigChanges(config *types.Config) {
req.Equal(config.DKGSetSize, uint32(6))
}
+func (s *StateTestSuite) TestEqual() {
+ var (
+ req = s.Require()
+ lambda = 250 * time.Millisecond
+ )
+ _, genesisNodes, err := NewKeys(20)
+ req.NoError(err)
+ st := NewState(genesisNodes, lambda, true)
+ req.NoError(st.Equal(st))
+ // One node is missing.
+ st1 := NewState(genesisNodes, lambda, true)
+ for nID := range st1.nodes {
+ delete(st1.nodes, nID)
+ break
+ }
+ req.Equal(st.Equal(st1), ErrStateNodeSetNotEqual)
+ // Make some changes.
+ st2 := st.Clone()
+ req.NoError(st.Equal(st2))
+ s.makeConfigChanges(st)
+ req.Equal(st.Equal(st2), ErrStateConfigNotEqual)
+ crs := common.NewRandomHash()
+ req.NoError(st.ProposeCRS(1, crs))
+ masterPubKey := s.newDKGMasterPublicKey(2)
+ comp := s.newDKGComplaint(2)
+ final := s.newDKGFinal(2)
+ s.makeDKGChanges(st, masterPubKey, comp, final)
+ // Remove dkg complaints from cloned one to check if equal.
+ st3 := st.Clone()
+ req.NoError(st.Equal(st3))
+ delete(st3.dkgComplaints, uint64(2))
+ req.Equal(st.Equal(st3), ErrStateDKGComplaintsNotEqual)
+ // Remove dkg master public key from cloned one to check if equal.
+ st4 := st.Clone()
+ req.NoError(st.Equal(st4))
+ delete(st4.dkgMasterPublicKeys, uint64(2))
+ req.Equal(st.Equal(st4), ErrStateDKGMasterPublicKeysNotEqual)
+ // Remove dkg finalize from cloned one to check if equal.
+ st5 := st.Clone()
+ req.NoError(st.Equal(st5))
+ delete(st5.dkgFinals, uint64(2))
+ req.Equal(st.Equal(st5), ErrStateDKGFinalsNotEqual)
+}
+
+func (s *StateTestSuite) TestPendingChangesEqual() {
+ var (
+ req = s.Require()
+ lambda = 250 * time.Millisecond
+ )
+ // Setup a non-local mode State instance.
+ _, genesisNodes, err := NewKeys(20)
+ req.NoError(err)
+ st := NewState(genesisNodes, lambda, false)
+ req.NoError(st.Equal(st))
+ // Apply some changes.
+ s.makeConfigChanges(st)
+ crs := common.NewRandomHash()
+ req.NoError(st.ProposeCRS(1, crs))
+ masterPubKey := s.newDKGMasterPublicKey(2)
+ comp := s.newDKGComplaint(2)
+ final := s.newDKGFinal(2)
+ s.makeDKGChanges(st, masterPubKey, comp, final)
+ // Remove pending config changes.
+ st1 := st.Clone()
+ req.NoError(st.Equal(st1))
+ st1.pendingChangedConfigs = make(map[StateChangeType]interface{})
+ req.Equal(st.Equal(st1), ErrStatePendingChangesNotEqual)
+ // Remove pending crs changes.
+ st2 := st.Clone()
+ req.NoError(st.Equal(st2))
+ st2.pendingCRS = []*crsAdditionRequest{}
+ req.Equal(st.Equal(st2), ErrStatePendingChangesNotEqual)
+ // Remove pending dkg complaints changes.
+ st3 := st.Clone()
+ req.NoError(st.Equal(st3))
+ st3.pendingDKGComplaints = []*typesDKG.Complaint{}
+ req.Equal(st.Equal(st3), ErrStatePendingChangesNotEqual)
+ // Remove pending dkg master public key changes.
+ st4 := st.Clone()
+ req.NoError(st.Equal(st4))
+ st4.pendingDKGMasterPublicKeys = []*typesDKG.MasterPublicKey{}
+ req.Equal(st.Equal(st4), ErrStatePendingChangesNotEqual)
+ // Remove pending dkg finalize changes.
+ st5 := st.Clone()
+ req.NoError(st.Equal(st5))
+ st5.pendingDKGFinals = []*typesDKG.Finalize{}
+ req.Equal(st.Equal(st5), ErrStatePendingChangesNotEqual)
+}
+
func (s *StateTestSuite) TestLocalMode() {
// Test State with local mode.
var (
req = s.Require()
lambda = 250 * time.Millisecond
)
- genesisNodes := s.genNodes(20)
+ _, genesisNodes, err := NewKeys(20)
+ req.NoError(err)
st := NewState(genesisNodes, lambda, true)
config1, nodes1 := st.Snapshot()
req.True(s.compareNodes(genesisNodes, nodes1))
@@ -228,7 +309,8 @@ func (s *StateTestSuite) TestPacking() {
lambda = 250 * time.Millisecond
)
// Make config changes.
- genesisNodes := s.genNodes(20)
+ _, genesisNodes, err := NewKeys(20)
+ req.NoError(err)
st := NewState(genesisNodes, lambda, false)
s.makeConfigChanges(st)
// Add new CRS.