mirror of
https://github.com/status-im/nimbus-eth1.git
synced 2025-01-24 03:00:25 +00:00
Add support for injecting receipts into the Portal Network (#1523)
Allow for the Fluffy Portal bridge to inject receipts. This requires a web3 endpoint to be provided, and currently only Alchemy is supported due to the used JSON-RPC endpoint.
This commit is contained in:
parent
0ea08655a0
commit
4c19a0e48e
@ -111,7 +111,7 @@ func fromBlockBody(T: type BlockBodySSZ, body: BlockBody): T =
|
|||||||
|
|
||||||
BlockBodySSZ(transactions: transactions, uncles: uncles)
|
BlockBodySSZ(transactions: transactions, uncles: uncles)
|
||||||
|
|
||||||
func fromReceipts(T: type ReceiptsSSZ, receipts: seq[Receipt]): T =
|
func fromReceipts*(T: type ReceiptsSSZ, receipts: seq[Receipt]): T =
|
||||||
var receiptsSSZ: ReceiptsSSZ
|
var receiptsSSZ: ReceiptsSSZ
|
||||||
for receipt in receipts:
|
for receipt in receipts:
|
||||||
discard receiptsSSZ.add(ReceiptByteList(rlp.encode(receipt)))
|
discard receiptsSSZ.add(ReceiptByteList(rlp.encode(receipt)))
|
||||||
@ -190,7 +190,7 @@ proc validateBlockBodyBytes*(
|
|||||||
|
|
||||||
BlockBody.fromPortalBlockBody(body)
|
BlockBody.fromPortalBlockBody(body)
|
||||||
|
|
||||||
proc validateReceipts(
|
proc validateReceipts*(
|
||||||
receipts: ReceiptsSSZ, receiptsRoot: KeccakHash): Result[void, string] =
|
receipts: ReceiptsSSZ, receiptsRoot: KeccakHash): Result[void, string] =
|
||||||
let calculatedReceiptsRoot = calcReceiptsRoot(receipts)
|
let calculatedReceiptsRoot = calcReceiptsRoot(receipts)
|
||||||
|
|
||||||
|
@ -12,6 +12,6 @@ import
|
|||||||
web3/conversions, # sigh
|
web3/conversions, # sigh
|
||||||
../../nimbus/rpc/[rpc_types, hexstrings]
|
../../nimbus/rpc/[rpc_types, hexstrings]
|
||||||
|
|
||||||
export rpcclient, rpc_types, errors
|
export rpcclient, rpc_types, hexstrings, errors
|
||||||
|
|
||||||
createRpcSigs(RpcClient, currentSourcePath.parentDir / "rpc_calls" / "rpc_eth_calls.nim")
|
createRpcSigs(RpcClient, currentSourcePath.parentDir / "rpc_calls" / "rpc_eth_calls.nim")
|
||||||
|
@ -2,4 +2,8 @@ proc eth_chaindId(): HexQuantityStr
|
|||||||
proc eth_getBlockByHash(data: EthHashStr, fullTransactions: bool): Option[BlockObject]
|
proc eth_getBlockByHash(data: EthHashStr, fullTransactions: bool): Option[BlockObject]
|
||||||
proc eth_getBlockByNumber(quantityTag: string, fullTransactions: bool): Option[BlockObject]
|
proc eth_getBlockByNumber(quantityTag: string, fullTransactions: bool): Option[BlockObject]
|
||||||
proc eth_getBlockTransactionCountByHash(data: EthHashStr): HexQuantityStr
|
proc eth_getBlockTransactionCountByHash(data: EthHashStr): HexQuantityStr
|
||||||
|
proc eth_getTransactionReceipt(data: Hash256): Option[ReceiptObject]
|
||||||
proc eth_getLogs(filterOptions: FilterOptions): seq[FilterLog]
|
proc eth_getLogs(filterOptions: FilterOptions): seq[FilterLog]
|
||||||
|
|
||||||
|
# Not supported: Only supported by Alchemy
|
||||||
|
proc eth_getBlockReceipts(data: Hash256): seq[ReceiptObject]
|
||||||
|
@ -16,6 +16,13 @@
|
|||||||
# The injection into the Portal network is done via the `portal_historyGossip`
|
# The injection into the Portal network is done via the `portal_historyGossip`
|
||||||
# JSON-RPC endpoint of a running Fluffy node.
|
# JSON-RPC endpoint of a running Fluffy node.
|
||||||
#
|
#
|
||||||
|
# If a web3 provider is configured, then block receipts will also be injected
|
||||||
|
# into the network whenever there is a new block. The web3 provider is needed
|
||||||
|
# to request the receipts. The receipts root is verified against the root
|
||||||
|
# provided bij the exection payload of the beacon block.
|
||||||
|
# To get the block receipts, the web3 provider currently needs to support the
|
||||||
|
# `eth_getBlockReceipts` JSON-RPC endpoint (not in standard specification).
|
||||||
|
#
|
||||||
# Other, currently not implemented, options to seed data:
|
# Other, currently not implemented, options to seed data:
|
||||||
# - Backfill post-merge block headers & bodies block into the network. Could
|
# - 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
|
# walk down the parent blocks and seed them. Could also verify if the data is
|
||||||
@ -31,10 +38,6 @@
|
|||||||
# (epoch accumulator) fluffy -> bridge
|
# (epoch accumulator) fluffy -> bridge
|
||||||
# (portal content) bridge -> fluffy
|
# (portal content) bridge -> fluffy
|
||||||
# This seems awfully cumbersome. Other options sound better, see comment down.
|
# 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.
|
# 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
|
# It is currently done by first downloading and storing all headers into files
|
||||||
@ -64,7 +67,6 @@
|
|||||||
|
|
||||||
import
|
import
|
||||||
std/[os, strutils, options],
|
std/[os, strutils, options],
|
||||||
web3/ethtypes,
|
|
||||||
chronicles, chronos, confutils,
|
chronicles, chronos, confutils,
|
||||||
eth/[keys, rlp], eth/[trie, trie/db],
|
eth/[keys, rlp], eth/[trie, trie/db],
|
||||||
# Need to rename this because of web3 ethtypes and ambigious indentifier mess
|
# Need to rename this because of web3 ethtypes and ambigious indentifier mess
|
||||||
@ -80,12 +82,17 @@ import
|
|||||||
# Weirdness. Need to import this to be able to do errors.ValidationResult as
|
# Weirdness. Need to import this to be able to do errors.ValidationResult as
|
||||||
# else we get an ambiguous identifier, ValidationResult from eth & libp2p.
|
# else we get an ambiguous identifier, ValidationResult from eth & libp2p.
|
||||||
libp2p/protocols/pubsub/errors,
|
libp2p/protocols/pubsub/errors,
|
||||||
../../rpc/portal_rpc_client,
|
../../../nimbus/rpc/rpc_types,
|
||||||
../../network/history/history_content,
|
../../rpc/[portal_rpc_client, eth_rpc_client],
|
||||||
|
../../network/history/[history_content, history_network],
|
||||||
../../network/beacon_light_client/beacon_light_client_content,
|
../../network/beacon_light_client/beacon_light_client_content,
|
||||||
../../common/common_types,
|
../../common/common_types,
|
||||||
./beacon_chain_bridge_conf
|
./beacon_chain_bridge_conf
|
||||||
|
|
||||||
|
from stew/objects import checkedEnumAssign
|
||||||
|
from stew/byteutils import readHexChar
|
||||||
|
from web3/ethtypes import BlockHash
|
||||||
|
|
||||||
from beacon_chain/gossip_processing/block_processor import newExecutionPayload
|
from beacon_chain/gossip_processing/block_processor import newExecutionPayload
|
||||||
from beacon_chain/gossip_processing/eth2_processor import toValidationResult
|
from beacon_chain/gossip_processing/eth2_processor import toValidationResult
|
||||||
|
|
||||||
@ -98,6 +105,86 @@ template asEthHash(hash: ethtypes.BlockHash): Hash256 =
|
|||||||
template unsafeQuantityToInt64(q: Quantity): int64 =
|
template unsafeQuantityToInt64(q: Quantity): int64 =
|
||||||
int64 q
|
int64 q
|
||||||
|
|
||||||
|
# TODO: Cannot use the `hexToInt` from rpc_utils as it importing that causes a
|
||||||
|
# strange "Exception can raise an unlisted exception: Exception` compile error.
|
||||||
|
func hexToInt(
|
||||||
|
s: string, T: typedesc[SomeInteger]): T {.raises: [ValueError].} =
|
||||||
|
var i = 0
|
||||||
|
if s[i] == '0' and (s[i+1] in {'x', 'X'}):
|
||||||
|
inc(i, 2)
|
||||||
|
if s.len - i > sizeof(T) * 2:
|
||||||
|
raise newException(ValueError, "Input hex too big for destination int")
|
||||||
|
|
||||||
|
var res: T = 0
|
||||||
|
while i < s.len:
|
||||||
|
res = res shl 4 or readHexChar(s[i]).T
|
||||||
|
inc(i)
|
||||||
|
|
||||||
|
res
|
||||||
|
|
||||||
|
func asTxType(quantity: HexQuantityStr): Result[TxType, string] =
|
||||||
|
let value =
|
||||||
|
try:
|
||||||
|
hexToInt(quantity.string, uint8)
|
||||||
|
except ValueError as e:
|
||||||
|
return err("Invalid data for TxType: " & e.msg)
|
||||||
|
|
||||||
|
var txType: TxType
|
||||||
|
if not checkedEnumAssign(txType, value):
|
||||||
|
err("Invalid data for TxType: " & $value)
|
||||||
|
else:
|
||||||
|
ok(txType)
|
||||||
|
|
||||||
|
func asReceipt(
|
||||||
|
receiptObject: rpc_types.ReceiptObject): Result[Receipt, string] =
|
||||||
|
let receiptType = asTxType(receiptObject.`type`).valueOr:
|
||||||
|
return err("Failed conversion to TxType" & error)
|
||||||
|
|
||||||
|
var logs: seq[Log]
|
||||||
|
if receiptObject.logs.len > 0:
|
||||||
|
for log in receiptObject.logs:
|
||||||
|
var topics: seq[Topic]
|
||||||
|
for topic in log.topics:
|
||||||
|
topics.add(Topic(topic.data))
|
||||||
|
|
||||||
|
logs.add(Log(
|
||||||
|
address: log.address,
|
||||||
|
data: log.data,
|
||||||
|
topics: topics
|
||||||
|
))
|
||||||
|
|
||||||
|
let cumulativeGasUsed =
|
||||||
|
try:
|
||||||
|
hexToInt(receiptObject.cumulativeGasUsed.string, GasInt)
|
||||||
|
except ValueError as e:
|
||||||
|
return err("Invalid data for cumulativeGasUsed: " & e.msg)
|
||||||
|
|
||||||
|
if receiptObject.status.isSome():
|
||||||
|
let status =
|
||||||
|
try:
|
||||||
|
hexToInt(receiptObject.status.get().string, int)
|
||||||
|
except ValueError as e:
|
||||||
|
return err("Invalid data for status: " & e.msg)
|
||||||
|
ok(Receipt(
|
||||||
|
receiptType: receiptType,
|
||||||
|
isHash: false,
|
||||||
|
status: status == 1,
|
||||||
|
cumulativeGasUsed: cumulativeGasUsed,
|
||||||
|
bloom: BloomFilter(receiptObject.logsBloom),
|
||||||
|
logs: logs
|
||||||
|
))
|
||||||
|
elif receiptObject.root.isSome():
|
||||||
|
ok(Receipt(
|
||||||
|
receiptType: receiptType,
|
||||||
|
isHash: true,
|
||||||
|
hash: receiptObject.root.get(),
|
||||||
|
cumulativeGasUsed: cumulativeGasUsed,
|
||||||
|
bloom: BloomFilter(receiptObject.logsBloom),
|
||||||
|
logs: logs
|
||||||
|
))
|
||||||
|
else:
|
||||||
|
err("No root nor status field in the JSON receipt object")
|
||||||
|
|
||||||
proc asPortalBlockData*(
|
proc asPortalBlockData*(
|
||||||
payload: ExecutionPayloadV1 | ExecutionPayloadV2 | ExecutionPayloadV3):
|
payload: ExecutionPayloadV1 | ExecutionPayloadV2 | ExecutionPayloadV3):
|
||||||
(common_types.BlockHash, BlockHeaderWithProof, BlockBodySSZ) =
|
(common_types.BlockHash, BlockHeaderWithProof, BlockBodySSZ) =
|
||||||
@ -162,6 +249,58 @@ func forkDigestAtEpoch(
|
|||||||
forkDigests: ForkDigests, epoch: Epoch, cfg: RuntimeConfig): ForkDigest =
|
forkDigests: ForkDigests, epoch: Epoch, cfg: RuntimeConfig): ForkDigest =
|
||||||
forkDigests.atEpoch(epoch, cfg)
|
forkDigests.atEpoch(epoch, cfg)
|
||||||
|
|
||||||
|
proc getBlockReceipts(
|
||||||
|
client: RpcClient, transactions: seq[TypedTransaction], blockHash: Hash256):
|
||||||
|
Future[Result[seq[Receipt], string]] {.async.} =
|
||||||
|
## Note: This makes use of `eth_getBlockReceipts` JSON-RPC endpoint which is
|
||||||
|
## only supported by Alchemy.
|
||||||
|
var receipts: seq[Receipt]
|
||||||
|
if transactions.len() > 0:
|
||||||
|
let receiptObjects =
|
||||||
|
# TODO: Add some retries depending on the failure
|
||||||
|
try:
|
||||||
|
await client.eth_getBlockReceipts(blockHash)
|
||||||
|
except CatchableError as e:
|
||||||
|
await client.close()
|
||||||
|
return err("JSON-RPC eth_getBlockReceipts failed: " & e.msg)
|
||||||
|
|
||||||
|
await client.close()
|
||||||
|
|
||||||
|
for receiptObject in receiptObjects:
|
||||||
|
let receipt = asReceipt(receiptObject).valueOr:
|
||||||
|
return err(error)
|
||||||
|
receipts.add(receipt)
|
||||||
|
|
||||||
|
return ok(receipts)
|
||||||
|
|
||||||
|
# TODO: This requires a seperate call for each transactions, which in reality
|
||||||
|
# takes too long and causes too much overhead. To make this usable the JSON-RPC
|
||||||
|
# code needs to get support for batch requests.
|
||||||
|
proc getBlockReceipts(
|
||||||
|
client: RpcClient, transactions: seq[TypedTransaction]):
|
||||||
|
Future[Result[seq[Receipt], string]] {.async.} =
|
||||||
|
var receipts: seq[Receipt]
|
||||||
|
for tx in transactions:
|
||||||
|
let txHash = keccakHash(tx.distinctBase)
|
||||||
|
let receiptObjectOpt =
|
||||||
|
# TODO: Add some retries depending on the failure
|
||||||
|
try:
|
||||||
|
await client.eth_getTransactionReceipt(txHash)
|
||||||
|
except CatchableError as e:
|
||||||
|
await client.close()
|
||||||
|
return err("JSON-RPC eth_getTransactionReceipt failed: " & e.msg)
|
||||||
|
|
||||||
|
await client.close()
|
||||||
|
|
||||||
|
if receiptObjectOpt.isNone():
|
||||||
|
return err("eth_getTransactionReceipt returned no receipt")
|
||||||
|
|
||||||
|
let receipt = asReceipt(receiptObjectOpt.get()).valueOr:
|
||||||
|
return err(error)
|
||||||
|
receipts.add(receipt)
|
||||||
|
|
||||||
|
return ok(receipts)
|
||||||
|
|
||||||
proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
||||||
# Required as both Eth2Node and LightClient requires correct config type
|
# Required as both Eth2Node and LightClient requires correct config type
|
||||||
var lcConfig = config.asLightClientConf()
|
var lcConfig = config.asLightClientConf()
|
||||||
@ -206,7 +345,19 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
forkDigests, getBeaconTime, genesis_validators_root
|
forkDigests, getBeaconTime, genesis_validators_root
|
||||||
)
|
)
|
||||||
|
|
||||||
rpcHttpclient = newRpcHttpClient()
|
portalRpcClient = newRpcHttpClient()
|
||||||
|
|
||||||
|
web3Client: Opt[RpcClient] =
|
||||||
|
if config.web3Url.isNone():
|
||||||
|
Opt.none(RpcClient)
|
||||||
|
else:
|
||||||
|
let client: RpcClient =
|
||||||
|
case config.web3Url.get().kind
|
||||||
|
of HttpUrl:
|
||||||
|
newRpcHttpClient()
|
||||||
|
of WsUrl:
|
||||||
|
newRpcWebSocketClient()
|
||||||
|
Opt.some(client)
|
||||||
|
|
||||||
optimisticHandler = proc(signedBlock: ForkedMsgTrustedSignedBeaconBlock):
|
optimisticHandler = proc(signedBlock: ForkedMsgTrustedSignedBeaconBlock):
|
||||||
Future[void] {.async.} =
|
Future[void] {.async.} =
|
||||||
@ -222,9 +373,10 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
if blck.message.is_execution_block:
|
if blck.message.is_execution_block:
|
||||||
template payload(): auto = blck.message.body.execution_payload
|
template payload(): auto = blck.message.body.execution_payload
|
||||||
|
|
||||||
# TODO: Get rid of the asEngineExecutionPayload step
|
# TODO: Get rid of the asEngineExecutionPayload step?
|
||||||
|
let executionPayload = payload.asEngineExecutionPayload()
|
||||||
let (hash, headerWithProof, body) =
|
let (hash, headerWithProof, body) =
|
||||||
asPortalBlockData(payload.asEngineExecutionPayload())
|
asPortalBlockData(executionPayload)
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
blockhash = history_content.`$`hash
|
blockhash = history_content.`$`hash
|
||||||
@ -234,7 +386,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
let encodedContentKey = contentKey.encode.asSeq()
|
let encodedContentKey = contentKey.encode.asSeq()
|
||||||
|
|
||||||
try:
|
try:
|
||||||
let peers = await rpcHttpclient.portal_historyGossip(
|
let peers = await portalRpcClient.portal_historyGossip(
|
||||||
toHex(encodedContentKey),
|
toHex(encodedContentKey),
|
||||||
SSZ.encode(headerWithProof).toHex())
|
SSZ.encode(headerWithProof).toHex())
|
||||||
info "Block header gossiped", peers,
|
info "Block header gossiped", peers,
|
||||||
@ -242,19 +394,19 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
error "JSON-RPC error", error = $e.msg
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
await rpcHttpclient.close()
|
await portalRpcClient.close()
|
||||||
|
|
||||||
# For bodies to get verified, the header needs to be available on
|
# 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. Wait a little to get the headers propagated through
|
||||||
# the network.
|
# the network.
|
||||||
await sleepAsync(1.seconds)
|
await sleepAsync(2.seconds)
|
||||||
|
|
||||||
block: # gossip block
|
block: # gossip block
|
||||||
let contentKey = history_content.ContentKey.init(blockBody, hash)
|
let contentKey = history_content.ContentKey.init(blockBody, hash)
|
||||||
let encodedContentKey = contentKey.encode.asSeq()
|
let encodedContentKey = contentKey.encode.asSeq()
|
||||||
|
|
||||||
try:
|
try:
|
||||||
let peers = await rpcHttpclient.portal_historyGossip(
|
let peers = await portalRpcClient.portal_historyGossip(
|
||||||
encodedContentKey.toHex(),
|
encodedContentKey.toHex(),
|
||||||
SSZ.encode(body).toHex())
|
SSZ.encode(body).toHex())
|
||||||
info "Block body gossiped", peers,
|
info "Block body gossiped", peers,
|
||||||
@ -262,7 +414,39 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
error "JSON-RPC error", error = $e.msg
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
await rpcHttpclient.close()
|
await portalRpcClient.close()
|
||||||
|
|
||||||
|
if web3Client.isSome():
|
||||||
|
# get receipts
|
||||||
|
let receipts =
|
||||||
|
(await web3Client.get().getBlockReceipts(
|
||||||
|
executionPayload.transactions, hash)).valueOr:
|
||||||
|
# (await web3Client.get().getBlockReceipts(
|
||||||
|
# executionPayload.transactions)).valueOr:
|
||||||
|
error "Error getting block receipts", error
|
||||||
|
return
|
||||||
|
|
||||||
|
let sszReceipts = ReceiptsSSZ.fromReceipts(receipts)
|
||||||
|
if validateReceipts(sszReceipts, payload.receiptsRoot).isErr():
|
||||||
|
error "Receipts root is invalid"
|
||||||
|
return
|
||||||
|
|
||||||
|
# gossip receipts
|
||||||
|
let contentKey = history_content.ContentKey.init(
|
||||||
|
history_content.ContentType.receipts, hash)
|
||||||
|
let encodedContentKeyHex = contentKey.encode.asSeq().toHex()
|
||||||
|
|
||||||
|
try:
|
||||||
|
let peers = await portalRpcClient.portal_historyGossip(
|
||||||
|
encodedContentKeyHex,
|
||||||
|
SSZ.encode(sszReceipts).toHex())
|
||||||
|
info "Block receipts gossiped", peers,
|
||||||
|
contentKey = encodedContentKeyHex
|
||||||
|
except CatchableError as e:
|
||||||
|
error "JSON-RPC error for portal_historyGossip", error = $e.msg
|
||||||
|
|
||||||
|
await portalRpcClient.close()
|
||||||
|
|
||||||
return
|
return
|
||||||
|
|
||||||
optimisticProcessor = initOptimisticProcessor(
|
optimisticProcessor = initOptimisticProcessor(
|
||||||
@ -296,7 +480,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
try:
|
try:
|
||||||
let
|
let
|
||||||
contentKeyHex = contentKey.asSeq().toHex()
|
contentKeyHex = contentKey.asSeq().toHex()
|
||||||
peers = await rpcHttpclient.portal_beaconLightClientGossip(
|
peers = await portalRpcClient.portal_beaconLightClientGossip(
|
||||||
contentKeyHex,
|
contentKeyHex,
|
||||||
content.toHex())
|
content.toHex())
|
||||||
info "Beacon LC bootstrap gossiped", peers,
|
info "Beacon LC bootstrap gossiped", peers,
|
||||||
@ -304,7 +488,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
error "JSON-RPC error", error = $e.msg
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
await rpcHttpclient.close()
|
await portalRpcClient.close()
|
||||||
|
|
||||||
asyncSpawn(GossipRpcAndClose())
|
asyncSpawn(GossipRpcAndClose())
|
||||||
|
|
||||||
@ -329,7 +513,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
try:
|
try:
|
||||||
let
|
let
|
||||||
contentKeyHex = contentKey.asSeq().toHex()
|
contentKeyHex = contentKey.asSeq().toHex()
|
||||||
peers = await rpcHttpclient.portal_beaconLightClientGossip(
|
peers = await portalRpcClient.portal_beaconLightClientGossip(
|
||||||
contentKeyHex,
|
contentKeyHex,
|
||||||
content.toHex())
|
content.toHex())
|
||||||
info "Beacon LC bootstrap gossiped", peers,
|
info "Beacon LC bootstrap gossiped", peers,
|
||||||
@ -337,7 +521,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
error "JSON-RPC error", error = $e.msg
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
await rpcHttpclient.close()
|
await portalRpcClient.close()
|
||||||
|
|
||||||
asyncSpawn(GossipRpcAndClose())
|
asyncSpawn(GossipRpcAndClose())
|
||||||
|
|
||||||
@ -364,7 +548,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
try:
|
try:
|
||||||
let
|
let
|
||||||
contentKeyHex = contentKey.asSeq().toHex()
|
contentKeyHex = contentKey.asSeq().toHex()
|
||||||
peers = await rpcHttpclient.portal_beaconLightClientGossip(
|
peers = await portalRpcClient.portal_beaconLightClientGossip(
|
||||||
contentKeyHex,
|
contentKeyHex,
|
||||||
content.toHex())
|
content.toHex())
|
||||||
info "Beacon LC bootstrap gossiped", peers,
|
info "Beacon LC bootstrap gossiped", peers,
|
||||||
@ -372,7 +556,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
error "JSON-RPC error", error = $e.msg
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
await rpcHttpclient.close()
|
await portalRpcClient.close()
|
||||||
|
|
||||||
asyncSpawn(GossipRpcAndClose())
|
asyncSpawn(GossipRpcAndClose())
|
||||||
|
|
||||||
@ -400,7 +584,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
try:
|
try:
|
||||||
let
|
let
|
||||||
contentKeyHex = contentKey.asSeq().toHex()
|
contentKeyHex = contentKey.asSeq().toHex()
|
||||||
peers = await rpcHttpclient.portal_beaconLightClientGossip(
|
peers = await portalRpcClient.portal_beaconLightClientGossip(
|
||||||
contentKeyHex,
|
contentKeyHex,
|
||||||
content.toHex())
|
content.toHex())
|
||||||
info "Beacon LC bootstrap gossiped", peers,
|
info "Beacon LC bootstrap gossiped", peers,
|
||||||
@ -408,13 +592,17 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} =
|
|||||||
except CatchableError as e:
|
except CatchableError as e:
|
||||||
error "JSON-RPC error", error = $e.msg
|
error "JSON-RPC error", error = $e.msg
|
||||||
|
|
||||||
await rpcHttpclient.close()
|
await portalRpcClient.close()
|
||||||
|
|
||||||
asyncSpawn(GossipRpcAndClose())
|
asyncSpawn(GossipRpcAndClose())
|
||||||
|
|
||||||
###
|
###
|
||||||
|
|
||||||
waitFor rpcHttpclient.connect(config.rpcAddress, Port(config.rpcPort), false)
|
waitFor portalRpcClient.connect(config.rpcAddress, Port(config.rpcPort), false)
|
||||||
|
|
||||||
|
if web3Client.isSome():
|
||||||
|
if config.web3Url.get().kind == HttpUrl:
|
||||||
|
waitFor (RpcHttpClient(web3Client.get())).connect(config.web3Url.get().web3Url)
|
||||||
|
|
||||||
info "Listening to incoming network requests"
|
info "Listening to incoming network requests"
|
||||||
network.initBeaconSync(cfg, forkDigests, genesisBlockRoot, getBeaconTime)
|
network.initBeaconSync(cfg, forkDigests, genesisBlockRoot, getBeaconTime)
|
||||||
|
@ -12,7 +12,7 @@ import
|
|||||||
json_serialization/std/net,
|
json_serialization/std/net,
|
||||||
beacon_chain/light_client,
|
beacon_chain/light_client,
|
||||||
beacon_chain/conf,
|
beacon_chain/conf,
|
||||||
json_rpc/[rpcproxy]
|
json_rpc/rpcproxy
|
||||||
|
|
||||||
export net, conf
|
export net, conf
|
||||||
|
|
||||||
@ -33,7 +33,7 @@ type
|
|||||||
Web3UrlKind* = enum
|
Web3UrlKind* = enum
|
||||||
HttpUrl, WsUrl
|
HttpUrl, WsUrl
|
||||||
|
|
||||||
ValidatedWeb3Url* = object
|
Web3Url* = object
|
||||||
kind*: Web3UrlKind
|
kind*: Web3UrlKind
|
||||||
web3Url*: string
|
web3Url*: string
|
||||||
|
|
||||||
@ -58,41 +58,47 @@ type BeaconBridgeConf* = object
|
|||||||
|
|
||||||
# Storage
|
# Storage
|
||||||
dataDir* {.
|
dataDir* {.
|
||||||
desc: "The directory where nimbus_verified_proxy will store all data"
|
desc: "The directory where beacon_chain_bridge will store all data"
|
||||||
defaultValue: defaultVerifiedProxyDataDir()
|
defaultValue: defaultVerifiedProxyDataDir()
|
||||||
defaultValueDesc: $defaultDataVerifiedProxyDirDesc
|
defaultValueDesc: $defaultDataVerifiedProxyDirDesc
|
||||||
abbr: "d"
|
abbr: "d"
|
||||||
name: "data-dir" .}: OutDir
|
name: "data-dir" .}: OutDir
|
||||||
|
|
||||||
# Bridge options
|
# Portal JSON-RPC API server to connect to
|
||||||
|
rpcAddress* {.
|
||||||
|
desc: "Listening address of the Portal JSON-RPC server"
|
||||||
|
defaultValue: "127.0.0.1"
|
||||||
|
name: "rpc-address" .}: string
|
||||||
|
|
||||||
|
rpcPort* {.
|
||||||
|
desc: "Listening port of the Portal JSON-RPC server"
|
||||||
|
defaultValue: 8545
|
||||||
|
name: "rpc-port" .}: Port
|
||||||
|
|
||||||
|
## Bridge options
|
||||||
|
|
||||||
beaconLightClient* {.
|
beaconLightClient* {.
|
||||||
desc: "Enable beacon light client content bridging"
|
desc: "Enable beacon light client content bridging"
|
||||||
defaultValue: false
|
defaultValue: false
|
||||||
name: "beacon-light-client" .}: bool
|
name: "beacon-light-client" .}: bool
|
||||||
|
|
||||||
|
web3Url* {.
|
||||||
|
desc: "Execution layer JSON-RPC API URL"
|
||||||
|
name: "web3-url" .}: Option[Web3Url]
|
||||||
|
|
||||||
|
## Beacon chain light client specific options
|
||||||
|
|
||||||
|
# For 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
|
||||||
|
|
||||||
# Network
|
# Network
|
||||||
eth2Network* {.
|
eth2Network* {.
|
||||||
desc: "The Eth2 network to join"
|
desc: "The Eth2 network to join"
|
||||||
defaultValueDesc: "mainnet"
|
defaultValueDesc: "mainnet"
|
||||||
name: "network" .}: Option[string]
|
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
|
# Libp2p
|
||||||
bootstrapNodes* {.
|
bootstrapNodes* {.
|
||||||
desc: "Specifies one or more bootstrap nodes to use when connecting to the network"
|
desc: "Specifies one or more bootstrap nodes to use when connecting to the network"
|
||||||
@ -164,22 +170,23 @@ type BeaconBridgeConf* = object
|
|||||||
"Peering agreements are established out of band and must be reciprocal"
|
"Peering agreements are established out of band and must be reciprocal"
|
||||||
name: "direct-peer" .}: seq[string]
|
name: "direct-peer" .}: seq[string]
|
||||||
|
|
||||||
|
proc parseCmdArg*(
|
||||||
|
T: type Web3Url, p: string): T {.raises: [ConfigurationError].} =
|
||||||
|
let
|
||||||
|
url = parseUri(p)
|
||||||
|
normalizedScheme = url.scheme.toLowerAscii()
|
||||||
|
|
||||||
|
|
||||||
proc parseCmdArg*(T: type ValidatedWeb3Url, p: string): T
|
|
||||||
{.raises: [ConfigurationError].} =
|
|
||||||
let url = parseUri(p)
|
|
||||||
let normalizedScheme = url.scheme.toLowerAscii()
|
|
||||||
if (normalizedScheme == "http" or normalizedScheme == "https"):
|
if (normalizedScheme == "http" or normalizedScheme == "https"):
|
||||||
ValidatedWeb3Url(kind: HttpUrl, web3Url: p)
|
Web3Url(kind: HttpUrl, web3Url: p)
|
||||||
elif (normalizedScheme == "ws" or normalizedScheme == "wss"):
|
elif (normalizedScheme == "ws" or normalizedScheme == "wss"):
|
||||||
ValidatedWeb3Url(kind: WsUrl, web3Url: p)
|
Web3Url(kind: WsUrl, web3Url: p)
|
||||||
else:
|
else:
|
||||||
raise newException(
|
raise newException(
|
||||||
ConfigurationError, "Web3 url should have defined scheme (http/https/ws/wss)"
|
ConfigurationError,
|
||||||
|
"The Web3 URL must specify one of following protocols: http/https/ws/wss"
|
||||||
)
|
)
|
||||||
|
|
||||||
proc completeCmdArg*(T: type ValidatedWeb3Url, val: string): seq[string] =
|
proc completeCmdArg*(T: type Web3Url, val: string): seq[string] =
|
||||||
return @[]
|
return @[]
|
||||||
|
|
||||||
func asLightClientConf*(pc: BeaconBridgeConf): LightClientConf =
|
func asLightClientConf*(pc: BeaconBridgeConf): LightClientConf =
|
||||||
@ -207,12 +214,3 @@ func asLightClientConf*(pc: BeaconBridgeConf): LightClientConf =
|
|||||||
jwtSecret: none(InputFile),
|
jwtSecret: none(InputFile),
|
||||||
stopAtEpoch: 0
|
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 = {})
|
|
||||||
|
@ -33,7 +33,7 @@ type
|
|||||||
Web3UrlKind* = enum
|
Web3UrlKind* = enum
|
||||||
HttpUrl, WsUrl
|
HttpUrl, WsUrl
|
||||||
|
|
||||||
ValidatedWeb3Url* = object
|
Web3Url* = object
|
||||||
kind*: Web3UrlKind
|
kind*: Web3UrlKind
|
||||||
web3Url*: string
|
web3Url*: string
|
||||||
|
|
||||||
@ -80,7 +80,7 @@ type VerifiedProxyConf* = object
|
|||||||
# No default - Needs to be provided by the user
|
# No default - Needs to be provided by the user
|
||||||
web3url* {.
|
web3url* {.
|
||||||
desc: "URL of the web3 data provider"
|
desc: "URL of the web3 data provider"
|
||||||
name: "web3-url" .}: ValidatedWeb3Url
|
name: "web3-url" .}: Web3Url
|
||||||
|
|
||||||
# Local JSON-RPC server
|
# Local JSON-RPC server
|
||||||
rpcAddress* {.
|
rpcAddress* {.
|
||||||
@ -166,20 +166,22 @@ type VerifiedProxyConf* = object
|
|||||||
name: "direct-peer" .}: seq[string]
|
name: "direct-peer" .}: seq[string]
|
||||||
|
|
||||||
|
|
||||||
proc parseCmdArg*(T: type ValidatedWeb3Url, p: string): T
|
proc parseCmdArg*(
|
||||||
{.raises: [ConfigurationError].} =
|
T: type Web3Url, p: string): T {.raises: [ConfigurationError].} =
|
||||||
let url = parseUri(p)
|
let
|
||||||
let normalizedScheme = url.scheme.toLowerAscii()
|
url = parseUri(p)
|
||||||
|
normalizedScheme = url.scheme.toLowerAscii()
|
||||||
|
|
||||||
if (normalizedScheme == "http" or normalizedScheme == "https"):
|
if (normalizedScheme == "http" or normalizedScheme == "https"):
|
||||||
ValidatedWeb3Url(kind: HttpUrl, web3Url: p)
|
Web3Url(kind: HttpUrl, web3Url: p)
|
||||||
elif (normalizedScheme == "ws" or normalizedScheme == "wss"):
|
elif (normalizedScheme == "ws" or normalizedScheme == "wss"):
|
||||||
ValidatedWeb3Url(kind: WsUrl, web3Url: p)
|
Web3Url(kind: WsUrl, web3Url: p)
|
||||||
else:
|
else:
|
||||||
raise newException(
|
raise newException(
|
||||||
ConfigurationError, "Web3 url should have defined scheme (http/https/ws/wss)"
|
ConfigurationError, "Web3 url should have defined scheme (http/https/ws/wss)"
|
||||||
)
|
)
|
||||||
|
|
||||||
proc completeCmdArg*(T: type ValidatedWeb3Url, val: string): seq[string] =
|
proc completeCmdArg*(T: type Web3Url, val: string): seq[string] =
|
||||||
return @[]
|
return @[]
|
||||||
|
|
||||||
func asLightClientConf*(pc: VerifiedProxyConf): LightClientConf =
|
func asLightClientConf*(pc: VerifiedProxyConf): LightClientConf =
|
||||||
@ -210,10 +212,9 @@ func asLightClientConf*(pc: VerifiedProxyConf): LightClientConf =
|
|||||||
|
|
||||||
# TODO: Cannot use ClientConfig in VerifiedProxyConf due to the fact that
|
# TODO: Cannot use ClientConfig in VerifiedProxyConf due to the fact that
|
||||||
# it contain `set[TLSFlags]` which does not have proper toml serialization
|
# it contain `set[TLSFlags]` which does not have proper toml serialization
|
||||||
func asClientConfig*(url: ValidatedWeb3Url): ClientConfig =
|
func asClientConfig*(url: Web3Url): ClientConfig =
|
||||||
case url.kind
|
case url.kind
|
||||||
of HttpUrl:
|
of HttpUrl:
|
||||||
getHttpClientConfig(url.web3Url)
|
getHttpClientConfig(url.web3Url)
|
||||||
of WsUrl:
|
of WsUrl:
|
||||||
getWebSocketClientConfig(url.web3Url, flags = {})
|
getWebSocketClientConfig(url.web3Url, flags = {})
|
||||||
|
|
||||||
|
Loading…
x
Reference in New Issue
Block a user