There are a few locations in the code that compare the current epoch to
the various FORK_EPOCH constants and branch off into fork-specific code.
When a new fork is introduced, it is sometimes forgotten to update all
of those branch locations. This patch introduces a compile-time check
that ensures that all branches need to be covered exhaustively. This is
done by replacing if-elif structures with case expressions.
There are a number of locations in the code that get attestations on a
forked beacon state. For attestation pools test, a convenience wrapper
was available to reduce clutter. This patch integrates that wrapper into
the core component so that it can also take advantage of the wrapper.
The initialization of a `SyncAggregate` to its default value is not very
intuitive. There is an `init` function in `sync_committee_msg_pool` that
provides a convenience wrapper. This patch exports that initializer so
that the rest of the code base can also take advantage of it.
Adds a function that constructs a Merkle proof for a generalized index.
This will be used during light client sync to update light clients with
a new state (see NEXT_SYNC_COMMITTEE_INDEX / FINALIZED_ROOT_INDEX).
The phase0 and altair overloads of `makeBeaconBlock` slightly differ in
their signatures which makes using them unnecessarily verbose.
- A placeholder `sync_aggregate` argument similar to `executionPayload`
is added to the phase0 overload to match the altair signature.
- A wrapper operating on `ForkedHashedBeaconState` is introduced.
This still doesn't work properly because it leads to a change in
the "eth" ENR field of the client. Since the client uses the value
of this field to look for other nodes on the network, the change
effectively prevents us from finding peers.
There were still a few instances that used the expansion of `errReject`
instead of using the template itself. It seems that those cases were
forgotten as part of other cleanups in #2809. Done now for readability.
* simplify state fork access pattern
* fixes
* unsafeAddr needs to be dereferenced outside of case for best effect
* remove hash_tree_root of ForkedXxx (doesn't make sense)
* simplify state transition
* fix vc
* readd hash_tree_root(forkedbeaconblock)
* readd htr(fhbs) as well
...and add some protections to not hash the wrong items elsewhere
There are still a few cases with manual loops through sync subcommittees
even though an `allSyncCommittees` iterator exists. Adjusted the few
remaining instances to also use the iterator instead.
When sync committee message handling was introduced in #2830, the edge
case of the same validator being selected multiple times as part of a
sync subcommittee was not covered. Not handling that edge case makes
sync contributions have a lower-than-expected participation rate as each
sync validator is only counted up through once per subcommittee.
This patch ensures that this edge case is properly covered.
The P2P spec defines how certain error classes should be handled through
either IGNORE or REJECT verdicts. For sync committee message, the spec
defines that only the first message from each validator per subcommittee
and slot shall be accepted, the rest is ignored. However, current code
rejects those messages instead of ignoring them. Fixed to match spec.
The README file explaining gossip_processing, and the attestation_flow
docs were no longer accurate, as attestations and aggregates no longer
go through a queue (pending batching). This patch updates the docs
accordingly. It also improves some grammar and fixes some typos.
Currently there is a mix of comment formats in the various preset files.
Altair presets have a leading header line, Phase0 presets don't.
Furthermore, the Altair `minimal` header refers to `mainnet` (#2710).
Updated all of the header lines to match the spec.
The current `getStateField` implementation fails at run-time when called
on a post-Altair state. This is improved by replacing the `if` structure
with a `case` expression, which is checked for exhaustive coverage at
compile time. Care is taken to preserve the `unsafeAddr` optimization.
* Placing callbacks into strategic places.
* Initial events call implementation.
* Post rebase fixes.
* Change addSyncContribution() implementation.
* Add `attestation-sent` event.
Remove gcsafe, raises from callbacks implementations.
Move `attestation-received` fire at the end of attestation processing.
* Address review comments.
Some clients disable their metadata V1 after the Altair fork, so we are unable to ping them correctly.
This PR adds a fallback to try metadata V2 if V1 fails.
* Add parallel attestation verification
* Update tests, batchVerify doesn't use the threadpool with only single core (nim-blscurve update)
* bump nim-blscurve
* Debug info for failing eth2 test vectors
* remove submodule eth2-testnets
* verbose debugging of make failure on Windows (libbacktrace?)
* Remove CI debug mode
* initialization convention
* Fix new altair tests
The template `message` extracts fork specific data from a forked block.
However, its current implementation does not compile due to each fork
returning a different type, i.e., the generated `case` code attempts to
return different types from each branch. As the template is unused, it
is safely removed for now.
* add merge SSZ consensus object tests
* add merge attestations, attester slashing, proposer slashing, and voluntary exit test fixtures
* throw catchable exception rather than assert on invalid SingleMemberUnion selector
* hash_tree_root can assert, because it's trusted data by that point
* re-assert on writing, since also trusted data
* beta.4 update
* implement upgrade_to_merge()
Other changes:
* Add server getBlockV2(), and produceBlockV2().
* Add getBlockV2() to REST test suite.
* Add client getBlockV2(), and produceBlockV2().
* Fix URLs in comments.
* Add some primitives and fix some issues in forks.nim.
* Switch `validator_client` to V2 calls usage.
* Bump `chronos` with imports fixes.
* Bump `nim-json-serialization` for `requireAllFields`.
* construct all unslashed, participating balances in one validator scan
* remove altair benchmarking setup for block_sim
* revert a benchmarking change from proc to func
* remove more benchmarking func/proc tweaks
* re-add asSeq, which is necessary because unsafeAddr
* ... except for the block_sim benchmark part
* approximately double altair epoch processing speed
* don't redundantly clear state balances cache
* only invalidate altair state balances cache once in process_rewards_and_penalties()
* cleanups
* use ForkedTrustedSignedBeaconBlock.ionit where appropriate
* move `is_aggregator` to `spec/`
* use `errReject` in a few more places
* update enr fork id when time is auspicious
* use network broadcast functions
* Return Ignore for aggregate signature validation timeouts
...consistently between aggregates and attestations.
* clean up some more reject/ignore rules
* shorten texts a bit
* errReject->checkedReject, use err helpers throughout
* get rid of quarantine in exitpool as well
* Fix getForkSchedule call.
Create cache of all configuration endpoints at node startup.
Add prepareJsonResponse() call to create cached responses.
Mark all procedures with `raises`.
* Add getForkSchedule to VC.
Fix getForkSchedule return type for API.
More `raises` annotations.
Fix VC fork_service.nim.
* Use `push raises` instead of inline `raises`.
* Improvements for REST API aggregated attestations and attestations processing.
* Rename eth2_network.sendXXX procedures to eth2_network.broadcastXXX.
Add broadcastBeaconBlock() and broadcastAggregateAndProof().
Fix links to specification in REST API declarations.
Add implementation for v2 getStateV2().
Add validator_duties.sendXXX procedures which not only broadcast data, but also validate it.
Fix JSON-RPC/REST to use new validator_duties.sendXXX procedures instead of own implementations.
* Fix validator_client online nodes count incorrect value.
Fix aggregate and proof attestation could be sent too late.
* Adding timeout for block wait in attestations processing.
Fix compilation errors.
* Attempt to debug aggregate and proofs.
* Fix Beacon AIP to use `sendAttestation`.
Add link comment to produceBlockV2.
* Add debug logs before publish operation for blocks, attestations and aggregated attestations.
Fix attestations publishing issue.
* logging fixes
`indexInCommnittee` already logged in attestation
Co-authored-by: Jacek Sieka <jacek@status.im>
* start of peer cycling
* discover new peers when low coverage
* disconnect unresponsive peers
* cleaning up a bit
* smarter cycling
* remove lowpeer event
* use sync_protocol to get metadata
* moved cycling to discovery
* fix discovery for tests
* change test
* various tweaks
* Faster discovery
* start of big refacto
* removed peer_balancer
* updated test
* added lazydial
* allow at least 10 incoming conn
* popcount -> countOnes
* allow max_conn - outgoingconn incoming connections
* add dOut check
* Removed lazyDial option and better kick
* use attnets in scoring
* only score peer with metadata
* cleanup
* use topic subscription
* rework dialing filtering, fix metadata compat issue
* send attestations and exit messages on fork-appropriate topic
* document why use wall clock over attestation slot
* centralize some fork-topic-picking-logic in eth2_network
* pick up new test in summary
* allow specified GetTimeFn for testing purposes
* add GenesisTime and use it in eth2_network
* replace GetTimeFn and GenesisTime with GetBeaconTimeFn
* reorganize ssz dependencies
This PR continues the work in
https://github.com/status-im/nimbus-eth2/pull/2646,
https://github.com/status-im/nimbus-eth2/pull/2779 as well as past
issues with serialization and type, to disentangle SSZ from eth2 and at
the same time simplify imports and exports with a structured approach.
The principal idea here is that when a library wants to introduce SSZ
support, they do so via 3 files:
* `ssz_codecs` which imports and reexports `codecs` - this covers the
basic byte conversions and ensures no overloads get lost
* `xxx_merkleization` imports and exports `merkleization` to specialize
and get access to `hash_tree_root` and friends
* `xxx_ssz_serialization` imports and exports `ssz_serialization` to
specialize ssz for a specific library
Those that need to interact with SSZ always import the `xxx_` versions
of the modules and never `ssz` itself so as to keep imports simple and
safe.
This is similar to how the REST / JSON-RPC serializers are structured in
that someone wanting to serialize spec types to REST-JSON will import
`eth2_rest_serialization` and nothing else.
* split up ssz into a core library that is independendent of eth2 types
* rename `bytes_reader` to `codec` to highlight that it contains coding
and decoding of bytes and native ssz types
* remove tricky List init overload that causes compile issues
* get rid of top-level ssz import
* reenable merkleization tests
* move some "standard" json serializers to spec
* remove `ValidatorIndex` serialization for now
* remove test_ssz_merkleization
* add tests for over/underlong byte sequences
* fix broken seq[byte] test - seq[byte] is not an SSZ type
There are a few things this PR doesn't solve:
* like #2646 this PR is weak on how to handle root and other
dontSerialize fields that "sometimes" should be computed - the same
problem appears in REST / JSON-RPC etc
* Fix a build problem on macOS
* Another way to fix the macOS builds
Co-authored-by: Zahary Karadjov <zahary@gmail.com>
The spec imports are a mess to work with, so this branch cleans them up
a bit to ensure that we avoid generic sandwitches and that importing
stuff generally becomes easier.
* reexport crypto/digest/presets because these are part of the public
symbol set of the rest of the spec types
* don't export `merge` types from `base` - this causes circular deps
* fix circular deps in `ssz/spec_types` - this is the first step in
disentangling ssz from spec
* be explicit about phase0 vs altair - longer term, `altair` will become
the "natural" type set, then merge and so on, so no point in giving
`phase0` special preferential treatment
Simpler module name for stuff that covers forks
* check that runtime config matches database state
* also include some assorted altair cleanups
* use "standard" genesis fork in local testnet to work around missing
runtime config support
* update validator key cache on startup
Versions prior to 1.1.0 do not write a validator key cache at all.
Versions from 1.4.0 and upwards require an immutable validator key cache
to verify blocks - normally, block verification fills the cache but that
assumes that at least one block was verified by a version that has the
key cache.
Taken together, this breaks direct upgrades from anything <1.1.0 to
1.4.0.
The fix is simply to refresh fill the cache from an existing state on
startup.
* also log serious block validation failures at info level
This refactoring puts the JSON-RPC and REST APIs on more equal footing
by renaming and moving things around, creating a separation between
client and server, and documenting what they are - the aim is to have a
simple-to-use base to start from when developing API clients, as well as
make it easier to navigate the code when looking for the legacy JSON-RPC
interface vs the new REST API.
* move REST client, serialization and supporting types to spec/eth2_apis
* REST stuff now starts with `rest_`, JSON-RPC stuff starts with `rpc_`,
more or less
* simplify imports such that there's a simple module to import for both
server and client
* map REST type and proc names to yaml spec more closely - in
particular, reuse operation and type names in `rest_types` to make
comparisons against spec more easy
* cleaner separation between client and server modules - modules common
between server and client such as `rest_types` and serialization move to
the spec folder - this allows the client to be built with less knowledge
about server internals
* Fix firstSuccess() template missing timeouts.
* Fix validator race condition.
Fix logs to be compatible with beacon_node logs.
Add CatchableError handlers to avoid crashes.
Move some logs from Notice to Debug level.
Fix some [unused] warnings.
* Fix block proposal issue for slots in the past and from the future.
* Change sent to published.
* Address review comments #1.
* some whole-file copies from altair branch
* rpc/node_api and rpc/node_rest_api also need to be copied
* remove new sync committee-related functionality
* bump libp2p
* altair sync v2
Use V2 sync requests after the altair fork has happened, according to
the wall clock
* Fix the behavior of the v1 req/resp calls after Altair
Co-authored-by: Zahary Karadjov <zahary@gmail.com>
* Initial commit.
* Exporting getConfig().
* Add beacon node checking procedures.
* Post rebase fixes.
* Use runSlotLoop() from nimbus_beacon_node.
Fallback implementation.
Fixes for ETH2 REST serialization.
* Add beacon_clock.durationToNextSlot().
Move type declarations from beacon_rest_api to json_rest_serialization.
Fix seq[ValidatorIndex] serialization.
Refactor ValidatorPool and add some utility procedures.
Create separate version of validator_client.
* Post-rebase fixes.
Remove CookedPubKey from validator_pool.nim.
* Now we should be able to produce attestations and aggregate and proofs.
But its not working yet.
* Debugging attestation sending.
* Add durationToNextAttestation.
Optimize some debug logs.
Fix aggregation_bits encoding.
Bump chronos/presto.
* Its alive.
* Fixes for launch_local_testnet script.
Bump chronos.
* Switch client API to not use `/api` prefix.
* Post-rebase adjustments.
* Fix endpoint for publishBlock().
* Add CONFIG_NAME.
Add more checks to ensure that beacon_node is compatible.
* Add beacon committee subscription support to validator_client.
* Fix stacktrace should be an array of strings.
Fix committee subscriptions should not be `data` keyed.
* Log duration to next block proposal.
* Fix beacon_node_status import.
* Use jsonMsgResponse() instead of jsonError().
* Fix graffityBytes usage.
Remove unnecessary `await`.
Adjust creation of SignedBlock instance.
Remove legacy files.
* Rework durationToNextSlot() and durationToNextEpoch() to use `fromNow`.
* Fix race condition for block proposal and attestations for same slot.
Fix local_testnet script to properly kill tasks on Windows.
Bump chronos and nim-http-tools, to allow connections to infura.io (basic auth).
* Catch services errors.
Improve performance of local_testnet.sh script on Windows.
Fix race condition when attestation producing.
* Post-rebase fixes.
* Bump chronos and presto.
* Calculate block publishing delay.
Fix pkill in one more place.
* Add error handling and timeouts to firstSuccess() template.
Add onceToAll() template.
Add checkNodes() procedure.
Refactor firstSuccess() template.
Add error checking to api.nim calls.
* Deprecated usage onceToAll() for better stability.
Address comment and send attestations asap.
* Avoid unnecessary loop when calculating minimal duration.
* Implement split preset/config support
This is the initial bulk refactor to introduce runtime config values in
a number of places, somewhat replacing the existing mechanism of loading
network metadata.
It still needs more work, this is the initial refactor that introduces
runtime configuration in some of the places that need it.
The PR changes the way presets and constants work, to match the spec. In
particular, a "preset" now refers to the compile-time configuration
while a "cfg" or "RuntimeConfig" is the dynamic part.
A single binary can support either mainnet or minimal, but not both.
Support for other presets has been removed completely (can be readded,
in case there's need).
There's a number of outstanding tasks:
* `SECONDS_PER_SLOT` still needs fixing
* loading custom runtime configs needs redoing
* checking constants against YAML file
* yeerongpilly support
`build/nimbus_beacon_node --network=yeerongpilly --discv5:no --log-level=DEBUG`
* load fork epoch from config
* fix fork digest sent in status
* nicer error string for request failures
* fix tools
* one more
* fixup
* fixup
* fixup
* use "standard" network definition folder in local testnet
Files are loaded from their standard locations, including genesis etc,
to conform to the format used in the `eth2-networks` repo.
* fix launch scripts, allow unknown config values
* fix base config of rest test
* cleanups
* bundle mainnet config using common loader
* fix spec links and names
* only include supported preset in binary
* drop yeerongpilly, add altair-devnet-0, support boot_enr.yaml
We generally send out attestations 250 ms after the block arrives.
Recent efficiency improvements have led to a slightly increased
incidence of "slot 0" issues where attestations are dropped by other
nodes because they have not yet had time to process the block due to
epoch processing taking time.
This PR mitigates the problem by increasing the window between receiving
the block and sending out attestations.
* Implement the new Altair req/resp protocols
Also fixes the altair message-id computation by providing the correct
forkdigest prefix in `isAltairTopic`.
Co-authored-by: Tanguy Cizain <tanguycizain@gmail.com>
We generally send out attestations 250 ms after the block arrives.
Recent efficiency improvements have led to a slightly increased
incidence of "slot 0" issues where attestations are dropped by other
nodes because they have not yet had time to process the block due to
epoch processing taking time.
This PR mitigates the problem by increasing the window between receiving
the block and sending out attestations.
* add blockchain_dag altair database reading; add rollback tests; fix some unnecessary type conversions
* remove debugging scaffolding
* proposeSignedBlock() will need to be async for merge; introduce altair types to VC
* introduce immutable Altair BeaconState
* add database support for Altair blocks and states
* add tests for Altair get/put/contains/delete state
* enable blockchain_dag Altair state database storing
* properly return error on getting missing altair block
* add Altair support to the block quarantine
* switch some spec/datatypes imports to spec/datatypes/base
* add Altair support to block_clearance
* allow runtime configuration of Altair transition slot
* enable Altair in block_sim, including in CI
* remove false OnBlockAdded dependency on phase.HashedBeaconState
* introduce altair data types into block_clearance; update some alpha.6 spec refs to alpha.7; add get_active_validator_indices_len ForkedHashedBeaconState wrapper
* switch many modules from using datatypes (with phase0 states/blocks) to datatypes/base (fork-independent); update spec refs from alpha.6 to alpha.7 and remove rm'd G2_POINT_AT_INFINITY
* switch more modules from using datatypes (with phase0 states/blocks) to datatypes/base (fork-independent); update spec refs from alpha.6 to alpha.7
* remove unnecessary phase0-only wrapper of get_attesting_indices(); allow signatures_batch to process either fork; remove O(n^2) nested loop in process_inactivity_updates(); add altair support to getAttestationsforTestBlock()
* add Altair versions of asSigVerified(), asTrusted(), and makeBeaconBlock()
* fix spec URL to be Altair for Altair makeBeaconBlock()
* strawman doppelganger detection walltime refactor
* move DoppelgangerProtection to Eth2Processor
* increase comment precision
* document difference between broadcastStartEpoch and nodeLaunchSlot, and allow for one-slot overlap to avoid false positives on intra-slot restarts
* strawman doppelganger detection walltime refactor
* move DoppelgangerProtection to Eth2Processor
* increase comment precision
* document difference between broadcastStartEpoch and nodeLaunchSlot, and allow for one-slot overlap to avoid false positives on intra-slot restarts
* update to Altair as of v1.1.0-alpha.7
* introduce Altair types into attestation pool
* avoid allocating/copying pubkeys excessively in get_next_sync_committee()
* use ForkedHashedBeaconState in StateData
* fix FAR_FUTURE_EPOCH -> slot overflow; almost always use assign()
* avoid stack allocation in maybeUpgradeStateToAltair()
* create and use dispatch functions for check_attester_slashing(), check_proposer_slashing(), and check_voluntary_exit()
* use getStateRoot() instead of various state.data.hbsPhase0.root
* remove withStateVars.hashedState(), which doesn't work as a design anymore
* introduce spec/datatypes/altair into beacon_chain_db
* fix inefficient codegen for getStateField(largeStateField)
* state_transition_slots() doesn't either need/use blocks or runtime presets
* combine process_slots(HBS)/state_transition_slots(HBS) which differ only in last-slot htr optimization
* getStateField(StateData, ...) was replaced by getStateField(ForkedHashedBeaconState, ...)
* fix rollback
* switch some state_transition(), process_slots, makeTestBlocks(), etc to use ForkedHashedBeaconState
* remove state_transition(phase0.HashedBeaconState)
* remove process_slots(phase0.HashedBeaconState)
* remove state_transition_block(phase0.HashedBeaconState)
* remove unused callWithBS(); separate case expression from if statement
* switch back from nested-ref-object construction to (ref Foo)(Bar())
* write uncompressed validator keys to database
Loading 150k+ validator keys on startup in compressed format takes a lot
of time - better store them in uncompressed format which makes behaviour
just after startup faster / more predictable.
* refactor cached validator key access
* fix isomorphic cast to work with non-var instances
* remove cooked pubkey cache - directly use database cache in chaindag
as well (one less cache to keep in sync)
* bump blscurve, introduce loadValid for known-to-be-valid keys
* load the cache of the current state epoch instead of the target state
epoch, when applying states and slots
* load state cache for each slot/block (for longer slot jumps)
* load state cache after full updateStateData
* look up two state cache epochs, instead of the same epoch twice :)
Instead of keeping a validator key list per EpochRef, this PR introduces
a single shared validator key list in ChainDAG, and cleans up some other
ChainDAG and key-related issues.
The PR does not introduce the validator key list in the state transition
- this is because we batch-check all signatures before entering the spec
code, thus the spec code never hits the cache.
A future refactor should _probably_ remove the threadvar altogether.
There's a few other small fixes in here that make the flow easier to
read:
* fix `var ChainDAGRef` -> `ChainDAGRef`
* fix `var QuarantineRef` -> `QuarantineRef`
* consistent `dag` variable name
* avoid using threadvar pubkey cache in most cases
* better error messages in batch signature checking
* update ChainDAG.effective_balance() to use StateData; rm unused ChainDAG.getBlockByPreciseSlot()
* update get_effective_balances to avoid god object; avoid most memory allocation in Altair epoch reward and penalty processing
* don't consider legacy database when writing state - this read is slow
on kvstore
* avoid epoch transition when there's an exact match in cache already
* simplify init to only consider checkpoint states
This way we perform the expensive epoch processing before the block
arrives.
Of course, this may lead to speculative misses which in turn lead to
replays - it's likely that in the case of a miss, we'll see a replay
regardless.
* gossip_to_consensus -> block_processor (it's processing only blocks,
but not only from gossip)
* measure queue and validation time for blocks
* measure assignment and state loading times for updateStateData
* avoid some unnecessary block copies in block sync
* warn that database is corrupt if we hit tail without a state
* proposed structure for hf1
* refactor datatypes.nim into datatypes/{base, phase0, hf1}.nim
* hf1 is Altair
* some syncing with alpha 2
* adjust epoch processing to disambiguate access to RewardFlags
* relocate StateData to stay consistent with meaning phase 0 StateData
* passes v1.1.0 alpha 5 SSZ consensus object tests
* Altair block header test fixtures work
* fix slash_validator() so that Altair attester slashings, proposer slashings, and voluntary exit textures work
* deposit operation Altair test fixtures work
* slot sanity and all but a couple epoch transition tests switched to Altair
* attestation Altair test fixtures work
* Altair block sanity test fixtures work
* add working altair sync committee tests
* improve workarounds for sum-types-across-modules Nim bug; incorporate SignedBeaconBlock root reconstuction to SSZ byte reader
This reverts commit eebc828778.
Adding a separate file turns out not to be enough. This PR reverts the
separate file change.
Another theory is that the large kvstore table causes cache thrashing -
all database connections share a common page cache which would explain
the poor performance of the separate file solution.
The V1 table structure shows great improvements in performance, but if
there's an old `kvstore` without rowid:s, these benefits are nullified:
reorgs during writes and deletes remain expensive (even if the
degradation is reduced somewhat).
This PR creates the tables in a new file instead, and uses the old file
as a read-only store - this has several interesting properties:
* the old database is left completely untouched - this guarantees that
downgrades work smooth (they'll only need to resync their missing
portions)
* starting sync after this PR means only a v1 database is created
* v0 databases stick around - no migration is performed (for now)
Future PR:s can introduce migration of the data from one database to
another - a simply copy will take hours which is downtime we want to
avoid - at that point, it might make sense to migrate straight to era
files instead.
* use StateData in place of BeaconState outside state transition code
* propagate more StateData usage
* remove withStateVars().state
* wrap get_beacon_committee(BeaconState, ...) as gbc(StateData, ...)
* switch makeAttestation() to use StateData
* use StateData wrapper/dispatcher for get_committee_count_per_slot()
* convert AttestationCache.init(), weak subjectivity functions, and updateValidatorMetrics()
* add get_shuffled_active_validator_indices(StateData) and get_block_root_at_slot(StateData)
* switch makeAttestationData() to StateData
* sync AllTests-mainnet.md after rebase
* Error when specifying an invalid --data-dir (or --validator-dir)
* Error when entering an invalid validator public key (e.g. invalid hex value)
* Warning when attempting to export a validator not present in the local database
Some unnecessary remains of the v1 mode has been removed as well
* Revert "Revert "Upgrade database schema" (#2570)"
This reverts commit 6057c2ffb4.
* ssz: fix loading empty lists into existing instances
Not a problem earlier because we didn't reuse instances
* bump nim-eth
* bump nim-web3
The `kvstore` design we're using now turns out to not be the best way to
use `sqlite` - in particular, there are some significant benefits to
using rowid in certain situations and to keep data in separate tables.
With this branch, there are massive improvements in startup time
(seconds instead of minutes) and state/block storage and pruning times
(milliseconds instead of seconds) - these improvements can in particular
be seen on slow drives and translate directly into better attestation
performance.
* update kvstore to new keyspace design
* remove `DirStoreRef` and the hidden `--state-db-kind` option - this
was an experiment to store large blobs in files, but with the new
kvstore, there's no compelling reason to do so
* remove `DbMap` - unused and would need updating for new keyspace
design
* introduce separate tables for each data type (blocks, states etc)
* remove "WITHOUT ROWID" pessimization for tables with large blobs
* close DbSeq statements explicitly (and earlier)
* store beacon block summaries in separate table, without SSZ
compression and load them all with single query on startup
* stop storing backwards compat full states
* mark genesis beacon block as trusted
* avoid faststreams when loading SSZ data
* remove `DisagreementBehavior` (unused)
This PR decreases the lead subscription time which should help
decrease bandwidth usage and CPU making the subscription for future
aggregation happen a bit later. There's room for more tuning here,
probably.
* fix missing negation from in #2550
* fix silly bitarray issues
* decrease subnet lead subscription time
* log all subnet switching source data
* rename subnet trackers to refer to stability and aggregate subnets
* more tests
Currently, we have a bit of a convoluted flow where when sending
attestations, we start broadcasting them over gossip then pass them to
the attestation validation to include them in the local attestation pool
- it should be the other way around: we should be checking attestations
_before_ gossipping them - this serves as an additional safety net to
ensure that we don't publish junk - this becomes more important when
publishing attestations from the API.
Also, the REST API was performing its own validation meaning
attestations coming from REST would be validated twice - finally, the
JSON RPC wasn't pre-validating and would happily broadcast invalid
attestations.
* Unified attestation production pipeline with the same flow for gossip,
locally and API-produced attestations: all are now validated and entered
into the pool, then broadcast/republished
* Refactor subnet handling with specific SubnetId alias, streamlining
where subnets are computed, avoiding the need to pass around the number
of active validators
* Move some of the subnet handling code to eth2_network
* Use BitArray throughout for subnet handling
This also makes future efforts to provide metrics and logs for
attestation efficiency easier
* Export rewards from epoch transition
* Use less memory for reward calculation (bool -> set[enum], field
alignment)
* Reuse reward memory when replaying, avoiding spike
* Allow replaying any range in ncli_db benchmark
When applying a block, we'll currently compute a state root for the
state after slot processing but before block processing - this is
unnecessary when a block is being applied because the intermediate state
root is never observed.
This PR reduces the number of database queries for slashing protection
from 5 reads and 1 write to 2 reads and 1 write in the optimistic case.
In the process, it removes user-level support for writing the database
in the version 1 format in order to simplify the code flow, and prevent
code rot. In particular, the v1 format was not covered by any unit tests
and has no advantages over v2. The concrete code to read and write it
remains for now, in particular to support upgrades from v1 to v2.
The branch also removes the use of concepts which doesn't work with
checked exceptions - in particular, this highlights code that both
raises exceptions and returns error codes, which could be cleaned up in
the future.
* Cache internal validator ID
* Rely on unique index to check for trivial duplicate votes
* Combine two surround vote queries into one
* Combine API for checking and registering slashing into single function
The slashing DB is normally not a bottleneck, but may become one with
high attached validator counts.
* Introduce unittest2 and junit reports
* fix XML path
* don't combine multiple CI runs
* fixup
* public combined report also
Co-authored-by: Ștefan Talpalaru <stefantalpalaru@yahoo.com>
* REST API test framework and tests.
* Fix ValidatorIndex tests to properly handle int32, but not uint32 values.
* Fix tests to follow latest REST fixes.
* refactor restapi.sh
and add it to the test suite
* Fix issues.
Add delay timeout which is required.
* Fix restapi.sh script for Windows.
Co-authored-by: Ștefan Talpalaru <stefantalpalaru@yahoo.com>
With the introduction of batching and lazy attestation aggregation, it
no longer makes sense to enqueue attestations between the signature
check and adding them to the attestation pool - this only takes up
valuable CPU without any real benefit.
* add successfully validated attestations to attestion pool directly
* avoid copying participant list around for single-vote attestations,
pass single validator index instead
* release decompressed gossip memory earlier, specially during async
message validation
* use cooked signatures in a few more places to avoid reloads and errors
* remove some Defect-raising versions of signature-loading
* release decompressed data memory before validating message
This patch writes a full genesis state to `kvstore` if one was missing,
which fixes 1.2.0 restarting sync when upgrading from 1.1.0, or when
downgrading to a pre-1.1.0 release.
* avoid creating indexed attestation just to check signatures - above
all, don't create it when not checking signatures ;)
* avoid pointer op when adding attestation to pool
* better iterator for yielding attestations
* add metric / log for attestation packing time
This is a revamp of the attestation pool that cleans up several aspects
of attestation processing as the network grows larger and block space
becomes more precious.
The aim is to better exploit the divide between attestation subnets and
aggregations by keeping the two kinds separate until it's time to either
produce a block or aggregate. This means we're no longer eagerly
combining single-vote attestations, but rather wait until the last
moment, and then try to add singles to all aggregates, including those
coming from the network.
Importantly, the branch improves on poor aggregate quality and poor
attestation packing in cases where block space is running out.
A basic greed scoring mechanism is used to select attestations for
blocks - attestations are added based on how much many new votes they
bring to the table.
* Collect single-vote attestations separately and store these until it's
time to make aggregates
* Create aggregates based on single-vote attestations
* Select _best_ aggregate rather than _first_ aggregate when on
aggregation duty
* Top up all aggregates with singles when it's time make the attestation
cut, thus improving the chances of grabbing the best aggregates out
there
* Improve aggregation test coverage
* Improve bitseq operations
* Simplify aggregate signature creation
* Make attestation cache temporary instead of storing it in attestation
pool - most of the time, blocks are not being produced, no need to keep
the data around
* Remove redundant aggregate storage that was used only for RPC
* Use tables to avoid some linear seeks when looking up attestation data
* Fix long cleanup on large slot jumps
* Avoid some pointers
* Speed up iterating all attestations for a slot (fixes#2490)
* only deserialize attestation and aggregation gossiped signatures once
* re-indent some aggregate checks into block scope
* spelling
* remove debugging assertion
* put part of gossip validation back into block context
* attestation pool test signature loading isn't so unsafe, and exportRaw isn't free
* remove more development doAsserts; don't exportRaw in loops
* Reset cached indices when resetting cache on SSZ read
When deserializing into an existing structure, the cache should be
cleared - goes for json also. Also improve error messages.
* batch attestations
* Fixes (but now need to investigate the chronos 0 .. 4095 crash similar to https://github.com/status-im/nimbus-eth2/issues/1518
* Try to remove the processing loop to no avail :/
* batch aggregates
* use resultsBuffer size for triggering deadline schedule
* pass attestation pool tests
* Introduce async gossip validators. May fix the 4096 bug (reentrancy issue?) (similar to sync unknown blocks #1518)
* Put logging at debug level, add speed info
* remove unnecessary batch info when it is known to be one
* downgrade some logs to trace level
* better comments [skip ci]
* Address most review comments
* only use ref for async proc
* fix exceptions in eth2_network
* update async exceptions in gossip_validation
* eth2_network 2nd pass
* change to sleepAsync
* Update beacon_chain/gossip_processing/batch_validation.nim
Co-authored-by: Jacek Sieka <jacek@status.im>
Co-authored-by: Jacek Sieka <jacek@status.im>
* update some v1.1.0 alpha1 to alpha2
* remove unused getDepositMessage overload and move other out of datatypes/base
* bump nim-eth2-scenarios to download v1.1.0-alpha.2 test vectors
* construct object rather than result
Since quite a lot of additional procs were now compiled as generics, this lead to compiler bugs that had
to be worked-around:
* The `Domain` type was renamed to `Eth2Domain` to avoid compilation errors
due to conflicts with `nativesockets.Domain`.
Similarly, `eth2_network.KeyPair` was renamed to `NetKeyPair`.
* A new more robust version of `hexToByteArray` was added to stew
* Add default port info
* Add default port info
* Edit CLI and docs to reflect 9000 port
* Apply suggestions from code review
Co-authored-by: Jacek Sieka <arnetheduck@gmail.com>
* set upper bound on EpochRef cache
* max 32 EpochRef instances
* less memory waste in BlockRef by removing EpochRef seq that is mostly
unused (~20mb)
* less memory waste in dag block lookup by not keeping an extra copy of
digest (~70mb)
* fix `==` and `$` for Eth2Digest
* remove `ChainDAG.tmpState` (~50mb?)
all in all, this branch cuts mainnet memory usage by ~160-180mb and puts
limits on EpochRef cache usage - where normally it hovered around 950mb
before, it's now sitting at 600-700mb on my machine.
* docs
* initial immutable validator database factoring
* remove changes from chain_dag: this abstraction properly belongs in beacon_chain_db
* add merging mutable/immutable validator portions; individually test database roundtripping of immutable validators and states-sans-immutable-validators
* update test summaries
* use stew/assign2 instead of Nim assignment
* add reading/writing of immutable validators in chaindag
* remove unused import
* replace chunked k/v store of immutable validators with per-row SQL table storage
* use List instead of HashList
* un-stub some ncli_db code so that it uses
* switch HashArray to array; move BeaconStateNoImmutableValidators from datatypes to beacon_chain_db
* begin only-mutable-part state storage
* uncomment some assigns
* work around https://github.com/nim-lang/Nim/issues/17253
* fix most of the issues/oversights; local sim runs again
* fix test suite by adding missing beaconstate field to copy function
* have ncli bench also store immutable validators
* extract some immutable-validator-specific code from the beacon chain db module
* add more rigorous database state roundtripping, with changing validator sets
* adjust ncli_db to use new schema
* simplify putState/getState by moving all immutable validator accounting into beacon state DB
* remove redundant test case and move code to immutable-beacon-chain module
* more efficient, but still brute-force, mutable+immutable validator merging
* reuse BeaconState in getState
* ensure HashList/HashArray caches are cleared when reusing getState buffers; add ncli_db and a unit test to verify this
* HashList.clear() -> HashList.clearCache()
* only copy incrementally necessary immutable validators
* increase strictness of test cases and fix/work around resulting HashList cache invalidation issues
* remove explanatory scaffolding
* allow for storage of full (with all validators) states for backwards/forwards-compatibility
* adjust DbSeq type usage
* store full, with-validators, state every 64 epochs to enable reverting versions
* reduce memory allocation and intermediate objects in state storage codepath
* eliminate allocation/copying through intermediate BeaconStateNoImmutableValidators objects
* skip benchmarking initial genesis-validator-heavy state store
* always store new-style state and sometimes old-style state
* document intent behind BeaconState/Validator type-punnery
* more accurate failure message on SQLite in-memory database initialization failure
Era files contain 8192 blocks and a state corresponding to the length of
the array holding block roots in the state, meaning that each block is
verifiable using the pubkeys and block roots from the state. Of course,
one would need to know the root of the state as well, which is available
in the first block of the _next_ file - or known from outside.
This PR also adds an implementation to write e2s, e2i and era files, as
well as a python script to inspect them.
All in all, the format is very similar to what goes on in the network
requests meaning it can trivially serve as a backing format for serving
said requests.
Mainnet, up to the first 671k slots, take up 3.5gb - in each era file,
the BeaconState contributes about 9mb at current validator set sizes, up
from ~3mb in the early blocks, for a grand total of ~558mb for the 82 eras
tested - this overhead could potentially be calculated but one would lose
the ability to verify individual blocks (eras could still be verified using
historical roots).
```
-rw-rw-r--. 1 arnetheduck arnetheduck 16 5 mar 11.47 ethereum2-mainnet-00000000-00000001.e2i
-rw-rw-r--. 1 arnetheduck arnetheduck 1,8M 5 mar 11.47 ethereum2-mainnet-00000000-00000001.e2s
-rw-rw-r--. 1 arnetheduck arnetheduck 65K 5 mar 11.47 ethereum2-mainnet-00000001-00000001.e2i
-rw-rw-r--. 1 arnetheduck arnetheduck 18M 5 mar 11.47 ethereum2-mainnet-00000001-00000001.e2s
...
-rw-rw-r--. 1 arnetheduck arnetheduck 65K 5 mar 11.52 ethereum2-mainnet-00000051-00000001.e2i
-rw-rw-r--. 1 arnetheduck arnetheduck 68M 5 mar 11.52 ethereum2-mainnet-00000051-00000001.e2s
-rw-rw-r--. 1 arnetheduck arnetheduck 61K 5 mar 11.11 ethereum2-mainnet-00000052-00000001.e2i
-rw-rw-r--. 1 arnetheduck arnetheduck 62M 5 mar 11.11 ethereum2-mainnet-00000052-00000001.e2s
```
* Deferred DAG and fork choice pruning
* fixup
* Address https://github.com/status-im/nimbus-eth2/pull/2384/files#r589448448, rely only on onSLotEnd for state pruning
* no need to store needPruning in the data structure
* lastPrunePoint is updated in pruning proc
* Split eager and LazyPruning
* enforce pruning in updateHead
* allow multiple hard fork datatypes to coexist
* update to 1.0.1
* merge recent datatypes.nim updates
* trigger rebuild now the out-of-disk-space machine offline
* fix replays stalling processing
Occasionally, attestations will arrive that vote for a target derived
either from the finalized block or earlier. In these cases, Nimbus would
replay the state transition of up to 32 epochs worth of blocks because
the finalized state has been pruned, delaying other processing and
leading to poor inclusion distance.
* put cheap attestation checks before forming EpochRef
* check that attestation target is not from an unviable history with
regards to finalization
* fix overly aggressive state pruning removing the state close to the
finalized checkpoint resulting in rare long replays for valid
attestations
* log long replays
* harden logging and traversal of nil BlockSlot
* simplify target check
no need to lookup target in chain dag again
* fixup
* fixup
* refactor slot loop
* fix attestations being sent out early when _any_ block arrives (as
opposed to the block for the "correct" slot)
* fix attestations being sent out late when block already arrived
* refactor slot processing loop
* shutdown if clock moves backwards significantly
* fix docs
* notify caller whether the block actually arrived
* 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
* database state storage benchmarking via ncli_db
* more cleanups from immutable validator state branch
* unexport some eth2_network constants and remove unused variables/templates
* make two PeerScore constants public
* Create CLI tool for slashing export
* Use SQLite as a DB instead of a KV-store
* Keeps v1 and v2 DBs around
* Uses the same schema as Lighthouse v1.1.0
* Passes all interchange tests + skeleton of finalization pruning
* Removes tests that would violate v5 / minimal slashing DB and MinSlot rules
* Migration tool added using low-watermark scheme for faster migration of large number of validators
* force pushing to fix unstable base
* increase attestation/aggregate queue sizes
when there are many validators, many aggregates and attestations arrive
every slot - increase the queue size a bit - also do batches on each
idle loop iteration since it's fairly quick
* don't score subnets for now
* wrapping up
* refactor and cleanups
* gossip parameters fixes
* comment fix
Co-authored-by: Jacek Sieka <jacek@status.im>
* hotfix gossip scoring
* skip gossip scoring parameters validation as they violate for now (but does not matter cos we don't score)
* workaround again gossip validation
* expose node signatures
* format bitseqs as hex strings
* format trusted sigs as hex strings (same as untrusted)
* reuse rpc client sigs
* include validator index in duties
* move SyncInfo to spec
when there are many validators, many aggregates and attestations arrive
every slot - increase the queue size a bit - also do batches on each
idle loop iteration since it's fairly quick
* use IntSet rather than HashSet[ValidatorIndex]
* add bounds check before uint64 -> int conversion
* use intsets in block transitions
* remove superfluous Nim issue explanation/reference
* performance fixes
* don't mark tree cache as dirty on read-only List accesses
* store only blob in memory for keys and signatures, parse blob lazily
* compare public keys by blob instead of parsing / converting to raw
* compare Eth2Digest using non-constant-time comparison
* avoid some unnecessary validator copying
This branch will in particular speed up deposit processing which has
been slowing down block replay.
Pre (mainnet, 1600 blocks):
```
All time are ms
Average, StdDev, Min, Max, Samples, Test
Validation is turned off meaning that no BLS operations are performed
3450.269, 0.000, 3450.269, 3450.269, 1, Initialize DB
0.417, 0.822, 0.036, 21.098, 1400, Load block from database
16.521, 0.000, 16.521, 16.521, 1, Load state from database
27.906, 50.846, 8.104, 1507.633, 1350, Apply block
52.617, 37.029, 20.640, 135.938, 50, Apply epoch block
```
Post:
```
3502.715, 0.000, 3502.715, 3502.715, 1, Initialize DB
0.080, 0.560, 0.035, 21.015, 1400, Load block from database
17.595, 0.000, 17.595, 17.595, 1, Load state from database
15.706, 11.028, 8.300, 107.537, 1350, Apply block
33.217, 12.622, 17.331, 60.580, 50, Apply epoch block
```
* more perf fixes
* load EpochRef cache into StateCache more aggressively
* point out security concern with public key cache
* reuse proposer index from state when processing block
* avoid genericAssign in a few more places
* don't parse key when signature is unparseable
* fix `==` overload for Eth2Digest
* preallocate validator list when getting active validators
* speed up proposer index calculation a little bit
* reuse cache when replaying blocks in ncli_db
* avoid a few more copying loops
```
Average, StdDev, Min, Max, Samples, Test
Validation is turned off meaning that no BLS operations are performed
3279.158, 0.000, 3279.158, 3279.158, 1, Initialize DB
0.072, 0.357, 0.035, 13.400, 1400, Load block from database
17.295, 0.000, 17.295, 17.295, 1, Load state from database
5.918, 9.896, 0.198, 98.028, 1350, Apply block
15.888, 10.951, 7.902, 39.535, 50, Apply epoch block
0.000, 0.000, 0.000, 0.000, 0, Database block store
```
* clear full balance cache before processing rewards and penalties
```
All time are ms
Average, StdDev, Min, Max, Samples, Test
Validation is turned off meaning that no BLS operations are performed
3947.901, 0.000, 3947.901, 3947.901, 1, Initialize DB
0.124, 0.506, 0.026, 202.370, 363345, Load block from database
97.614, 0.000, 97.614, 97.614, 1, Load state from database
0.186, 0.188, 0.012, 99.561, 357262, Advance slot, non-epoch
14.161, 5.966, 1.099, 395.511, 11524, Advance slot, epoch
1.372, 4.170, 0.017, 276.401, 363345, Apply block, no slot processing
0.000, 0.000, 0.000, 0.000, 0, Database block store
```
* use `idleAsync` to more evenly divide cpu attention when syncing in
particular - this gives networking better latency
* more strict exception handling in eth2_processor
* allow always-on subscription to all attestation subnets when gossiping
* in subscribe-all-subnets mode, consider all subnets to be stability subnets for ENR purposes
* remove await/async from sub/unsub
* fix unsubscribe wrong key (missed _snappy)
* use the right libp2p commit hash
* remove unused async
* fix inspector
* fix subnet calculation in RPC and insert broadcast attestations into node's pool
* unify codepaths to ensure only mostly-checked-to-be-valid attestations enter the pool, even from node's own broadcasts
* update attestation pool tests for new validateAttestation param
Co-authored-by: Dustin Brody <tersec@users.noreply.github.com>
* fix subnet calculation in RPC and insert broadcast attestations into node's pool
* unify codepaths to ensure only mostly-checked-to-be-valid attestations enter the pool, even from node's own broadcasts
* update attestation pool tests for new validateAttestation param