diff --git a/beacon-chain/blockchain/execution_engine.go b/beacon-chain/blockchain/execution_engine.go index 564648a69..af2eeeef3 100644 --- a/beacon-chain/blockchain/execution_engine.go +++ b/beacon-chain/blockchain/execution_engine.go @@ -387,10 +387,7 @@ func (s *Service) removeInvalidBlockAndState(ctx context.Context, blkRoots [][32 // This is an irreparable condition, it would me a justified or finalized block has become invalid. return err } - // No op if the sidecar does not exist. - if err := s.cfg.BeaconDB.DeleteBlobSidecars(ctx, root); err != nil { - return err - } + // TODO: Remove blob here } return nil } diff --git a/beacon-chain/blockchain/kzg/BUILD.bazel b/beacon-chain/blockchain/kzg/BUILD.bazel index 4f30dc2dc..21c8fe9d7 100644 --- a/beacon-chain/blockchain/kzg/BUILD.bazel +++ b/beacon-chain/blockchain/kzg/BUILD.bazel @@ -11,7 +11,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//consensus-types/blocks:go_default_library", - "//proto/prysm/v1alpha1:go_default_library", "@com_github_crate_crypto_go_kzg_4844//:go_default_library", "@com_github_pkg_errors//:go_default_library", ], @@ -25,7 +24,7 @@ go_test( ], embed = [":go_default_library"], deps = [ - "//proto/prysm/v1alpha1:go_default_library", + "//consensus-types/blocks:go_default_library", "//testing/require:go_default_library", "@com_github_consensys_gnark_crypto//ecc/bls12-381/fr:go_default_library", "@com_github_crate_crypto_go_kzg_4844//:go_default_library", diff --git a/beacon-chain/blockchain/kzg/validation.go b/beacon-chain/blockchain/kzg/validation.go index dfd09ce3b..1549a3f45 100644 --- a/beacon-chain/blockchain/kzg/validation.go +++ b/beacon-chain/blockchain/kzg/validation.go @@ -1,37 +1,10 @@ package kzg import ( - "fmt" - GoKZG "github.com/crate-crypto/go-kzg-4844" "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks" - ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1" ) -// IsDataAvailable checks that -// - all blobs in the block are available -// - Expected KZG commitments match the number of blobs in the block -// - That the number of proofs match the number of blobs -// - That the proofs are verified against the KZG commitments -func IsDataAvailable(commitments [][]byte, sidecars []*ethpb.DeprecatedBlobSidecar) error { - if len(commitments) != len(sidecars) { - return fmt.Errorf("could not check data availability, expected %d commitments, obtained %d", - len(commitments), len(sidecars)) - } - if len(commitments) == 0 { - return nil - } - blobs := make([]GoKZG.Blob, len(commitments)) - proofs := make([]GoKZG.KZGProof, len(commitments)) - cmts := make([]GoKZG.KZGCommitment, len(commitments)) - for i, sidecar := range sidecars { - blobs[i] = bytesToBlob(sidecar.Blob) - proofs[i] = bytesToKZGProof(sidecar.KzgProof) - cmts[i] = bytesToCommitment(commitments[i]) - } - return kzgContext.VerifyBlobKZGProofBatch(blobs, cmts, proofs) -} - // Verify performs single or batch verification of commitments depending on the number of given BlobSidecars. func Verify(sidecars ...blocks.ROBlob) error { if len(sidecars) == 0 { diff --git a/beacon-chain/blockchain/kzg/validation_test.go b/beacon-chain/blockchain/kzg/validation_test.go index b6f905ef0..032fbadf0 100644 --- a/beacon-chain/blockchain/kzg/validation_test.go +++ b/beacon-chain/blockchain/kzg/validation_test.go @@ -8,7 +8,7 @@ import ( "github.com/consensys/gnark-crypto/ecc/bls12-381/fr" GoKZG "github.com/crate-crypto/go-kzg-4844" - ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1" + "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks" "github.com/prysmaticlabs/prysm/v4/testing/require" "github.com/sirupsen/logrus" ) @@ -58,10 +58,9 @@ func GenerateCommitmentAndProof(blob GoKZG.Blob) (GoKZG.KZGCommitment, GoKZG.KZG return commitment, proof, err } -func TestIsDataAvailable(t *testing.T) { - sidecars := make([]*ethpb.DeprecatedBlobSidecar, 0) - commitments := make([][]byte, 0) - require.NoError(t, IsDataAvailable(commitments, sidecars)) +func TestVerify(t *testing.T) { + sidecars := make([]blocks.ROBlob, 0) + require.NoError(t, Verify(sidecars...)) } func TestBytesToAny(t *testing.T) { diff --git a/beacon-chain/db/iface/interface.go b/beacon-chain/db/iface/interface.go index e80f34d2b..509341004 100644 --- a/beacon-chain/db/iface/interface.go +++ b/beacon-chain/db/iface/interface.go @@ -55,9 +55,6 @@ type ReadOnlyDatabase interface { FeeRecipientByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (common.Address, error) RegistrationByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (*ethpb.ValidatorRegistrationV1, error) - // Blob operations. - BlobSidecarsByRoot(ctx context.Context, beaconBlockRoot [32]byte, indices ...uint64) ([]*ethpb.DeprecatedBlobSidecar, error) - BlobSidecarsBySlot(ctx context.Context, slot primitives.Slot, indices ...uint64) ([]*ethpb.DeprecatedBlobSidecar, error) // origin checkpoint sync support OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error) BackfillBlockRoot(ctx context.Context) ([32]byte, error) @@ -93,9 +90,6 @@ type NoHeadAccessDatabase interface { SaveFeeRecipientsByValidatorIDs(ctx context.Context, ids []primitives.ValidatorIndex, addrs []common.Address) error SaveRegistrationsByValidatorIDs(ctx context.Context, ids []primitives.ValidatorIndex, regs []*ethpb.ValidatorRegistrationV1) error - // Blob operations. - DeleteBlobSidecars(ctx context.Context, beaconBlockRoot [32]byte) error - CleanUpDirtyStates(ctx context.Context, slotsPerArchivedPoint primitives.Slot) error } diff --git a/beacon-chain/db/kv/BUILD.bazel b/beacon-chain/db/kv/BUILD.bazel index 48c9d50b8..896ad73c0 100644 --- a/beacon-chain/db/kv/BUILD.bazel +++ b/beacon-chain/db/kv/BUILD.bazel @@ -5,7 +5,6 @@ go_library( srcs = [ "archived_point.go", "backup.go", - "blob.go", "blocks.go", "checkpoint.go", "deposit_contract.go", @@ -39,7 +38,6 @@ go_library( "//beacon-chain/state/genesis:go_default_library", "//beacon-chain/state/state-native:go_default_library", "//config/features:go_default_library", - "//config/fieldparams:go_default_library", "//config/params:go_default_library", "//consensus-types/blocks:go_default_library", "//consensus-types/interfaces:go_default_library", @@ -76,7 +74,6 @@ go_test( srcs = [ "archived_point_test.go", "backup_test.go", - "blob_test.go", "blocks_test.go", "checkpoint_test.go", "deposit_contract_test.go", @@ -114,7 +111,6 @@ go_test( "//proto/prysm/v1alpha1:go_default_library", "//proto/testing:go_default_library", "//testing/assert:go_default_library", - "//testing/assertions:go_default_library", "//testing/require:go_default_library", "//testing/util:go_default_library", "@com_github_ethereum_go_ethereum//common:go_default_library", diff --git a/beacon-chain/db/kv/blob.go b/beacon-chain/db/kv/blob.go deleted file mode 100644 index 322df4f32..000000000 --- a/beacon-chain/db/kv/blob.go +++ /dev/null @@ -1,320 +0,0 @@ -package kv - -import ( - "bytes" - "context" - "sort" - - "github.com/pkg/errors" - fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams" - "github.com/prysmaticlabs/prysm/v4/config/params" - types "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives" - "github.com/prysmaticlabs/prysm/v4/encoding/bytesutil" - ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1" - bolt "go.etcd.io/bbolt" - "go.opencensus.io/trace" -) - -var ( - errBlobSlotMismatch = errors.New("sidecar slot mismatch") - errBlobParentMismatch = errors.New("sidecar parent root mismatch") - errBlobRootMismatch = errors.New("sidecar root mismatch") - errBlobProposerMismatch = errors.New("sidecar proposer index mismatch") - errBlobSidecarLimit = errors.New("sidecar exceeds maximum number of blobs") - errEmptySidecar = errors.New("nil or empty blob sidecars") - errNewerBlobExists = errors.New("Will not overwrite newer blobs in db") -) - -// A blob rotating key is represented as bytes(slot_to_rotating_buffer(blob.slot)) ++ bytes(blob.slot) ++ blob.block_root -type blobRotatingKey []byte - -// BufferPrefix returns the first 8 bytes of the rotating key. -// This represents bytes(slot_to_rotating_buffer(blob.slot)) in the rotating key. -func (rk blobRotatingKey) BufferPrefix() []byte { - return rk[0:8] -} - -// Slot returns the information from the key. -func (rk blobRotatingKey) Slot() types.Slot { - slotBytes := rk[8:16] - return bytesutil.BytesToSlotBigEndian(slotBytes) -} - -// BlockRoot returns the block root information from the key. -func (rk blobRotatingKey) BlockRoot() []byte { - return rk[16:] -} - -// SaveBlobSidecar saves the blobs for a given epoch in the sidecar bucket. When we receive a blob: -// -// 1. Convert slot using a modulo operator to [0, maxSlots] where maxSlots = MAX_EPOCHS_TO_PERSIST_BLOBS*SLOTS_PER_EPOCH -// -// 2. Compute key for blob as bytes(slot_to_rotating_buffer(blob.slot)) ++ bytes(blob.slot) ++ blob.block_root -// -// 3. Begin the save algorithm: If the incoming blob has a slot bigger than the saved slot at the spot -// in the rotating keys buffer, we overwrite all elements for that slot. Otherwise, we merge the blob with an existing one. -// Trying to replace a newer blob with an older one is an error. -func (s *Store) SaveBlobSidecar(ctx context.Context, scs []*ethpb.DeprecatedBlobSidecar) error { - if len(scs) == 0 { - return errEmptySidecar - } - ctx, span := trace.StartSpan(ctx, "BeaconDB.SaveBlobSidecar") - defer span.End() - - first := scs[0] - newKey := s.blobSidecarKey(first) - prefix := newKey.BufferPrefix() - var prune []blobRotatingKey - return s.db.Update(func(tx *bolt.Tx) error { - var existing []byte - sc := ðpb.DeprecatedBlobSidecars{} - bkt := tx.Bucket(blobsBucket) - c := bkt.Cursor() - for k, v := c.Seek(prefix); bytes.HasPrefix(k, prefix); k, v = c.Next() { - key := blobRotatingKey(k) - ks := key.Slot() - if ks < first.Slot { - // Mark older blobs at the same position of the ring buffer for deletion. - prune = append(prune, key) - continue - } - if ks > first.Slot { - // We shouldn't be overwriting newer blobs with older blobs. Something is wrong. - return errNewerBlobExists - } - // The slot isn't older or newer, so it must be equal. - // If the roots match, then we want to merge the new sidecars with the existing data. - if bytes.Equal(first.BlockRoot, key.BlockRoot()) { - existing = v - if err := decode(ctx, v, sc); err != nil { - return err - } - } - // If the slot is equal but the roots don't match, leave the existing key alone and allow the sidecar - // to be written to the new key with the same prefix. In this case sc will be empty, so it will just - // contain the incoming sidecars when we write it. - } - sc.Sidecars = append(sc.Sidecars, scs...) - sortSidecars(sc.Sidecars) - var err error - sc.Sidecars, err = validUniqueSidecars(sc.Sidecars) - if err != nil { - return err - } - encoded, err := encode(ctx, sc) - if err != nil { - return err - } - // don't write if the merged result is the same as before - if len(existing) == len(encoded) && bytes.Equal(existing, encoded) { - return nil - } - // Only prune if we're actually going through with the update. - for _, k := range prune { - if err := bkt.Delete(k); err != nil { - // note: attempting to delete a key that does not exist should not return an error. - log.WithError(err).Warnf("Could not delete blob key %#x.", k) - } - } - return bkt.Put(newKey, encoded) - }) -} - -// validUniqueSidecars ensures that all sidecars have the same slot, parent root, block root, and proposer index, and -// there are no more than MAX_BLOBS_PER_BLOCK sidecars. -func validUniqueSidecars(scs []*ethpb.DeprecatedBlobSidecar) ([]*ethpb.DeprecatedBlobSidecar, error) { - if len(scs) == 0 { - return nil, errEmptySidecar - } - - // If there's only 1 sidecar, we've got nothing to compare. - if len(scs) == 1 { - return scs, nil - } - - prev := scs[0] - didx := 1 - for i := 1; i < len(scs); i++ { - sc := scs[i] - if sc.Slot != prev.Slot { - return nil, errors.Wrapf(errBlobSlotMismatch, "%d != %d", sc.Slot, prev.Slot) - } - if !bytes.Equal(sc.BlockParentRoot, prev.BlockParentRoot) { - return nil, errors.Wrapf(errBlobParentMismatch, "%x != %x", sc.BlockParentRoot, prev.BlockParentRoot) - } - if !bytes.Equal(sc.BlockRoot, prev.BlockRoot) { - return nil, errors.Wrapf(errBlobRootMismatch, "%x != %x", sc.BlockRoot, prev.BlockRoot) - } - if sc.ProposerIndex != prev.ProposerIndex { - return nil, errors.Wrapf(errBlobProposerMismatch, "%d != %d", sc.ProposerIndex, prev.ProposerIndex) - } - // skip duplicate - if sc.Index == prev.Index { - continue - } - if didx != i { - scs[didx] = scs[i] - } - prev = scs[i] - didx += 1 - } - - if didx > fieldparams.MaxBlobsPerBlock { - return nil, errors.Wrapf(errBlobSidecarLimit, "%d > %d", didx, fieldparams.MaxBlobsPerBlock) - } - return scs[0:didx], nil -} - -// sortSidecars sorts the sidecars by their index. -func sortSidecars(scs []*ethpb.DeprecatedBlobSidecar) { - sort.Slice(scs, func(i, j int) bool { - return scs[i].Index < scs[j].Index - }) -} - -// BlobSidecarsByRoot retrieves the blobs for the given beacon block root. -// If the `indices` argument is omitted, all blobs for the root will be returned. -// Otherwise, the result will be filtered to only include the specified indices. -// An error will result if an invalid index is specified. -// The bucket size is bounded by 131072 entries. That's the most blobs a node will keep before rotating it out. -func (s *Store) BlobSidecarsByRoot(ctx context.Context, root [32]byte, indices ...uint64) ([]*ethpb.DeprecatedBlobSidecar, error) { - ctx, span := trace.StartSpan(ctx, "BeaconDB.BlobSidecarsByRoot") - defer span.End() - - var enc []byte - if err := s.db.View(func(tx *bolt.Tx) error { - c := tx.Bucket(blobsBucket).Cursor() - // Bucket size is bounded and bolt cursors are fast. Moreover, a thin caching layer can be added. - for k, v := c.First(); k != nil; k, v = c.Next() { - if bytes.HasSuffix(k, root[:]) { - enc = v - break - } - } - return nil - }); err != nil { - return nil, err - } - if enc == nil { - return nil, ErrNotFound - } - sc := ðpb.DeprecatedBlobSidecars{} - if err := decode(ctx, enc, sc); err != nil { - return nil, err - } - - return filterForIndices(sc, indices...) -} - -func filterForIndices(sc *ethpb.DeprecatedBlobSidecars, indices ...uint64) ([]*ethpb.DeprecatedBlobSidecar, error) { - if len(indices) == 0 { - return sc.Sidecars, nil - } - // This loop assumes that the BlobSidecars value stores the complete set of blobs for a block - // in ascending order from eg 0..3, without gaps. This allows us to assume the indices argument - // maps 1:1 with indices in the BlobSidecars storage object. - maxIdx := uint64(len(sc.Sidecars)) - 1 - sidecars := make([]*ethpb.DeprecatedBlobSidecar, len(indices)) - for i, idx := range indices { - if idx > maxIdx { - return nil, errors.Wrapf(ErrNotFound, "BlobSidecars missing index: index %d", idx) - } - sidecars[i] = sc.Sidecars[idx] - } - return sidecars, nil -} - -// BlobSidecarsBySlot retrieves BlobSidecars for the given slot. -// If the `indices` argument is omitted, all blobs for the slot will be returned. -// Otherwise, the result will be filtered to only include the specified indices. -// An error will result if an invalid index is specified. -// The bucket size is bounded by 131072 entries. That's the most blobs a node will keep before rotating it out. -func (s *Store) BlobSidecarsBySlot(ctx context.Context, slot types.Slot, indices ...uint64) ([]*ethpb.DeprecatedBlobSidecar, error) { - ctx, span := trace.StartSpan(ctx, "BeaconDB.BlobSidecarsBySlot") - defer span.End() - - var enc []byte - sk := s.slotKey(slot) - if err := s.db.View(func(tx *bolt.Tx) error { - c := tx.Bucket(blobsBucket).Cursor() - // Bucket size is bounded and bolt cursors are fast. Moreover, a thin caching layer can be added. - for k, v := c.Seek(sk); bytes.HasPrefix(k, sk); k, _ = c.Next() { - slotInKey := bytesutil.BytesToSlotBigEndian(k[8:16]) - if slotInKey == slot { - enc = v - break - } - } - return nil - }); err != nil { - return nil, err - } - if enc == nil { - return nil, ErrNotFound - } - sc := ðpb.DeprecatedBlobSidecars{} - if err := decode(ctx, enc, sc); err != nil { - return nil, err - } - - return filterForIndices(sc, indices...) -} - -// DeleteBlobSidecars returns true if the blobs are in the db. -func (s *Store) DeleteBlobSidecars(ctx context.Context, beaconBlockRoot [32]byte) error { - _, span := trace.StartSpan(ctx, "BeaconDB.DeleteBlobSidecar") - defer span.End() - return s.db.Update(func(tx *bolt.Tx) error { - bkt := tx.Bucket(blobsBucket) - c := bkt.Cursor() - for k, _ := c.First(); k != nil; k, _ = c.Next() { - if bytes.HasSuffix(k, beaconBlockRoot[:]) { - if err := bkt.Delete(k); err != nil { - return err - } - } - } - return nil - }) -} - -// We define a blob sidecar key as: bytes(slot_to_rotating_buffer(blob.slot)) ++ bytes(blob.slot) ++ blob.block_root -// where slot_to_rotating_buffer(slot) = slot % MAX_SLOTS_TO_PERSIST_BLOBS. -func (s *Store) blobSidecarKey(blob *ethpb.DeprecatedBlobSidecar) blobRotatingKey { - key := s.slotKey(blob.Slot) - key = append(key, bytesutil.SlotToBytesBigEndian(blob.Slot)...) - key = append(key, blob.BlockRoot...) - return key -} - -func (s *Store) slotKey(slot types.Slot) []byte { - return bytesutil.SlotToBytesBigEndian(slot.ModSlot(s.blobRetentionSlots())) -} - -func (s *Store) blobRetentionSlots() types.Slot { - return types.Slot(s.blobRetentionEpochs.Mul(uint64(params.BeaconConfig().SlotsPerEpoch))) -} - -var errBlobRetentionEpochMismatch = errors.New("epochs for blobs request value in DB does not match runtime config") - -func (s *Store) checkEpochsForBlobSidecarsRequestBucket(db *bolt.DB) error { - uRetentionEpochs := uint64(s.blobRetentionEpochs) - if err := db.Update(func(tx *bolt.Tx) error { - b := tx.Bucket(chainMetadataBucket) - v := b.Get(blobRetentionEpochsKey) - if v == nil { - if err := b.Put(blobRetentionEpochsKey, bytesutil.Uint64ToBytesBigEndian(uRetentionEpochs)); err != nil { - return err - } - return nil - } - e := bytesutil.BytesToUint64BigEndian(v) - if e != uRetentionEpochs { - return errors.Wrapf(errBlobRetentionEpochMismatch, "db=%d, config=%d", e, uRetentionEpochs) - } - return nil - }); err != nil { - return err - } - return nil -} diff --git a/beacon-chain/db/kv/blob_test.go b/beacon-chain/db/kv/blob_test.go deleted file mode 100644 index 527f39b2e..000000000 --- a/beacon-chain/db/kv/blob_test.go +++ /dev/null @@ -1,532 +0,0 @@ -package kv - -import ( - "context" - "crypto/rand" - "fmt" - "testing" - - "github.com/pkg/errors" - fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams" - "github.com/prysmaticlabs/prysm/v4/config/params" - "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives" - types "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives" - "github.com/prysmaticlabs/prysm/v4/encoding/bytesutil" - ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1" - "github.com/prysmaticlabs/prysm/v4/testing/assertions" - "github.com/prysmaticlabs/prysm/v4/testing/require" - bolt "go.etcd.io/bbolt" -) - -func equalBlobSlices(expect []*ethpb.DeprecatedBlobSidecar, got []*ethpb.DeprecatedBlobSidecar) error { - if len(expect) != len(got) { - return fmt.Errorf("mismatched lengths, expect=%d, got=%d", len(expect), len(got)) - } - for i := 0; i < len(expect); i++ { - es := expect[i] - gs := got[i] - var e string - assertions.DeepEqual(assertions.SprintfAssertionLoggerFn(&e), es, gs) - if e != "" { - return errors.New(e) - } - } - return nil -} - -func TestStore_BlobSidecars(t *testing.T) { - ctx := context.Background() - - t.Run("empty", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, 0) - require.ErrorContains(t, "nil or empty blob sidecars", db.SaveBlobSidecar(ctx, scs)) - }) - t.Run("empty by root", func(t *testing.T) { - db := setupDB(t) - got, err := db.BlobSidecarsByRoot(ctx, [32]byte{}) - require.ErrorIs(t, ErrNotFound, err) - require.Equal(t, 0, len(got)) - }) - t.Run("empty by slot", func(t *testing.T) { - db := setupDB(t) - got, err := db.BlobSidecarsBySlot(ctx, 1) - require.ErrorIs(t, ErrNotFound, err) - require.Equal(t, 0, len(got)) - }) - t.Run("save and retrieve by root (one)", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, 1) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, 1, len(scs)) - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save and retrieve by root (max), per batch", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save and retrieve by root, max and individually", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - for _, sc := range scs { - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{sc})) - } - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save and retrieve valid subset by root", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - - // we'll request indices 0 and 3, so make a slice with those indices for comparison - expect := make([]*ethpb.DeprecatedBlobSidecar, 2) - expect[0] = scs[0] - expect[1] = scs[3] - - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot), 0, 3) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(expect, got)) - require.Equal(t, uint64(0), got[0].Index) - require.Equal(t, uint64(3), got[1].Index) - }) - t.Run("error for invalid index when retrieving by root", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot), uint64(len(scs))) - require.ErrorIs(t, err, ErrNotFound) - require.Equal(t, 0, len(got)) - }) - t.Run("save and retrieve by slot (one)", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, 1) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, 1, len(scs)) - got, err := db.BlobSidecarsBySlot(ctx, scs[0].Slot) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save and retrieve by slot (max)", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - got, err := db.BlobSidecarsBySlot(ctx, scs[0].Slot) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save and retrieve by slot, max and individually", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - for _, sc := range scs { - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{sc})) - } - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - got, err := db.BlobSidecarsBySlot(ctx, scs[0].Slot) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save and retrieve valid subset by slot", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - - // we'll request indices 0 and 3, so make a slice with those indices for comparison - expect := make([]*ethpb.DeprecatedBlobSidecar, 2) - expect[0] = scs[0] - expect[1] = scs[3] - - got, err := db.BlobSidecarsBySlot(ctx, scs[0].Slot, 0, 3) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(expect, got)) - - require.Equal(t, uint64(0), got[0].Index) - require.Equal(t, uint64(3), got[1].Index) - }) - t.Run("error for invalid index when retrieving by slot", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - - got, err := db.BlobSidecarsBySlot(ctx, scs[0].Slot, uint64(len(scs))) - require.ErrorIs(t, err, ErrNotFound) - require.Equal(t, 0, len(got)) - }) - t.Run("delete works", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - require.NoError(t, db.DeleteBlobSidecars(ctx, bytesutil.ToBytes32(scs[0].BlockRoot))) - got, err = db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.ErrorIs(t, ErrNotFound, err) - require.Equal(t, 0, len(got)) - }) - t.Run("saving blob different times", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - - for i := 0; i < fieldparams.MaxBlobsPerBlock; i++ { - scs[i].Slot = primitives.Slot(i) - scs[i].BlockRoot = bytesutil.PadTo([]byte{byte(i)}, 32) - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{scs[i]})) - br := bytesutil.ToBytes32(scs[i].BlockRoot) - saved, err := db.BlobSidecarsByRoot(ctx, br) - require.NoError(t, err) - require.NoError(t, equalBlobSlices([]*ethpb.DeprecatedBlobSidecar{scs[i]}, saved)) - } - }) - t.Run("saving a new blob for rotation (batch)", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - oldBlockRoot := scs[0].BlockRoot - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(oldBlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - - newScs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - newRetentionSlot := primitives.Slot(params.BeaconConfig().MinEpochsForBlobsSidecarsRequest.Mul(uint64(params.BeaconConfig().SlotsPerEpoch))) - for _, sc := range newScs { - sc.Slot = sc.Slot + newRetentionSlot - } - require.NoError(t, db.SaveBlobSidecar(ctx, newScs)) - - _, err = db.BlobSidecarsBySlot(ctx, 100) - require.ErrorIs(t, ErrNotFound, err) - - got, err = db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(newScs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(newScs, got)) - }) - t.Run("save multiple blobs after new rotation (individually)", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - for _, sc := range scs { - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{sc})) - } - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - - scs = generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - newRetentionSlot := primitives.Slot(params.BeaconConfig().MinEpochsForBlobsSidecarsRequest.Mul(uint64(params.BeaconConfig().SlotsPerEpoch))) - for _, sc := range scs { - sc.Slot = sc.Slot + newRetentionSlot - } - for _, sc := range scs { - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{sc})) - } - - _, err = db.BlobSidecarsBySlot(ctx, 100) - require.ErrorIs(t, ErrNotFound, err) - - got, err = db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save multiple blobs after new rotation (batch then individually)", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - require.Equal(t, fieldparams.MaxBlobsPerBlock, len(scs)) - oldBlockRoot := scs[0].BlockRoot - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(oldBlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - - scs = generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - newRetentionSlot := primitives.Slot(params.BeaconConfig().MinEpochsForBlobsSidecarsRequest.Mul(uint64(params.BeaconConfig().SlotsPerEpoch))) - for _, sc := range scs { - sc.Slot = sc.Slot + newRetentionSlot - } - for _, sc := range scs { - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{sc})) - } - - _, err = db.BlobSidecarsBySlot(ctx, 100) - require.ErrorIs(t, ErrNotFound, err) - - got, err = db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save multiple blobs after new rotation (individually then batch)", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - for _, sc := range scs { - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{sc})) - } - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - - scs = generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock) - newRetentionSlot := primitives.Slot(params.BeaconConfig().MinEpochsForBlobsSidecarsRequest.Mul(uint64(params.BeaconConfig().SlotsPerEpoch))) - for _, sc := range scs { - sc.Slot = sc.Slot + newRetentionSlot - } - require.NoError(t, db.SaveBlobSidecar(ctx, scs)) - - _, err = db.BlobSidecarsBySlot(ctx, 100) - require.ErrorIs(t, ErrNotFound, err) - - got, err = db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - }) - t.Run("save equivocating blobs", func(t *testing.T) { - db := setupDB(t) - scs := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock/2) - eScs := generateEquivocatingBlobSidecars(t, fieldparams.MaxBlobsPerBlock/2) - - for i, sc := range scs { - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{sc})) - require.NoError(t, db.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{eScs[i]})) - } - - got, err := db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(scs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(scs, got)) - - got, err = db.BlobSidecarsByRoot(ctx, bytesutil.ToBytes32(eScs[0].BlockRoot)) - require.NoError(t, err) - require.NoError(t, equalBlobSlices(eScs, got)) - }) -} - -func generateBlobSidecars(t *testing.T, n uint64) []*ethpb.DeprecatedBlobSidecar { - blobSidecars := make([]*ethpb.DeprecatedBlobSidecar, n) - for i := uint64(0); i < n; i++ { - blobSidecars[i] = generateBlobSidecar(t, i) - } - return blobSidecars -} - -func generateBlobSidecar(t *testing.T, index uint64) *ethpb.DeprecatedBlobSidecar { - blob := make([]byte, 131072) - _, err := rand.Read(blob) - require.NoError(t, err) - kzgCommitment := make([]byte, 48) - _, err = rand.Read(kzgCommitment) - require.NoError(t, err) - kzgProof := make([]byte, 48) - _, err = rand.Read(kzgProof) - require.NoError(t, err) - return ðpb.DeprecatedBlobSidecar{ - BlockRoot: bytesutil.PadTo([]byte{'a'}, 32), - Index: index, - Slot: 100, - BlockParentRoot: bytesutil.PadTo([]byte{'b'}, 32), - ProposerIndex: 101, - Blob: blob, - KzgCommitment: kzgCommitment, - KzgProof: kzgProof, - } -} - -func generateEquivocatingBlobSidecars(t *testing.T, n uint64) []*ethpb.DeprecatedBlobSidecar { - blobSidecars := make([]*ethpb.DeprecatedBlobSidecar, n) - for i := uint64(0); i < n; i++ { - blobSidecars[i] = generateEquivocatingBlobSidecar(t, i) - } - return blobSidecars -} - -func generateEquivocatingBlobSidecar(t *testing.T, index uint64) *ethpb.DeprecatedBlobSidecar { - blob := make([]byte, 131072) - _, err := rand.Read(blob) - require.NoError(t, err) - kzgCommitment := make([]byte, 48) - _, err = rand.Read(kzgCommitment) - require.NoError(t, err) - kzgProof := make([]byte, 48) - _, err = rand.Read(kzgProof) - require.NoError(t, err) - - return ðpb.DeprecatedBlobSidecar{ - BlockRoot: bytesutil.PadTo([]byte{'c'}, 32), - Index: index, - Slot: 100, - BlockParentRoot: bytesutil.PadTo([]byte{'b'}, 32), - ProposerIndex: 102, - Blob: blob, - KzgCommitment: kzgCommitment, - KzgProof: kzgProof, - } -} - -func Test_validUniqueSidecars_validation(t *testing.T) { - tests := []struct { - name string - scs []*ethpb.DeprecatedBlobSidecar - err error - }{ - {name: "empty", scs: []*ethpb.DeprecatedBlobSidecar{}, err: errEmptySidecar}, - {name: "too many sidecars", scs: generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock+1), err: errBlobSidecarLimit}, - {name: "invalid slot", scs: []*ethpb.DeprecatedBlobSidecar{{Slot: 1}, {Slot: 2}}, err: errBlobSlotMismatch}, - {name: "invalid proposer index", scs: []*ethpb.DeprecatedBlobSidecar{{ProposerIndex: 1}, {ProposerIndex: 2}}, err: errBlobProposerMismatch}, - {name: "invalid root", scs: []*ethpb.DeprecatedBlobSidecar{{BlockRoot: []byte{1}}, {BlockRoot: []byte{2}}}, err: errBlobRootMismatch}, - {name: "invalid parent root", scs: []*ethpb.DeprecatedBlobSidecar{{BlockParentRoot: []byte{1}}, {BlockParentRoot: []byte{2}}}, err: errBlobParentMismatch}, - {name: "happy path", scs: []*ethpb.DeprecatedBlobSidecar{{Index: 0}, {Index: 1}}}, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - _, err := validUniqueSidecars(tt.scs) - if tt.err != nil { - require.ErrorIs(t, err, tt.err) - } else { - require.NoError(t, err) - } - }) - } -} - -func Test_validUniqueSidecars_dedup(t *testing.T) { - cases := []struct { - name string - scs []*ethpb.DeprecatedBlobSidecar - expected []*ethpb.DeprecatedBlobSidecar - err error - }{ - { - name: "duplicate sidecar", - scs: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 1}}, - expected: []*ethpb.DeprecatedBlobSidecar{{Index: 1}}, - }, - { - name: "single sidecar", - scs: []*ethpb.DeprecatedBlobSidecar{{Index: 1}}, - expected: []*ethpb.DeprecatedBlobSidecar{{Index: 1}}, - }, - { - name: "multiple duplicates", - scs: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 2}, {Index: 2}, {Index: 3}, {Index: 3}}, - expected: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 2}, {Index: 3}}, - }, - { - name: "ok number after de-dupe, > 6 before", - scs: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 2}, {Index: 2}, {Index: 2}, {Index: 2}, {Index: 3}, {Index: 3}}, - expected: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 2}, {Index: 3}}, - }, - { - name: "max unique, no dupes", - scs: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 2}, {Index: 3}, {Index: 4}, {Index: 5}, {Index: 6}}, - expected: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 2}, {Index: 3}, {Index: 4}, {Index: 5}, {Index: 6}}, - }, - { - name: "too many unique", - scs: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 2}, {Index: 3}, {Index: 4}, {Index: 5}, {Index: 6}, {Index: 7}}, - err: errBlobSidecarLimit, - }, - { - name: "too many unique with dupes", - scs: []*ethpb.DeprecatedBlobSidecar{{Index: 1}, {Index: 1}, {Index: 1}, {Index: 2}, {Index: 3}, {Index: 4}, {Index: 5}, {Index: 6}, {Index: 7}}, - err: errBlobSidecarLimit, - }, - } - for _, c := range cases { - t.Run(c.name, func(t *testing.T) { - u, err := validUniqueSidecars(c.scs) - if c.err != nil { - require.ErrorIs(t, err, c.err) - } else { - require.NoError(t, err) - } - require.Equal(t, len(c.expected), len(u)) - }) - } -} - -func TestStore_sortSidecars(t *testing.T) { - scs := []*ethpb.DeprecatedBlobSidecar{ - {Index: 6}, - {Index: 4}, - {Index: 2}, - {Index: 1}, - {Index: 3}, - {Index: 5}, - {}, - } - sortSidecars(scs) - for i := 0; i < len(scs)-1; i++ { - require.Equal(t, uint64(i), scs[i].Index) - } -} - -func BenchmarkStore_BlobSidecarsByRoot(b *testing.B) { - s := setupDB(b) - ctx := context.Background() - require.NoError(b, s.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{ - {BlockRoot: bytesutil.PadTo([]byte{'a'}, 32), Slot: 0}, - })) - - err := s.db.Update(func(tx *bolt.Tx) error { - bkt := tx.Bucket(blobsBucket) - for i := 1; i < 131071; i++ { - r := make([]byte, 32) - _, err := rand.Read(r) - require.NoError(b, err) - scs := []*ethpb.DeprecatedBlobSidecar{ - {BlockRoot: r, Slot: primitives.Slot(i)}, - } - k := s.blobSidecarKey(scs[0]) - encodedBlobSidecar, err := encode(ctx, ðpb.DeprecatedBlobSidecars{Sidecars: scs}) - require.NoError(b, err) - require.NoError(b, bkt.Put(k, encodedBlobSidecar)) - } - return nil - }) - require.NoError(b, err) - - require.NoError(b, s.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{ - {BlockRoot: bytesutil.PadTo([]byte{'b'}, 32), Slot: 131071}, - })) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - _, err := s.BlobSidecarsByRoot(ctx, [32]byte{'b'}) - require.NoError(b, err) - } -} - -func Test_checkEpochsForBlobSidecarsRequestBucket(t *testing.T) { - s := setupDB(t) - - require.NoError(t, s.checkEpochsForBlobSidecarsRequestBucket(s.db)) // First write - require.NoError(t, s.checkEpochsForBlobSidecarsRequestBucket(s.db)) // First check - - s.blobRetentionEpochs += 1 - require.ErrorIs(t, s.checkEpochsForBlobSidecarsRequestBucket(s.db), errBlobRetentionEpochMismatch) -} - -func TestBlobRotatingKey(t *testing.T) { - s := setupDB(t) - k := s.blobSidecarKey(ðpb.DeprecatedBlobSidecar{ - Slot: 1, - BlockRoot: []byte{2}, - }) - - require.Equal(t, types.Slot(1), k.Slot()) - require.DeepEqual(t, []byte{2}, k.BlockRoot()) - require.DeepEqual(t, s.slotKey(types.Slot(1)), k.BufferPrefix()) -} diff --git a/beacon-chain/db/kv/kv.go b/beacon-chain/db/kv/kv.go index a75c82f30..4edd992fd 100644 --- a/beacon-chain/db/kv/kv.go +++ b/beacon-chain/db/kv/kv.go @@ -18,7 +18,6 @@ import ( "github.com/prysmaticlabs/prysm/v4/config/features" "github.com/prysmaticlabs/prysm/v4/config/params" "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks" - "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives" "github.com/prysmaticlabs/prysm/v4/io/file" bolt "go.etcd.io/bbolt" ) @@ -91,7 +90,6 @@ type Store struct { validatorEntryCache *ristretto.Cache stateSummaryCache *stateSummaryCache ctx context.Context - blobRetentionEpochs primitives.Epoch } // StoreDatafilePath is the canonical construction of a full @@ -138,13 +136,6 @@ var Buckets = [][]byte{ // KVStoreOption is a functional option that modifies a kv.Store. type KVStoreOption func(*Store) -// WithBlobRetentionEpochs sets the variable configuring the blob retention window. -func WithBlobRetentionEpochs(e primitives.Epoch) KVStoreOption { - return func(s *Store) { - s.blobRetentionEpochs = e - } -} - // NewKVStore initializes a new boltDB key-value store at the directory // path specified, creates the kv-buckets based on the schema, and stores // an open connection db object as a property of the Store struct. @@ -217,14 +208,6 @@ func NewKVStore(ctx context.Context, dirPath string, opts ...KVStoreOption) (*St return nil, err } - if err := kv.checkEpochsForBlobSidecarsRequestBucket(boltDB); err != nil { - return nil, errors.Wrap(err, "failed to check epochs for blob sidecars request bucket") - } - - // set a default so that tests don't break - if kv.blobRetentionEpochs == 0 { - kv.blobRetentionEpochs = params.BeaconConfig().MinEpochsForBlobsSidecarsRequest - } return kv, nil } diff --git a/beacon-chain/db/kv/kv_test.go b/beacon-chain/db/kv/kv_test.go index d20d8430d..3941d468c 100644 --- a/beacon-chain/db/kv/kv_test.go +++ b/beacon-chain/db/kv/kv_test.go @@ -6,7 +6,6 @@ import ( "testing" "github.com/prysmaticlabs/prysm/v4/config/features" - "github.com/prysmaticlabs/prysm/v4/config/params" "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks" ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1" "github.com/prysmaticlabs/prysm/v4/testing/require" @@ -16,8 +15,7 @@ import ( // setupDB instantiates and returns a Store instance. func setupDB(t testing.TB) *Store { - opt := WithBlobRetentionEpochs(params.BeaconConfig().MinEpochsForBlobsSidecarsRequest) - db, err := NewKVStore(context.Background(), t.TempDir(), opt) + db, err := NewKVStore(context.Background(), t.TempDir()) require.NoError(t, err, "Failed to instantiate DB") t.Cleanup(func() { require.NoError(t, db.Close(), "Failed to close database") diff --git a/beacon-chain/db/kv/schema.go b/beacon-chain/db/kv/schema.go index b7b1f1af5..a5c89230a 100644 --- a/beacon-chain/db/kv/schema.go +++ b/beacon-chain/db/kv/schema.go @@ -47,10 +47,6 @@ var ( finalizedCheckpointKey = []byte("finalized-checkpoint") powchainDataKey = []byte("powchain-data") lastValidatedCheckpointKey = []byte("last-validated-checkpoint") - // blobRetentionEpochsKey determines the size of the blob circular buffer and how the keys in that buffer are - // determined. If this value changes, the existing data is invalidated, so storing it in the db - // allows us to assert at runtime that the db state is still consistent with the runtime state. - blobRetentionEpochsKey = []byte("blob-retention-epochs") // Below keys are used to identify objects are to be fork compatible. // Objects that are only compatible with specific forks should be prefixed with such keys. diff --git a/beacon-chain/node/node.go b/beacon-chain/node/node.go index c148734ad..916481074 100644 --- a/beacon-chain/node/node.go +++ b/beacon-chain/node/node.go @@ -398,7 +398,7 @@ func (b *BeaconNode) startDB(cliCtx *cli.Context, depositAddress string) error { log.WithField("database-path", dbPath).Info("Checking DB") - d, err := kv.NewKVStore(b.ctx, dbPath, kv.WithBlobRetentionEpochs(b.blobRetentionEpochs)) + d, err := kv.NewKVStore(b.ctx, dbPath) if err != nil { return err } @@ -421,7 +421,7 @@ func (b *BeaconNode) startDB(cliCtx *cli.Context, depositAddress string) error { return errors.Wrap(err, "could not clear database") } - d, err = kv.NewKVStore(b.ctx, dbPath, kv.WithBlobRetentionEpochs(b.blobRetentionEpochs)) + d, err = kv.NewKVStore(b.ctx, dbPath) if err != nil { return errors.Wrap(err, "could not create new database") } diff --git a/proto/prysm/v1alpha1/BUILD.bazel b/proto/prysm/v1alpha1/BUILD.bazel index 754d0428b..e9a969bb6 100644 --- a/proto/prysm/v1alpha1/BUILD.bazel +++ b/proto/prysm/v1alpha1/BUILD.bazel @@ -127,10 +127,8 @@ ssz_gen_marshal( "BuilderBid", "BuilderBidCapella", "BuilderBidDeneb", - "DeprecatedBlobSidecar", "BlobSidecar", "BlobSidecars", - "SignedBlobSidecar", "BlobIdentifier", ], ) diff --git a/proto/prysm/v1alpha1/blobs.pb.go b/proto/prysm/v1alpha1/blobs.pb.go index 0c7c4b339..0d8e36937 100755 --- a/proto/prysm/v1alpha1/blobs.pb.go +++ b/proto/prysm/v1alpha1/blobs.pb.go @@ -23,211 +23,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -type DeprecatedBlobSidecars struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Sidecars []*DeprecatedBlobSidecar `protobuf:"bytes,1,rep,name=sidecars,proto3" json:"sidecars,omitempty"` -} - -func (x *DeprecatedBlobSidecars) Reset() { - *x = DeprecatedBlobSidecars{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *DeprecatedBlobSidecars) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*DeprecatedBlobSidecars) ProtoMessage() {} - -func (x *DeprecatedBlobSidecars) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use DeprecatedBlobSidecars.ProtoReflect.Descriptor instead. -func (*DeprecatedBlobSidecars) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{0} -} - -func (x *DeprecatedBlobSidecars) GetSidecars() []*DeprecatedBlobSidecar { - if x != nil { - return x.Sidecars - } - return nil -} - -type DeprecatedBlobSidecar struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - BlockRoot []byte `protobuf:"bytes,1,opt,name=block_root,json=blockRoot,proto3" json:"block_root,omitempty" ssz-size:"32"` - Index uint64 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"` - Slot github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.Slot `protobuf:"varint,3,opt,name=slot,proto3" json:"slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives.Slot"` - BlockParentRoot []byte `protobuf:"bytes,4,opt,name=block_parent_root,json=blockParentRoot,proto3" json:"block_parent_root,omitempty" ssz-size:"32"` - ProposerIndex github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.ValidatorIndex `protobuf:"varint,5,opt,name=proposer_index,json=proposerIndex,proto3" json:"proposer_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives.ValidatorIndex"` - Blob []byte `protobuf:"bytes,6,opt,name=blob,proto3" json:"blob,omitempty" ssz-size:"131072"` - KzgCommitment []byte `protobuf:"bytes,7,opt,name=kzg_commitment,json=kzgCommitment,proto3" json:"kzg_commitment,omitempty" ssz-size:"48"` - KzgProof []byte `protobuf:"bytes,8,opt,name=kzg_proof,json=kzgProof,proto3" json:"kzg_proof,omitempty" ssz-size:"48"` -} - -func (x *DeprecatedBlobSidecar) Reset() { - *x = DeprecatedBlobSidecar{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *DeprecatedBlobSidecar) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*DeprecatedBlobSidecar) ProtoMessage() {} - -func (x *DeprecatedBlobSidecar) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use DeprecatedBlobSidecar.ProtoReflect.Descriptor instead. -func (*DeprecatedBlobSidecar) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{1} -} - -func (x *DeprecatedBlobSidecar) GetBlockRoot() []byte { - if x != nil { - return x.BlockRoot - } - return nil -} - -func (x *DeprecatedBlobSidecar) GetIndex() uint64 { - if x != nil { - return x.Index - } - return 0 -} - -func (x *DeprecatedBlobSidecar) GetSlot() github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.Slot { - if x != nil { - return x.Slot - } - return github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.Slot(0) -} - -func (x *DeprecatedBlobSidecar) GetBlockParentRoot() []byte { - if x != nil { - return x.BlockParentRoot - } - return nil -} - -func (x *DeprecatedBlobSidecar) GetProposerIndex() github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.ValidatorIndex { - if x != nil { - return x.ProposerIndex - } - return github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.ValidatorIndex(0) -} - -func (x *DeprecatedBlobSidecar) GetBlob() []byte { - if x != nil { - return x.Blob - } - return nil -} - -func (x *DeprecatedBlobSidecar) GetKzgCommitment() []byte { - if x != nil { - return x.KzgCommitment - } - return nil -} - -func (x *DeprecatedBlobSidecar) GetKzgProof() []byte { - if x != nil { - return x.KzgProof - } - return nil -} - -type SignedBlobSidecar struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Message *DeprecatedBlobSidecar `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` - Signature []byte `protobuf:"bytes,2,opt,name=signature,proto3" json:"signature,omitempty" ssz-size:"96"` -} - -func (x *SignedBlobSidecar) Reset() { - *x = SignedBlobSidecar{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *SignedBlobSidecar) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*SignedBlobSidecar) ProtoMessage() {} - -func (x *SignedBlobSidecar) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use SignedBlobSidecar.ProtoReflect.Descriptor instead. -func (*SignedBlobSidecar) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{2} -} - -func (x *SignedBlobSidecar) GetMessage() *DeprecatedBlobSidecar { - if x != nil { - return x.Message - } - return nil -} - -func (x *SignedBlobSidecar) GetSignature() []byte { - if x != nil { - return x.Signature - } - return nil -} - type BlindedBlobSidecars struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -239,7 +34,7 @@ type BlindedBlobSidecars struct { func (x *BlindedBlobSidecars) Reset() { *x = BlindedBlobSidecars{} if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -252,7 +47,7 @@ func (x *BlindedBlobSidecars) String() string { func (*BlindedBlobSidecars) ProtoMessage() {} func (x *BlindedBlobSidecars) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -265,7 +60,7 @@ func (x *BlindedBlobSidecars) ProtoReflect() protoreflect.Message { // Deprecated: Use BlindedBlobSidecars.ProtoReflect.Descriptor instead. func (*BlindedBlobSidecars) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{3} + return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{0} } func (x *BlindedBlobSidecars) GetSidecars() []*BlindedBlobSidecar { @@ -293,7 +88,7 @@ type BlindedBlobSidecar struct { func (x *BlindedBlobSidecar) Reset() { *x = BlindedBlobSidecar{} if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[4] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -306,7 +101,7 @@ func (x *BlindedBlobSidecar) String() string { func (*BlindedBlobSidecar) ProtoMessage() {} func (x *BlindedBlobSidecar) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[4] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -319,7 +114,7 @@ func (x *BlindedBlobSidecar) ProtoReflect() protoreflect.Message { // Deprecated: Use BlindedBlobSidecar.ProtoReflect.Descriptor instead. func (*BlindedBlobSidecar) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{4} + return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{1} } func (x *BlindedBlobSidecar) GetBlockRoot() []byte { @@ -390,7 +185,7 @@ type SignedBlindedBlobSidecar struct { func (x *SignedBlindedBlobSidecar) Reset() { *x = SignedBlindedBlobSidecar{} if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[5] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -403,7 +198,7 @@ func (x *SignedBlindedBlobSidecar) String() string { func (*SignedBlindedBlobSidecar) ProtoMessage() {} func (x *SignedBlindedBlobSidecar) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[5] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -416,7 +211,7 @@ func (x *SignedBlindedBlobSidecar) ProtoReflect() protoreflect.Message { // Deprecated: Use SignedBlindedBlobSidecar.ProtoReflect.Descriptor instead. func (*SignedBlindedBlobSidecar) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{5} + return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{2} } func (x *SignedBlindedBlobSidecar) GetMessage() *BlindedBlobSidecar { @@ -445,7 +240,7 @@ type BlobIdentifier struct { func (x *BlobIdentifier) Reset() { *x = BlobIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[6] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -458,7 +253,7 @@ func (x *BlobIdentifier) String() string { func (*BlobIdentifier) ProtoMessage() {} func (x *BlobIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[6] + mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -471,7 +266,7 @@ func (x *BlobIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use BlobIdentifier.ProtoReflect.Descriptor instead. func (*BlobIdentifier) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{6} + return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{3} } func (x *BlobIdentifier) GetBlockRoot() []byte { @@ -496,109 +291,66 @@ var file_proto_prysm_v1alpha1_blobs_proto_rawDesc = []byte{ 0x74, 0x6f, 0x12, 0x15, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x1a, 0x1b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x65, 0x78, 0x74, 0x2f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x62, 0x0a, 0x16, 0x44, 0x65, 0x70, 0x72, 0x65, 0x63, - 0x61, 0x74, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, - 0x12, 0x48, 0x0a, 0x08, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, - 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x72, 0x65, - 0x63, 0x61, 0x74, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, - 0x52, 0x08, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x22, 0xcf, 0x03, 0x0a, 0x15, 0x44, - 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, - 0x65, 0x63, 0x61, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, - 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, - 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x12, 0x59, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x42, - 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, - 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, - 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, - 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x32, 0x0a, 0x11, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x6f, - 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, - 0x0f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x6f, 0x74, - 0x12, 0x76, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, - 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, - 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, - 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x70, 0x6f, - 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x1e, 0x0a, 0x04, 0x62, 0x6c, 0x6f, 0x62, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a, 0x8a, 0xb5, 0x18, 0x06, 0x31, 0x33, 0x31, 0x30, - 0x37, 0x32, 0x52, 0x04, 0x62, 0x6c, 0x6f, 0x62, 0x12, 0x2d, 0x0a, 0x0e, 0x6b, 0x7a, 0x67, 0x5f, - 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, - 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x0d, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, - 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x23, 0x0a, 0x09, 0x6b, 0x7a, 0x67, 0x5f, 0x70, - 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, - 0x34, 0x38, 0x52, 0x08, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22, 0x81, 0x01, 0x0a, - 0x11, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, - 0x61, 0x72, 0x12, 0x46, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, - 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x72, - 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, - 0x72, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, - 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, - 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, - 0x22, 0x63, 0x0a, 0x13, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, - 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x12, 0x4c, 0x0a, 0x08, 0x73, 0x69, 0x64, 0x65, 0x63, - 0x61, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, - 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, - 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, - 0x65, 0x63, 0x61, 0x72, 0x42, 0x05, 0x92, 0xb5, 0x18, 0x01, 0x36, 0x52, 0x08, 0x73, 0x69, 0x64, - 0x65, 0x63, 0x61, 0x72, 0x73, 0x22, 0xd1, 0x03, 0x0a, 0x12, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, - 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x25, 0x0a, 0x0a, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, - 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, - 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x59, 0x0a, 0x04, 0x73, 0x6c, 0x6f, - 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, - 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, - 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, 0x63, - 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, - 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, - 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x32, 0x0a, 0x11, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x61, - 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, - 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x61, - 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x76, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x70, - 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, - 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, - 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, - 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, - 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x12, 0x23, 0x0a, 0x09, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x08, 0x62, 0x6c, 0x6f, - 0x62, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2d, 0x0a, 0x0e, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, - 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, - 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x0d, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, - 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x23, 0x0a, 0x09, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, - 0x66, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, - 0x08, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22, 0x85, 0x01, 0x0a, 0x18, 0x53, 0x69, - 0x67, 0x6e, 0x65, 0x64, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, - 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x43, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, - 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x63, 0x0a, 0x13, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, + 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x12, 0x4c, 0x0a, + 0x08, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, + 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, + 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x42, 0x05, 0x92, 0xb5, 0x18, 0x01, + 0x36, 0x52, 0x08, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x22, 0xd1, 0x03, 0x0a, 0x12, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, - 0x61, 0x72, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x24, 0x0a, 0x09, 0x73, - 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, - 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, - 0x65, 0x22, 0x4d, 0x0a, 0x0e, 0x42, 0x6c, 0x6f, 0x62, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, - 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, - 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x42, 0x95, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, - 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x0a, - 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, - 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, - 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, - 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, - 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, - 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x61, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x59, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, + 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, + 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, + 0x6d, 0x2f, 0x76, 0x34, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, + 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, + 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x32, 0x0a, 0x11, 0x62, 0x6c, + 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x76, + 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, + 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, + 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, 0x63, 0x6f, + 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, + 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, + 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x23, 0x0a, 0x09, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x72, + 0x6f, 0x6f, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, + 0x32, 0x52, 0x08, 0x62, 0x6c, 0x6f, 0x62, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2d, 0x0a, 0x0e, 0x6b, + 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x0d, 0x6b, 0x7a, 0x67, + 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x23, 0x0a, 0x09, 0x6b, 0x7a, + 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, + 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x08, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22, + 0x85, 0x01, 0x0a, 0x18, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, + 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x43, 0x0a, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, + 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, + 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, + 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, + 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x4d, 0x0a, 0x0e, 0x42, 0x6c, 0x6f, 0x62, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, + 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, + 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, + 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x95, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, + 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, + 0x70, 0x68, 0x61, 0x31, 0x42, 0x0a, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, + 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, + 0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, + 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, + 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, + 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, + 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -613,26 +365,21 @@ func file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP() []byte { return file_proto_prysm_v1alpha1_blobs_proto_rawDescData } -var file_proto_prysm_v1alpha1_blobs_proto_msgTypes = make([]protoimpl.MessageInfo, 7) +var file_proto_prysm_v1alpha1_blobs_proto_msgTypes = make([]protoimpl.MessageInfo, 4) var file_proto_prysm_v1alpha1_blobs_proto_goTypes = []interface{}{ - (*DeprecatedBlobSidecars)(nil), // 0: ethereum.eth.v1alpha1.DeprecatedBlobSidecars - (*DeprecatedBlobSidecar)(nil), // 1: ethereum.eth.v1alpha1.DeprecatedBlobSidecar - (*SignedBlobSidecar)(nil), // 2: ethereum.eth.v1alpha1.SignedBlobSidecar - (*BlindedBlobSidecars)(nil), // 3: ethereum.eth.v1alpha1.BlindedBlobSidecars - (*BlindedBlobSidecar)(nil), // 4: ethereum.eth.v1alpha1.BlindedBlobSidecar - (*SignedBlindedBlobSidecar)(nil), // 5: ethereum.eth.v1alpha1.SignedBlindedBlobSidecar - (*BlobIdentifier)(nil), // 6: ethereum.eth.v1alpha1.BlobIdentifier + (*BlindedBlobSidecars)(nil), // 0: ethereum.eth.v1alpha1.BlindedBlobSidecars + (*BlindedBlobSidecar)(nil), // 1: ethereum.eth.v1alpha1.BlindedBlobSidecar + (*SignedBlindedBlobSidecar)(nil), // 2: ethereum.eth.v1alpha1.SignedBlindedBlobSidecar + (*BlobIdentifier)(nil), // 3: ethereum.eth.v1alpha1.BlobIdentifier } var file_proto_prysm_v1alpha1_blobs_proto_depIdxs = []int32{ - 1, // 0: ethereum.eth.v1alpha1.DeprecatedBlobSidecars.sidecars:type_name -> ethereum.eth.v1alpha1.DeprecatedBlobSidecar - 1, // 1: ethereum.eth.v1alpha1.SignedBlobSidecar.message:type_name -> ethereum.eth.v1alpha1.DeprecatedBlobSidecar - 4, // 2: ethereum.eth.v1alpha1.BlindedBlobSidecars.sidecars:type_name -> ethereum.eth.v1alpha1.BlindedBlobSidecar - 4, // 3: ethereum.eth.v1alpha1.SignedBlindedBlobSidecar.message:type_name -> ethereum.eth.v1alpha1.BlindedBlobSidecar - 4, // [4:4] is the sub-list for method output_type - 4, // [4:4] is the sub-list for method input_type - 4, // [4:4] is the sub-list for extension type_name - 4, // [4:4] is the sub-list for extension extendee - 0, // [0:4] is the sub-list for field type_name + 1, // 0: ethereum.eth.v1alpha1.BlindedBlobSidecars.sidecars:type_name -> ethereum.eth.v1alpha1.BlindedBlobSidecar + 1, // 1: ethereum.eth.v1alpha1.SignedBlindedBlobSidecar.message:type_name -> ethereum.eth.v1alpha1.BlindedBlobSidecar + 2, // [2:2] is the sub-list for method output_type + 2, // [2:2] is the sub-list for method input_type + 2, // [2:2] is the sub-list for extension type_name + 2, // [2:2] is the sub-list for extension extendee + 0, // [0:2] is the sub-list for field type_name } func init() { file_proto_prysm_v1alpha1_blobs_proto_init() } @@ -642,42 +389,6 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() { } if !protoimpl.UnsafeEnabled { file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeprecatedBlobSidecars); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeprecatedBlobSidecar); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SignedBlobSidecar); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BlindedBlobSidecars); i { case 0: return &v.state @@ -689,7 +400,7 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() { return nil } } - file_proto_prysm_v1alpha1_blobs_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BlindedBlobSidecar); i { case 0: return &v.state @@ -701,7 +412,7 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() { return nil } } - file_proto_prysm_v1alpha1_blobs_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SignedBlindedBlobSidecar); i { case 0: return &v.state @@ -713,7 +424,7 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() { return nil } } - file_proto_prysm_v1alpha1_blobs_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BlobIdentifier); i { case 0: return &v.state @@ -732,7 +443,7 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_proto_prysm_v1alpha1_blobs_proto_rawDesc, NumEnums: 0, - NumMessages: 7, + NumMessages: 4, NumExtensions: 0, NumServices: 0, }, diff --git a/proto/prysm/v1alpha1/blobs.proto b/proto/prysm/v1alpha1/blobs.proto index e198f636d..014dbd3b7 100644 --- a/proto/prysm/v1alpha1/blobs.proto +++ b/proto/prysm/v1alpha1/blobs.proto @@ -24,26 +24,6 @@ option java_outer_classname = "BlobsProto"; option java_package = "org.ethereum.eth.v1alpha1"; option php_namespace = "Ethereum\\Eth\\v1alpha1"; -message DeprecatedBlobSidecars { - repeated DeprecatedBlobSidecar sidecars = 1; -} - -message DeprecatedBlobSidecar { - bytes block_root = 1 [(ethereum.eth.ext.ssz_size) = "32"]; - uint64 index = 2; - uint64 slot = 3 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives.Slot"]; - bytes block_parent_root = 4 [(ethereum.eth.ext.ssz_size) = "32"]; - uint64 proposer_index = 5 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives.ValidatorIndex"]; - bytes blob = 6 [(ethereum.eth.ext.ssz_size) = "blob.size"]; - bytes kzg_commitment = 7 [(ethereum.eth.ext.ssz_size) = "48"]; - bytes kzg_proof = 8 [(ethereum.eth.ext.ssz_size) = "48"]; -} - -message SignedBlobSidecar { - DeprecatedBlobSidecar message = 1; - bytes signature = 2 [(ethereum.eth.ext.ssz_size) = "96"]; -} - message BlindedBlobSidecars { repeated BlindedBlobSidecar sidecars = 1 [(ethereum.eth.ext.ssz_max) = "max_blobs_per_block.size"]; } diff --git a/proto/prysm/v1alpha1/generated.ssz.go b/proto/prysm/v1alpha1/generated.ssz.go index e29e48489..3f286e1c9 100644 --- a/proto/prysm/v1alpha1/generated.ssz.go +++ b/proto/prysm/v1alpha1/generated.ssz.go @@ -1,5 +1,5 @@ // Code generated by fastssz. DO NOT EDIT. -// Hash: 3f430aea0c076f79689535373e221ae26a7f8d581d9ab214f15d4fd3a761a82c +// Hash: 28837d1c80ab7592d4877ebbd9881adf756b7da77c5614263c8442c2b3dea5c2 package eth import ( @@ -15047,266 +15047,6 @@ func (h *HistoricalSummary) HashTreeRootWith(hh *ssz.Hasher) (err error) { return } -// MarshalSSZ ssz marshals the DeprecatedBlobSidecar object -func (d *DeprecatedBlobSidecar) MarshalSSZ() ([]byte, error) { - return ssz.MarshalSSZ(d) -} - -// MarshalSSZTo ssz marshals the DeprecatedBlobSidecar object to a target array -func (d *DeprecatedBlobSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) { - dst = buf - - // Field (0) 'BlockRoot' - if size := len(d.BlockRoot); size != 32 { - err = ssz.ErrBytesLengthFn("--.BlockRoot", size, 32) - return - } - dst = append(dst, d.BlockRoot...) - - // Field (1) 'Index' - dst = ssz.MarshalUint64(dst, d.Index) - - // Field (2) 'Slot' - dst = ssz.MarshalUint64(dst, uint64(d.Slot)) - - // Field (3) 'BlockParentRoot' - if size := len(d.BlockParentRoot); size != 32 { - err = ssz.ErrBytesLengthFn("--.BlockParentRoot", size, 32) - return - } - dst = append(dst, d.BlockParentRoot...) - - // Field (4) 'ProposerIndex' - dst = ssz.MarshalUint64(dst, uint64(d.ProposerIndex)) - - // Field (5) 'Blob' - if size := len(d.Blob); size != 131072 { - err = ssz.ErrBytesLengthFn("--.Blob", size, 131072) - return - } - dst = append(dst, d.Blob...) - - // Field (6) 'KzgCommitment' - if size := len(d.KzgCommitment); size != 48 { - err = ssz.ErrBytesLengthFn("--.KzgCommitment", size, 48) - return - } - dst = append(dst, d.KzgCommitment...) - - // Field (7) 'KzgProof' - if size := len(d.KzgProof); size != 48 { - err = ssz.ErrBytesLengthFn("--.KzgProof", size, 48) - return - } - dst = append(dst, d.KzgProof...) - - return -} - -// UnmarshalSSZ ssz unmarshals the DeprecatedBlobSidecar object -func (d *DeprecatedBlobSidecar) UnmarshalSSZ(buf []byte) error { - var err error - size := uint64(len(buf)) - if size != 131256 { - return ssz.ErrSize - } - - // Field (0) 'BlockRoot' - if cap(d.BlockRoot) == 0 { - d.BlockRoot = make([]byte, 0, len(buf[0:32])) - } - d.BlockRoot = append(d.BlockRoot, buf[0:32]...) - - // Field (1) 'Index' - d.Index = ssz.UnmarshallUint64(buf[32:40]) - - // Field (2) 'Slot' - d.Slot = github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[40:48])) - - // Field (3) 'BlockParentRoot' - if cap(d.BlockParentRoot) == 0 { - d.BlockParentRoot = make([]byte, 0, len(buf[48:80])) - } - d.BlockParentRoot = append(d.BlockParentRoot, buf[48:80]...) - - // Field (4) 'ProposerIndex' - d.ProposerIndex = github_com_prysmaticlabs_prysm_v4_consensus_types_primitives.ValidatorIndex(ssz.UnmarshallUint64(buf[80:88])) - - // Field (5) 'Blob' - if cap(d.Blob) == 0 { - d.Blob = make([]byte, 0, len(buf[88:131160])) - } - d.Blob = append(d.Blob, buf[88:131160]...) - - // Field (6) 'KzgCommitment' - if cap(d.KzgCommitment) == 0 { - d.KzgCommitment = make([]byte, 0, len(buf[131160:131208])) - } - d.KzgCommitment = append(d.KzgCommitment, buf[131160:131208]...) - - // Field (7) 'KzgProof' - if cap(d.KzgProof) == 0 { - d.KzgProof = make([]byte, 0, len(buf[131208:131256])) - } - d.KzgProof = append(d.KzgProof, buf[131208:131256]...) - - return err -} - -// SizeSSZ returns the ssz encoded size in bytes for the DeprecatedBlobSidecar object -func (d *DeprecatedBlobSidecar) SizeSSZ() (size int) { - size = 131256 - return -} - -// HashTreeRoot ssz hashes the DeprecatedBlobSidecar object -func (d *DeprecatedBlobSidecar) HashTreeRoot() ([32]byte, error) { - return ssz.HashWithDefaultHasher(d) -} - -// HashTreeRootWith ssz hashes the DeprecatedBlobSidecar object with a hasher -func (d *DeprecatedBlobSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) { - indx := hh.Index() - - // Field (0) 'BlockRoot' - if size := len(d.BlockRoot); size != 32 { - err = ssz.ErrBytesLengthFn("--.BlockRoot", size, 32) - return - } - hh.PutBytes(d.BlockRoot) - - // Field (1) 'Index' - hh.PutUint64(d.Index) - - // Field (2) 'Slot' - hh.PutUint64(uint64(d.Slot)) - - // Field (3) 'BlockParentRoot' - if size := len(d.BlockParentRoot); size != 32 { - err = ssz.ErrBytesLengthFn("--.BlockParentRoot", size, 32) - return - } - hh.PutBytes(d.BlockParentRoot) - - // Field (4) 'ProposerIndex' - hh.PutUint64(uint64(d.ProposerIndex)) - - // Field (5) 'Blob' - if size := len(d.Blob); size != 131072 { - err = ssz.ErrBytesLengthFn("--.Blob", size, 131072) - return - } - hh.PutBytes(d.Blob) - - // Field (6) 'KzgCommitment' - if size := len(d.KzgCommitment); size != 48 { - err = ssz.ErrBytesLengthFn("--.KzgCommitment", size, 48) - return - } - hh.PutBytes(d.KzgCommitment) - - // Field (7) 'KzgProof' - if size := len(d.KzgProof); size != 48 { - err = ssz.ErrBytesLengthFn("--.KzgProof", size, 48) - return - } - hh.PutBytes(d.KzgProof) - - if ssz.EnableVectorizedHTR { - hh.MerkleizeVectorizedHTR(indx) - } else { - hh.Merkleize(indx) - } - return -} - -// MarshalSSZ ssz marshals the SignedBlobSidecar object -func (s *SignedBlobSidecar) MarshalSSZ() ([]byte, error) { - return ssz.MarshalSSZ(s) -} - -// MarshalSSZTo ssz marshals the SignedBlobSidecar object to a target array -func (s *SignedBlobSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) { - dst = buf - - // Field (0) 'Message' - if s.Message == nil { - s.Message = new(DeprecatedBlobSidecar) - } - if dst, err = s.Message.MarshalSSZTo(dst); err != nil { - return - } - - // Field (1) 'Signature' - if size := len(s.Signature); size != 96 { - err = ssz.ErrBytesLengthFn("--.Signature", size, 96) - return - } - dst = append(dst, s.Signature...) - - return -} - -// UnmarshalSSZ ssz unmarshals the SignedBlobSidecar object -func (s *SignedBlobSidecar) UnmarshalSSZ(buf []byte) error { - var err error - size := uint64(len(buf)) - if size != 131352 { - return ssz.ErrSize - } - - // Field (0) 'Message' - if s.Message == nil { - s.Message = new(DeprecatedBlobSidecar) - } - if err = s.Message.UnmarshalSSZ(buf[0:131256]); err != nil { - return err - } - - // Field (1) 'Signature' - if cap(s.Signature) == 0 { - s.Signature = make([]byte, 0, len(buf[131256:131352])) - } - s.Signature = append(s.Signature, buf[131256:131352]...) - - return err -} - -// SizeSSZ returns the ssz encoded size in bytes for the SignedBlobSidecar object -func (s *SignedBlobSidecar) SizeSSZ() (size int) { - size = 131352 - return -} - -// HashTreeRoot ssz hashes the SignedBlobSidecar object -func (s *SignedBlobSidecar) HashTreeRoot() ([32]byte, error) { - return ssz.HashWithDefaultHasher(s) -} - -// HashTreeRootWith ssz hashes the SignedBlobSidecar object with a hasher -func (s *SignedBlobSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) { - indx := hh.Index() - - // Field (0) 'Message' - if err = s.Message.HashTreeRootWith(hh); err != nil { - return - } - - // Field (1) 'Signature' - if size := len(s.Signature); size != 96 { - err = ssz.ErrBytesLengthFn("--.Signature", size, 96) - return - } - hh.PutBytes(s.Signature) - - if ssz.EnableVectorizedHTR { - hh.MerkleizeVectorizedHTR(indx) - } else { - hh.Merkleize(indx) - } - return -} - // MarshalSSZ ssz marshals the BlobIdentifier object func (b *BlobIdentifier) MarshalSSZ() ([]byte, error) { return ssz.MarshalSSZ(b) diff --git a/testing/spectest/general/deneb/kzg/BUILD.bazel b/testing/spectest/general/deneb/kzg/BUILD.bazel index b513dd394..90b3f91dd 100644 --- a/testing/spectest/general/deneb/kzg/BUILD.bazel +++ b/testing/spectest/general/deneb/kzg/BUILD.bazel @@ -10,6 +10,7 @@ go_test( tags = ["spectest"], deps = [ "//beacon-chain/blockchain/kzg:go_default_library", + "//consensus-types/blocks:go_default_library", "//proto/prysm/v1alpha1:go_default_library", "//testing/require:go_default_library", "//testing/spectest/utils:go_default_library", diff --git a/testing/spectest/general/deneb/kzg/verify_blob_kzg_proof_batch_test.go b/testing/spectest/general/deneb/kzg/verify_blob_kzg_proof_batch_test.go index 938768da6..2c9b5283b 100644 --- a/testing/spectest/general/deneb/kzg/verify_blob_kzg_proof_batch_test.go +++ b/testing/spectest/general/deneb/kzg/verify_blob_kzg_proof_batch_test.go @@ -7,6 +7,7 @@ import ( "github.com/ghodss/yaml" kzgPrysm "github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain/kzg" + "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks" ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1" "github.com/prysmaticlabs/prysm/v4/testing/require" "github.com/prysmaticlabs/prysm/v4/testing/spectest/utils" @@ -36,37 +37,40 @@ func TestVerifyBlobKZGProofBatch(t *testing.T) { require.NoError(t, err) test := &KZGTestData{} require.NoError(t, yaml.Unmarshal(file, test)) - var sidecars []*ethpb.DeprecatedBlobSidecar + var sidecars []blocks.ROBlob blobs := test.Input.Blobs - commitments := test.Input.Commitments proofs := test.Input.Proofs + kzgs := test.Input.Commitments if len(proofs) != len(blobs) { require.Equal(t, false, test.Output) return } - var kzgs [][]byte - // Need separate loops to test length checks in - // `IsDataAvailable` + if len(kzgs) != len(blobs) { + require.Equal(t, false, test.Output) + return + } + for i, blob := range blobs { blobBytes, err := hex.DecodeString(blob[2:]) require.NoError(t, err) proofBytes, err := hex.DecodeString(proofs[i][2:]) require.NoError(t, err) - sidecar := ðpb.DeprecatedBlobSidecar{ - Blob: blobBytes, - KzgProof: proofBytes, - } - sidecars = append(sidecars, sidecar) - } - for _, commitment := range commitments { - commitmentBytes, err := hex.DecodeString(commitment[2:]) + kzgBytes, err := hex.DecodeString(kzgs[i][2:]) require.NoError(t, err) - kzgs = append(kzgs, commitmentBytes) + sidecar := ðpb.BlobSidecar{ + Blob: blobBytes, + KzgProof: proofBytes, + KzgCommitment: kzgBytes, + } + sidecar.SignedBlockHeader = util.HydrateSignedBeaconHeader(ðpb.SignedBeaconBlockHeader{}) + sc, err := blocks.NewROBlob(sidecar) + require.NoError(t, err) + sidecars = append(sidecars, sc) } if test.Output { - require.NoError(t, kzgPrysm.IsDataAvailable(kzgs, sidecars)) + require.NoError(t, kzgPrysm.Verify(sidecars...)) } else { - require.NotNil(t, kzgPrysm.IsDataAvailable(kzgs, sidecars)) + require.NotNil(t, kzgPrysm.Verify(sidecars...)) } }) } diff --git a/testing/spectest/shared/deneb/ssz_static/ssz_static.go b/testing/spectest/shared/deneb/ssz_static/ssz_static.go index 91a0000f9..0a30b7983 100644 --- a/testing/spectest/shared/deneb/ssz_static/ssz_static.go +++ b/testing/spectest/shared/deneb/ssz_static/ssz_static.go @@ -131,8 +131,6 @@ func UnmarshalledSSZ(t *testing.T, serializedBytes []byte, folderName string) (i obj = ðpb.BlobIdentifier{} case "BlobSidecar": obj = ðpb.BlobSidecar{} - case "SignedBlobSidecar": - obj = ðpb.SignedBlobSidecar{} case "PowBlock": obj = ðpb.PowBlock{} case "Withdrawal":