aboutsummaryrefslogtreecommitdiffstats
path: root/core/test/state.go
diff options
context:
space:
mode:
Diffstat (limited to 'core/test/state.go')
-rw-r--r--core/test/state.go395
1 files changed, 142 insertions, 253 deletions
diff --git a/core/test/state.go b/core/test/state.go
index 05394ed..2bb04e5 100644
--- a/core/test/state.go
+++ b/core/test/state.go
@@ -21,7 +21,6 @@ import (
"bytes"
"errors"
"math"
- "reflect"
"sync"
"time"
@@ -33,9 +32,6 @@ import (
"github.com/dexon-foundation/dexon/rlp"
)
-// StateChangeType is the type of state change request.
-type StateChangeType uint8
-
var (
// ErrDuplicatedChange means the change request is already applied.
ErrDuplicatedChange = errors.New("duplicated change")
@@ -70,29 +66,10 @@ var (
// ErrStatePendingChangesNotEqual means pending change requests of two
// states are not equal.
ErrStatePendingChangesNotEqual = errors.New("pending changes not equal")
-)
-
-// Types of state change.
-const (
- StateChangeNothing StateChangeType = iota
- // DKG & CRS
- StateAddCRS
- StateAddDKGComplaint
- StateAddDKGMasterPublicKey
- StateAddDKGFinal
- // Configuration related.
- StateChangeNumChains
- StateChangeLambdaBA
- StateChangeLambdaDKG
- StateChangeRoundInterval
- StateChangeMinBlockInterval
- StateChangeMaxBlockInterval
- StateChangeK
- StateChangePhiRatio
- StateChangeNotarySetSize
- StateChangeDKGSetSize
- // Node set related.
- StateAddNode
+ // ErrNotInRemoteMode means callers attempts to call functions for remote
+ // mode when the State instance is still in local mode.
+ ErrNotInRemoteMode = errors.New(
+ "attempting to use remote functions in local mode")
)
type crsAdditionRequest struct {
@@ -100,17 +77,6 @@ type crsAdditionRequest struct {
CRS common.Hash `json:"crs"`
}
-// StateChangeRequest carries information of state change request.
-type StateChangeRequest struct {
- Type StateChangeType `json:"type"`
- Payload interface{} `json:"payload"`
-}
-
-type rawStateChangeRequest struct {
- Type StateChangeType
- Payload rlp.RawValue
-}
-
// State emulates what the global state in governace contract on a fullnode.
type State struct {
// Configuration related.
@@ -134,16 +100,9 @@ type State struct {
// Other stuffs
local bool
lock sync.RWMutex
- // ChangeRequest(s) are organized as map, indexed by type of state change.
- // For each time to apply state change, only the last request would be
- // applied.
- pendingChangedConfigs map[StateChangeType]interface{}
- pendingNodes [][]byte
- pendingDKGComplaints []*typesDKG.Complaint
- pendingDKGFinals []*typesDKG.Finalize
- pendingDKGMasterPublicKeys []*typesDKG.MasterPublicKey
- pendingCRS []*crsAdditionRequest
- pendingChangesLock sync.Mutex
+ // Pending change requests.
+ ownRequests map[common.Hash]*StateChangeRequest
+ globalRequests map[common.Hash]*StateChangeRequest
}
// NewState constructs an State instance with genesis information, including:
@@ -157,20 +116,21 @@ func NewState(
}
genesisCRS := crypto.Keccak256Hash([]byte("__ DEXON"))
return &State{
- local: local,
- numChains: uint32(len(nodes)),
- lambdaBA: lambda,
- lambdaDKG: lambda * 10,
- roundInterval: lambda * 10000,
- minBlockInterval: time.Millisecond * 1,
- maxBlockInterval: lambda * 8,
- crs: []common.Hash{genesisCRS},
- nodes: nodes,
- phiRatio: 0.667,
- k: 0,
- notarySetSize: uint32(len(nodes)),
- dkgSetSize: uint32(len(nodes)),
- pendingChangedConfigs: make(map[StateChangeType]interface{}),
+ local: local,
+ numChains: uint32(len(nodes)),
+ lambdaBA: lambda,
+ lambdaDKG: lambda * 10,
+ roundInterval: lambda * 10000,
+ minBlockInterval: time.Millisecond * 1,
+ maxBlockInterval: lambda * 8,
+ crs: []common.Hash{genesisCRS},
+ nodes: nodes,
+ phiRatio: 0.667,
+ k: 0,
+ notarySetSize: uint32(len(nodes)),
+ dkgSetSize: uint32(len(nodes)),
+ ownRequests: make(map[common.Hash]*StateChangeRequest),
+ globalRequests: make(map[common.Hash]*StateChangeRequest),
dkgFinals: make(
map[uint64]map[types.NodeID]*typesDKG.Finalize),
dkgComplaints: make(
@@ -181,7 +141,8 @@ func NewState(
}
// SwitchToRemoteMode turn this State instance into remote mode: all changes
-// are pending, and need to be packed/unpacked to apply.
+// are pending, and need to be packed/unpacked to apply. Once this state switch
+// to remote mode, there would be no way to switch back to local mode.
func (s *State) SwitchToRemoteMode() {
s.lock.Lock()
defer s.lock.Unlock()
@@ -278,41 +239,24 @@ 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)
+func (s *State) unpackRequests(
+ b []byte) (reqs []*StateChangeRequest, err error) {
+ // Try to unmarshal this byte stream into []*StateChangeRequest.
+ rawReqs := []*rawStateChangeRequest{}
+ if err = rlp.DecodeBytes(b, &rawReqs); err != nil {
+ return
}
- copied = &typesDKG.Finalize{}
- if err = rlp.DecodeBytes(b, copied); err != nil {
- panic(err)
+ for _, r := range rawReqs {
+ var payload interface{}
+ if payload, err = s.unpackPayload(r); err != nil {
+ return
+ }
+ reqs = append(reqs, &StateChangeRequest{
+ Type: r.Type,
+ Payload: payload,
+ Hash: r.Hash,
+ Timestamp: r.Timestamp,
+ })
}
return
}
@@ -433,43 +377,27 @@ func (s *State) Equal(other *State) error {
}
}
// 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]) {
+ checkPending := func(
+ src, target map[common.Hash]*StateChangeRequest) error {
+ if len(src) != len(target) {
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
+ for k, v := range src {
+ otherV, exists := target[k]
+ if !exists {
+ return ErrStatePendingChangesNotEqual
+ }
+ if err := v.Equal(otherV); err != nil {
+ return err
+ }
}
+ return nil
}
- // Check pending DKG Master public keys.
- if len(s.pendingDKGMasterPublicKeys) !=
- len(other.pendingDKGMasterPublicKeys) {
- return ErrStatePendingChangesNotEqual
+ if err := checkPending(s.ownRequests, other.ownRequests); err != nil {
+ return err
}
- for idx, mKey := range s.pendingDKGMasterPublicKeys {
- if !mKey.Equal(other.pendingDKGMasterPublicKeys[idx]) {
- return ErrStatePendingChangesNotEqual
- }
+ if err := checkPending(s.globalRequests, other.globalRequests); err != nil {
+ return err
}
return nil
}
@@ -494,8 +422,7 @@ func (s *State) Clone() (copied *State) {
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{}),
+ dkgFinals: make(map[uint64]map[types.NodeID]*typesDKG.Finalize),
}
// Nodes
for nID, key := range s.nodes {
@@ -508,7 +435,7 @@ func (s *State) Clone() (copied *State) {
for nID, comps := range complaintsForRound {
tmpComps := []*typesDKG.Complaint{}
for _, comp := range comps {
- tmpComps = append(tmpComps, s.cloneDKGComplaint(comp))
+ tmpComps = append(tmpComps, cloneDKGComplaint(comp))
}
copied.dkgComplaints[round][nID] = tmpComps
}
@@ -518,44 +445,26 @@ func (s *State) Clone() (copied *State) {
make(map[types.NodeID]*typesDKG.MasterPublicKey)
for nID, mKey := range mKeysForRound {
copied.dkgMasterPublicKeys[round][nID] =
- s.cloneDKGMasterPublicKey(mKey)
+ 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)
+ copied.dkgFinals[round][nID] = 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,
- })
+ copied.ownRequests = make(map[common.Hash]*StateChangeRequest)
+ for k, req := range s.ownRequests {
+ copied.ownRequests[k] = req.Clone()
+ }
+ copied.globalRequests = make(map[common.Hash]*StateChangeRequest)
+ for k, req := range s.globalRequests {
+ copied.globalRequests[k] = req.Clone()
}
return
}
@@ -564,24 +473,16 @@ func (s *State) Clone() (copied *State) {
// be called when we extract these request from delivered blocks.
func (s *State) Apply(reqsAsBytes []byte) (err error) {
// Try to unmarshal this byte stream into []*StateChangeRequest.
- rawReqs := []*rawStateChangeRequest{}
- if err = rlp.DecodeBytes(reqsAsBytes, &rawReqs); err != nil {
+ reqs, err := s.unpackRequests(reqsAsBytes)
+ if err != nil {
return
}
- var reqs []*StateChangeRequest
- for _, r := range rawReqs {
- var payload interface{}
- if payload, err = s.unpackPayload(r); err != nil {
- return
- }
- reqs = append(reqs, &StateChangeRequest{
- Type: r.Type,
- Payload: payload,
- })
- }
s.lock.Lock()
defer s.lock.Unlock()
for _, req := range reqs {
+ // Remove this request from pending set once it's about to apply.
+ delete(s.globalRequests, req.Hash)
+ delete(s.ownRequests, req.Hash)
if err = s.isValidRequest(req); err != nil {
if err == ErrDuplicatedChange {
err = nil
@@ -596,53 +497,57 @@ func (s *State) Apply(reqsAsBytes []byte) (err error) {
return
}
-// PackRequests pack current pending requests as byte slice, which
-// could be sent as blocks' payload and unmarshall back to apply.
-func (s *State) PackRequests() (b []byte, err error) {
- packed := []*StateChangeRequest{}
- s.pendingChangesLock.Lock()
- defer s.pendingChangesLock.Unlock()
- // Pack simple configuration changes first. There should be no
- // validity problems for those changes.
- for k, v := range s.pendingChangedConfigs {
- packed = append(packed, &StateChangeRequest{
- Type: k,
- Payload: v,
- })
+// AddRequestsFromOthers add requests from others, they won't be packed by
+// 'PackOwnRequests'.
+func (s *State) AddRequestsFromOthers(reqsAsBytes []byte) (err error) {
+ if s.local {
+ err = ErrNotInRemoteMode
+ return
}
- s.pendingChangedConfigs = make(map[StateChangeType]interface{})
- // For other changes, we need to check their validity.
- s.lock.RLock()
- defer s.lock.RUnlock()
- for _, bytesOfKey := range s.pendingNodes {
- packed = append(packed, &StateChangeRequest{
- Type: StateAddNode,
- Payload: bytesOfKey,
- })
+ reqs, err := s.unpackRequests(reqsAsBytes)
+ if err != nil {
+ return
}
- for _, comp := range s.pendingDKGComplaints {
- packed = append(packed, &StateChangeRequest{
- Type: StateAddDKGComplaint,
- Payload: comp,
- })
+ s.lock.Lock()
+ defer s.lock.Unlock()
+ for _, req := range reqs {
+ s.globalRequests[req.Hash] = req
}
- for _, final := range s.pendingDKGFinals {
- packed = append(packed, &StateChangeRequest{
- Type: StateAddDKGFinal,
- Payload: final,
- })
+ return
+}
+
+// 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
}
- for _, masterPubKey := range s.pendingDKGMasterPublicKeys {
- packed = append(packed, &StateChangeRequest{
- Type: StateAddDKGMasterPublicKey,
- Payload: masterPubKey,
- })
+ // Pack requests in global pool.
+ packed := []*StateChangeRequest{}
+ s.lock.Lock()
+ defer s.lock.Unlock()
+ for _, v := range s.globalRequests {
+ packed = append(packed, v)
}
- for _, crs := range s.pendingCRS {
- packed = append(packed, &StateChangeRequest{
- Type: StateAddCRS,
- Payload: crs,
- })
+ return rlp.EncodeToBytes(packed)
+}
+
+// PackOwnRequests pack current pending requests as byte slice, which
+// could be sent as blocks' payload and unmarshall back to apply.
+//
+// Once a request is packed as own request, it would be turned into a normal
+// pending request and won't be packed by this function. This would ensure
+// each request broadcasted(gossip) once.
+//
+// This function is not required to call in local mode.
+func (s *State) PackOwnRequests() (b []byte, err error) {
+ packed := []*StateChangeRequest{}
+ s.lock.Lock()
+ defer s.lock.Unlock()
+ for k, v := range s.ownRequests {
+ delete(s.ownRequests, k)
+ s.globalRequests[k] = v
+ packed = append(packed, v)
}
if b, err = rlp.EncodeToBytes(packed); err != nil {
return
@@ -782,45 +687,29 @@ func (s *State) RequestChange(
payload = uint64(payload.(int))
case StateChangePhiRatio:
payload = math.Float32bits(payload.(float32))
- }
- req := &StateChangeRequest{
- Type: t,
- Payload: payload,
- }
- if s.local {
- err = func() error {
- s.lock.Lock()
- defer s.lock.Unlock()
- if err := s.isValidRequest(req); err != nil {
- return err
- }
- return s.applyRequest(req)
- }()
- return
- }
- s.lock.RLock()
- defer s.lock.RUnlock()
- if err = s.isValidRequest(req); err != nil {
- return
- }
- s.pendingChangesLock.Lock()
- defer s.pendingChangesLock.Unlock()
- switch t {
- case StateAddNode:
- s.pendingNodes = append(s.pendingNodes, payload.([]byte))
+ // These cases for for type assertion, make sure callers pass expected types.
case StateAddCRS:
- s.pendingCRS = append(s.pendingCRS, payload.(*crsAdditionRequest))
- case StateAddDKGComplaint:
- s.pendingDKGComplaints = append(
- s.pendingDKGComplaints, payload.(*typesDKG.Complaint))
- case StateAddDKGMasterPublicKey:
- s.pendingDKGMasterPublicKeys = append(
- s.pendingDKGMasterPublicKeys, payload.(*typesDKG.MasterPublicKey))
+ payload = payload.(*crsAdditionRequest)
case StateAddDKGFinal:
- s.pendingDKGFinals = append(
- s.pendingDKGFinals, payload.(*typesDKG.Finalize))
- default:
- s.pendingChangedConfigs[t] = payload
+ payload = payload.(*typesDKG.Finalize)
+ case StateAddDKGMasterPublicKey:
+ payload = payload.(*typesDKG.MasterPublicKey)
+ case StateAddDKGComplaint:
+ payload = payload.(*typesDKG.Complaint)
+ }
+ req := NewStateChangeRequest(t, payload)
+ s.lock.Lock()
+ defer s.lock.Unlock()
+ if s.local {
+ if err = s.isValidRequest(req); err != nil {
+ return
+ }
+ err = s.applyRequest(req)
+ } else {
+ if err = s.isValidRequest(req); err != nil {
+ return
+ }
+ s.ownRequests[req.Hash] = req
}
return
}
@@ -847,7 +736,7 @@ func (s *State) DKGComplaints(round uint64) []*typesDKG.Complaint {
tmpComps := make([]*typesDKG.Complaint, 0, len(comps))
for _, compProp := range comps {
for _, comp := range compProp {
- tmpComps = append(tmpComps, s.cloneDKGComplaint(comp))
+ tmpComps = append(tmpComps, cloneDKGComplaint(comp))
}
}
return tmpComps
@@ -865,7 +754,7 @@ func (s *State) DKGMasterPublicKeys(round uint64) []*typesDKG.MasterPublicKey {
}
mpks := make([]*typesDKG.MasterPublicKey, 0, len(masterPublicKeys))
for _, mpk := range masterPublicKeys {
- mpks = append(mpks, s.cloneDKGMasterPublicKey(mpk))
+ mpks = append(mpks, cloneDKGMasterPublicKey(mpk))
}
return mpks
}