mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-10 05:47:59 -05:00
Compare commits
4 Commits
poc-onbloc
...
batch-veri
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
0d4f8a2af4 | ||
|
|
3d4f3cd973 | ||
|
|
6528fb9cea | ||
|
|
5021131811 |
10
WORKSPACE
10
WORKSPACE
@@ -253,16 +253,16 @@ filegroup(
|
||||
url = "https://github.com/ethereum/EIPs/archive/5480440fe51742ed23342b68cf106cefd427e39d.tar.gz",
|
||||
)
|
||||
|
||||
consensus_spec_version = "v1.6.0-alpha.1"
|
||||
consensus_spec_version = "v1.6.0-alpha.4"
|
||||
|
||||
load("@prysm//tools:download_spectests.bzl", "consensus_spec_tests")
|
||||
|
||||
consensus_spec_tests(
|
||||
name = "consensus_spec_tests",
|
||||
flavors = {
|
||||
"general": "sha256-o4t9p3R+fQHF4KOykGmwlG3zDw5wUdVWprkzId8aIsk=",
|
||||
"minimal": "sha256-sU7ToI8t3MR8x0vVjC8ERmAHZDWpEmnAC9FWIpHi5x4=",
|
||||
"mainnet": "sha256-YKS4wngg0LgI9Upp4MYJ77aG+8+e/G4YeqEIlp06LZw=",
|
||||
"general": "sha256-MaN4zu3o0vWZypUHS5r4D8WzJF4wANoadM8qm6iyDs4=",
|
||||
"minimal": "sha256-aZGNPp/bBvJgq3Wf6vyR0H6G3DOkbSuggEmOL4jEmtg=",
|
||||
"mainnet": "sha256-C7jjosvpzUgw3GPajlsWBV02ZbkZ5Uv4ikmOqfDGajI=",
|
||||
},
|
||||
version = consensus_spec_version,
|
||||
)
|
||||
@@ -278,7 +278,7 @@ filegroup(
|
||||
visibility = ["//visibility:public"],
|
||||
)
|
||||
""",
|
||||
integrity = "sha256-Nv4TEuEJPQIM4E6T9J0FOITsmappmXZjGtlhe1HEXnU=",
|
||||
integrity = "sha256-qreawRS77l8CebiNww8z727qUItw7KlHY1Xqj7IrPdk=",
|
||||
strip_prefix = "consensus-specs-" + consensus_spec_version[1:],
|
||||
url = "https://github.com/ethereum/consensus-specs/archive/refs/tags/%s.tar.gz" % consensus_spec_version,
|
||||
)
|
||||
|
||||
@@ -822,6 +822,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
|
||||
regularsync.WithSlasherEnabled(b.slasherEnabled),
|
||||
regularsync.WithLightClientStore(b.lcStore),
|
||||
regularsync.WithBatchVerifierLimit(b.cliCtx.Int(flags.BatchVerifierLimit.Name)),
|
||||
regularsync.WithKzgBatchVerifierLimit(b.cliCtx.Int(flags.KzgBatchVerifierLimit.Name)),
|
||||
)
|
||||
return b.services.RegisterService(rs)
|
||||
}
|
||||
|
||||
@@ -72,6 +72,7 @@ go_library(
|
||||
go_test(
|
||||
name = "go_default_test",
|
||||
srcs = [
|
||||
"handlers_equivocation_test.go",
|
||||
"handlers_pool_test.go",
|
||||
"handlers_state_test.go",
|
||||
"handlers_test.go",
|
||||
|
||||
@@ -701,7 +701,7 @@ func (s *Server) publishBlockSSZ(ctx context.Context, w http.ResponseWriter, r *
|
||||
// Validate and optionally broadcast sidecars on equivocation.
|
||||
if err := s.validateBroadcast(ctx, r, genericBlock); err != nil {
|
||||
if errors.Is(err, errEquivocatedBlock) {
|
||||
b, err := blocks.NewSignedBeaconBlock(genericBlock)
|
||||
b, err := blocks.NewSignedBeaconBlock(genericBlock.Block)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, err.Error(), http.StatusBadRequest)
|
||||
return
|
||||
@@ -855,7 +855,7 @@ func (s *Server) publishBlock(ctx context.Context, w http.ResponseWriter, r *htt
|
||||
// Validate and optionally broadcast sidecars on equivocation.
|
||||
if err := s.validateBroadcast(ctx, r, genericBlock); err != nil {
|
||||
if errors.Is(err, errEquivocatedBlock) {
|
||||
b, err := blocks.NewSignedBeaconBlock(genericBlock)
|
||||
b, err := blocks.NewSignedBeaconBlock(genericBlock.Block)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, err.Error(), http.StatusBadRequest)
|
||||
return
|
||||
|
||||
35
beacon-chain/rpc/eth/beacon/handlers_equivocation_test.go
Normal file
35
beacon-chain/rpc/eth/beacon/handlers_equivocation_test.go
Normal file
@@ -0,0 +1,35 @@
|
||||
package beacon
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"testing"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/api/server/structs"
|
||||
rpctesting "github.com/OffchainLabs/prysm/v6/beacon-chain/rpc/eth/shared/testing"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
|
||||
"github.com/OffchainLabs/prysm/v6/testing/require"
|
||||
)
|
||||
|
||||
// TestBlocks_NewSignedBeaconBlock_EquivocationFix tests that blocks.NewSignedBeaconBlock
|
||||
// correctly handles the fixed case where genericBlock.Block is passed instead of genericBlock
|
||||
func TestBlocks_NewSignedBeaconBlock_EquivocationFix(t *testing.T) {
|
||||
// Parse the Phase0 JSON block
|
||||
var block structs.SignedBeaconBlock
|
||||
err := json.Unmarshal([]byte(rpctesting.Phase0Block), &block)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Convert to generic format
|
||||
genericBlock, err := block.ToGeneric()
|
||||
require.NoError(t, err)
|
||||
|
||||
// Test the FIX: pass genericBlock.Block instead of genericBlock
|
||||
// This is what our fix changed in handlers.go line 704 and 858
|
||||
_, err = blocks.NewSignedBeaconBlock(genericBlock.Block)
|
||||
require.NoError(t, err, "NewSignedBeaconBlock should work with genericBlock.Block")
|
||||
|
||||
// Test the BROKEN version: pass genericBlock directly (this should fail)
|
||||
_, err = blocks.NewSignedBeaconBlock(genericBlock)
|
||||
if err == nil {
|
||||
t.Errorf("NewSignedBeaconBlock should fail with whole genericBlock but succeeded")
|
||||
}
|
||||
}
|
||||
@@ -4,6 +4,8 @@ import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
|
||||
"github.com/OffchainLabs/prysm/v6/crypto/bls"
|
||||
"github.com/OffchainLabs/prysm/v6/monitoring/tracing"
|
||||
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
|
||||
@@ -11,13 +13,21 @@ import (
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const signatureVerificationInterval = 50 * time.Millisecond
|
||||
const (
|
||||
signatureVerificationInterval = 50 * time.Millisecond
|
||||
kzgVerificationInterval = 250 * time.Millisecond
|
||||
)
|
||||
|
||||
type signatureVerifier struct {
|
||||
set *bls.SignatureBatch
|
||||
resChan chan error
|
||||
}
|
||||
|
||||
type kzgVerifier struct {
|
||||
dataColumns []blocks.RODataColumn
|
||||
resChan chan error
|
||||
}
|
||||
|
||||
// A routine that runs in the background to perform batch
|
||||
// verifications of incoming messages from gossip.
|
||||
func (s *Service) verifierRoutine() {
|
||||
@@ -47,6 +57,34 @@ func (s *Service) verifierRoutine() {
|
||||
}
|
||||
}
|
||||
|
||||
// A routine that runs in the background to perform batch
|
||||
// KZG verifications of data column sidecars from gossip.
|
||||
func (s *Service) kzgVerifierRoutine() {
|
||||
kzgBatch := make([]*kzgVerifier, 0)
|
||||
ticker := time.NewTicker(kzgVerificationInterval)
|
||||
for {
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
ticker.Stop()
|
||||
for i := 0; i < len(kzgBatch); i++ {
|
||||
kzgBatch[i].resChan <- s.ctx.Err()
|
||||
}
|
||||
return
|
||||
case kzg := <-s.kzgChan:
|
||||
kzgBatch = append(kzgBatch, kzg)
|
||||
if len(kzgBatch) >= s.cfg.kzgBatchVerifierLimit {
|
||||
verifyKzgBatch(kzgBatch)
|
||||
kzgBatch = []*kzgVerifier{}
|
||||
}
|
||||
case <-ticker.C:
|
||||
if len(kzgBatch) > 0 {
|
||||
verifyKzgBatch(kzgBatch)
|
||||
kzgBatch = []*kzgVerifier{}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Service) validateWithBatchVerifier(ctx context.Context, message string, set *bls.SignatureBatch) (pubsub.ValidationResult, error) {
|
||||
_, span := trace.StartSpan(ctx, "sync.validateWithBatchVerifier")
|
||||
defer span.End()
|
||||
@@ -120,3 +158,47 @@ func performBatchAggregation(aggSet *bls.SignatureBatch) (*bls.SignatureBatch, e
|
||||
}
|
||||
return aggSet, nil
|
||||
}
|
||||
|
||||
func (s *Service) validateWithKzgBatchVerifier(ctx context.Context, message string, dataColumns []blocks.RODataColumn) (pubsub.ValidationResult, error) {
|
||||
_, span := trace.StartSpan(ctx, "sync.validateWithKzgBatchVerifier")
|
||||
defer span.End()
|
||||
|
||||
resChan := make(chan error)
|
||||
verificationSet := &kzgVerifier{dataColumns: dataColumns, resChan: resChan}
|
||||
s.kzgChan <- verificationSet
|
||||
|
||||
resErr := <-resChan
|
||||
close(resChan)
|
||||
if resErr != nil {
|
||||
log.WithError(resErr).Tracef("Could not perform batch verification of %s", message)
|
||||
err := peerdas.VerifyDataColumnsSidecarKZGProofs(dataColumns)
|
||||
if err != nil {
|
||||
verErr := errors.Wrapf(err, "Could not verify %s", message)
|
||||
tracing.AnnotateError(span, verErr)
|
||||
return pubsub.ValidationReject, verErr
|
||||
}
|
||||
}
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
func verifyKzgBatch(kzgBatch []*kzgVerifier) {
|
||||
if len(kzgBatch) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
allDataColumns := make([]blocks.RODataColumn, 0)
|
||||
for _, kzgVerifier := range kzgBatch {
|
||||
allDataColumns = append(allDataColumns, kzgVerifier.dataColumns...)
|
||||
}
|
||||
|
||||
var verificationErr error
|
||||
err := peerdas.VerifyDataColumnsSidecarKZGProofs(allDataColumns)
|
||||
if err != nil {
|
||||
verificationErr = errors.Wrap(err, "batch KZG verification failed")
|
||||
}
|
||||
|
||||
// Send the same result to all verifiers in the batch
|
||||
for i := 0; i < len(kzgBatch); i++ {
|
||||
kzgBatch[i].resChan <- verificationErr
|
||||
}
|
||||
}
|
||||
|
||||
@@ -229,3 +229,11 @@ func WithBatchVerifierLimit(limit int) Option {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// WithKzgBatchVerifierLimit sets the maximum number of KZG proofs to batch verify at once.
|
||||
func WithKzgBatchVerifierLimit(limit int) Option {
|
||||
return func(s *Service) error {
|
||||
s.cfg.kzgBatchVerifierLimit = limit
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
@@ -106,6 +106,7 @@ type config struct {
|
||||
blobStorage *filesystem.BlobStorage
|
||||
dataColumnStorage *filesystem.DataColumnStorage
|
||||
batchVerifierLimit int
|
||||
kzgBatchVerifierLimit int
|
||||
}
|
||||
|
||||
// This defines the interface for interacting with block chain service
|
||||
@@ -164,6 +165,7 @@ type Service struct {
|
||||
syncContributionBitsOverlapLock sync.RWMutex
|
||||
syncContributionBitsOverlapCache *lru.Cache
|
||||
signatureChan chan *signatureVerifier
|
||||
kzgChan chan *kzgVerifier
|
||||
clockWaiter startup.ClockWaiter
|
||||
initialSyncComplete chan struct{}
|
||||
verifierWaiter *verification.InitializerWaiter
|
||||
@@ -202,6 +204,8 @@ func NewService(ctx context.Context, opts ...Option) *Service {
|
||||
}
|
||||
// Initialize signature channel with configured limit
|
||||
r.signatureChan = make(chan *signatureVerifier, r.cfg.batchVerifierLimit)
|
||||
// Initialize KZG channel with configured limit
|
||||
r.kzgChan = make(chan *kzgVerifier, r.cfg.kzgBatchVerifierLimit)
|
||||
// Correctly remove it from our seen pending block map.
|
||||
// The eviction method always assumes that the mutex is held.
|
||||
r.slotToPendingBlocks.OnEvicted(func(s string, i interface{}) {
|
||||
@@ -254,6 +258,7 @@ func (s *Service) Start() {
|
||||
s.newColumnsVerifier = newDataColumnsVerifierFromInitializer(v)
|
||||
|
||||
go s.verifierRoutine()
|
||||
go s.kzgVerifierRoutine()
|
||||
go s.startTasksPostInitialSync()
|
||||
go s.processDataColumnLogs()
|
||||
|
||||
|
||||
@@ -145,9 +145,12 @@ func (s *Service) validateDataColumn(ctx context.Context, pid peer.ID, msg *pubs
|
||||
}
|
||||
|
||||
// [REJECT] The sidecar's column data is valid as verified by `verify_data_column_sidecar_kzg_proofs(sidecar)`.
|
||||
if err := verifier.SidecarKzgProofVerified(); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
validationResult, err := s.validateWithKzgBatchVerifier(ctx, "data column KZG proof", roDataColumns)
|
||||
if validationResult != pubsub.ValidationAccept {
|
||||
return validationResult, err
|
||||
}
|
||||
// Mark KZG verification as satisfied since we did it via batch verifier
|
||||
verifier.SatisfyRequirement(verification.RequireSidecarKzgProofVerified)
|
||||
|
||||
// [IGNORE] The sidecar is the first sidecar for the tuple `(block_header.slot, block_header.proposer_index, sidecar.index)`
|
||||
// with valid header signature, sidecar inclusion proof, and kzg proof.
|
||||
|
||||
@@ -7,6 +7,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/kzg"
|
||||
mock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
|
||||
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
|
||||
@@ -25,6 +26,9 @@ import (
|
||||
)
|
||||
|
||||
func TestValidateDataColumn(t *testing.T) {
|
||||
err := kzg.Start()
|
||||
require.NoError(t, err)
|
||||
|
||||
ctx := t.Context()
|
||||
|
||||
t.Run("from self", func(t *testing.T) {
|
||||
@@ -63,10 +67,14 @@ func TestValidateDataColumn(t *testing.T) {
|
||||
|
||||
clock := startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)
|
||||
service := &Service{
|
||||
cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: clock, chain: chainService},
|
||||
cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: clock, chain: chainService, batchVerifierLimit: 10, kzgBatchVerifierLimit: 10},
|
||||
ctx: ctx,
|
||||
newColumnsVerifier: newDataColumnsVerifier,
|
||||
seenDataColumnCache: newSlotAwareCache(seenDataColumnSize),
|
||||
kzgChan: make(chan *kzgVerifier, 10),
|
||||
}
|
||||
// Start the KZG verifier routine for batch verification
|
||||
go service.kzgVerifierRoutine()
|
||||
|
||||
// Encode a `beaconBlock` message instead of expected.
|
||||
buf := new(bytes.Buffer)
|
||||
|
||||
@@ -47,6 +47,10 @@ var (
|
||||
RequireSidecarKzgProofVerified,
|
||||
}
|
||||
|
||||
// SpectestDataColumnSidecarRequirements is used by the forkchoice spectests when verifying data columns used in the on_block tests.
|
||||
SpectestDataColumnSidecarRequirements = requirementList(GossipDataColumnSidecarRequirements).excluding(
|
||||
RequireSidecarParentSeen, RequireSidecarParentValid)
|
||||
|
||||
errColumnsInvalid = errors.New("data columns failed verification")
|
||||
errBadTopicLength = errors.New("topic length is invalid")
|
||||
errBadTopic = errors.New("topic is not of the one expected")
|
||||
|
||||
3
changelog/ttsao_fix-equivocation-block-field.md
Normal file
3
changelog/ttsao_fix-equivocation-block-field.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Fixed
|
||||
|
||||
- Fixed NewSignedBeaconBlock calls to use Block field for proper equivocation handling
|
||||
3
changelog/ttsao_implement-kzg-batch-verification.md
Normal file
3
changelog/ttsao_implement-kzg-batch-verification.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Added
|
||||
|
||||
- KZG proof batch verification for data column gossip validation
|
||||
3
changelog/ttsao_update-consensus-spec-v160-alpha4.md
Normal file
3
changelog/ttsao_update-consensus-spec-v160-alpha4.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Changed
|
||||
|
||||
- Update consensus spec to v1.6.0-alpha.4 and implement data column support for forkchoice spectests
|
||||
@@ -344,4 +344,10 @@ var (
|
||||
Usage: "Maximum number of signatures to batch verify at once for beacon attestation p2p gossip.",
|
||||
Value: 1000,
|
||||
}
|
||||
// KzgBatchVerifierLimit sets the maximum number of KZG proofs to batch verify at once.
|
||||
KzgBatchVerifierLimit = &cli.IntFlag{
|
||||
Name: "kzg-batch-verifier-limit",
|
||||
Usage: "Maximum number of KZG proofs to batch verify at once for data column p2p gossip.",
|
||||
Value: 25,
|
||||
}
|
||||
)
|
||||
|
||||
@@ -148,6 +148,7 @@ var appFlags = []cli.Flag{
|
||||
bflags.BackfillWorkerCount,
|
||||
bflags.BackfillOldestSlot,
|
||||
flags.BatchVerifierLimit,
|
||||
flags.KzgBatchVerifierLimit,
|
||||
}
|
||||
|
||||
func init() {
|
||||
|
||||
@@ -73,6 +73,7 @@ var appHelpFlagGroups = []flagGroup{
|
||||
flags.RPCHost,
|
||||
flags.RPCPort,
|
||||
flags.BatchVerifierLimit,
|
||||
flags.KzgBatchVerifierLimit,
|
||||
},
|
||||
},
|
||||
{
|
||||
|
||||
@@ -37,11 +37,12 @@ var placeholderFields = []string{
|
||||
"EIP7805_FORK_EPOCH",
|
||||
"EIP7805_FORK_VERSION",
|
||||
"EPOCHS_PER_SHUFFLING_PHASE",
|
||||
"INCLUSION_LIST_SUBMISSION_DEADLINE",
|
||||
"MAX_BYTES_PER_INCLUSION_LIST",
|
||||
"MAX_REQUEST_BLOB_SIDECARS_FULU",
|
||||
"MAX_REQUEST_INCLUSION_LIST",
|
||||
"MAX_REQUEST_PAYLOADS", // Compile time constant on BeaconBlockBody.ExecutionRequests
|
||||
"PROPOSER_INCLUSION_LIST_CUT_OFF",
|
||||
"PROPOSER_INCLUSION_LIST_CUTOFF",
|
||||
"PROPOSER_SCORE_BOOST_EIP7732",
|
||||
"PROPOSER_SELECTION_GAP",
|
||||
"TARGET_NUMBER_OF_PEERS",
|
||||
|
||||
26
proto/prysm/v1alpha1/gloas.proto
Normal file
26
proto/prysm/v1alpha1/gloas.proto
Normal file
@@ -0,0 +1,26 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package ethereum.eth.v1alpha1;
|
||||
|
||||
import "proto/eth/ext/options.proto";
|
||||
|
||||
message ExecutionPayloadHeaderGloas {
|
||||
bytes parent_block_hash = 1 [ (ethereum.eth.ext.ssz_size) = "32" ];
|
||||
bytes parent_block_root = 2 [ (ethereum.eth.ext.ssz_size) = "32" ];
|
||||
bytes block_hash = 3 [ (ethereum.eth.ext.ssz_size) = "32" ];
|
||||
uint64 gas_limit = 4;
|
||||
uint64 builder_index = 5 [ (ethereum.eth.ext.cast_type) =
|
||||
"github.com/OffchainLabs/prysm/v6/"
|
||||
"consensus-types/primitives.ValidatorIndex" ];
|
||||
uint64 slot = 6 [
|
||||
(ethereum.eth.ext.cast_type) =
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Slot"
|
||||
];
|
||||
uint64 value = 7;
|
||||
bytes blob_kzg_commitments_root = 8 [ (ethereum.eth.ext.ssz_size) = "32" ];
|
||||
}
|
||||
|
||||
message SignedExecutionPayloadHeader {
|
||||
ExecutionPayloadHeaderGloas message = 1;
|
||||
bytes signature = 2 [ (ethereum.eth.ext.ssz_size) = "96" ];
|
||||
}
|
||||
@@ -17,6 +17,7 @@ go_library(
|
||||
"//beacon-chain/blockchain/testing:go_default_library",
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/core/helpers:go_default_library",
|
||||
"//beacon-chain/core/time:go_default_library",
|
||||
"//beacon-chain/core/transition:go_default_library",
|
||||
"//beacon-chain/db/filesystem:go_default_library",
|
||||
|
||||
@@ -10,6 +10,7 @@ import (
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
|
||||
state_native "github.com/OffchainLabs/prysm/v6/beacon-chain/state/state-native"
|
||||
@@ -62,6 +63,7 @@ func runTest(t *testing.T, config string, fork int, basePath string) { // nolint
|
||||
|
||||
for _, folder := range testFolders {
|
||||
t.Run(folder.Name(), func(t *testing.T) {
|
||||
helpers.ClearCache()
|
||||
preStepsFile, err := util.BazelFileBytes(testsFolderPath, folder.Name(), "steps.yaml")
|
||||
require.NoError(t, err)
|
||||
var steps []Step
|
||||
@@ -148,6 +150,9 @@ func runTest(t *testing.T, config string, fork int, basePath string) { // nolint
|
||||
}
|
||||
}
|
||||
runBlobStep(t, step, beaconBlock, fork, folder, testsFolderPath, builder)
|
||||
if len(step.DataColumns) > 0 {
|
||||
runDataColumnStep(t, step, beaconBlock, fork, folder, testsFolderPath, builder)
|
||||
}
|
||||
if beaconBlock != nil {
|
||||
if step.Valid != nil && !*step.Valid {
|
||||
builder.InvalidBlock(t, beaconBlock)
|
||||
@@ -293,10 +298,154 @@ func runBlobStep(t *testing.T,
|
||||
}
|
||||
}
|
||||
|
||||
func runDataColumnStep(t *testing.T,
|
||||
step Step,
|
||||
beaconBlock interfaces.ReadOnlySignedBeaconBlock,
|
||||
fork int,
|
||||
folder os.DirEntry,
|
||||
testsFolderPath string,
|
||||
builder *Builder,
|
||||
) {
|
||||
columnFiles := step.DataColumns
|
||||
|
||||
require.NotNil(t, beaconBlock)
|
||||
require.Equal(t, true, fork >= version.Fulu)
|
||||
|
||||
block := beaconBlock.Block()
|
||||
root, err := block.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
kzgs, err := block.Body().BlobKzgCommitments()
|
||||
require.NoError(t, err)
|
||||
sh, err := beaconBlock.Header()
|
||||
require.NoError(t, err)
|
||||
// Use the same error that the verification system returns for data columns
|
||||
errDataColumnsInvalid := errors.New("data columns failed verification")
|
||||
requireVerifyExpected := errAssertionForStep(step, errDataColumnsInvalid)
|
||||
|
||||
var allColumns []blocks.RODataColumn
|
||||
|
||||
for columnIndex, columnFile := range columnFiles {
|
||||
if columnFile == nil || *columnFile == "null" {
|
||||
continue
|
||||
}
|
||||
|
||||
dataColumnFile, err := util.BazelFileBytes(testsFolderPath, folder.Name(), fmt.Sprint(*columnFile, ".ssz_snappy"))
|
||||
require.NoError(t, err)
|
||||
dataColumnSSZ, err := snappy.Decode(nil /* dst */, dataColumnFile)
|
||||
require.NoError(t, err)
|
||||
|
||||
var pb *ethpb.DataColumnSidecar
|
||||
|
||||
if step.Valid != nil && !*step.Valid {
|
||||
pb = ðpb.DataColumnSidecar{}
|
||||
if err := pb.UnmarshalSSZ(dataColumnSSZ); err != nil {
|
||||
pb = ðpb.DataColumnSidecar{
|
||||
Index: uint64(columnIndex),
|
||||
Column: [][]byte{},
|
||||
KzgCommitments: kzgs,
|
||||
KzgProofs: make([][]byte, 0),
|
||||
SignedBlockHeader: sh,
|
||||
}
|
||||
}
|
||||
} else {
|
||||
numCells := len(kzgs)
|
||||
column := make([][]byte, numCells)
|
||||
for cellIndex := 0; cellIndex < numCells; cellIndex++ {
|
||||
cell := make([]byte, 2048)
|
||||
cellStart := cellIndex * 2048
|
||||
cellEnd := cellStart + 2048
|
||||
if cellEnd <= len(dataColumnSSZ) {
|
||||
copy(cell, dataColumnSSZ[cellStart:cellEnd])
|
||||
}
|
||||
column[cellIndex] = cell
|
||||
}
|
||||
|
||||
inclusionProof, err := blocks.MerkleProofKZGCommitments(block.Body())
|
||||
require.NoError(t, err)
|
||||
|
||||
pb = ðpb.DataColumnSidecar{
|
||||
Index: uint64(columnIndex),
|
||||
Column: column,
|
||||
KzgCommitments: kzgs,
|
||||
SignedBlockHeader: sh,
|
||||
KzgCommitmentsInclusionProof: inclusionProof,
|
||||
}
|
||||
}
|
||||
|
||||
ro, err := blocks.NewRODataColumnWithRoot(pb, root)
|
||||
require.NoError(t, err)
|
||||
allColumns = append(allColumns, ro)
|
||||
}
|
||||
|
||||
if len(allColumns) > 0 {
|
||||
ini, err := builder.vwait.WaitForInitializer(context.Background())
|
||||
require.NoError(t, err)
|
||||
// Use different verification requirements based on whether this is a valid or invalid test case
|
||||
var forkchoiceReqs []verification.Requirement
|
||||
if step.Valid != nil && !*step.Valid {
|
||||
forkchoiceReqs = verification.SpectestDataColumnSidecarRequirements
|
||||
} else {
|
||||
forkchoiceReqs = []verification.Requirement{
|
||||
verification.RequireNotFromFutureSlot,
|
||||
verification.RequireSlotAboveFinalized,
|
||||
verification.RequireValidProposerSignature,
|
||||
verification.RequireSidecarParentSlotLower,
|
||||
verification.RequireSidecarDescendsFromFinalized,
|
||||
verification.RequireSidecarInclusionProven,
|
||||
verification.RequireSidecarProposerExpected,
|
||||
}
|
||||
}
|
||||
dv := ini.NewDataColumnsVerifier(allColumns, forkchoiceReqs)
|
||||
ctx := t.Context()
|
||||
|
||||
if step.Valid != nil && !*step.Valid {
|
||||
if err := dv.ValidFields(); err != nil {
|
||||
t.Logf("ValidFields error: %s", err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
if err := dv.NotFromFutureSlot(); err != nil {
|
||||
t.Logf("NotFromFutureSlot error: %s", err.Error())
|
||||
}
|
||||
if err := dv.SlotAboveFinalized(); err != nil {
|
||||
t.Logf("SlotAboveFinalized error: %s", err.Error())
|
||||
}
|
||||
if err := dv.SidecarInclusionProven(); err != nil {
|
||||
t.Logf("SidecarInclusionProven error: %s", err.Error())
|
||||
}
|
||||
if err := dv.ValidProposerSignature(ctx); err != nil {
|
||||
t.Logf("ValidProposerSignature error: %s", err.Error())
|
||||
}
|
||||
if err := dv.SidecarParentSlotLower(); err != nil {
|
||||
t.Logf("SidecarParentSlotLower error: %s", err.Error())
|
||||
}
|
||||
if err := dv.SidecarDescendsFromFinalized(); err != nil {
|
||||
t.Logf("SidecarDescendsFromFinalized error: %s", err.Error())
|
||||
}
|
||||
if err := dv.SidecarProposerExpected(ctx); err != nil {
|
||||
t.Logf("SidecarProposerExpected error: %s", err.Error())
|
||||
}
|
||||
|
||||
vdc, err := dv.VerifiedRODataColumns()
|
||||
requireVerifyExpected(t, err)
|
||||
|
||||
if err == nil {
|
||||
for _, column := range vdc {
|
||||
require.NoError(t, builder.service.ReceiveDataColumn(column))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func errAssertionForStep(step Step, expect error) func(t *testing.T, err error) {
|
||||
if !*step.Valid {
|
||||
return func(t *testing.T, err error) {
|
||||
require.ErrorIs(t, err, expect)
|
||||
if expect.Error() == "data columns failed verification" {
|
||||
require.NotNil(t, err)
|
||||
require.Equal(t, true, strings.Contains(err.Error(), expect.Error()))
|
||||
} else {
|
||||
require.ErrorIs(t, err, expect)
|
||||
}
|
||||
}
|
||||
}
|
||||
return func(t *testing.T, err error) {
|
||||
|
||||
@@ -11,6 +11,7 @@ type Step struct {
|
||||
PayloadStatus *MockEngineResp `json:"payload_status"`
|
||||
PowBlock *string `json:"pow_block"`
|
||||
Check *Check `json:"checks"`
|
||||
DataColumns []*string `json:"columns"`
|
||||
}
|
||||
|
||||
type Check struct {
|
||||
|
||||
Reference in New Issue
Block a user