les, les/flowcontrol: improved request serving and flow control (#18230)

This change

- implements concurrent LES request serving even for a single peer.
- replaces the request cost estimation method with a cost table based on
  benchmarks which gives much more consistent results. Until now the
  allowed number of light peers was just a guess which probably contributed
  a lot to the fluctuating quality of available service. Everything related
  to request cost is implemented in a single object, the 'cost tracker'. It
  uses a fixed cost table with a global 'correction factor'. Benchmark code
  is included and can be run at any time to adapt costs to low-level
  implementation changes.
- reimplements flowcontrol.ClientManager in a cleaner and more efficient
  way, with added capabilities: There is now control over bandwidth, which
  allows using the flow control parameters for client prioritization.
  Target utilization over 100 percent is now supported to model concurrent
  request processing. Total serving bandwidth is reduced during block
  processing to prevent database contention.
- implements an RPC API for the LES servers allowing server operators to
  assign priority bandwidth to certain clients and change prioritized
  status even while the client is connected. The new API is meant for
  cases where server operators charge for LES using an off-protocol mechanism.
- adds a unit test for the new client manager.
- adds an end-to-end test using the network simulator that tests bandwidth
  control functions through the new API.
This commit is contained in:
Felföldi Zsolt
2019-02-26 12:32:48 +01:00
committed by Felix Lange
parent c2b33a117f
commit c2003ed63b
39 changed files with 3705 additions and 999 deletions

View File

@ -18,166 +18,339 @@
package flowcontrol
import (
"fmt"
"sync"
"time"
"github.com/ethereum/go-ethereum/common/mclock"
"github.com/ethereum/go-ethereum/log"
)
const fcTimeConst = time.Millisecond
const (
// fcTimeConst is the time constant applied for MinRecharge during linear
// buffer recharge period
fcTimeConst = time.Millisecond
// DecParamDelay is applied at server side when decreasing capacity in order to
// avoid a buffer underrun error due to requests sent by the client before
// receiving the capacity update announcement
DecParamDelay = time.Second * 2
// keepLogs is the duration of keeping logs; logging is not used if zero
keepLogs = 0
)
// ServerParams are the flow control parameters specified by a server for a client
//
// Note: a server can assign different amounts of capacity to each client by giving
// different parameters to them.
type ServerParams struct {
BufLimit, MinRecharge uint64
}
type ClientNode struct {
params *ServerParams
bufValue uint64
lastTime mclock.AbsTime
lock sync.Mutex
cm *ClientManager
cmNode *cmNode
// scheduledUpdate represents a delayed flow control parameter update
type scheduledUpdate struct {
time mclock.AbsTime
params ServerParams
}
func NewClientNode(cm *ClientManager, params *ServerParams) *ClientNode {
// ClientNode is the flow control system's representation of a client
// (used in server mode only)
type ClientNode struct {
params ServerParams
bufValue uint64
lastTime mclock.AbsTime
updateSchedule []scheduledUpdate
sumCost uint64 // sum of req costs received from this client
accepted map[uint64]uint64 // value = sumCost after accepting the given req
lock sync.Mutex
cm *ClientManager
log *logger
cmNodeFields
}
// NewClientNode returns a new ClientNode
func NewClientNode(cm *ClientManager, params ServerParams) *ClientNode {
node := &ClientNode{
cm: cm,
params: params,
bufValue: params.BufLimit,
lastTime: mclock.Now(),
lastTime: cm.clock.Now(),
accepted: make(map[uint64]uint64),
}
node.cmNode = cm.addNode(node)
if keepLogs > 0 {
node.log = newLogger(keepLogs)
}
cm.connect(node)
return node
}
func (peer *ClientNode) Remove(cm *ClientManager) {
cm.removeNode(peer.cmNode)
// Disconnect should be called when a client is disconnected
func (node *ClientNode) Disconnect() {
node.cm.disconnect(node)
}
func (peer *ClientNode) recalcBV(time mclock.AbsTime) {
dt := uint64(time - peer.lastTime)
if time < peer.lastTime {
// update recalculates the buffer value at a specified time while also performing
// scheduled flow control parameter updates if necessary
func (node *ClientNode) update(now mclock.AbsTime) {
for len(node.updateSchedule) > 0 && node.updateSchedule[0].time <= now {
node.recalcBV(node.updateSchedule[0].time)
node.updateParams(node.updateSchedule[0].params, now)
node.updateSchedule = node.updateSchedule[1:]
}
node.recalcBV(now)
}
// recalcBV recalculates the buffer value at a specified time
func (node *ClientNode) recalcBV(now mclock.AbsTime) {
dt := uint64(now - node.lastTime)
if now < node.lastTime {
dt = 0
}
peer.bufValue += peer.params.MinRecharge * dt / uint64(fcTimeConst)
if peer.bufValue > peer.params.BufLimit {
peer.bufValue = peer.params.BufLimit
node.bufValue += node.params.MinRecharge * dt / uint64(fcTimeConst)
if node.bufValue > node.params.BufLimit {
node.bufValue = node.params.BufLimit
}
peer.lastTime = time
if node.log != nil {
node.log.add(now, fmt.Sprintf("updated bv=%d MRR=%d BufLimit=%d", node.bufValue, node.params.MinRecharge, node.params.BufLimit))
}
node.lastTime = now
}
func (peer *ClientNode) AcceptRequest() (uint64, bool) {
peer.lock.Lock()
defer peer.lock.Unlock()
// UpdateParams updates the flow control parameters of a client node
func (node *ClientNode) UpdateParams(params ServerParams) {
node.lock.Lock()
defer node.lock.Unlock()
time := mclock.Now()
peer.recalcBV(time)
return peer.bufValue, peer.cm.accept(peer.cmNode, time)
}
func (peer *ClientNode) RequestProcessed(cost uint64) (bv, realCost uint64) {
peer.lock.Lock()
defer peer.lock.Unlock()
time := mclock.Now()
peer.recalcBV(time)
peer.bufValue -= cost
rcValue, rcost := peer.cm.processed(peer.cmNode, time)
if rcValue < peer.params.BufLimit {
bv := peer.params.BufLimit - rcValue
if bv > peer.bufValue {
peer.bufValue = bv
now := node.cm.clock.Now()
node.update(now)
if params.MinRecharge >= node.params.MinRecharge {
node.updateSchedule = nil
node.updateParams(params, now)
} else {
for i, s := range node.updateSchedule {
if params.MinRecharge >= s.params.MinRecharge {
s.params = params
node.updateSchedule = node.updateSchedule[:i+1]
return
}
}
node.updateSchedule = append(node.updateSchedule, scheduledUpdate{time: now + mclock.AbsTime(DecParamDelay), params: params})
}
return peer.bufValue, rcost
}
// updateParams updates the flow control parameters of the node
func (node *ClientNode) updateParams(params ServerParams, now mclock.AbsTime) {
diff := params.BufLimit - node.params.BufLimit
if int64(diff) > 0 {
node.bufValue += diff
} else if node.bufValue > params.BufLimit {
node.bufValue = params.BufLimit
}
node.cm.updateParams(node, params, now)
}
// AcceptRequest returns whether a new request can be accepted and the missing
// buffer amount if it was rejected due to a buffer underrun. If accepted, maxCost
// is deducted from the flow control buffer.
func (node *ClientNode) AcceptRequest(reqID, index, maxCost uint64) (accepted bool, bufShort uint64, priority int64) {
node.lock.Lock()
defer node.lock.Unlock()
now := node.cm.clock.Now()
node.update(now)
if maxCost > node.bufValue {
if node.log != nil {
node.log.add(now, fmt.Sprintf("rejected reqID=%d bv=%d maxCost=%d", reqID, node.bufValue, maxCost))
node.log.dump(now)
}
return false, maxCost - node.bufValue, 0
}
node.bufValue -= maxCost
node.sumCost += maxCost
if node.log != nil {
node.log.add(now, fmt.Sprintf("accepted reqID=%d bv=%d maxCost=%d sumCost=%d", reqID, node.bufValue, maxCost, node.sumCost))
}
node.accepted[index] = node.sumCost
return true, 0, node.cm.accepted(node, maxCost, now)
}
// RequestProcessed should be called when the request has been processed
func (node *ClientNode) RequestProcessed(reqID, index, maxCost, realCost uint64) (bv uint64) {
node.lock.Lock()
defer node.lock.Unlock()
now := node.cm.clock.Now()
node.update(now)
node.cm.processed(node, maxCost, realCost, now)
bv = node.bufValue + node.sumCost - node.accepted[index]
if node.log != nil {
node.log.add(now, fmt.Sprintf("processed reqID=%d bv=%d maxCost=%d realCost=%d sumCost=%d oldSumCost=%d reportedBV=%d", reqID, node.bufValue, maxCost, realCost, node.sumCost, node.accepted[index], bv))
}
delete(node.accepted, index)
return
}
// ServerNode is the flow control system's representation of a server
// (used in client mode only)
type ServerNode struct {
clock mclock.Clock
bufEstimate uint64
bufRecharge bool
lastTime mclock.AbsTime
params *ServerParams
params ServerParams
sumCost uint64 // sum of req costs sent to this server
pending map[uint64]uint64 // value = sumCost after sending the given req
log *logger
lock sync.RWMutex
}
func NewServerNode(params *ServerParams) *ServerNode {
return &ServerNode{
// NewServerNode returns a new ServerNode
func NewServerNode(params ServerParams, clock mclock.Clock) *ServerNode {
node := &ServerNode{
clock: clock,
bufEstimate: params.BufLimit,
lastTime: mclock.Now(),
bufRecharge: false,
lastTime: clock.Now(),
params: params,
pending: make(map[uint64]uint64),
}
if keepLogs > 0 {
node.log = newLogger(keepLogs)
}
return node
}
func (peer *ServerNode) recalcBLE(time mclock.AbsTime) {
dt := uint64(time - peer.lastTime)
if time < peer.lastTime {
dt = 0
// UpdateParams updates the flow control parameters of the node
func (node *ServerNode) UpdateParams(params ServerParams) {
node.lock.Lock()
defer node.lock.Unlock()
node.recalcBLE(mclock.Now())
if params.BufLimit > node.params.BufLimit {
node.bufEstimate += params.BufLimit - node.params.BufLimit
} else {
if node.bufEstimate > params.BufLimit {
node.bufEstimate = params.BufLimit
}
}
peer.bufEstimate += peer.params.MinRecharge * dt / uint64(fcTimeConst)
if peer.bufEstimate > peer.params.BufLimit {
peer.bufEstimate = peer.params.BufLimit
node.params = params
}
// recalcBLE recalculates the lowest estimate for the client's buffer value at
// the given server at the specified time
func (node *ServerNode) recalcBLE(now mclock.AbsTime) {
if now < node.lastTime {
return
}
if node.bufRecharge {
dt := uint64(now - node.lastTime)
node.bufEstimate += node.params.MinRecharge * dt / uint64(fcTimeConst)
if node.bufEstimate >= node.params.BufLimit {
node.bufEstimate = node.params.BufLimit
node.bufRecharge = false
}
}
node.lastTime = now
if node.log != nil {
node.log.add(now, fmt.Sprintf("updated bufEst=%d MRR=%d BufLimit=%d", node.bufEstimate, node.params.MinRecharge, node.params.BufLimit))
}
peer.lastTime = time
}
// safetyMargin is added to the flow control waiting time when estimated buffer value is low
const safetyMargin = time.Millisecond
func (peer *ServerNode) canSend(maxCost uint64) (time.Duration, float64) {
peer.recalcBLE(mclock.Now())
maxCost += uint64(safetyMargin) * peer.params.MinRecharge / uint64(fcTimeConst)
if maxCost > peer.params.BufLimit {
maxCost = peer.params.BufLimit
}
if peer.bufEstimate >= maxCost {
return 0, float64(peer.bufEstimate-maxCost) / float64(peer.params.BufLimit)
}
return time.Duration((maxCost - peer.bufEstimate) * uint64(fcTimeConst) / peer.params.MinRecharge), 0
}
// CanSend returns the minimum waiting time required before sending a request
// with the given maximum estimated cost. Second return value is the relative
// estimated buffer level after sending the request (divided by BufLimit).
func (peer *ServerNode) CanSend(maxCost uint64) (time.Duration, float64) {
peer.lock.RLock()
defer peer.lock.RUnlock()
func (node *ServerNode) CanSend(maxCost uint64) (time.Duration, float64) {
node.lock.RLock()
defer node.lock.RUnlock()
return peer.canSend(maxCost)
}
// QueueRequest should be called when the request has been assigned to the given
// server node, before putting it in the send queue. It is mandatory that requests
// are sent in the same order as the QueueRequest calls are made.
func (peer *ServerNode) QueueRequest(reqID, maxCost uint64) {
peer.lock.Lock()
defer peer.lock.Unlock()
peer.bufEstimate -= maxCost
peer.sumCost += maxCost
peer.pending[reqID] = peer.sumCost
}
// GotReply adjusts estimated buffer value according to the value included in
// the latest request reply.
func (peer *ServerNode) GotReply(reqID, bv uint64) {
peer.lock.Lock()
defer peer.lock.Unlock()
if bv > peer.params.BufLimit {
bv = peer.params.BufLimit
now := node.clock.Now()
node.recalcBLE(now)
maxCost += uint64(safetyMargin) * node.params.MinRecharge / uint64(fcTimeConst)
if maxCost > node.params.BufLimit {
maxCost = node.params.BufLimit
}
sc, ok := peer.pending[reqID]
if node.bufEstimate >= maxCost {
relBuf := float64(node.bufEstimate-maxCost) / float64(node.params.BufLimit)
if node.log != nil {
node.log.add(now, fmt.Sprintf("canSend bufEst=%d maxCost=%d true relBuf=%f", node.bufEstimate, maxCost, relBuf))
}
return 0, relBuf
}
timeLeft := time.Duration((maxCost - node.bufEstimate) * uint64(fcTimeConst) / node.params.MinRecharge)
if node.log != nil {
node.log.add(now, fmt.Sprintf("canSend bufEst=%d maxCost=%d false timeLeft=%v", node.bufEstimate, maxCost, timeLeft))
}
return timeLeft, 0
}
// QueuedRequest should be called when the request has been assigned to the given
// server node, before putting it in the send queue. It is mandatory that requests
// are sent in the same order as the QueuedRequest calls are made.
func (node *ServerNode) QueuedRequest(reqID, maxCost uint64) {
node.lock.Lock()
defer node.lock.Unlock()
now := node.clock.Now()
node.recalcBLE(now)
// Note: we do not know when requests actually arrive to the server so bufRecharge
// is not turned on here if buffer was full; in this case it is going to be turned
// on by the first reply's bufValue feedback
if node.bufEstimate >= maxCost {
node.bufEstimate -= maxCost
} else {
log.Error("Queued request with insufficient buffer estimate")
node.bufEstimate = 0
}
node.sumCost += maxCost
node.pending[reqID] = node.sumCost
if node.log != nil {
node.log.add(now, fmt.Sprintf("queued reqID=%d bufEst=%d maxCost=%d sumCost=%d", reqID, node.bufEstimate, maxCost, node.sumCost))
}
}
// ReceivedReply adjusts estimated buffer value according to the value included in
// the latest request reply.
func (node *ServerNode) ReceivedReply(reqID, bv uint64) {
node.lock.Lock()
defer node.lock.Unlock()
now := node.clock.Now()
node.recalcBLE(now)
if bv > node.params.BufLimit {
bv = node.params.BufLimit
}
sc, ok := node.pending[reqID]
if !ok {
return
}
delete(peer.pending, reqID)
cc := peer.sumCost - sc
peer.bufEstimate = 0
delete(node.pending, reqID)
cc := node.sumCost - sc
newEstimate := uint64(0)
if bv > cc {
peer.bufEstimate = bv - cc
newEstimate = bv - cc
}
if newEstimate > node.bufEstimate {
// Note: we never reduce the buffer estimate based on the reported value because
// this can only happen because of the delayed delivery of the latest reply.
// The lowest estimate based on the previous reply can still be considered valid.
node.bufEstimate = newEstimate
}
node.bufRecharge = node.bufEstimate < node.params.BufLimit
node.lastTime = now
if node.log != nil {
node.log.add(now, fmt.Sprintf("received reqID=%d bufEst=%d reportedBv=%d sumCost=%d oldSumCost=%d", reqID, node.bufEstimate, bv, node.sumCost, sc))
}
}
// DumpLogs dumps the event log if logging is used
func (node *ServerNode) DumpLogs() {
node.lock.Lock()
defer node.lock.Unlock()
if node.log != nil {
node.log.dump(node.clock.Now())
}
peer.lastTime = mclock.Now()
}

65
les/flowcontrol/logger.go Normal file
View File

@ -0,0 +1,65 @@
// Copyright 2018 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 flowcontrol
import (
"fmt"
"time"
"github.com/ethereum/go-ethereum/common/mclock"
)
// logger collects events in string format and discards events older than the
// "keep" parameter
type logger struct {
events map[uint64]logEvent
writePtr, delPtr uint64
keep time.Duration
}
// logEvent describes a single event
type logEvent struct {
time mclock.AbsTime
event string
}
// newLogger creates a new logger
func newLogger(keep time.Duration) *logger {
return &logger{
events: make(map[uint64]logEvent),
keep: keep,
}
}
// add adds a new event and discards old events if possible
func (l *logger) add(now mclock.AbsTime, event string) {
keepAfter := now - mclock.AbsTime(l.keep)
for l.delPtr < l.writePtr && l.events[l.delPtr].time <= keepAfter {
delete(l.events, l.delPtr)
l.delPtr++
}
l.events[l.writePtr] = logEvent{now, event}
l.writePtr++
}
// dump prints all stored events
func (l *logger) dump(now mclock.AbsTime) {
for i := l.delPtr; i < l.writePtr; i++ {
e := l.events[i]
fmt.Println(time.Duration(e.time-now), e.event)
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2016 The go-ethereum Authors
// Copyright 2018 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
@ -14,211 +14,388 @@
// 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 flowcontrol implements a client side flow control mechanism
package flowcontrol
import (
"fmt"
"math"
"sync"
"time"
"github.com/ethereum/go-ethereum/common/mclock"
"github.com/ethereum/go-ethereum/common/prque"
)
const rcConst = 1000000
type cmNode struct {
node *ClientNode
lastUpdate mclock.AbsTime
serving, recharging bool
rcWeight uint64
rcValue, rcDelta, startValue int64
finishRecharge mclock.AbsTime
// cmNodeFields are ClientNode fields used by the client manager
// Note: these fields are locked by the client manager's mutex
type cmNodeFields struct {
corrBufValue int64 // buffer value adjusted with the extra recharge amount
rcLastIntValue int64 // past recharge integrator value when corrBufValue was last updated
rcFullIntValue int64 // future recharge integrator value when corrBufValue will reach maximum
queueIndex int // position in the recharge queue (-1 if not queued)
}
func (node *cmNode) update(time mclock.AbsTime) {
dt := int64(time - node.lastUpdate)
node.rcValue += node.rcDelta * dt / rcConst
node.lastUpdate = time
if node.recharging && time >= node.finishRecharge {
node.recharging = false
node.rcDelta = 0
node.rcValue = 0
}
}
// FixedPointMultiplier is applied to the recharge integrator and the recharge curve.
//
// Note: fixed point arithmetic is required for the integrator because it is a
// constantly increasing value that can wrap around int64 limits (which behavior is
// also supported by the priority queue). A floating point value would gradually lose
// precision in this application.
// The recharge curve and all recharge values are encoded as fixed point because
// sumRecharge is frequently updated by adding or subtracting individual recharge
// values and perfect precision is required.
const FixedPointMultiplier = 1000000
func (node *cmNode) set(serving bool, simReqCnt, sumWeight uint64) {
if node.serving && !serving {
node.recharging = true
sumWeight += node.rcWeight
}
node.serving = serving
if node.recharging && serving {
node.recharging = false
sumWeight -= node.rcWeight
}
node.rcDelta = 0
if serving {
node.rcDelta = int64(rcConst / simReqCnt)
}
if node.recharging {
node.rcDelta = -int64(node.node.cm.rcRecharge * node.rcWeight / sumWeight)
node.finishRecharge = node.lastUpdate + mclock.AbsTime(node.rcValue*rcConst/(-node.rcDelta))
}
}
var (
capFactorDropTC = 1 / float64(time.Second*10) // time constant for dropping the capacity factor
capFactorRaiseTC = 1 / float64(time.Hour) // time constant for raising the capacity factor
capFactorRaiseThreshold = 0.75 // connected / total capacity ratio threshold for raising the capacity factor
)
// ClientManager controls the capacity assigned to the clients of a server.
// Since ServerParams guarantee a safe lower estimate for processable requests
// even in case of all clients being active, ClientManager calculates a
// corrigated buffer value and usually allows a higher remaining buffer value
// to be returned with each reply.
type ClientManager struct {
lock sync.Mutex
nodes map[*cmNode]struct{}
simReqCnt, sumWeight, rcSumValue uint64
maxSimReq, maxRcSum uint64
rcRecharge uint64
resumeQueue chan chan bool
time mclock.AbsTime
clock mclock.Clock
lock sync.Mutex
enabledCh chan struct{}
curve PieceWiseLinear
sumRecharge, totalRecharge, totalConnected uint64
capLogFactor, totalCapacity float64
capLastUpdate mclock.AbsTime
totalCapacityCh chan uint64
// recharge integrator is increasing in each moment with a rate of
// (totalRecharge / sumRecharge)*FixedPointMultiplier or 0 if sumRecharge==0
rcLastUpdate mclock.AbsTime // last time the recharge integrator was updated
rcLastIntValue int64 // last updated value of the recharge integrator
// recharge queue is a priority queue with currently recharging client nodes
// as elements. The priority value is rcFullIntValue which allows to quickly
// determine which client will first finish recharge.
rcQueue *prque.Prque
}
func NewClientManager(rcTarget, maxSimReq, maxRcSum uint64) *ClientManager {
// NewClientManager returns a new client manager.
// Client manager enhances flow control performance by allowing client buffers
// to recharge quicker than the minimum guaranteed recharge rate if possible.
// The sum of all minimum recharge rates (sumRecharge) is updated each time
// a clients starts or finishes buffer recharging. Then an adjusted total
// recharge rate is calculated using a piecewise linear recharge curve:
//
// totalRecharge = curve(sumRecharge)
// (totalRecharge >= sumRecharge is enforced)
//
// Then the "bonus" buffer recharge is distributed between currently recharging
// clients proportionally to their minimum recharge rates.
//
// Note: total recharge is proportional to the average number of parallel running
// serving threads. A recharge value of 1000000 corresponds to one thread in average.
// The maximum number of allowed serving threads should always be considerably
// higher than the targeted average number.
//
// Note 2: although it is possible to specify a curve allowing the total target
// recharge starting from zero sumRecharge, it makes sense to add a linear ramp
// starting from zero in order to not let a single low-priority client use up
// the entire server capacity and thus ensure quick availability for others at
// any moment.
func NewClientManager(curve PieceWiseLinear, clock mclock.Clock) *ClientManager {
cm := &ClientManager{
nodes: make(map[*cmNode]struct{}),
resumeQueue: make(chan chan bool),
rcRecharge: rcConst * rcConst / (100*rcConst/rcTarget - rcConst),
maxSimReq: maxSimReq,
maxRcSum: maxRcSum,
clock: clock,
rcQueue: prque.New(func(a interface{}, i int) { a.(*ClientNode).queueIndex = i }),
capLastUpdate: clock.Now(),
}
if curve != nil {
cm.SetRechargeCurve(curve)
}
go cm.queueProc()
return cm
}
func (self *ClientManager) Stop() {
self.lock.Lock()
defer self.lock.Unlock()
// SetRechargeCurve updates the recharge curve
func (cm *ClientManager) SetRechargeCurve(curve PieceWiseLinear) {
cm.lock.Lock()
defer cm.lock.Unlock()
// signal any waiting accept routines to return false
self.nodes = make(map[*cmNode]struct{})
close(self.resumeQueue)
}
func (self *ClientManager) addNode(cnode *ClientNode) *cmNode {
time := mclock.Now()
node := &cmNode{
node: cnode,
lastUpdate: time,
finishRecharge: time,
rcWeight: 1,
now := cm.clock.Now()
cm.updateRecharge(now)
cm.updateCapFactor(now, false)
cm.curve = curve
if len(curve) > 0 {
cm.totalRecharge = curve[len(curve)-1].Y
} else {
cm.totalRecharge = 0
}
self.lock.Lock()
defer self.lock.Unlock()
self.nodes[node] = struct{}{}
self.update(mclock.Now())
return node
cm.refreshCapacity()
}
func (self *ClientManager) removeNode(node *cmNode) {
self.lock.Lock()
defer self.lock.Unlock()
// connect should be called when a client is connected, before passing it to any
// other ClientManager function
func (cm *ClientManager) connect(node *ClientNode) {
cm.lock.Lock()
defer cm.lock.Unlock()
time := mclock.Now()
self.stop(node, time)
delete(self.nodes, node)
self.update(time)
now := cm.clock.Now()
cm.updateRecharge(now)
node.corrBufValue = int64(node.params.BufLimit)
node.rcLastIntValue = cm.rcLastIntValue
node.queueIndex = -1
cm.updateCapFactor(now, true)
cm.totalConnected += node.params.MinRecharge
}
// recalc sumWeight
func (self *ClientManager) updateNodes(time mclock.AbsTime) (rce bool) {
var sumWeight, rcSum uint64
for node := range self.nodes {
rc := node.recharging
node.update(time)
if rc && !node.recharging {
rce = true
}
if node.recharging {
sumWeight += node.rcWeight
}
rcSum += uint64(node.rcValue)
// disconnect should be called when a client is disconnected
func (cm *ClientManager) disconnect(node *ClientNode) {
cm.lock.Lock()
defer cm.lock.Unlock()
now := cm.clock.Now()
cm.updateRecharge(cm.clock.Now())
cm.updateCapFactor(now, true)
cm.totalConnected -= node.params.MinRecharge
}
// accepted is called when a request with given maximum cost is accepted.
// It returns a priority indicator for the request which is used to determine placement
// in the serving queue. Older requests have higher priority by default. If the client
// is almost out of buffer, request priority is reduced.
func (cm *ClientManager) accepted(node *ClientNode, maxCost uint64, now mclock.AbsTime) (priority int64) {
cm.lock.Lock()
defer cm.lock.Unlock()
cm.updateNodeRc(node, -int64(maxCost), &node.params, now)
rcTime := (node.params.BufLimit - uint64(node.corrBufValue)) * FixedPointMultiplier / node.params.MinRecharge
return -int64(now) - int64(rcTime)
}
// processed updates the client buffer according to actual request cost after
// serving has been finished.
//
// Note: processed should always be called for all accepted requests
func (cm *ClientManager) processed(node *ClientNode, maxCost, realCost uint64, now mclock.AbsTime) {
cm.lock.Lock()
defer cm.lock.Unlock()
if realCost > maxCost {
realCost = maxCost
}
cm.updateNodeRc(node, int64(maxCost-realCost), &node.params, now)
if uint64(node.corrBufValue) > node.bufValue {
if node.log != nil {
node.log.add(now, fmt.Sprintf("corrected bv=%d oldBv=%d", node.corrBufValue, node.bufValue))
}
node.bufValue = uint64(node.corrBufValue)
}
self.sumWeight = sumWeight
self.rcSumValue = rcSum
return
}
func (self *ClientManager) update(time mclock.AbsTime) {
for {
firstTime := time
for node := range self.nodes {
if node.recharging && node.finishRecharge < firstTime {
firstTime = node.finishRecharge
}
// updateParams updates the flow control parameters of a client node
func (cm *ClientManager) updateParams(node *ClientNode, params ServerParams, now mclock.AbsTime) {
cm.lock.Lock()
defer cm.lock.Unlock()
cm.updateRecharge(now)
cm.updateCapFactor(now, true)
cm.totalConnected += params.MinRecharge - node.params.MinRecharge
cm.updateNodeRc(node, 0, &params, now)
}
// updateRecharge updates the recharge integrator and checks the recharge queue
// for nodes with recently filled buffers
func (cm *ClientManager) updateRecharge(now mclock.AbsTime) {
lastUpdate := cm.rcLastUpdate
cm.rcLastUpdate = now
// updating is done in multiple steps if node buffers are filled and sumRecharge
// is decreased before the given target time
for cm.sumRecharge > 0 {
bonusRatio := cm.curve.ValueAt(cm.sumRecharge) / float64(cm.sumRecharge)
if bonusRatio < 1 {
bonusRatio = 1
}
if self.updateNodes(firstTime) {
for node := range self.nodes {
if node.recharging {
node.set(node.serving, self.simReqCnt, self.sumWeight)
}
}
} else {
self.time = time
dt := now - lastUpdate
// fetch the client that finishes first
rcqNode := cm.rcQueue.PopItem().(*ClientNode) // if sumRecharge > 0 then the queue cannot be empty
// check whether it has already finished
dtNext := mclock.AbsTime(float64(rcqNode.rcFullIntValue-cm.rcLastIntValue) / bonusRatio)
if dt < dtNext {
// not finished yet, put it back, update integrator according
// to current bonusRatio and return
cm.rcQueue.Push(rcqNode, -rcqNode.rcFullIntValue)
cm.rcLastIntValue += int64(bonusRatio * float64(dt))
return
}
lastUpdate += dtNext
// finished recharging, update corrBufValue and sumRecharge if necessary and do next step
if rcqNode.corrBufValue < int64(rcqNode.params.BufLimit) {
rcqNode.corrBufValue = int64(rcqNode.params.BufLimit)
cm.updateCapFactor(lastUpdate, true)
cm.sumRecharge -= rcqNode.params.MinRecharge
}
cm.rcLastIntValue = rcqNode.rcFullIntValue
}
}
func (self *ClientManager) canStartReq() bool {
return self.simReqCnt < self.maxSimReq && self.rcSumValue < self.maxRcSum
// updateNodeRc updates a node's corrBufValue and adds an external correction value.
// It also adds or removes the rcQueue entry and updates ServerParams and sumRecharge if necessary.
func (cm *ClientManager) updateNodeRc(node *ClientNode, bvc int64, params *ServerParams, now mclock.AbsTime) {
cm.updateRecharge(now)
wasFull := true
if node.corrBufValue != int64(node.params.BufLimit) {
wasFull = false
node.corrBufValue += (cm.rcLastIntValue - node.rcLastIntValue) * int64(node.params.MinRecharge) / FixedPointMultiplier
if node.corrBufValue > int64(node.params.BufLimit) {
node.corrBufValue = int64(node.params.BufLimit)
}
node.rcLastIntValue = cm.rcLastIntValue
}
node.corrBufValue += bvc
if node.corrBufValue < 0 {
node.corrBufValue = 0
}
diff := int64(params.BufLimit - node.params.BufLimit)
if diff > 0 {
node.corrBufValue += diff
}
isFull := false
if node.corrBufValue >= int64(params.BufLimit) {
node.corrBufValue = int64(params.BufLimit)
isFull = true
}
sumRecharge := cm.sumRecharge
if !wasFull {
sumRecharge -= node.params.MinRecharge
}
if params != &node.params {
node.params = *params
}
if !isFull {
sumRecharge += node.params.MinRecharge
if node.queueIndex != -1 {
cm.rcQueue.Remove(node.queueIndex)
}
node.rcLastIntValue = cm.rcLastIntValue
node.rcFullIntValue = cm.rcLastIntValue + (int64(node.params.BufLimit)-node.corrBufValue)*FixedPointMultiplier/int64(node.params.MinRecharge)
cm.rcQueue.Push(node, -node.rcFullIntValue)
}
if sumRecharge != cm.sumRecharge {
cm.updateCapFactor(now, true)
cm.sumRecharge = sumRecharge
}
}
func (self *ClientManager) queueProc() {
for rc := range self.resumeQueue {
for {
time.Sleep(time.Millisecond * 10)
self.lock.Lock()
self.update(mclock.Now())
cs := self.canStartReq()
self.lock.Unlock()
if cs {
break
// updateCapFactor updates the total capacity factor. The capacity factor allows
// the total capacity of the system to go over the allowed total recharge value
// if the sum of momentarily recharging clients only exceeds the total recharge
// allowance in a very small fraction of time.
// The capacity factor is dropped quickly (with a small time constant) if sumRecharge
// exceeds totalRecharge. It is raised slowly (with a large time constant) if most
// of the total capacity is used by connected clients (totalConnected is larger than
// totalCapacity*capFactorRaiseThreshold) and sumRecharge stays under
// totalRecharge*totalConnected/totalCapacity.
func (cm *ClientManager) updateCapFactor(now mclock.AbsTime, refresh bool) {
if cm.totalRecharge == 0 {
return
}
dt := now - cm.capLastUpdate
cm.capLastUpdate = now
var d float64
if cm.sumRecharge > cm.totalRecharge {
d = (1 - float64(cm.sumRecharge)/float64(cm.totalRecharge)) * capFactorDropTC
} else {
totalConnected := float64(cm.totalConnected)
var connRatio float64
if totalConnected < cm.totalCapacity {
connRatio = totalConnected / cm.totalCapacity
} else {
connRatio = 1
}
if connRatio > capFactorRaiseThreshold {
sumRecharge := float64(cm.sumRecharge)
limit := float64(cm.totalRecharge) * connRatio
if sumRecharge < limit {
d = (1 - sumRecharge/limit) * (connRatio - capFactorRaiseThreshold) * (1 / (1 - capFactorRaiseThreshold)) * capFactorRaiseTC
}
}
close(rc)
}
}
func (self *ClientManager) accept(node *cmNode, time mclock.AbsTime) bool {
self.lock.Lock()
defer self.lock.Unlock()
self.update(time)
if !self.canStartReq() {
resume := make(chan bool)
self.lock.Unlock()
self.resumeQueue <- resume
<-resume
self.lock.Lock()
if _, ok := self.nodes[node]; !ok {
return false // reject if node has been removed or manager has been stopped
if d != 0 {
cm.capLogFactor += d * float64(dt)
if cm.capLogFactor < 0 {
cm.capLogFactor = 0
}
if refresh {
cm.refreshCapacity()
}
}
self.simReqCnt++
node.set(true, self.simReqCnt, self.sumWeight)
node.startValue = node.rcValue
self.update(self.time)
return true
}
func (self *ClientManager) stop(node *cmNode, time mclock.AbsTime) {
if node.serving {
self.update(time)
self.simReqCnt--
node.set(false, self.simReqCnt, self.sumWeight)
self.update(time)
// refreshCapacity recalculates the total capacity value and sends an update to the subscription
// channel if the relative change of the value since the last update is more than 0.1 percent
func (cm *ClientManager) refreshCapacity() {
totalCapacity := float64(cm.totalRecharge) * math.Exp(cm.capLogFactor)
if totalCapacity >= cm.totalCapacity*0.999 && totalCapacity <= cm.totalCapacity*1.001 {
return
}
cm.totalCapacity = totalCapacity
if cm.totalCapacityCh != nil {
select {
case cm.totalCapacityCh <- uint64(cm.totalCapacity):
default:
}
}
}
func (self *ClientManager) processed(node *cmNode, time mclock.AbsTime) (rcValue, rcCost uint64) {
self.lock.Lock()
defer self.lock.Unlock()
// SubscribeTotalCapacity returns all future updates to the total capacity value
// through a channel and also returns the current value
func (cm *ClientManager) SubscribeTotalCapacity(ch chan uint64) uint64 {
cm.lock.Lock()
defer cm.lock.Unlock()
self.stop(node, time)
return uint64(node.rcValue), uint64(node.rcValue - node.startValue)
cm.totalCapacityCh = ch
return uint64(cm.totalCapacity)
}
// PieceWiseLinear is used to describe recharge curves
type PieceWiseLinear []struct{ X, Y uint64 }
// ValueAt returns the curve's value at a given point
func (pwl PieceWiseLinear) ValueAt(x uint64) float64 {
l := 0
h := len(pwl)
if h == 0 {
return 0
}
for h != l {
m := (l + h) / 2
if x > pwl[m].X {
l = m + 1
} else {
h = m
}
}
if l == 0 {
return float64(pwl[0].Y)
}
l--
if h == len(pwl) {
return float64(pwl[l].Y)
}
dx := pwl[h].X - pwl[l].X
if dx < 1 {
return float64(pwl[l].Y)
}
return float64(pwl[l].Y) + float64(pwl[h].Y-pwl[l].Y)*float64(x-pwl[l].X)/float64(dx)
}
// Valid returns true if the X coordinates of the curve points are non-strictly monotonic
func (pwl PieceWiseLinear) Valid() bool {
var lastX uint64
for _, i := range pwl {
if i.X < lastX {
return false
}
lastX = i.X
}
return true
}

View File

@ -0,0 +1,123 @@
// Copyright 2018 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 flowcontrol
import (
"math/rand"
"testing"
"time"
"github.com/ethereum/go-ethereum/common/mclock"
)
type testNode struct {
node *ClientNode
bufLimit, capacity uint64
waitUntil mclock.AbsTime
index, totalCost uint64
}
const (
testMaxCost = 1000000
testLength = 100000
)
// testConstantTotalCapacity simulates multiple request sender nodes and verifies
// whether the total amount of served requests matches the expected value based on
// the total capacity and the duration of the test.
// Some nodes are sending requests occasionally so that their buffer should regularly
// reach the maximum while other nodes (the "max capacity nodes") are sending at the
// maximum permitted rate. The max capacity nodes are changed multiple times during
// a single test.
func TestConstantTotalCapacity(t *testing.T) {
testConstantTotalCapacity(t, 10, 1, 0)
testConstantTotalCapacity(t, 10, 1, 1)
testConstantTotalCapacity(t, 30, 1, 0)
testConstantTotalCapacity(t, 30, 2, 3)
testConstantTotalCapacity(t, 100, 1, 0)
testConstantTotalCapacity(t, 100, 3, 5)
testConstantTotalCapacity(t, 100, 5, 10)
}
func testConstantTotalCapacity(t *testing.T, nodeCount, maxCapacityNodes, randomSend int) {
clock := &mclock.Simulated{}
nodes := make([]*testNode, nodeCount)
var totalCapacity uint64
for i := range nodes {
nodes[i] = &testNode{capacity: uint64(50000 + rand.Intn(100000))}
totalCapacity += nodes[i].capacity
}
m := NewClientManager(PieceWiseLinear{{0, totalCapacity}}, clock)
for _, n := range nodes {
n.bufLimit = n.capacity * 6000 //uint64(2000+rand.Intn(10000))
n.node = NewClientNode(m, ServerParams{BufLimit: n.bufLimit, MinRecharge: n.capacity})
}
maxNodes := make([]int, maxCapacityNodes)
for i := range maxNodes {
// we don't care if some indexes are selected multiple times
// in that case we have fewer max nodes
maxNodes[i] = rand.Intn(nodeCount)
}
for i := 0; i < testLength; i++ {
now := clock.Now()
for _, idx := range maxNodes {
for nodes[idx].send(t, now) {
}
}
if rand.Intn(testLength) < maxCapacityNodes*3 {
maxNodes[rand.Intn(maxCapacityNodes)] = rand.Intn(nodeCount)
}
sendCount := randomSend
for sendCount > 0 {
if nodes[rand.Intn(nodeCount)].send(t, now) {
sendCount--
}
}
clock.Run(time.Millisecond)
}
var totalCost uint64
for _, n := range nodes {
totalCost += n.totalCost
}
ratio := float64(totalCost) / float64(totalCapacity) / testLength
if ratio < 0.98 || ratio > 1.02 {
t.Errorf("totalCost/totalCapacity/testLength ratio incorrect (expected: 1, got: %f)", ratio)
}
}
func (n *testNode) send(t *testing.T, now mclock.AbsTime) bool {
if now < n.waitUntil {
return false
}
n.index++
if ok, _, _ := n.node.AcceptRequest(0, n.index, testMaxCost); !ok {
t.Fatalf("Rejected request after expected waiting time has passed")
}
rcost := uint64(rand.Int63n(testMaxCost))
bv := n.node.RequestProcessed(0, n.index, testMaxCost, rcost)
if bv < testMaxCost {
n.waitUntil = now + mclock.AbsTime((testMaxCost-bv)*1001000/n.capacity)
}
//n.waitUntil = now + mclock.AbsTime(float64(testMaxCost)*1001000/float64(n.capacity)*(1-float64(bv)/float64(n.bufLimit)))
n.totalCost += rcost
return true
}