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:
Jordan Hrycaj 2022-08-04 09:04:30 +01:00 committed by GitHub
parent 71f9e37482
commit 5f0e89a41e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 6096 additions and 3401 deletions

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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,

View File

@ -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

View File

@ -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

View File

@ -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.

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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)
# ------------------------------------------------------------------------------ # ------------------------------------------------------------------------------

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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:

282
nimbus/sync/sync_sched.nim Normal file
View File

@ -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
# ------------------------------------------------------------------------------

View File

@ -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

View File

@ -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 =

View File

@ -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
# ------------------------------------------------------------------------------ # ------------------------------------------------------------------------------

View File

@ -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

View File

@ -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

View File

@ -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

2
vendor/nim-stew vendored

@ -1 +1 @@
Subproject commit 598246620da5c41d0e92a8dd6aab0755381b21cd Subproject commit 49db5b27b9933165cf53287e7302b9d2a37a8d26