les: move server pool to les/vflux/client (#22377)
* les: move serverPool to les/vflux/client * les: add metrics * les: moved ValueTracker inside ServerPool * les: protect against node registration before server pool is started * les/vflux/client: fixed tests * les: make peer registration safe
This commit is contained in:
@@ -26,17 +26,6 @@ import (
|
||||
"github.com/ethereum/go-ethereum/p2p/nodestate"
|
||||
)
|
||||
|
||||
func testNodeID(i int) enode.ID {
|
||||
return enode.ID{42, byte(i % 256), byte(i / 256)}
|
||||
}
|
||||
|
||||
func testNodeIndex(id enode.ID) int {
|
||||
if id[0] != 42 {
|
||||
return -1
|
||||
}
|
||||
return int(id[1]) + int(id[2])*256
|
||||
}
|
||||
|
||||
func testNode(i int) *enode.Node {
|
||||
return enode.SignNull(new(enr.Record), testNodeID(i))
|
||||
}
|
||||
|
522
les/vflux/client/serverpool.go
Normal file
522
les/vflux/client/serverpool.go
Normal file
@@ -0,0 +1,522 @@
|
||||
// Copyright 2020 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum 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 go-ethereum 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 go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package client
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"math/rand"
|
||||
"reflect"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/mclock"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/les/utils"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/metrics"
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
"github.com/ethereum/go-ethereum/p2p/enr"
|
||||
"github.com/ethereum/go-ethereum/p2p/nodestate"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
)
|
||||
|
||||
const (
|
||||
minTimeout = time.Millisecond * 500 // minimum request timeout suggested by the server pool
|
||||
timeoutRefresh = time.Second * 5 // recalculate timeout if older than this
|
||||
dialCost = 10000 // cost of a TCP dial (used for known node selection weight calculation)
|
||||
dialWaitStep = 1.5 // exponential multiplier of redial wait time when no value was provided by the server
|
||||
queryCost = 500 // cost of a UDP pre-negotiation query
|
||||
queryWaitStep = 1.02 // exponential multiplier of redial wait time when no value was provided by the server
|
||||
waitThreshold = time.Hour * 2000 // drop node if waiting time is over the threshold
|
||||
nodeWeightMul = 1000000 // multiplier constant for node weight calculation
|
||||
nodeWeightThreshold = 100 // minimum weight for keeping a node in the the known (valuable) set
|
||||
minRedialWait = 10 // minimum redial wait time in seconds
|
||||
preNegLimit = 5 // maximum number of simultaneous pre-negotiation queries
|
||||
maxQueryFails = 100 // number of consecutive UDP query failures before we print a warning
|
||||
)
|
||||
|
||||
// ServerPool provides a node iterator for dial candidates. The output is a mix of newly discovered
|
||||
// nodes, a weighted random selection of known (previously valuable) nodes and trusted/paid nodes.
|
||||
type ServerPool struct {
|
||||
clock mclock.Clock
|
||||
unixTime func() int64
|
||||
db ethdb.KeyValueStore
|
||||
|
||||
ns *nodestate.NodeStateMachine
|
||||
vt *ValueTracker
|
||||
mixer *enode.FairMix
|
||||
mixSources []enode.Iterator
|
||||
dialIterator enode.Iterator
|
||||
validSchemes enr.IdentityScheme
|
||||
trustedURLs []string
|
||||
fillSet *FillSet
|
||||
started, queryFails uint32
|
||||
|
||||
timeoutLock sync.RWMutex
|
||||
timeout time.Duration
|
||||
timeWeights ResponseTimeWeights
|
||||
timeoutRefreshed mclock.AbsTime
|
||||
|
||||
suggestedTimeoutGauge, totalValueGauge metrics.Gauge
|
||||
sessionValueMeter metrics.Meter
|
||||
}
|
||||
|
||||
// nodeHistory keeps track of dial costs which determine node weight together with the
|
||||
// service value calculated by ValueTracker.
|
||||
type nodeHistory struct {
|
||||
dialCost utils.ExpiredValue
|
||||
redialWaitStart, redialWaitEnd int64 // unix time (seconds)
|
||||
}
|
||||
|
||||
type nodeHistoryEnc struct {
|
||||
DialCost utils.ExpiredValue
|
||||
RedialWaitStart, RedialWaitEnd uint64
|
||||
}
|
||||
|
||||
// queryFunc sends a pre-negotiation query and blocks until a response arrives or timeout occurs.
|
||||
// It returns 1 if the remote node has confirmed that connection is possible, 0 if not
|
||||
// possible and -1 if no response arrived (timeout).
|
||||
type queryFunc func(*enode.Node) int
|
||||
|
||||
var (
|
||||
clientSetup = &nodestate.Setup{Version: 1}
|
||||
sfHasValue = clientSetup.NewPersistentFlag("hasValue")
|
||||
sfQueried = clientSetup.NewFlag("queried")
|
||||
sfCanDial = clientSetup.NewFlag("canDial")
|
||||
sfDialing = clientSetup.NewFlag("dialed")
|
||||
sfWaitDialTimeout = clientSetup.NewFlag("dialTimeout")
|
||||
sfConnected = clientSetup.NewFlag("connected")
|
||||
sfRedialWait = clientSetup.NewFlag("redialWait")
|
||||
sfAlwaysConnect = clientSetup.NewFlag("alwaysConnect")
|
||||
sfDisableSelection = nodestate.MergeFlags(sfQueried, sfCanDial, sfDialing, sfConnected, sfRedialWait)
|
||||
|
||||
sfiNodeHistory = clientSetup.NewPersistentField("nodeHistory", reflect.TypeOf(nodeHistory{}),
|
||||
func(field interface{}) ([]byte, error) {
|
||||
if n, ok := field.(nodeHistory); ok {
|
||||
ne := nodeHistoryEnc{
|
||||
DialCost: n.dialCost,
|
||||
RedialWaitStart: uint64(n.redialWaitStart),
|
||||
RedialWaitEnd: uint64(n.redialWaitEnd),
|
||||
}
|
||||
enc, err := rlp.EncodeToBytes(&ne)
|
||||
return enc, err
|
||||
}
|
||||
return nil, errors.New("invalid field type")
|
||||
},
|
||||
func(enc []byte) (interface{}, error) {
|
||||
var ne nodeHistoryEnc
|
||||
err := rlp.DecodeBytes(enc, &ne)
|
||||
n := nodeHistory{
|
||||
dialCost: ne.DialCost,
|
||||
redialWaitStart: int64(ne.RedialWaitStart),
|
||||
redialWaitEnd: int64(ne.RedialWaitEnd),
|
||||
}
|
||||
return n, err
|
||||
},
|
||||
)
|
||||
sfiNodeWeight = clientSetup.NewField("nodeWeight", reflect.TypeOf(uint64(0)))
|
||||
sfiConnectedStats = clientSetup.NewField("connectedStats", reflect.TypeOf(ResponseTimeStats{}))
|
||||
)
|
||||
|
||||
// newServerPool creates a new server pool
|
||||
func NewServerPool(db ethdb.KeyValueStore, dbKey []byte, mixTimeout time.Duration, query queryFunc, clock mclock.Clock, trustedURLs []string, requestList []RequestInfo) (*ServerPool, enode.Iterator) {
|
||||
s := &ServerPool{
|
||||
db: db,
|
||||
clock: clock,
|
||||
unixTime: func() int64 { return time.Now().Unix() },
|
||||
validSchemes: enode.ValidSchemes,
|
||||
trustedURLs: trustedURLs,
|
||||
vt: NewValueTracker(db, &mclock.System{}, requestList, time.Minute, 1/float64(time.Hour), 1/float64(time.Hour*100), 1/float64(time.Hour*1000)),
|
||||
ns: nodestate.NewNodeStateMachine(db, []byte(string(dbKey)+"ns:"), clock, clientSetup),
|
||||
}
|
||||
s.recalTimeout()
|
||||
s.mixer = enode.NewFairMix(mixTimeout)
|
||||
knownSelector := NewWrsIterator(s.ns, sfHasValue, sfDisableSelection, sfiNodeWeight)
|
||||
alwaysConnect := NewQueueIterator(s.ns, sfAlwaysConnect, sfDisableSelection, true, nil)
|
||||
s.mixSources = append(s.mixSources, knownSelector)
|
||||
s.mixSources = append(s.mixSources, alwaysConnect)
|
||||
|
||||
iter := enode.Iterator(s.mixer)
|
||||
if query != nil {
|
||||
iter = s.addPreNegFilter(iter, query)
|
||||
}
|
||||
s.dialIterator = enode.Filter(iter, func(node *enode.Node) bool {
|
||||
s.ns.SetState(node, sfDialing, sfCanDial, 0)
|
||||
s.ns.SetState(node, sfWaitDialTimeout, nodestate.Flags{}, time.Second*10)
|
||||
return true
|
||||
})
|
||||
|
||||
s.ns.SubscribeState(nodestate.MergeFlags(sfWaitDialTimeout, sfConnected), func(n *enode.Node, oldState, newState nodestate.Flags) {
|
||||
if oldState.Equals(sfWaitDialTimeout) && newState.IsEmpty() {
|
||||
// dial timeout, no connection
|
||||
s.setRedialWait(n, dialCost, dialWaitStep)
|
||||
s.ns.SetStateSub(n, nodestate.Flags{}, sfDialing, 0)
|
||||
}
|
||||
})
|
||||
|
||||
return s, s.dialIterator
|
||||
}
|
||||
|
||||
// AddMetrics adds metrics to the server pool. Should be called before Start().
|
||||
func (s *ServerPool) AddMetrics(
|
||||
suggestedTimeoutGauge, totalValueGauge, serverSelectableGauge, serverConnectedGauge metrics.Gauge,
|
||||
sessionValueMeter, serverDialedMeter metrics.Meter) {
|
||||
|
||||
s.suggestedTimeoutGauge = suggestedTimeoutGauge
|
||||
s.totalValueGauge = totalValueGauge
|
||||
s.sessionValueMeter = sessionValueMeter
|
||||
if serverSelectableGauge != nil {
|
||||
s.ns.AddLogMetrics(sfHasValue, sfDisableSelection, "selectable", nil, nil, serverSelectableGauge)
|
||||
}
|
||||
if serverDialedMeter != nil {
|
||||
s.ns.AddLogMetrics(sfDialing, nodestate.Flags{}, "dialed", serverDialedMeter, nil, nil)
|
||||
}
|
||||
if serverConnectedGauge != nil {
|
||||
s.ns.AddLogMetrics(sfConnected, nodestate.Flags{}, "connected", nil, nil, serverConnectedGauge)
|
||||
}
|
||||
}
|
||||
|
||||
// AddSource adds a node discovery source to the server pool (should be called before start)
|
||||
func (s *ServerPool) AddSource(source enode.Iterator) {
|
||||
if source != nil {
|
||||
s.mixSources = append(s.mixSources, source)
|
||||
}
|
||||
}
|
||||
|
||||
// addPreNegFilter installs a node filter mechanism that performs a pre-negotiation query.
|
||||
// Nodes that are filtered out and does not appear on the output iterator are put back
|
||||
// into redialWait state.
|
||||
func (s *ServerPool) addPreNegFilter(input enode.Iterator, query queryFunc) enode.Iterator {
|
||||
s.fillSet = NewFillSet(s.ns, input, sfQueried)
|
||||
s.ns.SubscribeState(sfQueried, func(n *enode.Node, oldState, newState nodestate.Flags) {
|
||||
if newState.Equals(sfQueried) {
|
||||
fails := atomic.LoadUint32(&s.queryFails)
|
||||
if fails == maxQueryFails {
|
||||
log.Warn("UDP pre-negotiation query does not seem to work")
|
||||
}
|
||||
if fails > maxQueryFails {
|
||||
fails = maxQueryFails
|
||||
}
|
||||
if rand.Intn(maxQueryFails*2) < int(fails) {
|
||||
// skip pre-negotiation with increasing chance, max 50%
|
||||
// this ensures that the client can operate even if UDP is not working at all
|
||||
s.ns.SetStateSub(n, sfCanDial, nodestate.Flags{}, time.Second*10)
|
||||
// set canDial before resetting queried so that FillSet will not read more
|
||||
// candidates unnecessarily
|
||||
s.ns.SetStateSub(n, nodestate.Flags{}, sfQueried, 0)
|
||||
return
|
||||
}
|
||||
go func() {
|
||||
q := query(n)
|
||||
if q == -1 {
|
||||
atomic.AddUint32(&s.queryFails, 1)
|
||||
} else {
|
||||
atomic.StoreUint32(&s.queryFails, 0)
|
||||
}
|
||||
s.ns.Operation(func() {
|
||||
// we are no longer running in the operation that the callback belongs to, start a new one because of setRedialWait
|
||||
if q == 1 {
|
||||
s.ns.SetStateSub(n, sfCanDial, nodestate.Flags{}, time.Second*10)
|
||||
} else {
|
||||
s.setRedialWait(n, queryCost, queryWaitStep)
|
||||
}
|
||||
s.ns.SetStateSub(n, nodestate.Flags{}, sfQueried, 0)
|
||||
})
|
||||
}()
|
||||
}
|
||||
})
|
||||
return NewQueueIterator(s.ns, sfCanDial, nodestate.Flags{}, false, func(waiting bool) {
|
||||
if waiting {
|
||||
s.fillSet.SetTarget(preNegLimit)
|
||||
} else {
|
||||
s.fillSet.SetTarget(0)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// start starts the server pool. Note that NodeStateMachine should be started first.
|
||||
func (s *ServerPool) Start() {
|
||||
s.ns.Start()
|
||||
for _, iter := range s.mixSources {
|
||||
// add sources to mixer at startup because the mixer instantly tries to read them
|
||||
// which should only happen after NodeStateMachine has been started
|
||||
s.mixer.AddSource(iter)
|
||||
}
|
||||
for _, url := range s.trustedURLs {
|
||||
if node, err := enode.Parse(s.validSchemes, url); err == nil {
|
||||
s.ns.SetState(node, sfAlwaysConnect, nodestate.Flags{}, 0)
|
||||
} else {
|
||||
log.Error("Invalid trusted server URL", "url", url, "error", err)
|
||||
}
|
||||
}
|
||||
unixTime := s.unixTime()
|
||||
s.ns.Operation(func() {
|
||||
s.ns.ForEach(sfHasValue, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) {
|
||||
s.calculateWeight(node)
|
||||
if n, ok := s.ns.GetField(node, sfiNodeHistory).(nodeHistory); ok && n.redialWaitEnd > unixTime {
|
||||
wait := n.redialWaitEnd - unixTime
|
||||
lastWait := n.redialWaitEnd - n.redialWaitStart
|
||||
if wait > lastWait {
|
||||
// if the time until expiration is larger than the last suggested
|
||||
// waiting time then the system clock was probably adjusted
|
||||
wait = lastWait
|
||||
}
|
||||
s.ns.SetStateSub(node, sfRedialWait, nodestate.Flags{}, time.Duration(wait)*time.Second)
|
||||
}
|
||||
})
|
||||
})
|
||||
atomic.StoreUint32(&s.started, 1)
|
||||
}
|
||||
|
||||
// stop stops the server pool
|
||||
func (s *ServerPool) Stop() {
|
||||
s.dialIterator.Close()
|
||||
if s.fillSet != nil {
|
||||
s.fillSet.Close()
|
||||
}
|
||||
s.ns.Operation(func() {
|
||||
s.ns.ForEach(sfConnected, nodestate.Flags{}, func(n *enode.Node, state nodestate.Flags) {
|
||||
// recalculate weight of connected nodes in order to update hasValue flag if necessary
|
||||
s.calculateWeight(n)
|
||||
})
|
||||
})
|
||||
s.ns.Stop()
|
||||
s.vt.Stop()
|
||||
}
|
||||
|
||||
// registerPeer implements serverPeerSubscriber
|
||||
func (s *ServerPool) RegisterNode(node *enode.Node) (*NodeValueTracker, error) {
|
||||
if atomic.LoadUint32(&s.started) == 0 {
|
||||
return nil, errors.New("server pool not started yet")
|
||||
}
|
||||
s.ns.SetState(node, sfConnected, sfDialing.Or(sfWaitDialTimeout), 0)
|
||||
nvt := s.vt.Register(node.ID())
|
||||
s.ns.SetField(node, sfiConnectedStats, nvt.RtStats())
|
||||
return nvt, nil
|
||||
}
|
||||
|
||||
// unregisterPeer implements serverPeerSubscriber
|
||||
func (s *ServerPool) UnregisterNode(node *enode.Node) {
|
||||
s.ns.Operation(func() {
|
||||
s.setRedialWait(node, dialCost, dialWaitStep)
|
||||
s.ns.SetStateSub(node, nodestate.Flags{}, sfConnected, 0)
|
||||
s.ns.SetFieldSub(node, sfiConnectedStats, nil)
|
||||
})
|
||||
s.vt.Unregister(node.ID())
|
||||
}
|
||||
|
||||
// recalTimeout calculates the current recommended timeout. This value is used by
|
||||
// the client as a "soft timeout" value. It also affects the service value calculation
|
||||
// of individual nodes.
|
||||
func (s *ServerPool) recalTimeout() {
|
||||
// Use cached result if possible, avoid recalculating too frequently.
|
||||
s.timeoutLock.RLock()
|
||||
refreshed := s.timeoutRefreshed
|
||||
s.timeoutLock.RUnlock()
|
||||
now := s.clock.Now()
|
||||
if refreshed != 0 && time.Duration(now-refreshed) < timeoutRefresh {
|
||||
return
|
||||
}
|
||||
// Cached result is stale, recalculate a new one.
|
||||
rts := s.vt.RtStats()
|
||||
|
||||
// Add a fake statistic here. It is an easy way to initialize with some
|
||||
// conservative values when the database is new. As soon as we have a
|
||||
// considerable amount of real stats this small value won't matter.
|
||||
rts.Add(time.Second*2, 10, s.vt.StatsExpFactor())
|
||||
|
||||
// Use either 10% failure rate timeout or twice the median response time
|
||||
// as the recommended timeout.
|
||||
timeout := minTimeout
|
||||
if t := rts.Timeout(0.1); t > timeout {
|
||||
timeout = t
|
||||
}
|
||||
if t := rts.Timeout(0.5) * 2; t > timeout {
|
||||
timeout = t
|
||||
}
|
||||
s.timeoutLock.Lock()
|
||||
if s.timeout != timeout {
|
||||
s.timeout = timeout
|
||||
s.timeWeights = TimeoutWeights(s.timeout)
|
||||
|
||||
if s.suggestedTimeoutGauge != nil {
|
||||
s.suggestedTimeoutGauge.Update(int64(s.timeout / time.Millisecond))
|
||||
}
|
||||
if s.totalValueGauge != nil {
|
||||
s.totalValueGauge.Update(int64(rts.Value(s.timeWeights, s.vt.StatsExpFactor())))
|
||||
}
|
||||
}
|
||||
s.timeoutRefreshed = now
|
||||
s.timeoutLock.Unlock()
|
||||
}
|
||||
|
||||
// GetTimeout returns the recommended request timeout.
|
||||
func (s *ServerPool) GetTimeout() time.Duration {
|
||||
s.recalTimeout()
|
||||
s.timeoutLock.RLock()
|
||||
defer s.timeoutLock.RUnlock()
|
||||
return s.timeout
|
||||
}
|
||||
|
||||
// getTimeoutAndWeight returns the recommended request timeout as well as the
|
||||
// response time weight which is necessary to calculate service value.
|
||||
func (s *ServerPool) getTimeoutAndWeight() (time.Duration, ResponseTimeWeights) {
|
||||
s.recalTimeout()
|
||||
s.timeoutLock.RLock()
|
||||
defer s.timeoutLock.RUnlock()
|
||||
return s.timeout, s.timeWeights
|
||||
}
|
||||
|
||||
// addDialCost adds the given amount of dial cost to the node history and returns the current
|
||||
// amount of total dial cost
|
||||
func (s *ServerPool) addDialCost(n *nodeHistory, amount int64) uint64 {
|
||||
logOffset := s.vt.StatsExpirer().LogOffset(s.clock.Now())
|
||||
if amount > 0 {
|
||||
n.dialCost.Add(amount, logOffset)
|
||||
}
|
||||
totalDialCost := n.dialCost.Value(logOffset)
|
||||
if totalDialCost < dialCost {
|
||||
totalDialCost = dialCost
|
||||
}
|
||||
return totalDialCost
|
||||
}
|
||||
|
||||
// serviceValue returns the service value accumulated in this session and in total
|
||||
func (s *ServerPool) serviceValue(node *enode.Node) (sessionValue, totalValue float64) {
|
||||
nvt := s.vt.GetNode(node.ID())
|
||||
if nvt == nil {
|
||||
return 0, 0
|
||||
}
|
||||
currentStats := nvt.RtStats()
|
||||
_, timeWeights := s.getTimeoutAndWeight()
|
||||
expFactor := s.vt.StatsExpFactor()
|
||||
|
||||
totalValue = currentStats.Value(timeWeights, expFactor)
|
||||
if connStats, ok := s.ns.GetField(node, sfiConnectedStats).(ResponseTimeStats); ok {
|
||||
diff := currentStats
|
||||
diff.SubStats(&connStats)
|
||||
sessionValue = diff.Value(timeWeights, expFactor)
|
||||
if s.sessionValueMeter != nil {
|
||||
s.sessionValueMeter.Mark(int64(sessionValue))
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// updateWeight calculates the node weight and updates the nodeWeight field and the
|
||||
// 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) {
|
||||
weight := uint64(totalValue * nodeWeightMul / float64(totalDialCost))
|
||||
if weight >= nodeWeightThreshold {
|
||||
s.ns.SetStateSub(node, sfHasValue, nodestate.Flags{}, 0)
|
||||
s.ns.SetFieldSub(node, sfiNodeWeight, weight)
|
||||
} else {
|
||||
s.ns.SetStateSub(node, nodestate.Flags{}, sfHasValue, 0)
|
||||
s.ns.SetFieldSub(node, sfiNodeWeight, nil)
|
||||
s.ns.SetFieldSub(node, sfiNodeHistory, nil)
|
||||
}
|
||||
s.ns.Persist(node) // saved if node history or hasValue changed
|
||||
}
|
||||
|
||||
// setRedialWait calculates and sets the redialWait timeout based on the service value
|
||||
// and dial cost accumulated during the last session/attempt and in total.
|
||||
// The waiting time is raised exponentially if no service value has been received in order
|
||||
// to prevent dialing an unresponsive node frequently for a very long time just because it
|
||||
// was useful in the past. It can still be occasionally dialed though and once it provides
|
||||
// a significant amount of service value again its waiting time is quickly reduced or reset
|
||||
// to the minimum.
|
||||
// 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) {
|
||||
n, _ := s.ns.GetField(node, sfiNodeHistory).(nodeHistory)
|
||||
sessionValue, totalValue := s.serviceValue(node)
|
||||
totalDialCost := s.addDialCost(&n, addDialCost)
|
||||
|
||||
// if the current dial session has yielded at least the average value/dial cost ratio
|
||||
// then the waiting time should be reset to the minimum. If the session value
|
||||
// is below average but still positive then timeout is limited to the ratio of
|
||||
// average / current service value multiplied by the minimum timeout. If the attempt
|
||||
// was unsuccessful then timeout is raised exponentially without limitation.
|
||||
// Note: dialCost is used in the formula below even if dial was not attempted at all
|
||||
// because the pre-negotiation query did not return a positive result. In this case
|
||||
// the ratio has no meaning anyway and waitFactor is always raised, though in smaller
|
||||
// steps because queries are cheaper and therefore we can allow more failed attempts.
|
||||
unixTime := s.unixTime()
|
||||
plannedTimeout := float64(n.redialWaitEnd - n.redialWaitStart) // last planned redialWait timeout
|
||||
var actualWait float64 // actual waiting time elapsed
|
||||
if unixTime > n.redialWaitEnd {
|
||||
// the planned timeout has elapsed
|
||||
actualWait = plannedTimeout
|
||||
} else {
|
||||
// if the node was redialed earlier then we do not raise the planned timeout
|
||||
// exponentially because that could lead to the timeout rising very high in
|
||||
// a short amount of time
|
||||
// Note that in case of an early redial actualWait also includes the dial
|
||||
// timeout or connection time of the last attempt but it still serves its
|
||||
// purpose of preventing the timeout rising quicker than linearly as a function
|
||||
// of total time elapsed without a successful connection.
|
||||
actualWait = float64(unixTime - n.redialWaitStart)
|
||||
}
|
||||
// raise timeout exponentially if the last planned timeout has elapsed
|
||||
// (use at least the last planned timeout otherwise)
|
||||
nextTimeout := actualWait * waitStep
|
||||
if plannedTimeout > nextTimeout {
|
||||
nextTimeout = plannedTimeout
|
||||
}
|
||||
// we reduce the waiting time if the server has provided service value during the
|
||||
// connection (but never under the minimum)
|
||||
a := totalValue * dialCost * float64(minRedialWait)
|
||||
b := float64(totalDialCost) * sessionValue
|
||||
if a < b*nextTimeout {
|
||||
nextTimeout = a / b
|
||||
}
|
||||
if nextTimeout < minRedialWait {
|
||||
nextTimeout = minRedialWait
|
||||
}
|
||||
wait := time.Duration(float64(time.Second) * nextTimeout)
|
||||
if wait < waitThreshold {
|
||||
n.redialWaitStart = unixTime
|
||||
n.redialWaitEnd = unixTime + int64(nextTimeout)
|
||||
s.ns.SetFieldSub(node, sfiNodeHistory, n)
|
||||
s.ns.SetStateSub(node, sfRedialWait, nodestate.Flags{}, wait)
|
||||
s.updateWeight(node, totalValue, totalDialCost)
|
||||
} else {
|
||||
// discard known node statistics if waiting time is very long because the node
|
||||
// hasn't been responsive for a very long time
|
||||
s.ns.SetFieldSub(node, sfiNodeHistory, nil)
|
||||
s.ns.SetFieldSub(node, sfiNodeWeight, nil)
|
||||
s.ns.SetStateSub(node, nodestate.Flags{}, sfHasValue, 0)
|
||||
}
|
||||
}
|
||||
|
||||
// calculateWeight calculates and sets the node weight without altering the node history.
|
||||
// This function should be called during startup and shutdown only, otherwise setRedialWait
|
||||
// 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) {
|
||||
n, _ := s.ns.GetField(node, sfiNodeHistory).(nodeHistory)
|
||||
_, totalValue := s.serviceValue(node)
|
||||
totalDialCost := s.addDialCost(&n, 0)
|
||||
s.updateWeight(node, totalValue, totalDialCost)
|
||||
}
|
||||
|
||||
// API returns the vflux client API
|
||||
func (s *ServerPool) API() *PrivateClientAPI {
|
||||
return NewPrivateClientAPI(s.vt)
|
||||
}
|
352
les/vflux/client/serverpool_test.go
Normal file
352
les/vflux/client/serverpool_test.go
Normal file
@@ -0,0 +1,352 @@
|
||||
// Copyright 2020 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum 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 go-ethereum 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 go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package client
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/mclock"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/ethdb/memorydb"
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
"github.com/ethereum/go-ethereum/p2p/enr"
|
||||
)
|
||||
|
||||
const (
|
||||
spTestNodes = 1000
|
||||
spTestTarget = 5
|
||||
spTestLength = 10000
|
||||
spMinTotal = 40000
|
||||
spMaxTotal = 50000
|
||||
)
|
||||
|
||||
func testNodeID(i int) enode.ID {
|
||||
return enode.ID{42, byte(i % 256), byte(i / 256)}
|
||||
}
|
||||
|
||||
func testNodeIndex(id enode.ID) int {
|
||||
if id[0] != 42 {
|
||||
return -1
|
||||
}
|
||||
return int(id[1]) + int(id[2])*256
|
||||
}
|
||||
|
||||
type ServerPoolTest struct {
|
||||
db ethdb.KeyValueStore
|
||||
clock *mclock.Simulated
|
||||
quit chan struct{}
|
||||
preNeg, preNegFail bool
|
||||
vt *ValueTracker
|
||||
sp *ServerPool
|
||||
input enode.Iterator
|
||||
testNodes []spTestNode
|
||||
trusted []string
|
||||
waitCount, waitEnded int32
|
||||
|
||||
cycle, conn, servedConn int
|
||||
serviceCycles, dialCount int
|
||||
disconnect map[int][]int
|
||||
}
|
||||
|
||||
type spTestNode struct {
|
||||
connectCycles, waitCycles int
|
||||
nextConnCycle, totalConn int
|
||||
connected, service bool
|
||||
node *enode.Node
|
||||
}
|
||||
|
||||
func newServerPoolTest(preNeg, preNegFail bool) *ServerPoolTest {
|
||||
nodes := make([]*enode.Node, spTestNodes)
|
||||
for i := range nodes {
|
||||
nodes[i] = enode.SignNull(&enr.Record{}, testNodeID(i))
|
||||
}
|
||||
return &ServerPoolTest{
|
||||
clock: &mclock.Simulated{},
|
||||
db: memorydb.New(),
|
||||
input: enode.CycleNodes(nodes),
|
||||
testNodes: make([]spTestNode, spTestNodes),
|
||||
preNeg: preNeg,
|
||||
preNegFail: preNegFail,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) beginWait() {
|
||||
// ensure that dialIterator and the maximal number of pre-neg queries are not all stuck in a waiting state
|
||||
for atomic.AddInt32(&s.waitCount, 1) > preNegLimit {
|
||||
atomic.AddInt32(&s.waitCount, -1)
|
||||
s.clock.Run(time.Second)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) endWait() {
|
||||
atomic.AddInt32(&s.waitCount, -1)
|
||||
atomic.AddInt32(&s.waitEnded, 1)
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) addTrusted(i int) {
|
||||
s.trusted = append(s.trusted, enode.SignNull(&enr.Record{}, testNodeID(i)).String())
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) start() {
|
||||
var testQuery queryFunc
|
||||
if s.preNeg {
|
||||
testQuery = func(node *enode.Node) int {
|
||||
idx := testNodeIndex(node.ID())
|
||||
n := &s.testNodes[idx]
|
||||
canConnect := !n.connected && n.connectCycles != 0 && s.cycle >= n.nextConnCycle
|
||||
if s.preNegFail {
|
||||
// simulate a scenario where UDP queries never work
|
||||
s.beginWait()
|
||||
s.clock.Sleep(time.Second * 5)
|
||||
s.endWait()
|
||||
return -1
|
||||
}
|
||||
switch idx % 3 {
|
||||
case 0:
|
||||
// pre-neg returns true only if connection is possible
|
||||
if canConnect {
|
||||
return 1
|
||||
}
|
||||
return 0
|
||||
case 1:
|
||||
// pre-neg returns true but connection might still fail
|
||||
return 1
|
||||
case 2:
|
||||
// pre-neg returns true if connection is possible, otherwise timeout (node unresponsive)
|
||||
if canConnect {
|
||||
return 1
|
||||
}
|
||||
s.beginWait()
|
||||
s.clock.Sleep(time.Second * 5)
|
||||
s.endWait()
|
||||
return -1
|
||||
}
|
||||
return -1
|
||||
}
|
||||
}
|
||||
|
||||
requestList := make([]RequestInfo, testReqTypes)
|
||||
for i := range requestList {
|
||||
requestList[i] = RequestInfo{Name: "testreq" + strconv.Itoa(i), InitAmount: 1, InitValue: 1}
|
||||
}
|
||||
|
||||
s.sp, _ = NewServerPool(s.db, []byte("sp:"), 0, testQuery, s.clock, s.trusted, requestList)
|
||||
s.sp.AddSource(s.input)
|
||||
s.sp.validSchemes = enode.ValidSchemesForTesting
|
||||
s.sp.unixTime = func() int64 { return int64(s.clock.Now()) / int64(time.Second) }
|
||||
s.disconnect = make(map[int][]int)
|
||||
s.sp.Start()
|
||||
s.quit = make(chan struct{})
|
||||
go func() {
|
||||
last := int32(-1)
|
||||
for {
|
||||
select {
|
||||
case <-time.After(time.Millisecond * 100):
|
||||
c := atomic.LoadInt32(&s.waitEnded)
|
||||
if c == last {
|
||||
// advance clock if test is stuck (might happen in rare cases)
|
||||
s.clock.Run(time.Second)
|
||||
}
|
||||
last = c
|
||||
case <-s.quit:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) stop() {
|
||||
close(s.quit)
|
||||
s.sp.Stop()
|
||||
for i := range s.testNodes {
|
||||
n := &s.testNodes[i]
|
||||
if n.connected {
|
||||
n.totalConn += s.cycle
|
||||
}
|
||||
n.connected = false
|
||||
n.node = nil
|
||||
n.nextConnCycle = 0
|
||||
}
|
||||
s.conn, s.servedConn = 0, 0
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) run() {
|
||||
for count := spTestLength; count > 0; count-- {
|
||||
if dcList := s.disconnect[s.cycle]; dcList != nil {
|
||||
for _, idx := range dcList {
|
||||
n := &s.testNodes[idx]
|
||||
s.sp.UnregisterNode(n.node)
|
||||
n.totalConn += s.cycle
|
||||
n.connected = false
|
||||
n.node = nil
|
||||
s.conn--
|
||||
if n.service {
|
||||
s.servedConn--
|
||||
}
|
||||
n.nextConnCycle = s.cycle + n.waitCycles
|
||||
}
|
||||
delete(s.disconnect, s.cycle)
|
||||
}
|
||||
if s.conn < spTestTarget {
|
||||
s.dialCount++
|
||||
s.beginWait()
|
||||
s.sp.dialIterator.Next()
|
||||
s.endWait()
|
||||
dial := s.sp.dialIterator.Node()
|
||||
id := dial.ID()
|
||||
idx := testNodeIndex(id)
|
||||
n := &s.testNodes[idx]
|
||||
if !n.connected && n.connectCycles != 0 && s.cycle >= n.nextConnCycle {
|
||||
s.conn++
|
||||
if n.service {
|
||||
s.servedConn++
|
||||
}
|
||||
n.totalConn -= s.cycle
|
||||
n.connected = true
|
||||
dc := s.cycle + n.connectCycles
|
||||
s.disconnect[dc] = append(s.disconnect[dc], idx)
|
||||
n.node = dial
|
||||
nv, _ := s.sp.RegisterNode(n.node)
|
||||
if n.service {
|
||||
nv.Served([]ServedRequest{{ReqType: 0, Amount: 100}}, 0)
|
||||
}
|
||||
}
|
||||
}
|
||||
s.serviceCycles += s.servedConn
|
||||
s.clock.Run(time.Second)
|
||||
s.cycle++
|
||||
}
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) setNodes(count, conn, wait int, service, trusted bool) (res []int) {
|
||||
for ; count > 0; count-- {
|
||||
idx := rand.Intn(spTestNodes)
|
||||
for s.testNodes[idx].connectCycles != 0 || s.testNodes[idx].connected {
|
||||
idx = rand.Intn(spTestNodes)
|
||||
}
|
||||
res = append(res, idx)
|
||||
s.testNodes[idx] = spTestNode{
|
||||
connectCycles: conn,
|
||||
waitCycles: wait,
|
||||
service: service,
|
||||
}
|
||||
if trusted {
|
||||
s.addTrusted(idx)
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) resetNodes() {
|
||||
for i, n := range s.testNodes {
|
||||
if n.connected {
|
||||
n.totalConn += s.cycle
|
||||
s.sp.UnregisterNode(n.node)
|
||||
}
|
||||
s.testNodes[i] = spTestNode{totalConn: n.totalConn}
|
||||
}
|
||||
s.conn, s.servedConn = 0, 0
|
||||
s.disconnect = make(map[int][]int)
|
||||
s.trusted = nil
|
||||
}
|
||||
|
||||
func (s *ServerPoolTest) checkNodes(t *testing.T, nodes []int) {
|
||||
var sum int
|
||||
for _, idx := range nodes {
|
||||
n := &s.testNodes[idx]
|
||||
if n.connected {
|
||||
n.totalConn += s.cycle
|
||||
}
|
||||
sum += n.totalConn
|
||||
n.totalConn = 0
|
||||
if n.connected {
|
||||
n.totalConn -= s.cycle
|
||||
}
|
||||
}
|
||||
if sum < spMinTotal || sum > spMaxTotal {
|
||||
t.Errorf("Total connection amount %d outside expected range %d to %d", sum, spMinTotal, spMaxTotal)
|
||||
}
|
||||
}
|
||||
|
||||
func TestServerPool(t *testing.T) { testServerPool(t, false, false) }
|
||||
func TestServerPoolWithPreNeg(t *testing.T) { testServerPool(t, true, false) }
|
||||
func TestServerPoolWithPreNegFail(t *testing.T) { testServerPool(t, true, true) }
|
||||
func testServerPool(t *testing.T, preNeg, fail bool) {
|
||||
s := newServerPoolTest(preNeg, fail)
|
||||
nodes := s.setNodes(100, 200, 200, true, false)
|
||||
s.setNodes(100, 20, 20, false, false)
|
||||
s.start()
|
||||
s.run()
|
||||
s.stop()
|
||||
s.checkNodes(t, nodes)
|
||||
}
|
||||
|
||||
func TestServerPoolChangedNodes(t *testing.T) { testServerPoolChangedNodes(t, false) }
|
||||
func TestServerPoolChangedNodesWithPreNeg(t *testing.T) { testServerPoolChangedNodes(t, true) }
|
||||
func testServerPoolChangedNodes(t *testing.T, preNeg bool) {
|
||||
s := newServerPoolTest(preNeg, false)
|
||||
nodes := s.setNodes(100, 200, 200, true, false)
|
||||
s.setNodes(100, 20, 20, false, false)
|
||||
s.start()
|
||||
s.run()
|
||||
s.checkNodes(t, nodes)
|
||||
for i := 0; i < 3; i++ {
|
||||
s.resetNodes()
|
||||
nodes := s.setNodes(100, 200, 200, true, false)
|
||||
s.setNodes(100, 20, 20, false, false)
|
||||
s.run()
|
||||
s.checkNodes(t, nodes)
|
||||
}
|
||||
s.stop()
|
||||
}
|
||||
|
||||
func TestServerPoolRestartNoDiscovery(t *testing.T) { testServerPoolRestartNoDiscovery(t, false) }
|
||||
func TestServerPoolRestartNoDiscoveryWithPreNeg(t *testing.T) {
|
||||
testServerPoolRestartNoDiscovery(t, true)
|
||||
}
|
||||
func testServerPoolRestartNoDiscovery(t *testing.T, preNeg bool) {
|
||||
s := newServerPoolTest(preNeg, false)
|
||||
nodes := s.setNodes(100, 200, 200, true, false)
|
||||
s.setNodes(100, 20, 20, false, false)
|
||||
s.start()
|
||||
s.run()
|
||||
s.stop()
|
||||
s.checkNodes(t, nodes)
|
||||
s.input = nil
|
||||
s.start()
|
||||
s.run()
|
||||
s.stop()
|
||||
s.checkNodes(t, nodes)
|
||||
}
|
||||
|
||||
func TestServerPoolTrustedNoDiscovery(t *testing.T) { testServerPoolTrustedNoDiscovery(t, false) }
|
||||
func TestServerPoolTrustedNoDiscoveryWithPreNeg(t *testing.T) {
|
||||
testServerPoolTrustedNoDiscovery(t, true)
|
||||
}
|
||||
func testServerPoolTrustedNoDiscovery(t *testing.T, preNeg bool) {
|
||||
s := newServerPoolTest(preNeg, false)
|
||||
trusted := s.setNodes(200, 200, 200, true, true)
|
||||
s.input = nil
|
||||
s.start()
|
||||
s.run()
|
||||
s.stop()
|
||||
s.checkNodes(t, trusted)
|
||||
}
|
@@ -45,6 +45,7 @@ var (
|
||||
type NodeValueTracker struct {
|
||||
lock sync.Mutex
|
||||
|
||||
vt *ValueTracker
|
||||
rtStats, lastRtStats ResponseTimeStats
|
||||
lastTransfer mclock.AbsTime
|
||||
basket serverBasket
|
||||
@@ -52,15 +53,12 @@ type NodeValueTracker struct {
|
||||
reqValues *[]float64
|
||||
}
|
||||
|
||||
// init initializes a NodeValueTracker.
|
||||
// Note that the contents of the referenced reqValues slice will not change; a new
|
||||
// reference is passed if the values are updated by ValueTracker.
|
||||
func (nv *NodeValueTracker) init(now mclock.AbsTime, reqValues *[]float64) {
|
||||
reqTypeCount := len(*reqValues)
|
||||
nv.reqCosts = make([]uint64, reqTypeCount)
|
||||
nv.lastTransfer = now
|
||||
nv.reqValues = reqValues
|
||||
nv.basket.init(reqTypeCount)
|
||||
// UpdateCosts updates the node value tracker's request cost table
|
||||
func (nv *NodeValueTracker) UpdateCosts(reqCosts []uint64) {
|
||||
nv.vt.lock.Lock()
|
||||
defer nv.vt.lock.Unlock()
|
||||
|
||||
nv.updateCosts(reqCosts, &nv.vt.refBasket.reqValues, nv.vt.refBasket.reqValueFactor(reqCosts))
|
||||
}
|
||||
|
||||
// updateCosts updates the request cost table of the server. The request value factor
|
||||
@@ -97,6 +95,28 @@ func (nv *NodeValueTracker) transferStats(now mclock.AbsTime, transferRate float
|
||||
return nv.basket.transfer(-math.Expm1(-transferRate * float64(dt))), recentRtStats
|
||||
}
|
||||
|
||||
type ServedRequest struct {
|
||||
ReqType, Amount uint32
|
||||
}
|
||||
|
||||
// Served adds a served request to the node's statistics. An actual request may be composed
|
||||
// of one or more request types (service vector indices).
|
||||
func (nv *NodeValueTracker) Served(reqs []ServedRequest, respTime time.Duration) {
|
||||
nv.vt.statsExpLock.RLock()
|
||||
expFactor := nv.vt.statsExpFactor
|
||||
nv.vt.statsExpLock.RUnlock()
|
||||
|
||||
nv.lock.Lock()
|
||||
defer nv.lock.Unlock()
|
||||
|
||||
var value float64
|
||||
for _, r := range reqs {
|
||||
nv.basket.add(r.ReqType, r.Amount, nv.reqCosts[r.ReqType]*uint64(r.Amount), expFactor)
|
||||
value += (*nv.reqValues)[r.ReqType] * float64(r.Amount)
|
||||
}
|
||||
nv.rtStats.Add(respTime, value, expFactor)
|
||||
}
|
||||
|
||||
// RtStats returns the node's own response time distribution statistics
|
||||
func (nv *NodeValueTracker) RtStats() ResponseTimeStats {
|
||||
nv.lock.Lock()
|
||||
@@ -333,7 +353,12 @@ func (vt *ValueTracker) Register(id enode.ID) *NodeValueTracker {
|
||||
return nil
|
||||
}
|
||||
nv := vt.loadOrNewNode(id)
|
||||
nv.init(vt.clock.Now(), &vt.refBasket.reqValues)
|
||||
reqTypeCount := len(vt.refBasket.reqValues)
|
||||
nv.reqCosts = make([]uint64, reqTypeCount)
|
||||
nv.lastTransfer = vt.clock.Now()
|
||||
nv.reqValues = &vt.refBasket.reqValues
|
||||
nv.basket.init(reqTypeCount)
|
||||
|
||||
vt.connected[id] = nv
|
||||
return nv
|
||||
}
|
||||
@@ -364,7 +389,7 @@ func (vt *ValueTracker) loadOrNewNode(id enode.ID) *NodeValueTracker {
|
||||
if nv, ok := vt.connected[id]; ok {
|
||||
return nv
|
||||
}
|
||||
nv := &NodeValueTracker{lastTransfer: vt.clock.Now()}
|
||||
nv := &NodeValueTracker{vt: vt, lastTransfer: vt.clock.Now()}
|
||||
enc, err := vt.db.Get(append(vtNodeKey, id[:]...))
|
||||
if err != nil {
|
||||
return nv
|
||||
@@ -425,14 +450,6 @@ func (vt *ValueTracker) saveNode(id enode.ID, nv *NodeValueTracker) {
|
||||
}
|
||||
}
|
||||
|
||||
// UpdateCosts updates the node value tracker's request cost table
|
||||
func (vt *ValueTracker) UpdateCosts(nv *NodeValueTracker, reqCosts []uint64) {
|
||||
vt.lock.Lock()
|
||||
defer vt.lock.Unlock()
|
||||
|
||||
nv.updateCosts(reqCosts, &vt.refBasket.reqValues, vt.refBasket.reqValueFactor(reqCosts))
|
||||
}
|
||||
|
||||
// RtStats returns the global response time distribution statistics
|
||||
func (vt *ValueTracker) RtStats() ResponseTimeStats {
|
||||
vt.lock.Lock()
|
||||
@@ -464,28 +481,6 @@ func (vt *ValueTracker) periodicUpdate() {
|
||||
vt.saveToDb()
|
||||
}
|
||||
|
||||
type ServedRequest struct {
|
||||
ReqType, Amount uint32
|
||||
}
|
||||
|
||||
// Served adds a served request to the node's statistics. An actual request may be composed
|
||||
// of one or more request types (service vector indices).
|
||||
func (vt *ValueTracker) Served(nv *NodeValueTracker, reqs []ServedRequest, respTime time.Duration) {
|
||||
vt.statsExpLock.RLock()
|
||||
expFactor := vt.statsExpFactor
|
||||
vt.statsExpLock.RUnlock()
|
||||
|
||||
nv.lock.Lock()
|
||||
defer nv.lock.Unlock()
|
||||
|
||||
var value float64
|
||||
for _, r := range reqs {
|
||||
nv.basket.add(r.ReqType, r.Amount, nv.reqCosts[r.ReqType]*uint64(r.Amount), expFactor)
|
||||
value += (*nv.reqValues)[r.ReqType] * float64(r.Amount)
|
||||
}
|
||||
nv.rtStats.Add(respTime, value, vt.statsExpFactor)
|
||||
}
|
||||
|
||||
type RequestStatsItem struct {
|
||||
Name string
|
||||
ReqAmount, ReqValue float64
|
||||
|
@@ -64,7 +64,7 @@ func TestValueTracker(t *testing.T) {
|
||||
for j := range costList {
|
||||
costList[j] = uint64(baseCost * relPrices[j])
|
||||
}
|
||||
vt.UpdateCosts(nodes[i], costList)
|
||||
nodes[i].UpdateCosts(costList)
|
||||
}
|
||||
for i := range nodes {
|
||||
nodes[i] = vt.Register(enode.ID{byte(i)})
|
||||
@@ -77,7 +77,7 @@ func TestValueTracker(t *testing.T) {
|
||||
node := rand.Intn(testNodeCount)
|
||||
respTime := time.Duration((rand.Float64() + 1) * float64(time.Second) * float64(node+1) / testNodeCount)
|
||||
totalAmount[reqType] += uint64(reqAmount)
|
||||
vt.Served(nodes[node], []ServedRequest{{uint32(reqType), uint32(reqAmount)}}, respTime)
|
||||
nodes[node].Served([]ServedRequest{{uint32(reqType), uint32(reqAmount)}}, respTime)
|
||||
clock.Run(time.Second)
|
||||
}
|
||||
} else {
|
||||
|
Reference in New Issue
Block a user