* Fix REST some rest call signatures and implement a simple API benchmark tool * Implement #3129 (Optimized history traversals in the REST API) Other notable changes: The `updateStateData` procedure in the `blockchain_dag.nim` module is optimized to not rewind down to the last snapshot state saved in the database if the supplied input state can be used as a starting point instead. * Disallow await in withStateForBlockSlot
This commit is contained in:
parent
5878d34117
commit
fba1f08a5e
|
@ -23,7 +23,8 @@ import
|
|||
sync_committee_msg_pool],
|
||||
./spec/datatypes/base,
|
||||
./sync/[sync_manager, request_manager],
|
||||
./validators/[action_tracker, validator_monitor, validator_pool]
|
||||
./validators/[action_tracker, validator_monitor, validator_pool],
|
||||
./rpc/state_ttl_cache
|
||||
|
||||
export
|
||||
osproc, chronos, httpserver, presto, action_tracker, beacon_clock,
|
||||
|
@ -70,6 +71,7 @@ type
|
|||
onAttestationSent*: OnAttestationCallback
|
||||
restKeysCache*: Table[ValidatorPubKey, ValidatorIndex]
|
||||
validatorMonitor*: ref ValidatorMonitor
|
||||
stateTtlCache*: StateTtlCache
|
||||
|
||||
const
|
||||
MaxEmptySlotCount* = uint64(10*60) div SECONDS_PER_SLOT
|
||||
|
|
|
@ -328,6 +328,18 @@ type
|
|||
defaultValueDesc: "127.0.0.1"
|
||||
name: "rest-address" }: ValidIpAddress
|
||||
|
||||
restCacheSize* {.
|
||||
defaultValue: 3
|
||||
desc: "The maximum number of recently accessed states that are kept in " &
|
||||
"memory. Speeds up requests obtaining information for consecutive " &
|
||||
"slots or epochs."
|
||||
name: "rest-statecache-size" }: Natural
|
||||
|
||||
restCacheTtl* {.
|
||||
defaultValue: 60
|
||||
desc: "The number of seconds to keep recently accessed states in memory"
|
||||
name: "rest-statecache-ttl" }: Natural
|
||||
|
||||
keymanagerEnabled* {.
|
||||
desc: "Enable the REST keymanager API (BETA version)"
|
||||
defaultValue: false
|
||||
|
|
|
@ -609,6 +609,7 @@ proc getEpochRef*(dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef =
|
|||
let
|
||||
ancestor = epochAncestor(blck, epoch)
|
||||
|
||||
dag.epochRefState.blck = BlockRef()
|
||||
dag.withState(
|
||||
dag.epochRefState, ancestor.blck.atEpochStart(ancestor.epoch)):
|
||||
dag.getEpochRef(stateData, cache)
|
||||
|
@ -944,8 +945,10 @@ proc updateStateData*(
|
|||
|
||||
# Look for a state in the database and load it - as long as it cannot be
|
||||
# found, keep track of the blocks that are needed to reach it from the
|
||||
# state that eventually will be found
|
||||
while not dag.getState(state, cur):
|
||||
# state that eventually will be found. Besides finding the state in the
|
||||
# database we may also reach the input state provided to the function.
|
||||
# It can also act as a viable starting point for the block replay later.
|
||||
while not canAdvance(state, cur) and not dag.getState(state, cur):
|
||||
# There's no state saved for this particular BlockSlot combination, keep
|
||||
# looking...
|
||||
if cur.slot == cur.blck.slot:
|
||||
|
|
|
@ -34,7 +34,7 @@ import
|
|||
validator_duties, validator_monitor, validator_pool,
|
||||
slashing_protection, keystore_management],
|
||||
./sync/[sync_protocol],
|
||||
./rpc/[rest_api, rpc_api],
|
||||
./rpc/[rest_api, rpc_api, state_ttl_cache],
|
||||
./spec/datatypes/[altair, merge, phase0],
|
||||
./spec/eth2_apis/rpc_beacon_client,
|
||||
./spec/[
|
||||
|
@ -469,6 +469,13 @@ proc init*(T: type BeaconNode,
|
|||
network.peerPool, SyncQueueKind.Forward, getLocalHeadSlot, getLocalWallSlot,
|
||||
getFirstSlotAtFinalizedEpoch, getBackfillSlot, blockVerifier)
|
||||
|
||||
let stateTtlCache = if config.restCacheSize > 0:
|
||||
StateTtlCache.init(
|
||||
cacheSize = config.restCacheSize,
|
||||
cacheTtl = chronos.seconds(config.restCacheTtl))
|
||||
else:
|
||||
nil
|
||||
|
||||
var node = BeaconNode(
|
||||
nickname: nickname,
|
||||
graffitiBytes: if config.graffiti.isSome: config.graffiti.get
|
||||
|
@ -498,7 +505,8 @@ proc init*(T: type BeaconNode,
|
|||
gossipState: {},
|
||||
beaconClock: beaconClock,
|
||||
onAttestationSent: onAttestationSent,
|
||||
validatorMonitor: validatorMonitor
|
||||
validatorMonitor: validatorMonitor,
|
||||
stateTtlCache: stateTtlCache
|
||||
)
|
||||
|
||||
debug "Loading validators", validatorsDir = config.validatorsDir()
|
||||
|
|
|
@ -15,7 +15,8 @@ import
|
|||
../consensus_object_pools/[blockchain_dag, exit_pool, spec_cache],
|
||||
../validators/validator_duties,
|
||||
../spec/[eth2_merkleization, forks, network, validator],
|
||||
../spec/datatypes/[phase0, altair]
|
||||
../spec/datatypes/[phase0, altair],
|
||||
./state_ttl_cache
|
||||
|
||||
export rest_utils
|
||||
|
||||
|
|
|
@ -9,7 +9,7 @@ import std/sequtils
|
|||
import chronicles
|
||||
import ".."/[version, beacon_node],
|
||||
".."/spec/forks,
|
||||
"."/rest_utils
|
||||
"."/[rest_utils, state_ttl_cache]
|
||||
|
||||
export rest_utils
|
||||
|
||||
|
|
|
@ -1,11 +1,11 @@
|
|||
import std/options,
|
||||
import std/[options, macros],
|
||||
presto,
|
||||
nimcrypto/utils as ncrutils,
|
||||
../spec/[forks],
|
||||
../spec/eth2_apis/[rest_types, eth2_rest_serialization],
|
||||
../beacon_node,
|
||||
../consensus_object_pools/blockchain_dag,
|
||||
./rest_constants
|
||||
"."/[rest_constants, state_ttl_cache]
|
||||
|
||||
export
|
||||
options, eth2_rest_serialization, blockchain_dag, presto, rest_types,
|
||||
|
@ -116,19 +116,77 @@ proc getBlockDataFromBlockIdent*(node: BeaconNode,
|
|||
id: BlockIdent): Result[BlockData, cstring] =
|
||||
ok(node.dag.get(? node.getBlockRef(id)))
|
||||
|
||||
template withStateForBlockSlot*(node: BeaconNode,
|
||||
blockSlot: BlockSlot, body: untyped): untyped =
|
||||
template isState(state: StateData): bool =
|
||||
state.blck.atSlot(getStateField(state.data, slot)) == blockSlot
|
||||
proc disallowInterruptionsAux(body: NimNode) =
|
||||
for n in body:
|
||||
const because =
|
||||
"because the `state` variable may be mutated (and thus invalidated) " &
|
||||
"before the function resumes execution."
|
||||
|
||||
if isState(node.dag.headState):
|
||||
withStateVars(node.dag.headState):
|
||||
var cache {.inject, used.}: StateCache
|
||||
body
|
||||
else:
|
||||
let rpcState = assignClone(node.dag.headState)
|
||||
node.dag.withState(rpcState[], blockSlot):
|
||||
body
|
||||
if n.kind == nnkYieldStmt:
|
||||
macros.error "You cannot use yield in this block " & because, n
|
||||
|
||||
if (n.kind in {nnkCall, nnkCommand} and
|
||||
n[0].kind in {nnkIdent, nnkSym} and
|
||||
$n[0] == "await"):
|
||||
macros.error "You cannot use await in this block " & because, n
|
||||
|
||||
disallowInterruptionsAux(n)
|
||||
|
||||
macro disallowInterruptions(body: untyped) =
|
||||
disallowInterruptionsAux(body)
|
||||
|
||||
template withStateForBlockSlot*(nodeParam: BeaconNode,
|
||||
blockSlotParam: BlockSlot,
|
||||
body: untyped): untyped =
|
||||
|
||||
block:
|
||||
let
|
||||
node = nodeParam
|
||||
blockSlot = blockSlotParam
|
||||
|
||||
template isState(state: StateData): bool =
|
||||
state.blck.atSlot(getStateField(state.data, slot)) == blockSlot
|
||||
|
||||
var cache {.inject, used.}: StateCache
|
||||
|
||||
# If we have a cache hit, there is a concern that the REST request
|
||||
# handler may continue executing asynchronously while we hit the same
|
||||
# advanced state is another request. We don't want the two requests
|
||||
# to work over the same state object because mutations to it will be
|
||||
# visible in both, so we must outlaw yielding within the `body` block.
|
||||
# Please note that the problem is not limited to the situations where
|
||||
# we have a cache hit. Working with the `headState` will result in the
|
||||
# same problem as it may change while the request is executing.
|
||||
#
|
||||
# TODO
|
||||
# The solution below is only partion, because it theory yields or awaits
|
||||
# can still be hidden in the body through the use of helper templates
|
||||
disallowInterruptions(body)
|
||||
|
||||
# TODO view-types
|
||||
# Avoid the code bloat produced by the double `body` reference through a lent var
|
||||
if isState(node.dag.headState):
|
||||
withStateVars(node.dag.headState):
|
||||
body
|
||||
else:
|
||||
let cachedState = if node.stateTtlCache != nil:
|
||||
node.stateTtlCache.getClosestState(blockSlot)
|
||||
else:
|
||||
nil
|
||||
|
||||
let stateToAdvance = if cachedState != nil:
|
||||
cachedState
|
||||
else:
|
||||
assignClone(node.dag.headState)
|
||||
|
||||
node.dag.updateStateData(stateToAdvance[], blockSlot, false, cache)
|
||||
|
||||
if cachedState == nil and node.stateTtlCache != nil:
|
||||
# This was not a cached state, we can cache it now
|
||||
node.stateTtlCache.add(stateToAdvance)
|
||||
|
||||
withStateVars(stateToAdvance[]):
|
||||
body
|
||||
|
||||
proc toValidatorIndex*(value: RestValidatorIndex): Result[ValidatorIndex,
|
||||
ValidatorIndexError] =
|
||||
|
|
|
@ -6,14 +6,14 @@
|
|||
import std/[typetraits, strutils, sets]
|
||||
import stew/[results, base10], chronicles,
|
||||
nimcrypto/utils as ncrutils
|
||||
import "."/rest_utils,
|
||||
".."/[beacon_chain_db, beacon_node],
|
||||
import ".."/[beacon_chain_db, beacon_node],
|
||||
".."/networking/eth2_network,
|
||||
".."/consensus_object_pools/[blockchain_dag, spec_cache,
|
||||
attestation_pool, sync_committee_msg_pool],
|
||||
".."/validators/validator_duties,
|
||||
".."/spec/[beaconstate, forks, network],
|
||||
".."/spec/datatypes/[phase0, altair]
|
||||
".."/spec/datatypes/[phase0, altair],
|
||||
"."/[rest_utils, state_ttl_cache]
|
||||
|
||||
export rest_utils
|
||||
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
# beacon_chain
|
||||
# Copyright (c) 2018-2021 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
|
||||
chronos,
|
||||
chronicles,
|
||||
../consensus_object_pools/block_pools_types
|
||||
|
||||
type
|
||||
CacheEntry = ref object
|
||||
state: ref StateData
|
||||
lastUsed: Moment
|
||||
|
||||
# This is ref object because we need to capture it by
|
||||
# reference in the `scheduleEntryExpiration` function.
|
||||
StateTtlCache* = ref object
|
||||
entries: seq[CacheEntry]
|
||||
ttl: Duration
|
||||
|
||||
const
|
||||
slotDifferenceForCacheHit = 5 * SLOTS_PER_EPOCH
|
||||
|
||||
logScope:
|
||||
topics = "state_ttl_cache"
|
||||
|
||||
proc init*(T: type StateTtlCache,
|
||||
cacheSize: Natural,
|
||||
cacheTtl: Duration): T =
|
||||
doAssert cacheSize > 0
|
||||
|
||||
StateTtlCache(
|
||||
entries: newSeq[CacheEntry](cacheSize),
|
||||
ttl: cacheTtl)
|
||||
|
||||
proc scheduleEntryExpiration(cache: StateTtlCache,
|
||||
entryIdx: int) =
|
||||
proc removeElement(arg: pointer) =
|
||||
if cache.entries[entryIdx] == nil:
|
||||
return
|
||||
let expirationTime = cache.entries[entryIdx].lastUsed + cache.ttl
|
||||
if expirationTime > Moment.now:
|
||||
return
|
||||
cache.entries[entryIdx] = nil
|
||||
debug "Cached REST state expired", index = entryIdx
|
||||
|
||||
discard setTimer(Moment.now + cache.ttl, removeElement)
|
||||
|
||||
proc add*(cache: StateTtlCache, state: ref StateData) =
|
||||
var
|
||||
now = Moment.now
|
||||
lruTime = now
|
||||
index = -1
|
||||
|
||||
for i in 0 ..< cache.entries.len:
|
||||
if cache.entries[i] == nil:
|
||||
index = i
|
||||
break
|
||||
if cache.entries[i].lastUsed <= lruTime:
|
||||
index = i
|
||||
lruTime = cache.entries[i].lastUsed
|
||||
|
||||
doAssert index != -1
|
||||
cache.entries[index] = CacheEntry(state: state, lastUsed: now)
|
||||
debug "Cached REST state added", index = index
|
||||
|
||||
cache.scheduleEntryExpiration(index)
|
||||
|
||||
proc getClosestState*(cache: StateTtlCache, bs: BlockSlot): ref StateData =
|
||||
var
|
||||
bestSlotDifference = Slot.high
|
||||
index = -1
|
||||
|
||||
for i in 0 ..< cache.entries.len:
|
||||
if cache.entries[i] == nil:
|
||||
continue
|
||||
|
||||
let stateSlot = getStateField(cache.entries[i].state.data, slot)
|
||||
if stateSlot > bs.slot:
|
||||
# We can use only states that can be advanced forward in time.
|
||||
continue
|
||||
|
||||
let slotDifference = bs.slot - stateSlot
|
||||
if slotDifference > slotDifferenceForCacheHit:
|
||||
# The state is too old to be useful as a rewind starting point.
|
||||
continue
|
||||
|
||||
var cur = bs
|
||||
for j in 0 ..< slotDifference:
|
||||
cur = cur.parentOrSlot
|
||||
|
||||
if cur.blck != cache.entries[i].state.blck:
|
||||
# The cached state and the requested BlockSlot are at different branches
|
||||
# of history.
|
||||
continue
|
||||
|
||||
if slotDifference < bestSlotDifference:
|
||||
bestSlotDifference = slotDifference.Slot
|
||||
index = i
|
||||
|
||||
if index == -1:
|
||||
return nil
|
||||
|
||||
cache.entries[index].lastUsed = Moment.now
|
||||
cache.scheduleEntryExpiration(index)
|
||||
|
||||
return cache.entries[index].state
|
|
@ -61,18 +61,17 @@ proc getStateValidatorBalances*(state_id: StateIdent
|
|||
meth: MethodGet.}
|
||||
## https://ethereum.github.io/beacon-APIs/#/Beacon/getStateValidators
|
||||
|
||||
proc getEpochCommittees*(state_id: StateIdent
|
||||
proc getEpochCommittees*(state_id: StateIdent, epoch: Option[Epoch],
|
||||
): RestResponse[GetEpochCommitteesResponse] {.
|
||||
rest, endpoint: "/eth/v1/beacon/states/{state_id}/committees",
|
||||
meth: MethodGet.}
|
||||
## https://ethereum.github.io/beacon-APIs/#/Beacon/getEpochCommittees
|
||||
|
||||
# TODO altair
|
||||
# proc getEpochSyncCommittees*(state_id: StateIdent
|
||||
# ): RestResponse[GetEpochSyncCommitteesResponse] {.
|
||||
# rest, endpoint: "/eth/v1/beacon/states/{state_id}/sync_committees",
|
||||
# meth: MethodGet.}
|
||||
# ## https://ethereum.github.io/beacon-APIs/#/Beacon/getEpochSyncCommittees
|
||||
proc getEpochSyncCommittees*(state_id: StateIdent, epoch: Option[Epoch],
|
||||
): RestResponse[GetEpochSyncCommitteesResponse] {.
|
||||
rest, endpoint: "/eth/v1/beacon/states/{state_id}/sync_committees",
|
||||
meth: MethodGet.}
|
||||
## https://ethereum.github.io/beacon-APIs/#/Beacon/getEpochSyncCommittees
|
||||
|
||||
proc getBlockHeaders*(slot: Option[Slot], parent_root: Option[Eth2Digest]
|
||||
): RestResponse[GetBlockHeadersResponse] {.
|
||||
|
|
|
@ -219,6 +219,7 @@ type
|
|||
|
||||
RestMetadata* = object
|
||||
seq_number*: string
|
||||
syncnets*: string
|
||||
attnets*: string
|
||||
|
||||
RestNetworkIdentity* = object
|
||||
|
@ -502,16 +503,19 @@ type
|
|||
GetPhase0BlockSszResponse* = phase0.SignedBeaconBlock
|
||||
GetAltairBlockSszResponse* = altair.SignedBeaconBlock
|
||||
|
||||
RestRoot* = object
|
||||
root*: Eth2Digest
|
||||
|
||||
# Types based on the OAPI yaml file - used in responses to requests
|
||||
GetAggregatedAttestationResponse* = DataEnclosedObject[Attestation]
|
||||
GetAttesterDutiesResponse* = DataRootEnclosedObject[seq[RestAttesterDuty]]
|
||||
GetBlockAttestationsResponse* = DataEnclosedObject[seq[Attestation]]
|
||||
GetBlockHeaderResponse* = DataEnclosedObject[RestBlockHeaderInfo]
|
||||
GetBlockHeadersResponse* = DataEnclosedObject[seq[RestBlockHeaderInfo]]
|
||||
GetBlockRootResponse* = DataEnclosedObject[Eth2Digest]
|
||||
GetBlockRootResponse* = DataEnclosedObject[RestRoot]
|
||||
GetDebugChainHeadsResponse* = DataEnclosedObject[seq[RestChainHead]]
|
||||
GetDepositContractResponse* = DataEnclosedObject[RestDepositContract]
|
||||
GetEpochCommitteesResponse* = DataEnclosedObject[RestGenesis]
|
||||
GetEpochCommitteesResponse* = DataEnclosedObject[seq[RestBeaconStatesCommittees]]
|
||||
GetForkScheduleResponse* = DataEnclosedObject[seq[Fork]]
|
||||
GetGenesisResponse* = DataEnclosedObject[RestGenesis]
|
||||
GetNetworkIdentityResponse* = DataEnclosedObject[RestNetworkIdentity]
|
||||
|
@ -528,7 +532,7 @@ type
|
|||
GetSpecVCResponse* = DataEnclosedObject[RestSpecVC]
|
||||
GetStateFinalityCheckpointsResponse* = DataEnclosedObject[RestBeaconStatesFinalityCheckpoints]
|
||||
GetStateForkResponse* = DataEnclosedObject[Fork]
|
||||
GetStateRootResponse* = DataEnclosedObject[Eth2Digest]
|
||||
GetStateRootResponse* = DataEnclosedObject[RestRoot]
|
||||
GetStateValidatorBalancesResponse* = DataEnclosedObject[seq[RestValidatorBalance]]
|
||||
GetStateValidatorResponse* = DataEnclosedObject[RestValidator]
|
||||
GetStateValidatorsResponse* = DataEnclosedObject[seq[RestValidator]]
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
import
|
||||
chronicles, chronicles/[topics_registry, timings],
|
||||
confutils, confutils/std/net,
|
||||
../beacon_chain/spec/eth2_apis/rest_beacon_client
|
||||
|
||||
type
|
||||
Config = object
|
||||
serverIpAddress {.
|
||||
defaultValue: ValidIpAddress.init("127.0.0.1")
|
||||
defaultValueDesc: "127.0.0.1"
|
||||
desc: "IP address of the beacon node's REST server"
|
||||
abbr: "a"
|
||||
name: "address" }: ValidIpAddress
|
||||
|
||||
serverPort {.
|
||||
defaultValue: 5052
|
||||
desc: "Listening port of the beacon node's REST server"
|
||||
abbr: "p"
|
||||
name: "port" }: Port
|
||||
|
||||
startSlot {.
|
||||
defaultValue: 0
|
||||
desc: "The starting slot from which to start history traversal"
|
||||
abbr: "s"
|
||||
name: "start-slot" }: uint
|
||||
|
||||
requestsCount {.
|
||||
desc: "Number of requests to send to the beacon node's REST server"
|
||||
abbr: "n"
|
||||
name: "count" }: uint
|
||||
|
||||
proc main =
|
||||
let config = Config.load
|
||||
let serverAddress = initTAddress(config.serverIpAddress, config.serverPort)
|
||||
let client = RestClientRef.new(serverAddress)
|
||||
|
||||
setLogLevel(LogLevel.INFO)
|
||||
|
||||
template benchmark(apiNameIdent: untyped): untyped {.dirty.} =
|
||||
block:
|
||||
const apiName = astToStr apiNameIdent
|
||||
info "Benchmarking ...", apiName
|
||||
info.logTime(apiName):
|
||||
for slot in config.startSlot ..< (config.startSlot + config.requestsCount):
|
||||
let ident = StateIdent(kind: StateQueryKind.Slot, slot: slot.Slot)
|
||||
discard waitFor client.`apiNameIdent`(ident)
|
||||
|
||||
benchmark(getStateRoot)
|
||||
benchmark(getStateFork)
|
||||
benchmark(getStateFinalityCheckpoints)
|
||||
benchmark(getStateValidatorBalances)
|
||||
|
||||
when isMainModule:
|
||||
main()
|
|
@ -0,0 +1,2 @@
|
|||
-d:"chronicles_runtime_filtering=on"
|
||||
-d:"chronicles_disable_thread_id"
|
|
@ -21,61 +21,45 @@ nimbus_beacon_node [OPTIONS]... command
|
|||
|
||||
The following options are available:
|
||||
|
||||
--log-level Sets the log level for process and topics (e.g. "DEBUG;
|
||||
TRACE:discv5,libp2p; REQUIRED:none; DISABLED:none") [=INFO].
|
||||
--log-format Specifies what kind of logs should be written to stdout (beta)
|
||||
(auto, colors, nocolors, json) [=auto].
|
||||
--log-file Specifies a path for the written Json log file.
|
||||
--log-level Sets the log level for process and topics (e.g. "DEBUG; TRACE:discv5,libp2p;
|
||||
REQUIRED:none; DISABLED:none") [=INFO].
|
||||
--log-file Specifies a path for the written Json log file (deprecated).
|
||||
--network The Eth2 network to join [=mainnet].
|
||||
-d, --data-dir The directory where nimbus will store all blockchain data.
|
||||
--validators-dir A directory containing validator keystores.
|
||||
--secrets-dir A directory containing validator keystore passwords.
|
||||
--wallets-dir A directory containing wallet files.
|
||||
--web3-url One or more Web3 provider URLs used for obtaining deposit
|
||||
contract data.
|
||||
--non-interactive Do not display interative prompts. Quit on missing
|
||||
configuration.
|
||||
--netkey-file Source of network (secp256k1) private key file
|
||||
(random|<path>) [=random].
|
||||
--insecure-netkey-password Use pre-generated INSECURE password for network private key
|
||||
file [=false].
|
||||
--agent-string Node agent string which is used as identifier in network
|
||||
[=nimbus].
|
||||
--subscribe-all-subnets Subscribe to all attestation subnet topics when gossiping
|
||||
[=false].
|
||||
--num-threads Number of worker threads (set this to 0 to use as many
|
||||
threads as there are CPU cores available) [=1].
|
||||
-b, --bootstrap-node Specifies one or more bootstrap nodes to use when connecting
|
||||
to the network.
|
||||
--bootstrap-file Specifies a line-delimited file of bootstrap Ethereum network
|
||||
addresses.
|
||||
--listen-address Listening address for the Ethereum LibP2P and Discovery v5
|
||||
traffic [=0.0.0.0].
|
||||
--web3-url One or more Web3 provider URLs used for obtaining deposit contract data.
|
||||
--non-interactive Do not display interative prompts. Quit on missing configuration.
|
||||
--netkey-file Source of network (secp256k1) private key file (random|<path>) [=random].
|
||||
--insecure-netkey-password Use pre-generated INSECURE password for network private key file [=false].
|
||||
--agent-string Node agent string which is used as identifier in network [=nimbus].
|
||||
--subscribe-all-subnets Subscribe to all attestation subnet topics when gossiping [=false].
|
||||
--num-threads Number of worker threads (set this to 0 to use as many threads as there are CPU
|
||||
cores available) [=1].
|
||||
-b, --bootstrap-node Specifies one or more bootstrap nodes to use when connecting to the network.
|
||||
--bootstrap-file Specifies a line-delimited file of bootstrap Ethereum network addresses.
|
||||
--listen-address Listening address for the Ethereum LibP2P and Discovery v5 traffic [=0.0.0.0].
|
||||
--tcp-port Listening TCP port for Ethereum LibP2P traffic [=9000].
|
||||
--udp-port Listening UDP port for node discovery [=9000].
|
||||
--max-peers The maximum number of peers to connect to [=160].
|
||||
--nat Specify method to use for determining public address. Must be
|
||||
one of: any, none, upnp, pmp, extip:<IP> [=any].
|
||||
--enr-auto-update Discovery can automatically update its ENR with the IP
|
||||
address and UDP port as seen by other nodes it communicates
|
||||
with. This option allows to enable/disable this functionality
|
||||
[=false].
|
||||
--weak-subjectivity-checkpoint Weak subjectivity checkpoint in the format
|
||||
block_root:epoch_number.
|
||||
--node-name A name for this node that will appear in the logs. If you set
|
||||
this to 'auto', a persistent automatically generated ID will
|
||||
be selected for each --data-dir folder.
|
||||
--graffiti The graffiti value that will appear in proposed blocks. You
|
||||
can use a 0x-prefixed hex encoded string to specify raw
|
||||
bytes.
|
||||
--verify-finalization Specify whether to verify finalization occurs on schedule,
|
||||
for testing [=false].
|
||||
--nat Specify method to use for determining public address. Must be one of: any, none,
|
||||
upnp, pmp, extip:<IP> [=any].
|
||||
--enr-auto-update Discovery can automatically update its ENR with the IP address and UDP port as
|
||||
seen by other nodes it communicates with. This option allows to enable/disable
|
||||
this functionality [=false].
|
||||
--weak-subjectivity-checkpoint Weak subjectivity checkpoint in the format block_root:epoch_number.
|
||||
--node-name A name for this node that will appear in the logs. If you set this to 'auto', a
|
||||
persistent automatically generated ID will be selected for each --data-dir
|
||||
folder.
|
||||
--graffiti The graffiti value that will appear in proposed blocks. You can use a
|
||||
0x-prefixed hex encoded string to specify raw bytes.
|
||||
--verify-finalization Specify whether to verify finalization occurs on schedule, for testing [=false].
|
||||
--stop-at-epoch A positive epoch selects the epoch at which to stop [=0].
|
||||
--metrics Enable the metrics server [=false].
|
||||
--metrics-address Listening address of the metrics server [=127.0.0.1].
|
||||
--metrics-port Listening HTTP port of the metrics server [=8008].
|
||||
--status-bar Display a status bar at the bottom of the terminal screen
|
||||
[=true].
|
||||
--status-bar Display a status bar at the bottom of the terminal screen [=true].
|
||||
--status-bar-contents Textual template for the contents of the status bar.
|
||||
--rpc Enable the JSON-RPC server [=false].
|
||||
--rpc-port HTTP port for the JSON-RPC service [=9190].
|
||||
|
@ -83,27 +67,32 @@ The following options are available:
|
|||
--rest Enable the REST server [=false].
|
||||
--rest-port Port for the REST server [=5052].
|
||||
--rest-address Listening address of the REST server [=127.0.0.1].
|
||||
--validator-api Enable the REST (BETA version) validator keystore management
|
||||
API [=false].
|
||||
--in-process-validators 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
|
||||
[=true].
|
||||
--rest-statecache-size The maximum number of recently accessed states that are kept in memory. Speeds
|
||||
up requests obtaining information for consecutive slots or epochs. [=3].
|
||||
--rest-statecache-ttl The number of seconds to keep recently accessed states in memory [=60].
|
||||
--keymanager Enable the REST keymanager API (BETA version) [=false].
|
||||
--keymanager-port Listening port for the REST keymanager API [=5052].
|
||||
--keymanager-address Listening port for the REST keymanager API [=127.0.0.1].
|
||||
--keymanager-token-file A file specifying the authorizition token required for accessing the keymanager
|
||||
API.
|
||||
--in-process-validators 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 [=true].
|
||||
--discv5 Enable Discovery v5 [=true].
|
||||
--dump Write SSZ dumps of blocks, attestations and states to data
|
||||
dir [=false].
|
||||
--direct-peer The list of priviledged, secure and known peers to connect
|
||||
and maintain the connection to, this requires a not random
|
||||
netkey-file. In the complete multiaddress format like:
|
||||
/ip4/<address>/tcp/<port>/p2p/<peerId-public-key>. Peering
|
||||
agreements are established out of band and must be
|
||||
reciprocal..
|
||||
--doppelganger-detection If enabled, the beacon node prudently listens for 2 epochs
|
||||
for attestations from a validator with the same index (a
|
||||
doppelganger), before sending an attestation itself. This
|
||||
protects against slashing (due to double-voting) but means
|
||||
you will miss two attestations when restarting. [=true].
|
||||
|
||||
--dump Write SSZ dumps of blocks, attestations and states to data dir [=false].
|
||||
--direct-peer The list of priviledged, secure and known peers to connect and maintain the
|
||||
connection to, this requires a not random netkey-file. In the complete
|
||||
multiaddress format like: /ip4/<address>/tcp/<port>/p2p/<peerId-public-key>.
|
||||
Peering agreements are established out of band and must be reciprocal..
|
||||
--doppelganger-detection If enabled, the beacon node prudently listens for 2 epochs for attestations from
|
||||
a validator with the same index (a doppelganger), before sending an attestation
|
||||
itself. This protects against slashing (due to double-voting) but means you will
|
||||
miss two attestations when restarting. [=true].
|
||||
--validator-monitor-auto Automatically monitor locally active validators (BETA) [=false].
|
||||
--validator-monitor-pubkey One or more validators to monitor - works best when --subscribe-all-subnets is
|
||||
enabled (BETA).
|
||||
--validator-monitor-totals Publish metrics to single 'totals' label for better collection performance when
|
||||
monitoring many validators (BETA) [=false].
|
||||
|
||||
...
|
||||
```
|
||||
|
|
Loading…
Reference in New Issue