Eth1 validators

This commit is contained in:
Yuriy Glukhov 2019-09-04 17:06:11 +03:00 committed by zah
parent f882c92a42
commit bb8955bdbc
7 changed files with 129 additions and 89 deletions

View File

@ -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, genesis, validator_keygen, interop sync_protocol, request_manager, 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 getGenesisFromEth1(conf) tailState = await node.mainchainMonitor.getGenesis()
else: else:
var snapshotFile = conf.dataDir / genesisFile var snapshotFile = conf.dataDir / genesisFile
if conf.stateSnapshot.isSome: if conf.stateSnapshot.isSome:
@ -180,7 +180,8 @@ 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
init result.mainchainMonitor, "", Port(0) # TODO: specify geth address and port result.mainchainMonitor = MainchainMonitor.init(conf.depositWeb3Url, conf.depositContractAddress)
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)
@ -347,6 +348,8 @@ 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
@ -359,7 +362,8 @@ 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(

View File

@ -206,14 +206,6 @@ 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,

View File

@ -1,59 +0,0 @@
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

View File

@ -1,25 +1,120 @@
import import
chronos, json_rpc/rpcclient, chronos, web3, json,
spec/[datatypes, digest] spec/[datatypes, digest, crypto, beaconstate, helpers],
./extras
import web3/ethtypes
type type
MainchainMonitor* = object MainchainMonitor* = ref object
gethAddress: string web3Url: string
gethPort: Port web3: Web3
depositContractAddress: Address
proc init*(T: type MainchainMonitor, gethAddress: string, gethPort: Port): T = genesisState: ref BeaconState
result.gethAddress = gethAddress genesisStateFut: Future[void]
result.gethPort = gethPort
proc start*(m: var MainchainMonitor) = pendingDeposits: seq[Deposit]
# TODO depositCount: uint64
# 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
proc getBeaconBlockRef*(m: MainchainMonitor): Eth1Data = curBlock: uint64
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
discard m.pendingDeposits
# TODO update after spec change removed Specials # TODO update after spec change removed Specials
# iterator getValidatorActions*(m: MainchainMonitor, # iterator getValidatorActions*(m: MainchainMonitor,

View File

@ -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, ../beacon_node_types, ../extras, ../ssz,
./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

View File

@ -18,7 +18,7 @@
# types / composition # types / composition
import import
macros, hashes, math, json, strutils, macros, hashes, math, json, strutils, tables,
stew/[byteutils, bitseqs], chronicles, eth/common, stew/[byteutils, bitseqs], chronicles, eth/common,
../version, ../ssz/types, ./crypto, ./digest ../version, ../ssz/types, ./crypto, ./digest
@ -394,6 +394,14 @@ 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

View File

@ -8,7 +8,7 @@
import import
options, nimcrypto, sequtils, math, tables, chronicles, options, nimcrypto, sequtils, math, tables, chronicles,
../ssz, ../beacon_node_types, ../ssz,
./crypto, ./datatypes, ./digest, ./helpers ./crypto, ./datatypes, ./digest, ./helpers
# TODO: Proceed to renaming and signature changes # TODO: Proceed to renaming and signature changes