From ee2ebc96a89884009d7d39672cad533a7b7dc99c Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Tue, 27 Oct 2020 10:00:57 +0100 Subject: [PATCH] move rpc api to own folder, mimic upstream structure (#1905) also implements a few more endpoints --- beacon_chain/beacon_node.nim | 95 +------- beacon_chain/beacon_node_common.nim | 3 + beacon_chain/block_pools/chain_dag.nim | 2 +- beacon_chain/eth2_discovery.nim | 2 + beacon_chain/eth2_network.nim | 8 +- beacon_chain/nimbus_binary_common.nim | 2 +- beacon_chain/peer_pool.nim | 2 + .../{validator_api.nim => rpc/beacon_api.nim} | 225 ++++-------------- beacon_chain/rpc/config_api.nim | 28 +++ beacon_chain/rpc/debug_api.nim | 22 ++ beacon_chain/rpc/event_api.nim | 22 ++ beacon_chain/rpc/nimbus_api.nim | 84 +++++++ beacon_chain/rpc/node_api.nim | 44 ++++ beacon_chain/rpc/rpc_utils.nim | 68 ++++++ beacon_chain/rpc/validator_api.nim | 119 +++++++++ .../spec/eth2_apis/callsigs_types.nim | 11 +- 16 files changed, 463 insertions(+), 274 deletions(-) rename beacon_chain/{validator_api.nim => rpc/beacon_api.nim} (50%) create mode 100644 beacon_chain/rpc/config_api.nim create mode 100644 beacon_chain/rpc/debug_api.nim create mode 100644 beacon_chain/rpc/event_api.nim create mode 100644 beacon_chain/rpc/nimbus_api.nim create mode 100644 beacon_chain/rpc/node_api.nim create mode 100644 beacon_chain/rpc/rpc_utils.nim create mode 100644 beacon_chain/rpc/validator_api.nim diff --git a/beacon_chain/beacon_node.nim b/beacon_chain/beacon_node.nim index b5382937e..5bc34ab76 100644 --- a/beacon_chain/beacon_node.nim +++ b/beacon_chain/beacon_node.nim @@ -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 diff --git a/beacon_chain/beacon_node_common.nim b/beacon_chain/beacon_node_common.nim index 53caa92cc..fc8a4a873 100644 --- a/beacon_chain/beacon_node_common.nim +++ b/beacon_chain/beacon_node_common.nim @@ -76,3 +76,6 @@ template findIt*(s: openarray, predicate: untyped): int = res = i break res + +proc currentSlot*(node: BeaconNode): Slot = + node.beaconClock.now.slotOrZero diff --git a/beacon_chain/block_pools/chain_dag.nim b/beacon_chain/block_pools/chain_dag.nim index 314cc9190..23a8e1f43 100644 --- a/beacon_chain/block_pools/chain_dag.nim +++ b/beacon_chain/block_pools/chain_dag.nim @@ -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" diff --git a/beacon_chain/eth2_discovery.nim b/beacon_chain/eth2_discovery.nim index 3d1c92573..6125b0f08 100644 --- a/beacon_chain/eth2_discovery.nim +++ b/beacon_chain/eth2_discovery.nim @@ -6,6 +6,8 @@ import eth/keys, eth/p2p/discoveryv5/[enr, protocol, node], conf +export protocol, keys + type Eth2DiscoveryProtocol* = protocol.Protocol Eth2DiscoveryId* = NodeId diff --git a/beacon_chain/eth2_network.nim b/beacon_chain/eth2_network.nim index d364a1f13..49eabb1cc 100644 --- a/beacon_chain/eth2_network.nim +++ b/beacon_chain/eth2_network.nim @@ -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[] diff --git a/beacon_chain/nimbus_binary_common.nim b/beacon_chain/nimbus_binary_common.nim index 2021c4072..73d59e331 100644 --- a/beacon_chain/nimbus_binary_common.nim +++ b/beacon_chain/nimbus_binary_common.nim @@ -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 diff --git a/beacon_chain/peer_pool.nim b/beacon_chain/peer_pool.nim index 0edf469c6..9f940e076 100644 --- a/beacon_chain/peer_pool.nim +++ b/beacon_chain/peer_pool.nim @@ -1,6 +1,8 @@ import std/[tables, heapqueue] import chronos +export tables + type PeerType* = enum Incoming, Outgoing diff --git a/beacon_chain/validator_api.nim b/beacon_chain/rpc/beacon_api.nim similarity index 50% rename from beacon_chain/validator_api.nim rename to beacon_chain/rpc/beacon_api.nim index b2e3955b5..308d8c9ec 100644 --- a/beacon_chain/validator_api.nim +++ b/beacon_chain/rpc/beacon_api.nim @@ -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.. 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() diff --git a/beacon_chain/rpc/config_api.nim b/beacon_chain/rpc/config_api.nim new file mode 100644 index 000000000..400d2668c --- /dev/null +++ b/beacon_chain/rpc/config_api.nim @@ -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() diff --git a/beacon_chain/rpc/debug_api.nim b/beacon_chain/rpc/debug_api.nim new file mode 100644 index 000000000..1eb539897 --- /dev/null +++ b/beacon_chain/rpc/debug_api.nim @@ -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)) diff --git a/beacon_chain/rpc/event_api.nim b/beacon_chain/rpc/event_api.nim new file mode 100644 index 000000000..abf4ba13a --- /dev/null +++ b/beacon_chain/rpc/event_api.nim @@ -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() diff --git a/beacon_chain/rpc/nimbus_api.nim b/beacon_chain/rpc/nimbus_api.nim new file mode 100644 index 000000000..0f2ff2847 --- /dev/null +++ b/beacon_chain/rpc/nimbus_api.nim @@ -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 diff --git a/beacon_chain/rpc/node_api.nim b/beacon_chain/rpc/node_api.nim new file mode 100644 index 000000000..e041182a2 --- /dev/null +++ b/beacon_chain/rpc/node_api.nim @@ -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() diff --git a/beacon_chain/rpc/rpc_utils.nim b/beacon_chain/rpc/rpc_utils.nim new file mode 100644 index 000000000..bae202463 --- /dev/null +++ b/beacon_chain/rpc/rpc_utils.nim @@ -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) diff --git a/beacon_chain/rpc/validator_api.nim b/beacon_chain/rpc/validator_api.nim new file mode 100644 index 000000000..f59e81d24 --- /dev/null +++ b/beacon_chain/rpc/validator_api.nim @@ -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.. 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") diff --git a/beacon_chain/spec/eth2_apis/callsigs_types.nim b/beacon_chain/spec/eth2_apis/callsigs_types.nim index ff7198016..37e5cf6c7 100644 --- a/beacon_chain/spec/eth2_apis/callsigs_types.nim +++ b/beacon_chain/spec/eth2_apis/callsigs_types.nim @@ -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]