Rest API initial implementation.

This commit is contained in:
cheatfate 2021-03-17 20:46:45 +02:00 committed by zah
parent 3d701d8973
commit 72695dd62a
9 changed files with 1751 additions and 4 deletions

View File

@ -11,7 +11,7 @@ import
std/osproc,
# Nimble packages
chronos, json_rpc/servers/httpserver,
chronos, json_rpc/servers/httpserver, presto
# Local modules
./conf, ./beacon_clock, ./beacon_chain_db,
@ -47,6 +47,7 @@ type
eth1Monitor*: Eth1Monitor
beaconClock*: BeaconClock
rpcServer*: RpcServer
restServer*: RestServerRef
vcProcess*: Process
forkDigest*: ForkDigest
requestManager*: RequestManager

View File

@ -281,6 +281,21 @@ type
desc: "Listening address of the RPC server [=127.0.0.1]"
name: "rpc-address" }: ValidIpAddress
restEnabled* {.
defaultValue: false
desc: "Enable the REST server"
name: "rest" }: bool
restPort* {.
defaultValue: defaultEth2RpcPort
desc: "HTTP port for the REST service"
name: "rest-port" }: Port
restAddress* {.
defaultValue: defaultAdminListenAddress(config)
desc: "Listening address of the REST server"
name: "rest-address" }: ValidIpAddress
inProcessValidators* {.
defaultValue: true # the use of the nimbus_signing_process binary by default will be delayed until async I/O over stdin/stdout is developed for the child process.
desc: "Disable the push model (the beacon node tells a signing process with the private keys of the validators what to sign and when) and load the validators in the beacon node itself"

View File

@ -16,7 +16,7 @@ import
# Nimble packages
stew/[objects, byteutils, endians2, io2], stew/shims/macros,
chronos, confutils, metrics, metrics/chronos_httpserver,
chronicles, bearssl, blscurve,
chronicles, bearssl, blscurve, presto,
json_serialization/std/[options, sets, net], serialization/errors,
eth/[keys, async_utils], eth/net/nat,
@ -36,6 +36,8 @@ import
attestation_aggregation, validator_duties, validator_pool,
slashing_protection, keystore_management],
./sync/[sync_manager, sync_protocol, request_manager],
./rpc/[rest_utils, config_rest_api, debug_rest_api, node_rest_api,
beacon_rest_api],
./rpc/[beacon_api, config_api, debug_api, event_api, nimbus_api, node_api,
validator_api],
./spec/[
@ -62,6 +64,16 @@ type
template init(T: type RpcHttpServer, ip: ValidIpAddress, port: Port): T =
newRpcHttpServer([initTAddress(ip, port)])
template init(T: type RestServerRef, ip: ValidIpAddress, port: Port): T =
let address = initTAddress(ip, port)
let res = RestServerRef.new(getRouter(), address)
if res.isErr():
notice "Rest server could not be started", address = $address,
reason = res.error()
nil
else:
res.get()
# https://github.com/ethereum/eth2.0-metrics/blob/master/metrics.md#interop-metrics
declareGauge beacon_slot, "Latest slot of the beacon chain state"
declareGauge beacon_current_epoch, "Current epoch"
@ -292,6 +304,11 @@ proc init*(T: type BeaconNode,
else:
nil
let restServer = if config.restEnabled:
RestServerRef.init(config.restAddress, config.restPort)
else:
nil
let
netKeys = getPersistentNetKeys(rng[], config)
nickname = if config.nodeName == "auto": shortForm(netKeys)
@ -365,6 +382,7 @@ proc init*(T: type BeaconNode,
eth1Monitor: eth1Monitor,
beaconClock: beaconClock,
rpcServer: rpcServer,
restServer: restServer,
forkDigest: enrForkId.forkDigest,
topicBeaconBlocks: topicBeaconBlocks,
topicAggregateAndProofs: topicAggregateAndProofs,
@ -1170,6 +1188,15 @@ proc installRpcHandlers(rpcServer: RpcServer, node: BeaconNode) =
rpcServer.installValidatorApiHandlers(node)
except Exception as exc: raiseAssert exc.msg # TODO fix json-rpc
proc installRestHandlers(restServer: RestServerRef, node: BeaconNode) =
restServer.router.installBeaconApiHandlers(node)
restServer.router.installConfigApiHandlers(node)
restServer.router.installDebugApiHandlers(node)
# restServer.router.installEventApiHandlers(node)
# restServer.router.installNimbusApiHandlers(node)
restServer.router.installNodeApiHandlers(node)
# restServer.router.installValidatorApiHandlers(node)
proc installMessageValidators(node: BeaconNode) =
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#attestations-and-aggregation
# These validators stay around the whole time, regardless of which specific
@ -1230,10 +1257,14 @@ proc run*(node: BeaconNode) {.raises: [Defect, CatchableError].} =
# it might have been set to "Stopping" with Ctrl+C
bnStatus = BeaconNodeStatus.Running
if node.rpcServer != nil:
if not(isNil(node.rpcServer)):
node.rpcServer.installRpcHandlers(node)
node.rpcServer.start()
if not(isNil(node.restServer)):
node.restServer.installRestHandlers(node)
node.restServer.start()
node.installMessageValidators()
let startTime = node.beaconClock.now()

View File

@ -0,0 +1,822 @@
# 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/[parseutils, typetraits, sequtils, strutils, deques, sets, options],
stew/[results, base10],
chronicles,
nimcrypto/utils as ncrutils,
../beacon_node_common, ../eth2_network, ../validator_duties,
../block_pools/chain_dag, ../exit_pool,
../spec/[crypto, digest, validator, network],
../ssz/merkleization,
./rest_utils
import ../spec/datatypes except readValue, writeValue
logScope: topics = "rest_beaconapi"
type
ValidatorTuple = tuple
index: ValidatorIndex
balance: string
status: string
validator: Validator
ValidatorBalanceTuple = tuple
index: ValidatorIndex
balance: string
StateCommitteeTuple = tuple
index: CommitteeIndex
slot: Slot
validators: seq[ValidatorIndex]
# # BlockHeaderMessageTuple = tuple
# # slot: Slot
# # proposer_index: string
# # parent_root: Eth2Digest
# # state_root: Eth2Digest
# # body_root: Eth2Digest
# # SignedHeaderMessageTuple = tuple
# # message: BlockHeaderMessageTuple
# # signature: ValidatorSig
# # BlockHeaderTuple = tuple
# # root: Eth2Digest
# # canonical: bool
# # header: SignedHeaderMessageTuple
proc validateFilter(filters: seq[ValidatorFilter]): Result[ValidatorFilter,
cstring] =
var res: ValidatorFilter
for item in filters:
if res * item != {}:
return err("Validator status must be unique")
res.incl(item)
if res == {}:
res = {ValidatorFilterKind.PendingInitialized,
ValidatorFilterKind.PendingQueued,
ValidatorFilterKind.ActiveOngoing,
ValidatorFilterKind.ActiveExiting,
ValidatorFilterKind.ActiveSlashed,
ValidatorFilterKind.ExitedUnslashed,
ValidatorFilterKind.ExitedSlashed,
ValidatorFilterKind.WithdrawalPossible,
ValidatorFilterKind.WithdrawalDone}
ok(res)
proc getStatus(validator: Validator,
current_epoch: Epoch): Result[ValidatorFilterKind, cstring] =
if validator.activation_epoch > current_epoch:
# pending
if validator.activation_eligibility_epoch == FAR_FUTURE_EPOCH:
ok(ValidatorFilterKind.PendingInitialized)
else:
# validator.activation_eligibility_epoch < FAR_FUTURE_EPOCH:
ok(ValidatorFilterKind.PendingQueued)
elif (validator.activation_epoch <= current_epoch) and
(current_epoch < validator.exit_epoch):
# active
if validator.exit_epoch == FAR_FUTURE_EPOCH:
ok(ValidatorFilterKind.ActiveOngoing)
elif not validator.slashed:
# validator.exit_epoch < FAR_FUTURE_EPOCH
ok(ValidatorFilterKind.ActiveExiting)
else:
# validator.exit_epoch < FAR_FUTURE_EPOCH and validator.slashed:
ok(ValidatorFilterKind.ActiveSlashed)
elif (validator.exit_epoch <= current_epoch) and
(current_epoch < validator.withdrawable_epoch):
# exited
if not validator.slashed:
ok(ValidatorFilterKind.ExitedUnslashed)
else:
# validator.slashed
ok(ValidatorFilterKind.ExitedSlashed)
elif validator.withdrawable_epoch <= current_epoch:
# withdrawal
if validator.effective_balance != 0:
ok(ValidatorFilterKind.WithdrawalPossible)
else:
# validator.effective_balance == 0
ok(ValidatorFilterKind.WithdrawalDone)
else:
err("Invalid validator status")
proc toString*(digest: Eth2Digest): string =
"0x" & ncrutils.toHex(digest.data, true)
proc toString*(version: Version): string =
"0x" & ncrutils.toHex(cast[array[4, byte]](version))
proc toString*(kind: ValidatorFilterKind): string =
case kind
of ValidatorFilterKind.PendingInitialized:
"pending_initialized"
of ValidatorFilterKind.PendingQueued:
"pending_queued"
of ValidatorFilterKind.ActiveOngoing:
"active_ongoing"
of ValidatorFilterKind.ActiveExiting:
"active_exiting"
of ValidatorFilterKind.ActiveSlashed:
"active_slashed"
of ValidatorFilterKind.ExitedUnslashed:
"exited_unslashed"
of ValidatorFilterKind.ExitedSlashed:
"exited_slashed"
of ValidatorFilterKind.WithdrawalPossible:
"withdrawal_possible"
of ValidatorFilterKind.WithdrawalDone:
"withdrawal_done"
proc `%`*(s: Epoch): JsonNode = newJString(Base10.toString(uint64(s)))
proc `%`*(s: Slot): JsonNode = newJString(Base10.toString(uint64(s)))
proc `%`*(s: uint64): JsonNode = newJString(Base10.toString(s))
proc `%`*(s: ValidatorIndex): JsonNode = newJString(Base10.toString(uint64(s)))
proc `%`*(s: CommitteeIndex): JsonNode = newJString(Base10.toString(uint64(s)))
proc `%`*(s: Checkpoint): JsonNode = %(epoch: s.epoch, root: s.root)
proc `%`*(s: GraffitiBytes): JsonNode =
newJString("0x" & ncrutils.toHex(distinctBase(s), true))
proc `%`*(s: ValidatorSig): JsonNode =
newJString("0x" & ncrutils.toHex(toRaw(s), true))
proc `%`*(pubkey: ValidatorPubKey): JsonNode =
newJString("0x" & ncrutils.toHex(toRaw(pubkey), true))
proc `%`*(digest: Eth2Digest): JsonNode =
newJString("0x" & ncrutils.toHex(digest.data, true))
proc `%`*(bitlist: BitList): JsonNode =
newJString("0x" & ncrutils.toHex(seq[byte](BitSeq(bitlist)), true))
proc `%`*(s: Validator): JsonNode =
let activation_eligibility_epoch =
if s.activation_eligibility_epoch < 1:
FarFutureEpochString
else:
Base10.toString(uint64(s.activation_eligibility_epoch))
let activation_epoch =
if s.activation_epoch < 1:
FarFutureEpochString
else:
Base10.toString(uint64(s.activation_epoch))
let exit_epoch =
if s.exit_epoch < 1:
FarFutureEpochString
else:
Base10.toString(uint64(s.exit_epoch))
let withdrawable_epoch =
if s.withdrawable_epoch < 1:
FarFutureEpochString
else:
Base10.toString(uint64(s.withdrawable_epoch))
%(
pubkey: s.pubkey,
withdrawal_credentials: s.withdrawal_credentials,
effective_balance: Base10.toString(s.effective_balance),
slashed: s.slashed,
activation_eligibility_epoch: activation_eligibility_epoch,
activation_epoch: activation_epoch,
exit_epoch: exit_epoch,
withdrawable_epoch: withdrawable_epoch
)
proc `%`*(s: AttestationData): JsonNode =
%(
slot: s.slot,
index: Base10.toString(s.index),
beacon_block_root: s.beacon_block_root,
source: s.source,
target: s.target
)
proc `%`*(s: TrustedAttestation): JsonNode =
%(
aggregation_bits: s.aggregation_bits,
signature: cast[ValidatorSig](s.signature),
data: s.data
)
proc `%`*(s: Attestation): JsonNode =
%(
aggregation_bits: s.aggregation_bits,
signature: s.signature,
data: s.data
)
proc `%`*(s: VoluntaryExit): JsonNode =
%(epoch: s.epoch, validator_index: Base10.toString(s.validator_index))
proc `%`*(s: SignedVoluntaryExit): JsonNode =
%(message: s.message, signature: s.signature)
proc `%`*(s: DepositData): JsonNode =
%(
pubkey: s.pubkey,
withdrawal_credentials: s.withdrawal_credentials,
amount: Base10.toString(s.amount),
signature: s.signature
)
proc `%`*(s: Deposit): JsonNode =
%(proof: s.proof, data: s.data)
proc `%`*(s: BeaconBlockHeader): JsonNode =
%(
slot: s.slot,
proposer_index: Base10.toString(s.proposer_index),
parent_root: s.parent_root,
state_root: s.state_root,
body_root: s.body_root,
)
proc `%`*(s: SignedBeaconBlockHeader): JsonNode =
%(message: s.message, signature: s.signature)
proc `%`*(s: ProposerSlashing): JsonNode =
%(signed_header_1: s.signed_header_1, signed_header_2: s.signed_header_2)
proc `%`*(s: IndexedAttestation): JsonNode =
%(
attesting_indices: s.attesting_indices,
data: s.data,
signature: s.signature
)
proc `%`*(s: AttesterSlashing): JsonNode =
%(attestation_1: s.attestation_1, attestation_2: s.attestation_2)
proc `%`*(s: Eth1Data): JsonNode =
%(
deposit_root: s.deposit_root,
deposit_count: Base10.toString(s.deposit_count),
block_hash: s.block_hash
)
proc `%`*(s: TrustedBeaconBlockBody): JsonNode =
%(
randao_reveal: cast[ValidatorSig](s.randao_reveal),
graffiti: s.graffiti,
proposer_slashings: s.proposer_slashings,
attester_slashings: s.attester_slashings,
attestations: s.attestations,
deposits: s.deposits,
voluntary_exits: s.voluntary_exits
)
proc `%`*(s: TrustedBeaconBlock): JsonNode =
%(
slot: s.slot,
proposer_index: Base10.toString(s.proposer_index),
parent_root: s.parent_root,
state_root: s.state_root,
body: s.body
)
proc `%`*(s: TrustedSignedBeaconBlock): JsonNode =
%(message: s.message, signature: cast[ValidatorSig](s.signature))
proc readValue*(reader: var JsonReader, value: var ValidatorSig)
{.raises: [IOError, SerializationError, Defect].} =
let hexValue = reader.readValue(string)
let res = ValidatorSig.fromHex(hexValue)
if res.isOk():
value = res.get()
else:
reader.raiseUnexpectedValue($res.error())
proc readValue*(reader: var JsonReader, value: var Epoch)
{.raises: [IOError, SerializationError, Defect].} =
let svalue = reader.readValue(string)
let res = Base10.decode(uint64, svalue)
if res.isOk():
value = Epoch(res.get())
else:
reader.raiseUnexpectedValue($res.error())
proc readValue*(reader: var JsonReader, value: var Slot)
{.raises: [IOError, SerializationError, Defect].} =
let svalue = reader.readValue(string)
let res = Base10.decode(uint64, svalue)
if res.isOk():
value = Slot(res.get())
else:
reader.raiseUnexpectedValue($res.error())
# proc readValue*(reader: var JsonReader, value: var ValidatorIndex)
# {.raises: [IOError, SerializationError, Defect].} =
# let svalue = reader.readValue(string)
# let res = Base10.decode(uint64, svalue)
# if res.isOk():
# let v = res.get()
# if v < VALIDATOR_REGISTRY_LIMIT:
# value = ValidatorIndex(v)
# else:
# reader.raiseUnexpectedValue(
# "Validator index is bigger then VALIDATOR_REGISTRY_LIMIT")
# else:
# reader.raiseUnexpectedValue($res.error())
proc decodeBody*[T](t: typedesc[T],
body: ContentBody): Result[T, cstring] =
if body.contentType != "application/json":
return err("Unsupported content type")
warn "Decoding data", data = cast[string](body.data)
let data =
try:
Json.decode(cast[string](body.data), T)
except SerializationError as exc:
warn "Error happens while processing json", errMsg = exc.formatMsg("tmp.nim")
return err("Unable to process data")
except CatchableError as exc:
warn "Error happens while parsing json", exc = exc.name, excMsg = exc.msg
return err("Unable to parse application/json data")
ok(data)
proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getGenesis
router.api(MethodGet, "/api/eth/v1/beacon/genesis") do () -> RestApiResponse:
return RestApiResponse.jsonResponse(
%(
genesis_time: toString(node.chainDag.headState.data.data.genesis_time),
genesis_validators_root:
node.chainDag.headState.data.data.genesis_validators_root,
genesis_fork_version: node.runtimePreset.GENESIS_FORK_VERSION
)
)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getStateRoot
router.api(MethodGet, "/api/eth/v1/beacon/states/{state_id}/root") do (
state_id: StateIdent) -> RestApiResponse:
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
return RestApiResponse.jsonResponse(%(root: hashedState().root))
return RestApiResponse.jsonError(Http500, "Internal server error")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getStateFork
router.api(MethodGet, "/api/eth/v1/beacon/states/{state_id}/fork") do (
state_id: StateIdent) -> RestApiResponse:
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
return RestApiResponse.jsonResponse(
%(
previous_version: state().fork.previous_version,
current_version: state().fork.current_version,
epoch: state().fork.epoch
)
)
return RestApiResponse.jsonError(Http500, "Internal server error")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getStateFinalityCheckpoints
router.api(MethodGet,
"/api/eth/v1/beacon/states/{state_id}/finality_checkpoints") do (
state_id: StateIdent) -> RestApiResponse:
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
return RestApiResponse.jsonResponse(
%(
previous_justified: state().previous_justified_checkpoint,
current_justified: state().current_justified_checkpoint,
finalized: state().finalized_checkpoint
)
)
return RestApiResponse.jsonError(Http500, "Internal server error")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getStateValidators
router.api(MethodGet, "/api/eth/v1/beacon/states/{state_id}/validators") do (
state_id: StateIdent, id: seq[ValidatorIdent],
status: seq[ValidatorFilter]) -> RestApiResponse:
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
let validatorIds =
block:
if id.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid validator identifier(s)")
id.get()
let validatorsMask =
block:
if status.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid validator status(es)")
let res = validateFilter(status.get())
if res.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid validator status value",
$res.error())
res.get()
let (keySet, indexSet) =
block:
var res1: HashSet[ValidatorPubKey]
var res2: HashSet[ValidatorIndex]
for item in validatorIds:
case item.kind
of ValidatorQueryKind.Key:
if item.key in res1:
return RestApiResponse.jsonError(Http400,
"Only unique validator keys allowed")
res1.incl(item.key)
of ValidatorQueryKind.Index:
if item.index in res2:
return RestApiResponse.jsonError(Http400,
"Only unique validator indexes allowed")
res2.incl(item.index)
(res1, res2)
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
let current_epoch = get_current_epoch(node.chainDag.headState.data.data)
var res: seq[ValidatorTuple]
for index, validator in state().validators.pairs():
let r1 =
if len(keySet) == 0:
true
else:
(validator.pubkey in keySet)
let r2 =
if len(indexSet) == 0:
true
else:
(ValidatorIndex(index) in indexSet)
let sres = validator.getStatus(current_epoch)
if sres.isOk():
let vstatus = sres.get()
let r3 = vstatus in validatorsMask
if (r1 or r2) and r3:
res.add((
index: ValidatorIndex(index),
balance: Base10.toString(state().balances[index]),
status: toString(vstatus),
validator: validator
))
return RestApiResponse.jsonResponse(%res)
return RestApiResponse.jsonError(Http500, "Internal server error")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getStateValidator
router.api(MethodGet,
"/api/eth/v1/beacon/states/{state_id}/validators/{validator_id}") do (
state_id: StateIdent, validator_id: ValidatorIdent) -> RestApiResponse:
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
if validator_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid validator_id",
$validator_id.error())
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
let current_epoch = get_current_epoch(node.chainDag.headState.data.data)
let vid = validator_id.get()
case vid.kind
of ValidatorQueryKind.Key:
for index, validator in state().validators.pairs():
if validator.pubkey == vid.key:
let sres = validator.getStatus(current_epoch)
if sres.isOk():
return RestApiResponse.jsonResponse(
%(
index: ValidatorIndex(index),
balance: Base10.toString(state().balances[index]),
status: toString(sres.get()),
validator: validator
)
)
else:
return RestApiResponse.jsonError(Http400,
"Could not obtain validator's status")
return RestApiResponse.jsonError(Http404, "Could not find validator")
of ValidatorQueryKind.Index:
let index = uint64(vid.index)
if index >= uint64(len(state().validators)):
return RestApiResponse.jsonError(Http404, "Could not find validator")
let validator = state().validators[index]
let sres = validator.getStatus(current_epoch)
if sres.isOk():
return RestApiResponse.jsonResponse(
%(
index: ValidatorIndex(index),
balance: Base10.toString(state().balances[index]),
status: toString(sres.get()),
validator: validator
)
)
else:
return RestApiResponse.jsonError(Http400,
"Could not obtain validator's status")
return RestApiResponse.jsonError(Http500, "Internal server error")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getStateValidatorBalances
router.api(MethodGet,
"/api/eth/v1/beacon/states/{state_id}/validator_balances") do (
state_id: StateIdent, id: seq[ValidatorIdent]) -> RestApiResponse:
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
let validatorIds =
block:
if id.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid validator identifier(s)")
id.get()
let (keySet, indexSet) =
block:
var res1: HashSet[ValidatorPubKey]
var res2: HashSet[ValidatorIndex]
for item in validatorIds:
case item.kind
of ValidatorQueryKind.Key:
if item.key in res1:
return RestApiResponse.jsonError(Http400,
"Only unique validator keys allowed")
res1.incl(item.key)
of ValidatorQueryKind.Index:
if item.index in res2:
return RestApiResponse.jsonError(Http400,
"Only unique validator indexes allowed")
res2.incl(item.index)
(res1, res2)
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
let current_epoch = get_current_epoch(node.chainDag.headState.data.data)
var res: seq[ValidatorBalanceTuple]
for index, validator in state().validators.pairs():
let rflag =
if (len(keySet) == 0) and (len(indexSet) == 0):
true
else:
(validator.pubkey in keySet) or (ValidatorIndex(index) in indexSet)
let sres = validator.getStatus(current_epoch)
if sres.isOk():
let vstatus = sres.get()
if rflag:
res.add((
index: ValidatorIndex(index),
balance: Base10.toString(state().balances[index]),
))
return RestApiResponse.jsonResponse(%res)
return RestApiResponse.jsonError(Http500, "Internal server error")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getEpochCommittees
router.api(MethodGet,
"/api/eth/v1/beacon/states/{state_id}/committees") do (
state_id: StateIdent, epoch: Option[Epoch], index: Option[CommitteeIndex],
slot: Option[Slot]) -> RestApiResponse:
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
let vepoch =
if epoch.isSome():
let repoch = epoch.get()
if repoch.isErr():
return RestApiResponse.jsonError(Http400, "Invalid epoch value",
$repoch.error())
some(repoch.get())
else:
none[Epoch]()
let vindex =
if index.isSome():
let rindex = index.get()
if rindex.isErr():
return RestApiResponse.jsonError(Http400, "Invalid index value",
$rindex.error())
some(rindex.get())
else:
none[CommitteeIndex]()
let vslot =
if slot.isSome():
let rslot = slot.get()
if rslot.isErr():
return RestApiResponse.jsonError(Http400, "Invalid slot value",
$rslot.error())
some(rslot.get())
else:
none[Slot]()
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
proc getCommittee(slot: Slot,
index: CommitteeIndex): StateCommitteeTuple =
let validators = get_beacon_committee(state, slot, index,
cache).mapIt(it)
(index: index, slot: slot, validators: validators)
proc forSlot(slot: Slot, cindex: Option[CommitteeIndex],
res: var seq[StateCommitteeTuple]) =
let committees_per_slot =
get_committee_count_per_slot(state, Epoch(slot), cache)
if cindex.isNone:
for committee_index in 0'u64 ..< committees_per_slot:
res.add(getCommittee(slot, CommitteeIndex(committee_index)))
else:
let idx = cindex.get()
if uint64(idx) < committees_per_slot:
res.add(getCommittee(slot, CommitteeIndex(idx)))
var res: seq[StateCommitteeTuple]
let qepoch =
if vepoch.isNone:
compute_epoch_at_slot(state().slot)
else:
vepoch.get()
if vslot.isNone():
for i in 0 ..< SLOTS_PER_EPOCH:
forSlot(compute_start_slot_at_epoch(qepoch) + i, vindex, res)
else:
forSlot(vslot.get(), vindex, res)
return RestApiResponse.jsonResponse(%res)
return RestApiResponse.jsonError(Http500, "Internal server error")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlockHeaders
router.api(MethodGet, "/api/eth/v1/beacon/headers") do (
slot: Option[Slot], parent_root: Option[Eth2Digest]) -> RestApiResponse:
return RestApiResponse.jsonError(Http500, "Not implemented yet")
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlockHeader
router.api(MethodGet, "/api/eth/v1/beacon/headers/{block_id}") do (
block_id: BlockIdent) -> RestApiResponse:
if block_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid block_id",
$block_id.error())
let res = node.getBlockDataFromBlockIdent(block_id.get())
if res.isErr():
return RestApiResponse.jsonError(Http404, "Block not found")
let data = res.get()
return RestApiResponse.jsonResponse(
%(
root: data.data.root,
canonical: data.refs.isAncestorOf(node.chainDag.head),
header: (
message: (
slot: data.data.message.slot,
proposer_index: Base10.toString(data.data.message.proposer_index),
parent_root: data.data.message.parent_root,
state_root: data.data.message.state_root,
body_root: data.data.message.body.hash_tree_root()
),
signature: cast[ValidatorSig](data.data.signature)
)
)
)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/publishBlock
router.api(MethodPost, "/api/eth/v1/beacon/blocks") do (
contentBody: Option[ContentBody]) -> RestApiResponse:
discard
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlock
router.api(MethodGet, "/api/eth/v1/beacon/blocks/{block_id}") do (
block_id: BlockIdent) -> RestApiResponse:
if block_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid block_id",
$block_id.error())
let res = node.getBlockDataFromBlockIdent(block_id.get())
if res.isErr():
return RestApiResponse.jsonError(Http404, "Block not found")
let data = res.get()
return RestApiResponse.jsonResponse(%(data.data))
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlockRoot
router.api(MethodGet, "/api/eth/v1/beacon/blocks/{block_id}/root") do (
block_id: BlockIdent) -> RestApiResponse:
if block_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid block_id",
$block_id.error())
let res = node.getBlockDataFromBlockIdent(block_id.get())
if res.isErr():
return RestApiResponse.jsonError(Http404, "Block not found")
let data = res.get()
return RestApiResponse.jsonResponse(%(root: data.data.root))
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlockAttestations
router.api(MethodGet,
"/api/eth/v1/beacon/blocks/{block_id}/attestations") do (
block_id: BlockIdent) -> RestApiResponse:
if block_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid block_id",
$block_id.error())
let res = node.getBlockDataFromBlockIdent(block_id.get())
if res.isErr():
return RestApiResponse.jsonError(Http404, "Block not found")
let data = res.get()
return RestApiResponse.jsonResponse(
%data.data.message.body.attestations.asSeq()
)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getPoolAttestations
router.api(MethodGet, "/api/eth/v1/beacon/pool/attestations") do (
slot: Option[Slot],
committee_index: Option[CommitteeIndex]) -> RestApiResponse:
let vindex =
if committee_index.isSome():
let rindex = committee_index.get()
if rindex.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid committee_index value",
$rindex.error())
some(rindex.get())
else:
none[CommitteeIndex]()
let vslot =
if slot.isSome():
let rslot = slot.get()
if rslot.isErr():
return RestApiResponse.jsonError(Http400, "Invalid slot value",
$rslot.error())
some(rslot.get())
else:
none[Slot]()
var res: seq[Attestation]
for item in node.attestationPool[].attestations(vslot, vindex):
res.add(item)
return RestApiResponse.jsonResponse(%res)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/submitPoolAttestations
router.api(MethodPost, "/api/eth/v1/beacon/pool/attestations") do (
contentBody: Option[ContentBody]) -> RestApiResponse:
discard
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getPoolAttesterSlashings
router.api(MethodGet, "/api/eth/v1/beacon/pool/attester_slashings") do (
) -> RestApiResponse:
var res: seq[AttesterSlashing]
if isNil(node.exitPool):
return RestApiResponse.jsonResponse(%res)
let length = len(node.exitPool.attester_slashings)
res = newSeqOfCap[AttesterSlashing](length)
for item in node.exitPool.attester_slashings.items():
res.add(item)
return RestApiResponse.jsonResponse(%res)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/submitPoolAttesterSlashings
router.api(MethodPost, "/api/eth/v1/beacon/pool/attester_slashings") do (
contentBody: Option[ContentBody]) -> RestApiResponse:
discard
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getPoolProposerSlashings
router.api(MethodGet, "/api/eth/v1/beacon/pool/proposer_slashings") do (
) -> RestApiResponse:
var res: seq[ProposerSlashing]
if isNil(node.exitPool):
return RestApiResponse.jsonResponse(%res)
let length = len(node.exitPool.proposer_slashings)
res = newSeqOfCap[ProposerSlashing](length)
for item in node.exitPool.proposer_slashings.items():
res.add(item)
return RestApiResponse.jsonResponse(%res)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/submitPoolProposerSlashings
router.api(MethodPost, "/api/eth/v1/beacon/pool/proposer_slashings") do (
contentBody: Option[ContentBody]) -> RestApiResponse:
discard
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getPoolVoluntaryExits
router.api(MethodGet, "/api/eth/v1/beacon/pool/voluntary_exits") do (
) -> RestApiResponse:
var res: seq[SignedVoluntaryExit]
if isNil(node.exitPool):
return RestApiResponse.jsonResponse(%res)
let length = len(node.exitPool.voluntary_exits)
res = newSeqOfCap[SignedVoluntaryExit](length)
for item in node.exitPool.voluntary_exits.items():
res.add(item)
return RestApiResponse.jsonResponse(%res)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/submitPoolVoluntaryExit
router.api(MethodPost, "/api/eth/v1/beacon/pool/voluntary_exits") do (
contentBody: Option[ContentBody]) -> RestApiResponse:
if contentBody.isNone():
return RestApiResponse.jsonError(Http400, "Empty request's body")
let res = decodeBody(SignedVoluntaryExit, contentBody.get())
warn "VoluntaryExit received", value = $res.get()

View File

@ -0,0 +1,116 @@
# 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/json,
stew/endians2,
presto,
rest_utils,
chronicles,
nimcrypto/utils as ncrutils,
../beacon_node_common, ../eth1_monitor,
../spec/[datatypes, digest, presets]
logScope: topics = "rest_config"
func getDepositAddress(node: BeaconNode): string =
if isNil(node.eth1Monitor):
""
else:
$node.eth1Monitor.depositContractAddress
proc installConfigApiHandlers*(router: var RestRouter, node: BeaconNode) =
router.api(MethodGet,
"/api/eth/v1/config/fork/schedule") do () -> RestApiResponse:
return RestApiResponse.jsonResponse(
%[node.chainDag.headState.data.data.fork]
)
router.api(MethodGet,
"/api/eth/v1/config/spec") do () -> RestApiResponse:
return RestApiResponse.jsonResponse(
%*{
"MAX_COMMITTEES_PER_SLOT": $MAX_COMMITTEES_PER_SLOT,
"TARGET_COMMITTEE_SIZE": $TARGET_COMMITTEE_SIZE,
"MAX_VALIDATORS_PER_COMMITTEE": $MAX_VALIDATORS_PER_COMMITTEE,
"MIN_PER_EPOCH_CHURN_LIMIT": $MIN_PER_EPOCH_CHURN_LIMIT,
"CHURN_LIMIT_QUOTIENT": $CHURN_LIMIT_QUOTIENT,
"SHUFFLE_ROUND_COUNT": $SHUFFLE_ROUND_COUNT,
"MIN_GENESIS_ACTIVE_VALIDATOR_COUNT":
$node.runtimePreset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT,
"MIN_GENESIS_TIME": $node.runtimePreset.MIN_GENESIS_TIME,
"HYSTERESIS_QUOTIENT": $HYSTERESIS_QUOTIENT,
"HYSTERESIS_DOWNWARD_MULTIPLIER": $HYSTERESIS_DOWNWARD_MULTIPLIER,
"HYSTERESIS_UPWARD_MULTIPLIER": $HYSTERESIS_UPWARD_MULTIPLIER,
"SAFE_SLOTS_TO_UPDATE_JUSTIFIED": $SAFE_SLOTS_TO_UPDATE_JUSTIFIED,
"ETH1_FOLLOW_DISTANCE": $node.runtimePreset.ETH1_FOLLOW_DISTANCE,
"TARGET_AGGREGATORS_PER_COMMITTEE": $TARGET_AGGREGATORS_PER_COMMITTEE,
"RANDOM_SUBNETS_PER_VALIDATOR": $RANDOM_SUBNETS_PER_VALIDATOR,
"EPOCHS_PER_RANDOM_SUBNET_SUBSCRIPTION":
$EPOCHS_PER_RANDOM_SUBNET_SUBSCRIPTION,
"SECONDS_PER_ETH1_BLOCK": $SECONDS_PER_ETH1_BLOCK,
"DEPOSIT_CHAIN_ID": $DEPOSIT_CHAIN_ID,
"DEPOSIT_NETWORK_ID": $DEPOSIT_NETWORK_ID,
"DEPOSIT_CONTRACT_ADDRESS": node.getDepositAddress,
"MIN_DEPOSIT_AMOUNT": $MIN_DEPOSIT_AMOUNT,
"MAX_EFFECTIVE_BALANCE": $MAX_EFFECTIVE_BALANCE,
"EJECTION_BALANCE": $EJECTION_BALANCE,
"EFFECTIVE_BALANCE_INCREMENT": $EFFECTIVE_BALANCE_INCREMENT,
"GENESIS_FORK_VERSION":
"0x" & $node.runtimePreset.GENESIS_FORK_VERSION,
"BLS_WITHDRAWAL_PREFIX": "0x" & ncrutils.toHex([BLS_WITHDRAWAL_PREFIX]),
"GENESIS_DELAY": $node.runtimePreset.GENESIS_DELAY,
"SECONDS_PER_SLOT": $SECONDS_PER_SLOT,
"MIN_ATTESTATION_INCLUSION_DELAY": $MIN_ATTESTATION_INCLUSION_DELAY,
"SLOTS_PER_EPOCH": $SLOTS_PER_EPOCH,
"MIN_SEED_LOOKAHEAD": $MIN_SEED_LOOKAHEAD,
"MAX_SEED_LOOKAHEAD": $MAX_SEED_LOOKAHEAD,
"EPOCHS_PER_ETH1_VOTING_PERIOD": $EPOCHS_PER_ETH1_VOTING_PERIOD,
"SLOTS_PER_HISTORICAL_ROOT": $SLOTS_PER_HISTORICAL_ROOT,
"MIN_VALIDATOR_WITHDRAWABILITY_DELAY":
$MIN_VALIDATOR_WITHDRAWABILITY_DELAY,
"SHARD_COMMITTEE_PERIOD": $SHARD_COMMITTEE_PERIOD,
"MIN_EPOCHS_TO_INACTIVITY_PENALTY": $MIN_EPOCHS_TO_INACTIVITY_PENALTY,
"EPOCHS_PER_HISTORICAL_VECTOR": $EPOCHS_PER_HISTORICAL_VECTOR,
"EPOCHS_PER_SLASHINGS_VECTOR": $EPOCHS_PER_SLASHINGS_VECTOR,
"HISTORICAL_ROOTS_LIMIT": $HISTORICAL_ROOTS_LIMIT,
"VALIDATOR_REGISTRY_LIMIT": $VALIDATOR_REGISTRY_LIMIT,
"BASE_REWARD_FACTOR": $BASE_REWARD_FACTOR,
"WHISTLEBLOWER_REWARD_QUOTIENT": $WHISTLEBLOWER_REWARD_QUOTIENT,
"PROPOSER_REWARD_QUOTIENT": $PROPOSER_REWARD_QUOTIENT,
"INACTIVITY_PENALTY_QUOTIENT": $INACTIVITY_PENALTY_QUOTIENT,
"MIN_SLASHING_PENALTY_QUOTIENT": $MIN_SLASHING_PENALTY_QUOTIENT,
"PROPORTIONAL_SLASHING_MULTIPLIER": $PROPORTIONAL_SLASHING_MULTIPLIER,
"MAX_PROPOSER_SLASHINGS": $MAX_PROPOSER_SLASHINGS,
"MAX_ATTESTER_SLASHINGS": $MAX_ATTESTER_SLASHINGS,
"MAX_ATTESTATIONS": $MAX_ATTESTATIONS,
"MAX_DEPOSITS": $MAX_DEPOSITS,
"MAX_VOLUNTARY_EXITS": $MAX_VOLUNTARY_EXITS,
"DOMAIN_BEACON_PROPOSER":
"0x" & ncrutils.toHex(uint32(DOMAIN_BEACON_PROPOSER).toBytesLE()),
"DOMAIN_BEACON_ATTESTER":
"0x" & ncrutils.toHex(uint32(DOMAIN_BEACON_ATTESTER).toBytesLE()),
"DOMAIN_RANDAO":
"0x" & ncrutils.toHex(uint32(DOMAIN_RANDAO).toBytesLE()),
"DOMAIN_DEPOSIT":
"0x" & ncrutils.toHex(uint32(DOMAIN_DEPOSIT).toBytesLE()),
"DOMAIN_VOLUNTARY_EXIT":
"0x" & ncrutils.toHex(uint32(DOMAIN_VOLUNTARY_EXIT).toBytesLE()),
"DOMAIN_SELECTION_PROOF":
"0x" & ncrutils.toHex(uint32(DOMAIN_SELECTION_PROOF).toBytesLE()),
"DOMAIN_AGGREGATE_AND_PROOF":
"0x" & ncrutils.toHex(uint32(DOMAIN_AGGREGATE_AND_PROOF).toBytesLE())
}
)
router.api(MethodGet,
"/api/eth/v1/config/deposit_contract") do () -> RestApiResponse:
return RestApiResponse.jsonResponse(
%*{
"chain_id": $DEPOSIT_CHAIN_ID,
"address": node.getDepositAddress()
}
)

View File

@ -0,0 +1,33 @@
import
std/sequtils,
presto,
chronicles,
../version, ../beacon_node_common,
../eth2_network, ../peer_pool,
../spec/[datatypes, digest, presets],
./rest_utils
logScope: topics = "rest_debug"
proc installDebugApiHandlers*(router: var RestRouter, node: BeaconNode) =
router.api(MethodGet,
"/api/eth/v1/debug/beacon/states/{state_id}") do (
state_id: StateIdent) -> RestApiResponse:
# TODO: This is very expensive call
if state_id.isErr():
return RestApiResponse.jsonError(Http400, "Invalid state_id",
$state_id.error())
let bres = node.getBlockSlot(state_id.get())
if bres.isErr():
return RestApiResponse.jsonError(Http404, "State not found",
$bres.error())
node.withStateForStateIdent(bres.get()):
return RestApiResponse.jsonResponse(%state())
return RestApiResponse.jsonError(Http500, "Internal server error")
router.api(MethodGet,
"/api/eth/v1/debug/beacon/heads") do () -> RestApiResponse:
return RestApiResponse.jsonResponse(
%node.chainDag.heads.mapIt((root: it.root, slot: it.slot))
)

View File

@ -0,0 +1,247 @@
import
stew/results,
presto,
chronicles,
eth/p2p/discoveryv5/enr,
libp2p/[multiaddress, multicodec],
nimcrypto/utils as ncrutils,
../version, ../beacon_node_common, ../sync_manager,
../eth2_network, ../peer_pool,
../spec/[datatypes, digest, presets],
../spec/eth2_apis/callsigs_types,
./rest_utils
logScope: topics = "rest_node"
type
ConnectionStateSet* = set[ConnectionState]
PeerTypeSet* = set[PeerType]
proc validateState(states: seq[PeerStateKind]): Result[ConnectionStateSet,
cstring] =
var res: set[ConnectionState]
for item in states:
case item
of PeerStateKind.Disconnected:
if ConnectionState.Disconnected in res:
return err("Peer connection states must be unique")
res.incl(ConnectionState.Disconnected)
of PeerStateKind.Connecting:
if ConnectionState.Connecting in res:
return err("Peer connection states must be unique")
res.incl(ConnectionState.Connecting)
of PeerStateKind.Connected:
if ConnectionState.Connected in res:
return err("Peer connection states must be unique")
res.incl(ConnectionState.Connected)
of PeerStateKind.Disconnecting:
if ConnectionState.Disconnecting in res:
return err("Peer connection states must be unique")
res.incl(ConnectionState.Disconnecting)
if res == {}:
res = {ConnectionState.Connecting, ConnectionState.Connected,
ConnectionState.Disconnecting, ConnectionState.Disconnected}
ok(res)
proc validateDirection(directions: seq[PeerDirectKind]): Result[PeerTypeSet,
cstring] =
var res: set[PeerType]
for item in directions:
case item
of PeerDirectKind.Inbound:
if PeerType.Incoming in res:
return err("Peer direction states must be unique")
res.incl(PeerType.Incoming)
of PeerDirectKind.Outbound:
if PeerType.Outgoing in res:
return err("Peer direction states must be unique")
res.incl(PeerType.Outgoing)
if res == {}:
res = {PeerType.Incoming, PeerType.Outgoing}
ok(res)
proc toString(state: ConnectionState): string =
case state
of ConnectionState.Disconnected:
"disconnected"
of ConnectionState.Connecting:
"connecting"
of ConnectionState.Connected:
"connected"
of ConnectionState.Disconnecting:
"disconnecting"
else:
""
proc toString(direction: PeerType): string =
case direction:
of PeerType.Incoming:
"inbound"
of PeerType.Outgoing:
"outbound"
proc getLastSeenAddress(info: PeerInfo): string =
# TODO (cheatfate): We need to provide filter here, which will be able to
# filter such multiaddresses like `/ip4/0.0.0.0` or local addresses or
# addresses with peer ids.
if len(info.addrs) > 0:
$info.addrs[len(info.addrs) - 1]
else:
""
proc getDiscoveryAddresses(node: BeaconNode): Option[seq[string]] =
let restr = node.network.enrRecord().toTypedRecord()
if restr.isErr():
return none[seq[string]]()
let respa = restr.get().toPeerAddr(udpProtocol)
if respa.isErr():
return none[seq[string]]()
let pa = respa.get()
let mpa = MultiAddress.init(multicodec("p2p"), pa.peerId)
if mpa.isErr():
return none[seq[string]]()
var addresses = newSeqOfCap[string](len(pa.addrs))
for item in pa.addrs:
let resa = concat(item, mpa.get())
if resa.isOk():
addresses.add($(resa.get()))
return some(addresses)
proc getP2PAddresses(node: BeaconNode): Option[seq[string]] =
let pinfo = node.network.switch.peerInfo
let mpa = MultiAddress.init(multicodec("p2p"), pinfo.peerId)
if mpa.isErr():
return none[seq[string]]()
var addresses = newSeqOfCap[string](len(pinfo.addrs))
for item in pinfo.addrs:
let resa = concat(item, mpa.get())
if resa.isOk():
addresses.add($(resa.get()))
return some(addresses)
proc installNodeApiHandlers*(router: var RestRouter, node: BeaconNode) =
router.api(MethodGet, "/api/eth/v1/node/identity") do () -> RestApiResponse:
let discoveryAddresses =
block:
let res = node.getDiscoveryAddresses()
if res.isSome():
res.get()
else:
newSeq[string](0)
let p2pAddresses =
block:
let res = node.getP2PAddresses()
if res.isSome():
res.get()
else:
newSeq[string]()
return RestApiResponse.jsonResponse(
%(
peer_id: $node.network.peerId(),
enr: node.network.enrRecord().toUri(),
p2p_addresses: p2pAddresses,
discovery_addresses: discoveryAddresses,
metadata: (node.network.metadata.seq_number,
"0x" & ncrutils.toHex(node.network.metadata.attnets.bytes))
)
)
router.api(MethodGet, "/api/eth/v1/node/peers") do (
states: seq[PeerStateKind],
directions: seq[PeerDirectKind]) -> RestApiResponse:
if states.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid state value(s)",
$states.error())
if directions.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid direction value(s)",
$directions.error())
let sres = validateState(states.get())
if sres.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid state value(s)",
$sres.error())
let dres = validateDirection(directions.get())
if dres.isErr():
return RestApiResponse.jsonError(Http400,
"Invalid direction value(s)",
$dres.error())
var res: seq[NodePeerTuple]
let connectionMask = sres.get()
let directionMask = dres.get()
for item in node.network.peers.values():
if (item.connectionState in connectionMask) and
(item.direction in directionMask):
let peer = (
peer_id: $item.info.peerId,
enr: if item.enr.isSome(): item.enr.get().toUri() else: "",
last_seen_p2p_address: item.info.getLastSeenAddress(),
state: item.connectionState.toString(),
direction: item.direction.toString(),
agent: item.info.agentVersion, # Fields `agent` and `proto` are not
proto: item.info.protoVersion # part of specification.
)
res.add(peer)
return RestApiResponse.jsonResponse(%res)
router.api(MethodGet, "/api/eth/v1/node/peer_count") do () -> RestApiResponse:
var res: NodePeerCountTuple
for item in node.network.peers.values():
case item.connectionState
of Connecting:
inc(res.connecting)
of Connected:
inc(res.connected)
of Disconnecting:
inc(res.disconnecting)
of Disconnected:
inc(res.disconnected)
of ConnectionState.None:
discard
return RestApiResponse.jsonResponse(%res)
router.api(MethodGet, "/api/eth/v1/node/peers/{peer_id}") do (
peer_id: PeerID) -> RestApiResponse:
if peer_id.isErr():
return RestApiResponse.jsonError(Http400, "PeerID could not be parsed",
$peer_id.error())
let peer = node.network.peers.getOrDefault(peer_id.get())
if isNil(peer):
return RestApiResponse.jsonError(Http404, "Peer not found")
return RestApiResponse.jsonResponse(
%(
peer_id: $peer.info.peerId,
enr: if peer.enr.isSome(): peer.enr.get().toUri() else: "",
last_seen_p2p_address: peer.info.getLastSeenAddress(),
state: peer.connectionState.toString(),
direction: peer.direction.toString(),
agent: peer.info.agentVersion, # Fields `agent` and `proto` are not
proto: peer.info.protoVersion # part of specification
)
)
router.api(MethodGet, "/api/eth/v1/node/version") do () -> RestApiResponse:
return RestApiResponse.jsonResponse(
%(version: "Nimbus/" & fullVersionStr)
)
router.api(MethodGet, "/api/eth/v1/node/syncing") do () -> RestApiResponse:
return RestApiResponse.jsonResponse(
%node.syncManager.getInfo()
)
router.api(MethodGet, "/api/eth/v1/node/health") do () -> RestApiResponse:
# TODO: Add ability to detect node's issues and return 503 error according
# to specification.
let res =
if node.syncManager.inProgress:
(health: 206)
else:
(health: 200)
return RestApiResponse.jsonResponse(%res)

View File

@ -0,0 +1,482 @@
import std/json
import presto
import libp2p/peerid
import stew/[base10, byteutils]
import nimcrypto/utils as ncrutils
import ../spec/[crypto, digest, datatypes]
import ../beacon_node_common, ../validator_duties
import ../block_pools/[block_pools_types, chain_dag]
export chain_dag, presto
const
DecimalSet = {'0' .. '9'}
# Base10 (decimal) set of chars
HexadecimalSet = {'0'..'9', 'A'..'F', 'a'..'f'}
# Base16 (hexadecimal) set of chars
Base58Set = {'1'..'9', 'A'..'H', 'J'..'N', 'P'..'Z', 'a'..'k', 'm'..'z'}
# Base58 set of chars
MaxDecimalSize = len($high(uint64))
# Maximum size of `uint64` decimal value
MaxPeerIdSize = 128
# Maximum size of `PeerID` base58 encoded value
ValidatorKeySize = RawPubKeySize * 2
# Size of `ValidatorPubKey` hexadecimal value (without 0x)
ValidatorSigSize = RawSigSize * 2
# Size of `ValidatorSig` hexadecimal value (without 0x)
ValidatorIndexSize = len($(1 shl 40))
# Maximum size of `ValidatorIndex` decimal value
RootHashSize = sizeof(Eth2Digest) * 2
# Size of `xxx_root` hexadecimal value (without 0x)
FarFutureEpochString* = "18446744073709551615"
type
ValidatorQueryKind* {.pure.} = enum
Index, Key
ValidatorIdent* = object
case kind*: ValidatorQueryKind
of ValidatorQueryKind.Index:
index*: ValidatorIndex
of ValidatorQueryKind.Key:
key*: ValidatorPubKey
ValidatorFilterKind* {.pure.} = enum
PendingInitialized, PendingQueued,
ActiveOngoing, ActiveExiting, ActiveSlashed,
ExitedUnslashed, ExitedSlashed,
WithdrawalPossible, WithdrawalDone
ValidatorFilter* = set[ValidatorFilterKind]
StateQueryKind* {.pure.} = enum
Slot, Root, Named
StateIdentType* {.pure.} = enum
Head, Genesis, Finalized, Justified
StateIdent* = object
case kind*: StateQueryKind
of StateQueryKind.Slot:
slot*: Slot
of StateQueryKind.Root:
root*: Eth2Digest
of StateQueryKind.Named:
value*: StateIdentType
BlockQueryKind* {.pure.} = enum
Slot, Root, Named
BlockIdentType* {.pure.} = enum
Head, Genesis, Finalized
BlockIdent* = object
case kind*: BlockQueryKind
of BlockQueryKind.Slot:
slot*: Slot
of BlockQueryKind.Root:
root*: Eth2Digest
of BlockQueryKind.Named:
value*: BlockIdentType
PeerStateKind* {.pure.} = enum
Disconnected, Connecting, Connected, Disconnecting
PeerDirectKind* {.pure.} = enum
Inbound, Outbound
proc toString*(s: uint64): string =
Base10.toString(s)
proc `%`*(s: Eth2Digest): JsonNode =
JsonNode(kind: JString,
str: "0x" & ncrutils.toHex(s.data, true))
proc toJsonHex(data: openArray[byte]): string =
# Per the eth2 API spec, hex arrays are printed with leading 0x
"0x" & ncrutils.toHex(data, true)
proc `%`*(list: List): JsonNode =
%(asSeq(list))
proc `%`*(bitlist: BitList): JsonNode =
newJString(toJsonHex(seq[byte](BitSeq(bitlist))))
proc `%`*(s: Version): JsonNode =
JsonNode(kind: JString,
str: "0x" & ncrutils.toHex(cast[array[4, byte]](s), true))
func match(data: openarray[char], charset: set[char]): int =
for ch in data:
if ch notin charset:
return 1
0
proc validate(key: string, value: string): int =
## This is rough validation procedure which should be simple and fast,
## because it will be used for query routing.
case key
of "{epoch}":
# Can be any decimal 64bit value.
if len(value) > MaxDecimalSize: 1 else: match(value, DecimalSet)
of "{slot}":
# Can be any decimal 64bit value.
if len(value) > MaxDecimalSize: 1 else: match(value, DecimalSet)
of "{peer_id}":
# Can be base58 encoded value.
if len(value) > MaxPeerIdSize: 1 else: match(value, Base58Set)
of "{state_id}":
# Can be one of: "head" (canonical head in node's view), "genesis",
# "finalized", "justified", <slot>, <hex encoded stateRoot with 0x prefix>.
if len(value) > 2:
if (value[0] == '0') and (value[1] == 'x'):
if len(value) != 2 + RootHashSize:
1
else:
match(value.toOpenArray(2, len(value) - 1), HexadecimalSet)
elif (value[0] in DecimalSet) and (value[1] in DecimalSet):
if len(value) > MaxDecimalSize:
1
else:
match(value.toOpenArray(2, len(value) - 1), DecimalSet)
else:
case value
of "head": 0
of "genesis": 0
of "finalized": 0
of "justified": 0
else: 1
else:
match(value, DecimalSet)
of "{block_id}":
# Can be one of: "head" (canonical head in node's view), "genesis",
# "finalized", <slot>, <hex encoded blockRoot with 0x prefix>.
if len(value) > 2:
if (value[0] == '0') and (value[1] == 'x'):
if len(value) != 2 + RootHashSize:
1
else:
match(value.toOpenArray(2, len(value) - 1), HexadecimalSet)
elif (value[0] in DecimalSet) and (value[1] in DecimalSet):
if len(value) > MaxDecimalSize:
1
else:
match(value.toOpenArray(2, len(value) - 1), DecimalSet)
else:
case value
of "head": 0
of "genesis": 0
of "finalized": 0
else: 1
else:
match(value, DecimalSet)
of "{validator_id}":
# Either hex encoded public key (with 0x prefix) or validator index.
if len(value) > 2:
if (value[0] == '0') and (value[1] == 'x'):
if len(value) != 2 + ValidatorKeySize:
1
else:
match(value.toOpenArray(2, len(value) - 1), HexadecimalSet)
else:
if len(value) > ValidatorIndexSize:
1
else:
match(value, DecimalSet)
else:
match(value, DecimalSet)
else:
1
proc parseRoot(value: string): Result[Eth2Digest, cstring] =
try:
ok(Eth2Digest(data: hexToByteArray[32](value)))
except ValueError:
err("Unable to decode root value")
proc decodeString*(t: typedesc[Slot], value: string): Result[Slot, cstring] =
let res = ? Base10.decode(uint64, value)
ok(Slot(res))
proc decodeString*(t: typedesc[Epoch], value: string): Result[Epoch, cstring] =
let res = ? Base10.decode(uint64, value)
ok(Epoch(res))
proc decodeString*(t: typedesc[StateIdent],
value: string): Result[StateIdent, cstring] =
if len(value) > 2:
if (value[0] == '0') and (value[1] == 'x'):
if len(value) != RootHashSize + 2:
err("Incorrect state root value length")
else:
let res = ? parseRoot(value)
ok(StateIdent(kind: StateQueryKind.Root, root: res))
elif (value[0] in DecimalSet) and (value[1] in DecimalSet):
let res = ? Base10.decode(uint64, value)
ok(StateIdent(kind: StateQueryKind.Slot, slot: Slot(res)))
else:
case value
of "head":
ok(StateIdent(kind: StateQueryKind.Named,
value: StateIdentType.Head))
of "genesis":
ok(StateIdent(kind: StateQueryKind.Named,
value: StateIdentType.Genesis))
of "finalized":
ok(StateIdent(kind: StateQueryKind.Named,
value: StateIdentType.Finalized))
of "justified":
ok(StateIdent(kind: StateQueryKind.Named,
value: StateIdentType.Justified))
else:
err("Incorrect state identifier value")
else:
let res = ? Base10.decode(uint64, value)
ok(StateIdent(kind: StateQueryKind.Slot, slot: Slot(res)))
proc decodeString*(t: typedesc[BlockIdent],
value: string): Result[BlockIdent, cstring] =
if len(value) > 2:
if (value[0] == '0') and (value[1] == 'x'):
if len(value) != RootHashSize + 2:
err("Incorrect block root value length")
else:
let res = ? parseRoot(value)
ok(BlockIdent(kind: BlockQueryKind.Root, root: res))
elif (value[0] in DecimalSet) and (value[1] in DecimalSet):
let res = ? Base10.decode(uint64, value)
ok(BlockIdent(kind: BlockQueryKind.Slot, slot: Slot(res)))
else:
case value
of "head":
ok(BlockIdent(kind: BlockQueryKind.Named,
value: BlockIdentType.Head))
of "genesis":
ok(BlockIdent(kind: BlockQueryKind.Named,
value: BlockIdentType.Genesis))
of "finalized":
ok(BlockIdent(kind: BlockQueryKind.Named,
value: BlockIdentType.Finalized))
else:
err("Incorrect block identifier value")
else:
let res = ? Base10.decode(uint64, value)
ok(BlockIdent(kind: BlockQueryKind.Slot, slot: Slot(res)))
proc decodeString*(t: typedesc[ValidatorIdent],
value: string): Result[ValidatorIdent, cstring] =
# This should raise exception if ValidatorIndex type will be changed,
# because currently it `uint32` but in 40bits size in specification.
doAssert(sizeof(uint32) == sizeof(ValidatorIndex))
if len(value) > 2:
if (value[0] == '0') and (value[1] == 'x'):
if len(value) != ValidatorKeySize + 2:
err("Incorrect validator's key value length")
else:
let res = ? ValidatorPubKey.fromHex(value)
ok(ValidatorIdent(kind: ValidatorQueryKind.Key,
key: res))
elif (value[0] in DecimalSet) and (value[1] in DecimalSet):
let res = ? Base10.decode(uint32, value)
ok(ValidatorIdent(kind: ValidatorQueryKind.Index,
index: ValidatorIndex(res)))
else:
err("Incorrect validator identifier value")
else:
let res = ? Base10.decode(uint32, value)
ok(ValidatorIdent(kind: ValidatorQueryKind.Index,
index: ValidatorIndex(res)))
proc decodeString*(t: typedesc[PeerID],
value: string): Result[PeerID, cstring] =
PeerID.init(value)
proc decodeString*(t: typedesc[CommitteeIndex],
value: string): Result[CommitteeIndex, cstring] =
let res = ? Base10.decode(uint64, value)
ok(CommitteeIndex(res))
proc decodeString*(t: typedesc[Eth2Digest],
value: string): Result[Eth2Digest, cstring] =
if len(value) != RootHashSize + 2:
return err("Incorrect root value length")
if value[0] != '0' and value[1] != 'x':
return err("Incorrect root value encoding")
parseRoot(value)
proc decodeString*(t: typedesc[ValidatorFilter],
value: string): Result[ValidatorFilter, cstring] =
case value
of "pending_initialized":
ok({ValidatorFilterKind.PendingInitialized})
of "pending_queued":
ok({ValidatorFilterKind.PendingQueued})
of "active_ongoing":
ok({ValidatorFilterKind.ActiveOngoing})
of "active_exiting":
ok({ValidatorFilterKind.ActiveExiting})
of "active_slashed":
ok({ValidatorFilterKind.ActiveSlashed})
of "exited_unslashed":
ok({ValidatorFilterKind.ExitedUnslashed})
of "exited_slashed":
ok({ValidatorFilterKind.ExitedSlashed})
of "withdrawal_possible":
ok({ValidatorFilterKind.WithdrawalPossible})
of "withdrawal_done":
ok({ValidatorFilterKind.WithdrawalDone})
of "pending":
ok({
ValidatorFilterKind.PendingInitialized,
ValidatorFilterKind.PendingQueued
})
of "active":
ok({
ValidatorFilterKind.ActiveOngoing,
ValidatorFilterKind.ActiveExiting,
ValidatorFilterKind.ActiveSlashed
})
of "exited":
ok({
ValidatorFilterKind.ExitedUnslashed,
ValidatorFilterKind.ExitedSlashed
})
of "withdrawal":
ok({
ValidatorFilterKind.WithdrawalPossible,
ValidatorFilterKind.WithdrawalDone
})
else:
err("Incorrect validator state identifier value")
proc decodeString*(t: typedesc[PeerStateKind],
value: string): Result[PeerStateKind, cstring] =
case value
of "disconnected":
ok(PeerStateKind.Disconnected)
of "connecting":
ok(PeerStateKind.Connecting)
of "connected":
ok(PeerStateKind.Connected)
of "disconnecting":
ok(PeerStateKind.Disconnecting)
else:
err("Incorrect peer's state value")
proc decodeString*(t: typedesc[PeerDirectKind],
value: string): Result[PeerDirectKind, cstring] =
case value
of "inbound":
ok(PeerDirectKind.Inbound)
of "outbound":
ok(PeerDirectKind.Outbound)
else:
err("Incorrect peer's direction value")
proc decodeString*(t: typedesc[ValidatorSig],
value: string): Result[ValidatorSig, cstring] =
if len(value) != ValidatorSigSize + 2:
return err("Incorrect validator signature value length")
if value[0] != '0' and value[1] != 'x':
return err("Incorrect validator signature encoding")
ValidatorSig.fromHex(value)
proc decodeString*(t: typedesc[GraffitiBytes],
value: string): Result[GraffitiBytes, cstring] =
try:
ok(GraffitiBytes.init(value))
except ValueError:
err("Unable to decode graffiti value")
proc jsonResponse*(t: typedesc[RestApiResponse], j: JsonNode): RestApiResponse =
let data = %*{"data": j}
ok(ContentBody(contentType: "application/json",
data: cast[seq[byte]]($data)))
proc getRouter*(): RestRouter =
RestRouter.init(validate)
proc getCurrentHead*(node: BeaconNode,
slot: Slot): Result[BlockRef, cstring] =
let res = node.chainDag.head
# if not(node.isSynced(res)):
# return err("Cannot fulfill request until node is synced")
if res.slot + uint64(2 * SLOTS_PER_EPOCH) < slot:
return err("Requesting way ahead of the current head")
ok(res)
proc getCurrentHead*(node: BeaconNode,
epoch: Epoch): Result[BlockRef, cstring] =
const maxEpoch = compute_epoch_at_slot(not(0'u64))
if epoch >= maxEpoch:
return err("Requesting epoch for which slot would overflow")
node.getCurrentHead(compute_start_slot_at_epoch(epoch))
proc toBlockSlot*(blckRef: BlockRef): BlockSlot =
blckRef.atSlot(blckRef.slot)
proc getBlockSlot*(node: BeaconNode,
stateIdent: StateIdent): Result[BlockSlot, cstring] =
case stateIdent.kind
of StateQueryKind.Slot:
let head = ? getCurrentHead(node, stateIdent.slot)
let bslot = head.atSlot(stateIdent.slot)
if isNil(bslot.blck):
return err("Block not found")
ok(bslot)
of StateQueryKind.Root:
let blckRef = node.chainDag.getRef(stateIdent.root)
if isNil(blckRef):
return err("Block not found")
ok(blckRef.toBlockSlot())
of StateQueryKind.Named:
case stateIdent.value
of StateIdentType.Head:
ok(node.chainDag.head.toBlockSlot())
of StateIdentType.Genesis:
ok(node.chainDag.getGenesisBlockSlot())
of StateIdentType.Finalized:
ok(node.chainDag.finalizedHead)
of StateIdentType.Justified:
ok(node.chainDag.head.atEpochStart(
node.chainDag.headState.data.data.current_justified_checkpoint.epoch))
proc getBlockDataFromBlockIdent*(node: BeaconNode,
id: BlockIdent): Result[BlockData, cstring] =
warn "Searching for block", ident = $id
case id.kind
of BlockQueryKind.Named:
case id.value
of BlockIdentType.Head:
ok(node.chainDag.get(node.chainDag.head))
of BlockIdentType.Genesis:
ok(node.chainDag.getGenesisBlockData())
of BlockIdentType.Finalized:
ok(node.chainDag.get(node.chainDag.finalizedHead.blck))
of BlockQueryKind.Root:
let res = node.chainDag.get(id.root)
if res.isNone():
return err("Block not found")
ok(res.get())
of BlockQueryKind.Slot:
let head = ? node.getCurrentHead(id.slot)
let blockSlot = head.atSlot(id.slot)
if isNil(blockSlot.blck):
return err("Block not found")
ok(node.chainDag.get(blockSlot.blck))
template withStateForStateIdent*(node: BeaconNode,
blockSlot: BlockSlot, body: untyped): untyped =
# TODO this can be optimized for the "head" case since that should be most
# common.
node.chainDag.withState(node.chainDag.tmpState, blockSlot):
body
proc jsonError*(t: typedesc[RestApiResponse], status: HttpCode = Http200,
msg: string = "", stacktrace: string = ""): RestApiResponse =
let data =
if len(stacktrace) > 0:
%*{"code": status.toInt(), "message": msg, "stacktrace": stacktrace}
else:
%*{"code": status.toInt(), "message": msg}
RestApiResponse.error(status, $data, "application/json")

2
vendor/nim-presto vendored

@ -1 +1 @@
Subproject commit d41dfd8ea2197ccdb96d63d05ef79bbf3d7d898c
Subproject commit ae185d2038dd80d366238ec00af4dc0c65785bf8