Compare commits

...

4 Commits

Author SHA1 Message Date
terence tsao
0d4f8a2af4 Implement KZG proof batch verification for data column gossip validation 2025-08-20 10:44:07 -07:00
terence tsao
3d4f3cd973 Create gloas.proto 2025-08-18 06:58:42 -07:00
terence
6528fb9cea Update consensus spec to v1.6.0-alpha.4 and implement data column support (#15590)
* Update consensus spec to v1.6.0-alpha.4 and implement data column support for forkchoice spectests

* Apply suggestion from @prestonvanloon

Co-Authored-By: Preston Van Loon <pvanloon@offchainlabs.com>

---------

Co-authored-by: Preston Van Loon <pvanloon@offchainlabs.com>
2025-08-16 15:49:12 +00:00
terence
5021131811 Fix NewSignedBeaconBlock calls to use Block field for equivocation handling (#15595) 2025-08-16 14:19:11 +00:00
22 changed files with 355 additions and 13 deletions

View File

@@ -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,
)

View File

@@ -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)
}

View File

@@ -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",

View File

@@ -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

View 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")
}
}

View File

@@ -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
}
}

View File

@@ -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
}
}

View File

@@ -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()

View File

@@ -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.

View File

@@ -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)

View File

@@ -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")

View File

@@ -0,0 +1,3 @@
### Fixed
- Fixed NewSignedBeaconBlock calls to use Block field for proper equivocation handling

View File

@@ -0,0 +1,3 @@
### Added
- KZG proof batch verification for data column gossip validation

View File

@@ -0,0 +1,3 @@
### Changed
- Update consensus spec to v1.6.0-alpha.4 and implement data column support for forkchoice spectests

View File

@@ -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,
}
)

View File

@@ -148,6 +148,7 @@ var appFlags = []cli.Flag{
bflags.BackfillWorkerCount,
bflags.BackfillOldestSlot,
flags.BatchVerifierLimit,
flags.KzgBatchVerifierLimit,
}
func init() {

View File

@@ -73,6 +73,7 @@ var appHelpFlagGroups = []flagGroup{
flags.RPCHost,
flags.RPCPort,
flags.BatchVerifierLimit,
flags.KzgBatchVerifierLimit,
},
},
{

View File

@@ -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",

View 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" ];
}

View File

@@ -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",

View File

@@ -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 = &ethpb.DataColumnSidecar{}
if err := pb.UnmarshalSSZ(dataColumnSSZ); err != nil {
pb = &ethpb.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 = &ethpb.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) {

View File

@@ -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 {