les: move client pool to les/vflux/server (#22495)

* les: move client pool to les/vflux/server

* les/vflux/server: un-expose NodeBalance, remove unused fn, fix bugs

* tests/fuzzers/vflux: add ClientPool fuzzer

* les/vflux/server: fixed balance tests

* les: rebase fix

* les/vflux/server: fixed more bugs

* les/vflux/server: unexported NodeStateMachine fields and flags

* les/vflux/server: unexport all internal components and functions

* les/vflux/server: fixed priorityPool test

* les/vflux/server: polish balance

* les/vflux/server: fixed mutex locking error

* les/vflux/server: priorityPool bug fixed

* common/prque: make Prque wrap-around priority handling optional

* les/vflux/server: rename funcs, small optimizations

* les/vflux/server: fixed timeUntil

* les/vflux/server: separated balance.posValue and negValue

* les/vflux/server: polish setup

* les/vflux/server: enforce capacity curve monotonicity

* les/vflux/server: simplified requestCapacity

* les/vflux/server: requestCapacity with target range, no iterations in SetCapacity

* les/vflux/server: minor changes

* les/vflux/server: moved default factors to balanceTracker

* les/vflux/server: set inactiveFlag in priorityPool

* les/vflux/server: moved related metrics to vfs package

* les/vflux/client: make priorityPool temp state logic cleaner

* les/vflux/server: changed log.Crit to log.Error

* add vflux fuzzer to oss-fuzz

Co-authored-by: rjl493456442 <garyrong0905@gmail.com>
This commit is contained in:
Felföldi Zsolt 2021-04-06 20:42:50 +02:00 committed by GitHub
parent e275b1a293
commit 2d89fe0883
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 1987 additions and 1545 deletions

View File

@ -55,7 +55,7 @@ type (
// NewLazyQueue creates a new lazy queue // NewLazyQueue creates a new lazy queue
func NewLazyQueue(setIndex SetIndexCallback, priority PriorityCallback, maxPriority MaxPriorityCallback, clock mclock.Clock, refreshPeriod time.Duration) *LazyQueue { func NewLazyQueue(setIndex SetIndexCallback, priority PriorityCallback, maxPriority MaxPriorityCallback, clock mclock.Clock, refreshPeriod time.Duration) *LazyQueue {
q := &LazyQueue{ q := &LazyQueue{
popQueue: newSstack(nil), popQueue: newSstack(nil, false),
setIndex: setIndex, setIndex: setIndex,
priority: priority, priority: priority,
maxPriority: maxPriority, maxPriority: maxPriority,
@ -71,8 +71,8 @@ func NewLazyQueue(setIndex SetIndexCallback, priority PriorityCallback, maxPrior
// Reset clears the contents of the queue // Reset clears the contents of the queue
func (q *LazyQueue) Reset() { func (q *LazyQueue) Reset() {
q.queue[0] = newSstack(q.setIndex0) q.queue[0] = newSstack(q.setIndex0, false)
q.queue[1] = newSstack(q.setIndex1) q.queue[1] = newSstack(q.setIndex1, false)
} }
// Refresh performs queue re-evaluation if necessary // Refresh performs queue re-evaluation if necessary

View File

@ -28,7 +28,12 @@ type Prque struct {
// New creates a new priority queue. // New creates a new priority queue.
func New(setIndex SetIndexCallback) *Prque { func New(setIndex SetIndexCallback) *Prque {
return &Prque{newSstack(setIndex)} return &Prque{newSstack(setIndex, false)}
}
// NewWrapAround creates a new priority queue with wrap-around priority handling.
func NewWrapAround(setIndex SetIndexCallback) *Prque {
return &Prque{newSstack(setIndex, true)}
} }
// Pushes a value with a given priority into the queue, expanding if necessary. // Pushes a value with a given priority into the queue, expanding if necessary.

View File

@ -31,22 +31,24 @@ type SetIndexCallback func(data interface{}, index int)
// the stack (heap) functionality and the Len, Less and Swap methods for the // the stack (heap) functionality and the Len, Less and Swap methods for the
// sortability requirements of the heaps. // sortability requirements of the heaps.
type sstack struct { type sstack struct {
setIndex SetIndexCallback setIndex SetIndexCallback
size int size int
capacity int capacity int
offset int offset int
wrapAround bool
blocks [][]*item blocks [][]*item
active []*item active []*item
} }
// Creates a new, empty stack. // Creates a new, empty stack.
func newSstack(setIndex SetIndexCallback) *sstack { func newSstack(setIndex SetIndexCallback, wrapAround bool) *sstack {
result := new(sstack) result := new(sstack)
result.setIndex = setIndex result.setIndex = setIndex
result.active = make([]*item, blockSize) result.active = make([]*item, blockSize)
result.blocks = [][]*item{result.active} result.blocks = [][]*item{result.active}
result.capacity = blockSize result.capacity = blockSize
result.wrapAround = wrapAround
return result return result
} }
@ -94,7 +96,11 @@ func (s *sstack) Len() int {
// Compares the priority of two elements of the stack (higher is first). // Compares the priority of two elements of the stack (higher is first).
// Required by sort.Interface. // Required by sort.Interface.
func (s *sstack) Less(i, j int) bool { func (s *sstack) Less(i, j int) bool {
return (s.blocks[i/blockSize][i%blockSize].priority - s.blocks[j/blockSize][j%blockSize].priority) > 0 a, b := s.blocks[i/blockSize][i%blockSize].priority, s.blocks[j/blockSize][j%blockSize].priority
if s.wrapAround {
return a-b > 0
}
return a > b
} }
// Swaps two elements in the stack. Required by sort.Interface. // Swaps two elements in the stack. Required by sort.Interface.
@ -110,5 +116,5 @@ func (s *sstack) Swap(i, j int) {
// Resets the stack, effectively clearing its contents. // Resets the stack, effectively clearing its contents.
func (s *sstack) Reset() { func (s *sstack) Reset() {
*s = *newSstack(s.setIndex) *s = *newSstack(s.setIndex, false)
} }

View File

@ -21,7 +21,7 @@ func TestSstack(t *testing.T) {
for i := 0; i < size; i++ { for i := 0; i < size; i++ {
data[i] = &item{rand.Int(), rand.Int63()} data[i] = &item{rand.Int(), rand.Int63()}
} }
stack := newSstack(nil) stack := newSstack(nil, false)
for rep := 0; rep < 2; rep++ { for rep := 0; rep < 2; rep++ {
// Push all the data into the stack, pop out every second // Push all the data into the stack, pop out every second
secs := []*item{} secs := []*item{}
@ -55,7 +55,7 @@ func TestSstackSort(t *testing.T) {
data[i] = &item{rand.Int(), int64(i)} data[i] = &item{rand.Int(), int64(i)}
} }
// Push all the data into the stack // Push all the data into the stack
stack := newSstack(nil) stack := newSstack(nil, false)
for _, val := range data { for _, val := range data {
stack.Push(val) stack.Push(val)
} }
@ -76,7 +76,7 @@ func TestSstackReset(t *testing.T) {
for i := 0; i < size; i++ { for i := 0; i < size; i++ {
data[i] = &item{rand.Int(), rand.Int63()} data[i] = &item{rand.Int(), rand.Int63()}
} }
stack := newSstack(nil) stack := newSstack(nil, false)
for rep := 0; rep < 2; rep++ { for rep := 0; rep < 2; rep++ {
// Push all the data into the stack, pop out every second // Push all the data into the stack, pop out every second
secs := []*item{} secs := []*item{}

View File

@ -31,7 +31,6 @@ var (
errNoCheckpoint = errors.New("no local checkpoint provided") errNoCheckpoint = errors.New("no local checkpoint provided")
errNotActivated = errors.New("checkpoint registrar is not activated") errNotActivated = errors.New("checkpoint registrar is not activated")
errUnknownBenchmarkType = errors.New("unknown benchmark type") errUnknownBenchmarkType = errors.New("unknown benchmark type")
errNoPriority = errors.New("priority too low to raise capacity")
) )
// PrivateLightServerAPI provides an API to access the LES light server. // PrivateLightServerAPI provides an API to access the LES light server.
@ -44,8 +43,8 @@ type PrivateLightServerAPI struct {
func NewPrivateLightServerAPI(server *LesServer) *PrivateLightServerAPI { func NewPrivateLightServerAPI(server *LesServer) *PrivateLightServerAPI {
return &PrivateLightServerAPI{ return &PrivateLightServerAPI{
server: server, server: server,
defaultPosFactors: server.clientPool.defaultPosFactors, defaultPosFactors: defaultPosFactors,
defaultNegFactors: server.clientPool.defaultNegFactors, defaultNegFactors: defaultNegFactors,
} }
} }
@ -66,7 +65,9 @@ func (api *PrivateLightServerAPI) ServerInfo() map[string]interface{} {
res := make(map[string]interface{}) res := make(map[string]interface{})
res["minimumCapacity"] = api.server.minCapacity res["minimumCapacity"] = api.server.minCapacity
res["maximumCapacity"] = api.server.maxCapacity res["maximumCapacity"] = api.server.maxCapacity
res["totalCapacity"], res["totalConnectedCapacity"], res["priorityConnectedCapacity"] = api.server.clientPool.capacityInfo() _, res["totalCapacity"] = api.server.clientPool.Limits()
_, res["totalConnectedCapacity"] = api.server.clientPool.Active()
res["priorityConnectedCapacity"] = 0 //TODO connect when token sale module is added
return res return res
} }
@ -80,9 +81,18 @@ func (api *PrivateLightServerAPI) ClientInfo(nodes []string) map[enode.ID]map[st
} }
res := make(map[enode.ID]map[string]interface{}) res := make(map[enode.ID]map[string]interface{})
api.server.clientPool.forClients(ids, func(client *clientInfo) { if len(ids) == 0 {
res[client.node.ID()] = api.clientInfo(client) ids = api.server.peers.ids()
}) }
for _, id := range ids {
if peer := api.server.peers.peer(id); peer != nil {
res[id] = api.clientInfo(peer, peer.balance)
} else {
api.server.clientPool.BalanceOperation(id, "", func(balance vfs.AtomicBalanceOperator) {
res[id] = api.clientInfo(nil, balance)
})
}
}
return res return res
} }
@ -94,31 +104,35 @@ func (api *PrivateLightServerAPI) ClientInfo(nodes []string) map[enode.ID]map[st
// assigned to it. // assigned to it.
func (api *PrivateLightServerAPI) PriorityClientInfo(start, stop enode.ID, maxCount int) map[enode.ID]map[string]interface{} { func (api *PrivateLightServerAPI) PriorityClientInfo(start, stop enode.ID, maxCount int) map[enode.ID]map[string]interface{} {
res := make(map[enode.ID]map[string]interface{}) res := make(map[enode.ID]map[string]interface{})
ids := api.server.clientPool.bt.GetPosBalanceIDs(start, stop, maxCount+1) ids := api.server.clientPool.GetPosBalanceIDs(start, stop, maxCount+1)
if len(ids) > maxCount { if len(ids) > maxCount {
res[ids[maxCount]] = make(map[string]interface{}) res[ids[maxCount]] = make(map[string]interface{})
ids = ids[:maxCount] ids = ids[:maxCount]
} }
if len(ids) != 0 { for _, id := range ids {
api.server.clientPool.forClients(ids, func(client *clientInfo) { if peer := api.server.peers.peer(id); peer != nil {
res[client.node.ID()] = api.clientInfo(client) res[id] = api.clientInfo(peer, peer.balance)
}) } else {
api.server.clientPool.BalanceOperation(id, "", func(balance vfs.AtomicBalanceOperator) {
res[id] = api.clientInfo(nil, balance)
})
}
} }
return res return res
} }
// clientInfo creates a client info data structure // clientInfo creates a client info data structure
func (api *PrivateLightServerAPI) clientInfo(c *clientInfo) map[string]interface{} { func (api *PrivateLightServerAPI) clientInfo(peer *clientPeer, balance vfs.ReadOnlyBalance) map[string]interface{} {
info := make(map[string]interface{}) info := make(map[string]interface{})
pb, nb := c.balance.GetBalance() pb, nb := balance.GetBalance()
info["isConnected"] = c.connected info["isConnected"] = peer != nil
info["pricing/balance"] = pb info["pricing/balance"] = pb
info["priority"] = pb != 0 info["priority"] = pb != 0
// cb := api.server.clientPool.ndb.getCurrencyBalance(id) // cb := api.server.clientPool.ndb.getCurrencyBalance(id)
// info["pricing/currency"] = cb.amount // info["pricing/currency"] = cb.amount
if c.connected { if peer != nil {
info["connectionTime"] = float64(mclock.Now()-c.connectedAt) / float64(time.Second) info["connectionTime"] = float64(mclock.Now()-peer.connectedAt) / float64(time.Second)
info["capacity"], _ = api.server.clientPool.ns.GetField(c.node, priorityPoolSetup.CapacityField).(uint64) info["capacity"] = peer.getCapacity()
info["pricing/negBalance"] = nb info["pricing/negBalance"] = nb
} }
return info return info
@ -126,7 +140,7 @@ func (api *PrivateLightServerAPI) clientInfo(c *clientInfo) map[string]interface
// setParams either sets the given parameters for a single connected client (if specified) // setParams either sets the given parameters for a single connected client (if specified)
// or the default parameters applicable to clients connected in the future // or the default parameters applicable to clients connected in the future
func (api *PrivateLightServerAPI) setParams(params map[string]interface{}, client *clientInfo, posFactors, negFactors *vfs.PriceFactors) (updateFactors bool, err error) { func (api *PrivateLightServerAPI) setParams(params map[string]interface{}, client *clientPeer, posFactors, negFactors *vfs.PriceFactors) (updateFactors bool, err error) {
defParams := client == nil defParams := client == nil
for name, value := range params { for name, value := range params {
errValue := func() error { errValue := func() error {
@ -156,9 +170,8 @@ func (api *PrivateLightServerAPI) setParams(params map[string]interface{}, clien
setFactor(&negFactors.RequestFactor) setFactor(&negFactors.RequestFactor)
case !defParams && name == "capacity": case !defParams && name == "capacity":
if capacity, ok := value.(float64); ok && uint64(capacity) >= api.server.minCapacity { if capacity, ok := value.(float64); ok && uint64(capacity) >= api.server.minCapacity {
_, err = api.server.clientPool.setCapacity(client.node, client.address, uint64(capacity), 0, true) _, err = api.server.clientPool.SetCapacity(client.Node(), uint64(capacity), 0, false)
// Don't have to call factor update explicitly. It's already done // time factor recalculation is performed automatically by the balance tracker
// in setCapacity function.
} else { } else {
err = errValue() err = errValue()
} }
@ -179,31 +192,25 @@ func (api *PrivateLightServerAPI) setParams(params map[string]interface{}, clien
// SetClientParams sets client parameters for all clients listed in the ids list // SetClientParams sets client parameters for all clients listed in the ids list
// or all connected clients if the list is empty // or all connected clients if the list is empty
func (api *PrivateLightServerAPI) SetClientParams(nodes []string, params map[string]interface{}) error { func (api *PrivateLightServerAPI) SetClientParams(nodes []string, params map[string]interface{}) error {
var ( var err error
ids []enode.ID
err error
)
for _, node := range nodes { for _, node := range nodes {
if id, err := parseNode(node); err != nil { var id enode.ID
if id, err = parseNode(node); err != nil {
return err return err
} else {
ids = append(ids, id)
} }
} if peer := api.server.peers.peer(id); peer != nil {
api.server.clientPool.forClients(ids, func(client *clientInfo) { posFactors, negFactors := peer.balance.GetPriceFactors()
if client.connected { update, e := api.setParams(params, peer, &posFactors, &negFactors)
posFactors, negFactors := client.balance.GetPriceFactors()
update, e := api.setParams(params, client, &posFactors, &negFactors)
if update { if update {
client.balance.SetPriceFactors(posFactors, negFactors) peer.balance.SetPriceFactors(posFactors, negFactors)
} }
if e != nil { if e != nil {
err = e err = e
} }
} else { } else {
err = fmt.Errorf("client %064x is not connected", client.node.ID()) err = fmt.Errorf("client %064x is not connected", id)
} }
}) }
return err return err
} }
@ -211,7 +218,7 @@ func (api *PrivateLightServerAPI) SetClientParams(nodes []string, params map[str
func (api *PrivateLightServerAPI) SetDefaultParams(params map[string]interface{}) error { func (api *PrivateLightServerAPI) SetDefaultParams(params map[string]interface{}) error {
update, err := api.setParams(params, nil, &api.defaultPosFactors, &api.defaultNegFactors) update, err := api.setParams(params, nil, &api.defaultPosFactors, &api.defaultNegFactors)
if update { if update {
api.server.clientPool.setDefaultFactors(api.defaultPosFactors, api.defaultNegFactors) api.server.clientPool.SetDefaultFactors(api.defaultPosFactors, api.defaultNegFactors)
} }
return err return err
} }
@ -224,7 +231,7 @@ func (api *PrivateLightServerAPI) SetConnectedBias(bias time.Duration) error {
if bias < time.Duration(0) { if bias < time.Duration(0) {
return fmt.Errorf("bias illegal: %v less than 0", bias) return fmt.Errorf("bias illegal: %v less than 0", bias)
} }
api.server.clientPool.setConnectedBias(bias) api.server.clientPool.SetConnectedBias(bias)
return nil return nil
} }
@ -235,8 +242,8 @@ func (api *PrivateLightServerAPI) AddBalance(node string, amount int64) (balance
if id, err = parseNode(node); err != nil { if id, err = parseNode(node); err != nil {
return return
} }
api.server.clientPool.forClients([]enode.ID{id}, func(c *clientInfo) { api.server.clientPool.BalanceOperation(id, "", func(nb vfs.AtomicBalanceOperator) {
balance[0], balance[1], err = c.balance.AddBalance(amount) balance[0], balance[1], err = nb.AddBalance(amount)
}) })
return return
} }
@ -338,14 +345,12 @@ func (api *PrivateDebugAPI) FreezeClient(node string) error {
if id, err = parseNode(node); err != nil { if id, err = parseNode(node); err != nil {
return err return err
} }
api.server.clientPool.forClients([]enode.ID{id}, func(c *clientInfo) { if peer := api.server.peers.peer(id); peer != nil {
if c.connected { peer.freeze()
c.peer.freeze() return nil
} else { } else {
err = fmt.Errorf("client %064x is not connected", id[:]) return fmt.Errorf("client %064x is not connected", id[:])
} }
})
return err
} }
// PrivateLightAPI provides an API to access the LES light server or light client. // PrivateLightAPI provides an API to access the LES light server or light client.

View File

@ -1,453 +0,0 @@
// Copyright 2019 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 les
import (
"fmt"
"sync"
"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/les/vflux"
vfs "github.com/ethereum/go-ethereum/les/vflux/server"
"github.com/ethereum/go-ethereum/log"
"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 (
defaultNegExpTC = 3600 // default time constant (in seconds) for exponentially reducing negative balance
// defaultConnectedBias is applied to already connected clients So that
// already connected client won't be kicked out very soon and we
// can ensure all connected clients can have enough time to request
// or sync some data.
//
// todo(rjl493456442) make it configurable. It can be the option of
// free trial time!
defaultConnectedBias = time.Minute * 3
inactiveTimeout = time.Second * 10
)
// clientPool implements a client database that assigns a priority to each client
// based on a positive and negative balance. Positive balance is externally assigned
// to prioritized clients and is decreased with connection time and processed
// requests (unless the price factors are zero). If the positive balance is zero
// then negative balance is accumulated.
//
// Balance tracking and priority calculation for connected clients is done by
// balanceTracker. activeQueue ensures that clients with the lowest positive or
// highest negative balance get evicted when the total capacity allowance is full
// and new clients with a better balance want to connect.
//
// Already connected nodes receive a small bias in their favor in order to avoid
// accepting and instantly kicking out clients. In theory, we try to ensure that
// each client can have several minutes of connection time.
//
// Balances of disconnected clients are stored in nodeDB including positive balance
// and negative banalce. Boeth positive balance and negative balance will decrease
// exponentially. If the balance is low enough, then the record will be dropped.
type clientPool struct {
vfs.BalanceTrackerSetup
vfs.PriorityPoolSetup
lock sync.Mutex
clock mclock.Clock
closed bool
removePeer func(enode.ID)
synced func() bool
ns *nodestate.NodeStateMachine
pp *vfs.PriorityPool
bt *vfs.BalanceTracker
defaultPosFactors, defaultNegFactors vfs.PriceFactors
posExpTC, negExpTC uint64
minCap uint64 // The minimal capacity value allowed for any client
connectedBias time.Duration
capLimit uint64
}
// clientPoolPeer represents a client peer in the pool.
// Positive balances are assigned to node key while negative balances are assigned
// to freeClientId. Currently network IP address without port is used because
// clients have a limited access to IP addresses while new node keys can be easily
// generated so it would be useless to assign a negative value to them.
type clientPoolPeer interface {
Node() *enode.Node
freeClientId() string
updateCapacity(uint64)
freeze()
allowInactive() bool
}
// clientInfo defines all information required by clientpool.
type clientInfo struct {
node *enode.Node
address string
peer clientPoolPeer
connected, priority bool
connectedAt mclock.AbsTime
balance *vfs.NodeBalance
}
// newClientPool creates a new client pool
func newClientPool(ns *nodestate.NodeStateMachine, lesDb ethdb.Database, minCap uint64, connectedBias time.Duration, clock mclock.Clock, removePeer func(enode.ID), synced func() bool) *clientPool {
pool := &clientPool{
ns: ns,
BalanceTrackerSetup: balanceTrackerSetup,
PriorityPoolSetup: priorityPoolSetup,
clock: clock,
minCap: minCap,
connectedBias: connectedBias,
removePeer: removePeer,
synced: synced,
}
pool.bt = vfs.NewBalanceTracker(ns, balanceTrackerSetup, lesDb, clock, &utils.Expirer{}, &utils.Expirer{})
pool.pp = vfs.NewPriorityPool(ns, priorityPoolSetup, clock, minCap, connectedBias, 4)
// set default expiration constants used by tests
// Note: server overwrites this if token sale is active
pool.bt.SetExpirationTCs(0, defaultNegExpTC)
ns.SubscribeState(pool.InactiveFlag.Or(pool.PriorityFlag), func(node *enode.Node, oldState, newState nodestate.Flags) {
if newState.Equals(pool.InactiveFlag) {
ns.AddTimeout(node, pool.InactiveFlag, inactiveTimeout)
}
if oldState.Equals(pool.InactiveFlag) && newState.Equals(pool.InactiveFlag.Or(pool.PriorityFlag)) {
ns.SetStateSub(node, pool.InactiveFlag, nodestate.Flags{}, 0) // remove timeout
}
})
ns.SubscribeState(pool.ActiveFlag.Or(pool.PriorityFlag), func(node *enode.Node, oldState, newState nodestate.Flags) {
c, _ := ns.GetField(node, clientInfoField).(*clientInfo)
if c == nil {
return
}
c.priority = newState.HasAll(pool.PriorityFlag)
if newState.Equals(pool.ActiveFlag) {
cap, _ := ns.GetField(node, pool.CapacityField).(uint64)
if cap > minCap {
pool.pp.RequestCapacity(node, minCap, 0, true)
}
}
})
ns.SubscribeState(pool.InactiveFlag.Or(pool.ActiveFlag), func(node *enode.Node, oldState, newState nodestate.Flags) {
if oldState.IsEmpty() {
clientConnectedMeter.Mark(1)
log.Debug("Client connected", "id", node.ID())
}
if oldState.Equals(pool.InactiveFlag) && newState.Equals(pool.ActiveFlag) {
clientActivatedMeter.Mark(1)
log.Debug("Client activated", "id", node.ID())
}
if oldState.Equals(pool.ActiveFlag) && newState.Equals(pool.InactiveFlag) {
clientDeactivatedMeter.Mark(1)
log.Debug("Client deactivated", "id", node.ID())
c, _ := ns.GetField(node, clientInfoField).(*clientInfo)
if c == nil || !c.peer.allowInactive() {
pool.removePeer(node.ID())
}
}
if newState.IsEmpty() {
clientDisconnectedMeter.Mark(1)
log.Debug("Client disconnected", "id", node.ID())
pool.removePeer(node.ID())
}
})
var totalConnected uint64
ns.SubscribeField(pool.CapacityField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) {
oldCap, _ := oldValue.(uint64)
newCap, _ := newValue.(uint64)
totalConnected += newCap - oldCap
totalConnectedGauge.Update(int64(totalConnected))
c, _ := ns.GetField(node, clientInfoField).(*clientInfo)
if c != nil {
c.peer.updateCapacity(newCap)
}
})
return pool
}
// stop shuts the client pool down
func (f *clientPool) stop() {
f.lock.Lock()
f.closed = true
f.lock.Unlock()
f.ns.ForEach(nodestate.Flags{}, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) {
// enforces saving all balances in BalanceTracker
f.disconnectNode(node)
})
f.bt.Stop()
}
// connect should be called after a successful handshake. If the connection was
// rejected, there is no need to call disconnect.
func (f *clientPool) connect(peer clientPoolPeer) (uint64, error) {
f.lock.Lock()
defer f.lock.Unlock()
// Short circuit if clientPool is already closed.
if f.closed {
return 0, fmt.Errorf("Client pool is already closed")
}
// Dedup connected peers.
node, freeID := peer.Node(), peer.freeClientId()
if f.ns.GetField(node, clientInfoField) != nil {
log.Debug("Client already connected", "address", freeID, "id", node.ID().String())
return 0, fmt.Errorf("Client already connected address=%s id=%s", freeID, node.ID().String())
}
now := f.clock.Now()
c := &clientInfo{
node: node,
address: freeID,
peer: peer,
connected: true,
connectedAt: now,
}
f.ns.SetField(node, clientInfoField, c)
f.ns.SetField(node, connAddressField, freeID)
if c.balance, _ = f.ns.GetField(node, f.BalanceField).(*vfs.NodeBalance); c.balance == nil {
f.disconnect(peer)
return 0, nil
}
c.balance.SetPriceFactors(f.defaultPosFactors, f.defaultNegFactors)
f.ns.SetState(node, f.InactiveFlag, nodestate.Flags{}, 0)
var allowed bool
f.ns.Operation(func() {
_, allowed = f.pp.RequestCapacity(node, f.minCap, f.connectedBias, true)
})
if allowed {
return f.minCap, nil
}
if !peer.allowInactive() {
f.disconnect(peer)
}
return 0, nil
}
// setConnectedBias sets the connection bias, which is applied to already connected clients
// So that already connected client won't be kicked out very soon and we can ensure all
// connected clients can have enough time to request or sync some data.
func (f *clientPool) setConnectedBias(bias time.Duration) {
f.lock.Lock()
defer f.lock.Unlock()
f.connectedBias = bias
f.pp.SetActiveBias(bias)
}
// disconnect should be called when a connection is terminated. If the disconnection
// was initiated by the pool itself using disconnectFn then calling disconnect is
// not necessary but permitted.
func (f *clientPool) disconnect(p clientPoolPeer) {
f.disconnectNode(p.Node())
}
// disconnectNode removes node fields and flags related to connected status
func (f *clientPool) disconnectNode(node *enode.Node) {
f.ns.SetField(node, connAddressField, nil)
f.ns.SetField(node, clientInfoField, nil)
}
// setDefaultFactors sets the default price factors applied to subsequently connected clients
func (f *clientPool) setDefaultFactors(posFactors, negFactors vfs.PriceFactors) {
f.lock.Lock()
defer f.lock.Unlock()
f.defaultPosFactors = posFactors
f.defaultNegFactors = negFactors
}
// capacityInfo returns the total capacity allowance, the total capacity of connected
// clients and the total capacity of connected and prioritized clients
func (f *clientPool) capacityInfo() (uint64, uint64, uint64) {
f.lock.Lock()
defer f.lock.Unlock()
// total priority active cap will be supported when the token issuer module is added
_, activeCap := f.pp.Active()
return f.capLimit, activeCap, 0
}
// setLimits sets the maximum number and total capacity of connected clients,
// dropping some of them if necessary.
func (f *clientPool) setLimits(totalConn int, totalCap uint64) {
f.lock.Lock()
defer f.lock.Unlock()
f.capLimit = totalCap
f.pp.SetLimits(uint64(totalConn), totalCap)
}
// setCapacity sets the assigned capacity of a connected client
func (f *clientPool) setCapacity(node *enode.Node, freeID string, capacity uint64, bias time.Duration, setCap bool) (uint64, error) {
c, _ := f.ns.GetField(node, clientInfoField).(*clientInfo)
if c == nil {
if setCap {
return 0, fmt.Errorf("client %064x is not connected", node.ID())
}
c = &clientInfo{node: node}
f.ns.SetField(node, clientInfoField, c)
f.ns.SetField(node, connAddressField, freeID)
if c.balance, _ = f.ns.GetField(node, f.BalanceField).(*vfs.NodeBalance); c.balance == nil {
log.Error("BalanceField is missing", "node", node.ID())
return 0, fmt.Errorf("BalanceField of %064x is missing", node.ID())
}
defer func() {
f.ns.SetField(node, connAddressField, nil)
f.ns.SetField(node, clientInfoField, nil)
}()
}
var (
minPriority int64
allowed bool
)
f.ns.Operation(func() {
if !setCap || c.priority {
// check clientInfo.priority inside Operation to ensure thread safety
minPriority, allowed = f.pp.RequestCapacity(node, capacity, bias, setCap)
}
})
if allowed {
return 0, nil
}
missing := c.balance.PosBalanceMissing(minPriority, capacity, bias)
if missing < 1 {
// ensure that we never return 0 missing and insufficient priority error
missing = 1
}
return missing, errNoPriority
}
// setCapacityLocked is the equivalent of setCapacity used when f.lock is already locked
func (f *clientPool) setCapacityLocked(node *enode.Node, freeID string, capacity uint64, minConnTime time.Duration, setCap bool) (uint64, error) {
f.lock.Lock()
defer f.lock.Unlock()
return f.setCapacity(node, freeID, capacity, minConnTime, setCap)
}
// forClients calls the supplied callback for either the listed node IDs or all connected
// nodes. It passes a valid clientInfo to the callback and ensures that the necessary
// fields and flags are set in order for BalanceTracker and PriorityPool to work even if
// the node is not connected.
func (f *clientPool) forClients(ids []enode.ID, cb func(client *clientInfo)) {
f.lock.Lock()
defer f.lock.Unlock()
if len(ids) == 0 {
f.ns.ForEach(nodestate.Flags{}, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) {
c, _ := f.ns.GetField(node, clientInfoField).(*clientInfo)
if c != nil {
cb(c)
}
})
} else {
for _, id := range ids {
node := f.ns.GetNode(id)
if node == nil {
node = enode.SignNull(&enr.Record{}, id)
}
c, _ := f.ns.GetField(node, clientInfoField).(*clientInfo)
if c != nil {
cb(c)
} else {
c = &clientInfo{node: node}
f.ns.SetField(node, clientInfoField, c)
f.ns.SetField(node, connAddressField, "")
if c.balance, _ = f.ns.GetField(node, f.BalanceField).(*vfs.NodeBalance); c.balance != nil {
cb(c)
} else {
log.Error("BalanceField is missing")
}
f.ns.SetField(node, connAddressField, nil)
f.ns.SetField(node, clientInfoField, nil)
}
}
}
}
// serveCapQuery serves a vflux capacity query. It receives multiple token amount values
// and a bias time value. For each given token amount it calculates the maximum achievable
// capacity in case the amount is added to the balance.
func (f *clientPool) serveCapQuery(id enode.ID, freeID string, data []byte) []byte {
var req vflux.CapacityQueryReq
if rlp.DecodeBytes(data, &req) != nil {
return nil
}
if l := len(req.AddTokens); l == 0 || l > vflux.CapacityQueryMaxLen {
return nil
}
result := make(vflux.CapacityQueryReply, len(req.AddTokens))
if !f.synced() {
capacityQueryZeroMeter.Mark(1)
reply, _ := rlp.EncodeToBytes(&result)
return reply
}
node := f.ns.GetNode(id)
if node == nil {
node = enode.SignNull(&enr.Record{}, id)
}
c, _ := f.ns.GetField(node, clientInfoField).(*clientInfo)
if c == nil {
c = &clientInfo{node: node}
f.ns.SetField(node, clientInfoField, c)
f.ns.SetField(node, connAddressField, freeID)
defer func() {
f.ns.SetField(node, connAddressField, nil)
f.ns.SetField(node, clientInfoField, nil)
}()
if c.balance, _ = f.ns.GetField(node, f.BalanceField).(*vfs.NodeBalance); c.balance == nil {
log.Error("BalanceField is missing", "node", node.ID())
return nil
}
}
// use vfs.CapacityCurve to answer request for multiple newly bought token amounts
curve := f.pp.GetCapacityCurve().Exclude(id)
bias := time.Second * time.Duration(req.Bias)
if f.connectedBias > bias {
bias = f.connectedBias
}
pb, _ := c.balance.GetBalance()
for i, addTokens := range req.AddTokens {
add := addTokens.Int64()
result[i] = curve.MaxCapacity(func(capacity uint64) int64 {
return c.balance.EstimatePriority(capacity, add, 0, bias, false) / int64(capacity)
})
if add <= 0 && uint64(-add) >= pb && result[i] > f.minCap {
result[i] = f.minCap
}
if result[i] < f.minCap {
result[i] = 0
}
}
// add first result to metrics (don't care about priority client multi-queries yet)
if result[0] == 0 {
capacityQueryZeroMeter.Mark(1)
} else {
capacityQueryNonZeroMeter.Mark(1)
}
reply, _ := rlp.EncodeToBytes(&result)
return reply
}

View File

@ -108,7 +108,7 @@ type ClientManager struct {
func NewClientManager(curve PieceWiseLinear, clock mclock.Clock) *ClientManager { func NewClientManager(curve PieceWiseLinear, clock mclock.Clock) *ClientManager {
cm := &ClientManager{ cm := &ClientManager{
clock: clock, clock: clock,
rcQueue: prque.New(func(a interface{}, i int) { a.(*ClientNode).queueIndex = i }), rcQueue: prque.NewWrapAround(func(a interface{}, i int) { a.(*ClientNode).queueIndex = i }),
capLastUpdate: clock.Now(), capLastUpdate: clock.Now(),
stop: make(chan chan struct{}), stop: make(chan chan struct{}),
} }

View File

@ -73,12 +73,9 @@ var (
serverConnectionGauge = metrics.NewRegisteredGauge("les/connection/server", nil) serverConnectionGauge = metrics.NewRegisteredGauge("les/connection/server", nil)
clientConnectionGauge = metrics.NewRegisteredGauge("les/connection/client", nil) clientConnectionGauge = metrics.NewRegisteredGauge("les/connection/client", nil)
totalCapacityGauge = metrics.NewRegisteredGauge("les/server/totalCapacity", nil) totalCapacityGauge = metrics.NewRegisteredGauge("les/server/totalCapacity", nil)
totalRechargeGauge = metrics.NewRegisteredGauge("les/server/totalRecharge", nil) totalRechargeGauge = metrics.NewRegisteredGauge("les/server/totalRecharge", nil)
totalConnectedGauge = metrics.NewRegisteredGauge("les/server/totalConnected", nil) blockProcessingTimer = metrics.NewRegisteredTimer("les/server/blockProcessingTime", nil)
blockProcessingTimer = metrics.NewRegisteredTimer("les/server/blockProcessingTime", nil)
capacityQueryZeroMeter = metrics.NewRegisteredMeter("les/server/capQueryZero", nil)
capacityQueryNonZeroMeter = metrics.NewRegisteredMeter("les/server/capQueryNonZero", nil)
requestServedMeter = metrics.NewRegisteredMeter("les/server/req/avgServedTime", nil) requestServedMeter = metrics.NewRegisteredMeter("les/server/req/avgServedTime", nil)
requestServedTimer = metrics.NewRegisteredTimer("les/server/req/servedTime", nil) requestServedTimer = metrics.NewRegisteredTimer("les/server/req/servedTime", nil)
@ -100,12 +97,8 @@ var (
sqServedGauge = metrics.NewRegisteredGauge("les/server/servingQueue/served", nil) sqServedGauge = metrics.NewRegisteredGauge("les/server/servingQueue/served", nil)
sqQueuedGauge = metrics.NewRegisteredGauge("les/server/servingQueue/queued", nil) sqQueuedGauge = metrics.NewRegisteredGauge("les/server/servingQueue/queued", nil)
clientConnectedMeter = metrics.NewRegisteredMeter("les/server/clientEvent/connected", nil) clientFreezeMeter = metrics.NewRegisteredMeter("les/server/clientEvent/freeze", nil)
clientActivatedMeter = metrics.NewRegisteredMeter("les/server/clientEvent/activated", nil) clientErrorMeter = metrics.NewRegisteredMeter("les/server/clientEvent/error", nil)
clientDeactivatedMeter = metrics.NewRegisteredMeter("les/server/clientEvent/deactivated", nil)
clientDisconnectedMeter = metrics.NewRegisteredMeter("les/server/clientEvent/disconnected", nil)
clientFreezeMeter = metrics.NewRegisteredMeter("les/server/clientEvent/freeze", nil)
clientErrorMeter = metrics.NewRegisteredMeter("les/server/clientEvent/error", nil)
requestRTT = metrics.NewRegisteredTimer("les/client/req/rtt", nil) requestRTT = metrics.NewRegisteredTimer("les/client/req/rtt", nil)
requestSendDelay = metrics.NewRegisteredTimer("les/client/req/sendDelay", nil) requestSendDelay = metrics.NewRegisteredTimer("les/client/req/sendDelay", nil)

View File

@ -17,6 +17,7 @@
package les package les
import ( import (
"crypto/ecdsa"
"errors" "errors"
"fmt" "fmt"
"math/big" "math/big"
@ -37,6 +38,7 @@ import (
vfs "github.com/ethereum/go-ethereum/les/vflux/server" vfs "github.com/ethereum/go-ethereum/les/vflux/server"
"github.com/ethereum/go-ethereum/light" "github.com/ethereum/go-ethereum/light"
"github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
) )
@ -762,15 +764,22 @@ type clientPeer struct {
responseLock sync.Mutex responseLock sync.Mutex
responseCount uint64 // Counter to generate an unique id for request processing. responseCount uint64 // Counter to generate an unique id for request processing.
balance *vfs.NodeBalance balance vfs.ConnectedBalance
// invalidLock is used for protecting invalidCount. // invalidLock is used for protecting invalidCount.
invalidLock sync.RWMutex invalidLock sync.RWMutex
invalidCount utils.LinearExpiredValue // Counter the invalid request the client peer has made. invalidCount utils.LinearExpiredValue // Counter the invalid request the client peer has made.
server bool capacity uint64
errCh chan error // lastAnnounce is the last broadcast created by the server; may be newer than the last head
fcClient *flowcontrol.ClientNode // Server side mirror token bucket. // sent to the specific client (stored in headInfo) if capacity is zero. In this case the
// latest head is sent when the client gains non-zero capacity.
lastAnnounce announceData
connectedAt mclock.AbsTime
server bool
errCh chan error
fcClient *flowcontrol.ClientNode // Server side mirror token bucket.
} }
func newClientPeer(version int, network uint64, p *p2p.Peer, rw p2p.MsgReadWriter) *clientPeer { func newClientPeer(version int, network uint64, p *p2p.Peer, rw p2p.MsgReadWriter) *clientPeer {
@ -789,9 +798,9 @@ func newClientPeer(version int, network uint64, p *p2p.Peer, rw p2p.MsgReadWrite
} }
} }
// freeClientId returns a string identifier for the peer. Multiple peers with // FreeClientId returns a string identifier for the peer. Multiple peers with
// the same identifier can not be connected in free mode simultaneously. // the same identifier can not be connected in free mode simultaneously.
func (p *clientPeer) freeClientId() string { func (p *clientPeer) FreeClientId() string {
if addr, ok := p.RemoteAddr().(*net.TCPAddr); ok { if addr, ok := p.RemoteAddr().(*net.TCPAddr); ok {
if addr.IP.IsLoopback() { if addr.IP.IsLoopback() {
// using peer id instead of loopback ip address allows multiple free // using peer id instead of loopback ip address allows multiple free
@ -921,25 +930,69 @@ func (p *clientPeer) sendAnnounce(request announceData) error {
return p2p.Send(p.rw, AnnounceMsg, request) return p2p.Send(p.rw, AnnounceMsg, request)
} }
// allowInactive implements clientPoolPeer // InactiveAllowance implements vfs.clientPeer
func (p *clientPeer) allowInactive() bool { func (p *clientPeer) InactiveAllowance() time.Duration {
return false return 0 // will return more than zero for les/5 clients
} }
// updateCapacity updates the request serving capacity assigned to a given client // getCapacity returns the current capacity of the peer
// and also sends an announcement about the updated flow control parameters func (p *clientPeer) getCapacity() uint64 {
func (p *clientPeer) updateCapacity(cap uint64) { p.lock.RLock()
defer p.lock.RUnlock()
return p.capacity
}
// UpdateCapacity updates the request serving capacity assigned to a given client
// and also sends an announcement about the updated flow control parameters.
// Note: UpdateCapacity implements vfs.clientPeer and should not block. The requested
// parameter is true if the callback was initiated by ClientPool.SetCapacity on the given peer.
func (p *clientPeer) UpdateCapacity(newCap uint64, requested bool) {
p.lock.Lock() p.lock.Lock()
defer p.lock.Unlock() defer p.lock.Unlock()
if cap != p.fcParams.MinRecharge { if newCap != p.fcParams.MinRecharge {
p.fcParams = flowcontrol.ServerParams{MinRecharge: cap, BufLimit: cap * bufLimitRatio} p.fcParams = flowcontrol.ServerParams{MinRecharge: newCap, BufLimit: newCap * bufLimitRatio}
p.fcClient.UpdateParams(p.fcParams) p.fcClient.UpdateParams(p.fcParams)
var kvList keyValueList var kvList keyValueList
kvList = kvList.add("flowControl/MRR", cap) kvList = kvList.add("flowControl/MRR", newCap)
kvList = kvList.add("flowControl/BL", cap*bufLimitRatio) kvList = kvList.add("flowControl/BL", newCap*bufLimitRatio)
p.queueSend(func() { p.sendAnnounce(announceData{Update: kvList}) }) p.queueSend(func() { p.sendAnnounce(announceData{Update: kvList}) })
} }
if p.capacity == 0 && newCap != 0 {
p.sendLastAnnounce()
}
p.capacity = newCap
}
// announceOrStore sends the given head announcement to the client if the client is
// active (capacity != 0) and the same announcement hasn't been sent before. If the
// client is inactive the announcement is stored and sent later if the client is
// activated again.
func (p *clientPeer) announceOrStore(announce announceData) {
p.lock.Lock()
defer p.lock.Unlock()
p.lastAnnounce = announce
if p.capacity != 0 {
p.sendLastAnnounce()
}
}
// announce sends the given head announcement to the client if it hasn't been sent before
func (p *clientPeer) sendLastAnnounce() {
if p.lastAnnounce.Td == nil {
return
}
if p.headInfo.Td == nil || p.lastAnnounce.Td.Cmp(p.headInfo.Td) > 0 {
if !p.queueSend(func() { p.sendAnnounce(p.lastAnnounce) }) {
p.Log().Debug("Dropped announcement because queue is full", "number", p.lastAnnounce.Number, "hash", p.lastAnnounce.Hash)
} else {
p.Log().Debug("Sent announcement", "number", p.lastAnnounce.Number, "hash", p.lastAnnounce.Hash)
}
p.headInfo = blockInfo{Hash: p.lastAnnounce.Hash, Number: p.lastAnnounce.Number, Td: p.lastAnnounce.Td}
}
} }
// freezeClient temporarily puts the client in a frozen state which means all // freezeClient temporarily puts the client in a frozen state which means all
@ -1064,6 +1117,11 @@ func (p *clientPeer) getInvalid() uint64 {
return p.invalidCount.Value(mclock.Now()) return p.invalidCount.Value(mclock.Now())
} }
// Disconnect implements vfs.clientPeer
func (p *clientPeer) Disconnect() {
p.Peer.Disconnect(p2p.DiscRequested)
}
// serverPeerSubscriber is an interface to notify services about added or // serverPeerSubscriber is an interface to notify services about added or
// removed server peers // removed server peers
type serverPeerSubscriber interface { type serverPeerSubscriber interface {
@ -1221,3 +1279,181 @@ func (ps *serverPeerSet) close() {
} }
ps.closed = true ps.closed = true
} }
// clientPeerSet represents the set of active client peers currently
// participating in the Light Ethereum sub-protocol.
type clientPeerSet struct {
peers map[enode.ID]*clientPeer
lock sync.RWMutex
closed bool
privateKey *ecdsa.PrivateKey
lastAnnounce, signedAnnounce announceData
}
// newClientPeerSet creates a new peer set to track the client peers.
func newClientPeerSet() *clientPeerSet {
return &clientPeerSet{peers: make(map[enode.ID]*clientPeer)}
}
// register adds a new peer into the peer set, or returns an error if the
// peer is already known.
func (ps *clientPeerSet) register(peer *clientPeer) error {
ps.lock.Lock()
defer ps.lock.Unlock()
if ps.closed {
return errClosed
}
if _, exist := ps.peers[peer.ID()]; exist {
return errAlreadyRegistered
}
ps.peers[peer.ID()] = peer
ps.announceOrStore(peer)
return nil
}
// unregister removes a remote peer from the peer set, disabling any further
// actions to/from that particular entity. It also initiates disconnection
// at the networking layer.
func (ps *clientPeerSet) unregister(id enode.ID) error {
ps.lock.Lock()
defer ps.lock.Unlock()
p, ok := ps.peers[id]
if !ok {
return errNotRegistered
}
delete(ps.peers, id)
p.Peer.Disconnect(p2p.DiscRequested)
return nil
}
// ids returns a list of all registered peer IDs
func (ps *clientPeerSet) ids() []enode.ID {
ps.lock.RLock()
defer ps.lock.RUnlock()
var ids []enode.ID
for id := range ps.peers {
ids = append(ids, id)
}
return ids
}
// peer retrieves the registered peer with the given id.
func (ps *clientPeerSet) peer(id enode.ID) *clientPeer {
ps.lock.RLock()
defer ps.lock.RUnlock()
return ps.peers[id]
}
// len returns if the current number of peers in the set.
func (ps *clientPeerSet) len() int {
ps.lock.RLock()
defer ps.lock.RUnlock()
return len(ps.peers)
}
// setSignerKey sets the signer key for signed announcements. Should be called before
// starting the protocol handler.
func (ps *clientPeerSet) setSignerKey(privateKey *ecdsa.PrivateKey) {
ps.privateKey = privateKey
}
// broadcast sends the given announcements to all active peers
func (ps *clientPeerSet) broadcast(announce announceData) {
ps.lock.Lock()
defer ps.lock.Unlock()
ps.lastAnnounce = announce
for _, peer := range ps.peers {
ps.announceOrStore(peer)
}
}
// announceOrStore sends the requested type of announcement to the given peer or stores
// it for later if the peer is inactive (capacity == 0).
func (ps *clientPeerSet) announceOrStore(p *clientPeer) {
if ps.lastAnnounce.Td == nil {
return
}
switch p.announceType {
case announceTypeSimple:
p.announceOrStore(ps.lastAnnounce)
case announceTypeSigned:
if ps.signedAnnounce.Hash != ps.lastAnnounce.Hash {
ps.signedAnnounce = ps.lastAnnounce
ps.signedAnnounce.sign(ps.privateKey)
}
p.announceOrStore(ps.signedAnnounce)
}
}
// close disconnects all peers. No new peers can be registered
// after close has returned.
func (ps *clientPeerSet) close() {
ps.lock.Lock()
defer ps.lock.Unlock()
for _, p := range ps.peers {
p.Peer.Disconnect(p2p.DiscQuitting)
}
ps.closed = true
}
// serverSet is a special set which contains all connected les servers.
// Les servers will also be discovered by discovery protocol because they
// also run the LES protocol. We can't drop them although they are useless
// for us(server) but for other protocols(e.g. ETH) upon the devp2p they
// may be useful.
type serverSet struct {
lock sync.Mutex
set map[string]*clientPeer
closed bool
}
func newServerSet() *serverSet {
return &serverSet{set: make(map[string]*clientPeer)}
}
func (s *serverSet) register(peer *clientPeer) error {
s.lock.Lock()
defer s.lock.Unlock()
if s.closed {
return errClosed
}
if _, exist := s.set[peer.id]; exist {
return errAlreadyRegistered
}
s.set[peer.id] = peer
return nil
}
func (s *serverSet) unregister(peer *clientPeer) error {
s.lock.Lock()
defer s.lock.Unlock()
if s.closed {
return errClosed
}
if _, exist := s.set[peer.id]; !exist {
return errNotRegistered
}
delete(s.set, peer.id)
peer.Peer.Disconnect(p2p.DiscQuitting)
return nil
}
func (s *serverSet) close() {
s.lock.Lock()
defer s.lock.Unlock()
for _, p := range s.set {
p.Peer.Disconnect(p2p.DiscQuitting)
}
s.closed = true
}

View File

@ -18,7 +18,6 @@ package les
import ( import (
"crypto/ecdsa" "crypto/ecdsa"
"reflect"
"time" "time"
"github.com/ethereum/go-ethereum/common/mclock" "github.com/ethereum/go-ethereum/common/mclock"
@ -26,7 +25,6 @@ import (
"github.com/ethereum/go-ethereum/eth/ethconfig" "github.com/ethereum/go-ethereum/eth/ethconfig"
"github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/les/flowcontrol" "github.com/ethereum/go-ethereum/les/flowcontrol"
"github.com/ethereum/go-ethereum/les/vflux"
vfs "github.com/ethereum/go-ethereum/les/vflux/server" vfs "github.com/ethereum/go-ethereum/les/vflux/server"
"github.com/ethereum/go-ethereum/light" "github.com/ethereum/go-ethereum/light"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
@ -34,24 +32,16 @@ import (
"github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr" "github.com/ethereum/go-ethereum/p2p/enr"
"github.com/ethereum/go-ethereum/p2p/nodestate"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/rpc"
) )
var ( var (
serverSetup = &nodestate.Setup{} defaultPosFactors = vfs.PriceFactors{TimeFactor: 0, CapacityFactor: 1, RequestFactor: 1}
clientPeerField = serverSetup.NewField("clientPeer", reflect.TypeOf(&clientPeer{})) defaultNegFactors = vfs.PriceFactors{TimeFactor: 0, CapacityFactor: 1, RequestFactor: 1}
clientInfoField = serverSetup.NewField("clientInfo", reflect.TypeOf(&clientInfo{}))
connAddressField = serverSetup.NewField("connAddr", reflect.TypeOf(""))
balanceTrackerSetup = vfs.NewBalanceTrackerSetup(serverSetup)
priorityPoolSetup = vfs.NewPriorityPoolSetup(serverSetup)
) )
func init() { const defaultConnectedBias = time.Minute * 3
balanceTrackerSetup.Connect(connAddressField, priorityPoolSetup.CapacityField)
priorityPoolSetup.Connect(balanceTrackerSetup.BalanceField, balanceTrackerSetup.UpdateFlag) // NodeBalance implements nodePriority
}
type ethBackend interface { type ethBackend interface {
ArchiveMode() bool ArchiveMode() bool
@ -65,10 +55,10 @@ type ethBackend interface {
type LesServer struct { type LesServer struct {
lesCommons lesCommons
ns *nodestate.NodeStateMachine
archiveMode bool // Flag whether the ethereum node runs in archive mode. archiveMode bool // Flag whether the ethereum node runs in archive mode.
handler *serverHandler handler *serverHandler
broadcaster *broadcaster peers *clientPeerSet
serverset *serverSet
vfluxServer *vfs.Server vfluxServer *vfs.Server
privateKey *ecdsa.PrivateKey privateKey *ecdsa.PrivateKey
@ -77,7 +67,7 @@ type LesServer struct {
costTracker *costTracker costTracker *costTracker
defParams flowcontrol.ServerParams defParams flowcontrol.ServerParams
servingQueue *servingQueue servingQueue *servingQueue
clientPool *clientPool clientPool *vfs.ClientPool
minCapacity, maxCapacity uint64 minCapacity, maxCapacity uint64
threadsIdle int // Request serving threads count when system is idle. threadsIdle int // Request serving threads count when system is idle.
@ -91,7 +81,6 @@ func NewLesServer(node *node.Node, e ethBackend, config *ethconfig.Config) (*Les
if err != nil { if err != nil {
return nil, err return nil, err
} }
ns := nodestate.NewNodeStateMachine(nil, nil, mclock.System{}, serverSetup)
// Calculate the number of threads used to service the light client // Calculate the number of threads used to service the light client
// requests based on the user-specified value. // requests based on the user-specified value.
threads := config.LightServ * 4 / 100 threads := config.LightServ * 4 / 100
@ -111,9 +100,9 @@ func NewLesServer(node *node.Node, e ethBackend, config *ethconfig.Config) (*Les
bloomTrieIndexer: light.NewBloomTrieIndexer(e.ChainDb(), nil, params.BloomBitsBlocks, params.BloomTrieFrequency, true), bloomTrieIndexer: light.NewBloomTrieIndexer(e.ChainDb(), nil, params.BloomBitsBlocks, params.BloomTrieFrequency, true),
closeCh: make(chan struct{}), closeCh: make(chan struct{}),
}, },
ns: ns,
archiveMode: e.ArchiveMode(), archiveMode: e.ArchiveMode(),
broadcaster: newBroadcaster(ns), peers: newClientPeerSet(),
serverset: newServerSet(),
vfluxServer: vfs.NewServer(time.Millisecond * 10), vfluxServer: vfs.NewServer(time.Millisecond * 10),
fcManager: flowcontrol.NewClientManager(nil, &mclock.System{}), fcManager: flowcontrol.NewClientManager(nil, &mclock.System{}),
servingQueue: newServingQueue(int64(time.Millisecond*10), float64(config.LightServ)/100), servingQueue: newServingQueue(int64(time.Millisecond*10), float64(config.LightServ)/100),
@ -121,7 +110,6 @@ func NewLesServer(node *node.Node, e ethBackend, config *ethconfig.Config) (*Les
threadsIdle: threads, threadsIdle: threads,
p2pSrv: node.Server(), p2pSrv: node.Server(),
} }
srv.vfluxServer.Register(srv)
issync := e.Synced issync := e.Synced
if config.LightNoSyncServe { if config.LightNoSyncServe {
issync = func() bool { return true } issync = func() bool { return true }
@ -149,8 +137,10 @@ func NewLesServer(node *node.Node, e ethBackend, config *ethconfig.Config) (*Les
srv.maxCapacity = totalRecharge srv.maxCapacity = totalRecharge
} }
srv.fcManager.SetCapacityLimits(srv.minCapacity, srv.maxCapacity, srv.minCapacity*2) srv.fcManager.SetCapacityLimits(srv.minCapacity, srv.maxCapacity, srv.minCapacity*2)
srv.clientPool = newClientPool(ns, lesDb, srv.minCapacity, defaultConnectedBias, mclock.System{}, srv.dropClient, issync) srv.clientPool = vfs.NewClientPool(lesDb, srv.minCapacity, defaultConnectedBias, mclock.System{}, issync)
srv.clientPool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 0, CapacityFactor: 1, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 0, CapacityFactor: 1, RequestFactor: 1}) srv.clientPool.Start()
srv.clientPool.SetDefaultFactors(defaultPosFactors, defaultNegFactors)
srv.vfluxServer.Register(srv.clientPool, "les", "Ethereum light client service")
checkpoint := srv.latestLocalCheckpoint() checkpoint := srv.latestLocalCheckpoint()
if !checkpoint.Empty() { if !checkpoint.Empty() {
@ -162,14 +152,6 @@ func NewLesServer(node *node.Node, e ethBackend, config *ethconfig.Config) (*Les
node.RegisterProtocols(srv.Protocols()) node.RegisterProtocols(srv.Protocols())
node.RegisterAPIs(srv.APIs()) node.RegisterAPIs(srv.APIs())
node.RegisterLifecycle(srv) node.RegisterLifecycle(srv)
// disconnect all peers at nsm shutdown
ns.SubscribeField(clientPeerField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) {
if state.Equals(serverSetup.OfflineFlag()) && oldValue != nil {
oldValue.(*clientPeer).Peer.Disconnect(p2p.DiscRequested)
}
})
ns.Start()
return srv, nil return srv, nil
} }
@ -198,7 +180,7 @@ func (s *LesServer) APIs() []rpc.API {
func (s *LesServer) Protocols() []p2p.Protocol { func (s *LesServer) Protocols() []p2p.Protocol {
ps := s.makeProtocols(ServerProtocolVersions, s.handler.runPeer, func(id enode.ID) interface{} { ps := s.makeProtocols(ServerProtocolVersions, s.handler.runPeer, func(id enode.ID) interface{} {
if p := s.getClient(id); p != nil { if p := s.peers.peer(id); p != nil {
return p.Info() return p.Info()
} }
return nil return nil
@ -215,7 +197,7 @@ func (s *LesServer) Protocols() []p2p.Protocol {
// Start starts the LES server // Start starts the LES server
func (s *LesServer) Start() error { func (s *LesServer) Start() error {
s.privateKey = s.p2pSrv.PrivateKey s.privateKey = s.p2pSrv.PrivateKey
s.broadcaster.setSignerKey(s.privateKey) s.peers.setSignerKey(s.privateKey)
s.handler.start() s.handler.start()
s.wg.Add(1) s.wg.Add(1)
go s.capacityManagement() go s.capacityManagement()
@ -229,8 +211,9 @@ func (s *LesServer) Start() error {
func (s *LesServer) Stop() error { func (s *LesServer) Stop() error {
close(s.closeCh) close(s.closeCh)
s.clientPool.stop() s.clientPool.Stop()
s.ns.Stop() s.serverset.close()
s.peers.close()
s.fcManager.Stop() s.fcManager.Stop()
s.costTracker.stop() s.costTracker.stop()
s.handler.stop() s.handler.stop()
@ -261,7 +244,7 @@ func (s *LesServer) capacityManagement() {
totalCapacityCh := make(chan uint64, 100) totalCapacityCh := make(chan uint64, 100)
totalCapacity := s.fcManager.SubscribeTotalCapacity(totalCapacityCh) totalCapacity := s.fcManager.SubscribeTotalCapacity(totalCapacityCh)
s.clientPool.setLimits(s.config.LightPeers, totalCapacity) s.clientPool.SetLimits(uint64(s.config.LightPeers), totalCapacity)
var ( var (
busy bool busy bool
@ -298,39 +281,9 @@ func (s *LesServer) capacityManagement() {
log.Warn("Reduced free peer connections", "from", freePeers, "to", newFreePeers) log.Warn("Reduced free peer connections", "from", freePeers, "to", newFreePeers)
} }
freePeers = newFreePeers freePeers = newFreePeers
s.clientPool.setLimits(s.config.LightPeers, totalCapacity) s.clientPool.SetLimits(uint64(s.config.LightPeers), totalCapacity)
case <-s.closeCh: case <-s.closeCh:
return return
} }
} }
} }
func (s *LesServer) getClient(id enode.ID) *clientPeer {
if node := s.ns.GetNode(id); node != nil {
if p, ok := s.ns.GetField(node, clientPeerField).(*clientPeer); ok {
return p
}
}
return nil
}
func (s *LesServer) dropClient(id enode.ID) {
if p := s.getClient(id); p != nil {
p.Peer.Disconnect(p2p.DiscRequested)
}
}
// ServiceInfo implements vfs.Service
func (s *LesServer) ServiceInfo() (string, string) {
return "les", "Ethereum light client service"
}
// Handle implements vfs.Service
func (s *LesServer) Handle(id enode.ID, address string, name string, data []byte) []byte {
switch name {
case vflux.CapacityQueryName:
return s.clientPool.serveCapQuery(id, address, data)
default:
return nil
}
}

View File

@ -17,7 +17,6 @@
package les package les
import ( import (
"crypto/ecdsa"
"errors" "errors"
"sync" "sync"
"sync/atomic" "sync/atomic"
@ -31,13 +30,10 @@ import (
"github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/ethdb"
vfs "github.com/ethereum/go-ethereum/les/vflux/server"
"github.com/ethereum/go-ethereum/light" "github.com/ethereum/go-ethereum/light"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/nodestate"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
) )
@ -59,7 +55,6 @@ const (
var ( var (
errTooManyInvalidRequest = errors.New("too many invalid requests made") errTooManyInvalidRequest = errors.New("too many invalid requests made")
errFullClientPool = errors.New("client pool is full")
) )
// serverHandler is responsible for serving light client and process // serverHandler is responsible for serving light client and process
@ -128,32 +123,18 @@ func (h *serverHandler) handle(p *clientPeer) error {
p.Log().Debug("Light Ethereum handshake failed", "err", err) p.Log().Debug("Light Ethereum handshake failed", "err", err)
return err return err
} }
// Reject the duplicated peer, otherwise register it to peerset.
var registered bool
if err := h.server.ns.Operation(func() {
if h.server.ns.GetField(p.Node(), clientPeerField) != nil {
registered = true
} else {
h.server.ns.SetFieldSub(p.Node(), clientPeerField, p)
}
}); err != nil {
return err
}
if registered {
return errAlreadyRegistered
}
defer func() {
h.server.ns.SetField(p.Node(), clientPeerField, nil)
if p.fcClient != nil { // is nil when connecting another server
p.fcClient.Disconnect()
}
}()
if p.server { if p.server {
if err := h.server.serverset.register(p); err != nil {
return err
}
// connected to another server, no messages expected, just wait for disconnection // connected to another server, no messages expected, just wait for disconnection
_, err := p.rw.ReadMsg() _, err := p.rw.ReadMsg()
h.server.serverset.unregister(p)
return err return err
} }
defer p.fcClient.Disconnect() // set by handshake if it's not another server
// Reject light clients if server is not synced. // Reject light clients if server is not synced.
// //
// Put this checking here, so that "non-synced" les-server peers are still allowed // Put this checking here, so that "non-synced" les-server peers are still allowed
@ -162,30 +143,31 @@ func (h *serverHandler) handle(p *clientPeer) error {
p.Log().Debug("Light server not synced, rejecting peer") p.Log().Debug("Light server not synced, rejecting peer")
return p2p.DiscRequested return p2p.DiscRequested
} }
// Disconnect the inbound peer if it's rejected by clientPool if err := h.server.peers.register(p); err != nil {
if cap, err := h.server.clientPool.connect(p); cap != p.fcParams.MinRecharge || err != nil { return err
p.Log().Debug("Light Ethereum peer rejected", "err", errFullClientPool)
return errFullClientPool
} }
p.balance, _ = h.server.ns.GetField(p.Node(), h.server.clientPool.BalanceField).(*vfs.NodeBalance) if p.balance = h.server.clientPool.Register(p); p.balance == nil {
if p.balance == nil { h.server.peers.unregister(p.ID())
p.Log().Debug("Client pool already closed")
return p2p.DiscRequested return p2p.DiscRequested
} }
activeCount, _ := h.server.clientPool.pp.Active() activeCount, _ := h.server.clientPool.Active()
clientConnectionGauge.Update(int64(activeCount)) clientConnectionGauge.Update(int64(activeCount))
p.connectedAt = mclock.Now()
var wg sync.WaitGroup // Wait group used to track all in-flight task routines. var wg sync.WaitGroup // Wait group used to track all in-flight task routines.
connectedAt := mclock.Now()
defer func() { defer func() {
wg.Wait() // Ensure all background task routines have exited. wg.Wait() // Ensure all background task routines have exited.
h.server.clientPool.disconnect(p) h.server.clientPool.Unregister(p)
h.server.peers.unregister(p.ID())
p.balance = nil p.balance = nil
activeCount, _ := h.server.clientPool.pp.Active() activeCount, _ := h.server.clientPool.Active()
clientConnectionGauge.Update(int64(activeCount)) clientConnectionGauge.Update(int64(activeCount))
connectionTimer.Update(time.Duration(mclock.Now() - connectedAt)) connectionTimer.Update(time.Duration(mclock.Now() - p.connectedAt))
}() }()
// Mark the peer starts to be served.
// Mark the peer as being served.
atomic.StoreUint32(&p.serving, 1) atomic.StoreUint32(&p.serving, 1)
defer atomic.StoreUint32(&p.serving, 0) defer atomic.StoreUint32(&p.serving, 0)
@ -448,78 +430,9 @@ func (h *serverHandler) broadcastLoop() {
} }
lastHead, lastTd = header, td lastHead, lastTd = header, td
log.Debug("Announcing block to peers", "number", number, "hash", hash, "td", td, "reorg", reorg) log.Debug("Announcing block to peers", "number", number, "hash", hash, "td", td, "reorg", reorg)
h.server.broadcaster.broadcast(announceData{Hash: hash, Number: number, Td: td, ReorgDepth: reorg}) h.server.peers.broadcast(announceData{Hash: hash, Number: number, Td: td, ReorgDepth: reorg})
case <-h.closeCh: case <-h.closeCh:
return return
} }
} }
} }
// broadcaster sends new header announcements to active client peers
type broadcaster struct {
ns *nodestate.NodeStateMachine
privateKey *ecdsa.PrivateKey
lastAnnounce, signedAnnounce announceData
}
// newBroadcaster creates a new broadcaster
func newBroadcaster(ns *nodestate.NodeStateMachine) *broadcaster {
b := &broadcaster{ns: ns}
ns.SubscribeState(priorityPoolSetup.ActiveFlag, func(node *enode.Node, oldState, newState nodestate.Flags) {
if newState.Equals(priorityPoolSetup.ActiveFlag) {
// send last announcement to activated peers
b.sendTo(node)
}
})
return b
}
// setSignerKey sets the signer key for signed announcements. Should be called before
// starting the protocol handler.
func (b *broadcaster) setSignerKey(privateKey *ecdsa.PrivateKey) {
b.privateKey = privateKey
}
// broadcast sends the given announcements to all active peers
func (b *broadcaster) broadcast(announce announceData) {
b.ns.Operation(func() {
// iterate in an Operation to ensure that the active set does not change while iterating
b.lastAnnounce = announce
b.ns.ForEach(priorityPoolSetup.ActiveFlag, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) {
b.sendTo(node)
})
})
}
// sendTo sends the most recent announcement to the given node unless the same or higher Td
// announcement has already been sent.
func (b *broadcaster) sendTo(node *enode.Node) {
if b.lastAnnounce.Td == nil {
return
}
if p, _ := b.ns.GetField(node, clientPeerField).(*clientPeer); p != nil {
if p.headInfo.Td == nil || b.lastAnnounce.Td.Cmp(p.headInfo.Td) > 0 {
announce := b.lastAnnounce
switch p.announceType {
case announceTypeSimple:
if !p.queueSend(func() { p.sendAnnounce(announce) }) {
log.Debug("Drop announcement because queue is full", "number", announce.Number, "hash", announce.Hash)
} else {
log.Debug("Sent announcement", "number", announce.Number, "hash", announce.Hash)
}
case announceTypeSigned:
if b.signedAnnounce.Hash != b.lastAnnounce.Hash {
b.signedAnnounce = b.lastAnnounce
b.signedAnnounce.sign(b.privateKey)
}
announce := b.signedAnnounce
if !p.queueSend(func() { p.sendAnnounce(announce) }) {
log.Debug("Drop announcement because queue is full", "number", announce.Number, "hash", announce.Hash)
} else {
log.Debug("Sent announcement", "number", announce.Number, "hash", announce.Hash)
}
}
p.headInfo = blockInfo{b.lastAnnounce.Hash, b.lastAnnounce.Number, b.lastAnnounce.Td}
}
}
}

View File

@ -123,7 +123,7 @@ func (t *servingTask) waitOrStop() bool {
// newServingQueue returns a new servingQueue // newServingQueue returns a new servingQueue
func newServingQueue(suspendBias int64, utilTarget float64) *servingQueue { func newServingQueue(suspendBias int64, utilTarget float64) *servingQueue {
sq := &servingQueue{ sq := &servingQueue{
queue: prque.New(nil), queue: prque.NewWrapAround(nil),
suspendBias: suspendBias, suspendBias: suspendBias,
queueAddCh: make(chan *servingTask, 100), queueAddCh: make(chan *servingTask, 100),
queueBestCh: make(chan *servingTask), queueBestCh: make(chan *servingTask),
@ -279,7 +279,7 @@ func (sq *servingQueue) updateRecentTime() {
func (sq *servingQueue) addTask(task *servingTask) { func (sq *servingQueue) addTask(task *servingTask) {
if sq.best == nil { if sq.best == nil {
sq.best = task sq.best = task
} else if task.priority > sq.best.priority { } else if task.priority-sq.best.priority > 0 {
sq.queue.Push(sq.best, sq.best.priority) sq.queue.Push(sq.best, sq.best.priority)
sq.best = task sq.best = task
} else { } else {

View File

@ -45,10 +45,10 @@ import (
"github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/les/checkpointoracle" "github.com/ethereum/go-ethereum/les/checkpointoracle"
"github.com/ethereum/go-ethereum/les/flowcontrol" "github.com/ethereum/go-ethereum/les/flowcontrol"
vfs "github.com/ethereum/go-ethereum/les/vflux/server"
"github.com/ethereum/go-ethereum/light" "github.com/ethereum/go-ethereum/light"
"github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/nodestate"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
) )
@ -284,7 +284,6 @@ func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Da
} }
oracle = checkpointoracle.New(checkpointConfig, getLocal) oracle = checkpointoracle.New(checkpointConfig, getLocal)
} }
ns := nodestate.NewNodeStateMachine(nil, nil, mclock.System{}, serverSetup)
server := &LesServer{ server := &LesServer{
lesCommons: lesCommons{ lesCommons: lesCommons{
genesis: genesis.Hash(), genesis: genesis.Hash(),
@ -296,8 +295,7 @@ func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Da
oracle: oracle, oracle: oracle,
closeCh: make(chan struct{}), closeCh: make(chan struct{}),
}, },
ns: ns, peers: newClientPeerSet(),
broadcaster: newBroadcaster(ns),
servingQueue: newServingQueue(int64(time.Millisecond*10), 1), servingQueue: newServingQueue(int64(time.Millisecond*10), 1),
defParams: flowcontrol.ServerParams{ defParams: flowcontrol.ServerParams{
BufLimit: testBufLimit, BufLimit: testBufLimit,
@ -307,14 +305,14 @@ func newTestServerHandler(blocks int, indexers []*core.ChainIndexer, db ethdb.Da
} }
server.costTracker, server.minCapacity = newCostTracker(db, server.config) server.costTracker, server.minCapacity = newCostTracker(db, server.config)
server.costTracker.testCostList = testCostList(0) // Disable flow control mechanism. server.costTracker.testCostList = testCostList(0) // Disable flow control mechanism.
server.clientPool = newClientPool(ns, db, testBufRecharge, defaultConnectedBias, clock, func(id enode.ID) {}, alwaysTrueFn) server.clientPool = vfs.NewClientPool(db, testBufRecharge, defaultConnectedBias, clock, alwaysTrueFn)
server.clientPool.setLimits(10000, 10000) // Assign enough capacity for clientpool server.clientPool.Start()
server.clientPool.SetLimits(10000, 10000) // Assign enough capacity for clientpool
server.handler = newServerHandler(server, simulation.Blockchain(), db, txpool, func() bool { return true }) server.handler = newServerHandler(server, simulation.Blockchain(), db, txpool, func() bool { return true })
if server.oracle != nil { if server.oracle != nil {
server.oracle.Start(simulation) server.oracle.Start(simulation)
} }
server.servingQueue.setThreads(4) server.servingQueue.setThreads(4)
ns.Start()
server.handler.start() server.handler.start()
return server.handler, simulation return server.handler, simulation
} }

View File

@ -47,21 +47,57 @@ type PriceFactors struct {
TimeFactor, CapacityFactor, RequestFactor float64 TimeFactor, CapacityFactor, RequestFactor float64
} }
// timePrice returns the price of connection per nanosecond at the given capacity // connectionPrice returns the price of connection per nanosecond at the given capacity
func (p PriceFactors) timePrice(cap uint64) float64 { // and the estimated average request cost.
return p.TimeFactor + float64(cap)*p.CapacityFactor/1000000 func (p PriceFactors) connectionPrice(cap uint64, avgReqCost float64) float64 {
return p.TimeFactor + float64(cap)*p.CapacityFactor/1000000 + p.RequestFactor*avgReqCost
} }
// NodeBalance keeps track of the positive and negative balances of a connected type (
// nodePriority interface provides current and estimated future priorities on demand
nodePriority interface {
// priority should return the current priority of the node (higher is better)
priority(cap uint64) int64
// estimatePriority should return a lower estimate for the minimum of the node priority
// value starting from the current moment until the given time. If the priority goes
// under the returned estimate before the specified moment then it is the caller's
// responsibility to signal with updateFlag.
estimatePriority(cap uint64, addBalance int64, future, bias time.Duration, update bool) int64
}
// ReadOnlyBalance provides read-only operations on the node balance
ReadOnlyBalance interface {
nodePriority
GetBalance() (uint64, uint64)
GetRawBalance() (utils.ExpiredValue, utils.ExpiredValue)
GetPriceFactors() (posFactor, negFactor PriceFactors)
}
// ConnectedBalance provides operations permitted on connected nodes (non-read-only
// operations are not permitted inside a BalanceOperation)
ConnectedBalance interface {
ReadOnlyBalance
SetPriceFactors(posFactor, negFactor PriceFactors)
RequestServed(cost uint64) uint64
}
// AtomicBalanceOperator provides operations permitted in an atomic BalanceOperation
AtomicBalanceOperator interface {
ReadOnlyBalance
AddBalance(amount int64) (uint64, uint64, error)
SetBalance(pos, neg uint64) error
}
)
// nodeBalance keeps track of the positive and negative balances of a connected
// client and calculates actual and projected future priority values. // client and calculates actual and projected future priority values.
// Implements nodePriority interface. // Implements nodePriority interface.
type NodeBalance struct { type nodeBalance struct {
bt *BalanceTracker bt *balanceTracker
lock sync.RWMutex lock sync.RWMutex
node *enode.Node node *enode.Node
connAddress string connAddress string
active bool active, hasPriority, setFlags bool
priority bool
capacity uint64 capacity uint64
balance balance balance balance
posFactor, negFactor PriceFactors posFactor, negFactor PriceFactors
@ -78,7 +114,62 @@ type NodeBalance struct {
// balance represents a pair of positive and negative balances // balance represents a pair of positive and negative balances
type balance struct { type balance struct {
pos, neg utils.ExpiredValue pos, neg utils.ExpiredValue
posExp, negExp utils.ValueExpirer
}
// posValue returns the value of positive balance at a given timestamp.
func (b balance) posValue(now mclock.AbsTime) uint64 {
return b.pos.Value(b.posExp.LogOffset(now))
}
// negValue returns the value of negative balance at a given timestamp.
func (b balance) negValue(now mclock.AbsTime) uint64 {
return b.neg.Value(b.negExp.LogOffset(now))
}
// addValue adds the value of a given amount to the balance. The original value and
// updated value will also be returned if the addition is successful.
// Returns the error if the given value is too large and the value overflows.
func (b *balance) addValue(now mclock.AbsTime, amount int64, pos bool, force bool) (uint64, uint64, int64, error) {
var (
val utils.ExpiredValue
offset utils.Fixed64
)
if pos {
offset, val = b.posExp.LogOffset(now), b.pos
} else {
offset, val = b.negExp.LogOffset(now), b.neg
}
old := val.Value(offset)
if amount > 0 && (amount > maxBalance || old > maxBalance-uint64(amount)) {
if !force {
return old, 0, 0, errBalanceOverflow
}
val = utils.ExpiredValue{}
amount = maxBalance
}
net := val.Add(amount, offset)
if pos {
b.pos = val
} else {
b.neg = val
}
return old, val.Value(offset), net, nil
}
// setValue sets the internal balance amount to the given values. Returns the
// error if the given value is too large.
func (b *balance) setValue(now mclock.AbsTime, pos uint64, neg uint64) error {
if pos > maxBalance || neg > maxBalance {
return errBalanceOverflow
}
var pb, nb utils.ExpiredValue
pb.Add(int64(pos), b.posExp.LogOffset(now))
nb.Add(int64(neg), b.negExp.LogOffset(now))
b.pos = pb
b.neg = nb
return nil
} }
// balanceCallback represents a single callback that is activated when client priority // balanceCallback represents a single callback that is activated when client priority
@ -90,18 +181,18 @@ type balanceCallback struct {
} }
// GetBalance returns the current positive and negative balance. // GetBalance returns the current positive and negative balance.
func (n *NodeBalance) GetBalance() (uint64, uint64) { func (n *nodeBalance) GetBalance() (uint64, uint64) {
n.lock.Lock() n.lock.Lock()
defer n.lock.Unlock() defer n.lock.Unlock()
now := n.bt.clock.Now() now := n.bt.clock.Now()
n.updateBalance(now) n.updateBalance(now)
return n.balance.pos.Value(n.bt.posExp.LogOffset(now)), n.balance.neg.Value(n.bt.negExp.LogOffset(now)) return n.balance.posValue(now), n.balance.negValue(now)
} }
// GetRawBalance returns the current positive and negative balance // GetRawBalance returns the current positive and negative balance
// but in the raw(expired value) format. // but in the raw(expired value) format.
func (n *NodeBalance) GetRawBalance() (utils.ExpiredValue, utils.ExpiredValue) { func (n *nodeBalance) GetRawBalance() (utils.ExpiredValue, utils.ExpiredValue) {
n.lock.Lock() n.lock.Lock()
defer n.lock.Unlock() defer n.lock.Unlock()
@ -114,164 +205,147 @@ func (n *NodeBalance) GetRawBalance() (utils.ExpiredValue, utils.ExpiredValue) {
// before and after the operation. Exceeding maxBalance results in an error (balance is // before and after the operation. Exceeding maxBalance results in an error (balance is
// unchanged) while adding a negative amount higher than the current balance results in // unchanged) while adding a negative amount higher than the current balance results in
// zero balance. // zero balance.
func (n *NodeBalance) AddBalance(amount int64) (uint64, uint64, error) { // Note: this function should run inside a NodeStateMachine operation
func (n *nodeBalance) AddBalance(amount int64) (uint64, uint64, error) {
var ( var (
err error err error
old, new uint64 old, new uint64
now = n.bt.clock.Now()
callbacks []func()
setPriority bool
) )
n.bt.ns.Operation(func() { // Operation with holding the lock
var ( n.bt.updateTotalBalance(n, func() bool {
callbacks []func() n.updateBalance(now)
setPriority bool if old, new, _, err = n.balance.addValue(now, amount, true, false); err != nil {
) return false
n.bt.updateTotalBalance(n, func() bool {
now := n.bt.clock.Now()
n.updateBalance(now)
// Ensure the given amount is valid to apply.
offset := n.bt.posExp.LogOffset(now)
old = n.balance.pos.Value(offset)
if amount > 0 && (amount > maxBalance || old > maxBalance-uint64(amount)) {
err = errBalanceOverflow
return false
}
// Update the total positive balance counter.
n.balance.pos.Add(amount, offset)
callbacks = n.checkCallbacks(now)
setPriority = n.checkPriorityStatus()
new = n.balance.pos.Value(offset)
n.storeBalance(true, false)
return true
})
for _, cb := range callbacks {
cb()
} }
if setPriority { callbacks, setPriority = n.checkCallbacks(now), n.checkPriorityStatus()
n.bt.ns.SetStateSub(n.node, n.bt.PriorityFlag, nodestate.Flags{}, 0) n.storeBalance(true, false)
} return true
n.signalPriorityUpdate()
}) })
if err != nil { if err != nil {
return old, old, err return old, old, err
} }
// Operation without holding the lock
for _, cb := range callbacks {
cb()
}
if n.setFlags {
if setPriority {
n.bt.ns.SetStateSub(n.node, n.bt.setup.priorityFlag, nodestate.Flags{}, 0)
}
// Note: priority flag is automatically removed by the zero priority callback if necessary
n.signalPriorityUpdate()
}
return old, new, nil return old, new, nil
} }
// SetBalance sets the positive and negative balance to the given values // SetBalance sets the positive and negative balance to the given values
func (n *NodeBalance) SetBalance(pos, neg uint64) error { // Note: this function should run inside a NodeStateMachine operation
if pos > maxBalance || neg > maxBalance { func (n *nodeBalance) SetBalance(pos, neg uint64) error {
return errBalanceOverflow var (
} now = n.bt.clock.Now()
n.bt.ns.Operation(func() { callbacks []func()
var ( setPriority bool
callbacks []func() )
setPriority bool // Operation with holding the lock
) n.bt.updateTotalBalance(n, func() bool {
n.bt.updateTotalBalance(n, func() bool { n.updateBalance(now)
now := n.bt.clock.Now() if err := n.balance.setValue(now, pos, neg); err != nil {
n.updateBalance(now) return false
var pb, nb utils.ExpiredValue
pb.Add(int64(pos), n.bt.posExp.LogOffset(now))
nb.Add(int64(neg), n.bt.negExp.LogOffset(now))
n.balance.pos = pb
n.balance.neg = nb
callbacks = n.checkCallbacks(now)
setPriority = n.checkPriorityStatus()
n.storeBalance(true, true)
return true
})
for _, cb := range callbacks {
cb()
} }
if setPriority { callbacks, setPriority = n.checkCallbacks(now), n.checkPriorityStatus()
n.bt.ns.SetStateSub(n.node, n.bt.PriorityFlag, nodestate.Flags{}, 0) n.storeBalance(true, true)
} return true
n.signalPriorityUpdate()
}) })
// Operation without holding the lock
for _, cb := range callbacks {
cb()
}
if n.setFlags {
if setPriority {
n.bt.ns.SetStateSub(n.node, n.bt.setup.priorityFlag, nodestate.Flags{}, 0)
}
// Note: priority flag is automatically removed by the zero priority callback if necessary
n.signalPriorityUpdate()
}
return nil return nil
} }
// RequestServed should be called after serving a request for the given peer // RequestServed should be called after serving a request for the given peer
func (n *NodeBalance) RequestServed(cost uint64) uint64 { func (n *nodeBalance) RequestServed(cost uint64) (newBalance uint64) {
n.lock.Lock() n.lock.Lock()
var callbacks []func()
defer func() {
n.lock.Unlock()
if callbacks != nil {
n.bt.ns.Operation(func() {
for _, cb := range callbacks {
cb()
}
})
}
}()
now := n.bt.clock.Now() var (
check bool
fcost = float64(cost)
now = n.bt.clock.Now()
)
n.updateBalance(now) n.updateBalance(now)
fcost := float64(cost)
posExp := n.bt.posExp.LogOffset(now)
var check bool
if !n.balance.pos.IsZero() { if !n.balance.pos.IsZero() {
if n.posFactor.RequestFactor != 0 { posCost := -int64(fcost * n.posFactor.RequestFactor)
c := -int64(fcost * n.posFactor.RequestFactor) if posCost == 0 {
cc := n.balance.pos.Add(c, posExp) fcost = 0
if c == cc { newBalance = n.balance.posValue(now)
} else {
var net int64
_, newBalance, net, _ = n.balance.addValue(now, posCost, true, false)
if posCost == net {
fcost = 0 fcost = 0
} else { } else {
fcost *= 1 - float64(cc)/float64(c) fcost *= 1 - float64(net)/float64(posCost)
} }
check = true check = true
} else {
fcost = 0
} }
} }
if fcost > 0 { if fcost > 0 && n.negFactor.RequestFactor != 0 {
if n.negFactor.RequestFactor != 0 { n.balance.addValue(now, int64(fcost*n.negFactor.RequestFactor), false, false)
n.balance.neg.Add(int64(fcost*n.negFactor.RequestFactor), n.bt.negExp.LogOffset(now)) check = true
check = true
}
} }
n.sumReqCost += cost
var callbacks []func()
if check { if check {
callbacks = n.checkCallbacks(now) callbacks = n.checkCallbacks(now)
} }
n.sumReqCost += cost n.lock.Unlock()
return n.balance.pos.Value(posExp)
if callbacks != nil {
n.bt.ns.Operation(func() {
for _, cb := range callbacks {
cb()
}
})
}
return
} }
// Priority returns the actual priority based on the current balance // priority returns the actual priority based on the current balance
func (n *NodeBalance) Priority(capacity uint64) int64 { func (n *nodeBalance) priority(capacity uint64) int64 {
n.lock.Lock() n.lock.Lock()
defer n.lock.Unlock() defer n.lock.Unlock()
n.updateBalance(n.bt.clock.Now()) now := n.bt.clock.Now()
return n.balanceToPriority(n.balance, capacity) n.updateBalance(now)
return n.balanceToPriority(now, n.balance, capacity)
} }
// EstMinPriority gives a lower estimate for the priority at a given time in the future. // EstMinPriority gives a lower estimate for the priority at a given time in the future.
// An average request cost per time is assumed that is twice the average cost per time // An average request cost per time is assumed that is twice the average cost per time
// in the current session. // in the current session.
// If update is true then a priority callback is added that turns UpdateFlag on and off // If update is true then a priority callback is added that turns updateFlag on and off
// in case the priority goes below the estimated minimum. // in case the priority goes below the estimated minimum.
func (n *NodeBalance) EstimatePriority(capacity uint64, addBalance int64, future, bias time.Duration, update bool) int64 { func (n *nodeBalance) estimatePriority(capacity uint64, addBalance int64, future, bias time.Duration, update bool) int64 {
n.lock.Lock() n.lock.Lock()
defer n.lock.Unlock() defer n.lock.Unlock()
now := n.bt.clock.Now() now := n.bt.clock.Now()
n.updateBalance(now) n.updateBalance(now)
b := n.balance
b := n.balance // copy the balance
if addBalance != 0 { if addBalance != 0 {
offset := n.bt.posExp.LogOffset(now) b.addValue(now, addBalance, true, true)
old := n.balance.pos.Value(offset)
if addBalance > 0 && (addBalance > maxBalance || old > maxBalance-uint64(addBalance)) {
b.pos = utils.ExpiredValue{}
b.pos.Add(maxBalance, offset)
} else {
b.pos.Add(addBalance, offset)
}
} }
if future > 0 { if future > 0 {
var avgReqCost float64 var avgReqCost float64
@ -284,52 +358,20 @@ func (n *NodeBalance) EstimatePriority(capacity uint64, addBalance int64, future
if bias > 0 { if bias > 0 {
b = n.reducedBalance(b, now+mclock.AbsTime(future), bias, capacity, 0) b = n.reducedBalance(b, now+mclock.AbsTime(future), bias, capacity, 0)
} }
pri := n.balanceToPriority(b, capacity) // Note: we subtract one from the estimated priority in order to ensure that biased
// estimates are always lower than actual priorities, even if the bias is very small.
// This ensures that two nodes will not ping-pong update signals forever if both of
// them have zero estimated priority drop in the projected future.
pri := n.balanceToPriority(now, b, capacity) - 1
if update { if update {
n.addCallback(balanceCallbackUpdate, pri, n.signalPriorityUpdate) n.addCallback(balanceCallbackUpdate, pri, n.signalPriorityUpdate)
} }
return pri return pri
} }
// PosBalanceMissing calculates the missing amount of positive balance in order to
// connect at targetCapacity, stay connected for the given amount of time and then
// still have a priority of targetPriority
func (n *NodeBalance) PosBalanceMissing(targetPriority int64, targetCapacity uint64, after time.Duration) uint64 {
n.lock.Lock()
defer n.lock.Unlock()
now := n.bt.clock.Now()
if targetPriority < 0 {
timePrice := n.negFactor.timePrice(targetCapacity)
timeCost := uint64(float64(after) * timePrice)
negBalance := n.balance.neg.Value(n.bt.negExp.LogOffset(now))
if timeCost+negBalance < uint64(-targetPriority) {
return 0
}
if uint64(-targetPriority) > negBalance && timePrice > 1e-100 {
if negTime := time.Duration(float64(uint64(-targetPriority)-negBalance) / timePrice); negTime < after {
after -= negTime
} else {
after = 0
}
}
targetPriority = 0
}
timePrice := n.posFactor.timePrice(targetCapacity)
posRequired := uint64(float64(targetPriority)*float64(targetCapacity)+float64(after)*timePrice) + 1
if posRequired >= maxBalance {
return math.MaxUint64 // target not reachable
}
posBalance := n.balance.pos.Value(n.bt.posExp.LogOffset(now))
if posRequired > posBalance {
return posRequired - posBalance
}
return 0
}
// SetPriceFactors sets the price factors. TimeFactor is the price of a nanosecond of // SetPriceFactors sets the price factors. TimeFactor is the price of a nanosecond of
// connection while RequestFactor is the price of a request cost unit. // connection while RequestFactor is the price of a request cost unit.
func (n *NodeBalance) SetPriceFactors(posFactor, negFactor PriceFactors) { func (n *nodeBalance) SetPriceFactors(posFactor, negFactor PriceFactors) {
n.lock.Lock() n.lock.Lock()
now := n.bt.clock.Now() now := n.bt.clock.Now()
n.updateBalance(now) n.updateBalance(now)
@ -346,7 +388,7 @@ func (n *NodeBalance) SetPriceFactors(posFactor, negFactor PriceFactors) {
} }
// GetPriceFactors returns the price factors // GetPriceFactors returns the price factors
func (n *NodeBalance) GetPriceFactors() (posFactor, negFactor PriceFactors) { func (n *nodeBalance) GetPriceFactors() (posFactor, negFactor PriceFactors) {
n.lock.Lock() n.lock.Lock()
defer n.lock.Unlock() defer n.lock.Unlock()
@ -354,7 +396,7 @@ func (n *NodeBalance) GetPriceFactors() (posFactor, negFactor PriceFactors) {
} }
// activate starts time/capacity cost deduction. // activate starts time/capacity cost deduction.
func (n *NodeBalance) activate() { func (n *nodeBalance) activate() {
n.bt.updateTotalBalance(n, func() bool { n.bt.updateTotalBalance(n, func() bool {
if n.active { if n.active {
return false return false
@ -366,7 +408,7 @@ func (n *NodeBalance) activate() {
} }
// deactivate stops time/capacity cost deduction and saves the balances in the database // deactivate stops time/capacity cost deduction and saves the balances in the database
func (n *NodeBalance) deactivate() { func (n *nodeBalance) deactivate() {
n.bt.updateTotalBalance(n, func() bool { n.bt.updateTotalBalance(n, func() bool {
if !n.active { if !n.active {
return false return false
@ -383,7 +425,7 @@ func (n *NodeBalance) deactivate() {
} }
// updateBalance updates balance based on the time factor // updateBalance updates balance based on the time factor
func (n *NodeBalance) updateBalance(now mclock.AbsTime) { func (n *nodeBalance) updateBalance(now mclock.AbsTime) {
if n.active && now > n.lastUpdate { if n.active && now > n.lastUpdate {
n.balance = n.reducedBalance(n.balance, n.lastUpdate, time.Duration(now-n.lastUpdate), n.capacity, 0) n.balance = n.reducedBalance(n.balance, n.lastUpdate, time.Duration(now-n.lastUpdate), n.capacity, 0)
n.lastUpdate = now n.lastUpdate = now
@ -391,7 +433,7 @@ func (n *NodeBalance) updateBalance(now mclock.AbsTime) {
} }
// storeBalance stores the positive and/or negative balance of the node in the database // storeBalance stores the positive and/or negative balance of the node in the database
func (n *NodeBalance) storeBalance(pos, neg bool) { func (n *nodeBalance) storeBalance(pos, neg bool) {
if pos { if pos {
n.bt.storeBalance(n.node.ID().Bytes(), false, n.balance.pos) n.bt.storeBalance(n.node.ID().Bytes(), false, n.balance.pos)
} }
@ -405,7 +447,7 @@ func (n *NodeBalance) storeBalance(pos, neg bool) {
// immediately. // immediately.
// Note: should be called while n.lock is held // Note: should be called while n.lock is held
// Note 2: the callback function runs inside a NodeStateMachine operation // Note 2: the callback function runs inside a NodeStateMachine operation
func (n *NodeBalance) addCallback(id int, threshold int64, callback func()) { func (n *nodeBalance) addCallback(id int, threshold int64, callback func()) {
n.removeCallback(id) n.removeCallback(id)
idx := 0 idx := 0
for idx < n.callbackCount && threshold > n.callbacks[idx].threshold { for idx < n.callbackCount && threshold > n.callbacks[idx].threshold {
@ -425,7 +467,7 @@ func (n *NodeBalance) addCallback(id int, threshold int64, callback func()) {
// removeCallback removes the given callback and returns true if it was active // removeCallback removes the given callback and returns true if it was active
// Note: should be called while n.lock is held // Note: should be called while n.lock is held
func (n *NodeBalance) removeCallback(id int) bool { func (n *nodeBalance) removeCallback(id int) bool {
idx := n.callbackIndex[id] idx := n.callbackIndex[id]
if idx == -1 { if idx == -1 {
return false return false
@ -442,11 +484,11 @@ func (n *NodeBalance) removeCallback(id int) bool {
// checkCallbacks checks whether the threshold of any of the active callbacks // checkCallbacks checks whether the threshold of any of the active callbacks
// have been reached and returns triggered callbacks. // have been reached and returns triggered callbacks.
// Note: checkCallbacks assumes that the balance has been recently updated. // Note: checkCallbacks assumes that the balance has been recently updated.
func (n *NodeBalance) checkCallbacks(now mclock.AbsTime) (callbacks []func()) { func (n *nodeBalance) checkCallbacks(now mclock.AbsTime) (callbacks []func()) {
if n.callbackCount == 0 || n.capacity == 0 { if n.callbackCount == 0 || n.capacity == 0 {
return return
} }
pri := n.balanceToPriority(n.balance, n.capacity) pri := n.balanceToPriority(now, n.balance, n.capacity)
for n.callbackCount != 0 && n.callbacks[n.callbackCount-1].threshold >= pri { for n.callbackCount != 0 && n.callbacks[n.callbackCount-1].threshold >= pri {
n.callbackCount-- n.callbackCount--
n.callbackIndex[n.callbacks[n.callbackCount].id] = -1 n.callbackIndex[n.callbacks[n.callbackCount].id] = -1
@ -458,7 +500,7 @@ func (n *NodeBalance) checkCallbacks(now mclock.AbsTime) (callbacks []func()) {
// scheduleCheck sets up or updates a scheduled event to ensure that it will be called // scheduleCheck sets up or updates a scheduled event to ensure that it will be called
// again just after the next threshold has been reached. // again just after the next threshold has been reached.
func (n *NodeBalance) scheduleCheck(now mclock.AbsTime) { func (n *nodeBalance) scheduleCheck(now mclock.AbsTime) {
if n.callbackCount != 0 { if n.callbackCount != 0 {
d, ok := n.timeUntil(n.callbacks[n.callbackCount-1].threshold) d, ok := n.timeUntil(n.callbacks[n.callbackCount-1].threshold)
if !ok { if !ok {
@ -484,7 +526,7 @@ func (n *NodeBalance) scheduleCheck(now mclock.AbsTime) {
} }
// updateAfter schedules a balance update and callback check in the future // updateAfter schedules a balance update and callback check in the future
func (n *NodeBalance) updateAfter(dt time.Duration) { func (n *nodeBalance) updateAfter(dt time.Duration) {
if n.updateEvent == nil || n.updateEvent.Stop() { if n.updateEvent == nil || n.updateEvent.Stop() {
if dt == 0 { if dt == 0 {
n.updateEvent = nil n.updateEvent = nil
@ -512,20 +554,22 @@ func (n *NodeBalance) updateAfter(dt time.Duration) {
// balanceExhausted should be called when the positive balance is exhausted (priority goes to zero/negative) // balanceExhausted should be called when the positive balance is exhausted (priority goes to zero/negative)
// Note: this function should run inside a NodeStateMachine operation // Note: this function should run inside a NodeStateMachine operation
func (n *NodeBalance) balanceExhausted() { func (n *nodeBalance) balanceExhausted() {
n.lock.Lock() n.lock.Lock()
n.storeBalance(true, false) n.storeBalance(true, false)
n.priority = false n.hasPriority = false
n.lock.Unlock() n.lock.Unlock()
n.bt.ns.SetStateSub(n.node, nodestate.Flags{}, n.bt.PriorityFlag, 0) if n.setFlags {
n.bt.ns.SetStateSub(n.node, nodestate.Flags{}, n.bt.setup.priorityFlag, 0)
}
} }
// checkPriorityStatus checks whether the node has gained priority status and sets the priority // checkPriorityStatus checks whether the node has gained priority status and sets the priority
// callback and flag if necessary. It assumes that the balance has been recently updated. // callback and flag if necessary. It assumes that the balance has been recently updated.
// Note that the priority flag has to be set by the caller after the mutex has been released. // Note that the priority flag has to be set by the caller after the mutex has been released.
func (n *NodeBalance) checkPriorityStatus() bool { func (n *nodeBalance) checkPriorityStatus() bool {
if !n.priority && !n.balance.pos.IsZero() { if !n.hasPriority && !n.balance.pos.IsZero() {
n.priority = true n.hasPriority = true
n.addCallback(balanceCallbackZero, 0, func() { n.balanceExhausted() }) n.addCallback(balanceCallbackZero, 0, func() { n.balanceExhausted() })
return true return true
} }
@ -534,15 +578,15 @@ func (n *NodeBalance) checkPriorityStatus() bool {
// signalPriorityUpdate signals that the priority fell below the previous minimum estimate // signalPriorityUpdate signals that the priority fell below the previous minimum estimate
// Note: this function should run inside a NodeStateMachine operation // Note: this function should run inside a NodeStateMachine operation
func (n *NodeBalance) signalPriorityUpdate() { func (n *nodeBalance) signalPriorityUpdate() {
n.bt.ns.SetStateSub(n.node, n.bt.UpdateFlag, nodestate.Flags{}, 0) n.bt.ns.SetStateSub(n.node, n.bt.setup.updateFlag, nodestate.Flags{}, 0)
n.bt.ns.SetStateSub(n.node, nodestate.Flags{}, n.bt.UpdateFlag, 0) n.bt.ns.SetStateSub(n.node, nodestate.Flags{}, n.bt.setup.updateFlag, 0)
} }
// setCapacity updates the capacity value used for priority calculation // setCapacity updates the capacity value used for priority calculation
// Note: capacity should never be zero // Note: capacity should never be zero
// Note 2: this function should run inside a NodeStateMachine operation // Note 2: this function should run inside a NodeStateMachine operation
func (n *NodeBalance) setCapacity(capacity uint64) { func (n *nodeBalance) setCapacity(capacity uint64) {
n.lock.Lock() n.lock.Lock()
now := n.bt.clock.Now() now := n.bt.clock.Now()
n.updateBalance(now) n.updateBalance(now)
@ -557,74 +601,89 @@ func (n *NodeBalance) setCapacity(capacity uint64) {
// balanceToPriority converts a balance to a priority value. Lower priority means // balanceToPriority converts a balance to a priority value. Lower priority means
// first to disconnect. Positive balance translates to positive priority. If positive // first to disconnect. Positive balance translates to positive priority. If positive
// balance is zero then negative balance translates to a negative priority. // balance is zero then negative balance translates to a negative priority.
func (n *NodeBalance) balanceToPriority(b balance, capacity uint64) int64 { func (n *nodeBalance) balanceToPriority(now mclock.AbsTime, b balance, capacity uint64) int64 {
if !b.pos.IsZero() { pos := b.posValue(now)
return int64(b.pos.Value(n.bt.posExp.LogOffset(n.bt.clock.Now())) / capacity) if pos > 0 {
return int64(pos / capacity)
} }
return -int64(b.neg.Value(n.bt.negExp.LogOffset(n.bt.clock.Now()))) return -int64(b.negValue(now))
}
// priorityToBalance converts a target priority to a requested balance value.
// If the priority is negative, then minimal negative balance is returned;
// otherwise the minimal positive balance is returned.
func (n *nodeBalance) priorityToBalance(priority int64, capacity uint64) (uint64, uint64) {
if priority > 0 {
return uint64(priority) * n.capacity, 0
}
return 0, uint64(-priority)
} }
// reducedBalance estimates the reduced balance at a given time in the fututre based // reducedBalance estimates the reduced balance at a given time in the fututre based
// on the given balance, the time factor and an estimated average request cost per time ratio // on the given balance, the time factor and an estimated average request cost per time ratio
func (n *NodeBalance) reducedBalance(b balance, start mclock.AbsTime, dt time.Duration, capacity uint64, avgReqCost float64) balance { func (n *nodeBalance) reducedBalance(b balance, start mclock.AbsTime, dt time.Duration, capacity uint64, avgReqCost float64) balance {
// since the costs are applied continuously during the dt time period we calculate // since the costs are applied continuously during the dt time period we calculate
// the expiration offset at the middle of the period // the expiration offset at the middle of the period
at := start + mclock.AbsTime(dt/2) var (
dtf := float64(dt) at = start + mclock.AbsTime(dt/2)
dtf = float64(dt)
)
if !b.pos.IsZero() { if !b.pos.IsZero() {
factor := n.posFactor.timePrice(capacity) + n.posFactor.RequestFactor*avgReqCost factor := n.posFactor.connectionPrice(capacity, avgReqCost)
diff := -int64(dtf * factor) diff := -int64(dtf * factor)
dd := b.pos.Add(diff, n.bt.posExp.LogOffset(at)) _, _, net, _ := b.addValue(at, diff, true, false)
if dd == diff { if net == diff {
dtf = 0 dtf = 0
} else { } else {
dtf += float64(dd) / factor dtf += float64(net) / factor
} }
} }
if dt > 0 { if dtf > 0 {
factor := n.negFactor.timePrice(capacity) + n.negFactor.RequestFactor*avgReqCost factor := n.negFactor.connectionPrice(capacity, avgReqCost)
b.neg.Add(int64(dtf*factor), n.bt.negExp.LogOffset(at)) b.addValue(at, int64(dtf*factor), false, false)
} }
return b return b
} }
// timeUntil calculates the remaining time needed to reach a given priority level // timeUntil calculates the remaining time needed to reach a given priority level
// assuming that no requests are processed until then. If the given level is never // assuming that no requests are processed until then. If the given level is never
// reached then (0, false) is returned. // reached then (0, false) is returned. If it has already been reached then (0, true)
// is returned.
// Note: the function assumes that the balance has been recently updated and // Note: the function assumes that the balance has been recently updated and
// calculates the time starting from the last update. // calculates the time starting from the last update.
func (n *NodeBalance) timeUntil(priority int64) (time.Duration, bool) { func (n *nodeBalance) timeUntil(priority int64) (time.Duration, bool) {
now := n.bt.clock.Now() var (
var dt float64 now = n.bt.clock.Now()
if !n.balance.pos.IsZero() { pos = n.balance.posValue(now)
posBalance := n.balance.pos.Value(n.bt.posExp.LogOffset(now)) targetPos, targetNeg = n.priorityToBalance(priority, n.capacity)
timePrice := n.posFactor.timePrice(n.capacity) diffTime float64
)
if pos > 0 {
timePrice := n.posFactor.connectionPrice(n.capacity, 0)
if timePrice < 1e-100 { if timePrice < 1e-100 {
return 0, false return 0, false
} }
if priority > 0 { if targetPos > 0 {
newBalance := uint64(priority) * n.capacity if targetPos > pos {
if newBalance > posBalance { return 0, true
return 0, false
} }
dt = float64(posBalance-newBalance) / timePrice diffTime = float64(pos-targetPos) / timePrice
return time.Duration(dt), true return time.Duration(diffTime), true
} else { } else {
dt = float64(posBalance) / timePrice diffTime = float64(pos) / timePrice
} }
} else { } else {
if priority > 0 { if targetPos > 0 {
return 0, false return 0, true
} }
} }
// if we have a positive balance then dt equals the time needed to get it to zero neg := n.balance.negValue(now)
negBalance := n.balance.neg.Value(n.bt.negExp.LogOffset(now)) if targetNeg > neg {
timePrice := n.negFactor.timePrice(n.capacity) timePrice := n.negFactor.connectionPrice(n.capacity, 0)
if uint64(-priority) > negBalance {
if timePrice < 1e-100 { if timePrice < 1e-100 {
return 0, false return 0, false
} }
dt += float64(uint64(-priority)-negBalance) / timePrice diffTime += float64(targetNeg-neg) / timePrice
} }
return time.Duration(dt), true return time.Duration(diffTime), true
} }

View File

@ -24,6 +24,7 @@ import (
"time" "time"
"github.com/ethereum/go-ethereum/common/mclock" "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/ethdb/memorydb"
"github.com/ethereum/go-ethereum/les/utils" "github.com/ethereum/go-ethereum/les/utils"
"github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enode"
@ -31,59 +32,82 @@ import (
"github.com/ethereum/go-ethereum/p2p/nodestate" "github.com/ethereum/go-ethereum/p2p/nodestate"
) )
var (
testFlag = testSetup.NewFlag("testFlag")
connAddrFlag = testSetup.NewField("connAddr", reflect.TypeOf(""))
btTestSetup = NewBalanceTrackerSetup(testSetup)
)
func init() {
btTestSetup.Connect(connAddrFlag, ppTestSetup.CapacityField)
}
type zeroExpirer struct{} type zeroExpirer struct{}
func (z zeroExpirer) SetRate(now mclock.AbsTime, rate float64) {} func (z zeroExpirer) SetRate(now mclock.AbsTime, rate float64) {}
func (z zeroExpirer) SetLogOffset(now mclock.AbsTime, logOffset utils.Fixed64) {} func (z zeroExpirer) SetLogOffset(now mclock.AbsTime, logOffset utils.Fixed64) {}
func (z zeroExpirer) LogOffset(now mclock.AbsTime) utils.Fixed64 { return 0 } func (z zeroExpirer) LogOffset(now mclock.AbsTime) utils.Fixed64 { return 0 }
type balanceTestClient struct{}
func (client balanceTestClient) FreeClientId() string { return "" }
type balanceTestSetup struct { type balanceTestSetup struct {
clock *mclock.Simulated clock *mclock.Simulated
db ethdb.KeyValueStore
ns *nodestate.NodeStateMachine ns *nodestate.NodeStateMachine
bt *BalanceTracker setup *serverSetup
bt *balanceTracker
} }
func newBalanceTestSetup() *balanceTestSetup { func newBalanceTestSetup(db ethdb.KeyValueStore, posExp, negExp utils.ValueExpirer) *balanceTestSetup {
// Initialize and customize the setup for the balance testing
clock := &mclock.Simulated{} clock := &mclock.Simulated{}
ns := nodestate.NewNodeStateMachine(nil, nil, clock, testSetup) setup := newServerSetup()
db := memorydb.New() setup.clientField = setup.setup.NewField("balancTestClient", reflect.TypeOf(balanceTestClient{}))
bt := NewBalanceTracker(ns, btTestSetup, db, clock, zeroExpirer{}, zeroExpirer{})
ns := nodestate.NewNodeStateMachine(nil, nil, clock, setup.setup)
if posExp == nil {
posExp = zeroExpirer{}
}
if negExp == nil {
negExp = zeroExpirer{}
}
if db == nil {
db = memorydb.New()
}
bt := newBalanceTracker(ns, setup, db, clock, posExp, negExp)
ns.Start() ns.Start()
return &balanceTestSetup{ return &balanceTestSetup{
clock: clock, clock: clock,
db: db,
ns: ns, ns: ns,
setup: setup,
bt: bt, bt: bt,
} }
} }
func (b *balanceTestSetup) newNode(capacity uint64) *NodeBalance { func (b *balanceTestSetup) newNode(capacity uint64) *nodeBalance {
node := enode.SignNull(&enr.Record{}, enode.ID{}) node := enode.SignNull(&enr.Record{}, enode.ID{})
b.ns.SetState(node, testFlag, nodestate.Flags{}, 0) b.ns.SetField(node, b.setup.clientField, balanceTestClient{})
b.ns.SetField(node, btTestSetup.connAddressField, "")
if capacity != 0 { if capacity != 0 {
b.ns.SetField(node, ppTestSetup.CapacityField, capacity) b.ns.SetField(node, b.setup.capacityField, capacity)
} }
n, _ := b.ns.GetField(node, btTestSetup.BalanceField).(*NodeBalance) n, _ := b.ns.GetField(node, b.setup.balanceField).(*nodeBalance)
return n return n
} }
func (b *balanceTestSetup) setBalance(node *nodeBalance, pos, neg uint64) (err error) {
b.bt.BalanceOperation(node.node.ID(), node.connAddress, func(balance AtomicBalanceOperator) {
err = balance.SetBalance(pos, neg)
})
return
}
func (b *balanceTestSetup) addBalance(node *nodeBalance, add int64) (old, new uint64, err error) {
b.bt.BalanceOperation(node.node.ID(), node.connAddress, func(balance AtomicBalanceOperator) {
old, new, err = balance.AddBalance(add)
})
return
}
func (b *balanceTestSetup) stop() { func (b *balanceTestSetup) stop() {
b.bt.Stop() b.bt.stop()
b.ns.Stop() b.ns.Stop()
} }
func TestAddBalance(t *testing.T) { func TestAddBalance(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000) node := b.newNode(1000)
@ -100,7 +124,7 @@ func TestAddBalance(t *testing.T) {
{maxBalance, [2]uint64{0, 0}, 0, true}, {maxBalance, [2]uint64{0, 0}, 0, true},
} }
for _, i := range inputs { for _, i := range inputs {
old, new, err := node.AddBalance(i.delta) old, new, err := b.addBalance(node, i.delta)
if i.expectErr { if i.expectErr {
if err == nil { if err == nil {
t.Fatalf("Expect get error but nil") t.Fatalf("Expect get error but nil")
@ -119,7 +143,7 @@ func TestAddBalance(t *testing.T) {
} }
func TestSetBalance(t *testing.T) { func TestSetBalance(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000) node := b.newNode(1000)
@ -130,9 +154,8 @@ func TestSetBalance(t *testing.T) {
{0, 1000}, {0, 1000},
{1000, 1000}, {1000, 1000},
} }
for _, i := range inputs { for _, i := range inputs {
node.SetBalance(i.pos, i.neg) b.setBalance(node, i.pos, i.neg)
pos, neg := node.GetBalance() pos, neg := node.GetBalance()
if pos != i.pos { if pos != i.pos {
t.Fatalf("Positive balance mismatch, want %v, got %v", i.pos, pos) t.Fatalf("Positive balance mismatch, want %v, got %v", i.pos, pos)
@ -144,13 +167,12 @@ func TestSetBalance(t *testing.T) {
} }
func TestBalanceTimeCost(t *testing.T) { func TestBalanceTimeCost(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000) node := b.newNode(1000)
b.ns.SetField(node.node, ppTestSetup.CapacityField, uint64(1))
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1}) node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
node.SetBalance(uint64(time.Minute), 0) // 1 minute time allowance b.setBalance(node, uint64(time.Minute), 0) // 1 minute time allowance
var inputs = []struct { var inputs = []struct {
runTime time.Duration runTime time.Duration
@ -172,7 +194,7 @@ func TestBalanceTimeCost(t *testing.T) {
} }
} }
node.SetBalance(uint64(time.Minute), 0) // Refill 1 minute time allowance b.setBalance(node, uint64(time.Minute), 0) // Refill 1 minute time allowance
for _, i := range inputs { for _, i := range inputs {
b.clock.Run(i.runTime) b.clock.Run(i.runTime)
if pos, _ := node.GetBalance(); pos != i.expPos { if pos, _ := node.GetBalance(); pos != i.expPos {
@ -185,13 +207,12 @@ func TestBalanceTimeCost(t *testing.T) {
} }
func TestBalanceReqCost(t *testing.T) { func TestBalanceReqCost(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000) node := b.newNode(1000)
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1}) node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
b.ns.SetField(node.node, ppTestSetup.CapacityField, uint64(1)) b.setBalance(node, uint64(time.Minute), 0) // 1 minute time serving time allowance
node.SetBalance(uint64(time.Minute), 0) // 1 minute time serving time allowance
var inputs = []struct { var inputs = []struct {
reqCost uint64 reqCost uint64
expPos uint64 expPos uint64
@ -214,7 +235,7 @@ func TestBalanceReqCost(t *testing.T) {
} }
func TestBalanceToPriority(t *testing.T) { func TestBalanceToPriority(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000) node := b.newNode(1000)
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1}) node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
@ -230,22 +251,20 @@ func TestBalanceToPriority(t *testing.T) {
{0, 1000, -1000}, {0, 1000, -1000},
} }
for _, i := range inputs { for _, i := range inputs {
node.SetBalance(i.pos, i.neg) b.setBalance(node, i.pos, i.neg)
priority := node.Priority(1000) priority := node.priority(1000)
if priority != i.priority { if priority != i.priority {
t.Fatalf("Priority mismatch, want %v, got %v", i.priority, priority) t.Fatalf("priority mismatch, want %v, got %v", i.priority, priority)
} }
} }
} }
func TestEstimatedPriority(t *testing.T) { func TestEstimatedPriority(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000000000) node := b.newNode(1000000000)
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1}) node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
b.setBalance(node, uint64(time.Minute), 0)
b.ns.SetField(node.node, ppTestSetup.CapacityField, uint64(1))
node.SetBalance(uint64(time.Minute), 0)
var inputs = []struct { var inputs = []struct {
runTime time.Duration // time cost runTime time.Duration // time cost
futureTime time.Duration // diff of future time futureTime time.Duration // diff of future time
@ -272,47 +291,18 @@ func TestEstimatedPriority(t *testing.T) {
for _, i := range inputs { for _, i := range inputs {
b.clock.Run(i.runTime) b.clock.Run(i.runTime)
node.RequestServed(i.reqCost) node.RequestServed(i.reqCost)
priority := node.EstimatePriority(1000000000, 0, i.futureTime, 0, false) priority := node.estimatePriority(1000000000, 0, i.futureTime, 0, false)
if priority != i.priority { if priority != i.priority-1 {
t.Fatalf("Estimated priority mismatch, want %v, got %v", i.priority, priority) t.Fatalf("Estimated priority mismatch, want %v, got %v", i.priority-1, priority)
}
}
}
func TestPosBalanceMissing(t *testing.T) {
b := newBalanceTestSetup()
defer b.stop()
node := b.newNode(1000)
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
b.ns.SetField(node.node, ppTestSetup.CapacityField, uint64(1))
var inputs = []struct {
pos, neg uint64
priority int64
cap uint64
after time.Duration
expect uint64
}{
{uint64(time.Second * 2), 0, 0, 1, time.Second, 0},
{uint64(time.Second * 2), 0, 0, 1, 2 * time.Second, 1},
{uint64(time.Second * 2), 0, int64(time.Second), 1, 2 * time.Second, uint64(time.Second) + 1},
{0, 0, int64(time.Second), 1, time.Second, uint64(2*time.Second) + 1},
{0, 0, -int64(time.Second), 1, time.Second, 1},
}
for _, i := range inputs {
node.SetBalance(i.pos, i.neg)
got := node.PosBalanceMissing(i.priority, i.cap, i.after)
if got != i.expect {
t.Fatalf("Missing budget mismatch, want %v, got %v", i.expect, got)
} }
} }
} }
func TestPostiveBalanceCounting(t *testing.T) { func TestPostiveBalanceCounting(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
var nodes []*NodeBalance var nodes []*nodeBalance
for i := 0; i < 100; i += 1 { for i := 0; i < 100; i += 1 {
node := b.newNode(1000000) node := b.newNode(1000000)
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1}) node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
@ -323,7 +313,7 @@ func TestPostiveBalanceCounting(t *testing.T) {
var sum uint64 var sum uint64
for i := 0; i < 100; i += 1 { for i := 0; i < 100; i += 1 {
amount := int64(rand.Intn(100) + 100) amount := int64(rand.Intn(100) + 100)
nodes[i].AddBalance(amount) b.addBalance(nodes[i], amount)
sum += uint64(amount) sum += uint64(amount)
} }
if b.bt.TotalTokenAmount() != sum { if b.bt.TotalTokenAmount() != sum {
@ -333,7 +323,7 @@ func TestPostiveBalanceCounting(t *testing.T) {
// Change client status // Change client status
for i := 0; i < 100; i += 1 { for i := 0; i < 100; i += 1 {
if rand.Intn(2) == 0 { if rand.Intn(2) == 0 {
b.ns.SetField(nodes[i].node, ppTestSetup.CapacityField, uint64(1)) b.ns.SetField(nodes[i].node, b.setup.capacityField, uint64(1))
} }
} }
if b.bt.TotalTokenAmount() != sum { if b.bt.TotalTokenAmount() != sum {
@ -341,7 +331,7 @@ func TestPostiveBalanceCounting(t *testing.T) {
} }
for i := 0; i < 100; i += 1 { for i := 0; i < 100; i += 1 {
if rand.Intn(2) == 0 { if rand.Intn(2) == 0 {
b.ns.SetField(nodes[i].node, ppTestSetup.CapacityField, uint64(1)) b.ns.SetField(nodes[i].node, b.setup.capacityField, uint64(1))
} }
} }
if b.bt.TotalTokenAmount() != sum { if b.bt.TotalTokenAmount() != sum {
@ -350,7 +340,7 @@ func TestPostiveBalanceCounting(t *testing.T) {
} }
func TestCallbackChecking(t *testing.T) { func TestCallbackChecking(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000000) node := b.newNode(1000000)
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1}) node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
@ -363,7 +353,7 @@ func TestCallbackChecking(t *testing.T) {
{0, time.Second}, {0, time.Second},
{-int64(time.Second), 2 * time.Second}, {-int64(time.Second), 2 * time.Second},
} }
node.SetBalance(uint64(time.Second), 0) b.setBalance(node, uint64(time.Second), 0)
for _, i := range inputs { for _, i := range inputs {
diff, _ := node.timeUntil(i.priority) diff, _ := node.timeUntil(i.priority)
if diff != i.expDiff { if diff != i.expDiff {
@ -373,14 +363,13 @@ func TestCallbackChecking(t *testing.T) {
} }
func TestCallback(t *testing.T) { func TestCallback(t *testing.T) {
b := newBalanceTestSetup() b := newBalanceTestSetup(nil, nil, nil)
defer b.stop() defer b.stop()
node := b.newNode(1000) node := b.newNode(1000)
node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1}) node.SetPriceFactors(PriceFactors{1, 0, 1}, PriceFactors{1, 0, 1})
b.ns.SetField(node.node, ppTestSetup.CapacityField, uint64(1))
callCh := make(chan struct{}, 1) callCh := make(chan struct{}, 1)
node.SetBalance(uint64(time.Minute), 0) b.setBalance(node, uint64(time.Minute), 0)
node.addCallback(balanceCallbackZero, 0, func() { callCh <- struct{}{} }) node.addCallback(balanceCallbackZero, 0, func() { callCh <- struct{}{} })
b.clock.Run(time.Minute) b.clock.Run(time.Minute)
@ -390,7 +379,7 @@ func TestCallback(t *testing.T) {
t.Fatalf("Callback hasn't been called yet") t.Fatalf("Callback hasn't been called yet")
} }
node.SetBalance(uint64(time.Minute), 0) b.setBalance(node, uint64(time.Minute), 0)
node.addCallback(balanceCallbackZero, 0, func() { callCh <- struct{}{} }) node.addCallback(balanceCallbackZero, 0, func() { callCh <- struct{}{} })
node.removeCallback(balanceCallbackZero) node.removeCallback(balanceCallbackZero)
@ -403,23 +392,14 @@ func TestCallback(t *testing.T) {
} }
func TestBalancePersistence(t *testing.T) { func TestBalancePersistence(t *testing.T) {
clock := &mclock.Simulated{}
ns := nodestate.NewNodeStateMachine(nil, nil, clock, testSetup)
db := memorydb.New()
posExp := &utils.Expirer{} posExp := &utils.Expirer{}
negExp := &utils.Expirer{} negExp := &utils.Expirer{}
posExp.SetRate(clock.Now(), math.Log(2)/float64(time.Hour*2)) // halves every two hours posExp.SetRate(0, math.Log(2)/float64(time.Hour*2)) // halves every two hours
negExp.SetRate(clock.Now(), math.Log(2)/float64(time.Hour)) // halves every hour negExp.SetRate(0, math.Log(2)/float64(time.Hour)) // halves every hour
bt := NewBalanceTracker(ns, btTestSetup, db, clock, posExp, negExp) setup := newBalanceTestSetup(nil, posExp, negExp)
ns.Start()
bts := &balanceTestSetup{ exp := func(balance *nodeBalance, expPos, expNeg uint64) {
clock: clock, pos, neg := balance.GetBalance()
ns: ns,
bt: bt,
}
var nb *NodeBalance
exp := func(expPos, expNeg uint64) {
pos, neg := nb.GetBalance()
if pos != expPos { if pos != expPos {
t.Fatalf("Positive balance incorrect, want %v, got %v", expPos, pos) t.Fatalf("Positive balance incorrect, want %v, got %v", expPos, pos)
} }
@ -428,44 +408,32 @@ func TestBalancePersistence(t *testing.T) {
} }
} }
expTotal := func(expTotal uint64) { expTotal := func(expTotal uint64) {
total := bt.TotalTokenAmount() total := setup.bt.TotalTokenAmount()
if total != expTotal { if total != expTotal {
t.Fatalf("Total token amount incorrect, want %v, got %v", expTotal, total) t.Fatalf("Total token amount incorrect, want %v, got %v", expTotal, total)
} }
} }
expTotal(0) expTotal(0)
nb = bts.newNode(0) balance := setup.newNode(0)
expTotal(0) expTotal(0)
nb.SetBalance(16000000000, 16000000000) setup.setBalance(balance, 16000000000, 16000000000)
exp(16000000000, 16000000000) exp(balance, 16000000000, 16000000000)
expTotal(16000000000) expTotal(16000000000)
clock.Run(time.Hour * 2)
exp(8000000000, 4000000000)
expTotal(8000000000)
bt.Stop()
ns.Stop()
clock = &mclock.Simulated{} setup.clock.Run(time.Hour * 2)
ns = nodestate.NewNodeStateMachine(nil, nil, clock, testSetup) exp(balance, 8000000000, 4000000000)
posExp = &utils.Expirer{}
negExp = &utils.Expirer{}
posExp.SetRate(clock.Now(), math.Log(2)/float64(time.Hour*2)) // halves every two hours
negExp.SetRate(clock.Now(), math.Log(2)/float64(time.Hour)) // halves every hour
bt = NewBalanceTracker(ns, btTestSetup, db, clock, posExp, negExp)
ns.Start()
bts = &balanceTestSetup{
clock: clock,
ns: ns,
bt: bt,
}
expTotal(8000000000) expTotal(8000000000)
nb = bts.newNode(0) setup.stop()
exp(8000000000, 4000000000)
// Test the functionalities after restart
setup = newBalanceTestSetup(setup.db, posExp, negExp)
expTotal(8000000000) expTotal(8000000000)
clock.Run(time.Hour * 2) balance = setup.newNode(0)
exp(4000000000, 1000000000) exp(balance, 8000000000, 4000000000)
expTotal(8000000000)
setup.clock.Run(time.Hour * 2)
exp(balance, 4000000000, 1000000000)
expTotal(4000000000) expTotal(4000000000)
bt.Stop() setup.stop()
ns.Stop()
} }

View File

@ -17,7 +17,6 @@
package server package server
import ( import (
"reflect"
"sync" "sync"
"time" "time"
@ -25,6 +24,7 @@ import (
"github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/les/utils" "github.com/ethereum/go-ethereum/les/utils"
"github.com/ethereum/go-ethereum/p2p/enode" "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/p2p/nodestate"
) )
@ -34,82 +34,56 @@ const (
persistExpirationRefresh = time.Minute * 5 // refresh period of the token expiration persistence persistExpirationRefresh = time.Minute * 5 // refresh period of the token expiration persistence
) )
// BalanceTrackerSetup contains node state flags and fields used by BalanceTracker // balanceTracker tracks positive and negative balances for connected nodes.
type BalanceTrackerSetup struct { // After clientField is set externally, a nodeBalance is created and previous
// controlled by PriorityPool
PriorityFlag, UpdateFlag nodestate.Flags
BalanceField nodestate.Field
// external connections
connAddressField, capacityField nodestate.Field
}
// NewBalanceTrackerSetup creates a new BalanceTrackerSetup and initializes the fields
// and flags controlled by BalanceTracker
func NewBalanceTrackerSetup(setup *nodestate.Setup) BalanceTrackerSetup {
return BalanceTrackerSetup{
// PriorityFlag is set if the node has a positive balance
PriorityFlag: setup.NewFlag("priorityNode"),
// UpdateFlag set and then immediately reset if the balance has been updated and
// therefore priority is suddenly changed
UpdateFlag: setup.NewFlag("balanceUpdate"),
// BalanceField contains the NodeBalance struct which implements nodePriority,
// allowing on-demand priority calculation and future priority estimation
BalanceField: setup.NewField("balance", reflect.TypeOf(&NodeBalance{})),
}
}
// Connect sets the fields used by BalanceTracker as an input
func (bts *BalanceTrackerSetup) Connect(connAddressField, capacityField nodestate.Field) {
bts.connAddressField = connAddressField
bts.capacityField = capacityField
}
// BalanceTracker tracks positive and negative balances for connected nodes.
// After connAddressField is set externally, a NodeBalance is created and previous
// balance values are loaded from the database. Both balances are exponentially expired // balance values are loaded from the database. Both balances are exponentially expired
// values. Costs are deducted from the positive balance if present, otherwise added to // values. Costs are deducted from the positive balance if present, otherwise added to
// the negative balance. If the capacity is non-zero then a time cost is applied // the negative balance. If the capacity is non-zero then a time cost is applied
// continuously while individual request costs are applied immediately. // continuously while individual request costs are applied immediately.
// The two balances are translated into a single priority value that also depends // The two balances are translated into a single priority value that also depends
// on the actual capacity. // on the actual capacity.
type BalanceTracker struct { type balanceTracker struct {
BalanceTrackerSetup setup *serverSetup
clock mclock.Clock clock mclock.Clock
lock sync.Mutex lock sync.Mutex
ns *nodestate.NodeStateMachine ns *nodestate.NodeStateMachine
ndb *nodeDB ndb *nodeDB
posExp, negExp utils.ValueExpirer posExp, negExp utils.ValueExpirer
posExpTC, negExpTC uint64
posExpTC, negExpTC uint64
defaultPosFactors, defaultNegFactors PriceFactors
active, inactive utils.ExpiredValue active, inactive utils.ExpiredValue
balanceTimer *utils.UpdateTimer balanceTimer *utils.UpdateTimer
quit chan struct{} quit chan struct{}
} }
// NewBalanceTracker creates a new BalanceTracker // newBalanceTracker creates a new balanceTracker
func NewBalanceTracker(ns *nodestate.NodeStateMachine, setup BalanceTrackerSetup, db ethdb.KeyValueStore, clock mclock.Clock, posExp, negExp utils.ValueExpirer) *BalanceTracker { func newBalanceTracker(ns *nodestate.NodeStateMachine, setup *serverSetup, db ethdb.KeyValueStore, clock mclock.Clock, posExp, negExp utils.ValueExpirer) *balanceTracker {
ndb := newNodeDB(db, clock) ndb := newNodeDB(db, clock)
bt := &BalanceTracker{ bt := &balanceTracker{
ns: ns, ns: ns,
BalanceTrackerSetup: setup, setup: setup,
ndb: ndb, ndb: ndb,
clock: clock, clock: clock,
posExp: posExp, posExp: posExp,
negExp: negExp, negExp: negExp,
balanceTimer: utils.NewUpdateTimer(clock, time.Second*10), balanceTimer: utils.NewUpdateTimer(clock, time.Second*10),
quit: make(chan struct{}), quit: make(chan struct{}),
} }
posOffset, negOffset := bt.ndb.getExpiration() posOffset, negOffset := bt.ndb.getExpiration()
posExp.SetLogOffset(clock.Now(), posOffset) posExp.SetLogOffset(clock.Now(), posOffset)
negExp.SetLogOffset(clock.Now(), negOffset) negExp.SetLogOffset(clock.Now(), negOffset)
// Load all persisted balance entries of priority nodes,
// calculate the total number of issued service tokens.
bt.ndb.forEachBalance(false, func(id enode.ID, balance utils.ExpiredValue) bool { bt.ndb.forEachBalance(false, func(id enode.ID, balance utils.ExpiredValue) bool {
bt.inactive.AddExp(balance) bt.inactive.AddExp(balance)
return true return true
}) })
ns.SubscribeField(bt.capacityField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) { ns.SubscribeField(bt.setup.capacityField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) {
n, _ := ns.GetField(node, bt.BalanceField).(*NodeBalance) n, _ := ns.GetField(node, bt.setup.balanceField).(*nodeBalance)
if n == nil { if n == nil {
return return
} }
@ -126,15 +100,22 @@ func NewBalanceTracker(ns *nodestate.NodeStateMachine, setup BalanceTrackerSetup
n.deactivate() n.deactivate()
} }
}) })
ns.SubscribeField(bt.connAddressField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) { ns.SubscribeField(bt.setup.clientField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) {
type peer interface {
FreeClientId() string
}
if newValue != nil { if newValue != nil {
ns.SetFieldSub(node, bt.BalanceField, bt.newNodeBalance(node, newValue.(string))) n := bt.newNodeBalance(node, newValue.(peer).FreeClientId(), true)
bt.lock.Lock()
n.SetPriceFactors(bt.defaultPosFactors, bt.defaultNegFactors)
bt.lock.Unlock()
ns.SetFieldSub(node, bt.setup.balanceField, n)
} else { } else {
ns.SetStateSub(node, nodestate.Flags{}, bt.PriorityFlag, 0) ns.SetStateSub(node, nodestate.Flags{}, bt.setup.priorityFlag, 0)
if b, _ := ns.GetField(node, bt.BalanceField).(*NodeBalance); b != nil { if b, _ := ns.GetField(node, bt.setup.balanceField).(*nodeBalance); b != nil {
b.deactivate() b.deactivate()
} }
ns.SetFieldSub(node, bt.BalanceField, nil) ns.SetFieldSub(node, bt.setup.balanceField, nil)
} }
}) })
@ -157,31 +138,31 @@ func NewBalanceTracker(ns *nodestate.NodeStateMachine, setup BalanceTrackerSetup
return bt return bt
} }
// Stop saves expiration offset and unsaved node balances and shuts BalanceTracker down // Stop saves expiration offset and unsaved node balances and shuts balanceTracker down
func (bt *BalanceTracker) Stop() { func (bt *balanceTracker) stop() {
now := bt.clock.Now() now := bt.clock.Now()
bt.ndb.setExpiration(bt.posExp.LogOffset(now), bt.negExp.LogOffset(now)) bt.ndb.setExpiration(bt.posExp.LogOffset(now), bt.negExp.LogOffset(now))
close(bt.quit) close(bt.quit)
bt.ns.ForEach(nodestate.Flags{}, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) { bt.ns.ForEach(nodestate.Flags{}, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) {
if n, ok := bt.ns.GetField(node, bt.BalanceField).(*NodeBalance); ok { if n, ok := bt.ns.GetField(node, bt.setup.balanceField).(*nodeBalance); ok {
n.lock.Lock() n.lock.Lock()
n.storeBalance(true, true) n.storeBalance(true, true)
n.lock.Unlock() n.lock.Unlock()
bt.ns.SetField(node, bt.BalanceField, nil) bt.ns.SetField(node, bt.setup.balanceField, nil)
} }
}) })
bt.ndb.close() bt.ndb.close()
} }
// TotalTokenAmount returns the current total amount of service tokens in existence // TotalTokenAmount returns the current total amount of service tokens in existence
func (bt *BalanceTracker) TotalTokenAmount() uint64 { func (bt *balanceTracker) TotalTokenAmount() uint64 {
bt.lock.Lock() bt.lock.Lock()
defer bt.lock.Unlock() defer bt.lock.Unlock()
bt.balanceTimer.Update(func(_ time.Duration) bool { bt.balanceTimer.Update(func(_ time.Duration) bool {
bt.active = utils.ExpiredValue{} bt.active = utils.ExpiredValue{}
bt.ns.ForEach(nodestate.Flags{}, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) { bt.ns.ForEach(nodestate.Flags{}, nodestate.Flags{}, func(node *enode.Node, state nodestate.Flags) {
if n, ok := bt.ns.GetField(node, bt.BalanceField).(*NodeBalance); ok && n.active { if n, ok := bt.ns.GetField(node, bt.setup.balanceField).(*nodeBalance); ok && n.active {
pos, _ := n.GetRawBalance() pos, _ := n.GetRawBalance()
bt.active.AddExp(pos) bt.active.AddExp(pos)
} }
@ -194,13 +175,21 @@ func (bt *BalanceTracker) TotalTokenAmount() uint64 {
} }
// GetPosBalanceIDs lists node IDs with an associated positive balance // GetPosBalanceIDs lists node IDs with an associated positive balance
func (bt *BalanceTracker) GetPosBalanceIDs(start, stop enode.ID, maxCount int) (result []enode.ID) { func (bt *balanceTracker) GetPosBalanceIDs(start, stop enode.ID, maxCount int) (result []enode.ID) {
return bt.ndb.getPosBalanceIDs(start, stop, maxCount) return bt.ndb.getPosBalanceIDs(start, stop, maxCount)
} }
// SetDefaultFactors sets the default price factors applied to subsequently connected clients
func (bt *balanceTracker) SetDefaultFactors(posFactors, negFactors PriceFactors) {
bt.lock.Lock()
bt.defaultPosFactors = posFactors
bt.defaultNegFactors = negFactors
bt.lock.Unlock()
}
// SetExpirationTCs sets positive and negative token expiration time constants. // SetExpirationTCs sets positive and negative token expiration time constants.
// Specified in seconds, 0 means infinite (no expiration). // Specified in seconds, 0 means infinite (no expiration).
func (bt *BalanceTracker) SetExpirationTCs(pos, neg uint64) { func (bt *balanceTracker) SetExpirationTCs(pos, neg uint64) {
bt.lock.Lock() bt.lock.Lock()
defer bt.lock.Unlock() defer bt.lock.Unlock()
@ -220,39 +209,55 @@ func (bt *BalanceTracker) SetExpirationTCs(pos, neg uint64) {
// GetExpirationTCs returns the current positive and negative token expiration // GetExpirationTCs returns the current positive and negative token expiration
// time constants // time constants
func (bt *BalanceTracker) GetExpirationTCs() (pos, neg uint64) { func (bt *balanceTracker) GetExpirationTCs() (pos, neg uint64) {
bt.lock.Lock() bt.lock.Lock()
defer bt.lock.Unlock() defer bt.lock.Unlock()
return bt.posExpTC, bt.negExpTC return bt.posExpTC, bt.negExpTC
} }
// newNodeBalance loads balances from the database and creates a NodeBalance instance // BalanceOperation allows atomic operations on the balance of a node regardless of whether
// for the given node. It also sets the PriorityFlag and adds balanceCallbackZero if // it is currently connected or not
func (bt *balanceTracker) BalanceOperation(id enode.ID, connAddress string, cb func(AtomicBalanceOperator)) {
bt.ns.Operation(func() {
var nb *nodeBalance
if node := bt.ns.GetNode(id); node != nil {
nb, _ = bt.ns.GetField(node, bt.setup.balanceField).(*nodeBalance)
} else {
node = enode.SignNull(&enr.Record{}, id)
nb = bt.newNodeBalance(node, connAddress, false)
}
cb(nb)
})
}
// newNodeBalance loads balances from the database and creates a nodeBalance instance
// for the given node. It also sets the priorityFlag and adds balanceCallbackZero if
// the node has a positive balance. // the node has a positive balance.
// Note: this function should run inside a NodeStateMachine operation // Note: this function should run inside a NodeStateMachine operation
func (bt *BalanceTracker) newNodeBalance(node *enode.Node, negBalanceKey string) *NodeBalance { func (bt *balanceTracker) newNodeBalance(node *enode.Node, connAddress string, setFlags bool) *nodeBalance {
pb := bt.ndb.getOrNewBalance(node.ID().Bytes(), false) pb := bt.ndb.getOrNewBalance(node.ID().Bytes(), false)
nb := bt.ndb.getOrNewBalance([]byte(negBalanceKey), true) nb := bt.ndb.getOrNewBalance([]byte(connAddress), true)
n := &NodeBalance{ n := &nodeBalance{
bt: bt, bt: bt,
node: node, node: node,
connAddress: negBalanceKey, setFlags: setFlags,
balance: balance{pos: pb, neg: nb}, connAddress: connAddress,
balance: balance{pos: pb, neg: nb, posExp: bt.posExp, negExp: bt.negExp},
initTime: bt.clock.Now(), initTime: bt.clock.Now(),
lastUpdate: bt.clock.Now(), lastUpdate: bt.clock.Now(),
} }
for i := range n.callbackIndex { for i := range n.callbackIndex {
n.callbackIndex[i] = -1 n.callbackIndex[i] = -1
} }
if n.checkPriorityStatus() { if setFlags && n.checkPriorityStatus() {
n.bt.ns.SetStateSub(n.node, n.bt.PriorityFlag, nodestate.Flags{}, 0) n.bt.ns.SetStateSub(n.node, n.bt.setup.priorityFlag, nodestate.Flags{}, 0)
} }
return n return n
} }
// storeBalance stores either a positive or a negative balance in the database // storeBalance stores either a positive or a negative balance in the database
func (bt *BalanceTracker) storeBalance(id []byte, neg bool, value utils.ExpiredValue) { func (bt *balanceTracker) storeBalance(id []byte, neg bool, value utils.ExpiredValue) {
if bt.canDropBalance(bt.clock.Now(), neg, value) { if bt.canDropBalance(bt.clock.Now(), neg, value) {
bt.ndb.delBalance(id, neg) // balance is small enough, drop it directly. bt.ndb.delBalance(id, neg) // balance is small enough, drop it directly.
} else { } else {
@ -262,7 +267,7 @@ func (bt *BalanceTracker) storeBalance(id []byte, neg bool, value utils.ExpiredV
// canDropBalance tells whether a positive or negative balance is below the threshold // canDropBalance tells whether a positive or negative balance is below the threshold
// and therefore can be dropped from the database // and therefore can be dropped from the database
func (bt *BalanceTracker) canDropBalance(now mclock.AbsTime, neg bool, b utils.ExpiredValue) bool { func (bt *balanceTracker) canDropBalance(now mclock.AbsTime, neg bool, b utils.ExpiredValue) bool {
if neg { if neg {
return b.Value(bt.negExp.LogOffset(now)) <= negThreshold return b.Value(bt.negExp.LogOffset(now)) <= negThreshold
} }
@ -270,7 +275,7 @@ func (bt *BalanceTracker) canDropBalance(now mclock.AbsTime, neg bool, b utils.E
} }
// updateTotalBalance adjusts the total balance after executing given callback. // updateTotalBalance adjusts the total balance after executing given callback.
func (bt *BalanceTracker) updateTotalBalance(n *NodeBalance, callback func() bool) { func (bt *balanceTracker) updateTotalBalance(n *nodeBalance, callback func() bool) {
bt.lock.Lock() bt.lock.Lock()
defer bt.lock.Unlock() defer bt.lock.Unlock()

View File

@ -0,0 +1,335 @@
// Copyright 2019 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 server
import (
"errors"
"sync"
"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/les/vflux"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/nodestate"
"github.com/ethereum/go-ethereum/rlp"
)
var (
ErrNotConnected = errors.New("client not connected")
ErrNoPriority = errors.New("priority too low to raise capacity")
ErrCantFindMaximum = errors.New("Unable to find maximum allowed capacity")
)
// ClientPool implements a client database that assigns a priority to each client
// based on a positive and negative balance. Positive balance is externally assigned
// to prioritized clients and is decreased with connection time and processed
// requests (unless the price factors are zero). If the positive balance is zero
// then negative balance is accumulated.
//
// Balance tracking and priority calculation for connected clients is done by
// balanceTracker. PriorityQueue ensures that clients with the lowest positive or
// highest negative balance get evicted when the total capacity allowance is full
// and new clients with a better balance want to connect.
//
// Already connected nodes receive a small bias in their favor in order to avoid
// accepting and instantly kicking out clients. In theory, we try to ensure that
// each client can have several minutes of connection time.
//
// Balances of disconnected clients are stored in nodeDB including positive balance
// and negative banalce. Boeth positive balance and negative balance will decrease
// exponentially. If the balance is low enough, then the record will be dropped.
type ClientPool struct {
*priorityPool
*balanceTracker
setup *serverSetup
clock mclock.Clock
closed bool
ns *nodestate.NodeStateMachine
synced func() bool
lock sync.RWMutex
connectedBias time.Duration
minCap uint64 // the minimal capacity value allowed for any client
capReqNode *enode.Node // node that is requesting capacity change; only used inside NSM operation
}
// clientPeer represents a peer in the client pool. None of the callbacks should block.
type clientPeer interface {
Node() *enode.Node
FreeClientId() string // unique id for non-priority clients (typically a prefix of the network address)
InactiveAllowance() time.Duration // disconnection timeout for inactive non-priority peers
UpdateCapacity(newCap uint64, requested bool) // signals a capacity update (requested is true if it is a result of a SetCapacity call on the given peer
Disconnect() // initiates disconnection (Unregister should always be called)
}
// NewClientPool creates a new client pool
func NewClientPool(balanceDb ethdb.KeyValueStore, minCap uint64, connectedBias time.Duration, clock mclock.Clock, synced func() bool) *ClientPool {
setup := newServerSetup()
ns := nodestate.NewNodeStateMachine(nil, nil, clock, setup.setup)
cp := &ClientPool{
priorityPool: newPriorityPool(ns, setup, clock, minCap, connectedBias, 4, 100),
balanceTracker: newBalanceTracker(ns, setup, balanceDb, clock, &utils.Expirer{}, &utils.Expirer{}),
setup: setup,
ns: ns,
clock: clock,
minCap: minCap,
connectedBias: connectedBias,
synced: synced,
}
ns.SubscribeState(nodestate.MergeFlags(setup.activeFlag, setup.inactiveFlag, setup.priorityFlag), func(node *enode.Node, oldState, newState nodestate.Flags) {
if newState.Equals(setup.inactiveFlag) {
// set timeout for non-priority inactive client
var timeout time.Duration
if c, ok := ns.GetField(node, setup.clientField).(clientPeer); ok {
timeout = c.InactiveAllowance()
}
if timeout > 0 {
ns.AddTimeout(node, setup.inactiveFlag, timeout)
} else {
// Note: if capacity is immediately available then priorityPool will set the active
// flag simultaneously with removing the inactive flag and therefore this will not
// initiate disconnection
ns.SetStateSub(node, nodestate.Flags{}, setup.inactiveFlag, 0)
}
}
if oldState.Equals(setup.inactiveFlag) && newState.Equals(setup.inactiveFlag.Or(setup.priorityFlag)) {
ns.SetStateSub(node, setup.inactiveFlag, nodestate.Flags{}, 0) // priority gained; remove timeout
}
if newState.Equals(setup.activeFlag) {
// active with no priority; limit capacity to minCap
cap, _ := ns.GetField(node, setup.capacityField).(uint64)
if cap > minCap {
cp.requestCapacity(node, minCap, minCap, 0)
}
}
if newState.Equals(nodestate.Flags{}) {
if c, ok := ns.GetField(node, setup.clientField).(clientPeer); ok {
c.Disconnect()
}
}
})
ns.SubscribeField(setup.capacityField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) {
if c, ok := ns.GetField(node, setup.clientField).(clientPeer); ok {
newCap, _ := newValue.(uint64)
c.UpdateCapacity(newCap, node == cp.capReqNode)
}
})
// add metrics
cp.ns.SubscribeState(nodestate.MergeFlags(cp.setup.activeFlag, cp.setup.inactiveFlag), func(node *enode.Node, oldState, newState nodestate.Flags) {
if oldState.IsEmpty() && !newState.IsEmpty() {
clientConnectedMeter.Mark(1)
}
if !oldState.IsEmpty() && newState.IsEmpty() {
clientDisconnectedMeter.Mark(1)
}
if oldState.HasNone(cp.setup.activeFlag) && oldState.HasAll(cp.setup.activeFlag) {
clientActivatedMeter.Mark(1)
}
if oldState.HasAll(cp.setup.activeFlag) && oldState.HasNone(cp.setup.activeFlag) {
clientDeactivatedMeter.Mark(1)
}
_, connected := cp.Active()
totalConnectedGauge.Update(int64(connected))
})
return cp
}
// Start starts the client pool. Should be called before Register/Unregister.
func (cp *ClientPool) Start() {
cp.ns.Start()
}
// Stop shuts the client pool down. The clientPeer interface callbacks will not be called
// after Stop. Register calls will return nil.
func (cp *ClientPool) Stop() {
cp.balanceTracker.stop()
cp.ns.Stop()
}
// Register registers the peer into the client pool. If the peer has insufficient
// priority and remains inactive for longer than the allowed timeout then it will be
// disconnected by calling the Disconnect function of the clientPeer interface.
func (cp *ClientPool) Register(peer clientPeer) ConnectedBalance {
cp.ns.SetField(peer.Node(), cp.setup.clientField, peerWrapper{peer})
balance, _ := cp.ns.GetField(peer.Node(), cp.setup.balanceField).(*nodeBalance)
return balance
}
// Unregister removes the peer from the client pool
func (cp *ClientPool) Unregister(peer clientPeer) {
cp.ns.SetField(peer.Node(), cp.setup.clientField, nil)
}
// setConnectedBias sets the connection bias, which is applied to already connected clients
// So that already connected client won't be kicked out very soon and we can ensure all
// connected clients can have enough time to request or sync some data.
func (cp *ClientPool) SetConnectedBias(bias time.Duration) {
cp.lock.Lock()
cp.connectedBias = bias
cp.setActiveBias(bias)
cp.lock.Unlock()
}
// SetCapacity sets the assigned capacity of a connected client
func (cp *ClientPool) SetCapacity(node *enode.Node, reqCap uint64, bias time.Duration, requested bool) (capacity uint64, err error) {
cp.lock.RLock()
if cp.connectedBias > bias {
bias = cp.connectedBias
}
cp.lock.RUnlock()
cp.ns.Operation(func() {
balance, _ := cp.ns.GetField(node, cp.setup.balanceField).(*nodeBalance)
if balance == nil {
err = ErrNotConnected
return
}
capacity, _ = cp.ns.GetField(node, cp.setup.capacityField).(uint64)
if capacity == 0 {
// if the client is inactive then it has insufficient priority for the minimal capacity
// (will be activated automatically with minCap when possible)
return
}
if reqCap < cp.minCap {
// can't request less than minCap; switching between 0 (inactive state) and minCap is
// performed by the server automatically as soon as necessary/possible
reqCap = cp.minCap
}
if reqCap > cp.minCap && cp.ns.GetState(node).HasNone(cp.setup.priorityFlag) {
err = ErrNoPriority
return
}
if reqCap == capacity {
return
}
if requested {
// mark the requested node so that the UpdateCapacity callback can signal
// whether the update is the direct result of a SetCapacity call on the given node
cp.capReqNode = node
defer func() {
cp.capReqNode = nil
}()
}
var minTarget, maxTarget uint64
if reqCap > capacity {
// Estimate maximum available capacity at the current priority level and request
// the estimated amount.
// Note: requestCapacity could find the highest available capacity between the
// current and the requested capacity but it could cost a lot of iterations with
// fine step adjustment if the requested capacity is very high. By doing a quick
// estimation of the maximum available capacity based on the capacity curve we
// can limit the number of required iterations.
curve := cp.getCapacityCurve().exclude(node.ID())
maxTarget = curve.maxCapacity(func(capacity uint64) int64 {
return balance.estimatePriority(capacity, 0, 0, bias, false)
})
if maxTarget <= capacity {
return
}
if maxTarget > reqCap {
maxTarget = reqCap
}
// Specify a narrow target range that allows a limited number of fine step
// iterations
minTarget = maxTarget - maxTarget/20
if minTarget < capacity {
minTarget = capacity
}
} else {
minTarget, maxTarget = reqCap, reqCap
}
if newCap := cp.requestCapacity(node, minTarget, maxTarget, bias); newCap >= minTarget && newCap <= maxTarget {
capacity = newCap
return
}
// we should be able to find the maximum allowed capacity in a few iterations
log.Error("Unable to find maximum allowed capacity")
err = ErrCantFindMaximum
})
return
}
// serveCapQuery serves a vflux capacity query. It receives multiple token amount values
// and a bias time value. For each given token amount it calculates the maximum achievable
// capacity in case the amount is added to the balance.
func (cp *ClientPool) serveCapQuery(id enode.ID, freeID string, data []byte) []byte {
var req vflux.CapacityQueryReq
if rlp.DecodeBytes(data, &req) != nil {
return nil
}
if l := len(req.AddTokens); l == 0 || l > vflux.CapacityQueryMaxLen {
return nil
}
result := make(vflux.CapacityQueryReply, len(req.AddTokens))
if !cp.synced() {
capacityQueryZeroMeter.Mark(1)
reply, _ := rlp.EncodeToBytes(&result)
return reply
}
bias := time.Second * time.Duration(req.Bias)
cp.lock.RLock()
if cp.connectedBias > bias {
bias = cp.connectedBias
}
cp.lock.RUnlock()
// use capacityCurve to answer request for multiple newly bought token amounts
curve := cp.getCapacityCurve().exclude(id)
cp.BalanceOperation(id, freeID, func(balance AtomicBalanceOperator) {
pb, _ := balance.GetBalance()
for i, addTokens := range req.AddTokens {
add := addTokens.Int64()
result[i] = curve.maxCapacity(func(capacity uint64) int64 {
return balance.estimatePriority(capacity, add, 0, bias, false) / int64(capacity)
})
if add <= 0 && uint64(-add) >= pb && result[i] > cp.minCap {
result[i] = cp.minCap
}
if result[i] < cp.minCap {
result[i] = 0
}
}
})
// add first result to metrics (don't care about priority client multi-queries yet)
if result[0] == 0 {
capacityQueryZeroMeter.Mark(1)
} else {
capacityQueryNonZeroMeter.Mark(1)
}
reply, _ := rlp.EncodeToBytes(&result)
return reply
}
// Handle implements Service
func (cp *ClientPool) Handle(id enode.ID, address string, name string, data []byte) []byte {
switch name {
case vflux.CapacityQueryName:
return cp.serveCapQuery(id, address, data)
default:
return nil
}
}

View File

@ -14,7 +14,7 @@
// You should have received a copy of the GNU Lesser General Public License // 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/>. // along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package les package server
import ( import (
"fmt" "fmt"
@ -24,12 +24,13 @@ import (
"github.com/ethereum/go-ethereum/common/mclock" "github.com/ethereum/go-ethereum/common/mclock"
"github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/rawdb"
vfs "github.com/ethereum/go-ethereum/les/vflux/server"
"github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr" "github.com/ethereum/go-ethereum/p2p/enr"
"github.com/ethereum/go-ethereum/p2p/nodestate" "github.com/ethereum/go-ethereum/p2p/nodestate"
) )
const defaultConnectedBias = time.Minute * 3
func TestClientPoolL10C100Free(t *testing.T) { func TestClientPoolL10C100Free(t *testing.T) {
testClientPool(t, 10, 100, 0, true) testClientPool(t, 10, 100, 0, true)
} }
@ -64,11 +65,6 @@ type poolTestPeer struct {
inactiveAllowed bool inactiveAllowed bool
} }
func testStateMachine() *nodestate.NodeStateMachine {
return nodestate.NewNodeStateMachine(nil, nil, mclock.System{}, serverSetup)
}
func newPoolTestPeer(i int, disconnCh chan int) *poolTestPeer { func newPoolTestPeer(i int, disconnCh chan int) *poolTestPeer {
return &poolTestPeer{ return &poolTestPeer{
index: i, index: i,
@ -81,36 +77,39 @@ func (i *poolTestPeer) Node() *enode.Node {
return i.node return i.node
} }
func (i *poolTestPeer) freeClientId() string { func (i *poolTestPeer) FreeClientId() string {
return fmt.Sprintf("addr #%d", i.index) return fmt.Sprintf("addr #%d", i.index)
} }
func (i *poolTestPeer) updateCapacity(cap uint64) { func (i *poolTestPeer) InactiveAllowance() time.Duration {
i.cap = cap if i.inactiveAllowed {
return time.Second * 10
}
return 0
} }
func (i *poolTestPeer) freeze() {} func (i *poolTestPeer) UpdateCapacity(capacity uint64, requested bool) {
i.cap = capacity
func (i *poolTestPeer) allowInactive() bool {
return i.inactiveAllowed
} }
func getBalance(pool *clientPool, p *poolTestPeer) (pos, neg uint64) { func (i *poolTestPeer) Disconnect() {
temp := pool.ns.GetField(p.node, clientInfoField) == nil if i.disconnCh == nil {
if temp { return
pool.ns.SetField(p.node, connAddressField, p.freeClientId())
}
n, _ := pool.ns.GetField(p.node, pool.BalanceField).(*vfs.NodeBalance)
pos, neg = n.GetBalance()
if temp {
pool.ns.SetField(p.node, connAddressField, nil)
} }
id := i.node.ID()
i.disconnCh <- int(id[0]) + int(id[1])<<8
}
func getBalance(pool *ClientPool, p *poolTestPeer) (pos, neg uint64) {
pool.BalanceOperation(p.node.ID(), p.FreeClientId(), func(nb AtomicBalanceOperator) {
pos, neg = nb.GetBalance()
})
return return
} }
func addBalance(pool *clientPool, id enode.ID, amount int64) { func addBalance(pool *ClientPool, id enode.ID, amount int64) {
pool.forClients([]enode.ID{id}, func(c *clientInfo) { pool.BalanceOperation(id, "", func(nb AtomicBalanceOperator) {
c.balance.AddBalance(amount) nb.AddBalance(amount)
}) })
} }
@ -122,6 +121,19 @@ func checkDiff(a, b uint64) bool {
return a > b+maxDiff || b > a+maxDiff return a > b+maxDiff || b > a+maxDiff
} }
func connect(pool *ClientPool, peer *poolTestPeer) uint64 {
pool.Register(peer)
return peer.cap
}
func disconnect(pool *ClientPool, peer *poolTestPeer) {
pool.Unregister(peer)
}
func alwaysTrueFn() bool {
return true
}
func testClientPool(t *testing.T, activeLimit, clientCount, paidCount int, randomDisconnect bool) { func testClientPool(t *testing.T, activeLimit, clientCount, paidCount int, randomDisconnect bool) {
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
var ( var (
@ -130,19 +142,17 @@ func testClientPool(t *testing.T, activeLimit, clientCount, paidCount int, rando
connected = make([]bool, clientCount) connected = make([]bool, clientCount)
connTicks = make([]int, clientCount) connTicks = make([]int, clientCount)
disconnCh = make(chan int, clientCount) disconnCh = make(chan int, clientCount)
disconnFn = func(id enode.ID) { pool = NewClientPool(db, 1, 0, &clock, alwaysTrueFn)
disconnCh <- int(id[0]) + int(id[1])<<8
}
pool = newClientPool(testStateMachine(), db, 1, 0, &clock, disconnFn, alwaysTrueFn)
) )
pool.ns.Start() pool.Start()
pool.SetExpirationTCs(0, 1000)
pool.setLimits(activeLimit, uint64(activeLimit)) pool.SetLimits(uint64(activeLimit), uint64(activeLimit))
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}) pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
// pool should accept new peers up to its connected limit // pool should accept new peers up to its connected limit
for i := 0; i < activeLimit; i++ { for i := 0; i < activeLimit; i++ {
if cap, _ := pool.connect(newPoolTestPeer(i, disconnCh)); cap != 0 { if cap := connect(pool, newPoolTestPeer(i, disconnCh)); cap != 0 {
connected[i] = true connected[i] = true
} else { } else {
t.Fatalf("Test peer #%d rejected", i) t.Fatalf("Test peer #%d rejected", i)
@ -163,23 +173,23 @@ func testClientPool(t *testing.T, activeLimit, clientCount, paidCount int, rando
i := rand.Intn(clientCount) i := rand.Intn(clientCount)
if connected[i] { if connected[i] {
if randomDisconnect { if randomDisconnect {
pool.disconnect(newPoolTestPeer(i, disconnCh)) disconnect(pool, newPoolTestPeer(i, disconnCh))
connected[i] = false connected[i] = false
connTicks[i] += tickCounter connTicks[i] += tickCounter
} }
} else { } else {
if cap, _ := pool.connect(newPoolTestPeer(i, disconnCh)); cap != 0 { if cap := connect(pool, newPoolTestPeer(i, disconnCh)); cap != 0 {
connected[i] = true connected[i] = true
connTicks[i] -= tickCounter connTicks[i] -= tickCounter
} else { } else {
pool.disconnect(newPoolTestPeer(i, disconnCh)) disconnect(pool, newPoolTestPeer(i, disconnCh))
} }
} }
pollDisconnects: pollDisconnects:
for { for {
select { select {
case i := <-disconnCh: case i := <-disconnCh:
pool.disconnect(newPoolTestPeer(i, disconnCh)) disconnect(pool, newPoolTestPeer(i, disconnCh))
if connected[i] { if connected[i] {
connTicks[i] += tickCounter connTicks[i] += tickCounter
connected[i] = false connected[i] = false
@ -211,18 +221,18 @@ func testClientPool(t *testing.T, activeLimit, clientCount, paidCount int, rando
t.Errorf("Total connected time of test node #%d (%d) outside expected range (%d to %d)", i, connTicks[i], min, max) t.Errorf("Total connected time of test node #%d (%d) outside expected range (%d to %d)", i, connTicks[i], min, max)
} }
} }
pool.stop() pool.Stop()
} }
func testPriorityConnect(t *testing.T, pool *clientPool, p *poolTestPeer, cap uint64, expSuccess bool) { func testPriorityConnect(t *testing.T, pool *ClientPool, p *poolTestPeer, cap uint64, expSuccess bool) {
if cap, _ := pool.connect(p); cap == 0 { if cap := connect(pool, p); cap == 0 {
if expSuccess { if expSuccess {
t.Fatalf("Failed to connect paid client") t.Fatalf("Failed to connect paid client")
} else { } else {
return return
} }
} }
if _, err := pool.setCapacity(p.node, "", cap, defaultConnectedBias, true); err != nil { if newCap, _ := pool.SetCapacity(p.node, cap, defaultConnectedBias, true); newCap != cap {
if expSuccess { if expSuccess {
t.Fatalf("Failed to raise capacity of paid client") t.Fatalf("Failed to raise capacity of paid client")
} else { } else {
@ -239,11 +249,11 @@ func TestConnectPaidClient(t *testing.T) {
clock mclock.Simulated clock mclock.Simulated
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
) )
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, func(id enode.ID) {}, alwaysTrueFn) pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool.ns.Start() pool.Start()
defer pool.stop() defer pool.Stop()
pool.setLimits(10, uint64(10)) pool.SetLimits(10, uint64(10))
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}) pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
// Add balance for an external client and mark it as paid client // Add balance for an external client and mark it as paid client
addBalance(pool, newPoolTestPeer(0, nil).node.ID(), int64(time.Minute)) addBalance(pool, newPoolTestPeer(0, nil).node.ID(), int64(time.Minute))
@ -255,16 +265,16 @@ func TestConnectPaidClientToSmallPool(t *testing.T) {
clock mclock.Simulated clock mclock.Simulated
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
) )
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, func(id enode.ID) {}, alwaysTrueFn) pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool.ns.Start() pool.Start()
defer pool.stop() defer pool.Stop()
pool.setLimits(10, uint64(10)) // Total capacity limit is 10 pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}) pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
// Add balance for an external client and mark it as paid client // Add balance for an external client and mark it as paid client
addBalance(pool, newPoolTestPeer(0, nil).node.ID(), int64(time.Minute)) addBalance(pool, newPoolTestPeer(0, nil).node.ID(), int64(time.Minute))
// Connect a fat paid client to pool, should reject it. // connect a fat paid client to pool, should reject it.
testPriorityConnect(t, pool, newPoolTestPeer(0, nil), 100, false) testPriorityConnect(t, pool, newPoolTestPeer(0, nil), 100, false)
} }
@ -273,24 +283,23 @@ func TestConnectPaidClientToFullPool(t *testing.T) {
clock mclock.Simulated clock mclock.Simulated
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
) )
removeFn := func(enode.ID) {} // Noop pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, removeFn, alwaysTrueFn) pool.Start()
pool.ns.Start() defer pool.Stop()
defer pool.stop() pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.setLimits(10, uint64(10)) // Total capacity limit is 10 pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
addBalance(pool, newPoolTestPeer(i, nil).node.ID(), int64(time.Second*20)) addBalance(pool, newPoolTestPeer(i, nil).node.ID(), int64(time.Second*20))
pool.connect(newPoolTestPeer(i, nil)) connect(pool, newPoolTestPeer(i, nil))
} }
addBalance(pool, newPoolTestPeer(11, nil).node.ID(), int64(time.Second*2)) // Add low balance to new paid client addBalance(pool, newPoolTestPeer(11, nil).node.ID(), int64(time.Second*2)) // Add low balance to new paid client
if cap, _ := pool.connect(newPoolTestPeer(11, nil)); cap != 0 { if cap := connect(pool, newPoolTestPeer(11, nil)); cap != 0 {
t.Fatalf("Low balance paid client should be rejected") t.Fatalf("Low balance paid client should be rejected")
} }
clock.Run(time.Second) clock.Run(time.Second)
addBalance(pool, newPoolTestPeer(12, nil).node.ID(), int64(time.Minute*5)) // Add high balance to new paid client addBalance(pool, newPoolTestPeer(12, nil).node.ID(), int64(time.Minute*5)) // Add high balance to new paid client
if cap, _ := pool.connect(newPoolTestPeer(12, nil)); cap == 0 { if cap := connect(pool, newPoolTestPeer(12, nil)); cap == 0 {
t.Fatalf("High balance paid client should be accepted") t.Fatalf("High balance paid client should be accepted")
} }
} }
@ -301,23 +310,20 @@ func TestPaidClientKickedOut(t *testing.T) {
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
kickedCh = make(chan int, 100) kickedCh = make(chan int, 100)
) )
removeFn := func(id enode.ID) { pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
kickedCh <- int(id[0]) pool.Start()
} pool.SetExpirationTCs(0, 0)
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, removeFn, alwaysTrueFn) defer pool.Stop()
pool.ns.Start() pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.bt.SetExpirationTCs(0, 0) pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
defer pool.stop()
pool.setLimits(10, uint64(10)) // Total capacity limit is 10
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
addBalance(pool, newPoolTestPeer(i, kickedCh).node.ID(), 10000000000) // 10 second allowance addBalance(pool, newPoolTestPeer(i, kickedCh).node.ID(), 10000000000) // 10 second allowance
pool.connect(newPoolTestPeer(i, kickedCh)) connect(pool, newPoolTestPeer(i, kickedCh))
clock.Run(time.Millisecond) clock.Run(time.Millisecond)
} }
clock.Run(defaultConnectedBias + time.Second*11) clock.Run(defaultConnectedBias + time.Second*11)
if cap, _ := pool.connect(newPoolTestPeer(11, kickedCh)); cap == 0 { if cap := connect(pool, newPoolTestPeer(11, kickedCh)); cap == 0 {
t.Fatalf("Free client should be accepted") t.Fatalf("Free client should be accepted")
} }
select { select {
@ -335,12 +341,12 @@ func TestConnectFreeClient(t *testing.T) {
clock mclock.Simulated clock mclock.Simulated
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
) )
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, func(id enode.ID) {}, alwaysTrueFn) pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool.ns.Start() pool.Start()
defer pool.stop() defer pool.Stop()
pool.setLimits(10, uint64(10)) pool.SetLimits(10, uint64(10))
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}) pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
if cap, _ := pool.connect(newPoolTestPeer(0, nil)); cap == 0 { if cap := connect(pool, newPoolTestPeer(0, nil)); cap == 0 {
t.Fatalf("Failed to connect free client") t.Fatalf("Failed to connect free client")
} }
testPriorityConnect(t, pool, newPoolTestPeer(0, nil), 2, false) testPriorityConnect(t, pool, newPoolTestPeer(0, nil), 2, false)
@ -351,26 +357,25 @@ func TestConnectFreeClientToFullPool(t *testing.T) {
clock mclock.Simulated clock mclock.Simulated
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
) )
removeFn := func(enode.ID) {} // Noop pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, removeFn, alwaysTrueFn) pool.Start()
pool.ns.Start() defer pool.Stop()
defer pool.stop() pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.setLimits(10, uint64(10)) // Total capacity limit is 10 pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
pool.connect(newPoolTestPeer(i, nil)) connect(pool, newPoolTestPeer(i, nil))
} }
if cap, _ := pool.connect(newPoolTestPeer(11, nil)); cap != 0 { if cap := connect(pool, newPoolTestPeer(11, nil)); cap != 0 {
t.Fatalf("New free client should be rejected") t.Fatalf("New free client should be rejected")
} }
clock.Run(time.Minute) clock.Run(time.Minute)
if cap, _ := pool.connect(newPoolTestPeer(12, nil)); cap != 0 { if cap := connect(pool, newPoolTestPeer(12, nil)); cap != 0 {
t.Fatalf("New free client should be rejected") t.Fatalf("New free client should be rejected")
} }
clock.Run(time.Millisecond) clock.Run(time.Millisecond)
clock.Run(4 * time.Minute) clock.Run(4 * time.Minute)
if cap, _ := pool.connect(newPoolTestPeer(13, nil)); cap == 0 { if cap := connect(pool, newPoolTestPeer(13, nil)); cap == 0 {
t.Fatalf("Old client connects more than 5min should be kicked") t.Fatalf("Old client connects more than 5min should be kicked")
} }
} }
@ -381,18 +386,17 @@ func TestFreeClientKickedOut(t *testing.T) {
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
kicked = make(chan int, 100) kicked = make(chan int, 100)
) )
removeFn := func(id enode.ID) { kicked <- int(id[0]) } pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, removeFn, alwaysTrueFn) pool.Start()
pool.ns.Start() defer pool.Stop()
defer pool.stop() pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.setLimits(10, uint64(10)) // Total capacity limit is 10 pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
pool.connect(newPoolTestPeer(i, kicked)) connect(pool, newPoolTestPeer(i, kicked))
clock.Run(time.Millisecond) clock.Run(time.Millisecond)
} }
if cap, _ := pool.connect(newPoolTestPeer(10, kicked)); cap != 0 { if cap := connect(pool, newPoolTestPeer(10, kicked)); cap != 0 {
t.Fatalf("New free client should be rejected") t.Fatalf("New free client should be rejected")
} }
select { select {
@ -400,10 +404,10 @@ func TestFreeClientKickedOut(t *testing.T) {
case <-time.NewTimer(time.Second).C: case <-time.NewTimer(time.Second).C:
t.Fatalf("timeout") t.Fatalf("timeout")
} }
pool.disconnect(newPoolTestPeer(10, kicked)) disconnect(pool, newPoolTestPeer(10, kicked))
clock.Run(5 * time.Minute) clock.Run(5 * time.Minute)
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
pool.connect(newPoolTestPeer(i+10, kicked)) connect(pool, newPoolTestPeer(i+10, kicked))
} }
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
select { select {
@ -423,18 +427,17 @@ func TestPositiveBalanceCalculation(t *testing.T) {
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
kicked = make(chan int, 10) kicked = make(chan int, 10)
) )
removeFn := func(id enode.ID) { kicked <- int(id[0]) } // Noop pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, removeFn, alwaysTrueFn) pool.Start()
pool.ns.Start() defer pool.Stop()
defer pool.stop() pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.setLimits(10, uint64(10)) // Total capacity limit is 10 pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
addBalance(pool, newPoolTestPeer(0, kicked).node.ID(), int64(time.Minute*3)) addBalance(pool, newPoolTestPeer(0, kicked).node.ID(), int64(time.Minute*3))
testPriorityConnect(t, pool, newPoolTestPeer(0, kicked), 10, true) testPriorityConnect(t, pool, newPoolTestPeer(0, kicked), 10, true)
clock.Run(time.Minute) clock.Run(time.Minute)
pool.disconnect(newPoolTestPeer(0, kicked)) disconnect(pool, newPoolTestPeer(0, kicked))
pb, _ := getBalance(pool, newPoolTestPeer(0, kicked)) pb, _ := getBalance(pool, newPoolTestPeer(0, kicked))
if checkDiff(pb, uint64(time.Minute*2)) { if checkDiff(pb, uint64(time.Minute*2)) {
t.Fatalf("Positive balance mismatch, want %v, got %v", uint64(time.Minute*2), pb) t.Fatalf("Positive balance mismatch, want %v, got %v", uint64(time.Minute*2), pb)
@ -447,12 +450,11 @@ func TestDowngradePriorityClient(t *testing.T) {
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
kicked = make(chan int, 10) kicked = make(chan int, 10)
) )
removeFn := func(id enode.ID) { kicked <- int(id[0]) } // Noop pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, removeFn, alwaysTrueFn) pool.Start()
pool.ns.Start() defer pool.Stop()
defer pool.stop() pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.setLimits(10, uint64(10)) // Total capacity limit is 10 pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 1})
p := newPoolTestPeer(0, kicked) p := newPoolTestPeer(0, kicked)
addBalance(pool, p.node.ID(), int64(time.Minute)) addBalance(pool, p.node.ID(), int64(time.Minute))
@ -483,30 +485,31 @@ func TestNegativeBalanceCalculation(t *testing.T) {
clock mclock.Simulated clock mclock.Simulated
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
) )
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, func(id enode.ID) {}, alwaysTrueFn) pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool.ns.Start() pool.Start()
defer pool.stop() defer pool.Stop()
pool.setLimits(10, uint64(10)) // Total capacity limit is 10 pool.SetExpirationTCs(0, 3600)
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1e-3, CapacityFactor: 0, RequestFactor: 1}, vfs.PriceFactors{TimeFactor: 1e-3, CapacityFactor: 0, RequestFactor: 1}) pool.SetLimits(10, uint64(10)) // Total capacity limit is 10
pool.SetDefaultFactors(PriceFactors{TimeFactor: 1e-3, CapacityFactor: 0, RequestFactor: 1}, PriceFactors{TimeFactor: 1e-3, CapacityFactor: 0, RequestFactor: 1})
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
pool.connect(newPoolTestPeer(i, nil)) connect(pool, newPoolTestPeer(i, nil))
} }
clock.Run(time.Second) clock.Run(time.Second)
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
pool.disconnect(newPoolTestPeer(i, nil)) disconnect(pool, newPoolTestPeer(i, nil))
_, nb := getBalance(pool, newPoolTestPeer(i, nil)) _, nb := getBalance(pool, newPoolTestPeer(i, nil))
if nb != 0 { if nb != 0 {
t.Fatalf("Short connection shouldn't be recorded") t.Fatalf("Short connection shouldn't be recorded")
} }
} }
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
pool.connect(newPoolTestPeer(i, nil)) connect(pool, newPoolTestPeer(i, nil))
} }
clock.Run(time.Minute) clock.Run(time.Minute)
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
pool.disconnect(newPoolTestPeer(i, nil)) disconnect(pool, newPoolTestPeer(i, nil))
_, nb := getBalance(pool, newPoolTestPeer(i, nil)) _, nb := getBalance(pool, newPoolTestPeer(i, nil))
exp := uint64(time.Minute) / 1000 exp := uint64(time.Minute) / 1000
exp -= exp / 120 // correct for negative balance expiration exp -= exp / 120 // correct for negative balance expiration
@ -521,10 +524,10 @@ func TestInactiveClient(t *testing.T) {
clock mclock.Simulated clock mclock.Simulated
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
) )
pool := newClientPool(testStateMachine(), db, 1, defaultConnectedBias, &clock, func(id enode.ID) {}, alwaysTrueFn) pool := NewClientPool(db, 1, defaultConnectedBias, &clock, alwaysTrueFn)
pool.ns.Start() pool.Start()
defer pool.stop() defer pool.Stop()
pool.setLimits(2, uint64(2)) pool.SetLimits(2, uint64(2))
p1 := newPoolTestPeer(1, nil) p1 := newPoolTestPeer(1, nil)
p1.inactiveAllowed = true p1.inactiveAllowed = true
@ -535,15 +538,15 @@ func TestInactiveClient(t *testing.T) {
addBalance(pool, p1.node.ID(), 1000*int64(time.Second)) addBalance(pool, p1.node.ID(), 1000*int64(time.Second))
addBalance(pool, p3.node.ID(), 2000*int64(time.Second)) addBalance(pool, p3.node.ID(), 2000*int64(time.Second))
// p1: 1000 p2: 0 p3: 2000 // p1: 1000 p2: 0 p3: 2000
p1.cap, _ = pool.connect(p1) p1.cap = connect(pool, p1)
if p1.cap != 1 { if p1.cap != 1 {
t.Fatalf("Failed to connect peer #1") t.Fatalf("Failed to connect peer #1")
} }
p2.cap, _ = pool.connect(p2) p2.cap = connect(pool, p2)
if p2.cap != 1 { if p2.cap != 1 {
t.Fatalf("Failed to connect peer #2") t.Fatalf("Failed to connect peer #2")
} }
p3.cap, _ = pool.connect(p3) p3.cap = connect(pool, p3)
if p3.cap != 1 { if p3.cap != 1 {
t.Fatalf("Failed to connect peer #3") t.Fatalf("Failed to connect peer #3")
} }
@ -566,11 +569,11 @@ func TestInactiveClient(t *testing.T) {
if p2.cap != 0 { if p2.cap != 0 {
t.Fatalf("Failed to deactivate peer #2") t.Fatalf("Failed to deactivate peer #2")
} }
pool.setDefaultFactors(vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 0}, vfs.PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 0}) pool.SetDefaultFactors(PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 0}, PriceFactors{TimeFactor: 1, CapacityFactor: 0, RequestFactor: 0})
p4 := newPoolTestPeer(4, nil) p4 := newPoolTestPeer(4, nil)
addBalance(pool, p4.node.ID(), 1500*int64(time.Second)) addBalance(pool, p4.node.ID(), 1500*int64(time.Second))
// p1: 1000 p2: 500 p3: 2000 p4: 1500 // p1: 1000 p2: 500 p3: 2000 p4: 1500
p4.cap, _ = pool.connect(p4) p4.cap = connect(pool, p4)
if p4.cap != 1 { if p4.cap != 1 {
t.Fatalf("Failed to activate peer #4") t.Fatalf("Failed to activate peer #4")
} }
@ -579,8 +582,8 @@ func TestInactiveClient(t *testing.T) {
} }
clock.Run(time.Second * 600) clock.Run(time.Second * 600)
// manually trigger a check to avoid a long real-time wait // manually trigger a check to avoid a long real-time wait
pool.ns.SetState(p1.node, pool.UpdateFlag, nodestate.Flags{}, 0) pool.ns.SetState(p1.node, pool.setup.updateFlag, nodestate.Flags{}, 0)
pool.ns.SetState(p1.node, nodestate.Flags{}, pool.UpdateFlag, 0) pool.ns.SetState(p1.node, nodestate.Flags{}, pool.setup.updateFlag, 0)
// p1: 1000 p2: 500 p3: 2000 p4: 900 // p1: 1000 p2: 500 p3: 2000 p4: 900
if p1.cap != 1 { if p1.cap != 1 {
t.Fatalf("Failed to activate peer #1") t.Fatalf("Failed to activate peer #1")
@ -588,8 +591,8 @@ func TestInactiveClient(t *testing.T) {
if p4.cap != 0 { if p4.cap != 0 {
t.Fatalf("Failed to deactivate peer #4") t.Fatalf("Failed to deactivate peer #4")
} }
pool.disconnect(p2) disconnect(pool, p2)
pool.disconnect(p4) disconnect(pool, p4)
addBalance(pool, p1.node.ID(), -1000*int64(time.Second)) addBalance(pool, p1.node.ID(), -1000*int64(time.Second))
if p1.cap != 1 { if p1.cap != 1 {
t.Fatalf("Should not deactivate peer #1") t.Fatalf("Should not deactivate peer #1")

View File

@ -0,0 +1,33 @@
// Copyright 2021 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 server
import (
"github.com/ethereum/go-ethereum/metrics"
)
var (
totalConnectedGauge = metrics.NewRegisteredGauge("vflux/server/totalConnected", nil)
clientConnectedMeter = metrics.NewRegisteredMeter("vflux/server/clientEvent/connected", nil)
clientActivatedMeter = metrics.NewRegisteredMeter("vflux/server/clientEvent/activated", nil)
clientDeactivatedMeter = metrics.NewRegisteredMeter("vflux/server/clientEvent/deactivated", nil)
clientDisconnectedMeter = metrics.NewRegisteredMeter("vflux/server/clientEvent/disconnected", nil)
capacityQueryZeroMeter = metrics.NewRegisteredMeter("vflux/server/capQueryZero", nil)
capacityQueryNonZeroMeter = metrics.NewRegisteredMeter("vflux/server/capQueryNonZero", nil)
)

View File

@ -18,7 +18,6 @@ package server
import ( import (
"math" "math"
"reflect"
"sync" "sync"
"time" "time"
@ -33,36 +32,7 @@ const (
lazyQueueRefresh = time.Second * 10 // refresh period of the active queue lazyQueueRefresh = time.Second * 10 // refresh period of the active queue
) )
// PriorityPoolSetup contains node state flags and fields used by PriorityPool // priorityPool handles a set of nodes where each node has a capacity (a scalar value)
// Note: ActiveFlag and InactiveFlag can be controlled both externally and by the pool,
// see PriorityPool description for details.
type PriorityPoolSetup struct {
// controlled by PriorityPool
ActiveFlag, InactiveFlag nodestate.Flags
CapacityField, ppNodeInfoField nodestate.Field
// external connections
updateFlag nodestate.Flags
priorityField nodestate.Field
}
// NewPriorityPoolSetup creates a new PriorityPoolSetup and initializes the fields
// and flags controlled by PriorityPool
func NewPriorityPoolSetup(setup *nodestate.Setup) PriorityPoolSetup {
return PriorityPoolSetup{
ActiveFlag: setup.NewFlag("active"),
InactiveFlag: setup.NewFlag("inactive"),
CapacityField: setup.NewField("capacity", reflect.TypeOf(uint64(0))),
ppNodeInfoField: setup.NewField("ppNodeInfo", reflect.TypeOf(&ppNodeInfo{})),
}
}
// Connect sets the fields and flags used by PriorityPool as an input
func (pps *PriorityPoolSetup) Connect(priorityField nodestate.Field, updateFlag nodestate.Flags) {
pps.priorityField = priorityField // should implement nodePriority
pps.updateFlag = updateFlag // triggers an immediate priority update
}
// PriorityPool handles a set of nodes where each node has a capacity (a scalar value)
// and a priority (which can change over time and can also depend on the capacity). // and a priority (which can change over time and can also depend on the capacity).
// A node is active if it has at least the necessary minimal amount of capacity while // A node is active if it has at least the necessary minimal amount of capacity while
// inactive nodes have 0 capacity (values between 0 and the minimum are not allowed). // inactive nodes have 0 capacity (values between 0 and the minimum are not allowed).
@ -79,70 +49,70 @@ func (pps *PriorityPoolSetup) Connect(priorityField nodestate.Field, updateFlag
// This time bias can be interpreted as minimum expected active time at the given // This time bias can be interpreted as minimum expected active time at the given
// capacity (if the threshold priority stays the same). // capacity (if the threshold priority stays the same).
// //
// Nodes in the pool always have either InactiveFlag or ActiveFlag set. A new node is // Nodes in the pool always have either inactiveFlag or activeFlag set. A new node is
// added to the pool by externally setting InactiveFlag. PriorityPool can switch a node // added to the pool by externally setting inactiveFlag. priorityPool can switch a node
// between InactiveFlag and ActiveFlag at any time. Nodes can be removed from the pool // between inactiveFlag and activeFlag at any time. Nodes can be removed from the pool
// by externally resetting both flags. ActiveFlag should not be set externally. // by externally resetting both flags. activeFlag should not be set externally.
// //
// The highest priority nodes in "inactive" state are moved to "active" state as soon as // The highest priority nodes in "inactive" state are moved to "active" state as soon as
// the minimum capacity can be granted for them. The capacity of lower priority active // the minimum capacity can be granted for them. The capacity of lower priority active
// nodes is reduced or they are demoted to "inactive" state if their priority is // nodes is reduced or they are demoted to "inactive" state if their priority is
// insufficient even at minimal capacity. // insufficient even at minimal capacity.
type PriorityPool struct { type priorityPool struct {
PriorityPoolSetup setup *serverSetup
ns *nodestate.NodeStateMachine ns *nodestate.NodeStateMachine
clock mclock.Clock clock mclock.Clock
lock sync.Mutex lock sync.Mutex
activeQueue *prque.LazyQueue inactiveQueue *prque.Prque
inactiveQueue *prque.Prque maxCount, maxCap uint64
changed []*ppNodeInfo minCap uint64
activeCount, activeCap uint64 activeBias time.Duration
maxCount, maxCap uint64 capacityStepDiv, fineStepDiv uint64
minCap uint64
activeBias time.Duration
capacityStepDiv uint64
cachedCurve *CapacityCurve cachedCurve *capacityCurve
ccUpdatedAt mclock.AbsTime ccUpdatedAt mclock.AbsTime
ccUpdateForced bool ccUpdateForced bool
tempState []*ppNodeInfo // nodes currently in temporary state
// the following fields represent the temporary state if tempState is not empty
activeCount, activeCap uint64
activeQueue *prque.LazyQueue
} }
// nodePriority interface provides current and estimated future priorities on demand // ppNodeInfo is the internal node descriptor of priorityPool
type nodePriority interface {
// Priority should return the current priority of the node (higher is better)
Priority(cap uint64) int64
// EstMinPriority should return a lower estimate for the minimum of the node priority
// value starting from the current moment until the given time. If the priority goes
// under the returned estimate before the specified moment then it is the caller's
// responsibility to signal with updateFlag.
EstimatePriority(cap uint64, addBalance int64, future, bias time.Duration, update bool) int64
}
// ppNodeInfo is the internal node descriptor of PriorityPool
type ppNodeInfo struct { type ppNodeInfo struct {
nodePriority nodePriority nodePriority nodePriority
node *enode.Node node *enode.Node
connected bool connected bool
capacity, origCap uint64 capacity uint64 // only changed when temporary state is committed
bias time.Duration
forced, changed bool
activeIndex, inactiveIndex int activeIndex, inactiveIndex int
tempState bool // should only be true while the priorityPool lock is held
tempCapacity uint64 // equals capacity when tempState is false
// the following fields only affect the temporary state and they are set to their
// default value when entering the temp state
minTarget, stepDiv uint64
bias time.Duration
} }
// NewPriorityPool creates a new PriorityPool // newPriorityPool creates a new priorityPool
func NewPriorityPool(ns *nodestate.NodeStateMachine, setup PriorityPoolSetup, clock mclock.Clock, minCap uint64, activeBias time.Duration, capacityStepDiv uint64) *PriorityPool { func newPriorityPool(ns *nodestate.NodeStateMachine, setup *serverSetup, clock mclock.Clock, minCap uint64, activeBias time.Duration, capacityStepDiv, fineStepDiv uint64) *priorityPool {
pp := &PriorityPool{ pp := &priorityPool{
ns: ns, setup: setup,
PriorityPoolSetup: setup, ns: ns,
clock: clock, clock: clock,
inactiveQueue: prque.New(inactiveSetIndex), inactiveQueue: prque.New(inactiveSetIndex),
minCap: minCap, minCap: minCap,
activeBias: activeBias, activeBias: activeBias,
capacityStepDiv: capacityStepDiv, capacityStepDiv: capacityStepDiv,
fineStepDiv: fineStepDiv,
}
if pp.activeBias < time.Duration(1) {
pp.activeBias = time.Duration(1)
} }
pp.activeQueue = prque.NewLazyQueue(activeSetIndex, activePriority, pp.activeMaxPriority, clock, lazyQueueRefresh) pp.activeQueue = prque.NewLazyQueue(activeSetIndex, activePriority, pp.activeMaxPriority, clock, lazyQueueRefresh)
ns.SubscribeField(pp.priorityField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) { ns.SubscribeField(pp.setup.balanceField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) {
if newValue != nil { if newValue != nil {
c := &ppNodeInfo{ c := &ppNodeInfo{
node: node, node: node,
@ -150,18 +120,19 @@ func NewPriorityPool(ns *nodestate.NodeStateMachine, setup PriorityPoolSetup, cl
activeIndex: -1, activeIndex: -1,
inactiveIndex: -1, inactiveIndex: -1,
} }
ns.SetFieldSub(node, pp.ppNodeInfoField, c) ns.SetFieldSub(node, pp.setup.queueField, c)
ns.SetStateSub(node, setup.inactiveFlag, nodestate.Flags{}, 0)
} else { } else {
ns.SetStateSub(node, nodestate.Flags{}, pp.ActiveFlag.Or(pp.InactiveFlag), 0) ns.SetStateSub(node, nodestate.Flags{}, pp.setup.activeFlag.Or(pp.setup.inactiveFlag), 0)
if n, _ := pp.ns.GetField(node, pp.ppNodeInfoField).(*ppNodeInfo); n != nil { if n, _ := pp.ns.GetField(node, pp.setup.queueField).(*ppNodeInfo); n != nil {
pp.disconnectedNode(n) pp.disconnectedNode(n)
} }
ns.SetFieldSub(node, pp.CapacityField, nil) ns.SetFieldSub(node, pp.setup.capacityField, nil)
ns.SetFieldSub(node, pp.ppNodeInfoField, nil) ns.SetFieldSub(node, pp.setup.queueField, nil)
} }
}) })
ns.SubscribeState(pp.ActiveFlag.Or(pp.InactiveFlag), func(node *enode.Node, oldState, newState nodestate.Flags) { ns.SubscribeState(pp.setup.activeFlag.Or(pp.setup.inactiveFlag), func(node *enode.Node, oldState, newState nodestate.Flags) {
if c, _ := pp.ns.GetField(node, pp.ppNodeInfoField).(*ppNodeInfo); c != nil { if c, _ := pp.ns.GetField(node, pp.setup.queueField).(*ppNodeInfo); c != nil {
if oldState.IsEmpty() { if oldState.IsEmpty() {
pp.connectedNode(c) pp.connectedNode(c)
} }
@ -170,7 +141,7 @@ func NewPriorityPool(ns *nodestate.NodeStateMachine, setup PriorityPoolSetup, cl
} }
} }
}) })
ns.SubscribeState(pp.updateFlag, func(node *enode.Node, oldState, newState nodestate.Flags) { ns.SubscribeState(pp.setup.updateFlag, func(node *enode.Node, oldState, newState nodestate.Flags) {
if !newState.IsEmpty() { if !newState.IsEmpty() {
pp.updatePriority(node) pp.updatePriority(node)
} }
@ -178,18 +149,12 @@ func NewPriorityPool(ns *nodestate.NodeStateMachine, setup PriorityPoolSetup, cl
return pp return pp
} }
// RequestCapacity checks whether changing the capacity of a node to the given target // requestCapacity tries to set the capacity of a connected node to the highest possible
// is possible (bias is applied in favor of other active nodes if the target is higher // value inside the given target range. If maxTarget is not reachable then the capacity is
// than the current capacity). // iteratively reduced in fine steps based on the fineStepDiv parameter until minTarget is reached.
// If setCap is true then it also performs the change if possible. The function returns // The function returns the new capacity if successful and the original capacity otherwise.
// the minimum priority needed to do the change and whether it is currently allowed. // Note: this function should run inside a NodeStateMachine operation
// If setCap and allowed are both true then the caller can assume that the change was func (pp *priorityPool) requestCapacity(node *enode.Node, minTarget, maxTarget uint64, bias time.Duration) uint64 {
// successful.
// Note: priorityField should always be set before calling RequestCapacity. If setCap
// is false then both InactiveFlag and ActiveFlag can be unset and they are not changed
// by this function call either.
// Note 2: this function should run inside a NodeStateMachine operation
func (pp *PriorityPool) RequestCapacity(node *enode.Node, targetCap uint64, bias time.Duration, setCap bool) (minPriority int64, allowed bool) {
pp.lock.Lock() pp.lock.Lock()
pp.activeQueue.Refresh() pp.activeQueue.Refresh()
var updates []capUpdate var updates []capUpdate
@ -198,39 +163,37 @@ func (pp *PriorityPool) RequestCapacity(node *enode.Node, targetCap uint64, bias
pp.updateFlags(updates) pp.updateFlags(updates)
}() }()
if targetCap < pp.minCap { if minTarget < pp.minCap {
targetCap = pp.minCap minTarget = pp.minCap
}
if maxTarget < minTarget {
maxTarget = minTarget
} }
if bias < pp.activeBias { if bias < pp.activeBias {
bias = pp.activeBias bias = pp.activeBias
} }
c, _ := pp.ns.GetField(node, pp.ppNodeInfoField).(*ppNodeInfo) c, _ := pp.ns.GetField(node, pp.setup.queueField).(*ppNodeInfo)
if c == nil { if c == nil {
log.Error("RequestCapacity called for unknown node", "id", node.ID()) log.Error("requestCapacity called for unknown node", "id", node.ID())
return math.MaxInt64, false return 0
} }
var priority int64 pp.setTempState(c)
if targetCap > c.capacity { if maxTarget > c.capacity {
priority = c.nodePriority.EstimatePriority(targetCap, 0, 0, bias, false) c.bias = bias
} else { c.stepDiv = pp.fineStepDiv
priority = c.nodePriority.Priority(targetCap)
} }
pp.markForChange(c) pp.setTempCapacity(c, maxTarget)
pp.setCapacity(c, targetCap) c.minTarget = minTarget
c.forced = true
pp.activeQueue.Remove(c.activeIndex) pp.activeQueue.Remove(c.activeIndex)
pp.inactiveQueue.Remove(c.inactiveIndex) pp.inactiveQueue.Remove(c.inactiveIndex)
pp.activeQueue.Push(c) pp.activeQueue.Push(c)
_, minPriority = pp.enforceLimits() pp.enforceLimits()
// if capacity update is possible now then minPriority == math.MinInt64 updates = pp.finalizeChanges(c.tempCapacity >= minTarget && c.tempCapacity <= maxTarget && c.tempCapacity != c.capacity)
// if it is not possible at all then minPriority == math.MaxInt64 return c.capacity
allowed = priority > minPriority
updates = pp.finalizeChanges(setCap && allowed)
return
} }
// SetLimits sets the maximum number and total capacity of simultaneously active nodes // SetLimits sets the maximum number and total capacity of simultaneously active nodes
func (pp *PriorityPool) SetLimits(maxCount, maxCap uint64) { func (pp *priorityPool) SetLimits(maxCount, maxCap uint64) {
pp.lock.Lock() pp.lock.Lock()
pp.activeQueue.Refresh() pp.activeQueue.Refresh()
var updates []capUpdate var updates []capUpdate
@ -247,27 +210,38 @@ func (pp *PriorityPool) SetLimits(maxCount, maxCap uint64) {
updates = pp.finalizeChanges(true) updates = pp.finalizeChanges(true)
} }
if inc { if inc {
updates = pp.tryActivate() updates = append(updates, pp.tryActivate(false)...)
} }
} }
// SetActiveBias sets the bias applied when trying to activate inactive nodes // setActiveBias sets the bias applied when trying to activate inactive nodes
func (pp *PriorityPool) SetActiveBias(bias time.Duration) { func (pp *priorityPool) setActiveBias(bias time.Duration) {
pp.lock.Lock() pp.lock.Lock()
defer pp.lock.Unlock()
pp.activeBias = bias pp.activeBias = bias
pp.tryActivate() if pp.activeBias < time.Duration(1) {
pp.activeBias = time.Duration(1)
}
updates := pp.tryActivate(false)
pp.lock.Unlock()
pp.ns.Operation(func() { pp.updateFlags(updates) })
} }
// Active returns the number and total capacity of currently active nodes // Active returns the number and total capacity of currently active nodes
func (pp *PriorityPool) Active() (uint64, uint64) { func (pp *priorityPool) Active() (uint64, uint64) {
pp.lock.Lock() pp.lock.Lock()
defer pp.lock.Unlock() defer pp.lock.Unlock()
return pp.activeCount, pp.activeCap return pp.activeCount, pp.activeCap
} }
// Limits returns the maximum allowed number and total capacity of active nodes
func (pp *priorityPool) Limits() (uint64, uint64) {
pp.lock.Lock()
defer pp.lock.Unlock()
return pp.maxCount, pp.maxCap
}
// inactiveSetIndex callback updates ppNodeInfo item index in inactiveQueue // inactiveSetIndex callback updates ppNodeInfo item index in inactiveQueue
func inactiveSetIndex(a interface{}, index int) { func inactiveSetIndex(a interface{}, index int) {
a.(*ppNodeInfo).inactiveIndex = index a.(*ppNodeInfo).inactiveIndex = index
@ -290,37 +264,31 @@ func invertPriority(p int64) int64 {
// activePriority callback returns actual priority of ppNodeInfo item in activeQueue // activePriority callback returns actual priority of ppNodeInfo item in activeQueue
func activePriority(a interface{}) int64 { func activePriority(a interface{}) int64 {
c := a.(*ppNodeInfo) c := a.(*ppNodeInfo)
if c.forced {
return math.MinInt64
}
if c.bias == 0 { if c.bias == 0 {
return invertPriority(c.nodePriority.Priority(c.capacity)) return invertPriority(c.nodePriority.priority(c.tempCapacity))
} else { } else {
return invertPriority(c.nodePriority.EstimatePriority(c.capacity, 0, 0, c.bias, true)) return invertPriority(c.nodePriority.estimatePriority(c.tempCapacity, 0, 0, c.bias, true))
} }
} }
// activeMaxPriority callback returns estimated maximum priority of ppNodeInfo item in activeQueue // activeMaxPriority callback returns estimated maximum priority of ppNodeInfo item in activeQueue
func (pp *PriorityPool) activeMaxPriority(a interface{}, until mclock.AbsTime) int64 { func (pp *priorityPool) activeMaxPriority(a interface{}, until mclock.AbsTime) int64 {
c := a.(*ppNodeInfo) c := a.(*ppNodeInfo)
if c.forced {
return math.MinInt64
}
future := time.Duration(until - pp.clock.Now()) future := time.Duration(until - pp.clock.Now())
if future < 0 { if future < 0 {
future = 0 future = 0
} }
return invertPriority(c.nodePriority.EstimatePriority(c.capacity, 0, future, c.bias, false)) return invertPriority(c.nodePriority.estimatePriority(c.tempCapacity, 0, future, c.bias, false))
} }
// inactivePriority callback returns actual priority of ppNodeInfo item in inactiveQueue // inactivePriority callback returns actual priority of ppNodeInfo item in inactiveQueue
func (pp *PriorityPool) inactivePriority(p *ppNodeInfo) int64 { func (pp *priorityPool) inactivePriority(p *ppNodeInfo) int64 {
return p.nodePriority.Priority(pp.minCap) return p.nodePriority.priority(pp.minCap)
} }
// connectedNode is called when a new node has been added to the pool (InactiveFlag set) // connectedNode is called when a new node has been added to the pool (inactiveFlag set)
// Note: this function should run inside a NodeStateMachine operation // Note: this function should run inside a NodeStateMachine operation
func (pp *PriorityPool) connectedNode(c *ppNodeInfo) { func (pp *priorityPool) connectedNode(c *ppNodeInfo) {
pp.lock.Lock() pp.lock.Lock()
pp.activeQueue.Refresh() pp.activeQueue.Refresh()
var updates []capUpdate var updates []capUpdate
@ -334,13 +302,13 @@ func (pp *PriorityPool) connectedNode(c *ppNodeInfo) {
} }
c.connected = true c.connected = true
pp.inactiveQueue.Push(c, pp.inactivePriority(c)) pp.inactiveQueue.Push(c, pp.inactivePriority(c))
updates = pp.tryActivate() updates = pp.tryActivate(false)
} }
// disconnectedNode is called when a node has been removed from the pool (both InactiveFlag // disconnectedNode is called when a node has been removed from the pool (both inactiveFlag
// and ActiveFlag reset) // and activeFlag reset)
// Note: this function should run inside a NodeStateMachine operation // Note: this function should run inside a NodeStateMachine operation
func (pp *PriorityPool) disconnectedNode(c *ppNodeInfo) { func (pp *priorityPool) disconnectedNode(c *ppNodeInfo) {
pp.lock.Lock() pp.lock.Lock()
pp.activeQueue.Refresh() pp.activeQueue.Refresh()
var updates []capUpdate var updates []capUpdate
@ -356,42 +324,51 @@ func (pp *PriorityPool) disconnectedNode(c *ppNodeInfo) {
pp.activeQueue.Remove(c.activeIndex) pp.activeQueue.Remove(c.activeIndex)
pp.inactiveQueue.Remove(c.inactiveIndex) pp.inactiveQueue.Remove(c.inactiveIndex)
if c.capacity != 0 { if c.capacity != 0 {
pp.setCapacity(c, 0) pp.setTempState(c)
updates = pp.tryActivate() pp.setTempCapacity(c, 0)
updates = pp.tryActivate(true)
} }
} }
// markForChange internally puts a node in a temporary state that can either be reverted // setTempState internally puts a node in a temporary state that can either be reverted
// or confirmed later. This temporary state allows changing the capacity of a node and // or confirmed later. This temporary state allows changing the capacity of a node and
// moving it between the active and inactive queue. ActiveFlag/InactiveFlag and // moving it between the active and inactive queue. activeFlag/inactiveFlag and
// CapacityField are not changed while the changes are still temporary. // capacityField are not changed while the changes are still temporary.
func (pp *PriorityPool) markForChange(c *ppNodeInfo) { func (pp *priorityPool) setTempState(c *ppNodeInfo) {
if c.changed { if c.tempState {
return return
} }
c.changed = true c.tempState = true
c.origCap = c.capacity if c.tempCapacity != c.capacity { // should never happen
pp.changed = append(pp.changed, c) log.Error("tempCapacity != capacity when entering tempState")
}
c.minTarget = pp.minCap
c.stepDiv = pp.capacityStepDiv
pp.tempState = append(pp.tempState, c)
} }
// setCapacity changes the capacity of a node and adjusts activeCap and activeCount // setTempCapacity changes the capacity of a node in the temporary state and adjusts
// accordingly. Note that this change is performed in the temporary state so it should // activeCap and activeCount accordingly. Since this change is performed in the temporary
// be called after markForChange and before finalizeChanges. // state it should be called after setTempState and before finalizeChanges.
func (pp *PriorityPool) setCapacity(n *ppNodeInfo, cap uint64) { func (pp *priorityPool) setTempCapacity(n *ppNodeInfo, cap uint64) {
pp.activeCap += cap - n.capacity if !n.tempState { // should never happen
if n.capacity == 0 { log.Error("Node is not in temporary state")
return
}
pp.activeCap += cap - n.tempCapacity
if n.tempCapacity == 0 {
pp.activeCount++ pp.activeCount++
} }
if cap == 0 { if cap == 0 {
pp.activeCount-- pp.activeCount--
} }
n.capacity = cap n.tempCapacity = cap
} }
// enforceLimits enforces active node count and total capacity limits. It returns the // enforceLimits enforces active node count and total capacity limits. It returns the
// lowest active node priority. Note that this function is performed on the temporary // lowest active node priority. Note that this function is performed on the temporary
// internal state. // internal state.
func (pp *PriorityPool) enforceLimits() (*ppNodeInfo, int64) { func (pp *priorityPool) enforceLimits() (*ppNodeInfo, int64) {
if pp.activeCap <= pp.maxCap && pp.activeCount <= pp.maxCount { if pp.activeCap <= pp.maxCap && pp.activeCount <= pp.maxCount {
return nil, math.MinInt64 return nil, math.MinInt64
} }
@ -401,16 +378,19 @@ func (pp *PriorityPool) enforceLimits() (*ppNodeInfo, int64) {
) )
pp.activeQueue.MultiPop(func(data interface{}, priority int64) bool { pp.activeQueue.MultiPop(func(data interface{}, priority int64) bool {
c = data.(*ppNodeInfo) c = data.(*ppNodeInfo)
pp.markForChange(c) pp.setTempState(c)
maxActivePriority = priority maxActivePriority = priority
if c.capacity == pp.minCap || pp.activeCount > pp.maxCount { if c.tempCapacity == c.minTarget || pp.activeCount > pp.maxCount {
pp.setCapacity(c, 0) pp.setTempCapacity(c, 0)
} else { } else {
sub := c.capacity / pp.capacityStepDiv sub := c.tempCapacity / c.stepDiv
if c.capacity-sub < pp.minCap { if sub == 0 {
sub = c.capacity - pp.minCap sub = 1
} }
pp.setCapacity(c, c.capacity-sub) if c.tempCapacity-sub < c.minTarget {
sub = c.tempCapacity - c.minTarget
}
pp.setTempCapacity(c, c.tempCapacity-sub)
pp.activeQueue.Push(c) pp.activeQueue.Push(c)
} }
return pp.activeCap > pp.maxCap || pp.activeCount > pp.maxCount return pp.activeCap > pp.maxCap || pp.activeCount > pp.maxCount
@ -421,71 +401,74 @@ func (pp *PriorityPool) enforceLimits() (*ppNodeInfo, int64) {
// finalizeChanges either commits or reverts temporary changes. The necessary capacity // finalizeChanges either commits or reverts temporary changes. The necessary capacity
// field and according flag updates are not performed here but returned in a list because // field and according flag updates are not performed here but returned in a list because
// they should be performed while the mutex is not held. // they should be performed while the mutex is not held.
func (pp *PriorityPool) finalizeChanges(commit bool) (updates []capUpdate) { func (pp *priorityPool) finalizeChanges(commit bool) (updates []capUpdate) {
for _, c := range pp.changed { for _, c := range pp.tempState {
// always remove and push back in order to update biased/forced priority // always remove and push back in order to update biased priority
pp.activeQueue.Remove(c.activeIndex) pp.activeQueue.Remove(c.activeIndex)
pp.inactiveQueue.Remove(c.inactiveIndex) pp.inactiveQueue.Remove(c.inactiveIndex)
c.bias = 0 oldCapacity := c.capacity
c.forced = false if commit {
c.changed = false c.capacity = c.tempCapacity
if !commit { } else {
pp.setCapacity(c, c.origCap) pp.setTempCapacity(c, c.capacity) // revert activeCount/activeCap
} }
c.tempState = false
c.bias = 0
c.stepDiv = pp.capacityStepDiv
c.minTarget = pp.minCap
if c.connected { if c.connected {
if c.capacity != 0 { if c.capacity != 0 {
pp.activeQueue.Push(c) pp.activeQueue.Push(c)
} else { } else {
pp.inactiveQueue.Push(c, pp.inactivePriority(c)) pp.inactiveQueue.Push(c, pp.inactivePriority(c))
} }
if c.capacity != c.origCap && commit { if c.capacity != oldCapacity {
updates = append(updates, capUpdate{c.node, c.origCap, c.capacity}) updates = append(updates, capUpdate{c.node, oldCapacity, c.capacity})
} }
} }
c.origCap = 0
} }
pp.changed = nil pp.tempState = nil
if commit { if commit {
pp.ccUpdateForced = true pp.ccUpdateForced = true
} }
return return
} }
// capUpdate describes a CapacityField and ActiveFlag/InactiveFlag update // capUpdate describes a capacityField and activeFlag/inactiveFlag update
type capUpdate struct { type capUpdate struct {
node *enode.Node node *enode.Node
oldCap, newCap uint64 oldCap, newCap uint64
} }
// updateFlags performs CapacityField and ActiveFlag/InactiveFlag updates while the // updateFlags performs capacityField and activeFlag/inactiveFlag updates while the
// pool mutex is not held // pool mutex is not held
// Note: this function should run inside a NodeStateMachine operation // Note: this function should run inside a NodeStateMachine operation
func (pp *PriorityPool) updateFlags(updates []capUpdate) { func (pp *priorityPool) updateFlags(updates []capUpdate) {
for _, f := range updates { for _, f := range updates {
if f.oldCap == 0 { if f.oldCap == 0 {
pp.ns.SetStateSub(f.node, pp.ActiveFlag, pp.InactiveFlag, 0) pp.ns.SetStateSub(f.node, pp.setup.activeFlag, pp.setup.inactiveFlag, 0)
} }
if f.newCap == 0 { if f.newCap == 0 {
pp.ns.SetStateSub(f.node, pp.InactiveFlag, pp.ActiveFlag, 0) pp.ns.SetStateSub(f.node, pp.setup.inactiveFlag, pp.setup.activeFlag, 0)
pp.ns.SetFieldSub(f.node, pp.CapacityField, nil) pp.ns.SetFieldSub(f.node, pp.setup.capacityField, nil)
} else { } else {
pp.ns.SetFieldSub(f.node, pp.CapacityField, f.newCap) pp.ns.SetFieldSub(f.node, pp.setup.capacityField, f.newCap)
} }
} }
} }
// tryActivate tries to activate inactive nodes if possible // tryActivate tries to activate inactive nodes if possible
func (pp *PriorityPool) tryActivate() []capUpdate { func (pp *priorityPool) tryActivate(commit bool) []capUpdate {
var commit bool
for pp.inactiveQueue.Size() > 0 { for pp.inactiveQueue.Size() > 0 {
c := pp.inactiveQueue.PopItem().(*ppNodeInfo) c := pp.inactiveQueue.PopItem().(*ppNodeInfo)
pp.markForChange(c) pp.setTempState(c)
pp.setCapacity(c, pp.minCap) pp.setTempCapacity(c, pp.minCap)
c.bias = pp.activeBias c.bias = pp.activeBias
pp.activeQueue.Push(c) pp.activeQueue.Push(c)
pp.enforceLimits() pp.enforceLimits()
if c.capacity > 0 { if c.tempCapacity > 0 {
commit = true commit = true
c.bias = 0
} else { } else {
break break
} }
@ -497,7 +480,7 @@ func (pp *PriorityPool) tryActivate() []capUpdate {
// updatePriority gets the current priority value of the given node from the nodePriority // updatePriority gets the current priority value of the given node from the nodePriority
// interface and performs the necessary changes. It is triggered by updateFlag. // interface and performs the necessary changes. It is triggered by updateFlag.
// Note: this function should run inside a NodeStateMachine operation // Note: this function should run inside a NodeStateMachine operation
func (pp *PriorityPool) updatePriority(node *enode.Node) { func (pp *priorityPool) updatePriority(node *enode.Node) {
pp.lock.Lock() pp.lock.Lock()
pp.activeQueue.Refresh() pp.activeQueue.Refresh()
var updates []capUpdate var updates []capUpdate
@ -506,7 +489,7 @@ func (pp *PriorityPool) updatePriority(node *enode.Node) {
pp.updateFlags(updates) pp.updateFlags(updates)
}() }()
c, _ := pp.ns.GetField(node, pp.ppNodeInfoField).(*ppNodeInfo) c, _ := pp.ns.GetField(node, pp.setup.queueField).(*ppNodeInfo)
if c == nil || !c.connected { if c == nil || !c.connected {
return return
} }
@ -517,15 +500,15 @@ func (pp *PriorityPool) updatePriority(node *enode.Node) {
} else { } else {
pp.inactiveQueue.Push(c, pp.inactivePriority(c)) pp.inactiveQueue.Push(c, pp.inactivePriority(c))
} }
updates = pp.tryActivate() updates = pp.tryActivate(false)
} }
// CapacityCurve is a snapshot of the priority pool contents in a format that can efficiently // capacityCurve is a snapshot of the priority pool contents in a format that can efficiently
// estimate how much capacity could be granted to a given node at a given priority level. // estimate how much capacity could be granted to a given node at a given priority level.
type CapacityCurve struct { type capacityCurve struct {
points []curvePoint // curve points sorted in descending order of priority points []curvePoint // curve points sorted in descending order of priority
index map[enode.ID][]int // curve point indexes belonging to each node index map[enode.ID][]int // curve point indexes belonging to each node
exclude []int // curve point indexes of excluded node excludeList []int // curve point indexes of excluded node
excludeFirst bool // true if activeCount == maxCount excludeFirst bool // true if activeCount == maxCount
} }
@ -534,8 +517,8 @@ type curvePoint struct {
nextPri int64 // next priority level where more capacity will be available nextPri int64 // next priority level where more capacity will be available
} }
// GetCapacityCurve returns a new or recently cached CapacityCurve based on the contents of the pool // getCapacityCurve returns a new or recently cached capacityCurve based on the contents of the pool
func (pp *PriorityPool) GetCapacityCurve() *CapacityCurve { func (pp *priorityPool) getCapacityCurve() *capacityCurve {
pp.lock.Lock() pp.lock.Lock()
defer pp.lock.Unlock() defer pp.lock.Unlock()
@ -547,7 +530,7 @@ func (pp *PriorityPool) GetCapacityCurve() *CapacityCurve {
pp.ccUpdateForced = false pp.ccUpdateForced = false
pp.ccUpdatedAt = now pp.ccUpdatedAt = now
curve := &CapacityCurve{ curve := &capacityCurve{
index: make(map[enode.ID][]int), index: make(map[enode.ID][]int),
} }
pp.cachedCurve = curve pp.cachedCurve = curve
@ -556,6 +539,7 @@ func (pp *PriorityPool) GetCapacityCurve() *CapacityCurve {
excludeFirst := pp.maxCount == pp.activeCount excludeFirst := pp.maxCount == pp.activeCount
// reduce node capacities or remove nodes until nothing is left in the queue; // reduce node capacities or remove nodes until nothing is left in the queue;
// record the available capacity and the necessary priority after each step // record the available capacity and the necessary priority after each step
lastPri := int64(math.MinInt64)
for pp.activeCap > 0 { for pp.activeCap > 0 {
cp := curvePoint{} cp := curvePoint{}
if pp.activeCap > pp.maxCap { if pp.activeCap > pp.maxCap {
@ -570,9 +554,15 @@ func (pp *PriorityPool) GetCapacityCurve() *CapacityCurve {
// enforceLimits removes the lowest priority node if it has minimal capacity, // enforceLimits removes the lowest priority node if it has minimal capacity,
// otherwise reduces its capacity // otherwise reduces its capacity
next, cp.nextPri = pp.enforceLimits() next, cp.nextPri = pp.enforceLimits()
if cp.nextPri < lastPri {
// enforce monotonicity which may be broken by continuously changing priorities
cp.nextPri = lastPri
} else {
lastPri = cp.nextPri
}
pp.activeCap -= tempCap pp.activeCap -= tempCap
if next == nil { if next == nil {
log.Error("GetCapacityCurve: cannot remove next element from the priority queue") log.Error("getCapacityCurve: cannot remove next element from the priority queue")
break break
} }
id := next.node.ID() id := next.node.ID()
@ -595,34 +585,34 @@ func (pp *PriorityPool) GetCapacityCurve() *CapacityCurve {
nextPri: math.MaxInt64, nextPri: math.MaxInt64,
}) })
if curve.excludeFirst { if curve.excludeFirst {
curve.exclude = curve.index[excludeID] curve.excludeList = curve.index[excludeID]
} }
return curve return curve
} }
// Exclude returns a CapacityCurve with the given node excluded from the original curve // exclude returns a capacityCurve with the given node excluded from the original curve
func (cc *CapacityCurve) Exclude(id enode.ID) *CapacityCurve { func (cc *capacityCurve) exclude(id enode.ID) *capacityCurve {
if exclude, ok := cc.index[id]; ok { if excludeList, ok := cc.index[id]; ok {
// return a new version of the curve (only one excluded node can be selected) // return a new version of the curve (only one excluded node can be selected)
// Note: if the first node was excluded by default (excludeFirst == true) then // Note: if the first node was excluded by default (excludeFirst == true) then
// we can forget about that and exclude the node with the given id instead. // we can forget about that and exclude the node with the given id instead.
return &CapacityCurve{ return &capacityCurve{
points: cc.points, points: cc.points,
index: cc.index, index: cc.index,
exclude: exclude, excludeList: excludeList,
} }
} }
return cc return cc
} }
func (cc *CapacityCurve) getPoint(i int) curvePoint { func (cc *capacityCurve) getPoint(i int) curvePoint {
cp := cc.points[i] cp := cc.points[i]
if i == 0 && cc.excludeFirst { if i == 0 && cc.excludeFirst {
cp.freeCap = 0 cp.freeCap = 0
return cp return cp
} }
for ii := len(cc.exclude) - 1; ii >= 0; ii-- { for ii := len(cc.excludeList) - 1; ii >= 0; ii-- {
ei := cc.exclude[ii] ei := cc.excludeList[ii]
if ei < i { if ei < i {
break break
} }
@ -632,11 +622,11 @@ func (cc *CapacityCurve) getPoint(i int) curvePoint {
return cp return cp
} }
// MaxCapacity calculates the maximum capacity available for a node with a given // maxCapacity calculates the maximum capacity available for a node with a given
// (monotonically decreasing) priority vs. capacity function. Note that if the requesting // (monotonically decreasing) priority vs. capacity function. Note that if the requesting
// node is already in the pool then it should be excluded from the curve in order to get // node is already in the pool then it should be excluded from the curve in order to get
// the correct result. // the correct result.
func (cc *CapacityCurve) MaxCapacity(priority func(cap uint64) int64) uint64 { func (cc *capacityCurve) maxCapacity(priority func(cap uint64) int64) uint64 {
min, max := 0, len(cc.points)-1 // the curve always has at least one point min, max := 0, len(cc.points)-1 // the curve always has at least one point
for min < max { for min < max {
mid := (min + max) / 2 mid := (min + max) / 2

View File

@ -28,18 +28,6 @@ import (
"github.com/ethereum/go-ethereum/p2p/nodestate" "github.com/ethereum/go-ethereum/p2p/nodestate"
) )
var (
testSetup = &nodestate.Setup{}
ppTestClientFlag = testSetup.NewFlag("ppTestClientFlag")
ppTestClientField = testSetup.NewField("ppTestClient", reflect.TypeOf(&ppTestClient{}))
ppUpdateFlag = testSetup.NewFlag("ppUpdateFlag")
ppTestSetup = NewPriorityPoolSetup(testSetup)
)
func init() {
ppTestSetup.Connect(ppTestClientField, ppUpdateFlag)
}
const ( const (
testCapacityStepDiv = 100 testCapacityStepDiv = 100
testCapacityToleranceDiv = 10 testCapacityToleranceDiv = 10
@ -51,25 +39,27 @@ type ppTestClient struct {
balance, cap uint64 balance, cap uint64
} }
func (c *ppTestClient) Priority(cap uint64) int64 { func (c *ppTestClient) priority(cap uint64) int64 {
return int64(c.balance / cap) return int64(c.balance / cap)
} }
func (c *ppTestClient) EstimatePriority(cap uint64, addBalance int64, future, bias time.Duration, update bool) int64 { func (c *ppTestClient) estimatePriority(cap uint64, addBalance int64, future, bias time.Duration, update bool) int64 {
return int64(c.balance / cap) return int64(c.balance / cap)
} }
func TestPriorityPool(t *testing.T) { func TestPriorityPool(t *testing.T) {
clock := &mclock.Simulated{} clock := &mclock.Simulated{}
ns := nodestate.NewNodeStateMachine(nil, nil, clock, testSetup) setup := newServerSetup()
setup.balanceField = setup.setup.NewField("ppTestClient", reflect.TypeOf(&ppTestClient{}))
ns := nodestate.NewNodeStateMachine(nil, nil, clock, setup.setup)
ns.SubscribeField(ppTestSetup.CapacityField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) { ns.SubscribeField(setup.capacityField, func(node *enode.Node, state nodestate.Flags, oldValue, newValue interface{}) {
if n := ns.GetField(node, ppTestSetup.priorityField); n != nil { if n := ns.GetField(node, setup.balanceField); n != nil {
c := n.(*ppTestClient) c := n.(*ppTestClient)
c.cap = newValue.(uint64) c.cap = newValue.(uint64)
} }
}) })
pp := NewPriorityPool(ns, ppTestSetup, clock, testMinCap, 0, testCapacityStepDiv) pp := newPriorityPool(ns, setup, clock, testMinCap, 0, testCapacityStepDiv, testCapacityStepDiv)
ns.Start() ns.Start()
pp.SetLimits(100, 1000000) pp.SetLimits(100, 1000000)
clients := make([]*ppTestClient, 100) clients := make([]*ppTestClient, 100)
@ -77,7 +67,8 @@ func TestPriorityPool(t *testing.T) {
for { for {
var ok bool var ok bool
ns.Operation(func() { ns.Operation(func() {
_, ok = pp.RequestCapacity(c.node, c.cap+c.cap/testCapacityStepDiv, 0, true) newCap := c.cap + c.cap/testCapacityStepDiv
ok = pp.requestCapacity(c.node, newCap, newCap, 0) == newCap
}) })
if !ok { if !ok {
return return
@ -101,9 +92,8 @@ func TestPriorityPool(t *testing.T) {
} }
sumBalance += c.balance sumBalance += c.balance
clients[i] = c clients[i] = c
ns.SetState(c.node, ppTestClientFlag, nodestate.Flags{}, 0) ns.SetField(c.node, setup.balanceField, c)
ns.SetField(c.node, ppTestSetup.priorityField, c) ns.SetState(c.node, setup.inactiveFlag, nodestate.Flags{}, 0)
ns.SetState(c.node, ppTestSetup.InactiveFlag, nodestate.Flags{}, 0)
raise(c) raise(c)
check(c) check(c)
} }
@ -113,8 +103,8 @@ func TestPriorityPool(t *testing.T) {
oldBalance := c.balance oldBalance := c.balance
c.balance = uint64(rand.Int63n(100000000000) + 100000000000) c.balance = uint64(rand.Int63n(100000000000) + 100000000000)
sumBalance += c.balance - oldBalance sumBalance += c.balance - oldBalance
pp.ns.SetState(c.node, ppUpdateFlag, nodestate.Flags{}, 0) pp.ns.SetState(c.node, setup.updateFlag, nodestate.Flags{}, 0)
pp.ns.SetState(c.node, nodestate.Flags{}, ppUpdateFlag, 0) pp.ns.SetState(c.node, nodestate.Flags{}, setup.updateFlag, 0)
if c.balance > oldBalance { if c.balance > oldBalance {
raise(c) raise(c)
} else { } else {
@ -129,32 +119,28 @@ func TestPriorityPool(t *testing.T) {
if count%10 == 0 { if count%10 == 0 {
// test available capacity calculation with capacity curve // test available capacity calculation with capacity curve
c = clients[rand.Intn(len(clients))] c = clients[rand.Intn(len(clients))]
curve := pp.GetCapacityCurve().Exclude(c.node.ID()) curve := pp.getCapacityCurve().exclude(c.node.ID())
add := uint64(rand.Int63n(10000000000000)) add := uint64(rand.Int63n(10000000000000))
c.balance += add c.balance += add
sumBalance += add sumBalance += add
expCap := curve.MaxCapacity(func(cap uint64) int64 { expCap := curve.maxCapacity(func(cap uint64) int64 {
return int64(c.balance / cap) return int64(c.balance / cap)
}) })
//fmt.Println(expCap, c.balance, sumBalance)
/*for i, cp := range curve.points {
fmt.Println("cp", i, cp, "ex", curve.getPoint(i))
}*/
var ok bool var ok bool
expFail := expCap + 1 expFail := expCap + 10
if expFail < testMinCap { if expFail < testMinCap {
expFail = testMinCap expFail = testMinCap
} }
ns.Operation(func() { ns.Operation(func() {
_, ok = pp.RequestCapacity(c.node, expFail, 0, true) ok = pp.requestCapacity(c.node, expFail, expFail, 0) == expFail
}) })
if ok { if ok {
t.Errorf("Request for more than expected available capacity succeeded") t.Errorf("Request for more than expected available capacity succeeded")
} }
if expCap >= testMinCap { if expCap >= testMinCap {
ns.Operation(func() { ns.Operation(func() {
_, ok = pp.RequestCapacity(c.node, expCap, 0, true) ok = pp.requestCapacity(c.node, expCap, expCap, 0) == expCap
}) })
if !ok { if !ok {
t.Errorf("Request for expected available capacity failed") t.Errorf("Request for expected available capacity failed")
@ -162,8 +148,8 @@ func TestPriorityPool(t *testing.T) {
} }
c.balance -= add c.balance -= add
sumBalance -= add sumBalance -= add
pp.ns.SetState(c.node, ppUpdateFlag, nodestate.Flags{}, 0) pp.ns.SetState(c.node, setup.updateFlag, nodestate.Flags{}, 0)
pp.ns.SetState(c.node, nodestate.Flags{}, ppUpdateFlag, 0) pp.ns.SetState(c.node, nodestate.Flags{}, setup.updateFlag, 0)
for _, c := range clients { for _, c := range clients {
raise(c) raise(c)
} }
@ -175,8 +161,11 @@ func TestPriorityPool(t *testing.T) {
func TestCapacityCurve(t *testing.T) { func TestCapacityCurve(t *testing.T) {
clock := &mclock.Simulated{} clock := &mclock.Simulated{}
ns := nodestate.NewNodeStateMachine(nil, nil, clock, testSetup) setup := newServerSetup()
pp := NewPriorityPool(ns, ppTestSetup, clock, 400000, 0, 2) setup.balanceField = setup.setup.NewField("ppTestClient", reflect.TypeOf(&ppTestClient{}))
ns := nodestate.NewNodeStateMachine(nil, nil, clock, setup.setup)
pp := newPriorityPool(ns, setup, clock, 400000, 0, 2, 2)
ns.Start() ns.Start()
pp.SetLimits(10, 10000000) pp.SetLimits(10, 10000000)
clients := make([]*ppTestClient, 10) clients := make([]*ppTestClient, 10)
@ -188,17 +177,16 @@ func TestCapacityCurve(t *testing.T) {
cap: 1000000, cap: 1000000,
} }
clients[i] = c clients[i] = c
ns.SetState(c.node, ppTestClientFlag, nodestate.Flags{}, 0) ns.SetField(c.node, setup.balanceField, c)
ns.SetField(c.node, ppTestSetup.priorityField, c) ns.SetState(c.node, setup.inactiveFlag, nodestate.Flags{}, 0)
ns.SetState(c.node, ppTestSetup.InactiveFlag, nodestate.Flags{}, 0)
ns.Operation(func() { ns.Operation(func() {
pp.RequestCapacity(c.node, c.cap, 0, true) pp.requestCapacity(c.node, c.cap, c.cap, 0)
}) })
} }
curve := pp.GetCapacityCurve() curve := pp.getCapacityCurve()
check := func(balance, expCap uint64) { check := func(balance, expCap uint64) {
cap := curve.MaxCapacity(func(cap uint64) int64 { cap := curve.maxCapacity(func(cap uint64) int64 {
return int64(balance / cap) return int64(balance / cap)
}) })
var fail bool var fail bool
@ -226,7 +214,7 @@ func TestCapacityCurve(t *testing.T) {
check(1000000000000, 2500000) check(1000000000000, 2500000)
pp.SetLimits(11, 10000000) pp.SetLimits(11, 10000000)
curve = pp.GetCapacityCurve() curve = pp.getCapacityCurve()
check(0, 0) check(0, 0)
check(10000000000, 100000) check(10000000000, 100000)

View File

@ -40,7 +40,6 @@ type (
// Service is a service registered at the Server and identified by a string id // Service is a service registered at the Server and identified by a string id
Service interface { Service interface {
ServiceInfo() (id, desc string) // only called during registration
Handle(id enode.ID, address string, name string, data []byte) []byte // never called concurrently Handle(id enode.ID, address string, name string, data []byte) []byte // never called concurrently
} }
@ -60,9 +59,8 @@ func NewServer(delayPerRequest time.Duration) *Server {
} }
// Register registers a Service // Register registers a Service
func (s *Server) Register(b Service) { func (s *Server) Register(b Service, id, desc string) {
srv := &serviceEntry{backend: b} srv := &serviceEntry{backend: b, id: id, desc: desc}
srv.id, srv.desc = b.ServiceInfo()
if strings.Contains(srv.id, ":") { if strings.Contains(srv.id, ":") {
// srv.id + ":" will be used as a service database prefix // srv.id + ":" will be used as a service database prefix
log.Error("Service ID contains ':'", "id", srv.id) log.Error("Service ID contains ':'", "id", srv.id)

View File

@ -0,0 +1,59 @@
// Copyright 2021 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 server
import (
"reflect"
"github.com/ethereum/go-ethereum/p2p/nodestate"
)
type peerWrapper struct{ clientPeer } // the NodeStateMachine type system needs this wrapper
// serverSetup is a wrapper of the node state machine setup, which contains
// all the created flags and fields used in the vflux server side.
type serverSetup struct {
setup *nodestate.Setup
clientField nodestate.Field // Field contains the client peer handler
// Flags and fields controlled by balance tracker. BalanceTracker
// is responsible for setting/deleting these flags or fields.
priorityFlag nodestate.Flags // Flag is set if the node has a positive balance
updateFlag nodestate.Flags // Flag is set whenever the node balance is changed(priority changed)
balanceField nodestate.Field // Field contains the client balance for priority calculation
// Flags and fields controlled by priority queue. Priority queue
// is responsible for setting/deleting these flags or fields.
activeFlag nodestate.Flags // Flag is set if the node is active
inactiveFlag nodestate.Flags // Flag is set if the node is inactive
capacityField nodestate.Field // Field contains the capacity of the node
queueField nodestate.Field // Field contains the infomration in the priority queue
}
// newServerSetup initializes the setup for state machine and returns the flags/fields group.
func newServerSetup() *serverSetup {
setup := &serverSetup{setup: &nodestate.Setup{}}
setup.clientField = setup.setup.NewField("client", reflect.TypeOf(peerWrapper{}))
setup.priorityFlag = setup.setup.NewFlag("priority")
setup.updateFlag = setup.setup.NewFlag("update")
setup.balanceField = setup.setup.NewField("balance", reflect.TypeOf(&nodeBalance{}))
setup.activeFlag = setup.setup.NewFlag("active")
setup.inactiveFlag = setup.setup.NewFlag("inactive")
setup.capacityField = setup.setup.NewField("capacity", reflect.TypeOf(uint64(0)))
setup.queueField = setup.setup.NewField("queue", reflect.TypeOf(&ppNodeInfo{}))
return setup
}

View File

@ -102,6 +102,7 @@ compile_fuzzer tests/fuzzers/stacktrie Fuzz fuzzStackTrie
compile_fuzzer tests/fuzzers/difficulty Fuzz fuzzDifficulty compile_fuzzer tests/fuzzers/difficulty Fuzz fuzzDifficulty
compile_fuzzer tests/fuzzers/abi Fuzz fuzzAbi compile_fuzzer tests/fuzzers/abi Fuzz fuzzAbi
compile_fuzzer tests/fuzzers/les Fuzz fuzzLes compile_fuzzer tests/fuzzers/les Fuzz fuzzLes
compile_fuzzer tests/fuzzers/vflux FuzzClientPool fuzzClientPool
compile_fuzzer tests/fuzzers/bls12381 FuzzG1Add fuzz_g1_add compile_fuzzer tests/fuzzers/bls12381 FuzzG1Add fuzz_g1_add
compile_fuzzer tests/fuzzers/bls12381 FuzzG1Mul fuzz_g1_mul compile_fuzzer tests/fuzzers/bls12381 FuzzG1Mul fuzz_g1_mul

View File

@ -858,6 +858,23 @@ func (ns *NodeStateMachine) GetField(n *enode.Node, field Field) interface{} {
return nil return nil
} }
// GetState retrieves the current state 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) GetState(n *enode.Node) Flags {
ns.lock.Lock()
defer ns.lock.Unlock()
ns.checkStarted()
if ns.closed {
return Flags{}
}
if _, node := ns.updateEnode(n); node != nil {
return Flags{mask: node.state, setup: ns.setup}
}
return Flags{}
}
// SetField sets the given field of the given node and blocks until the operation is finished // 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()

View File

@ -0,0 +1,289 @@
// Copyright 2021 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 vflux
import (
"bytes"
"encoding/binary"
"io"
"math"
"math/big"
"time"
"github.com/ethereum/go-ethereum/common/mclock"
"github.com/ethereum/go-ethereum/ethdb/memorydb"
"github.com/ethereum/go-ethereum/les/vflux"
vfs "github.com/ethereum/go-ethereum/les/vflux/server"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr"
"github.com/ethereum/go-ethereum/rlp"
)
type fuzzer struct {
peers [256]*clientPeer
disconnectList []*clientPeer
input io.Reader
exhausted bool
activeCount, activeCap uint64
maxCount, maxCap uint64
}
type clientPeer struct {
fuzzer *fuzzer
node *enode.Node
freeID string
timeout time.Duration
balance vfs.ConnectedBalance
capacity uint64
}
func (p *clientPeer) Node() *enode.Node {
return p.node
}
func (p *clientPeer) FreeClientId() string {
return p.freeID
}
func (p *clientPeer) InactiveAllowance() time.Duration {
return p.timeout
}
func (p *clientPeer) UpdateCapacity(newCap uint64, requested bool) {
p.fuzzer.activeCap -= p.capacity
if p.capacity != 0 {
p.fuzzer.activeCount--
}
p.capacity = newCap
p.fuzzer.activeCap += p.capacity
if p.capacity != 0 {
p.fuzzer.activeCount++
}
}
func (p *clientPeer) Disconnect() {
p.fuzzer.disconnectList = append(p.fuzzer.disconnectList, p)
p.fuzzer.activeCap -= p.capacity
if p.capacity != 0 {
p.fuzzer.activeCount--
}
p.capacity = 0
p.balance = nil
}
func newFuzzer(input []byte) *fuzzer {
f := &fuzzer{
input: bytes.NewReader(input),
}
for i := range f.peers {
f.peers[i] = &clientPeer{
fuzzer: f,
node: enode.SignNull(new(enr.Record), enode.ID{byte(i)}),
freeID: string([]byte{byte(i)}),
timeout: f.randomDelay(),
}
}
return f
}
func (f *fuzzer) read(size int) []byte {
out := make([]byte, size)
if _, err := f.input.Read(out); err != nil {
f.exhausted = true
}
return out
}
func (f *fuzzer) randomByte() byte {
d := f.read(1)
return d[0]
}
func (f *fuzzer) randomBool() bool {
d := f.read(1)
return d[0]&1 == 1
}
func (f *fuzzer) randomInt(max int) int {
if max == 0 {
return 0
}
if max <= 256 {
return int(f.randomByte()) % max
}
var a uint16
if err := binary.Read(f.input, binary.LittleEndian, &a); err != nil {
f.exhausted = true
}
return int(a % uint16(max))
}
func (f *fuzzer) randomTokenAmount(signed bool) int64 {
x := uint64(f.randomInt(65000))
x = x * x * x * x
if signed && (x&1) == 1 {
if x <= math.MaxInt64 {
return -int64(x)
}
return math.MinInt64
}
if x <= math.MaxInt64 {
return int64(x)
}
return math.MaxInt64
}
func (f *fuzzer) randomDelay() time.Duration {
delay := f.randomByte()
if delay < 128 {
return time.Duration(delay) * time.Second
}
return 0
}
func (f *fuzzer) randomFactors() vfs.PriceFactors {
return vfs.PriceFactors{
TimeFactor: float64(f.randomByte()) / 25500,
CapacityFactor: float64(f.randomByte()) / 255,
RequestFactor: float64(f.randomByte()) / 255,
}
}
func (f *fuzzer) connectedBalanceOp(balance vfs.ConnectedBalance) {
switch f.randomInt(3) {
case 0:
balance.RequestServed(uint64(f.randomTokenAmount(false)))
case 1:
balance.SetPriceFactors(f.randomFactors(), f.randomFactors())
case 2:
balance.GetBalance()
balance.GetRawBalance()
balance.GetPriceFactors()
}
}
func (f *fuzzer) atomicBalanceOp(balance vfs.AtomicBalanceOperator) {
switch f.randomInt(3) {
case 0:
balance.AddBalance(f.randomTokenAmount(true))
case 1:
balance.SetBalance(uint64(f.randomTokenAmount(false)), uint64(f.randomTokenAmount(false)))
case 2:
balance.GetBalance()
balance.GetRawBalance()
balance.GetPriceFactors()
}
}
func FuzzClientPool(input []byte) int {
if len(input) > 10000 {
return -1
}
f := newFuzzer(input)
if f.exhausted {
return 0
}
clock := &mclock.Simulated{}
db := memorydb.New()
pool := vfs.NewClientPool(db, 10, f.randomDelay(), clock, func() bool { return true })
pool.Start()
defer pool.Stop()
count := 0
for !f.exhausted && count < 1000 {
count++
switch f.randomInt(11) {
case 0:
i := int(f.randomByte())
f.peers[i].balance = pool.Register(f.peers[i])
case 1:
i := int(f.randomByte())
f.peers[i].Disconnect()
case 2:
f.maxCount = uint64(f.randomByte())
f.maxCap = uint64(f.randomByte())
f.maxCap *= f.maxCap
pool.SetLimits(f.maxCount, f.maxCap)
case 3:
pool.SetConnectedBias(f.randomDelay())
case 4:
pool.SetDefaultFactors(f.randomFactors(), f.randomFactors())
case 5:
pool.SetExpirationTCs(uint64(f.randomInt(50000)), uint64(f.randomInt(50000)))
case 6:
if _, err := pool.SetCapacity(f.peers[f.randomByte()].node, uint64(f.randomByte()), f.randomDelay(), f.randomBool()); err == vfs.ErrCantFindMaximum {
panic(nil)
}
case 7:
if balance := f.peers[f.randomByte()].balance; balance != nil {
f.connectedBalanceOp(balance)
}
case 8:
pool.BalanceOperation(f.peers[f.randomByte()].node.ID(), f.peers[f.randomByte()].freeID, func(balance vfs.AtomicBalanceOperator) {
count := f.randomInt(4)
for i := 0; i < count; i++ {
f.atomicBalanceOp(balance)
}
})
case 9:
pool.TotalTokenAmount()
pool.GetExpirationTCs()
pool.Active()
pool.Limits()
pool.GetPosBalanceIDs(f.peers[f.randomByte()].node.ID(), f.peers[f.randomByte()].node.ID(), f.randomInt(100))
case 10:
req := vflux.CapacityQueryReq{
Bias: uint64(f.randomByte()),
AddTokens: make([]vflux.IntOrInf, f.randomInt(vflux.CapacityQueryMaxLen+1)),
}
for i := range req.AddTokens {
v := vflux.IntOrInf{Type: uint8(f.randomInt(4))}
if v.Type < 2 {
v.Value = *big.NewInt(f.randomTokenAmount(false))
}
req.AddTokens[i] = v
}
reqEnc, err := rlp.EncodeToBytes(&req)
if err != nil {
panic(err)
}
p := int(f.randomByte())
if p < len(reqEnc) {
reqEnc[p] = f.randomByte()
}
pool.Handle(f.peers[f.randomByte()].node.ID(), f.peers[f.randomByte()].freeID, vflux.CapacityQueryName, reqEnc)
}
for _, peer := range f.disconnectList {
pool.Unregister(peer)
}
f.disconnectList = nil
if d := f.randomDelay(); d > 0 {
clock.Run(d)
}
//fmt.Println(f.activeCount, f.maxCount, f.activeCap, f.maxCap)
if activeCount, activeCap := pool.Active(); activeCount != f.activeCount || activeCap != f.activeCap {
panic(nil)
}
if f.activeCount > f.maxCount || f.activeCap > f.maxCap {
panic(nil)
}
}
return 0
}

View File

@ -0,0 +1,41 @@
// 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 main
import (
"fmt"
"io/ioutil"
"os"
"github.com/ethereum/go-ethereum/tests/fuzzers/vflux"
)
func main() {
if len(os.Args) != 2 {
fmt.Fprintf(os.Stderr, "Usage: debug <file>\n")
fmt.Fprintf(os.Stderr, "Example\n")
fmt.Fprintf(os.Stderr, " $ debug ../crashers/4bbef6857c733a87ecf6fd8b9e7238f65eb9862a\n")
os.Exit(1)
}
crasher := os.Args[1]
data, err := ioutil.ReadFile(crasher)
if err != nil {
fmt.Fprintf(os.Stderr, "error loading crasher %v: %v", crasher, err)
os.Exit(1)
}
vflux.FuzzClientPool(data)
}