Snap accounts bulk import preparer (#1183)
* Provided common scheduler API, applied to `full` sync * Use hexary trie as storage for proofs_db records also: + Store metadata with account for keeping track of account state + add iterator over accounts * Common scheduler API applied to `snap` sync * Prepare for accounts bulk import details: + Added some ad-hoc checks for proving accounts data received from the snap/1 (will be replaced by proper database version when ready) + Added code that dumps some of the received snap/1 data into a file (turned of by default, see `worker_desc.nim`)
This commit is contained in:
parent
71f9e37482
commit
5f0e89a41e
|
@ -15,19 +15,12 @@ type
|
||||||
terminalHash
|
terminalHash
|
||||||
safeHash
|
safeHash
|
||||||
finalizedHash
|
finalizedHash
|
||||||
snapSyncStatus
|
|
||||||
snapSyncAccount
|
|
||||||
snapSyncProof
|
|
||||||
|
|
||||||
DbKey* = object
|
DbKey* = object
|
||||||
# The first byte stores the key type. The rest are key-specific values
|
# The first byte stores the key type. The rest are key-specific values
|
||||||
data*: array[33, byte]
|
data*: array[33, byte]
|
||||||
dataEndPos*: uint8 # the last populated position in the data
|
dataEndPos*: uint8 # the last populated position in the data
|
||||||
|
|
||||||
DbXKey* = object
|
|
||||||
data*: array[65, byte]
|
|
||||||
dataEndPos*: uint8 # the last populated position in the data
|
|
||||||
|
|
||||||
proc genericHashKey*(h: Hash256): DbKey {.inline.} =
|
proc genericHashKey*(h: Hash256): DbKey {.inline.} =
|
||||||
result.data[0] = byte ord(genericHash)
|
result.data[0] = byte ord(genericHash)
|
||||||
result.data[1 .. 32] = h.data
|
result.data[1 .. 32] = h.data
|
||||||
|
@ -86,28 +79,9 @@ proc finalizedHashKey*(): DbKey {.inline.} =
|
||||||
result.data[0] = byte ord(finalizedHash)
|
result.data[0] = byte ord(finalizedHash)
|
||||||
result.dataEndPos = uint8 1
|
result.dataEndPos = uint8 1
|
||||||
|
|
||||||
proc snapSyncStatusKey*(h: Hash256): DbKey =
|
template toOpenArray*(k: DbKey): openArray[byte] =
|
||||||
result.data[0] = byte ord(snapSyncStatus)
|
|
||||||
result.data[1 .. 32] = h.data
|
|
||||||
result.dataEndPos = uint8 32
|
|
||||||
|
|
||||||
proc snapSyncAccountKey*(h, b: Hash256): DbXKey =
|
|
||||||
result.data[0] = byte ord(snapSyncAccount)
|
|
||||||
result.data[1 .. 32] = h.data
|
|
||||||
result.data[33 .. 64] = b.data
|
|
||||||
result.dataEndPos = uint8 64
|
|
||||||
|
|
||||||
proc snapSyncProofKey*(h: Hash256): DbKey =
|
|
||||||
result.data[0] = byte ord(snapSyncProof)
|
|
||||||
result.data[1 .. 32] = h.data
|
|
||||||
result.dataEndPos = uint8 32
|
|
||||||
|
|
||||||
template toOpenArray*(k: DbKey|DbXKey): openArray[byte] =
|
|
||||||
k.data.toOpenArray(0, int(k.dataEndPos))
|
k.data.toOpenArray(0, int(k.dataEndPos))
|
||||||
|
|
||||||
proc hash*(k: DbKey|DbXKey): Hash =
|
|
||||||
result = hash(k.toOpenArray)
|
|
||||||
|
|
||||||
proc `==`*(a, b: DbKey): bool {.inline.} =
|
proc `==`*(a, b: DbKey): bool {.inline.} =
|
||||||
a.toOpenArray == b.toOpenArray
|
a.toOpenArray == b.toOpenArray
|
||||||
|
|
||||||
|
|
|
@ -148,7 +148,8 @@ proc setupP2P(nimbus: NimbusNode, conf: NimbusConf,
|
||||||
of SyncMode.Full:
|
of SyncMode.Full:
|
||||||
FullSyncRef.init(nimbus.ethNode, conf.maxPeers, tickerOK).start
|
FullSyncRef.init(nimbus.ethNode, conf.maxPeers, tickerOK).start
|
||||||
of SyncMode.Snap:
|
of SyncMode.Snap:
|
||||||
SnapSyncRef.init(nimbus.ethNode, conf.maxPeers).start
|
SnapSyncRef.init(nimbus.ethNode, nimbus.chainRef, nimbus.ctx.rng,
|
||||||
|
conf.maxPeers, tickerOK).start
|
||||||
of SyncMode.Default:
|
of SyncMode.Default:
|
||||||
discard
|
discard
|
||||||
|
|
||||||
|
|
|
@ -1,5 +1,4 @@
|
||||||
# Nimbus - New sync approach - A fusion of snap, trie, beam and other methods
|
# Nimbus
|
||||||
#
|
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
# Licensed under either of
|
# Licensed under either of
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
@ -10,13 +9,11 @@
|
||||||
# except according to those terms.
|
# except according to those terms.
|
||||||
|
|
||||||
import
|
import
|
||||||
std/hashes,
|
eth/[common/eth_types, p2p],
|
||||||
chronicles,
|
chronicles,
|
||||||
chronos,
|
chronos,
|
||||||
eth/[common/eth_types, p2p, p2p/peer_pool, p2p/private/p2p_types],
|
stew/[interval_set, sorted_set],
|
||||||
stew/keyed_queue,
|
"."/[full/worker, sync_desc, sync_sched, protocol]
|
||||||
./protocol,
|
|
||||||
./full/[full_desc, worker]
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
@ -24,153 +21,32 @@ logScope:
|
||||||
topics = "full-sync"
|
topics = "full-sync"
|
||||||
|
|
||||||
type
|
type
|
||||||
ActiveBuddies = ##\
|
FullSyncRef* = RunnerSyncRef[CtxData,BuddyData]
|
||||||
## List of active workers
|
|
||||||
KeyedQueue[Peer,BuddyRef]
|
|
||||||
|
|
||||||
FullSyncRef* = ref object of CtxRef
|
|
||||||
pool: PeerPool ## for starting the system
|
|
||||||
buddies: ActiveBuddies ## LRU cache with worker descriptors
|
|
||||||
tickerOk: bool ## Ticker logger
|
|
||||||
singleRunLock: bool ## For worker initialisation
|
|
||||||
monitorLock: bool ## For worker monitor
|
|
||||||
activeMulti: int ## Activated runners
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private helpers
|
# Virtual methods/interface, `mixin` functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc nsCtx(buddy: BuddyRef): FullSyncRef =
|
proc runSetup(ctx: FullCtxRef; ticker: bool): bool =
|
||||||
buddy.ctx.FullSyncRef
|
worker.setup(ctx,ticker)
|
||||||
|
|
||||||
proc hash(peer: Peer): Hash =
|
proc runRelease(ctx: FullCtxRef) =
|
||||||
## Needed for `buddies` table key comparison
|
worker.release(ctx)
|
||||||
hash(peer.remote.id)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
proc runStart(buddy: FullBuddyRef): bool =
|
||||||
# Private functions
|
worker.start(buddy)
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc workerLoop(buddy: BuddyRef) {.async.} =
|
proc runStop(buddy: FullBuddyRef) =
|
||||||
let
|
worker.stop(buddy)
|
||||||
ctx = buddy.nsCtx
|
|
||||||
peer = buddy.peer
|
|
||||||
trace "Starting peer worker", peer,
|
|
||||||
peers=ctx.pool.len, workers=ctx.buddies.len, maxWorkers=ctx.buddiesMax
|
|
||||||
|
|
||||||
# Continue until stopped
|
proc runPool(buddy: FullBuddyRef) =
|
||||||
while not buddy.ctrl.stopped:
|
worker.runPool(buddy)
|
||||||
if ctx.monitorLock:
|
|
||||||
await sleepAsync(500.milliseconds)
|
|
||||||
continue
|
|
||||||
|
|
||||||
# Rotate connection table so the most used entry is at the top/right
|
proc runSingle(buddy: FullBuddyRef) {.async.} =
|
||||||
# end. So zombies will implicitely be pushed left.
|
await worker.runSingle(buddy)
|
||||||
discard ctx.buddies.lruFetch(peer)
|
|
||||||
|
|
||||||
# Invoke `runPool()` over all buddies if requested
|
proc runMulti(buddy: FullBuddyRef) {.async.} =
|
||||||
if ctx.poolMode:
|
await worker.runMulti(buddy)
|
||||||
# Grab `monitorLock` (was `false` as checked above) and wait until clear
|
|
||||||
# to run as the only activated instance.
|
|
||||||
ctx.monitorLock = true
|
|
||||||
while 0 < ctx.activeMulti:
|
|
||||||
await sleepAsync(500.milliseconds)
|
|
||||||
while ctx.singleRunLock:
|
|
||||||
await sleepAsync(500.milliseconds)
|
|
||||||
trace "Starting pool mode for repair & recovery"
|
|
||||||
for w in ctx.buddies.nextValues:
|
|
||||||
buddy.runPool()
|
|
||||||
trace "Pool mode done"
|
|
||||||
ctx.monitorLock = false
|
|
||||||
continue
|
|
||||||
|
|
||||||
await sleepAsync(50.milliseconds)
|
|
||||||
|
|
||||||
# Multi mode
|
|
||||||
if buddy.ctrl.multiOk:
|
|
||||||
if not ctx.singleRunLock:
|
|
||||||
ctx.activeMulti.inc
|
|
||||||
# Continue doing something, work a bit
|
|
||||||
await buddy.runMulti()
|
|
||||||
ctx.activeMulti.dec
|
|
||||||
continue
|
|
||||||
|
|
||||||
# Single mode as requested. The `multiOk` flag for this worker was just
|
|
||||||
# found `false` in the pervious clause.
|
|
||||||
if not ctx.singleRunLock:
|
|
||||||
# Lock single instance mode and wait for other workers to finish
|
|
||||||
ctx.singleRunLock = true
|
|
||||||
while 0 < ctx.activeMulti:
|
|
||||||
await sleepAsync(500.milliseconds)
|
|
||||||
# Run single instance and release afterwards
|
|
||||||
await buddy.runSingle()
|
|
||||||
ctx.singleRunLock = false
|
|
||||||
|
|
||||||
# End while
|
|
||||||
|
|
||||||
buddy.stop()
|
|
||||||
|
|
||||||
trace "Peer worker done", peer, ctrlState=buddy.ctrl.state,
|
|
||||||
peers=ctx.pool.len, workers=ctx.buddies.len, maxWorkers=ctx.buddiesMax
|
|
||||||
|
|
||||||
|
|
||||||
proc onPeerConnected(ctx: FullSyncRef; peer: Peer) =
|
|
||||||
# Check for known entry (which should not exist.)
|
|
||||||
if ctx.buddies.hasKey(peer):
|
|
||||||
trace "Reconnecting zombie peer rejected", peer,
|
|
||||||
peers=ctx.pool.len, workers=ctx.buddies.len, maxWorkers=ctx.buddiesMax
|
|
||||||
return
|
|
||||||
|
|
||||||
# Initialise worker for this peer
|
|
||||||
let buddy = BuddyRef(ctx: ctx, peer: peer)
|
|
||||||
if not buddy.start():
|
|
||||||
trace "Ignoring useless peer", peer,
|
|
||||||
peers=ctx.pool.len, workers=ctx.buddies.len, maxWorkers=ctx.buddiesMax
|
|
||||||
buddy.ctrl.zombie = true
|
|
||||||
return
|
|
||||||
|
|
||||||
# Check for table overflow. An overflow might happen if there are zombies
|
|
||||||
# in the table (though preventing them from re-connecting for a while.)
|
|
||||||
if ctx.buddiesMax <= ctx.buddies.len:
|
|
||||||
let leastPeer = ctx.buddies.shift.value.data
|
|
||||||
if leastPeer.ctrl.zombie:
|
|
||||||
trace "Dequeuing zombie peer", leastPeer,
|
|
||||||
peers=ctx.pool.len, workers=ctx.buddies.len, maxWorkers=ctx.buddiesMax
|
|
||||||
discard
|
|
||||||
else:
|
|
||||||
# This could happen if there are idle entries in the table, i.e.
|
|
||||||
# somehow hanging runners.
|
|
||||||
trace "Peer table full! Dequeuing least used entry", leastPeer,
|
|
||||||
peers=ctx.pool.len, workers=ctx.buddies.len, maxWorkers=ctx.buddiesMax
|
|
||||||
leastPeer.stop()
|
|
||||||
leastPeer.ctrl.zombie = true
|
|
||||||
|
|
||||||
# Add peer entry
|
|
||||||
discard ctx.buddies.lruAppend(peer, buddy, ctx.buddiesMax)
|
|
||||||
|
|
||||||
# Run worker
|
|
||||||
asyncSpawn buddy.workerLoop()
|
|
||||||
|
|
||||||
|
|
||||||
proc onPeerDisconnected(ctx: FullSyncRef, peer: Peer) =
|
|
||||||
let
|
|
||||||
peers = ctx.pool.len
|
|
||||||
maxWorkers = ctx.buddiesMax
|
|
||||||
rc = ctx.buddies.eq(peer)
|
|
||||||
if rc.isErr:
|
|
||||||
debug "Disconnected from unregistered peer", peer, peers,
|
|
||||||
workers=ctx.buddies.len, maxWorkers
|
|
||||||
return
|
|
||||||
if rc.value.ctrl.zombie:
|
|
||||||
# Don't disconnect, leave them fall out of the LRU cache. The effect is,
|
|
||||||
# that reconnecting might be blocked, for a while.
|
|
||||||
trace "Disconnected zombie", peer, peers,
|
|
||||||
workers=ctx.buddies.len, maxWorkers
|
|
||||||
else:
|
|
||||||
rc.value.ctrl.stopped = true # in case it is hanging somewhere
|
|
||||||
ctx.buddies.del(peer)
|
|
||||||
trace "Disconnected buddy", peer, peers,
|
|
||||||
workers=ctx.buddies.len, maxWorkers
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public functions
|
# Public functions
|
||||||
|
@ -180,38 +56,15 @@ proc init*(
|
||||||
T: type FullSyncRef;
|
T: type FullSyncRef;
|
||||||
ethNode: EthereumNode;
|
ethNode: EthereumNode;
|
||||||
maxPeers: int;
|
maxPeers: int;
|
||||||
enableTicker: bool): T =
|
enableTicker = false): T =
|
||||||
## Constructor
|
new result
|
||||||
# Leave one extra slot so that it can holds a *zombie* even if all slots
|
result.initSync(ethNode, maxPeers, enableTicker)
|
||||||
# are full. The effect is that a re-connect on the latest zombie will be
|
|
||||||
# rejected as long as its worker descriptor is registered.
|
|
||||||
let lruSize = max(1,maxPeers+1)
|
|
||||||
result = T(
|
|
||||||
buddiesMax: lruSize,
|
|
||||||
chain: ethNode.chain,
|
|
||||||
pool: ethNode.peerPool,
|
|
||||||
tickerOk: enableTicker)
|
|
||||||
result.buddies.init(lruSize)
|
|
||||||
|
|
||||||
proc start*(ctx: FullSyncRef) =
|
proc start*(ctx: FullSyncRef) =
|
||||||
## Set up syncing. This call should come early.
|
doAssert ctx.startSync()
|
||||||
var po = PeerObserver(
|
|
||||||
onPeerConnected:
|
|
||||||
proc(p: Peer) {.gcsafe.} =
|
|
||||||
ctx.onPeerConnected(p),
|
|
||||||
onPeerDisconnected:
|
|
||||||
proc(p: Peer) {.gcsafe.} =
|
|
||||||
ctx.onPeerDisconnected(p))
|
|
||||||
|
|
||||||
# Initialise sub-systems
|
|
||||||
doAssert ctx.workerSetup(ctx.tickerOk)
|
|
||||||
po.setProtocol eth
|
|
||||||
ctx.pool.addObserver(ctx, po)
|
|
||||||
|
|
||||||
proc stop*(ctx: FullSyncRef) =
|
proc stop*(ctx: FullSyncRef) =
|
||||||
## Stop syncing
|
ctx.stopSync()
|
||||||
ctx.pool.delObserver(ctx)
|
|
||||||
ctx.workerRelease()
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# End
|
# End
|
||||||
|
|
|
@ -1,5 +1,4 @@
|
||||||
# Nimbus - Fetch account and storage states from peers efficiently
|
# Nimbus
|
||||||
#
|
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
# Licensed under either of
|
# Licensed under either of
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
@ -15,7 +14,7 @@ import
|
||||||
eth/[common/eth_types, p2p],
|
eth/[common/eth_types, p2p],
|
||||||
stint,
|
stint,
|
||||||
../../utils/prettify,
|
../../utils/prettify,
|
||||||
".."/[timer_helper, types]
|
../timer_helper
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
@ -33,7 +32,7 @@ type
|
||||||
TickerStatsUpdater* =
|
TickerStatsUpdater* =
|
||||||
proc: TickerStats {.gcsafe, raises: [Defect].}
|
proc: TickerStats {.gcsafe, raises: [Defect].}
|
||||||
|
|
||||||
Ticker* = ref object
|
TickerRef* = ref object
|
||||||
nBuddies: int
|
nBuddies: int
|
||||||
lastStats: TickerStats
|
lastStats: TickerStats
|
||||||
lastTick: uint64
|
lastTick: uint64
|
||||||
|
@ -56,9 +55,9 @@ proc pp(n: BlockNumber): string =
|
||||||
proc pp(n: Option[BlockNumber]): string =
|
proc pp(n: Option[BlockNumber]): string =
|
||||||
if n.isNone: "n/a" else: n.get.pp
|
if n.isNone: "n/a" else: n.get.pp
|
||||||
|
|
||||||
proc setLogTicker(t: Ticker; at: Moment) {.gcsafe.}
|
proc setLogTicker(t: TickerRef; at: Moment) {.gcsafe.}
|
||||||
|
|
||||||
proc runLogTicker(t: Ticker) {.gcsafe.} =
|
proc runLogTicker(t: TickerRef) {.gcsafe.} =
|
||||||
let data = t.statsCb()
|
let data = t.statsCb()
|
||||||
|
|
||||||
if data != t.lastStats or
|
if data != t.lastStats or
|
||||||
|
@ -83,7 +82,7 @@ proc runLogTicker(t: Ticker) {.gcsafe.} =
|
||||||
t.setLogTicker(Moment.fromNow(tickerLogInterval))
|
t.setLogTicker(Moment.fromNow(tickerLogInterval))
|
||||||
|
|
||||||
|
|
||||||
proc setLogTicker(t: Ticker; at: Moment) =
|
proc setLogTicker(t: TickerRef; at: Moment) =
|
||||||
if not t.logTicker.isNil:
|
if not t.logTicker.isNil:
|
||||||
t.logTicker = safeSetTimer(at, runLogTicker, t)
|
t.logTicker = safeSetTimer(at, runLogTicker, t)
|
||||||
|
|
||||||
|
@ -91,16 +90,16 @@ proc setLogTicker(t: Ticker; at: Moment) =
|
||||||
# Public constructor and start/stop functions
|
# Public constructor and start/stop functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc init*(T: type Ticker; cb: TickerStatsUpdater): T =
|
proc init*(T: type TickerRef; cb: TickerStatsUpdater): T =
|
||||||
## Constructor
|
## Constructor
|
||||||
T(statsCb: cb)
|
T(statsCb: cb)
|
||||||
|
|
||||||
proc start*(t: Ticker) =
|
proc start*(t: TickerRef) =
|
||||||
## Re/start ticker unconditionally
|
## Re/start ticker unconditionally
|
||||||
#debug "Started ticker"
|
#debug "Started ticker"
|
||||||
t.logTicker = safeSetTimer(Moment.fromNow(tickerStartDelay), runLogTicker, t)
|
t.logTicker = safeSetTimer(Moment.fromNow(tickerStartDelay), runLogTicker, t)
|
||||||
|
|
||||||
proc stop*(t: Ticker) =
|
proc stop*(t: TickerRef) =
|
||||||
## Stop ticker unconditionally
|
## Stop ticker unconditionally
|
||||||
t.logTicker = nil
|
t.logTicker = nil
|
||||||
#debug "Stopped ticker"
|
#debug "Stopped ticker"
|
||||||
|
@ -109,7 +108,7 @@ proc stop*(t: Ticker) =
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc startBuddy*(t: Ticker) =
|
proc startBuddy*(t: TickerRef) =
|
||||||
## Increment buddies counter and start ticker unless running.
|
## Increment buddies counter and start ticker unless running.
|
||||||
if t.nBuddies <= 0:
|
if t.nBuddies <= 0:
|
||||||
t.nBuddies = 1
|
t.nBuddies = 1
|
||||||
|
@ -117,7 +116,7 @@ proc startBuddy*(t: Ticker) =
|
||||||
else:
|
else:
|
||||||
t.nBuddies.inc
|
t.nBuddies.inc
|
||||||
|
|
||||||
proc stopBuddy*(t: Ticker) =
|
proc stopBuddy*(t: TickerRef) =
|
||||||
## Decrement buddies counter and stop ticker if there are no more registered
|
## Decrement buddies counter and stop ticker if there are no more registered
|
||||||
## buddies.
|
## buddies.
|
||||||
t.nBuddies.dec
|
t.nBuddies.dec
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
# nim-eth
|
# Nimbus
|
||||||
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
||||||
# Licensed and distributed under either of
|
# Licensed and distributed under either of
|
||||||
# * MIT license (license terms in the root directory or at
|
# * MIT license (license terms in the root directory or at
|
||||||
|
@ -62,9 +62,9 @@ import
|
||||||
chronos,
|
chronos,
|
||||||
eth/[common/eth_types, p2p],
|
eth/[common/eth_types, p2p],
|
||||||
stew/[byteutils, interval_set, sorted_set],
|
stew/[byteutils, interval_set, sorted_set],
|
||||||
../../utils,
|
"../.."/[db/db_chain, utils],
|
||||||
../protocol,
|
".."/[protocol, sync_desc],
|
||||||
"."/[full_desc, ticker]
|
./ticker
|
||||||
|
|
||||||
{.push raises:[Defect].}
|
{.push raises:[Defect].}
|
||||||
|
|
||||||
|
@ -85,63 +85,64 @@ const
|
||||||
## staged.
|
## staged.
|
||||||
50
|
50
|
||||||
|
|
||||||
static:
|
|
||||||
doAssert stagedWorkItemsTrigger < maxStagedWorkItems
|
|
||||||
|
|
||||||
type
|
type
|
||||||
BlockRangeSetRef* = ##\
|
BlockRangeSetRef = ##\
|
||||||
## Disjunct sets of block number intervals
|
## Disjunct sets of block number intervals
|
||||||
IntervalSetRef[BlockNumber,UInt256]
|
IntervalSetRef[BlockNumber,UInt256]
|
||||||
|
|
||||||
BlockRange* = ##\
|
BlockRange = ##\
|
||||||
## Block number interval
|
## Block number interval
|
||||||
Interval[BlockNumber,UInt256]
|
Interval[BlockNumber,UInt256]
|
||||||
|
|
||||||
WorkItemQueue* = ##\
|
WorkItemQueue = ##\
|
||||||
## Block intervals sorted by least block number
|
## Block intervals sorted by least block number
|
||||||
SortedSet[BlockNumber,WorkItemRef]
|
SortedSet[BlockNumber,WorkItemRef]
|
||||||
|
|
||||||
WorkItemWalkRef* = ##\
|
WorkItemWalkRef = ##\
|
||||||
## Fast traversal descriptor for `WorkItemQueue`
|
## Fast traversal descriptor for `WorkItemQueue`
|
||||||
SortedSetWalkRef[BlockNumber,WorkItemRef]
|
SortedSetWalkRef[BlockNumber,WorkItemRef]
|
||||||
|
|
||||||
WorkItemRef* = ref object
|
WorkItemRef = ref object
|
||||||
## Block worker item wrapper for downloading a block range
|
## Block worker item wrapper for downloading a block range
|
||||||
blocks: BlockRange ## Block numbers to fetch
|
blocks: BlockRange ## Block numbers to fetch
|
||||||
topHash: Option[Hash256] ## Fetch by top hash rather than blocks
|
topHash: Option[Hash256] ## Fetch by top hash rather than blocks
|
||||||
headers: seq[BlockHeader] ## Block headers received
|
headers: seq[BlockHeader] ## Block headers received
|
||||||
hashes: seq[Hash256] ## Hashed from `headers[]` for convenience
|
hashes: seq[Hash256] ## Hashed from `headers[]` for convenience
|
||||||
bodies: seq[BlockBody] ## Block bodies received
|
bodies: seq[BlockBody] ## Block bodies received
|
||||||
|
|
||||||
BuddyDataEx = ref object of BuddyDataRef
|
BuddyData* = object
|
||||||
## Local descriptor data extension
|
## Local descriptor data extension
|
||||||
bestNumber: Option[BlockNumber] ## Largest block number reported
|
bestNumber: Option[BlockNumber] ## Largest block number reported
|
||||||
|
|
||||||
CtxDataEx = ref object of CtxDataRef
|
CtxData* = object
|
||||||
## Globally shared data extension
|
## Globally shared data extension
|
||||||
backtrack: Option[Hash256] ## Find reverse block after re-org
|
backtrack: Option[Hash256] ## Find reverse block after re-org
|
||||||
unprocessed: BlockRangeSetRef ## Block ranges to fetch
|
unprocessed: BlockRangeSetRef ## Block ranges to fetch
|
||||||
staged: WorkItemQueue ## Blocks fetched but not stored yet
|
staged: WorkItemQueue ## Blocks fetched but not stored yet
|
||||||
untrusted: seq[Peer] ## Clean up list
|
untrusted: seq[Peer] ## Clean up list
|
||||||
trusted: HashSet[Peer] ## Peers ready for delivery
|
trusted: HashSet[Peer] ## Peers ready for delivery
|
||||||
topPersistent: BlockNumber ## Up to this block number stored OK
|
topPersistent: BlockNumber ## Up to this block number stored OK
|
||||||
ticker: Ticker ## Logger ticker
|
ticker: TickerRef ## Logger ticker
|
||||||
|
|
||||||
|
FullBuddyRef* = ##\
|
||||||
|
## Extended worker peer descriptor
|
||||||
|
BuddyRef[CtxData,BuddyData]
|
||||||
|
|
||||||
|
FullCtxRef* = ##\
|
||||||
|
## Extended global descriptor
|
||||||
|
CtxRef[CtxData]
|
||||||
|
|
||||||
let
|
let
|
||||||
highBlockRange =
|
highBlockNumber = high(BlockNumber)
|
||||||
BlockRange.new(high(BlockNumber),high(BlockNumber))
|
highBlockRange = BlockRange.new(highBlockNumber,highBlockNumber)
|
||||||
|
|
||||||
|
static:
|
||||||
|
doAssert stagedWorkItemsTrigger < maxStagedWorkItems
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private helpers
|
# Private helpers
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc getOrHigh(b: Option[BlockNumber]): BlockNumber =
|
|
||||||
## Syntactic sugar
|
|
||||||
if b.isSome: b.get else: high(BlockNumber)
|
|
||||||
|
|
||||||
proc getOrHigh(b: Option[BlockRange]): BlockRange =
|
|
||||||
if b.isSome: b.get else: highBlockRange
|
|
||||||
|
|
||||||
proc hash(peer: Peer): Hash =
|
proc hash(peer: Peer): Hash =
|
||||||
## Mixin `HashSet[Peer]` handler
|
## Mixin `HashSet[Peer]` handler
|
||||||
hash(cast[pointer](peer))
|
hash(cast[pointer](peer))
|
||||||
|
@ -165,7 +166,7 @@ proc reduce(ivSet: BlockRangeSetRef; wi: WorkItemRef): Uint256 =
|
||||||
|
|
||||||
proc pp(n: BlockNumber): string =
|
proc pp(n: BlockNumber): string =
|
||||||
## Dedicated pretty printer (`$` is defined elsewhere using `UInt256`)
|
## Dedicated pretty printer (`$` is defined elsewhere using `UInt256`)
|
||||||
if n == high(BlockNumber): "high" else:"#" & $n
|
if n == highBlockNumber: "high" else:"#" & $n
|
||||||
|
|
||||||
proc `$`(iv: BlockRange): string =
|
proc `$`(iv: BlockRange): string =
|
||||||
## Needed for macro generated DSL files like `snap.nim` because the
|
## Needed for macro generated DSL files like `snap.nim` because the
|
||||||
|
@ -188,27 +189,15 @@ proc `$`(brs: BlockRangeSetRef): string =
|
||||||
# Private getters
|
# Private getters
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc local(buddy: BuddyRef): BuddyDataEx =
|
proc nextUnprocessed(desc: var CtxData): Option[BlockNumber] =
|
||||||
## Parameters local to this peer worker
|
|
||||||
buddy.data.BuddyDataEx
|
|
||||||
|
|
||||||
proc pool(ctx: CtxRef): CtxDataEx =
|
|
||||||
## Parameters shared between all peer workers
|
|
||||||
ctx.data.CtxDataEx
|
|
||||||
|
|
||||||
proc pool(buddy: BuddyRef): CtxDataEx =
|
|
||||||
## Ditto
|
|
||||||
buddy.ctx.data.CtxDataEx
|
|
||||||
|
|
||||||
proc nextUnprocessed(pool: CtxDataEx): Option[BlockNumber] =
|
|
||||||
## Pseudo getter
|
## Pseudo getter
|
||||||
let rc = pool.unprocessed.ge()
|
let rc = desc.unprocessed.ge()
|
||||||
if rc.isOK:
|
if rc.isOK:
|
||||||
result = some(rc.value.minPt)
|
result = some(rc.value.minPt)
|
||||||
|
|
||||||
proc nextStaged(pool: CtxDataEx): Option[BlockRange] =
|
proc nextStaged(desc: var CtxData): Option[BlockRange] =
|
||||||
## Pseudo getter
|
## Pseudo getter
|
||||||
let rc = pool.staged.ge(low(BlockNumber))
|
let rc = desc.staged.ge(low(BlockNumber))
|
||||||
if rc.isOK:
|
if rc.isOK:
|
||||||
result = some(rc.value.data.blocks)
|
result = some(rc.value.data.blocks)
|
||||||
|
|
||||||
|
@ -216,7 +205,7 @@ proc nextStaged(pool: CtxDataEx): Option[BlockRange] =
|
||||||
# Private functions affecting all shared data
|
# Private functions affecting all shared data
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc globalReset(ctx: CtxRef; backBlocks = maxHeadersFetch): bool =
|
proc globalReset(ctx: FullCtxRef; backBlocks = maxHeadersFetch): bool =
|
||||||
## Globally flush `pending` and `staged` items and update `unprocessed`
|
## Globally flush `pending` and `staged` items and update `unprocessed`
|
||||||
## ranges and set the `unprocessed` back before the best block number/
|
## ranges and set the `unprocessed` back before the best block number/
|
||||||
var topPersistent: BlockNumber
|
var topPersistent: BlockNumber
|
||||||
|
@ -232,32 +221,35 @@ proc globalReset(ctx: CtxRef; backBlocks = maxHeadersFetch): bool =
|
||||||
error "Best block header problem", backBlocks, error=($e.name), msg=e.msg
|
error "Best block header problem", backBlocks, error=($e.name), msg=e.msg
|
||||||
return false
|
return false
|
||||||
|
|
||||||
ctx.pool.unprocessed.clear()
|
ctx.data.unprocessed.clear()
|
||||||
ctx.pool.staged.clear()
|
ctx.data.staged.clear()
|
||||||
ctx.pool.trusted.clear()
|
ctx.data.trusted.clear()
|
||||||
ctx.pool.topPersistent = topPersistent
|
ctx.data.topPersistent = topPersistent
|
||||||
discard ctx.pool.unprocessed.merge(topPersistent + 1, high(BlockNumber))
|
discard ctx.data.unprocessed.merge(topPersistent + 1, highBlockNumber)
|
||||||
|
|
||||||
true
|
true
|
||||||
|
|
||||||
proc tickerUpdater(ctx: CtxRef): TickerStatsUpdater =
|
proc tickerUpdater(ctx: FullCtxRef): TickerStatsUpdater =
|
||||||
result = proc: TickerStats =
|
result = proc: TickerStats =
|
||||||
let
|
let
|
||||||
stagedRange = ctx.pool.nextStaged
|
stagedRange = ctx.data.nextStaged
|
||||||
nextStaged = if stagedRange.isSome: some(stagedRange.get.minPt)
|
nextStaged = if stagedRange.isSome: some(stagedRange.get.minPt)
|
||||||
else: none(BlockNumber)
|
else: none(BlockNumber)
|
||||||
TickerStats(
|
TickerStats(
|
||||||
topPersistent: ctx.pool.topPersistent,
|
topPersistent: ctx.data.topPersistent,
|
||||||
nextStaged: nextStaged,
|
nextStaged: nextStaged,
|
||||||
nextUnprocessed: ctx.pool.nextUnprocessed,
|
nextUnprocessed: ctx.data.nextUnprocessed,
|
||||||
nStagedQueue: ctx.pool.staged.len,
|
nStagedQueue: ctx.data.staged.len,
|
||||||
reOrg: ctx.pool.backtrack.isSome)
|
reOrg: ctx.data.backtrack.isSome)
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private functions
|
# Private functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
template safeTransport(buddy: BuddyRef; info: static[string]; code: untyped) =
|
template safeTransport(
|
||||||
|
buddy: FullBuddyRef;
|
||||||
|
info: static[string];
|
||||||
|
code: untyped) =
|
||||||
try:
|
try:
|
||||||
code
|
code
|
||||||
except TransportError as e:
|
except TransportError as e:
|
||||||
|
@ -265,17 +257,18 @@ template safeTransport(buddy: BuddyRef; info: static[string]; code: untyped) =
|
||||||
buddy.ctrl.stopped = true
|
buddy.ctrl.stopped = true
|
||||||
|
|
||||||
|
|
||||||
proc getRandomTrustedPeer(buddy: BuddyRef): Result[Peer,void] =
|
proc getRandomTrustedPeer(buddy: FullBuddyRef): Result[Peer,void] =
|
||||||
## Return random entry from `trusted` peer different from this peer set if
|
## Return random entry from `trusted` peer different from this peer set if
|
||||||
## there are enough
|
## there are enough
|
||||||
##
|
##
|
||||||
## Ackn: nim-eth/eth/p2p/blockchain_sync.nim: `randomTrustedPeer()`
|
## Ackn: nim-eth/eth/p2p/blockchain_sync.nim: `randomTrustedPeer()`
|
||||||
let
|
let
|
||||||
nPeers = buddy.pool.trusted.len
|
ctx = buddy.ctx
|
||||||
offInx = if buddy.peer in buddy.pool.trusted: 2 else: 1
|
nPeers = ctx.data.trusted.len
|
||||||
|
offInx = if buddy.peer in ctx.data.trusted: 2 else: 1
|
||||||
if 0 < nPeers:
|
if 0 < nPeers:
|
||||||
var (walkInx, stopInx) = (0, rand(nPeers - offInx))
|
var (walkInx, stopInx) = (0, rand(nPeers - offInx))
|
||||||
for p in buddy.pool.trusted:
|
for p in ctx.data.trusted:
|
||||||
if p == buddy.peer:
|
if p == buddy.peer:
|
||||||
continue
|
continue
|
||||||
if walkInx == stopInx:
|
if walkInx == stopInx:
|
||||||
|
@ -284,21 +277,22 @@ proc getRandomTrustedPeer(buddy: BuddyRef): Result[Peer,void] =
|
||||||
err()
|
err()
|
||||||
|
|
||||||
|
|
||||||
proc newWorkItem(buddy: BuddyRef): Result[WorkItemRef,void] =
|
proc newWorkItem(buddy: FullBuddyRef): Result[WorkItemRef,void] =
|
||||||
## Fetch the next unprocessed block range and register it as work item.
|
## Fetch the next unprocessed block range and register it as work item.
|
||||||
##
|
##
|
||||||
## This function will grab a block range from the `unprocessed` range set,
|
## This function will grab a block range from the `unprocessed` range set,
|
||||||
## ove it and return it as a `WorkItemRef`. The returned range is registered
|
## ove it and return it as a `WorkItemRef`. The returned range is registered
|
||||||
## in the `pending` list.
|
## in the `pending` list.
|
||||||
let
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
peer = buddy.peer
|
peer = buddy.peer
|
||||||
rc = buddy.pool.unprocessed.ge()
|
rc = ctx.data.unprocessed.ge()
|
||||||
if rc.isErr:
|
if rc.isErr:
|
||||||
return err() # no more data for this peer
|
return err() # no more data for this peer
|
||||||
|
|
||||||
# Check whether there is somthing to do at all
|
# Check whether there is somthing to do at all
|
||||||
if buddy.local.bestNumber.isNone or
|
if buddy.data.bestNumber.isNone or
|
||||||
buddy.local.bestNumber.get < rc.value.minPt:
|
buddy.data.bestNumber.get < rc.value.minPt:
|
||||||
return err() # no more data for this peer
|
return err() # no more data for this peer
|
||||||
|
|
||||||
# Compute interval
|
# Compute interval
|
||||||
|
@ -306,32 +300,37 @@ proc newWorkItem(buddy: BuddyRef): Result[WorkItemRef,void] =
|
||||||
rc.value.minPt,
|
rc.value.minPt,
|
||||||
min(rc.value.maxPt,
|
min(rc.value.maxPt,
|
||||||
min(rc.value.minPt + maxHeadersFetch - 1,
|
min(rc.value.minPt + maxHeadersFetch - 1,
|
||||||
buddy.local.bestNumber.get)))
|
buddy.data.bestNumber.get)))
|
||||||
|
|
||||||
discard buddy.pool.unprocessed.reduce(iv)
|
discard ctx.data.unprocessed.reduce(iv)
|
||||||
return ok(WorkItemRef(blocks: iv))
|
return ok(WorkItemRef(blocks: iv))
|
||||||
|
|
||||||
|
|
||||||
proc recycleStaged(buddy: BuddyRef) =
|
proc recycleStaged(buddy: FullBuddyRef) =
|
||||||
## Flush list of staged items and store the block ranges
|
## Flush list of staged items and store the block ranges
|
||||||
## back to the `unprocessed` ranges set
|
## back to the `unprocessed` ranges set
|
||||||
##
|
##
|
||||||
# using fast traversal
|
# using fast traversal
|
||||||
let walk = WorkItemWalkRef.init(buddy.pool.staged)
|
let
|
||||||
var rc = walk.first()
|
ctx = buddy.ctx
|
||||||
|
walk = WorkItemWalkRef.init(ctx.data.staged)
|
||||||
|
var
|
||||||
|
rc = walk.first()
|
||||||
while rc.isOk:
|
while rc.isOk:
|
||||||
# Store back into `unprocessed` ranges set
|
# Store back into `unprocessed` ranges set
|
||||||
discard buddy.pool.unprocessed.merge(rc.value.data)
|
discard ctx.data.unprocessed.merge(rc.value.data)
|
||||||
rc = walk.next()
|
rc = walk.next()
|
||||||
# optional clean up, see comments on the destroy() directive
|
# optional clean up, see comments on the destroy() directive
|
||||||
walk.destroy()
|
walk.destroy()
|
||||||
buddy.pool.staged.clear()
|
ctx.data.staged.clear()
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private `Future` helpers
|
# Private `Future` helpers
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc getBestNumber(buddy: BuddyRef): Future[Result[BlockNumber,void]]{.async.} =
|
proc getBestNumber(
|
||||||
|
buddy: FullBuddyRef
|
||||||
|
): Future[Result[BlockNumber,void]] {.async.} =
|
||||||
## Get best block number from best block hash.
|
## Get best block number from best block hash.
|
||||||
##
|
##
|
||||||
## Ackn: nim-eth/eth/p2p/blockchain_sync.nim: `getBestBlockNumber()`
|
## Ackn: nim-eth/eth/p2p/blockchain_sync.nim: `getBestBlockNumber()`
|
||||||
|
@ -370,7 +369,7 @@ proc getBestNumber(buddy: BuddyRef): Future[Result[BlockNumber,void]]{.async.} =
|
||||||
return err()
|
return err()
|
||||||
|
|
||||||
|
|
||||||
proc agreesOnChain(buddy: BuddyRef; other: Peer): Future[bool] {.async.} =
|
proc agreesOnChain(buddy: FullBuddyRef; other: Peer): Future[bool] {.async.} =
|
||||||
## Returns `true` if one of the peers `buddy.peer` or `other` acknowledges
|
## Returns `true` if one of the peers `buddy.peer` or `other` acknowledges
|
||||||
## existence of the best block of the other peer.
|
## existence of the best block of the other peer.
|
||||||
##
|
##
|
||||||
|
@ -417,49 +416,51 @@ proc agreesOnChain(buddy: BuddyRef; other: Peer): Future[bool] {.async.} =
|
||||||
# Private functions, worker sub-tasks
|
# Private functions, worker sub-tasks
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc initaliseWorker(buddy: BuddyRef): Future[bool] {.async.} =
|
proc initaliseWorker(buddy: FullBuddyRef): Future[bool] {.async.} =
|
||||||
## Initalise worker. This function must be run in single mode at the
|
## Initalise worker. This function must be run in single mode at the
|
||||||
## beginning of running worker peer.
|
## beginning of running worker peer.
|
||||||
##
|
##
|
||||||
## Ackn: nim-eth/eth/p2p/blockchain_sync.nim: `startSyncWithPeer()`
|
## Ackn: nim-eth/eth/p2p/blockchain_sync.nim: `startSyncWithPeer()`
|
||||||
##
|
##
|
||||||
let peer = buddy.peer
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
|
||||||
# Delayed clean up batch list
|
# Delayed clean up batch list
|
||||||
if 0 < buddy.pool.untrusted.len:
|
if 0 < ctx.data.untrusted.len:
|
||||||
trace "Removing untrused peers", peer, count=buddy.pool.untrusted.len
|
trace "Removing untrused peers", peer, count=ctx.data.untrusted.len
|
||||||
for p in buddy.pool.untrusted:
|
for p in ctx.data.untrusted:
|
||||||
buddy.pool.trusted.excl p
|
ctx.data.trusted.excl p
|
||||||
buddy.pool.untrusted.setLen(0)
|
ctx.data.untrusted.setLen(0)
|
||||||
|
|
||||||
if buddy.local.bestNumber.isNone:
|
if buddy.data.bestNumber.isNone:
|
||||||
let rc = await buddy.getBestNumber()
|
let rc = await buddy.getBestNumber()
|
||||||
# Beware of peer terminating the session right after communicating
|
# Beware of peer terminating the session right after communicating
|
||||||
if rc.isErr or buddy.ctrl.stopped:
|
if rc.isErr or buddy.ctrl.stopped:
|
||||||
return false
|
return false
|
||||||
if rc.value <= buddy.pool.topPersistent:
|
if rc.value <= ctx.data.topPersistent:
|
||||||
buddy.ctrl.zombie = true
|
buddy.ctrl.zombie = true
|
||||||
trace "Useless peer, best number too low", peer,
|
trace "Useless peer, best number too low", peer,
|
||||||
topPersistent=buddy.pool.topPersistent, bestNumber=rc.value
|
topPersistent=ctx.data.topPersistent, bestNumber=rc.value
|
||||||
buddy.local.bestNumber = some(rc.value)
|
buddy.data.bestNumber = some(rc.value)
|
||||||
|
|
||||||
if minPeersToStartSync <= buddy.pool.trusted.len:
|
if minPeersToStartSync <= ctx.data.trusted.len:
|
||||||
# We have enough trusted peers. Validate new peer against trusted
|
# We have enough trusted peers. Validate new peer against trusted
|
||||||
let rc = buddy.getRandomTrustedPeer()
|
let rc = buddy.getRandomTrustedPeer()
|
||||||
if rc.isOK:
|
if rc.isOK:
|
||||||
if await buddy.agreesOnChain(rc.value):
|
if await buddy.agreesOnChain(rc.value):
|
||||||
# Beware of peer terminating the session
|
# Beware of peer terminating the session
|
||||||
if not buddy.ctrl.stopped:
|
if not buddy.ctrl.stopped:
|
||||||
buddy.pool.trusted.incl peer
|
ctx.data.trusted.incl peer
|
||||||
return true
|
return true
|
||||||
|
|
||||||
# If there are no trusted peers yet, assume this very peer is trusted,
|
# If there are no trusted peers yet, assume this very peer is trusted,
|
||||||
# but do not finish initialisation until there are more peers.
|
# but do not finish initialisation until there are more peers.
|
||||||
elif buddy.pool.trusted.len == 0:
|
elif ctx.data.trusted.len == 0:
|
||||||
trace "Assume initial trusted peer", peer
|
trace "Assume initial trusted peer", peer
|
||||||
buddy.pool.trusted.incl peer
|
ctx.data.trusted.incl peer
|
||||||
|
|
||||||
elif buddy.pool.trusted.len == 1 and buddy.peer in buddy.pool.trusted:
|
elif ctx.data.trusted.len == 1 and buddy.peer in ctx.data.trusted:
|
||||||
# Ignore degenerate case, note that `trusted.len < minPeersToStartSync`
|
# Ignore degenerate case, note that `trusted.len < minPeersToStartSync`
|
||||||
discard
|
discard
|
||||||
|
|
||||||
|
@ -472,7 +473,7 @@ proc initaliseWorker(buddy: BuddyRef): Future[bool] {.async.} =
|
||||||
var
|
var
|
||||||
agreeScore = 0
|
agreeScore = 0
|
||||||
otherPeer: Peer
|
otherPeer: Peer
|
||||||
for p in buddy.pool.trusted:
|
for p in ctx.data.trusted:
|
||||||
if peer == p:
|
if peer == p:
|
||||||
inc agreeScore
|
inc agreeScore
|
||||||
else:
|
else:
|
||||||
|
@ -486,29 +487,34 @@ proc initaliseWorker(buddy: BuddyRef): Future[bool] {.async.} =
|
||||||
otherPeer = p
|
otherPeer = p
|
||||||
|
|
||||||
# Check for the number of peers that disagree
|
# Check for the number of peers that disagree
|
||||||
case buddy.pool.trusted.len - agreeScore
|
case ctx.data.trusted.len - agreeScore
|
||||||
of 0:
|
of 0:
|
||||||
trace "Peer trusted by score", peer,
|
trace "Peer trusted by score", peer,
|
||||||
trusted=buddy.pool.trusted.len
|
trusted=ctx.data.trusted.len
|
||||||
buddy.pool.trusted.incl peer # best possible outcome
|
ctx.data.trusted.incl peer # best possible outcome
|
||||||
of 1:
|
of 1:
|
||||||
trace "Other peer no longer trusted", peer,
|
trace "Other peer no longer trusted", peer,
|
||||||
otherPeer, trusted=buddy.pool.trusted.len
|
otherPeer, trusted=ctx.data.trusted.len
|
||||||
buddy.pool.trusted.excl otherPeer
|
ctx.data.trusted.excl otherPeer
|
||||||
buddy.pool.trusted.incl peer
|
ctx.data.trusted.incl peer
|
||||||
else:
|
else:
|
||||||
trace "Peer not trusted", peer,
|
trace "Peer not trusted", peer,
|
||||||
trusted=buddy.pool.trusted.len
|
trusted=ctx.data.trusted.len
|
||||||
discard
|
discard
|
||||||
|
|
||||||
if minPeersToStartSync <= buddy.pool.trusted.len:
|
if minPeersToStartSync <= ctx.data.trusted.len:
|
||||||
return true
|
return true
|
||||||
|
|
||||||
|
|
||||||
proc fetchHeaders(buddy: BuddyRef; wi: WorkItemRef): Future[bool] {.async.} =
|
proc fetchHeaders(
|
||||||
|
buddy: FullBuddyRef;
|
||||||
|
wi: WorkItemRef
|
||||||
|
): Future[bool] {.async.} =
|
||||||
## Get the work item with the least interval and complete it. The function
|
## Get the work item with the least interval and complete it. The function
|
||||||
## returns `true` if bodies were fetched and there were no inconsistencies.
|
## returns `true` if bodies were fetched and there were no inconsistencies.
|
||||||
let peer = buddy.peer
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
|
||||||
if 0 < wi.hashes.len:
|
if 0 < wi.hashes.len:
|
||||||
return true
|
return true
|
||||||
|
@ -563,7 +569,7 @@ proc fetchHeaders(buddy: BuddyRef; wi: WorkItemRef): Future[bool] {.async.} =
|
||||||
wi.headers = hdrResp.get.headers.reversed
|
wi.headers = hdrResp.get.headers.reversed
|
||||||
wi.blocks = BlockRange.new(
|
wi.blocks = BlockRange.new(
|
||||||
wi.headers[0].blockNumber, wi.headers[^1].blockNumber)
|
wi.headers[0].blockNumber, wi.headers[^1].blockNumber)
|
||||||
discard buddy.pool.unprocessed.reduce(wi)
|
discard ctx.data.unprocessed.reduce(wi)
|
||||||
trace "Updated reverse header range", peer, range=($wi.blocks)
|
trace "Updated reverse header range", peer, range=($wi.blocks)
|
||||||
|
|
||||||
# Verify start block number
|
# Verify start block number
|
||||||
|
@ -600,13 +606,13 @@ proc fetchHeaders(buddy: BuddyRef; wi: WorkItemRef): Future[bool] {.async.} =
|
||||||
let redRng = BlockRange.new(
|
let redRng = BlockRange.new(
|
||||||
wi.headers[0].blockNumber, wi.headers[^1].blockNumber)
|
wi.headers[0].blockNumber, wi.headers[^1].blockNumber)
|
||||||
trace "Adjusting block range", peer, range=($wi.blocks), reduced=($redRng)
|
trace "Adjusting block range", peer, range=($wi.blocks), reduced=($redRng)
|
||||||
discard buddy.pool.unprocessed.merge(redRng.maxPt + 1, wi.blocks.maxPt)
|
discard ctx.data.unprocessed.merge(redRng.maxPt + 1, wi.blocks.maxPt)
|
||||||
wi.blocks = redRng
|
wi.blocks = redRng
|
||||||
|
|
||||||
return true
|
return true
|
||||||
|
|
||||||
|
|
||||||
proc fetchBodies(buddy: BuddyRef; wi: WorkItemRef): Future[bool] {.async.} =
|
proc fetchBodies(buddy: FullBuddyRef; wi: WorkItemRef): Future[bool] {.async.} =
|
||||||
## Get the work item with the least interval and complete it. The function
|
## Get the work item with the least interval and complete it. The function
|
||||||
## returns `true` if bodies were fetched and there were no inconsistencies.
|
## returns `true` if bodies were fetched and there were no inconsistencies.
|
||||||
let peer = buddy.peer
|
let peer = buddy.peer
|
||||||
|
@ -646,33 +652,34 @@ proc fetchBodies(buddy: BuddyRef; wi: WorkItemRef): Future[bool] {.async.} =
|
||||||
return true
|
return true
|
||||||
|
|
||||||
|
|
||||||
proc stageItem(buddy: BuddyRef; wi: WorkItemRef) =
|
proc stageItem(buddy: FullBuddyRef; wi: WorkItemRef) =
|
||||||
## Add work item to the list of staged items
|
## Add work item to the list of staged items
|
||||||
let peer = buddy.peer
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
let rc = buddy.pool.staged.insert(wi.blocks.minPt)
|
peer = buddy.peer
|
||||||
|
rc = ctx.data.staged.insert(wi.blocks.minPt)
|
||||||
if rc.isOk:
|
if rc.isOk:
|
||||||
rc.value.data = wi
|
rc.value.data = wi
|
||||||
|
|
||||||
# Turn on pool mode if there are too may staged work items queued.
|
# Turn on pool mode if there are too may staged work items queued.
|
||||||
# This must only be done when the added work item is not backtracking.
|
# This must only be done when the added work item is not backtracking.
|
||||||
if stagedWorkItemsTrigger < buddy.pool.staged.len and
|
if stagedWorkItemsTrigger < ctx.data.staged.len and
|
||||||
buddy.pool.backtrack.isNone and
|
ctx.data.backtrack.isNone and
|
||||||
wi.topHash.isNone:
|
wi.topHash.isNone:
|
||||||
buddy.ctx.poolMode = true
|
buddy.ctx.poolMode = true
|
||||||
|
|
||||||
# The list size is limited. So cut if necessary and recycle back the block
|
# The list size is limited. So cut if necessary and recycle back the block
|
||||||
# range of the discarded item (tough luck if the current work item is the
|
# range of the discarded item (tough luck if the current work item is the
|
||||||
# one removed from top.)
|
# one removed from top.)
|
||||||
while maxStagedWorkItems < buddy.pool.staged.len:
|
while maxStagedWorkItems < ctx.data.staged.len:
|
||||||
let topValue = buddy.pool.staged.le(high(BlockNumber)).value
|
let topValue = ctx.data.staged.le(highBlockNumber).value
|
||||||
discard buddy.pool.unprocessed.merge(topValue.data)
|
discard ctx.data.unprocessed.merge(topValue.data)
|
||||||
discard buddy.pool.staged.delete(topValue.key)
|
discard ctx.data.staged.delete(topValue.key)
|
||||||
return
|
return
|
||||||
|
|
||||||
# Ooops, duplicates should not exist (but anyway ...)
|
# Ooops, duplicates should not exist (but anyway ...)
|
||||||
let wj = block:
|
let wj = block:
|
||||||
let rc = buddy.pool.staged.eq(wi.blocks.minPt)
|
let rc = ctx.data.staged.eq(wi.blocks.minPt)
|
||||||
doAssert rc.isOk
|
doAssert rc.isOk
|
||||||
# Store `wi` and return offending entry
|
# Store `wi` and return offending entry
|
||||||
let rcData = rc.value.data
|
let rcData = rc.value.data
|
||||||
|
@ -685,26 +692,27 @@ proc stageItem(buddy: BuddyRef; wi: WorkItemRef) =
|
||||||
block:
|
block:
|
||||||
let rc = wi.blocks - wj.blocks
|
let rc = wi.blocks - wj.blocks
|
||||||
if rc.isOk:
|
if rc.isOk:
|
||||||
discard buddy.pool.unprocessed.merge(rc.value)
|
discard ctx.data.unprocessed.merge(rc.value)
|
||||||
|
|
||||||
|
|
||||||
proc processStaged(buddy: BuddyRef): bool =
|
proc processStaged(buddy: FullBuddyRef): bool =
|
||||||
## Fetch a work item from the `staged` queue an process it to be
|
## Fetch a work item from the `staged` queue an process it to be
|
||||||
## stored on the persistent block chain.
|
## stored on the persistent block chain.
|
||||||
|
|
||||||
let
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
peer = buddy.peer
|
peer = buddy.peer
|
||||||
chainDb = buddy.ctx.chain
|
chainDb = buddy.ctx.chain
|
||||||
rc = buddy.pool.staged.ge(low(BlockNumber))
|
rc = ctx.data.staged.ge(low(BlockNumber))
|
||||||
|
|
||||||
if rc.isErr:
|
if rc.isErr:
|
||||||
# No more items in the database
|
# No more items in the database
|
||||||
return false
|
return false
|
||||||
|
|
||||||
let
|
let
|
||||||
wi = rc.value.data
|
wi = rc.value.data
|
||||||
topPersistent = buddy.pool.topPersistent
|
topPersistent = ctx.data.topPersistent
|
||||||
startNumber = wi.headers[0].blockNumber
|
startNumber = wi.headers[0].blockNumber
|
||||||
stagedRecords = buddy.pool.staged.len
|
stagedRecords = ctx.data.staged.len
|
||||||
|
|
||||||
# Check whether this record of blocks can be stored, at all
|
# Check whether this record of blocks can be stored, at all
|
||||||
if topPersistent + 1 < startNumber:
|
if topPersistent + 1 < startNumber:
|
||||||
|
@ -717,11 +725,11 @@ proc processStaged(buddy: BuddyRef): bool =
|
||||||
topPersistent, range=($wi.blocks)
|
topPersistent, range=($wi.blocks)
|
||||||
|
|
||||||
# remove from staged DB
|
# remove from staged DB
|
||||||
discard buddy.pool.staged.delete(wi.blocks.minPt)
|
discard ctx.data.staged.delete(wi.blocks.minPt)
|
||||||
|
|
||||||
try:
|
try:
|
||||||
if chainDb.persistBlocks(wi.headers, wi.bodies) == ValidationResult.OK:
|
if chainDb.persistBlocks(wi.headers, wi.bodies) == ValidationResult.OK:
|
||||||
buddy.pool.topPersistent = wi.blocks.maxPt
|
ctx.data.topPersistent = wi.blocks.maxPt
|
||||||
return true
|
return true
|
||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
error "Storing persistent blocks failed", peer, range=($wi.blocks),
|
error "Storing persistent blocks failed", peer, range=($wi.blocks),
|
||||||
|
@ -749,7 +757,7 @@ proc processStaged(buddy: BuddyRef): bool =
|
||||||
# the blocks from another peer.
|
# the blocks from another peer.
|
||||||
trace "Storing persistent blocks failed", peer,
|
trace "Storing persistent blocks failed", peer,
|
||||||
range=($wi.blocks)
|
range=($wi.blocks)
|
||||||
discard buddy.pool.unprocessed.merge(wi.blocks)
|
discard ctx.data.unprocessed.merge(wi.blocks)
|
||||||
buddy.ctrl.zombie = true
|
buddy.ctrl.zombie = true
|
||||||
return false
|
return false
|
||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
|
@ -758,7 +766,7 @@ proc processStaged(buddy: BuddyRef): bool =
|
||||||
|
|
||||||
# Parent block header problem, so we might be in the middle of a re-org.
|
# Parent block header problem, so we might be in the middle of a re-org.
|
||||||
# Set single mode backtrack following the offending parent hash.
|
# Set single mode backtrack following the offending parent hash.
|
||||||
buddy.pool.backtrack = some(parentHash)
|
ctx.data.backtrack = some(parentHash)
|
||||||
buddy.ctrl.multiOk = false
|
buddy.ctrl.multiOk = false
|
||||||
|
|
||||||
if wi.topHash.isNone:
|
if wi.topHash.isNone:
|
||||||
|
@ -778,42 +786,43 @@ proc processStaged(buddy: BuddyRef): bool =
|
||||||
# Public start/stop and admin functions
|
# Public start/stop and admin functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc workerSetup*(ctx: CtxRef; tickerOK: bool): bool =
|
proc setup*(ctx: FullCtxRef; tickerOK: bool): bool =
|
||||||
## Global set up
|
## Global set up
|
||||||
ctx.data = CtxDataEx(unprocessed: BlockRangeSetRef.init()) # `pool` extension
|
ctx.data.unprocessed = BlockRangeSetRef.init()
|
||||||
ctx.pool.staged.init()
|
ctx.data.staged.init()
|
||||||
if tickerOK:
|
if tickerOK:
|
||||||
ctx.pool.ticker = Ticker.init(ctx.tickerUpdater)
|
ctx.data.ticker = TickerRef.init(ctx.tickerUpdater)
|
||||||
else:
|
else:
|
||||||
debug "Ticker is disabled"
|
debug "Ticker is disabled"
|
||||||
return ctx.globalReset(0)
|
return ctx.globalReset(0)
|
||||||
|
|
||||||
proc workerRelease*(ctx: CtxRef) =
|
proc release*(ctx: FullCtxRef) =
|
||||||
## Global clean up
|
## Global clean up
|
||||||
if not ctx.pool.ticker.isNil:
|
if not ctx.data.ticker.isNil:
|
||||||
ctx.pool.ticker.stop()
|
ctx.data.ticker.stop()
|
||||||
|
|
||||||
proc start*(buddy: BuddyRef): bool =
|
proc start*(buddy: FullBuddyRef): bool =
|
||||||
## Initialise worker peer
|
## Initialise worker peer
|
||||||
|
let ctx = buddy.ctx
|
||||||
if buddy.peer.supports(protocol.eth) and
|
if buddy.peer.supports(protocol.eth) and
|
||||||
buddy.peer.state(protocol.eth).initialized:
|
buddy.peer.state(protocol.eth).initialized:
|
||||||
buddy.data = BuddyDataEx.new() # `local` extension
|
if not ctx.data.ticker.isNil:
|
||||||
if not buddy.pool.ticker.isNil:
|
ctx.data.ticker.startBuddy()
|
||||||
buddy.pool.ticker.startBuddy()
|
|
||||||
return true
|
return true
|
||||||
|
|
||||||
proc stop*(buddy: BuddyRef) =
|
proc stop*(buddy: FullBuddyRef) =
|
||||||
## Clean up this peer
|
## Clean up this peer
|
||||||
|
let ctx = buddy.ctx
|
||||||
buddy.ctrl.stopped = true
|
buddy.ctrl.stopped = true
|
||||||
buddy.pool.untrusted.add buddy.peer
|
ctx.data.untrusted.add buddy.peer
|
||||||
if not buddy.pool.ticker.isNil:
|
if not ctx.data.ticker.isNil:
|
||||||
buddy.pool.ticker.stopBuddy()
|
ctx.data.ticker.stopBuddy()
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc runSingle*(buddy: BuddyRef) {.async.} =
|
proc runSingle*(buddy: FullBuddyRef) {.async.} =
|
||||||
## This peer worker is invoked if the peer-local flag `buddy.ctrl.multiOk`
|
## This peer worker is invoked if the peer-local flag `buddy.ctrl.multiOk`
|
||||||
## is set `false` which is the default mode. This flag is updated by the
|
## is set `false` which is the default mode. This flag is updated by the
|
||||||
## worker when deemed appropriate.
|
## worker when deemed appropriate.
|
||||||
|
@ -825,37 +834,40 @@ proc runSingle*(buddy: BuddyRef) {.async.} =
|
||||||
##
|
##
|
||||||
## Note that this function runs in `async` mode.
|
## Note that this function runs in `async` mode.
|
||||||
##
|
##
|
||||||
let peer = buddy.peer
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
|
||||||
if buddy.pool.backtrack.isSome:
|
if ctx.data.backtrack.isSome:
|
||||||
trace "Single run mode, re-org backtracking", peer
|
trace "Single run mode, re-org backtracking", peer
|
||||||
let wi = WorkItemRef(
|
let wi = WorkItemRef(
|
||||||
# This dummy interval can savely merged back without any effect
|
# This dummy interval can savely merged back without any effect
|
||||||
blocks: highBlockRange,
|
blocks: highBlockRange,
|
||||||
# Enable backtrack
|
# Enable backtrack
|
||||||
topHash: some(buddy.pool.backtrack.get))
|
topHash: some(ctx.data.backtrack.get))
|
||||||
|
|
||||||
# Fetch headers and bodies for the current work item
|
# Fetch headers and bodies for the current work item
|
||||||
if await buddy.fetchHeaders(wi):
|
if await buddy.fetchHeaders(wi):
|
||||||
if await buddy.fetchBodies(wi):
|
if await buddy.fetchBodies(wi):
|
||||||
buddy.pool.backtrack = none(Hash256)
|
ctx.data.backtrack = none(Hash256)
|
||||||
buddy.stageItem(wi)
|
buddy.stageItem(wi)
|
||||||
|
|
||||||
# Update pool and persistent database (may reset `multiOk`)
|
# Update persistent database (may reset `multiOk`)
|
||||||
buddy.ctrl.multiOk = true
|
buddy.ctrl.multiOk = true
|
||||||
while buddy.processStaged():
|
while buddy.processStaged() and not buddy.ctrl.stopped:
|
||||||
discard
|
# Allow thread switch as `persistBlocks()` might be slow
|
||||||
|
await sleepAsync(10.milliseconds)
|
||||||
return
|
return
|
||||||
|
|
||||||
# This work item failed, nothing to do anymore.
|
# This work item failed, nothing to do anymore.
|
||||||
discard buddy.pool.unprocessed.merge(wi)
|
discard ctx.data.unprocessed.merge(wi)
|
||||||
buddy.ctrl.zombie = true
|
buddy.ctrl.zombie = true
|
||||||
|
|
||||||
else:
|
else:
|
||||||
if buddy.local.bestNumber.isNone:
|
if buddy.data.bestNumber.isNone:
|
||||||
# Only log for the first time, or so
|
# Only log for the first time, or so
|
||||||
trace "Single run mode, initialisation", peer,
|
trace "Single run mode, initialisation", peer,
|
||||||
trusted=buddy.pool.trusted.len
|
trusted=ctx.data.trusted.len
|
||||||
discard
|
discard
|
||||||
|
|
||||||
# Initialise/re-initialise this worker
|
# Initialise/re-initialise this worker
|
||||||
|
@ -865,7 +877,7 @@ proc runSingle*(buddy: BuddyRef) {.async.} =
|
||||||
await sleepAsync(2.seconds)
|
await sleepAsync(2.seconds)
|
||||||
|
|
||||||
|
|
||||||
proc runPool*(buddy: BuddyRef) =
|
proc runPool*(buddy: FullBuddyRef) =
|
||||||
## Ocne started, the function `runPool()` is called for all worker peers in
|
## Ocne started, the function `runPool()` is called for all worker peers in
|
||||||
## a row (as the body of an iteration.) There will be no other worker peer
|
## a row (as the body of an iteration.) There will be no other worker peer
|
||||||
## functions activated simultaneously.
|
## functions activated simultaneously.
|
||||||
|
@ -877,29 +889,32 @@ proc runPool*(buddy: BuddyRef) =
|
||||||
##
|
##
|
||||||
## Note that this function does not run in `async` mode.
|
## Note that this function does not run in `async` mode.
|
||||||
##
|
##
|
||||||
if buddy.ctx.poolMode:
|
let ctx = buddy.ctx
|
||||||
|
if ctx.poolMode:
|
||||||
# Mind the gap, fill in if necessary
|
# Mind the gap, fill in if necessary
|
||||||
let
|
let
|
||||||
topPersistent = buddy.pool.topPersistent
|
topPersistent = ctx.data.topPersistent
|
||||||
covered = min(
|
covered = min(
|
||||||
buddy.pool.nextUnprocessed.getOrHigh,
|
ctx.data.nextUnprocessed.get(highBlockNumber),
|
||||||
buddy.pool.nextStaged.getOrHigh.minPt)
|
ctx.data.nextStaged.get(highBlockRange).minPt)
|
||||||
if topPersistent + 1 < covered:
|
if topPersistent + 1 < covered:
|
||||||
discard buddy.pool.unprocessed.merge(topPersistent + 1, covered - 1)
|
discard ctx.data.unprocessed.merge(topPersistent + 1, covered - 1)
|
||||||
buddy.ctx.poolMode = false
|
ctx.poolMode = false
|
||||||
|
|
||||||
|
|
||||||
proc runMulti*(buddy: BuddyRef) {.async.} =
|
proc runMulti*(buddy: FullBuddyRef) {.async.} =
|
||||||
## This peer worker is invoked if the `buddy.ctrl.multiOk` flag is set
|
## This peer worker is invoked if the `buddy.ctrl.multiOk` flag is set
|
||||||
## `true` which is typically done after finishing `runSingle()`. This
|
## `true` which is typically done after finishing `runSingle()`. This
|
||||||
## instance can be simultaneously active for all peer workers.
|
## instance can be simultaneously active for all peer workers.
|
||||||
##
|
##
|
||||||
# Fetch work item
|
# Fetch work item
|
||||||
let rc = buddy.newWorkItem()
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
rc = buddy.newWorkItem()
|
||||||
if rc.isErr:
|
if rc.isErr:
|
||||||
# No way, end of capacity for this peer => re-calibrate
|
# No way, end of capacity for this peer => re-calibrate
|
||||||
buddy.ctrl.multiOk = false
|
buddy.ctrl.multiOk = false
|
||||||
buddy.local.bestNumber = none(BlockNumber)
|
buddy.data.bestNumber = none(BlockNumber)
|
||||||
return
|
return
|
||||||
let wi = rc.value
|
let wi = rc.value
|
||||||
|
|
||||||
|
@ -908,13 +923,14 @@ proc runMulti*(buddy: BuddyRef) {.async.} =
|
||||||
if await buddy.fetchBodies(wi):
|
if await buddy.fetchBodies(wi):
|
||||||
buddy.stageItem(wi)
|
buddy.stageItem(wi)
|
||||||
|
|
||||||
# Update pool and persistent database
|
# Update persistent database
|
||||||
while buddy.processStaged():
|
while buddy.processStaged() and not buddy.ctrl.stopped:
|
||||||
discard
|
# Allow thread switch as `persistBlocks()` might be slow
|
||||||
|
await sleepAsync(10.milliseconds)
|
||||||
return
|
return
|
||||||
|
|
||||||
# This work item failed
|
# This work item failed
|
||||||
discard buddy.pool.unprocessed.merge(wi)
|
discard ctx.data.unprocessed.merge(wi)
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# End
|
# End
|
||||||
|
|
|
@ -27,4 +27,10 @@ type
|
||||||
#eth* = eth67
|
#eth* = eth67
|
||||||
snap* = snap1
|
snap* = snap1
|
||||||
|
|
||||||
|
SnapAccountRange* = accountRangeObj
|
||||||
|
## Syntactic sugar, type defined in `snap1`
|
||||||
|
|
||||||
|
SnapTrieNodes* = trieNodesObj
|
||||||
|
## Ditto
|
||||||
|
|
||||||
# End
|
# End
|
||||||
|
|
|
@ -40,7 +40,6 @@ import
|
||||||
chronos,
|
chronos,
|
||||||
eth/[common/eth_types, p2p, p2p/private/p2p_types, p2p/blockchain_utils],
|
eth/[common/eth_types, p2p, p2p/private/p2p_types, p2p/blockchain_utils],
|
||||||
stew/byteutils,
|
stew/byteutils,
|
||||||
../types,
|
|
||||||
./trace_config
|
./trace_config
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
|
@ -112,6 +111,16 @@ const
|
||||||
trEthSendDelaying* =
|
trEthSendDelaying* =
|
||||||
">> " & prettyEthProtoName & " Delaying "
|
">> " & prettyEthProtoName & " Delaying "
|
||||||
|
|
||||||
|
func toHex(hash: Hash256): string =
|
||||||
|
## Shortcut for `byteutils.toHex(hash.data)`
|
||||||
|
hash.data.toHex
|
||||||
|
|
||||||
|
func traceStep(request: BlocksRequest): string =
|
||||||
|
var str = if request.reverse: "-" else: "+"
|
||||||
|
if request.skip < high(typeof(request.skip)):
|
||||||
|
return str & $(request.skip + 1)
|
||||||
|
return static($(high(typeof(request.skip)).u256 + 1))
|
||||||
|
|
||||||
p2pProtocol eth66(version = ethVersion,
|
p2pProtocol eth66(version = ethVersion,
|
||||||
rlpxName = "eth",
|
rlpxName = "eth",
|
||||||
peerState = PeerState,
|
peerState = PeerState,
|
||||||
|
|
|
@ -141,7 +141,6 @@ import
|
||||||
nimcrypto/hash,
|
nimcrypto/hash,
|
||||||
stew/byteutils,
|
stew/byteutils,
|
||||||
../../constants,
|
../../constants,
|
||||||
../snap/path_desc,
|
|
||||||
./trace_config
|
./trace_config
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
|
@ -149,13 +148,13 @@ logScope:
|
||||||
|
|
||||||
type
|
type
|
||||||
SnapAccount* = object
|
SnapAccount* = object
|
||||||
accHash*: NodeTag
|
accHash*: Hash256
|
||||||
accBody* {.rlpCustomSerialization.}: Account
|
accBody* {.rlpCustomSerialization.}: Account
|
||||||
|
|
||||||
SnapAccountProof* = seq[Blob]
|
SnapAccountProof* = seq[Blob]
|
||||||
|
|
||||||
SnapStorage* = object
|
SnapStorage* = object
|
||||||
slotHash*: NodeTag
|
slotHash*: Hash256
|
||||||
slotData*: Blob
|
slotData*: Blob
|
||||||
|
|
||||||
SnapStorageProof* = seq[Blob]
|
SnapStorageProof* = seq[Blob]
|
||||||
|
@ -186,6 +185,55 @@ const
|
||||||
# avoids transmitting these hashes in about 90% of accounts. We need to
|
# avoids transmitting these hashes in about 90% of accounts. We need to
|
||||||
# recognise or set these hashes in `Account` when serialising RLP for `snap`.
|
# recognise or set these hashes in `Account` when serialising RLP for `snap`.
|
||||||
|
|
||||||
|
proc snapRead*(rlp: var Rlp; T: type Account; strict: static[bool] = false): T
|
||||||
|
{.gcsafe, raises: [Defect, RlpError]} =
|
||||||
|
## RLP decoding for `Account`. The `snap` RLP representation of the account
|
||||||
|
## differs from standard `Account` RLP. Empty storage hash and empty code
|
||||||
|
## hash are each represented by an RLP zero-length string instead of the
|
||||||
|
## full hash.
|
||||||
|
##
|
||||||
|
## Normally, this read function will silently handle standard encodinig and
|
||||||
|
## `snap` enciding. Setting the argument strict as `false` the function will
|
||||||
|
## throw an exception if `snap` encoding is violated.
|
||||||
|
rlp.tryEnterList()
|
||||||
|
result.nonce = rlp.read(typeof(result.nonce))
|
||||||
|
result.balance = rlp.read(typeof(result.balance))
|
||||||
|
if rlp.blobLen != 0 or not rlp.isBlob:
|
||||||
|
result.storageRoot = rlp.read(typeof(result.storageRoot))
|
||||||
|
when strict:
|
||||||
|
if result.storageRoot == BLANK_ROOT_HASH:
|
||||||
|
raise newException(RlpTypeMismatch,
|
||||||
|
"BLANK_ROOT_HASH not encoded as empty string in Snap protocol")
|
||||||
|
else:
|
||||||
|
rlp.skipElem()
|
||||||
|
result.storageRoot = BLANK_ROOT_HASH
|
||||||
|
if rlp.blobLen != 0 or not rlp.isBlob:
|
||||||
|
result.codeHash = rlp.read(typeof(result.codeHash))
|
||||||
|
when strict:
|
||||||
|
if result.codeHash == EMPTY_SHA3:
|
||||||
|
raise newException(RlpTypeMismatch,
|
||||||
|
"EMPTY_SHA3 not encoded as empty string in Snap protocol")
|
||||||
|
else:
|
||||||
|
rlp.skipElem()
|
||||||
|
result.codeHash = EMPTY_SHA3
|
||||||
|
|
||||||
|
proc snapAppend*(writer: var RlpWriter; account: Account) =
|
||||||
|
## RLP encoding for `Account`. The snap RLP representation of the account
|
||||||
|
## differs from standard `Account` RLP. Empty storage hash and empty code
|
||||||
|
## hash are each represented by an RLP zero-length string instead of the
|
||||||
|
## full hash.
|
||||||
|
writer.startList(4)
|
||||||
|
writer.append(account.nonce)
|
||||||
|
writer.append(account.balance)
|
||||||
|
if account.storageRoot == BLANK_ROOT_HASH:
|
||||||
|
writer.append("")
|
||||||
|
else:
|
||||||
|
writer.append(account.storageRoot)
|
||||||
|
if account.codeHash == EMPTY_SHA3:
|
||||||
|
writer.append("")
|
||||||
|
else:
|
||||||
|
writer.append(account.codeHash)
|
||||||
|
|
||||||
proc read(rlp: var Rlp, t: var SnapAccount, T: type Account): T =
|
proc read(rlp: var Rlp, t: var SnapAccount, T: type Account): T =
|
||||||
## RLP Mixin: decoding for `SnapAccount`.
|
## RLP Mixin: decoding for `SnapAccount`.
|
||||||
result = rlp.snapRead(T)
|
result = rlp.snapRead(T)
|
||||||
|
@ -202,12 +250,10 @@ p2pProtocol snap1(version = 1,
|
||||||
requestResponse:
|
requestResponse:
|
||||||
# User message 0x00: GetAccountRange.
|
# User message 0x00: GetAccountRange.
|
||||||
# Note: `origin` and `limit` differs from the specification to match Geth.
|
# Note: `origin` and `limit` differs from the specification to match Geth.
|
||||||
proc getAccountRange(peer: Peer, rootHash: Hash256,
|
proc getAccountRange(peer: Peer, rootHash: Hash256, origin: Hash256,
|
||||||
origin: NodeTag, limit: NodeTag,
|
limit: Hash256, responseBytes: uint64) =
|
||||||
responseBytes: uint64) =
|
|
||||||
trace trSnapRecvReceived & "GetAccountRange (0x00)", peer,
|
trace trSnapRecvReceived & "GetAccountRange (0x00)", peer,
|
||||||
accountRange=leafRangePp(origin, limit),
|
accountRange=(origin,limit), stateRoot=($rootHash), responseBytes
|
||||||
stateRoot=($rootHash), responseBytes
|
|
||||||
|
|
||||||
trace trSnapSendReplying & "EMPTY AccountRange (0x01)", peer, sent=0
|
trace trSnapSendReplying & "EMPTY AccountRange (0x01)", peer, sent=0
|
||||||
await response.send(@[], @[])
|
await response.send(@[], @[])
|
||||||
|
@ -220,9 +266,8 @@ p2pProtocol snap1(version = 1,
|
||||||
# User message 0x02: GetStorageRanges.
|
# User message 0x02: GetStorageRanges.
|
||||||
# Note: `origin` and `limit` differs from the specification to match Geth.
|
# Note: `origin` and `limit` differs from the specification to match Geth.
|
||||||
proc getStorageRanges(peer: Peer, rootHash: Hash256,
|
proc getStorageRanges(peer: Peer, rootHash: Hash256,
|
||||||
accounts: openArray[NodeTag],
|
accounts: openArray[Hash256], origin: openArray[byte],
|
||||||
origin: openArray[byte], limit: openArray[byte],
|
limit: openArray[byte], responseBytes: uint64) =
|
||||||
responseBytes: uint64) =
|
|
||||||
when trSnapTracePacketsOk:
|
when trSnapTracePacketsOk:
|
||||||
var definiteFullRange = ((origin.len == 32 or origin.len == 0) and
|
var definiteFullRange = ((origin.len == 32 or origin.len == 0) and
|
||||||
(limit.len == 32 or limit.len == 0))
|
(limit.len == 32 or limit.len == 0))
|
||||||
|
@ -286,7 +331,7 @@ p2pProtocol snap1(version = 1,
|
||||||
# User message 0x06: GetTrieNodes.
|
# User message 0x06: GetTrieNodes.
|
||||||
requestResponse:
|
requestResponse:
|
||||||
proc getTrieNodes(peer: Peer, rootHash: Hash256,
|
proc getTrieNodes(peer: Peer, rootHash: Hash256,
|
||||||
paths: openArray[PathSegment], responseBytes: uint64) =
|
paths: openArray[seq[Blob]], responseBytes: uint64) =
|
||||||
trace trSnapRecvReceived & "GetTrieNodes (0x06)", peer,
|
trace trSnapRecvReceived & "GetTrieNodes (0x06)", peer,
|
||||||
nodePaths=paths.len, stateRoot=($rootHash), responseBytes
|
nodePaths=paths.len, stateRoot=($rootHash), responseBytes
|
||||||
|
|
||||||
|
|
|
@ -1,5 +1,4 @@
|
||||||
# Nimbus - New sync approach - A fusion of snap, trie, beam and other methods
|
# Nimbus
|
||||||
#
|
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
# Licensed under either of
|
# Licensed under either of
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
@ -10,13 +9,12 @@
|
||||||
# except according to those terms.
|
# except according to those terms.
|
||||||
|
|
||||||
import
|
import
|
||||||
std/hashes,
|
eth/[common/eth_types, p2p],
|
||||||
chronicles,
|
chronicles,
|
||||||
chronos,
|
chronos,
|
||||||
eth/[common/eth_types, p2p, p2p/peer_pool, p2p/private/p2p_types],
|
../p2p/chain,
|
||||||
stew/keyed_queue,
|
./snap/[worker, worker_desc],
|
||||||
"."/[protocol, types],
|
"."/[sync_desc, sync_sched, protocol]
|
||||||
./snap/worker
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
@ -24,134 +22,54 @@ logScope:
|
||||||
topics = "snap-sync"
|
topics = "snap-sync"
|
||||||
|
|
||||||
type
|
type
|
||||||
SnapSyncRef* = ref object of Worker
|
SnapSyncRef* = RunnerSyncRef[CtxData,BuddyData]
|
||||||
chain: AbstractChainDB
|
|
||||||
buddies: KeyedQueue[Peer,WorkerBuddy] ## LRU cache with worker descriptors
|
|
||||||
pool: PeerPool ## for starting the system
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private helpers
|
# Virtual methods/interface, `mixin` functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc nsCtx(sp: WorkerBuddy): SnapSyncRef =
|
proc runSetup(ctx: SnapCtxRef; ticker: bool): bool =
|
||||||
sp.ns.SnapSyncRef
|
worker.setup(ctx,ticker)
|
||||||
|
|
||||||
proc hash(peer: Peer): Hash =
|
proc runRelease(ctx: SnapCtxRef) =
|
||||||
## Needed for `buddies` table key comparison
|
worker.release(ctx)
|
||||||
hash(peer.remote.id)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
proc runStart(buddy: SnapBuddyRef): bool =
|
||||||
# Private functions
|
worker.start(buddy)
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc workerLoop(sp: WorkerBuddy) {.async.} =
|
proc runStop(buddy: SnapBuddyRef) =
|
||||||
let ns = sp.nsCtx
|
worker.stop(buddy)
|
||||||
trace "Starting peer worker", peer=sp,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
|
|
||||||
# Do something, work a bit
|
proc runPool(buddy: SnapBuddyRef) =
|
||||||
await sp.workerExec
|
worker.runPool(buddy)
|
||||||
|
|
||||||
# Continue until stopped
|
proc runSingle(buddy: SnapBuddyRef) {.async.} =
|
||||||
while not sp.ctrl.stopped:
|
await worker.runSingle(buddy)
|
||||||
# Rotate connection table so the most used entry is at the end
|
|
||||||
discard sp.nsCtx.buddies.lruFetch(sp.peer)
|
|
||||||
|
|
||||||
let delayMs = if sp.workerLockedOk: 1000 else: 50
|
proc runMulti(buddy: SnapBuddyRef) {.async.} =
|
||||||
await sleepAsync(chronos.milliseconds(delayMs))
|
await worker.runMulti(buddy)
|
||||||
|
|
||||||
# Do something, work a bit
|
|
||||||
await sp.workerExec
|
|
||||||
|
|
||||||
trace "Peer worker done", peer=sp, ctrlState=sp.ctrl.state,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
|
|
||||||
|
|
||||||
proc onPeerConnected(ns: SnapSyncRef, peer: Peer) =
|
|
||||||
let sp = WorkerBuddy.new(ns, peer)
|
|
||||||
|
|
||||||
# Check for known entry (which should not exist.)
|
|
||||||
if ns.buddies.hasKey(peer):
|
|
||||||
trace "Ignoring already registered peer!", peer,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
return
|
|
||||||
|
|
||||||
# Initialise worker for this peer
|
|
||||||
if not sp.workerStart():
|
|
||||||
trace "Ignoring useless peer", peer,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
sp.ctrl.zombie = true
|
|
||||||
return
|
|
||||||
|
|
||||||
# Check for table overflow. An overflow should not happen if the table is
|
|
||||||
# as large as the peer connection table.
|
|
||||||
if ns.buddiesMax <= ns.buddies.len:
|
|
||||||
let leastPeer = ns.buddies.shift.value.data
|
|
||||||
if leastPeer.ctrl.zombie:
|
|
||||||
trace "Dequeuing zombie peer", leastPeer,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
discard
|
|
||||||
else:
|
|
||||||
trace "Peer table full! Dequeuing least used entry", leastPeer,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
leastPeer.workerStop()
|
|
||||||
leastPeer.ctrl.zombie = true
|
|
||||||
|
|
||||||
# Add peer entry
|
|
||||||
discard ns.buddies.lruAppend(sp.peer, sp, ns.buddiesMax)
|
|
||||||
|
|
||||||
# Run worker
|
|
||||||
asyncSpawn sp.workerLoop()
|
|
||||||
|
|
||||||
|
|
||||||
proc onPeerDisconnected(ns: SnapSyncRef, peer: Peer) =
|
|
||||||
let rc = ns.buddies.eq(peer)
|
|
||||||
if rc.isErr:
|
|
||||||
debug "Disconnected from unregistered peer", peer,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
return
|
|
||||||
let sp = rc.value
|
|
||||||
if sp.ctrl.zombie:
|
|
||||||
trace "Disconnected zombie peer", peer,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
else:
|
|
||||||
sp.workerStop()
|
|
||||||
ns.buddies.del(peer)
|
|
||||||
trace "Disconnected peer", peer,
|
|
||||||
peers=ns.pool.len, workers=ns.buddies.len, maxWorkers=ns.buddiesMax
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc init*(T: type SnapSyncRef; ethNode: EthereumNode; maxPeers: int): T =
|
proc init*(
|
||||||
## Constructor
|
T: type SnapSyncRef;
|
||||||
|
ethNode: EthereumNode;
|
||||||
|
chain: Chain;
|
||||||
|
rng: ref HmacDrbgContext;
|
||||||
|
maxPeers: int;
|
||||||
|
enableTicker = false): T =
|
||||||
new result
|
new result
|
||||||
let size = max(1,maxPeers)
|
result.initSync(ethNode, maxPeers, enableTicker)
|
||||||
result.chain = ethNode.chain
|
result.ctx.chain = chain # explicitely override
|
||||||
result.buddies.init(size)
|
result.ctx.data.rng = rng
|
||||||
result.buddiesMax = size
|
|
||||||
result.pool = ethNode.peerPool
|
|
||||||
|
|
||||||
proc start*(ctx: SnapSyncRef) =
|
proc start*(ctx: SnapSyncRef) =
|
||||||
## Set up syncing. This call should come early.
|
doAssert ctx.startSync()
|
||||||
var po = PeerObserver(
|
|
||||||
onPeerConnected:
|
|
||||||
proc(p: Peer) {.gcsafe.} =
|
|
||||||
ctx.onPeerConnected(p),
|
|
||||||
onPeerDisconnected:
|
|
||||||
proc(p: Peer) {.gcsafe.} =
|
|
||||||
ctx.onPeerDisconnected(p))
|
|
||||||
|
|
||||||
# Initialise sub-systems
|
|
||||||
ctx.workerSetup(ctx.chain)
|
|
||||||
po.setProtocol eth
|
|
||||||
ctx.pool.addObserver(ctx, po)
|
|
||||||
|
|
||||||
proc stop*(ctx: SnapSyncRef) =
|
proc stop*(ctx: SnapSyncRef) =
|
||||||
## Stop syncing
|
ctx.stopSync()
|
||||||
ctx.pool.delObserver(ctx)
|
|
||||||
ctx.workerRelease()
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# End
|
# End
|
||||||
|
|
|
@ -1,83 +0,0 @@
|
||||||
# Collected change log from Jamie's snap branch squash merge
|
|
||||||
|
|
||||||
The comments are collected in chronological order, oldest first (as opposed to
|
|
||||||
squash merge order which is oldest last.)
|
|
||||||
|
|
||||||
If a similar comment is found in a source file it was deleted here.
|
|
||||||
|
|
||||||
|
|
||||||
## Sync: Chain head: Promote peer chain head updates to debug level
|
|
||||||
|
|
||||||
This way, you can see peer chain head updates at `--log-level:DEBUG` without
|
|
||||||
being flooded by trace messages.
|
|
||||||
|
|
||||||
These occur about once every 15 seconds from each good peer.
|
|
||||||
|
|
||||||
|
|
||||||
## Sync: Chain head: Rate limit "blocked overlapping" error states
|
|
||||||
|
|
||||||
Under some conditions when a peer is not responding (but stays connected),
|
|
||||||
these messages happen continuously. Don't output them and don't waste CPU
|
|
||||||
trying.
|
|
||||||
|
|
||||||
|
|
||||||
## Sync: Set and update `syncStateRoot` for each peer
|
|
||||||
|
|
||||||
State syncing requires the `stateRoot` value of the selected block to sync to.
|
|
||||||
|
|
||||||
The chain head tracker selects a block and uses `block.stateRoot`. State sync
|
|
||||||
reads that value to sync to. It can change at any time, but that's ok, the
|
|
||||||
state sync algorithm is designed around that idea.
|
|
||||||
|
|
||||||
Aside from getting an initial `stateRoot`, the regular updates are essential
|
|
||||||
because state sync is so slow.
|
|
||||||
|
|
||||||
On Mainnet, it is normal for the initial selected block to become too old
|
|
||||||
before state sync is complete, and then peers stop providing data in their
|
|
||||||
replies. The solution is for `stateRoot` to be updated by the chain head
|
|
||||||
tracker so it's always recent enough. (On Goerli and a fast peer we can fetch
|
|
||||||
the whole state just in time without this.)
|
|
||||||
|
|
||||||
There are a number of issues with the simple implementation here:
|
|
||||||
|
|
||||||
- The selected `stateRoot` block shouldn't be the most recent canonical head,
|
|
||||||
because it is prone to change due to small reorgs. It should be a more stable
|
|
||||||
block choice, slightly further back in time.
|
|
||||||
|
|
||||||
However, any block close to the head is reasonably harmless during the state
|
|
||||||
"snap" phase. Small block differences cause a small state delta, which are
|
|
||||||
patched automatically during "heal" traversals.
|
|
||||||
|
|
||||||
- During the state "heal" phase, `stateRoot` should not be updated on every
|
|
||||||
block change, because it disrupts the "heal" traversal when this happens.
|
|
||||||
|
|
||||||
It should be kept the same for longer, but not too long because the `snap/1`
|
|
||||||
protocol does not provide state older than 128 blocks ago.
|
|
||||||
|
|
||||||
So during "heal", `stateRoot` should be updated roughly every N blocks where
|
|
||||||
N is close to 128, except when the heal is disrupted due to chain reorgs
|
|
||||||
taking place or other loss of available state from the peer.
|
|
||||||
|
|
||||||
- During the state "heal" phase, `stateRoot` must be coordinated among all
|
|
||||||
the peers. This is because "heal" converges a patchwork of states from
|
|
||||||
different times into a unified point-in-time whole state, so that execution
|
|
||||||
can proceed using entirely local data from there.
|
|
||||||
|
|
||||||
|
|
||||||
## Sync: Add `genesisStateRoot` for state syncing
|
|
||||||
|
|
||||||
State syncing requires the `stateRoot` value of the selected block to sync to.
|
|
||||||
Normally the chain head tracker selects a block and uses `block.stateRoot`.
|
|
||||||
|
|
||||||
However, in some cases in test environments, the chain head tracker finds the
|
|
||||||
sync block is 0, the genesis block, without receiving that block from a peer.
|
|
||||||
Of course this only happens when connecting to peers that are on block 0
|
|
||||||
themselves, but it can happen and must be handled.
|
|
||||||
|
|
||||||
Perhaps we should not run state sync on block 0, and instead the local trie.
|
|
||||||
But to get the correct "flat" or "snap sync" style representation that requires
|
|
||||||
special code.
|
|
||||||
|
|
||||||
In order to exercise the state sync code and see how peers behave when block 0
|
|
||||||
is selected, and avoid special code, use the genesis `stateRoot` found locally,
|
|
||||||
and sync that state from peers like any other.
|
|
|
@ -1,404 +0,0 @@
|
||||||
# Nimbus - Types, data structures and shared utilities used in network sync
|
|
||||||
#
|
|
||||||
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
|
||||||
# Licensed under either of
|
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
|
||||||
# http://www.apache.org/licenses/LICENSE-2.0)
|
|
||||||
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
|
||||||
# http://opensource.org/licenses/MIT)
|
|
||||||
# at your option. This file may not be copied, modified, or
|
|
||||||
# distributed except according to those terms.
|
|
||||||
|
|
||||||
import
|
|
||||||
std/[math, sequtils, strutils, hashes],
|
|
||||||
eth/common/eth_types,
|
|
||||||
nimcrypto/keccak,
|
|
||||||
stew/[byteutils, interval_set],
|
|
||||||
stint,
|
|
||||||
../../constants,
|
|
||||||
../types
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
|
||||||
|
|
||||||
type
|
|
||||||
NodeTag* = ##\
|
|
||||||
## Trie leaf item, account hash etc.
|
|
||||||
distinct UInt256
|
|
||||||
|
|
||||||
LeafRange* = ##\
|
|
||||||
## Interval `[minPt,maxPt]` of` NodeTag` elements, can be managed in an
|
|
||||||
## `IntervalSet` data type.
|
|
||||||
Interval[NodeTag,UInt256]
|
|
||||||
|
|
||||||
LeafRangeSet* = ##\
|
|
||||||
## Managed structure to handle non-adjacent `LeafRange` intervals
|
|
||||||
IntervalSetRef[NodeTag,UInt256]
|
|
||||||
|
|
||||||
PathSegment* = object
|
|
||||||
## Path prefix or trailer for an interior node in a hexary trie. See also
|
|
||||||
## the implementation of `NibblesSeq` from `eth/trie/nibbles` for a more
|
|
||||||
## general implementation.
|
|
||||||
bytes: seq[byte] ## <tag> + at most 32 bytes (aka 64 nibbles)
|
|
||||||
|
|
||||||
PathSegmentError = enum
|
|
||||||
isNoError = 0
|
|
||||||
isTooLongEvenLength ## More than 64 nibbles (even number)
|
|
||||||
isTooLongOddLength ## More than 63 nibbles (odd number)
|
|
||||||
isUnknownType ## Unknown encoduing type
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public helpers
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc to*(nid: NodeTag; T: type Hash256): T =
|
|
||||||
result.data = nid.UInt256.toBytesBE
|
|
||||||
|
|
||||||
proc to*(nid: NodeTag; T: type NodeHash): T =
|
|
||||||
nid.to(Hash256).T
|
|
||||||
|
|
||||||
proc to*(h: Hash256; T: type NodeTag): T =
|
|
||||||
UInt256.fromBytesBE(h.data).T
|
|
||||||
|
|
||||||
proc to*(nh: NodeHash; T: type NodeTag): T =
|
|
||||||
nh.Hash256.to(T)
|
|
||||||
|
|
||||||
proc to*(n: SomeUnsignedInt|UInt256; T: type NodeTag): T =
|
|
||||||
n.u256.T
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public constructors
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc new*(T: type NodeHash; ps: PathSegment): T =
|
|
||||||
## Import `PathSegment` argument into a `LeafTtemData`. Missing nibbles on the
|
|
||||||
## right will be zero padded.
|
|
||||||
if (ps.bytes[0] and 0x10) == 0:
|
|
||||||
for n in 1 ..< ps.bytes.len:
|
|
||||||
result.Hash256.data[n-1] = ps.bytes[n]
|
|
||||||
else:
|
|
||||||
for n in 0 ..< ps.bytes.len:
|
|
||||||
result.Hash256.data[n] = (ps.bytes[n] shl 4) or (ps.bytes[n+1] shr 4)
|
|
||||||
|
|
||||||
proc new*(T: type NodeTag; ps: PathSegment): T =
|
|
||||||
## Import `PathSegment` argument into a `LeafTtem`. Missing nibbles on the
|
|
||||||
## right will be zero padded.
|
|
||||||
NodeHash.new(ps).to(NodeTag)
|
|
||||||
|
|
||||||
proc init*(nh: var NodeHash; data: openArray[byte]): bool =
|
|
||||||
## Import argument `data` into `nh` which must have length either `32` or `0`.
|
|
||||||
## The latter case is equivalent to an all zero byte array of size `32`.
|
|
||||||
if data.len == 32:
|
|
||||||
for n in 0 ..< 32:
|
|
||||||
nh.Hash256.data[n] = data[n]
|
|
||||||
return true
|
|
||||||
elif data.len == 0:
|
|
||||||
nh.reset
|
|
||||||
return true
|
|
||||||
|
|
||||||
proc init*(nt: var NodeTag; data: openArray[byte]): bool =
|
|
||||||
## Similar to `init(li: var NodeTag; ps: PathSegment)`
|
|
||||||
var h: NodeHash
|
|
||||||
if h.init(data):
|
|
||||||
nt = h.to(NodeTag)
|
|
||||||
return true
|
|
||||||
|
|
||||||
proc init*(ps: var PathSegment; data: openArray[byte]): bool =
|
|
||||||
## Import argument `data` into `ps` which must be a valid path as found
|
|
||||||
## in a trie extension or leaf node starting with:
|
|
||||||
## * 0x00, or 0x20: followed by at most 64 nibbles (i.e. by 32 bytes max),
|
|
||||||
## Here, data path is made up of the at most 32 pairs of nibbles.
|
|
||||||
## * 0x1x, or 0x3x: followed by at most 62 nibbles (31 bytes max). Here the
|
|
||||||
## data path value starts with the `x` followed by the at most 62 pairs of
|
|
||||||
## nibbles.
|
|
||||||
if 0 < data.len:
|
|
||||||
# Check first byte for marker
|
|
||||||
if ((data[0] and 0xdf) == 0x00 and data.len <= 33) or # right nibble 0
|
|
||||||
((data[0] and 0xd0) == 0x10 and data.len <= 32): # right nibble 1st dgt
|
|
||||||
ps.bytes = data.toSeq
|
|
||||||
return true
|
|
||||||
|
|
||||||
proc new*(T: type PathSegment; tag: NodeTag; isLeaf = false): T =
|
|
||||||
## Create `PathSegment` from `NodeTag`. If the `isLeaf` argument is set, the
|
|
||||||
## path segment is marked as a leaf node (trie prefix' 0x20').
|
|
||||||
result.bytes = @[0.byte] & tag.to(Hash256).data.toSeq
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public `PathSegment` functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc verify*(ps: PathSegment): Result[void,PathSegmentError] =
|
|
||||||
## Check `ip` for consistency
|
|
||||||
if ps.bytes.len == 0:
|
|
||||||
return ok()
|
|
||||||
if (ps.bytes[0] and 0xdf) == 0:
|
|
||||||
if 33 < ps.bytes.len:
|
|
||||||
return err(isTooLongEvenLength)
|
|
||||||
elif (ps.bytes[0] and 0xd0) == 0x10:
|
|
||||||
if 32 < ps.bytes.len:
|
|
||||||
return err(isTooLongOddLength)
|
|
||||||
else:
|
|
||||||
return err(isUnknownType)
|
|
||||||
ok()
|
|
||||||
|
|
||||||
proc len*(ps: PathSegment): int =
|
|
||||||
## Returns the number of nibbles in the range 0..64.
|
|
||||||
if ps.bytes.len == 0:
|
|
||||||
0
|
|
||||||
elif (ps.bytes[0] and 0x10) == 0:
|
|
||||||
2 * ps.bytes.len - 2
|
|
||||||
else:
|
|
||||||
2 * ps.bytes.len - 1
|
|
||||||
|
|
||||||
proc setLen*(ps: var PathSegment; newLen: int) =
|
|
||||||
## Truncate or extend the length (i.e. the number of nibbles) of the argument
|
|
||||||
## `ip` to `newLen` bertwwn 0..63. When extending, new nibbles are zero
|
|
||||||
## initialised.
|
|
||||||
## This function throws an assertion defect if the `newLen` argument is
|
|
||||||
## outside the range 0..64.
|
|
||||||
doAssert 0 <= newLen and newLen <= 64
|
|
||||||
if ps.bytes.len == 0:
|
|
||||||
ps.bytes = @[0.byte]
|
|
||||||
if (ps.bytes[0] and 0x10) == 0:
|
|
||||||
if (newLen and 1) == 0: # both, old and new lengths are even
|
|
||||||
ps.bytes.setLen(1 + (newLen shr 1))
|
|
||||||
else: # new length odd, need to shift nibbles
|
|
||||||
let newBytesLen = (newLen + 1) shr 1
|
|
||||||
ps.bytes[0] = ps.bytes[0] or 0x10
|
|
||||||
if 1 < ps.bytes.len:
|
|
||||||
ps.bytes[0] = ps.bytes[0] or (ps.bytes[1] shr 4)
|
|
||||||
for n in 1 ..< min(ps.bytes.len-1, newBytesLen):
|
|
||||||
ps.bytes[n] = (ps.bytes[n] shl 4) or (ps.bytes[n+1] shr 4)
|
|
||||||
ps.bytes.setLen(newBytesLen)
|
|
||||||
else:
|
|
||||||
if (newLen and 1) == 1: # both, old and new lengths are odd
|
|
||||||
ps.bytes.setLen((newLen + 1) shr 1)
|
|
||||||
else: # new even length => shift nibbles right
|
|
||||||
let oldBytesLen = ps.bytes.len
|
|
||||||
ps.bytes.setLen((newLen shr 1) + 1)
|
|
||||||
for n in countDown(min(ps.bytes.len-1,oldBytesLen),1):
|
|
||||||
ps.bytes[n] = (ps.bytes[n-1] shl 4) or (ps.bytes[n] shr 4)
|
|
||||||
ps.bytes[0] = ps.bytes[0] and 0xd0
|
|
||||||
|
|
||||||
proc `[]`*(ps: PathSegment; nibbleInx: int): int =
|
|
||||||
## Extract the nibble (aka hex digit) value at the argument position index
|
|
||||||
## `nibbleInx`. If the position index `nibbleInx` does not relate to a valid
|
|
||||||
## nibble position, `0` is returned
|
|
||||||
##
|
|
||||||
## This function throws an assertion defect if the `nibbleInx` is outside
|
|
||||||
## the range 0..63.
|
|
||||||
doAssert 0 <= nibbleInx and nibbleInx < 64
|
|
||||||
if ps.bytes.len == 0:
|
|
||||||
result = 0
|
|
||||||
elif (ps.bytes[0] and 0x10) == 0:
|
|
||||||
let byteInx = (nibbleInx shr 1) + 1
|
|
||||||
if (nibbleInx and 1) == 0:
|
|
||||||
result = ps.bytes[byteInx].int shr 4
|
|
||||||
else:
|
|
||||||
result = ps.bytes[byteInx].int and 0x0f
|
|
||||||
else:
|
|
||||||
let byteInx = (nibbleInx + 1) shr 1
|
|
||||||
if (nibbleInx and 1) == 0:
|
|
||||||
result = ps.bytes[byteInx].int and 0x0f
|
|
||||||
else:
|
|
||||||
result = ps.bytes[byteInx].int shr 4
|
|
||||||
|
|
||||||
proc `[]=`*(ps: var PathSegment; nibbleInx: int; value: int) =
|
|
||||||
## Assign a nibble (aka hex) value `value` at position `nibbleInx`. If the
|
|
||||||
## length of the argument `ip` was smaller than the `nibbleInx`, the length
|
|
||||||
## will be extended to include that nibble.
|
|
||||||
##
|
|
||||||
## This function throws an assertion defect if the `nibbleInx` is outside
|
|
||||||
## the range 0..63, or if `value` is outside 0..15.
|
|
||||||
doAssert 0 <= nibbleInx and nibbleInx < 64
|
|
||||||
doAssert 0 <= value and value < 16
|
|
||||||
if ps.len <= nibbleInx:
|
|
||||||
if ps.bytes.len == 0:
|
|
||||||
ps.bytes = @[0.byte]
|
|
||||||
ps.setLen(nibbleInx + 1)
|
|
||||||
if (ps.bytes[0] and 0x10) == 0:
|
|
||||||
let byteInx = (nibbleInx shr 1) + 1
|
|
||||||
if (nibbleInx and 1) == 0:
|
|
||||||
ps.bytes[byteInx] = (value.uint8 shl 4) or (ps.bytes[byteInx] and 0x0f)
|
|
||||||
else:
|
|
||||||
ps.bytes[byteInx] = (ps.bytes[byteInx] and 0xf0) or value.uint8
|
|
||||||
else:
|
|
||||||
let byteInx = (nibbleInx + 1) shr 1
|
|
||||||
if (nibbleInx and 1) == 0:
|
|
||||||
ps.bytes[byteInx] = (ps.bytes[byteInx] and 0xf0) or value.uint8
|
|
||||||
else:
|
|
||||||
ps.bytes[byteInx] = (value.uint8 shl 4) or (ps.bytes[byteInx] and 0x0f)
|
|
||||||
|
|
||||||
proc `$`*(ps: PathSegment): string =
|
|
||||||
$ps.len & "#" & ps.bytes.mapIt(it.toHex(2)).join.toLowerAscii
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public rlp support
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc read*(rlp: var Rlp, T: type NodeTag): T
|
|
||||||
{.gcsafe, raises: [Defect,RlpError]} =
|
|
||||||
rlp.read(Hash256).to(T)
|
|
||||||
|
|
||||||
proc append*(writer: var RlpWriter, nid: NodeTag) =
|
|
||||||
writer.append(nid.to(Hash256))
|
|
||||||
|
|
||||||
# -------------
|
|
||||||
|
|
||||||
proc snapRead*(rlp: var Rlp; T: type Account; strict: static[bool] = false): T
|
|
||||||
{.gcsafe, raises: [Defect, RlpError]} =
|
|
||||||
## RLP decoding for `Account`. The `snap` RLP representation of the account
|
|
||||||
## differs from standard `Account` RLP. Empty storage hash and empty code
|
|
||||||
## hash are each represented by an RLP zero-length string instead of the
|
|
||||||
## full hash.
|
|
||||||
##
|
|
||||||
## Normally, this read function will silently handle standard encodinig and
|
|
||||||
## `snap` enciding. Setting the argument strict as `false` the function will
|
|
||||||
## throw an exception if `snap` encoding is violated.
|
|
||||||
rlp.tryEnterList()
|
|
||||||
result.nonce = rlp.read(typeof(result.nonce))
|
|
||||||
result.balance = rlp.read(typeof(result.balance))
|
|
||||||
if rlp.blobLen != 0 or not rlp.isBlob:
|
|
||||||
result.storageRoot = rlp.read(typeof(result.storageRoot))
|
|
||||||
when strict:
|
|
||||||
if result.storageRoot == BLANK_ROOT_HASH:
|
|
||||||
raise newException(RlpTypeMismatch,
|
|
||||||
"BLANK_ROOT_HASH not encoded as empty string in Snap protocol")
|
|
||||||
else:
|
|
||||||
rlp.skipElem()
|
|
||||||
result.storageRoot = BLANK_ROOT_HASH
|
|
||||||
if rlp.blobLen != 0 or not rlp.isBlob:
|
|
||||||
result.codeHash = rlp.read(typeof(result.codeHash))
|
|
||||||
when strict:
|
|
||||||
if result.codeHash == EMPTY_SHA3:
|
|
||||||
raise newException(RlpTypeMismatch,
|
|
||||||
"EMPTY_SHA3 not encoded as empty string in Snap protocol")
|
|
||||||
else:
|
|
||||||
rlp.skipElem()
|
|
||||||
result.codeHash = EMPTY_SHA3
|
|
||||||
|
|
||||||
proc snapAppend*(writer: var RlpWriter; account: Account) =
|
|
||||||
## RLP encoding for `Account`. The snap RLP representation of the account
|
|
||||||
## differs from standard `Account` RLP. Empty storage hash and empty code
|
|
||||||
## hash are each represented by an RLP zero-length string instead of the
|
|
||||||
## full hash.
|
|
||||||
writer.startList(4)
|
|
||||||
writer.append(account.nonce)
|
|
||||||
writer.append(account.balance)
|
|
||||||
if account.storageRoot == BLANK_ROOT_HASH:
|
|
||||||
writer.append("")
|
|
||||||
else:
|
|
||||||
writer.append(account.storageRoot)
|
|
||||||
if account.codeHash == EMPTY_SHA3:
|
|
||||||
writer.append("")
|
|
||||||
else:
|
|
||||||
writer.append(account.codeHash)
|
|
||||||
|
|
||||||
# -------------
|
|
||||||
|
|
||||||
proc compactRead*(rlp: var Rlp, T: type PathSegment): T
|
|
||||||
{.gcsafe, raises: [Defect,RlpError]} =
|
|
||||||
## Read compact encoded path segment
|
|
||||||
rlp.tryEnterList()
|
|
||||||
let
|
|
||||||
path = rlp.read(array[32, byte])
|
|
||||||
length = rlp.read(byte)
|
|
||||||
if 64 < length:
|
|
||||||
raise newException(
|
|
||||||
MalformedRlpError, "More the most 64 nibbles for PathSegment")
|
|
||||||
if (length and 1) == 0:
|
|
||||||
# initalise as even extension
|
|
||||||
result.bytes.setLen(1 + (length shr 1))
|
|
||||||
for n in 1 ..< result.bytes.len:
|
|
||||||
result.bytes[n] = path[n-1]
|
|
||||||
else:
|
|
||||||
# initalise as odd extension
|
|
||||||
result.bytes.setLen((length + 1) shr 1)
|
|
||||||
result.bytes[0] = 0x10 or (path[0] shl 4)
|
|
||||||
for n in 1 ..< result.bytes.len:
|
|
||||||
result.bytes[n] = (path[n-1] shl 4) or (path[n] shr 4)
|
|
||||||
|
|
||||||
proc compactAppend*(writer: var RlpWriter, ps: PathSegment) =
|
|
||||||
## Append compact encoded path segment
|
|
||||||
var path: array[32, byte]
|
|
||||||
if (ps.bytes[0] and 0x10) == 0:
|
|
||||||
for n in 1 ..< ps.bytes.len:
|
|
||||||
path[n-1] = ps.bytes[n]
|
|
||||||
else:
|
|
||||||
for n in 1 ..< ps.bytes.len:
|
|
||||||
path[n-1] = (ps.bytes[n-1] shl 4) or (ps.bytes[n] shr 4)
|
|
||||||
path[ps.bytes.len-1] = ps.bytes[^1] shl 4
|
|
||||||
writer.startList(2)
|
|
||||||
writer.append(path)
|
|
||||||
writer.append(ps.len.byte)
|
|
||||||
|
|
||||||
# -------------
|
|
||||||
|
|
||||||
proc dbRead*(rlp: var Rlp, T: type PathSegment): T
|
|
||||||
{.gcsafe, raises: [Defect,RlpError]} =
|
|
||||||
## Read as stored in the database
|
|
||||||
result.bytes = rlp.read(Blob)
|
|
||||||
|
|
||||||
proc dbAppend*(writer: var RlpWriter, ps: PathSegment) =
|
|
||||||
## Append in database record format
|
|
||||||
writer.append(ps.bytes)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public `NodeTag` and `LeafRange` functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc u256*(lp: NodeTag): UInt256 = lp.UInt256
|
|
||||||
proc low*(T: type NodeTag): T = low(UInt256).T
|
|
||||||
proc high*(T: type NodeTag): T = high(UInt256).T
|
|
||||||
|
|
||||||
proc `+`*(a: NodeTag; b: UInt256): NodeTag = (a.u256+b).NodeTag
|
|
||||||
proc `-`*(a: NodeTag; b: UInt256): NodeTag = (a.u256-b).NodeTag
|
|
||||||
proc `-`*(a, b: NodeTag): UInt256 = (a.u256 - b.u256)
|
|
||||||
|
|
||||||
proc `==`*(a, b: NodeTag): bool = a.u256 == b.u256
|
|
||||||
proc `<=`*(a, b: NodeTag): bool = a.u256 <= b.u256
|
|
||||||
proc `<`*(a, b: NodeTag): bool = a.u256 < b.u256
|
|
||||||
|
|
||||||
proc hash*(a: NodeTag): Hash =
|
|
||||||
## Mixin for `Table` or `keyedQueue`
|
|
||||||
a.to(Hash256).data.hash
|
|
||||||
|
|
||||||
proc digestTo*(data: Blob; T: type NodeTag): T =
|
|
||||||
## Hash the `data` argument
|
|
||||||
keccak256.digest(data).to(T)
|
|
||||||
|
|
||||||
proc freeFactor*(lrs: LeafRangeSet): float =
|
|
||||||
## Free factor, ie. `#items-free / 2^256` to be used in statistics
|
|
||||||
if 0 < lrs.total:
|
|
||||||
((high(NodeTag) - lrs.total).u256 + 1).to(float) / (2.0^256)
|
|
||||||
elif lrs.chunks == 0:
|
|
||||||
1.0
|
|
||||||
else:
|
|
||||||
0.0
|
|
||||||
|
|
||||||
# Printing & pretty printing
|
|
||||||
proc `$`*(nt: NodeTag): string =
|
|
||||||
if nt == high(NodeTag):
|
|
||||||
"high(NodeTag)"
|
|
||||||
elif nt == 0.u256.NodeTag:
|
|
||||||
"0"
|
|
||||||
else:
|
|
||||||
nt.to(Hash256).data.toHex
|
|
||||||
|
|
||||||
proc leafRangePp*(a, b: NodeTag): string =
|
|
||||||
## Needed for macro generated DSL files like `snap.nim` because the
|
|
||||||
## `distinct` flavour of `NodeTag` is discarded there.
|
|
||||||
result = "[" & $a
|
|
||||||
if a != b:
|
|
||||||
result &= ',' & $b
|
|
||||||
result &= "]"
|
|
||||||
|
|
||||||
proc `$`*(a, b: NodeTag): string =
|
|
||||||
## Prettyfied prototype
|
|
||||||
leafRangePp(a,b)
|
|
||||||
|
|
||||||
proc `$`*(iv: LeafRange): string =
|
|
||||||
leafRangePp(iv.minPt, iv.maxPt)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# End
|
|
||||||
# ------------------------------------------------------------------------------
|
|
|
@ -0,0 +1,153 @@
|
||||||
|
# Nimbus - Types, data structures and shared utilities used in network sync
|
||||||
|
#
|
||||||
|
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
||||||
|
# Licensed under either of
|
||||||
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0)
|
||||||
|
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
||||||
|
# http://opensource.org/licenses/MIT)
|
||||||
|
# at your option. This file may not be copied, modified, or
|
||||||
|
# distributed except according to those terms.
|
||||||
|
|
||||||
|
import
|
||||||
|
std/[math, hashes],
|
||||||
|
eth/common/eth_types,
|
||||||
|
nimcrypto/keccak,
|
||||||
|
stew/[byteutils, interval_set],
|
||||||
|
stint,
|
||||||
|
../../constants,
|
||||||
|
../types
|
||||||
|
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
type
|
||||||
|
NodeTag* = ##\
|
||||||
|
## Trie leaf item, account hash etc.
|
||||||
|
distinct UInt256
|
||||||
|
|
||||||
|
LeafRange* = ##\
|
||||||
|
## Interval `[minPt,maxPt]` of` NodeTag` elements, can be managed in an
|
||||||
|
## `IntervalSet` data type.
|
||||||
|
Interval[NodeTag,UInt256]
|
||||||
|
|
||||||
|
LeafRangeSet* = ##\
|
||||||
|
## Managed structure to handle non-adjacent `LeafRange` intervals
|
||||||
|
IntervalSetRef[NodeTag,UInt256]
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public helpers
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc to*(nid: NodeTag; T: type Hash256): T =
|
||||||
|
## Convert to serialised equivalent
|
||||||
|
result.data = nid.UInt256.toBytesBE
|
||||||
|
|
||||||
|
proc to*(nid: NodeTag; T: type NodeHash): T =
|
||||||
|
## Syntactic sugar
|
||||||
|
nid.to(Hash256).T
|
||||||
|
|
||||||
|
proc to*(h: Hash256; T: type NodeTag): T =
|
||||||
|
## Convert from serialised equivalent
|
||||||
|
UInt256.fromBytesBE(h.data).T
|
||||||
|
|
||||||
|
proc to*(nh: NodeHash; T: type NodeTag): T =
|
||||||
|
## Syntactic sugar
|
||||||
|
nh.Hash256.to(T)
|
||||||
|
|
||||||
|
proc to*(n: SomeUnsignedInt|UInt256; T: type NodeTag): T =
|
||||||
|
## Syntactic sugar
|
||||||
|
n.u256.T
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public constructors
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc init*(nh: var NodeHash; data: openArray[byte]): bool =
|
||||||
|
## Import argument `data` into `nh` which must have length either `32` or `0`.
|
||||||
|
## The latter case is equivalent to an all zero byte array of size `32`.
|
||||||
|
if data.len == 32:
|
||||||
|
for n in 0 ..< 32:
|
||||||
|
nh.Hash256.data[n] = data[n]
|
||||||
|
return true
|
||||||
|
elif data.len == 0:
|
||||||
|
nh.reset
|
||||||
|
return true
|
||||||
|
|
||||||
|
proc init*(nt: var NodeTag; data: openArray[byte]): bool =
|
||||||
|
## Similar to `init(nh: var NodeHash; .)`.
|
||||||
|
var h: NodeHash
|
||||||
|
if h.init(data):
|
||||||
|
nt = h.to(NodeTag)
|
||||||
|
return true
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public rlp support
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc read*(rlp: var Rlp, T: type NodeTag): T
|
||||||
|
{.gcsafe, raises: [Defect,RlpError]} =
|
||||||
|
rlp.read(Hash256).to(T)
|
||||||
|
|
||||||
|
proc append*(writer: var RlpWriter, nid: NodeTag) =
|
||||||
|
writer.append(nid.to(Hash256))
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public `NodeTag` and `LeafRange` functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc u256*(lp: NodeTag): UInt256 = lp.UInt256
|
||||||
|
proc low*(T: type NodeTag): T = low(UInt256).T
|
||||||
|
proc high*(T: type NodeTag): T = high(UInt256).T
|
||||||
|
|
||||||
|
proc `+`*(a: NodeTag; b: UInt256): NodeTag = (a.u256+b).NodeTag
|
||||||
|
proc `-`*(a: NodeTag; b: UInt256): NodeTag = (a.u256-b).NodeTag
|
||||||
|
proc `-`*(a, b: NodeTag): UInt256 = (a.u256 - b.u256)
|
||||||
|
|
||||||
|
proc `==`*(a, b: NodeTag): bool = a.u256 == b.u256
|
||||||
|
proc `<=`*(a, b: NodeTag): bool = a.u256 <= b.u256
|
||||||
|
proc `<`*(a, b: NodeTag): bool = a.u256 < b.u256
|
||||||
|
|
||||||
|
proc hash*(a: NodeTag): Hash =
|
||||||
|
## Mixin for `Table` or `keyedQueue`
|
||||||
|
a.to(Hash256).data.hash
|
||||||
|
|
||||||
|
proc digestTo*(data: Blob; T: type NodeTag): T =
|
||||||
|
## Hash the `data` argument
|
||||||
|
keccak256.digest(data).to(T)
|
||||||
|
|
||||||
|
proc freeFactor*(lrs: LeafRangeSet): float =
|
||||||
|
## Free factor, ie. `#items-free / 2^256` to be used in statistics
|
||||||
|
if 0 < lrs.total:
|
||||||
|
((high(NodeTag) - lrs.total).u256 + 1).to(float) / (2.0^256)
|
||||||
|
elif lrs.chunks == 0:
|
||||||
|
1.0
|
||||||
|
else:
|
||||||
|
0.0
|
||||||
|
|
||||||
|
# Printing & pretty printing
|
||||||
|
proc `$`*(nt: NodeTag): string =
|
||||||
|
if nt == high(NodeTag):
|
||||||
|
"high(NodeTag)"
|
||||||
|
elif nt == 0.u256.NodeTag:
|
||||||
|
"0"
|
||||||
|
else:
|
||||||
|
nt.to(Hash256).data.toHex
|
||||||
|
|
||||||
|
proc leafRangePp*(a, b: NodeTag): string =
|
||||||
|
## Needed for macro generated DSL files like `snap.nim` because the
|
||||||
|
## `distinct` flavour of `NodeTag` is discarded there.
|
||||||
|
result = "[" & $a
|
||||||
|
if a != b:
|
||||||
|
result &= ',' & $b
|
||||||
|
result &= "]"
|
||||||
|
|
||||||
|
proc `$`*(a, b: NodeTag): string =
|
||||||
|
## Prettyfied prototype
|
||||||
|
leafRangePp(a,b)
|
||||||
|
|
||||||
|
proc `$`*(iv: LeafRange): string =
|
||||||
|
leafRangePp(iv.minPt, iv.maxPt)
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# End
|
||||||
|
# ------------------------------------------------------------------------------
|
|
@ -1,4 +1,4 @@
|
||||||
# Nimbus - Rapidly converge on and track the canonical chain head of each peer
|
# Nimbus
|
||||||
#
|
#
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
# Licensed under either of
|
# Licensed under either of
|
||||||
|
@ -9,652 +9,221 @@
|
||||||
# at your option. This file may not be copied, modified, or distributed
|
# at your option. This file may not be copied, modified, or distributed
|
||||||
# except according to those terms.
|
# except according to those terms.
|
||||||
|
|
||||||
## This module fetches and tracks the canonical chain head of each connected
|
|
||||||
## peer. (Or in future, each peer we care about; we won't poll them all so
|
|
||||||
## often.)
|
|
||||||
##
|
|
||||||
## This is for when we aren't sure of the block number of a peer's canonical
|
|
||||||
## chain head. Most of the time, after finding which block, it quietly polls
|
|
||||||
## to track small updates to the "best" block number and hash of each peer.
|
|
||||||
##
|
|
||||||
## But sometimes that can get out of step. If there has been a deeper reorg
|
|
||||||
## than our tracking window, or a burst of more than a few new blocks, network
|
|
||||||
## delays, downtime, or the peer is itself syncing. Perhaps we stopped Nimbus
|
|
||||||
## and restarted a while later, e.g. suspending a laptop or Control-Z. Then
|
|
||||||
## this will catch up. It is even possible that the best hash the peer gave us
|
|
||||||
## in the `Status` handshake has disappeared by the time we query for the
|
|
||||||
## corresponding block number, so we start at zero.
|
|
||||||
##
|
|
||||||
## The steps here perform a robust and efficient O(log N) search to rapidly
|
|
||||||
## converge on the new best block if it's moved out of the polling window no
|
|
||||||
## matter where it starts, confirm the peer's canonical chain head boundary,
|
|
||||||
## then track the peer's chain head in real-time by polling. The method is
|
|
||||||
## robust to peer state changes at any time.
|
|
||||||
##
|
|
||||||
## The purpose is to:
|
|
||||||
##
|
|
||||||
## - Help with finding a peer common chain prefix ("fast sync pivot") in a
|
|
||||||
## consistent, fast and explicit way.
|
|
||||||
##
|
|
||||||
## - Catch up quickly after any long pauses of network downtime, program not
|
|
||||||
## running, or deep chain reorgs.
|
|
||||||
##
|
|
||||||
## - Be able to display real-time peer states, so they are less mysterious.
|
|
||||||
##
|
|
||||||
## - Tell the beam/snap/trie sync processes when to start and what blocks to
|
|
||||||
## fetch, and keep those fetchers in the head-adjacent window of the
|
|
||||||
## ever-changing chain.
|
|
||||||
##
|
|
||||||
## - Help the sync process bootstrap usefully when we only have one peer,
|
|
||||||
## speculatively fetching and validating what data we can before we have more
|
|
||||||
## peers to corroborate the consensus.
|
|
||||||
##
|
|
||||||
## - Help detect consensus failures in the network.
|
|
||||||
##
|
|
||||||
## We cannot assume a peer's canonical chain stays the same or only gains new
|
|
||||||
## blocks from one query to the next. There can be reorgs, including deep
|
|
||||||
## reorgs. When a reorg happens, the best block number can decrease if the new
|
|
||||||
## canonical chain is shorter than the old one, and the best block hash we
|
|
||||||
## previously knew can become unavailable on the peer. So we must detect when
|
|
||||||
## the current best block disappears and be able to reduce block number.
|
|
||||||
|
|
||||||
import
|
import
|
||||||
std/bitops,
|
std/[hashes, math, options, sets],
|
||||||
chronicles,
|
chronicles,
|
||||||
chronos,
|
chronos,
|
||||||
eth/[common/eth_types, p2p, p2p/private/p2p_types],
|
eth/[common/eth_types, p2p],
|
||||||
"../.."/[constants, p2p/chain/chain_desc],
|
stew/[interval_set, keyed_queue],
|
||||||
".."/[protocol, types],
|
".."/[protocol, sync_desc],
|
||||||
./worker/[worker_desc, fetch]
|
./worker/[accounts_db, fetch_accounts, pivot, ticker],
|
||||||
|
"."/[range_desc, worker_desc]
|
||||||
{.push raises: [Defect].}
|
|
||||||
|
|
||||||
export
|
|
||||||
worker_desc
|
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
topics = "snap-worker"
|
topics = "snap-sync"
|
||||||
|
|
||||||
const
|
|
||||||
syncLockedMinimumReply = 8
|
|
||||||
## Minimum number of headers we assume any peers will send if they have
|
|
||||||
## them in contiguous ascending queries. Fewer than this confirms we have
|
|
||||||
## found the peer's canonical chain head boundary. Must be at least 2, and
|
|
||||||
## at least `syncLockedQueryOverlap+2` to stay `SyncLocked` when the chain
|
|
||||||
## extends. Should not be large as that would be stretching assumptions
|
|
||||||
## about peer implementations. 8 is chosen as it allows 3-deep extensions
|
|
||||||
## and 3-deep reorgs to be followed in a single round trip.
|
|
||||||
|
|
||||||
syncLockedQueryOverlap = 4
|
|
||||||
## Number of headers to re-query on each poll when `SyncLocked` so that we
|
|
||||||
## get small reorg updates in one round trip. Must be no more than
|
|
||||||
## `syncLockedMinimumReply-1`, no more than `syncLockedMinimumReply-2` to
|
|
||||||
## stay `SyncLocked` when the chain extends, and not too large to avoid
|
|
||||||
## excessive duplicate fetching. 4 is chosen as it allows 3-deep reorgs
|
|
||||||
## to be followed in single round trip.
|
|
||||||
|
|
||||||
syncLockedQuerySize = 192
|
|
||||||
## Query size when polling `SyncLocked`. Must be at least
|
|
||||||
## `syncLockedMinimumReply`. Large is fine, if we get a large reply the
|
|
||||||
## values are almost always useful.
|
|
||||||
|
|
||||||
huntQuerySize = 16
|
|
||||||
## Query size when hunting for canonical head boundary. Small is good
|
|
||||||
## because we don't want to keep most of the headers at hunt time.
|
|
||||||
|
|
||||||
huntForwardExpandShift = 4
|
|
||||||
## Expansion factor during `HuntForward` exponential search.
|
|
||||||
## 16 is chosen for rapid convergence when bootstrapping or catching up.
|
|
||||||
|
|
||||||
huntBackwardExpandShift = 1
|
|
||||||
## Expansion factor during `HuntBackward` exponential search.
|
|
||||||
## 2 is chosen for better convergence when tracking a chain reorg.
|
|
||||||
|
|
||||||
type
|
|
||||||
WorkerMode = enum
|
|
||||||
## The current state of tracking the peer's canonical chain head.
|
|
||||||
## `bestBlockNumber` is only valid when this is `SyncLocked`.
|
|
||||||
SyncLocked
|
|
||||||
SyncOnlyHash
|
|
||||||
HuntForward
|
|
||||||
HuntBackward
|
|
||||||
HuntRange
|
|
||||||
HuntRangeFinal
|
|
||||||
|
|
||||||
WorkerHuntEx = ref object of WorkerBase
|
|
||||||
## Peer canonical chain head ("best block") search state.
|
|
||||||
syncMode: WorkerMode ## Action mode
|
|
||||||
startedFetch: bool ## Start download once, only
|
|
||||||
lowNumber: BlockNumber ## Recent lowest known block number.
|
|
||||||
highNumber: BlockNumber ## Recent highest known block number.
|
|
||||||
bestNumber: BlockNumber
|
|
||||||
bestHash: BlockHash
|
|
||||||
step: uint
|
|
||||||
|
|
||||||
static:
|
|
||||||
doAssert syncLockedMinimumReply >= 2
|
|
||||||
doAssert syncLockedMinimumReply >= syncLockedQueryOverlap + 2
|
|
||||||
doAssert syncLockedQuerySize <= maxHeadersFetch
|
|
||||||
doAssert huntQuerySize >= 1 and huntQuerySize <= maxHeadersFetch
|
|
||||||
doAssert huntForwardExpandShift >= 1 and huntForwardExpandShift <= 8
|
|
||||||
doAssert huntBackwardExpandShift >= 1 and huntBackwardExpandShift <= 8
|
|
||||||
|
|
||||||
# Make sure that request/response wire protocol messages are id-tracked and
|
|
||||||
# would not overlap (no multi-protocol legacy support)
|
|
||||||
doAssert 66 <= protocol.ethVersion
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private helpers
|
# Private helpers
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc hunt(sp: WorkerBuddy): WorkerHuntEx =
|
proc hash(h: Hash256): Hash =
|
||||||
sp.workerBase.WorkerHuntEx
|
## Mixin for `Table` or `keyedQueue`
|
||||||
|
h.data.hash
|
||||||
|
|
||||||
proc `hunt=`(sp: WorkerBuddy; value: WorkerHuntEx) =
|
proc meanStdDev(sum, sqSum: float; length: int): (float,float) =
|
||||||
sp.workerBase = value
|
if 0 < length:
|
||||||
|
result[0] = sum / length.float
|
||||||
proc new(T: type WorkerHuntEx; syncMode: WorkerMode): T =
|
result[1] = sqrt(sqSum / length.float - result[0] * result[0])
|
||||||
T(syncMode: syncMode,
|
|
||||||
lowNumber: 0.toBlockNumber.BlockNumber,
|
|
||||||
highNumber: high(BlockNumber).BlockNumber, # maximum uncertainty range.
|
|
||||||
bestNumber: 0.toBlockNumber.BlockNumber,
|
|
||||||
bestHash: ZERO_HASH256.BlockHash, # whatever
|
|
||||||
step: 0u)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private logging helpers
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc traceSyncLocked(sp: WorkerBuddy, number: BlockNumber, hash: BlockHash) =
|
|
||||||
## Trace messages when peer canonical head is confirmed or updated.
|
|
||||||
let
|
|
||||||
bestBlock = sp.ns.pp(hash, number)
|
|
||||||
peer = $sp
|
|
||||||
if sp.hunt.syncMode != SyncLocked:
|
|
||||||
debug "Now tracking chain head of peer", peer, bestBlock
|
|
||||||
elif number > sp.hunt.bestNumber:
|
|
||||||
if number == sp.hunt.bestNumber + 1:
|
|
||||||
debug "Peer chain head advanced one block", peer,
|
|
||||||
advance=1, bestBlock
|
|
||||||
else:
|
|
||||||
debug "Peer chain head advanced some blocks", peer,
|
|
||||||
advance=(sp.hunt.bestNumber - number), bestBlock
|
|
||||||
elif number < sp.hunt.bestNumber or hash != sp.hunt.bestHash:
|
|
||||||
debug "Peer chain head reorg detected", peer,
|
|
||||||
advance=(sp.hunt.bestNumber - number), bestBlock
|
|
||||||
|
|
||||||
# proc peerSyncChainTrace(sp: WorkerBuddy) =
|
|
||||||
# ## To be called after `peerSyncChainRequest` has updated state.
|
|
||||||
# case sp.hunt.syncMode:
|
|
||||||
# of SyncLocked:
|
|
||||||
# trace "SyncLocked",
|
|
||||||
# bestBlock = sp.ns.pp(sp.hunt.bestHash, sp.hunt.bestNumber)
|
|
||||||
# of SyncOnlyHash:
|
|
||||||
# trace "OnlyHash",
|
|
||||||
# bestBlock = sp.ns.pp(sp.hunt.bestHash, sp.hunt.bestNumber)
|
|
||||||
# of HuntForward:
|
|
||||||
# template highMax(n: BlockNumber): string =
|
|
||||||
# if n == high(BlockNumber): "max" else: $n
|
|
||||||
# trace "HuntForward",
|
|
||||||
# low=sp.hunt.lowNumber, high=highMax(sp.hunt.highNumber),
|
|
||||||
# step=sp.hunt.step
|
|
||||||
# of HuntBackward:
|
|
||||||
# trace "HuntBackward",
|
|
||||||
# low=sp.hunt.lowNumber, high=sp.hunt.highNumber, step=sp.hunt.step
|
|
||||||
# of HuntRange:
|
|
||||||
# trace "HuntRange",
|
|
||||||
# low=sp.hunt.lowNumber, high=sp.hunt.highNumber, step=sp.hunt.step
|
|
||||||
# of HuntRangeFinal:
|
|
||||||
# trace "HuntRangeFinal",
|
|
||||||
# low=sp.hunt.lowNumber, high=sp.hunt.highNumber, step=1
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private functions
|
# Private functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc setSyncLocked(sp: WorkerBuddy, number: BlockNumber, hash: BlockHash) =
|
proc rndNodeTag(buddy: SnapBuddyRef): NodeTag =
|
||||||
## Actions to take when peer canonical head is confirmed or updated.
|
## Create random node tag
|
||||||
sp.traceSyncLocked(number, hash)
|
let
|
||||||
sp.hunt.bestNumber = number
|
ctx = buddy.ctx
|
||||||
sp.hunt.bestHash = hash
|
peer = buddy.peer
|
||||||
sp.hunt.syncMode = SyncLocked
|
var data: array[32,byte]
|
||||||
|
ctx.data.rng[].generate(data)
|
||||||
|
UInt256.fromBytesBE(data).NodeTag
|
||||||
|
|
||||||
proc clearSyncStateRoot(sp: WorkerBuddy) =
|
|
||||||
if sp.ctrl.stateRoot.isSome:
|
|
||||||
debug "Stopping state sync from this peer", peer=sp
|
|
||||||
sp.ctrl.stateRoot = none(TrieHash)
|
|
||||||
|
|
||||||
proc lockSyncStateAndFetch(
|
proc setPivotEnv(buddy: SnapBuddyRef; header: BlockHeader) =
|
||||||
sp: WorkerBuddy,
|
## Activate environment for state root implied by `header` argument
|
||||||
number: BlockNumber,
|
let
|
||||||
hash: BlockHash,
|
ctx = buddy.ctx
|
||||||
stateRoot: TrieHash) =
|
key = header.stateRoot
|
||||||
sp.setSyncLocked(number, hash)
|
rc = ctx.data.pivotTable.lruFetch(key)
|
||||||
|
if rc.isOk:
|
||||||
let thisBlock = sp.ns.pp(hash, number)
|
ctx.data.pivotEnv = rc.value
|
||||||
if sp.ctrl.stateRoot.isNone:
|
|
||||||
debug "Starting state sync from this peer", peer=sp,
|
|
||||||
thisBlock, stateRoot
|
|
||||||
elif sp.ctrl.stateRoot.unsafeGet != stateRoot:
|
|
||||||
trace "Adjusting state sync root from this peer", peer=sp,
|
|
||||||
thisBlock, stateRoot
|
|
||||||
|
|
||||||
sp.ctrl.stateRoot = some(stateRoot)
|
|
||||||
|
|
||||||
if not sp.hunt.startedFetch:
|
|
||||||
sp.hunt.startedFetch = true
|
|
||||||
trace "Starting to download block state", peer=sp,
|
|
||||||
thisBlock, stateRoot
|
|
||||||
asyncSpawn sp.fetch()
|
|
||||||
|
|
||||||
proc setHuntBackward(sp: WorkerBuddy, lowestAbsent: BlockNumber) =
|
|
||||||
## Start exponential search mode backward due to new uncertainty.
|
|
||||||
sp.hunt.syncMode = HuntBackward
|
|
||||||
sp.hunt.step = 0
|
|
||||||
# Block zero is always present.
|
|
||||||
sp.hunt.lowNumber = 0.toBlockNumber
|
|
||||||
# Zero `lowestAbsent` is never correct, but an incorrect peer could send it.
|
|
||||||
sp.hunt.highNumber = if lowestAbsent > 0: lowestAbsent else: 1.toBlockNumber
|
|
||||||
sp.clearSyncStateRoot()
|
|
||||||
|
|
||||||
proc setHuntForward(sp: WorkerBuddy, highestPresent: BlockNumber) =
|
|
||||||
## Start exponential search mode forward due to new uncertainty.
|
|
||||||
sp.hunt.syncMode = HuntForward
|
|
||||||
sp.hunt.step = 0
|
|
||||||
sp.hunt.lowNumber = highestPresent
|
|
||||||
sp.hunt.highNumber = high(BlockNumber)
|
|
||||||
sp.clearSyncStateRoot()
|
|
||||||
|
|
||||||
proc updateHuntAbsent(sp: WorkerBuddy, lowestAbsent: BlockNumber) =
|
|
||||||
## Converge uncertainty range backward.
|
|
||||||
if lowestAbsent < sp.hunt.highNumber:
|
|
||||||
sp.hunt.highNumber = lowestAbsent
|
|
||||||
# If uncertainty range has moved outside the search window, change to hunt
|
|
||||||
# backward to block zero. Note that empty uncertainty range is allowed
|
|
||||||
# (empty range is `hunt.lowNumber + 1 == hunt.highNumber`).
|
|
||||||
if sp.hunt.highNumber <= sp.hunt.lowNumber:
|
|
||||||
sp.setHuntBackward(lowestAbsent)
|
|
||||||
sp.clearSyncStateRoot()
|
|
||||||
|
|
||||||
proc updateHuntPresent(sp: WorkerBuddy, highestPresent: BlockNumber) =
|
|
||||||
## Converge uncertainty range forward.
|
|
||||||
if highestPresent > sp.hunt.lowNumber:
|
|
||||||
sp.hunt.lowNumber = highestPresent
|
|
||||||
# If uncertainty range has moved outside the search window, change to hunt
|
|
||||||
# forward to no upper limit. Note that empty uncertainty range is allowed
|
|
||||||
# (empty range is `hunt.lowNumber + 1 == hunt.highNumber`).
|
|
||||||
if sp.hunt.lowNumber >= sp.hunt.highNumber:
|
|
||||||
sp.setHuntForward(highestPresent)
|
|
||||||
sp.clearSyncStateRoot()
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private functions, assemble request
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc peerSyncChainRequest(sp: WorkerBuddy): BlocksRequest =
|
|
||||||
## Choose `GetBlockHeaders` parameters when hunting or following the canonical
|
|
||||||
## chain of a peer.
|
|
||||||
if sp.hunt.syncMode == SyncLocked:
|
|
||||||
# Stable and locked. This is just checking for changes including reorgs.
|
|
||||||
# `sp.hunt.bestNumber` was recently the head of the peer's canonical
|
|
||||||
# chain. We must include this block number to detect when the canonical
|
|
||||||
# chain gets shorter versus no change.
|
|
||||||
result.startBlock.number =
|
|
||||||
if sp.hunt.bestNumber <= syncLockedQueryOverlap:
|
|
||||||
# Every peer should send genesis for block 0, so don't ask for it.
|
|
||||||
# `peerSyncChainEmptyReply` has logic to handle this reply as if it
|
|
||||||
# was for block 0. Aside from saving bytes, this is more robust if
|
|
||||||
# some client doesn't do genesis reply correctly.
|
|
||||||
1.toBlockNumber
|
|
||||||
else:
|
|
||||||
min(sp.hunt.bestNumber - syncLockedQueryOverlap.toBlockNumber,
|
|
||||||
high(BlockNumber) - (syncLockedQuerySize - 1).toBlockNumber)
|
|
||||||
result.maxResults = syncLockedQuerySize
|
|
||||||
return
|
return
|
||||||
|
|
||||||
if sp.hunt.syncMode == SyncOnlyHash:
|
let env = SnapPivotRef(
|
||||||
# We only have the hash of the recent head of the peer's canonical chain.
|
stateHeader: header,
|
||||||
# Like `SyncLocked`, query more than one item to detect when the
|
pivotAccount: buddy.rndNodeTag,
|
||||||
# canonical chain gets shorter, no change or longer.
|
availAccounts: LeafRangeSet.init())
|
||||||
result.startBlock = sp.hunt.bestHash.to(HashOrNum)
|
# Pre-filled with the largest possible interval
|
||||||
result.maxResults = syncLockedQuerySize
|
discard env.availAccounts.merge(low(NodeTag),high(NodeTag))
|
||||||
return
|
|
||||||
|
|
||||||
# Searching for the peers's canonical head. An ascending query is always
|
# Statistics
|
||||||
# used, regardless of search direction. This is because a descending query
|
ctx.data.pivotCount.inc
|
||||||
# (`reverse = true` and `maxResults > 1`) is useless for searching: Either
|
|
||||||
# `startBlock` is present, in which case the extra descending results
|
|
||||||
# contribute no more information about the canonical head boundary, or
|
|
||||||
# `startBlock` is absent in which case there are zero results. It's not
|
|
||||||
# defined in the `eth` specification that there must be zero results (in
|
|
||||||
# principle peers could return the lower numbered blocks), but in practice
|
|
||||||
# peers stop at the first absent block in the sequence from `startBlock`.
|
|
||||||
#
|
|
||||||
# Guaranteeing O(log N) time convergence in all scenarios requires some
|
|
||||||
# properties to be true in both exponential search (expanding) and
|
|
||||||
# quasi-binary search (converging in a range). The most important is that
|
|
||||||
# the gap to `startBlock` after `hunt.lowNumber` and also before
|
|
||||||
# `hunt.highNumber` are proportional to the query step, where the query step
|
|
||||||
# is `hunt.step` exponentially expanding each round, or `maxStep`
|
|
||||||
# approximately evenly distributed in the range.
|
|
||||||
#
|
|
||||||
# `hunt.lowNumber+1` must not be used consistently as the start, even with a
|
|
||||||
# large enough query step size, as that will sometimes take O(N) to converge
|
|
||||||
# in both the exponential and quasi-binary searches. (Ending at
|
|
||||||
# `hunt.highNumber-1` is fine if `huntQuerySize > 1`. This asymmetry is
|
|
||||||
# due to ascending queries (see earlier comment), and non-empty truncated
|
|
||||||
# query reply being proof of presence before the truncation point, but not
|
|
||||||
# proof of absence after it. A reply can be truncated just because the peer
|
|
||||||
# decides to.)
|
|
||||||
#
|
|
||||||
# The proportional gap requirement is why we divide by query size here,
|
|
||||||
# instead of stretching to fit more strictly with `(range-1)/(size-1)`.
|
|
||||||
|
|
||||||
const huntFinalSize = max(2, huntQuerySize)
|
ctx.data.pivotEnv = ctx.data.pivotTable.lruAppend(key, env, ctx.buddiesMax)
|
||||||
var maxStep = 0u
|
# -----
|
||||||
|
if ctx.data.proofDumpOk:
|
||||||
let fullRangeClamped =
|
let peer = buddy.peer
|
||||||
if sp.hunt.highNumber <= sp.hunt.lowNumber: 0u
|
trace "Snap proofs dump enabled", peer
|
||||||
else: min(high(uint).toBlockNumber,
|
ctx.data.proofDumpOk = false
|
||||||
sp.hunt.highNumber - sp.hunt.lowNumber).truncate(uint) - 1
|
env.proofDumpOk = true
|
||||||
|
#env.pivotAccount = 0.to(NodeTag)
|
||||||
if fullRangeClamped >= huntFinalSize: # `HuntRangeFinal` condition.
|
|
||||||
maxStep = if huntQuerySize == 1:
|
|
||||||
fullRangeClamped
|
|
||||||
elif (huntQuerySize and (huntQuerySize-1)) == 0:
|
|
||||||
fullRangeClamped shr fastLog2(huntQuerySize)
|
|
||||||
else:
|
|
||||||
fullRangeClamped div huntQuerySize
|
|
||||||
doAssert huntFinalSize >= huntQuerySize
|
|
||||||
doAssert maxStep >= 1 # Ensured by the above assertion.
|
|
||||||
|
|
||||||
# Check for exponential search (expanding). Iterate `hunt.step`. O(log N)
|
|
||||||
# requires `startBlock` to be offset from `hunt.lowNumber`/`hunt.highNumber`.
|
|
||||||
if sp.hunt.syncMode in {HuntForward, HuntBackward} and
|
|
||||||
fullRangeClamped >= huntFinalSize:
|
|
||||||
let forward = sp.hunt.syncMode == HuntForward
|
|
||||||
let expandShift = if forward: huntForwardExpandShift
|
|
||||||
else: huntBackwardExpandShift
|
|
||||||
# Switches to range search when this condition is no longer true.
|
|
||||||
if sp.hunt.step < maxStep shr expandShift:
|
|
||||||
# The `if` above means the next line cannot overflow.
|
|
||||||
sp.hunt.step = if sp.hunt.step > 0: sp.hunt.step shl expandShift else: 1
|
|
||||||
# Satisfy the O(log N) convergence conditions.
|
|
||||||
result.startBlock.number =
|
|
||||||
if forward: sp.hunt.lowNumber + sp.hunt.step.toBlockNumber
|
|
||||||
else: sp.hunt.highNumber - (sp.hunt.step * huntQuerySize).toBlockNumber
|
|
||||||
result.maxResults = huntQuerySize
|
|
||||||
result.skip = sp.hunt.step - 1
|
|
||||||
return
|
|
||||||
|
|
||||||
# For tracing/display.
|
|
||||||
sp.hunt.step = maxStep
|
|
||||||
sp.hunt.syncMode = HuntRange
|
|
||||||
if maxStep > 0:
|
|
||||||
# Quasi-binary search (converging in a range). O(log N) requires
|
|
||||||
# `startBlock` to satisfy the constraints described above, with the
|
|
||||||
# proportionality from both ends of the range. The optimal information
|
|
||||||
# gathering position is tricky and doesn't make much difference, so don't
|
|
||||||
# bother. We'll centre the query in the range.
|
|
||||||
var offset = fullRangeClamped - maxStep * (huntQuerySize-1)
|
|
||||||
# Rounding must bias towards end to ensure `offset >= 1` after this.
|
|
||||||
offset -= offset shr 1
|
|
||||||
result.startBlock.number = sp.hunt.lowNumber + offset.toBlockNumber
|
|
||||||
result.maxResults = huntQuerySize
|
|
||||||
result.skip = maxStep - 1
|
|
||||||
else:
|
|
||||||
# Small range, final step. At `fullRange == 0` we must query at least one
|
|
||||||
# block before and after the range to confirm the canonical head boundary,
|
|
||||||
# or find it has moved. This ensures progress without getting stuck. When
|
|
||||||
# `fullRange` is small this is also beneficial, to get `SyncLocked` in one
|
|
||||||
# round trip from hereand it simplifies the other search branches below.
|
|
||||||
# Ideally the query is similar to `SyncLocked`, enough to get `SyncLocked`
|
|
||||||
# in one round trip, and accommodate a small reorg or extension.
|
|
||||||
const afterSoftMax = syncLockedMinimumReply - syncLockedQueryOverlap
|
|
||||||
const beforeHardMax = syncLockedQueryOverlap
|
|
||||||
let extra = huntFinalSize - fullRangeClamped
|
|
||||||
var before = (extra + 1) shr 1
|
|
||||||
before = max(before + afterSoftMax, extra) - afterSoftMax
|
|
||||||
before = min(before, beforeHardMax)
|
|
||||||
# See `SyncLocked` case.
|
|
||||||
result.startBlock.number =
|
|
||||||
if sp.hunt.bestNumber <= before.toBlockNumber: 1.toBlockNumber
|
|
||||||
else: min(sp.hunt.bestNumber - before.toBlockNumber,
|
|
||||||
high(BlockNumber) - (huntFinalSize - 1).toBlockNumber)
|
|
||||||
result.maxResults = huntFinalSize
|
|
||||||
sp.hunt.syncMode = HuntRangeFinal
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private functions, reply handling
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc peerSyncChainEmptyReply(
|
|
||||||
sp: WorkerBuddy,
|
|
||||||
request: BlocksRequest) =
|
|
||||||
## Handle empty `GetBlockHeaders` reply. This means `request.startBlock` is
|
|
||||||
## absent on the peer. If it was `SyncLocked` there must have been a reorg
|
|
||||||
## and the previous canonical chain head has disappeared. If hunting, this
|
|
||||||
## updates the range of uncertainty.
|
|
||||||
|
|
||||||
# Treat empty response to a request starting from block 1 as equivalent to
|
|
||||||
# length 1 starting from block 0 in `peerSyncChainNonEmptyReply`. We treat
|
|
||||||
# every peer as if it would send genesis for block 0, without asking for it.
|
|
||||||
if request.skip == 0 and
|
|
||||||
not request.reverse and
|
|
||||||
not request.startBlock.isHash and
|
|
||||||
request.startBlock.number == 1.toBlockNumber:
|
|
||||||
sp.lockSyncStateAndFetch(
|
|
||||||
0.toBlockNumber,
|
|
||||||
sp.peer.network.chain.genesisHash.BlockHash,
|
|
||||||
sp.peer.network.chain.Chain.genesisStateRoot.TrieHash)
|
|
||||||
return
|
|
||||||
|
|
||||||
if sp.hunt.syncMode in {SyncLocked, SyncOnlyHash}:
|
|
||||||
inc sp.stats.ok.reorgDetected
|
|
||||||
trace "Peer reorg detected, best block disappeared", peer=sp,
|
|
||||||
startBlock=request.startBlock
|
|
||||||
|
|
||||||
let lowestAbsent = request.startBlock.number
|
|
||||||
case sp.hunt.syncMode:
|
|
||||||
of SyncLocked:
|
|
||||||
# If this message doesn't change our knowledge, ignore it.
|
|
||||||
if lowestAbsent > sp.hunt.bestNumber:
|
|
||||||
return
|
|
||||||
# Due to a reorg, peer's canonical head has lower block number, outside
|
|
||||||
# our tracking window. Sync lock is no longer valid. Switch to hunt
|
|
||||||
# backward to find the new canonical head.
|
|
||||||
sp.setHuntBackward(lowestAbsent)
|
|
||||||
of SyncOnlyHash:
|
|
||||||
# Due to a reorg, peer doesn't have the block hash it originally gave us.
|
|
||||||
# Switch to hunt forward from block zero to find the canonical head.
|
|
||||||
sp.setHuntForward(0.toBlockNumber)
|
|
||||||
of HuntForward, HuntBackward, HuntRange, HuntRangeFinal:
|
|
||||||
# Update the hunt range.
|
|
||||||
sp.updateHuntAbsent(lowestAbsent)
|
|
||||||
|
|
||||||
# Update best block number. It is invalid except when `SyncLocked`, but
|
|
||||||
# still useful as a hint of what we knew recently, for example in displays.
|
|
||||||
if lowestAbsent <= sp.hunt.bestNumber:
|
|
||||||
sp.hunt.bestNumber =
|
|
||||||
if lowestAbsent == 0.toBlockNumber: lowestAbsent
|
|
||||||
else: lowestAbsent - 1.toBlockNumber
|
|
||||||
sp.hunt.bestHash = default(typeof(sp.hunt.bestHash))
|
|
||||||
sp.ns.seen(sp.hunt.bestHash,sp.hunt.bestNumber)
|
|
||||||
|
|
||||||
|
|
||||||
proc peerSyncChainNonEmptyReply(
|
proc updatePivotEnv(buddy: SnapBuddyRef): bool =
|
||||||
sp: WorkerBuddy,
|
## Update global state root environment from local `pivotHeader`. Choose the
|
||||||
request: BlocksRequest,
|
## latest block number. Returns `true` if the environment was changed
|
||||||
headers: openArray[BlockHeader]) =
|
if buddy.data.pivotHeader.isSome:
|
||||||
## Handle non-empty `GetBlockHeaders` reply. This means `request.startBlock`
|
let
|
||||||
## is present on the peer and in its canonical chain (unless the request was
|
ctx = buddy.ctx
|
||||||
## made with a hash). If it's a short, contiguous, ascending order reply, it
|
newStateNumber = buddy.data.pivotHeader.unsafeGet.blockNumber
|
||||||
## reveals the abrupt transition at the end of the chain and we have learned
|
stateNumber = if ctx.data.pivotEnv.isNil: 0.toBlockNumber
|
||||||
## or reconfirmed the real-time head block. If hunting, this updates the
|
else: ctx.data.pivotEnv.stateHeader.blockNumber
|
||||||
## range of uncertainty.
|
if stateNumber + maxPivotBlockWindow < newStateNumber:
|
||||||
|
buddy.setPivotEnv(buddy.data.pivotHeader.get)
|
||||||
|
return true
|
||||||
|
|
||||||
let len = headers.len
|
|
||||||
let highestIndex = if request.reverse: 0 else: len - 1
|
|
||||||
|
|
||||||
# We assume a short enough reply means we've learned the peer's canonical
|
proc tickerUpdate*(ctx: SnapCtxRef): TickerStatsUpdater =
|
||||||
# head, because it would have replied with another header if not at the head.
|
result = proc: TickerStats =
|
||||||
# This is not justified when the request used a general hash, because the
|
var
|
||||||
# peer doesn't have to reply with its canonical chain in that case, except it
|
aSum, aSqSum, uSum, uSqSum: float
|
||||||
# is still justified if the hash was the known canonical head, which is
|
count = 0
|
||||||
# the case in a `SyncOnlyHash` request.
|
for kvp in ctx.data.pivotTable.nextPairs:
|
||||||
if len < syncLockedMinimumReply and
|
|
||||||
request.skip == 0 and not request.reverse and
|
|
||||||
len.uint < request.maxResults:
|
|
||||||
sp.lockSyncStateAndFetch(
|
|
||||||
headers[highestIndex].blockNumber,
|
|
||||||
headers[highestIndex].blockHash.BlockHash,
|
|
||||||
headers[highestIndex].stateRoot.TrieHash)
|
|
||||||
return
|
|
||||||
|
|
||||||
# Be careful, this number is from externally supplied data and arithmetic
|
# Accounts mean & variance
|
||||||
# in the upward direction could overflow.
|
let aLen = kvp.data.nAccounts.float
|
||||||
let highestPresent = headers[highestIndex].blockNumber
|
if 0 < aLen:
|
||||||
|
count.inc
|
||||||
|
aSum += aLen
|
||||||
|
aSqSum += aLen * aLen
|
||||||
|
|
||||||
# A reply that isn't short enough for the canonical head criterion above
|
# Fill utilisation mean & variance
|
||||||
# tells us headers up to some number, but it doesn't tell us if there are
|
let fill = kvp.data.availAccounts.freeFactor
|
||||||
# more after it in the peer's canonical chain. We have to request more
|
uSum += fill
|
||||||
# headers to find out.
|
uSqSum += fill * fill
|
||||||
case sp.hunt.syncMode:
|
|
||||||
of SyncLocked:
|
|
||||||
# If this message doesn't change our knowledge, ignore it.
|
|
||||||
if highestPresent <= sp.hunt.bestNumber:
|
|
||||||
return
|
|
||||||
# Sync lock is no longer valid as we don't have confirmed canonical head.
|
|
||||||
# Switch to hunt forward to find the new canonical head.
|
|
||||||
sp.setHuntForward(highestPresent)
|
|
||||||
of SyncOnlyHash:
|
|
||||||
# As `SyncLocked` but without the block number check.
|
|
||||||
sp.setHuntForward(highestPresent)
|
|
||||||
of HuntForward, HuntBackward, HuntRange, HuntRangeFinal:
|
|
||||||
# Update the hunt range.
|
|
||||||
sp.updateHuntPresent(highestPresent)
|
|
||||||
|
|
||||||
# Update best block number. It is invalid except when `SyncLocked`, but
|
let
|
||||||
# still useful as a hint of what we knew recently, for example in displays.
|
tabLen = ctx.data.pivotTable.len
|
||||||
if highestPresent > sp.hunt.bestNumber:
|
pivotBlock = if ctx.data.pivotEnv.isNil: none(BlockNumber)
|
||||||
sp.hunt.bestNumber = highestPresent
|
else: some(ctx.data.pivotEnv.stateHeader.blockNumber)
|
||||||
sp.hunt.bestHash = headers[highestIndex].blockHash.BlockHash
|
TickerStats(
|
||||||
sp.ns.seen(sp.hunt.bestHash,sp.hunt.bestNumber)
|
pivotBlock: pivotBlock,
|
||||||
|
activeQueues: tabLen,
|
||||||
|
flushedQueues: ctx.data.pivotCount.int64 - tabLen,
|
||||||
|
accounts: meanStdDev(aSum, aSqSum, count),
|
||||||
|
fillFactor: meanStdDev(uSum, uSqSum, count))
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public start/stop and admin functions
|
# Public start/stop and admin functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc workerSetup*(ns: Worker; chainDb: AbstractChainDB) =
|
proc setup*(ctx: SnapCtxRef; tickerOK: bool): bool =
|
||||||
## Global set up
|
## Global set up
|
||||||
ns.fetchSetup(chainDb)
|
ctx.data.accountRangeMax = high(UInt256) div ctx.buddiesMax.u256
|
||||||
|
ctx.data.accountsDb = AccountsDbRef.init(ctx.chain.getTrieDB)
|
||||||
|
if tickerOK:
|
||||||
|
ctx.data.ticker = TickerRef.init(ctx.tickerUpdate)
|
||||||
|
else:
|
||||||
|
trace "Ticker is disabled"
|
||||||
|
# ----
|
||||||
|
if snapAccountsDumpEnable:
|
||||||
|
doAssert ctx.data.proofDumpFile.open("./dump-stream.out", fmWrite)
|
||||||
|
ctx.data.proofDumpOk = true
|
||||||
|
# ----
|
||||||
|
true
|
||||||
|
|
||||||
proc workerRelease*(ns: Worker) =
|
proc release*(ctx: SnapCtxRef) =
|
||||||
## Global clean up
|
## Global clean up
|
||||||
ns.fetchRelease()
|
if not ctx.data.ticker.isNil:
|
||||||
|
ctx.data.ticker.stop()
|
||||||
|
ctx.data.ticker = nil
|
||||||
|
|
||||||
proc workerStart*(sp: WorkerBuddy): bool =
|
proc start*(buddy: SnapBuddyRef): bool =
|
||||||
## Initialise `WorkerBuddy` to support `workerBlockHeaders()` calls
|
## Initialise worker peer
|
||||||
if sp.peer.supports(protocol.snap) and
|
let
|
||||||
sp.peer.supports(protocol.eth) and
|
ctx = buddy.ctx
|
||||||
sp.peer.state(protocol.eth).initialized:
|
peer = buddy.peer
|
||||||
|
if peer.supports(protocol.snap) and
|
||||||
sp.ctrl.init(running = true)
|
peer.supports(protocol.eth) and
|
||||||
|
peer.state(protocol.eth).initialized:
|
||||||
# Initialise data retrieval
|
buddy.pivotStart()
|
||||||
sp.fetchStart()
|
if not ctx.data.ticker.isNil:
|
||||||
|
ctx.data.ticker.startBuddy()
|
||||||
# Link in hunt descriptor
|
|
||||||
sp.hunt = WorkerHuntEx.new(HuntForward)
|
|
||||||
|
|
||||||
# We know the hash but not the block number.
|
|
||||||
sp.hunt.bestHash = sp.peer.state(protocol.eth).bestBlockHash.BlockHash
|
|
||||||
# TODO: Temporarily disabled because it's useful to test the worker.
|
|
||||||
# sp.syncMode = SyncOnlyHash
|
|
||||||
return true
|
return true
|
||||||
|
|
||||||
proc workerStop*(sp: WorkerBuddy) =
|
proc stop*(buddy: SnapBuddyRef) =
|
||||||
## Clean up this peer
|
## Clean up this peer
|
||||||
if not sp.ctrl.stopped:
|
let
|
||||||
sp.ctrl.stopped = true
|
ctx = buddy.ctx
|
||||||
sp.fetchStop()
|
peer = buddy.peer
|
||||||
|
buddy.ctrl.stopped = true
|
||||||
proc workerLockedOk*(sp: WorkerBuddy): bool =
|
buddy.pivotStop()
|
||||||
sp.hunt.syncMode == SyncLocked
|
if not ctx.data.ticker.isNil:
|
||||||
|
ctx.data.ticker.stopBuddy()
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc workerExec*(sp: WorkerBuddy) {.async.} =
|
proc runSingle*(buddy: SnapBuddyRef) {.async.} =
|
||||||
## Query a peer to update our knowledge of its canonical chain and its best
|
## This peer worker is invoked if the peer-local flag `buddy.ctrl.multiOk`
|
||||||
## block, which is its canonical chain head. This can be called at any time
|
## is set `false` which is the default mode. This flag is updated by the
|
||||||
## after a peer has negotiated the connection.
|
## worker when deemed appropriate.
|
||||||
|
## * For all workers, there can be only one `runSingle()` function active
|
||||||
|
## simultaneously for all worker peers.
|
||||||
|
## * There will be no `runMulti()` function active for the same worker peer
|
||||||
|
## simultaneously
|
||||||
|
## * There will be no `runPool()` iterator active simultaneously.
|
||||||
##
|
##
|
||||||
## This function is called in an exponential then binary search style
|
## Note that this function runs in `async` mode.
|
||||||
## during initial sync to find the canonical head, real-time polling
|
|
||||||
## afterwards to check for updates.
|
|
||||||
##
|
##
|
||||||
## All replies to this query are part of the peer's canonical chain at the
|
buddy.ctrl.multiOk = true
|
||||||
## time the peer sends them.
|
|
||||||
|
|
||||||
let request = sp.peerSyncChainRequest
|
|
||||||
|
|
||||||
trace trEthSendSendingGetBlockHeaders, peer=sp,
|
proc runPool*(buddy: SnapBuddyRef) =
|
||||||
count=request.maxResults,
|
## Ocne started, the function `runPool()` is called for all worker peers in
|
||||||
startBlock=sp.ns.pp(request.startBlock), step=request.traceStep
|
## a row (as the body of an iteration.) There will be no other worker peer
|
||||||
|
## functions activated simultaneously.
|
||||||
|
##
|
||||||
|
## This procedure is started if the global flag `buddy.ctx.poolMode` is set
|
||||||
|
## `true` (default is `false`.) It is the responsibility of the `runPool()`
|
||||||
|
## instance to reset the flag `buddy.ctx.poolMode`, typically at the first
|
||||||
|
## peer instance as the number of active instances is unknown to `runPool()`.
|
||||||
|
##
|
||||||
|
## Note that this function does not run in `async` mode.
|
||||||
|
##
|
||||||
|
discard
|
||||||
|
|
||||||
inc sp.stats.ok.getBlockHeaders
|
|
||||||
var reply: Option[protocol.blockHeadersObj]
|
|
||||||
try:
|
|
||||||
reply = await sp.peer.getBlockHeaders(request)
|
|
||||||
except CatchableError as e:
|
|
||||||
trace trEthRecvError & "waiting for GetBlockHeaders reply", peer=sp,
|
|
||||||
error=e.msg
|
|
||||||
inc sp.stats.major.networkErrors
|
|
||||||
sp.workerStop()
|
|
||||||
return
|
|
||||||
|
|
||||||
if reply.isNone:
|
proc runMulti*(buddy: SnapBuddyRef) {.async.} =
|
||||||
trace trEthRecvTimeoutWaiting & "for GetBlockHeaders reply", peer=sp
|
## This peer worker is invoked if the `buddy.ctrl.multiOk` flag is set
|
||||||
# TODO: Should disconnect?
|
## `true` which is typically done after finishing `runSingle()`. This
|
||||||
inc sp.stats.minor.timeoutBlockHeaders
|
## instance can be simultaneously active for all peer workers.
|
||||||
return
|
##
|
||||||
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
|
||||||
let nHeaders = reply.get.headers.len
|
if buddy.data.pivotHeader.isNone:
|
||||||
if nHeaders == 0:
|
|
||||||
trace trEthRecvReceivedBlockHeaders, peer=sp,
|
|
||||||
got=0, requested=request.maxResults
|
|
||||||
else:
|
|
||||||
trace trEthRecvReceivedBlockHeaders, peer=sp,
|
|
||||||
got=nHeaders, requested=request.maxResults,
|
|
||||||
firstBlock=reply.get.headers[0].blockNumber,
|
|
||||||
lastBlock=reply.get.headers[^1].blockNumber
|
|
||||||
|
|
||||||
if request.maxResults.int < nHeaders:
|
await buddy.pivotExec()
|
||||||
trace trEthRecvProtocolViolation & "excess headers in BlockHeaders message",
|
|
||||||
peer=sp, got=nHeaders, requested=request.maxResults
|
|
||||||
# TODO: Should disconnect.
|
|
||||||
inc sp.stats.major.excessBlockHeaders
|
|
||||||
return
|
|
||||||
|
|
||||||
if 0 < nHeaders:
|
if not buddy.updatePivotEnv():
|
||||||
# TODO: Check this is not copying the `headers`.
|
return
|
||||||
sp.peerSyncChainNonEmptyReply(request, reply.get.headers)
|
|
||||||
else:
|
|
||||||
sp.peerSyncChainEmptyReply(request)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
if await buddy.fetchAccounts():
|
||||||
# Debugging
|
buddy.ctrl.multiOk = false
|
||||||
# ------------------------------------------------------------------------------
|
buddy.data.pivotHeader = none(BlockHeader)
|
||||||
|
|
||||||
proc huntPp*(sn: WorkerBuddy): string =
|
|
||||||
let hx = sn.hunt
|
|
||||||
result &= "(mode=" & $hx.syncMode
|
|
||||||
result &= ",num=(" & hx.lowNumber.pp & "," & hx.highNumber.pp & ")"
|
|
||||||
result &= ",best=(" & hx.bestNumber.pp & "," & hx.bestHash.pp & ")"
|
|
||||||
result &= ",step=" & $hx.step
|
|
||||||
result &= ")"
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# End
|
# End
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1,339 +0,0 @@
|
||||||
# Nimbus - Fetch account and storage states from peers efficiently
|
|
||||||
#
|
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
|
||||||
# Licensed under either of
|
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
|
||||||
# http://www.apache.org/licenses/LICENSE-2.0)
|
|
||||||
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
|
||||||
# http://opensource.org/licenses/MIT)
|
|
||||||
# at your option. This file may not be copied, modified, or distributed
|
|
||||||
# except according to those terms.
|
|
||||||
|
|
||||||
import
|
|
||||||
std/math,
|
|
||||||
chronos,
|
|
||||||
eth/[common/eth_types, p2p],
|
|
||||||
nimcrypto/keccak,
|
|
||||||
stew/[interval_set, keyed_queue],
|
|
||||||
stint,
|
|
||||||
../../types,
|
|
||||||
../path_desc,
|
|
||||||
./fetch/[fetch_accounts, proof_db],
|
|
||||||
"."/[ticker, worker_desc]
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
|
||||||
|
|
||||||
type
|
|
||||||
FetchEx = ref object of WorkerFetchBase
|
|
||||||
accTab: AccLruCache ## Global worker data
|
|
||||||
quCount: uint64 ## Count visited roots
|
|
||||||
lastPivot: NodeTag ## Used for calculating pivots
|
|
||||||
accRangeMaxLen: UInt256 ## Maximap interval length, high(u256)/#peers
|
|
||||||
pdb: ProofDb ## Proof processing
|
|
||||||
|
|
||||||
AccTabEntryRef = ref object
|
|
||||||
## Global worker table
|
|
||||||
avail: LeafRangeSet ## Accounts to visit (organised as ranges)
|
|
||||||
pivot: NodeTag ## Where to to start fetching from
|
|
||||||
base: WorkerFetchBase ## Back reference (`FetchEx` not working, here)
|
|
||||||
|
|
||||||
AccLruCache =
|
|
||||||
KeyedQueue[TrieHash,AccTabEntryRef]
|
|
||||||
|
|
||||||
logScope:
|
|
||||||
topics = "snap-fetch"
|
|
||||||
|
|
||||||
const
|
|
||||||
accRangeMaxLen = ##\
|
|
||||||
## ask for that many accounts at once (not the range is sparse)
|
|
||||||
(high(NodeTag) - low(NodeTag)) div 1000
|
|
||||||
|
|
||||||
pivotAccIncrement = ##\
|
|
||||||
## increment when `lastPivot` would stay put
|
|
||||||
10_000_000.u256
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private helpers
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc `==`(a, b: AccTabEntryRef): bool =
|
|
||||||
## Just to make things clear, should be default action anyway
|
|
||||||
cast[pointer](a) == cast[pointer](b)
|
|
||||||
|
|
||||||
proc fetchEx(ns: Worker): FetchEx =
|
|
||||||
## Getter
|
|
||||||
ns.fetchBase.FetchEx
|
|
||||||
|
|
||||||
proc fetchEx(sp: WorkerBuddy): FetchEx =
|
|
||||||
## Getter
|
|
||||||
sp.ns.fetchEx
|
|
||||||
|
|
||||||
proc withMaxLen(atb: AccTabEntryRef; iv: LeafRange): LeafRange =
|
|
||||||
## Reduce accounts interval to maximal size
|
|
||||||
let maxlen = atb.base.FetchEx.accRangeMaxLen
|
|
||||||
if 0 < iv.len and iv.len <= maxLen:
|
|
||||||
iv
|
|
||||||
else:
|
|
||||||
LeafRange.new(iv.minPt, iv.minPt + maxLen - 1.u256)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc getAccTab(sp: WorkerBuddy; key: TrieHash): AccTabEntryRef =
|
|
||||||
## Fetch LRU table item, create a new one if missing.
|
|
||||||
# fetch existing table (if any)
|
|
||||||
block:
|
|
||||||
let rc = sp.fetchEx.accTab.lruFetch(key)
|
|
||||||
if rc.isOk:
|
|
||||||
# Item was moved to the end of queue
|
|
||||||
return rc.value
|
|
||||||
|
|
||||||
# Calculate some new start address for the range fetcher
|
|
||||||
while true:
|
|
||||||
# Derive pivot from last interval set in table
|
|
||||||
let rc = sp.fetchEx.accTab.last
|
|
||||||
if rc.isErr:
|
|
||||||
break # no more => stop
|
|
||||||
# Check last interval
|
|
||||||
let blkRc = rc.value.data.avail.le() # rightmost interval
|
|
||||||
if blkRc.isErr:
|
|
||||||
# Delete useless interval set, repeat
|
|
||||||
sp.fetchEx.accTab.del(rc.value.key)
|
|
||||||
continue
|
|
||||||
# use increasing `pivot` values
|
|
||||||
if sp.fetchEx.lastPivot < blkRc.value.minPt:
|
|
||||||
sp.ns.fetchEx.lastPivot = blkRc.value.minPt
|
|
||||||
break
|
|
||||||
if sp.fetchEx.lastPivot < high(NodeTag) - pivotAccIncrement:
|
|
||||||
sp.fetchEx.lastPivot = sp.ns.fetchEx.lastPivot + pivotAccIncrement
|
|
||||||
break
|
|
||||||
# Otherwise start at 0
|
|
||||||
sp.fetchEx.lastPivot = 0.to(NodeTag)
|
|
||||||
break
|
|
||||||
|
|
||||||
let accRange = AccTabEntryRef(
|
|
||||||
pivot: sp.fetchEx.lastPivot,
|
|
||||||
avail: LeafRangeSet.init(),
|
|
||||||
base: sp.fetchEx)
|
|
||||||
|
|
||||||
trace "New accounts list for syncing",
|
|
||||||
peer=sp, stateRoot=key, pivot=sp.fetchEx.lastPivot
|
|
||||||
|
|
||||||
# Statistics
|
|
||||||
sp.fetchEx.quCount.inc
|
|
||||||
|
|
||||||
# Pre-filled with the largest possible interval
|
|
||||||
discard accRange.avail.merge(low(NodeTag),high(NodeTag))
|
|
||||||
|
|
||||||
# Append and curb LRU table as needed
|
|
||||||
return sp.fetchEx.accTab.lruAppend(key, accRange, sp.ns.buddiesMax)
|
|
||||||
|
|
||||||
|
|
||||||
proc sameAccTab(sp: WorkerBuddy; key: TrieHash; accTab: AccTabEntryRef): bool =
|
|
||||||
## Verify that account list entry has not changed.
|
|
||||||
let rc = sp.fetchEx.accTab.eq(key)
|
|
||||||
if rc.isErr:
|
|
||||||
return accTab.isNil
|
|
||||||
if not accTab.isNil:
|
|
||||||
return accTab == rc.value
|
|
||||||
|
|
||||||
|
|
||||||
proc fetchAccRange(atb: AccTabEntryRef): Result[LeafRange,void] =
|
|
||||||
## Fetch an interval from the account range list. Use the `atb.pivot` value
|
|
||||||
## as a start entry to fetch data from, wrapping around if necessary.
|
|
||||||
block:
|
|
||||||
# Check whether there the start point is in the middle of an interval
|
|
||||||
let rc = atb.avail.le(atb.pivot)
|
|
||||||
if rc.isOk:
|
|
||||||
if atb.pivot <= rc.value.maxPt:
|
|
||||||
let iv = LeafRange.new(atb.pivot, rc.value.maxPt)
|
|
||||||
discard atb.avail.reduce(iv)
|
|
||||||
return ok(iv)
|
|
||||||
|
|
||||||
block:
|
|
||||||
# Take the next interval to the right
|
|
||||||
let rc = atb.avail.ge(atb.pivot)
|
|
||||||
if rc.isOk:
|
|
||||||
let iv = atb.withMaxLen(rc.value)
|
|
||||||
discard atb.avail.reduce(iv)
|
|
||||||
return ok(iv)
|
|
||||||
|
|
||||||
# Otherwise wrap around
|
|
||||||
let rc = atb.avail.ge()
|
|
||||||
if rc.isOk:
|
|
||||||
let iv = atb.withMaxLen(rc.value)
|
|
||||||
discard atb.avail.reduce(iv)
|
|
||||||
return ok(iv)
|
|
||||||
|
|
||||||
err()
|
|
||||||
|
|
||||||
|
|
||||||
proc putAccRange(atb: AccTabEntryRef; iv: LeafRange) =
|
|
||||||
discard atb.avail.merge(iv)
|
|
||||||
|
|
||||||
proc putAccRange(atb: AccTabEntryRef; a, b: NodeTag) =
|
|
||||||
discard atb.avail.merge(a, b)
|
|
||||||
|
|
||||||
proc haveAccRange(atb: AccTabEntryRef): bool =
|
|
||||||
0 < atb.avail.chunks
|
|
||||||
|
|
||||||
|
|
||||||
proc meanStdDev(sum, sqSum: float; length: int): (float,float) =
|
|
||||||
if 0 < length:
|
|
||||||
result[0] = sum / length.float
|
|
||||||
result[1] = sqrt(sqSum / length.float - result[0] * result[0])
|
|
||||||
|
|
||||||
proc tickerStats(ns: Worker): TickerStats {.gcsafe.} =
|
|
||||||
var aSum, aSqSum, uSum, uSqSum: float
|
|
||||||
for kvp in ns.fetchEx.accTab.nextPairs:
|
|
||||||
|
|
||||||
# Accounts mean & variance
|
|
||||||
let aLen = ns.fetchEx.pdb.accountsLen(kvp.key).float
|
|
||||||
aSum += aLen
|
|
||||||
aSqSum += aLen * aLen
|
|
||||||
|
|
||||||
# Fill utilisation mean & variance
|
|
||||||
let fill = kvp.data.avail.freeFactor
|
|
||||||
uSum += fill
|
|
||||||
uSqSum += fill * fill
|
|
||||||
|
|
||||||
result.activeQueues = ns.fetchEx.accTab.len
|
|
||||||
result.flushedQueues = ns.fetchEx.quCount.int64 - result.activeQueues
|
|
||||||
result.accounts = meanStdDev(aSum, aSqSum, result.activeQueues)
|
|
||||||
result.fillFactor = meanStdDev(uSum, uSqSum, result.activeQueues)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public start/stop and admin functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc fetchSetup*(ns: Worker; chainDb: AbstractChainDB) =
|
|
||||||
## Global set up
|
|
||||||
ns.fetchBase = FetchEx()
|
|
||||||
ns.fetchEx.accTab.init(ns.buddiesMax)
|
|
||||||
ns.fetchEx.accRangeMaxLen = high(UInt256) div ns.buddiesMax.u256
|
|
||||||
ns.fetchEx.pdb.init(chainDb.getTrieDB)
|
|
||||||
ns.tickerSetup(cb = tickerStats)
|
|
||||||
|
|
||||||
proc fetchRelease*(ns: Worker) =
|
|
||||||
## Global clean up
|
|
||||||
ns.tickerRelease()
|
|
||||||
ns.fetchBase = nil
|
|
||||||
|
|
||||||
proc fetchStart*(sp: WorkerBuddy) =
|
|
||||||
## Initialise fetching for particular peer
|
|
||||||
discard
|
|
||||||
|
|
||||||
proc fetchStop*(sp: WorkerBuddy) =
|
|
||||||
## Clean up for this peer
|
|
||||||
discard
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc fetch*(sp: WorkerBuddy) {.async.} =
|
|
||||||
## Concurrently fetch account data. The data are fetched from `sp.peer` where
|
|
||||||
## `sp` is the argument descriptor. Currently, accounts data are fetched but
|
|
||||||
## not further processed (i.e. discarded.)
|
|
||||||
##
|
|
||||||
## The accounts requested depend on
|
|
||||||
## * the currrent state root `sp.ctrl.stateRoot`,
|
|
||||||
## * an account list `accTab(stateRoot)` depending on the current state root.
|
|
||||||
##
|
|
||||||
## The account list keeps track of account ranges already requested. It is
|
|
||||||
## shared among all instances of `fetch()` (sharing the same `ds`
|
|
||||||
## descriptor.) So the accounts requested for a shared accounts list are
|
|
||||||
## mutually exclusive.
|
|
||||||
##
|
|
||||||
## Currently the accounts list to retrieve by `accTab()` is implemented as
|
|
||||||
## follows.
|
|
||||||
## * For each state root there is a separate accounts list.
|
|
||||||
## * If the state root changes and there is no account list yet, create a
|
|
||||||
## new one.
|
|
||||||
## * Account ranges are fetched from an accoiunts list with increasing values
|
|
||||||
## starting at a (typically positive) `pivot` value. The fetch wraps around
|
|
||||||
## when the highest values are exhausted. This `pivot` value is increased
|
|
||||||
## with each new accounts list (derived from the last used accounts list.)
|
|
||||||
## * Accounts list are kept in a LRU table and automatically cleared. The
|
|
||||||
## size of the LRU table is set to `sp.ns.buddiesMax`, the maximal number
|
|
||||||
## of workers or peers.
|
|
||||||
|
|
||||||
trace "Fetching from peer", peer=sp, ctrlState=sp.ctrl.state
|
|
||||||
sp.tickerStartPeer()
|
|
||||||
|
|
||||||
while not sp.ctrl.stopped:
|
|
||||||
|
|
||||||
# We need a state root and an access range list (depending on state root)
|
|
||||||
if sp.ctrl.stateRoot.isNone:
|
|
||||||
trace "Currently no state root", peer=sp
|
|
||||||
# Wait for a new state root
|
|
||||||
while not sp.ctrl.stopped and
|
|
||||||
sp.ctrl.stateRoot.isNone:
|
|
||||||
await sleepAsync(5.seconds)
|
|
||||||
continue
|
|
||||||
|
|
||||||
# Ok, here is the `stateRoot`, tentatively try the access range list
|
|
||||||
let
|
|
||||||
stateRoot = sp.ctrl.stateRoot.get
|
|
||||||
accTab = sp.getAccTab(stateRoot)
|
|
||||||
|
|
||||||
if not accTab.haveAccRange():
|
|
||||||
trace "Currently no account ranges", peer=sp
|
|
||||||
# Account ranges exhausted, wait for a new state root
|
|
||||||
while not sp.ctrl.stopped and
|
|
||||||
sp.ctrl.stateRoot.isSome and
|
|
||||||
stateRoot == sp.ctrl.stateRoot.get and
|
|
||||||
sp.sameAccTab(stateRoot, accTab) and
|
|
||||||
not accTab.haveAccRange():
|
|
||||||
await sleepAsync(5.seconds)
|
|
||||||
continue
|
|
||||||
|
|
||||||
# Get a range of accounts to fetch from
|
|
||||||
let iv = block:
|
|
||||||
let rc = accTab.fetchAccRange()
|
|
||||||
if rc.isErr:
|
|
||||||
continue
|
|
||||||
rc.value
|
|
||||||
|
|
||||||
# Fetch data for this range delegated to `fetchAccounts()`
|
|
||||||
let dd = block:
|
|
||||||
let rc = await sp.fetchAccounts(stateRoot, iv)
|
|
||||||
if rc.isErr:
|
|
||||||
accTab.putAccRange(iv) # fail => interval back to pool
|
|
||||||
case rc.error:
|
|
||||||
of NetworkProblem, MissingProof, AccountsMinTooSmall,
|
|
||||||
AccountsMaxTooLarge:
|
|
||||||
# Mark this peer dead, i.e. avoid fetching from this peer for a while
|
|
||||||
sp.stats.major.networkErrors.inc()
|
|
||||||
sp.ctrl.zombie = true
|
|
||||||
of NothingSerious:
|
|
||||||
discard
|
|
||||||
of NoAccountsForStateRoot:
|
|
||||||
# One could wait for a new state root but this may result in a
|
|
||||||
# temporary standstill if all `fetch()` instances do the same. So
|
|
||||||
# waiting for a while here might be preferable in the hope that the
|
|
||||||
# situation changes at the peer.
|
|
||||||
await sleepAsync(5.seconds)
|
|
||||||
continue
|
|
||||||
rc.value
|
|
||||||
|
|
||||||
# Register consumed accounts range
|
|
||||||
if dd.consumed < iv.len:
|
|
||||||
# return some unused range
|
|
||||||
accTab.putAccRange(iv.minPt + dd.consumed.u256, iv.maxPt)
|
|
||||||
|
|
||||||
# Process data
|
|
||||||
block:
|
|
||||||
let rc = sp.ns.fetchEx.pdb.mergeProved(stateRoot, iv.minPt, dd.data)
|
|
||||||
if rc.isErr:
|
|
||||||
discard # ??
|
|
||||||
|
|
||||||
# while end
|
|
||||||
|
|
||||||
trace "Done syncing for this peer", peer=sp, ctrlState=sp.ctrl.state
|
|
||||||
sp.tickerStopPeer()
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# End
|
|
||||||
# ------------------------------------------------------------------------------
|
|
|
@ -1,780 +0,0 @@
|
||||||
# Nimbus - Fetch account and storage states from peers by snapshot traversal
|
|
||||||
#
|
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
|
||||||
# Licensed under either of
|
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
|
||||||
# http://www.apache.org/licenses/LICENSE-2.0)
|
|
||||||
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
|
||||||
# http://opensource.org/licenses/MIT)
|
|
||||||
# at your option. This file may not be copied, modified, or distributed
|
|
||||||
# except according to those terms.
|
|
||||||
|
|
||||||
import
|
|
||||||
std/[algorithm, hashes, options, sequtils, sets, strutils, strformat, tables],
|
|
||||||
chronos,
|
|
||||||
eth/[common/eth_types, p2p, rlp, trie/db],
|
|
||||||
nimcrypto/keccak,
|
|
||||||
stew/[byteutils, interval_set],
|
|
||||||
stint,
|
|
||||||
../../../../db/storage_types,
|
|
||||||
"../../.."/[protocol, types],
|
|
||||||
../../path_desc,
|
|
||||||
../worker_desc
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
|
||||||
|
|
||||||
logScope:
|
|
||||||
topics = "snap-proof"
|
|
||||||
|
|
||||||
const
|
|
||||||
RowColumnParserDump = false
|
|
||||||
NibbleFollowDump = false # true
|
|
||||||
|
|
||||||
type
|
|
||||||
ProofError* = enum
|
|
||||||
RlpEncoding
|
|
||||||
RlpBlobExpected
|
|
||||||
RlpNonEmptyBlobExpected
|
|
||||||
RlpBranchLinkExpected
|
|
||||||
RlpListExpected
|
|
||||||
Rlp2Or17ListEntries
|
|
||||||
RlpExtPathEncoding
|
|
||||||
RlpLeafPathEncoding
|
|
||||||
RlpRecTypeError
|
|
||||||
ImpossibleKeyError
|
|
||||||
RowUnreferenced
|
|
||||||
AccountSmallerThanBase
|
|
||||||
AccountsNotSrictlyIncreasing
|
|
||||||
LastAccountProofFailed
|
|
||||||
MissingMergeBeginDirective
|
|
||||||
StateRootDiffers
|
|
||||||
|
|
||||||
ProofRecType = enum
|
|
||||||
Branch,
|
|
||||||
Extension,
|
|
||||||
Leaf
|
|
||||||
|
|
||||||
StatusRec = object
|
|
||||||
nAccounts: int
|
|
||||||
nProofs: int
|
|
||||||
|
|
||||||
AccountRec = ##\
|
|
||||||
## Account entry record
|
|
||||||
distinct Account
|
|
||||||
|
|
||||||
ProofRec = object
|
|
||||||
## Proofs entry record
|
|
||||||
case kind: ProofRecType
|
|
||||||
of Branch:
|
|
||||||
vertex: array[16,NodeTag]
|
|
||||||
value: Blob # name starts with a `v` as in vertex
|
|
||||||
of Extension:
|
|
||||||
extend: PathSegment
|
|
||||||
follow: NodeTag
|
|
||||||
of Leaf:
|
|
||||||
path: PathSegment
|
|
||||||
payload: Blob # name starts with a `p` as in path
|
|
||||||
|
|
||||||
ProofKvp = object
|
|
||||||
key: NodeTag
|
|
||||||
data: Option[ProofRec]
|
|
||||||
|
|
||||||
ProofDb* = object
|
|
||||||
keyMap: Table[NodeTag,uint] ## For debugging only
|
|
||||||
|
|
||||||
rootTag: NodeTag ## Current root node
|
|
||||||
rootHash: TrieHash ## Root node as hash
|
|
||||||
stat: StatusRec ## table statistics
|
|
||||||
|
|
||||||
db: TrieDatabaseRef ## general database
|
|
||||||
dbTx: DbTransaction ## Rollback state capture
|
|
||||||
|
|
||||||
newAccs: seq[(NodeTag,NodeTag)] ## New accounts group: (base,last)
|
|
||||||
newProofs: seq[NodeTag] ## Newly added proofs records
|
|
||||||
refPool: HashSet[NodeTag] ## New proofs references recs
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private helpers
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
template noRlpError(info: static[string]; code: untyped) =
|
|
||||||
try:
|
|
||||||
code
|
|
||||||
except RlpError as e:
|
|
||||||
raiseAssert "Inconveivable (" & info & "): " & e.msg
|
|
||||||
|
|
||||||
proc read(rlp: var Rlp; T: type ProofRec): T =
|
|
||||||
## RLP mixin
|
|
||||||
noRlpError("read(ProofRec)"):
|
|
||||||
result.kind = rlp.read(typeof result.kind)
|
|
||||||
rlp.tryEnterList()
|
|
||||||
case result.kind:
|
|
||||||
of Branch:
|
|
||||||
result.vertex = rlp.read(typeof result.vertex)
|
|
||||||
result.value = rlp.read(typeof result.value)
|
|
||||||
of Extension:
|
|
||||||
result.extend = rlp.dbRead(typeof result.extend)
|
|
||||||
result.follow = rlp.read(typeof result.follow)
|
|
||||||
of Leaf:
|
|
||||||
result.path = rlp.dbRead(typeof result.path)
|
|
||||||
result.payload = rlp.read(typeof result.payload)
|
|
||||||
|
|
||||||
proc append(writer: var RlpWriter; rec: ProofRec) =
|
|
||||||
## RLP mixin
|
|
||||||
append(writer, rec.kind)
|
|
||||||
startList(writer, 2)
|
|
||||||
case rec.kind:
|
|
||||||
of Branch:
|
|
||||||
append(writer, rec.vertex)
|
|
||||||
append(writer, rec.value)
|
|
||||||
of Extension:
|
|
||||||
dbAppend(writer, rec.extend)
|
|
||||||
append(writer, rec.follow)
|
|
||||||
of Leaf:
|
|
||||||
dbAppend(writer, rec.path)
|
|
||||||
append(writer, rec.payload)
|
|
||||||
|
|
||||||
proc to(w: TrieHash; T: type NodeTag): T =
|
|
||||||
## Syntactic sugar
|
|
||||||
w.Hash256.to(T)
|
|
||||||
|
|
||||||
proc to(w: AccountRec; T: type Account): T =
|
|
||||||
## Syntactic sugar
|
|
||||||
w.T
|
|
||||||
|
|
||||||
proc to(w: Account; T: type AccountRec): T =
|
|
||||||
## Syntactic sugar
|
|
||||||
w.T
|
|
||||||
|
|
||||||
|
|
||||||
func nibble(a: array[32,byte]; inx: int): int =
|
|
||||||
let byteInx = inx shr 1
|
|
||||||
if byteInx < 32:
|
|
||||||
if (inx and 1) == 0:
|
|
||||||
result = (a[byteInx] shr 4).int
|
|
||||||
else:
|
|
||||||
result = (a[byteInx] and 15).int
|
|
||||||
|
|
||||||
proc clearJournal(pv: var ProofDb) =
|
|
||||||
pv.newAccs.setLen(0)
|
|
||||||
pv.newProofs.setLen(0)
|
|
||||||
pv.refPool.clear
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private debugging helpers
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
import
|
|
||||||
../../../../constants
|
|
||||||
|
|
||||||
template noPpError(info: static[string]; code: untyped) =
|
|
||||||
try:
|
|
||||||
code
|
|
||||||
except ValueError as e:
|
|
||||||
raiseAssert "Inconveivable (" & info & "): " & e.msg
|
|
||||||
except KeyError as e:
|
|
||||||
raiseAssert "Not possible (" & info & "): " & e.msg
|
|
||||||
|
|
||||||
proc pp(s: string; hex = false): string =
|
|
||||||
if hex:
|
|
||||||
let n = (s.len + 1) div 2
|
|
||||||
(if s.len < 20: s else: s[0 .. 5] & ".." & s[s.len-8 .. s.len-1]) &
|
|
||||||
"[" & (if 0 < n: "#" & $n else: "") & "]"
|
|
||||||
elif s.len <= 30:
|
|
||||||
s
|
|
||||||
else:
|
|
||||||
(if (s.len and 1) == 0: s[0 ..< 8] else: "0" & s[0 ..< 7]) &
|
|
||||||
"..(" & $s.len & ").." & s[s.len-16 ..< s.len]
|
|
||||||
|
|
||||||
proc pp(a: Hash256; collapse = true): string =
|
|
||||||
if not collapse:
|
|
||||||
a.data.mapIt(it.toHex(2)).join.toLowerAscii
|
|
||||||
elif a == ZERO_HASH256:
|
|
||||||
"ZERO_HASH256"
|
|
||||||
elif a == BLANK_ROOT_HASH:
|
|
||||||
"BLANK_ROOT_HASH"
|
|
||||||
elif a == EMPTY_UNCLE_HASH:
|
|
||||||
"EMPTY_UNCLE_HASH"
|
|
||||||
elif a == EMPTY_SHA3:
|
|
||||||
"EMPTY_SHA3"
|
|
||||||
elif a == ZERO_HASH256:
|
|
||||||
"ZERO_HASH256"
|
|
||||||
else:
|
|
||||||
a.data.mapIt(it.toHex(2)).join[56 .. 63].toLowerAscii
|
|
||||||
|
|
||||||
proc pp(a: NodeHash|TrieHash; collapse = true): string =
|
|
||||||
a.Hash256.pp(collapse)
|
|
||||||
|
|
||||||
proc pp(a: NodeTag; collapse = true): string =
|
|
||||||
a.to(Hash256).pp(collapse)
|
|
||||||
|
|
||||||
proc toKey(a: NodeTag; pv: var ProofDb): uint =
|
|
||||||
noPpError("pp(NodeTag)"):
|
|
||||||
if not pv.keyMap.hasKey(a):
|
|
||||||
pv.keyMap[a] = pv.keyMap.len.uint + 1
|
|
||||||
result = pv.keyMap[a]
|
|
||||||
|
|
||||||
proc pp(a: NodeTag; pv: var ProofDb): string =
|
|
||||||
$a.toKey(pv)
|
|
||||||
|
|
||||||
proc pp(q: openArray[byte]; noHash = false): string =
|
|
||||||
if q.len == 32 and not noHash:
|
|
||||||
var a: array[32,byte]
|
|
||||||
for n in 0..31: a[n] = q[n]
|
|
||||||
($Hash256(data: a)).pp
|
|
||||||
else:
|
|
||||||
q.toSeq.mapIt(it.toHex(2)).join.toLowerAscii.pp(hex = true)
|
|
||||||
|
|
||||||
proc pp(blob: Blob): string =
|
|
||||||
blob.mapIt(it.toHex(2)).join
|
|
||||||
|
|
||||||
proc pp(a: Account): string =
|
|
||||||
noPpError("pp(Account)"):
|
|
||||||
result = &"({a.nonce},{a.balance},{a.storageRoot},{a.codeHash})"
|
|
||||||
|
|
||||||
proc pp(sa: SnapAccount): string =
|
|
||||||
"(" & $sa.accHash & "," & sa.accBody.pp & ")"
|
|
||||||
|
|
||||||
proc pp(al: seq[SnapAccount]): string =
|
|
||||||
result = " @["
|
|
||||||
noPpError("pp(seq[SnapAccount])"):
|
|
||||||
for n,rec in al:
|
|
||||||
result &= &"| # <{n}>| {rec.pp},"
|
|
||||||
if 10 < result.len:
|
|
||||||
result[^1] = ']'
|
|
||||||
else:
|
|
||||||
result &= "]"
|
|
||||||
|
|
||||||
proc pp(blobs: seq[Blob]): string =
|
|
||||||
result = " @["
|
|
||||||
noPpError("pp(seq[Blob])"):
|
|
||||||
for n,rec in blobs:
|
|
||||||
result &= "| # <" & $n & ">| \"" & rec.pp & "\".hexToSeqByte,"
|
|
||||||
if 10 < result.len:
|
|
||||||
result[^1] = ']'
|
|
||||||
else:
|
|
||||||
result &= "]"
|
|
||||||
|
|
||||||
proc pp(hs: seq[NodeTag]; pv: var ProofDb): string =
|
|
||||||
"<" & hs.mapIt(it.pp(pv)).join(",") & ">"
|
|
||||||
|
|
||||||
proc pp(hs: HashSet[NodeTag]; pv: var ProofDb): string =
|
|
||||||
"{" & toSeq(hs.items).mapIt(it.toKey(pv)).sorted.mapIt($it).join(",") & "}"
|
|
||||||
|
|
||||||
proc pp(rec: ProofRec; pv: var ProofDb): string =
|
|
||||||
noPpError("pp(ProofRec)"):
|
|
||||||
case rec.kind:
|
|
||||||
of Branch: result &=
|
|
||||||
"b(" & rec.vertex.mapIt(it.pp(pv)).join(",") & "," &
|
|
||||||
rec.value.pp.pp(true) & ")"
|
|
||||||
of Leaf: result &=
|
|
||||||
"l(" & ($rec.path).pp(true) & "," & rec.payload.pp.pp(true) & ")"
|
|
||||||
of Extension: result &=
|
|
||||||
"x(" & ($rec.extend).pp(true) & "," & rec.follow.pp(pv) & ")"
|
|
||||||
|
|
||||||
proc pp(rec: Option[ProofRec]; pv: var ProofDb): string =
|
|
||||||
if rec.isSome:
|
|
||||||
rec.get.pp(pv)
|
|
||||||
else:
|
|
||||||
"n/a"
|
|
||||||
|
|
||||||
proc pp(q: seq[ProofKvp]; pv: var ProofDb): string =
|
|
||||||
result="@["
|
|
||||||
for kvp in q:
|
|
||||||
result &= "(" & kvp.key.pp(pv) & "," & kvp.data.pp(pv) & "),"
|
|
||||||
if q.len == 0:
|
|
||||||
result &= "]"
|
|
||||||
else:
|
|
||||||
result[^1] = ']'
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Private functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
template mkProofKey(pv: ProofDb; tag: NodeTag): openArray[byte] =
|
|
||||||
tag.to(Hash256).snapSyncProofKey.toOpenArray
|
|
||||||
|
|
||||||
proc getProofsRec(pv: ProofDb; tag: NodeTag): Result[ProofRec,void] =
|
|
||||||
let recData = pv.db.get(pv.mkProofKey(tag))
|
|
||||||
if 0 < recData.len:
|
|
||||||
return ok(recData.decode(ProofRec))
|
|
||||||
err()
|
|
||||||
|
|
||||||
proc hasProofsRec(pv: ProofDb; tag: NodeTag): bool =
|
|
||||||
pv.db.contains(pv.mkProofKey(tag))
|
|
||||||
|
|
||||||
proc collectRefs(pv: var ProofDb; rec: ProofRec) =
|
|
||||||
case rec.kind:
|
|
||||||
of Branch:
|
|
||||||
for v in rec.vertex:
|
|
||||||
pv.refPool.incl v
|
|
||||||
of Extension:
|
|
||||||
pv.refPool.incl rec.follow
|
|
||||||
of Leaf:
|
|
||||||
discard
|
|
||||||
|
|
||||||
proc collectRefs(pv: var ProofDb; tag: NodeTag) =
|
|
||||||
let rc = pv.getProofsRec(tag)
|
|
||||||
if rc.isOk:
|
|
||||||
pv.collectRefs(rc.value)
|
|
||||||
|
|
||||||
proc addProofsRec(pv: var ProofDb; tag: NodeTag; rec: ProofRec) =
|
|
||||||
#debug "addProofsRec", size=pv.nProofs, tag=tag.pp(pv), rec=rec.pp(pv)
|
|
||||||
if not pv.hasProofsRec(tag):
|
|
||||||
pv.db.put(pv.mkProofKey(tag), rlp.encode(rec))
|
|
||||||
pv.stat.nProofs.inc
|
|
||||||
pv.newProofs.add tag # to be committed
|
|
||||||
# Always add references, the rec might have been added earlier outside
|
|
||||||
# the current transaction.
|
|
||||||
pv.collectRefs(rec)
|
|
||||||
|
|
||||||
# -----------
|
|
||||||
|
|
||||||
template mkAccKey(pv: ProofDb; tag: NodeTag): openArray[byte] =
|
|
||||||
snapSyncAccountKey(tag.to(Hash256), pv.rootHash.Hash256).toOpenArray
|
|
||||||
|
|
||||||
proc hasAccountRec(pv: ProofDb; tag: NodeTag): bool =
|
|
||||||
pv.db.contains(pv.mkAccKey(tag))
|
|
||||||
|
|
||||||
proc getAccountRec(pv: ProofDb; tag: NodeTag): Result[AccountRec,void] =
|
|
||||||
let rec = pv.db.get(pv.mkAccKey(tag))
|
|
||||||
if 0 < rec.len:
|
|
||||||
noRlpError("read(AccountRec)"):
|
|
||||||
return ok(rec.decode(Account).to(AccountRec))
|
|
||||||
err()
|
|
||||||
|
|
||||||
proc addAccountRec(pv: var ProofDb; tag: NodeTag; rec: AccountRec) =
|
|
||||||
if not pv.hasAccountRec(tag):
|
|
||||||
pv.db.put(pv.mkAccKey(tag), rlp.encode(rec.to(Account)))
|
|
||||||
pv.stat.nAccounts.inc
|
|
||||||
|
|
||||||
# -----------
|
|
||||||
|
|
||||||
template mkStatusKey(pv: ProofDb; root: TrieHash): openArray[byte] =
|
|
||||||
snapSyncStatusKey(root.Hash256).toOpenArray
|
|
||||||
|
|
||||||
proc hasStatusRec(pv: ProofDb; root: TrieHash): bool =
|
|
||||||
pv.db.contains(pv.mkStatusKey(root))
|
|
||||||
|
|
||||||
proc getStatusRec(pv: ProofDb; root: TrieHash): Result[StatusRec,void] =
|
|
||||||
let rec = pv.db.get(pv.mkStatusKey(root))
|
|
||||||
if 0 < rec.len:
|
|
||||||
noRlpError("getStatusRec"):
|
|
||||||
return ok(rec.decode(StatusRec))
|
|
||||||
err()
|
|
||||||
|
|
||||||
proc useStatusRec(pv: ProofDb; root: TrieHash): StatusRec =
|
|
||||||
let rec = pv.db.get(pv.mkStatusKey(root))
|
|
||||||
if 0 < rec.len:
|
|
||||||
noRlpError("findStatusRec"):
|
|
||||||
return rec.decode(StatusRec)
|
|
||||||
|
|
||||||
proc putStatusRec(pv: ProofDb; root: TrieHash; rec: StatusRec) =
|
|
||||||
pv.db.put(pv.mkStatusKey(root), rlp.encode(rec))
|
|
||||||
|
|
||||||
# Example trie from https://eth.wiki/en/fundamentals/patricia-tree
|
|
||||||
#
|
|
||||||
# lookup data:
|
|
||||||
# "do": "verb"
|
|
||||||
# "dog": "puppy"
|
|
||||||
# "dodge": "coin"
|
|
||||||
# "horse": "stallion"
|
|
||||||
#
|
|
||||||
# trie DB:
|
|
||||||
# root: [16 A]
|
|
||||||
# A: [* * * * B * * * [20+"orse" "stallion"] * * * * * * * *]
|
|
||||||
# B: [00+"o" D]
|
|
||||||
# D: [* * * * * * E * * * * * * * * * "verb"]
|
|
||||||
# E: [17 [* * * * * * [35 "coin"] * * * * * * * * * "puppy"]]
|
|
||||||
#
|
|
||||||
# with first nibble of two-column rows:
|
|
||||||
# hex bits | node type length
|
|
||||||
# ---------+------------------
|
|
||||||
# 0 0000 | extension even
|
|
||||||
# 1 0001 | extension odd
|
|
||||||
# 2 0010 | leaf even
|
|
||||||
# 3 0011 | leaf odd
|
|
||||||
#
|
|
||||||
# and key path:
|
|
||||||
# "do": 6 4 6 f
|
|
||||||
# "dog": 6 4 6 f 6 7
|
|
||||||
# "dodge": 6 4 6 f 6 7 6 5
|
|
||||||
# "horse": 6 8 6 f 7 2 7 3 6 5
|
|
||||||
#
|
|
||||||
|
|
||||||
proc parse(pv: ProofDb; rlpData: Blob): Result[ProofKvp,ProofError]
|
|
||||||
{.gcsafe, raises: [Defect, RlpError].} =
|
|
||||||
## Decode a single trie item for adding to the table
|
|
||||||
|
|
||||||
let recTag = rlpData.digestTo(NodeTag)
|
|
||||||
when RowColumnParserDump:
|
|
||||||
debug "Rlp column parser", recTag
|
|
||||||
if pv.hasProofsRec(recTag):
|
|
||||||
# No need to do this rec again
|
|
||||||
return ok(ProofKvp(key: recTag, data: none(ProofRec)))
|
|
||||||
|
|
||||||
var
|
|
||||||
# Inut data
|
|
||||||
rlp = rlpData.rlpFromBytes
|
|
||||||
|
|
||||||
# Result data
|
|
||||||
blobs = newSeq[Blob](2) # temporary, cache
|
|
||||||
rec = ProofRec(kind: Branch) # part of output, default type
|
|
||||||
top = 0 # count entries
|
|
||||||
|
|
||||||
# Collect lists of either 2 or 17 blob entries.
|
|
||||||
for w in rlp.items:
|
|
||||||
when RowColumnParserDump:
|
|
||||||
debug "Rlp column parser", col=top, data=w.toBytes.pp
|
|
||||||
case top
|
|
||||||
of 0, 1:
|
|
||||||
if not w.isBlob:
|
|
||||||
return err(RlpBlobExpected)
|
|
||||||
blobs[top] = rlp.read(Blob)
|
|
||||||
of 2 .. 15:
|
|
||||||
if not rec.vertex[top].init(rlp.read(Blob)):
|
|
||||||
return err(RlpBranchLinkExpected)
|
|
||||||
of 16:
|
|
||||||
if not w.isBlob:
|
|
||||||
return err(RlpBlobExpected)
|
|
||||||
rec.value = rlp.read(Blob)
|
|
||||||
else:
|
|
||||||
return err(Rlp2Or17ListEntries)
|
|
||||||
top.inc
|
|
||||||
|
|
||||||
when RowColumnParserDump:
|
|
||||||
debug "Rlp column parser done collecting columns", col=top
|
|
||||||
|
|
||||||
# Assemble collected data
|
|
||||||
case top:
|
|
||||||
of 2:
|
|
||||||
if blobs[0].len == 0:
|
|
||||||
return err(RlpNonEmptyBlobExpected)
|
|
||||||
case blobs[0][0] shr 4:
|
|
||||||
of 0, 1:
|
|
||||||
rec.kind = Extension
|
|
||||||
if not (rec.extend.init(blobs[0]) and rec.follow.init(blobs[1])):
|
|
||||||
return err(RlpExtPathEncoding)
|
|
||||||
of 2, 3:
|
|
||||||
rec.kind = Leaf
|
|
||||||
if not rec.path.init(blobs[0]):
|
|
||||||
return err(RlpLeafPathEncoding)
|
|
||||||
rec.payload = blobs[1]
|
|
||||||
else:
|
|
||||||
return err(RlpRecTypeError)
|
|
||||||
of 17:
|
|
||||||
# Branch entry, complete the first two vertices
|
|
||||||
for n,blob in blobs:
|
|
||||||
if not rec.vertex[n].init(blob):
|
|
||||||
return err(RlpBranchLinkExpected)
|
|
||||||
else:
|
|
||||||
return err(Rlp2Or17ListEntries)
|
|
||||||
|
|
||||||
ok(ProofKvp(key: recTag, data: some(rec)))
|
|
||||||
|
|
||||||
|
|
||||||
proc parse(pv: var ProofDb; proof: SnapAccountProof): Result[void,ProofError] =
|
|
||||||
## Decode a list of RLP encoded trie entries and add it to the rec pool
|
|
||||||
try:
|
|
||||||
for n,rlpRec in proof:
|
|
||||||
when RowColumnParserDump:
|
|
||||||
debug "Rlp rec parser", rec=n, data=rec.pp
|
|
||||||
|
|
||||||
let kvp = block:
|
|
||||||
let rc = pv.parse(rlpRec)
|
|
||||||
if rc.isErr:
|
|
||||||
return err(rc.error)
|
|
||||||
rc.value
|
|
||||||
|
|
||||||
if kvp.data.isNone: # avoids dups, stoll collects references
|
|
||||||
pv.collectRefs(kvp.key)
|
|
||||||
else:
|
|
||||||
pv.addProofsRec(kvp.key, kvp.data.get)
|
|
||||||
except RlpError:
|
|
||||||
return err(RlpEncoding)
|
|
||||||
except KeyError:
|
|
||||||
return err(ImpossibleKeyError)
|
|
||||||
|
|
||||||
ok()
|
|
||||||
|
|
||||||
proc follow(pv: ProofDb; path: NodeTag): (int, Blob) =
|
|
||||||
## Returns the number of matching digits/nibbles from the argument `tag`
|
|
||||||
## found in the proofs trie.
|
|
||||||
var
|
|
||||||
inTop = 0
|
|
||||||
inPath = path.UInt256.toBytesBE
|
|
||||||
recTag = pv.rootTag
|
|
||||||
leafBlob: Blob
|
|
||||||
|
|
||||||
when NibbleFollowDump:
|
|
||||||
trace "follow", root=pv.rootTag, path
|
|
||||||
|
|
||||||
noRlpError("follow"):
|
|
||||||
block loop:
|
|
||||||
while true:
|
|
||||||
|
|
||||||
let rec = block:
|
|
||||||
let rc = pv.getProofsRec(recTag)
|
|
||||||
if rc.isErr:
|
|
||||||
break loop
|
|
||||||
rc.value
|
|
||||||
|
|
||||||
let recType = rec.kind
|
|
||||||
case recType:
|
|
||||||
of Branch:
|
|
||||||
let
|
|
||||||
nibble = inPath.nibble(inTop)
|
|
||||||
newTag = rec.vertex[nibble]
|
|
||||||
when NibbleFollowDump:
|
|
||||||
trace "follow branch", recType, recTag, inTop, nibble, newTag
|
|
||||||
recTag = newTag
|
|
||||||
|
|
||||||
of Leaf:
|
|
||||||
for n in 0 ..< rec.path.len:
|
|
||||||
if rec.path[n] != inPath.nibble(inTop + n):
|
|
||||||
inTop += n
|
|
||||||
when NibbleFollowDump:
|
|
||||||
let tail = rec.path
|
|
||||||
trace "follow leaf failed", recType, recTag, tail
|
|
||||||
break loop
|
|
||||||
inTop += rec.path.len
|
|
||||||
leafBlob = rec.payload
|
|
||||||
when NibbleFollowDump:
|
|
||||||
trace "follow leaf", recType, recTag, inTop, done=true
|
|
||||||
break loop
|
|
||||||
|
|
||||||
of Extension:
|
|
||||||
for n in 0 ..< rec.extend.len:
|
|
||||||
if rec.extend[n] != inPath.nibble(inTop + n):
|
|
||||||
inTop += n
|
|
||||||
when NibbleFollowDump:
|
|
||||||
let tail = rec.extend
|
|
||||||
trace "follow extension failed", recType, recTag, tail
|
|
||||||
break loop
|
|
||||||
inTop += rec.extend.len
|
|
||||||
let newTag = rec.follow
|
|
||||||
when NibbleFollowDump:
|
|
||||||
trace "follow extension", recType, recTag, inTop, newTag
|
|
||||||
recTag = newTag
|
|
||||||
|
|
||||||
# end case
|
|
||||||
inTop.inc
|
|
||||||
|
|
||||||
# end while
|
|
||||||
inTop.dec
|
|
||||||
|
|
||||||
when NibbleFollowDump:
|
|
||||||
trace "follow done", tag, inTop
|
|
||||||
|
|
||||||
(inTop, leafBlob)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public constructor
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc init*(pv: var ProofDb; db: TrieDatabaseRef) =
|
|
||||||
pv = ProofDb(db: db)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public functions, transaction frame
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc isMergeTx*(pv: ProofDb): bool =
|
|
||||||
## The function returns `true` exactly if a merge transaction was initialised
|
|
||||||
## with `mergeBegin()`.
|
|
||||||
not pv.dbTx.isNil
|
|
||||||
|
|
||||||
proc mergeBegin*(pv: var ProofDb; root: TrieHash): bool =
|
|
||||||
## Prepare the system for accepting data input unless there is an open
|
|
||||||
## transaction, already. The function returns `true` if
|
|
||||||
## * There was no transaction initialised, yet
|
|
||||||
## * There is an open transaction for the same state root argument `root`
|
|
||||||
## In all other cases, `false` is returned.
|
|
||||||
if pv.dbTx.isNil:
|
|
||||||
# Update state root
|
|
||||||
pv.rootTag = root.to(NodeTag)
|
|
||||||
pv.rootHash = root
|
|
||||||
# Fetch status record for this `root`
|
|
||||||
pv.stat = pv.useStatusRec(root)
|
|
||||||
# New DB transaction
|
|
||||||
pv.dbTx = pv.db.beginTransaction
|
|
||||||
return true
|
|
||||||
# Make sure that the state roots are the same
|
|
||||||
pv.rootHash == root
|
|
||||||
|
|
||||||
proc mergeCommit*(pv: var ProofDb): bool =
|
|
||||||
## Accept merges and clear rollback journal if there was a transaction
|
|
||||||
## initialised with `mergeBegin()`. If successful, `true` is returned, and
|
|
||||||
## `false` otherwise.
|
|
||||||
if not pv.dbTx.isNil:
|
|
||||||
pv.dbTx.commit
|
|
||||||
pv.dbTx = nil
|
|
||||||
pv.clearJournal()
|
|
||||||
pv.putStatusRec(pv.rootHash, pv.stat) # persistent new status for this root
|
|
||||||
return true
|
|
||||||
|
|
||||||
proc mergeRollback*(pv: var ProofDb): bool =
|
|
||||||
## Rewind discaring merges and clear rollback journal if there was a
|
|
||||||
## transaction initialised with `mergeBegin()`. If successful, `true` is
|
|
||||||
## returned, and `false` otherwise.
|
|
||||||
if not pv.dbTx.isNil:
|
|
||||||
pv.dbTx.rollback
|
|
||||||
pv.dbTx = nil
|
|
||||||
# restore previous status for this root
|
|
||||||
pv.stat = pv.useStatusRec(pv.rootHash)
|
|
||||||
pv.clearJournal()
|
|
||||||
return true
|
|
||||||
|
|
||||||
proc merge*(
|
|
||||||
pv: var ProofDb;
|
|
||||||
proofs: SnapAccountProof
|
|
||||||
): Result[void,ProofError] =
|
|
||||||
## Merge account proofs (as received with the snap message `AccountRange`)
|
|
||||||
## into the database. A rollback journal is maintained so that this operation
|
|
||||||
## can be reverted.
|
|
||||||
if pv.dbTx.isNil:
|
|
||||||
return err(MissingMergeBeginDirective)
|
|
||||||
let rc = pv.parse(proofs)
|
|
||||||
if rc.isErr:
|
|
||||||
trace "Merge() proof failed", proofs=proofs.len, error=rc.error
|
|
||||||
return err(rc.error)
|
|
||||||
ok()
|
|
||||||
|
|
||||||
proc merge*(
|
|
||||||
pv: var ProofDb;
|
|
||||||
base: NodeTag;
|
|
||||||
acc: seq[SnapAccount]
|
|
||||||
): Result[void,ProofError] =
|
|
||||||
## Merge accounts (as received with the snap message `AccountRange`) into
|
|
||||||
## the database. A rollback journal is maintained so that this operation
|
|
||||||
## can be reverted.
|
|
||||||
if pv.dbTx.isNil:
|
|
||||||
return err(MissingMergeBeginDirective)
|
|
||||||
if acc.len != 0:
|
|
||||||
# Verify lower bound
|
|
||||||
if acc[0].accHash < base:
|
|
||||||
return err(AccountSmallerThanBase)
|
|
||||||
# Verify strictly increasing account hashes
|
|
||||||
for n in 1 ..< acc.len:
|
|
||||||
if acc[n].accHash <= acc[n-1].accHash:
|
|
||||||
return err(AccountsNotSrictlyIncreasing)
|
|
||||||
# Add to database
|
|
||||||
for sa in acc:
|
|
||||||
pv.addAccountRec(sa.accHash, sa.accBody.to(AccountRec))
|
|
||||||
# Stash boundary values, needed for later boundary proof
|
|
||||||
pv.newAccs.add (base, acc[^1].accHash)
|
|
||||||
ok()
|
|
||||||
|
|
||||||
proc mergeValidate*(pv: ProofDb): Result[void,ProofError] =
|
|
||||||
## Verify non-commited accounts and proofs:
|
|
||||||
## * The prosfs entries must all be referenced from within the rollback
|
|
||||||
## journal
|
|
||||||
## * For each group of accounts, the base `NodeTag` must be found in the
|
|
||||||
## proof database with a partial path of length ???
|
|
||||||
## * The last entry in a group of accounts must habe the `accBody` in the
|
|
||||||
## proof database
|
|
||||||
if pv.dbTx.isNil:
|
|
||||||
return err(MissingMergeBeginDirective)
|
|
||||||
|
|
||||||
# Make sure that all recs are referenced
|
|
||||||
if 0 < pv.newProofs.len:
|
|
||||||
#debug "Ref check",refPool=pv.refPool.pp(pv),newProofs=pv.newProofs.pp(pv)
|
|
||||||
for tag in pv.newProofs:
|
|
||||||
if tag notin pv.refPool and tag != pv.rootTag:
|
|
||||||
#debug "Unreferenced proofs rec", tag, tag=tag.pp(pv)
|
|
||||||
return err(RowUnreferenced)
|
|
||||||
|
|
||||||
## verify accounts
|
|
||||||
for (baseTag,accTag) in pv.newAccs:
|
|
||||||
|
|
||||||
# Validate increasing accounts
|
|
||||||
|
|
||||||
# Base and last account must be in database
|
|
||||||
let
|
|
||||||
nBaseDgts = pv.follow(baseTag)[0]
|
|
||||||
(nAccDgts, accData) = pv.follow(accTag)
|
|
||||||
|
|
||||||
# Verify account base
|
|
||||||
# ...
|
|
||||||
|
|
||||||
# Verify last account
|
|
||||||
if nAccDgts == 64:
|
|
||||||
let rc = pv.getAccountRec(accTag)
|
|
||||||
if rc.isOk:
|
|
||||||
noRlpError("validate(Account)"):
|
|
||||||
if accData.decode(Account) == rc.value.to(Account):
|
|
||||||
continue
|
|
||||||
|
|
||||||
# This account list did not verify
|
|
||||||
return err(LastAccountProofFailed)
|
|
||||||
|
|
||||||
ok()
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc mergeProved*(
|
|
||||||
pv: var ProofDb;
|
|
||||||
root: TrieHash;
|
|
||||||
base: NodeTag;
|
|
||||||
data: WorkerAccountRange
|
|
||||||
): Result[void,ProofError] =
|
|
||||||
## Validate and merge accounts and proofs (as received with the snap message
|
|
||||||
## `AccountRange`) into the database. Any open transaction initialised with
|
|
||||||
## `mergeBegin()` is continued ans finished.
|
|
||||||
if not pv.mergeBegin(root):
|
|
||||||
return err(StateRootDiffers)
|
|
||||||
|
|
||||||
block:
|
|
||||||
let rc = pv.merge(data.proof)
|
|
||||||
if rc.isErr:
|
|
||||||
trace "Merge proofs failed",
|
|
||||||
proof=data.proof.len, error=rc.error
|
|
||||||
discard pv.mergeRollback()
|
|
||||||
return err(rc.error)
|
|
||||||
block:
|
|
||||||
let rc = pv.merge(base, data.accounts)
|
|
||||||
if rc.isErr:
|
|
||||||
trace "Merge accounts failed",
|
|
||||||
accounts=data.accounts.len, error=rc.error
|
|
||||||
discard pv.mergeRollback()
|
|
||||||
return err(rc.error)
|
|
||||||
block:
|
|
||||||
let rc = pv.mergeValidate()
|
|
||||||
if rc.isErr:
|
|
||||||
trace "Proofs or accounts do not valdate",
|
|
||||||
accounts=data.accounts.len, error=rc.error
|
|
||||||
discard pv.mergeRollback()
|
|
||||||
return err(rc.error)
|
|
||||||
|
|
||||||
#trace "Merge accounts and proofs ok",
|
|
||||||
# root=pv.rootTag, base=base, accounts=data.accounts.pp, proof=data.proof.pp
|
|
||||||
discard pv.mergeCommit()
|
|
||||||
ok()
|
|
||||||
|
|
||||||
proc proofsLen*(pv: ProofDb; root: TrieHash): int =
|
|
||||||
## Number of entries in the proofs table for the argument state root `root`.
|
|
||||||
if pv.rootHash == root:
|
|
||||||
pv.stat.nProofs
|
|
||||||
else:
|
|
||||||
pv.useStatusRec(root).nProofs
|
|
||||||
|
|
||||||
proc accountsLen*(pv: ProofDb; root: TrieHash): int =
|
|
||||||
## Number of entries in the accounts table for the argument state root `root`.
|
|
||||||
if pv.rootHash == root:
|
|
||||||
pv.stat.nAccounts
|
|
||||||
else:
|
|
||||||
pv.useStatusRec(root).nAccounts
|
|
||||||
|
|
||||||
proc journalLen*(pv: ProofDb): (bool,int,int,int) =
|
|
||||||
## Size of the current rollback journal:
|
|
||||||
## * oepn transaction, see `mergeBegin()`
|
|
||||||
## * number of added recs
|
|
||||||
## * number of added references implied by recs
|
|
||||||
## * number of added accounts
|
|
||||||
(not pv.dbTx.isNil, pv.newProofs.len, pv.refPool.len, pv.newAccs.len)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# End
|
|
||||||
# ------------------------------------------------------------------------------
|
|
|
@ -0,0 +1,193 @@
|
||||||
|
# Nimbus - Fetch account and storage states from peers efficiently
|
||||||
|
#
|
||||||
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
|
# Licensed under either of
|
||||||
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0)
|
||||||
|
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
||||||
|
# http://opensource.org/licenses/MIT)
|
||||||
|
# at your option. This file may not be copied, modified, or distributed
|
||||||
|
# except according to those terms.
|
||||||
|
|
||||||
|
import
|
||||||
|
chronicles,
|
||||||
|
chronos,
|
||||||
|
eth/[common/eth_types, p2p],
|
||||||
|
nimcrypto/keccak,
|
||||||
|
stew/[interval_set, keyed_queue],
|
||||||
|
stint,
|
||||||
|
../../sync_desc,
|
||||||
|
".."/[range_desc, worker_desc],
|
||||||
|
"."/[accounts_db, get_account_range]
|
||||||
|
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "snap-fetch"
|
||||||
|
|
||||||
|
const
|
||||||
|
accRangeMaxLen = ##\
|
||||||
|
## Ask for that many accounts at once (not the range is sparse)
|
||||||
|
(high(NodeTag) - low(NodeTag)) div 1000
|
||||||
|
|
||||||
|
maxTimeoutErrors = ##\
|
||||||
|
## maximal number of non-resonses accepted in a row
|
||||||
|
2
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private helpers
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc withMaxLen(buddy: SnapBuddyRef; iv: LeafRange): LeafRange =
|
||||||
|
## Reduce accounts interval to maximal size
|
||||||
|
let maxlen =
|
||||||
|
if buddy.ctx.data.pivotEnv.proofDumpOk: snapAccountsDumpRange
|
||||||
|
else: buddy.ctx.data.accountRangeMax
|
||||||
|
if 0 < iv.len and iv.len <= maxLen:
|
||||||
|
iv
|
||||||
|
else:
|
||||||
|
LeafRange.new(iv.minPt, iv.minPt + (maxLen - 1.u256))
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc getUnprocessed(buddy: SnapBuddyRef): Result[LeafRange,void] =
|
||||||
|
## Fetch an interval from the account range list. Use the `pivotAccount`
|
||||||
|
## value as a start entry to fetch data from, wrapping around if necessary.
|
||||||
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
env = ctx.data.pivotEnv
|
||||||
|
pivotPt = env.pivotAccount
|
||||||
|
|
||||||
|
block:
|
||||||
|
# Take the next interval to the right (aka ge) `pivotPt`
|
||||||
|
let rc = env.availAccounts.ge(pivotPt)
|
||||||
|
if rc.isOk:
|
||||||
|
let iv = buddy.withMaxLen(rc.value)
|
||||||
|
discard env.availAccounts.reduce(iv)
|
||||||
|
return ok(iv)
|
||||||
|
|
||||||
|
block:
|
||||||
|
# Check whether the `pivotPt` is in the middle of an interval
|
||||||
|
let rc = env.availAccounts.envelope(pivotPt)
|
||||||
|
if rc.isOk:
|
||||||
|
let iv = buddy.withMaxLen(LeafRange.new(pivotPt, rc.value.maxPt))
|
||||||
|
discard env.availAccounts.reduce(iv)
|
||||||
|
return ok(iv)
|
||||||
|
|
||||||
|
block:
|
||||||
|
# Otherwise wrap around
|
||||||
|
let rc = env.availAccounts.ge()
|
||||||
|
if rc.isOk:
|
||||||
|
let iv = buddy.withMaxLen(rc.value)
|
||||||
|
discard env.availAccounts.reduce(iv)
|
||||||
|
return ok(iv)
|
||||||
|
|
||||||
|
err()
|
||||||
|
|
||||||
|
proc putUnprocessed(buddy: SnapBuddyRef; iv: LeafRange) =
|
||||||
|
discard buddy.ctx.data.pivotEnv.availAccounts.merge(iv)
|
||||||
|
|
||||||
|
proc delUnprocessed(buddy: SnapBuddyRef; iv: LeafRange) =
|
||||||
|
discard buddy.ctx.data.pivotEnv.availAccounts.reduce(iv)
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc fetchAccounts*(buddy: SnapBuddyRef): Future[bool] {.async.} =
|
||||||
|
## Fetch accounts data and store them in the database. The function returns
|
||||||
|
## `true` if there are no more unprocessed accounts.
|
||||||
|
##
|
||||||
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
env = ctx.data.pivotEnv
|
||||||
|
stateRoot = env.stateHeader.stateRoot
|
||||||
|
|
||||||
|
# Get a range of accounts to fetch from
|
||||||
|
let iv = block:
|
||||||
|
let rc = buddy.getUnprocessed()
|
||||||
|
if rc.isErr:
|
||||||
|
trace "No more unprocessed accounts", peer, stateRoot
|
||||||
|
return true
|
||||||
|
rc.value
|
||||||
|
|
||||||
|
# Fetch data for this range delegated to `fetchAccounts()`
|
||||||
|
let dd = block:
|
||||||
|
let rc = await buddy.getAccountRange(stateRoot, iv)
|
||||||
|
if rc.isErr:
|
||||||
|
buddy.putUnprocessed(iv) # fail => interval back to pool
|
||||||
|
case rc.error:
|
||||||
|
of ResponseTimeout:
|
||||||
|
if maxTimeoutErrors <= buddy.data.errors.nTimeouts:
|
||||||
|
# Mark this peer dead, i.e. avoid fetching from this peer for a while
|
||||||
|
buddy.ctrl.zombie = true
|
||||||
|
else:
|
||||||
|
buddy.data.errors.nTimeouts.inc
|
||||||
|
await sleepAsync(5.seconds)
|
||||||
|
of NetworkProblem, MissingProof, AccountsMinTooSmall, AccountsMaxTooLarge:
|
||||||
|
# Mark this peer dead, i.e. avoid fetching from this peer for a while
|
||||||
|
buddy.data.stats.major.networkErrors.inc()
|
||||||
|
buddy.ctrl.zombie = true
|
||||||
|
of GetAccountRangeError.NothingSerious:
|
||||||
|
discard
|
||||||
|
of NoAccountsForStateRoot:
|
||||||
|
# Mark this peer dead, i.e. avoid fetching from this peer for a while
|
||||||
|
buddy.ctrl.zombie = true
|
||||||
|
return
|
||||||
|
rc.value
|
||||||
|
|
||||||
|
# Reset error counts
|
||||||
|
buddy.data.errors.nTimeouts = 0
|
||||||
|
|
||||||
|
|
||||||
|
# Process accounts data
|
||||||
|
let
|
||||||
|
nAccounts = dd.data.accounts.len
|
||||||
|
rc = ctx.data.accountsDb.importAccounts(peer, stateRoot, iv.minPt, dd.data)
|
||||||
|
if rc.isErr:
|
||||||
|
# TODO: Prevent deadlock in case there is a problem with the approval
|
||||||
|
# data which is not in production state, yet.
|
||||||
|
trace "Import failed, restoring unprocessed accounts", peer, stateRoot,
|
||||||
|
range=dd.consumed, nAccounts, error=rc.error
|
||||||
|
|
||||||
|
# Just try another peer
|
||||||
|
buddy.ctrl.zombie = true
|
||||||
|
else:
|
||||||
|
# Statistics
|
||||||
|
env.nAccounts.inc(nAccounts)
|
||||||
|
|
||||||
|
# Register consumed and bulk-imported (well, not yet) accounts range
|
||||||
|
let rx = iv - dd.consumed
|
||||||
|
if rx.isOk:
|
||||||
|
# Return some unused range
|
||||||
|
buddy.putUnprocessed(rx.value)
|
||||||
|
else:
|
||||||
|
# The processed interval might be a bit larger
|
||||||
|
let ry = dd.consumed - iv
|
||||||
|
if ry.isOk:
|
||||||
|
# Remove from unprocessed data. If it is not unprocessed, anymore then
|
||||||
|
# it was double processed which if ok.
|
||||||
|
buddy.delUnprocessed(ry.value)
|
||||||
|
|
||||||
|
# ----
|
||||||
|
# For dumping data ready to be used in unit tests
|
||||||
|
if env.proofDumpOk:
|
||||||
|
var fd = ctx.data.proofDumpFile
|
||||||
|
if env.proofDumpInx == 0:
|
||||||
|
fd.write dumpRoot(stateRoot)
|
||||||
|
fd.write "\n"
|
||||||
|
if rc.isErr:
|
||||||
|
fd.write " # Error: base=" & $iv.minPt & " msg=" & $rc.error & "\n"
|
||||||
|
fd.write dumpSnapAccountRange(
|
||||||
|
iv.minPt, dd.data, "snapProofData" & $env.proofDumpInx & "*")
|
||||||
|
fd.flushFile
|
||||||
|
env.proofDumpInx.inc
|
||||||
|
if snapAccountsDumpMax <= env.proofDumpInx:
|
||||||
|
env.proofDumpOk = false
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# End
|
||||||
|
# ------------------------------------------------------------------------------
|
|
@ -16,50 +16,46 @@
|
||||||
import
|
import
|
||||||
chronos,
|
chronos,
|
||||||
eth/[common/eth_types, p2p],
|
eth/[common/eth_types, p2p],
|
||||||
nimcrypto/keccak,
|
|
||||||
stew/interval_set,
|
stew/interval_set,
|
||||||
"../../.."/[protocol, protocol/trace_config, types],
|
"../.."/[protocol, protocol/trace_config],
|
||||||
../../path_desc,
|
".."/[range_desc, worker_desc]
|
||||||
../worker_desc
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
topics = "snap-fetch"
|
topics = "snap-fetch"
|
||||||
|
|
||||||
const
|
|
||||||
snapRequestBytesLimit = 2 * 1024 * 1024
|
|
||||||
## Soft bytes limit to request in `snap` protocol calls.
|
|
||||||
|
|
||||||
type
|
type
|
||||||
FetchError* = enum
|
GetAccountRangeError* = enum
|
||||||
NothingSerious,
|
NothingSerious
|
||||||
MissingProof,
|
MissingProof
|
||||||
AccountsMinTooSmall,
|
AccountsMinTooSmall
|
||||||
AccountsMaxTooLarge,
|
AccountsMaxTooLarge
|
||||||
NoAccountsForStateRoot,
|
NoAccountsForStateRoot
|
||||||
NetworkProblem
|
NetworkProblem
|
||||||
|
ResponseTimeout
|
||||||
|
|
||||||
FetchAccounts* = object
|
GetAccountRange* = object
|
||||||
consumed*: UInt256 ## Leftmost accounts used from argument range
|
consumed*: LeafRange ## Real accounts interval covered
|
||||||
data*: WorkerAccountRange ## reply data
|
data*: SnapAccountRange ## reply data
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private functions
|
# Private functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc getAccountRange(
|
proc getAccountRangeReq(
|
||||||
sp: WorkerBuddy;
|
buddy: SnapBuddyRef;
|
||||||
root: TrieHash;
|
root: Hash256;
|
||||||
iv: LeafRange
|
iv: LeafRange
|
||||||
): Future[Result[Option[WorkerAccountRange],void]] {.async.} =
|
): Future[Result[Option[SnapAccountRange],void]] {.async.} =
|
||||||
|
let
|
||||||
|
peer = buddy.peer
|
||||||
try:
|
try:
|
||||||
let reply = await sp.peer.getAccountRange(
|
let reply = await peer.getAccountRange(
|
||||||
root.to(Hash256), iv.minPt, iv.maxPt, snapRequestBytesLimit)
|
root, iv.minPt.to(Hash256), iv.maxPt.to(Hash256), snapRequestBytesLimit)
|
||||||
return ok(reply)
|
return ok(reply)
|
||||||
|
|
||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
trace trSnapRecvError & "waiting for reply to GetAccountRange", peer=sp,
|
trace trSnapRecvError & "waiting for GetAccountRange reply", peer,
|
||||||
error=e.msg
|
error=e.msg
|
||||||
return err()
|
return err()
|
||||||
|
|
||||||
|
@ -67,25 +63,27 @@ proc getAccountRange(
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc fetchAccounts*(
|
proc getAccountRange*(
|
||||||
peer: WorkerBuddy;
|
buddy: SnapBuddyRef;
|
||||||
stateRoot: TrieHash;
|
stateRoot: Hash256;
|
||||||
iv: LeafRange
|
iv: LeafRange
|
||||||
): Future[Result[FetchAccounts,FetchError]] {.async.} =
|
): Future[Result[GetAccountRange,GetAccountRangeError]] {.async.} =
|
||||||
## Fetch data using the `snap#` protocol, returns the range covered.
|
## Fetch data using the `snap#` protocol, returns the range covered.
|
||||||
|
let
|
||||||
|
peer = buddy.peer
|
||||||
if trSnapTracePacketsOk:
|
if trSnapTracePacketsOk:
|
||||||
trace trSnapSendSending & "GetAccountRange", peer,
|
trace trSnapSendSending & "GetAccountRange", peer,
|
||||||
accRange=iv, stateRoot, bytesLimit=snapRequestBytesLimit
|
accRange=iv, stateRoot, bytesLimit=snapRequestBytesLimit
|
||||||
|
|
||||||
var dd = block:
|
var dd = block:
|
||||||
let rc = await peer.getAccountRange(stateRoot, iv)
|
let rc = await buddy.getAccountRangeReq(stateRoot, iv)
|
||||||
if rc.isErr:
|
if rc.isErr:
|
||||||
return err(NetworkProblem)
|
return err(NetworkProblem)
|
||||||
if rc.value.isNone:
|
if rc.value.isNone:
|
||||||
trace trSnapRecvTimeoutWaiting & "for reply to GetAccountRange", peer
|
trace trSnapRecvTimeoutWaiting & "for reply to GetAccountRange", peer
|
||||||
return err(NothingSerious)
|
return err(ResponseTimeout)
|
||||||
FetchAccounts(
|
GetAccountRange(
|
||||||
consumed: iv.len,
|
consumed: iv,
|
||||||
data: rc.value.get)
|
data: rc.value.get)
|
||||||
|
|
||||||
let
|
let
|
||||||
|
@ -105,20 +103,20 @@ proc fetchAccounts*(
|
||||||
# any) account after limitHash must be provided.
|
# any) account after limitHash must be provided.
|
||||||
if nProof == 0:
|
if nProof == 0:
|
||||||
# Maybe try another peer
|
# Maybe try another peer
|
||||||
trace trSnapRecvReceived & "EMPTY AccountRange reply", peer,
|
trace trSnapRecvReceived & "empty AccountRange", peer,
|
||||||
nAccounts, nProof, accRange="n/a", reqRange=iv, stateRoot
|
nAccounts, nProof, accRange="n/a", reqRange=iv, stateRoot
|
||||||
return err(NoAccountsForStateRoot)
|
return err(NoAccountsForStateRoot)
|
||||||
|
|
||||||
# So there is no data, otherwise an account beyond the interval end
|
# So there is no data, otherwise an account beyond the interval end
|
||||||
# `iv.maxPt` would have been returned.
|
# `iv.maxPt` would have been returned.
|
||||||
trace trSnapRecvReceived & "END AccountRange message", peer,
|
dd.consumed = LeafRange.new(iv.minPt, high(NodeTag))
|
||||||
nAccounts, nProof, accRange=LeafRange.new(iv.minPt, high(NodeTag)),
|
trace trSnapRecvReceived & "terminal AccountRange", peer,
|
||||||
reqRange=iv, stateRoot
|
nAccounts, nProof, accRange=dd.consumed, reqRange=iv, stateRoot
|
||||||
dd.consumed = high(NodeTag) - iv.minPt
|
|
||||||
return ok(dd)
|
return ok(dd)
|
||||||
|
|
||||||
let (accMinPt, accMaxPt) =
|
let (accMinPt, accMaxPt) = (
|
||||||
(dd.data.accounts[0].accHash, dd.data.accounts[^1].accHash)
|
dd.data.accounts[0].accHash.to(NodeTag),
|
||||||
|
dd.data.accounts[^1].accHash.to(NodeTag))
|
||||||
|
|
||||||
if nProof == 0:
|
if nProof == 0:
|
||||||
# github.com/ethereum/devp2p/blob/master/caps/snap.md#accountrange-0x01
|
# github.com/ethereum/devp2p/blob/master/caps/snap.md#accountrange-0x01
|
||||||
|
@ -129,11 +127,10 @@ proc fetchAccounts*(
|
||||||
# situation for storage slots, this clause keeps the behavior the same
|
# situation for storage slots, this clause keeps the behavior the same
|
||||||
# across both.
|
# across both.
|
||||||
if 0.to(NodeTag) < iv.minPt:
|
if 0.to(NodeTag) < iv.minPt:
|
||||||
trace trSnapRecvProtocolViolation & "missing proof in AccountRange", peer,
|
trace trSnapRecvProtocolViolation & "proof-less AccountRange", peer,
|
||||||
nAccounts, nProof, accRange=LeafRange.new(iv.minPt, accMaxPt),
|
nAccounts, nProof, accRange=LeafRange.new(iv.minPt, accMaxPt),
|
||||||
reqRange=iv, stateRoot
|
reqRange=iv, stateRoot
|
||||||
return err(MissingProof)
|
return err(MissingProof)
|
||||||
# TODO: How do I know that the full accounts list is correct?
|
|
||||||
|
|
||||||
if accMinPt < iv.minPt:
|
if accMinPt < iv.minPt:
|
||||||
# Not allowed
|
# Not allowed
|
||||||
|
@ -150,16 +147,18 @@ proc fetchAccounts*(
|
||||||
# * [..] If no accounts exist between startingHash and limitHash, then the
|
# * [..] If no accounts exist between startingHash and limitHash, then the
|
||||||
# first (if any) account after limitHash must be provided.
|
# first (if any) account after limitHash must be provided.
|
||||||
if 1 < nAccounts:
|
if 1 < nAccounts:
|
||||||
trace trSnapRecvProtocolViolation & "max too large in AccountRange", peer,
|
# Geth always seems to allow the last account to be larger than the
|
||||||
nAccounts, nProof, accRange=LeafRange.new(iv.minPt, accMaxPt),
|
# limit (seen with Geth/v1.10.18-unstable-4b309c70-20220517.)
|
||||||
reqRange=iv, stateRoot
|
if iv.maxPt < dd.data.accounts[^2].accHash.to(NodeTag):
|
||||||
return err(AccountsMaxTooLarge)
|
# The segcond largest should not excceed the top one requested.
|
||||||
|
trace trSnapRecvProtocolViolation & "AccountRange top exceeded", peer,
|
||||||
|
nAccounts, nProof, accRange=LeafRange.new(iv.minPt, accMaxPt),
|
||||||
|
reqRange=iv, stateRoot
|
||||||
|
return err(AccountsMaxTooLarge)
|
||||||
|
|
||||||
trace trSnapRecvReceived & "AccountRange message", peer,
|
dd.consumed = LeafRange.new(iv.minPt, accMaxPt)
|
||||||
nAccounts, nProof, accRange=LeafRange.new(iv.minPt, accMaxPt),
|
trace trSnapRecvReceived & "AccountRange", peer,
|
||||||
reqRange=iv, stateRoot
|
nAccounts, nProof, accRange=dd.consumed, reqRange=iv, stateRoot
|
||||||
|
|
||||||
dd.consumed = (accMaxPt - iv.minPt) + 1
|
|
||||||
return ok(dd)
|
return ok(dd)
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
|
@ -0,0 +1,616 @@
|
||||||
|
# Nimbus - Rapidly converge on and track the canonical chain head of each peer
|
||||||
|
#
|
||||||
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
|
# Licensed under either of
|
||||||
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0)
|
||||||
|
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
||||||
|
# http://opensource.org/licenses/MIT)
|
||||||
|
# at your option. This file may not be copied, modified, or distributed
|
||||||
|
# except according to those terms.
|
||||||
|
|
||||||
|
## This module fetches and tracks the canonical chain head of each connected
|
||||||
|
## peer. (Or in future, each peer we care about; we won't poll them all so
|
||||||
|
## often.)
|
||||||
|
##
|
||||||
|
## This is for when we aren't sure of the block number of a peer's canonical
|
||||||
|
## chain head. Most of the time, after finding which block, it quietly polls
|
||||||
|
## to track small updates to the "best" block number and hash of each peer.
|
||||||
|
##
|
||||||
|
## But sometimes that can get out of step. If there has been a deeper reorg
|
||||||
|
## than our tracking window, or a burst of more than a few new blocks, network
|
||||||
|
## delays, downtime, or the peer is itself syncing. Perhaps we stopped Nimbus
|
||||||
|
## and restarted a while later, e.g. suspending a laptop or Control-Z. Then
|
||||||
|
## this will catch up. It is even possible that the best hash the peer gave us
|
||||||
|
## in the `Status` handshake has disappeared by the time we query for the
|
||||||
|
## corresponding block number, so we start at zero.
|
||||||
|
##
|
||||||
|
## The steps here perform a robust and efficient O(log N) search to rapidly
|
||||||
|
## converge on the new best block if it's moved out of the polling window no
|
||||||
|
## matter where it starts, confirm the peer's canonical chain head boundary,
|
||||||
|
## then track the peer's chain head in real-time by polling. The method is
|
||||||
|
## robust to peer state changes at any time.
|
||||||
|
##
|
||||||
|
## The purpose is to:
|
||||||
|
##
|
||||||
|
## - Help with finding a peer common chain prefix ("fast sync pivot") in a
|
||||||
|
## consistent, fast and explicit way.
|
||||||
|
##
|
||||||
|
## - Catch up quickly after any long pauses of network downtime, program not
|
||||||
|
## running, or deep chain reorgs.
|
||||||
|
##
|
||||||
|
## - Be able to display real-time peer states, so they are less mysterious.
|
||||||
|
##
|
||||||
|
## - Tell the beam/snap/trie sync processes when to start and what blocks to
|
||||||
|
## fetch, and keep those fetchers in the head-adjacent window of the
|
||||||
|
## ever-changing chain.
|
||||||
|
##
|
||||||
|
## - Help the sync process bootstrap usefully when we only have one peer,
|
||||||
|
## speculatively fetching and validating what data we can before we have more
|
||||||
|
## peers to corroborate the consensus.
|
||||||
|
##
|
||||||
|
## - Help detect consensus failures in the network.
|
||||||
|
##
|
||||||
|
## We cannot assume a peer's canonical chain stays the same or only gains new
|
||||||
|
## blocks from one query to the next. There can be reorgs, including deep
|
||||||
|
## reorgs. When a reorg happens, the best block number can decrease if the new
|
||||||
|
## canonical chain is shorter than the old one, and the best block hash we
|
||||||
|
## previously knew can become unavailable on the peer. So we must detect when
|
||||||
|
## the current best block disappears and be able to reduce block number.
|
||||||
|
|
||||||
|
import
|
||||||
|
std/bitops,
|
||||||
|
chronicles,
|
||||||
|
chronos,
|
||||||
|
eth/[common/eth_types, p2p, p2p/private/p2p_types],
|
||||||
|
"../../.."/[constants, genesis, p2p/chain/chain_desc],
|
||||||
|
"../.."/[protocol, sync_desc, types],
|
||||||
|
../worker_desc
|
||||||
|
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
export
|
||||||
|
worker_desc
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "snap-pivot"
|
||||||
|
|
||||||
|
const
|
||||||
|
syncLockedMinimumReply = 8
|
||||||
|
## Minimum number of headers we assume any peers will send if they have
|
||||||
|
## them in contiguous ascending queries. Fewer than this confirms we have
|
||||||
|
## found the peer's canonical chain head boundary. Must be at least 2, and
|
||||||
|
## at least `syncLockedQueryOverlap+2` to stay `SyncLocked` when the chain
|
||||||
|
## extends. Should not be large as that would be stretching assumptions
|
||||||
|
## about peer implementations. 8 is chosen as it allows 3-deep extensions
|
||||||
|
## and 3-deep reorgs to be followed in a single round trip.
|
||||||
|
|
||||||
|
syncLockedQueryOverlap = 4
|
||||||
|
## Number of headers to re-query on each poll when `SyncLocked` so that we
|
||||||
|
## get small reorg updates in one round trip. Must be no more than
|
||||||
|
## `syncLockedMinimumReply-1`, no more than `syncLockedMinimumReply-2` to
|
||||||
|
## stay `SyncLocked` when the chain extends, and not too large to avoid
|
||||||
|
## excessive duplicate fetching. 4 is chosen as it allows 3-deep reorgs
|
||||||
|
## to be followed in single round trip.
|
||||||
|
|
||||||
|
syncLockedQuerySize = 192
|
||||||
|
## Query size when polling `SyncLocked`. Must be at least
|
||||||
|
## `syncLockedMinimumReply`. Large is fine, if we get a large reply the
|
||||||
|
## values are almost always useful.
|
||||||
|
|
||||||
|
huntQuerySize = 16
|
||||||
|
## Query size when hunting for canonical head boundary. Small is good
|
||||||
|
## because we don't want to keep most of the headers at hunt time.
|
||||||
|
|
||||||
|
huntForwardExpandShift = 4
|
||||||
|
## Expansion factor during `HuntForward` exponential search.
|
||||||
|
## 16 is chosen for rapid convergence when bootstrapping or catching up.
|
||||||
|
|
||||||
|
huntBackwardExpandShift = 1
|
||||||
|
## Expansion factor during `HuntBackward` exponential search.
|
||||||
|
## 2 is chosen for better convergence when tracking a chain reorg.
|
||||||
|
|
||||||
|
type
|
||||||
|
WorkerMode = enum
|
||||||
|
## The current state of tracking the peer's canonical chain head.
|
||||||
|
## `bestBlockNumber` is only valid when this is `SyncLocked`.
|
||||||
|
SyncLocked
|
||||||
|
SyncOnlyHash
|
||||||
|
HuntForward
|
||||||
|
HuntBackward
|
||||||
|
HuntRange
|
||||||
|
HuntRangeFinal
|
||||||
|
|
||||||
|
WorkerHuntEx = ref object of WorkerBase
|
||||||
|
## Peer canonical chain head ("best block") search state.
|
||||||
|
syncMode: WorkerMode ## Action mode
|
||||||
|
lowNumber: BlockNumber ## Recent lowest known block number.
|
||||||
|
highNumber: BlockNumber ## Recent highest known block number.
|
||||||
|
bestNumber: BlockNumber
|
||||||
|
bestHash: BlockHash
|
||||||
|
step: uint
|
||||||
|
|
||||||
|
static:
|
||||||
|
doAssert syncLockedMinimumReply >= 2
|
||||||
|
doAssert syncLockedMinimumReply >= syncLockedQueryOverlap + 2
|
||||||
|
doAssert syncLockedQuerySize <= maxHeadersFetch
|
||||||
|
doAssert huntQuerySize >= 1 and huntQuerySize <= maxHeadersFetch
|
||||||
|
doAssert huntForwardExpandShift >= 1 and huntForwardExpandShift <= 8
|
||||||
|
doAssert huntBackwardExpandShift >= 1 and huntBackwardExpandShift <= 8
|
||||||
|
|
||||||
|
# Make sure that request/response wire protocol messages are id-tracked and
|
||||||
|
# would not overlap (no multi-protocol legacy support)
|
||||||
|
doAssert 66 <= protocol.ethVersion
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private helpers
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc hunt(buddy: SnapBuddyRef): WorkerHuntEx =
|
||||||
|
buddy.data.workerBase.WorkerHuntEx
|
||||||
|
|
||||||
|
proc `hunt=`(buddy: SnapBuddyRef; value: WorkerHuntEx) =
|
||||||
|
buddy.data.workerBase = value
|
||||||
|
|
||||||
|
proc new(T: type WorkerHuntEx; syncMode: WorkerMode): T =
|
||||||
|
T(syncMode: syncMode,
|
||||||
|
lowNumber: 0.toBlockNumber.BlockNumber,
|
||||||
|
highNumber: high(BlockNumber).BlockNumber, # maximum uncertainty range.
|
||||||
|
bestNumber: 0.toBlockNumber.BlockNumber,
|
||||||
|
bestHash: ZERO_HASH256.BlockHash, # whatever
|
||||||
|
step: 0u)
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private logging helpers
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc traceSyncLocked(buddy: SnapBuddyRef, num: BlockNumber, hash: BlockHash) =
|
||||||
|
## Trace messages when peer canonical head is confirmed or updated.
|
||||||
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
bestBlock = ctx.pp(hash, num)
|
||||||
|
if buddy.hunt.syncMode != SyncLocked:
|
||||||
|
debug "Now tracking chain head of peer", peer,
|
||||||
|
bestBlock
|
||||||
|
elif num > buddy.hunt.bestNumber:
|
||||||
|
if num == buddy.hunt.bestNumber + 1:
|
||||||
|
debug "Peer chain head advanced one block", peer,
|
||||||
|
advance=1, bestBlock
|
||||||
|
else:
|
||||||
|
debug "Peer chain head advanced some blocks", peer,
|
||||||
|
advance=(buddy.hunt.bestNumber - num), bestBlock
|
||||||
|
elif num < buddy.hunt.bestNumber or hash != buddy.hunt.bestHash:
|
||||||
|
debug "Peer chain head reorg detected", peer,
|
||||||
|
advance=(buddy.hunt.bestNumber - num), bestBlock
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc clearSyncStateRoot(buddy: SnapBuddyRef) =
|
||||||
|
if buddy.data.pivotHeader.isSome:
|
||||||
|
debug "Stopping state sync from this peer", peer=buddy.peer
|
||||||
|
buddy.data.pivotHeader = none(BlockHeader)
|
||||||
|
|
||||||
|
proc lockSyncStateAndFetch(buddy: SnapBuddyRef; header: BlockHeader) =
|
||||||
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
stateRoot = header.stateRoot
|
||||||
|
hash = header.blockHash.BlockHash
|
||||||
|
thisBlock = ctx.pp(hash, header.blockNumber)
|
||||||
|
|
||||||
|
buddy.traceSyncLocked(header.blockNumber, hash)
|
||||||
|
buddy.hunt.bestNumber = header.blockNumber
|
||||||
|
buddy.hunt.bestHash = hash
|
||||||
|
buddy.hunt.syncMode = SyncLocked
|
||||||
|
|
||||||
|
if buddy.data.pivotHeader.isNone:
|
||||||
|
debug "Starting state sync from this peer", peer, thisBlock, stateRoot
|
||||||
|
elif buddy.data.pivotHeader.unsafeGet.stateRoot != stateRoot:
|
||||||
|
trace "Adjusting state sync root from this peer", peer, thisBlock, stateRoot
|
||||||
|
|
||||||
|
buddy.data.pivotHeader = some(header)
|
||||||
|
|
||||||
|
proc setHuntBackward(buddy: SnapBuddyRef, lowestAbsent: BlockNumber) =
|
||||||
|
## Start exponential search mode backward due to new uncertainty.
|
||||||
|
buddy.hunt.syncMode = HuntBackward
|
||||||
|
buddy.hunt.step = 0
|
||||||
|
# Block zero is always present.
|
||||||
|
buddy.hunt.lowNumber = 0.toBlockNumber
|
||||||
|
# Zero `lowestAbsent` is never correct, but an incorrect peer could send it.
|
||||||
|
buddy.hunt.highNumber =
|
||||||
|
if lowestAbsent > 0: lowestAbsent
|
||||||
|
else: 1.toBlockNumber
|
||||||
|
buddy.clearSyncStateRoot()
|
||||||
|
|
||||||
|
proc setHuntForward(buddy: SnapBuddyRef, highestPresent: BlockNumber) =
|
||||||
|
## Start exponential search mode forward due to new uncertainty.
|
||||||
|
buddy.hunt.syncMode = HuntForward
|
||||||
|
buddy.hunt.step = 0
|
||||||
|
buddy.hunt.lowNumber = highestPresent
|
||||||
|
buddy.hunt.highNumber = high(BlockNumber)
|
||||||
|
buddy.clearSyncStateRoot()
|
||||||
|
|
||||||
|
proc updateHuntAbsent(buddy: SnapBuddyRef, lowestAbsent: BlockNumber) =
|
||||||
|
## Converge uncertainty range backward.
|
||||||
|
if lowestAbsent < buddy.hunt.highNumber:
|
||||||
|
buddy.hunt.highNumber = lowestAbsent
|
||||||
|
# If uncertainty range has moved outside the search window, change to hunt
|
||||||
|
# backward to block zero. Note that empty uncertainty range is allowed
|
||||||
|
# (empty range is `hunt.lowNumber + 1 == hunt.highNumber`).
|
||||||
|
if buddy.hunt.highNumber <= buddy.hunt.lowNumber:
|
||||||
|
buddy.setHuntBackward(lowestAbsent)
|
||||||
|
buddy.clearSyncStateRoot()
|
||||||
|
|
||||||
|
proc updateHuntPresent(buddy: SnapBuddyRef, highestPresent: BlockNumber) =
|
||||||
|
## Converge uncertainty range forward.
|
||||||
|
if highestPresent > buddy.hunt.lowNumber:
|
||||||
|
buddy.hunt.lowNumber = highestPresent
|
||||||
|
# If uncertainty range has moved outside the search window, change to hunt
|
||||||
|
# forward to no upper limit. Note that empty uncertainty range is allowed
|
||||||
|
# (empty range is `hunt.lowNumber + 1 == hunt.highNumber`).
|
||||||
|
if buddy.hunt.lowNumber >= buddy.hunt.highNumber:
|
||||||
|
buddy.setHuntForward(highestPresent)
|
||||||
|
buddy.clearSyncStateRoot()
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private functions, assemble request
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc peerSyncChainRequest(buddy: SnapBuddyRef): BlocksRequest =
|
||||||
|
## Choose `GetBlockHeaders` parameters when hunting or following the canonical
|
||||||
|
## chain of a peer.
|
||||||
|
if buddy.hunt.syncMode == SyncLocked:
|
||||||
|
# Stable and locked. This is just checking for changes including reorgs.
|
||||||
|
# `buddy.hunt.bestNumber` was recently the head of the peer's canonical
|
||||||
|
# chain. We must include this block number to detect when the canonical
|
||||||
|
# chain gets shorter versus no change.
|
||||||
|
result.startBlock.number =
|
||||||
|
if buddy.hunt.bestNumber <= syncLockedQueryOverlap:
|
||||||
|
# Every peer should send genesis for block 0, so don't ask for it.
|
||||||
|
# `peerSyncChainEmptyReply` has logic to handle this reply as if it
|
||||||
|
# was for block 0. Aside from saving bytes, this is more robust if
|
||||||
|
# some client doesn't do genesis reply correctly.
|
||||||
|
1.toBlockNumber
|
||||||
|
else:
|
||||||
|
min(buddy.hunt.bestNumber - syncLockedQueryOverlap.toBlockNumber,
|
||||||
|
high(BlockNumber) - (syncLockedQuerySize - 1).toBlockNumber)
|
||||||
|
result.maxResults = syncLockedQuerySize
|
||||||
|
return
|
||||||
|
|
||||||
|
if buddy.hunt.syncMode == SyncOnlyHash:
|
||||||
|
# We only have the hash of the recent head of the peer's canonical chain.
|
||||||
|
# Like `SyncLocked`, query more than one item to detect when the
|
||||||
|
# canonical chain gets shorter, no change or longer.
|
||||||
|
result.startBlock = buddy.hunt.bestHash.to(HashOrNum)
|
||||||
|
result.maxResults = syncLockedQuerySize
|
||||||
|
return
|
||||||
|
|
||||||
|
# Searching for the peers's canonical head. An ascending query is always
|
||||||
|
# used, regardless of search direction. This is because a descending query
|
||||||
|
# (`reverse = true` and `maxResults > 1`) is useless for searching: Either
|
||||||
|
# `startBlock` is present, in which case the extra descending results
|
||||||
|
# contribute no more information about the canonical head boundary, or
|
||||||
|
# `startBlock` is absent in which case there are zero results. It's not
|
||||||
|
# defined in the `eth` specification that there must be zero results (in
|
||||||
|
# principle peers could return the lower numbered blocks), but in practice
|
||||||
|
# peers stop at the first absent block in the sequence from `startBlock`.
|
||||||
|
#
|
||||||
|
# Guaranteeing O(log N) time convergence in all scenarios requires some
|
||||||
|
# properties to be true in both exponential search (expanding) and
|
||||||
|
# quasi-binary search (converging in a range). The most important is that
|
||||||
|
# the gap to `startBlock` after `hunt.lowNumber` and also before
|
||||||
|
# `hunt.highNumber` are proportional to the query step, where the query step
|
||||||
|
# is `hunt.step` exponentially expanding each round, or `maxStep`
|
||||||
|
# approximately evenly distributed in the range.
|
||||||
|
#
|
||||||
|
# `hunt.lowNumber+1` must not be used consistently as the start, even with a
|
||||||
|
# large enough query step size, as that will sometimes take O(N) to converge
|
||||||
|
# in both the exponential and quasi-binary searches. (Ending at
|
||||||
|
# `hunt.highNumber-1` is fine if `huntQuerySize > 1`. This asymmetry is
|
||||||
|
# due to ascending queries (see earlier comment), and non-empty truncated
|
||||||
|
# query reply being proof of presence before the truncation point, but not
|
||||||
|
# proof of absence after it. A reply can be truncated just because the peer
|
||||||
|
# decides to.)
|
||||||
|
#
|
||||||
|
# The proportional gap requirement is why we divide by query size here,
|
||||||
|
# instead of stretching to fit more strictly with `(range-1)/(size-1)`.
|
||||||
|
|
||||||
|
const huntFinalSize = max(2, huntQuerySize)
|
||||||
|
var maxStep = 0u
|
||||||
|
|
||||||
|
let fullRangeClamped =
|
||||||
|
if buddy.hunt.highNumber <= buddy.hunt.lowNumber: 0u
|
||||||
|
else: min(high(uint).toBlockNumber,
|
||||||
|
buddy.hunt.highNumber - buddy.hunt.lowNumber).truncate(uint) - 1
|
||||||
|
|
||||||
|
if fullRangeClamped >= huntFinalSize: # `HuntRangeFinal` condition.
|
||||||
|
maxStep = if huntQuerySize == 1:
|
||||||
|
fullRangeClamped
|
||||||
|
elif (huntQuerySize and (huntQuerySize-1)) == 0:
|
||||||
|
fullRangeClamped shr fastLog2(huntQuerySize)
|
||||||
|
else:
|
||||||
|
fullRangeClamped div huntQuerySize
|
||||||
|
doAssert huntFinalSize >= huntQuerySize
|
||||||
|
doAssert maxStep >= 1 # Ensured by the above assertion.
|
||||||
|
|
||||||
|
# Check for exponential search (expanding). Iterate `hunt.step`. O(log N)
|
||||||
|
# requires `startBlock` to be offset from `hunt.lowNumber`/`hunt.highNumber`.
|
||||||
|
if buddy.hunt.syncMode in {HuntForward, HuntBackward} and
|
||||||
|
fullRangeClamped >= huntFinalSize:
|
||||||
|
let forward = buddy.hunt.syncMode == HuntForward
|
||||||
|
let expandShift = if forward: huntForwardExpandShift
|
||||||
|
else: huntBackwardExpandShift
|
||||||
|
# Switches to range search when this condition is no longer true.
|
||||||
|
if buddy.hunt.step < maxStep shr expandShift:
|
||||||
|
# The `if` above means the next line cannot overflow.
|
||||||
|
buddy.hunt.step = if buddy.hunt.step > 0: buddy.hunt.step shl expandShift
|
||||||
|
else: 1
|
||||||
|
# Satisfy the O(log N) convergence conditions.
|
||||||
|
result.startBlock.number =
|
||||||
|
if forward: buddy.hunt.lowNumber + buddy.hunt.step.toBlockNumber
|
||||||
|
else: buddy.hunt.highNumber -
|
||||||
|
(buddy.hunt.step * huntQuerySize).toBlockNumber
|
||||||
|
result.maxResults = huntQuerySize
|
||||||
|
result.skip = buddy.hunt.step - 1
|
||||||
|
return
|
||||||
|
|
||||||
|
# For tracing/display.
|
||||||
|
buddy.hunt.step = maxStep
|
||||||
|
buddy.hunt.syncMode = HuntRange
|
||||||
|
if maxStep > 0:
|
||||||
|
# Quasi-binary search (converging in a range). O(log N) requires
|
||||||
|
# `startBlock` to satisfy the constraints described above, with the
|
||||||
|
# proportionality from both ends of the range. The optimal information
|
||||||
|
# gathering position is tricky and doesn't make much difference, so don't
|
||||||
|
# bother. We'll centre the query in the range.
|
||||||
|
var offset = fullRangeClamped - maxStep * (huntQuerySize-1)
|
||||||
|
# Rounding must bias towards end to ensure `offset >= 1` after this.
|
||||||
|
offset -= offset shr 1
|
||||||
|
result.startBlock.number = buddy.hunt.lowNumber + offset.toBlockNumber
|
||||||
|
result.maxResults = huntQuerySize
|
||||||
|
result.skip = maxStep - 1
|
||||||
|
else:
|
||||||
|
# Small range, final step. At `fullRange == 0` we must query at least one
|
||||||
|
# block before and after the range to confirm the canonical head boundary,
|
||||||
|
# or find it has moved. This ensures progress without getting stuck. When
|
||||||
|
# `fullRange` is small this is also beneficial, to get `SyncLocked` in one
|
||||||
|
# round trip from hereand it simplifies the other search branches below.
|
||||||
|
# Ideally the query is similar to `SyncLocked`, enough to get `SyncLocked`
|
||||||
|
# in one round trip, and accommodate a small reorg or extension.
|
||||||
|
const afterSoftMax = syncLockedMinimumReply - syncLockedQueryOverlap
|
||||||
|
const beforeHardMax = syncLockedQueryOverlap
|
||||||
|
let extra = huntFinalSize - fullRangeClamped
|
||||||
|
var before = (extra + 1) shr 1
|
||||||
|
before = max(before + afterSoftMax, extra) - afterSoftMax
|
||||||
|
before = min(before, beforeHardMax)
|
||||||
|
# See `SyncLocked` case.
|
||||||
|
result.startBlock.number =
|
||||||
|
if buddy.hunt.bestNumber <= before.toBlockNumber: 1.toBlockNumber
|
||||||
|
else: min(buddy.hunt.bestNumber - before.toBlockNumber,
|
||||||
|
high(BlockNumber) - (huntFinalSize - 1).toBlockNumber)
|
||||||
|
result.maxResults = huntFinalSize
|
||||||
|
buddy.hunt.syncMode = HuntRangeFinal
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private functions, reply handling
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc peerSyncChainEmptyReply(buddy: SnapBuddyRef; request: BlocksRequest) =
|
||||||
|
## Handle empty `GetBlockHeaders` reply. This means `request.startBlock` is
|
||||||
|
## absent on the peer. If it was `SyncLocked` there must have been a reorg
|
||||||
|
## and the previous canonical chain head has disappeared. If hunting, this
|
||||||
|
## updates the range of uncertainty.
|
||||||
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
peer = buddy.peer
|
||||||
|
|
||||||
|
# Treat empty response to a request starting from block 1 as equivalent to
|
||||||
|
# length 1 starting from block 0 in `peerSyncChainNonEmptyReply`. We treat
|
||||||
|
# every peer as if it would send genesis for block 0, without asking for it.
|
||||||
|
if request.skip == 0 and
|
||||||
|
not request.reverse and
|
||||||
|
not request.startBlock.isHash and
|
||||||
|
request.startBlock.number == 1.toBlockNumber:
|
||||||
|
try:
|
||||||
|
buddy.lockSyncStateAndFetch(ctx.chain.Chain.db.toGenesisHeader)
|
||||||
|
except RlpError as e:
|
||||||
|
raiseAssert "Gensis/chain problem (" & $e.name & "): " & e.msg
|
||||||
|
return
|
||||||
|
|
||||||
|
if buddy.hunt.syncMode in {SyncLocked, SyncOnlyHash}:
|
||||||
|
inc buddy.data.stats.ok.reorgDetected
|
||||||
|
trace "Peer reorg detected, best block disappeared", peer,
|
||||||
|
startBlock=request.startBlock
|
||||||
|
|
||||||
|
let lowestAbsent = request.startBlock.number
|
||||||
|
case buddy.hunt.syncMode:
|
||||||
|
of SyncLocked:
|
||||||
|
# If this message doesn't change our knowledge, ignore it.
|
||||||
|
if lowestAbsent > buddy.hunt.bestNumber:
|
||||||
|
return
|
||||||
|
# Due to a reorg, peer's canonical head has lower block number, outside
|
||||||
|
# our tracking window. Sync lock is no longer valid. Switch to hunt
|
||||||
|
# backward to find the new canonical head.
|
||||||
|
buddy.setHuntBackward(lowestAbsent)
|
||||||
|
of SyncOnlyHash:
|
||||||
|
# Due to a reorg, peer doesn't have the block hash it originally gave us.
|
||||||
|
# Switch to hunt forward from block zero to find the canonical head.
|
||||||
|
buddy.setHuntForward(0.toBlockNumber)
|
||||||
|
of HuntForward, HuntBackward, HuntRange, HuntRangeFinal:
|
||||||
|
# Update the hunt range.
|
||||||
|
buddy.updateHuntAbsent(lowestAbsent)
|
||||||
|
|
||||||
|
# Update best block number. It is invalid except when `SyncLocked`, but
|
||||||
|
# still useful as a hint of what we knew recently, for example in displays.
|
||||||
|
if lowestAbsent <= buddy.hunt.bestNumber:
|
||||||
|
buddy.hunt.bestNumber =
|
||||||
|
if lowestAbsent == 0.toBlockNumber: lowestAbsent
|
||||||
|
else: lowestAbsent - 1.toBlockNumber
|
||||||
|
buddy.hunt.bestHash = default(typeof(buddy.hunt.bestHash))
|
||||||
|
ctx.seen(buddy.hunt.bestHash,buddy.hunt.bestNumber)
|
||||||
|
|
||||||
|
|
||||||
|
proc peerSyncChainNonEmptyReply(
|
||||||
|
buddy: SnapBuddyRef;
|
||||||
|
request: BlocksRequest;
|
||||||
|
headers: openArray[BlockHeader]) =
|
||||||
|
## Handle non-empty `GetBlockHeaders` reply. This means `request.startBlock`
|
||||||
|
## is present on the peer and in its canonical chain (unless the request was
|
||||||
|
## made with a hash). If it's a short, contiguous, ascending order reply, it
|
||||||
|
## reveals the abrupt transition at the end of the chain and we have learned
|
||||||
|
## or reconfirmed the real-time head block. If hunting, this updates the
|
||||||
|
## range of uncertainty.
|
||||||
|
|
||||||
|
let
|
||||||
|
ctx = buddy.ctx
|
||||||
|
len = headers.len
|
||||||
|
highestIndex = if request.reverse: 0 else: len - 1
|
||||||
|
|
||||||
|
# We assume a short enough reply means we've learned the peer's canonical
|
||||||
|
# head, because it would have replied with another header if not at the head.
|
||||||
|
# This is not justified when the request used a general hash, because the
|
||||||
|
# peer doesn't have to reply with its canonical chain in that case, except it
|
||||||
|
# is still justified if the hash was the known canonical head, which is
|
||||||
|
# the case in a `SyncOnlyHash` request.
|
||||||
|
if len < syncLockedMinimumReply and
|
||||||
|
request.skip == 0 and not request.reverse and
|
||||||
|
len.uint < request.maxResults:
|
||||||
|
buddy.lockSyncStateAndFetch(headers[highestIndex])
|
||||||
|
return
|
||||||
|
|
||||||
|
# Be careful, this number is from externally supplied data and arithmetic
|
||||||
|
# in the upward direction could overflow.
|
||||||
|
let highestPresent = headers[highestIndex].blockNumber
|
||||||
|
|
||||||
|
# A reply that isn't short enough for the canonical head criterion above
|
||||||
|
# tells us headers up to some number, but it doesn't tell us if there are
|
||||||
|
# more after it in the peer's canonical chain. We have to request more
|
||||||
|
# headers to find out.
|
||||||
|
case buddy.hunt.syncMode:
|
||||||
|
of SyncLocked:
|
||||||
|
# If this message doesn't change our knowledge, ignore it.
|
||||||
|
if highestPresent <= buddy.hunt.bestNumber:
|
||||||
|
return
|
||||||
|
# Sync lock is no longer valid as we don't have confirmed canonical head.
|
||||||
|
# Switch to hunt forward to find the new canonical head.
|
||||||
|
buddy.setHuntForward(highestPresent)
|
||||||
|
of SyncOnlyHash:
|
||||||
|
# As `SyncLocked` but without the block number check.
|
||||||
|
buddy.setHuntForward(highestPresent)
|
||||||
|
of HuntForward, HuntBackward, HuntRange, HuntRangeFinal:
|
||||||
|
# Update the hunt range.
|
||||||
|
buddy.updateHuntPresent(highestPresent)
|
||||||
|
|
||||||
|
# Update best block number. It is invalid except when `SyncLocked`, but
|
||||||
|
# still useful as a hint of what we knew recently, for example in displays.
|
||||||
|
if highestPresent > buddy.hunt.bestNumber:
|
||||||
|
buddy.hunt.bestNumber = highestPresent
|
||||||
|
buddy.hunt.bestHash = headers[highestIndex].blockHash.BlockHash
|
||||||
|
ctx.seen(buddy.hunt.bestHash,buddy.hunt.bestNumber)
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public start/stop and admin functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc pivotStart*(buddy: SnapBuddyRef) =
|
||||||
|
## Setup state root hunter
|
||||||
|
buddy.hunt = WorkerHuntEx.new(HuntForward)
|
||||||
|
|
||||||
|
# We know the hash but not the block number.
|
||||||
|
buddy.hunt.bestHash = buddy.peer.state(protocol.eth).bestBlockHash.BlockHash
|
||||||
|
|
||||||
|
# TODO: Temporarily disabled because it's useful to test the worker.
|
||||||
|
# buddy.syncMode = SyncOnlyHash
|
||||||
|
|
||||||
|
proc pivotStop*(buddy: SnapBuddyRef) =
|
||||||
|
## Clean up this peer
|
||||||
|
discard
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc pivotExec*(buddy: SnapBuddyRef) {.async.} =
|
||||||
|
## Query a peer to update our knowledge of its canonical chain and its best
|
||||||
|
## block, which is its canonical chain head. This can be called at any time
|
||||||
|
## after a peer has negotiated the connection.
|
||||||
|
##
|
||||||
|
## This function is called in an exponential then binary search style
|
||||||
|
## during initial sync to find the canonical head, real-time polling
|
||||||
|
## afterwards to check for updates.
|
||||||
|
##
|
||||||
|
## All replies to this query are part of the peer's canonical chain at the
|
||||||
|
## time the peer sends them.
|
||||||
|
let
|
||||||
|
peer = buddy.peer
|
||||||
|
ctx = buddy.ctx
|
||||||
|
|
||||||
|
trace "Starting pivotExec()", peer
|
||||||
|
|
||||||
|
let
|
||||||
|
request = buddy.peerSyncChainRequest
|
||||||
|
|
||||||
|
trace trEthSendSendingGetBlockHeaders, peer,
|
||||||
|
count=request.maxResults,
|
||||||
|
startBlock=ctx.pp(request.startBlock), step=request.traceStep
|
||||||
|
|
||||||
|
inc buddy.data.stats.ok.getBlockHeaders
|
||||||
|
var reply: Option[protocol.blockHeadersObj]
|
||||||
|
try:
|
||||||
|
reply = await peer.getBlockHeaders(request)
|
||||||
|
except CatchableError as e:
|
||||||
|
trace trEthRecvError & "waiting for GetBlockHeaders reply", peer,
|
||||||
|
error=e.msg
|
||||||
|
inc buddy.data.stats.major.networkErrors
|
||||||
|
buddy.pivotStop()
|
||||||
|
return
|
||||||
|
|
||||||
|
if reply.isNone:
|
||||||
|
trace trEthRecvTimeoutWaiting & "for GetBlockHeaders reply", peer
|
||||||
|
# TODO: Should disconnect?
|
||||||
|
inc buddy.data.stats.minor.timeoutBlockHeaders
|
||||||
|
return
|
||||||
|
|
||||||
|
let nHeaders = reply.get.headers.len
|
||||||
|
if nHeaders == 0:
|
||||||
|
trace trEthRecvReceivedBlockHeaders, peer,
|
||||||
|
got=0, requested=request.maxResults
|
||||||
|
else:
|
||||||
|
trace trEthRecvReceivedBlockHeaders, peer,
|
||||||
|
got=nHeaders, requested=request.maxResults,
|
||||||
|
firstBlock=reply.get.headers[0].blockNumber,
|
||||||
|
lastBlock=reply.get.headers[^1].blockNumber
|
||||||
|
|
||||||
|
if request.maxResults.int < nHeaders:
|
||||||
|
trace trEthRecvProtocolViolation & "excess headers in BlockHeaders message",
|
||||||
|
peer, got=nHeaders, requested=request.maxResults
|
||||||
|
# TODO: Should disconnect.
|
||||||
|
inc buddy.data.stats.major.excessBlockHeaders
|
||||||
|
return
|
||||||
|
|
||||||
|
if 0 < nHeaders:
|
||||||
|
# TODO: Check this is not copying the `headers`.
|
||||||
|
buddy.peerSyncChainNonEmptyReply(request, reply.get.headers)
|
||||||
|
else:
|
||||||
|
buddy.peerSyncChainEmptyReply(request)
|
||||||
|
|
||||||
|
trace "Done pivotExec()", peer
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Debugging
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc huntPp*(buddy: SnapBuddyRef): string =
|
||||||
|
let hx = buddy.hunt
|
||||||
|
result &= "(mode=" & $hx.syncMode
|
||||||
|
result &= ",num=(" & hx.lowNumber.pp & "," & hx.highNumber.pp & ")"
|
||||||
|
result &= ",best=(" & hx.bestNumber.pp & "," & hx.bestHash.pp & ")"
|
||||||
|
result &= ",step=" & $hx.step
|
||||||
|
result &= ")"
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# End
|
||||||
|
# ------------------------------------------------------------------------------
|
|
@ -16,8 +16,7 @@ import
|
||||||
eth/[common/eth_types, p2p],
|
eth/[common/eth_types, p2p],
|
||||||
stint,
|
stint,
|
||||||
../../../utils/prettify,
|
../../../utils/prettify,
|
||||||
../../timer_helper,
|
../../timer_helper
|
||||||
./worker_desc
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
@ -26,25 +25,28 @@ logScope:
|
||||||
|
|
||||||
type
|
type
|
||||||
TickerStats* = object
|
TickerStats* = object
|
||||||
|
pivotBlock*: Option[BlockNumber]
|
||||||
accounts*: (float,float) ## mean and standard deviation
|
accounts*: (float,float) ## mean and standard deviation
|
||||||
fillFactor*: (float,float) ## mean and standard deviation
|
fillFactor*: (float,float) ## mean and standard deviation
|
||||||
activeQueues*: int
|
activeQueues*: int
|
||||||
flushedQueues*: int64
|
flushedQueues*: int64
|
||||||
|
|
||||||
TickerStatsUpdater* =
|
TickerStatsUpdater* =
|
||||||
proc(ns: Worker): TickerStats {.gcsafe, raises: [Defect].}
|
proc: TickerStats {.gcsafe, raises: [Defect].}
|
||||||
|
|
||||||
TickerEx = ref object of WorkerTickerBase
|
TickerRef* = ref object
|
||||||
## Account fetching state that is shared among all peers.
|
## Account fetching state that is shared among all peers.
|
||||||
ns: Worker
|
nBuddies: int
|
||||||
peersActive: int
|
lastStats: TickerStats
|
||||||
statsCb: TickerStatsUpdater
|
lastTick: uint64
|
||||||
logTicker: TimerCallback
|
statsCb: TickerStatsUpdater
|
||||||
tick: uint64 # more than 5*10^11y before wrap when ticking every sec
|
logTicker: TimerCallback
|
||||||
|
tick: uint64 # more than 5*10^11y before wrap when ticking every sec
|
||||||
|
|
||||||
const
|
const
|
||||||
defaultTickerStartDelay = 100.milliseconds
|
tickerStartDelay = 100.milliseconds
|
||||||
tickerLogInterval = 1.seconds
|
tickerLogInterval = 1.seconds
|
||||||
|
tickerLogSuppressMax = 100
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private functions: ticking log messages
|
# Private functions: ticking log messages
|
||||||
|
@ -56,92 +58,81 @@ template noFmtError(info: static[string]; code: untyped) =
|
||||||
except ValueError as e:
|
except ValueError as e:
|
||||||
raiseAssert "Inconveivable (" & info & "): " & e.msg
|
raiseAssert "Inconveivable (" & info & "): " & e.msg
|
||||||
|
|
||||||
proc setLogTicker(sf: TickerEx; at: Moment) {.gcsafe.}
|
proc setLogTicker(t: TickerRef; at: Moment) {.gcsafe.}
|
||||||
|
|
||||||
proc runLogTicker(sf: TickerEx) {.gcsafe.} =
|
proc runLogTicker(t: TickerRef) {.gcsafe.} =
|
||||||
var
|
let data = t.statsCb()
|
||||||
avAccounts = ""
|
|
||||||
avUtilisation = ""
|
|
||||||
let
|
|
||||||
tick = sf.tick.toSI
|
|
||||||
peers = sf.peersActive
|
|
||||||
|
|
||||||
y = sf.statsCb(sf.ns)
|
if data != t.lastStats or
|
||||||
queues = y.activeQueues
|
t.lastTick + tickerLogSuppressMax < t.tick:
|
||||||
flushed = y.flushedQueues
|
t.lastStats = data
|
||||||
mem = getTotalMem().uint.toSI
|
t.lastTick = t.tick
|
||||||
|
var
|
||||||
|
avAccounts = ""
|
||||||
|
avUtilisation = ""
|
||||||
|
pivot = "n/a"
|
||||||
|
let
|
||||||
|
flushed = data.flushedQueues
|
||||||
|
|
||||||
noFmtError("runLogTicker"):
|
buddies = t.nBuddies
|
||||||
avAccounts = (&"{(y.accounts[0]+0.5).int64}({(y.accounts[1]+0.5).int64})")
|
tick = t.tick.toSI
|
||||||
avUtilisation = &"{y.fillFactor[0]*100.0:.2f}%({y.fillFactor[1]*100.0:.2f}%)"
|
mem = getTotalMem().uint.toSI
|
||||||
|
|
||||||
info "Sync queue average statistics",
|
noFmtError("runLogTicker"):
|
||||||
tick, peers, queues, avAccounts, avUtilisation, flushed, mem
|
if data.pivotBlock.isSome:
|
||||||
|
pivot = &"#{data.pivotBlock.get}({data.activeQueues})"
|
||||||
|
avAccounts =
|
||||||
|
&"{(data.accounts[0]+0.5).int64}({(data.accounts[1]+0.5).int64})"
|
||||||
|
avUtilisation =
|
||||||
|
&"{data.fillFactor[0]*100.0:.2f}%({data.fillFactor[1]*100.0:.2f}%)"
|
||||||
|
|
||||||
sf.tick.inc
|
info "Snap sync statistics",
|
||||||
sf.setLogTicker(Moment.fromNow(tickerLogInterval))
|
tick, buddies, pivot, avAccounts, avUtilisation, flushed, mem
|
||||||
|
|
||||||
proc setLogTicker(sf: TickerEx; at: Moment) =
|
t.tick.inc
|
||||||
if sf.logTicker.isNil:
|
t.setLogTicker(Moment.fromNow(tickerLogInterval))
|
||||||
debug "Sync accounts progress ticker has stopped"
|
|
||||||
else:
|
|
||||||
sf.logTicker = safeSetTimer(at, runLogTicker, sf)
|
proc setLogTicker(t: TickerRef; at: Moment) =
|
||||||
|
if not t.logTicker.isNil:
|
||||||
|
t.logTicker = safeSetTimer(at, runLogTicker, t)
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Private getters/setters
|
# Public constructor and start/stop functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc tickerEx(ns: Worker): TickerEx =
|
proc init*(T: type TickerRef; cb: TickerStatsUpdater): T =
|
||||||
## Handy helper
|
## Constructor
|
||||||
ns.tickerBase.TickerEx
|
T(statsCb: cb)
|
||||||
|
|
||||||
proc `tickerEx=`(ns: Worker; value: TickerEx) =
|
proc start*(t: TickerRef) =
|
||||||
## Handy helper
|
|
||||||
ns.tickerBase = value
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public start/stop functions!
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc tickerSetup*(ns: Worker; cb: TickerStatsUpdater) =
|
|
||||||
## Global set up
|
|
||||||
if ns.tickerEx.isNil:
|
|
||||||
ns.tickerEx = TickerEx(ns: ns)
|
|
||||||
ns.tickerEx.statsCb = cb
|
|
||||||
|
|
||||||
proc tickerRelease*(ns: Worker) =
|
|
||||||
## Global clean up
|
|
||||||
if not ns.tickerEx.isNil:
|
|
||||||
ns.tickerEx.logTicker = nil # stop timer
|
|
||||||
ns.tickerEx = nil # unlink `TickerEx` object
|
|
||||||
|
|
||||||
proc tickerStart*(ns: Worker) =
|
|
||||||
## Re/start ticker unconditionally
|
## Re/start ticker unconditionally
|
||||||
ns.tickerEx.tick = 0
|
#debug "Started ticker"
|
||||||
ns.tickerEx.logTicker = safeSetTimer(
|
t.logTicker = safeSetTimer(Moment.fromNow(tickerStartDelay), runLogTicker, t)
|
||||||
Moment.fromNow(defaultTickerStartDelay),
|
|
||||||
runLogTicker,
|
|
||||||
ns.tickerEx)
|
|
||||||
|
|
||||||
proc tickerStop*(ns: Worker) =
|
proc stop*(t: TickerRef) =
|
||||||
## Stop ticker unconditionally
|
## Stop ticker unconditionally
|
||||||
ns.tickerEx.logTicker = nil
|
t.logTicker = nil
|
||||||
|
#debug "Stopped ticker"
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc tickerStartPeer*(sp: WorkerBuddy) =
|
proc startBuddy*(t: TickerRef) =
|
||||||
if sp.ns.tickerEx.peersActive <= 0:
|
## Increment buddies counter and start ticker unless running.
|
||||||
sp.ns.tickerEx.peersActive = 1
|
if t.nBuddies <= 0:
|
||||||
sp.ns.tickerStart()
|
t.nBuddies = 1
|
||||||
|
t.start()
|
||||||
else:
|
else:
|
||||||
sp.ns.tickerEx.peersActive.inc
|
t.nBuddies.inc
|
||||||
|
|
||||||
proc tickerStopPeer*(sp: WorkerBuddy) =
|
proc stopBuddy*(t: TickerRef) =
|
||||||
sp.ns.tickerEx.peersActive.dec
|
## Decrement buddies counter and stop ticker if there are no more registered
|
||||||
if sp.ns.tickerEx.peersActive <= 0:
|
## buddies.
|
||||||
sp.ns.tickerStop()
|
t.nBuddies.dec
|
||||||
|
if t.nBuddies <= 0:
|
||||||
|
t.stop()
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# End
|
# End
|
||||||
|
|
|
@ -1,237 +0,0 @@
|
||||||
# Nimbus - New sync approach - A fusion of snap, trie, beam and other methods
|
|
||||||
#
|
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
|
||||||
# Licensed under either of
|
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
|
||||||
# http://www.apache.org/licenses/LICENSE-2.0)
|
|
||||||
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
|
||||||
# http://opensource.org/licenses/MIT)
|
|
||||||
# at your option. This file may not be copied, modified, or distributed
|
|
||||||
# except according to those terms.
|
|
||||||
|
|
||||||
import
|
|
||||||
std/[sequtils, strutils],
|
|
||||||
eth/[common/eth_types, p2p],
|
|
||||||
nimcrypto/hash,
|
|
||||||
stew/[byteutils, keyed_queue],
|
|
||||||
../../../constants,
|
|
||||||
"../.."/[protocol/snap1, types]
|
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
|
||||||
|
|
||||||
const
|
|
||||||
seenBlocksMax = 500
|
|
||||||
## Internal size of LRU cache (for debugging)
|
|
||||||
|
|
||||||
type
|
|
||||||
WorkerBase* = ref object of RootObj
|
|
||||||
## Stub object, to be inherited in file `worker.nim`
|
|
||||||
|
|
||||||
BuddyStat* = distinct uint
|
|
||||||
|
|
||||||
BuddyRunState = enum
|
|
||||||
## Combined state of two boolean values (`stopped`,`stopThisState`) as used
|
|
||||||
## in the original source set up (should be double checked and simplified.)
|
|
||||||
Running ## running, not requested to stop
|
|
||||||
Stopped ## stopped, stop request
|
|
||||||
ZombieStop ## abanon/ignore (LRU tab overflow, odd packets)
|
|
||||||
ZombieRun ## additional zombie state to potentially recover from
|
|
||||||
|
|
||||||
WorkerBuddyStats* = tuple
|
|
||||||
## Statistics counters for events associated with this peer.
|
|
||||||
## These may be used to recognise errors and select good peers.
|
|
||||||
ok: tuple[
|
|
||||||
reorgDetected: BuddyStat,
|
|
||||||
getBlockHeaders: BuddyStat,
|
|
||||||
getNodeData: BuddyStat]
|
|
||||||
minor: tuple[
|
|
||||||
timeoutBlockHeaders: BuddyStat,
|
|
||||||
unexpectedBlockHash: BuddyStat]
|
|
||||||
major: tuple[
|
|
||||||
networkErrors: BuddyStat,
|
|
||||||
excessBlockHeaders: BuddyStat,
|
|
||||||
wrongBlockHeader: BuddyStat]
|
|
||||||
|
|
||||||
WorkerBuddyCtrl* = object
|
|
||||||
## Control and state settings
|
|
||||||
stateRoot*: Option[TrieHash]
|
|
||||||
## State root to fetch state for. This changes during sync and is
|
|
||||||
## slightly different for each peer.
|
|
||||||
runState: BuddyRunState
|
|
||||||
## Access with getters
|
|
||||||
|
|
||||||
# -------
|
|
||||||
|
|
||||||
WorkerSeenBlocks = KeyedQueue[array[32,byte],BlockNumber]
|
|
||||||
## Temporary for pretty debugging, `BlockHash` keyed lru cache
|
|
||||||
|
|
||||||
WorkerTickerBase* = ref object of RootObj
|
|
||||||
## Stub object, to be inherited in file `ticker.nim`
|
|
||||||
|
|
||||||
WorkerFetchBase* = ref object of RootObj
|
|
||||||
## Stub object, to be inherited in file `fetch.nim`
|
|
||||||
|
|
||||||
# -------
|
|
||||||
|
|
||||||
WorkerBuddy* = ref object
|
|
||||||
## Non-inheritable peer state tracking descriptor.
|
|
||||||
ns*: Worker ## Worker descriptor object back reference
|
|
||||||
peer*: Peer ## Reference to eth p2pProtocol entry
|
|
||||||
stats*: WorkerBuddyStats ## Statistics counters
|
|
||||||
ctrl*: WorkerBuddyCtrl ## Control and state settings
|
|
||||||
|
|
||||||
workerBase*: WorkerBase ## Opaque object reference for sub-module
|
|
||||||
# ...
|
|
||||||
|
|
||||||
Worker* = ref object of RootObj
|
|
||||||
## Shared state among all peers of a snap syncing node. Will be
|
|
||||||
## amended/inherited into `SnapSyncCtx` by the `snap` module which
|
|
||||||
## will also manage a list of `WorkerBuddy` objects.
|
|
||||||
seenBlock: WorkerSeenBlocks ## Temporary, debugging, pretty logs
|
|
||||||
|
|
||||||
buddiesMax*: int ## Max number of buddies (for LRU caches)
|
|
||||||
|
|
||||||
fetchBase*: WorkerFetchBase ## Opaque object reference
|
|
||||||
tickerBase*: WorkerTickerBase ## Opaque object reference
|
|
||||||
|
|
||||||
# -------
|
|
||||||
|
|
||||||
WorkerAccountRange* = accountRangeObj
|
|
||||||
## Syntactic sugar, type defined in `snap1`
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public Constructor
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc new*(T: type WorkerBuddy; ns: Worker; peer: Peer): T =
|
|
||||||
## Initial state all default settings.
|
|
||||||
T(ns: ns, peer: peer)
|
|
||||||
|
|
||||||
proc init*(ctrl: var WorkerBuddyCtrl; running: bool) =
|
|
||||||
## Set initial running state `Running` if the argument `running`
|
|
||||||
## is `true`. Otherwise the running state is set `stopped`.
|
|
||||||
ctrl.runState = if running: Running else: Stopped
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc `$`*(sp: WorkerBuddy): string =
|
|
||||||
$sp.peer
|
|
||||||
|
|
||||||
proc inc(stat: var BuddyStat) {.borrow.}
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public getters, `BuddyRunState` execution control functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc state*(ctrl: WorkerBuddyCtrl): BuddyRunState =
|
|
||||||
## Getter (logging only, details of `BuddyRunState` are private)
|
|
||||||
ctrl.runState
|
|
||||||
|
|
||||||
proc running*(ctrl: WorkerBuddyCtrl): bool =
|
|
||||||
## Getter, if `true` if `ctrl.state()` is `Running`
|
|
||||||
ctrl.runState == Running
|
|
||||||
|
|
||||||
proc stopped*(ctrl: WorkerBuddyCtrl): bool =
|
|
||||||
## Getter, if `true`, if `ctrl.state()` is not `Running`
|
|
||||||
ctrl.runState in {Stopped, ZombieStop, ZombieRun}
|
|
||||||
|
|
||||||
proc zombie*(ctrl: WorkerBuddyCtrl): bool =
|
|
||||||
## Getter, `true` if `ctrl.state()` is `Zombie` (i.e. not `running()` and
|
|
||||||
## not `stopped()`)
|
|
||||||
ctrl.runState in {ZombieStop, ZombieRun}
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public setters, `BuddyRunState` execution control functions
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc `zombie=`*(ctrl: var WorkerBuddyCtrl; value: bool) =
|
|
||||||
## Setter
|
|
||||||
if value:
|
|
||||||
case ctrl.runState:
|
|
||||||
of Running:
|
|
||||||
ctrl.runState = ZombieRun
|
|
||||||
of Stopped:
|
|
||||||
ctrl.runState = ZombieStop
|
|
||||||
else:
|
|
||||||
discard
|
|
||||||
else:
|
|
||||||
case ctrl.runState:
|
|
||||||
of ZombieRun:
|
|
||||||
ctrl.runState = Running
|
|
||||||
of ZombieStop:
|
|
||||||
ctrl.runState = Stopped
|
|
||||||
else:
|
|
||||||
discard
|
|
||||||
|
|
||||||
proc `stopped=`*(ctrl: var WorkerBuddyCtrl; value: bool) =
|
|
||||||
## Setter
|
|
||||||
if value:
|
|
||||||
case ctrl.runState:
|
|
||||||
of Running:
|
|
||||||
ctrl.runState = Stopped
|
|
||||||
else:
|
|
||||||
discard
|
|
||||||
else:
|
|
||||||
case ctrl.runState:
|
|
||||||
of Stopped:
|
|
||||||
ctrl.runState = Running
|
|
||||||
else:
|
|
||||||
discard
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# Public functions, debugging helpers (will go away eventually)
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
|
|
||||||
proc pp*(sn: Worker; bh: BlockHash): string =
|
|
||||||
## Pretty printer for debugging
|
|
||||||
let rc = sn.seenBlock.lruFetch(bh.to(Hash256).data)
|
|
||||||
if rc.isOk:
|
|
||||||
return "#" & $rc.value
|
|
||||||
$bh.to(Hash256).data.toHex
|
|
||||||
|
|
||||||
proc pp*(sn: Worker; bh: BlockHash; bn: BlockNumber): string =
|
|
||||||
## Pretty printer for debugging
|
|
||||||
let rc = sn.seenBlock.lruFetch(bh.to(Hash256).data)
|
|
||||||
if rc.isOk:
|
|
||||||
return "#" & $rc.value
|
|
||||||
"#" & $sn.seenBlock.lruAppend(bh.to(Hash256).data, bn, seenBlocksMax)
|
|
||||||
|
|
||||||
proc pp*(sn: Worker; bhn: HashOrNum): string =
|
|
||||||
if not bhn.isHash:
|
|
||||||
return "num(#" & $bhn.number & ")"
|
|
||||||
let rc = sn.seenBlock.lruFetch(bhn.hash.data)
|
|
||||||
if rc.isOk:
|
|
||||||
return "hash(#" & $rc.value & ")"
|
|
||||||
return "hash(" & $bhn.hash.data.toHex & ")"
|
|
||||||
|
|
||||||
proc seen*(sn: Worker; bh: BlockHash; bn: BlockNumber) =
|
|
||||||
## Register for pretty printing
|
|
||||||
if not sn.seenBlock.lruFetch(bh.to(Hash256).data).isOk:
|
|
||||||
discard sn.seenBlock.lruAppend(bh.to(Hash256).data, bn, seenBlocksMax)
|
|
||||||
|
|
||||||
proc pp*(a: MDigest[256]; collapse = true): string =
|
|
||||||
if not collapse:
|
|
||||||
a.data.mapIt(it.toHex(2)).join.toLowerAscii
|
|
||||||
elif a == BLANK_ROOT_HASH:
|
|
||||||
"BLANK_ROOT_HASH"
|
|
||||||
elif a == EMPTY_UNCLE_HASH:
|
|
||||||
"EMPTY_UNCLE_HASH"
|
|
||||||
elif a == EMPTY_SHA3:
|
|
||||||
"EMPTY_SHA3"
|
|
||||||
elif a == ZERO_HASH256:
|
|
||||||
"ZERO_HASH256"
|
|
||||||
else:
|
|
||||||
a.data.mapIt(it.toHex(2)).join[56 .. 63].toLowerAscii
|
|
||||||
|
|
||||||
proc pp*(bh: BlockHash): string =
|
|
||||||
bh.Hash256.pp
|
|
||||||
|
|
||||||
proc pp*(bn: BlockNumber): string =
|
|
||||||
if bn == high(BlockNumber): "#max"
|
|
||||||
else: "#" & $bn
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
|
||||||
# End
|
|
||||||
# ------------------------------------------------------------------------------
|
|
|
@ -0,0 +1,193 @@
|
||||||
|
# Nimbus - New sync approach - A fusion of snap, trie, beam and other methods
|
||||||
|
#
|
||||||
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
|
# Licensed under either of
|
||||||
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0)
|
||||||
|
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
||||||
|
# http://opensource.org/licenses/MIT)
|
||||||
|
# at your option. This file may not be copied, modified, or distributed
|
||||||
|
# except according to those terms.
|
||||||
|
|
||||||
|
import
|
||||||
|
std/[sequtils, strutils],
|
||||||
|
eth/[common/eth_types, p2p],
|
||||||
|
nimcrypto,
|
||||||
|
stew/[byteutils, keyed_queue],
|
||||||
|
../../constants,
|
||||||
|
".."/[sync_desc, types],
|
||||||
|
./worker/[accounts_db, ticker],
|
||||||
|
./range_desc
|
||||||
|
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
const
|
||||||
|
snapRequestBytesLimit* = 2 * 1024 * 1024
|
||||||
|
## Soft bytes limit to request in `snap` protocol calls.
|
||||||
|
|
||||||
|
maxPivotBlockWindow* = 500
|
||||||
|
## The maximal depth of two block headers. If the pivot block header
|
||||||
|
## (containing the current state root) is more than this many blocks
|
||||||
|
## away from a new pivot block header candidate, then the latter one
|
||||||
|
## replaces the current block header.
|
||||||
|
|
||||||
|
snapAccountsDumpRangeKiln = (high(UInt256) div 300000)
|
||||||
|
## Sample size for a single snap dump on `kiln` (for debugging)
|
||||||
|
|
||||||
|
snapAccountsDumpRange* = snapAccountsDumpRangeKiln
|
||||||
|
## Activated size of a data slice if dump is anabled
|
||||||
|
|
||||||
|
snapAccountsDumpMax* = 20
|
||||||
|
## Max number of snap proof dumps (for debugging)
|
||||||
|
|
||||||
|
snapAccountsDumpEnable* = false
|
||||||
|
## Enable data dump
|
||||||
|
|
||||||
|
seenBlocksMax = 500
|
||||||
|
## Internal size of LRU cache (for debugging)
|
||||||
|
|
||||||
|
type
|
||||||
|
WorkerBase* = ref object of RootObj
|
||||||
|
## Stub object, to be inherited in file `worker.nim`
|
||||||
|
|
||||||
|
BuddyStat* = distinct uint
|
||||||
|
|
||||||
|
SnapBuddyStats* = tuple
|
||||||
|
## Statistics counters for events associated with this peer.
|
||||||
|
## These may be used to recognise errors and select good peers.
|
||||||
|
ok: tuple[
|
||||||
|
reorgDetected: BuddyStat,
|
||||||
|
getBlockHeaders: BuddyStat,
|
||||||
|
getNodeData: BuddyStat]
|
||||||
|
minor: tuple[
|
||||||
|
timeoutBlockHeaders: BuddyStat,
|
||||||
|
unexpectedBlockHash: BuddyStat]
|
||||||
|
major: tuple[
|
||||||
|
networkErrors: BuddyStat,
|
||||||
|
excessBlockHeaders: BuddyStat,
|
||||||
|
wrongBlockHeader: BuddyStat]
|
||||||
|
|
||||||
|
SnapBuddyErrors* = tuple
|
||||||
|
## particular error counters so connections will not be cut immediately
|
||||||
|
## after a particular error.
|
||||||
|
nTimeouts: uint
|
||||||
|
|
||||||
|
# -------
|
||||||
|
|
||||||
|
WorkerSeenBlocks = KeyedQueue[array[32,byte],BlockNumber]
|
||||||
|
## Temporary for pretty debugging, `BlockHash` keyed lru cache
|
||||||
|
|
||||||
|
WorkerTickerBase* = ref object of RootObj
|
||||||
|
## Stub object, to be inherited in file `ticker.nim`
|
||||||
|
|
||||||
|
WorkerFetchBase* = ref object of RootObj
|
||||||
|
## Stub object, to be inherited in file `fetch.nim`
|
||||||
|
|
||||||
|
WorkerFetchEnvBase* = ref object of RootObj
|
||||||
|
## Stub object, to be inherited in file `fetch.nim`
|
||||||
|
|
||||||
|
SnapPivotRef* = ref object
|
||||||
|
## Stub object, cache for particular snap data environment
|
||||||
|
stateHeader*: BlockHeader ## Pivot state, containg state root
|
||||||
|
pivotAccount*: NodeTag ## Random account
|
||||||
|
availAccounts*: LeafRangeSet ## Accounts to fetch (organised as ranges)
|
||||||
|
nAccounts*: uint64 ## Number of accounts imported
|
||||||
|
# fetchEnv*: WorkerFetchEnvBase ## Opaque object reference
|
||||||
|
# ---
|
||||||
|
proofDumpOk*: bool
|
||||||
|
proofDumpInx*: int
|
||||||
|
|
||||||
|
SnapPivotTable* = ##\
|
||||||
|
## LRU table, indexed by state root
|
||||||
|
KeyedQueue[Hash256,SnapPivotRef]
|
||||||
|
|
||||||
|
BuddyData* = object
|
||||||
|
## Local descriptor data extension
|
||||||
|
stats*: SnapBuddyStats ## Statistics counters
|
||||||
|
errors*: SnapBuddyErrors ## For error handling
|
||||||
|
pivotHeader*: Option[BlockHeader] ## For pivot state hunter
|
||||||
|
workerBase*: WorkerBase ## Opaque object reference for sub-module
|
||||||
|
|
||||||
|
CtxData* = object
|
||||||
|
## Globally shared data extension
|
||||||
|
seenBlock: WorkerSeenBlocks ## Temporary, debugging, pretty logs
|
||||||
|
rng*: ref HmacDrbgContext ## Random generator
|
||||||
|
ticker*: TickerRef ## Ticker, logger
|
||||||
|
pivotTable*: SnapPivotTable ## Per state root environment
|
||||||
|
pivotCount*: uint64 ## Total of all created tab entries
|
||||||
|
pivotEnv*: SnapPivotRef ## Environment containing state root
|
||||||
|
accountRangeMax*: UInt256 ## Maximal length, high(u256)/#peers
|
||||||
|
accountsDb*: AccountsDbRef ## Proof processing for accounts
|
||||||
|
# ---
|
||||||
|
proofDumpOk*: bool
|
||||||
|
proofDumpFile*: File
|
||||||
|
|
||||||
|
SnapBuddyRef* = ##\
|
||||||
|
## Extended worker peer descriptor
|
||||||
|
BuddyRef[CtxData,BuddyData]
|
||||||
|
|
||||||
|
SnapCtxRef* = ##\
|
||||||
|
## Extended global descriptor
|
||||||
|
CtxRef[CtxData]
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc inc(stat: var BuddyStat) {.borrow.}
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public functions, debugging helpers (will go away eventually)
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc pp*(ctx: SnapCtxRef; bh: BlockHash): string =
|
||||||
|
## Pretty printer for debugging
|
||||||
|
let rc = ctx.data.seenBlock.lruFetch(bh.to(Hash256).data)
|
||||||
|
if rc.isOk:
|
||||||
|
return "#" & $rc.value
|
||||||
|
"%" & $bh.to(Hash256).data.toHex
|
||||||
|
|
||||||
|
proc pp*(ctx: SnapCtxRef; bh: BlockHash; bn: BlockNumber): string =
|
||||||
|
## Pretty printer for debugging
|
||||||
|
let rc = ctx.data.seenBlock.lruFetch(bh.to(Hash256).data)
|
||||||
|
if rc.isOk:
|
||||||
|
return "#" & $rc.value
|
||||||
|
"#" & $ctx.data.seenBlock.lruAppend(bh.to(Hash256).data, bn, seenBlocksMax)
|
||||||
|
|
||||||
|
proc pp*(ctx: SnapCtxRef; bhn: HashOrNum): string =
|
||||||
|
if not bhn.isHash:
|
||||||
|
return "#" & $bhn.number
|
||||||
|
let rc = ctx.data.seenBlock.lruFetch(bhn.hash.data)
|
||||||
|
if rc.isOk:
|
||||||
|
return "%" & $rc.value
|
||||||
|
return "%" & $bhn.hash.data.toHex
|
||||||
|
|
||||||
|
proc seen*(ctx: SnapCtxRef; bh: BlockHash; bn: BlockNumber) =
|
||||||
|
## Register for pretty printing
|
||||||
|
if not ctx.data.seenBlock.lruFetch(bh.to(Hash256).data).isOk:
|
||||||
|
discard ctx.data.seenBlock.lruAppend(bh.to(Hash256).data, bn, seenBlocksMax)
|
||||||
|
|
||||||
|
proc pp*(a: MDigest[256]; collapse = true): string =
|
||||||
|
if not collapse:
|
||||||
|
a.data.mapIt(it.toHex(2)).join.toLowerAscii
|
||||||
|
elif a == BLANK_ROOT_HASH:
|
||||||
|
"BLANK_ROOT_HASH"
|
||||||
|
elif a == EMPTY_UNCLE_HASH:
|
||||||
|
"EMPTY_UNCLE_HASH"
|
||||||
|
elif a == EMPTY_SHA3:
|
||||||
|
"EMPTY_SHA3"
|
||||||
|
elif a == ZERO_HASH256:
|
||||||
|
"ZERO_HASH256"
|
||||||
|
else:
|
||||||
|
a.data.mapIt(it.toHex(2)).join[56 .. 63].toLowerAscii
|
||||||
|
|
||||||
|
proc pp*(bh: BlockHash): string =
|
||||||
|
"%" & bh.Hash256.pp
|
||||||
|
|
||||||
|
proc pp*(bn: BlockNumber): string =
|
||||||
|
if bn == high(BlockNumber): "#high"
|
||||||
|
else: "#" & $bn
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# End
|
||||||
|
# ------------------------------------------------------------------------------
|
|
@ -1,5 +1,4 @@
|
||||||
# Nimbus - New sync approach - A fusion of snap, trie, beam and other methods
|
# Nimbus
|
||||||
#
|
|
||||||
# Copyright (c) 2021 Status Research & Development GmbH
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
# Licensed under either of
|
# Licensed under either of
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
@ -9,63 +8,53 @@
|
||||||
# at your option. This file may not be copied, modified, or distributed
|
# at your option. This file may not be copied, modified, or distributed
|
||||||
# except according to those terms.
|
# except according to those terms.
|
||||||
|
|
||||||
|
## Worker peers scheduler template
|
||||||
|
## ===============================
|
||||||
|
##
|
||||||
|
## Public descriptors
|
||||||
|
|
||||||
import
|
import
|
||||||
eth/[common/eth_types, p2p]
|
eth/[common/eth_types, p2p]
|
||||||
|
|
||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
type
|
type
|
||||||
BuddyRunState = enum
|
BuddyRunState* = enum
|
||||||
## Combined state of two boolean values (`stopped`,`stopThisState`) as used
|
Running = 0 ## Running, default state
|
||||||
## in the original source set up (should be double checked and simplified.)
|
Stopped ## Stopped or about stopping
|
||||||
Running = 0 ## running, default state
|
ZombieStop ## Abandon/ignore (wait for pushed out of LRU table)
|
||||||
Stopped ## stopped or about stopping
|
ZombieRun ## Extra zombie state to potentially recover from
|
||||||
ZombieStop ## abandon/ignore (LRU tab overflow, odd packets)
|
|
||||||
ZombieRun ## extra zombie state to potentially recover from
|
|
||||||
|
|
||||||
BuddyCtrl* = object
|
BuddyCtrl* = object
|
||||||
## Control and state settings
|
## Control and state settings
|
||||||
runState: BuddyRunState ## Access with getters
|
runState: BuddyRunState ## Access with getters
|
||||||
multiPeer: bool ## Triggers `runSingle()` mode unless `true`
|
multiOk*: bool ## Triggers `runSingle()` mode unless `true`
|
||||||
|
|
||||||
BuddyDataRef* = ref object of RootObj
|
BuddyRef*[S,W] = ref object
|
||||||
## Stub object, to be inherited in file `worker.nim`
|
## Worker peer state descriptor.
|
||||||
|
ctx*: CtxRef[S] ## Shared data descriptor back reference
|
||||||
BuddyRef* = ref object
|
|
||||||
## Non-inheritable peer state tracking descriptor.
|
|
||||||
ctx*: CtxRef ## Shared data back reference
|
|
||||||
peer*: Peer ## Reference to eth p2pProtocol entry
|
peer*: Peer ## Reference to eth p2pProtocol entry
|
||||||
ctrl*: BuddyCtrl ## Control and state settings
|
ctrl*: BuddyCtrl ## Control and state settings
|
||||||
data*: BuddyDataRef ## Opaque object reference for sub-module
|
data*: W ## Worker peer specific data
|
||||||
|
|
||||||
# -----
|
CtxRef*[S] = ref object
|
||||||
|
## Shared state among all syncing peer workers (aka buddies.)
|
||||||
CtxDataRef* = ref object of RootObj
|
buddiesMax*: int ## Max number of buddies
|
||||||
## Stub object, to be inherited in file `worker.nim`
|
chain*: AbstractChainDB ## Block chain database (no need for `Peer`)
|
||||||
|
|
||||||
CtxRef* = ref object of RootObj
|
|
||||||
## Shared state among all syncing peer workers (aka buddies.) This object
|
|
||||||
## Will be amended/inherited main module which controls the peer workers.
|
|
||||||
buddiesMax*: int ## Max number of buddies (for LRU cache, read only)
|
|
||||||
chain*: AbstractChainDB ## Block chain database (read only reference)
|
|
||||||
poolMode*: bool ## Activate `runPool()` workers if set `true`
|
poolMode*: bool ## Activate `runPool()` workers if set `true`
|
||||||
data*: CtxDataRef ## Opaque object reference for sub-module
|
data*: S ## Shared context for all worker peers
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc `$`*(buddy: BuddyRef): string =
|
proc `$`*[S,W](worker: BuddyRef[S,W]): string =
|
||||||
$buddy.peer & "$" & $buddy.ctrl.runState
|
$worker.peer & "$" & $worker.ctrl.runState
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public getters, `BuddyRunState` execution control functions
|
# Public getters, `BuddyRunState` execution control functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc multiOk*(ctrl: BuddyCtrl): bool =
|
|
||||||
## Getter
|
|
||||||
ctrl.multiPeer
|
|
||||||
|
|
||||||
proc state*(ctrl: BuddyCtrl): BuddyRunState =
|
proc state*(ctrl: BuddyCtrl): BuddyRunState =
|
||||||
## Getter (logging only, details of `BuddyCtrl` are private)
|
## Getter (logging only, details of `BuddyCtrl` are private)
|
||||||
ctrl.runState
|
ctrl.runState
|
||||||
|
@ -87,10 +76,6 @@ proc zombie*(ctrl: BuddyCtrl): bool =
|
||||||
# Public setters, `BuddyRunState` execution control functions
|
# Public setters, `BuddyRunState` execution control functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc `multiOk=`*(ctrl: var BuddyCtrl; val: bool) =
|
|
||||||
## Setter
|
|
||||||
ctrl.multiPeer = val
|
|
||||||
|
|
||||||
proc `zombie=`*(ctrl: var BuddyCtrl; value: bool) =
|
proc `zombie=`*(ctrl: var BuddyCtrl; value: bool) =
|
||||||
## Setter
|
## Setter
|
||||||
if value:
|
if value:
|
|
@ -0,0 +1,282 @@
|
||||||
|
# Nimbus
|
||||||
|
# Copyright (c) 2021 Status Research & Development GmbH
|
||||||
|
# Licensed under either of
|
||||||
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0)
|
||||||
|
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
||||||
|
# http://opensource.org/licenses/MIT)
|
||||||
|
# at your option. This file may not be copied, modified, or distributed
|
||||||
|
# except according to those terms.
|
||||||
|
|
||||||
|
## Sync worker peers scheduler template
|
||||||
|
## ====================================
|
||||||
|
##
|
||||||
|
## Virtual method/interface functions to be provided as `mixin`:
|
||||||
|
##
|
||||||
|
## *runSetup(ctx: CtxRef[S]; tickerOK: bool): bool*
|
||||||
|
## Global set up. This function will be called before any worker peer is
|
||||||
|
## started. If that function returns `false`, no worker peers will be run.
|
||||||
|
##
|
||||||
|
## *runRelease(ctx: CtxRef[S])*
|
||||||
|
## Global clean up, done with all the worker peers.
|
||||||
|
##
|
||||||
|
##
|
||||||
|
## *runStart(buddy: BuddyRef[S,W]): bool*
|
||||||
|
## Initialise a new worker peer.
|
||||||
|
##
|
||||||
|
## *runStop(buddy: BuddyRef[S,W])*
|
||||||
|
## Clean up this worker peer.
|
||||||
|
##
|
||||||
|
##
|
||||||
|
## *runPool(buddy: BuddyRef[S,W])*
|
||||||
|
## Once started, the function `runPool()` is called for all worker peers in
|
||||||
|
## sequence as the body of an iteration. There will be no other worker peer
|
||||||
|
## functions activated simultaneously.
|
||||||
|
##
|
||||||
|
## This procedure is started if the global flag `buddy.ctx.poolMode` is set
|
||||||
|
## `true` (default is `false`.) It is the responsibility of the `runPool()`
|
||||||
|
## instance to reset the flag `buddy.ctx.poolMode`, typically at the first
|
||||||
|
## peer instance as the number of active instances is unknown to `runPool()`.
|
||||||
|
##
|
||||||
|
## Note that this function does not run in `async` mode.
|
||||||
|
##
|
||||||
|
##
|
||||||
|
## *runSingle(buddy: BuddyRef[S,W]) {.async.}*
|
||||||
|
## This worker peer method is invoked if the peer-local flag
|
||||||
|
## `buddy.ctrl.multiOk` is set `false` which is the default mode. This flag
|
||||||
|
## is updated by the worker peer when deemed appropriate.
|
||||||
|
## * For all workers, there can be only one `runSingle()` function active
|
||||||
|
## simultaneously for all worker peers.
|
||||||
|
## * There will be no `runMulti()` function active for the same worker peer
|
||||||
|
## simultaneously
|
||||||
|
## * There will be no `runPool()` iterator active simultaneously.
|
||||||
|
##
|
||||||
|
## Note that this function runs in `async` mode.
|
||||||
|
##
|
||||||
|
## *runMulti(buddy: BuddyRef[S,W]) {.async.}*
|
||||||
|
## This worker peer method is invoked if the `buddy.ctrl.multiOk` flag is
|
||||||
|
## set `true` which is typically done after finishing `runSingle()`. This
|
||||||
|
## instance can be simultaneously active for all worker peers.
|
||||||
|
##
|
||||||
|
##
|
||||||
|
## Additional import files needed when using this template:
|
||||||
|
## * eth/[common/eth_types, p2p]
|
||||||
|
## * chronicles
|
||||||
|
## * chronos
|
||||||
|
## * stew/[interval_set, sorted_set],
|
||||||
|
## * "."/[sync_desc, sync_sched, protocol]
|
||||||
|
##
|
||||||
|
|
||||||
|
import
|
||||||
|
std/hashes,
|
||||||
|
chronos,
|
||||||
|
eth/[common/eth_types, p2p, p2p/peer_pool, p2p/private/p2p_types],
|
||||||
|
stew/keyed_queue,
|
||||||
|
./sync_desc
|
||||||
|
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
type
|
||||||
|
ActiveBuddies[S,W] = ##\
|
||||||
|
## List of active workers, using `Hash(Peer)` rather than `Peer`
|
||||||
|
KeyedQueue[Hash,RunnerBuddyRef[S,W]]
|
||||||
|
|
||||||
|
RunnerSyncRef*[S,W] = ref object
|
||||||
|
## Module descriptor
|
||||||
|
ctx*: CtxRef[S] ## Shared data
|
||||||
|
pool: PeerPool ## For starting the system
|
||||||
|
buddies: ActiveBuddies[S,W] ## LRU cache with worker descriptors
|
||||||
|
tickerOk: bool ## Ticker logger
|
||||||
|
singleRunLock: bool ## For worker initialisation
|
||||||
|
monitorLock: bool ## For worker monitor
|
||||||
|
activeMulti: int ## Activated runners
|
||||||
|
|
||||||
|
RunnerBuddyRef[S,W] = ref object
|
||||||
|
## Per worker peer descriptor
|
||||||
|
dsc: RunnerSyncRef[S,W] ## Scheduler descriptor
|
||||||
|
worker: BuddyRef[S,W] ## Worker peer data
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private helpers
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc hash(peer: Peer): Hash =
|
||||||
|
## Needed for `buddies` table key comparison
|
||||||
|
peer.remote.id.hash
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Private functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc workerLoop[S,W](buddy: RunnerBuddyRef[S,W]) {.async.} =
|
||||||
|
mixin runMulti, runSingle, runPool, runStop
|
||||||
|
let
|
||||||
|
dsc = buddy.dsc
|
||||||
|
ctx = dsc.ctx
|
||||||
|
worker = buddy.worker
|
||||||
|
peer = worker.peer
|
||||||
|
|
||||||
|
# Continue until stopped
|
||||||
|
while not worker.ctrl.stopped:
|
||||||
|
if dsc.monitorLock:
|
||||||
|
await sleepAsync(50.milliseconds)
|
||||||
|
continue
|
||||||
|
|
||||||
|
# Invoke `runPool()` over all buddies if requested
|
||||||
|
if ctx.poolMode:
|
||||||
|
# Grab `monitorLock` (was `false` as checked above) and wait until clear
|
||||||
|
# to run as the only activated instance.
|
||||||
|
dsc.monitorLock = true
|
||||||
|
while 0 < dsc.activeMulti:
|
||||||
|
await sleepAsync(50.milliseconds)
|
||||||
|
while dsc.singleRunLock:
|
||||||
|
await sleepAsync(50.milliseconds)
|
||||||
|
for w in dsc.buddies.nextValues:
|
||||||
|
worker.runPool()
|
||||||
|
dsc.monitorLock = false
|
||||||
|
continue
|
||||||
|
|
||||||
|
# Rotate connection table so the most used entry is at the top/right
|
||||||
|
# end. So zombies will end up leftish.
|
||||||
|
discard dsc.buddies.lruFetch(peer.hash)
|
||||||
|
|
||||||
|
# Allow task switch
|
||||||
|
await sleepAsync(50.milliseconds)
|
||||||
|
|
||||||
|
# Multi mode
|
||||||
|
if worker.ctrl.multiOk:
|
||||||
|
if not dsc.singleRunLock:
|
||||||
|
dsc.activeMulti.inc
|
||||||
|
# Continue doing something, work a bit
|
||||||
|
await worker.runMulti()
|
||||||
|
dsc.activeMulti.dec
|
||||||
|
continue
|
||||||
|
|
||||||
|
# Single mode as requested. The `multiOk` flag for this worker was just
|
||||||
|
# found `false` in the pervious clause.
|
||||||
|
if not dsc.singleRunLock:
|
||||||
|
# Lock single instance mode and wait for other workers to finish
|
||||||
|
dsc.singleRunLock = true
|
||||||
|
while 0 < dsc.activeMulti:
|
||||||
|
await sleepAsync(50.milliseconds)
|
||||||
|
# Run single instance and release afterwards
|
||||||
|
await worker.runSingle()
|
||||||
|
dsc.singleRunLock = false
|
||||||
|
|
||||||
|
# End while
|
||||||
|
|
||||||
|
# Note that `runStart()` was dispatched in `onPeerConnected()`
|
||||||
|
worker.runStop()
|
||||||
|
|
||||||
|
|
||||||
|
proc onPeerConnected[S,W](dsc: RunnerSyncRef[S,W]; peer: Peer) =
|
||||||
|
mixin runStart, runStop
|
||||||
|
# Check for known entry (which should not exist.)
|
||||||
|
let
|
||||||
|
maxWorkers = dsc.ctx.buddiesMax
|
||||||
|
peers = dsc.pool.len
|
||||||
|
workers = dsc.buddies.len
|
||||||
|
if dsc.buddies.hasKey(peer.hash):
|
||||||
|
trace "Reconnecting zombie peer rejected", peer, peers, workers, maxWorkers
|
||||||
|
return
|
||||||
|
|
||||||
|
# Initialise worker for this peer
|
||||||
|
let buddy = RunnerBuddyRef[S,W](
|
||||||
|
dsc: dsc,
|
||||||
|
worker: BuddyRef[S,W](
|
||||||
|
ctx: dsc.ctx,
|
||||||
|
peer: peer))
|
||||||
|
if not buddy.worker.runStart():
|
||||||
|
trace "Ignoring useless peer", peer, peers, workers, maxWorkers
|
||||||
|
buddy.worker.ctrl.zombie = true
|
||||||
|
return
|
||||||
|
|
||||||
|
# Check for table overflow. An overflow might happen if there are zombies
|
||||||
|
# in the table (though preventing them from re-connecting for a while.)
|
||||||
|
if dsc.ctx.buddiesMax <= workers:
|
||||||
|
let leastPeer = dsc.buddies.shift.value.data
|
||||||
|
if leastPeer.worker.ctrl.zombie:
|
||||||
|
trace "Dequeuing zombie peer",
|
||||||
|
oldest=leastPeer.worker, peers, workers=dsc.buddies.len, maxWorkers
|
||||||
|
discard
|
||||||
|
else:
|
||||||
|
# This could happen if there are idle entries in the table, i.e.
|
||||||
|
# somehow hanging runners.
|
||||||
|
trace "Peer table full! Dequeuing least used entry",
|
||||||
|
oldest=leastPeer.worker, peers, workers=dsc.buddies.len, maxWorkers
|
||||||
|
leastPeer.worker.runStop()
|
||||||
|
leastPeer.worker.ctrl.zombie = true
|
||||||
|
|
||||||
|
# Add peer entry
|
||||||
|
discard dsc.buddies.lruAppend(peer.hash, buddy, dsc.ctx.buddiesMax)
|
||||||
|
|
||||||
|
trace "Running peer worker", peer, peers,
|
||||||
|
workers=dsc.buddies.len, maxWorkers
|
||||||
|
|
||||||
|
asyncSpawn buddy.workerLoop()
|
||||||
|
|
||||||
|
|
||||||
|
proc onPeerDisconnected[S,W](dsc: RunnerSyncRef[S,W], peer: Peer) =
|
||||||
|
let
|
||||||
|
peers = dsc.pool.len
|
||||||
|
maxWorkers = dsc.ctx.buddiesMax
|
||||||
|
workers = dsc.buddies.len
|
||||||
|
rc = dsc.buddies.eq(peer.hash)
|
||||||
|
if rc.isErr:
|
||||||
|
debug "Disconnected, unregistered peer", peer, peers, workers, maxWorkers
|
||||||
|
return
|
||||||
|
if rc.value.worker.ctrl.zombie:
|
||||||
|
# Don't disconnect, leave them fall out of the LRU cache. The effect is,
|
||||||
|
# that reconnecting might be blocked, for a while.
|
||||||
|
trace "Disconnected, zombie", peer, peers, workers, maxWorkers
|
||||||
|
else:
|
||||||
|
rc.value.worker.ctrl.stopped = true # in case it is hanging somewhere
|
||||||
|
dsc.buddies.del(peer.hash)
|
||||||
|
trace "Disconnected buddy", peer, peers, workers=dsc.buddies.len, maxWorkers
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# Public functions
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
proc initSync*[S,W](
|
||||||
|
dsc: RunnerSyncRef[S,W];
|
||||||
|
node: EthereumNode;
|
||||||
|
slots: int;
|
||||||
|
noisy = false) =
|
||||||
|
## Constructor
|
||||||
|
# Leave one extra slot so that it can holds a *zombie* even if all slots
|
||||||
|
# are full. The effect is that a re-connect on the latest zombie will be
|
||||||
|
# rejected as long as its worker descriptor is registered.
|
||||||
|
dsc.ctx = CtxRef[S](
|
||||||
|
buddiesMax: max(1, slots + 1),
|
||||||
|
chain: node.chain)
|
||||||
|
dsc.pool = node.peerPool
|
||||||
|
dsc.tickerOk = noisy
|
||||||
|
dsc.buddies.init(dsc.ctx.buddiesMax)
|
||||||
|
|
||||||
|
proc startSync*[S,W](dsc: RunnerSyncRef[S,W]): bool =
|
||||||
|
## Set up syncing. This call should come early.
|
||||||
|
mixin runSetup
|
||||||
|
# Initialise sub-systems
|
||||||
|
if dsc.ctx.runSetup(dsc.tickerOk):
|
||||||
|
var po = PeerObserver(
|
||||||
|
onPeerConnected:
|
||||||
|
proc(p: Peer) {.gcsafe.} =
|
||||||
|
dsc.onPeerConnected(p),
|
||||||
|
onPeerDisconnected:
|
||||||
|
proc(p: Peer) {.gcsafe.} =
|
||||||
|
dsc.onPeerDisconnected(p))
|
||||||
|
|
||||||
|
po.setProtocol eth
|
||||||
|
dsc.pool.addObserver(dsc, po)
|
||||||
|
return true
|
||||||
|
|
||||||
|
proc stopSync*[S,W](dsc: RunnerSyncRef[S,W]) =
|
||||||
|
## Stop syncing
|
||||||
|
mixin runRelease
|
||||||
|
dsc.pool.delObserver(dsc)
|
||||||
|
dsc.ctx.runRelease()
|
||||||
|
|
||||||
|
# ------------------------------------------------------------------------------
|
||||||
|
# End
|
||||||
|
# ------------------------------------------------------------------------------
|
|
@ -1,5 +1,4 @@
|
||||||
# Nimbus - Types, data structures and shared utilities used in network sync
|
# Nimbus
|
||||||
#
|
|
||||||
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
||||||
# Licensed under either of
|
# Licensed under either of
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
|
|
@ -1,5 +1,4 @@
|
||||||
# Nimbus - Types, data structures and shared utilities used in network sync
|
# Nimbus
|
||||||
#
|
|
||||||
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
# Copyright (c) 2018-2021 Status Research & Development GmbH
|
||||||
# Licensed under either of
|
# Licensed under either of
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
||||||
|
@ -17,12 +16,6 @@ import
|
||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
type
|
type
|
||||||
TxHash* = distinct Hash256
|
|
||||||
## Hash of a transaction.
|
|
||||||
##
|
|
||||||
## Note that the `ethXX` protocol driver always uses the
|
|
||||||
## underlying `Hash256` type which needs to be converted to `TxHash`.
|
|
||||||
|
|
||||||
NodeHash* = distinct Hash256
|
NodeHash* = distinct Hash256
|
||||||
## Hash of a trie node or other blob carried over `NodeData` account trie
|
## Hash of a trie node or other blob carried over `NodeData` account trie
|
||||||
## nodes, storage trie nodes, contract code.
|
## nodes, storage trie nodes, contract code.
|
||||||
|
@ -34,46 +27,41 @@ type
|
||||||
## Hash of a block, goes with `BlockNumber`.
|
## Hash of a block, goes with `BlockNumber`.
|
||||||
##
|
##
|
||||||
## Note that the `ethXX` protocol driver always uses the
|
## Note that the `ethXX` protocol driver always uses the
|
||||||
## underlying `Hash256` type which needs to be converted to `TxHash`.
|
## underlying `Hash256` type which needs to be converted to `BlockHash`.
|
||||||
|
|
||||||
TrieHash* = distinct Hash256
|
SomeDistinctHash256 =
|
||||||
## Hash of a trie root: accounts, storage, receipts or transactions.
|
NodeHash | BlockHash
|
||||||
##
|
|
||||||
## Note that the `snapXX` protocol driver always uses the underlying
|
|
||||||
## `Hash256` type which needs to be converted to `TrieHash`.
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public constructors
|
# Public constructors
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc new*(T: type TxHash): T = Hash256().T
|
proc new*(T: type SomeDistinctHash256): T =
|
||||||
proc new*(T: type NodeHash): T = Hash256().T
|
Hash256().T
|
||||||
proc new*(T: type BlockHash): T = Hash256().T
|
|
||||||
proc new*(T: type TrieHash): T = Hash256().T
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Public (probably non-trivial) type conversions
|
# Public (probably non-trivial) type conversions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc to*(num: UInt256; T: type float): T =
|
|
||||||
## Convert to float
|
|
||||||
let mantissaLen = 256 - num.leadingZeros
|
|
||||||
if mantissaLen <= 64:
|
|
||||||
num.truncate(uint64).T
|
|
||||||
else:
|
|
||||||
let exp = mantissaLen - 64
|
|
||||||
(num shr exp).truncate(uint64).T * (2.0 ^ exp)
|
|
||||||
|
|
||||||
proc to*(num: SomeInteger; T: type float): T =
|
proc to*(num: SomeInteger; T: type float): T =
|
||||||
## Convert to float
|
## Convert to float. Result an d argument are not strictly equivalent. Though
|
||||||
|
## sort of `(num.to(float) + 0.5).int == num` might hold in many cases.
|
||||||
num.T
|
num.T
|
||||||
|
|
||||||
proc to*(w: TrieHash|NodeHash|BlockHash|TxHash; T: type Hash256): T =
|
proc to*(longNum: UInt256; T: type float): T =
|
||||||
## Get rid of `distinct` harness (needed for `snap1` and `eth1` protocol
|
## Convert to float (see also comment at `num.to(float)`, above.)
|
||||||
## driver access.)
|
let mantissaLen = 256 - longNum.leadingZeros
|
||||||
|
if mantissaLen <= 64:
|
||||||
|
longNum.truncate(uint64).T
|
||||||
|
else:
|
||||||
|
let exp = mantissaLen - 64
|
||||||
|
(longNum shr exp).truncate(uint64).T * (2.0 ^ exp)
|
||||||
|
|
||||||
|
proc to*(w: SomeDistinctHash256; T: type Hash256): T =
|
||||||
|
## Syntactic sugar
|
||||||
w.Hash256
|
w.Hash256
|
||||||
|
|
||||||
proc to*(w: seq[TrieHash|NodeHash|BlockHash|TxHash]; T: type seq[Hash256]): T =
|
proc to*(w: seq[SomeDistinctHash256]; T: type seq[Hash256]): T =
|
||||||
## Ditto
|
## Ditto
|
||||||
cast[seq[Hash256]](w)
|
cast[seq[Hash256]](w)
|
||||||
|
|
||||||
|
@ -85,17 +73,22 @@ proc to*(bh: BlockHash; T: type HashOrNum): T =
|
||||||
# Public functions
|
# Public functions
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc read*(rlp: var Rlp, T: type TrieHash): T
|
proc read*(rlp: var Rlp, T: type SomeDistinctHash256): T
|
||||||
{.gcsafe, raises: [Defect,RlpError]} =
|
{.gcsafe, raises: [Defect,RlpError]} =
|
||||||
## RLP mixin reader
|
## RLP mixin reader
|
||||||
rlp.read(Hash256).T
|
rlp.read(Hash256).T
|
||||||
|
|
||||||
proc `==`*(a: NodeHash; b: TrieHash): bool = a.Hash256 == b.Hash256
|
proc append*(writer: var RlpWriter; h: SomeDistinctHash256) =
|
||||||
proc `==`*(a,b: TrieHash): bool {.borrow.}
|
## RLP mixin
|
||||||
proc `==`*(a,b: NodeHash): bool {.borrow.}
|
append(writer, h.Hash256)
|
||||||
proc `==`*(a,b: BlockHash): bool {.borrow.}
|
|
||||||
|
|
||||||
proc hash*(root: TrieHash|NodeHash|BlockHash): Hash =
|
proc `==`*(a: SomeDistinctHash256; b: Hash256): bool =
|
||||||
|
a.Hash256 == b
|
||||||
|
|
||||||
|
proc `==`*[T: SomeDistinctHash256](a,b: T): bool =
|
||||||
|
a.Hash256 == b.Hash256
|
||||||
|
|
||||||
|
proc hash*(root: SomeDistinctHash256): Hash =
|
||||||
## Mixin for `Table` or `keyedQueue`
|
## Mixin for `Table` or `keyedQueue`
|
||||||
root.Hash256.data.hash
|
root.Hash256.data.hash
|
||||||
|
|
||||||
|
@ -123,7 +116,7 @@ func toHex*(hash: Hash256): string =
|
||||||
## Shortcut for `byteutils.toHex(hash.data)`
|
## Shortcut for `byteutils.toHex(hash.data)`
|
||||||
hash.data.toHex
|
hash.data.toHex
|
||||||
|
|
||||||
func `$`*(h: TrieHash|NodeHash|BlockHash|TxHash): string =
|
func `$`*(h: SomeDistinctHash256): string =
|
||||||
$h.Hash256.data.toHex
|
$h.Hash256.data.toHex
|
||||||
|
|
||||||
func `$`*(blob: Blob): string =
|
func `$`*(blob: Blob): string =
|
||||||
|
|
|
@ -78,6 +78,9 @@ proc pp*(h: BlockHeader; indent: int): string =
|
||||||
proc pp*(g: Genesis; indent: int): string =
|
proc pp*(g: Genesis; indent: int): string =
|
||||||
g.pp("\n" & " ".repeat(max(1,indent)))
|
g.pp("\n" & " ".repeat(max(1,indent)))
|
||||||
|
|
||||||
|
proc pp*(a: Account): string =
|
||||||
|
&"({a.nonce},{a.balance},{a.storageRoot.pp},{a.codeHash.pp})"
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# End
|
# End
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
|
@ -15,26 +15,12 @@
|
||||||
|
|
||||||
import
|
import
|
||||||
std/[sequtils, strformat, strutils, tables, times],
|
std/[sequtils, strformat, strutils, tables, times],
|
||||||
nimcrypto/hash
|
nimcrypto/hash,
|
||||||
|
../../nimbus/constants
|
||||||
|
|
||||||
export
|
export
|
||||||
sequtils, strformat, strutils
|
sequtils, strformat, strutils
|
||||||
|
|
||||||
const
|
|
||||||
ZeroHash256 = MDigest[256].default
|
|
||||||
|
|
||||||
EmptyUncleHash = ( "1dcc4de8dec75d7aab85b567b6ccd41a" &
|
|
||||||
"d312451b948a7413f0a142fd40d49347" ).toDigest
|
|
||||||
|
|
||||||
BlankRootHash = ( "56e81f171bcc55a6ff8345e692c0f86e" &
|
|
||||||
"5b48e01b996cadc001622fb5e363b421" ).toDigest
|
|
||||||
|
|
||||||
EmptySha3 = ( "c5d2460186f7233c927e7db2dcc703c0" &
|
|
||||||
"e500b653ca82273b7bfad8045d85a470" ).toDigest
|
|
||||||
|
|
||||||
EmptyRlpHash = ( "56e81f171bcc55a6ff8345e692c0f86e" &
|
|
||||||
"5b48e01b996cadc001622fb5e363b421" ).toDigest
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Helpers
|
# Helpers
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
@ -103,14 +89,16 @@ proc pp*(q: openArray[int]; itemsPerLine: int; lineSep: string): string =
|
||||||
proc pp*(a: MDigest[256]; collapse = true): string =
|
proc pp*(a: MDigest[256]; collapse = true): string =
|
||||||
if not collapse:
|
if not collapse:
|
||||||
a.data.mapIt(it.toHex(2)).join.toLowerAscii
|
a.data.mapIt(it.toHex(2)).join.toLowerAscii
|
||||||
elif a == EmptyRlpHash:
|
elif a == ZERO_HASH256:
|
||||||
"emptyRlpHash"
|
"ZERO_HASH256"
|
||||||
elif a == EmptyUncleHash:
|
elif a == BLANK_ROOT_HASH:
|
||||||
"emptyUncleHash"
|
"BLANK_ROOT_HASH"
|
||||||
elif a == EmptySha3:
|
elif a == EMPTY_UNCLE_HASH:
|
||||||
"EmptySha3"
|
"EMPTY_UNCLE_HASH"
|
||||||
elif a == ZeroHash256:
|
elif a == EMPTY_SHA3:
|
||||||
"zeroHash256"
|
"EMPTY_SHA3"
|
||||||
|
elif a == ZERO_HASH256:
|
||||||
|
"ZERO_HASH256"
|
||||||
else:
|
else:
|
||||||
a.data.mapIt(it.toHex(2)).join[56 .. 63].toLowerAscii
|
a.data.mapIt(it.toHex(2)).join[56 .. 63].toLowerAscii
|
||||||
|
|
||||||
|
|
|
@ -12,18 +12,19 @@
|
||||||
## Snap sync components tester
|
## Snap sync components tester
|
||||||
|
|
||||||
import
|
import
|
||||||
std/[distros, os, random, sequtils, strformat, strutils],
|
std/[distros, os, sequtils, strformat, strutils],
|
||||||
chronicles,
|
chronicles,
|
||||||
eth/[common/eth_types, trie/db],
|
eth/[common/eth_types, p2p, rlp, trie/db],
|
||||||
stint,
|
stint,
|
||||||
stew/results,
|
stew/results,
|
||||||
unittest2,
|
unittest2,
|
||||||
../nimbus/db/select_backend,
|
../nimbus/db/select_backend,
|
||||||
../nimbus/sync/[types, protocol/snap1],
|
../nimbus/sync/[types, protocol],
|
||||||
../nimbus/sync/snap/path_desc,
|
../nimbus/sync/snap/range_desc,
|
||||||
../nimbus/sync/snap/worker/[fetch/proof_db, worker_desc],
|
../nimbus/sync/snap/worker/accounts_db,
|
||||||
./replay/pp,
|
./replay/pp,
|
||||||
./test_sync_snap/accounts_and_proofs
|
#./test_sync_snap/sample1,
|
||||||
|
./test_sync_snap/sample0
|
||||||
|
|
||||||
const
|
const
|
||||||
baseDir = [".", "..", ".."/"..", $DirSep]
|
baseDir = [".", "..", ".."/"..", $DirSep]
|
||||||
|
@ -37,7 +38,7 @@ type
|
||||||
|
|
||||||
TestItem = object ## palatable input format for tests
|
TestItem = object ## palatable input format for tests
|
||||||
base: NodeTag
|
base: NodeTag
|
||||||
data: WorkerAccountRange
|
data: SnapAccountRange
|
||||||
|
|
||||||
TestDbInstances =
|
TestDbInstances =
|
||||||
array[3,TrieDatabaseRef]
|
array[3,TrieDatabaseRef]
|
||||||
|
@ -56,7 +57,7 @@ else:
|
||||||
|
|
||||||
let
|
let
|
||||||
# Forces `check()` to print the error (as opposed when using `isOk()`)
|
# Forces `check()` to print the error (as opposed when using `isOk()`)
|
||||||
OkProof = Result[void,ProofError].ok()
|
OkAccDb = Result[void,AccountsDbError].ok()
|
||||||
|
|
||||||
# There was a problem with the Github/CI which results in spurious crashes
|
# There was a problem with the Github/CI which results in spurious crashes
|
||||||
# when leaving the `runner()` if the persistent BaseChainDB initialisation
|
# when leaving the `runner()` if the persistent BaseChainDB initialisation
|
||||||
|
@ -75,8 +76,14 @@ proc findFilePath(file: string): Result[string,void] =
|
||||||
return ok(path)
|
return ok(path)
|
||||||
err()
|
err()
|
||||||
|
|
||||||
proc pp(w: TrieHash): string =
|
proc pp(w: Hash256): string =
|
||||||
pp.pp(w.Hash256) # `pp()` also available from `worker-desc`
|
pp.pp(w) # `pp()` also available from `worker_desc`
|
||||||
|
|
||||||
|
proc pp(w: NodeTag; collapse = true): string =
|
||||||
|
pp.pp(w.to(Hash256),collapse)
|
||||||
|
|
||||||
|
proc pp(w: seq[(string,string)]; indent = 4): string =
|
||||||
|
w.mapIt(&"({it[0]},{it[1]})").join("\n" & " ".repeat(indent))
|
||||||
|
|
||||||
proc setTraceLevel =
|
proc setTraceLevel =
|
||||||
discard
|
discard
|
||||||
|
@ -97,25 +104,25 @@ proc to(data: seq[TestSample]; T: type seq[TestItem]): T =
|
||||||
for r in data:
|
for r in data:
|
||||||
result.add TestItem(
|
result.add TestItem(
|
||||||
base: r.base.to(NodeTag),
|
base: r.base.to(NodeTag),
|
||||||
data: WorkerAccountRange(
|
data: SnapAccountRange(
|
||||||
proof: r.proofs,
|
proof: r.proofs,
|
||||||
accounts: r.accounts.mapIt(
|
accounts: r.accounts.mapIt(
|
||||||
SnapAccount(
|
SnapAccount(
|
||||||
accHash: it[0].to(NodeTag),
|
accHash: it[0],
|
||||||
accBody: Account(
|
accBody: Account(
|
||||||
nonce: it[1],
|
nonce: it[1],
|
||||||
balance: it[2],
|
balance: it[2],
|
||||||
storageRoot: it[3],
|
storageRoot: it[3],
|
||||||
codeHash: it[4])))))
|
codeHash: it[4])))))
|
||||||
|
|
||||||
proc permute(r: var Rand; qLen: int): seq[int] =
|
#proc permute(r: var Rand; qLen: int): seq[int] =
|
||||||
result = (0 ..< qLen).toSeq
|
# result = (0 ..< qLen).toSeq
|
||||||
let
|
# let
|
||||||
halfLen = result.len shr 1
|
# halfLen = result.len shr 1
|
||||||
randMax = result.len - halfLen - 1
|
# randMax = result.len - halfLen - 1
|
||||||
for left in 0 ..< halfLen:
|
# for left in 0 ..< halfLen:
|
||||||
let right = halfLen + r.rand(randMax)
|
# let right = halfLen + r.rand(randMax)
|
||||||
result[left].swap(result[right])
|
# result[left].swap(result[right])
|
||||||
|
|
||||||
proc flushDbDir(s: string) =
|
proc flushDbDir(s: string) =
|
||||||
if s != "":
|
if s != "":
|
||||||
|
@ -153,10 +160,11 @@ proc lastTwo(a: openArray[string]): seq[string] =
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc accountsRunner(
|
proc accountsRunner(
|
||||||
noisy = true; persistent: bool; root: TrieHash; data: seq[TestSample]) =
|
noisy = true; persistent: bool; root: Hash256; data: seq[TestSample]) =
|
||||||
let
|
let
|
||||||
lst = data.to(seq[TestItem])
|
peer = Peer.new
|
||||||
tmpDir = "accounts_and_proofs.nim".findFilePath.value.splitFile.dir
|
testItemLst = data.to(seq[TestItem])
|
||||||
|
tmpDir = "sample0.nim".findFilePath.value.splitFile.dir
|
||||||
db = if persistent: tmpDir.testDbs() else: testDbs()
|
db = if persistent: tmpDir.testDbs() else: testDbs()
|
||||||
dbDir = db.dbDir.split($DirSep).lastTwo.join($DirSep)
|
dbDir = db.dbDir.split($DirSep).lastTwo.join($DirSep)
|
||||||
info = if db.persistent: &"persistent db on \"{dbDir}\""
|
info = if db.persistent: &"persistent db on \"{dbDir}\""
|
||||||
|
@ -168,86 +176,48 @@ proc accountsRunner(
|
||||||
|
|
||||||
suite &"SyncSnap: accounts and proofs for {info}":
|
suite &"SyncSnap: accounts and proofs for {info}":
|
||||||
var
|
var
|
||||||
desc: ProofDb
|
base: AccountsDbRef
|
||||||
nRows: seq[int]
|
desc: AccountsDbSessionRef
|
||||||
|
|
||||||
test &"Merging {lst.len} proofs for state root ..{root.pp}":
|
test &"Verifying {testItemLst.len} snap items for state root ..{root.pp}":
|
||||||
desc.init(db.inst[0])
|
base = AccountsDbRef.init(db.inst[0])
|
||||||
check desc.mergeBegin(root)
|
for n,w in testItemLst:
|
||||||
for proofs in lst.mapIt(it.data.proof):
|
check base.importAccounts(peer, root, w.base, w.data) == OkAccDb
|
||||||
check desc.merge(proofs) == OkProof
|
|
||||||
check desc.mergeValidate == OkProof
|
|
||||||
nRows.add desc.proofsLen(root)
|
|
||||||
check 1 < nRows.len # otherwise test makes no sense
|
|
||||||
check 0 < nRows[^1]
|
|
||||||
|
|
||||||
test "Rollback full database":
|
test &"Merging {testItemLst.len} proofs for state root ..{root.pp}":
|
||||||
check desc.mergeRollback()
|
base = AccountsDbRef.init(db.inst[1])
|
||||||
check desc.proofsLen(root) == 0
|
desc = AccountsDbSessionRef.init(base, root, peer)
|
||||||
check desc.accountsLen(root) == 0
|
for n,w in testItemLst:
|
||||||
check desc.journalLen == (false,0,0,0)
|
check desc.merge(w.data.proof) == OkAccDb
|
||||||
|
check desc.merge(w.base, w.data.accounts) == OkAccDb
|
||||||
|
desc.assignPrettyKeys() # for debugging (if any)
|
||||||
|
check desc.interpolate() == OkAccDb
|
||||||
|
|
||||||
test "Merging and committing all except the last":
|
# echo ">>> ", desc.dumpProofsDB.join("\n ")
|
||||||
for n,proofs in lst.mapIt(it.data.proof):
|
|
||||||
check desc.mergeBegin(root)
|
|
||||||
check desc.merge(proofs) == OkProof
|
|
||||||
check nRows[n] == desc.proofsLen(root)
|
|
||||||
check desc.mergeValidate == OkProof
|
|
||||||
if n < nRows.len - 1:
|
|
||||||
check desc.mergeCommit
|
|
||||||
check nRows[n] == desc.proofsLen(root)
|
|
||||||
check desc.mergeRollback
|
|
||||||
check 1 < nRows.len and nRows[^2] == desc.proofsLen(root)
|
|
||||||
|
|
||||||
test &"Merging/committing {lst.len} proofs, transposed rows":
|
|
||||||
desc.init(db.inst[1])
|
|
||||||
check desc.proofsLen(root) == 0
|
|
||||||
check desc.journalLen == (false,0,0,0)
|
|
||||||
var r = initRand(42)
|
|
||||||
for n,proofs in lst.mapIt(it.data.proof):
|
|
||||||
let permProof = r.permute(proofs.len).mapIt(proofs[it])
|
|
||||||
check desc.mergeBegin(root)
|
|
||||||
check desc.merge(permProof) == OkProof
|
|
||||||
check desc.mergeValidate == OkProof
|
|
||||||
check desc.mergeCommit
|
|
||||||
check nRows[n] == desc.proofsLen(root)
|
|
||||||
|
|
||||||
test &"Merging {lst.len} proved account groups"&
|
|
||||||
&" for state root ..{root.pp}":
|
|
||||||
desc.init(db.inst[2])
|
|
||||||
for n,w in lst:
|
|
||||||
check desc.mergeProved(root, w.base, w.data) == OkProof
|
|
||||||
check desc.journalLen == (false,0,0,0)
|
|
||||||
check nRows[n] == desc.proofsLen(root)
|
|
||||||
check desc.journalLen == (false,0,0,0)
|
|
||||||
check 1 < nRows.len # otherwise test makes no sense
|
|
||||||
check 0 < nRows[^1]
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# Main function(s)
|
# Main function(s)
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
|
|
||||||
proc syncSnapMain*(noisy = defined(debug)) =
|
proc syncSnapMain*(noisy = defined(debug)) =
|
||||||
noisy.accountsRunner(persistent = true, testRoot.TrieHash, testSamples)
|
noisy.accountsRunner(
|
||||||
|
persistent = false, sample0.snapRoot, sample0.snapProofData)
|
||||||
|
|
||||||
when isMainModule:
|
when isMainModule:
|
||||||
const noisy = defined(debug) or true
|
const
|
||||||
|
noisy = defined(debug) or true
|
||||||
when true: # false:
|
test00 = (sample0.snapRoot, @[sample0.snapProofData0])
|
||||||
# Import additional data from test data repo
|
test01 = (sample0.snapRoot, sample0.snapProofData)
|
||||||
import ../../nimbus-eth1-blobs/replay/accounts_and_proofs_ex
|
#test10 = (sample1.snapRoot, @[sample1.snapProofData1])
|
||||||
else:
|
#test11 = (sample1.snapRoot, sample1.snapProofData)
|
||||||
const
|
|
||||||
testRootEx = testRoot
|
|
||||||
testSamplesEx = newSeq[TestSample]()
|
|
||||||
|
|
||||||
setTraceLevel()
|
setTraceLevel()
|
||||||
|
setErrorLevel()
|
||||||
|
|
||||||
# Verify sample state roots
|
noisy.accountsRunner(persistent=false, test00[0], test00[1])
|
||||||
doAssert testRoot == testRootEx
|
noisy.accountsRunner(persistent=false, test01[0], test01[1])
|
||||||
|
#noisy.accountsRunner(persistent=false, test10[0], test10[1])
|
||||||
let samplesList = (testSamples & testSamplesEx)
|
#noisy.accountsRunner(persistent=false, test11[0], test11[1])
|
||||||
noisy.accountsRunner(persistent = true, testRoot.TrieHash, samplesList)
|
|
||||||
|
|
||||||
# ------------------------------------------------------------------------------
|
# ------------------------------------------------------------------------------
|
||||||
# End
|
# End
|
||||||
|
|
|
@ -1,134 +0,0 @@
|
||||||
# Licensed under either of
|
|
||||||
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
|
|
||||||
# http://www.apache.org/licenses/LICENSE-2.0)
|
|
||||||
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
|
|
||||||
# http://opensource.org/licenses/MIT)
|
|
||||||
# at your option. This file may not be copied, modified, or
|
|
||||||
# distributed except according to those terms.
|
|
||||||
|
|
||||||
## Collected snap/1 accounts and proofs when fetching accounts
|
|
||||||
|
|
||||||
import
|
|
||||||
std/[sequtils],
|
|
||||||
eth/common/eth_types,
|
|
||||||
nimcrypto/hash,
|
|
||||||
stew/byteutils,
|
|
||||||
../../nimbus/constants
|
|
||||||
|
|
||||||
const
|
|
||||||
root =
|
|
||||||
"b538f1067958013728e013b52c3e37eaecf86ddc83fe5f7b4a045e50deb08810".toDigest
|
|
||||||
|
|
||||||
rec0 = (
|
|
||||||
ZERO_HASH256,
|
|
||||||
@[
|
|
||||||
# <0>
|
|
||||||
("00000013653234c2d78dcdc645c5141e358ef2e590fe5278778ba729ff5ffd95".toDigest,
|
|
||||||
1u64,
|
|
||||||
"7931794000000000".parse(Uint256),
|
|
||||||
"56E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421".toDigest,
|
|
||||||
"C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".toDigest),
|
|
||||||
|
|
||||||
# <1>
|
|
||||||
("0000008c38d769d75c1ad1de6660da51edc10394c11c50ff9a0ca9e8b8b35dc2".toDigest,
|
|
||||||
9u64,
|
|
||||||
"143314596029971".parse(Uint256),
|
|
||||||
"56E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421".toDigest,
|
|
||||||
"C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".toDigest),
|
|
||||||
|
|
||||||
# <43695>
|
|
||||||
("001048467d0933750604fb19cf5dd096f02f60279cc0d9cf03f9b3424a7fb95f".toDigest,
|
|
||||||
4u64,
|
|
||||||
0.u256,
|
|
||||||
"56E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421".toDigest,
|
|
||||||
"C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".toDigest)],
|
|
||||||
@[
|
|
||||||
# <0>
|
|
||||||
"F90211A0FFBB364A8CB0D565F70271627D91A255FB50D751D52A3348B61C9D7B4E98AB28A0FB1ED3251BBD153BFE1415F9946ABFF90C207678BC09EB006C2538D5EE181944A04EC56E4DC562B7C1C3DC3B0D917CE07B4975D9D4459B3F433EAF5D466DA5FF05A0F5283E423E1112E8E50A68D03E339F914F0D18883C4128571B1D14A64F2C9F2DA09414298F9C3AC243DD13F0412CFE069020D4268767E37ADC529D8923312E6519A07978D8ADDBF59DF2D472783308BB735D8BF9FC951FC694E4249B268F0B67CA67A0A17D1D539EEF8747147C2B77281AC355FF16FA42D6941489AB3A11B46D06DD2EA0D7D9CD27EDEEA84EDD53B201DEC05DDB8F6ADD8CDDC77628FFDE9CABBE4F6C1DA03C45D84EEFF0128C6D19BE1D8CAF2797C7332D5E12040B87E1F4E7E17D9D4977A0A8B7AA500844BCA70F76E20BB231291A54CBC71039D183DA4B1FB058FC79FC69A087682429DABD006289751A0EA2C05AA1FD277DA89BF8C7E26DBAEBC774F246A8A0DA0F3AAB84168AF6C0101C0994B881B4FC9EDC4E99E7F28BA90C26F65EE0C819A0A3D9721D23C8B8118B50FAAA40FB3424B8C2BA1B91A2EAC0AAD29868B74B8497A0D1C18AA65CCA65A7118E2C42C429BADE088FC61987B86575145B5A84CA5379A3A0AD509B03FDE185D3ED8CC884A4D0AC03390E7DB8FEC78EC3127DB28CEB670353A0403A13695F15EAAA0588C74282DFF5A9C05BD9039F44336F10BA5590E087043780".hexToSeqByte,
|
|
||||||
# <1>
|
|
||||||
"F90211A050B2D95C218D12F40BE4549EE50709E479B19157BA095501AA94293F662DCA7FA00FB68AA15AD8AD8E2773DC61A370AFE82DAB79EDFDEE1A076F9C3C39B90A30B2A0F7E3E89841383EA0264230C6E5F8BB7A9383E31B13D4333F7417AC389C47C368A0ADF864ED54A756828CA1E8B3C0D434C495CAE24FA3547B10D4B037628BEBD1F4A0ADEEBF028C5D866AC5E6F0D91234E599B2606D89FCFD90F3CF332B46A03AB057A065433307CF0FF284D6A3A7E6D0D434BCD3D732757CCCFA1020B21E5F4762BA5EA0FEC203B690FB1AB74055EF240EA0F9A0E5F27AE2FFED6DA9D2B64883AB456EFEA03C4C09C4F72C15A1CE661C60EB7725403337B46D979FEE259AA3BCC6B0AD6161A05E9BE5120BDF2E94D6E64D1DE4CBAFB7C12E830498228A7D12F6CE2461E93990A066B7F2AD805E04708D28CA06AEE29F88700EB51AB38F4BC109DD9F4ABAAC041BA02C2413B49872A483B1F6B97A741B4B8C59040F4F4759AE444643330CD9907F29A08651CEF990EF0A0C5AB52F4FA4AD4265ACB8F7D2358ABE9AC87BC926F04C1B24A0D29829A570067E446DA03CDFFA7203766F8365C65FBF0E23BF3233BB96D6C658A00F68202899DB482FAFF2AAB2635EDB72E43EBD170D4F0554BAF828E970C4DBC7A06D3D6F17ED39FBB36A6065AC34BE5C633F1B8928B5514DEFFD5A0FFA501AF202A0BE7035655FB3801E87902731C9F0041D8CAFBE471B48F0212293ACCD8C40CACC80".hexToSeqByte,
|
|
||||||
# <2>
|
|
||||||
"F90211A000F354080583902374DBAD850D0E8E33803412F22C7BA81CBC2778A3B3350761A0809A2CF3A2F87798CE2B4047BB288F17C000307BC7C57FA251CD2C7292596AECA04B40B0EF7E160F09D3DA0EA477C385A49074D35C937A2B74B3F51ABD8A5F9BCAA0F972583DC80407F31517059FCC80E3D31262D8637BB819E94F2D2CD0736A324CA033539BA750904CED59B6B37F3879FDB62AAA850DCF7994DA64DA41B3A9E78850A0B4F5AA77948FC84800F8A9365531A166B56D3D1C5BBC3367554D0C1DC7766811A0FF28D18F41F88909A7CDF60CE8F5B56171B6FFFC97CF299730AC08436AD562B1A0F83235BB3F0C276386895A503BEF8B61F7A8D65D9F6ED4A92C4FD0616E05DE1EA05DC6F966B54000C0B830DF4BB61E47D4D4357822FE4474823DF4862F92602E2AA067D7F3201504A6BC2CF96A010A856CABC4489BEE2F325AB40C6E3ED230898D68A082FCBFA9FCB388E1EC0CC70F14072B3011CACADC212FFB11DCA1A0387064427FA03F4EB0BC6BB0AF1B9AC31A64FB77C6B75F41301DEFBB3803A660E526D7A8D577A01813C0B7A37EBAA16B78E63E23D6E1EF58B9646C572723FCBAF706EFB0958C77A00E112F5A43F599A7858758D3783525C8BC57CFA1BC9D973045363A6091721A28A0879D607834EC77D3D62A5C30BE61615BFB9DAA671DABCC7294C7C3A8633DB6AFA05876CE1DD19DB3F2FCDE622F6C1AF61898481DD6C72BD9273106835A49C5248480".hexToSeqByte,
|
|
||||||
# <3>
|
|
||||||
"F90211A062A4606CBB57887CC2C4B9B686DF69A45791F8518E9B62FB6E86130B4B1C6D13A082126F32BE01C3EF18630C686074A2A393B323B8EC3C964705781631358B8E57A08A47D9820D50F48362B4EC12BCBCD759AC42B2F703401B40AA7699964ABA7B40A0C39C5E09856C11DCC6B3739D8846C5406F8FD462EB79095C419F233749A167C8A009D8A1308EBB7522740233D015BA1910584A7A0360BCFAA3C997FFDA7DB648FBA08728CFDBED8454545FAB8D7A587E24CBCA0AA9AF04F883F954399B1859EF91C1A082EE3DB9657745B7147DB490C653358D0E162C4C28F35D7416F7D720EBA69F48A0E301E9D346E2666B8E97D8A3719388B5FCF68B672D0ECEDC6ABACC3F6F906224A03FF691E4BCEB8DD2B604483F1A116FF8EAB64325F5F10AD426B431EDAE7C5ECEA0F92D8622AFA94F9C1A3C147A491897E39C522060C7FA283E21CD5FE23DA2A911A05995EFA81B02A46AD062F21F5C27E01CC16338AACD382DC796FF48D2B06B8A54A024EFE4D36BF0E3DD5587EB85B960051F2AD4F7E4522C770E5A76186F55A0CBF5A0E90C35D8AD9DEEEC60713B687F2830A9A20C705E46442A4EAFF7D867359F9009A0C2F59F608A467ABB02907A45B7446B5F375C660C256E63E20749C98FFD157299A02F08BF5CE0278F3A28A89D18BD44B2712792C362BF755BC545BC3C35E6895BB2A0BB6EDC6F46D9C76AE0CCDEBA9F1C18CA7F0EE94A561350BDAC731364007A525480".hexToSeqByte,
|
|
||||||
# <4>
|
|
||||||
"F90211A05A745895FC96B94DB4D0A16D034CF33F38059F2357ED7CB2EA35DB00DD92EF65A03FB18F877D09F9E2021D5EE638E2F2919E24CAEA933ED4AC51818323F7DDC29EA004E520D01462FC4C79A9A74FEE948EC2F36E708575E7AD4CD9C47F6B0B87F519A09CCEB99ADBC31003D903A288D6EE7DF583A4947FB4ADF0F22886E84A86D4D7E5A070D1C2E44D89E9A140F51F5D1C1E9906EF24A3E70A90614B4A8ACB1AEAB35A5CA001831185E3DBBAA2AEB1C18ED3E0F77B41F2E98E09490F399D8D2FAAB92CB3C3A067F57C912E5826406002CAC84732BF505702CA103ACB63A1024ED53F8AAC86C7A05D0D61D6026C41DFCF47FE3B19F277FC9FEBD94E4C2FF0AA12F15C4035B651B4A05CC597DD4F21C32D0EA8831C9CB585310C5B22CA8FAFEA15A4B3F93C2BAAF394A084807A504C68F016A6DBAB22370B77FAB6AD339DD8C6BFBE54BFD137C808D0CDA0ED42536EE6357BB3AA7DDC2A06FBB4E1D8DE2152A657183979151A8D04EFCA2FA078A48BF0F38B4F0972FBD2D876DD685F4FE8CCEFF168F1C6201C973ACEF1C1C8A0DBFAFB4F768292C4B23EB8F6F81CD29D956702E78390F2C4417F0C4E8F6C2A17A0E9B44679701E0B0F2EF944853AEAFB8CF9FFAC1CE6A52159AF74845E46F47125A0E41FC974110569D17E12190B596DE2C0E3C8B3BB451DC4C91154A0C2645D797AA01199389984707B7EC7157340E1D8B1174F5F47CE465FF4F8449F22E28EA56D0A80".hexToSeqByte,
|
|
||||||
# <5>
|
|
||||||
"F90211A050C20A151922301F0D1998EE0141A22B7C919BD0D716794EE7E3A5E0EC48DEC8A0AB31DFBEF2AC74B4E501DCE89315A6A89B10F20CBA64F761993A1037418613A7A0BF0D6EE592B2CAA6F302B27999438103809FAF702A2B30E5E1403965EF87B35EA0135D8AFE5EB5D20A1927A91F535BA7684490658EF272C933115BF0BF060CF5E6A0A1EE2F87381EA364E75D651631B5B00E62B1E7E2008458ACF29E0831D7760AFDA040AC269BEA082940F45ED7D813C47E21B930C88EF2B28BF187AE83789EF94BC5A02A03448BD5B58065042A47DB4310F364915C5D4E5FBDF32D292D6FB6BDD0708CA0E0C204B11B2EECD0A4986EEFD3413AD49DDDE082FEE6FAD50AD5C310EB7F22CDA0DA59FED5F92CC206DC2EA7BAD2EA92CC2E9C9FFE5F3A816BBF4EE2E585CE3DCAA073B2EB114312AAB7D2A765DC7923977FB60AF0447ECC7B381F193F68F658A1B7A031045FF0797D0309A4073C19A00172A2737B1D345269149A4AA4A82E6D202B4EA060BEC4F79BB8D0BCAF2B2E362B75845651A2FCC8876A3511D7843CA4E6D18DFDA01D4D7E3E579AA0FBADD67C3B9F09931DB5D733852B36F6201F9D7CF0961E1C17A0AAE7C5916D9FC01F0E7B27543F3991C0B48302A739F83030A951DA296BFCE7B8A0E3519D65CC3E19949E0D25424D5B3BFBD9DF02C84744FD39360471D197570803A0FD9C31C068D34B2701C0CDD3346012D84BB43E982BE5451E98CE50F9DB2D99DE80".hexToSeqByte,
|
|
||||||
# <6>
|
|
||||||
"F8B180A00642D4EC5F43A113E2E0C470A87AB859178187341C0BFA059AABBE2384BDFB62808080808080A0E5652B61A18FE7AC7F7C71DE1AE97B8CA1CF742CA89BE4BBE191A2F492C606C5808080A0B203CB3B8CF8825505939914BCBF2B7B7606768E503682093E9B08B99DB3B38AA014C13E3F5D55A90B2C7C654B7C1C34FC65D2268EEC0EB6296B3A8F6199154F4EA0A0CE8026B7C397771620B3B3620E2B0528157DB901C02EDB86BA87DF8DC23E268080".hexToSeqByte,
|
|
||||||
# <7>
|
|
||||||
"F90211A0175A0B1AEB1FFC698C61412794315A73B4C360EFA558142223EB50FDFB7C03E1A04AA803B66B8E66D570D504405C088CAFDE3F1568046360FD8AA0A365A4120214A0E6377FAD331E7EDD34F932BFFBDD7B0A3B828BBB7D2D6C73133B6919D9A49E20A0E7C4D8894D251DBDCE60AB039BF2B6B877FC032465CEEA46775BBD436630823CA0B5637ED98AF879C025DF5C7C0969A45BDD4061A891742DA7A5A95EF148A41623A05E301F8CA8969599E168E4C36276D7EA0CE70F88206CE550CBD211C5F011ED88A079C3DE527358AA2F1052BFDDBBCA68434044644E75EDD214D24281D6A0C58752A0086F191941F619D0B54136FD6E2582AB891066C8EB42113065E7E2ADF84FD5C1A01D1F5BE41598FF2FCAF1BA2A5C081120D1D70105DF11FA96729CBED928BBA2DEA07E7F5873A68A712E0D4952A9AE5216169488D12EB318FE4705396C58F1F0C88EA07585C154BFFE63F1600CD996C3551BB63E9ABF02470D9B92287A7746D6F87D30A090DE8B996508F04459B3FC3F75C65FC7F66CD7F0CB6E48D58B9853EC7DBD1F58A0F8D482EE79E5E29B6741861FE044346F6E5EA80BFD8A1378CCC73824A59EBB3EA0FDAD4E6FC2F866901830B8EB6FCD19ABC1AE8028BDC85C5A20D78D002F1F117CA0424A916B37993B3A8EAA67ABC370994D0F8931E2AD450F18FF820BCB5EBC88E3A032FE085815FE7CCA707217A0308A266BF6F63DEDEC466A8F89F7AE3F66876E7080".hexToSeqByte,
|
|
||||||
# <8>
|
|
||||||
"F90211A06C58C81EA693CAC268BD2F0733BEB9A2784C75AA20C39B47644BB8A5E2189B27A05945F9ECE092095BD10E7814A81A14DBEDB25342BEABFAA588BFCDAF6444BCA6A007CAC5ABE793C070CE26F2A21CD0F29A573252D2C88051BC3CD58BECEA261EBEA068D6CE58650D53BBFE68284E09749907136BD52EE69332329DC0B86256987290A0BD948923CEB4704B381153432836A6534DC48C5161558B5B4B2747E9618922E9A075244FB6B65AEAC7B016CB659B04F4F144C7A9093175BBEBD844090BF6548240A0932890EF1AE3A79F55162780E5C6B32B7DEE1DA34F3FC2EBEEDDD1E96FCD433FA0E264A2922852C7C489F91DAA2FCFF0C5285A7DA1DD7891A3D9408D74F63B394BA0BF60414930AC9613A6CEF84FEDD860D0878DF5862626F52DDC7D250CDC79D2CEA026EB321595E04C1C43C33A937E4429F9F731CDC1A737FCFD12AACCF445F26748A0FAD955C9809991E80713C556A6AE30D425F36C62BA701DB863DB601341AB9664A0D48C5E648623C5FEF68B95A7761F7CC59B82FFF0231857158CBAB82C0839B846A0F33215586D176E00AA996ACE3C47E9C7517FF4B2D8CFA3AE69A57C5D767AE8C5A02FC3250268C96860E1D52E2391C43BF1EE881987F96730A750E61C6CD91E6870A02E95A4BF0044815926DF4C81B09BE500DCCBBF98CFC9E624BF2E248EF625E2D3A0F346E1C14D8B033A03E6B8BFD8318B0DBACCA7B138B6AE9A72D84E712A52603380".hexToSeqByte,
|
|
||||||
# <9>
|
|
||||||
"F90211A0296F209978A24A9984C5E45D10D6B245947D83FA007DC2121C01A39577138663A055ACACB026401BA36C369FD68C335D30A0CCE097B25AD38819097CFE5D704867A031FF3A3298562295E6B44A13F79028A7DF49AB62CDBBC75B6B671B9265254A76A0BCA39714982A3DB01CF2211E5D13863968929DD3858000AA1970A577F14A5E8BA0A3E891D719D60B724951222664EAD2643E8B8A944BAF4EBAACAE702C8E1AEF42A0924AC4E2FC0B05E457C07660FBB4FC693FBE4ACA9F1290460A59057C742EB734A027C4975E9683463489E1BF7311A77689D566FFB7A9A0D9EBC944CFCE4265F2FBA0F7D781A27E6D5ABC32A251BAE37FCC0D87D6A021B3B24B22BF4E2EB184A9C397A0530A2DAD21BDF103E4A70448739D7B4905C5B23D6AC3505DF879960CEF80FAD6A0569BED1BACE3BF2BAF56C6A6EEE8E2D5722C5C3CB8116ECA93CB2F069BB1B2B2A06A040503F573725DAB1A804D4382492CF0E62AFAAC297AEF46D14D231AD07A24A05E08296620CBF5F661D98EC10AF23B81D3C42650657782E291D2EDE6FD0671C9A0A19F098F8D390CCF95A404C19B808C1E73BD637719458E43E3AA1AE667772657A007A61D4524CE417FD7E75A60C87E49D2ABE2B2F84DEB6195DC291E9227CF65E9A07EA8968C14011CD7F7ABE224736C000213124C9A0819B31E689CB8B534EC889CA004802E2FC25D0C7D828D66701727396F54AA6622453214DDA47F89ACA1616FDD80".hexToSeqByte,
|
|
||||||
# <10>
|
|
||||||
"F901318080A0A9B034F6DF142723288C01ABC1D11C5836D0584FCEB53D7F61E0216881474224A0B2B840F666627E35696A5E8B7764FD8C232A2AA09B5C30D6C72FD5AB75143DC8A0EAA66386D22922CFDDC505D0E4FB3A5B7F4C0F589C7200E58AE336284FBB6C02A0579C4844BABBC6F1343B8A2F88A497463533D0B8FA6351CF95D8A61B1457997680A0297155BBC073059DC7C59FB9DB094034DF55D6B93EC5FDEECE12806D1AA8EBC080A0C2DF13436F789DB4517236E714733A08D55239E7C72D9E12D657E39AB47553A5A0BF9B5757D3F5BEEF22AF958A4DED7F5D0CAD563DC3DC7C9BD22F7B61B3885BB0A076CE7BBA69CAFCFE416BC453A064610A9946330563312DD6C57923A1EE46FCDC8080A0866DEA9CB87A66E9C92F86A84526298955FE32F35B17B78DB21A28DF55B67E128080".hexToSeqByte,
|
|
||||||
# <11>
|
|
||||||
"F8518080A00AB67F3EB5F163D372289AF571D28A0CFE1EA9E5E8C1B12BED10B1F04ADD8163808080A0A43155ADC4D4F7C6F82A3AE1B2F5B0A91C13F8C8B91D2E4833BDCA163196CA2880808080808080808080".hexToSeqByte,
|
|
||||||
# <12>
|
|
||||||
"F8669D207D0933750604FB19CF5DD096F02F60279CC0D9CF03F9B3424A7FB95FB846F8440480A056E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421A0C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".hexToSeqByte])
|
|
||||||
|
|
||||||
rec1 = (
|
|
||||||
"001048467d0933750604fb19cf5dd096f02f60279cc0d9cf03f9b3424a7fb960".toDigest,
|
|
||||||
@[
|
|
||||||
# <0>
|
|
||||||
("00104852da00c6b5afbceb650f30322fc6e4406b508796d325ff4d3ef3a904e5".toDigest,
|
|
||||||
1u64,
|
|
||||||
4974.u256,
|
|
||||||
"56E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421".toDigest,
|
|
||||||
"C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".toDigest),
|
|
||||||
|
|
||||||
# <1>
|
|
||||||
("00104859ecda2e64d1a459062849603d4ea641c749f0c3bbbf9e9f5faf9c16ba".toDigest,
|
|
||||||
13u64,
|
|
||||||
0.u256,
|
|
||||||
"56E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421".toDigest,
|
|
||||||
"C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".toDigest),
|
|
||||||
|
|
||||||
# <43845>
|
|
||||||
("0020c98b155f8a165cdab1ec9865e8a96c2acd182a7f590593d48c9ef88b5d29".toDigest,
|
|
||||||
1u64,
|
|
||||||
"1549611000000000".parse(Uint256),
|
|
||||||
"56E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421".toDigest,
|
|
||||||
"C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".toDigest)],
|
|
||||||
@[
|
|
||||||
# <0>
|
|
||||||
"F90211A0FFBB364A8CB0D565F70271627D91A255FB50D751D52A3348B61C9D7B4E98AB28A0FB1ED3251BBD153BFE1415F9946ABFF90C207678BC09EB006C2538D5EE181944A04EC56E4DC562B7C1C3DC3B0D917CE07B4975D9D4459B3F433EAF5D466DA5FF05A0F5283E423E1112E8E50A68D03E339F914F0D18883C4128571B1D14A64F2C9F2DA09414298F9C3AC243DD13F0412CFE069020D4268767E37ADC529D8923312E6519A07978D8ADDBF59DF2D472783308BB735D8BF9FC951FC694E4249B268F0B67CA67A0A17D1D539EEF8747147C2B77281AC355FF16FA42D6941489AB3A11B46D06DD2EA0D7D9CD27EDEEA84EDD53B201DEC05DDB8F6ADD8CDDC77628FFDE9CABBE4F6C1DA03C45D84EEFF0128C6D19BE1D8CAF2797C7332D5E12040B87E1F4E7E17D9D4977A0A8B7AA500844BCA70F76E20BB231291A54CBC71039D183DA4B1FB058FC79FC69A087682429DABD006289751A0EA2C05AA1FD277DA89BF8C7E26DBAEBC774F246A8A0DA0F3AAB84168AF6C0101C0994B881B4FC9EDC4E99E7F28BA90C26F65EE0C819A0A3D9721D23C8B8118B50FAAA40FB3424B8C2BA1B91A2EAC0AAD29868B74B8497A0D1C18AA65CCA65A7118E2C42C429BADE088FC61987B86575145B5A84CA5379A3A0AD509B03FDE185D3ED8CC884A4D0AC03390E7DB8FEC78EC3127DB28CEB670353A0403A13695F15EAAA0588C74282DFF5A9C05BD9039F44336F10BA5590E087043780".hexToSeqByte,
|
|
||||||
# <1>
|
|
||||||
"F90211A050B2D95C218D12F40BE4549EE50709E479B19157BA095501AA94293F662DCA7FA00FB68AA15AD8AD8E2773DC61A370AFE82DAB79EDFDEE1A076F9C3C39B90A30B2A0F7E3E89841383EA0264230C6E5F8BB7A9383E31B13D4333F7417AC389C47C368A0ADF864ED54A756828CA1E8B3C0D434C495CAE24FA3547B10D4B037628BEBD1F4A0ADEEBF028C5D866AC5E6F0D91234E599B2606D89FCFD90F3CF332B46A03AB057A065433307CF0FF284D6A3A7E6D0D434BCD3D732757CCCFA1020B21E5F4762BA5EA0FEC203B690FB1AB74055EF240EA0F9A0E5F27AE2FFED6DA9D2B64883AB456EFEA03C4C09C4F72C15A1CE661C60EB7725403337B46D979FEE259AA3BCC6B0AD6161A05E9BE5120BDF2E94D6E64D1DE4CBAFB7C12E830498228A7D12F6CE2461E93990A066B7F2AD805E04708D28CA06AEE29F88700EB51AB38F4BC109DD9F4ABAAC041BA02C2413B49872A483B1F6B97A741B4B8C59040F4F4759AE444643330CD9907F29A08651CEF990EF0A0C5AB52F4FA4AD4265ACB8F7D2358ABE9AC87BC926F04C1B24A0D29829A570067E446DA03CDFFA7203766F8365C65FBF0E23BF3233BB96D6C658A00F68202899DB482FAFF2AAB2635EDB72E43EBD170D4F0554BAF828E970C4DBC7A06D3D6F17ED39FBB36A6065AC34BE5C633F1B8928B5514DEFFD5A0FFA501AF202A0BE7035655FB3801E87902731C9F0041D8CAFBE471B48F0212293ACCD8C40CACC80".hexToSeqByte,
|
|
||||||
# <2>
|
|
||||||
"F90211A000F354080583902374DBAD850D0E8E33803412F22C7BA81CBC2778A3B3350761A0809A2CF3A2F87798CE2B4047BB288F17C000307BC7C57FA251CD2C7292596AECA04B40B0EF7E160F09D3DA0EA477C385A49074D35C937A2B74B3F51ABD8A5F9BCAA0F972583DC80407F31517059FCC80E3D31262D8637BB819E94F2D2CD0736A324CA033539BA750904CED59B6B37F3879FDB62AAA850DCF7994DA64DA41B3A9E78850A0B4F5AA77948FC84800F8A9365531A166B56D3D1C5BBC3367554D0C1DC7766811A0FF28D18F41F88909A7CDF60CE8F5B56171B6FFFC97CF299730AC08436AD562B1A0F83235BB3F0C276386895A503BEF8B61F7A8D65D9F6ED4A92C4FD0616E05DE1EA05DC6F966B54000C0B830DF4BB61E47D4D4357822FE4474823DF4862F92602E2AA067D7F3201504A6BC2CF96A010A856CABC4489BEE2F325AB40C6E3ED230898D68A082FCBFA9FCB388E1EC0CC70F14072B3011CACADC212FFB11DCA1A0387064427FA03F4EB0BC6BB0AF1B9AC31A64FB77C6B75F41301DEFBB3803A660E526D7A8D577A01813C0B7A37EBAA16B78E63E23D6E1EF58B9646C572723FCBAF706EFB0958C77A00E112F5A43F599A7858758D3783525C8BC57CFA1BC9D973045363A6091721A28A0879D607834EC77D3D62A5C30BE61615BFB9DAA671DABCC7294C7C3A8633DB6AFA05876CE1DD19DB3F2FCDE622F6C1AF61898481DD6C72BD9273106835A49C5248480".hexToSeqByte,
|
|
||||||
# <3>
|
|
||||||
"F90211A0175A0B1AEB1FFC698C61412794315A73B4C360EFA558142223EB50FDFB7C03E1A04AA803B66B8E66D570D504405C088CAFDE3F1568046360FD8AA0A365A4120214A0E6377FAD331E7EDD34F932BFFBDD7B0A3B828BBB7D2D6C73133B6919D9A49E20A0E7C4D8894D251DBDCE60AB039BF2B6B877FC032465CEEA46775BBD436630823CA0B5637ED98AF879C025DF5C7C0969A45BDD4061A891742DA7A5A95EF148A41623A05E301F8CA8969599E168E4C36276D7EA0CE70F88206CE550CBD211C5F011ED88A079C3DE527358AA2F1052BFDDBBCA68434044644E75EDD214D24281D6A0C58752A0086F191941F619D0B54136FD6E2582AB891066C8EB42113065E7E2ADF84FD5C1A01D1F5BE41598FF2FCAF1BA2A5C081120D1D70105DF11FA96729CBED928BBA2DEA07E7F5873A68A712E0D4952A9AE5216169488D12EB318FE4705396C58F1F0C88EA07585C154BFFE63F1600CD996C3551BB63E9ABF02470D9B92287A7746D6F87D30A090DE8B996508F04459B3FC3F75C65FC7F66CD7F0CB6E48D58B9853EC7DBD1F58A0F8D482EE79E5E29B6741861FE044346F6E5EA80BFD8A1378CCC73824A59EBB3EA0FDAD4E6FC2F866901830B8EB6FCD19ABC1AE8028BDC85C5A20D78D002F1F117CA0424A916B37993B3A8EAA67ABC370994D0F8931E2AD450F18FF820BCB5EBC88E3A032FE085815FE7CCA707217A0308A266BF6F63DEDEC466A8F89F7AE3F66876E7080".hexToSeqByte,
|
|
||||||
# <4>
|
|
||||||
"F90211A06C58C81EA693CAC268BD2F0733BEB9A2784C75AA20C39B47644BB8A5E2189B27A05945F9ECE092095BD10E7814A81A14DBEDB25342BEABFAA588BFCDAF6444BCA6A007CAC5ABE793C070CE26F2A21CD0F29A573252D2C88051BC3CD58BECEA261EBEA068D6CE58650D53BBFE68284E09749907136BD52EE69332329DC0B86256987290A0BD948923CEB4704B381153432836A6534DC48C5161558B5B4B2747E9618922E9A075244FB6B65AEAC7B016CB659B04F4F144C7A9093175BBEBD844090BF6548240A0932890EF1AE3A79F55162780E5C6B32B7DEE1DA34F3FC2EBEEDDD1E96FCD433FA0E264A2922852C7C489F91DAA2FCFF0C5285A7DA1DD7891A3D9408D74F63B394BA0BF60414930AC9613A6CEF84FEDD860D0878DF5862626F52DDC7D250CDC79D2CEA026EB321595E04C1C43C33A937E4429F9F731CDC1A737FCFD12AACCF445F26748A0FAD955C9809991E80713C556A6AE30D425F36C62BA701DB863DB601341AB9664A0D48C5E648623C5FEF68B95A7761F7CC59B82FFF0231857158CBAB82C0839B846A0F33215586D176E00AA996ACE3C47E9C7517FF4B2D8CFA3AE69A57C5D767AE8C5A02FC3250268C96860E1D52E2391C43BF1EE881987F96730A750E61C6CD91E6870A02E95A4BF0044815926DF4C81B09BE500DCCBBF98CFC9E624BF2E248EF625E2D3A0F346E1C14D8B033A03E6B8BFD8318B0DBACCA7B138B6AE9A72D84E712A52603380".hexToSeqByte,
|
|
||||||
# <5>
|
|
||||||
"F90211A0296F209978A24A9984C5E45D10D6B245947D83FA007DC2121C01A39577138663A055ACACB026401BA36C369FD68C335D30A0CCE097B25AD38819097CFE5D704867A031FF3A3298562295E6B44A13F79028A7DF49AB62CDBBC75B6B671B9265254A76A0BCA39714982A3DB01CF2211E5D13863968929DD3858000AA1970A577F14A5E8BA0A3E891D719D60B724951222664EAD2643E8B8A944BAF4EBAACAE702C8E1AEF42A0924AC4E2FC0B05E457C07660FBB4FC693FBE4ACA9F1290460A59057C742EB734A027C4975E9683463489E1BF7311A77689D566FFB7A9A0D9EBC944CFCE4265F2FBA0F7D781A27E6D5ABC32A251BAE37FCC0D87D6A021B3B24B22BF4E2EB184A9C397A0530A2DAD21BDF103E4A70448739D7B4905C5B23D6AC3505DF879960CEF80FAD6A0569BED1BACE3BF2BAF56C6A6EEE8E2D5722C5C3CB8116ECA93CB2F069BB1B2B2A06A040503F573725DAB1A804D4382492CF0E62AFAAC297AEF46D14D231AD07A24A05E08296620CBF5F661D98EC10AF23B81D3C42650657782E291D2EDE6FD0671C9A0A19F098F8D390CCF95A404C19B808C1E73BD637719458E43E3AA1AE667772657A007A61D4524CE417FD7E75A60C87E49D2ABE2B2F84DEB6195DC291E9227CF65E9A07EA8968C14011CD7F7ABE224736C000213124C9A0819B31E689CB8B534EC889CA004802E2FC25D0C7D828D66701727396F54AA6622453214DDA47F89ACA1616FDD80".hexToSeqByte,
|
|
||||||
# <6>
|
|
||||||
"F901318080A0A9B034F6DF142723288C01ABC1D11C5836D0584FCEB53D7F61E0216881474224A0B2B840F666627E35696A5E8B7764FD8C232A2AA09B5C30D6C72FD5AB75143DC8A0EAA66386D22922CFDDC505D0E4FB3A5B7F4C0F589C7200E58AE336284FBB6C02A0579C4844BABBC6F1343B8A2F88A497463533D0B8FA6351CF95D8A61B1457997680A0297155BBC073059DC7C59FB9DB094034DF55D6B93EC5FDEECE12806D1AA8EBC080A0C2DF13436F789DB4517236E714733A08D55239E7C72D9E12D657E39AB47553A5A0BF9B5757D3F5BEEF22AF958A4DED7F5D0CAD563DC3DC7C9BD22F7B61B3885BB0A076CE7BBA69CAFCFE416BC453A064610A9946330563312DD6C57923A1EE46FCDC8080A0866DEA9CB87A66E9C92F86A84526298955FE32F35B17B78DB21A28DF55B67E128080".hexToSeqByte,
|
|
||||||
# <7>
|
|
||||||
"F8518080A00AB67F3EB5F163D372289AF571D28A0CFE1EA9E5E8C1B12BED10B1F04ADD8163808080A0A43155ADC4D4F7C6F82A3AE1B2F5B0A91C13F8C8B91D2E4833BDCA163196CA2880808080808080808080".hexToSeqByte,
|
|
||||||
# <8>
|
|
||||||
"F8669D207D0933750604FB19CF5DD096F02F60279CC0D9CF03F9B3424A7FB95FB846F8440480A056E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421A0C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".hexToSeqByte,
|
|
||||||
# <9>
|
|
||||||
"F90211A0D8DF8DD801321FCCFF6BC008D582698662A15ED4B29431A1CD1CB540169476ACA03B76B2DB6EE17A52E226B2301A7DD2FF21A957909FCD26D81DBA762DF65B678EA015C7B6DB1A993A54976301D2FB6AC31CEB33ED48018E322C8487F2DB3DC264EDA044B53BF21F700419E1A6F5340880311240A356596B095E8D8944D1240513F4DBA0BD2245AC2DD694A1823AC008FC02F2ADB917F5D3B5AF0A02CD8257C25BC4A9E1A033BA8C1554270E029F0FF6D43E4C903E179F5FBE2189BABC4744B7ADAD96178AA037CA87F94F696DE439973B89FE6DCCB39F64C0D95FEE05CC72DADF7C1F4063E0A054B143D6C582F858EDF2301467D3A0A6E11F6BFBA2B93C3F49C5AB9E418AEEAAA0C82405BC0E336A9C8BD8B171E4569086EF20CCA24FA032161D7E38522F7A6E3AA0187182D43F5A0E84692C755ECE4C5763CABFF4B88F8D960D60DF39B4E3ED80C6A0BD00857DCEB2AEFF0B3E6D94204C466B62A2F945DC5BA1C629B27B0149106E18A0865BCFFB1D3C36173CE347878A7E4FBC2DAB1DF7BF9BA80B9F53E06FE00D5461A0973E5E630F4E4FC9E65F6EB23A9B7C516201C9DAB309B29CA33B379B07BA9D29A0CC49BF76F5D3A790F3B7EC2392AA88B30BFF500BF03A17E4AC0D9093DE27A313A0B3ED1849C31A2B81446D87F590707E1A352D3B5431A8281F593FD153B9B75E33A028872B9C072274D6AAD1D3FAAFDD163D393ADB405C7435815E9367B06645941A80".hexToSeqByte,
|
|
||||||
# <10>
|
|
||||||
"F90211A0D50F2E39F45A2111D55147674DD3DFE9EF778723B49B17BBB5423338CF899966A0E30B43737727FF3F099B43478F3EB3F6CB58171A4D4914A49E18E4250BE7E17CA0EB0ADC11962D3A32FE947CBC5BD542D2E33BE338869CF2253E6EB4D5ABFFF308A00877B208516D96B6C8943261281568E787B0C4ED71C25F08DABE28B3CFA23311A07CE4931FB0A59EA544536ADD1C9731BF9A6806690CA5DE401304AABC57379E56A0C10B68793F9F0AEF92E0BC9511ADD7F32E64AE71325BE9FBC9A04ABE817C73F1A0E09B62F3EDDB32F66360E3CF098A11FAA5E6BA74FCBA58017B8AB1FBE322DC75A0A3D7CB9D94C9ABDCBE75EDDF9119EF6BA96EA469D4232EC16849C9F0D6A4D920A004CAC15D7CCEBBA9587B95F3656CEDAAD7F1180C63B47A7DCE5CEE3EB1F87655A0AEC2B3F123707530EDC5BB11DEF09AE196F3391DA1F7CD4555B3FB3200843B92A01B87103A5557E37231639416C5D76FD76E3D763C222198D6C554C64CF679F982A0D627D002CC4EE0C2F51E4304B350C16080FEBB3B1BB7267A42D245422C75826FA08F4307EB616ABD9A0FEDA83E3E12B41BDAF6F05215A6A92EE11CF68F830DF944A065D0E5A8EE025D5EFEBB60F767203221D4B7B243743283EA7EB745074098A2A1A04E4B7B9F232A245C7F7D1FD4AA5495DD8A6E27420366E4515A7C1A275764ADC3A0D53C6B29CB990E25619E4A1E4E92A1F35102985E6EC999D040896AFBD7D56AC480".hexToSeqByte,
|
|
||||||
# <11>
|
|
||||||
"F90211A04A89D57B9CB42F6E269D51AC8DDA52101DEF5163BFB415E0C7D5D838544905DEA0B9EA602CEB85FEB7F9763B870AD24845DCC4C47DD12D7CB091C9B640CED04166A0B323570ABA3DD34B2AD3042BD7A7E7A10F36BA3007A6C9F81182526960FCB297A0C15BB5B84B2E0E0145566DC26C33992B27D90DDAE3CAE6AD71E4CC6AB8FEC13DA066D2E06E2F9F5E3FCCF972BA9508A4E29316CEB1AD3E643DE4582073E90C9653A0DEA30438B2AB11706FD4EB35A0BE6B4C2E1A1FEB4D6F3C9D2105123FE325AF3AA0A76442E86C30687060E41A70112ED2D4B7F00AFD537D9562BA50F3A6870667B2A037E3C627AB7C7B4EE4BC86754B9C950D4BB992AA5F271CDCB9FDB280CFF74E4FA00673239BAF1BDB51BBC1E504B844C9275E39F998BE78153A6674B002844459EBA0D8C03E5D9B3B5295ADE0C34520CDE8BA36D3D4DDB49EC5B3C1C1A04E1C8213C9A00C7364BE1AB503A7A9017021E915D2AAB82FFA3B58E4FA4EF8A36D1BBAEF035AA0846D08C50C7978D5234C5D77565D4D3386D9FA6BBC0F20F58D726EE4CACA8C73A0C052ED2FF80CD00B598AD906101D2B539F1DA5745E2E39C9D6E51B6AB419A6E4A07817251C528F0D7297E4BB855EB66B767EE817B4D697BDAD59107734C6A13352A0B69712AA6A7D9BDDB4462F71EB5688583B277A13453F5CD37947B4561A7D5F23A0C7EFCD12218C436D3055B9DB0121964412A981BDC224ACDB6D8382B3B433DC0980".hexToSeqByte,
|
|
||||||
# <12>
|
|
||||||
"F9017180A00AB394B3794B7909B05FA99943D33B6C650A3EDA8278104EE6A815E63B23F294A00BEC2410E52705BCE45A0E7D536BC1BC720CCEAE09A82F259528EB52249BC7A580A0490DFF989D2CA627D0FFFE5A7624E5D1F93D13D148C8E895A533E637D72DF692A06963AF64D5582FF69C1AED0338830CFD29FEB6CA241D27CF2B003DF3872226AD80A051A0B35BC60C36717801FA70B97EF70F83DC87272C1BB1D8A018422A6881EFEBA0B240366AAF91CBE93688DF2012629A6E322BA7893BC155894912F1298727FC0280A0248DEDB0AE495BBDCC633729DB6755CF75154B487848EE1A2AF41B16651AB159A0F91E534C8C8B4C99673618DF0C8439134C5BE214CA88B31A9AF856F329280512A0740EF53248D750747B8A867C90B824C2D8D0B989BF5FBD9B0F85B1CC7EC8A5D780A02666267A3FF068897BDFE0B8CD363070ADC2A47F2B75245C5D95E2CE5C0BA665A04176569E211DE429567E85394E2851FC1858DB8AEADD2A9FDD76EA580BB62F2F80".hexToSeqByte,
|
|
||||||
# <13>
|
|
||||||
"F871808080808080808080A0E27FB04D7FEE1421851667D4958BF753B72531863F37128B1524F79036DA3DBBA0FC79B7B936154EFB48ED672C1C45F7ADE8D90C37C6876CCF0A8E67DAFB42CF57A0362D97C46FED60D536848D8A8F02A0B897606DA3841A83B68A206486B80F508D8080808080".hexToSeqByte,
|
|
||||||
# <14>
|
|
||||||
"F86D9D20155F8A165CDAB1EC9865E8A96C2ACD182A7F590593D48C9EF88B5D29B84DF84B018705815CED31EE00A056E81F171BCC55A6FF8345E692C0F86E5B48E01B996CADC001622FB5E363B421A0C5D2460186F7233C927E7DB2DCC703C0E500B653CA82273B7BFAD8045D85A470".hexToSeqByte])
|
|
||||||
|
|
||||||
# ----------
|
|
||||||
|
|
||||||
const
|
|
||||||
testRoot* = root
|
|
||||||
testSamples* = @[rec0, rec1]
|
|
||||||
|
|
||||||
# End
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1 +1 @@
|
||||||
Subproject commit 598246620da5c41d0e92a8dd6aab0755381b21cd
|
Subproject commit 49db5b27b9933165cf53287e7302b9d2a37a8d26
|
Loading…
Reference in New Issue