Add first iteration of a fluffy beacon bridge for block data (#1437)
This commit is contained in:
parent
fda7971aaf
commit
3a079bf30e
|
@ -1,8 +1,10 @@
|
||||||
# Seeding data into the Portal history network
|
# Seeding data into the Portal history network
|
||||||
|
|
||||||
## Building and seeding epoch accumulators into the Portal history network
|
## Seeding from locally stored history data
|
||||||
|
|
||||||
### Step 1: Building the epoch accumulators
|
### Building and seeding epoch accumulators into the Portal history network
|
||||||
|
|
||||||
|
#### Step 1: Building the epoch accumulators
|
||||||
1. Set-up access to an Ethereum JSON-RPC endpoint (e.g. local geth instance)
|
1. Set-up access to an Ethereum JSON-RPC endpoint (e.g. local geth instance)
|
||||||
that can serve the data.
|
that can serve the data.
|
||||||
|
|
||||||
|
@ -27,7 +29,7 @@ Ethereum JSON-RPC endpoint.
|
||||||
./build/eth_data_exporter exportAccumulatorData --writeEpochAccumulators --data-dir:"./user_data_dir/"
|
./build/eth_data_exporter exportAccumulatorData --writeEpochAccumulators --data-dir:"./user_data_dir/"
|
||||||
```
|
```
|
||||||
|
|
||||||
### Step 2: Seed the epoch accumulators into the Portal network
|
#### Step 2: Seed the epoch accumulators into the Portal network
|
||||||
Run Fluffy and trigger the propagation of data with the
|
Run Fluffy and trigger the propagation of data with the
|
||||||
`portal_history_propagateEpochAccumulators` JSON-RPC API call:
|
`portal_history_propagateEpochAccumulators` JSON-RPC API call:
|
||||||
|
|
||||||
|
@ -39,7 +41,7 @@ curl -s -X POST -H 'Content-Type: application/json' -d '{"jsonrpc":"2.0","id":"1
|
||||||
```
|
```
|
||||||
|
|
||||||
|
|
||||||
### Step 3 (Optional): Verify that all epoch accumulators are available
|
#### Step 3 (Optional): Verify that all epoch accumulators are available
|
||||||
Run Fluffy and run the `content_verifier` tool to verify that all epoch
|
Run Fluffy and run the `content_verifier` tool to verify that all epoch
|
||||||
accumulators are available on the history network:
|
accumulators are available on the history network:
|
||||||
|
|
||||||
|
@ -53,7 +55,7 @@ Run the `content_verifier` tool and see if all epoch accumulators are found:
|
||||||
./build/content_verifier
|
./build/content_verifier
|
||||||
```
|
```
|
||||||
|
|
||||||
## Seeding block data into the Portal network
|
### Downloading & seeding block data into the Portal network
|
||||||
|
|
||||||
1. Set-up access to an Ethereum JSON-RPC endpoint (e.g. local geth instance)
|
1. Set-up access to an Ethereum JSON-RPC endpoint (e.g. local geth instance)
|
||||||
that can serve the data.
|
that can serve the data.
|
||||||
|
@ -82,3 +84,33 @@ Ethereum JSON-RPC endpoint.
|
||||||
# From another shell
|
# From another shell
|
||||||
curl -s -X POST -H 'Content-Type: application/json' -d '{"jsonrpc":"2.0","id":"1","method":"portal_history_propagate","params":["./user_data_dir/eth-history-data.json"]}' http://localhost:8545 | jq
|
curl -s -X POST -H 'Content-Type: application/json' -d '{"jsonrpc":"2.0","id":"1","method":"portal_history_propagate","params":["./user_data_dir/eth-history-data.json"]}' http://localhost:8545 | jq
|
||||||
```
|
```
|
||||||
|
|
||||||
|
## Seeding from content bridges
|
||||||
|
|
||||||
|
### Seeding post-merge block headers and bodies through the beacon chain bridge
|
||||||
|
|
||||||
|
Run a Fluffy node with the JSON-RPC API enabled.
|
||||||
|
|
||||||
|
```bash
|
||||||
|
./build/fluffy --network:testnet0 --rpc --table-ip-limit:1024 --bucket-ip-limit:24
|
||||||
|
```
|
||||||
|
|
||||||
|
Build & run the `beacon_chain_bridge`:
|
||||||
|
```bash
|
||||||
|
make fluffy-tools
|
||||||
|
|
||||||
|
TRUSTED_BLOCK_ROOT=0x1234567890123456789012345678901234567890123456789012345678901234 # Replace this
|
||||||
|
./build/beacon_chain_bridge --trusted-block-root=${TRUSTED_BLOCK_ROOT}
|
||||||
|
```
|
||||||
|
|
||||||
|
The `beacon_chain_bridge` will start with the consensus light client sync follow
|
||||||
|
beacon block gossip. Once it is synced, the execution payload of new beacon
|
||||||
|
blocks will be extracted and injected in the Portal network as execution headers
|
||||||
|
and blocks.
|
||||||
|
|
||||||
|
> Note: The execution headers will come without a proof for now.
|
||||||
|
|
||||||
|
The injection into the Portal network is done via the
|
||||||
|
`portal_historyGossip` JSON-RPC endpoint of the running Fluffy node.
|
||||||
|
|
||||||
|
> Note: Backfilling of block bodies and headers is not yet supported.
|
||||||
|
|
|
@ -161,3 +161,6 @@ type
|
||||||
|
|
||||||
func init*(T: type BlockHeaderProof, proof: AccumulatorProof): T =
|
func init*(T: type BlockHeaderProof, proof: AccumulatorProof): T =
|
||||||
BlockHeaderProof(proofType: accumulatorProof, accumulatorProof: proof)
|
BlockHeaderProof(proofType: accumulatorProof, accumulatorProof: proof)
|
||||||
|
|
||||||
|
func init*(T: type BlockHeaderProof): T =
|
||||||
|
BlockHeaderProof(proofType: none)
|
||||||
|
|
|
@ -0,0 +1,391 @@
|
||||||
|
# Nimbus
|
||||||
|
# Copyright (c) 2023 Status Research & Development GmbH
|
||||||
|
# Licensed and distributed under either of
|
||||||
|
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
|
||||||
|
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
|
||||||
|
# at your option. This file may not be copied, modified, or distributed except according to those terms.
|
||||||
|
|
||||||
|
#
|
||||||
|
# This beacon_chain_bridge allows for following the head of the beacon chain and
|
||||||
|
# seeding the latest execution block headers and bodies into the Portal network.
|
||||||
|
#
|
||||||
|
# The bridge does consensus light client sync and follows beacon block gossip.
|
||||||
|
# Once it is synced, the execution payload of new beacon blocks will be
|
||||||
|
# extracted and injected in the Portal network as execution headers and blocks.
|
||||||
|
#
|
||||||
|
# The injection into the Portal network is done via the `portal_historyGossip`
|
||||||
|
# JSON-RPC endpoint of a running Fluffy node.
|
||||||
|
#
|
||||||
|
# Other, currently not implemented, options to seed data:
|
||||||
|
# - Backfill post-merge block headers & bodies block into the network. Could
|
||||||
|
# walk down the parent blocks and seed them. Could also verify if the data is
|
||||||
|
# already available on the network before seeding it, potentially jumping in
|
||||||
|
# steps > 1.
|
||||||
|
# - For backfill of pre-merge headers and blocks, access to epoch accumulators
|
||||||
|
# is needed to be able to build the proofs. These could be retrieved from the
|
||||||
|
# network, but would require usage of the `portal_historyRecursiveFindContent`
|
||||||
|
# JSON-RPC endpoint. Additionally, the actualy block headers and bodies need
|
||||||
|
# to be requested from an execution JSON-RPC endpoint.
|
||||||
|
# Data would flow from:
|
||||||
|
# (block data) execution client -> bridge
|
||||||
|
# (epoch accumulator) fluffy -> bridge
|
||||||
|
# (portal content) bridge -> fluffy
|
||||||
|
# This seems awfully cumbersome. Other options sound better, see comment down.
|
||||||
|
# - Also receipts need to be requested from an execution JSON-RPC endpoint, but
|
||||||
|
# they can be verified because of consensus light client sync.
|
||||||
|
# Of course, if you are using a trusted execution endpoint for that, you can
|
||||||
|
# get the block headers and bodies also through that channel.
|
||||||
|
#
|
||||||
|
# Data seeding of Epoch accumulators is unlikely to be supported by this bridge.
|
||||||
|
# It is currently done by first downloading and storing all headers into files
|
||||||
|
# per epoch. Then the accumulator and epoch accumulators can be build from this
|
||||||
|
# data.
|
||||||
|
# The reason for this approach is because downloading all the headers from an
|
||||||
|
# execution endpoint takes long (you actually request the full blocks). An
|
||||||
|
# intermediate local storage step is preferred because of this. The accumulator
|
||||||
|
# build itself can be done in minutes when the data is locally available. These
|
||||||
|
# locally stored accumulators can then be seeded directly from a Fluffy node via
|
||||||
|
# a (currently) non standardized JSON-RPC endpoint.
|
||||||
|
#
|
||||||
|
# Data seeding of the block headers, bodies and receipts can be done the same
|
||||||
|
# way. Downloading and storing them first locally in files. Then seeding them
|
||||||
|
# into the network.
|
||||||
|
# For the headers, the proof needs to be build and added from the right
|
||||||
|
# epoch accumulator, so access to the epoch accumulator is a requirement
|
||||||
|
# (offline or from the network).
|
||||||
|
# This functionality is currently directly part of Fluffy and triggered via
|
||||||
|
# non standardized JSON-RPC calls
|
||||||
|
# Alternatively, this could also be moved to a seperate tool which gossips the
|
||||||
|
# data with a portal_historyGossip JSON-RPC call, but the building of the header
|
||||||
|
# proofs would be slighty more cumbersome.
|
||||||
|
#
|
||||||
|
|
||||||
|
when (NimMajor, NimMinor) < (1, 4):
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
else:
|
||||||
|
{.push raises: [].}
|
||||||
|
|
||||||
|
import
|
||||||
|
std/[os, strutils, options],
|
||||||
|
web3/ethtypes,
|
||||||
|
chronicles, chronicles/chronos_tools, chronos,
|
||||||
|
eth/[keys, rlp], eth/[trie, trie/db],
|
||||||
|
# Need to rename this because of web3 ethtypes and ambigious indentifier mess
|
||||||
|
# for `BlockHeader`.
|
||||||
|
eth/common/eth_types as etypes,
|
||||||
|
eth/common/eth_types_rlp,
|
||||||
|
beacon_chain/eth1/eth1_monitor,
|
||||||
|
beacon_chain/gossip_processing/optimistic_processor,
|
||||||
|
beacon_chain/networking/topic_params,
|
||||||
|
beacon_chain/spec/beaconstate,
|
||||||
|
beacon_chain/spec/datatypes/[phase0, altair, bellatrix],
|
||||||
|
beacon_chain/[light_client, nimbus_binary_common, version],
|
||||||
|
# Weirdness. Need to import this to be able to do errors.ValidationResult as
|
||||||
|
# else we get an ambiguous identifier, ValidationResult from eth & libp2p.
|
||||||
|
libp2p/protocols/pubsub/errors,
|
||||||
|
../../rpc/portal_rpc_client,
|
||||||
|
../../network/history/history_content,
|
||||||
|
../../common/common_types,
|
||||||
|
./beacon_chain_bridge_conf
|
||||||
|
|
||||||
|
from beacon_chain/gossip_processing/block_processor import newExecutionPayload
|
||||||
|
from beacon_chain/gossip_processing/eth2_processor import toValidationResult
|
||||||
|
|
||||||
|
template asEthHash(hash: ethtypes.BlockHash): Hash256 =
|
||||||
|
Hash256(data: distinctBase(hash))
|
||||||
|
|
||||||
|
# TODO: Ugh why isn't gasLimit and gasUsed a uint64 in nim-eth / nimbus-eth1 :(
|
||||||
|
template unsafeQuantityToInt64(q: Quantity): int64 =
|
||||||
|
int64 q
|
||||||
|
|
||||||
|
proc asPortalBlockData*(
|
||||||
|
payload: ExecutionPayloadV1 | ExecutionPayloadV2):
|
||||||
|
(common_types.BlockHash, BlockHeaderWithProof, BlockBodySSZ) =
|
||||||
|
proc calculateTransactionData(
|
||||||
|
items: openArray[TypedTransaction]):
|
||||||
|
Hash256 {.raises: [Defect].} =
|
||||||
|
|
||||||
|
var tr = initHexaryTrie(newMemoryDB())
|
||||||
|
for i, t in items:
|
||||||
|
try:
|
||||||
|
let tx = distinctBase(t)
|
||||||
|
tr.put(rlp.encode(i), tx)
|
||||||
|
except RlpError as e:
|
||||||
|
# TODO: Investigate this RlpError as it doesn't sound like this is
|
||||||
|
# something that can actually occur.
|
||||||
|
raiseAssert(e.msg)
|
||||||
|
|
||||||
|
return tr.rootHash()
|
||||||
|
|
||||||
|
let
|
||||||
|
txRoot = calculateTransactionData(payload.transactions)
|
||||||
|
|
||||||
|
header = etypes.BlockHeader(
|
||||||
|
parentHash: payload.parentHash.asEthHash,
|
||||||
|
ommersHash: EMPTY_UNCLE_HASH,
|
||||||
|
coinbase: EthAddress payload.feeRecipient,
|
||||||
|
stateRoot: payload.stateRoot.asEthHash,
|
||||||
|
txRoot: txRoot,
|
||||||
|
receiptRoot: payload.receiptsRoot.asEthHash,
|
||||||
|
bloom: distinctBase(payload.logsBloom),
|
||||||
|
difficulty: default(DifficultyInt),
|
||||||
|
blockNumber: payload.blockNumber.distinctBase.u256,
|
||||||
|
gasLimit: payload.gasLimit.unsafeQuantityToInt64,
|
||||||
|
gasUsed: payload.gasUsed.unsafeQuantityToInt64,
|
||||||
|
timestamp: fromUnix payload.timestamp.unsafeQuantityToInt64,
|
||||||
|
extraData: bytes payload.extraData,
|
||||||
|
mixDigest: payload.prevRandao.asEthHash,
|
||||||
|
nonce: default(BlockNonce),
|
||||||
|
fee: some(payload.baseFeePerGas),
|
||||||
|
withdrawalsRoot: options.none(Hash256), # TODO: Update later
|
||||||
|
excessDataGas: options.none(UInt256) # TODO: Update later
|
||||||
|
)
|
||||||
|
|
||||||
|
headerWithProof = BlockHeaderWithProof(
|
||||||
|
header: ByteList(rlp.encode(header)),
|
||||||
|
proof: BlockHeaderProof.init())
|
||||||
|
|
||||||
|
var transactions: Transactions
|
||||||
|
for tx in payload.transactions:
|
||||||
|
discard transactions.add(TransactionByteList(distinctBase(tx)))
|
||||||
|
|
||||||
|
let body = BlockBodySSZ(
|
||||||
|
transactions: transactions,
|
||||||
|
uncles: Uncles(@[byte 0xc0]))
|
||||||
|
|
||||||
|
let hash = common_types.BlockHash(data: distinctBase(payload.blockHash))
|
||||||
|
|
||||||
|
(hash, headerWithProof, body)
|
||||||
|
|
||||||
|
# TODO Find what can throw exception
|
||||||
|
proc run() {.raises: [Exception, Defect].} =
|
||||||
|
{.pop.}
|
||||||
|
var config = makeBannerAndConfig(
|
||||||
|
"Nimbus beacon chain bridge", BeaconBridgeConf)
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
|
# Required as both Eth2Node and LightClient requires correct config type
|
||||||
|
var lcConfig = config.asLightClientConf()
|
||||||
|
|
||||||
|
setupLogging(config.logLevel, config.logStdout, none(OutFile))
|
||||||
|
|
||||||
|
notice "Launching Nimbus beacon chain bridge",
|
||||||
|
cmdParams = commandLineParams(), config
|
||||||
|
|
||||||
|
let metadata = loadEth2Network(config.eth2Network)
|
||||||
|
|
||||||
|
for node in metadata.bootstrapNodes:
|
||||||
|
lcConfig.bootstrapNodes.add node
|
||||||
|
|
||||||
|
template cfg(): auto = metadata.cfg
|
||||||
|
|
||||||
|
let
|
||||||
|
genesisState =
|
||||||
|
try:
|
||||||
|
template genesisData(): auto = metadata.genesisData
|
||||||
|
newClone(readSszForkedHashedBeaconState(
|
||||||
|
cfg, genesisData.toOpenArrayByte(genesisData.low, genesisData.high)))
|
||||||
|
except CatchableError as err:
|
||||||
|
raiseAssert "Invalid baked-in state: " & err.msg
|
||||||
|
|
||||||
|
beaconClock = BeaconClock.init(getStateField(genesisState[], genesis_time))
|
||||||
|
|
||||||
|
getBeaconTime = beaconClock.getBeaconTimeFn()
|
||||||
|
|
||||||
|
genesis_validators_root =
|
||||||
|
getStateField(genesisState[], genesis_validators_root)
|
||||||
|
forkDigests = newClone ForkDigests.init(cfg, genesis_validators_root)
|
||||||
|
|
||||||
|
genesisBlockRoot = get_initial_beacon_block(genesisState[]).root
|
||||||
|
|
||||||
|
rng = keys.newRng()
|
||||||
|
|
||||||
|
netKeys = getRandomNetKeys(rng[])
|
||||||
|
|
||||||
|
network = createEth2Node(
|
||||||
|
rng, lcConfig, netKeys, cfg,
|
||||||
|
forkDigests, getBeaconTime, genesis_validators_root
|
||||||
|
)
|
||||||
|
|
||||||
|
rpcHttpclient = newRpcHttpClient()
|
||||||
|
|
||||||
|
optimisticHandler = proc(signedBlock: ForkedMsgTrustedSignedBeaconBlock):
|
||||||
|
Future[void] {.async.} =
|
||||||
|
# TODO: Should not be gossiping optimistic blocks, but instead store them
|
||||||
|
# in a cache and only gossip them after they are confirmed due to an LC
|
||||||
|
# finalized header.
|
||||||
|
notice "New LC optimistic block",
|
||||||
|
opt = signedBlock.toBlockId(),
|
||||||
|
wallSlot = getBeaconTime().slotOrZero
|
||||||
|
|
||||||
|
withBlck(signedBlock):
|
||||||
|
when stateFork >= BeaconStateFork.Bellatrix:
|
||||||
|
if blck.message.is_execution_block:
|
||||||
|
template payload(): auto = blck.message.body.execution_payload
|
||||||
|
|
||||||
|
# TODO: Get rid of the asEngineExecutionPayload step
|
||||||
|
let (hash, headerWithProof, body) =
|
||||||
|
asPortalBlockData(payload.asEngineExecutionPayload())
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
blockhash = history_content.`$`hash
|
||||||
|
|
||||||
|
block: # gossip header
|
||||||
|
let contentKey = ContentKey.init(blockHeaderWithProof, hash)
|
||||||
|
let encodedContentKey = contentKey.encode.asSeq()
|
||||||
|
|
||||||
|
try:
|
||||||
|
let peers = await rpcHttpclient.portal_historyGossip(
|
||||||
|
encodedContentKey.toHex(),
|
||||||
|
SSZ.encode(headerWithProof).toHex())
|
||||||
|
info "Block header gossiped", peers,
|
||||||
|
contentKey = encodedContentKey.toHex()
|
||||||
|
except CatchableError as e:
|
||||||
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
|
await rpcHttpclient.close()
|
||||||
|
|
||||||
|
# For bodies to get verified, the header needs to be available on
|
||||||
|
# the network. Wait a little to get the headers propagated through
|
||||||
|
# the network.
|
||||||
|
await sleepAsync(1.seconds)
|
||||||
|
|
||||||
|
block: # gossip block
|
||||||
|
let contentKey = ContentKey.init(blockBody, hash)
|
||||||
|
let encodedContentKey = contentKey.encode.asSeq()
|
||||||
|
|
||||||
|
try:
|
||||||
|
let peers = await rpcHttpclient.portal_historyGossip(
|
||||||
|
encodedContentKey.toHex(),
|
||||||
|
SSZ.encode(body).toHex())
|
||||||
|
info "Block body gossiped", peers,
|
||||||
|
contentKey = encodedContentKey.toHex()
|
||||||
|
except CatchableError as e:
|
||||||
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
|
await rpcHttpclient.close()
|
||||||
|
return
|
||||||
|
|
||||||
|
optimisticProcessor = initOptimisticProcessor(
|
||||||
|
getBeaconTime, optimisticHandler)
|
||||||
|
|
||||||
|
lightClient = createLightClient(
|
||||||
|
network, rng, lcConfig, cfg, forkDigests, getBeaconTime,
|
||||||
|
genesis_validators_root, LightClientFinalizationMode.Optimistic)
|
||||||
|
|
||||||
|
waitFor rpcHttpclient.connect(config.rpcAddress, Port(config.rpcPort), false)
|
||||||
|
|
||||||
|
info "Listening to incoming network requests"
|
||||||
|
network.initBeaconSync(cfg, forkDigests, genesisBlockRoot, getBeaconTime)
|
||||||
|
network.addValidator(
|
||||||
|
getBeaconBlocksTopic(forkDigests.phase0),
|
||||||
|
proc (signedBlock: phase0.SignedBeaconBlock): errors.ValidationResult =
|
||||||
|
toValidationResult(
|
||||||
|
optimisticProcessor.processSignedBeaconBlock(signedBlock)))
|
||||||
|
network.addValidator(
|
||||||
|
getBeaconBlocksTopic(forkDigests.altair),
|
||||||
|
proc (signedBlock: altair.SignedBeaconBlock): errors.ValidationResult =
|
||||||
|
toValidationResult(
|
||||||
|
optimisticProcessor.processSignedBeaconBlock(signedBlock)))
|
||||||
|
network.addValidator(
|
||||||
|
getBeaconBlocksTopic(forkDigests.bellatrix),
|
||||||
|
proc (signedBlock: bellatrix.SignedBeaconBlock): errors.ValidationResult =
|
||||||
|
toValidationResult(
|
||||||
|
optimisticProcessor.processSignedBeaconBlock(signedBlock)))
|
||||||
|
lightClient.installMessageValidators()
|
||||||
|
|
||||||
|
waitFor network.startListening()
|
||||||
|
waitFor network.start()
|
||||||
|
|
||||||
|
proc onFinalizedHeader(
|
||||||
|
lightClient: LightClient, finalizedHeader: BeaconBlockHeader) =
|
||||||
|
info "New LC finalized header",
|
||||||
|
finalized_header = shortLog(finalizedHeader)
|
||||||
|
|
||||||
|
proc onOptimisticHeader(
|
||||||
|
lightClient: LightClient, optimisticHeader: BeaconBlockHeader) =
|
||||||
|
info "New LC optimistic header",
|
||||||
|
optimistic_header = shortLog(optimisticHeader)
|
||||||
|
optimisticProcessor.setOptimisticHeader(optimisticHeader)
|
||||||
|
|
||||||
|
lightClient.onFinalizedHeader = onFinalizedHeader
|
||||||
|
lightClient.onOptimisticHeader = onOptimisticHeader
|
||||||
|
lightClient.trustedBlockRoot = some config.trustedBlockRoot
|
||||||
|
|
||||||
|
func shouldSyncOptimistically(wallSlot: Slot): bool =
|
||||||
|
# Check whether light client is used
|
||||||
|
let optimisticHeader = lightClient.optimisticHeader.valueOr:
|
||||||
|
return false
|
||||||
|
|
||||||
|
# Check whether light client has synced sufficiently close to wall slot
|
||||||
|
const maxAge = 2 * SLOTS_PER_EPOCH
|
||||||
|
if optimisticHeader.slot < max(wallSlot, maxAge.Slot) - maxAge:
|
||||||
|
return false
|
||||||
|
|
||||||
|
true
|
||||||
|
|
||||||
|
var blocksGossipState: GossipState = {}
|
||||||
|
proc updateBlocksGossipStatus(slot: Slot) =
|
||||||
|
let
|
||||||
|
isBehind = not shouldSyncOptimistically(slot)
|
||||||
|
|
||||||
|
targetGossipState = getTargetGossipState(
|
||||||
|
slot.epoch, cfg.ALTAIR_FORK_EPOCH, cfg.BELLATRIX_FORK_EPOCH,
|
||||||
|
cfg.CAPELLA_FORK_EPOCH, cfg.EIP4844_FORK_EPOCH, isBehind)
|
||||||
|
|
||||||
|
template currentGossipState(): auto = blocksGossipState
|
||||||
|
if currentGossipState == targetGossipState:
|
||||||
|
return
|
||||||
|
|
||||||
|
if currentGossipState.card == 0 and targetGossipState.card > 0:
|
||||||
|
debug "Enabling blocks topic subscriptions",
|
||||||
|
wallSlot = slot, targetGossipState
|
||||||
|
elif currentGossipState.card > 0 and targetGossipState.card == 0:
|
||||||
|
debug "Disabling blocks topic subscriptions",
|
||||||
|
wallSlot = slot
|
||||||
|
else:
|
||||||
|
# Individual forks added / removed
|
||||||
|
discard
|
||||||
|
|
||||||
|
let
|
||||||
|
newGossipForks = targetGossipState - currentGossipState
|
||||||
|
oldGossipForks = currentGossipState - targetGossipState
|
||||||
|
|
||||||
|
for gossipFork in oldGossipForks:
|
||||||
|
let forkDigest = forkDigests[].atStateFork(gossipFork)
|
||||||
|
network.unsubscribe(getBeaconBlocksTopic(forkDigest))
|
||||||
|
|
||||||
|
for gossipFork in newGossipForks:
|
||||||
|
let forkDigest = forkDigests[].atStateFork(gossipFork)
|
||||||
|
network.subscribe(
|
||||||
|
getBeaconBlocksTopic(forkDigest), blocksTopicParams,
|
||||||
|
enableTopicMetrics = true)
|
||||||
|
|
||||||
|
blocksGossipState = targetGossipState
|
||||||
|
|
||||||
|
proc onSecond(time: Moment) =
|
||||||
|
let wallSlot = getBeaconTime().slotOrZero()
|
||||||
|
updateBlocksGossipStatus(wallSlot + 1)
|
||||||
|
lightClient.updateGossipStatus(wallSlot + 1)
|
||||||
|
|
||||||
|
proc runOnSecondLoop() {.async.} =
|
||||||
|
let sleepTime = chronos.seconds(1)
|
||||||
|
while true:
|
||||||
|
let start = chronos.now(chronos.Moment)
|
||||||
|
await chronos.sleepAsync(sleepTime)
|
||||||
|
let afterSleep = chronos.now(chronos.Moment)
|
||||||
|
let sleepTime = afterSleep - start
|
||||||
|
onSecond(start)
|
||||||
|
let finished = chronos.now(chronos.Moment)
|
||||||
|
let processingTime = finished - afterSleep
|
||||||
|
trace "onSecond task completed", sleepTime, processingTime
|
||||||
|
|
||||||
|
onSecond(Moment.now())
|
||||||
|
lightClient.start()
|
||||||
|
|
||||||
|
asyncSpawn runOnSecondLoop()
|
||||||
|
while true:
|
||||||
|
poll()
|
||||||
|
|
||||||
|
when isMainModule:
|
||||||
|
run()
|
|
@ -0,0 +1,215 @@
|
||||||
|
# Nimbus
|
||||||
|
# Copyright (c) 2023 Status Research & Development GmbH
|
||||||
|
# Licensed and distributed under either of
|
||||||
|
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
|
||||||
|
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
|
||||||
|
# at your option. This file may not be copied, modified, or distributed except according to those terms.
|
||||||
|
|
||||||
|
when (NimMajor, NimMinor) < (1, 4):
|
||||||
|
{.push raises: [Defect].}
|
||||||
|
else:
|
||||||
|
{.push raises: [].}
|
||||||
|
|
||||||
|
import
|
||||||
|
std/os,
|
||||||
|
json_serialization/std/net,
|
||||||
|
beacon_chain/light_client,
|
||||||
|
beacon_chain/conf,
|
||||||
|
json_rpc/[rpcproxy]
|
||||||
|
|
||||||
|
export net, conf
|
||||||
|
|
||||||
|
proc defaultVerifiedProxyDataDir*(): string =
|
||||||
|
let dataDir = when defined(windows):
|
||||||
|
"AppData" / "Roaming" / "NimbusVerifiedProxy"
|
||||||
|
elif defined(macosx):
|
||||||
|
"Library" / "Application Support" / "NimbusVerifiedProxy"
|
||||||
|
else:
|
||||||
|
".cache" / "nimbus-verified-proxy"
|
||||||
|
|
||||||
|
getHomeDir() / dataDir
|
||||||
|
|
||||||
|
const
|
||||||
|
defaultDataVerifiedProxyDirDesc* = defaultVerifiedProxyDataDir()
|
||||||
|
|
||||||
|
type
|
||||||
|
Web3UrlKind* = enum
|
||||||
|
HttpUrl, WsUrl
|
||||||
|
|
||||||
|
ValidatedWeb3Url* = object
|
||||||
|
kind*: Web3UrlKind
|
||||||
|
web3Url*: string
|
||||||
|
|
||||||
|
type BeaconBridgeConf* = object
|
||||||
|
# Config
|
||||||
|
configFile* {.
|
||||||
|
desc: "Loads the configuration from a TOML file"
|
||||||
|
name: "config-file" .}: Option[InputFile]
|
||||||
|
|
||||||
|
# Logging
|
||||||
|
logLevel* {.
|
||||||
|
desc: "Sets the log level"
|
||||||
|
defaultValue: "INFO"
|
||||||
|
name: "log-level" .}: string
|
||||||
|
|
||||||
|
logStdout* {.
|
||||||
|
hidden
|
||||||
|
desc: "Specifies what kind of logs should be written to stdout (auto, colors, nocolors, json)"
|
||||||
|
defaultValueDesc: "auto"
|
||||||
|
defaultValue: StdoutLogKind.Auto
|
||||||
|
name: "log-format" .}: StdoutLogKind
|
||||||
|
|
||||||
|
# Storage
|
||||||
|
dataDir* {.
|
||||||
|
desc: "The directory where nimbus_verified_proxy will store all data"
|
||||||
|
defaultValue: defaultVerifiedProxyDataDir()
|
||||||
|
defaultValueDesc: $defaultDataVerifiedProxyDirDesc
|
||||||
|
abbr: "d"
|
||||||
|
name: "data-dir" .}: OutDir
|
||||||
|
|
||||||
|
# Network
|
||||||
|
eth2Network* {.
|
||||||
|
desc: "The Eth2 network to join"
|
||||||
|
defaultValueDesc: "mainnet"
|
||||||
|
name: "network" .}: Option[string]
|
||||||
|
|
||||||
|
# Consensus light sync
|
||||||
|
# No default - Needs to be provided by the user
|
||||||
|
trustedBlockRoot* {.
|
||||||
|
desc: "Recent trusted finalized block root to initialize the consensus light client from"
|
||||||
|
name: "trusted-block-root" .}: Eth2Digest
|
||||||
|
|
||||||
|
# Local JSON-RPC server
|
||||||
|
rpcAddress* {.
|
||||||
|
desc: "Listening address of the JSON-RPC server"
|
||||||
|
defaultValue: "127.0.0.1"
|
||||||
|
name: "rpc-address" .}: string
|
||||||
|
|
||||||
|
rpcPort* {.
|
||||||
|
desc: "Listening port of the JSON-RPC server"
|
||||||
|
defaultValue: 8545
|
||||||
|
name: "rpc-port" .}: Port
|
||||||
|
|
||||||
|
# Libp2p
|
||||||
|
bootstrapNodes* {.
|
||||||
|
desc: "Specifies one or more bootstrap nodes to use when connecting to the network"
|
||||||
|
abbr: "b"
|
||||||
|
name: "bootstrap-node" .}: seq[string]
|
||||||
|
|
||||||
|
bootstrapNodesFile* {.
|
||||||
|
desc: "Specifies a line-delimited file of bootstrap Ethereum network addresses"
|
||||||
|
defaultValue: ""
|
||||||
|
name: "bootstrap-file" .}: InputFile
|
||||||
|
|
||||||
|
listenAddress* {.
|
||||||
|
desc: "Listening address for the Ethereum LibP2P and Discovery v5 traffic"
|
||||||
|
defaultValue: defaultListenAddress
|
||||||
|
defaultValueDesc: $defaultListenAddressDesc
|
||||||
|
name: "listen-address" .}: ValidIpAddress
|
||||||
|
|
||||||
|
tcpPort* {.
|
||||||
|
desc: "Listening TCP port for Ethereum LibP2P traffic"
|
||||||
|
defaultValue: defaultEth2TcpPort
|
||||||
|
defaultValueDesc: $defaultEth2TcpPortDesc
|
||||||
|
name: "tcp-port" .}: Port
|
||||||
|
|
||||||
|
udpPort* {.
|
||||||
|
desc: "Listening UDP port for node discovery"
|
||||||
|
defaultValue: defaultEth2TcpPort
|
||||||
|
defaultValueDesc: $defaultEth2TcpPortDesc
|
||||||
|
name: "udp-port" .}: Port
|
||||||
|
|
||||||
|
# TODO: Select a lower amount of peers.
|
||||||
|
maxPeers* {.
|
||||||
|
desc: "The target number of peers to connect to"
|
||||||
|
defaultValue: 160 # 5 (fanout) * 64 (subnets) / 2 (subs) for a healthy mesh
|
||||||
|
name: "max-peers" .}: int
|
||||||
|
|
||||||
|
hardMaxPeers* {.
|
||||||
|
desc: "The maximum number of peers to connect to. Defaults to maxPeers * 1.5"
|
||||||
|
name: "hard-max-peers" .}: Option[int]
|
||||||
|
|
||||||
|
nat* {.
|
||||||
|
desc: "Specify method to use for determining public address. " &
|
||||||
|
"Must be one of: any, none, upnp, pmp, extip:<IP>"
|
||||||
|
defaultValue: NatConfig(hasExtIp: false, nat: NatAny)
|
||||||
|
defaultValueDesc: "any"
|
||||||
|
name: "nat" .}: NatConfig
|
||||||
|
|
||||||
|
enrAutoUpdate* {.
|
||||||
|
desc: "Discovery can automatically update its ENR with the IP address " &
|
||||||
|
"and UDP port as seen by other nodes it communicates with. " &
|
||||||
|
"This option allows to enable/disable this functionality"
|
||||||
|
defaultValue: false
|
||||||
|
name: "enr-auto-update" .}: bool
|
||||||
|
|
||||||
|
agentString* {.
|
||||||
|
defaultValue: "nimbus",
|
||||||
|
desc: "Node agent string which is used as identifier in the LibP2P network"
|
||||||
|
name: "agent-string" .}: string
|
||||||
|
|
||||||
|
discv5Enabled* {.
|
||||||
|
desc: "Enable Discovery v5"
|
||||||
|
defaultValue: true
|
||||||
|
name: "discv5" .}: bool
|
||||||
|
|
||||||
|
directPeers* {.
|
||||||
|
desc: "The list of priviledged, secure and known peers to connect and" &
|
||||||
|
"maintain the connection to, this requires a not random netkey-file." &
|
||||||
|
"In the complete multiaddress format like:" &
|
||||||
|
"/ip4/<address>/tcp/<port>/p2p/<peerId-public-key>." &
|
||||||
|
"Peering agreements are established out of band and must be reciprocal"
|
||||||
|
name: "direct-peer" .}: seq[string]
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
proc parseCmdArg*(T: type ValidatedWeb3Url, p: TaintedString): T
|
||||||
|
{.raises: [Defect, ConfigurationError].} =
|
||||||
|
let url = parseUri(p)
|
||||||
|
let normalizedScheme = url.scheme.toLowerAscii()
|
||||||
|
if (normalizedScheme == "http" or normalizedScheme == "https"):
|
||||||
|
ValidatedWeb3Url(kind: HttpUrl, web3Url: p)
|
||||||
|
elif (normalizedScheme == "ws" or normalizedScheme == "wss"):
|
||||||
|
ValidatedWeb3Url(kind: WsUrl, web3Url: p)
|
||||||
|
else:
|
||||||
|
raise newException(
|
||||||
|
ConfigurationError, "Web3 url should have defined scheme (http/https/ws/wss)"
|
||||||
|
)
|
||||||
|
|
||||||
|
proc completeCmdArg*(T: type ValidatedWeb3Url, val: TaintedString): seq[string] =
|
||||||
|
return @[]
|
||||||
|
|
||||||
|
func asLightClientConf*(pc: BeaconBridgeConf): LightClientConf =
|
||||||
|
return LightClientConf(
|
||||||
|
configFile: pc.configFile,
|
||||||
|
logLevel: pc.logLevel,
|
||||||
|
logStdout: pc.logStdout,
|
||||||
|
logFile: none(OutFile),
|
||||||
|
dataDir: pc.dataDir,
|
||||||
|
eth2Network: pc.eth2Network,
|
||||||
|
bootstrapNodes: pc.bootstrapNodes,
|
||||||
|
bootstrapNodesFile: pc.bootstrapNodesFile,
|
||||||
|
listenAddress: pc.listenAddress,
|
||||||
|
tcpPort: pc.tcpPort,
|
||||||
|
udpPort: pc.udpPort,
|
||||||
|
maxPeers: pc.maxPeers,
|
||||||
|
hardMaxPeers: pc.hardMaxPeers,
|
||||||
|
nat: pc.nat,
|
||||||
|
enrAutoUpdate: pc.enrAutoUpdate,
|
||||||
|
agentString: pc.agentString,
|
||||||
|
discv5Enabled: pc.discv5Enabled,
|
||||||
|
directPeers: pc.directPeers,
|
||||||
|
trustedBlockRoot: pc.trustedBlockRoot,
|
||||||
|
web3Urls: @[],
|
||||||
|
jwtSecret: none(string),
|
||||||
|
stopAtEpoch: 0
|
||||||
|
)
|
||||||
|
|
||||||
|
# TODO: Cannot use ClientConfig in VerifiedProxyConf due to the fact that
|
||||||
|
# it contain `set[TLSFlags]` which does not have proper toml serialization
|
||||||
|
func asClientConfig*(url: ValidatedWeb3Url): ClientConfig =
|
||||||
|
case url.kind
|
||||||
|
of HttpUrl:
|
||||||
|
getHttpClientConfig(url.web3Url)
|
||||||
|
of WsUrl:
|
||||||
|
getWebSocketClientConfig(url.web3Url, flags = {})
|
|
@ -87,10 +87,11 @@ task fluffy_test, "Run fluffy tests":
|
||||||
test "fluffy/tests/beacon_light_client_tests", "all_beacon_light_client_tests", "-d:chronicles_log_level=ERROR -d:chronosStrictException -d:nimbus_db_backend=sqlite -d:PREFER_BLST_SHA256=false"
|
test "fluffy/tests/beacon_light_client_tests", "all_beacon_light_client_tests", "-d:chronicles_log_level=ERROR -d:chronosStrictException -d:nimbus_db_backend=sqlite -d:PREFER_BLST_SHA256=false"
|
||||||
|
|
||||||
task fluffy_tools, "Build fluffy tools":
|
task fluffy_tools, "Build fluffy tools":
|
||||||
buildBinary "portalcli", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException -d:PREFER_BLST_SHA256=false"
|
buildBinary "beacon_chain_bridge", "fluffy/tools/bridge/", "-d:chronicles_log_level=TRACE -d:chronosStrictException -d:PREFER_BLST_SHA256=false -d:libp2p_pki_schemes=secp256k1"
|
||||||
buildBinary "blockwalk", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException"
|
|
||||||
buildBinary "eth_data_exporter", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException -d:PREFER_BLST_SHA256=false"
|
buildBinary "eth_data_exporter", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException -d:PREFER_BLST_SHA256=false"
|
||||||
buildBinary "content_verifier", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException -d:PREFER_BLST_SHA256=false"
|
buildBinary "content_verifier", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException -d:PREFER_BLST_SHA256=false"
|
||||||
|
buildBinary "blockwalk", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException"
|
||||||
|
buildBinary "portalcli", "fluffy/tools/", "-d:chronicles_log_level=TRACE -d:chronosStrictException -d:PREFER_BLST_SHA256=false"
|
||||||
|
|
||||||
task utp_test_app, "Build uTP test app":
|
task utp_test_app, "Build uTP test app":
|
||||||
buildBinary "utp_test_app", "fluffy/tools/utp_testing/", "-d:chronicles_log_level=TRACE -d:chronosStrictException"
|
buildBinary "utp_test_app", "fluffy/tools/utp_testing/", "-d:chronicles_log_level=TRACE -d:chronosStrictException"
|
||||||
|
|
|
@ -222,8 +222,6 @@ proc run() {.raises: [Exception, Defect].} =
|
||||||
|
|
||||||
blocksGossipState = targetGossipState
|
blocksGossipState = targetGossipState
|
||||||
|
|
||||||
var nextExchangeTransitionConfTime: Moment
|
|
||||||
|
|
||||||
proc onSecond(time: Moment) =
|
proc onSecond(time: Moment) =
|
||||||
let wallSlot = getBeaconTime().slotOrZero()
|
let wallSlot = getBeaconTime().slotOrZero()
|
||||||
updateBlocksGossipStatus(wallSlot + 1)
|
updateBlocksGossipStatus(wallSlot + 1)
|
||||||
|
|
Loading…
Reference in New Issue