fixed upto electra

This commit is contained in:
Agnish Ghosh 2024-05-27 18:10:00 +05:30
parent 905f3b8170
commit aa2ce79770
No known key found for this signature in database
GPG Key ID: 7BDDA05D1B25E9F8
11 changed files with 204 additions and 22 deletions

2
.gitmodules vendored
View File

@ -214,7 +214,7 @@
path = vendor/nim-kzg4844
url = https://github.com/status-im/nim-kzg4844.git
ignore = untracked
branch = master
branch = peerdas
[submodule "vendor/nim-results"]
path = vendor/nim-results
url = https://github.com/arnetheduck/nim-results.git

View File

@ -12,7 +12,7 @@ import
stew/results,
chronicles, chronos, metrics, taskpools,
../spec/[helpers, forks],
../spec/datatypes/[altair, phase0, deneb],
../spec/datatypes/[altair, phase0, deneb, eip7594],
../consensus_object_pools/[
blob_quarantine, block_clearance, block_quarantine, blockchain_dag,
attestation_pool, light_client_pool, sync_committee_msg_pool,
@ -47,6 +47,10 @@ declareCounter blob_sidecars_received,
"Number of valid blobs processed by this node"
declareCounter blob_sidecars_dropped,
"Number of invalid blobs dropped by this node", labels = ["reason"]
declareCounter data_column_sidecars_received,
"Number of valid data column sidecars processed by this node"
declareCounter data_column_sidecars_dropped,
"Number of invalid data column sidecars dropped by this node", labels = ["reason"]
declareCounter beacon_attester_slashings_received,
"Number of valid attester slashings processed by this node"
declareCounter beacon_attester_slashings_dropped,
@ -94,6 +98,9 @@ declareHistogram beacon_block_delay,
declareHistogram blob_sidecar_delay,
"Time(s) between slot start and blob sidecar reception", buckets = delayBuckets
declareHistogram data_column_sidecar_delay,
"Time(s) between slot start and data column sidecar reception", buckets = delayBuckets
type
DoppelgangerProtection = object
broadcastStartEpoch*: Epoch ##\
@ -320,6 +327,41 @@ proc processBlobSidecar*(
v
proc processDataColumnSidecar*(
self: var Eth2Processor, src: MsgSource,
dataColumnSidecar: DataColumnSidecar, subnet_id: uint64): ValidationRes =
template block_header: untyped = dataColumnSidecar.signed_block_header.message
let
wallTime = self.getCurrentBeaconTime()
(_, wallSlot) = wallTime.toSlot()
logScope:
dcs = shortLog(dataColumnSidecar)
wallSlot
# Potential under/overflows are fine; would just create odd metrics and logs
let delay = wallTime - block_header.slot.start_beacon_time
debug "Data column received", delay
let v =
self.dag.validateDataColumnSidecar(self.quarantine, self.blobQuarantine,
dataColumnSidecar, wallTime, subnet_id)
if v.isErr():
debug "Dropping data column", error = v.error()
blob_sidecars_dropped.inc(1, [$v.error[0]])
return v
debug "Data column validated"
# TODO do something with it!
data_column_sidecars_received.inc()
data_column_sidecar_delay.observe(delay.toFloatSeconds())
v
proc setupDoppelgangerDetection*(self: var Eth2Processor, slot: Slot) =
# When another client's already running, this is very likely to detect
# potential duplicate validators, which can trigger slashing.
@ -342,8 +384,7 @@ proc clearDoppelgangerProtection*(self: var Eth2Processor) =
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
proc checkForPotentialDoppelganger(
self: var Eth2Processor,
attestation: phase0.Attestation | electra.Attestation,
self: var Eth2Processor, attestation: phase0.Attestation | electra.Attestation,
attesterIndices: openArray[ValidatorIndex]) =
# Only check for attestations after node launch. There might be one slot of
# overlap in quick intra-slot restarts so trade off a few true negatives in
@ -413,10 +454,8 @@ proc processAttestation*(
proc processSignedAggregateAndProof*(
self: ref Eth2Processor, src: MsgSource,
signedAggregateAndProof:
phase0.SignedAggregateAndProof | electra.SignedAggregateAndProof,
checkSignature = true, checkCover = true): Future[ValidationRes]
{.async: (raises: [CancelledError]).} =
signedAggregateAndProof: phase0.SignedAggregateAndProof | electra.SignedAggregateAndProof,
checkSignature = true, checkCover = true): Future[ValidationRes] {.async: (raises: [CancelledError]).} =
var wallTime = self.getCurrentBeaconTime()
let (afterGenesis, wallSlot) = wallTime.toSlot()
@ -685,4 +724,4 @@ proc processLightClientOptimisticUpdate*(
beacon_light_client_optimistic_update_received.inc()
else:
beacon_light_client_optimistic_update_dropped.inc(1, [$v.error[0]])
v
v

View File

@ -471,6 +471,120 @@ proc validateBlobSidecar*(
ok()
# https://github.com/ethereum/consensus-specs/blob/5f48840f4d768bf0e0a8156a3ed06ec333589007/specs/_features/eip7594/p2p-interface.md#the-gossip-domain-gossipsub
proc validateDataColumnSidecar*(
dag: ChainDAGRef, quarantine: ref Quarantine,
blobQuarantine: ref BlobQuarantine, data_column_sidecar: DataColumnSidecar,
wallTime: BeaconTime, subnet_id: uint64): Result[void, ValidationError] =
template block_header: untyped = data_column_sidecar.signed_block_header.message
# [REJECT] The sidecar's index is consistent with `NUMBER_OF_COLUMNS`
# -- i.e. `blob_sidecar.index < NUMBER_OF_COLUMNS`
if not (data_column_sidecar.index < NUMBER_OF_COLUMNS):
return dag.checkedReject("DataColumnSidecar: The sidecar's index should be consistent with NUMBER_OF_COLUMNS")
# [REJECT] The sidecar is for the correct subnet
# -- i.e. `compute_subnet_for_data_column_sidecar(blob_sidecar.index) == subnet_id`.
if not (compute_subnet_for_data_column_sidecar(data_column_sidecar.index) == subnet_id):
return dag.checkedReject("DataColumnSidecar: The sidecar is not for the correct subnet")
# [IGNORE] The sidecar is not from a future slot (with a `MAXIMUM_GOSSIP_CLOCK_DISPARITY` allowance)
# -- i.e. validate that `block_header.slot <= current_slot` (a client MAY queue future sidecars for
# processing at the appropriate slot).
if not (block_header.slot <=
(wallTime + MAXIMUM_GOSSIP_CLOCK_DISPARITY).slotOrZero):
return errIgnore("DataColumnSidecar: slot too high")
# [IGNORE] The sidecar is from a slot greater than the latest
# finalized slot -- i.e. validate that `block_header.slot >
# compute_start_slot_at_epoch(state.finalized_checkpoint.epoch)`
if not (block_header.slot > dag.finalizedHead.slot):
return errIgnore("DataColumnSidecar: slot already finalized")
# TODO: [REJECT] The sidecar's `kzg_commitments` inclusion proof is valid as verified by
# `verify_data_column_sidecar_inclusion_proof(sidecar)`.
# TODO: [REJECT] The sidecar's column data is valid as
# verified by `verify_data_column_kzg_proofs(sidecar)`
# [IGNORE] The sidecar is the first sidecar for the tuple
# (block_header.slot, block_header.proposer_index, blob_sidecar.index)
# with valid header signature, sidecar inclusion proof, and kzg proof.
let block_root = hash_tree_root(block_header)
if dag.getBlockRef(block_root).isSome():
return errIgnore("BlobSidecar: already have block")
if blobQuarantine[].hasBlob(
block_header.slot, block_header.proposer_index, data_column_sidecar.index):
return errIgnore("BlobSidecar: already have valid blob from same proposer")
# [IGNORE] The sidecar's block's parent (defined by
# `block_header.parent_root`) has been seen (via both gossip and
# non-gossip sources) (a client MAY queue sidecars for processing
# once the parent block is retrieved).
#
# [REJECT] The sidecar's block's parent (defined by
# `block_header.parent_root`) passes validation.
let parent = dag.getBlockRef(block_header.parent_root).valueOr:
if block_header.parent_root in quarantine[].unviable:
quarantine[].addUnviable(block_root)
return dag.checkedReject("DataColumnSidecar: parent not validated")
else:
quarantine[].addMissing(block_header.parent_root)
return errIgnore("DataColumnSidecar: parent not found")
# [REJECT] The sidecar is proposed by the expected `proposer_index`
# for the block's slot in the context of the current shuffling
# (defined by `block_header.parent_root`/`block_header.slot`).
# If the proposer_index cannot immediately be verified against the expected
# shuffling, the sidecar MAY be queued for later processing while proposers
# for the block's branch are calculated -- in such a case do not
# REJECT, instead IGNORE this message.
let proposer = getProposer(dag, parent, block_header.slot).valueOr:
warn "cannot compute proposer for blob"
return errIgnore("BlobSidecar: Cannot compute proposer") # internal issue
if uint64(proposer) != block_header.proposer_index:
return dag.checkedReject("BlobSidecar: Unexpected proposer")
# [REJECT] The proposer signature of `blob_sidecar.signed_block_header`,
# is valid with respect to the `block_header.proposer_index` pubkey.
if not verify_block_signature(
dag.forkAtEpoch(block_header.slot.epoch),
getStateField(dag.headState, genesis_validators_root),
block_header.slot,
block_root,
dag.validatorKey(proposer).get(),
data_column_sidecar.signed_block_header.signature):
return dag.checkedReject("DataColumnSidecar: Invalid proposer signature")
# [REJECT] The sidecar is from a higher slot than the sidecar's
# block's parent (defined by `block_header.parent_root`).
if not (block_header.slot > parent.bid.slot):
return dag.checkedReject("DataColumnSidecar: slot lower than parents'")
# [REJECT] The current finalized_checkpoint is an ancestor of the sidecar's
# block -- i.e. `get_checkpoint_block(store, block_header.parent_root,
# store.finalized_checkpoint.epoch) == store.finalized_checkpoint.root`.
let
finalized_checkpoint = getStateField(dag.headState, finalized_checkpoint)
ancestor = get_ancestor(parent, finalized_checkpoint.epoch.start_slot)
if ancestor.isNil:
# This shouldn't happen: we should always be able to trace the parent back
# to the finalized checkpoint (else it wouldn't be in the DAG)
return errIgnore("DataColumnSidecar: Can't find ancestor")
if not (
finalized_checkpoint.root == ancestor.root or
finalized_checkpoint.root.isZero):
quarantine[].addUnviable(block_root)
return dag.checkedReject(
"DataColumnSidecar: Finalized checkpoint not an ancestor")
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#beacon_block
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/bellatrix/p2p-interface.md#beacon_block
proc validateBeaconBlock*(

View File

@ -831,7 +831,7 @@ template gossipMaxSize(T: untyped): uint32 =
when isFixedSize(T):
fixedPortionSize(T).uint32
elif T is bellatrix.SignedBeaconBlock or T is capella.SignedBeaconBlock or
T is deneb.SignedBeaconBlock or T is electra.SignedBeaconBlock:
T is deneb.SignedBeaconBlock or T is electra.SignedBeaconBlock or T is DataColumnSidecar:
GOSSIP_MAX_SIZE
# TODO https://github.com/status-im/nim-ssz-serialization/issues/20 for
# Attestation, AttesterSlashing, and SignedAggregateAndProof, which all

View File

@ -1107,7 +1107,7 @@ proc addCapellaMessageHandlers(
proc addDenebMessageHandlers(
node: BeaconNode, forkDigest: ForkDigest, slot: Slot) =
node.addCapellaMessageHandlers(forkDigest, slot)
for topic in blobSidecarTopics(forkDigest):
for topic in dataColumnSidecarTopics(forkDigest):
node.network.subscribe(topic, basicParams)
proc addElectraMessageHandlers(
@ -1131,7 +1131,7 @@ proc removeCapellaMessageHandlers(node: BeaconNode, forkDigest: ForkDigest) =
proc removeDenebMessageHandlers(node: BeaconNode, forkDigest: ForkDigest) =
node.removeCapellaMessageHandlers(forkDigest)
for topic in blobSidecarTopics(forkDigest):
for topic in dataColumnSidecarTopics(forkDigest):
node.network.unsubscribe(topic)
proc removeElectraMessageHandlers(node: BeaconNode, forkDigest: ForkDigest) =

View File

@ -46,6 +46,7 @@ export results, blscurve, rand, json_serialization
const
RawSigSize* = 96
RawPubKeySize* = 48
RawPointSize* = 48
UncompressedPubKeySize* = 96
# RawPrivKeySize* = 32 for BLST
@ -94,6 +95,13 @@ type
key*: ValidatorPrivKey
id*: uint32
Point* = object
blob*: array[RawPointSize, byte]
BLSCommitment* = object
export
AggregateSignature
@ -390,7 +398,7 @@ func toRaw*(x: ValidatorPrivKey): array[32, byte] =
# TODO: distinct type - see https://github.com/status-im/nim-blscurve/pull/67
static: doAssert BLS_BACKEND == BLST
result = SecretKey(x).exportRaw()
template toRaw*(x: ValidatorPubKey | ValidatorSig): auto =
x.blob
@ -608,4 +616,4 @@ proc confirmShares*(pubKey: ValidatorPubKey,
let signature = share.key.blsSign(confirmationData).toSignatureShare(share.id);
signs.add(signature)
let recovered = signs.recoverSignature()
return pubKey.blsVerify(confirmationData, recovered)
return pubKey.blsVerify(confirmationData, recovered)

View File

@ -32,7 +32,7 @@ export json_serialization, base, kzg4844
const
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/deneb/polynomial-commitments.md#constants
BYTES_PER_FIELD_ELEMENT = 32
BYTES_PER_FIELD_ELEMENT* = 32
BLS_MODULUS* = "52435875175126190479447740508185965837690552500527637822603658699938581184513".u256
type

View File

@ -16,11 +16,11 @@ import
"."/[
block_id, eth2_merkleization, eth2_ssz_serialization,
forks_light_client, presets],
./datatypes/[phase0, altair, bellatrix, capella, deneb, electra],
./datatypes/[phase0, altair, bellatrix, capella, deneb, electra, eip7594],
./mev/[bellatrix_mev, capella_mev, deneb_mev, electra_mev]
export
extras, block_id, phase0, altair, bellatrix, capella, deneb, electra,
extras, block_id, phase0, altair, bellatrix, capella, deneb, electra, eip7594,
eth2_merkleization, eth2_ssz_serialization, forks_light_client,
presets, deneb_mev, electra_mev

View File

@ -9,7 +9,7 @@
import
"."/[helpers, forks],
"."/datatypes/base
"."/datatypes/[base, constants, eip7594]
export base
@ -155,8 +155,7 @@ func getDiscoveryForkID*(cfg: RuntimeConfig,
type GossipState* = set[ConsensusFork]
func getTargetGossipState*(
epoch, ALTAIR_FORK_EPOCH, BELLATRIX_FORK_EPOCH, CAPELLA_FORK_EPOCH,
DENEB_FORK_EPOCH: Epoch, ELECTRA_FORK_EPOCH: Epoch, isBehind: bool):
GossipState =
DENEB_FORK_EPOCH: Epoch, ELECTRA_FORK_EPOCH: Epoch, isBehind: bool): GossipState =
if isBehind:
return {}
@ -226,3 +225,20 @@ func getSyncSubnets*(
iterator blobSidecarTopics*(forkDigest: ForkDigest): string =
for subnet_id in BlobId:
yield getBlobSidecarTopic(forkDigest, subnet_id)
const
KZG_COMMITMENTS_INCLUSION_PROOF_DEPTH* = 32
MAX_REQUEST_DATA_COLUMN_SIDECARS* = MAX_REQUEST_BLOCKS_DENEB * NUMBER_OF_COLUMNS
MIN_EPOCHS_FOR_DATA_COLUMN_SIDECARS_REQUESTS* = 4096
func getDataColumnSidecarTopic*(forkDigest: ForkDigest,
subnet_id: uint64): string =
eth2Prefix(forkDigest) & "data_column_sidecar_" & $subnet_id & "/ssz_snappy"
func compute_subnet_for_data_column_sidecar*(column_index: ColumnIndex): uint64 =
uint64(column_index mod DATA_COLUMN_SIDECAR_SUBNET_COUNT)
iterator dataColumnSidecarTopics*(forkDigest: ForkDigest): string =
for subnet_id in 0'u64..<DATA_COLUMN_SIDECAR_SUBNET_COUNT:
yield getDataColumnSidecarTopic(forkDigest, subnet_id)

View File

@ -685,6 +685,11 @@ proc readRuntimeConfig*(
values[lineParts[0]] = lineParts[1].strip
values.withValue("DENEB_FORK_EPOCH", v):
values["EIP7594_FORK_EPOCH"] = v[]
values.withValue("DENEB_FORK_VERSION", v):
values["EIP7594_FORK_VERSION"] = v[]
# Certain config keys are baked into the binary at compile-time
# and cannot be overridden via config.
template checkCompatibility(
@ -830,4 +835,4 @@ template name*(cfg: RuntimeConfig): string =
func defaultLightClientDataMaxPeriods*(cfg: RuntimeConfig): uint64 =
const epochsPerPeriod = EPOCHS_PER_SYNC_COMMITTEE_PERIOD
let maxEpochs = cfg.MIN_EPOCHS_FOR_BLOCK_REQUESTS
(maxEpochs + epochsPerPeriod - 1) div epochsPerPeriod
(maxEpochs + epochsPerPeriod - 1) div epochsPerPeriod

2
vendor/nim-kzg4844 vendored

@ -1 +1 @@
Subproject commit f12616d0675d9f6346141ca95f0840ab227eb213
Subproject commit 01216c0b8ebcd0d3466fbfb08a87e96dc1b7c305