p2p/nodestate: ensure correct callback order (#21436)

This PR adds an extra guarantee to NodeStateMachine: it ensures that all
immediate effects of a certain change are processed before any subsequent
effects of any of the immediate effects on the same node. In the original
version, if a cascaded change caused a subscription callback to be called
multiple times for the same node then these calls might have happened in a
wrong chronological order.

For example:

- a subscription to flag0 changes flag1 and flag2
- a subscription to flag1 changes flag3
- a subscription to flag1, flag2 and flag3 was called in the following order:

   [flag1] -> [flag1, flag3]
   [] -> [flag1]
   [flag1, flag3] -> [flag1, flag2, flag3]

This happened because the tree of changes was traversed in a "depth-first
order". Now it is traversed in a "breadth-first order"; each node has a
FIFO queue for pending callbacks and each triggered subscription callback
is added to the end of the list. The already existing guarantees are
retained; no SetState or SetField returns until the callback queue of the
node is empty again. Just like before, it is the responsibility of the
state machine design to ensure that infinite state loops are not possible.
Multiple changes affecting the same node can still happen simultaneously;
in this case the changes can be interleaved in the FIFO of the node but the
correct order is still guaranteed.

A new unit test is also added to verify callback order in the above scenario.
pull/21563/head
Felföldi Zsolt 4 years ago committed by GitHub
parent a99ac5335c
commit 4eb9296910
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 83
      les/serverpool.go
  2. 344
      p2p/nodestate/nodestate.go
  3. 56
      p2p/nodestate/nodestate_test.go

@ -166,7 +166,7 @@ func newServerPool(db ethdb.KeyValueStore, dbKey []byte, vt *lpc.ValueTracker, d
if oldState.Equals(sfWaitDialTimeout) && newState.IsEmpty() { if oldState.Equals(sfWaitDialTimeout) && newState.IsEmpty() {
// dial timeout, no connection // dial timeout, no connection
s.setRedialWait(n, dialCost, dialWaitStep) s.setRedialWait(n, dialCost, dialWaitStep)
s.ns.SetState(n, nodestate.Flags{}, sfDialing, 0) s.ns.SetStateSub(n, nodestate.Flags{}, sfDialing, 0)
} }
}) })
@ -193,10 +193,10 @@ func (s *serverPool) addPreNegFilter(input enode.Iterator, query queryFunc) enod
if rand.Intn(maxQueryFails*2) < int(fails) { if rand.Intn(maxQueryFails*2) < int(fails) {
// skip pre-negotiation with increasing chance, max 50% // skip pre-negotiation with increasing chance, max 50%
// this ensures that the client can operate even if UDP is not working at all // this ensures that the client can operate even if UDP is not working at all
s.ns.SetState(n, sfCanDial, nodestate.Flags{}, time.Second*10) s.ns.SetStateSub(n, sfCanDial, nodestate.Flags{}, time.Second*10)
// set canDial before resetting queried so that FillSet will not read more // set canDial before resetting queried so that FillSet will not read more
// candidates unnecessarily // candidates unnecessarily
s.ns.SetState(n, nodestate.Flags{}, sfQueried, 0) s.ns.SetStateSub(n, nodestate.Flags{}, sfQueried, 0)
return return
} }
go func() { go func() {
@ -206,12 +206,15 @@ func (s *serverPool) addPreNegFilter(input enode.Iterator, query queryFunc) enod
} else { } else {
atomic.StoreUint32(&s.queryFails, 0) atomic.StoreUint32(&s.queryFails, 0)
} }
if q == 1 { s.ns.Operation(func() {
s.ns.SetState(n, sfCanDial, nodestate.Flags{}, time.Second*10) // we are no longer running in the operation that the callback belongs to, start a new one because of setRedialWait
} else { if q == 1 {
s.setRedialWait(n, queryCost, queryWaitStep) s.ns.SetStateSub(n, sfCanDial, nodestate.Flags{}, time.Second*10)
} } else {
s.ns.SetState(n, nodestate.Flags{}, sfQueried, 0) s.setRedialWait(n, queryCost, queryWaitStep)
}
s.ns.SetStateSub(n, nodestate.Flags{}, sfQueried, 0)
})
}() }()
} }
}) })
@ -240,18 +243,20 @@ func (s *serverPool) start() {
} }
} }
unixTime := s.unixTime() unixTime := s.unixTime()
s.ns.ForEach(sfHasValue, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) { s.ns.Operation(func() {
s.calculateWeight(node) s.ns.ForEach(sfHasValue, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) {
if n, ok := s.ns.GetField(node, sfiNodeHistory).(nodeHistory); ok && n.redialWaitEnd > unixTime { s.calculateWeight(node)
wait := n.redialWaitEnd - unixTime if n, ok := s.ns.GetField(node, sfiNodeHistory).(nodeHistory); ok && n.redialWaitEnd > unixTime {
lastWait := n.redialWaitEnd - n.redialWaitStart wait := n.redialWaitEnd - unixTime
if wait > lastWait { lastWait := n.redialWaitEnd - n.redialWaitStart
// if the time until expiration is larger than the last suggested if wait > lastWait {
// waiting time then the system clock was probably adjusted // if the time until expiration is larger than the last suggested
wait = lastWait // waiting time then the system clock was probably adjusted
wait = lastWait
}
s.ns.SetStateSub(node, sfRedialWait, nodestate.Flags{}, time.Duration(wait)*time.Second)
} }
s.ns.SetState(node, sfRedialWait, nodestate.Flags{}, time.Duration(wait)*time.Second) })
}
}) })
} }
@ -261,9 +266,11 @@ func (s *serverPool) stop() {
if s.fillSet != nil { if s.fillSet != nil {
s.fillSet.Close() s.fillSet.Close()
} }
s.ns.ForEach(sfConnected, nodestate.Flags{}, func(n *enode.Node, state nodestate.Flags) { s.ns.Operation(func() {
// recalculate weight of connected nodes in order to update hasValue flag if necessary s.ns.ForEach(sfConnected, nodestate.Flags{}, func(n *enode.Node, state nodestate.Flags) {
s.calculateWeight(n) // recalculate weight of connected nodes in order to update hasValue flag if necessary
s.calculateWeight(n)
})
}) })
s.ns.Stop() s.ns.Stop()
} }
@ -279,9 +286,11 @@ func (s *serverPool) registerPeer(p *serverPeer) {
// unregisterPeer implements serverPeerSubscriber // unregisterPeer implements serverPeerSubscriber
func (s *serverPool) unregisterPeer(p *serverPeer) { func (s *serverPool) unregisterPeer(p *serverPeer) {
s.setRedialWait(p.Node(), dialCost, dialWaitStep) s.ns.Operation(func() {
s.ns.SetState(p.Node(), nodestate.Flags{}, sfConnected, 0) s.setRedialWait(p.Node(), dialCost, dialWaitStep)
s.ns.SetField(p.Node(), sfiConnectedStats, nil) s.ns.SetStateSub(p.Node(), nodestate.Flags{}, sfConnected, 0)
s.ns.SetFieldSub(p.Node(), sfiConnectedStats, nil)
})
s.vt.Unregister(p.ID()) s.vt.Unregister(p.ID())
p.setValueTracker(nil, nil) p.setValueTracker(nil, nil)
} }
@ -380,14 +389,16 @@ func (s *serverPool) serviceValue(node *enode.Node) (sessionValue, totalValue fl
// updateWeight calculates the node weight and updates the nodeWeight field and the // updateWeight calculates the node weight and updates the nodeWeight field and the
// hasValue flag. It also saves the node state if necessary. // hasValue flag. It also saves the node state if necessary.
// Note: this function should run inside a NodeStateMachine operation
func (s *serverPool) updateWeight(node *enode.Node, totalValue float64, totalDialCost uint64) { func (s *serverPool) updateWeight(node *enode.Node, totalValue float64, totalDialCost uint64) {
weight := uint64(totalValue * nodeWeightMul / float64(totalDialCost)) weight := uint64(totalValue * nodeWeightMul / float64(totalDialCost))
if weight >= nodeWeightThreshold { if weight >= nodeWeightThreshold {
s.ns.SetState(node, sfHasValue, nodestate.Flags{}, 0) s.ns.SetStateSub(node, sfHasValue, nodestate.Flags{}, 0)
s.ns.SetField(node, sfiNodeWeight, weight) s.ns.SetFieldSub(node, sfiNodeWeight, weight)
} else { } else {
s.ns.SetState(node, nodestate.Flags{}, sfHasValue, 0) s.ns.SetStateSub(node, nodestate.Flags{}, sfHasValue, 0)
s.ns.SetField(node, sfiNodeWeight, nil) s.ns.SetFieldSub(node, sfiNodeWeight, nil)
s.ns.SetFieldSub(node, sfiNodeHistory, nil)
} }
s.ns.Persist(node) // saved if node history or hasValue changed s.ns.Persist(node) // saved if node history or hasValue changed
} }
@ -400,6 +411,7 @@ func (s *serverPool) updateWeight(node *enode.Node, totalValue float64, totalDia
// a significant amount of service value again its waiting time is quickly reduced or reset // a significant amount of service value again its waiting time is quickly reduced or reset
// to the minimum. // to the minimum.
// Note: node weight is also recalculated and updated by this function. // Note: node weight is also recalculated and updated by this function.
// Note 2: this function should run inside a NodeStateMachine operation
func (s *serverPool) setRedialWait(node *enode.Node, addDialCost int64, waitStep float64) { func (s *serverPool) setRedialWait(node *enode.Node, addDialCost int64, waitStep float64) {
n, _ := s.ns.GetField(node, sfiNodeHistory).(nodeHistory) n, _ := s.ns.GetField(node, sfiNodeHistory).(nodeHistory)
sessionValue, totalValue := s.serviceValue(node) sessionValue, totalValue := s.serviceValue(node)
@ -450,21 +462,22 @@ func (s *serverPool) setRedialWait(node *enode.Node, addDialCost int64, waitStep
if wait < waitThreshold { if wait < waitThreshold {
n.redialWaitStart = unixTime n.redialWaitStart = unixTime
n.redialWaitEnd = unixTime + int64(nextTimeout) n.redialWaitEnd = unixTime + int64(nextTimeout)
s.ns.SetField(node, sfiNodeHistory, n) s.ns.SetFieldSub(node, sfiNodeHistory, n)
s.ns.SetState(node, sfRedialWait, nodestate.Flags{}, wait) s.ns.SetStateSub(node, sfRedialWait, nodestate.Flags{}, wait)
s.updateWeight(node, totalValue, totalDialCost) s.updateWeight(node, totalValue, totalDialCost)
} else { } else {
// discard known node statistics if waiting time is very long because the node // discard known node statistics if waiting time is very long because the node
// hasn't been responsive for a very long time // hasn't been responsive for a very long time
s.ns.SetField(node, sfiNodeHistory, nil) s.ns.SetFieldSub(node, sfiNodeHistory, nil)
s.ns.SetField(node, sfiNodeWeight, nil) s.ns.SetFieldSub(node, sfiNodeWeight, nil)
s.ns.SetState(node, nodestate.Flags{}, sfHasValue, 0) s.ns.SetStateSub(node, nodestate.Flags{}, sfHasValue, 0)
} }
} }
// calculateWeight calculates and sets the node weight without altering the node history. // calculateWeight calculates and sets the node weight without altering the node history.
// This function should be called during startup and shutdown only, otherwise setRedialWait // This function should be called during startup and shutdown only, otherwise setRedialWait
// will keep the weights updated as the underlying statistics are adjusted. // will keep the weights updated as the underlying statistics are adjusted.
// Note: this function should run inside a NodeStateMachine operation
func (s *serverPool) calculateWeight(node *enode.Node) { func (s *serverPool) calculateWeight(node *enode.Node) {
n, _ := s.ns.GetField(node, sfiNodeHistory).(nodeHistory) n, _ := s.ns.GetField(node, sfiNodeHistory).(nodeHistory)
_, totalValue := s.serviceValue(node) _, totalValue := s.serviceValue(node)

@ -32,34 +32,46 @@ import (
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
) )
var (
ErrInvalidField = errors.New("invalid field type")
ErrClosed = errors.New("already closed")
)
type ( type (
// NodeStateMachine connects different system components operating on subsets of // NodeStateMachine implements a network node-related event subscription system.
// network nodes. Node states are represented by 64 bit vectors with each bit assigned // It can assign binary state flags and fields of arbitrary type to each node and allows
// to a state flag. Each state flag has a descriptor structure and the mapping is // subscriptions to flag/field changes which can also modify further flags and fields,
// created automatically. It is possible to subscribe to subsets of state flags and // potentially triggering further subscriptions. An operation includes an initial change
// receive a callback if one of the nodes has a relevant state flag changed. // and all resulting subsequent changes and always ends in a consistent global state.
// Callbacks can also modify further flags of the same node or other nodes. State // It is initiated by a "top level" SetState/SetField call that blocks (also blocking other
// updates only return after all immediate effects throughout the system have happened // top-level functions) until the operation is finished. Callbacks making further changes
// (deadlocks should be avoided by design of the implemented state logic). The caller // should use the non-blocking SetStateSub/SetFieldSub functions. The tree of events
// can also add timeouts assigned to a certain node and a subset of state flags. // resulting from the initial changes is traversed in a breadth-first order, ensuring for
// If the timeout elapses, the flags are reset. If all relevant flags are reset then // each subscription callback that all other callbacks caused by the same change triggering
// the timer is dropped. State flags with no timeout are persisted in the database // the current callback are processed before anything is triggered by the changes made in the
// if the flag descriptor enables saving. If a node has no state flags set at any // current callback. In practice this logic ensures that all subscriptions "see" events in
// moment then it is discarded. // the logical order, callbacks are never called concurrently and "back and forth" effects
// // are also possible. The state machine design should ensure that infinite event cycles
// Extra node fields can also be registered so system components can also store more // cannot happen.
// complex state for each node that is relevant to them, without creating a custom // The caller can also add timeouts assigned to a certain node and a subset of state flags.
// peer set. Fields can be shared across multiple components if they all know the // If the timeout elapses, the flags are reset. If all relevant flags are reset then the timer
// field ID. Subscription to fields is also possible. Persistent fields should have // is dropped. State flags with no timeout are persisted in the database if the flag
// an encoder and a decoder function. // descriptor enables saving. If a node has no state flags set at any moment then it is discarded.
// Note: in order to avoid mutex deadlocks the callbacks should never lock a mutex that
// might be locked when the top level SetState/SetField functions are called. If a function
// potentially performs state/field changes then it is recommended to mention this fact in the
// function description, along with whether it should run inside an operation callback.
NodeStateMachine struct { NodeStateMachine struct {
started, stopped bool started, closed bool
lock sync.Mutex lock sync.Mutex
clock mclock.Clock clock mclock.Clock
db ethdb.KeyValueStore db ethdb.KeyValueStore
dbNodeKey []byte dbNodeKey []byte
nodes map[enode.ID]*nodeInfo nodes map[enode.ID]*nodeInfo
offlineCallbackList []offlineCallback offlineCallbackList []offlineCallback
opFlag bool // an operation has started
opWait *sync.Cond // signaled when the operation ends
opPending []func() // pending callback list of the current operation
// Registered state flags or fields. Modifications are allowed // Registered state flags or fields. Modifications are allowed
// only when the node state machine has not been started. // only when the node state machine has not been started.
@ -128,11 +140,12 @@ type (
// nodeInfo contains node state, fields and state timeouts // nodeInfo contains node state, fields and state timeouts
nodeInfo struct { nodeInfo struct {
node *enode.Node node *enode.Node
state bitMask state bitMask
timeouts []*nodeStateTimeout timeouts []*nodeStateTimeout
fields []interface{} fields []interface{}
db, dirty bool fieldCount int
db, dirty bool
} }
nodeInfoEnc struct { nodeInfoEnc struct {
@ -158,7 +171,7 @@ type (
} }
offlineCallback struct { offlineCallback struct {
node *enode.Node node *nodeInfo
state bitMask state bitMask
fields []interface{} fields []interface{}
} }
@ -319,10 +332,11 @@ func NewNodeStateMachine(db ethdb.KeyValueStore, dbKey []byte, clock mclock.Cloc
nodes: make(map[enode.ID]*nodeInfo), nodes: make(map[enode.ID]*nodeInfo),
fields: make([]*fieldInfo, len(setup.fields)), fields: make([]*fieldInfo, len(setup.fields)),
} }
ns.opWait = sync.NewCond(&ns.lock)
stateNameMap := make(map[string]int) stateNameMap := make(map[string]int)
for index, flag := range setup.flags { for index, flag := range setup.flags {
if _, ok := stateNameMap[flag.name]; ok { if _, ok := stateNameMap[flag.name]; ok {
panic("Node state flag name collision") panic("Node state flag name collision: " + flag.name)
} }
stateNameMap[flag.name] = index stateNameMap[flag.name] = index
if flag.persistent { if flag.persistent {
@ -332,7 +346,7 @@ func NewNodeStateMachine(db ethdb.KeyValueStore, dbKey []byte, clock mclock.Cloc
fieldNameMap := make(map[string]int) fieldNameMap := make(map[string]int)
for index, field := range setup.fields { for index, field := range setup.fields {
if _, ok := fieldNameMap[field.name]; ok { if _, ok := fieldNameMap[field.name]; ok {
panic("Node field name collision") panic("Node field name collision: " + field.name)
} }
ns.fields[index] = &fieldInfo{fieldDefinition: field} ns.fields[index] = &fieldInfo{fieldDefinition: field}
fieldNameMap[field.name] = index fieldNameMap[field.name] = index
@ -357,10 +371,12 @@ func (ns *NodeStateMachine) fieldIndex(field Field) int {
} }
// SubscribeState adds a node state subscription. The callback is called while the state // SubscribeState adds a node state subscription. The callback is called while the state
// machine mutex is not held and it is allowed to make further state updates. All immediate // machine mutex is not held and it is allowed to make further state updates using the
// changes throughout the system are processed in the same thread/goroutine. It is the // non-blocking SetStateSub/SetFieldSub functions. All callbacks of an operation are running
// responsibility of the implemented state logic to avoid deadlocks caused by the callbacks, // from the thread/goroutine of the initial caller and parallel operations are not permitted.
// infinite toggling of flags or hazardous/non-deterministic state changes. // Therefore the callback is never called concurrently. It is the responsibility of the
// implemented state logic to avoid deadlocks and to reach a stable state in a finite amount
// of steps.
// State subscriptions should be installed before loading the node database or making the // State subscriptions should be installed before loading the node database or making the
// first state update. // first state update.
func (ns *NodeStateMachine) SubscribeState(flags Flags, callback StateCallback) { func (ns *NodeStateMachine) SubscribeState(flags Flags, callback StateCallback) {
@ -408,26 +424,33 @@ func (ns *NodeStateMachine) Start() {
if ns.db != nil { if ns.db != nil {
ns.loadFromDb() ns.loadFromDb()
} }
ns.lock.Unlock()
ns.opStart()
ns.offlineCallbacks(true) ns.offlineCallbacks(true)
ns.opFinish()
ns.lock.Unlock()
} }
// Stop stops the state machine and saves its state if a database was supplied // Stop stops the state machine and saves its state if a database was supplied
func (ns *NodeStateMachine) Stop() { func (ns *NodeStateMachine) Stop() {
ns.lock.Lock() ns.lock.Lock()
defer ns.lock.Unlock()
ns.checkStarted()
if !ns.opStart() {
panic("already closed")
}
for _, node := range ns.nodes { for _, node := range ns.nodes {
fields := make([]interface{}, len(node.fields)) fields := make([]interface{}, len(node.fields))
copy(fields, node.fields) copy(fields, node.fields)
ns.offlineCallbackList = append(ns.offlineCallbackList, offlineCallback{node.node, node.state, fields}) ns.offlineCallbackList = append(ns.offlineCallbackList, offlineCallback{node, node.state, fields})
} }
ns.stopped = true
if ns.db != nil { if ns.db != nil {
ns.saveToDb() ns.saveToDb()
ns.lock.Unlock()
} else {
ns.lock.Unlock()
} }
ns.offlineCallbacks(false) ns.offlineCallbacks(false)
ns.closed = true
ns.opFinish()
} }
// loadFromDb loads persisted node states from the database // loadFromDb loads persisted node states from the database
@ -477,6 +500,7 @@ func (ns *NodeStateMachine) decodeNode(id enode.ID, data []byte) {
if decode := ns.fields[i].decode; decode != nil { if decode := ns.fields[i].decode; decode != nil {
if field, err := decode(encField); err == nil { if field, err := decode(encField); err == nil {
node.fields[i] = field node.fields[i] = field
node.fieldCount++
} else { } else {
log.Error("Failed to decode node field", "id", id, "field name", ns.fields[i].name, "error", err) log.Error("Failed to decode node field", "id", id, "field name", ns.fields[i].name, "error", err)
return return
@ -491,7 +515,7 @@ func (ns *NodeStateMachine) decodeNode(id enode.ID, data []byte) {
node.state = enc.State node.state = enc.State
fields := make([]interface{}, len(node.fields)) fields := make([]interface{}, len(node.fields))
copy(fields, node.fields) copy(fields, node.fields)
ns.offlineCallbackList = append(ns.offlineCallbackList, offlineCallback{node.node, node.state, fields}) ns.offlineCallbackList = append(ns.offlineCallbackList, offlineCallback{node, node.state, fields})
log.Debug("Loaded node state", "id", id, "state", Flags{mask: enc.State, setup: ns.setup}) log.Debug("Loaded node state", "id", id, "state", Flags{mask: enc.State, setup: ns.setup})
} }
@ -505,15 +529,6 @@ func (ns *NodeStateMachine) saveNode(id enode.ID, node *nodeInfo) error {
for _, t := range node.timeouts { for _, t := range node.timeouts {
storedState &= ^t.mask storedState &= ^t.mask
} }
if storedState == 0 {
if node.db {
node.db = false
ns.deleteNode(id)
}
node.dirty = false
return nil
}
enc := nodeInfoEnc{ enc := nodeInfoEnc{
Enr: *node.node.Record(), Enr: *node.node.Record(),
Version: ns.setup.Version, Version: ns.setup.Version,
@ -537,6 +552,14 @@ func (ns *NodeStateMachine) saveNode(id enode.ID, node *nodeInfo) error {
enc.Fields[i] = blob enc.Fields[i] = blob
lastIndex = i lastIndex = i
} }
if storedState == 0 && lastIndex == -1 {
if node.db {
node.db = false
ns.deleteNode(id)
}
node.dirty = false
return nil
}
enc.Fields = enc.Fields[:lastIndex+1] enc.Fields = enc.Fields[:lastIndex+1]
data, err := rlp.EncodeToBytes(&enc) data, err := rlp.EncodeToBytes(&enc)
if err != nil { if err != nil {
@ -596,23 +619,36 @@ func (ns *NodeStateMachine) Persist(n *enode.Node) error {
return nil return nil
} }
// SetState updates the given node state flags and processes all resulting callbacks. // SetState updates the given node state flags and blocks until the operation is finished.
// It only returns after all subsequent immediate changes (including those changed by the // If a flag with a timeout is set again, the operation removes or replaces the existing timeout.
// callbacks) have been processed. If a flag with a timeout is set again, the operation func (ns *NodeStateMachine) SetState(n *enode.Node, setFlags, resetFlags Flags, timeout time.Duration) error {
// removes or replaces the existing timeout.
func (ns *NodeStateMachine) SetState(n *enode.Node, setFlags, resetFlags Flags, timeout time.Duration) {
ns.lock.Lock() ns.lock.Lock()
ns.checkStarted() defer ns.lock.Unlock()
if ns.stopped {
ns.lock.Unlock() if !ns.opStart() {
return return ErrClosed
} }
ns.setState(n, setFlags, resetFlags, timeout)
ns.opFinish()
return nil
}
// SetStateSub updates the given node state flags without blocking (should be called
// from a subscription/operation callback).
func (ns *NodeStateMachine) SetStateSub(n *enode.Node, setFlags, resetFlags Flags, timeout time.Duration) {
ns.lock.Lock()
defer ns.lock.Unlock()
ns.opCheck()
ns.setState(n, setFlags, resetFlags, timeout)
}
func (ns *NodeStateMachine) setState(n *enode.Node, setFlags, resetFlags Flags, timeout time.Duration) {
ns.checkStarted()
set, reset := ns.stateMask(setFlags), ns.stateMask(resetFlags) set, reset := ns.stateMask(setFlags), ns.stateMask(resetFlags)
id, node := ns.updateEnode(n) id, node := ns.updateEnode(n)
if node == nil { if node == nil {
if set == 0 { if set == 0 {
ns.lock.Unlock()
return return
} }
node = ns.newNode(n) node = ns.newNode(n)
@ -627,16 +663,14 @@ func (ns *NodeStateMachine) SetState(n *enode.Node, setFlags, resetFlags Flags,
// even they are not existent(it's noop). // even they are not existent(it's noop).
ns.removeTimeouts(node, set|reset) ns.removeTimeouts(node, set|reset)
// Register the timeout callback if the new state is not empty // Register the timeout callback if required
// and timeout itself is required. if timeout != 0 && set != 0 {
if timeout != 0 && newState != 0 {
ns.addTimeout(n, set, timeout) ns.addTimeout(n, set, timeout)
} }
if newState == oldState { if newState == oldState {
ns.lock.Unlock()
return return
} }
if newState == 0 { if newState == 0 && node.fieldCount == 0 {
delete(ns.nodes, id) delete(ns.nodes, id)
if node.db { if node.db {
ns.deleteNode(id) ns.deleteNode(id)
@ -646,68 +680,118 @@ func (ns *NodeStateMachine) SetState(n *enode.Node, setFlags, resetFlags Flags,
node.dirty = true node.dirty = true
} }
} }
ns.lock.Unlock() callback := func() {
// call state update subscription callbacks without holding the mutex for _, sub := range ns.stateSubs {
for _, sub := range ns.stateSubs { if changed&sub.mask != 0 {
if changed&sub.mask != 0 { sub.callback(n, Flags{mask: oldState & sub.mask, setup: ns.setup}, Flags{mask: newState & sub.mask, setup: ns.setup})
sub.callback(n, Flags{mask: oldState & sub.mask, setup: ns.setup}, Flags{mask: newState & sub.mask, setup: ns.setup})
}
}
if newState == 0 {
// call field subscriptions for discarded fields
for i, v := range node.fields {
if v != nil {
f := ns.fields[i]
if len(f.subs) > 0 {
for _, cb := range f.subs {
cb(n, Flags{setup: ns.setup}, v, nil)
}
}
} }
} }
} }
ns.opPending = append(ns.opPending, callback)
}
// opCheck checks whether an operation is active
func (ns *NodeStateMachine) opCheck() {
if !ns.opFlag {
panic("Operation has not started")
}
}
// opStart waits until other operations are finished and starts a new one
func (ns *NodeStateMachine) opStart() bool {
for ns.opFlag {
ns.opWait.Wait()
}
if ns.closed {
return false
}
ns.opFlag = true
return true
}
// opFinish finishes the current operation by running all pending callbacks.
// Callbacks resulting from a state/field change performed in a previous callback are always
// put at the end of the pending list and therefore processed after all callbacks resulting
// from the previous state/field change.
func (ns *NodeStateMachine) opFinish() {
for len(ns.opPending) != 0 {
list := ns.opPending
ns.lock.Unlock()
for _, cb := range list {
cb()
}
ns.lock.Lock()
ns.opPending = ns.opPending[len(list):]
}
ns.opPending = nil
ns.opFlag = false
ns.opWait.Signal()
}
// Operation calls the given function as an operation callback. This allows the caller
// to start an operation with multiple initial changes. The same rules apply as for
// subscription callbacks.
func (ns *NodeStateMachine) Operation(fn func()) error {
ns.lock.Lock()
started := ns.opStart()
ns.lock.Unlock()
if !started {
return ErrClosed
}
fn()
ns.lock.Lock()
ns.opFinish()
ns.lock.Unlock()
return nil
} }
// offlineCallbacks calls state update callbacks at startup or shutdown // offlineCallbacks calls state update callbacks at startup or shutdown
func (ns *NodeStateMachine) offlineCallbacks(start bool) { func (ns *NodeStateMachine) offlineCallbacks(start bool) {
for _, cb := range ns.offlineCallbackList { for _, cb := range ns.offlineCallbackList {
for _, sub := range ns.stateSubs { cb := cb
offState := offlineState & sub.mask callback := func() {
onState := cb.state & sub.mask for _, sub := range ns.stateSubs {
if offState != onState { offState := offlineState & sub.mask
onState := cb.state & sub.mask
if offState == onState {
continue
}
if start { if start {
sub.callback(cb.node, Flags{mask: offState, setup: ns.setup}, Flags{mask: onState, setup: ns.setup}) sub.callback(cb.node.node, Flags{mask: offState, setup: ns.setup}, Flags{mask: onState, setup: ns.setup})
} else { } else {
sub.callback(cb.node, Flags{mask: onState, setup: ns.setup}, Flags{mask: offState, setup: ns.setup}) sub.callback(cb.node.node, Flags{mask: onState, setup: ns.setup}, Flags{mask: offState, setup: ns.setup})
} }
} }
} for i, f := range cb.fields {
for i, f := range cb.fields { if f == nil || ns.fields[i].subs == nil {
if f != nil && ns.fields[i].subs != nil { continue
}
for _, fsub := range ns.fields[i].subs { for _, fsub := range ns.fields[i].subs {
if start { if start {
fsub(cb.node, Flags{mask: offlineState, setup: ns.setup}, nil, f) fsub(cb.node.node, Flags{mask: offlineState, setup: ns.setup}, nil, f)
} else { } else {
fsub(cb.node, Flags{mask: offlineState, setup: ns.setup}, f, nil) fsub(cb.node.node, Flags{mask: offlineState, setup: ns.setup}, f, nil)
} }
} }
} }
} }
ns.opPending = append(ns.opPending, callback)
} }
ns.offlineCallbackList = nil ns.offlineCallbackList = nil
} }
// AddTimeout adds a node state timeout associated to the given state flag(s). // AddTimeout adds a node state timeout associated to the given state flag(s).
// After the specified time interval, the relevant states will be reset. // After the specified time interval, the relevant states will be reset.
func (ns *NodeStateMachine) AddTimeout(n *enode.Node, flags Flags, timeout time.Duration) { func (ns *NodeStateMachine) AddTimeout(n *enode.Node, flags Flags, timeout time.Duration) error {
ns.lock.Lock() ns.lock.Lock()
defer ns.lock.Unlock() defer ns.lock.Unlock()
ns.checkStarted() ns.checkStarted()
if ns.stopped { if ns.closed {
return return ErrClosed
} }
ns.addTimeout(n, ns.stateMask(flags), timeout) ns.addTimeout(n, ns.stateMask(flags), timeout)
return nil
} }
// addTimeout adds a node state timeout associated to the given state flag(s). // addTimeout adds a node state timeout associated to the given state flag(s).
@ -756,13 +840,15 @@ func (ns *NodeStateMachine) removeTimeouts(node *nodeInfo, mask bitMask) {
} }
} }
// GetField retrieves the given field of the given node // GetField retrieves the given field of the given node. Note that when used in a
// subscription callback the result can be out of sync with the state change represented
// by the callback parameters so extra safety checks might be necessary.
func (ns *NodeStateMachine) GetField(n *enode.Node, field Field) interface{} { func (ns *NodeStateMachine) GetField(n *enode.Node, field Field) interface{} {
ns.lock.Lock() ns.lock.Lock()
defer ns.lock.Unlock() defer ns.lock.Unlock()
ns.checkStarted() ns.checkStarted()
if ns.stopped { if ns.closed {
return nil return nil
} }
if _, node := ns.updateEnode(n); node != nil { if _, node := ns.updateEnode(n); node != nil {
@ -771,48 +857,80 @@ func (ns *NodeStateMachine) GetField(n *enode.Node, field Field) interface{} {
return nil return nil
} }
// SetField sets the given field of the given node // SetField sets the given field of the given node and blocks until the operation is finished
func (ns *NodeStateMachine) SetField(n *enode.Node, field Field, value interface{}) error { func (ns *NodeStateMachine) SetField(n *enode.Node, field Field, value interface{}) error {
ns.lock.Lock() ns.lock.Lock()
ns.checkStarted() defer ns.lock.Unlock()
if ns.stopped {
ns.lock.Unlock() if !ns.opStart() {
return nil return ErrClosed
} }
_, node := ns.updateEnode(n) err := ns.setField(n, field, value)
ns.opFinish()
return err
}
// SetFieldSub sets the given field of the given node without blocking (should be called
// from a subscription/operation callback).
func (ns *NodeStateMachine) SetFieldSub(n *enode.Node, field Field, value interface{}) error {
ns.lock.Lock()
defer ns.lock.Unlock()
ns.opCheck()
return ns.setField(n, field, value)
}
func (ns *NodeStateMachine) setField(n *enode.Node, field Field, value interface{}) error {
ns.checkStarted()
id, node := ns.updateEnode(n)
if node == nil { if node == nil {
ns.lock.Unlock() if value == nil {
return nil return nil
}
node = ns.newNode(n)
ns.nodes[id] = node
} }
fieldIndex := ns.fieldIndex(field) fieldIndex := ns.fieldIndex(field)
f := ns.fields[fieldIndex] f := ns.fields[fieldIndex]
if value != nil && reflect.TypeOf(value) != f.ftype { if value != nil && reflect.TypeOf(value) != f.ftype {
log.Error("Invalid field type", "type", reflect.TypeOf(value), "required", f.ftype) log.Error("Invalid field type", "type", reflect.TypeOf(value), "required", f.ftype)
ns.lock.Unlock() return ErrInvalidField
return errors.New("invalid field type")
} }
oldValue := node.fields[fieldIndex] oldValue := node.fields[fieldIndex]
if value == oldValue { if value == oldValue {
ns.lock.Unlock()
return nil return nil
} }
if oldValue != nil {
node.fieldCount--
}
if value != nil {
node.fieldCount++
}
node.fields[fieldIndex] = value node.fields[fieldIndex] = value
if f.encode != nil { if node.state == 0 && node.fieldCount == 0 {
node.dirty = true delete(ns.nodes, id)
if node.db {
ns.deleteNode(id)
}
} else {
if f.encode != nil {
node.dirty = true
}
} }
state := node.state state := node.state
ns.lock.Unlock() callback := func() {
if len(f.subs) > 0 {
for _, cb := range f.subs { for _, cb := range f.subs {
cb(n, Flags{mask: state, setup: ns.setup}, oldValue, value) cb(n, Flags{mask: state, setup: ns.setup}, oldValue, value)
} }
} }
ns.opPending = append(ns.opPending, callback)
return nil return nil
} }
// ForEach calls the callback for each node having all of the required and none of the // ForEach calls the callback for each node having all of the required and none of the
// disabled flags set // disabled flags set.
// Note that this callback is not an operation callback but ForEach can be called from an
// Operation callback or Operation can also be called from a ForEach callback if necessary.
func (ns *NodeStateMachine) ForEach(requireFlags, disableFlags Flags, cb func(n *enode.Node, state Flags)) { func (ns *NodeStateMachine) ForEach(requireFlags, disableFlags Flags, cb func(n *enode.Node, state Flags)) {
ns.lock.Lock() ns.lock.Lock()
ns.checkStarted() ns.checkStarted()

@ -147,8 +147,13 @@ func TestSetField(t *testing.T) {
// Set field before setting state // Set field before setting state
ns.SetField(testNode(1), fields[0], "hello world") ns.SetField(testNode(1), fields[0], "hello world")
field := ns.GetField(testNode(1), fields[0]) field := ns.GetField(testNode(1), fields[0])
if field == nil {
t.Fatalf("Field should be set before setting states")
}
ns.SetField(testNode(1), fields[0], nil)
field = ns.GetField(testNode(1), fields[0])
if field != nil { if field != nil {
t.Fatalf("Field shouldn't be set before setting states") t.Fatalf("Field should be unset")
} }
// Set field after setting state // Set field after setting state
ns.SetState(testNode(1), flags[0], Flags{}, 0) ns.SetState(testNode(1), flags[0], Flags{}, 0)
@ -169,23 +174,6 @@ func TestSetField(t *testing.T) {
} }
} }
func TestUnsetField(t *testing.T) {
mdb, clock := rawdb.NewMemoryDatabase(), &mclock.Simulated{}
s, flags, fields := testSetup([]bool{false}, []reflect.Type{reflect.TypeOf("")})
ns := NewNodeStateMachine(mdb, []byte("-ns"), clock, s)
ns.Start()
ns.SetState(testNode(1), flags[0], Flags{}, time.Second)
ns.SetField(testNode(1), fields[0], "hello world")
ns.SetState(testNode(1), Flags{}, flags[0], 0)
if field := ns.GetField(testNode(1), fields[0]); field != nil {
t.Fatalf("Field should be unset")
}
}
func TestSetState(t *testing.T) { func TestSetState(t *testing.T) {
mdb, clock := rawdb.NewMemoryDatabase(), &mclock.Simulated{} mdb, clock := rawdb.NewMemoryDatabase(), &mclock.Simulated{}
@ -339,6 +327,7 @@ func TestFieldSub(t *testing.T) {
ns2.Start() ns2.Start()
check(s.OfflineFlag(), nil, uint64(100)) check(s.OfflineFlag(), nil, uint64(100))
ns2.SetState(testNode(1), Flags{}, flags[0], 0) ns2.SetState(testNode(1), Flags{}, flags[0], 0)
ns2.SetField(testNode(1), fields[0], nil)
check(Flags{}, uint64(100), nil) check(Flags{}, uint64(100), nil)
ns2.Stop() ns2.Stop()
} }
@ -387,3 +376,34 @@ func TestDuplicatedFlags(t *testing.T) {
clock.Run(2 * time.Second) clock.Run(2 * time.Second)
check(flags[0], Flags{}, true) check(flags[0], Flags{}, true)
} }
func TestCallbackOrder(t *testing.T) {
mdb, clock := rawdb.NewMemoryDatabase(), &mclock.Simulated{}
s, flags, _ := testSetup([]bool{false, false, false, false}, nil)
ns := NewNodeStateMachine(mdb, []byte("-ns"), clock, s)
ns.SubscribeState(flags[0], func(n *enode.Node, oldState, newState Flags) {
if newState.Equals(flags[0]) {
ns.SetStateSub(n, flags[1], Flags{}, 0)
ns.SetStateSub(n, flags[2], Flags{}, 0)
}
})
ns.SubscribeState(flags[1], func(n *enode.Node, oldState, newState Flags) {
if newState.Equals(flags[1]) {
ns.SetStateSub(n, flags[3], Flags{}, 0)
}
})
lastState := Flags{}
ns.SubscribeState(MergeFlags(flags[1], flags[2], flags[3]), func(n *enode.Node, oldState, newState Flags) {
if !oldState.Equals(lastState) {
t.Fatalf("Wrong callback order")
}
lastState = newState
})
ns.Start()
defer ns.Stop()
ns.SetState(testNode(1), flags[0], Flags{}, 0)
}

Loading…
Cancel
Save