parent
bb8955bdbc
commit
e3bd4410d8
|
@ -9,7 +9,7 @@ import
|
||||||
conf, time, state_transition, fork_choice, ssz, beacon_chain_db,
|
conf, time, state_transition, fork_choice, ssz, beacon_chain_db,
|
||||||
validator_pool, extras, attestation_pool, block_pool, eth2_network,
|
validator_pool, extras, attestation_pool, block_pool, eth2_network,
|
||||||
beacon_node_types, mainchain_monitor, trusted_state_snapshots, version,
|
beacon_node_types, mainchain_monitor, trusted_state_snapshots, version,
|
||||||
sync_protocol, request_manager, validator_keygen, interop
|
sync_protocol, request_manager, genesis, validator_keygen, interop
|
||||||
|
|
||||||
const
|
const
|
||||||
topicBeaconBlocks = "/eth2/beacon_block/ssz"
|
topicBeaconBlocks = "/eth2/beacon_block/ssz"
|
||||||
|
@ -89,7 +89,7 @@ proc initGenesis(node: BeaconNode) {.async.} =
|
||||||
var tailState: BeaconState
|
var tailState: BeaconState
|
||||||
if conf.depositWeb3Url.len != 0:
|
if conf.depositWeb3Url.len != 0:
|
||||||
info "Waiting for genesis state from eth1"
|
info "Waiting for genesis state from eth1"
|
||||||
tailState = await node.mainchainMonitor.getGenesis()
|
tailState = await getGenesisFromEth1(conf)
|
||||||
else:
|
else:
|
||||||
var snapshotFile = conf.dataDir / genesisFile
|
var snapshotFile = conf.dataDir / genesisFile
|
||||||
if conf.stateSnapshot.isSome:
|
if conf.stateSnapshot.isSome:
|
||||||
|
@ -180,8 +180,7 @@ proc init*(T: type BeaconNode, conf: BeaconNodeConf): Future[BeaconNode] {.async
|
||||||
result.bootstrapNodes.add BootstrapAddr.init(string ln)
|
result.bootstrapNodes.add BootstrapAddr.init(string ln)
|
||||||
|
|
||||||
result.attachedValidators = ValidatorPool.init
|
result.attachedValidators = ValidatorPool.init
|
||||||
result.mainchainMonitor = MainchainMonitor.init(conf.depositWeb3Url, conf.depositContractAddress)
|
init result.mainchainMonitor, "", Port(0) # TODO: specify geth address and port
|
||||||
result.mainchainMonitor.start()
|
|
||||||
|
|
||||||
let trieDB = trieDB newChainDb(string conf.databaseDir)
|
let trieDB = trieDB newChainDb(string conf.databaseDir)
|
||||||
result.db = BeaconChainDB.init(trieDB)
|
result.db = BeaconChainDB.init(trieDB)
|
||||||
|
@ -348,8 +347,6 @@ proc proposeBlock(node: BeaconNode,
|
||||||
doAssert false, "head slot matches proposal slot (!)"
|
doAssert false, "head slot matches proposal slot (!)"
|
||||||
# return
|
# return
|
||||||
|
|
||||||
# let eth1Data = await node.mainchainMonitor.getBeaconBlockRef()
|
|
||||||
|
|
||||||
var (nroot, nblck) = node.blockPool.withState(
|
var (nroot, nblck) = node.blockPool.withState(
|
||||||
node.stateCache, BlockSlot(blck: head, slot: slot - 1)):
|
node.stateCache, BlockSlot(blck: head, slot: slot - 1)):
|
||||||
# To create a block, we'll first apply a partial block to the state, skipping
|
# To create a block, we'll first apply a partial block to the state, skipping
|
||||||
|
@ -362,8 +359,7 @@ proc proposeBlock(node: BeaconNode,
|
||||||
eth1_data: get_eth1data_stub(
|
eth1_data: get_eth1data_stub(
|
||||||
state.eth1_deposit_index, slot.compute_epoch_of_slot()),
|
state.eth1_deposit_index, slot.compute_epoch_of_slot()),
|
||||||
attestations:
|
attestations:
|
||||||
node.attestationPool.getAttestationsForBlock(state, slot),
|
node.attestationPool.getAttestationsForBlock(state, slot))
|
||||||
deposits: node.mainchainMonitor.getPendingDeposits())
|
|
||||||
|
|
||||||
var
|
var
|
||||||
newBlock = BeaconBlock(
|
newBlock = BeaconBlock(
|
||||||
|
|
|
@ -206,6 +206,14 @@ type
|
||||||
## The block associated with the state found in data - in particular,
|
## The block associated with the state found in data - in particular,
|
||||||
## blck.state_root == rdata.root
|
## blck.state_root == rdata.root
|
||||||
|
|
||||||
|
StateCache* = object
|
||||||
|
crosslink_committee_cache*:
|
||||||
|
Table[tuple[a: int, b: Eth2Digest], seq[ValidatorIndex]]
|
||||||
|
active_validator_indices_cache*:
|
||||||
|
Table[Epoch, seq[ValidatorIndex]]
|
||||||
|
start_shard_cache*: Table[Epoch, Shard]
|
||||||
|
committee_count_cache*: Table[Epoch, uint64]
|
||||||
|
|
||||||
BlockSlot* = object
|
BlockSlot* = object
|
||||||
## Unique identifier for a particular fork in the block chain - normally,
|
## Unique identifier for a particular fork in the block chain - normally,
|
||||||
## there's a block for every slot, but in the case a block is not produced,
|
## there's a block for every slot, but in the case a block is not produced,
|
||||||
|
|
|
@ -0,0 +1,59 @@
|
||||||
|
import
|
||||||
|
chronos, web3, json,
|
||||||
|
spec/[datatypes, digest, crypto, beaconstate, helpers, validator],
|
||||||
|
conf, extras
|
||||||
|
|
||||||
|
contract(DepositContract):
|
||||||
|
proc deposit(pubkey: Bytes48, withdrawalCredentials: Bytes32, signature: Bytes96)
|
||||||
|
proc DepositEvent(pubkey: Bytes48, withdrawalCredentials: Bytes32, amount: Bytes8, signature: Bytes96, index: Bytes8) {.event.}
|
||||||
|
|
||||||
|
const MIN_GENESIS_TIME = 0
|
||||||
|
|
||||||
|
type
|
||||||
|
QueueElement = (BlockHash, DepositData)
|
||||||
|
|
||||||
|
DepositCollector = ref object
|
||||||
|
deposits: seq[datatypes.Deposit]
|
||||||
|
queue: AsyncQueue[QueueElement]
|
||||||
|
|
||||||
|
proc processDeposit(d: DepositCollector, web3: Web3): Future[BeaconState] {.async.} =
|
||||||
|
while true:
|
||||||
|
let (blkHash, data) = await d.queue.popFirst()
|
||||||
|
|
||||||
|
let blk = await web3.provider.eth_getBlockByHash(blkHash, false)
|
||||||
|
let dep = datatypes.Deposit(data: data)
|
||||||
|
d.deposits.add(dep)
|
||||||
|
|
||||||
|
if d.deposits.len >= SLOTS_PER_EPOCH and d.deposits.len >= MIN_GENESIS_ACTIVE_VALIDATOR_COUNT and blk.timestamp.uint64 >= MIN_GENESIS_TIME.uint64:
|
||||||
|
# This block is a genesis candidate
|
||||||
|
var h: Eth2Digest
|
||||||
|
h.data = array[32, byte](blkHash)
|
||||||
|
let s = initialize_beacon_state_from_eth1(h, blk.timestamp.uint64, d.deposits, {skipValidation})
|
||||||
|
|
||||||
|
if is_valid_genesis_state(s):
|
||||||
|
return s
|
||||||
|
|
||||||
|
proc getGenesisFromEth1*(conf: BeaconNodeConf): Future[BeaconState] {.async.} =
|
||||||
|
let web3 = await newWeb3(conf.depositWeb3Url)
|
||||||
|
|
||||||
|
let contractAddress = Address.fromHex(conf.depositContractAddress)
|
||||||
|
let ns = web3.contractSender(DepositContract, contractAddress)
|
||||||
|
|
||||||
|
var deposits = DepositCollector()
|
||||||
|
deposits.queue = newAsyncQueue[QueueElement]()
|
||||||
|
|
||||||
|
let s = await ns.subscribe(DepositEvent, %*{"fromBlock": "0x0"}) do(pubkey: Bytes48, withdrawalCredentials: Bytes32, amount: Bytes8, signature: Bytes96, merkleTreeIndex: Bytes8, j: JsonNode):
|
||||||
|
|
||||||
|
let blkHash = BlockHash.fromHex(j["blockHash"].getStr())
|
||||||
|
let amount = bytes_to_int(array[8, byte](amount))
|
||||||
|
|
||||||
|
deposits.queue.addLastNoWait((blkHash,
|
||||||
|
DepositData(pubkey: ValidatorPubKey.init(array[48, byte](pubkey)),
|
||||||
|
withdrawal_credentials: Eth2Digest(data: array[32, byte](withdrawalCredentials)),
|
||||||
|
amount: amount,
|
||||||
|
signature: ValidatorSig.init(array[96, byte](signature)))))
|
||||||
|
|
||||||
|
let genesisState = await processDeposit(deposits, web3)
|
||||||
|
await s.unsubscribe()
|
||||||
|
return genesisState
|
||||||
|
|
|
@ -1,120 +1,25 @@
|
||||||
import
|
import
|
||||||
chronos, web3, json,
|
chronos, json_rpc/rpcclient,
|
||||||
spec/[datatypes, digest, crypto, beaconstate, helpers],
|
spec/[datatypes, digest]
|
||||||
./extras
|
|
||||||
|
|
||||||
import web3/ethtypes
|
|
||||||
|
|
||||||
type
|
type
|
||||||
MainchainMonitor* = ref object
|
MainchainMonitor* = object
|
||||||
web3Url: string
|
gethAddress: string
|
||||||
web3: Web3
|
gethPort: Port
|
||||||
depositContractAddress: Address
|
|
||||||
|
|
||||||
genesisState: ref BeaconState
|
proc init*(T: type MainchainMonitor, gethAddress: string, gethPort: Port): T =
|
||||||
genesisStateFut: Future[void]
|
result.gethAddress = gethAddress
|
||||||
|
result.gethPort = gethPort
|
||||||
|
|
||||||
pendingDeposits: seq[Deposit]
|
proc start*(m: var MainchainMonitor) =
|
||||||
depositCount: uint64
|
# TODO
|
||||||
|
# Start an async loop following the new blocks using the ETH1 JSON-RPC
|
||||||
|
# interface and keep an always-up-to-date receipt reference here
|
||||||
|
discard
|
||||||
|
|
||||||
curBlock: uint64
|
proc getBeaconBlockRef*(m: MainchainMonitor): Eth1Data =
|
||||||
depositQueue: AsyncQueue[QueueElement]
|
|
||||||
|
|
||||||
eth1Block: BlockHash
|
|
||||||
|
|
||||||
QueueElement = (BlockHash, DepositData)
|
|
||||||
|
|
||||||
|
|
||||||
proc init*(T: type MainchainMonitor, web3Url, depositContractAddress: string): T =
|
|
||||||
result.new()
|
|
||||||
result.web3Url = web3Url
|
|
||||||
result.depositContractAddress = Address.fromHex(depositContractAddress)
|
|
||||||
result.depositQueue = newAsyncQueue[QueueElement]()
|
|
||||||
|
|
||||||
contract(DepositContract):
|
|
||||||
proc deposit(pubkey: Bytes48, withdrawalCredentials: Bytes32, signature: Bytes96)
|
|
||||||
proc get_hash_tree_root(): BlockHash
|
|
||||||
proc get_deposit_count(): Bytes8
|
|
||||||
proc DepositEvent(pubkey: Bytes48, withdrawalCredentials: Bytes32, amount: Bytes8, signature: Bytes96, index: Bytes8) {.event.}
|
|
||||||
|
|
||||||
const MIN_GENESIS_TIME = 0
|
|
||||||
|
|
||||||
proc processDeposits(m: MainchainMonitor) {.async.} =
|
|
||||||
while true:
|
|
||||||
let (blkHash, data) = await m.depositQueue.popFirst()
|
|
||||||
|
|
||||||
let blk = await m.web3.provider.eth_getBlockByHash(blkHash, false)
|
|
||||||
let dep = datatypes.Deposit(data: data)
|
|
||||||
m.pendingDeposits.add(dep)
|
|
||||||
inc m.depositCount
|
|
||||||
m.eth1Block = blkHash
|
|
||||||
|
|
||||||
if m.pendingDeposits.len >= SLOTS_PER_EPOCH and m.pendingDeposits.len >= MIN_GENESIS_ACTIVE_VALIDATOR_COUNT and blk.timestamp.uint64 >= MIN_GENESIS_TIME.uint64:
|
|
||||||
# This block is a genesis candidate
|
|
||||||
var h: Eth2Digest
|
|
||||||
h.data = array[32, byte](blkHash)
|
|
||||||
let s = initialize_beacon_state_from_eth1(h, blk.timestamp.uint64, m.pendingDeposits, {skipValidation})
|
|
||||||
|
|
||||||
if is_valid_genesis_state(s):
|
|
||||||
m.pendingDeposits.setLen(0)
|
|
||||||
m.genesisState.new()
|
|
||||||
m.genesisState[] = s
|
|
||||||
if not m.genesisStateFut.isNil:
|
|
||||||
m.genesisStateFut.complete()
|
|
||||||
m.genesisStateFut = nil
|
|
||||||
# TODO: Set curBlock to blk number
|
|
||||||
|
|
||||||
proc isRunning*(m: MainchainMonitor): bool =
|
|
||||||
not m.web3.isNil
|
|
||||||
|
|
||||||
proc getGenesis*(m: MainchainMonitor): Future[BeaconState] {.async.} =
|
|
||||||
if m.genesisState.isNil:
|
|
||||||
if m.genesisStateFut.isNil:
|
|
||||||
m.genesisStateFut = newFuture[void]("getGenesis")
|
|
||||||
await m.genesisStateFut
|
|
||||||
m.genesisStateFut = nil
|
|
||||||
|
|
||||||
doAssert(not m.genesisState.isNil)
|
|
||||||
return m.genesisState[]
|
|
||||||
|
|
||||||
proc run(m: MainchainMonitor) {.async.} =
|
|
||||||
m.web3 = await newWeb3(m.web3Url)
|
|
||||||
let ns = m.web3.contractSender(DepositContract, m.depositContractAddress)
|
|
||||||
|
|
||||||
let s = await ns.subscribe(DepositEvent, %*{"fromBlock": "0x0"}) do(pubkey: Bytes48, withdrawalCredentials: Bytes32, amount: Bytes8, signature: Bytes96, merkleTreeIndex: Bytes8, j: JsonNode):
|
|
||||||
|
|
||||||
let blkHash = BlockHash.fromHex(j["blockHash"].getStr())
|
|
||||||
let amount = bytes_to_int(array[8, byte](amount))
|
|
||||||
|
|
||||||
m.depositQueue.addLastNoWait((blkHash,
|
|
||||||
DepositData(pubkey: ValidatorPubKey.init(array[48, byte](pubkey)),
|
|
||||||
withdrawal_credentials: Eth2Digest(data: array[32, byte](withdrawalCredentials)),
|
|
||||||
amount: amount,
|
|
||||||
signature: ValidatorSig.init(array[96, byte](signature)))))
|
|
||||||
|
|
||||||
try:
|
|
||||||
await m.processDeposits()
|
|
||||||
finally:
|
|
||||||
await s.unsubscribe()
|
|
||||||
# await m.web3.close()
|
|
||||||
m.web3 = nil
|
|
||||||
|
|
||||||
proc start*(m: MainchainMonitor) =
|
|
||||||
asyncCheck m.run()
|
|
||||||
|
|
||||||
proc getBeaconBlockRef*(m: MainchainMonitor): Future[Eth1Data] {.async.} =
|
|
||||||
let ns = m.web3.contractSender(DepositContract, m.depositContractAddress)
|
|
||||||
|
|
||||||
# TODO: use m.eth1Block for web3 calls
|
|
||||||
let cnt = await ns.get_deposit_count().call()
|
|
||||||
let htr = await ns.get_hash_tree_root().call()
|
|
||||||
result.deposit_count = bytes_to_int(array[8, byte](cnt))
|
|
||||||
result.deposit_root.data = array[32, byte](htr)
|
|
||||||
result.block_hash.data = array[32, byte](m.eth1Block)
|
|
||||||
|
|
||||||
proc getPendingDeposits*(m: MainchainMonitor): seq[Deposit] =
|
|
||||||
# This should be a simple accessor for the reference kept above
|
# This should be a simple accessor for the reference kept above
|
||||||
m.pendingDeposits
|
discard
|
||||||
|
|
||||||
# TODO update after spec change removed Specials
|
# TODO update after spec change removed Specials
|
||||||
# iterator getValidatorActions*(m: MainchainMonitor,
|
# iterator getValidatorActions*(m: MainchainMonitor,
|
||||||
|
|
|
@ -8,7 +8,7 @@
|
||||||
import
|
import
|
||||||
tables, algorithm, math, options, sequtils,
|
tables, algorithm, math, options, sequtils,
|
||||||
json_serialization/std/sets, chronicles, stew/bitseqs,
|
json_serialization/std/sets, chronicles, stew/bitseqs,
|
||||||
../extras, ../ssz,
|
../extras, ../ssz, ../beacon_node_types,
|
||||||
./crypto, ./datatypes, ./digest, ./helpers, ./validator
|
./crypto, ./datatypes, ./digest, ./helpers, ./validator
|
||||||
|
|
||||||
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#is_valid_merkle_branch
|
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#is_valid_merkle_branch
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
# types / composition
|
# types / composition
|
||||||
|
|
||||||
import
|
import
|
||||||
macros, hashes, math, json, strutils, tables,
|
macros, hashes, math, json, strutils,
|
||||||
stew/[byteutils, bitseqs], chronicles, eth/common,
|
stew/[byteutils, bitseqs], chronicles, eth/common,
|
||||||
../version, ../ssz/types, ./crypto, ./digest
|
../version, ../ssz/types, ./crypto, ./digest
|
||||||
|
|
||||||
|
@ -394,14 +394,6 @@ type
|
||||||
data*: BeaconState
|
data*: BeaconState
|
||||||
root*: Eth2Digest # hash_tree_root (not signing_root!)
|
root*: Eth2Digest # hash_tree_root (not signing_root!)
|
||||||
|
|
||||||
StateCache* = object
|
|
||||||
crosslink_committee_cache*:
|
|
||||||
Table[tuple[a: int, b: Eth2Digest], seq[ValidatorIndex]]
|
|
||||||
active_validator_indices_cache*:
|
|
||||||
Table[Epoch, seq[ValidatorIndex]]
|
|
||||||
start_shard_cache*: Table[Epoch, Shard]
|
|
||||||
committee_count_cache*: Table[Epoch, uint64]
|
|
||||||
|
|
||||||
when networkBackend == rlpxBackend:
|
when networkBackend == rlpxBackend:
|
||||||
import eth/rlp/bitseqs as rlpBitseqs
|
import eth/rlp/bitseqs as rlpBitseqs
|
||||||
export read, append
|
export read, append
|
||||||
|
|
|
@ -8,7 +8,7 @@
|
||||||
|
|
||||||
import
|
import
|
||||||
options, nimcrypto, sequtils, math, tables, chronicles,
|
options, nimcrypto, sequtils, math, tables, chronicles,
|
||||||
../ssz,
|
../ssz, ../beacon_node_types,
|
||||||
./crypto, ./datatypes, ./digest, ./helpers
|
./crypto, ./datatypes, ./digest, ./helpers
|
||||||
|
|
||||||
# TODO: Proceed to renaming and signature changes
|
# TODO: Proceed to renaming and signature changes
|
||||||
|
|
Loading…
Reference in New Issue