startup cleanup
* fix several memory leaks due to temporaries not being reset during init * avoid massive main() function with lots of stuff in it * disable nim-prompt (unused) * reuse validator pool instance in eth2_processor * style cleanup
This commit is contained in:
parent
3f6834cce7
commit
0dbc7162ac
|
@ -37,7 +37,7 @@ type
|
||||||
netKeys*: KeyPair
|
netKeys*: KeyPair
|
||||||
db*: BeaconChainDB
|
db*: BeaconChainDB
|
||||||
config*: BeaconNodeConf
|
config*: BeaconNodeConf
|
||||||
attachedValidators*: ValidatorPool
|
attachedValidators*: ref ValidatorPool
|
||||||
chainDag*: ChainDAGRef
|
chainDag*: ChainDAGRef
|
||||||
quarantine*: QuarantineRef
|
quarantine*: QuarantineRef
|
||||||
attestationPool*: ref AttestationPool
|
attestationPool*: ref AttestationPool
|
||||||
|
|
|
@ -511,7 +511,7 @@ type
|
||||||
desc: "Delay in seconds between retries after unsuccessful attempts to connect to a beacon node"
|
desc: "Delay in seconds between retries after unsuccessful attempts to connect to a beacon node"
|
||||||
name: "retry-delay" }: int
|
name: "retry-delay" }: int
|
||||||
|
|
||||||
proc defaultDataDir*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
proc defaultDataDir*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
let dataDir = when defined(windows):
|
let dataDir = when defined(windows):
|
||||||
"AppData" / "Roaming" / "Nimbus"
|
"AppData" / "Roaming" / "Nimbus"
|
||||||
elif defined(macosx):
|
elif defined(macosx):
|
||||||
|
@ -521,29 +521,29 @@ proc defaultDataDir*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
|
|
||||||
getHomeDir() / dataDir / "BeaconNode"
|
getHomeDir() / dataDir / "BeaconNode"
|
||||||
|
|
||||||
func dumpDir*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
func dumpDir*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
conf.dataDir / "dump"
|
config.dataDir / "dump"
|
||||||
|
|
||||||
func dumpDirInvalid*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
func dumpDirInvalid*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
conf.dumpDir / "invalid" # things that failed validation
|
config.dumpDir / "invalid" # things that failed validation
|
||||||
|
|
||||||
func dumpDirIncoming*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
func dumpDirIncoming*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
conf.dumpDir / "incoming" # things that couldn't be validated (missingparent etc)
|
config.dumpDir / "incoming" # things that couldn't be validated (missingparent etc)
|
||||||
|
|
||||||
func dumpDirOutgoing*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
func dumpDirOutgoing*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
conf.dumpDir / "outgoing" # things we produced
|
config.dumpDir / "outgoing" # things we produced
|
||||||
|
|
||||||
proc createDumpDirs*(conf: BeaconNodeConf) =
|
proc createDumpDirs*(config: BeaconNodeConf) =
|
||||||
if conf.dumpEnabled:
|
if config.dumpEnabled:
|
||||||
let resInv = secureCreatePath(conf.dumpDirInvalid)
|
let resInv = secureCreatePath(config.dumpDirInvalid)
|
||||||
if resInv.isErr():
|
if resInv.isErr():
|
||||||
warn "Could not create dump directory", path = conf.dumpDirInvalid
|
warn "Could not create dump directory", path = config.dumpDirInvalid
|
||||||
let resInc = secureCreatePath(conf.dumpDirIncoming)
|
let resInc = secureCreatePath(config.dumpDirIncoming)
|
||||||
if resInc.isErr():
|
if resInc.isErr():
|
||||||
warn "Could not create dump directory", path = conf.dumpDirIncoming
|
warn "Could not create dump directory", path = config.dumpDirIncoming
|
||||||
let resOut = secureCreatePath(conf.dumpDirOutgoing)
|
let resOut = secureCreatePath(config.dumpDirOutgoing)
|
||||||
if resOut.isErr():
|
if resOut.isErr():
|
||||||
warn "Could not create dump directory", path = conf.dumpDirOutgoing
|
warn "Could not create dump directory", path = config.dumpDirOutgoing
|
||||||
|
|
||||||
func parseCmdArg*(T: type GraffitiBytes, input: TaintedString): T
|
func parseCmdArg*(T: type GraffitiBytes, input: TaintedString): T
|
||||||
{.raises: [ValueError, Defect].} =
|
{.raises: [ValueError, Defect].} =
|
||||||
|
@ -611,65 +611,65 @@ proc parseCmdArg*(T: type enr.Record, p: TaintedString): T
|
||||||
proc completeCmdArg*(T: type enr.Record, val: TaintedString): seq[string] =
|
proc completeCmdArg*(T: type enr.Record, val: TaintedString): seq[string] =
|
||||||
return @[]
|
return @[]
|
||||||
|
|
||||||
func validatorsDir*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
func validatorsDir*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
string conf.validatorsDirFlag.get(InputDir(conf.dataDir / "validators"))
|
string config.validatorsDirFlag.get(InputDir(config.dataDir / "validators"))
|
||||||
|
|
||||||
func secretsDir*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
func secretsDir*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
string conf.secretsDirFlag.get(InputDir(conf.dataDir / "secrets"))
|
string config.secretsDirFlag.get(InputDir(config.dataDir / "secrets"))
|
||||||
|
|
||||||
func walletsDir*(conf: BeaconNodeConf): string =
|
func walletsDir*(config: BeaconNodeConf): string =
|
||||||
if conf.walletsDirFlag.isSome:
|
if config.walletsDirFlag.isSome:
|
||||||
conf.walletsDirFlag.get.string
|
config.walletsDirFlag.get.string
|
||||||
else:
|
else:
|
||||||
conf.dataDir / "wallets"
|
config.dataDir / "wallets"
|
||||||
|
|
||||||
func outWalletName*(conf: BeaconNodeConf): Option[WalletName] =
|
func outWalletName*(config: BeaconNodeConf): Option[WalletName] =
|
||||||
proc fail {.noReturn.} =
|
proc fail {.noReturn.} =
|
||||||
raiseAssert "outWalletName should be used only in the right context"
|
raiseAssert "outWalletName should be used only in the right context"
|
||||||
|
|
||||||
case conf.cmd
|
case config.cmd
|
||||||
of wallets:
|
of wallets:
|
||||||
case conf.walletsCmd
|
case config.walletsCmd
|
||||||
of WalletsCmd.create: conf.createdWalletNameFlag
|
of WalletsCmd.create: config.createdWalletNameFlag
|
||||||
of WalletsCmd.restore: conf.restoredWalletNameFlag
|
of WalletsCmd.restore: config.restoredWalletNameFlag
|
||||||
of WalletsCmd.list: fail()
|
of WalletsCmd.list: fail()
|
||||||
of deposits:
|
of deposits:
|
||||||
# TODO: Uncomment when the deposits create command is restored
|
# TODO: Uncomment when the deposits create command is restored
|
||||||
#case conf.depositsCmd
|
#case config.depositsCmd
|
||||||
#of DepositsCmd.create: conf.newWalletNameFlag
|
#of DepositsCmd.create: config.newWalletNameFlag
|
||||||
#else: fail()
|
#else: fail()
|
||||||
fail()
|
fail()
|
||||||
else:
|
else:
|
||||||
fail()
|
fail()
|
||||||
|
|
||||||
func outWalletFile*(conf: BeaconNodeConf): Option[OutFile] =
|
func outWalletFile*(config: BeaconNodeConf): Option[OutFile] =
|
||||||
proc fail {.noReturn.} =
|
proc fail {.noReturn.} =
|
||||||
raiseAssert "outWalletName should be used only in the right context"
|
raiseAssert "outWalletName should be used only in the right context"
|
||||||
|
|
||||||
case conf.cmd
|
case config.cmd
|
||||||
of wallets:
|
of wallets:
|
||||||
case conf.walletsCmd
|
case config.walletsCmd
|
||||||
of WalletsCmd.create: conf.createdWalletFileFlag
|
of WalletsCmd.create: config.createdWalletFileFlag
|
||||||
of WalletsCmd.restore: conf.restoredWalletFileFlag
|
of WalletsCmd.restore: config.restoredWalletFileFlag
|
||||||
of WalletsCmd.list: fail()
|
of WalletsCmd.list: fail()
|
||||||
of deposits:
|
of deposits:
|
||||||
# TODO: Uncomment when the deposits create command is restored
|
# TODO: Uncomment when the deposits create command is restored
|
||||||
#case conf.depositsCmd
|
#case config.depositsCmd
|
||||||
#of DepositsCmd.create: conf.newWalletFileFlag
|
#of DepositsCmd.create: config.newWalletFileFlag
|
||||||
#else: fail()
|
#else: fail()
|
||||||
fail()
|
fail()
|
||||||
else:
|
else:
|
||||||
fail()
|
fail()
|
||||||
|
|
||||||
func databaseDir*(conf: BeaconNodeConf|ValidatorClientConf): string =
|
func databaseDir*(config: BeaconNodeConf|ValidatorClientConf): string =
|
||||||
conf.dataDir / "db"
|
config.dataDir / "db"
|
||||||
|
|
||||||
func defaultListenAddress*(conf: BeaconNodeConf|ValidatorClientConf): ValidIpAddress =
|
func defaultListenAddress*(config: BeaconNodeConf|ValidatorClientConf): ValidIpAddress =
|
||||||
# TODO: How should we select between IPv4 and IPv6
|
# TODO: How should we select between IPv4 and IPv6
|
||||||
# Maybe there should be a config option for this.
|
# Maybe there should be a config option for this.
|
||||||
(static ValidIpAddress.init("0.0.0.0"))
|
(static ValidIpAddress.init("0.0.0.0"))
|
||||||
|
|
||||||
func defaultAdminListenAddress*(conf: BeaconNodeConf|ValidatorClientConf): ValidIpAddress =
|
func defaultAdminListenAddress*(config: BeaconNodeConf|ValidatorClientConf): ValidIpAddress =
|
||||||
(static ValidIpAddress.init("127.0.0.1"))
|
(static ValidIpAddress.init("127.0.0.1"))
|
||||||
|
|
||||||
template writeValue*(writer: var JsonWriter,
|
template writeValue*(writer: var JsonWriter,
|
||||||
|
|
|
@ -1114,8 +1114,7 @@ proc getEth1BlockHash*(url: string, blockId: RtBlockIdentifier): Future[BlockHas
|
||||||
await web3.close()
|
await web3.close()
|
||||||
|
|
||||||
proc testWeb3Provider*(web3Url: Uri,
|
proc testWeb3Provider*(web3Url: Uri,
|
||||||
depositContractAddress: Option[Eth1Address],
|
depositContractAddress: Eth1Address) {.async.} =
|
||||||
depositContractDeployedAt: Option[BlockHashOrNumber]) {.async.} =
|
|
||||||
template mustSucceed(action: static string, expr: untyped): untyped =
|
template mustSucceed(action: static string, expr: untyped): untyped =
|
||||||
try: expr
|
try: expr
|
||||||
except CatchableError as err:
|
except CatchableError as err:
|
||||||
|
@ -1133,14 +1132,13 @@ proc testWeb3Provider*(web3Url: Uri,
|
||||||
echo "Network: ", network
|
echo "Network: ", network
|
||||||
echo "Latest block: ", latestBlock.number.uint64
|
echo "Latest block: ", latestBlock.number.uint64
|
||||||
|
|
||||||
if depositContractAddress.isSome:
|
let ns = web3.contractSender(DepositContract, depositContractAddress)
|
||||||
let ns = web3.contractSender(DepositContract, depositContractAddress.get)
|
|
||||||
try:
|
try:
|
||||||
let depositRoot = awaitWithRetries(
|
let depositRoot = awaitWithRetries(
|
||||||
ns.get_deposit_root.call(blockNumber = latestBlock.number.uint64))
|
ns.get_deposit_root.call(blockNumber = latestBlock.number.uint64))
|
||||||
echo "Deposit root: ", depositRoot
|
echo "Deposit root: ", depositRoot
|
||||||
except CatchableError as err:
|
except CatchableError as err:
|
||||||
echo "Web3 provider is not archive mode"
|
echo "Web3 provider is not archive mode: ", err.msg
|
||||||
|
|
||||||
when hasGenesisDetection:
|
when hasGenesisDetection:
|
||||||
proc init*(T: type Eth1Monitor,
|
proc init*(T: type Eth1Monitor,
|
||||||
|
@ -1261,4 +1259,3 @@ when hasGenesisDetection:
|
||||||
else:
|
else:
|
||||||
doAssert bnStatus == BeaconNodeStatus.Stopping
|
doAssert bnStatus == BeaconNodeStatus.Stopping
|
||||||
return new BeaconStateRef # cannot return nil...
|
return new BeaconStateRef # cannot return nil...
|
||||||
|
|
||||||
|
|
|
@ -4,7 +4,7 @@ import
|
||||||
std/[os, strutils],
|
std/[os, strutils],
|
||||||
chronicles, stew/shims/net, stew/results, bearssl,
|
chronicles, stew/shims/net, stew/results, bearssl,
|
||||||
eth/keys, eth/p2p/discoveryv5/[enr, protocol, node],
|
eth/keys, eth/p2p/discoveryv5/[enr, protocol, node],
|
||||||
conf
|
./conf
|
||||||
|
|
||||||
export protocol, keys
|
export protocol, keys
|
||||||
|
|
||||||
|
@ -74,7 +74,7 @@ proc loadBootstrapFile*(bootstrapFile: string,
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
proc new*(T: type Eth2DiscoveryProtocol,
|
proc new*(T: type Eth2DiscoveryProtocol,
|
||||||
conf: BeaconNodeConf,
|
config: BeaconNodeConf,
|
||||||
ip: Option[ValidIpAddress], tcpPort, udpPort: Port,
|
ip: Option[ValidIpAddress], tcpPort, udpPort: Port,
|
||||||
pk: PrivateKey,
|
pk: PrivateKey,
|
||||||
enrFields: openArray[(string, seq[byte])], rng: ref BrHmacDrbgContext):
|
enrFields: openArray[(string, seq[byte])], rng: ref BrHmacDrbgContext):
|
||||||
|
@ -84,14 +84,14 @@ proc new*(T: type Eth2DiscoveryProtocol,
|
||||||
# * for setting up a specific key
|
# * for setting up a specific key
|
||||||
# * for using a persistent database
|
# * for using a persistent database
|
||||||
var bootstrapEnrs: seq[enr.Record]
|
var bootstrapEnrs: seq[enr.Record]
|
||||||
for node in conf.bootstrapNodes:
|
for node in config.bootstrapNodes:
|
||||||
addBootstrapNode(node, bootstrapEnrs)
|
addBootstrapNode(node, bootstrapEnrs)
|
||||||
loadBootstrapFile(string conf.bootstrapNodesFile, bootstrapEnrs)
|
loadBootstrapFile(string config.bootstrapNodesFile, bootstrapEnrs)
|
||||||
|
|
||||||
let persistentBootstrapFile = conf.dataDir / "bootstrap_nodes.txt"
|
let persistentBootstrapFile = config.dataDir / "bootstrap_nodes.txt"
|
||||||
if fileExists(persistentBootstrapFile):
|
if fileExists(persistentBootstrapFile):
|
||||||
loadBootstrapFile(persistentBootstrapFile, bootstrapEnrs)
|
loadBootstrapFile(persistentBootstrapFile, bootstrapEnrs)
|
||||||
|
|
||||||
newProtocol(pk, ip, tcpPort, udpPort, enrFields, bootstrapEnrs,
|
newProtocol(pk, ip, tcpPort, udpPort, enrFields, bootstrapEnrs,
|
||||||
bindIp = conf.listenAddress, enrAutoUpdate = conf.enrAutoUpdate,
|
bindIp = config.listenAddress, enrAutoUpdate = config.enrAutoUpdate,
|
||||||
rng = rng)
|
rng = rng)
|
||||||
|
|
|
@ -21,11 +21,10 @@ import
|
||||||
libp2p/stream/connection,
|
libp2p/stream/connection,
|
||||||
eth/[keys, async_utils], eth/p2p/p2p_protocol_dsl,
|
eth/[keys, async_utils], eth/p2p/p2p_protocol_dsl,
|
||||||
eth/net/nat, eth/p2p/discoveryv5/[enr, node],
|
eth/net/nat, eth/p2p/discoveryv5/[enr, node],
|
||||||
# Beacon node modules
|
"."/[
|
||||||
version, conf, eth2_discovery, libp2p_json_serialization, conf,
|
version, conf, eth2_discovery, libp2p_json_serialization,
|
||||||
ssz/ssz_serialization,
|
ssz/ssz_serialization, peer_pool, time, keystore_management],
|
||||||
peer_pool, spec/[datatypes, digest, helpers, network], ./time,
|
./spec/[datatypes, digest, helpers, network]
|
||||||
keystore_management
|
|
||||||
|
|
||||||
import libp2p/protocols/pubsub/gossipsub
|
import libp2p/protocols/pubsub/gossipsub
|
||||||
|
|
||||||
|
@ -600,10 +599,11 @@ proc performProtocolHandshakes*(peer: Peer, incoming: bool) {.async.} =
|
||||||
proc initProtocol(name: string,
|
proc initProtocol(name: string,
|
||||||
peerInit: PeerStateInitializer,
|
peerInit: PeerStateInitializer,
|
||||||
networkInit: NetworkStateInitializer): ProtocolInfoObj =
|
networkInit: NetworkStateInitializer): ProtocolInfoObj =
|
||||||
result.name = name
|
ProtocolInfoObj(
|
||||||
result.messages = @[]
|
name: name,
|
||||||
result.peerStateInitializer = peerInit
|
messages: @[],
|
||||||
result.networkStateInitializer = networkInit
|
peerStateInitializer: peerInit,
|
||||||
|
networkStateInitializer: networkInit)
|
||||||
|
|
||||||
proc registerProtocol(protocol: ProtocolInfo) =
|
proc registerProtocol(protocol: ProtocolInfo) =
|
||||||
# TODO: This can be done at compile-time in the future
|
# TODO: This can be done at compile-time in the future
|
||||||
|
@ -913,8 +913,8 @@ proc runDiscoveryLoop*(node: Eth2Node) {.async.} =
|
||||||
# when no peers are in the routing table. Don't run it in continuous loop.
|
# when no peers are in the routing table. Don't run it in continuous loop.
|
||||||
await sleepAsync(1.seconds)
|
await sleepAsync(1.seconds)
|
||||||
|
|
||||||
proc getPersistentNetMetadata*(conf: BeaconNodeConf): Eth2Metadata =
|
proc getPersistentNetMetadata*(config: BeaconNodeConf): Eth2Metadata =
|
||||||
let metadataPath = conf.dataDir / nodeMetadataFilename
|
let metadataPath = config.dataDir / nodeMetadataFilename
|
||||||
if not fileExists(metadataPath):
|
if not fileExists(metadataPath):
|
||||||
result = Eth2Metadata()
|
result = Eth2Metadata()
|
||||||
for i in 0 ..< ATTESTATION_SUBNET_COUNT:
|
for i in 0 ..< ATTESTATION_SUBNET_COUNT:
|
||||||
|
@ -1058,53 +1058,60 @@ proc onConnEvent(node: Eth2Node, peerId: PeerID, event: ConnEvent) {.async.} =
|
||||||
peer = peerId, peer_state = peer.connectionState
|
peer = peerId, peer_state = peer.connectionState
|
||||||
peer.connectionState = Disconnected
|
peer.connectionState = Disconnected
|
||||||
|
|
||||||
proc init*(T: type Eth2Node, conf: BeaconNodeConf, enrForkId: ENRForkID,
|
proc new*(T: type Eth2Node, config: BeaconNodeConf, enrForkId: ENRForkID,
|
||||||
switch: Switch, pubsub: GossipSub, ip: Option[ValidIpAddress],
|
switch: Switch, pubsub: GossipSub, ip: Option[ValidIpAddress],
|
||||||
tcpPort, udpPort: Port, privKey: keys.PrivateKey, discovery: bool,
|
tcpPort, udpPort: Port, privKey: keys.PrivateKey, discovery: bool,
|
||||||
rng: ref BrHmacDrbgContext): T =
|
rng: ref BrHmacDrbgContext): T =
|
||||||
new result
|
let
|
||||||
result.switch = switch
|
metadata = getPersistentNetMetadata(config)
|
||||||
result.pubsub = pubsub
|
|
||||||
result.wantedPeers = conf.maxPeers
|
|
||||||
result.peerPool = newPeerPool[Peer, PeerID](maxPeers = conf.maxPeers)
|
|
||||||
when not defined(local_testnet):
|
when not defined(local_testnet):
|
||||||
result.connectTimeout = 1.minutes
|
let
|
||||||
result.seenThreshold = 5.minutes
|
connectTimeout = 1.minutes
|
||||||
|
seenThreshold = 5.minutes
|
||||||
else:
|
else:
|
||||||
result.connectTimeout = 10.seconds
|
let
|
||||||
result.seenThreshold = 10.seconds
|
connectTimeout = 10.seconds
|
||||||
result.seenTable = initTable[PeerID, SeenItem]()
|
seenThreshold = 10.seconds
|
||||||
result.connTable = initHashSet[PeerID]()
|
|
||||||
|
let node = T(
|
||||||
|
switch: switch,
|
||||||
|
pubsub: pubsub,
|
||||||
|
wantedPeers: config.maxPeers,
|
||||||
|
peerPool: newPeerPool[Peer, PeerID](maxPeers = config.maxPeers),
|
||||||
# Its important here to create AsyncQueue with limited size, otherwise
|
# Its important here to create AsyncQueue with limited size, otherwise
|
||||||
# it could produce HIGH cpu usage.
|
# it could produce HIGH cpu usage.
|
||||||
result.connQueue = newAsyncQueue[PeerAddr](ConcurrentConnections)
|
connQueue: newAsyncQueue[PeerAddr](ConcurrentConnections),
|
||||||
# TODO: The persistent net metadata should only be used in the case of reusing
|
# TODO: The persistent net metadata should only be used in the case of reusing
|
||||||
# the previous netkey.
|
# the previous netkey.
|
||||||
result.metadata = getPersistentNetMetadata(conf)
|
metadata: metadata,
|
||||||
result.forkId = enrForkId
|
forkId: enrForkId,
|
||||||
result.discovery = Eth2DiscoveryProtocol.new(
|
discovery: Eth2DiscoveryProtocol.new(
|
||||||
conf, ip, tcpPort, udpPort, privKey,
|
config, ip, tcpPort, udpPort, privKey,
|
||||||
{"eth2": SSZ.encode(result.forkId), "attnets": SSZ.encode(result.metadata.attnets)},
|
{"eth2": SSZ.encode(enrForkId), "attnets": SSZ.encode(metadata.attnets)},
|
||||||
rng)
|
rng),
|
||||||
result.discoveryEnabled = discovery
|
discoveryEnabled: discovery,
|
||||||
result.rng = rng
|
rng: rng,
|
||||||
|
connectTimeout: connectTimeout,
|
||||||
|
seenThreshold: seenThreshold,
|
||||||
|
)
|
||||||
|
|
||||||
newSeq result.protocolStates, allProtocols.len
|
newSeq node.protocolStates, allProtocols.len
|
||||||
for proto in allProtocols:
|
for proto in allProtocols:
|
||||||
if proto.networkStateInitializer != nil:
|
if proto.networkStateInitializer != nil:
|
||||||
result.protocolStates[proto.index] = proto.networkStateInitializer(result)
|
node.protocolStates[proto.index] = proto.networkStateInitializer(node)
|
||||||
|
|
||||||
for msg in proto.messages:
|
for msg in proto.messages:
|
||||||
if msg.protocolMounter != nil:
|
if msg.protocolMounter != nil:
|
||||||
msg.protocolMounter result
|
msg.protocolMounter node
|
||||||
|
|
||||||
let node = result
|
|
||||||
proc peerHook(peerId: PeerID, event: ConnEvent): Future[void] {.gcsafe.} =
|
proc peerHook(peerId: PeerID, event: ConnEvent): Future[void] {.gcsafe.} =
|
||||||
onConnEvent(node, peerId, event)
|
onConnEvent(node, peerId, event)
|
||||||
|
|
||||||
switch.addConnEventHandler(peerHook, ConnEventKind.Connected)
|
switch.addConnEventHandler(peerHook, ConnEventKind.Connected)
|
||||||
switch.addConnEventHandler(peerHook, ConnEventKind.Disconnected)
|
switch.addConnEventHandler(peerHook, ConnEventKind.Disconnected)
|
||||||
|
|
||||||
|
node
|
||||||
|
|
||||||
template publicKey*(node: Eth2Node): keys.PublicKey =
|
template publicKey*(node: Eth2Node): keys.PublicKey =
|
||||||
node.discovery.privKey.toPublicKey
|
node.discovery.privKey.toPublicKey
|
||||||
|
|
||||||
|
@ -1306,15 +1313,15 @@ proc p2pProtocolBackendImpl*(p: P2PProtocol): Backend =
|
||||||
result.implementProtocolInit = proc (p: P2PProtocol): NimNode =
|
result.implementProtocolInit = proc (p: P2PProtocol): NimNode =
|
||||||
return newCall(initProtocol, newLit(p.name), p.peerInit, p.netInit)
|
return newCall(initProtocol, newLit(p.name), p.peerInit, p.netInit)
|
||||||
|
|
||||||
proc setupNat(conf: BeaconNodeConf): tuple[ip: Option[ValidIpAddress],
|
proc setupNat(config: BeaconNodeConf): tuple[ip: Option[ValidIpAddress],
|
||||||
tcpPort: Port,
|
tcpPort: Port,
|
||||||
udpPort: Port] =
|
udpPort: Port] =
|
||||||
# defaults
|
# defaults
|
||||||
result.tcpPort = conf.tcpPort
|
result.tcpPort = config.tcpPort
|
||||||
result.udpPort = conf.udpPort
|
result.udpPort = config.udpPort
|
||||||
|
|
||||||
var nat: NatStrategy
|
var nat: NatStrategy
|
||||||
case conf.nat.toLowerAscii:
|
case config.nat.toLowerAscii:
|
||||||
of "any":
|
of "any":
|
||||||
nat = NatAny
|
nat = NatAny
|
||||||
of "none":
|
of "none":
|
||||||
|
@ -1324,16 +1331,16 @@ proc setupNat(conf: BeaconNodeConf): tuple[ip: Option[ValidIpAddress],
|
||||||
of "pmp":
|
of "pmp":
|
||||||
nat = NatPmp
|
nat = NatPmp
|
||||||
else:
|
else:
|
||||||
if conf.nat.startsWith("extip:"):
|
if config.nat.startsWith("extip:"):
|
||||||
try:
|
try:
|
||||||
# any required port redirection is assumed to be done by hand
|
# any required port redirection is assumed to be done by hand
|
||||||
result.ip = some(ValidIpAddress.init(conf.nat[6..^1]))
|
result.ip = some(ValidIpAddress.init(config.nat[6..^1]))
|
||||||
nat = NatNone
|
nat = NatNone
|
||||||
except ValueError:
|
except ValueError:
|
||||||
error "nor a valid IP address", address = conf.nat[6..^1]
|
error "nor a valid IP address", address = config.nat[6..^1]
|
||||||
quit QuitFailure
|
quit QuitFailure
|
||||||
else:
|
else:
|
||||||
error "not a valid NAT mechanism", value = conf.nat
|
error "not a valid NAT mechanism", value = config.nat
|
||||||
quit QuitFailure
|
quit QuitFailure
|
||||||
|
|
||||||
if nat != NatNone:
|
if nat != NatNone:
|
||||||
|
@ -1367,10 +1374,10 @@ template tcpEndPoint(address, port): auto =
|
||||||
MultiAddress.init(address, tcpProtocol, port)
|
MultiAddress.init(address, tcpProtocol, port)
|
||||||
|
|
||||||
proc getPersistentNetKeys*(rng: var BrHmacDrbgContext,
|
proc getPersistentNetKeys*(rng: var BrHmacDrbgContext,
|
||||||
conf: BeaconNodeConf): KeyPair =
|
config: BeaconNodeConf): KeyPair =
|
||||||
case conf.cmd
|
case config.cmd
|
||||||
of noCommand, record:
|
of noCommand, record:
|
||||||
if conf.netKeyFile == "random":
|
if config.netKeyFile == "random":
|
||||||
let res = PrivateKey.random(Secp256k1, rng)
|
let res = PrivateKey.random(Secp256k1, rng)
|
||||||
if res.isErr():
|
if res.isErr():
|
||||||
fatal "Could not generate random network key file"
|
fatal "Could not generate random network key file"
|
||||||
|
@ -1386,17 +1393,17 @@ proc getPersistentNetKeys*(rng: var BrHmacDrbgContext,
|
||||||
return KeyPair(seckey: privKey, pubkey: privKey.getKey().tryGet())
|
return KeyPair(seckey: privKey, pubkey: privKey.getKey().tryGet())
|
||||||
else:
|
else:
|
||||||
let keyPath =
|
let keyPath =
|
||||||
if isAbsolute(conf.netKeyFile):
|
if isAbsolute(config.netKeyFile):
|
||||||
conf.netKeyFile
|
config.netKeyFile
|
||||||
else:
|
else:
|
||||||
conf.dataDir / conf.netKeyFile
|
config.dataDir / config.netKeyFile
|
||||||
|
|
||||||
if fileAccessible(keyPath, {AccessFlags.Find}):
|
if fileAccessible(keyPath, {AccessFlags.Find}):
|
||||||
info "Network key storage is present, unlocking", key_path = keyPath
|
info "Network key storage is present, unlocking", key_path = keyPath
|
||||||
|
|
||||||
# Insecure password used only for automated testing.
|
# Insecure password used only for automated testing.
|
||||||
let insecurePassword =
|
let insecurePassword =
|
||||||
if conf.netKeyInsecurePassword:
|
if config.netKeyInsecurePassword:
|
||||||
some(NetworkInsecureKeyPassword)
|
some(NetworkInsecureKeyPassword)
|
||||||
else:
|
else:
|
||||||
none[string]()
|
none[string]()
|
||||||
|
@ -1423,7 +1430,7 @@ proc getPersistentNetKeys*(rng: var BrHmacDrbgContext,
|
||||||
|
|
||||||
# Insecure password used only for automated testing.
|
# Insecure password used only for automated testing.
|
||||||
let insecurePassword =
|
let insecurePassword =
|
||||||
if conf.netKeyInsecurePassword:
|
if config.netKeyInsecurePassword:
|
||||||
some(NetworkInsecureKeyPassword)
|
some(NetworkInsecureKeyPassword)
|
||||||
else:
|
else:
|
||||||
none[string]()
|
none[string]()
|
||||||
|
@ -1438,15 +1445,15 @@ proc getPersistentNetKeys*(rng: var BrHmacDrbgContext,
|
||||||
return KeyPair(seckey: privKey, pubkey: pubKey)
|
return KeyPair(seckey: privKey, pubkey: pubKey)
|
||||||
|
|
||||||
of createTestnet:
|
of createTestnet:
|
||||||
if conf.netKeyFile == "random":
|
if config.netKeyFile == "random":
|
||||||
fatal "Could not create testnet using `random` network key"
|
fatal "Could not create testnet using `random` network key"
|
||||||
quit QuitFailure
|
quit QuitFailure
|
||||||
|
|
||||||
let keyPath =
|
let keyPath =
|
||||||
if isAbsolute(conf.netKeyFile):
|
if isAbsolute(config.netKeyFile):
|
||||||
conf.netKeyFile
|
config.netKeyFile
|
||||||
else:
|
else:
|
||||||
conf.dataDir / conf.netKeyFile
|
config.dataDir / config.netKeyFile
|
||||||
|
|
||||||
let rres = PrivateKey.random(Secp256k1, rng)
|
let rres = PrivateKey.random(Secp256k1, rng)
|
||||||
if rres.isErr():
|
if rres.isErr():
|
||||||
|
@ -1458,7 +1465,7 @@ proc getPersistentNetKeys*(rng: var BrHmacDrbgContext,
|
||||||
|
|
||||||
# Insecure password used only for automated testing.
|
# Insecure password used only for automated testing.
|
||||||
let insecurePassword =
|
let insecurePassword =
|
||||||
if conf.netKeyInsecurePassword:
|
if config.netKeyInsecurePassword:
|
||||||
some(NetworkInsecureKeyPassword)
|
some(NetworkInsecureKeyPassword)
|
||||||
else:
|
else:
|
||||||
none[string]()
|
none[string]()
|
||||||
|
@ -1501,7 +1508,7 @@ func msgIdProvider(m: messages.Message): seq[byte] =
|
||||||
except CatchableError:
|
except CatchableError:
|
||||||
gossipId(m.data, false)
|
gossipId(m.data, false)
|
||||||
|
|
||||||
proc newBeaconSwitch*(conf: BeaconNodeConf, seckey: PrivateKey,
|
proc newBeaconSwitch*(config: BeaconNodeConf, seckey: PrivateKey,
|
||||||
address: MultiAddress,
|
address: MultiAddress,
|
||||||
rng: ref BrHmacDrbgContext): Switch =
|
rng: ref BrHmacDrbgContext): Switch =
|
||||||
proc createMplex(conn: Connection): Muxer =
|
proc createMplex(conn: Connection): Muxer =
|
||||||
|
@ -1514,7 +1521,7 @@ proc newBeaconSwitch*(conf: BeaconNodeConf, seckey: PrivateKey,
|
||||||
muxers = {MplexCodec: mplexProvider}.toTable
|
muxers = {MplexCodec: mplexProvider}.toTable
|
||||||
secureManagers = [Secure(newNoise(rng, seckey))]
|
secureManagers = [Secure(newNoise(rng, seckey))]
|
||||||
|
|
||||||
peerInfo.agentVersion = conf.agentString
|
peerInfo.agentVersion = config.agentString
|
||||||
|
|
||||||
let identify = newIdentify(peerInfo)
|
let identify = newIdentify(peerInfo)
|
||||||
|
|
||||||
|
@ -1524,15 +1531,15 @@ proc newBeaconSwitch*(conf: BeaconNodeConf, seckey: PrivateKey,
|
||||||
identify,
|
identify,
|
||||||
muxers,
|
muxers,
|
||||||
secureManagers,
|
secureManagers,
|
||||||
maxConnections = conf.maxPeers)
|
maxConnections = config.maxPeers)
|
||||||
|
|
||||||
proc createEth2Node*(rng: ref BrHmacDrbgContext,
|
proc createEth2Node*(rng: ref BrHmacDrbgContext,
|
||||||
conf: BeaconNodeConf,
|
config: BeaconNodeConf,
|
||||||
netKeys: KeyPair,
|
netKeys: KeyPair,
|
||||||
enrForkId: ENRForkID): Eth2Node =
|
enrForkId: ENRForkID): Eth2Node =
|
||||||
var
|
var
|
||||||
(extIp, extTcpPort, extUdpPort) = setupNat(conf)
|
(extIp, extTcpPort, extUdpPort) = setupNat(config)
|
||||||
hostAddress = tcpEndPoint(conf.listenAddress, conf.tcpPort)
|
hostAddress = tcpEndPoint(config.listenAddress, config.tcpPort)
|
||||||
announcedAddresses = if extIp.isNone(): @[]
|
announcedAddresses = if extIp.isNone(): @[]
|
||||||
else: @[tcpEndPoint(extIp.get(), extTcpPort)]
|
else: @[tcpEndPoint(extIp.get(), extTcpPort)]
|
||||||
|
|
||||||
|
@ -1543,7 +1550,7 @@ proc createEth2Node*(rng: ref BrHmacDrbgContext,
|
||||||
# TODO nim-libp2p still doesn't have support for announcing addresses
|
# TODO nim-libp2p still doesn't have support for announcing addresses
|
||||||
# that are different from the host address (this is relevant when we
|
# that are different from the host address (this is relevant when we
|
||||||
# are running behind a NAT).
|
# are running behind a NAT).
|
||||||
var switch = newBeaconSwitch(conf, netKeys.seckey, hostAddress, rng)
|
var switch = newBeaconSwitch(config, netKeys.seckey, hostAddress, rng)
|
||||||
|
|
||||||
let
|
let
|
||||||
params = GossipSubParams(
|
params = GossipSubParams(
|
||||||
|
@ -1587,10 +1594,10 @@ proc createEth2Node*(rng: ref BrHmacDrbgContext,
|
||||||
|
|
||||||
switch.mount(pubsub)
|
switch.mount(pubsub)
|
||||||
|
|
||||||
result = Eth2Node.init(conf, enrForkId, switch, pubsub,
|
Eth2Node.new(config, enrForkId, switch, pubsub,
|
||||||
extIp, extTcpPort, extUdpPort,
|
extIp, extTcpPort, extUdpPort,
|
||||||
netKeys.seckey.asEthKey,
|
netKeys.seckey.asEthKey,
|
||||||
discovery = conf.discv5Enabled,
|
discovery = config.discv5Enabled,
|
||||||
rng = rng)
|
rng = rng)
|
||||||
|
|
||||||
proc announcedENR*(node: Eth2Node): enr.Record =
|
proc announcedENR*(node: Eth2Node): enr.Record =
|
||||||
|
|
|
@ -5,7 +5,7 @@ import
|
||||||
spec/[datatypes, digest, crypto, keystore],
|
spec/[datatypes, digest, crypto, keystore],
|
||||||
stew/io2, libp2p/crypto/crypto as lcrypto,
|
stew/io2, libp2p/crypto/crypto as lcrypto,
|
||||||
nimcrypto/utils as ncrutils,
|
nimcrypto/utils as ncrutils,
|
||||||
conf, ssz/merkleization, network_metadata, filepath
|
"."/[conf, ssz/merkleization, network_metadata, filepath]
|
||||||
|
|
||||||
export
|
export
|
||||||
keystore
|
keystore
|
||||||
|
@ -280,13 +280,13 @@ iterator validatorKeysFromDirs*(validatorsDir, secretsDir: string): ValidatorPri
|
||||||
except OSError:
|
except OSError:
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
iterator validatorKeys*(conf: BeaconNodeConf|ValidatorClientConf): ValidatorPrivKey =
|
iterator validatorKeys*(config: BeaconNodeConf|ValidatorClientConf): ValidatorPrivKey =
|
||||||
let validatorsDir = conf.validatorsDir
|
let validatorsDir = config.validatorsDir
|
||||||
try:
|
try:
|
||||||
for kind, file in walkDir(validatorsDir):
|
for kind, file in walkDir(validatorsDir):
|
||||||
if kind == pcDir:
|
if kind == pcDir:
|
||||||
let keyName = splitFile(file).name
|
let keyName = splitFile(file).name
|
||||||
let key = loadKeystore(validatorsDir, conf.secretsDir, keyName, conf.nonInteractive)
|
let key = loadKeystore(validatorsDir, config.secretsDir, keyName, config.nonInteractive)
|
||||||
if key.isSome:
|
if key.isSome:
|
||||||
yield key.get
|
yield key.get
|
||||||
else:
|
else:
|
||||||
|
|
|
@ -22,23 +22,20 @@ import
|
||||||
eth/p2p/enode, eth/p2p/discoveryv5/[protocol, enr, random2],
|
eth/p2p/enode, eth/p2p/discoveryv5/[protocol, enr, random2],
|
||||||
|
|
||||||
# Local modules
|
# Local modules
|
||||||
|
"."/[
|
||||||
|
attestation_aggregation, attestation_pool, beacon_chain_db,
|
||||||
|
beacon_node_common, beacon_node_status, beacon_node_types, conf,
|
||||||
|
eth1_monitor, eth2_discovery, eth2_network, eth2_processor, exit_pool,
|
||||||
|
extras, filepath, interop, keystore_management, network_metadata,
|
||||||
|
nimbus_binary_common, request_manager, ssz/merkleization, statusbar,
|
||||||
|
sync_manager, sync_protocol, time, validator_duties, validator_pool,
|
||||||
|
validator_protection/slashing_protection, version,],
|
||||||
./rpc/[beacon_api, config_api, debug_api, event_api, nimbus_api, node_api,
|
./rpc/[beacon_api, config_api, debug_api, event_api, nimbus_api, node_api,
|
||||||
validator_api],
|
validator_api],
|
||||||
spec/[
|
./spec/[
|
||||||
datatypes, digest, crypto, beaconstate, helpers, network, presets,
|
datatypes, digest, crypto, beaconstate, eth2_apis/beacon_rpc_client,
|
||||||
validator],
|
helpers, network, presets, validator, weak_subjectivity, signatures],
|
||||||
spec/[weak_subjectivity, signatures],
|
./block_pools/[chain_dag, quarantine, clearance, block_pools_types]
|
||||||
spec/eth2_apis/beacon_rpc_client,
|
|
||||||
conf, time, beacon_chain_db, validator_pool, extras,
|
|
||||||
attestation_aggregation, attestation_pool, exit_pool, eth2_network,
|
|
||||||
eth2_discovery,
|
|
||||||
beacon_node_common, beacon_node_types, beacon_node_status,
|
|
||||||
block_pools/[chain_dag, quarantine, clearance, block_pools_types],
|
|
||||||
nimbus_binary_common, network_metadata,
|
|
||||||
eth1_monitor, version, ssz/merkleization,
|
|
||||||
sync_protocol, request_manager, keystore_management, interop, statusbar,
|
|
||||||
sync_manager, validator_duties, filepath,
|
|
||||||
validator_protection/slashing_protection, ./eth2_processor
|
|
||||||
|
|
||||||
from eth/common/eth_types import BlockHashOrNumber
|
from eth/common/eth_types import BlockHashOrNumber
|
||||||
|
|
||||||
|
@ -48,7 +45,7 @@ import
|
||||||
TopicParams, validateParameters, init
|
TopicParams, validateParameters, init
|
||||||
|
|
||||||
const
|
const
|
||||||
hasPrompt = not defined(withoutPrompt)
|
hasPrompt = false and not defined(withoutPrompt) # disabled, doesn't work
|
||||||
|
|
||||||
type
|
type
|
||||||
RpcServer* = RpcHttpServer
|
RpcServer* = RpcHttpServer
|
||||||
|
@ -82,21 +79,21 @@ func enrForkIdFromState(state: BeaconState): ENRForkID =
|
||||||
proc init*(T: type BeaconNode,
|
proc init*(T: type BeaconNode,
|
||||||
runtimePreset: RuntimePreset,
|
runtimePreset: RuntimePreset,
|
||||||
rng: ref BrHmacDrbgContext,
|
rng: ref BrHmacDrbgContext,
|
||||||
conf: BeaconNodeConf,
|
config: BeaconNodeConf,
|
||||||
depositContractAddress: Eth1Address,
|
depositContractAddress: Eth1Address,
|
||||||
depositContractDeployedAt: BlockHashOrNumber,
|
depositContractDeployedAt: BlockHashOrNumber,
|
||||||
eth1Network: Option[Eth1Network],
|
eth1Network: Option[Eth1Network],
|
||||||
genesisStateContents: ref string,
|
genesisStateContents: string,
|
||||||
genesisDepositsSnapshotContents: ref string): Future[BeaconNode] {.async.} =
|
genesisDepositsSnapshotContents: string): BeaconNode =
|
||||||
let
|
let
|
||||||
db = BeaconChainDB.init(runtimePreset, conf.databaseDir)
|
db = BeaconChainDB.init(runtimePreset, config.databaseDir)
|
||||||
|
|
||||||
var
|
var
|
||||||
genesisState, checkpointState: ref BeaconState
|
genesisState, checkpointState: ref BeaconState
|
||||||
checkpointBlock: SignedBeaconBlock
|
checkpointBlock: SignedBeaconBlock
|
||||||
|
|
||||||
if conf.finalizedCheckpointState.isSome:
|
if config.finalizedCheckpointState.isSome:
|
||||||
let checkpointStatePath = conf.finalizedCheckpointState.get.string
|
let checkpointStatePath = config.finalizedCheckpointState.get.string
|
||||||
checkpointState = try:
|
checkpointState = try:
|
||||||
newClone(SSZ.loadFile(checkpointStatePath, BeaconState))
|
newClone(SSZ.loadFile(checkpointStatePath, BeaconState))
|
||||||
except SerializationError as err:
|
except SerializationError as err:
|
||||||
|
@ -107,12 +104,12 @@ proc init*(T: type BeaconNode,
|
||||||
fatal "Failed to read checkpoint state file", err = err.msg
|
fatal "Failed to read checkpoint state file", err = err.msg
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
if conf.finalizedCheckpointBlock.isNone:
|
if config.finalizedCheckpointBlock.isNone:
|
||||||
if checkpointState.slot > 0:
|
if checkpointState.slot > 0:
|
||||||
fatal "Specifying a non-genesis --finalized-checkpoint-state requires specifying --finalized-checkpoint-block as well"
|
fatal "Specifying a non-genesis --finalized-checkpoint-state requires specifying --finalized-checkpoint-block as well"
|
||||||
quit 1
|
quit 1
|
||||||
else:
|
else:
|
||||||
let checkpointBlockPath = conf.finalizedCheckpointBlock.get.string
|
let checkpointBlockPath = config.finalizedCheckpointBlock.get.string
|
||||||
try:
|
try:
|
||||||
checkpointBlock = SSZ.loadFile(checkpointBlockPath, SignedBeaconBlock)
|
checkpointBlock = SSZ.loadFile(checkpointBlockPath, SignedBeaconBlock)
|
||||||
except SerializationError as err:
|
except SerializationError as err:
|
||||||
|
@ -121,7 +118,7 @@ proc init*(T: type BeaconNode,
|
||||||
except IOError as err:
|
except IOError as err:
|
||||||
fatal "Failed to load the checkpoint block", err = err.msg
|
fatal "Failed to load the checkpoint block", err = err.msg
|
||||||
quit 1
|
quit 1
|
||||||
elif conf.finalizedCheckpointBlock.isSome:
|
elif config.finalizedCheckpointBlock.isSome:
|
||||||
# TODO We can download the state from somewhere in the future relying
|
# TODO We can download the state from somewhere in the future relying
|
||||||
# on the trusted `state_root` appearing in the checkpoint block.
|
# on the trusted `state_root` appearing in the checkpoint block.
|
||||||
fatal "--finalized-checkpoint-block cannot be specified without --finalized-checkpoint-state"
|
fatal "--finalized-checkpoint-block cannot be specified without --finalized-checkpoint-state"
|
||||||
|
@ -133,7 +130,7 @@ proc init*(T: type BeaconNode,
|
||||||
tailState: ref BeaconState
|
tailState: ref BeaconState
|
||||||
tailBlock: SignedBeaconBlock
|
tailBlock: SignedBeaconBlock
|
||||||
|
|
||||||
if genesisStateContents == nil and checkpointState == nil:
|
if genesisStateContents.len == 0 and checkpointState == nil:
|
||||||
when hasGenesisDetection:
|
when hasGenesisDetection:
|
||||||
if genesisDepositsSnapshotContents != nil:
|
if genesisDepositsSnapshotContents != nil:
|
||||||
fatal "A deposits snapshot cannot be provided without also providing a matching beacon state snapshot"
|
fatal "A deposits snapshot cannot be provided without also providing a matching beacon state snapshot"
|
||||||
|
@ -142,16 +139,16 @@ proc init*(T: type BeaconNode,
|
||||||
# This is a fresh start without a known genesis state
|
# This is a fresh start without a known genesis state
|
||||||
# (most likely, it hasn't arrived yet). We'll try to
|
# (most likely, it hasn't arrived yet). We'll try to
|
||||||
# obtain a genesis through the Eth1 deposits monitor:
|
# obtain a genesis through the Eth1 deposits monitor:
|
||||||
if conf.web3Url.len == 0:
|
if config.web3Url.len == 0:
|
||||||
fatal "Web3 URL not specified"
|
fatal "Web3 URL not specified"
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
# TODO Could move this to a separate "GenesisMonitor" process or task
|
# TODO Could move this to a separate "GenesisMonitor" process or task
|
||||||
# that would do only this - see Paul's proposal for this.
|
# that would do only this - see Paul's proposal for this.
|
||||||
let eth1MonitorRes = await Eth1Monitor.init(
|
let eth1MonitorRes = waitFor Eth1Monitor.init(
|
||||||
runtimePreset,
|
runtimePreset,
|
||||||
db,
|
db,
|
||||||
conf.web3Url,
|
config.web3Url,
|
||||||
depositContractAddress,
|
depositContractAddress,
|
||||||
depositContractDeployedAt,
|
depositContractDeployedAt,
|
||||||
eth1Network)
|
eth1Network)
|
||||||
|
@ -159,14 +156,14 @@ proc init*(T: type BeaconNode,
|
||||||
if eth1MonitorRes.isErr:
|
if eth1MonitorRes.isErr:
|
||||||
fatal "Failed to start Eth1 monitor",
|
fatal "Failed to start Eth1 monitor",
|
||||||
reason = eth1MonitorRes.error,
|
reason = eth1MonitorRes.error,
|
||||||
web3Url = conf.web3Url,
|
web3Url = config.web3Url,
|
||||||
depositContractAddress,
|
depositContractAddress,
|
||||||
depositContractDeployedAt
|
depositContractDeployedAt
|
||||||
quit 1
|
quit 1
|
||||||
else:
|
else:
|
||||||
eth1Monitor = eth1MonitorRes.get
|
eth1Monitor = eth1MonitorRes.get
|
||||||
|
|
||||||
genesisState = await eth1Monitor.waitGenesis()
|
genesisState = waitFor eth1Monitor.waitGenesis()
|
||||||
if bnStatus == BeaconNodeStatus.Stopping:
|
if bnStatus == BeaconNodeStatus.Stopping:
|
||||||
return nil
|
return nil
|
||||||
|
|
||||||
|
@ -182,7 +179,7 @@ proc init*(T: type BeaconNode,
|
||||||
"in order to support monitoring for genesis events"
|
"in order to support monitoring for genesis events"
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
elif genesisStateContents == nil:
|
elif genesisStateContents.len == 0:
|
||||||
if checkpointState.slot == GENESIS_SLOT:
|
if checkpointState.slot == GENESIS_SLOT:
|
||||||
genesisState = checkpointState
|
genesisState = checkpointState
|
||||||
tailState = checkpointState
|
tailState = checkpointState
|
||||||
|
@ -192,7 +189,7 @@ proc init*(T: type BeaconNode,
|
||||||
quit 1
|
quit 1
|
||||||
else:
|
else:
|
||||||
try:
|
try:
|
||||||
genesisState = newClone(SSZ.decode(genesisStateContents[], BeaconState))
|
genesisState = newClone(SSZ.decode(genesisStateContents, BeaconState))
|
||||||
except CatchableError as err:
|
except CatchableError as err:
|
||||||
raiseAssert "Invalid baked-in state: " & err.msg
|
raiseAssert "Invalid baked-in state: " & err.msg
|
||||||
|
|
||||||
|
@ -210,10 +207,10 @@ proc init*(T: type BeaconNode,
|
||||||
error "Failed to initialize database", err = e.msg
|
error "Failed to initialize database", err = e.msg
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
info "Loading block dag from database", path = conf.databaseDir
|
info "Loading block dag from database", path = config.databaseDir
|
||||||
|
|
||||||
let
|
let
|
||||||
chainDagFlags = if conf.verifyFinalization: {verifyFinalization}
|
chainDagFlags = if config.verifyFinalization: {verifyFinalization}
|
||||||
else: {}
|
else: {}
|
||||||
chainDag = ChainDAGRef.init(runtimePreset, db, chainDagFlags)
|
chainDag = ChainDAGRef.init(runtimePreset, db, chainDagFlags)
|
||||||
beaconClock = BeaconClock.init(chainDag.headState.data.data)
|
beaconClock = BeaconClock.init(chainDag.headState.data.data)
|
||||||
|
@ -221,29 +218,29 @@ proc init*(T: type BeaconNode,
|
||||||
databaseGenesisValidatorsRoot =
|
databaseGenesisValidatorsRoot =
|
||||||
chainDag.headState.data.data.genesis_validators_root
|
chainDag.headState.data.data.genesis_validators_root
|
||||||
|
|
||||||
if genesisStateContents != nil:
|
if genesisStateContents.len != 0:
|
||||||
let
|
let
|
||||||
networkGenesisValidatorsRoot =
|
networkGenesisValidatorsRoot =
|
||||||
extractGenesisValidatorRootFromSnapshop(genesisStateContents[])
|
extractGenesisValidatorRootFromSnapshop(genesisStateContents)
|
||||||
|
|
||||||
if networkGenesisValidatorsRoot != databaseGenesisValidatorsRoot:
|
if networkGenesisValidatorsRoot != databaseGenesisValidatorsRoot:
|
||||||
fatal "The specified --data-dir contains data for a different network",
|
fatal "The specified --data-dir contains data for a different network",
|
||||||
networkGenesisValidatorsRoot, databaseGenesisValidatorsRoot,
|
networkGenesisValidatorsRoot, databaseGenesisValidatorsRoot,
|
||||||
dataDir = conf.dataDir
|
dataDir = config.dataDir
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
if conf.weakSubjectivityCheckpoint.isSome:
|
if config.weakSubjectivityCheckpoint.isSome:
|
||||||
let
|
let
|
||||||
currentSlot = beaconClock.now.slotOrZero
|
currentSlot = beaconClock.now.slotOrZero
|
||||||
isCheckpointStale = not is_within_weak_subjectivity_period(
|
isCheckpointStale = not is_within_weak_subjectivity_period(
|
||||||
currentSlot,
|
currentSlot,
|
||||||
chainDag.headState.data.data,
|
chainDag.headState.data.data,
|
||||||
conf.weakSubjectivityCheckpoint.get)
|
config.weakSubjectivityCheckpoint.get)
|
||||||
|
|
||||||
if isCheckpointStale:
|
if isCheckpointStale:
|
||||||
error "Weak subjectivity checkpoint is stale",
|
error "Weak subjectivity checkpoint is stale",
|
||||||
currentSlot,
|
currentSlot,
|
||||||
checkpoint = conf.weakSubjectivityCheckpoint.get,
|
checkpoint = config.weakSubjectivityCheckpoint.get,
|
||||||
headStateSlot = chainDag.headState.data.data.slot
|
headStateSlot = chainDag.headState.data.data.slot
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
|
@ -252,50 +249,81 @@ proc init*(T: type BeaconNode,
|
||||||
if checkpointGenesisValidatorsRoot != databaseGenesisValidatorsRoot:
|
if checkpointGenesisValidatorsRoot != databaseGenesisValidatorsRoot:
|
||||||
fatal "The specified checkpoint state is intended for a different network",
|
fatal "The specified checkpoint state is intended for a different network",
|
||||||
checkpointGenesisValidatorsRoot, databaseGenesisValidatorsRoot,
|
checkpointGenesisValidatorsRoot, databaseGenesisValidatorsRoot,
|
||||||
dataDir = conf.dataDir
|
dataDir = config.dataDir
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
chainDag.setTailState(checkpointState[], checkpointBlock)
|
chainDag.setTailState(checkpointState[], checkpointBlock)
|
||||||
|
|
||||||
if eth1Monitor.isNil and
|
if eth1Monitor.isNil and
|
||||||
conf.web3Url.len > 0 and
|
config.web3Url.len > 0 and
|
||||||
genesisDepositsSnapshotContents != nil:
|
genesisDepositsSnapshotContents.len > 0:
|
||||||
let genesisDepositsSnapshot = SSZ.decode(genesisDepositsSnapshotContents[],
|
let genesisDepositsSnapshot = SSZ.decode(genesisDepositsSnapshotContents,
|
||||||
DepositContractSnapshot)
|
DepositContractSnapshot)
|
||||||
eth1Monitor = Eth1Monitor.init(
|
eth1Monitor = Eth1Monitor.init(
|
||||||
runtimePreset,
|
runtimePreset,
|
||||||
db,
|
db,
|
||||||
conf.web3Url,
|
config.web3Url,
|
||||||
depositContractAddress,
|
depositContractAddress,
|
||||||
genesisDepositsSnapshot,
|
genesisDepositsSnapshot,
|
||||||
eth1Network)
|
eth1Network)
|
||||||
|
|
||||||
let rpcServer = if conf.rpcEnabled:
|
let rpcServer = if config.rpcEnabled:
|
||||||
RpcServer.init(conf.rpcAddress, conf.rpcPort)
|
RpcServer.init(config.rpcAddress, config.rpcPort)
|
||||||
else:
|
else:
|
||||||
nil
|
nil
|
||||||
|
|
||||||
let
|
let
|
||||||
netKeys = getPersistentNetKeys(rng[], conf)
|
netKeys = getPersistentNetKeys(rng[], config)
|
||||||
nickname = if conf.nodeName == "auto": shortForm(netKeys)
|
nickname = if config.nodeName == "auto": shortForm(netKeys)
|
||||||
else: conf.nodeName
|
else: config.nodeName
|
||||||
enrForkId = enrForkIdFromState(chainDag.headState.data.data)
|
enrForkId = enrForkIdFromState(chainDag.headState.data.data)
|
||||||
topicBeaconBlocks = getBeaconBlocksTopic(enrForkId.forkDigest)
|
topicBeaconBlocks = getBeaconBlocksTopic(enrForkId.forkDigest)
|
||||||
topicAggregateAndProofs = getAggregateAndProofsTopic(enrForkId.forkDigest)
|
topicAggregateAndProofs = getAggregateAndProofsTopic(enrForkId.forkDigest)
|
||||||
network = createEth2Node(rng, conf, netKeys, enrForkId)
|
network = createEth2Node(rng, config, netKeys, enrForkId)
|
||||||
attestationPool = newClone(AttestationPool.init(chainDag, quarantine))
|
attestationPool = newClone(AttestationPool.init(chainDag, quarantine))
|
||||||
exitPool = newClone(ExitPool.init(chainDag, quarantine))
|
exitPool = newClone(ExitPool.init(chainDag, quarantine))
|
||||||
|
|
||||||
|
slashingProtectionDB =
|
||||||
|
case config.slashingDbKind
|
||||||
|
of SlashingDbKind.v1:
|
||||||
|
info "Loading slashing protection database", path = config.validatorsDir()
|
||||||
|
SlashingProtectionDB.init(
|
||||||
|
chainDag.headState.data.data.genesis_validators_root,
|
||||||
|
config.validatorsDir(), "slashing_protection",
|
||||||
|
modes = {kCompleteArchiveV1},
|
||||||
|
disagreementBehavior = kChooseV1
|
||||||
|
)
|
||||||
|
of SlashingDbKind.v2:
|
||||||
|
info "Loading slashing protection database (v2)", path = config.validatorsDir()
|
||||||
|
SlashingProtectionDB.init(
|
||||||
|
chainDag.headState.data.data.genesis_validators_root,
|
||||||
|
config.validatorsDir(), "slashing_protection"
|
||||||
|
)
|
||||||
|
of SlashingDbKind.both:
|
||||||
|
info "Loading slashing protection database (dual DB mode)", path = config.validatorsDir()
|
||||||
|
SlashingProtectionDB.init(
|
||||||
|
chainDag.headState.data.data.genesis_validators_root,
|
||||||
|
config.validatorsDir(), "slashing_protection",
|
||||||
|
modes = {kCompleteArchiveV1, kCompleteArchiveV2},
|
||||||
|
disagreementBehavior = kChooseV2
|
||||||
|
)
|
||||||
|
validatorPool = newClone(ValidatorPool.init(slashingProtectionDB))
|
||||||
|
processor = Eth2Processor.new(
|
||||||
|
config, chainDag, attestationPool, exitPool, validatorPool,
|
||||||
|
quarantine, proc(): BeaconTime = beaconClock.now())
|
||||||
|
|
||||||
var res = BeaconNode(
|
var res = BeaconNode(
|
||||||
nickname: nickname,
|
nickname: nickname,
|
||||||
graffitiBytes: if conf.graffiti.isSome: conf.graffiti.get.GraffitiBytes
|
graffitiBytes: if config.graffiti.isSome: config.graffiti.get.GraffitiBytes
|
||||||
else: defaultGraffitiBytes(),
|
else: defaultGraffitiBytes(),
|
||||||
network: network,
|
network: network,
|
||||||
netKeys: netKeys,
|
netKeys: netKeys,
|
||||||
db: db,
|
db: db,
|
||||||
config: conf,
|
config: config,
|
||||||
chainDag: chainDag,
|
chainDag: chainDag,
|
||||||
quarantine: quarantine,
|
quarantine: quarantine,
|
||||||
attestationPool: attestationPool,
|
attestationPool: attestationPool,
|
||||||
|
attachedValidators: validatorPool,
|
||||||
exitPool: exitPool,
|
exitPool: exitPool,
|
||||||
eth1Monitor: eth1Monitor,
|
eth1Monitor: eth1Monitor,
|
||||||
beaconClock: beaconClock,
|
beaconClock: beaconClock,
|
||||||
|
@ -303,6 +331,8 @@ proc init*(T: type BeaconNode,
|
||||||
forkDigest: enrForkId.forkDigest,
|
forkDigest: enrForkId.forkDigest,
|
||||||
topicBeaconBlocks: topicBeaconBlocks,
|
topicBeaconBlocks: topicBeaconBlocks,
|
||||||
topicAggregateAndProofs: topicAggregateAndProofs,
|
topicAggregateAndProofs: topicAggregateAndProofs,
|
||||||
|
processor: processor,
|
||||||
|
requestManager: RequestManager.init(network, processor.blocksQueue)
|
||||||
)
|
)
|
||||||
|
|
||||||
# set topic validation routine
|
# set topic validation routine
|
||||||
|
@ -320,45 +350,6 @@ proc init*(T: type BeaconNode,
|
||||||
topics &= getAttestationTopic(enrForkId.forkDigest, subnet)
|
topics &= getAttestationTopic(enrForkId.forkDigest, subnet)
|
||||||
topics)
|
topics)
|
||||||
|
|
||||||
case conf.slashingDbKind
|
|
||||||
of SlashingDbKind.v1:
|
|
||||||
info "Loading slashing protection database", path = conf.validatorsDir()
|
|
||||||
res.attachedValidators = ValidatorPool.init(
|
|
||||||
SlashingProtectionDB.init(
|
|
||||||
chainDag.headState.data.data.genesis_validators_root,
|
|
||||||
conf.validatorsDir(), "slashing_protection",
|
|
||||||
modes = {kCompleteArchiveV1},
|
|
||||||
disagreementBehavior = kChooseV1
|
|
||||||
)
|
|
||||||
)
|
|
||||||
of SlashingDbKind.v2:
|
|
||||||
info "Loading slashing protection database (v2)", path = conf.validatorsDir()
|
|
||||||
res.attachedValidators = ValidatorPool.init(
|
|
||||||
SlashingProtectionDB.init(
|
|
||||||
chainDag.headState.data.data.genesis_validators_root,
|
|
||||||
conf.validatorsDir(), "slashing_protection"
|
|
||||||
)
|
|
||||||
)
|
|
||||||
of SlashingDbKind.both:
|
|
||||||
info "Loading slashing protection database (dual DB mode)", path = conf.validatorsDir()
|
|
||||||
res.attachedValidators = ValidatorPool.init(
|
|
||||||
SlashingProtectionDB.init(
|
|
||||||
chainDag.headState.data.data.genesis_validators_root,
|
|
||||||
conf.validatorsDir(), "slashing_protection",
|
|
||||||
modes = {kCompleteArchiveV1, kCompleteArchiveV2},
|
|
||||||
disagreementBehavior = kChooseV2
|
|
||||||
)
|
|
||||||
)
|
|
||||||
|
|
||||||
proc getWallTime(): BeaconTime = res.beaconClock.now()
|
|
||||||
|
|
||||||
res.processor = Eth2Processor.new(
|
|
||||||
conf, chainDag, attestationPool, exitPool, newClone(res.attachedValidators),
|
|
||||||
quarantine, getWallTime)
|
|
||||||
|
|
||||||
res.requestManager = RequestManager.init(
|
|
||||||
network, res.processor.blocksQueue)
|
|
||||||
|
|
||||||
if res.config.inProcessValidators:
|
if res.config.inProcessValidators:
|
||||||
res.addLocalValidators()
|
res.addLocalValidators()
|
||||||
else:
|
else:
|
||||||
|
@ -692,8 +683,8 @@ proc getAttestationSubnetHandlers(node: BeaconNode) =
|
||||||
var initialStabilitySubnets: set[uint8]
|
var initialStabilitySubnets: set[uint8]
|
||||||
|
|
||||||
node.attestationSubnets.stabilitySubnets.setLen(
|
node.attestationSubnets.stabilitySubnets.setLen(
|
||||||
node.attachedValidators.count)
|
node.attachedValidators[].count)
|
||||||
for i in 0 ..< node.attachedValidators.count:
|
for i in 0 ..< node.attachedValidators[].count:
|
||||||
node.attestationSubnets.stabilitySubnets[i] = (
|
node.attestationSubnets.stabilitySubnets[i] = (
|
||||||
subnet: node.network.rng[].rand(ATTESTATION_SUBNET_COUNT - 1).uint8,
|
subnet: node.network.rng[].rand(ATTESTATION_SUBNET_COUNT - 1).uint8,
|
||||||
expiration: wallEpoch + node.getStabilitySubnetLength())
|
expiration: wallEpoch + node.getStabilitySubnetLength())
|
||||||
|
@ -1254,7 +1245,7 @@ proc initializeNetworking(node: BeaconNode) {.async.} =
|
||||||
|
|
||||||
func shouldWeStartWeb3(node: BeaconNode): bool =
|
func shouldWeStartWeb3(node: BeaconNode): bool =
|
||||||
(node.config.web3Mode == Web3Mode.enabled) or
|
(node.config.web3Mode == Web3Mode.enabled) or
|
||||||
(node.config.web3Mode == Web3Mode.auto and node.attachedValidators.count > 0)
|
(node.config.web3Mode == Web3Mode.auto and node.attachedValidators[].count > 0)
|
||||||
|
|
||||||
proc start(node: BeaconNode) =
|
proc start(node: BeaconNode) =
|
||||||
let
|
let
|
||||||
|
@ -1274,13 +1265,14 @@ proc start(node: BeaconNode) =
|
||||||
SECONDS_PER_SLOT,
|
SECONDS_PER_SLOT,
|
||||||
SPEC_VERSION,
|
SPEC_VERSION,
|
||||||
dataDir = node.config.dataDir.string,
|
dataDir = node.config.dataDir.string,
|
||||||
validators = node.attachedValidators.count
|
validators = node.attachedValidators[].count
|
||||||
|
|
||||||
if genesisTime.inFuture:
|
if genesisTime.inFuture:
|
||||||
notice "Waiting for genesis", genesisIn = genesisTime.offset
|
notice "Waiting for genesis", genesisIn = genesisTime.offset
|
||||||
|
|
||||||
waitFor node.initializeNetworking()
|
waitFor node.initializeNetworking()
|
||||||
|
|
||||||
|
# TODO this does not account for validators getting attached "later"
|
||||||
if node.eth1Monitor != nil and node.shouldWeStartWeb3:
|
if node.eth1Monitor != nil and node.shouldWeStartWeb3:
|
||||||
node.eth1Monitor.start()
|
node.eth1Monitor.start()
|
||||||
|
|
||||||
|
@ -1304,37 +1296,12 @@ func formatGwei(amount: uint64): string =
|
||||||
while result[^1] == '0':
|
while result[^1] == '0':
|
||||||
result.setLen(result.len - 1)
|
result.setLen(result.len - 1)
|
||||||
|
|
||||||
when hasPrompt:
|
proc initStatusBar(node: BeaconNode) =
|
||||||
from unicode import Rune
|
if not isatty(stdout): return
|
||||||
import prompt
|
if not node.config.statusBarEnabled: return
|
||||||
|
|
||||||
func providePromptCompletions*(line: seq[Rune], cursorPos: int): seq[string] =
|
|
||||||
# TODO
|
|
||||||
# The completions should be generated with the general-purpose command-line
|
|
||||||
# parsing API of Confutils
|
|
||||||
result = @[]
|
|
||||||
|
|
||||||
proc processPromptCommands(p: ptr Prompt) {.thread.} =
|
|
||||||
while true:
|
|
||||||
var cmd = p[].readLine()
|
|
||||||
case cmd
|
|
||||||
of "quit":
|
|
||||||
quit 0
|
|
||||||
else:
|
|
||||||
p[].writeLine("Unknown command: " & cmd)
|
|
||||||
|
|
||||||
proc initPrompt(node: BeaconNode) =
|
|
||||||
if isatty(stdout) and node.config.statusBarEnabled:
|
|
||||||
enableTrueColors()
|
enableTrueColors()
|
||||||
|
|
||||||
# TODO: nim-prompt seems to have threading issues at the moment
|
|
||||||
# which result in sporadic crashes. We should introduce a
|
|
||||||
# lock that guards the access to the internal prompt line
|
|
||||||
# variable.
|
|
||||||
#
|
|
||||||
# var p = Prompt.init("nimbus > ", providePromptCompletions)
|
|
||||||
# p.useHistoryFile()
|
|
||||||
|
|
||||||
proc dataResolver(expr: string): string =
|
proc dataResolver(expr: string): string =
|
||||||
template justified: untyped = node.chainDag.head.atEpochStart(
|
template justified: untyped = node.chainDag.head.atEpochStart(
|
||||||
node.chainDag.headState.data.data.current_justified_checkpoint.epoch)
|
node.chainDag.headState.data.data.current_justified_checkpoint.epoch)
|
||||||
|
@ -1428,16 +1395,48 @@ when hasPrompt:
|
||||||
logLoggingFailure(cstring(msg), e)
|
logLoggingFailure(cstring(msg), e)
|
||||||
|
|
||||||
proc statusBarUpdatesPollingLoop() {.async.} =
|
proc statusBarUpdatesPollingLoop() {.async.} =
|
||||||
|
try:
|
||||||
while true:
|
while true:
|
||||||
update statusBar
|
update statusBar
|
||||||
erase statusBar
|
erase statusBar
|
||||||
render statusBar
|
render statusBar
|
||||||
await sleepAsync(chronos.seconds(1))
|
await sleepAsync(chronos.seconds(1))
|
||||||
|
except CatchableError as exc:
|
||||||
|
warn "Failed to update status bar, no further updates", err = exc.msg
|
||||||
|
|
||||||
traceAsyncErrors statusBarUpdatesPollingLoop()
|
asyncSpawn statusBarUpdatesPollingLoop()
|
||||||
|
|
||||||
|
when hasPrompt:
|
||||||
|
# TODO: nim-prompt seems to have threading issues at the moment
|
||||||
|
# which result in sporadic crashes. We should introduce a
|
||||||
|
# lock that guards the access to the internal prompt line
|
||||||
|
# variable.
|
||||||
|
#
|
||||||
|
# var p = Prompt.init("nimbus > ", providePromptCompletions)
|
||||||
|
# p.useHistoryFile()
|
||||||
|
|
||||||
|
from unicode import Rune
|
||||||
|
import prompt
|
||||||
|
|
||||||
|
func providePromptCompletions*(line: seq[Rune], cursorPos: int): seq[string] =
|
||||||
|
# TODO
|
||||||
|
# The completions should be generated with the general-purpose command-line
|
||||||
|
# parsing API of Confutils
|
||||||
|
result = @[]
|
||||||
|
|
||||||
|
proc processPromptCommands(p: ptr Prompt) {.thread.} =
|
||||||
|
while true:
|
||||||
|
var cmd = p[].readLine()
|
||||||
|
case cmd
|
||||||
|
of "quit":
|
||||||
|
quit 0
|
||||||
|
else:
|
||||||
|
p[].writeLine("Unknown command: " & cmd)
|
||||||
|
|
||||||
|
proc initPrompt(node: BeaconNode) =
|
||||||
# var t: Thread[ptr Prompt]
|
# var t: Thread[ptr Prompt]
|
||||||
# createThread(t, processPromptCommands, addr p)
|
# createThread(t, processPromptCommands, addr p)
|
||||||
|
discard
|
||||||
|
|
||||||
proc handleValidatorExitCommand(config: BeaconNodeConf) {.async.} =
|
proc handleValidatorExitCommand(config: BeaconNodeConf) {.async.} =
|
||||||
let port = try:
|
let port = try:
|
||||||
|
@ -1577,91 +1576,75 @@ proc handleValidatorExitCommand(config: BeaconNodeConf) {.async.} =
|
||||||
err = err.msg
|
err = err.msg
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
programMain:
|
proc loadEth2Network(config: BeaconNodeConf): Eth2NetworkMetadata =
|
||||||
var
|
|
||||||
config = makeBannerAndConfig(clientId, BeaconNodeConf)
|
|
||||||
# This is ref so we can mutate it (to erase it) after the initial loading.
|
|
||||||
genesisStateContents: ref string
|
|
||||||
genesisDepositsSnapshotContents: ref string
|
|
||||||
eth1Network: Option[Eth1Network]
|
|
||||||
depositContractAddress: Option[Eth1Address]
|
|
||||||
depositContractDeployedAt: Option[BlockHashOrNumber]
|
|
||||||
runtimePreset: RuntimePreset
|
|
||||||
|
|
||||||
setupStdoutLogging(config.logLevel)
|
|
||||||
|
|
||||||
if not(checkAndCreateDataDir(string(config.dataDir))):
|
|
||||||
# We are unable to access/create data folder or data folder's
|
|
||||||
# permissions are insecure.
|
|
||||||
quit QuitFailure
|
|
||||||
|
|
||||||
setupLogging(config.logLevel, config.logFile)
|
|
||||||
|
|
||||||
## This Ctrl+C handler exits the program in non-graceful way.
|
|
||||||
## It's responsible for handling Ctrl+C in sub-commands such
|
|
||||||
## as `wallets *` and `deposits *`. In a regular beacon node
|
|
||||||
## run, it will be overwritten later with a different handler
|
|
||||||
## performing a graceful exit.
|
|
||||||
proc exitImmediatelyOnCtrlC() {.noconv.} =
|
|
||||||
when defined(windows):
|
|
||||||
# workaround for https://github.com/nim-lang/Nim/issues/4057
|
|
||||||
setupForeignThreadGc()
|
|
||||||
echo "" # If we interrupt during an interactive prompt, this
|
|
||||||
# will move the cursor to the next line
|
|
||||||
notice "Shutting down after having received SIGINT"
|
|
||||||
quit 0
|
|
||||||
setControlCHook(exitImmediatelyOnCtrlC)
|
|
||||||
# equivalent SIGTERM handler
|
|
||||||
when defined(posix):
|
|
||||||
proc exitImmediatelyOnSIGTERM(signal: cint) {.noconv.} =
|
|
||||||
notice "Shutting down after having received SIGTERM"
|
|
||||||
quit 0
|
|
||||||
c_signal(SIGTERM, exitImmediatelyOnSIGTERM)
|
|
||||||
|
|
||||||
if config.eth2Network.isSome:
|
if config.eth2Network.isSome:
|
||||||
let metadata = getMetadataForNetwork(config.eth2Network.get)
|
getMetadataForNetwork(config.eth2Network.get)
|
||||||
runtimePreset = metadata.runtimePreset
|
else:
|
||||||
|
when const_preset == "mainnet":
|
||||||
|
mainnetMetadata
|
||||||
|
else:
|
||||||
|
# Presumably other configurations can have other defaults, but for now
|
||||||
|
# this simplifies the flow
|
||||||
|
echo "Must specify network on non-mainnet node"
|
||||||
|
quit 1
|
||||||
|
|
||||||
if config.cmd == noCommand:
|
proc loadBeaconNode(config: var BeaconNodeConf, rng: ref BrHmacDrbgContext): BeaconNode =
|
||||||
|
let metadata = config.loadEth2Network()
|
||||||
|
|
||||||
|
# Updating the config based on the metadata certainly is not beautiful but it
|
||||||
|
# works
|
||||||
for node in metadata.bootstrapNodes:
|
for node in metadata.bootstrapNodes:
|
||||||
config.bootstrapNodes.add node
|
config.bootstrapNodes.add node
|
||||||
|
|
||||||
if metadata.genesisData.len > 0:
|
BeaconNode.init(
|
||||||
genesisStateContents = newClone metadata.genesisData
|
metadata.runtimePreset,
|
||||||
|
rng,
|
||||||
|
config,
|
||||||
|
metadata.depositContractAddress,
|
||||||
|
metadata.depositContractDeployedAt,
|
||||||
|
metadata.eth1Network,
|
||||||
|
metadata.genesisData,
|
||||||
|
metadata.genesisDepositsSnapshot)
|
||||||
|
|
||||||
if metadata.genesisDepositsSnapshot.len > 0:
|
proc doRunBeaconNode(config: var BeaconNodeConf, rng: ref BrHmacDrbgContext) =
|
||||||
genesisDepositsSnapshotContents = newClone metadata.genesisDepositsSnapshot
|
info "Launching beacon node",
|
||||||
|
version = fullVersionStr,
|
||||||
|
bls_backend = $BLS_BACKEND,
|
||||||
|
cmdParams = commandLineParams(),
|
||||||
|
config
|
||||||
|
|
||||||
depositContractAddress = some metadata.depositContractAddress
|
createPidFile(config.dataDir.string / "beacon_node.pid")
|
||||||
depositContractDeployedAt = some metadata.depositContractDeployedAt
|
|
||||||
eth1Network = metadata.eth1Network
|
config.createDumpDirs()
|
||||||
|
|
||||||
|
if config.metricsEnabled:
|
||||||
|
when useInsecureFeatures:
|
||||||
|
let metricsAddress = config.metricsAddress
|
||||||
|
notice "Starting metrics HTTP server",
|
||||||
|
url = "http://" & $metricsAddress & ":" & $config.metricsPort & "/metrics"
|
||||||
|
metrics.startHttpServer($metricsAddress, config.metricsPort)
|
||||||
else:
|
else:
|
||||||
runtimePreset = defaultRuntimePreset
|
warn "Metrics support disabled, see https://status-im.github.io/nimbus-eth2/metrics-pretty-pictures.html#simple-metrics"
|
||||||
when const_preset == "mainnet":
|
|
||||||
if config.cmd == noCommand:
|
|
||||||
depositContractAddress = some mainnetMetadata.depositContractAddress
|
|
||||||
depositContractDeployedAt = some mainnetMetadata.depositContractDeployedAt
|
|
||||||
|
|
||||||
for node in mainnetMetadata.bootstrapNodes:
|
# There are no managed event loops in here, to do a graceful shutdown, but
|
||||||
config.bootstrapNodes.add node
|
# letting the default Ctrl+C handler exit is safe, since we only read from
|
||||||
|
# the db.
|
||||||
|
let node = loadBeaconNode(config, rng)
|
||||||
|
|
||||||
genesisStateContents = newClone mainnetMetadata.genesisData
|
if bnStatus == BeaconNodeStatus.Stopping:
|
||||||
genesisDepositsSnapshotContents = newClone mainnetMetadata.genesisDepositsSnapshot
|
return
|
||||||
eth1Network = some mainnet
|
|
||||||
|
|
||||||
# Single RNG instance for the application - will be seeded on construction
|
initStatusBar(node)
|
||||||
# and avoid using system resources (such as urandom) after that
|
|
||||||
let rng = keys.newRng()
|
|
||||||
|
|
||||||
template findWalletWithoutErrors(name: WalletName): auto =
|
when hasPrompt:
|
||||||
let res = keystore_management.findWallet(config, name)
|
initPrompt(node)
|
||||||
if res.isErr:
|
|
||||||
fatal "Failed to locate wallet", error = res.error
|
|
||||||
quit 1
|
|
||||||
res.get
|
|
||||||
|
|
||||||
case config.cmd
|
if node.nickname != "":
|
||||||
of createTestnet:
|
dynamicLogScope(node = node.nickname): node.start()
|
||||||
|
else:
|
||||||
|
node.start()
|
||||||
|
|
||||||
|
proc doCreateTestnet(config: BeaconNodeConf, rng: var BrHmacDrbgContext) =
|
||||||
let launchPadDeposits = try:
|
let launchPadDeposits = try:
|
||||||
Json.loadFile(config.testnetDepositsFile.string, seq[LaunchPadDeposit])
|
Json.loadFile(config.testnetDepositsFile.string, seq[LaunchPadDeposit])
|
||||||
except SerializationError as err:
|
except SerializationError as err:
|
||||||
|
@ -1678,6 +1661,7 @@ programMain:
|
||||||
outGenesis = config.outputGenesis.string
|
outGenesis = config.outputGenesis.string
|
||||||
eth1Hash = if config.web3Url.len == 0: eth1BlockHash
|
eth1Hash = if config.web3Url.len == 0: eth1BlockHash
|
||||||
else: (waitFor getEth1BlockHash(config.web3Url, blockId("latest"))).asEth2Digest
|
else: (waitFor getEth1BlockHash(config.web3Url, blockId("latest"))).asEth2Digest
|
||||||
|
runtimePreset = getRuntimePresetForNetwork(config.eth2Network)
|
||||||
var
|
var
|
||||||
initialState = initialize_beacon_state_from_eth1(
|
initialState = initialize_beacon_state_from_eth1(
|
||||||
runtimePreset, eth1Hash, startTime, deposits, {skipBlsValidation})
|
runtimePreset, eth1Hash, startTime, deposits, {skipBlsValidation})
|
||||||
|
@ -1699,7 +1683,7 @@ programMain:
|
||||||
let bootstrapFile = config.outputBootstrapFile.string
|
let bootstrapFile = config.outputBootstrapFile.string
|
||||||
if bootstrapFile.len > 0:
|
if bootstrapFile.len > 0:
|
||||||
let
|
let
|
||||||
networkKeys = getPersistentNetKeys(rng[], config)
|
networkKeys = getPersistentNetKeys(rng, config)
|
||||||
netMetadata = getPersistentNetMetadata(config)
|
netMetadata = getPersistentNetMetadata(config)
|
||||||
bootstrapEnr = enr.Record.init(
|
bootstrapEnr = enr.Record.init(
|
||||||
1, # sequence number
|
1, # sequence number
|
||||||
|
@ -1713,61 +1697,7 @@ programMain:
|
||||||
writeFile(bootstrapFile, bootstrapEnr.tryGet().toURI)
|
writeFile(bootstrapFile, bootstrapEnr.tryGet().toURI)
|
||||||
echo "Wrote ", bootstrapFile
|
echo "Wrote ", bootstrapFile
|
||||||
|
|
||||||
of noCommand:
|
proc doDeposits(config: BeaconNodeConf, rng: var BrHmacDrbgContext) =
|
||||||
info "Launching beacon node",
|
|
||||||
version = fullVersionStr,
|
|
||||||
bls_backend = $BLS_BACKEND,
|
|
||||||
cmdParams = commandLineParams(),
|
|
||||||
config
|
|
||||||
|
|
||||||
createPidFile(config.dataDir.string / "beacon_node.pid")
|
|
||||||
|
|
||||||
config.createDumpDirs()
|
|
||||||
|
|
||||||
if config.metricsEnabled:
|
|
||||||
when useInsecureFeatures:
|
|
||||||
let metricsAddress = config.metricsAddress
|
|
||||||
notice "Starting metrics HTTP server",
|
|
||||||
url = "http://" & $metricsAddress & ":" & $config.metricsPort & "/metrics"
|
|
||||||
metrics.startHttpServer($metricsAddress, config.metricsPort)
|
|
||||||
else:
|
|
||||||
warn "Metrics support disabled, see https://status-im.github.io/nimbus-eth2/metrics-pretty-pictures.html#simple-metrics"
|
|
||||||
|
|
||||||
if depositContractAddress.isNone or depositContractDeployedAt.isNone:
|
|
||||||
echo "Please specify the a network through the --network option"
|
|
||||||
quit 1
|
|
||||||
|
|
||||||
# There are no managed event loops in here, to do a graceful shutdown, but
|
|
||||||
# letting the default Ctrl+C handler exit is safe, since we only read from
|
|
||||||
# the db.
|
|
||||||
var node = waitFor BeaconNode.init(
|
|
||||||
runtimePreset,
|
|
||||||
rng,
|
|
||||||
config,
|
|
||||||
depositContractAddress.get,
|
|
||||||
depositContractDeployedAt.get,
|
|
||||||
eth1Network,
|
|
||||||
genesisStateContents,
|
|
||||||
genesisDepositsSnapshotContents)
|
|
||||||
|
|
||||||
if bnStatus == BeaconNodeStatus.Stopping:
|
|
||||||
return
|
|
||||||
|
|
||||||
# The memory for the initial snapshot won't be needed anymore
|
|
||||||
if genesisStateContents != nil:
|
|
||||||
genesisStateContents[] = ""
|
|
||||||
if genesisDepositsSnapshotContents != nil:
|
|
||||||
genesisDepositsSnapshotContents[] = ""
|
|
||||||
|
|
||||||
when hasPrompt:
|
|
||||||
initPrompt(node)
|
|
||||||
|
|
||||||
if node.nickname != "":
|
|
||||||
dynamicLogScope(node = node.nickname): node.start()
|
|
||||||
else:
|
|
||||||
node.start()
|
|
||||||
|
|
||||||
of deposits:
|
|
||||||
case config.depositsCmd
|
case config.depositsCmd
|
||||||
#[
|
#[
|
||||||
of DepositsCmd.create:
|
of DepositsCmd.create:
|
||||||
|
@ -1867,14 +1797,21 @@ programMain:
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
importKeystoresFromDir(
|
importKeystoresFromDir(
|
||||||
rng[],
|
rng,
|
||||||
validatorKeysDir.string,
|
validatorKeysDir.string,
|
||||||
config.validatorsDir, config.secretsDir)
|
config.validatorsDir, config.secretsDir)
|
||||||
|
|
||||||
of DepositsCmd.exit:
|
of DepositsCmd.exit:
|
||||||
waitFor handleValidatorExitCommand(config)
|
waitFor handleValidatorExitCommand(config)
|
||||||
|
|
||||||
of wallets:
|
proc doWallets(config: BeaconNodeConf, rng: var BrHmacDrbgContext) =
|
||||||
|
template findWalletWithoutErrors(name: WalletName): auto =
|
||||||
|
let res = keystore_management.findWallet(config, name)
|
||||||
|
if res.isErr:
|
||||||
|
fatal "Failed to locate wallet", error = res.error
|
||||||
|
quit 1
|
||||||
|
res.get
|
||||||
|
|
||||||
case config.walletsCmd:
|
case config.walletsCmd:
|
||||||
of WalletsCmd.create:
|
of WalletsCmd.create:
|
||||||
if config.createdWalletNameFlag.isSome:
|
if config.createdWalletNameFlag.isSome:
|
||||||
|
@ -1885,7 +1822,7 @@ programMain:
|
||||||
echo "The Wallet '" & name.string & "' already exists."
|
echo "The Wallet '" & name.string & "' already exists."
|
||||||
quit 1
|
quit 1
|
||||||
|
|
||||||
var walletRes = createWalletInteractively(rng[], config)
|
var walletRes = createWalletInteractively(rng, config)
|
||||||
if walletRes.isErr:
|
if walletRes.isErr:
|
||||||
fatal "Unable to create wallet", err = walletRes.error
|
fatal "Unable to create wallet", err = walletRes.error
|
||||||
quit 1
|
quit 1
|
||||||
|
@ -1906,12 +1843,12 @@ programMain:
|
||||||
wallet = walletFile
|
wallet = walletFile
|
||||||
|
|
||||||
of WalletsCmd.restore:
|
of WalletsCmd.restore:
|
||||||
restoreWalletInteractively(rng[], config)
|
restoreWalletInteractively(rng, config)
|
||||||
|
|
||||||
of record:
|
proc doRecord(config: BeaconNodeConf, rng: var BrHmacDrbgContext) =
|
||||||
case config.recordCmd:
|
case config.recordCmd:
|
||||||
of RecordCmd.create:
|
of RecordCmd.create:
|
||||||
let netKeys = getPersistentNetKeys(rng[], config)
|
let netKeys = getPersistentNetKeys(rng, config)
|
||||||
|
|
||||||
var fieldPairs: seq[FieldPair]
|
var fieldPairs: seq[FieldPair]
|
||||||
for field in config.fields:
|
for field in config.fields:
|
||||||
|
@ -1935,9 +1872,55 @@ programMain:
|
||||||
of RecordCmd.print:
|
of RecordCmd.print:
|
||||||
echo $config.recordPrint
|
echo $config.recordPrint
|
||||||
|
|
||||||
of web3:
|
proc doWeb3Cmd(config: BeaconNodeConf) =
|
||||||
case config.web3Cmd:
|
case config.web3Cmd:
|
||||||
of Web3Cmd.test:
|
of Web3Cmd.test:
|
||||||
|
let metadata = config.loadEth2Network()
|
||||||
waitFor testWeb3Provider(config.web3TestUrl,
|
waitFor testWeb3Provider(config.web3TestUrl,
|
||||||
depositContractAddress,
|
metadata.depositContractAddress)
|
||||||
depositContractDeployedAt)
|
|
||||||
|
programMain:
|
||||||
|
var
|
||||||
|
config = makeBannerAndConfig(clientId, BeaconNodeConf)
|
||||||
|
|
||||||
|
setupStdoutLogging(config.logLevel)
|
||||||
|
|
||||||
|
if not(checkAndCreateDataDir(string(config.dataDir))):
|
||||||
|
# We are unable to access/create data folder or data folder's
|
||||||
|
# permissions are insecure.
|
||||||
|
quit QuitFailure
|
||||||
|
|
||||||
|
setupLogging(config.logLevel, config.logFile)
|
||||||
|
|
||||||
|
## This Ctrl+C handler exits the program in non-graceful way.
|
||||||
|
## It's responsible for handling Ctrl+C in sub-commands such
|
||||||
|
## as `wallets *` and `deposits *`. In a regular beacon node
|
||||||
|
## run, it will be overwritten later with a different handler
|
||||||
|
## performing a graceful exit.
|
||||||
|
proc exitImmediatelyOnCtrlC() {.noconv.} =
|
||||||
|
when defined(windows):
|
||||||
|
# workaround for https://github.com/nim-lang/Nim/issues/4057
|
||||||
|
setupForeignThreadGc()
|
||||||
|
echo "" # If we interrupt during an interactive prompt, this
|
||||||
|
# will move the cursor to the next line
|
||||||
|
notice "Shutting down after having received SIGINT"
|
||||||
|
quit 0
|
||||||
|
setControlCHook(exitImmediatelyOnCtrlC)
|
||||||
|
# equivalent SIGTERM handler
|
||||||
|
when defined(posix):
|
||||||
|
proc exitImmediatelyOnSIGTERM(signal: cint) {.noconv.} =
|
||||||
|
notice "Shutting down after having received SIGTERM"
|
||||||
|
quit 0
|
||||||
|
c_signal(SIGTERM, exitImmediatelyOnSIGTERM)
|
||||||
|
|
||||||
|
# Single RNG instance for the application - will be seeded on construction
|
||||||
|
# and avoid using system resources (such as urandom) after that
|
||||||
|
let rng = keys.newRng()
|
||||||
|
|
||||||
|
case config.cmd
|
||||||
|
of createTestnet: doCreateTestnet(config, rng[])
|
||||||
|
of noCommand: doRunBeaconNode(config, rng)
|
||||||
|
of deposits: doDeposits(config, rng[])
|
||||||
|
of wallets: doWallets(config, rng[])
|
||||||
|
of record: doRecord(config, rng[])
|
||||||
|
of web3: doWeb3Cmd(config)
|
||||||
|
|
|
@ -64,7 +64,7 @@ proc addLocalValidator*(node: BeaconNode,
|
||||||
state: BeaconState,
|
state: BeaconState,
|
||||||
privKey: ValidatorPrivKey) =
|
privKey: ValidatorPrivKey) =
|
||||||
let pubKey = privKey.toPubKey()
|
let pubKey = privKey.toPubKey()
|
||||||
node.attachedValidators.addLocalValidator(
|
node.attachedValidators[].addLocalValidator(
|
||||||
pubKey, privKey, findValidator(state, pubKey))
|
pubKey, privKey, findValidator(state, pubKey))
|
||||||
|
|
||||||
proc addLocalValidators*(node: BeaconNode) =
|
proc addLocalValidators*(node: BeaconNode) =
|
||||||
|
@ -87,11 +87,11 @@ proc addRemoteValidators*(node: BeaconNode) =
|
||||||
inStream: node.vcProcess.inputStream,
|
inStream: node.vcProcess.inputStream,
|
||||||
outStream: node.vcProcess.outputStream,
|
outStream: node.vcProcess.outputStream,
|
||||||
pubKeyStr: $key))
|
pubKeyStr: $key))
|
||||||
node.attachedValidators.addRemoteValidator(key, v)
|
node.attachedValidators[].addRemoteValidator(key, v)
|
||||||
|
|
||||||
proc getAttachedValidator*(node: BeaconNode,
|
proc getAttachedValidator*(node: BeaconNode,
|
||||||
pubkey: ValidatorPubKey): AttachedValidator =
|
pubkey: ValidatorPubKey): AttachedValidator =
|
||||||
node.attachedValidators.getValidator(pubkey)
|
node.attachedValidators[].getValidator(pubkey)
|
||||||
|
|
||||||
proc getAttachedValidator*(node: BeaconNode,
|
proc getAttachedValidator*(node: BeaconNode,
|
||||||
state: BeaconState,
|
state: BeaconState,
|
||||||
|
@ -464,7 +464,7 @@ proc handleProposal(node: BeaconNode, head: BlockRef, slot: Slot):
|
||||||
return head
|
return head
|
||||||
|
|
||||||
let validator =
|
let validator =
|
||||||
node.attachedValidators.getValidator(proposer.get()[1])
|
node.attachedValidators[].getValidator(proposer.get()[1])
|
||||||
|
|
||||||
if validator != nil:
|
if validator != nil:
|
||||||
return await proposeBlock(node, validator, proposer.get()[0], head, slot)
|
return await proposeBlock(node, validator, proposer.get()[0], head, slot)
|
||||||
|
@ -569,7 +569,7 @@ proc updateValidatorMetrics*(node: BeaconNode) =
|
||||||
|
|
||||||
var total: Gwei
|
var total: Gwei
|
||||||
var i = 0
|
var i = 0
|
||||||
for _, v in node.attachedValidators.validators:
|
for _, v in node.attachedValidators[].validators:
|
||||||
let balance =
|
let balance =
|
||||||
if v.index.isNone():
|
if v.index.isNone():
|
||||||
0.Gwei
|
0.Gwei
|
||||||
|
@ -597,7 +597,7 @@ proc updateValidatorMetrics*(node: BeaconNode) =
|
||||||
|
|
||||||
proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
|
proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
|
||||||
## Perform validator duties - create blocks, vote and aggregate existing votes
|
## Perform validator duties - create blocks, vote and aggregate existing votes
|
||||||
if node.attachedValidators.count == 0:
|
if node.attachedValidators[].count == 0:
|
||||||
# Nothing to do because we have no validator attached
|
# Nothing to do because we have no validator attached
|
||||||
return
|
return
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue