mirror of
https://github.com/status-im/nimbus-eth2.git
synced 2025-01-10 22:36:01 +00:00
move rpc api to own folder, mimic upstream structure (#1905)
also implements a few more endpoints
This commit is contained in:
parent
677f099110
commit
ee2ebc96a8
@ -17,21 +17,22 @@ import
|
||||
json_serialization/std/[options, sets, net], serialization/errors,
|
||||
|
||||
eth/[keys, async_utils],
|
||||
eth/common/eth_types_json_serialization,
|
||||
eth/db/[kvstore, kvstore_sqlite3],
|
||||
eth/p2p/enode, eth/p2p/discoveryv5/[protocol, enr],
|
||||
|
||||
# Local modules
|
||||
./rpc/[beacon_api, config_api, debug_api, event_api, nimbus_api, node_api,
|
||||
validator_api],
|
||||
spec/[datatypes, digest, crypto, beaconstate, helpers, network, presets],
|
||||
spec/[state_transition, weak_subjectivity],
|
||||
spec/[weak_subjectivity],
|
||||
conf, time, beacon_chain_db, validator_pool, extras,
|
||||
attestation_pool, exit_pool, eth2_network, eth2_discovery,
|
||||
beacon_node_common, beacon_node_types, beacon_node_status,
|
||||
block_pools/[spec_cache, chain_dag, quarantine, clearance, block_pools_types],
|
||||
block_pools/[chain_dag, quarantine, clearance, block_pools_types],
|
||||
nimbus_binary_common, network_metadata,
|
||||
mainchain_monitor, version, ssz/[merkleization], merkle_minimal,
|
||||
sync_protocol, request_manager, keystore_management, interop, statusbar,
|
||||
sync_manager, validator_duties, validator_api,
|
||||
sync_manager, validator_duties,
|
||||
validator_slashing_protection,
|
||||
./eth2_processor
|
||||
|
||||
@ -680,93 +681,17 @@ proc startSyncManager(node: BeaconNode) =
|
||||
)
|
||||
node.syncManager.start()
|
||||
|
||||
proc currentSlot(node: BeaconNode): Slot =
|
||||
node.beaconClock.now.slotOrZero
|
||||
|
||||
proc connectedPeersCount(node: BeaconNode): int =
|
||||
len(node.network.peerPool)
|
||||
|
||||
proc installBeaconApiHandlers(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("getBeaconHead") do () -> Slot:
|
||||
return node.chainDag.head.slot
|
||||
|
||||
rpcServer.rpc("getChainHead") do () -> JsonNode:
|
||||
let
|
||||
head = node.chainDag.head
|
||||
finalized = node.chainDag.headState.data.data.finalized_checkpoint
|
||||
justified = node.chainDag.headState.data.data.current_justified_checkpoint
|
||||
return %* {
|
||||
"head_slot": head.slot,
|
||||
"head_block_root": head.root.data.toHex(),
|
||||
"finalized_slot": finalized.epoch * SLOTS_PER_EPOCH,
|
||||
"finalized_block_root": finalized.root.data.toHex(),
|
||||
"justified_slot": justified.epoch * SLOTS_PER_EPOCH,
|
||||
"justified_block_root": justified.root.data.toHex(),
|
||||
}
|
||||
|
||||
rpcServer.rpc("getSyncing") do () -> bool:
|
||||
let
|
||||
wallSlot = currentSlot(node)
|
||||
headSlot = node.chainDag.head.slot
|
||||
# FIXME: temporary hack: If more than 1 block away from expected head, then we are "syncing"
|
||||
return (headSlot + 1) < wallSlot
|
||||
|
||||
template requireOneOf(x, y: distinct Option) =
|
||||
if x.isNone xor y.isNone:
|
||||
raise newException(CatchableError,
|
||||
"Please specify one of " & astToStr(x) & " or " & astToStr(y))
|
||||
|
||||
rpcServer.rpc("getNetworkPeerId") do () -> string:
|
||||
return $publicKey(node.network)
|
||||
|
||||
rpcServer.rpc("getNetworkPeers") do () -> seq[string]:
|
||||
for peerId, peer in node.network.peerPool:
|
||||
result.add $peerId
|
||||
|
||||
rpcServer.rpc("getNetworkEnr") do () -> string:
|
||||
return $node.network.discovery.localNode.record
|
||||
|
||||
proc installDebugApiHandlers(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("getNodeVersion") do () -> string:
|
||||
return "Nimbus/" & fullVersionStr
|
||||
|
||||
rpcServer.rpc("getSpecPreset") do () -> JsonNode:
|
||||
var res = newJObject()
|
||||
genStmtList:
|
||||
for presetValue in PresetValue:
|
||||
if presetValue notin ignoredValues + runtimeValues:
|
||||
let
|
||||
settingSym = ident($presetValue)
|
||||
settingKey = newLit(toLowerAscii($presetValue))
|
||||
let f = quote do:
|
||||
res[`settingKey`] = %(presets.`settingSym`)
|
||||
yield f
|
||||
|
||||
for field, value in fieldPairs(node.config.runtimePreset):
|
||||
res[field] = when value isnot Version: %value
|
||||
else: %value.toUInt64
|
||||
|
||||
return res
|
||||
|
||||
rpcServer.rpc("peers") do () -> JsonNode:
|
||||
var res = newJObject()
|
||||
var peers = newJArray()
|
||||
for id, peer in node.network.peerPool:
|
||||
peers.add(
|
||||
%(
|
||||
info: shortLog(peer.info),
|
||||
connectionState: $peer.connectionState,
|
||||
score: peer.score,
|
||||
)
|
||||
)
|
||||
res.add("peers", peers)
|
||||
|
||||
return res
|
||||
|
||||
proc installRpcHandlers(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.installValidatorApiHandlers(node)
|
||||
rpcServer.installBeaconApiHandlers(node)
|
||||
rpcServer.installConfigApiHandlers(node)
|
||||
rpcServer.installDebugApiHandlers(node)
|
||||
rpcServer.installEventApiHandlers(node)
|
||||
rpcServer.installNimbusApiHandlers(node)
|
||||
rpcServer.installNodeApiHandlers(node)
|
||||
rpcServer.installValidatorApiHandlers(node)
|
||||
|
||||
proc installMessageValidators(node: BeaconNode) =
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0-rc.0/specs/phase0/p2p-interface.md#attestations-and-aggregation
|
||||
|
@ -76,3 +76,6 @@ template findIt*(s: openarray, predicate: untyped): int =
|
||||
res = i
|
||||
break
|
||||
res
|
||||
|
||||
proc currentSlot*(node: BeaconNode): Slot =
|
||||
node.beaconClock.now.slotOrZero
|
||||
|
@ -16,7 +16,7 @@ import
|
||||
beaconstate],
|
||||
block_pools_types, quarantine
|
||||
|
||||
export block_pools_types
|
||||
export block_pools_types, helpers
|
||||
|
||||
declareCounter beacon_reorgs_total, "Total occurrences of reorganizations of the chain" # On fork choice
|
||||
declareCounter beacon_state_data_cache_hits, "EpochRef hits"
|
||||
|
@ -6,6 +6,8 @@ import
|
||||
eth/keys, eth/p2p/discoveryv5/[enr, protocol, node],
|
||||
conf
|
||||
|
||||
export protocol, keys
|
||||
|
||||
type
|
||||
Eth2DiscoveryProtocol* = protocol.Protocol
|
||||
Eth2DiscoveryId* = NodeId
|
||||
|
@ -36,7 +36,7 @@ when chronicles.enabledLogLevel == LogLevel.TRACE:
|
||||
|
||||
export
|
||||
version, multiaddress, peer_pool, peerinfo, p2pProtocol, connection,
|
||||
libp2p_json_serialization, ssz_serialization, results
|
||||
libp2p_json_serialization, ssz_serialization, results, eth2_discovery
|
||||
|
||||
logScope:
|
||||
topics = "networking"
|
||||
@ -309,6 +309,12 @@ proc openStream(node: Eth2Node,
|
||||
|
||||
proc init*(T: type Peer, network: Eth2Node, info: PeerInfo): Peer {.gcsafe.}
|
||||
|
||||
func peerId*(node: Eth2Node): PeerID =
|
||||
node.switch.peerInfo.peerId
|
||||
|
||||
func enrRecord*(node: Eth2Node): Record =
|
||||
node.discovery.localNode.record
|
||||
|
||||
proc getPeer*(node: Eth2Node, peerId: PeerID): Peer =
|
||||
node.peers.withValue(peerId, peer) do:
|
||||
return peer[]
|
||||
|
@ -13,7 +13,7 @@ import
|
||||
|
||||
# Nimble packages
|
||||
chronos, confutils/defs,
|
||||
chronicles, chronicles/helpers as chroniclesHelpers,
|
||||
chronicles, chronicles/helpers as chroniclesHelpers, chronicles/topics_registry,
|
||||
stew/io2,
|
||||
|
||||
# Local modules
|
||||
|
@ -1,6 +1,8 @@
|
||||
import std/[tables, heapqueue]
|
||||
import chronos
|
||||
|
||||
export tables
|
||||
|
||||
type
|
||||
PeerType* = enum
|
||||
Incoming, Outgoing
|
||||
|
@ -1,4 +1,3 @@
|
||||
# beacon_chain
|
||||
# Copyright (c) 2018-2020 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).
|
||||
@ -6,32 +5,25 @@
|
||||
# at your option. This file may not be copied, modified, or distributed except according to those terms.
|
||||
|
||||
import
|
||||
# Standard library
|
||||
tables, strutils, parseutils, sequtils,
|
||||
std/[parseutils, sequtils, strutils],
|
||||
|
||||
# Nimble packages
|
||||
stew/[byteutils, objects],
|
||||
chronos, metrics, json_rpc/[rpcserver, jsonmarshal],
|
||||
json_rpc/[rpcserver, jsonmarshal],
|
||||
chronicles,
|
||||
../beacon_node_common, ../eth2_json_rpc_serialization, ../eth2_network,
|
||||
../validator_duties,
|
||||
../block_pools/chain_dag,
|
||||
../spec/[crypto, digest, datatypes, validator],
|
||||
../spec/eth2_apis/callsigs_types,
|
||||
../ssz/merkleization,
|
||||
./rpc_utils
|
||||
|
||||
# Local modules
|
||||
spec/[datatypes, digest, crypto, validator, helpers],
|
||||
block_pools/[chain_dag, spec_cache], ssz/merkleization,
|
||||
beacon_node_common, beacon_node_types, attestation_pool,
|
||||
validator_duties, eth2_network,
|
||||
spec/eth2_apis/callsigs_types,
|
||||
eth2_json_rpc_serialization
|
||||
logScope: topics = "beaconapi"
|
||||
|
||||
type
|
||||
RpcServer* = RpcHttpServer
|
||||
RpcServer = RpcHttpServer
|
||||
|
||||
logScope: topics = "valapi"
|
||||
|
||||
proc toBlockSlot(blckRef: BlockRef): BlockSlot =
|
||||
blckRef.atSlot(blckRef.slot)
|
||||
|
||||
proc parseRoot(str: string): Eth2Digest =
|
||||
return Eth2Digest(data: hexToByteArray[32](str))
|
||||
template unimplemented() =
|
||||
raise (ref CatchableError)(msg: "Unimplemented")
|
||||
|
||||
proc parsePubkey(str: string): ValidatorPubKey =
|
||||
if str.len != RawPubKeySize + 2: # +2 because of the `0x` prefix
|
||||
@ -41,24 +33,6 @@ proc parsePubkey(str: string): ValidatorPubKey =
|
||||
raise newException(CatchableError, "Not a valid public key")
|
||||
return pubkeyRes[]
|
||||
|
||||
func checkEpochToSlotOverflow(epoch: Epoch) =
|
||||
const maxEpoch = compute_epoch_at_slot(not 0'u64)
|
||||
if epoch >= maxEpoch:
|
||||
raise newException(
|
||||
ValueError, "Requesting epoch for which slot would overflow")
|
||||
|
||||
proc doChecksAndGetCurrentHead(node: BeaconNode, slot: Slot): BlockRef =
|
||||
result = node.chainDag.head
|
||||
if not node.isSynced(result):
|
||||
raise newException(CatchableError, "Cannot fulfill request until node is synced")
|
||||
# TODO for now we limit the requests arbitrarily by up to 2 epochs into the future
|
||||
if result.slot + uint64(2 * SLOTS_PER_EPOCH) < slot:
|
||||
raise newException(CatchableError, "Requesting way ahead of the current head")
|
||||
|
||||
proc doChecksAndGetCurrentHead(node: BeaconNode, epoch: Epoch): BlockRef =
|
||||
checkEpochToSlotOverflow(epoch)
|
||||
node.doChecksAndGetCurrentHead(epoch.compute_start_slot_at_epoch)
|
||||
|
||||
# TODO currently this function throws if the validator isn't found - is this OK?
|
||||
proc getValidatorInfoFromValidatorId(
|
||||
state: BeaconState,
|
||||
@ -132,15 +106,6 @@ proc getValidatorInfoFromValidatorId(
|
||||
return some((validator: validator, status: actual_status,
|
||||
balance: validator.effective_balance))
|
||||
|
||||
proc getBlockSlotFromString(node: BeaconNode, slot: string): BlockSlot =
|
||||
if slot.len == 0:
|
||||
raise newException(ValueError, "Empty slot number not allowed")
|
||||
var parsed: BiggestUInt
|
||||
if parseBiggestUInt(slot, parsed) != slot.len:
|
||||
raise newException(ValueError, "Not a valid slot number")
|
||||
let head = node.doChecksAndGetCurrentHead(parsed.Slot)
|
||||
return head.atSlot(parsed.Slot)
|
||||
|
||||
proc getBlockDataFromBlockId(node: BeaconNode, blockId: string): BlockData =
|
||||
result = case blockId:
|
||||
of "head":
|
||||
@ -162,44 +127,14 @@ proc getBlockDataFromBlockId(node: BeaconNode, blockId: string): BlockData =
|
||||
raise newException(CatchableError, "Block not found")
|
||||
node.chainDag.get(blockSlot.blck)
|
||||
|
||||
proc stateIdToBlockSlot(node: BeaconNode, stateId: string): BlockSlot =
|
||||
result = case stateId:
|
||||
of "head":
|
||||
node.chainDag.head.toBlockSlot()
|
||||
of "genesis":
|
||||
node.chainDag.getGenesisBlockSlot()
|
||||
of "finalized":
|
||||
node.chainDag.finalizedHead
|
||||
of "justified":
|
||||
node.chainDag.head.atEpochStart(
|
||||
node.chainDag.headState.data.data.current_justified_checkpoint.epoch)
|
||||
else:
|
||||
if stateId.startsWith("0x"):
|
||||
let blckRoot = parseRoot(stateId)
|
||||
let blckRef = node.chainDag.getRef(blckRoot)
|
||||
if blckRef.isNil:
|
||||
raise newException(CatchableError, "Block not found")
|
||||
blckRef.toBlockSlot()
|
||||
else:
|
||||
node.getBlockSlotFromString(stateId)
|
||||
|
||||
# TODO Probably the `beacon` ones should be defined elsewhere...?
|
||||
|
||||
proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
|
||||
let GENESIS_FORK_VERSION = node.config.runtimePreset.GENESIS_FORK_VERSION
|
||||
|
||||
template withStateForStateId(stateId: string, body: untyped): untyped =
|
||||
# TODO this can be optimized for the "head" case since that should be most common
|
||||
node.chainDag.withState(node.chainDag.tmpState,
|
||||
node.stateIdToBlockSlot(stateId)):
|
||||
body
|
||||
|
||||
proc installBeaconApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("get_v1_beacon_genesis") do () -> BeaconGenesisTuple:
|
||||
return (genesis_time: node.chainDag.headState.data.data.genesis_time,
|
||||
genesis_validators_root:
|
||||
node.chainDag.headState.data.data.genesis_validators_root,
|
||||
genesis_fork_version: GENESIS_FORK_VERSION)
|
||||
return (
|
||||
genesis_time: node.chainDag.headState.data.data.genesis_time,
|
||||
genesis_validators_root:
|
||||
node.chainDag.headState.data.data.genesis_validators_root,
|
||||
genesis_fork_version: node.config.runtimePreset.GENESIS_FORK_VERSION
|
||||
)
|
||||
|
||||
rpcServer.rpc("get_v1_beacon_states_root") do (stateId: string) -> Eth2Digest:
|
||||
withStateForStateId(stateId):
|
||||
@ -237,6 +172,10 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
raise newException(CatchableError, "Validator status differs")
|
||||
return res.get()
|
||||
|
||||
rpcServer.rpc("get_v1_beacon_states_stateId_validator_balances") do (
|
||||
stateId: string) -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_beacon_states_stateId_committees_epoch") do (
|
||||
stateId: string, epoch: uint64, index: uint64, slot: uint64) ->
|
||||
seq[BeaconStatesCommitteesTuple]:
|
||||
@ -265,14 +204,7 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
|
||||
rpcServer.rpc("get_v1_beacon_headers") do (
|
||||
slot: uint64, parent_root: Eth2Digest) -> seq[BeaconHeadersTuple]:
|
||||
# @mratsim: I'm adding a toposorted iterator that returns all blocks from last finalization to all heads in the dual fork choice PR @viktor
|
||||
|
||||
# filterIt(dag.blocks.values(), it.blck.slot == slot_of_interest)
|
||||
# maybe usesBlockPool.heads ??? or getBlockRange ???
|
||||
|
||||
# https://discordapp.com/channels/613988663034118151/614014714590134292/726095138484518912
|
||||
|
||||
discard # raise newException(CatchableError, "Not implemented") # cannot compile...
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_beacon_headers_blockId") do (
|
||||
blockId: string) -> tuple[canonical: bool, header: SignedBeaconBlockHeader]:
|
||||
@ -300,105 +232,28 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
blockId: string) -> seq[TrustedAttestation]:
|
||||
return node.getBlockDataFromBlockId(blockId).data.message.body.attestations.asSeq
|
||||
|
||||
rpcServer.rpc("get_v1_beacon_pool_attestations") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("post_v1_beacon_pool_attestations") do (
|
||||
attestation: Attestation) -> bool:
|
||||
node.sendAttestation(attestation)
|
||||
return true
|
||||
|
||||
rpcServer.rpc("get_v1_config_fork_schedule") do (
|
||||
) -> seq[tuple[epoch: uint64, version: Version]]:
|
||||
discard # raise newException(CatchableError, "Not implemented") # cannot compile...
|
||||
rpcServer.rpc("get_v1_beacon_pool_attester_slahsings") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_debug_beacon_states_stateId") do (
|
||||
stateId: string) -> BeaconState:
|
||||
withStateForStateId(stateId):
|
||||
return state
|
||||
rpcServer.rpc("post_v1_beacon_pool_attester_slahsings") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_validator_block") do (
|
||||
slot: Slot, graffiti: GraffitiBytes, randao_reveal: ValidatorSig) -> BeaconBlock:
|
||||
debug "get_v1_validator_block", slot = slot
|
||||
let head = node.doChecksAndGetCurrentHead(slot)
|
||||
let proposer = node.chainDag.getProposer(head, slot)
|
||||
if proposer.isNone():
|
||||
raise newException(CatchableError, "could not retrieve block for slot: " & $slot)
|
||||
let message = makeBeaconBlockForHeadAndSlot(
|
||||
node, randao_reveal, proposer.get()[0], graffiti, head, slot)
|
||||
if message.isNone():
|
||||
raise newException(CatchableError, "could not retrieve block for slot: " & $slot)
|
||||
return message.get()
|
||||
rpcServer.rpc("get_v1_beacon_pool_proposer_slashings") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("post_v1_validator_block") do (body: SignedBeaconBlock) -> bool:
|
||||
debug "post_v1_validator_block",
|
||||
slot = body.message.slot,
|
||||
prop_idx = body.message.proposer_index
|
||||
let head = node.doChecksAndGetCurrentHead(body.message.slot)
|
||||
rpcServer.rpc("post_v1_beacon_pool_proposer_slashings") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
if head.slot >= body.message.slot:
|
||||
raise newException(CatchableError,
|
||||
"Proposal is for a past slot: " & $body.message.slot)
|
||||
if head == await proposeSignedBlock(node, head, AttachedValidator(), body):
|
||||
raise newException(CatchableError, "Could not propose block")
|
||||
return true
|
||||
rpcServer.rpc("get_v1_beacon_pool_voluntary_exits") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_validator_attestation") do (
|
||||
slot: Slot, committee_index: CommitteeIndex) -> AttestationData:
|
||||
debug "get_v1_validator_attestation", slot = slot
|
||||
let
|
||||
head = node.doChecksAndGetCurrentHead(slot)
|
||||
epochRef = node.chainDag.getEpochRef(head, slot.epoch)
|
||||
return makeAttestationData(epochRef, head.atSlot(slot), committee_index.uint64)
|
||||
|
||||
rpcServer.rpc("get_v1_validator_aggregate_attestation") do (
|
||||
slot: Slot, attestation_data_root: Eth2Digest)-> Attestation:
|
||||
debug "get_v1_validator_aggregate_attestation"
|
||||
let res = node.attestationPool[].getAggregatedAttestation(slot, attestation_data_root)
|
||||
if res.isSome:
|
||||
return res.get
|
||||
raise newException(CatchableError, "Could not retrieve an aggregated attestation")
|
||||
|
||||
rpcServer.rpc("post_v1_validator_aggregate_and_proofs") do (
|
||||
payload: SignedAggregateAndProof) -> bool:
|
||||
debug "post_v1_validator_aggregate_and_proofs"
|
||||
node.network.broadcast(node.topicAggregateAndProofs, payload)
|
||||
notice "Aggregated attestation sent",
|
||||
attestation = shortLog(payload.message.aggregate)
|
||||
|
||||
rpcServer.rpc("get_v1_validator_duties_attester") do (
|
||||
epoch: Epoch, public_keys: seq[ValidatorPubKey]) -> seq[AttesterDuties]:
|
||||
debug "get_v1_validator_duties_attester", epoch = epoch
|
||||
let
|
||||
head = node.doChecksAndGetCurrentHead(epoch)
|
||||
epochRef = node.chainDag.getEpochRef(head, epoch)
|
||||
committees_per_slot = get_committee_count_per_slot(epochRef)
|
||||
for i in 0 ..< SLOTS_PER_EPOCH:
|
||||
let slot = compute_start_slot_at_epoch(epoch) + i
|
||||
for committee_index in 0'u64..<committees_per_slot:
|
||||
let committee = get_beacon_committee(
|
||||
epochRef, slot, committee_index.CommitteeIndex)
|
||||
for index_in_committee, validatorIdx in committee:
|
||||
if validatorIdx < epochRef.validator_keys.len.ValidatorIndex:
|
||||
let curr_val_pubkey = epochRef.validator_keys[validatorIdx].initPubKey
|
||||
if public_keys.findIt(it == curr_val_pubkey) != -1:
|
||||
result.add((public_key: curr_val_pubkey,
|
||||
validator_index: validatorIdx,
|
||||
committee_index: committee_index.CommitteeIndex,
|
||||
committee_length: committee.lenu64,
|
||||
validator_committee_index: index_in_committee.uint64,
|
||||
slot: slot))
|
||||
|
||||
rpcServer.rpc("get_v1_validator_duties_proposer") do (
|
||||
epoch: Epoch) -> seq[ValidatorPubkeySlotPair]:
|
||||
debug "get_v1_validator_duties_proposer", epoch = epoch
|
||||
let
|
||||
head = node.doChecksAndGetCurrentHead(epoch)
|
||||
epochRef = node.chainDag.getEpochRef(head, epoch)
|
||||
for i in 0 ..< SLOTS_PER_EPOCH:
|
||||
if epochRef.beacon_proposers[i].isSome():
|
||||
result.add((public_key: epochRef.beacon_proposers[i].get()[1].initPubKey(),
|
||||
slot: compute_start_slot_at_epoch(epoch) + i))
|
||||
|
||||
rpcServer.rpc("post_v1_validator_beacon_committee_subscriptions") do (
|
||||
committee_index: CommitteeIndex, slot: Slot, aggregator: bool,
|
||||
validator_pubkey: ValidatorPubKey, slot_signature: ValidatorSig) -> bool:
|
||||
debug "post_v1_validator_beacon_committee_subscriptions"
|
||||
raise newException(CatchableError, "Not implemented")
|
||||
rpcServer.rpc("post_v1_beacon_pool_voluntary_exits") do () -> JsonNode:
|
||||
unimplemented()
|
28
beacon_chain/rpc/config_api.nim
Normal file
28
beacon_chain/rpc/config_api.nim
Normal file
@ -0,0 +1,28 @@
|
||||
# Copyright (c) 2018-2020 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.
|
||||
|
||||
import
|
||||
json_rpc/[rpcserver, jsonmarshal],
|
||||
chronicles,
|
||||
../beacon_node_common
|
||||
|
||||
logScope: topics = "configapi"
|
||||
|
||||
type
|
||||
RpcServer = RpcHttpServer
|
||||
|
||||
template unimplemented() =
|
||||
raise (ref CatchableError)(msg: "Unimplemented")
|
||||
|
||||
proc installConfigApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("get_v1_config_fork_schedule") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_config_spec") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_config_deposit_contract") do () -> JsonNode:
|
||||
unimplemented()
|
22
beacon_chain/rpc/debug_api.nim
Normal file
22
beacon_chain/rpc/debug_api.nim
Normal file
@ -0,0 +1,22 @@
|
||||
import
|
||||
std/sequtils,
|
||||
json_rpc/[rpcserver, jsonmarshal],
|
||||
chronicles,
|
||||
../beacon_node_common, ../eth2_json_rpc_serialization, ../eth2_network, ../peer_pool, ../version,
|
||||
../spec/[datatypes, digest, presets],
|
||||
./rpc_utils
|
||||
|
||||
logScope: topics = "debugapi"
|
||||
|
||||
type
|
||||
RpcServer = RpcHttpServer
|
||||
|
||||
proc installDebugApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("get_v1_debug_beacon_states_stateId") do (
|
||||
stateId: string) -> BeaconState:
|
||||
withStateForStateId(stateId):
|
||||
return state
|
||||
|
||||
rpcServer.rpc("get_v1_debug_beacon_heads") do (
|
||||
stateId: string) -> seq[tuple[root: Eth2Digest, slot: Slot]]:
|
||||
return node.chainDag.heads.mapIt((it.root, it.slot))
|
22
beacon_chain/rpc/event_api.nim
Normal file
22
beacon_chain/rpc/event_api.nim
Normal file
@ -0,0 +1,22 @@
|
||||
# Copyright (c) 2018-2020 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.
|
||||
|
||||
import
|
||||
json_rpc/[rpcserver, jsonmarshal],
|
||||
chronicles,
|
||||
../beacon_node_common
|
||||
|
||||
logScope: topics = "eventapi"
|
||||
|
||||
type
|
||||
RpcServer = RpcHttpServer
|
||||
|
||||
template unimplemented() =
|
||||
raise (ref CatchableError)(msg: "Unimplemented")
|
||||
|
||||
proc installEventApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("get_v1_events") do () -> JsonNode:
|
||||
unimplemented()
|
84
beacon_chain/rpc/nimbus_api.nim
Normal file
84
beacon_chain/rpc/nimbus_api.nim
Normal file
@ -0,0 +1,84 @@
|
||||
# Copyright (c) 2018-2020 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.
|
||||
|
||||
import
|
||||
std/strutils,
|
||||
stew/shims/macros,
|
||||
stew/byteutils,
|
||||
json_rpc/[rpcserver, jsonmarshal],
|
||||
|
||||
../beacon_node_common, ../eth2_network,
|
||||
../spec/[digest, datatypes, presets]
|
||||
|
||||
logScope: topics = "nimbusapi"
|
||||
|
||||
type
|
||||
RpcServer = RpcHttpServer
|
||||
|
||||
proc installNimbusApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("getBeaconHead") do () -> Slot:
|
||||
return node.chainDag.head.slot
|
||||
|
||||
rpcServer.rpc("getChainHead") do () -> JsonNode:
|
||||
let
|
||||
head = node.chainDag.head
|
||||
finalized = node.chainDag.headState.data.data.finalized_checkpoint
|
||||
justified = node.chainDag.headState.data.data.current_justified_checkpoint
|
||||
return %* {
|
||||
"head_slot": head.slot,
|
||||
"head_block_root": head.root.data.toHex(),
|
||||
"finalized_slot": finalized.epoch * SLOTS_PER_EPOCH,
|
||||
"finalized_block_root": finalized.root.data.toHex(),
|
||||
"justified_slot": justified.epoch * SLOTS_PER_EPOCH,
|
||||
"justified_block_root": justified.root.data.toHex(),
|
||||
}
|
||||
|
||||
rpcServer.rpc("getSyncing") do () -> bool:
|
||||
let
|
||||
wallSlot = currentSlot(node)
|
||||
headSlot = node.chainDag.head.slot
|
||||
# FIXME: temporary hack: If more than 1 block away from expected head, then we are "syncing"
|
||||
return (headSlot + 1) < wallSlot
|
||||
|
||||
rpcServer.rpc("getNetworkPeerId") do () -> string:
|
||||
return $publicKey(node.network)
|
||||
|
||||
rpcServer.rpc("getNetworkPeers") do () -> seq[string]:
|
||||
for peerId, peer in node.network.peerPool:
|
||||
result.add $peerId
|
||||
|
||||
rpcServer.rpc("getSpecPreset") do () -> JsonNode:
|
||||
var res = newJObject()
|
||||
genStmtList:
|
||||
for presetValue in PresetValue:
|
||||
if presetValue notin ignoredValues + runtimeValues:
|
||||
let
|
||||
settingSym = ident($presetValue)
|
||||
settingKey = newLit(toLowerAscii($presetValue))
|
||||
let f = quote do:
|
||||
res[`settingKey`] = %(presets.`settingSym`)
|
||||
yield f
|
||||
|
||||
for field, value in fieldPairs(node.config.runtimePreset):
|
||||
res[field] = when value isnot Version: %value
|
||||
else: %value.toUInt64
|
||||
|
||||
return res
|
||||
|
||||
rpcServer.rpc("peers") do () -> JsonNode:
|
||||
var res = newJObject()
|
||||
var peers = newJArray()
|
||||
for id, peer in node.network.peerPool:
|
||||
peers.add(
|
||||
%(
|
||||
info: shortLog(peer.info),
|
||||
connectionState: $peer.connectionState,
|
||||
score: peer.score,
|
||||
)
|
||||
)
|
||||
res.add("peers", peers)
|
||||
|
||||
return res
|
44
beacon_chain/rpc/node_api.nim
Normal file
44
beacon_chain/rpc/node_api.nim
Normal file
@ -0,0 +1,44 @@
|
||||
import
|
||||
chronicles,
|
||||
json_rpc/[rpcserver, jsonmarshal],
|
||||
|
||||
../beacon_node_common, ../eth2_network,
|
||||
../peer_pool, ../version,
|
||||
../spec/[datatypes, digest, presets],
|
||||
../spec/eth2_apis/callsigs_types
|
||||
|
||||
logScope: topics = "nodeapi"
|
||||
|
||||
type
|
||||
RpcServer = RpcHttpServer
|
||||
|
||||
template unimplemented() =
|
||||
raise (ref CatchableError)(msg: "Unimplemented")
|
||||
|
||||
proc installNodeApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("get_v1_node_identity") do () -> NodeIdentityTuple:
|
||||
# TODO rest of fields
|
||||
return (
|
||||
peer_id: node.network.peerId(),
|
||||
enr: node.network.enrRecord(),
|
||||
p2p_addresses: newSeq[MultiAddress](0),
|
||||
discovery_addresses: newSeq[MultiAddress](0),
|
||||
metadata: (0'u64, "")
|
||||
)
|
||||
|
||||
rpcServer.rpc("get_v1_node_peers") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_node_peers_peerId") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_node_version") do () -> JsonNode:
|
||||
return %{
|
||||
"version": "Nimbus/" & fullVersionStr
|
||||
}
|
||||
|
||||
rpcServer.rpc("get_v1_node_syncing") do () -> JsonNode:
|
||||
unimplemented()
|
||||
|
||||
rpcServer.rpc("get_v1_node_health") do () -> JsonNode:
|
||||
unimplemented()
|
68
beacon_chain/rpc/rpc_utils.nim
Normal file
68
beacon_chain/rpc/rpc_utils.nim
Normal file
@ -0,0 +1,68 @@
|
||||
import
|
||||
std/[strutils, parseutils],
|
||||
stew/byteutils,
|
||||
../beacon_node_common, ../validator_duties,
|
||||
../block_pools/[block_pools_types, chain_dag],
|
||||
../spec/[datatypes, digest, helpers]
|
||||
|
||||
export chain_dag
|
||||
|
||||
template withStateForStateId*(stateId: string, body: untyped): untyped =
|
||||
# TODO this can be optimized for the "head" case since that should be most common
|
||||
node.chainDag.withState(node.chainDag.tmpState,
|
||||
node.stateIdToBlockSlot(stateId)):
|
||||
body
|
||||
|
||||
proc toBlockSlot*(blckRef: BlockRef): BlockSlot =
|
||||
blckRef.atSlot(blckRef.slot)
|
||||
|
||||
proc parseRoot*(str: string): Eth2Digest =
|
||||
return Eth2Digest(data: hexToByteArray[32](str))
|
||||
|
||||
func checkEpochToSlotOverflow*(epoch: Epoch) =
|
||||
const maxEpoch = compute_epoch_at_slot(not 0'u64)
|
||||
if epoch >= maxEpoch:
|
||||
raise newException(
|
||||
ValueError, "Requesting epoch for which slot would overflow")
|
||||
|
||||
proc doChecksAndGetCurrentHead*(node: BeaconNode, slot: Slot): BlockRef =
|
||||
result = node.chainDag.head
|
||||
if not node.isSynced(result):
|
||||
raise newException(CatchableError, "Cannot fulfill request until node is synced")
|
||||
# TODO for now we limit the requests arbitrarily by up to 2 epochs into the future
|
||||
if result.slot + uint64(2 * SLOTS_PER_EPOCH) < slot:
|
||||
raise newException(CatchableError, "Requesting way ahead of the current head")
|
||||
|
||||
proc doChecksAndGetCurrentHead*(node: BeaconNode, epoch: Epoch): BlockRef =
|
||||
checkEpochToSlotOverflow(epoch)
|
||||
node.doChecksAndGetCurrentHead(epoch.compute_start_slot_at_epoch)
|
||||
|
||||
proc getBlockSlotFromString*(node: BeaconNode, slot: string): BlockSlot =
|
||||
if slot.len == 0:
|
||||
raise newException(ValueError, "Empty slot number not allowed")
|
||||
var parsed: BiggestUInt
|
||||
if parseBiggestUInt(slot, parsed) != slot.len:
|
||||
raise newException(ValueError, "Not a valid slot number")
|
||||
let head = node.doChecksAndGetCurrentHead(parsed.Slot)
|
||||
return head.atSlot(parsed.Slot)
|
||||
|
||||
proc stateIdToBlockSlot*(node: BeaconNode, stateId: string): BlockSlot =
|
||||
result = case stateId:
|
||||
of "head":
|
||||
node.chainDag.head.toBlockSlot()
|
||||
of "genesis":
|
||||
node.chainDag.getGenesisBlockSlot()
|
||||
of "finalized":
|
||||
node.chainDag.finalizedHead
|
||||
of "justified":
|
||||
node.chainDag.head.atEpochStart(
|
||||
node.chainDag.headState.data.data.current_justified_checkpoint.epoch)
|
||||
else:
|
||||
if stateId.startsWith("0x"):
|
||||
let blckRoot = parseRoot(stateId)
|
||||
let blckRef = node.chainDag.getRef(blckRoot)
|
||||
if blckRef.isNil:
|
||||
raise newException(CatchableError, "Block not found")
|
||||
blckRef.toBlockSlot()
|
||||
else:
|
||||
node.getBlockSlotFromString(stateId)
|
119
beacon_chain/rpc/validator_api.nim
Normal file
119
beacon_chain/rpc/validator_api.nim
Normal file
@ -0,0 +1,119 @@
|
||||
# beacon_chain
|
||||
# Copyright (c) 2018-2020 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.
|
||||
|
||||
import
|
||||
# Standard library
|
||||
std/[tables],
|
||||
|
||||
# Nimble packages
|
||||
stew/[objects],
|
||||
json_rpc/[rpcserver, jsonmarshal],
|
||||
chronicles,
|
||||
|
||||
# Local modules
|
||||
../spec/[datatypes, digest, crypto, helpers],
|
||||
../spec/eth2_apis/callsigs_types,
|
||||
../block_pools/[chain_dag, spec_cache], ../ssz/merkleization,
|
||||
../beacon_node_common, ../beacon_node_types, ../attestation_pool,
|
||||
../validator_duties, ../eth2_network,
|
||||
../eth2_json_rpc_serialization,
|
||||
./rpc_utils
|
||||
|
||||
logScope: topics = "valapi"
|
||||
|
||||
type
|
||||
RpcServer* = RpcHttpServer
|
||||
|
||||
proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
|
||||
rpcServer.rpc("get_v1_validator_block") do (
|
||||
slot: Slot, graffiti: GraffitiBytes, randao_reveal: ValidatorSig) -> BeaconBlock:
|
||||
debug "get_v1_validator_block", slot = slot
|
||||
let head = node.doChecksAndGetCurrentHead(slot)
|
||||
let proposer = node.chainDag.getProposer(head, slot)
|
||||
if proposer.isNone():
|
||||
raise newException(CatchableError, "could not retrieve block for slot: " & $slot)
|
||||
let message = makeBeaconBlockForHeadAndSlot(
|
||||
node, randao_reveal, proposer.get()[0], graffiti, head, slot)
|
||||
if message.isNone():
|
||||
raise newException(CatchableError, "could not retrieve block for slot: " & $slot)
|
||||
return message.get()
|
||||
|
||||
rpcServer.rpc("post_v1_validator_block") do (body: SignedBeaconBlock) -> bool:
|
||||
debug "post_v1_validator_block",
|
||||
slot = body.message.slot,
|
||||
prop_idx = body.message.proposer_index
|
||||
let head = node.doChecksAndGetCurrentHead(body.message.slot)
|
||||
|
||||
if head.slot >= body.message.slot:
|
||||
raise newException(CatchableError,
|
||||
"Proposal is for a past slot: " & $body.message.slot)
|
||||
if head == await proposeSignedBlock(node, head, AttachedValidator(), body):
|
||||
raise newException(CatchableError, "Could not propose block")
|
||||
return true
|
||||
|
||||
rpcServer.rpc("get_v1_validator_attestation") do (
|
||||
slot: Slot, committee_index: CommitteeIndex) -> AttestationData:
|
||||
debug "get_v1_validator_attestation", slot = slot
|
||||
let
|
||||
head = node.doChecksAndGetCurrentHead(slot)
|
||||
epochRef = node.chainDag.getEpochRef(head, slot.epoch)
|
||||
return makeAttestationData(epochRef, head.atSlot(slot), committee_index.uint64)
|
||||
|
||||
rpcServer.rpc("get_v1_validator_aggregate_attestation") do (
|
||||
slot: Slot, attestation_data_root: Eth2Digest)-> Attestation:
|
||||
debug "get_v1_validator_aggregate_attestation"
|
||||
let res = node.attestationPool[].getAggregatedAttestation(slot, attestation_data_root)
|
||||
if res.isSome:
|
||||
return res.get
|
||||
raise newException(CatchableError, "Could not retrieve an aggregated attestation")
|
||||
|
||||
rpcServer.rpc("post_v1_validator_aggregate_and_proofs") do (
|
||||
payload: SignedAggregateAndProof) -> bool:
|
||||
debug "post_v1_validator_aggregate_and_proofs"
|
||||
node.network.broadcast(node.topicAggregateAndProofs, payload)
|
||||
notice "Aggregated attestation sent",
|
||||
attestation = shortLog(payload.message.aggregate)
|
||||
|
||||
rpcServer.rpc("get_v1_validator_duties_attester") do (
|
||||
epoch: Epoch, public_keys: seq[ValidatorPubKey]) -> seq[AttesterDuties]:
|
||||
debug "get_v1_validator_duties_attester", epoch = epoch
|
||||
let
|
||||
head = node.doChecksAndGetCurrentHead(epoch)
|
||||
epochRef = node.chainDag.getEpochRef(head, epoch)
|
||||
committees_per_slot = get_committee_count_per_slot(epochRef)
|
||||
for i in 0 ..< SLOTS_PER_EPOCH:
|
||||
let slot = compute_start_slot_at_epoch(epoch) + i
|
||||
for committee_index in 0'u64..<committees_per_slot:
|
||||
let committee = get_beacon_committee(
|
||||
epochRef, slot, committee_index.CommitteeIndex)
|
||||
for index_in_committee, validatorIdx in committee:
|
||||
if validatorIdx < epochRef.validator_keys.len.ValidatorIndex:
|
||||
let curr_val_pubkey = epochRef.validator_keys[validatorIdx].initPubKey
|
||||
if public_keys.findIt(it == curr_val_pubkey) != -1:
|
||||
result.add((public_key: curr_val_pubkey,
|
||||
validator_index: validatorIdx,
|
||||
committee_index: committee_index.CommitteeIndex,
|
||||
committee_length: committee.lenu64,
|
||||
validator_committee_index: index_in_committee.uint64,
|
||||
slot: slot))
|
||||
|
||||
rpcServer.rpc("get_v1_validator_duties_proposer") do (
|
||||
epoch: Epoch) -> seq[ValidatorPubkeySlotPair]:
|
||||
debug "get_v1_validator_duties_proposer", epoch = epoch
|
||||
let
|
||||
head = node.doChecksAndGetCurrentHead(epoch)
|
||||
epochRef = node.chainDag.getEpochRef(head, epoch)
|
||||
for i in 0 ..< SLOTS_PER_EPOCH:
|
||||
if epochRef.beacon_proposers[i].isSome():
|
||||
result.add((public_key: epochRef.beacon_proposers[i].get()[1].initPubKey(),
|
||||
slot: compute_start_slot_at_epoch(epoch) + i))
|
||||
|
||||
rpcServer.rpc("post_v1_validator_beacon_committee_subscriptions") do (
|
||||
committee_index: CommitteeIndex, slot: Slot, aggregator: bool,
|
||||
validator_pubkey: ValidatorPubKey, slot_signature: ValidatorSig) -> bool:
|
||||
debug "post_v1_validator_beacon_committee_subscriptions"
|
||||
raise newException(CatchableError, "Not implemented")
|
@ -5,7 +5,9 @@ import
|
||||
# TODO for some reason "../[datatypes, digest, crypto]" results in "Error: cannot open file"
|
||||
../datatypes,
|
||||
../digest,
|
||||
../crypto
|
||||
../crypto,
|
||||
libp2p/[peerid, multiaddress],
|
||||
eth/p2p/discoveryv5/enr
|
||||
|
||||
type
|
||||
AttesterDuties* = tuple
|
||||
@ -42,3 +44,10 @@ type
|
||||
root: Eth2Digest
|
||||
canonical: bool
|
||||
header: SignedBeaconBlockHeader
|
||||
|
||||
NodeIdentityTuple* = tuple
|
||||
peer_id: PeerID
|
||||
enr: Record
|
||||
p2p_addresses: seq[MultiAddress]
|
||||
discovery_addresses: seq[MultiAddress]
|
||||
metadata: tuple[seq_number: uint64, attnets: string]
|
||||
|
Loading…
x
Reference in New Issue
Block a user