Don't require requests that might fail on non-archive Geth nodes

This commit is contained in:
Zahary Karadjov 2020-10-26 10:55:10 +02:00 committed by zah
parent 79f1b6117e
commit 18639c3eff
9 changed files with 183 additions and 132 deletions

View File

@ -28,12 +28,9 @@ proc init*(T: type AttestationPool, chainDag: ChainDAGRef, quarantine: Quarantin
## The `finalized_root` works around the finalized_checkpoint of the genesis block
## holding a zero_root.
let
finalizedEpochRef = chainDag.getEpochRef(
chainDag.finalizedHead.blck, chainDag.finalizedHead.slot.epoch())
var forkChoice = ForkChoice.init(
finalizedEpochRef, chainDag.finalizedHead.blck)
chainDag.getFinalizedEpochRef(),
chainDag.finalizedHead.blck)
# Feed fork choice with unfinalized history - during startup, block pool only
# keeps track of a single history so we just need to follow it

View File

@ -7,7 +7,7 @@ import
eth/db/[kvstore, kvstore_sqlite3],
./spec/[datatypes, digest, crypto, state_transition, signatures],
./ssz/[ssz_serialization, merkleization],
filepath
merkle_minimal, filepath
type
DbSeq*[T] = object
@ -23,6 +23,8 @@ type
ImmutableValidatorDataSeq = seq[ImmutableValidatorData]
ValidatorKeyToIndexMap = Table[ValidatorPubKey, ValidatorIndex]
DepositsMerkleizer* = SszMerkleizer[depositContractLimit]
BeaconChainDB* = ref object
## Database storing resolved blocks and states - resolved blocks are such
## blocks that form a chain back to the tail block.
@ -43,6 +45,16 @@ type
immutableValidatorData*: ImmutableValidatorDataSeq
validatorKeyToIndex*: ValidatorKeyToIndexMap
finalizedEth1DepositsMerkleizer*: DepositsMerkleizer
## A merkleizer keeping track of the `deposit_root` value obtained from
## Eth1 after finalizing blocks with ETH1_FOLLOW_DISTANCE confirmations.
## The value is used when voting for Eth1 heads.
finalizedEth2DepositsMerkleizer*: DepositsMerkleizer
## A separate merkleizer which is advanced when the Eth2 chain finalizes.
## It will lag behind the "eth1 merkleizer". We use to produce merkle
## proofs for deposits when they are added to Eth2 blocks.
Keyspaces* = enum
defaultKeyspace = "kvstore"
validatorIndexFromPubKey
@ -66,6 +78,12 @@ type
kEth1PersistedTo
## The latest ETH1 block hash which satisfied the follow distance and
## had its deposits persisted to disk.
kFinalizedEth1DepositsMarkleizer
## A merkleizer used to compute the `deposit_root` of all finalized
## deposits (i.e. deposits confirmed by ETH1_FOLLOW_DISTANCE blocks)
kFinalizedEth2DepositsMarkleizer
## A merkleizer used for computing merkle proofs of deposits added
## to Eth2 blocks (it may lag behind the finalized deposits merkleizer).
const
maxDecompressedDbRecordSize = 16*1024*1024
@ -171,9 +189,12 @@ template insert*[K, V](t: var Table[K, V], key: K, value: V) =
proc produceDerivedData(deposit: DepositData,
preset: RuntimePreset,
deposits: var DepositsSeq,
validators: var ImmutableValidatorDataSeq,
validatorKeyToIndex: var ValidatorKeyToIndexMap) =
validatorKeyToIndex: var ValidatorKeyToIndexMap,
finalizedEth1DepositsMerkleizer: var DepositsMerkleizer) =
let htr = hash_tree_root(deposit)
finalizedEth1DepositsMerkleizer.addChunk htr.data
if verify_deposit_signature(preset, deposit):
let pubkey = deposit.pubkey
if pubkey notin validatorKeyToIndex:
@ -189,9 +210,9 @@ proc processDeposit*(db: BeaconChainDB, newDeposit: DepositData) =
produceDerivedData(
newDeposit,
db.preset,
db.deposits,
db.immutableValidatorData,
db.validatorKeyToIndex)
db.validatorKeyToIndex,
db.finalizedEth1DepositsMerkleizer)
proc init*(T: type BeaconChainDB,
preset: RuntimePreset,
@ -201,7 +222,10 @@ proc init*(T: type BeaconChainDB,
# TODO
# The inMemory store shuold offer the complete functionality
# of the database-backed one (i.e. tracking of deposits and validators)
T(backend: kvStore MemStoreRef.init(), preset: preset)
T(backend: kvStore MemStoreRef.init(),
preset: preset,
finalizedEth1DepositsMerkleizer: init DepositsMerkleizer,
finalizedEth2DepositsMerkleizer: init DepositsMerkleizer)
else:
let s = secureCreatePath(dir)
doAssert s.isOk # TODO Handle this in a better way
@ -213,20 +237,30 @@ proc init*(T: type BeaconChainDB,
immutableValidatorData = newSeq[ImmutableValidatorData]()
validatorKeyToIndex = initTable[ValidatorPubKey, ValidatorIndex]()
depositsSeq = DbSeq[DepositData].init(sqliteStore, "deposits")
finalizedEth1DepositsMerkleizer = init DepositsMerkleizer
finalizedEth2DepositsMerkleizer = init DepositsMerkleizer
for i in 0 ..< depositsSeq.len:
produceDerivedData(
depositsSeq.get(i),
preset,
depositsSeq,
immutableValidatorData,
validatorKeyToIndex)
validatorKeyToIndex,
finalizedEth1DepositsMerkleizer)
T(backend: kvStore sqliteStore,
preset: preset,
deposits: depositsSeq,
immutableValidatorData: immutableValidatorData,
validatorKeyToIndex: validatorKeyToIndex)
validatorKeyToIndex: validatorKeyToIndex,
finalizedEth1DepositsMerkleizer: finalizedEth1DepositsMerkleizer,
finalizedEth2DepositsMerkleizer: finalizedEth2DepositsMerkleizer)
proc advanceTo*(merkleizer: var DepositsMerkleizer,
db: BeaconChainDB,
deposit_index: uint64) =
for i in merkleizer.totalChunks ..< depositIndex:
merkleizer.addChunk hash_tree_root(db.deposits.get(i)).data
proc snappyEncode(inp: openArray[byte]): seq[byte] =
try:

View File

@ -139,6 +139,7 @@ type
epoch*: Epoch
current_justified_checkpoint*: Checkpoint
finalized_checkpoint*: Checkpoint
eth1_data*: Eth1Data
beacon_proposers*: array[
SLOTS_PER_EPOCH, Option[(ValidatorIndex, ValidatorPubKey)]]
shuffled_active_validator_indices*: seq[ValidatorIndex]

View File

@ -94,6 +94,7 @@ proc init*(
epoch = state.get_current_epoch()
epochRef = EpochRef(
epoch: epoch,
eth1_data: state.eth1_data,
current_justified_checkpoint: state.current_justified_checkpoint,
finalized_checkpoint: state.finalized_checkpoint,
shuffled_active_validator_indices:
@ -459,6 +460,9 @@ proc getEpochRef*(dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef =
newEpochRef.updateKeyStores(blck.parent, dag.finalizedHead.blck)
newEpochRef
proc getFinalizedEpochRef*(dag: ChainDAGRef): EpochRef =
dag.getEpochRef(dag.finalizedHead.blck, dag.finalizedHead.slot.epoch)
proc getState(
dag: ChainDAGRef, state: var StateData, stateRoot: Eth2Digest,
blck: BlockRef): bool =

View File

@ -154,21 +154,9 @@ func trimHeight(eth1Chain: var Eth1Chain, blockNumber: Eth1BlockNumber) =
else:
break
func isSuccessorBlock(eth1Chain: Eth1Chain, newBlock: Eth1Block): bool =
let currentDepositCount = if eth1Chain.blocks.len == 0:
eth1Chain.knownStart.deposit_count
else:
let lastBlock = eth1Chain.blocks.peekLast
if lastBlock.number >= newBlock.number: return false
lastBlock.voteData.deposit_count
(currentDepositCount + newBlock.deposits.lenu64) == newBlock.voteData.deposit_count
func addSuccessorBlock*(eth1Chain: var Eth1Chain, newBlock: Eth1Block): bool =
result = isSuccessorBlock(eth1Chain, newBlock)
if result:
eth1Chain.blocks.addLast newBlock
eth1Chain.blocksByHash[newBlock.voteData.block_hash.asBlockHash] = newBlock
func addBlock*(eth1Chain: var Eth1Chain, newBlock: Eth1Block) =
eth1Chain.blocks.addLast newBlock
eth1Chain.blocksByHash[newBlock.voteData.block_hash.asBlockHash] = newBlock
proc allDepositsUpTo*(m: MainchainMonitor, totalDeposits: uint64): seq[Deposit] =
for i in 0'u64 ..< totalDeposits:
@ -252,7 +240,7 @@ proc readJsonDeposits(depositsList: JsonNode): seq[Eth1Block] =
proc fetchDepositData*(p: Web3DataProviderRef,
fromBlock, toBlock: Eth1BlockNumber): Future[seq[Eth1Block]]
{.async, locks: 0.} =
{.async, locks: 0.} =
var currentBlock = fromBlock
while currentBlock <= toBlock:
var blocksPerRequest = 128'u64
@ -279,37 +267,10 @@ proc fetchDepositData*(p: Web3DataProviderRef,
break # breaks the inner "retry" loop and continues
# to the next range of blocks to request
proc fetchBlockDetails(p: Web3DataProviderRef, blk: Eth1Block) {.async.} =
let
web3Block = p.getBlockByNumber(blk.number)
depositRoot = p.ns.get_deposit_root.call(blockNumber = blk.number)
rawCount = p.ns.get_deposit_count.call(blockNumber = blk.number)
discard await web3Block
discard await depositRoot
discard await rawCount
let depositCount = bytes_to_uint64(array[8, byte](rawCount.read))
blk.timestamp = Eth1BlockTimestamp(web3Block.read.timestamp)
blk.voteData.deposit_count = depositCount
blk.voteData.deposit_root = depositRoot.read.asEth2Digest
proc onDisconnect*(p: Web3DataProviderRef, handler: DisconnectHandler) {.
gcsafe
locks: 0
# raises: []
.} =
p.web3.onDisconnect = handler
proc onBlockHeaders*(p: Web3DataProviderRef,
blockHeaderHandler: BlockHeaderHandler,
errorHandler: SubscriptionErrorHandler): Future[void] {.
async
gcsafe
locks: 0
# raises: []
.} =
errorHandler: SubscriptionErrorHandler): Future[void]
{.async, gcsafe.} =
if p.blockHeadersSubscription != nil:
await p.blockHeadersSubscription.unsubscribe()
@ -320,7 +281,17 @@ proc onBlockHeaders*(p: Web3DataProviderRef,
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0-rc.0/specs/phase0/validator.md#get_eth1_data
proc getBlockProposalData*(m: MainchainMonitor,
state: BeaconState): (Eth1Data, seq[Deposit]) =
state: BeaconState,
finalizedEth1Data: Eth1Data): (Eth1Data, seq[Deposit]) =
# TODO To make block proposal cheaper, we can perform this action more regularly
# (e.g. in BeaconNode.onSlot). But keep in mind that this action needs to be
# performed only when there are validators attached to the node.
m.db.finalizedEth2DepositsMerkleizer.advanceTo(
m.db, finalizedEth1Data.deposit_count)
doAssert(m.db.finalizedEth2DepositsMerkleizer.getFinalHash ==
finalizedEth1Data.deposit_root)
let periodStart = voting_period_start_time(state)
var otherVotesCountTable = initCountTable[Eth1Block]()
@ -346,17 +317,27 @@ proc getBlockProposalData*(m: MainchainMonitor,
result[0] = latestBlock.voteData
if pendingDeposits > 0:
# TODO This can be significantly more optimal
let targetDepositCount = result[0].deposit_count
let totalDepositsInNewBlock = min(MAX_DEPOSITS, pendingDeposits)
var newAllDeposits = m.allDepositsUpTo(targetDepositCount)
attachMerkleProofs(newAllDeposits.toOpenArray(0, targetDepositCount.int - 1))
var
deposits = newSeq[Deposit](pendingDeposits)
depositRoots = newSeq[Eth2Digest](pendingDeposits)
depositsMerkleizerClone = clone m.db.finalizedEth2DepositsMerkleizer
let
totalDepositsInNewBlock = min(MAX_DEPOSITS, pendingDeposits)
newDepositIndex = state.eth1_deposit_index + totalDepositsInNewBlock
depositsMerkleizerClone.advanceTo(m.db, state.eth1_deposit_index)
result[1] = newAllDeposits[state.eth1_deposit_index ..< newDepositIndex]
for i in 0 ..< totalDepositsInNewBlock:
deposits[i].data = m.db.deposits.get(state.eth1_deposit_index + i)
depositRoots[i] = hash_tree_root(deposits[i].data)
let proofs = depositsMerkleizerClone.addChunksAndGenMerkleProofs(depositRoots)
for i in 0 ..< totalDepositsInNewBlock:
deposits[i].proof[0..31] = proofs.getProof(i.int)
deposits[i].proof[32].data[0..7] =
toBytesLE uint64(state.eth1_deposit_index + i + 1)
swap(result[1], deposits)
proc init*(T: type MainchainMonitor,
db: BeaconChainDB,
@ -397,10 +378,30 @@ proc init*(T: type MainchainMonitor,
depositQueue: newAsyncQueue[Eth1BlockHeader](),
eth1Chain: Eth1Chain(knownStart: knownStart))
proc persistFinalizedBlocks(m: MainchainMonitor, timeNow: float): tuple[
genesisBlock: Eth1Block,
previousBlock: Eth1Block
] =
proc fetchBlockDetails(p: Web3DataProviderRef, blk: Eth1Block) {.async.} =
let
web3Block = p.getBlockByNumber(blk.number)
depositRoot = p.ns.get_deposit_root.call(blockNumber = blk.number)
rawCount = p.ns.get_deposit_count.call(blockNumber = blk.number)
var error: ref CatchableError = nil
try: blk.voteData.deposit_root = asEth2Digest(await depositRoot)
except CatchableError as err: error = err
try: blk.voteData.deposit_count = bytes_to_uint64(array[8, byte](await rawCount))
except CatchableError as err: error = err
if error != nil:
debug "Deposit contract data not available",
blockNumber = blk.number,
err = error.msg
blk.timestamp = Eth1BlockTimestamp (await web3Block).timestamp
proc persistFinalizedBlocks(m: MainchainMonitor, timeNow: float):
Future[tuple[genesisBlock: Eth1Block, previousBlock: Eth1Block]] {.async.} =
let followDistanceInSeconds = uint64(SECONDS_PER_ETH1_BLOCK) *
m.preset.ETH1_FOLLOW_DISTANCE
var prevBlock: Eth1Block
@ -408,13 +409,25 @@ proc persistFinalizedBlocks(m: MainchainMonitor, timeNow: float): tuple[
# TODO: The DB operations should be executed as a transaction here
block: # TODO Begin Transaction
while m.eth1Chain.blocks.len > 0:
let blk = m.eth1Chain.blocks.peekFirst
# TODO keep a separate set of candidate blocks
var blk = m.eth1Chain.blocks.peekFirst
if blk.timestamp != 0:
await fetchBlockDetails(m.dataProvider, blk)
if float(blk.timestamp + followDistanceInSeconds) > timeNow:
break
for deposit in blk.deposits:
m.db.processDeposit(deposit.data)
# TODO compare our results against the web3 provider
blk.voteData.deposit_count = m.db.finalizedEth1DepositsMerkleizer.totalChunks
blk.voteData.deposit_root = m.db.finalizedEth1DepositsMerkleizer.getFinalHash
# TODO
# Try to confirm history by obtaining deposit_root from a more
# recent block
# TODO The len property is currently stored in memory which
# makes it unsafe in the face of failed transactions
let activeValidatorsCount = m.db.immutableValidatorData.lenu64
@ -528,7 +541,7 @@ proc checkForGenesisLoop(m: MainchainMonitor) {.async.} =
# TODO: check for a stale monitor
let
now = epochTime()
(genesisCandidate, genesisParent) = m.persistFinalizedBlocks(now)
(genesisCandidate, genesisParent) = await m.persistFinalizedBlocks(now)
if genesisCandidate != nil:
# We have a candidate state on our hands, but our current Eth1Chain
@ -617,7 +630,7 @@ proc processDeposits(m: MainchainMonitor,
m.checkIfShouldStopMainchainMonitor()
let now = epochTime()
discard m.persistFinalizedBlocks(now)
discard await m.persistFinalizedBlocks(now)
let blk = await m.depositQueue.popFirst()
m.eth1Chain.trimHeight(Eth1BlockNumber(blk.number) - 1)
@ -633,52 +646,8 @@ proc processDeposits(m: MainchainMonitor,
let eth1Blocks = await dataProvider.fetchDepositData(latestKnownBlock + 1,
Eth1BlockNumber blk.number)
if eth1Blocks.len == 0:
if m.eth1Chain.maxValidDeposits > m.preset.MIN_GENESIS_ACTIVE_VALIDATOR_COUNT and
m.eth1Chain.knownStart.deposit_count == 0:
let latestEth1Data = m.eth1Chain.latestEth1Data
for missingBlockNum in latestKnownBlock + 1 ..< Eth1BlockNumber(blk.number):
let missingBlock = await dataProvider.getBlockByNumber(missingBlockNum)
doAssert m.eth1Chain.addSuccessorBlock Eth1Block(
number: Eth1BlockNumber(missingBlock.number),
timestamp: Eth1BlockTimestamp(missingBlock.timestamp),
voteData: latestEth1Data)
doAssert m.eth1Chain.addSuccessorBlock Eth1Block(
number: Eth1BlockNumber(blk.number),
timestamp: Eth1BlockTimestamp(blk.timestamp),
voteData: latestEth1Data)
else:
template logBlockProcessed(blk) =
debug "Eth1 block processed",
`block` = shortLog(blk), totalDeposits = blk.voteData.deposit_count
await dataProvider.fetchBlockDetails(eth1Blocks[0])
if m.eth1Chain.addSuccessorBlock(eth1Blocks[0]):
logBlockProcessed eth1Blocks[0]
for i in 1 ..< eth1Blocks.len:
await dataProvider.fetchBlockDetails(eth1Blocks[i])
if m.eth1Chain.addSuccessorBlock(eth1Blocks[i]):
logBlockProcessed eth1Blocks[i]
else:
raise newException(CorruptDataProvider,
"A non-successor Eth1 block reported")
else:
# A non-continuous chain detected.
# This could be the result of a deeper fork that was not reported
# properly by the web3 provider. Since this should be an extremely
# rare event we can afford to handle it in a relatively inefficient
# manner. Let's delete half of our non-finalized chain and try again.
var blocksToPop = 0
if m.eth1Chain.blocks.len > 0:
blocksToPop = max(1, m.eth1Chain.totalNonFinalizedBlocks div 2)
for i in 0 ..< blocksToPop:
m.eth1Chain.popBlock()
warn "Web3 provider responded with a non-continous chain of deposits",
backtrackedDeposits = blocksToPop
m.depositQueue.addFirstNoWait blk
for i in 0 ..< eth1Blocks.len:
m.eth1Chain.addBlock eth1Blocks[i]
proc isRunning*(m: MainchainMonitor): bool =
not m.runFut.isNil

View File

@ -13,7 +13,7 @@
{.push raises: [Defect].}
import
sequtils, macros,
sequtils,
stew/endians2,
# Specs
../../beacon_chain/spec/[datatypes, digest],
@ -21,13 +21,21 @@ import
# TODO All tests need to be moved to the test suite.
const depositContractLimit* = Limit(1'u64 shl (DEPOSIT_CONTRACT_TREE_DEPTH - 1'u64))
func attachMerkleProofs*(deposits: var openarray[Deposit]) =
let depositsRoots = mapIt(deposits, hash_tree_root(it.data))
const depositContractLimit = Limit(1'u64 shl (DEPOSIT_CONTRACT_TREE_DEPTH - 1'u64))
var incrementalMerkleProofs = createMerkleizer(depositContractLimit)
for i in 0 ..< depositsRoots.len:
incrementalMerkleProofs.addChunkAndGenMerkleProof(depositsRoots[i], deposits[i].proof)
deposits[i].proof[32].data[0..7] = toBytesLE uint64(i + 1)
template getProof*(proofs: seq[Eth2Digest], idxParam: int): openarray[Eth2Digest] =
let
idx = idxParam
startIdx = idx * DEPOSIT_CONTRACT_TREE_DEPTH
endIdx = startIdx + DEPOSIT_CONTRACT_TREE_DEPTH - 1
proofs.toOpenArray(startIdx, endIdx)

View File

@ -29,7 +29,7 @@ const
bitsPerChunk = bytesPerChunk * 8
type
SszChunksMerkleizer* = object
SszChunksMerkleizer* {.requiresInit.} = object
combinedChunks: ptr UncheckedArray[Eth2Digest]
totalChunks: uint64
topIndex: int
@ -287,14 +287,51 @@ func binaryTreeHeight*(totalElements: Limit): int =
bitWidth nextPow2(uint64 totalElements)
type
SszHeapMerkleizer[limit: static[Limit]] = object
chunks: array[binaryTreeHeight limit, Eth2Digest]
SszMerkleizer*[limit: static[Limit]] = object
combinedChunks: ref array[binaryTreeHeight limit, Eth2Digest]
m: SszChunksMerkleizer
proc init*(S: type SszHeapMerkleizer): S =
result.m.combinedChunks = cast[ptr UncheckedArray[Eth2Digest]](addr result.chunks)
result.m.topIndex = result.limit - 1
result.m.totalChunks = 0
proc init*(S: type SszMerkleizer): S =
new result.combinedChunks
result.m = SszChunksMerkleizer(
combinedChunks: cast[ptr UncheckedArray[Eth2Digest]](
addr result.combinedChunks[][0]),
topIndex: binaryTreeHeight(result.limit) - 1,
totalChunks: 0)
proc init*(S: type SszMerkleizer,
combinedChunks: openarray[Eth2Digest],
totalChunks: uint64): S =
new result.combinedChunks
result.combinedChunks[][0 ..< combinedChunks.len] = combinedChunks
result.m = SszChunksMerkleizer(
combinedChunks: cast[ptr UncheckedArray[Eth2Digest]](
addr result.combinedChunks[][0]),
topIndex: binaryTreeHeight(result.limit) - 1,
totalChunks: totalChunks)
proc clone*[L: static[Limit]](cloned: SszMerkleizer[L]): SszMerkleizer[L] =
new result.combinedChunks
result.combinedChunks[] = cloned.combinedChunks[]
result.m = SszChunksMerkleizer(
combinedChunks: cast[ptr UncheckedArray[Eth2Digest]](
addr result.combinedChunks[][0]),
topIndex: binaryTreeHeight(L) - 1,
totalChunks: cloned.totalChunks)
template addChunksAndGenMerkleProofs*(
merkleizer: var SszMerkleizer,
chunks: openarray[Eth2Digest]): seq[Eth2Digest] =
addChunksAndGenMerkleProofs(merkleizer.m, chunks)
template addChunk*(merkleizer: var SszMerkleizer, data: openarray[byte]) =
addChunk(merkleizer.m, data)
template totalChunks*(merkleizer: SszMerkleizer): uint64 =
merkleizer.m.totalChunks
template getFinalHash*(merkleizer: SszMerkleizer): Eth2Digest =
merkleizer.m.getFinalHash
template createMerkleizer*(totalElements: static Limit): SszChunksMerkleizer =
trs "CREATING A MERKLEIZER FOR ", totalElements

View File

@ -188,7 +188,8 @@ proc makeBeaconBlockForHeadAndSlot*(node: BeaconNode,
if node.mainchainMonitor.isNil:
(state.eth1_data, newSeq[Deposit]())
else:
node.mainchainMonitor.getBlockProposalData(state)
let finalizedEth1Data = node.chainDag.getFinalizedEpochRef().eth1_data
node.mainchainMonitor.getBlockProposalData(state, finalizedEth1Data)
let
poolPtr = unsafeAddr node.chainDag # safe because restore is short-lived

@ -1 +1 @@
Subproject commit 2a5cc08a557051947c4076674e01a93bcefed46a
Subproject commit 7f63f72e46a8c5932c802931cefbe2683bb5e5e6