Address #1584 Don't keep all deposits in memory (persist them to disk)

This commit is contained in:
Zahary Karadjov 2020-10-12 04:07:20 +03:00 committed by zah
parent bc8acdb9de
commit e6320e5881
9 changed files with 274 additions and 171 deletions

View File

@ -1,13 +1,24 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
typetraits, stew/[results, objects, endians2], typetraits,
stew/[results, objects, endians2, io2],
serialization, chronicles, snappy, serialization, chronicles, snappy,
eth/db/kvstore, eth/db/[kvstore, kvstore_sqlite3],
./spec/[datatypes, digest, crypto, state_transition], ./spec/[datatypes, digest, crypto, state_transition],
./ssz/[ssz_serialization, merkleization] ./ssz/[ssz_serialization, merkleization]
type type
DbSeq*[T] = object
db: SqStoreRef
name: string
file: File
endPos: uint64
DbMap*[K, V] = object
db: SqStoreRef
keyspace: int
BeaconChainDB* = ref object BeaconChainDB* = ref object
## Database storing resolved blocks and states - resolved blocks are such ## Database storing resolved blocks and states - resolved blocks are such
## blocks that form a chain back to the tail block. ## blocks that form a chain back to the tail block.
@ -23,6 +34,15 @@ type
## database. ## database.
backend: KvStoreRef backend: KvStoreRef
deposits*: DbSeq[DepositData]
validators*: DbSeq[ImmutableValidatorData]
validatorsByKey*: DbMap[ValidatorPubKey, ValidatorIndex]
Keyspaces* = enum
defaultKeyspace = "kvstore"
seqMetadata
validatorIndexFromPubKey
DbKeyKind = enum DbKeyKind = enum
kHashToState kHashToState
kHashToBlock kHashToBlock
@ -39,6 +59,9 @@ type
kGenesisBlockRoot kGenesisBlockRoot
## Immutable reference to the network genesis state ## Immutable reference to the network genesis state
## (needed for satisfying requests to the beacon node API). ## (needed for satisfying requests to the beacon node API).
kEth1PersistedTo
## The latest ETH1 block hash which satisfied the follow distance and
## had its deposits persisted to disk.
const const
maxDecompressedDbRecordSize = 16*1024*1024 maxDecompressedDbRecordSize = 16*1024*1024
@ -71,8 +94,91 @@ func subkey(root: Eth2Digest, slot: Slot): array[40, byte] =
ret ret
proc init*(T: type BeaconChainDB, backend: KVStoreRef): BeaconChainDB = template panic =
T(backend: backend) # TODO: Could we recover from a corrupted database?
# Review all usages.
raiseAssert "The database should not be corrupted"
proc createSeq*(db: SqStoreRef, baseDir, seqFile: string, T: type): DbSeq[T] =
var endPos: uint64 = 0
proc onData(data: openArray[byte]) =
endPos = uint64.fromBytesBE(data)
discard db.get(
ord seqMetadata,
seqFile.toOpenArrayByte(0, seqFile.len - 1),
onData).expect("working database")
let f = try: open(baseDir / seqFile, fmWrite)
except IOError: panic()
let fileSize = try: getFileSize(f).uint64
except IOError: panic()
if endPos > fileSize: panic()
DbSeq[T](db: db, name: seqFile, file: f, endPos: endPos)
proc add*[T](s: var DbSeq[T], val: T) =
var bytes = SSZ.encode(val)
try:
setFilePos(s.file, s.endPos.int64)
write(s.file, bytes)
except IOError:
panic()
s.endPos += bytes.len.uint64
proc len*[T](s: DbSeq[T]): uint64 =
const elemSize = fixedPortionSize(T).uint64
s.endPos div elemSize
proc get*[T](s: DbSeq[T], idx: uint64): T =
const size = uint64 fixedPortionSize(T)
var recordBytes: array[size, byte]
try:
# TODO: check for invalid coercion here
let pos = size * idx
setFilePos(s.file, pos.int64)
let bytesRead = readBytes(s.file, recordBytes, 0, size)
# TODO Can we recover from a corrupted database?
if bytesRead.uint64 != size: panic()
except IOError:
panic()
try:
decode(SSZ, recordBytes, T)
except SerializationError:
panic()
proc flush*(s: DbSeq) =
s.file.flushFile()
s.db.put(
ord seqMetadata,
s.name.toOpenArrayByte(0, s.name.len - 1),
s.endPos.toBytesBE()).expect("working database")
proc createMap*(db: SqStoreRef, keyspace: int;
K, V: distinct type): DbMap[K, V] =
DbMap[K, V](db: db, keyspace: keyspace)
proc insert*[K, V](m: DbMap[K, V], key: K, value: V) =
m.db.put(m.keyspace, SSZ.encode key, SSZ.encode value).expect("working database")
proc contains*[K, V](m: DbMap[K, V], key: K): bool =
contains(m.db, SSZ.encode key).expect("working database")
proc init*(T: type BeaconChainDB, dir: string): BeaconChainDB =
let s = createPath(dir, 0o750)
doAssert s.isOk # TODO Handle this in a better way
let sqliteStore = SqStoreRef.init(dir, "nbc", Keyspaces).expect(
"working database")
T(backend: kvStore sqliteStore,
deposits: createSeq(sqliteStore, dir, "deposits", DepositData),
validators: createSeq(sqliteStore, dir, "validators", ImmutableValidatorData),
validatorsByKey: createMap(sqliteStore, int validatorIndexFromPubKey,
ValidatorPubKey, ValidatorIndex))
proc snappyEncode(inp: openArray[byte]): seq[byte] = proc snappyEncode(inp: openArray[byte]): seq[byte] =
try: try:
@ -173,6 +279,9 @@ proc putTailBlock*(db: BeaconChainDB, key: Eth2Digest) =
proc putGenesisBlockRoot*(db: BeaconChainDB, key: Eth2Digest) = proc putGenesisBlockRoot*(db: BeaconChainDB, key: Eth2Digest) =
db.put(subkey(kGenesisBlockRoot), key) db.put(subkey(kGenesisBlockRoot), key)
proc putEth1PersistedTo*(db: BeaconChainDB, key: Eth2Digest) =
db.put(subkey(kEth1PersistedTo), key)
proc getBlock*(db: BeaconChainDB, key: Eth2Digest): Opt[TrustedSignedBeaconBlock] = proc getBlock*(db: BeaconChainDB, key: Eth2Digest): Opt[TrustedSignedBeaconBlock] =
# We only store blocks that we trust in the database # We only store blocks that we trust in the database
result.ok(TrustedSignedBeaconBlock()) result.ok(TrustedSignedBeaconBlock())
@ -216,7 +325,12 @@ proc getTailBlock*(db: BeaconChainDB): Opt[Eth2Digest] =
db.get(subkey(kTailBlock), Eth2Digest) db.get(subkey(kTailBlock), Eth2Digest)
proc getGenesisBlockRoot*(db: BeaconChainDB): Eth2Digest = proc getGenesisBlockRoot*(db: BeaconChainDB): Eth2Digest =
db.get(subkey(kGenesisBlockRoot), Eth2Digest).expect("The database must be seeded with the genesis state") db.get(subkey(kGenesisBlockRoot), Eth2Digest).expect(
"The database must be seeded with the genesis state")
proc getEth1PersistedTo*(db: BeaconChainDB): Eth2Digest =
db.get(subkey(kGenesisBlockRoot), Eth2Digest).expect(
"The database must be seeded with genesis eth1 data")
proc containsBlock*(db: BeaconChainDB, key: Eth2Digest): bool = proc containsBlock*(db: BeaconChainDB, key: Eth2Digest): bool =
db.backend.contains(subkey(SignedBeaconBlock, key)).expect("working database") db.backend.contains(subkey(SignedBeaconBlock, key)).expect("working database")

View File

@ -77,7 +77,7 @@ proc init*(T: type BeaconNode,
netKeys = getPersistentNetKeys(rng[], conf) netKeys = getPersistentNetKeys(rng[], conf)
nickname = if conf.nodeName == "auto": shortForm(netKeys) nickname = if conf.nodeName == "auto": shortForm(netKeys)
else: conf.nodeName else: conf.nodeName
db = BeaconChainDB.init(kvStore SqStoreRef.init(conf.databaseDir, "nbc").tryGet()) db = BeaconChainDB.init(conf.databaseDir)
var var
mainchainMonitor: MainchainMonitor mainchainMonitor: MainchainMonitor
@ -156,6 +156,7 @@ proc init*(T: type BeaconNode,
# 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.
mainchainMonitor = MainchainMonitor.init( mainchainMonitor = MainchainMonitor.init(
db,
conf.runtimePreset, conf.runtimePreset,
web3, web3,
conf.depositContractAddress.get, conf.depositContractAddress.get,
@ -233,6 +234,7 @@ proc init*(T: type BeaconNode,
conf.web3Url.len > 0 and conf.web3Url.len > 0 and
conf.depositContractAddress.isSome: conf.depositContractAddress.isSome:
mainchainMonitor = MainchainMonitor.init( mainchainMonitor = MainchainMonitor.init(
db,
conf.runtimePreset, conf.runtimePreset,
web3Provider(conf.web3Url), web3Provider(conf.web3Url),
conf.depositContractAddress.get, conf.depositContractAddress.get,

View File

@ -2,9 +2,8 @@ import
std/[deques, tables, hashes, options, strformat], std/[deques, tables, hashes, options, strformat],
chronos, web3, web3/ethtypes as web3Types, json, chronicles, chronos, web3, web3/ethtypes as web3Types, json, chronicles,
eth/common/eth_types, eth/async_utils, eth/common/eth_types, eth/async_utils,
spec/[datatypes, digest, crypto, beaconstate, helpers, validator], spec/[datatypes, digest, crypto, beaconstate, helpers, signatures],
network_metadata, merkle_minimal, ssz, beacon_chain_db, network_metadata, merkle_minimal, beacon_node_status
beacon_node_status
from times import epochTime from times import epochTime
@ -34,12 +33,14 @@ type
Eth1BlockTimestamp* = uint64 Eth1BlockTimestamp* = uint64
Eth1BlockHeader = web3Types.BlockHeader Eth1BlockHeader = web3Types.BlockHeader
Database* = object
Eth1Block* = ref object Eth1Block* = ref object
number*: Eth1BlockNumber number*: Eth1BlockNumber
timestamp*: Eth1BlockTimestamp timestamp*: Eth1BlockTimestamp
deposits*: seq[Deposit] deposits*: seq[Deposit]
voteData*: Eth1Data voteData*: Eth1Data
knownGoodDepositsCount*: Option[uint64] knownValidatorsCount*: Option[uint64]
Eth1Chain* = object Eth1Chain* = object
knownStart: Eth1Data knownStart: Eth1Data
@ -47,9 +48,10 @@ type
blocks: Deque[Eth1Block] blocks: Deque[Eth1Block]
blocksByHash: Table[BlockHash, Eth1Block] blocksByHash: Table[BlockHash, Eth1Block]
allDeposits*: seq[Deposit]
MainchainMonitor* = ref object MainchainMonitor* = ref object
db: BeaconChainDB
preset: RuntimePreset preset: RuntimePreset
depositContractAddress: Address depositContractAddress: Address
dataProviderFactory*: DataProviderFactory dataProviderFactory*: DataProviderFactory
@ -181,11 +183,6 @@ func trimHeight(eth1Chain: var Eth1Chain, blockNumber: Eth1BlockNumber) =
else: else:
break break
if eth1Chain.blocks.len > 0:
eth1Chain.allDeposits.setLen(eth1Chain.blocks[^1].voteData.deposit_count)
else:
eth1Chain.allDeposits.setLen(0)
func isSuccessorBlock(eth1Chain: Eth1Chain, newBlock: Eth1Block): bool = func isSuccessorBlock(eth1Chain: Eth1Chain, newBlock: Eth1Block): bool =
let currentDepositCount = if eth1Chain.blocks.len == 0: let currentDepositCount = if eth1Chain.blocks.len == 0:
eth1Chain.knownStart.deposit_count eth1Chain.knownStart.deposit_count
@ -199,18 +196,12 @@ func isSuccessorBlock(eth1Chain: Eth1Chain, newBlock: Eth1Block): bool =
func addSuccessorBlock*(eth1Chain: var Eth1Chain, newBlock: Eth1Block): bool = func addSuccessorBlock*(eth1Chain: var Eth1Chain, newBlock: Eth1Block): bool =
result = isSuccessorBlock(eth1Chain, newBlock) result = isSuccessorBlock(eth1Chain, newBlock)
if result: if result:
eth1Chain.allDeposits.add newBlock.deposits
reset newBlock.deposits
eth1Chain.blocks.addLast newBlock eth1Chain.blocks.addLast newBlock
eth1Chain.blocksByHash[newBlock.voteData.block_hash.asBlockHash] = newBlock eth1Chain.blocksByHash[newBlock.voteData.block_hash.asBlockHash] = newBlock
func totalDeposits*(eth1Chain: Eth1Chain): int = proc allDepositsUpTo*(m: MainchainMonitor, totalDeposits: uint64): seq[Deposit] =
for blk in eth1Chain.blocks: for i in 0'u64 ..< totalDeposits:
result += blk.deposits.len result.add Deposit(data: m.db.deposits.get(i))
func allDeposits*(eth1Chain: Eth1Chain): seq[Deposit] =
for blk in eth1Chain.blocks:
result.add blk.deposits
func clear*(eth1Chain: var Eth1Chain) = func clear*(eth1Chain: var Eth1Chain) =
eth1Chain = default(Eth1Chain) eth1Chain = default(Eth1Chain)
@ -320,7 +311,7 @@ proc getBlockNumber(p: DataProviderRef, hash: BlockHash): Future[Eth1BlockNumber
return Eth1BlockNumber(blk.number) return Eth1BlockNumber(blk.number)
except CatchableError as exc: except CatchableError as exc:
debug "Failed to get Eth1 block number from hash", debug "Failed to get Eth1 block number from hash",
hash = $hash, err = exc.msg hash = $hash, err = exc.msg
raise exc raise exc
template readJsonField(j: JsonNode, template readJsonField(j: JsonNode,
@ -374,7 +365,7 @@ proc readJsonDeposits(depositsList: JsonNode): seq[Eth1Block] =
method fetchDepositData*(p: Web3DataProviderRef, method fetchDepositData*(p: Web3DataProviderRef,
fromBlock, toBlock: Eth1BlockNumber): Future[seq[Eth1Block]] fromBlock, toBlock: Eth1BlockNumber): Future[seq[Eth1Block]]
{.async, locks: 0.} = {.async, locks: 0.} =
debug "Obtaining deposit log events", fromBlock, toBlock info "Obtaining deposit log events", fromBlock, toBlock
return readJsonDeposits(await p.ns.getJsonLogs(DepositEvent, return readJsonDeposits(await p.ns.getJsonLogs(DepositEvent,
fromBlock = some blockId(fromBlock), fromBlock = some blockId(fromBlock),
toBlock = some blockId(toBlock))) toBlock = some blockId(toBlock)))
@ -413,18 +404,19 @@ method onBlockHeaders*(p: Web3DataProviderRef,
if p.blockHeadersSubscription != nil: if p.blockHeadersSubscription != nil:
await p.blockHeadersSubscription.unsubscribe() await p.blockHeadersSubscription.unsubscribe()
debug "Waiting for new Eth1 block headers" info "Waiting for new Eth1 block headers"
p.blockHeadersSubscription = await p.web3.subscribeForBlockHeaders( p.blockHeadersSubscription = await p.web3.subscribeForBlockHeaders(
blockHeaderHandler, errorHandler) blockHeaderHandler, errorHandler)
# https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/validator.md#get_eth1_data # https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/validator.md#get_eth1_data
func getBlockProposalData*(preset: RuntimePreset, eth1Chain: Eth1Chain, proc getBlockProposalData*(m: MainchainMonitor,
state: BeaconState): (Eth1Data, seq[Deposit]) = state: BeaconState): (Eth1Data, seq[Deposit]) =
template voteForNoChange() = template voteForNoChange() =
return (state.eth1_data, newSeq[Deposit]()) result[0] = state.eth1_data
return
let prevBlock = eth1Chain.findBlock(state.eth1_data) let prevBlock = m.eth1Chain.findBlock(state.eth1_data)
if prevBlock == nil: if prevBlock == nil:
# The Eth1 block currently referenced in the BeaconState is unknown to us. # The Eth1 block currently referenced in the BeaconState is unknown to us.
# This situation is not specifically covered in the honest validator spec, # This situation is not specifically covered in the honest validator spec,
@ -437,83 +429,108 @@ func getBlockProposalData*(preset: RuntimePreset, eth1Chain: Eth1Chain,
var otherVotesCountTable = initCountTable[Eth1Block]() var otherVotesCountTable = initCountTable[Eth1Block]()
for vote in state.eth1_data_votes: for vote in state.eth1_data_votes:
let eth1Block = eth1Chain.findBlock(vote) let eth1Block = m.eth1Chain.findBlock(vote)
if eth1Block != nil and is_candidate_block(preset, eth1Block, periodStart): if eth1Block != nil and is_candidate_block(m.preset, eth1Block, periodStart):
otherVotesCountTable.inc eth1Block otherVotesCountTable.inc eth1Block
var ourVote: Eth1Block
if otherVotesCountTable.len > 0: if otherVotesCountTable.len > 0:
ourVote = otherVotesCountTable.largest.key let (winningBlock, votes) = otherVotesCountTable.largest
result[0] = winningBlock.voteData
if uint64((votes + 1) * 2) > SLOTS_PER_ETH1_VOTING_PERIOD:
result[1] = m.eth1Chain.getDepositsInRange(prevBlock.number,
winningBlock.number)
# TODO This can be significantly more optimal
var newAllDeposits = m.allDepositsUpTo(winningBlock.voteData.deposit_count)
newAllDeposits.add result[1]
attachMerkleProofs(newAllDeposits)
for i in 0 ..< result[1].len:
result[1][i].proof = newAllDeposits[newAllDeposits.len - result[1].len + i].proof
else: else:
ourVote = eth1Chain.latestCandidateBlock(preset, periodStart) let latestBlock = m.eth1Chain.latestCandidateBlock(m.preset, periodStart)
if ourVote == nil: if latestBlock == nil:
voteForNoChange() voteForNoChange()
else:
(ourVote.voteData, eth1Chain.getDepositsInRange(prevBlock.number, ourVote.number)) result[0] = latestBlock.voteData
return
template getBlockProposalData*(m: MainchainMonitor, state: BeaconState): untyped =
getBlockProposalData(m.preset, m.eth1Chain, state)
proc init*(T: type MainchainMonitor, proc init*(T: type MainchainMonitor,
db: BeaconChainDB,
preset: RuntimePreset, preset: RuntimePreset,
dataProviderFactory: DataProviderFactory, dataProviderFactory: DataProviderFactory,
depositContractAddress: Eth1Address, depositContractAddress: Eth1Address,
startPosition: Eth1Data): T = startPosition: Eth1Data): T =
T(preset: preset, T(db: db,
preset: preset,
depositQueue: newAsyncQueue[Eth1BlockHeader](), depositQueue: newAsyncQueue[Eth1BlockHeader](),
dataProviderFactory: dataProviderFactory, dataProviderFactory: dataProviderFactory,
depositContractAddress: Address depositContractAddress, depositContractAddress: Address depositContractAddress,
eth1Chain: Eth1Chain(knownStart: startPosition)) eth1Chain: Eth1Chain(knownStart: startPosition))
proc isCandidateForGenesis(preset: RuntimePreset, proc persistFinalizedBlocks(m: MainchainMonitor, timeNow: float): tuple[
timeNow: float, genesisBlock: Eth1Block,
blk: Eth1Block): bool = previousBlock: Eth1Block
let followDistanceInSeconds = uint64(SECONDS_PER_ETH1_BLOCK) * preset.ETH1_FOLLOW_DISTANCE ] =
if float(blk.timestamp + followDistanceInSeconds) > timeNow:
return false
if genesis_time_from_eth1_timestamp(preset, blk.timestamp) < preset.MIN_GENESIS_TIME:
return false
if blk.knownGoodDepositsCount.isSome:
blk.knownGoodDepositsCount.get >= preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT
else:
blk.voteData.deposit_count >= preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT
proc minGenesisCandidateBlockIdx(m: MainchainMonitor): Option[int]
{.raises: [Defect].} =
if m.eth1Chain.blocks.len == 0: if m.eth1Chain.blocks.len == 0:
return return
let now = epochTime() let followDistanceInSeconds = uint64(SECONDS_PER_ETH1_BLOCK) *
if not isCandidateForGenesis(m.preset, now, m.eth1Chain.blocks.peekLast): m.preset.ETH1_FOLLOW_DISTANCE
return var prevBlock: Eth1Block
var candidatePos = m.eth1Chain.blocks.len - 1 # TODO: The DB operations should be executed as a transaction here
while candidatePos > 1: block: # TODO Begin Transaction
if not isCandidateForGenesis(m.preset, now, m.eth1Chain.blocks[candidatePos - 1]): while true:
break let blk = m.eth1Chain.blocks.peekFirst
dec candidatePos if float(blk.timestamp + followDistanceInSeconds) > timeNow:
break
return some(candidatePos) for deposit in blk.deposits:
m.db.deposits.add deposit.data
proc createBeaconStateAux(preset: RuntimePreset, if verify_deposit_signature(m.preset, deposit.data):
eth1Block: Eth1Block, let pubkey = deposit.data.pubkey
deposits: var openarray[Deposit]): BeaconStateRef = if pubkey notin m.db.validatorsByKey:
let idx = m.db.validators.len
m.db.validators.add ImmutableValidatorData(
pubkey: pubkey,
withdrawal_credentials: deposit.data.withdrawal_credentials)
m.db.validatorsByKey.insert(pubkey, ValidatorIndex idx)
blk.knownValidatorsCount = some m.db.validators.len
discard m.eth1Chain.blocks.popFirst()
m.eth1Chain.blocksByHash.del blk.voteData.block_hash.asBlockHash
let blockGenesisTime = genesis_time_from_eth1_timestamp(m.preset,
blk.timestamp)
if blockGenesisTime >= m.preset.MIN_GENESIS_TIME and
m.db.validators.len >= m.preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT:
result = (blk, prevBlock)
prevBlock = blk
if prevBlock != nil:
# TODO commit transaction
m.db.validators.flush()
m.db.deposits.flush()
m.db.putEth1PersistedTo prevBlock.voteData.block_hash
# TODO Commit
proc createGenesisState(m: MainchainMonitor, eth1Block: Eth1Block): BeaconStateRef =
notice "Generating genesis state",
blockNum = eth1Block.number,
blockHash = eth1Block.voteData.block_hash,
totalDeposits = eth1Block.voteData.deposit_count,
activeValidators = eth1Block.knownValidatorsCount.get
var deposits = m.allDepositsUpTo(eth1Block.voteData.deposit_count)
attachMerkleProofs deposits attachMerkleProofs deposits
result = initialize_beacon_state_from_eth1(preset,
eth1Block.voteData.block_hash,
eth1Block.timestamp.uint64,
deposits, {})
var cache = StateCache()
let activeValidators = count_active_validators(result[], GENESIS_EPOCH, cache)
eth1Block.knownGoodDepositsCount = some activeValidators
proc createBeaconState(m: MainchainMonitor, eth1Block: Eth1Block): BeaconStateRef = initialize_beacon_state_from_eth1(m.preset,
createBeaconStateAux( eth1Block.voteData.block_hash,
m.preset, eth1Block.timestamp.uint64,
eth1Block, deposits, {})
m.eth1Chain.allDeposits.toOpenArray(0, int(eth1Block.voteData.deposit_count - 1)))
proc signalGenesis(m: MainchainMonitor, genesisState: BeaconStateRef) = proc signalGenesis(m: MainchainMonitor, genesisState: BeaconStateRef) =
m.genesisState = genesisState m.genesisState = genesisState
@ -590,58 +607,37 @@ proc checkForGenesisLoop(m: MainchainMonitor) {.async.} =
return return
try: try:
let genesisCandidateIdx = m.minGenesisCandidateBlockIdx # TODO: check for a stale monitor
if genesisCandidateIdx.isSome: let
let now = epochTime()
genesisCandidateIdx = genesisCandidateIdx.get (genesisCandidate, genesisParent) = m.persistFinalizedBlocks(now)
genesisCandidate = m.eth1Chain.blocks[genesisCandidateIdx]
notice "Generating state for candidate block for genesis", if genesisCandidate != nil:
blockNum = genesisCandidate.number, # We have a candidate state on our hands, but our current Eth1Chain
blockHash = genesisCandidate.voteData.block_hash, # may consist only of blocks that have deposits attached to them
potentialDeposits = genesisCandidate.voteData.deposit_count # while the real genesis may have happened in a block without any
# deposits (triggered by MIN_GENESIS_TIME).
#
# This can happen when the beacon node is launched after the genesis
# event. We take a short cut when constructing the initial Eth1Chain
# by downloading only deposit log entries. Thus, we'll see all the
# blocks with deposits, but not the regular blocks in between.
#
# We'll handle this special case below by examing whether we are in
# this potential scenario and we'll use a fast guessing algorith to
# discover the ETh1 block with minimal valid genesis time.
if genesisParent != nil:
if genesisParent.knownValidatorsCount.get >= m.preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT and
genesisParent.number - genesisParent.number > 1:
let genesisBlock = await m.findGenesisBlockInRange(genesisParent, genesisCandidate)
if genesisBlock.number != genesisCandidate.number:
m.signalGenesis m.createGenesisState(genesisBlock)
return
let let candidateState = m.createGenesisState(genesisCandidate)
candidateState = m.createBeaconState(genesisCandidate) m.signalGenesis candidateState
return
if genesisCandidate.knownGoodDepositsCount.get >= m.preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT:
# We have a candidate state on our hands, but our current Eth1Chain
# may consist only of blocks that have deposits attached to them
# while the real genesis may have happened in a block without any
# deposits (triggered by MIN_GENESIS_TIME).
#
# This can happen when the beacon node is launched after the genesis
# event. We take a short cut when constructing the initial Eth1Chain
# by downloading only deposit log entries. Thus, we'll see all the
# blocks with deposits, but not the regular blocks in between.
#
# We'll handle this special case below by examing whether we are in
# this potential scenario and we'll use a fast guessing algorith to
# discover the ETh1 block with minimal valid genesis time.
if genesisCandidateIdx > 0:
let preceedingEth1Block = m.eth1Chain.blocks[genesisCandidateIdx - 1]
if preceedingEth1Block.voteData.deposit_root == genesisCandidate.voteData.deposit_root:
preceedingEth1Block.knownGoodDepositsCount = genesisCandidate.knownGoodDepositsCount
else:
discard m.createBeaconState(preceedingEth1Block)
if preceedingEth1Block.knownGoodDepositsCount.get >= m.preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT and
genesisCandidate.number - preceedingEth1Block.number > 1:
let genesisBlock = await m.findGenesisBlockInRange(preceedingEth1Block, genesisCandidate)
if genesisBlock.number != genesisCandidate.number:
m.signalGenesis m.createBeaconState(genesisBlock)
return
m.signalGenesis candidateState
return
else:
notice "Eth2 genesis candidate block rejected",
`block` = shortLog(genesisCandidate),
validDeposits = genesisCandidate.knownGoodDepositsCount.get,
needed = m.preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT
else:
# TODO: check for a stale monitor
discard
except CatchableError as err: except CatchableError as err:
debug "Unexpected error in checkForGenesisLoop", err = err.msg debug "Unexpected error in checkForGenesisLoop", err = err.msg
@ -678,15 +674,15 @@ func latestEth1Data(eth1Chain: Eth1Chain): Eth1Data =
func knownInvalidDepositsCount(eth1Chain: Eth1Chain): uint64 = func knownInvalidDepositsCount(eth1Chain: Eth1Chain): uint64 =
for i in countdown(eth1Chain.blocks.len - 1, 0): for i in countdown(eth1Chain.blocks.len - 1, 0):
let blk = eth1Chain.blocks[i] let blk = eth1Chain.blocks[i]
if blk.knownGoodDepositsCount.isSome: if blk.knownValidatorsCount.isSome:
return blk.voteData.deposit_count - blk.knownGoodDepositsCount.get return blk.voteData.deposit_count - blk.knownValidatorsCount.get
return 0 return 0
func maxValidDeposits(eth1Chain: Eth1Chain): uint64 = func maxValidDeposits(eth1Chain: Eth1Chain): uint64 =
if eth1Chain.blocks.len > 0: if eth1Chain.blocks.len > 0:
let lastBlock = eth1Chain.blocks[^1] let lastBlock = eth1Chain.blocks[^1]
lastBlock.knownGoodDepositsCount.get( lastBlock.knownValidatorsCount.get(
lastBlock.voteData.deposit_count - eth1Chain.knownInvalidDepositsCount) lastBlock.voteData.deposit_count - eth1Chain.knownInvalidDepositsCount)
else: else:
0 0
@ -697,7 +693,7 @@ proc processDeposits(m: MainchainMonitor,
# Please note that this code is using a queue to guarantee the # Please note that this code is using a queue to guarantee the
# strict serial order of processing of deposits. If we had the # strict serial order of processing of deposits. If we had the
# same code embedded in the deposit contracts events handler, # same code embedded in the deposit contracts events handler,
# it could easily re-order the steps due to the intruptable # it could easily re-order the steps due to the interruptible
# interleaved execution of async code. # interleaved execution of async code.
while true: while true:
m.checkIfShouldStopMainchainMonitor() m.checkIfShouldStopMainchainMonitor()
@ -734,7 +730,7 @@ proc processDeposits(m: MainchainMonitor,
voteData: latestEth1Data) voteData: latestEth1Data)
else: else:
template logBlockProcessed(blk) = template logBlockProcessed(blk) =
debug "Eth1 block processed", info "Eth1 block processed",
`block` = shortLog(blk), totalDeposits = blk.voteData.deposit_count `block` = shortLog(blk), totalDeposits = blk.voteData.deposit_count
await dataProvider.fetchBlockDetails(eth1Blocks[0]) await dataProvider.fetchBlockDetails(eth1Blocks[0])

View File

@ -342,6 +342,11 @@ type
BeaconStateRef* = ref BeaconState not nil BeaconStateRef* = ref BeaconState not nil
NilableBeaconStateRef* = ref BeaconState NilableBeaconStateRef* = ref BeaconState
# Please note that this type is not part of the spec
ImmutableValidatorData* = object
pubkey*: ValidatorPubKey
withdrawal_credentials*: Eth2Digest
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0-rc.0/specs/phase0/beacon-chain.md#validator # https://github.com/ethereum/eth2.0-specs/blob/v1.0.0-rc.0/specs/phase0/beacon-chain.md#validator
Validator* = object Validator* = object
pubkey*: ValidatorPubKey pubkey*: ValidatorPubKey

View File

@ -406,7 +406,6 @@ func get_attestation_deltas(
let let
total_balance = get_total_active_balance(state, cache) total_balance = get_total_active_balance(state, cache)
get_source_deltas(state, total_balance, rewards, penalties, cache) get_source_deltas(state, total_balance, rewards, penalties, cache)
get_target_deltas(state, total_balance, rewards, penalties, cache) get_target_deltas(state, total_balance, rewards, penalties, cache)
get_head_deltas(state, total_balance, rewards, penalties, cache) get_head_deltas(state, total_balance, rewards, penalties, cache)

View File

@ -23,7 +23,7 @@ import
conf, time, validator_pool, conf, time, validator_pool,
attestation_pool, exit_pool, block_pools/[spec_cache, chain_dag, clearance], attestation_pool, exit_pool, block_pools/[spec_cache, chain_dag, clearance],
eth2_network, keystore_management, beacon_node_common, beacon_node_types, eth2_network, keystore_management, beacon_node_common, beacon_node_types,
nimbus_binary_common, mainchain_monitor, version, ssz/merkleization, interop, nimbus_binary_common, mainchain_monitor, version, ssz/merkleization,
attestation_aggregation, sync_manager, sszdump, attestation_aggregation, sync_manager, sszdump,
validator_slashing_protection validator_slashing_protection
@ -199,8 +199,7 @@ proc makeBeaconBlockForHeadAndSlot*(node: BeaconNode,
node.chainDag.tmpState, head.atSlot(slot)): node.chainDag.tmpState, head.atSlot(slot)):
let (eth1data, deposits) = let (eth1data, deposits) =
if node.mainchainMonitor.isNil: if node.mainchainMonitor.isNil:
(get_eth1data_stub(state.eth1_deposit_index, slot.compute_epoch_at_slot()), (state.eth1_data, newSeq[Deposit]())
newSeq[Deposit]())
else: else:
node.mainchainMonitor.getBlockProposalData(state) node.mainchainMonitor.getBlockProposalData(state)

View File

@ -1,13 +1,12 @@
import import
confutils, stats, chronicles, strformat, tables, os, stats, strformat, tables,
stew/byteutils, chronicles, confutils, stew/byteutils, eth/db/kvstore_sqlite3,
../beacon_chain/network_metadata, ../beacon_chain/network_metadata,
../beacon_chain/[beacon_chain_db, extras], ../beacon_chain/[beacon_chain_db, extras],
../beacon_chain/block_pools/[chain_dag], ../beacon_chain/block_pools/[chain_dag],
../beacon_chain/spec/[crypto, datatypes, digest, helpers, ../beacon_chain/spec/[crypto, datatypes, digest, helpers,
state_transition, presets], state_transition, presets],
../beacon_chain/sszdump, ../research/simutils, ../beacon_chain/sszdump, ../research/simutils
eth/db/[kvstore, kvstore_sqlite3]
type Timers = enum type Timers = enum
tInit = "Initialize DB" tInit = "Initialize DB"
@ -85,10 +84,8 @@ proc cmdBench(conf: DbConf, runtimePreset: RuntimePreset) =
echo "Opening database..." echo "Opening database..."
let let
db = BeaconChainDB.init( db = BeaconChainDB.init(conf.databaseDir.string)
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc").tryGet()) dbBenchmark = BeaconChainDB.init("benchmark")
dbBenchmark = BeaconChainDB.init(
kvStore SqStoreRef.init(".", "benchmark").tryGet())
defer: db.close() defer: db.close()
if not ChainDAGRef.isInitialized(db): if not ChainDAGRef.isInitialized(db):
@ -140,9 +137,7 @@ proc cmdBench(conf: DbConf, runtimePreset: RuntimePreset) =
printTimers(false, timers) printTimers(false, timers)
proc cmdDumpState(conf: DbConf) = proc cmdDumpState(conf: DbConf) =
let let db = BeaconChainDB.init(conf.databaseDir.string)
db = BeaconChainDB.init(
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc").tryGet())
defer: db.close() defer: db.close()
for stateRoot in conf.stateRoot: for stateRoot in conf.stateRoot:
@ -157,9 +152,7 @@ proc cmdDumpState(conf: DbConf) =
echo "Couldn't load ", stateRoot, ": ", e.msg echo "Couldn't load ", stateRoot, ": ", e.msg
proc cmdDumpBlock(conf: DbConf) = proc cmdDumpBlock(conf: DbConf) =
let let db = BeaconChainDB.init(conf.databaseDir.string)
db = BeaconChainDB.init(
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc").tryGet())
defer: db.close() defer: db.close()
for blockRoot in conf.blockRootx: for blockRoot in conf.blockRootx:
@ -245,10 +238,9 @@ proc copyPrunedDatabase(
proc cmdPrune(conf: DbConf) = proc cmdPrune(conf: DbConf) =
let let
db = BeaconChainDB.init( db = BeaconChainDB.init(conf.databaseDir.string)
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc").tryGet()) # TODO: add the destination as CLI paramter
copyDb = BeaconChainDB.init( copyDb = BeaconChainDB.init("pruned_db")
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc_pruned").tryGet())
defer: defer:
db.close() db.close()
@ -258,9 +250,7 @@ proc cmdPrune(conf: DbConf) =
proc cmdRewindState(conf: DbConf, runtimePreset: RuntimePreset) = proc cmdRewindState(conf: DbConf, runtimePreset: RuntimePreset) =
echo "Opening database..." echo "Opening database..."
let let db = BeaconChainDB.init(conf.databaseDir.string)
db = BeaconChainDB.init(
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc").tryGet())
defer: db.close() defer: db.close()
if not ChainDAGRef.isInitialized(db): if not ChainDAGRef.isInitialized(db):

View File

@ -15,9 +15,9 @@
# a database, as if a real node was running. # a database, as if a real node was running.
import import
confutils, chronicles, stats, times, confutils, chronicles, stats, times, strformat,
strformat, options, random, tables, os,
options, random, tables, eth/db/kvstore_sqlite3,
../tests/[testblockutil], ../tests/[testblockutil],
../beacon_chain/spec/[beaconstate, crypto, datatypes, digest, presets, ../beacon_chain/spec/[beaconstate, crypto, datatypes, digest, presets,
helpers, validator, signatures, state_transition], helpers, validator, signatures, state_transition],
@ -26,7 +26,6 @@ import
interop, validator_pool], interop, validator_pool],
../beacon_chain/block_pools/[ ../beacon_chain/block_pools/[
spec_cache, chain_dag, quarantine, clearance], spec_cache, chain_dag, quarantine, clearance],
eth/db/[kvstore, kvstore_sqlite3],
../beacon_chain/ssz/[merkleization, ssz_serialization], ../beacon_chain/ssz/[merkleization, ssz_serialization],
./simutils ./simutils
@ -50,8 +49,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
echo "Starting simulation..." echo "Starting simulation..."
let let db = BeaconChainDB.init("block_sim_db")
db = BeaconChainDB.init(kvStore SqStoreRef.init(".", "block_sim").tryGet())
defer: db.close() defer: db.close()
ChainDAGRef.preInit(db, state[].data, state[].data, genesisBlock) ChainDAGRef.preInit(db, state[].data, state[].data, genesisBlock)

2
vendor/nim-eth vendored

@ -1 +1 @@
Subproject commit de2d43a7e7afb7b094ca251bdbbd58fbf47df031 Subproject commit c1037213910ac5ec2156740698d7ecc9dcbfbaeb