diff --git a/.gitmodules b/.gitmodules index a42ebf3cc..f7244d871 100644 --- a/.gitmodules +++ b/.gitmodules @@ -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 diff --git a/beacon_chain/gossip_processing/eth2_processor.nim b/beacon_chain/gossip_processing/eth2_processor.nim index f055096d1..d1aae1933 100644 --- a/beacon_chain/gossip_processing/eth2_processor.nim +++ b/beacon_chain/gossip_processing/eth2_processor.nim @@ -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 \ No newline at end of file diff --git a/beacon_chain/gossip_processing/gossip_validation.nim b/beacon_chain/gossip_processing/gossip_validation.nim index 3f3a15c4c..7855f166e 100644 --- a/beacon_chain/gossip_processing/gossip_validation.nim +++ b/beacon_chain/gossip_processing/gossip_validation.nim @@ -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*( diff --git a/beacon_chain/networking/eth2_network.nim b/beacon_chain/networking/eth2_network.nim index b3334ebf4..4779ca1be 100644 --- a/beacon_chain/networking/eth2_network.nim +++ b/beacon_chain/networking/eth2_network.nim @@ -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 diff --git a/beacon_chain/nimbus_beacon_node.nim b/beacon_chain/nimbus_beacon_node.nim index 6a7385105..267ad2066 100644 --- a/beacon_chain/nimbus_beacon_node.nim +++ b/beacon_chain/nimbus_beacon_node.nim @@ -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) = diff --git a/beacon_chain/spec/crypto.nim b/beacon_chain/spec/crypto.nim index 6dfb98f93..09798978e 100644 --- a/beacon_chain/spec/crypto.nim +++ b/beacon_chain/spec/crypto.nim @@ -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) \ No newline at end of file diff --git a/beacon_chain/spec/datatypes/deneb.nim b/beacon_chain/spec/datatypes/deneb.nim index 248ae5b99..6b66d9aa5 100644 --- a/beacon_chain/spec/datatypes/deneb.nim +++ b/beacon_chain/spec/datatypes/deneb.nim @@ -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 diff --git a/beacon_chain/spec/forks.nim b/beacon_chain/spec/forks.nim index 63ce8df4b..1c1558666 100644 --- a/beacon_chain/spec/forks.nim +++ b/beacon_chain/spec/forks.nim @@ -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 diff --git a/beacon_chain/spec/network.nim b/beacon_chain/spec/network.nim index 221253d6e..6867d07da 100644 --- a/beacon_chain/spec/network.nim +++ b/beacon_chain/spec/network.nim @@ -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..