Checkpoint Sync 2/5 - API support for retrieving weak subjectivity data (#10384)

also refactor replayer code for better reuse:
- separately expose stategen's canonical block func
- CanonicalHistory in ws api, NewCanonicalBuilder
- refactor CanonicalHistory into a ReplayerBuilder

Co-authored-by: Kasey Kirkham <kasey@users.noreply.github.com>
This commit is contained in:
kasey
2022-03-23 17:54:07 -05:00
committed by GitHub
parent 3ad6abd9c0
commit 49826ebe28
47 changed files with 2139 additions and 2383 deletions

View File

@@ -15,29 +15,12 @@ go_library(
"weak_subjectivity.go",
],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/core/helpers",
visibility = [
"//beacon-chain:__subpackages__",
"//cmd/beacon-chain:__subpackages__",
"//contracts/deposit:__pkg__",
"//crypto/keystore:__pkg__",
"//network/forks:__pkg__",
"//proto/prysm/v1alpha1:__subpackages__",
"//proto/prysm/v1alpha1/attestation:__pkg__",
"//runtime/interop:__pkg__",
"//slasher:__subpackages__",
"//testing/altair:__pkg__",
"//testing/benchmark/benchmark_files:__subpackages__",
"//testing/endtoend/evaluators:__pkg__",
"//testing/slasher/simulator:__pkg__",
"//testing/spectest:__subpackages__",
"//testing/util:__pkg__",
"//tools:__subpackages__",
"//validator:__subpackages__",
],
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/cache:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/state:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//container/slice:go_default_library",
"//container/trie:go_default_library",

View File

@@ -1,7 +1,6 @@
package helpers
import (
"bytes"
"context"
"encoding/hex"
"errors"
@@ -12,9 +11,11 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/math"
eth "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
v1alpha1 "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/time/slots"
)
@@ -122,20 +123,20 @@ func ComputeWeakSubjectivityPeriod(ctx context.Context, st state.ReadOnlyBeaconS
// current_epoch = compute_epoch_at_slot(get_current_slot(store))
// return current_epoch <= ws_state_epoch + ws_period
func IsWithinWeakSubjectivityPeriod(
ctx context.Context, currentEpoch types.Epoch, wsState state.ReadOnlyBeaconState, wsCheckpoint *eth.WeakSubjectivityCheckpoint) (bool, error) {
ctx context.Context, currentEpoch types.Epoch, wsState state.ReadOnlyBeaconState, wsStateRoot [fieldparams.RootLength]byte, wsEpoch types.Epoch) (bool, error) {
// Make sure that incoming objects are not nil.
if wsState == nil || wsState.IsNil() || wsState.LatestBlockHeader() == nil || wsCheckpoint == nil {
if wsState == nil || wsState.IsNil() || wsState.LatestBlockHeader() == nil {
return false, errors.New("invalid weak subjectivity state or checkpoint")
}
// Assert that state and checkpoint have the same root and epoch.
if !bytes.Equal(wsState.LatestBlockHeader().StateRoot, wsCheckpoint.StateRoot) {
if bytesutil.ToBytes32(wsState.LatestBlockHeader().StateRoot) != wsStateRoot {
return false, fmt.Errorf("state (%#x) and checkpoint (%#x) roots do not match",
wsState.LatestBlockHeader().StateRoot, wsCheckpoint.StateRoot)
wsState.LatestBlockHeader().StateRoot, wsStateRoot)
}
if slots.ToEpoch(wsState.Slot()) != wsCheckpoint.Epoch {
if slots.ToEpoch(wsState.Slot()) != wsEpoch {
return false, fmt.Errorf("state (%v) and checkpoint (%v) epochs do not match",
slots.ToEpoch(wsState.Slot()), wsCheckpoint.Epoch)
slots.ToEpoch(wsState.Slot()), wsEpoch)
}
// Compare given epoch to state epoch + weak subjectivity period.
@@ -164,7 +165,7 @@ func LatestWeakSubjectivityEpoch(ctx context.Context, st state.ReadOnlyBeaconSta
}
// ParseWeakSubjectivityInputString parses "blocks_root:epoch_number" string into a checkpoint.
func ParseWeakSubjectivityInputString(wsCheckpointString string) (*eth.Checkpoint, error) {
func ParseWeakSubjectivityInputString(wsCheckpointString string) (*v1alpha1.Checkpoint, error) {
if wsCheckpointString == "" {
return nil, nil
}
@@ -197,7 +198,7 @@ func ParseWeakSubjectivityInputString(wsCheckpointString string) (*eth.Checkpoin
return nil, err
}
return &eth.Checkpoint{
return &v1alpha1.Checkpoint{
Epoch: types.Epoch(epoch),
Root: bRoot,
}, nil

View File

@@ -54,12 +54,15 @@ func TestWeakSubjectivity_ComputeWeakSubjectivityPeriod(t *testing.T) {
})
}
}
type mockWsCheckpoint func() (stateRoot [32]byte, blockRoot [32]byte, e types.Epoch)
func TestWeakSubjectivity_IsWithinWeakSubjectivityPeriod(t *testing.T) {
tests := []struct {
name string
epoch types.Epoch
genWsState func() state.ReadOnlyBeaconState
genWsCheckpoint func() *ethpb.WeakSubjectivityCheckpoint
genWsCheckpoint mockWsCheckpoint
want bool
wantedErr string
}{
@@ -68,22 +71,8 @@ func TestWeakSubjectivity_IsWithinWeakSubjectivityPeriod(t *testing.T) {
genWsState: func() state.ReadOnlyBeaconState {
return nil
},
genWsCheckpoint: func() *ethpb.WeakSubjectivityCheckpoint {
return &ethpb.WeakSubjectivityCheckpoint{
BlockRoot: make([]byte, 32),
StateRoot: make([]byte, 32),
Epoch: 42,
}
},
wantedErr: "invalid weak subjectivity state or checkpoint",
},
{
name: "nil weak subjectivity checkpoint",
genWsState: func() state.ReadOnlyBeaconState {
return genState(t, 128, 32)
},
genWsCheckpoint: func() *ethpb.WeakSubjectivityCheckpoint {
return nil
genWsCheckpoint: func() ([32]byte, [32]byte, types.Epoch) {
return [32]byte{}, [32]byte{}, 42
},
wantedErr: "invalid weak subjectivity state or checkpoint",
},
@@ -99,11 +88,10 @@ func TestWeakSubjectivity_IsWithinWeakSubjectivityPeriod(t *testing.T) {
require.NoError(t, err)
return beaconState
},
genWsCheckpoint: func() *ethpb.WeakSubjectivityCheckpoint {
return &ethpb.WeakSubjectivityCheckpoint{
StateRoot: bytesutil.PadTo([]byte("stateroot2"), 32),
Epoch: 42,
}
genWsCheckpoint: func() ([32]byte, [32]byte, types.Epoch) {
var sr [32]byte
copy(sr[:], bytesutil.PadTo([]byte("stateroot2"), 32))
return sr, [32]byte{}, 42
},
wantedErr: fmt.Sprintf("state (%#x) and checkpoint (%#x) roots do not match",
bytesutil.PadTo([]byte("stateroot1"), 32), bytesutil.PadTo([]byte("stateroot2"), 32)),
@@ -120,11 +108,10 @@ func TestWeakSubjectivity_IsWithinWeakSubjectivityPeriod(t *testing.T) {
require.NoError(t, err)
return beaconState
},
genWsCheckpoint: func() *ethpb.WeakSubjectivityCheckpoint {
return &ethpb.WeakSubjectivityCheckpoint{
StateRoot: bytesutil.PadTo([]byte("stateroot"), 32),
Epoch: 43,
}
genWsCheckpoint: func() ([32]byte, [32]byte, types.Epoch) {
var sr [32]byte
copy(sr[:], bytesutil.PadTo([]byte("stateroot"), 32))
return sr, [32]byte{}, 43
},
wantedErr: "state (42) and checkpoint (43) epochs do not match",
},
@@ -140,11 +127,10 @@ func TestWeakSubjectivity_IsWithinWeakSubjectivityPeriod(t *testing.T) {
require.NoError(t, err)
return beaconState
},
genWsCheckpoint: func() *ethpb.WeakSubjectivityCheckpoint {
return &ethpb.WeakSubjectivityCheckpoint{
StateRoot: bytesutil.PadTo([]byte("stateroot"), 32),
Epoch: 42,
}
genWsCheckpoint: func() ([32]byte, [32]byte, types.Epoch) {
var sr [32]byte
copy(sr[:], bytesutil.PadTo([]byte("stateroot"), 32))
return sr, [32]byte{}, 42
},
wantedErr: "cannot compute weak subjectivity period: no active validators found",
},
@@ -161,11 +147,10 @@ func TestWeakSubjectivity_IsWithinWeakSubjectivityPeriod(t *testing.T) {
require.NoError(t, err)
return beaconState
},
genWsCheckpoint: func() *ethpb.WeakSubjectivityCheckpoint {
return &ethpb.WeakSubjectivityCheckpoint{
StateRoot: bytesutil.PadTo([]byte("stateroot"), 32),
Epoch: 42,
}
genWsCheckpoint: func() ([32]byte, [32]byte, types.Epoch) {
var sr [32]byte
copy(sr[:], bytesutil.PadTo([]byte("stateroot"), 32))
return sr, [32]byte{}, 42
},
want: false,
},
@@ -182,18 +167,18 @@ func TestWeakSubjectivity_IsWithinWeakSubjectivityPeriod(t *testing.T) {
require.NoError(t, err)
return beaconState
},
genWsCheckpoint: func() *ethpb.WeakSubjectivityCheckpoint {
return &ethpb.WeakSubjectivityCheckpoint{
StateRoot: bytesutil.PadTo([]byte("stateroot"), 32),
Epoch: 42,
}
genWsCheckpoint: func() ([32]byte, [32]byte, types.Epoch) {
var sr [32]byte
copy(sr[:], bytesutil.PadTo([]byte("stateroot"), 32))
return sr, [32]byte{}, 42
},
want: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := helpers.IsWithinWeakSubjectivityPeriod(context.Background(), tt.epoch, tt.genWsState(), tt.genWsCheckpoint())
sr, _, e := tt.genWsCheckpoint()
got, err := helpers.IsWithinWeakSubjectivityPeriod(context.Background(), tt.epoch, tt.genWsState(), sr, e)
if tt.wantedErr != "" {
assert.Equal(t, false, got)
assert.ErrorContains(t, tt.wantedErr, err)

View File

@@ -10,7 +10,7 @@ go_library(
"structs_marshalling.go",
],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/rpc/apimiddleware",
visibility = ["//beacon-chain:__subpackages__"],
visibility = ["//visibility:public"],
deps = [
"//api/gateway/apimiddleware:go_default_library",
"//api/grpc:go_default_library",

View File

@@ -37,6 +37,7 @@ func (_ *BeaconEndpointFactory) Paths() []string {
"/eth/v1/beacon/pool/proposer_slashings",
"/eth/v1/beacon/pool/voluntary_exits",
"/eth/v1/beacon/pool/sync_committees",
"/eth/v1/beacon/weak_subjectivity",
"/eth/v1/node/identity",
"/eth/v1/node/peers",
"/eth/v1/node/peers/{peer_id}",
@@ -142,6 +143,8 @@ func (_ *BeaconEndpointFactory) Create(path string) (*apimiddleware.Endpoint, er
endpoint.Hooks = apimiddleware.HookCollection{
OnPreDeserializeRequestBodyIntoContainer: wrapSyncCommitteeSignaturesArray,
}
case "/eth/v1/beacon/weak_subjectivity":
endpoint.GetResponse = &WeakSubjectivityResponse{}
case "/eth/v1/node/identity":
endpoint.GetResponse = &identityResponseJson{}
case "/eth/v1/node/peers":

View File

@@ -19,6 +19,15 @@ type genesisResponse_GenesisJson struct {
GenesisForkVersion string `json:"genesis_fork_version" hex:"true"`
}
// WeakSubjectivityResponse is used to marshal/unmarshal the response for the
// /eth/v1/beacon/weak_subjectivity endpoint.
type WeakSubjectivityResponse struct {
Data *struct {
Checkpoint *checkpointJson `json:"ws_checkpoint"`
StateRoot string `json:"state_root" hex:"true"`
} `json:"data"`
}
// feeRecipientsRequestJson is used in /validator/prepare_beacon_proposers API endpoint.
type feeRecipientsRequestJSON struct {
Recipients []*feeRecipientJson `json:"recipients"`

View File

@@ -35,6 +35,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/state/v1:go_default_library",
"//beacon-chain/sync:go_default_library",
"//config/features:go_default_library",
"//config/params:go_default_library",
"//crypto/bls:go_default_library",

View File

@@ -5,18 +5,21 @@ import (
"fmt"
"strconv"
"github.com/golang/protobuf/ptypes/empty"
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed"
blockfeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/block"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/db/filters"
rpchelpers "github.com/prysmaticlabs/prysm/beacon-chain/rpc/eth/helpers"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpbv1 "github.com/prysmaticlabs/prysm/proto/eth/v1"
ethpbv2 "github.com/prysmaticlabs/prysm/proto/eth/v2"
"github.com/prysmaticlabs/prysm/proto/migration"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/wrapper"
"github.com/prysmaticlabs/prysm/time/slots"
"go.opencensus.io/trace"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
@@ -40,6 +43,43 @@ func (e *blockIdParseError) Error() string {
return e.message
}
// GetWeakSubjectivity computes the starting epoch of the current weak subjectivity period, and then also
// determines the best block root and state root to use for a Checkpoint Sync starting from that point.
func (bs *Server) GetWeakSubjectivity(ctx context.Context, _ *empty.Empty) (*ethpbv1.WeakSubjectivityResponse, error) {
if err := rpchelpers.ValidateSync(ctx, bs.SyncChecker, bs.HeadFetcher, bs.GenesisTimeFetcher); err != nil {
// This is already a grpc error, so we can't wrap it any further
return nil, err
}
hs, err := bs.HeadFetcher.HeadState(ctx)
if err != nil {
return nil, status.Error(codes.Internal, "could not get head state")
}
wsEpoch, err := helpers.LatestWeakSubjectivityEpoch(ctx, hs)
if err != nil {
return nil, status.Errorf(codes.Internal, "could not get weak subjectivity epoch: %v", err)
}
wsSlot, err := slots.EpochStart(wsEpoch)
if err != nil {
return nil, status.Errorf(codes.Internal, "could not get weak subjectivity slot: %v", err)
}
cbr, cb, err := bs.CanonicalHistory.BlockForSlot(ctx, wsSlot)
if err != nil {
return nil, status.Errorf(codes.Internal, fmt.Sprintf("could not find highest block below slot %d", wsSlot))
}
stateRoot := bytesutil.ToBytes32(cb.Block().StateRoot())
log.Printf("weak subjectivity checkpoint reported as epoch=%d, block root=%#x, state root=%#x", wsEpoch, cbr, stateRoot)
return &ethpbv1.WeakSubjectivityResponse{
Data: &ethpbv1.WeakSubjectivityData{
WsCheckpoint: &ethpbv1.Checkpoint{
Epoch: wsEpoch,
Root: cbr[:],
},
StateRoot: stateRoot[:],
},
}, nil
}
// GetBlockHeader retrieves block header for given block id.
func (bs *Server) GetBlockHeader(ctx context.Context, req *ethpbv1.BlockRequest) (*ethpbv1.BlockHeaderResponse, error) {
ctx, span := trace.StartSpan(ctx, "beacon.GetBlockHeader")
@@ -48,7 +88,7 @@ func (bs *Server) GetBlockHeader(ctx context.Context, req *ethpbv1.BlockRequest)
blk, err := bs.blockFromBlockID(ctx, req.BlockId)
err = handleGetBlockError(blk, err)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "GetBlockHeader")
}
v1alpha1Header, err := blk.Header()
if err != nil {
@@ -214,7 +254,7 @@ func (bs *Server) GetBlock(ctx context.Context, req *ethpbv1.BlockRequest) (*eth
blk, err := bs.blockFromBlockID(ctx, req.BlockId)
err = handleGetBlockError(blk, err)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "GetBlock")
}
signedBeaconBlock, err := migration.SignedBeaconBlock(blk)
if err != nil {
@@ -237,7 +277,7 @@ func (bs *Server) GetBlockSSZ(ctx context.Context, req *ethpbv1.BlockRequest) (*
blk, err := bs.blockFromBlockID(ctx, req.BlockId)
err = handleGetBlockError(blk, err)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "GetBlockSSZ")
}
signedBeaconBlock, err := migration.SignedBeaconBlock(blk)
if err != nil {
@@ -259,7 +299,7 @@ func (bs *Server) GetBlockV2(ctx context.Context, req *ethpbv2.BlockRequestV2) (
blk, err := bs.blockFromBlockID(ctx, req.BlockId)
err = handleGetBlockError(blk, err)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "GetBlockV2")
}
_, err = blk.PbPhase0Block()
@@ -336,7 +376,7 @@ func (bs *Server) GetBlockSSZV2(ctx context.Context, req *ethpbv2.BlockRequestV2
blk, err := bs.blockFromBlockID(ctx, req.BlockId)
err = handleGetBlockError(blk, err)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "GetBlockSSZV2")
}
_, err = blk.PbPhase0Block()

View File

@@ -15,6 +15,7 @@ import (
v1alpha1validator "github.com/prysmaticlabs/prysm/beacon-chain/rpc/prysm/v1alpha1/validator"
"github.com/prysmaticlabs/prysm/beacon-chain/rpc/statefetcher"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/beacon-chain/sync"
)
// Server defines a server implementation of the gRPC Beacon Chain service,
@@ -34,4 +35,6 @@ type Server struct {
StateFetcher statefetcher.Fetcher
HeadFetcher blockchain.HeadFetcher
V1Alpha1ValidatorServer *v1alpha1validator.Server
SyncChecker sync.Checker
CanonicalHistory *stategen.CanonicalHistory
}

View File

@@ -117,12 +117,7 @@ func (bs *Server) GetFinalityCheckpoints(ctx context.Context, req *ethpb.StateRe
st, err = bs.StateFetcher.State(ctx, req.StateId)
if err != nil {
if stateNotFoundErr, ok := err.(*statefetcher.StateNotFoundError); ok {
return nil, status.Errorf(codes.NotFound, "State not found: %v", stateNotFoundErr)
} else if parseErr, ok := err.(*statefetcher.StateIdParseError); ok {
return nil, status.Errorf(codes.InvalidArgument, "Invalid state ID: %v", parseErr)
}
return nil, status.Errorf(codes.Internal, "Could not get state: %v", err)
return nil, helpers.PrepareStateFetchGRPCError(err)
}
return &ethpb.StateFinalityCheckpointResponse{

View File

@@ -14,6 +14,7 @@ go_library(
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/rpc/statefetcher:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync:go_default_library",
"//config/params:go_default_library",
"//proto/eth/v1:go_default_library",

View File

@@ -1,7 +1,10 @@
package helpers
import (
"errors"
"github.com/prysmaticlabs/prysm/beacon-chain/rpc/statefetcher"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
@@ -9,9 +12,13 @@ import (
// PrepareStateFetchGRPCError returns an appropriate gRPC error based on the supplied argument.
// The argument error should be a result of fetching state.
func PrepareStateFetchGRPCError(err error) error {
if errors.Is(err, stategen.ErrSlotBeforeOrigin) {
return status.Errorf(codes.NotFound, "lacking historical data needed to fulfill request")
}
if stateNotFoundErr, ok := err.(*statefetcher.StateNotFoundError); ok {
return status.Errorf(codes.NotFound, "State not found: %v", stateNotFoundErr)
} else if parseErr, ok := err.(*statefetcher.StateIdParseError); ok {
}
if parseErr, ok := err.(*statefetcher.StateIdParseError); ok {
return status.Errorf(codes.InvalidArgument, "Invalid state ID: %v", parseErr)
}
return status.Errorf(codes.Internal, "Invalid state ID: %v", err)

View File

@@ -59,7 +59,7 @@ func (bs *Server) ListValidatorAssignments(
if err != nil {
return nil, err
}
requestedState, err := bs.ReplayerBuilder.ForSlot(startSlot).ReplayBlocks(ctx)
requestedState, err := bs.ReplayerBuilder.ReplayerForSlot(startSlot).ReplayBlocks(ctx)
if err != nil {
msg := fmt.Sprintf("could not replay all blocks from the closest stored state (at slot %d) "+
"to the requested epoch (%d) - %v", startSlot, requestedEpoch, err)

View File

@@ -2,7 +2,6 @@ package beacon
import (
"context"
"fmt"
"strconv"
"github.com/pkg/errors"
@@ -394,46 +393,43 @@ func (bs *Server) chainHeadRetrieval(ctx context.Context) (*ethpb.ChainHead, err
return nil, status.Errorf(codes.Internal, "Could not get head block root: %v", err)
}
isGenesis := func(cp *ethpb.Checkpoint) bool {
return bytesutil.ToBytes32(cp.Root) == params.BeaconConfig().ZeroHash && cp.Epoch == 0
}
// Retrieve genesis block in the event we have genesis checkpoints.
genBlock, err := bs.BeaconDB.GenesisBlock(ctx)
if err != nil || genBlock == nil || genBlock.IsNil() || genBlock.Block().IsNil() {
return nil, status.Error(codes.Internal, "Could not get genesis block")
validGenesis := false
validateCP := func(cp *ethpb.Checkpoint, name string) error {
if bytesutil.ToBytes32(cp.Root) == params.BeaconConfig().ZeroHash && cp.Epoch == 0 {
if validGenesis {
return nil
}
// Retrieve genesis block in the event we have genesis checkpoints.
genBlock, err := bs.BeaconDB.GenesisBlock(ctx)
if err != nil || helpers.BeaconBlockIsNil(genBlock) != nil {
return status.Error(codes.Internal, "Could not get genesis block")
}
validGenesis = true
return nil
}
b, err := bs.BeaconDB.Block(ctx, bytesutil.ToBytes32(cp.Root))
if err != nil {
return status.Errorf(codes.Internal, "Could not get %s block: %v", name, err)
}
if err := helpers.BeaconBlockIsNil(b); err != nil {
return status.Errorf(codes.Internal, "Could not get %s block: %v", name, err)
}
return nil
}
finalizedCheckpoint := bs.FinalizationFetcher.FinalizedCheckpt()
if !isGenesis(finalizedCheckpoint) {
b, err := bs.BeaconDB.Block(ctx, bytesutil.ToBytes32(finalizedCheckpoint.Root))
if err != nil {
return nil, status.Error(codes.Internal, "Could not get finalized block")
}
if err := helpers.BeaconBlockIsNil(b); err != nil {
return nil, status.Errorf(codes.Internal, "Could not get finalized block: %v", err)
}
if err := validateCP(finalizedCheckpoint, "finalized"); err != nil {
return nil, err
}
justifiedCheckpoint := bs.FinalizationFetcher.CurrentJustifiedCheckpt()
if !isGenesis(justifiedCheckpoint) {
b, err := bs.BeaconDB.Block(ctx, bytesutil.ToBytes32(justifiedCheckpoint.Root))
if err != nil {
return nil, status.Error(codes.Internal, "Could not get justified block")
}
if err := helpers.BeaconBlockIsNil(b); err != nil {
return nil, status.Errorf(codes.Internal, "Could not get justified block: %v", err)
}
if err := validateCP(justifiedCheckpoint, "justified"); err != nil {
return nil, err
}
prevJustifiedCheckpoint := bs.FinalizationFetcher.PreviousJustifiedCheckpt()
if !isGenesis(prevJustifiedCheckpoint) {
b, err := bs.BeaconDB.Block(ctx, bytesutil.ToBytes32(prevJustifiedCheckpoint.Root))
if err != nil {
return nil, status.Error(codes.Internal, "Could not get prev justified block")
}
if err := helpers.BeaconBlockIsNil(b); err != nil {
return nil, status.Errorf(codes.Internal, "Could not get prev justified block: %v", err)
}
if err := validateCP(prevJustifiedCheckpoint, "prev justified"); err != nil {
return nil, err
}
fSlot, err := slots.EpochStart(finalizedCheckpoint.Epoch)
@@ -463,40 +459,3 @@ func (bs *Server) chainHeadRetrieval(ctx context.Context) (*ethpb.ChainHead, err
PreviousJustifiedBlockRoot: prevJustifiedCheckpoint.Root,
}, nil
}
// GetWeakSubjectivityCheckpoint retrieves weak subjectivity state root, block root, and epoch.
func (bs *Server) GetWeakSubjectivityCheckpoint(ctx context.Context, _ *emptypb.Empty) (*ethpb.WeakSubjectivityCheckpoint, error) {
hs, err := bs.HeadFetcher.HeadState(ctx)
if err != nil {
return nil, status.Error(codes.Internal, "Could not get head state")
}
wsEpoch, err := helpers.LatestWeakSubjectivityEpoch(ctx, hs)
if err != nil {
return nil, status.Error(codes.Internal, "Could not get weak subjectivity epoch")
}
wsSlot, err := slots.EpochStart(wsEpoch)
if err != nil {
return nil, status.Error(codes.Internal, "Could not get weak subjectivity slot")
}
wsState, err := bs.ReplayerBuilder.ForSlot(wsSlot).ReplayBlocks(ctx)
if err != nil {
msg := fmt.Sprintf("error replaying blocks for state at slot %d: %v", wsSlot, err)
return nil, status.Error(codes.Internal, msg)
}
stateRoot, err := wsState.HashTreeRoot(ctx)
if err != nil {
return nil, status.Error(codes.Internal, "Could not get weak subjectivity state root")
}
blkRoot, err := wsState.LatestBlockHeader().HashTreeRoot()
if err != nil {
return nil, status.Error(codes.Internal, "Could not get weak subjectivity block root")
}
return &ethpb.WeakSubjectivityCheckpoint{
BlockRoot: blkRoot[:],
StateRoot: stateRoot[:],
Epoch: wsEpoch,
}, nil
}

View File

@@ -366,6 +366,60 @@ func TestServer_ListBlocks_Errors(t *testing.T) {
assert.Equal(t, int32(0), res.TotalSize, "Wanted total size 0")
}
// ensures that if any of the checkpoints are zero-valued, an error will be generated without genesis being present
func TestServer_GetChainHead_NoGenesis(t *testing.T) {
db := dbTest.SetupDB(t)
s, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, s.SetSlot(1))
genBlock := util.NewBeaconBlock()
genBlock.Block.ParentRoot = bytesutil.PadTo([]byte{'G'}, fieldparams.RootLength)
require.NoError(t, db.SaveBlock(context.Background(), wrapper.WrappedPhase0SignedBeaconBlock(genBlock)))
gRoot, err := genBlock.Block.HashTreeRoot()
require.NoError(t, err)
cases := []struct {
name string
zeroSetter func(val *ethpb.Checkpoint) error
}{
{
name: "zero-value prev justified",
zeroSetter: s.SetPreviousJustifiedCheckpoint,
},
{
name: "zero-value current justified",
zeroSetter: s.SetCurrentJustifiedCheckpoint,
},
{
name: "zero-value finalized",
zeroSetter: s.SetFinalizedCheckpoint,
},
}
finalized := &ethpb.Checkpoint{Epoch: 1, Root: gRoot[:]}
prevJustified := &ethpb.Checkpoint{Epoch: 2, Root: gRoot[:]}
justified := &ethpb.Checkpoint{Epoch: 3, Root: gRoot[:]}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
require.NoError(t, s.SetPreviousJustifiedCheckpoint(prevJustified))
require.NoError(t, s.SetCurrentJustifiedCheckpoint(justified))
require.NoError(t, s.SetFinalizedCheckpoint(finalized))
require.NoError(t, c.zeroSetter(&ethpb.Checkpoint{Epoch: 0, Root: params.BeaconConfig().ZeroHash[:]}))
})
bs := &Server{
BeaconDB: db,
HeadFetcher: &chainMock.ChainService{Block: wrapper.WrappedPhase0SignedBeaconBlock(genBlock), State: s},
FinalizationFetcher: &chainMock.ChainService{
FinalizedCheckPoint: s.FinalizedCheckpoint(),
CurrentJustifiedCheckPoint: s.CurrentJustifiedCheckpoint(),
PreviousJustifiedCheckPoint: s.PreviousJustifiedCheckpoint()},
}
_, err = bs.GetChainHead(context.Background(), nil)
require.ErrorContains(t, "Could not get genesis block", err)
}
}
func TestServer_GetChainHead_NoFinalizedBlock(t *testing.T) {
db := dbTest.SetupDB(t)

View File

@@ -74,7 +74,7 @@ func (bs *Server) retrieveCommitteesForEpoch(
if err != nil {
return nil, nil, err
}
requestedState, err := bs.ReplayerBuilder.ForSlot(startSlot).ReplayBlocks(ctx)
requestedState, err := bs.ReplayerBuilder.ReplayerForSlot(startSlot).ReplayBlocks(ctx)
if err != nil {
return nil, nil, status.Errorf(codes.Internal, "error replaying blocks for state at slot %d: %v", startSlot, err)
}

View File

@@ -76,10 +76,9 @@ func TestServer_ListBeaconCommittees_CurrentEpoch(t *testing.T) {
func addDefaultReplayerBuilder(s *Server, h stategen.HistoryAccessor) {
cc := &mockstategen.MockCanonicalChecker{Is: true, Err: nil}
cs := &mockstategen.MockCurrentSlotter{Slot: math.MaxUint64 - 1}
s.ReplayerBuilder = stategen.NewCanonicalBuilder(h, cc, cs)
s.ReplayerBuilder = stategen.NewCanonicalHistory(h, cc, cs)
}
// TODO: test failure
func TestServer_ListBeaconCommittees_PreviousEpoch(t *testing.T) {
params.SetupTestConfigCleanup(t)
params.OverrideBeaconConfig(params.MainnetConfig())

View File

@@ -65,7 +65,7 @@ func (bs *Server) ListValidatorBalances(
if err != nil {
return nil, err
}
requestedState, err := bs.ReplayerBuilder.ForSlot(startSlot).ReplayBlocks(ctx)
requestedState, err := bs.ReplayerBuilder.ReplayerForSlot(startSlot).ReplayBlocks(ctx)
if err != nil {
return nil, status.Error(codes.Internal, fmt.Sprintf("error replaying blocks for state at slot %d: %v", startSlot, err))
}
@@ -220,7 +220,7 @@ func (bs *Server) ListValidators(
if err != nil {
return nil, err
}
reqState, err = bs.ReplayerBuilder.ForSlot(s).ReplayBlocks(ctx)
reqState, err = bs.ReplayerBuilder.ReplayerForSlot(s).ReplayBlocks(ctx)
if err != nil {
return nil, status.Error(codes.Internal, fmt.Sprintf("error replaying blocks for state at slot %d: %v", s, err))
}
@@ -415,7 +415,7 @@ func (bs *Server) GetValidatorActiveSetChanges(
if err != nil {
return nil, err
}
requestedState, err := bs.ReplayerBuilder.ForSlot(s).ReplayBlocks(ctx)
requestedState, err := bs.ReplayerBuilder.ReplayerForSlot(s).ReplayBlocks(ctx)
if err != nil {
return nil, status.Error(codes.Internal, fmt.Sprintf("error replaying blocks for state at slot %d: %v", s, err))
}
@@ -512,7 +512,7 @@ func (bs *Server) GetValidatorParticipation(
}
// ReplayerBuilder ensures that a canonical chain is followed to the slot
beaconState, err := bs.ReplayerBuilder.ForSlot(startSlot).ReplayBlocks(ctx)
beaconState, err := bs.ReplayerBuilder.ReplayerForSlot(startSlot).ReplayBlocks(ctx)
if err != nil {
return nil, status.Error(codes.Internal, fmt.Sprintf("error replaying blocks for state at slot %d: %v", startSlot, err))
}
@@ -831,7 +831,7 @@ func (bs *Server) GetIndividualVotes(
if err != nil {
return nil, err
}
st, err := bs.ReplayerBuilder.ForSlot(s).ReplayBlocks(ctx)
st, err := bs.ReplayerBuilder.ReplayerForSlot(s).ReplayBlocks(ctx)
if err != nil {
return nil, status.Errorf(codes.Internal, "failed to replay blocks for state at epoch %d: %v", req.Epoch, err)
}

View File

@@ -1505,6 +1505,8 @@ func TestServer_GetValidatorParticipation_CannotRequestFutureEpoch(t *testing.T)
func TestServer_GetValidatorParticipation_CurrentAndPrevEpoch(t *testing.T) {
helpers.ClearCache()
params.SetupTestConfigCleanup(t)
params.OverrideBeaconConfig(params.MainnetConfig())
beaconDB := dbTest.SetupDB(t)
ctx := context.Background()

View File

@@ -29,7 +29,7 @@ func (ds *Server) GetBeaconState(
)
}
st, err := ds.ReplayerBuilder.ForSlot(q.Slot).ReplayBlocks(ctx)
st, err := ds.ReplayerBuilder.ReplayerForSlot(q.Slot).ReplayBlocks(ctx)
if err != nil {
return nil, status.Error(codes.Internal, fmt.Sprintf("error replaying blocks for state at slot %d: %v", q.Slot, err))
}

View File

@@ -18,14 +18,10 @@ import (
"github.com/prysmaticlabs/prysm/testing/util"
)
func addReplayerBuilder(s *Server, h stategen.HistoryAccessor, is bool, canonErr error, currSlot types.Slot) {
cc := &mockstategen.MockCanonicalChecker{Is: is, Err: canonErr}
cs := &mockstategen.MockCurrentSlotter{Slot: currSlot}
s.ReplayerBuilder = stategen.NewCanonicalBuilder(h, cc, cs)
}
func addDefaultReplayerBuilder(s *Server, h stategen.HistoryAccessor) {
addReplayerBuilder(s, h, true, nil, math.MaxUint64-1)
cc := &mockstategen.MockCanonicalChecker{Is: true}
cs := &mockstategen.MockCurrentSlotter{Slot: math.MaxUint64 - 1}
s.ReplayerBuilder = stategen.NewCanonicalHistory(h, cc, cs)
}
func TestServer_GetBeaconState(t *testing.T) {

View File

@@ -362,7 +362,7 @@ func (vs *Server) retrieveAfterEpochTransition(ctx context.Context, epoch types.
return nil, err
}
// replay to first slot of following epoch
return vs.ReplayerBuilder.ForSlot(endSlot).ReplayToSlot(ctx, endSlot+1)
return vs.ReplayerBuilder.ReplayerForSlot(endSlot).ReplayToSlot(ctx, endSlot+1)
}
func checkValidatorsAreRecent(headEpoch types.Epoch, req *ethpb.DoppelGangerRequest) (bool, *ethpb.DoppelGangerResponse) {

View File

@@ -178,6 +178,7 @@ func (s *Service) Start() {
stateCache = s.cfg.StateGen.CombinedCache()
}
withCache := stategen.WithCache(stateCache)
ch := stategen.NewCanonicalHistory(s.cfg.BeaconDB, s.cfg.ChainInfoFetcher, s.cfg.ChainInfoFetcher, withCache)
validatorServer := &validatorv1alpha1.Server{
Ctx: s.ctx,
@@ -204,7 +205,7 @@ func (s *Service) Start() {
SlashingsPool: s.cfg.SlashingsPool,
StateGen: s.cfg.StateGen,
SyncCommitteePool: s.cfg.SyncCommitteeObjectPool,
ReplayerBuilder: stategen.NewCanonicalBuilder(s.cfg.BeaconDB, s.cfg.ChainInfoFetcher, s.cfg.ChainInfoFetcher, withCache),
ReplayerBuilder: ch,
ExecutionEngineCaller: s.cfg.ExecutionEngineCaller,
BeaconDB: s.cfg.BeaconDB,
}
@@ -221,7 +222,7 @@ func (s *Service) Start() {
ChainInfoFetcher: s.cfg.ChainInfoFetcher,
GenesisTimeFetcher: s.cfg.GenesisTimeFetcher,
StateGenService: s.cfg.StateGen,
ReplayerBuilder: stategen.NewCanonicalBuilder(s.cfg.BeaconDB, s.cfg.ChainInfoFetcher, s.cfg.ChainInfoFetcher, withCache),
ReplayerBuilder: ch,
},
SyncCommitteePool: s.cfg.SyncCommitteeObjectPool,
}
@@ -271,9 +272,10 @@ func (s *Service) Start() {
SyncChecker: s.cfg.SyncService,
ReceivedAttestationsBuffer: make(chan *ethpbv1alpha1.Attestation, attestationBufferSize),
CollectedAttestationsBuffer: make(chan []*ethpbv1alpha1.Attestation, attestationBufferSize),
ReplayerBuilder: stategen.NewCanonicalBuilder(s.cfg.BeaconDB, s.cfg.ChainInfoFetcher, s.cfg.ChainInfoFetcher, withCache),
ReplayerBuilder: ch,
}
beaconChainServerV1 := &beacon.Server{
CanonicalHistory: ch,
BeaconDB: s.cfg.BeaconDB,
AttestationsPool: s.cfg.AttestationsPool,
SlashingsPool: s.cfg.SlashingsPool,
@@ -293,6 +295,7 @@ func (s *Service) Start() {
HeadFetcher: s.cfg.HeadFetcher,
VoluntaryExitsPool: s.cfg.ExitPool,
V1Alpha1ValidatorServer: validatorServer,
SyncChecker: s.cfg.SyncService,
}
ethpbv1alpha1.RegisterNodeServer(s.grpcServer, nodeServer)
ethpbservice.RegisterBeaconNodeServer(s.grpcServer, nodeServerV1)
@@ -314,7 +317,7 @@ func (s *Service) Start() {
HeadFetcher: s.cfg.HeadFetcher,
PeerManager: s.cfg.PeerManager,
PeersFetcher: s.cfg.PeersFetcher,
ReplayerBuilder: stategen.NewCanonicalBuilder(s.cfg.BeaconDB, s.cfg.ChainInfoFetcher, s.cfg.ChainInfoFetcher, withCache),
ReplayerBuilder: ch,
}
debugServerV1 := &debug.Server{
BeaconDB: s.cfg.BeaconDB,
@@ -324,7 +327,7 @@ func (s *Service) Start() {
ChainInfoFetcher: s.cfg.ChainInfoFetcher,
GenesisTimeFetcher: s.cfg.GenesisTimeFetcher,
StateGenService: s.cfg.StateGen,
ReplayerBuilder: stategen.NewCanonicalBuilder(s.cfg.BeaconDB, s.cfg.ChainInfoFetcher, s.cfg.ChainInfoFetcher, withCache),
ReplayerBuilder: ch,
},
}
ethpbv1alpha1.RegisterDebugServer(s.grpcServer, debugServer)

View File

@@ -11,6 +11,7 @@ go_library(
"//beacon-chain/db:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//config/params:go_default_library",
"//encoding/bytesutil:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_eth2_types//:go_default_library",

View File

@@ -14,6 +14,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"go.opencensus.io/trace"
)
@@ -110,7 +111,7 @@ func (p *StateProvider) State(ctx context.Context, stateId []byte) (state.Beacon
return nil, errors.Wrap(err, "could not get head state")
}
case "genesis":
s, err = p.BeaconDB.GenesisState(ctx)
s, err = p.StateBySlot(ctx, params.BeaconConfig().GenesisSlot)
if err != nil {
return nil, errors.Wrap(err, "could not get genesis state")
}
@@ -203,7 +204,7 @@ func (p *StateProvider) StateBySlot(ctx context.Context, target types.Slot) (sta
ctx, span := trace.StartSpan(ctx, "statefetcher.StateBySlot")
defer span.End()
st, err := p.ReplayerBuilder.ForSlot(target).ReplayBlocks(ctx)
st, err := p.ReplayerBuilder.ReplayerForSlot(target).ReplayBlocks(ctx)
if err != nil {
msg := fmt.Sprintf("error while replaying history to slot=%d", target)
return nil, errors.Wrap(err, msg)

View File

@@ -73,8 +73,12 @@ func TestGetState(t *testing.T) {
require.NoError(t, db.SaveGenesisBlockRoot(ctx, r))
require.NoError(t, db.SaveState(ctx, bs, r))
cc := &mockstategen.MockCanonicalChecker{Is: true}
cs := &mockstategen.MockCurrentSlotter{Slot: bs.Slot() + 1}
ch := stategen.NewCanonicalHistory(db, cc, cs)
p := StateProvider{
BeaconDB: db,
BeaconDB: db,
ReplayerBuilder: ch,
}
s, err := p.State(ctx, []byte("genesis"))

View File

@@ -7,6 +7,7 @@ go_library(
"epoch_boundary_state_cache.go",
"errors.go",
"getter.go",
"history.go",
"hot_state_cache.go",
"log.go",
"metrics.go",
@@ -51,6 +52,7 @@ go_test(
srcs = [
"epoch_boundary_state_cache_test.go",
"getter_test.go",
"history_test.go",
"hot_state_cache_test.go",
"init_test.go",
"migrate_test.go",

View File

@@ -4,7 +4,6 @@ import (
"context"
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/config/params"
@@ -13,6 +12,8 @@ import (
"go.opencensus.io/trace"
)
var ErrSlotBeforeOrigin = errors.New("cannot retrieve data for slots before sync origin")
// HasState returns true if the state exists in cache or in DB.
func (s *State) HasState(ctx context.Context, blockRoot [32]byte) (bool, error) {
has, err := s.HasStateInCache(ctx, blockRoot)
@@ -111,14 +112,6 @@ func (s *State) StateByRootInitialSync(ctx context.Context, blockRoot [32]byte)
return startState, nil
}
// StateBySlot retrieves the state using input slot.
func (s *State) StateBySlot(ctx context.Context, slot types.Slot) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "stateGen.StateBySlot")
defer span.End()
return s.loadStateBySlot(ctx, slot)
}
// This returns the state summary object of a given block root, it first checks the cache
// then checks the DB. An error is returned if state summary object is nil.
func (s *State) stateSummary(ctx context.Context, blockRoot [32]byte) (*ethpb.StateSummary, error) {
@@ -208,36 +201,6 @@ func (s *State) loadStateByRoot(ctx context.Context, blockRoot [32]byte) (state.
return s.ReplayBlocks(ctx, startState, blks, targetSlot)
}
// This loads a state by slot.
func (s *State) loadStateBySlot(ctx context.Context, slot types.Slot) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "stateGen.loadStateBySlot")
defer span.End()
// Return genesis state if slot is 0.
if slot == 0 {
return s.beaconDB.GenesisState(ctx)
}
// Gather the last saved block root and the slot number.
lastValidRoot, lastValidSlot, err := s.lastSavedBlock(ctx, slot)
if err != nil {
return nil, errors.Wrap(err, "could not get last valid block for hot state using slot")
}
replayStartState, err := s.loadStateByRoot(ctx, lastValidRoot)
if err != nil {
return nil, err
}
if lastValidSlot < slot {
replayStartState, err = ReplayProcessSlots(ctx, replayStartState, slot)
if err != nil {
return nil, err
}
}
return replayStartState, nil
}
// This returns the highest available ancestor state of the input block root.
// It recursively look up block's parent until a corresponding state of the block root
// is found in the caches or DB.
@@ -266,6 +229,11 @@ func (s *State) LastAncestorState(ctx context.Context, root [32]byte) (state.Bea
if ctx.Err() != nil {
return nil, ctx.Err()
}
// return an error if we have rewound to before the checkpoint sync slot
if (b.Block().Slot() - 1) < s.minimumSlot() {
return nil, errors.Wrapf(ErrSlotBeforeOrigin, "no blocks in db prior to slot %d", s.minimumSlot())
}
// Is the state a genesis state.
parentRoot := bytesutil.ToBytes32(b.Block().ParentRoot())
if parentRoot == params.BeaconConfig().ZeroHash {

View File

@@ -231,63 +231,6 @@ func TestStateByRootInitialSync_CanProcessUpTo(t *testing.T) {
assert.Equal(t, targetSlot, loadedState.Slot(), "Did not correctly load state")
}
func TestStateBySlot_ColdState(t *testing.T) {
ctx := context.Background()
beaconDB := testDB.SetupDB(t)
service := New(beaconDB)
service.slotsPerArchivedPoint = params.BeaconConfig().SlotsPerEpoch * 2
service.finalizedInfo.slot = service.slotsPerArchivedPoint + 1
beaconState, pks := util.DeterministicGenesisState(t, 32)
genesisStateRoot, err := beaconState.HashTreeRoot(ctx)
require.NoError(t, err)
genesis := blocks.NewGenesisBlock(genesisStateRoot[:])
assert.NoError(t, beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(genesis)))
gRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err)
assert.NoError(t, beaconDB.SaveState(ctx, beaconState, gRoot))
assert.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
b, err := util.GenerateFullBlock(beaconState, pks, util.DefaultBlockGenConfig(), 1)
require.NoError(t, err)
require.NoError(t, beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b)))
bRoot, err := b.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveState(ctx, beaconState, bRoot))
require.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, bRoot))
r := [32]byte{}
require.NoError(t, service.beaconDB.SaveStateSummary(ctx, &ethpb.StateSummary{Slot: service.slotsPerArchivedPoint, Root: r[:]}))
slot := types.Slot(20)
loadedState, err := service.StateBySlot(ctx, slot)
require.NoError(t, err)
assert.Equal(t, slot, loadedState.Slot(), "Did not correctly save state")
}
func TestStateBySlot_HotStateDB(t *testing.T) {
ctx := context.Background()
beaconDB := testDB.SetupDB(t)
service := New(beaconDB)
beaconState, _ := util.DeterministicGenesisState(t, 32)
genesisStateRoot, err := beaconState.HashTreeRoot(ctx)
require.NoError(t, err)
genesis := blocks.NewGenesisBlock(genesisStateRoot[:])
assert.NoError(t, beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(genesis)))
gRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err)
assert.NoError(t, beaconDB.SaveState(ctx, beaconState, gRoot))
assert.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
slot := types.Slot(10)
loadedState, err := service.StateBySlot(ctx, slot)
require.NoError(t, err)
assert.Equal(t, slot, loadedState.Slot(), "Did not correctly load state")
}
func TestLoadeStateByRoot_Cached(t *testing.T) {
ctx := context.Background()
beaconDB := testDB.SetupDB(t)
@@ -379,83 +322,6 @@ func TestLoadeStateByRoot_FromDBBoundaryCase(t *testing.T) {
assert.Equal(t, types.Slot(10), loadedState.Slot(), "Did not correctly load state")
}
func TestLoadeStateBySlot_CanAdvanceSlotUsingDB(t *testing.T) {
ctx := context.Background()
beaconDB := testDB.SetupDB(t)
service := New(beaconDB)
beaconState, _ := util.DeterministicGenesisState(t, 32)
b := util.NewBeaconBlock()
require.NoError(t, service.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b)))
gRoot, err := b.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, service.beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
require.NoError(t, service.beaconDB.SaveState(ctx, beaconState, gRoot))
slot := types.Slot(10)
loadedState, err := service.loadStateBySlot(ctx, slot)
require.NoError(t, err)
assert.Equal(t, slot, loadedState.Slot(), "Did not correctly load state")
}
func TestLoadeStateBySlot_CanReplayBlock(t *testing.T) {
ctx := context.Background()
beaconDB := testDB.SetupDB(t)
service := New(beaconDB)
genesis, keys := util.DeterministicGenesisState(t, 64)
genesisBlockRoot := bytesutil.ToBytes32(nil)
require.NoError(t, beaconDB.SaveState(ctx, genesis, genesisBlockRoot))
stateRoot, err := genesis.HashTreeRoot(ctx)
require.NoError(t, err)
genesisBlk := blocks.NewGenesisBlock(stateRoot[:])
require.NoError(t, beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(genesisBlk)))
genesisBlkRoot, err := genesisBlk.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, genesisBlkRoot))
b1, err := util.GenerateFullBlock(genesis, keys, util.DefaultBlockGenConfig(), 1)
assert.NoError(t, err)
require.NoError(t, beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b1)))
r1, err := b1.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, service.beaconDB.SaveStateSummary(ctx, &ethpb.StateSummary{Slot: 1, Root: r1[:]}))
service.hotStateCache.put(bytesutil.ToBytes32(b1.Block.ParentRoot), genesis)
loadedState, err := service.loadStateBySlot(ctx, 2)
require.NoError(t, err)
assert.Equal(t, types.Slot(2), loadedState.Slot(), "Did not correctly load state")
}
func TestLoadeStateBySlot_DoesntReplayBlockOnRequestedSlot(t *testing.T) {
ctx := context.Background()
beaconDB := testDB.SetupDB(t)
service := New(beaconDB)
genesis, keys := util.DeterministicGenesisState(t, 64)
genesisBlockRoot := bytesutil.ToBytes32(nil)
require.NoError(t, beaconDB.SaveState(ctx, genesis, genesisBlockRoot))
stateRoot, err := genesis.HashTreeRoot(ctx)
require.NoError(t, err)
genesisBlk := blocks.NewGenesisBlock(stateRoot[:])
require.NoError(t, beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(genesisBlk)))
genesisBlkRoot, err := genesisBlk.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, beaconDB.SaveGenesisBlockRoot(ctx, genesisBlkRoot))
b1, err := util.GenerateFullBlock(genesis, keys, util.DefaultBlockGenConfig(), 1)
assert.NoError(t, err)
require.NoError(t, beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b1)))
r1, err := b1.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, service.beaconDB.SaveStateSummary(ctx, &ethpb.StateSummary{Slot: 1, Root: r1[:]}))
service.hotStateCache.put(bytesutil.ToBytes32(b1.Block.ParentRoot), genesis)
loadedState, err := service.loadStateBySlot(ctx, 1)
require.NoError(t, err)
assert.Equal(t, types.Slot(1), loadedState.Slot(), "Did not correctly load state")
// Latest block header's state root should not be zero. Zero means the current slot's block has been processed.
require.NotEqual(t, params.BeaconConfig().ZeroHash, bytesutil.ToBytes32(loadedState.LatestBlockHeader().StateRoot))
}
func TestLastAncestorState_CanGetUsingDB(t *testing.T) {
ctx := context.Background()
beaconDB := testDB.SetupDB(t)

View File

@@ -0,0 +1,206 @@
package stategen
import (
"context"
"fmt"
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block"
"go.opencensus.io/trace"
)
func WithCache(c CachedGetter) CanonicalHistoryOption {
return func(h *CanonicalHistory) {
h.cache = c
}
}
type CanonicalHistoryOption func(*CanonicalHistory)
func NewCanonicalHistory(h HistoryAccessor, cc CanonicalChecker, cs CurrentSlotter, opts ...CanonicalHistoryOption) *CanonicalHistory {
ch := &CanonicalHistory{
h: h,
cc: cc,
cs: cs,
}
for _, o := range opts {
o(ch)
}
return ch
}
type CanonicalHistory struct {
h HistoryAccessor
cc CanonicalChecker
cs CurrentSlotter
cache CachedGetter
}
func (ch *CanonicalHistory) ReplayerForSlot(target types.Slot) Replayer {
return &stateReplayer{chainer: ch, method: forSlot, target: target}
}
func (c *CanonicalHistory) BlockForSlot(ctx context.Context, target types.Slot) ([32]byte, block.SignedBeaconBlock, error) {
currentSlot := c.cs.CurrentSlot()
if target > currentSlot {
return [32]byte{}, nil, errors.Wrap(ErrFutureSlotRequested, fmt.Sprintf("requested=%d, current=%d", target, currentSlot))
}
for target > 0 {
if ctx.Err() != nil {
return [32]byte{}, nil, errors.Wrap(ctx.Err(), "context canceled during canonicalBlockForSlot")
}
hbs, err := c.h.HighestSlotBlocksBelow(ctx, target+1)
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, fmt.Sprintf("error finding highest block w/ slot <= %d", target))
}
if len(hbs) == 0 {
return [32]byte{}, nil, errors.Wrap(ErrNoBlocksBelowSlot, fmt.Sprintf("slot=%d", target))
}
r, b, err := c.bestForSlot(ctx, hbs)
if err == nil {
// we found a valid, canonical block!
return r, b, nil
}
// we found a block, but it wasn't considered canonical - keep looking
if errors.Is(err, ErrNoCanonicalBlockForSlot) {
// break once we've seen slot 0 (and prevent underflow)
if hbs[0].Block().Slot() == params.BeaconConfig().GenesisSlot {
break
}
target = hbs[0].Block().Slot() - 1
continue
}
return [32]byte{}, nil, err
}
b, err := c.h.GenesisBlock(ctx)
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, "db error while retrieving genesis block")
}
root, _, err := c.bestForSlot(ctx, []block.SignedBeaconBlock{b})
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, "problem retrieving genesis block")
}
return root, b, nil
}
// bestForSlot encapsulates several messy realities of the underlying db code, looping through multiple blocks,
// performing null/validity checks, and using CanonicalChecker to only pick canonical blocks.
func (c *CanonicalHistory) bestForSlot(ctx context.Context, hbs []block.SignedBeaconBlock) ([32]byte, block.SignedBeaconBlock, error) {
for _, b := range hbs {
if helpers.BeaconBlockIsNil(b) != nil {
continue
}
root, err := b.Block().HashTreeRoot()
if err != nil {
// use this error message to wrap a sentinel error for error type matching
wrapped := errors.Wrap(ErrInvalidDBBlock, err.Error())
msg := fmt.Sprintf("could not compute hash_tree_root for block at slot=%d", b.Block().Slot())
return [32]byte{}, nil, errors.Wrap(wrapped, msg)
}
canon, err := c.cc.IsCanonical(ctx, root)
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, "replayer could not check if block is canonical")
}
if canon {
return root, b, nil
}
}
return [32]byte{}, nil, errors.Wrap(ErrNoCanonicalBlockForSlot, "no good block for slot")
}
// ChainForSlot creates a value that satisfies the Replayer interface via db queries
// and the stategen transition helper methods. This implementation uses the following algorithm:
// - find the highest canonical block <= the target slot
// - starting with this block, recursively search backwards for a stored state, and accumulate intervening blocks
func (c *CanonicalHistory) chainForSlot(ctx context.Context, target types.Slot) (state.BeaconState, []block.SignedBeaconBlock, error) {
ctx, span := trace.StartSpan(ctx, "canonicalChainer.chainForSlot")
defer span.End()
_, b, err := c.BlockForSlot(ctx, target)
if err != nil {
return nil, nil, errors.Wrap(err, fmt.Sprintf("unable to find replay data for slot=%d", target))
}
s, descendants, err := c.ancestorChain(ctx, b)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to query for ancestor and descendant blocks")
}
return s, descendants, nil
}
func (c *CanonicalHistory) getState(ctx context.Context, root [32]byte) (state.BeaconState, error) {
if c.cache != nil {
st, err := c.cache.ByRoot(root)
if err == nil {
return st, nil
}
if !errors.Is(err, ErrNotInCache) {
return nil, errors.Wrap(err, "error reading from state cache during state replay")
}
}
return c.h.StateOrError(ctx, root)
}
// ancestorChain works backwards through the chain lineage, accumulating blocks and checking for a saved state.
// If it finds a saved state that the tail block was descended from, it returns this state and
// all blocks in the lineage, including the tail block. Blocks are returned in ascending order.
// Note that this function assumes that the tail is a canonical block, and therefore assumes that
// all ancestors are also canonical.
func (c *CanonicalHistory) ancestorChain(ctx context.Context, tail block.SignedBeaconBlock) (state.BeaconState, []block.SignedBeaconBlock, error) {
ctx, span := trace.StartSpan(ctx, "canonicalChainer.ancestorChain")
defer span.End()
chain := make([]block.SignedBeaconBlock, 0)
for {
if err := ctx.Err(); err != nil {
msg := fmt.Sprintf("context canceled while finding ancestors of block at slot %d", tail.Block().Slot())
return nil, nil, errors.Wrap(err, msg)
}
b := tail.Block()
// compute hash_tree_root of current block and try to look up the corresponding state
root, err := b.HashTreeRoot()
if err != nil {
msg := fmt.Sprintf("could not compute htr for descendant block at slot=%d", b.Slot())
return nil, nil, errors.Wrap(err, msg)
}
st, err := c.getState(ctx, root)
// err == nil, we've got a real state - the job is done!
// Note: in cases where there are skipped slots we could find a state that is a descendant
// of the block we are searching for. We don't want to return a future block, so in this case
// we keep working backwards.
if err == nil && st.Slot() == b.Slot() {
// we found the state by the root of the head, meaning it has already been applied.
// we only want to return the blocks descended from it.
reverseChain(chain)
return st, chain, nil
}
// ErrNotFoundState errors are fine, but other errors mean something is wrong with the db
if err != nil && !errors.Is(err, db.ErrNotFoundState) {
return nil, nil, errors.Wrap(err, fmt.Sprintf("error querying database for state w/ block root = %#x", root))
}
parent, err := c.h.Block(ctx, bytesutil.ToBytes32(b.ParentRoot()))
if err != nil {
msg := fmt.Sprintf("db error when retrieving parent of block at slot=%d by root=%#x", b.Slot(), b.ParentRoot())
return nil, nil, errors.Wrap(err, msg)
}
if helpers.BeaconBlockIsNil(parent) != nil {
msg := fmt.Sprintf("unable to retrieve parent of block at slot=%d by root=%#x", b.Slot(), b.ParentRoot())
return nil, nil, errors.Wrap(db.ErrNotFound, msg)
}
chain = append(chain, tail)
tail = parent
}
}
func reverseChain(c []block.SignedBeaconBlock) {
last := len(c) - 1
swaps := (last + 1) / 2
for i := 0; i < swaps; i++ {
c[i], c[last-i] = c[last-i], c[i]
}
}

View File

@@ -0,0 +1,615 @@
package stategen
import (
"context"
"encoding/binary"
"fmt"
"testing"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/pkg/errors"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block/mock"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/wrapper"
"github.com/prysmaticlabs/prysm/testing/require"
)
func TestBlockForSlotFuture(t *testing.T) {
ch := &CanonicalHistory{
cs: &mockCurrentSlotter{Slot: 0},
}
_, _, err := ch.BlockForSlot(context.Background(), 1)
require.ErrorIs(t, err, ErrFutureSlotRequested)
}
func TestChainForSlotFuture(t *testing.T) {
ch := &CanonicalHistory{
cs: &mockCurrentSlotter{Slot: 0},
}
_, _, err := ch.chainForSlot(context.Background(), 1)
require.ErrorIs(t, err, ErrFutureSlotRequested)
}
func TestBestForSlot(t *testing.T) {
nilBlock, err := wrapper.WrappedSignedBeaconBlock(&ethpb.SignedBeaconBlock{})
require.NoError(t, err)
nilBody, err := wrapper.WrappedSignedBeaconBlock(&ethpb.SignedBeaconBlock{Block: &ethpb.BeaconBlock{}})
require.NoError(t, err)
derp := errors.New("fake hash tree root method no hash good")
badHTR := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{HtrErr: derp, BeaconBlockBody: &mock.BeaconBlockBody{}}}
var goodHTR [32]byte
copy(goodHTR[:], []byte{23})
var betterHTR [32]byte
copy(betterHTR[:], []byte{42})
good := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{BeaconBlockBody: &mock.BeaconBlockBody{}, Htr: goodHTR}}
better := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{BeaconBlockBody: &mock.BeaconBlockBody{}, Htr: betterHTR}}
cases := []struct {
name string
err error
blocks []block.SignedBeaconBlock
best block.SignedBeaconBlock
root [32]byte
cc CanonicalChecker
}{
{
name: "empty list",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{},
},
{
name: "empty SignedBeaconBlock",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{nil},
},
{
name: "empty BeaconBlock",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{nilBlock},
},
{
name: "empty BeaconBlockBody",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{nilBody},
},
{
name: "bad HTR",
err: ErrInvalidDBBlock,
blocks: []block.SignedBeaconBlock{badHTR},
},
{
name: "IsCanonical fail",
blocks: []block.SignedBeaconBlock{good, better},
cc: &mockCanonicalChecker{is: true, err: derp},
err: derp,
},
{
name: "all non-canonical",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{good, better},
cc: &mockCanonicalChecker{is: false},
},
{
name: "one canonical",
blocks: []block.SignedBeaconBlock{good},
cc: &mockCanonicalChecker{is: true},
root: goodHTR,
best: good,
},
{
name: "all canonical",
blocks: []block.SignedBeaconBlock{better, good},
cc: &mockCanonicalChecker{is: true},
root: betterHTR,
best: better,
},
{
name: "first wins",
blocks: []block.SignedBeaconBlock{good, better},
cc: &mockCanonicalChecker{is: true},
root: goodHTR,
best: good,
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
chk := CanonicalChecker(&mockCanonicalChecker{is: true})
if c.cc != nil {
chk = c.cc
}
ch := &CanonicalHistory{cc: chk}
r, b, err := ch.bestForSlot(context.Background(), c.blocks)
if c.err == nil {
require.NoError(t, err)
require.DeepEqual(t, c.best, b)
require.Equal(t, c.root, r)
} else {
require.ErrorIs(t, err, c.err)
}
})
}
}
// happy path tests
func TestCanonicalBlockForSlotHappy(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin},
{slot: middle, savedState: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
ch := &CanonicalHistory{h: hist, cc: hist, cs: hist}
// since only the end block and genesis are canonical, once the slot drops below
// end, we should always get genesis
cases := []struct {
slot types.Slot
highest types.Slot
canon types.Slot
name string
}{
{slot: hist.current, highest: end, canon: end, name: "slot > end"},
{slot: end, highest: end, canon: end, name: "slot == end"},
{slot: end - 1, highest: middle, canon: 0, name: "middle < slot < end"},
{slot: middle, highest: middle, canon: 0, name: "slot == middle"},
{slot: middle - 1, highest: begin, canon: 0, name: "begin < slot < middle"},
{slot: begin, highest: begin, canon: 0, name: "slot == begin"},
{slot: begin - 1, highest: 0, canon: 0, name: "genesis < slot < begin"},
{slot: 0, highest: 0, canon: 0, name: "slot == genesis"},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
bs, err := hist.HighestSlotBlocksBelow(ctx, c.slot+1)
require.NoError(t, err)
require.Equal(t, len(bs), 1)
r, err := bs[0].Block().HashTreeRoot()
require.NoError(t, err)
require.Equal(t, hist.slotMap[c.highest], r)
cr, _, err := ch.BlockForSlot(ctx, c.slot)
require.NoError(t, err)
require.Equal(t, hist.slotMap[c.canon], cr)
})
}
}
func TestCanonicalBlockForSlotNonHappy(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin},
{slot: middle, savedState: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
slotOrderObserved := make([]types.Slot, 0)
derp := errors.New("HighestSlotBlocksBelow don't work")
// since only the end block and genesis are canonical, once the slot drops below
// end, we should always get genesis
cases := []struct {
name string
slot types.Slot
canon CanonicalChecker
overrideHighest func(context.Context, types.Slot) ([]block.SignedBeaconBlock, error)
slotOrderExpected []types.Slot
err error
root [32]byte
}{
{
name: "HighestSlotBlocksBelow not called for genesis",
overrideHighest: func(_ context.Context, _ types.Slot) ([]block.SignedBeaconBlock, error) {
return nil, derp
},
root: hist.slotMap[0],
},
{
name: "wrapped error from HighestSlotBlocksBelow returned",
err: derp,
overrideHighest: func(_ context.Context, _ types.Slot) ([]block.SignedBeaconBlock, error) {
return nil, derp
},
slot: end,
},
{
name: "HighestSlotBlocksBelow empty list",
err: ErrNoBlocksBelowSlot,
overrideHighest: func(_ context.Context, _ types.Slot) ([]block.SignedBeaconBlock, error) {
return []block.SignedBeaconBlock{}, nil
},
slot: end,
},
{
name: "HighestSlotBlocksBelow no canonical",
err: ErrNoCanonicalBlockForSlot,
canon: &mockCanonicalChecker{is: false},
slot: end,
},
{
name: "slot ordering correct - only genesis canonical",
canon: &mockCanonicalChecker{isCanon: func(root [32]byte) (bool, error) {
if root == hist.slotMap[0] {
return true, nil
}
return false, nil
}},
overrideHighest: func(_ context.Context, s types.Slot) ([]block.SignedBeaconBlock, error) {
slotOrderObserved = append(slotOrderObserved, s)
// this allows the mock HighestSlotBlocksBelow to continue to execute now that we've recorded
// the slot in our channel
return nil, errFallThroughOverride
},
slotOrderExpected: []types.Slot{156, 155, 150, 100},
slot: end,
root: hist.slotMap[0],
},
{
name: "slot ordering correct - slot 100 canonical",
canon: &mockCanonicalChecker{isCanon: func(root [32]byte) (bool, error) {
if root == hist.slotMap[100] {
return true, nil
}
return false, nil
}},
overrideHighest: func(_ context.Context, s types.Slot) ([]block.SignedBeaconBlock, error) {
slotOrderObserved = append(slotOrderObserved, s)
// this allows the mock HighestSlotBlocksBelow to continue to execute now that we've recorded
// the slot in our channel
return nil, errFallThroughOverride
},
slotOrderExpected: []types.Slot{156, 155, 150},
slot: end,
root: hist.slotMap[100],
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
var canon CanonicalChecker = hist
if c.canon != nil {
canon = c.canon
}
ch := &CanonicalHistory{h: hist, cc: canon, cs: hist}
hist.overrideHighestSlotBlocksBelow = c.overrideHighest
r, _, err := ch.BlockForSlot(ctx, c.slot)
if c.err == nil {
require.NoError(t, err)
} else {
require.ErrorIs(t, err, c.err)
}
if len(c.slotOrderExpected) > 0 {
require.Equal(t, len(c.slotOrderExpected), len(slotOrderObserved), "HighestSlotBlocksBelow not called the expected number of times")
for i := range c.slotOrderExpected {
require.Equal(t, c.slotOrderExpected[i], slotOrderObserved[i])
}
}
if c.root != [32]byte{} {
require.Equal(t, c.root, r)
}
slotOrderObserved = make([]types.Slot, 0)
})
}
}
type mockCurrentSlotter struct {
Slot types.Slot
}
func (c *mockCurrentSlotter) CurrentSlot() types.Slot {
return c.Slot
}
var _ CurrentSlotter = &mockCurrentSlotter{}
func TestAncestorChainCache(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin, canonicalBlock: true},
{slot: middle, canonicalBlock: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
ch := &CanonicalHistory{h: hist, cc: hist, cs: hist}
// should only contain the genesis block
require.Equal(t, 1, len(hist.states))
endBlock := hist.blocks[hist.slotMap[end]]
st, bs, err := ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 3, len(bs))
expectedHTR, err := hist.states[hist.slotMap[0]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
// now populate the cache, we should get the cached state instead of genesis
ch.cache = &mockCachedGetter{
cache: map[[32]byte]state.BeaconState{
hist.slotMap[end]: hist.hiddenStates[hist.slotMap[end]],
},
}
st, bs, err = ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 0, len(bs))
expectedHTR, err = hist.hiddenStates[hist.slotMap[end]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
// populate cache with a different state for good measure
ch.cache = &mockCachedGetter{
cache: map[[32]byte]state.BeaconState{
hist.slotMap[begin]: hist.hiddenStates[hist.slotMap[begin]],
},
}
st, bs, err = ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 2, len(bs))
expectedHTR, err = hist.hiddenStates[hist.slotMap[begin]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
// rebuild history w/ last state saved, make sure we get that instead of cache
specs[2].savedState = true
hist = newMockHistory(t, specs, end+1)
ch = &CanonicalHistory{h: hist, cc: hist, cs: hist}
ch.cache = &mockCachedGetter{
cache: map[[32]byte]state.BeaconState{
hist.slotMap[begin]: hist.hiddenStates[hist.slotMap[begin]],
},
}
st, bs, err = ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 0, len(bs))
expectedHTR, err = hist.states[hist.slotMap[end]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
}
func TestAncestorChainOK(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin},
{slot: middle, savedState: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
ch := &CanonicalHistory{h: hist, cc: hist, cs: hist}
endBlock := hist.blocks[hist.slotMap[end]]
st, bs, err := ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
// middle is the most recent slot where savedState == true
require.Equal(t, 1, len(bs))
require.DeepEqual(t, endBlock, bs[0])
expectedHTR, err := hist.states[hist.slotMap[middle]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
middleBlock := hist.blocks[hist.slotMap[middle]]
st, bs, err = ch.ancestorChain(ctx, middleBlock)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, 0, len(bs))
require.Equal(t, expectedHTR, actualHTR)
}
func TestChainForSlot(t *testing.T) {
ctx := context.Background()
var zero, one, two, three types.Slot = 50, 51, 150, 151
specs := []mockHistorySpec{
{slot: zero, canonicalBlock: true, savedState: true},
{slot: one, canonicalBlock: true},
{slot: two},
{slot: three, canonicalBlock: true},
}
hist := newMockHistory(t, specs, three+10)
ch := &CanonicalHistory{h: hist, cc: hist, cs: hist}
firstNonGenesisRoot := hist.slotMap[zero]
nonGenesisStateRoot, err := hist.states[firstNonGenesisRoot].HashTreeRoot(ctx)
require.NoError(t, err)
cases := []struct {
name string
slot types.Slot
stateRoot [32]byte
blockRoots [][32]byte
}{
{
name: "above latest slot (but before current slot)",
slot: three + 1,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one], hist.slotMap[two], hist.slotMap[three]},
},
{
name: "last canonical slot - two treated as canonical because it is parent of three",
slot: three,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one], hist.slotMap[two], hist.slotMap[three]},
},
{
name: "non-canonical slot skipped",
slot: two,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one]},
},
{
name: "first canonical slot",
slot: one,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one]},
},
{
name: "slot at saved state",
slot: zero,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{},
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
st, blocks, err := ch.chainForSlot(ctx, c.slot)
require.NoError(t, err)
actualStRoot, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, c.stateRoot, actualStRoot)
require.Equal(t, len(c.blockRoots), len(blocks))
for i, b := range blocks {
root, err := b.Block().HashTreeRoot()
require.NoError(t, err)
require.Equal(t, c.blockRoots[i], root)
}
})
}
}
func TestAncestorChainOrdering(t *testing.T) {
ctx := context.Background()
var zero, one, two, three, four, five types.Slot = 50, 51, 150, 151, 152, 200
specs := []mockHistorySpec{
{slot: zero},
{slot: one, savedState: true},
{slot: two},
{slot: three},
{slot: four},
{slot: five},
}
hist := newMockHistory(t, specs, five+1)
endRoot := hist.slotMap[specs[len(specs)-1].slot]
endBlock := hist.blocks[endRoot]
ch := &CanonicalHistory{h: hist, cc: hist, cs: hist}
st, bs, err := ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
expectedRoot, err := hist.states[hist.slotMap[one]].HashTreeRoot(ctx)
require.NoError(t, err)
actualRoot, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedRoot, actualRoot)
// we asked for the chain leading up to five
// one has the savedState. one is applied to the savedState, so it should be omitted
// that means we should get two, three, four, five (length of 4)
require.Equal(t, 4, len(bs))
for i, slot := range []types.Slot{two, three, four, five} {
require.Equal(t, slot, bs[i].Block().Slot(), fmt.Sprintf("wrong value at index %d", i))
}
// do the same query, but with the final state saved
// we should just get the final state w/o block to apply
specs[5].savedState = true
hist = newMockHistory(t, specs, five+1)
endRoot = hist.slotMap[specs[len(specs)-1].slot]
endBlock = hist.blocks[endRoot]
ch = &CanonicalHistory{h: hist, cc: hist, cs: hist}
st, bs, err = ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
expectedRoot, err = hist.states[endRoot].HashTreeRoot(ctx)
require.NoError(t, err)
actualRoot, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedRoot, actualRoot)
require.Equal(t, 0, len(bs))
// slice off the last element for an odd size list (to cover odd/even in the reverseChain func)
specs = specs[:len(specs)-1]
require.Equal(t, 5, len(specs))
hist = newMockHistory(t, specs, five+1)
ch = &CanonicalHistory{h: hist, cc: hist, cs: hist}
endRoot = hist.slotMap[specs[len(specs)-1].slot]
endBlock = hist.blocks[endRoot]
st, bs, err = ch.ancestorChain(ctx, endBlock)
require.NoError(t, err)
expectedRoot, err = hist.states[hist.slotMap[one]].HashTreeRoot(ctx)
require.NoError(t, err)
actualRoot, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedRoot, actualRoot)
require.Equal(t, 3, len(bs))
for i, slot := range []types.Slot{two, three, four} {
require.Equal(t, slot, bs[i].Block().Slot(), fmt.Sprintf("wrong value at index %d", i))
}
}
type mockCanonicalChecker struct {
isCanon func([32]byte) (bool, error)
is bool
err error
}
func (m *mockCanonicalChecker) IsCanonical(_ context.Context, root [32]byte) (bool, error) {
if m.isCanon != nil {
return m.isCanon(root)
}
return m.is, m.err
}
func TestReverseChain(t *testing.T) {
// test 0,1,2,3 elements to handle: zero case; single element; even number; odd number
for i := 0; i < 4; i++ {
t.Run(fmt.Sprintf("reverseChain with %d elements", i), func(t *testing.T) {
actual := mockBlocks(i, incrFwd)
expected := mockBlocks(i, incrBwd)
reverseChain(actual)
if len(actual) != len(expected) {
t.Errorf("different list lengths")
}
for i := 0; i < len(actual); i++ {
sblockA, ok := actual[i].(*mock.SignedBeaconBlock)
require.Equal(t, true, ok)
blockA, ok := sblockA.BeaconBlock.(*mock.BeaconBlock)
require.Equal(t, true, ok)
sblockE, ok := expected[i].(*mock.SignedBeaconBlock)
require.Equal(t, true, ok)
blockE, ok := sblockE.BeaconBlock.(*mock.BeaconBlock)
require.Equal(t, true, ok)
require.Equal(t, blockA.Htr, blockE.Htr)
}
})
}
}
func incrBwd(n int, c chan uint32) {
for i := n - 1; i >= 0; i-- {
c <- uint32(i)
}
close(c)
}
func incrFwd(n int, c chan uint32) {
for i := 0; i < n; i++ {
c <- uint32(i)
}
close(c)
}
func mockBlocks(n int, iter func(int, chan uint32)) []block.SignedBeaconBlock {
bchan := make(chan uint32)
go iter(n, bchan)
mb := make([]block.SignedBeaconBlock, 0)
for i := range bchan {
h := [32]byte{}
binary.LittleEndian.PutUint32(h[:], i)
b := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{BeaconBlockBody: &mock.BeaconBlockBody{}, Htr: h}}
mb = append(mb, b)
}
return mb
}

View File

@@ -48,7 +48,7 @@ func (b *MockReplayerBuilder) SetMockSlotError(s types.Slot, e error) {
b.forSlot[s] = &MockReplayer{Err: e}
}
func (b *MockReplayerBuilder) ForSlot(target types.Slot) stategen.Replayer {
func (b *MockReplayerBuilder) ReplayerForSlot(target types.Slot) stategen.Replayer {
return b.forSlot[target]
}

View File

@@ -235,79 +235,6 @@ func ReplayProcessSlots(ctx context.Context, state state.BeaconState, slot types
return state, nil
}
// This finds the last saved block in DB from searching backwards from input slot,
// it returns the block root and the slot of the block.
// This is used by both hot and cold state management.
func (s *State) lastSavedBlock(ctx context.Context, slot types.Slot) ([32]byte, types.Slot, error) {
ctx, span := trace.StartSpan(ctx, "stateGen.lastSavedBlock")
defer span.End()
// Handle the genesis case where the input slot is 0.
if slot == 0 {
gRoot, err := s.genesisRoot(ctx)
if err != nil {
return [32]byte{}, 0, err
}
return gRoot, 0, nil
}
lastSaved, err := s.beaconDB.HighestSlotBlocksBelow(ctx, slot)
if err != nil {
return [32]byte{}, 0, err
}
// Given this is used to query canonical block. There should only be one saved canonical block of a given slot.
if len(lastSaved) != 1 {
return [32]byte{}, 0, fmt.Errorf("highest saved block does not equal to 1, it equals to %d", len(lastSaved))
}
if lastSaved[0] == nil || lastSaved[0].IsNil() || lastSaved[0].Block().IsNil() {
return [32]byte{}, 0, nil
}
r, err := lastSaved[0].Block().HashTreeRoot()
if err != nil {
return [32]byte{}, 0, err
}
return r, lastSaved[0].Block().Slot(), nil
}
// This finds the last saved state in DB from searching backwards from input slot,
// it returns the block root of the block which was used to produce the state.
// This is used by both hot and cold state management.
func (s *State) lastSavedState(ctx context.Context, slot types.Slot) (state.ReadOnlyBeaconState, error) {
ctx, span := trace.StartSpan(ctx, "stateGen.lastSavedState")
defer span.End()
// Handle the genesis case where the input slot is 0.
if slot == 0 {
return s.beaconDB.GenesisState(ctx)
}
lastSaved, err := s.beaconDB.HighestSlotStatesBelow(ctx, slot+1)
if err != nil {
return nil, err
}
// Given this is used to query canonical state. There should only be one saved canonical block of a given slot.
if len(lastSaved) != 1 {
return nil, fmt.Errorf("highest saved state does not equal to 1, it equals to %d", len(lastSaved))
}
if lastSaved[0] == nil {
return nil, errUnknownState
}
return lastSaved[0], nil
}
// This returns the genesis root.
func (s *State) genesisRoot(ctx context.Context) ([32]byte, error) {
b, err := s.beaconDB.GenesisBlock(ctx)
if err != nil {
return [32]byte{}, err
}
return b.Block().HashTreeRoot()
}
// Given the start slot and the end slot, this returns the finalized beacon blocks in between.
// Since hot states don't have finalized blocks, this should ONLY be used for replaying cold state.
func (s *State) loadFinalizedBlocks(ctx context.Context, startSlot, endSlot types.Slot) ([]block.SignedBeaconBlock, error) {

View File

@@ -4,7 +4,6 @@ import (
"context"
"testing"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
testDB "github.com/prysmaticlabs/prysm/beacon-chain/db/testing"
@@ -344,135 +343,6 @@ func TestLoadBlocks_BadStart(t *testing.T) {
assert.ErrorContains(t, "end block roots don't match", err)
}
func TestLastSavedBlock_Genesis(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
s := &State{
beaconDB: beaconDB,
finalizedInfo: &finalizedInfo{slot: 128},
}
gBlk := util.NewBeaconBlock()
gRoot, err := gBlk.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(gBlk)))
require.NoError(t, s.beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
savedRoot, savedSlot, err := s.lastSavedBlock(ctx, 0)
require.NoError(t, err)
assert.Equal(t, types.Slot(0), savedSlot, "Did not save genesis slot")
assert.Equal(t, savedRoot, savedRoot, "Did not save genesis root")
}
func TestLastSavedBlock_CanGet(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
s := &State{
beaconDB: beaconDB,
finalizedInfo: &finalizedInfo{slot: 128},
}
b1 := util.NewBeaconBlock()
b1.Block.Slot = s.finalizedInfo.slot + 5
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b1)))
b2 := util.NewBeaconBlock()
b2.Block.Slot = s.finalizedInfo.slot + 10
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b2)))
b3 := util.NewBeaconBlock()
b3.Block.Slot = s.finalizedInfo.slot + 20
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b3)))
savedRoot, savedSlot, err := s.lastSavedBlock(ctx, s.finalizedInfo.slot+100)
require.NoError(t, err)
assert.Equal(t, s.finalizedInfo.slot+20, savedSlot)
wantedRoot, err := b3.Block.HashTreeRoot()
require.NoError(t, err)
assert.Equal(t, wantedRoot, savedRoot, "Did not save correct root")
}
func TestLastSavedBlock_NoSavedBlock(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
s := &State{
beaconDB: beaconDB,
finalizedInfo: &finalizedInfo{slot: 128},
}
root, slot, err := s.lastSavedBlock(ctx, s.finalizedInfo.slot+1)
require.NoError(t, err)
if slot != 0 && root != [32]byte{} {
t.Error("Did not get wanted block")
}
}
func TestLastSavedState_Genesis(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
s := &State{
beaconDB: beaconDB,
finalizedInfo: &finalizedInfo{slot: 128},
}
gBlk := util.NewBeaconBlock()
gState, err := util.NewBeaconState()
require.NoError(t, err)
gRoot, err := gBlk.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(gBlk)))
require.NoError(t, s.beaconDB.SaveGenesisBlockRoot(ctx, gRoot))
require.NoError(t, s.beaconDB.SaveState(ctx, gState, gRoot))
savedState, err := s.lastSavedState(ctx, 0)
require.NoError(t, err)
require.DeepSSZEqual(t, gState.InnerStateUnsafe(), savedState.InnerStateUnsafe())
}
func TestLastSavedState_CanGet(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
s := &State{
beaconDB: beaconDB,
finalizedInfo: &finalizedInfo{slot: 128},
}
b1 := util.NewBeaconBlock()
b1.Block.Slot = s.finalizedInfo.slot + 5
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b1)))
b2 := util.NewBeaconBlock()
b2.Block.Slot = s.finalizedInfo.slot + 10
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b2)))
b2Root, err := b2.Block.HashTreeRoot()
require.NoError(t, err)
st, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, st.SetSlot(s.finalizedInfo.slot+10))
require.NoError(t, s.beaconDB.SaveState(ctx, st, b2Root))
b3 := util.NewBeaconBlock()
b3.Block.Slot = s.finalizedInfo.slot + 20
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b3)))
savedState, err := s.lastSavedState(ctx, s.finalizedInfo.slot+100)
require.NoError(t, err)
require.DeepSSZEqual(t, st.InnerStateUnsafe(), savedState.InnerStateUnsafe())
}
func TestLastSavedState_NoSavedBlockState(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
s := &State{
beaconDB: beaconDB,
finalizedInfo: &finalizedInfo{slot: 128},
}
b1 := util.NewBeaconBlock()
b1.Block.Slot = 127
require.NoError(t, s.beaconDB.SaveBlock(ctx, wrapper.WrappedPhase0SignedBeaconBlock(b1)))
_, err := s.lastSavedState(ctx, s.finalizedInfo.slot+1)
assert.ErrorContains(t, errUnknownState.Error(), err)
}
// tree1 constructs the following tree:
// B0 - B1 - - B3 -- B5
// \- B2 -- B4 -- B6 ----- B8

View File

@@ -7,10 +7,7 @@ import (
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block"
"github.com/sirupsen/logrus"
"go.opencensus.io/trace"
@@ -22,6 +19,12 @@ var ErrNoBlocksBelowSlot = errors.New("no blocks found in db below slot")
var ErrInvalidDBBlock = errors.New("invalid block found in database")
var ErrReplayTargetSlotExceeded = errors.New("desired replay slot is less than state's slot")
type retrievalMethod int
const (
forSlot retrievalMethod = iota
)
// HistoryAccessor describes the minimum set of database methods needed to support the ReplayerBuilder.
type HistoryAccessor interface {
HighestSlotBlocksBelow(ctx context.Context, slot types.Slot) ([]block.SignedBeaconBlock, error)
@@ -48,12 +51,18 @@ type Replayer interface {
// ReplayToSlot invokes ReplayBlocks under the hood,
// but then also runs process_slots to advance the state past the root or slot used in the builder.
// For example, if you wanted the state to be at the target slot, but only integrating blocks up to
// slot-1, you could request Builder.ForSlot(slot-1).ReplayToSlot(slot)
// slot-1, you could request Builder.ReplayerForSlot(slot-1).ReplayToSlot(slot)
ReplayToSlot(ctx context.Context, target types.Slot) (state.BeaconState, error)
}
var _ Replayer = &stateReplayer{}
// chainer is responsible for supplying the chain components necessary to rebuild a state,
// namely a starting BeaconState and all available blocks from the starting state up to and including the target slot
type chainer interface {
chainForSlot(ctx context.Context, target types.Slot) (state.BeaconState, []block.SignedBeaconBlock, error)
}
type stateReplayer struct {
s state.BeaconState
target types.Slot
@@ -118,7 +127,7 @@ func (rs *stateReplayer) ReplayBlocks(ctx context.Context) (state.BeaconState, e
// ReplayToSlot invokes ReplayBlocks under the hood,
// but then also runs process_slots to advance the state past the root or slot used in the builder.
// for example, if you wanted the state to be at the target slot, but only integrating blocks up to
// slot-1, you could request Builder.ForSlot(slot-1).ReplayToSlot(slot)
// slot-1, you could request Builder.ReplayerForSlot(slot-1).ReplayToSlot(slot)
func (rs *stateReplayer) ReplayToSlot(ctx context.Context, replayTo types.Slot) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "stateGen.stateReplayer.ReplayToSlot")
defer span.End()
@@ -159,232 +168,9 @@ func (rs *stateReplayer) ReplayToSlot(ctx context.Context, replayTo types.Slot)
// (only ForSlot implemented so far).
// See documentation on Replayer for more on how to use this to obtain pre/post-block states
type ReplayerBuilder interface {
// ForSlot creates a builder that will create a state that includes blocks up to and including the requested slot
// ReplayerForSlot creates a builder that will create a state that includes blocks up to and including the requested slot
// The resulting Replayer will always yield a state with .Slot=target; if there are skipped blocks
// between the highest canonical block in the db and the target, the replayer will fast-forward past the intervening
// slots via process_slots.
ForSlot(target types.Slot) Replayer
}
func WithCache(c CachedGetter) CanonicalBuilderOption {
return func(b *CanonicalBuilder) {
b.chainer.useCache(c)
}
}
type CanonicalBuilderOption func(*CanonicalBuilder)
// NewCanonicalBuilder handles initializing the default concrete ReplayerBuilder implementation.
func NewCanonicalBuilder(h HistoryAccessor, c CanonicalChecker, cs CurrentSlotter, opts ...CanonicalBuilderOption) *CanonicalBuilder {
b := &CanonicalBuilder{
chainer: &canonicalChainer{
h: h,
c: c,
cs: cs,
},
}
for _, o := range opts {
o(b)
}
return b
}
type retrievalMethod int
const (
forSlot retrievalMethod = iota
)
// CanonicalBuilder builds a Replayer that uses a combination of database queries and a
// CanonicalChecker (which should usually be a fork choice store implementing an IsCanonical method)
// to determine the canonical chain and apply it to generate the desired state.
type CanonicalBuilder struct {
chainer chainer
}
var _ ReplayerBuilder = &CanonicalBuilder{}
func (r *CanonicalBuilder) ForSlot(target types.Slot) Replayer {
return &stateReplayer{chainer: r.chainer, method: forSlot, target: target}
}
// chainer is responsible for supplying the chain components necessary to rebuild a state,
// namely a starting BeaconState and all available blocks from the starting state up to and including the target slot
type chainer interface {
chainForSlot(ctx context.Context, target types.Slot) (state.BeaconState, []block.SignedBeaconBlock, error)
useCache(c CachedGetter)
}
type canonicalChainer struct {
h HistoryAccessor
c CanonicalChecker
cs CurrentSlotter
cache CachedGetter
}
var _ chainer = &canonicalChainer{}
func (c *canonicalChainer) useCache(cache CachedGetter) {
c.cache = cache
}
// ChainForSlot creates a value that satisfies the Replayer interface via db queries
// and the stategen transition helper methods. This implementation uses the following algorithm:
// - find the highest canonical block <= the target slot
// - starting with this block, recursively search backwards for a stored state, and accumulate intervening blocks
func (c *canonicalChainer) chainForSlot(ctx context.Context, target types.Slot) (state.BeaconState, []block.SignedBeaconBlock, error) {
ctx, span := trace.StartSpan(ctx, "canonicalChainer.chainForSlot")
defer span.End()
currentSlot := c.cs.CurrentSlot()
if target > currentSlot {
return nil, nil, errors.Wrap(ErrFutureSlotRequested, fmt.Sprintf("requested=%d, current=%d", target, currentSlot))
}
_, b, err := c.canonicalBlockForSlot(ctx, target)
if err != nil {
return nil, nil, errors.Wrap(err, fmt.Sprintf("unable to find replay data for slot=%d", target))
}
s, descendants, err := c.ancestorChain(ctx, b)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to query for ancestor and descendant blocks")
}
return s, descendants, nil
}
// canonicalBlockForSlot uses HighestSlotBlocksBelow(target+1) and the CanonicalChecker
// to find the highest canonical block available to replay to the given slot.
func (c *canonicalChainer) canonicalBlockForSlot(ctx context.Context, target types.Slot) ([32]byte, block.SignedBeaconBlock, error) {
for target > 0 {
if ctx.Err() != nil {
return [32]byte{}, nil, errors.Wrap(ctx.Err(), "context canceled during canonicalBlockForSlot")
}
hbs, err := c.h.HighestSlotBlocksBelow(ctx, target+1)
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, fmt.Sprintf("error finding highest block w/ slot <= %d", target))
}
if len(hbs) == 0 {
return [32]byte{}, nil, errors.Wrap(ErrNoBlocksBelowSlot, fmt.Sprintf("slot=%d", target))
}
r, b, err := c.bestForSlot(ctx, hbs)
if err == nil {
// we found a valid, canonical block!
return r, b, nil
}
// we found a block, but it wasn't considered canonical - keep looking
if errors.Is(err, ErrNoCanonicalBlockForSlot) {
// break once we've seen slot 0 (and prevent underflow)
if hbs[0].Block().Slot() == 0 {
break
}
target = hbs[0].Block().Slot() - 1
continue
}
return [32]byte{}, nil, err
}
b, err := c.h.GenesisBlock(ctx)
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, "db error while retrieving genesis block")
}
root, _, err := c.bestForSlot(ctx, []block.SignedBeaconBlock{b})
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, "problem retrieving genesis block")
}
return root, b, nil
}
// bestForSlot encapsulates several messy realities of the underlying db code, looping through multiple blocks,
// performing null/validity checks, and using CanonicalChecker to only pick canonical blocks.
func (c *canonicalChainer) bestForSlot(ctx context.Context, hbs []block.SignedBeaconBlock) ([32]byte, block.SignedBeaconBlock, error) {
for _, b := range hbs {
if helpers.BeaconBlockIsNil(b) != nil {
continue
}
root, err := b.Block().HashTreeRoot()
if err != nil {
// use this error message to wrap a sentinel error for error type matching
wrapped := errors.Wrap(ErrInvalidDBBlock, err.Error())
msg := fmt.Sprintf("could not compute hash_tree_root for block at slot=%d", b.Block().Slot())
return [32]byte{}, nil, errors.Wrap(wrapped, msg)
}
canon, err := c.c.IsCanonical(ctx, root)
if err != nil {
return [32]byte{}, nil, errors.Wrap(err, "replayer could not check if block is canonical")
}
if canon {
return root, b, nil
}
}
return [32]byte{}, nil, errors.Wrap(ErrNoCanonicalBlockForSlot, "no good block for slot")
}
func (c *canonicalChainer) getState(ctx context.Context, root [32]byte) (state.BeaconState, error) {
if c.cache != nil {
st, err := c.cache.ByRoot(root)
if err == nil {
return st, nil
}
if !errors.Is(err, ErrNotInCache) {
return nil, errors.Wrap(err, "error reading from state cache during state replay")
}
}
return c.h.StateOrError(ctx, root)
}
// ancestorChain works backwards through the chain lineage, accumulating blocks and checking for a saved state.
// If it finds a saved state that the tail block was descended from, it returns this state and
// all blocks in the lineage, including the tail block. Blocks are returned in ascending order.
// Note that this function assumes that the tail is a canonical block, and therefore assumes that
// all ancestors are also canonical.
func (c *canonicalChainer) ancestorChain(ctx context.Context, tail block.SignedBeaconBlock) (state.BeaconState, []block.SignedBeaconBlock, error) {
ctx, span := trace.StartSpan(ctx, "canonicalChainer.ancestorChain")
defer span.End()
chain := make([]block.SignedBeaconBlock, 0)
for {
if err := ctx.Err(); err != nil {
msg := fmt.Sprintf("context canceled while finding ancestors of block at slot %d", tail.Block().Slot())
return nil, nil, errors.Wrap(err, msg)
}
b := tail.Block()
// compute hash_tree_root of current block and try to look up the corresponding state
root, err := b.HashTreeRoot()
if err != nil {
msg := fmt.Sprintf("could not compute htr for descendant block at slot=%d", b.Slot())
return nil, nil, errors.Wrap(err, msg)
}
st, err := c.getState(ctx, root)
// err == nil, we've got a real state - the job is done!
// Note: in cases where there are skipped slots we could find a state that is a descendant
// of the block we are searching for. We don't want to return a future block, so in this case
// we keep working backwards.
if err == nil && st.Slot() == b.Slot() {
// we found the state by the root of the head, meaning it has already been applied.
// we only want to return the blocks descended from it.
reverseChain(chain)
return st, chain, nil
}
// ErrNotFoundState errors are fine, but other errors mean something is wrong with the db
if err != nil && !errors.Is(err, db.ErrNotFoundState) {
return nil, nil, errors.Wrap(err, fmt.Sprintf("error querying database for state w/ block root = %#x", root))
}
parent, err := c.h.Block(ctx, bytesutil.ToBytes32(b.ParentRoot()))
if err != nil {
msg := fmt.Sprintf("db error when retrieving parent of block at slot=%d by root=%#x", b.Slot(), b.ParentRoot())
return nil, nil, errors.Wrap(err, msg)
}
if helpers.BeaconBlockIsNil(parent) != nil {
msg := fmt.Sprintf("unable to retrieve parent of block at slot=%d by root=%#x", b.Slot(), b.ParentRoot())
return nil, nil, errors.Wrap(db.ErrNotFound, msg)
}
chain = append(chain, tail)
tail = parent
}
}
func reverseChain(c []block.SignedBeaconBlock) {
last := len(c) - 1
swaps := (last + 1) / 2
for i := 0; i < swaps; i++ {
c[i], c[last-i] = c[last-i], c[i]
}
ReplayerForSlot(target types.Slot) Replayer
}

View File

@@ -2,18 +2,12 @@ package stategen
import (
"context"
"encoding/binary"
"fmt"
"testing"
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block/mock"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/wrapper"
"github.com/prysmaticlabs/prysm/testing/require"
)
@@ -44,8 +38,8 @@ func TestReplayBlocks(t *testing.T) {
}
hist := newMockHistory(t, specs, five+1)
bld := NewCanonicalBuilder(hist, hist, hist)
st, err := bld.ForSlot(five).ReplayBlocks(ctx)
ch := NewCanonicalHistory(hist, hist, hist)
st, err := ch.ReplayerForSlot(five).ReplayBlocks(ctx)
require.NoError(t, err)
expected := hist.hiddenStates[hist.slotMap[five]]
expectedHTR, err := expected.HashTreeRoot(ctx)
@@ -61,7 +55,7 @@ func TestReplayBlocks(t *testing.T) {
require.Equal(t, bytesutil.ToBytes32(expectedLBH.BodyRoot), bytesutil.ToBytes32(actualLBH.BodyRoot))
require.Equal(t, expectedHTR, actualHTR)
st, err = bld.ForSlot(one).ReplayBlocks(ctx)
st, err = ch.ReplayerForSlot(one).ReplayBlocks(ctx)
require.NoError(t, err)
expected = hist.states[hist.slotMap[one]]
@@ -94,9 +88,9 @@ func TestReplayToSlot(t *testing.T) {
// first case tests that ReplayToSlot is equivalent to ReplayBlocks
hist := newMockHistory(t, specs, five+1)
bld := NewCanonicalBuilder(hist, hist, hist)
ch := NewCanonicalHistory(hist, hist, hist)
st, err := bld.ForSlot(five).ReplayToSlot(ctx, five)
st, err := ch.ReplayerForSlot(five).ReplayToSlot(ctx, five)
require.NoError(t, err)
expected := hist.hiddenStates[hist.slotMap[five]]
expectedHTR, err := expected.HashTreeRoot(ctx)
@@ -112,7 +106,7 @@ func TestReplayToSlot(t *testing.T) {
require.Equal(t, bytesutil.ToBytes32(expectedLBH.BodyRoot), bytesutil.ToBytes32(actualLBH.BodyRoot))
require.Equal(t, expectedHTR, actualHTR)
st, err = bld.ForSlot(five).ReplayToSlot(ctx, five+100)
st, err = ch.ReplayerForSlot(five).ReplayToSlot(ctx, five+100)
require.NoError(t, err)
require.Equal(t, five+100, st.Slot())
expectedLBH, err = headerFromBlock(hist.blocks[hist.slotMap[five]])
@@ -124,592 +118,3 @@ func TestReplayToSlot(t *testing.T) {
require.Equal(t, expectedLBH.ProposerIndex, actualLBH.ProposerIndex)
require.Equal(t, bytesutil.ToBytes32(expectedLBH.BodyRoot), bytesutil.ToBytes32(actualLBH.BodyRoot))
}
// happy path tests
func TestCanonicalBlockForSlotHappy(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin},
{slot: middle, savedState: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
cc := canonicalChainer{h: hist, c: hist, cs: hist}
// since only the end block and genesis are canonical, once the slot drops below
// end, we should always get genesis
cases := []struct {
slot types.Slot
highest types.Slot
canon types.Slot
name string
}{
{slot: hist.current, highest: end, canon: end, name: "slot > end"},
{slot: end, highest: end, canon: end, name: "slot == end"},
{slot: end - 1, highest: middle, canon: 0, name: "middle < slot < end"},
{slot: middle, highest: middle, canon: 0, name: "slot == middle"},
{slot: middle - 1, highest: begin, canon: 0, name: "begin < slot < middle"},
{slot: begin, highest: begin, canon: 0, name: "slot == begin"},
{slot: begin - 1, highest: 0, canon: 0, name: "genesis < slot < begin"},
{slot: 0, highest: 0, canon: 0, name: "slot == genesis"},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
bs, err := hist.HighestSlotBlocksBelow(ctx, c.slot+1)
require.NoError(t, err)
require.Equal(t, len(bs), 1)
r, err := bs[0].Block().HashTreeRoot()
require.NoError(t, err)
require.Equal(t, hist.slotMap[c.highest], r)
cr, _, err := cc.canonicalBlockForSlot(ctx, c.slot)
require.NoError(t, err)
require.Equal(t, hist.slotMap[c.canon], cr)
})
}
}
func TestCanonicalBlockForSlotNonHappy(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin},
{slot: middle, savedState: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
slotOrderObserved := make([]types.Slot, 0)
derp := errors.New("HighestSlotBlocksBelow don't work")
// since only the end block and genesis are canonical, once the slot drops below
// end, we should always get genesis
cases := []struct {
name string
slot types.Slot
canon CanonicalChecker
overrideHighest func(context.Context, types.Slot) ([]block.SignedBeaconBlock, error)
slotOrderExpected []types.Slot
err error
root [32]byte
}{
{
name: "HigestSlotBlocksBelow not called for genesis",
overrideHighest: func(_ context.Context, _ types.Slot) ([]block.SignedBeaconBlock, error) {
return nil, derp
},
root: hist.slotMap[0],
},
{
name: "wrapped error from HigestSlotBlocksBelow returned",
err: derp,
overrideHighest: func(_ context.Context, _ types.Slot) ([]block.SignedBeaconBlock, error) {
return nil, derp
},
slot: end,
},
{
name: "HigestSlotBlocksBelow empty list",
err: ErrNoBlocksBelowSlot,
overrideHighest: func(_ context.Context, _ types.Slot) ([]block.SignedBeaconBlock, error) {
return []block.SignedBeaconBlock{}, nil
},
slot: end,
},
{
name: "HigestSlotBlocksBelow no canonical",
err: ErrNoCanonicalBlockForSlot,
canon: &mockCanonicalChecker{is: false},
slot: end,
},
{
name: "slot ordering correct - only genesis canonical",
canon: &mockCanonicalChecker{isCanon: func(root [32]byte) (bool, error) {
if root == hist.slotMap[0] {
return true, nil
}
return false, nil
}},
overrideHighest: func(_ context.Context, s types.Slot) ([]block.SignedBeaconBlock, error) {
slotOrderObserved = append(slotOrderObserved, s)
// this allows the mock HighestSlotBlocksBelow to continue to execute now that we've recorded
// the slot in our channel
return nil, errFallThroughOverride
},
slotOrderExpected: []types.Slot{156, 155, 150, 100},
slot: end,
root: hist.slotMap[0],
},
{
name: "slot ordering correct - slot 100 canonical",
canon: &mockCanonicalChecker{isCanon: func(root [32]byte) (bool, error) {
if root == hist.slotMap[100] {
return true, nil
}
return false, nil
}},
overrideHighest: func(_ context.Context, s types.Slot) ([]block.SignedBeaconBlock, error) {
slotOrderObserved = append(slotOrderObserved, s)
// this allows the mock HighestSlotBlocksBelow to continue to execute now that we've recorded
// the slot in our channel
return nil, errFallThroughOverride
},
slotOrderExpected: []types.Slot{156, 155, 150},
slot: end,
root: hist.slotMap[100],
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
var canon CanonicalChecker = hist
if c.canon != nil {
canon = c.canon
}
cc := canonicalChainer{h: hist, c: canon, cs: hist}
hist.overrideHighestSlotBlocksBelow = c.overrideHighest
r, _, err := cc.canonicalBlockForSlot(ctx, c.slot)
if c.err == nil {
require.NoError(t, err)
} else {
require.ErrorIs(t, err, c.err)
}
if len(c.slotOrderExpected) > 0 {
require.Equal(t, len(c.slotOrderExpected), len(slotOrderObserved), "HighestSlotBlocksBelow not called the expected number of times")
for i := range c.slotOrderExpected {
require.Equal(t, c.slotOrderExpected[i], slotOrderObserved[i])
}
}
if c.root != [32]byte{} {
require.Equal(t, c.root, r)
}
slotOrderObserved = make([]types.Slot, 0)
})
}
}
type mockCurrentSlotter struct {
Slot types.Slot
}
func (c *mockCurrentSlotter) CurrentSlot() types.Slot {
return c.Slot
}
var _ CurrentSlotter = &mockCurrentSlotter{}
func TestCanonicalChainerFuture(t *testing.T) {
r := &canonicalChainer{
cs: &mockCurrentSlotter{Slot: 0},
}
_, _, err := r.chainForSlot(context.Background(), 1)
require.ErrorIs(t, err, ErrFutureSlotRequested)
}
func TestAncestorChainCache(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin, canonicalBlock: true},
{slot: middle, canonicalBlock: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
cc := &canonicalChainer{h: hist, c: hist, cs: hist}
// should only contain the genesis block
require.Equal(t, 1, len(hist.states))
endBlock := hist.blocks[hist.slotMap[end]]
st, bs, err := cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 3, len(bs))
expectedHTR, err := hist.states[hist.slotMap[0]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
// now populate the cache, we should get the cached state instead of genesis
cc.cache = &mockCachedGetter{
cache: map[[32]byte]state.BeaconState{
hist.slotMap[end]: hist.hiddenStates[hist.slotMap[end]],
},
}
st, bs, err = cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 0, len(bs))
expectedHTR, err = hist.hiddenStates[hist.slotMap[end]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
// populate cache with a different state for good measure
cc.cache = &mockCachedGetter{
cache: map[[32]byte]state.BeaconState{
hist.slotMap[begin]: hist.hiddenStates[hist.slotMap[begin]],
},
}
st, bs, err = cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 2, len(bs))
expectedHTR, err = hist.hiddenStates[hist.slotMap[begin]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
// rebuild history w/ last state saved, make sure we get that instead of cache
specs[2].savedState = true
hist = newMockHistory(t, specs, end+1)
cc = &canonicalChainer{h: hist, c: hist, cs: hist}
cc.cache = &mockCachedGetter{
cache: map[[32]byte]state.BeaconState{
hist.slotMap[begin]: hist.hiddenStates[hist.slotMap[begin]],
},
}
st, bs, err = cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
require.Equal(t, 0, len(bs))
expectedHTR, err = hist.states[hist.slotMap[end]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
}
func TestAncestorChainOK(t *testing.T) {
ctx := context.Background()
var begin, middle, end types.Slot = 100, 150, 155
specs := []mockHistorySpec{
{slot: begin},
{slot: middle, savedState: true},
{slot: end, canonicalBlock: true},
}
hist := newMockHistory(t, specs, end+1)
cc := &canonicalChainer{h: hist, c: hist, cs: hist}
endBlock := hist.blocks[hist.slotMap[end]]
st, bs, err := cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
// middle is the most recent slot where savedState == true
require.Equal(t, 1, len(bs))
require.DeepEqual(t, endBlock, bs[0])
expectedHTR, err := hist.states[hist.slotMap[middle]].HashTreeRoot(ctx)
require.NoError(t, err)
actualHTR, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedHTR, actualHTR)
middleBlock := hist.blocks[hist.slotMap[middle]]
st, bs, err = cc.ancestorChain(ctx, middleBlock)
require.NoError(t, err)
actualHTR, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, 0, len(bs))
require.Equal(t, expectedHTR, actualHTR)
}
func TestChainForSlot(t *testing.T) {
ctx := context.Background()
var zero, one, two, three types.Slot = 50, 51, 150, 151
specs := []mockHistorySpec{
{slot: zero, canonicalBlock: true, savedState: true},
{slot: one, canonicalBlock: true},
{slot: two},
{slot: three, canonicalBlock: true},
}
hist := newMockHistory(t, specs, three+10)
cc := &canonicalChainer{h: hist, c: hist, cs: hist}
firstNonGenesisRoot := hist.slotMap[zero]
nonGenesisStateRoot, err := hist.states[firstNonGenesisRoot].HashTreeRoot(ctx)
require.NoError(t, err)
cases := []struct {
name string
slot types.Slot
stateRoot [32]byte
blockRoots [][32]byte
}{
{
name: "above latest slot (but before current slot)",
slot: three + 1,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one], hist.slotMap[two], hist.slotMap[three]},
},
{
name: "last canonical slot - two treated as canonical because it is parent of three",
slot: three,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one], hist.slotMap[two], hist.slotMap[three]},
},
{
name: "non-canonical slot skipped",
slot: two,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one]},
},
{
name: "first canonical slot",
slot: one,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{hist.slotMap[one]},
},
{
name: "slot at saved state",
slot: zero,
stateRoot: nonGenesisStateRoot,
blockRoots: [][32]byte{},
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
st, blocks, err := cc.chainForSlot(ctx, c.slot)
require.NoError(t, err)
actualStRoot, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, c.stateRoot, actualStRoot)
require.Equal(t, len(c.blockRoots), len(blocks))
for i, b := range blocks {
root, err := b.Block().HashTreeRoot()
require.NoError(t, err)
require.Equal(t, c.blockRoots[i], root)
}
})
}
}
func TestAncestorChainOrdering(t *testing.T) {
ctx := context.Background()
var zero, one, two, three, four, five types.Slot = 50, 51, 150, 151, 152, 200
specs := []mockHistorySpec{
{slot: zero},
{slot: one, savedState: true},
{slot: two},
{slot: three},
{slot: four},
{slot: five},
}
hist := newMockHistory(t, specs, five+1)
endRoot := hist.slotMap[specs[len(specs)-1].slot]
endBlock := hist.blocks[endRoot]
cc := &canonicalChainer{h: hist, c: hist, cs: hist}
st, bs, err := cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
expectedRoot, err := hist.states[hist.slotMap[one]].HashTreeRoot(ctx)
require.NoError(t, err)
actualRoot, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedRoot, actualRoot)
// we asked for the chain leading up to five
// one has the savedState. one is applied to the savedState, so it should be omitted
// that means we should get two, three, four, five (length of 4)
require.Equal(t, 4, len(bs))
for i, slot := range []types.Slot{two, three, four, five} {
require.Equal(t, slot, bs[i].Block().Slot(), fmt.Sprintf("wrong value at index %d", i))
}
// do the same query, but with the final state saved
// we should just get the final state w/o block to apply
specs[5].savedState = true
hist = newMockHistory(t, specs, five+1)
endRoot = hist.slotMap[specs[len(specs)-1].slot]
endBlock = hist.blocks[endRoot]
cc = &canonicalChainer{h: hist, c: hist, cs: hist}
st, bs, err = cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
expectedRoot, err = hist.states[endRoot].HashTreeRoot(ctx)
require.NoError(t, err)
actualRoot, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedRoot, actualRoot)
require.Equal(t, 0, len(bs))
// slice off the last element for an odd size list (to cover odd/even in the reverseChain func)
specs = specs[:len(specs)-1]
require.Equal(t, 5, len(specs))
hist = newMockHistory(t, specs, five+1)
cc = &canonicalChainer{h: hist, c: hist, cs: hist}
endRoot = hist.slotMap[specs[len(specs)-1].slot]
endBlock = hist.blocks[endRoot]
st, bs, err = cc.ancestorChain(ctx, endBlock)
require.NoError(t, err)
expectedRoot, err = hist.states[hist.slotMap[one]].HashTreeRoot(ctx)
require.NoError(t, err)
actualRoot, err = st.HashTreeRoot(ctx)
require.NoError(t, err)
require.Equal(t, expectedRoot, actualRoot)
require.Equal(t, 3, len(bs))
for i, slot := range []types.Slot{two, three, four} {
require.Equal(t, slot, bs[i].Block().Slot(), fmt.Sprintf("wrong value at index %d", i))
}
}
func TestBestForSlot(t *testing.T) {
nilBlock, err := wrapper.WrappedSignedBeaconBlock(&ethpb.SignedBeaconBlock{})
require.NoError(t, err)
nilBody, err := wrapper.WrappedSignedBeaconBlock(&ethpb.SignedBeaconBlock{Block: &ethpb.BeaconBlock{}})
require.NoError(t, err)
derp := errors.New("fake hash tree root method no hash good")
badHTR := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{HtrErr: derp, BeaconBlockBody: &mock.BeaconBlockBody{}}}
var goodHTR [32]byte
copy(goodHTR[:], []byte{23})
var betterHTR [32]byte
copy(betterHTR[:], []byte{42})
good := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{BeaconBlockBody: &mock.BeaconBlockBody{}, Htr: goodHTR}}
better := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{BeaconBlockBody: &mock.BeaconBlockBody{}, Htr: betterHTR}}
cases := []struct {
name string
err error
blocks []block.SignedBeaconBlock
best block.SignedBeaconBlock
root [32]byte
cc CanonicalChecker
}{
{
name: "empty list",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{},
},
{
name: "empty SignedBeaconBlock",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{nil},
},
{
name: "empty BeaconBlock",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{nilBlock},
},
{
name: "empty BeaconBlockBody",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{nilBody},
},
{
name: "bad HTR",
err: ErrInvalidDBBlock,
blocks: []block.SignedBeaconBlock{badHTR},
},
{
name: "IsCanonical fail",
blocks: []block.SignedBeaconBlock{good, better},
cc: &mockCanonicalChecker{is: true, err: derp},
err: derp,
},
{
name: "all non-canonical",
err: ErrNoCanonicalBlockForSlot,
blocks: []block.SignedBeaconBlock{good, better},
cc: &mockCanonicalChecker{is: false},
},
{
name: "one canonical",
blocks: []block.SignedBeaconBlock{good},
cc: &mockCanonicalChecker{is: true},
root: goodHTR,
best: good,
},
{
name: "all canonical",
blocks: []block.SignedBeaconBlock{better, good},
cc: &mockCanonicalChecker{is: true},
root: betterHTR,
best: better,
},
{
name: "first wins",
blocks: []block.SignedBeaconBlock{good, better},
cc: &mockCanonicalChecker{is: true},
root: goodHTR,
best: good,
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
chk := CanonicalChecker(&mockCanonicalChecker{is: true})
if c.cc != nil {
chk = c.cc
}
cc := &canonicalChainer{c: chk}
r, b, err := cc.bestForSlot(context.Background(), c.blocks)
if c.err == nil {
require.NoError(t, err)
require.DeepEqual(t, c.best, b)
require.Equal(t, c.root, r)
} else {
require.ErrorIs(t, err, c.err)
}
})
}
}
type mockCanonicalChecker struct {
isCanon func([32]byte) (bool, error)
is bool
err error
}
func (m *mockCanonicalChecker) IsCanonical(_ context.Context, root [32]byte) (bool, error) {
if m.isCanon != nil {
return m.isCanon(root)
}
return m.is, m.err
}
func TestReverseChain(t *testing.T) {
// test 0,1,2,3 elements to handle: zero case; single element; even number; odd number
for i := 0; i < 4; i++ {
t.Run(fmt.Sprintf("reverseChain with %d elements", i), func(t *testing.T) {
actual := mockBlocks(i, incrFwd)
expected := mockBlocks(i, incrBwd)
reverseChain(actual)
if len(actual) != len(expected) {
t.Errorf("different list lengths")
}
for i := 0; i < len(actual); i++ {
sblockA, ok := actual[i].(*mock.SignedBeaconBlock)
require.Equal(t, true, ok)
blockA, ok := sblockA.BeaconBlock.(*mock.BeaconBlock)
require.Equal(t, true, ok)
sblockE, ok := expected[i].(*mock.SignedBeaconBlock)
require.Equal(t, true, ok)
blockE, ok := sblockE.BeaconBlock.(*mock.BeaconBlock)
require.Equal(t, true, ok)
require.Equal(t, blockA.Htr, blockE.Htr)
}
})
}
}
func incrBwd(n int, c chan uint32) {
for i := n - 1; i >= 0; i-- {
c <- uint32(i)
}
close(c)
}
func incrFwd(n int, c chan uint32) {
for i := 0; i < n; i++ {
c <- uint32(i)
}
close(c)
}
func mockBlocks(n int, iter func(int, chan uint32)) []block.SignedBeaconBlock {
bchan := make(chan uint32)
go iter(n, bchan)
mb := make([]block.SignedBeaconBlock, 0)
for i := range bchan {
h := [32]byte{}
binary.LittleEndian.PutUint32(h[:], i)
b := &mock.SignedBeaconBlock{BeaconBlock: &mock.BeaconBlock{BeaconBlockBody: &mock.BeaconBlockBody{}, Htr: h}}
mb = append(mb, b)
}
return mb
}

View File

@@ -33,7 +33,6 @@ type StateManager interface {
StateByRoot(ctx context.Context, blockRoot [32]byte) (state.BeaconState, error)
StateByRootIfCachedNoCopy(blockRoot [32]byte) state.BeaconState
StateByRootInitialSync(ctx context.Context, blockRoot [32]byte) (state.BeaconState, error)
StateBySlot(ctx context.Context, slot types.Slot) (state.BeaconState, error)
RecoverStateSummary(ctx context.Context, blockRoot [32]byte) (*ethpb.StateSummary, error)
SaveState(ctx context.Context, root [32]byte, st state.BeaconState) error
ForceCheckpoint(ctx context.Context, root []byte) error
@@ -49,6 +48,8 @@ type State struct {
finalizedInfo *finalizedInfo
epochBoundaryStateCache *epochBoundaryState
saveHotStateDB *saveHotStateDbConfig
minSlot types.Slot
beaconDBInitType BeaconDBInitType
}
// This tracks the config in the event of long non-finality,
@@ -70,9 +71,31 @@ type finalizedInfo struct {
lock sync.RWMutex
}
func WithMinimumSlot(min types.Slot) StateGenOption {
return func(s *State) {
s.minSlot = min
}
}
type BeaconDBInitType uint
const (
BeaconDBInitTypeGenesisState = iota
BeaconDBInitTypeCheckpoint
)
func WithInitType(t BeaconDBInitType) StateGenOption {
return func(s *State) {
s.beaconDBInitType = t
}
}
// StateGenOption is a functional option for controlling the initialization of a *State value
type StateGenOption func(*State)
// New returns a new state management object.
func New(beaconDB db.NoHeadAccessDatabase) *State {
return &State{
func New(beaconDB db.NoHeadAccessDatabase, opts ...StateGenOption) *State {
s := &State{
beaconDB: beaconDB,
hotStateCache: newHotStateCache(),
finalizedInfo: &finalizedInfo{slot: 0, root: params.BeaconConfig().ZeroHash},
@@ -81,7 +104,13 @@ func New(beaconDB db.NoHeadAccessDatabase) *State {
saveHotStateDB: &saveHotStateDbConfig{
duration: defaultHotStateDBInterval,
},
// defaults to minimumSlot of zero (genesis), overridden by checkpoint sync
minSlot: params.BeaconConfig().GenesisSlot,
}
for _, o := range opts {
o(s)
}
return s
}
// Resume resumes a new state management object from previously saved finalized check point in DB.
@@ -138,3 +167,7 @@ func (s *State) finalizedState() state.BeaconState {
defer s.finalizedInfo.lock.RUnlock()
return s.finalizedInfo.state.Copy()
}
func (s *State) minimumSlot() types.Slot {
return s.minSlot
}

View File

@@ -234,7 +234,7 @@ func (s *Service) processBlock(
s.logSyncStatus(genesis, blk.Block(), blkRoot)
parentRoot := bytesutil.ToBytes32(blk.Block().ParentRoot())
if !s.cfg.DB.HasBlock(ctx, parentRoot) && !s.cfg.Chain.HasInitSyncBlock(parentRoot) {
return fmt.Errorf("%w: %#x", errParentDoesNotExist, blk.Block().ParentRoot())
return fmt.Errorf("%w: (in processBlock, slot=%d) %#x", errParentDoesNotExist, blk.Block().Slot(), blk.Block().ParentRoot())
}
return blockReceiver(ctx, blk, blkRoot)
}
@@ -264,7 +264,7 @@ func (s *Service) processBatchedBlocks(ctx context.Context, genesis time.Time,
s.logBatchSyncStatus(genesis, blks, blkRoot)
parentRoot := bytesutil.ToBytes32(firstBlock.Block().ParentRoot())
if !s.cfg.DB.HasBlock(ctx, parentRoot) && !s.cfg.Chain.HasInitSyncBlock(parentRoot) {
return fmt.Errorf("%w: %#x", errParentDoesNotExist, firstBlock.Block().ParentRoot())
return fmt.Errorf("%w: %#x (in processBatchedBlocks, slot=%d)", errParentDoesNotExist, firstBlock.Block().ParentRoot(), firstBlock.Block().Slot())
}
blockRoots := make([][32]byte, len(blks))
blockRoots[0] = blkRoot

View File

@@ -50,7 +50,7 @@ var file_proto_eth_service_beacon_chain_service_proto_rawDesc = []byte{
0x32, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x21, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x68, 0x2f,
0x76, 0x32, 0x2f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xe8, 0x22, 0x0a, 0x0b, 0x42, 0x65, 0x61, 0x63,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xf6, 0x23, 0x0a, 0x0b, 0x42, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x12, 0x6f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x47, 0x65,
0x6e, 0x65, 0x73, 0x69, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x20, 0x2e,
@@ -58,288 +58,296 @@ var file_proto_eth_service_beacon_chain_service_proto_rawDesc = []byte{
0x47, 0x65, 0x6e, 0x65, 0x73, 0x69, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x27, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x21, 0x12, 0x1f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e,
0x2f, 0x67, 0x65, 0x6e, 0x65, 0x73, 0x69, 0x73, 0x12, 0x89, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74,
0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x2f, 0x67, 0x65, 0x6e, 0x65, 0x73, 0x69, 0x73, 0x12, 0x8b, 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74,
0x57, 0x65, 0x61, 0x6b, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79,
0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x65, 0x61, 0x6b, 0x53,
0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2b, 0x12, 0x29, 0x2f, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x77, 0x65, 0x61, 0x6b, 0x5f, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63,
0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x89, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x53, 0x74,
0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f,
0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93,
0x02, 0x30, 0x12, 0x2e, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74,
0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74,
0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x72, 0x6f,
0x6f, 0x74, 0x12, 0x89, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46,
0x6f, 0x72, 0x6b, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x30, 0x12, 0x2e,
0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31,
0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b,
0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x66, 0x6f, 0x72, 0x6b, 0x12, 0xb1,
0x01, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x43, 0x68,
0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74,
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65,
0x52, 0x6f, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3,
0xe4, 0x93, 0x02, 0x30, 0x12, 0x2e, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f,
0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74,
0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f,
0x72, 0x6f, 0x6f, 0x74, 0x12, 0x89, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74,
0x65, 0x46, 0x6f, 0x72, 0x6b, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x30,
0x12, 0x2e, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f,
0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73,
0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x66, 0x6f, 0x72, 0x6b,
0x12, 0xb1, 0x01, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79,
0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x1d, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74,
0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61,
0x74, 0x65, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70,
0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x46, 0x82, 0xd3,
0xe4, 0x93, 0x02, 0x40, 0x12, 0x3e, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f,
0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74,
0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f,
0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f,
0x69, 0x6e, 0x74, 0x73, 0x12, 0xa1, 0x01, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x56, 0x61, 0x6c,
0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56,
0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f,
0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3c, 0x82, 0xd3, 0xe4, 0x93,
0x02, 0x36, 0x12, 0x34, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74,
0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69,
0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x46, 0x82, 0xd3, 0xe4, 0x93,
0x02, 0x40, 0x12, 0x3e, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74,
0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74,
0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x76, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0xac, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74,
0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x26, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74,
0x65, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4b, 0x82, 0xd3, 0xe4, 0x93,
0x02, 0x45, 0x12, 0x43, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74,
0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74,
0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x76, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61,
0x74, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xb4, 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74,
0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65,
0x73, 0x12, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x61, 0x6c,
0x61, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x56,
0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x73,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x44, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3e,
0x12, 0x3c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f,
0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x66, 0x69,
0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e,
0x74, 0x73, 0x12, 0xa1, 0x01, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x56, 0x61, 0x6c, 0x69, 0x64,
0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x61, 0x6c,
0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3c, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x36,
0x12, 0x34, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f,
0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73,
0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x76, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x12, 0xa1,
0x01, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65,
0x73, 0x12, 0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74,
0x65, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61,
0x74, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3c, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x36, 0x12, 0x34, 0x2f, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74,
0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65,
0x65, 0x73, 0x12, 0xb2, 0x01, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x43,
0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x12, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x74, 0x61, 0x74,
0x65, 0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x79,
0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x41, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3b, 0x12, 0x39, 0x2f, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74,
0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x63, 0x6f, 0x6d,
0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x12, 0x88, 0x01, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x24, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x25, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x27, 0x82, 0xd3, 0xe4, 0x93, 0x02,
0x21, 0x12, 0x1f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68,
0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x68, 0x65, 0x61, 0x64, 0x65,
0x72, 0x73, 0x12, 0x89, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48,
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64,
0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x82, 0xd3, 0xe4, 0x93,
0x02, 0x2c, 0x12, 0x2a, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74,
0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x68, 0x65, 0x61, 0x64,
0x65, 0x72, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x7f,
0x0a, 0x0b, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x2d, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e,
0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x56, 0x32, 0x1a, 0x16, 0x2e, 0x67,
0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45,
0x6d, 0x70, 0x74, 0x79, 0x22, 0x29, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x23, 0x22, 0x1e, 0x2f, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x3a, 0x01, 0x2a, 0x12,
0x89, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74,
0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x30, 0x12, 0x2e, 0x2f, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x72, 0x6f, 0x6f, 0x74, 0x12, 0x7c, 0x0a, 0x08, 0x47,
0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2b, 0x12, 0x29,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0xac, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x56, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x26, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56,
0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x45,
0x12, 0x43, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f,
0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73,
0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x76, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f,
0x72, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xb4, 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x56, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x12,
0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x61, 0x6c, 0x61, 0x6e,
0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, 0x6c,
0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x44, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3e, 0x12, 0x3c,
0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31,
0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b,
0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x86, 0x01, 0x0a, 0x0b, 0x47, 0x65,
0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x53, 0x53, 0x5a, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b,
0x53, 0x53, 0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x35, 0x82, 0xd3, 0xe4,
0x93, 0x02, 0x2f, 0x12, 0x2d, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65,
0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73,
0x73, 0x7a, 0x12, 0x82, 0x01, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56,
0x32, 0x12, 0x1f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x32, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x56, 0x32, 0x1a, 0x20, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x56, 0x32, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2b, 0x12, 0x29, 0x2f, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x32, 0x2f, 0x62,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x8c, 0x01, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x53, 0x53, 0x5a, 0x56, 0x32, 0x12, 0x1f, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x32, 0x1a, 0x23, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x53, 0x53, 0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x56, 0x32, 0x22,
0x35, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2f, 0x12, 0x2d, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x32, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e,
0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69,
0x64, 0x7d, 0x2f, 0x73, 0x73, 0x7a, 0x12, 0xa2, 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73,
0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3e, 0x82, 0xd3, 0xe4,
0x93, 0x02, 0x38, 0x12, 0x36, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65,
0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x61,
0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x9e, 0x01, 0x0a, 0x14,
0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x73, 0x12, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x73, 0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29,
0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b,
0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61,
0x74, 0x6f, 0x72, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x12, 0xa1, 0x01, 0x0a,
0x0e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x12,
0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65,
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65,
0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x3c, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x36, 0x12, 0x34, 0x2f, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74,
0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73,
0x12, 0xb2, 0x01, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d,
0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x12, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53,
0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x79, 0x6e, 0x63,
0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x41, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3b, 0x12, 0x39, 0x2f, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x74, 0x61, 0x74,
0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69,
0x74, 0x74, 0x65, 0x65, 0x73, 0x12, 0x88, 0x01, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x24, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x25, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x27, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x21, 0x12,
0x1f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76,
0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73,
0x12, 0x89, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61,
0x64, 0x65, 0x72, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x24, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2c,
0x12, 0x2a, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f,
0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72,
0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x7f, 0x0a, 0x0b,
0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x2d, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x69,
0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43,
0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x56, 0x32, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f,
0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70,
0x74, 0x79, 0x22, 0x29, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x23, 0x22, 0x1e, 0x2f, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x89, 0x01,
0x0a, 0x0c, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1d,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x50, 0x6f, 0x6f,
0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02,
0x2b, 0x12, 0x29, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68,
0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f,
0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x8e, 0x01, 0x0a,
0x12, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x73, 0x12, 0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x41, 0x74, 0x74, 0x65,
0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x34, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2e, 0x22,
0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x30, 0x12, 0x2e, 0x2f, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x72, 0x6f, 0x6f, 0x74, 0x12, 0x7c, 0x0a, 0x08, 0x47, 0x65, 0x74,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2b, 0x12, 0x29, 0x2f, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x86, 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x53, 0x53, 0x5a, 0x12, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x53, 0x53,
0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x35, 0x82, 0xd3, 0xe4, 0x93, 0x02,
0x2f, 0x12, 0x2d, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68,
0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x73, 0x7a,
0x12, 0x82, 0x01, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x12,
0x1f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x32, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x32,
0x1a, 0x20, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x32, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x56, 0x32, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2b, 0x12, 0x29, 0x2f, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x32, 0x2f, 0x62, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x8c, 0x01, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x53, 0x53, 0x5a, 0x56, 0x32, 0x12, 0x1f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x32, 0x1a, 0x23, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b,
0x53, 0x53, 0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x56, 0x32, 0x22, 0x35, 0x82,
0xd3, 0xe4, 0x93, 0x02, 0x2f, 0x12, 0x2d, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c,
0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x32, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d,
0x2f, 0x73, 0x73, 0x7a, 0x12, 0xa2, 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3e, 0x82, 0xd3, 0xe4, 0x93, 0x02,
0x38, 0x12, 0x36, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68,
0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x73, 0x2f, 0x7b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x61, 0x74, 0x74,
0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x9e, 0x01, 0x0a, 0x14, 0x4c, 0x69,
0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f,
0x6e, 0x73, 0x12, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x73, 0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41,
0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x50, 0x6f, 0x6f, 0x6c, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2b, 0x12,
0x29, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76,
0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x61, 0x74,
0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x9c, 0x01,
0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74,
0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f,
0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d,
0x70, 0x74, 0x79, 0x1a, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c,
0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x12, 0x2f, 0x2f, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74,
0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x8f, 0x01, 0x0a,
0x16, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53,
0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x12, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74,
0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f,
0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70,
0x74, 0x79, 0x22, 0x3a, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x34, 0x22, 0x2f, 0x2f, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65,
0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x9b,
0x01, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x6f,
0x73, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x16, 0x2e, 0x67,
0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45,
0x6d, 0x70, 0x74, 0x79, 0x1a, 0x2d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53,
0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x12, 0x2f, 0x2f, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73,
0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x8f, 0x01, 0x0a,
0x16, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53,
0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x12, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73,
0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f,
0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70,
0x74, 0x79, 0x22, 0x3a, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x34, 0x22, 0x2f, 0x2f, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65,
0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x93,
0x01, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x56, 0x6f, 0x6c, 0x75, 0x6e,
0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74,
0x79, 0x1a, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69,
0x74, 0x73, 0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x34,
0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2e, 0x12, 0x2c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f,
0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65,
0x78, 0x69, 0x74, 0x73, 0x12, 0x8c, 0x01, 0x0a, 0x13, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x56,
0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x12, 0x24, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53,
0x69, 0x67, 0x6e, 0x65, 0x64, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78,
0x69, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93,
0x02, 0x31, 0x22, 0x2c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74,
0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c,
0x2f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x73,
0x3a, 0x01, 0x2a, 0x12, 0xa8, 0x01, 0x0a, 0x21, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x50, 0x6f,
0x6f, 0x6c, 0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x53,
0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x12, 0x32, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x75, 0x62, 0x6d,
0x69, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74,
0x74, 0x65, 0x65, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x1a, 0x16, 0x2e,
0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x8e, 0x01, 0x0a, 0x12, 0x53,
0x75, 0x62, 0x6d, 0x69, 0x74, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x73, 0x12, 0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x2e, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74,
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e,
0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x22, 0x2c, 0x2f,
0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x34, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2e, 0x22, 0x29, 0x2f,
0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f,
0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x73, 0x79, 0x6e, 0x63,
0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x7f,
0x0a, 0x0f, 0x47, 0x65, 0x74, 0x46, 0x6f, 0x72, 0x6b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c,
0x65, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x25, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x6f, 0x72, 0x6b,
0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x2d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x27, 0x12, 0x25, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72,
0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69,
0x67, 0x2f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12,
0x66, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x53, 0x70, 0x65, 0x63, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f,
0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x61, 0x74, 0x74, 0x65,
0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x9c, 0x01, 0x0a, 0x19,
0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72,
0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74,
0x79, 0x1a, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73,
0x68, 0x69, 0x6e, 0x67, 0x73, 0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x12, 0x2f, 0x2f, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72,
0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x8f, 0x01, 0x0a, 0x16, 0x53,
0x75, 0x62, 0x6d, 0x69, 0x74, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c, 0x61,
0x73, 0x68, 0x69, 0x6e, 0x67, 0x12, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72,
0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79,
0x22, 0x3a, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x34, 0x22, 0x2f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72,
0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f,
0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x5f,
0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x9b, 0x01, 0x0a,
0x19, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65,
0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f,
0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70,
0x74, 0x79, 0x1a, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x24, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1e, 0x12, 0x1c, 0x2f, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x66,
0x69, 0x67, 0x2f, 0x73, 0x70, 0x65, 0x63, 0x12, 0x88, 0x01, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x44,
0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x12, 0x16,
0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
0x43, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x30, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2a, 0x12, 0x28, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72,
0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69,
0x67, 0x2f, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61,
0x63, 0x74, 0x42, 0x95, 0x01, 0x0a, 0x18, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x42,
0x17, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x72, 0x76,
0x69, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x30, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2f, 0x65, 0x74, 0x68, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0xaa, 0x02, 0x14, 0x45,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x53, 0x65, 0x72, 0x76,
0x69, 0x63, 0x65, 0xca, 0x02, 0x14, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45,
0x74, 0x68, 0x5c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x33,
0x74, 0x79, 0x1a, 0x2d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61,
0x73, 0x68, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x12, 0x2f, 0x2f, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72,
0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x8f, 0x01, 0x0a, 0x16, 0x53,
0x75, 0x62, 0x6d, 0x69, 0x74, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61,
0x73, 0x68, 0x69, 0x6e, 0x67, 0x12, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72,
0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79,
0x22, 0x3a, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x34, 0x22, 0x2f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72,
0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f,
0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f,
0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x93, 0x01, 0x0a,
0x16, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6f, 0x6c, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61,
0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a,
0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x73,
0x50, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x34, 0x82, 0xd3,
0xe4, 0x93, 0x02, 0x2e, 0x12, 0x2c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f,
0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f,
0x6f, 0x6c, 0x2f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69,
0x74, 0x73, 0x12, 0x8c, 0x01, 0x0a, 0x13, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x56, 0x6f, 0x6c,
0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x12, 0x24, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67,
0x6e, 0x65, 0x64, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74,
0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31,
0x22, 0x2c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f,
0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x76,
0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x73, 0x3a, 0x01,
0x2a, 0x12, 0xa8, 0x01, 0x0a, 0x21, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x50, 0x6f, 0x6f, 0x6c,
0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x53, 0x69, 0x67,
0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x12, 0x32, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74,
0x50, 0x6f, 0x6f, 0x6c, 0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65,
0x65, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x1a, 0x16, 0x2e, 0x67, 0x6f,
0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d,
0x70, 0x74, 0x79, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x22, 0x2c, 0x2f, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x2f, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x63,
0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x73, 0x3a, 0x01, 0x2a, 0x12, 0x7f, 0x0a, 0x0f,
0x47, 0x65, 0x74, 0x46, 0x6f, 0x72, 0x6b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12,
0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x25, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x53, 0x63,
0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2d,
0x82, 0xd3, 0xe4, 0x93, 0x02, 0x27, 0x12, 0x25, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2f,
0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x66, 0x0a,
0x07, 0x47, 0x65, 0x74, 0x53, 0x70, 0x65, 0x63, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79,
0x1a, 0x1d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x2e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x24, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1e, 0x12, 0x1c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
0x61, 0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67,
0x2f, 0x73, 0x70, 0x65, 0x63, 0x12, 0x88, 0x01, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x44, 0x65, 0x70,
0x6f, 0x73, 0x69, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x12, 0x16, 0x2e, 0x67,
0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45,
0x6d, 0x70, 0x74, 0x79, 0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x43, 0x6f,
0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x30,
0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2a, 0x12, 0x28, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2f,
0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74,
0x42, 0x95, 0x01, 0x0a, 0x18, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x42, 0x17, 0x42,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x30, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65,
0x74, 0x68, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0xaa, 0x02, 0x14, 0x45, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
0x65, 0xca, 0x02, 0x14, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68,
0x5c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var file_proto_eth_service_beacon_chain_service_proto_goTypes = []interface{}{
@@ -361,93 +369,96 @@ var file_proto_eth_service_beacon_chain_service_proto_goTypes = []interface{}{
(*v1.SignedVoluntaryExit)(nil), // 15: ethereum.eth.v1.SignedVoluntaryExit
(*v2.SubmitPoolSyncCommitteeSignatures)(nil), // 16: ethereum.eth.v2.SubmitPoolSyncCommitteeSignatures
(*v1.GenesisResponse)(nil), // 17: ethereum.eth.v1.GenesisResponse
(*v1.StateRootResponse)(nil), // 18: ethereum.eth.v1.StateRootResponse
(*v1.StateForkResponse)(nil), // 19: ethereum.eth.v1.StateForkResponse
(*v1.StateFinalityCheckpointResponse)(nil), // 20: ethereum.eth.v1.StateFinalityCheckpointResponse
(*v1.StateValidatorsResponse)(nil), // 21: ethereum.eth.v1.StateValidatorsResponse
(*v1.StateValidatorResponse)(nil), // 22: ethereum.eth.v1.StateValidatorResponse
(*v1.ValidatorBalancesResponse)(nil), // 23: ethereum.eth.v1.ValidatorBalancesResponse
(*v1.StateCommitteesResponse)(nil), // 24: ethereum.eth.v1.StateCommitteesResponse
(*v2.StateSyncCommitteesResponse)(nil), // 25: ethereum.eth.v2.StateSyncCommitteesResponse
(*v1.BlockHeadersResponse)(nil), // 26: ethereum.eth.v1.BlockHeadersResponse
(*v1.BlockHeaderResponse)(nil), // 27: ethereum.eth.v1.BlockHeaderResponse
(*v1.BlockRootResponse)(nil), // 28: ethereum.eth.v1.BlockRootResponse
(*v1.BlockResponse)(nil), // 29: ethereum.eth.v1.BlockResponse
(*v1.BlockSSZResponse)(nil), // 30: ethereum.eth.v1.BlockSSZResponse
(*v2.BlockResponseV2)(nil), // 31: ethereum.eth.v2.BlockResponseV2
(*v2.BlockSSZResponseV2)(nil), // 32: ethereum.eth.v2.BlockSSZResponseV2
(*v1.BlockAttestationsResponse)(nil), // 33: ethereum.eth.v1.BlockAttestationsResponse
(*v1.AttestationsPoolResponse)(nil), // 34: ethereum.eth.v1.AttestationsPoolResponse
(*v1.AttesterSlashingsPoolResponse)(nil), // 35: ethereum.eth.v1.AttesterSlashingsPoolResponse
(*v1.ProposerSlashingPoolResponse)(nil), // 36: ethereum.eth.v1.ProposerSlashingPoolResponse
(*v1.VoluntaryExitsPoolResponse)(nil), // 37: ethereum.eth.v1.VoluntaryExitsPoolResponse
(*v1.ForkScheduleResponse)(nil), // 38: ethereum.eth.v1.ForkScheduleResponse
(*v1.SpecResponse)(nil), // 39: ethereum.eth.v1.SpecResponse
(*v1.DepositContractResponse)(nil), // 40: ethereum.eth.v1.DepositContractResponse
(*v1.WeakSubjectivityResponse)(nil), // 18: ethereum.eth.v1.WeakSubjectivityResponse
(*v1.StateRootResponse)(nil), // 19: ethereum.eth.v1.StateRootResponse
(*v1.StateForkResponse)(nil), // 20: ethereum.eth.v1.StateForkResponse
(*v1.StateFinalityCheckpointResponse)(nil), // 21: ethereum.eth.v1.StateFinalityCheckpointResponse
(*v1.StateValidatorsResponse)(nil), // 22: ethereum.eth.v1.StateValidatorsResponse
(*v1.StateValidatorResponse)(nil), // 23: ethereum.eth.v1.StateValidatorResponse
(*v1.ValidatorBalancesResponse)(nil), // 24: ethereum.eth.v1.ValidatorBalancesResponse
(*v1.StateCommitteesResponse)(nil), // 25: ethereum.eth.v1.StateCommitteesResponse
(*v2.StateSyncCommitteesResponse)(nil), // 26: ethereum.eth.v2.StateSyncCommitteesResponse
(*v1.BlockHeadersResponse)(nil), // 27: ethereum.eth.v1.BlockHeadersResponse
(*v1.BlockHeaderResponse)(nil), // 28: ethereum.eth.v1.BlockHeaderResponse
(*v1.BlockRootResponse)(nil), // 29: ethereum.eth.v1.BlockRootResponse
(*v1.BlockResponse)(nil), // 30: ethereum.eth.v1.BlockResponse
(*v1.BlockSSZResponse)(nil), // 31: ethereum.eth.v1.BlockSSZResponse
(*v2.BlockResponseV2)(nil), // 32: ethereum.eth.v2.BlockResponseV2
(*v2.BlockSSZResponseV2)(nil), // 33: ethereum.eth.v2.BlockSSZResponseV2
(*v1.BlockAttestationsResponse)(nil), // 34: ethereum.eth.v1.BlockAttestationsResponse
(*v1.AttestationsPoolResponse)(nil), // 35: ethereum.eth.v1.AttestationsPoolResponse
(*v1.AttesterSlashingsPoolResponse)(nil), // 36: ethereum.eth.v1.AttesterSlashingsPoolResponse
(*v1.ProposerSlashingPoolResponse)(nil), // 37: ethereum.eth.v1.ProposerSlashingPoolResponse
(*v1.VoluntaryExitsPoolResponse)(nil), // 38: ethereum.eth.v1.VoluntaryExitsPoolResponse
(*v1.ForkScheduleResponse)(nil), // 39: ethereum.eth.v1.ForkScheduleResponse
(*v1.SpecResponse)(nil), // 40: ethereum.eth.v1.SpecResponse
(*v1.DepositContractResponse)(nil), // 41: ethereum.eth.v1.DepositContractResponse
}
var file_proto_eth_service_beacon_chain_service_proto_depIdxs = []int32{
0, // 0: ethereum.eth.service.BeaconChain.GetGenesis:input_type -> google.protobuf.Empty
1, // 1: ethereum.eth.service.BeaconChain.GetStateRoot:input_type -> ethereum.eth.v1.StateRequest
1, // 2: ethereum.eth.service.BeaconChain.GetStateFork:input_type -> ethereum.eth.v1.StateRequest
1, // 3: ethereum.eth.service.BeaconChain.GetFinalityCheckpoints:input_type -> ethereum.eth.v1.StateRequest
2, // 4: ethereum.eth.service.BeaconChain.ListValidators:input_type -> ethereum.eth.v1.StateValidatorsRequest
3, // 5: ethereum.eth.service.BeaconChain.GetValidator:input_type -> ethereum.eth.v1.StateValidatorRequest
4, // 6: ethereum.eth.service.BeaconChain.ListValidatorBalances:input_type -> ethereum.eth.v1.ValidatorBalancesRequest
5, // 7: ethereum.eth.service.BeaconChain.ListCommittees:input_type -> ethereum.eth.v1.StateCommitteesRequest
6, // 8: ethereum.eth.service.BeaconChain.ListSyncCommittees:input_type -> ethereum.eth.v2.StateSyncCommitteesRequest
7, // 9: ethereum.eth.service.BeaconChain.ListBlockHeaders:input_type -> ethereum.eth.v1.BlockHeadersRequest
8, // 10: ethereum.eth.service.BeaconChain.GetBlockHeader:input_type -> ethereum.eth.v1.BlockRequest
9, // 11: ethereum.eth.service.BeaconChain.SubmitBlock:input_type -> ethereum.eth.v2.SignedBeaconBlockContainerV2
8, // 12: ethereum.eth.service.BeaconChain.GetBlockRoot:input_type -> ethereum.eth.v1.BlockRequest
8, // 13: ethereum.eth.service.BeaconChain.GetBlock:input_type -> ethereum.eth.v1.BlockRequest
8, // 14: ethereum.eth.service.BeaconChain.GetBlockSSZ:input_type -> ethereum.eth.v1.BlockRequest
10, // 15: ethereum.eth.service.BeaconChain.GetBlockV2:input_type -> ethereum.eth.v2.BlockRequestV2
10, // 16: ethereum.eth.service.BeaconChain.GetBlockSSZV2:input_type -> ethereum.eth.v2.BlockRequestV2
8, // 17: ethereum.eth.service.BeaconChain.ListBlockAttestations:input_type -> ethereum.eth.v1.BlockRequest
11, // 18: ethereum.eth.service.BeaconChain.ListPoolAttestations:input_type -> ethereum.eth.v1.AttestationsPoolRequest
12, // 19: ethereum.eth.service.BeaconChain.SubmitAttestations:input_type -> ethereum.eth.v1.SubmitAttestationsRequest
0, // 20: ethereum.eth.service.BeaconChain.ListPoolAttesterSlashings:input_type -> google.protobuf.Empty
13, // 21: ethereum.eth.service.BeaconChain.SubmitAttesterSlashing:input_type -> ethereum.eth.v1.AttesterSlashing
0, // 22: ethereum.eth.service.BeaconChain.ListPoolProposerSlashings:input_type -> google.protobuf.Empty
14, // 23: ethereum.eth.service.BeaconChain.SubmitProposerSlashing:input_type -> ethereum.eth.v1.ProposerSlashing
0, // 24: ethereum.eth.service.BeaconChain.ListPoolVoluntaryExits:input_type -> google.protobuf.Empty
15, // 25: ethereum.eth.service.BeaconChain.SubmitVoluntaryExit:input_type -> ethereum.eth.v1.SignedVoluntaryExit
16, // 26: ethereum.eth.service.BeaconChain.SubmitPoolSyncCommitteeSignatures:input_type -> ethereum.eth.v2.SubmitPoolSyncCommitteeSignatures
0, // 27: ethereum.eth.service.BeaconChain.GetForkSchedule:input_type -> google.protobuf.Empty
0, // 28: ethereum.eth.service.BeaconChain.GetSpec:input_type -> google.protobuf.Empty
0, // 29: ethereum.eth.service.BeaconChain.GetDepositContract:input_type -> google.protobuf.Empty
17, // 30: ethereum.eth.service.BeaconChain.GetGenesis:output_type -> ethereum.eth.v1.GenesisResponse
18, // 31: ethereum.eth.service.BeaconChain.GetStateRoot:output_type -> ethereum.eth.v1.StateRootResponse
19, // 32: ethereum.eth.service.BeaconChain.GetStateFork:output_type -> ethereum.eth.v1.StateForkResponse
20, // 33: ethereum.eth.service.BeaconChain.GetFinalityCheckpoints:output_type -> ethereum.eth.v1.StateFinalityCheckpointResponse
21, // 34: ethereum.eth.service.BeaconChain.ListValidators:output_type -> ethereum.eth.v1.StateValidatorsResponse
22, // 35: ethereum.eth.service.BeaconChain.GetValidator:output_type -> ethereum.eth.v1.StateValidatorResponse
23, // 36: ethereum.eth.service.BeaconChain.ListValidatorBalances:output_type -> ethereum.eth.v1.ValidatorBalancesResponse
24, // 37: ethereum.eth.service.BeaconChain.ListCommittees:output_type -> ethereum.eth.v1.StateCommitteesResponse
25, // 38: ethereum.eth.service.BeaconChain.ListSyncCommittees:output_type -> ethereum.eth.v2.StateSyncCommitteesResponse
26, // 39: ethereum.eth.service.BeaconChain.ListBlockHeaders:output_type -> ethereum.eth.v1.BlockHeadersResponse
27, // 40: ethereum.eth.service.BeaconChain.GetBlockHeader:output_type -> ethereum.eth.v1.BlockHeaderResponse
0, // 41: ethereum.eth.service.BeaconChain.SubmitBlock:output_type -> google.protobuf.Empty
28, // 42: ethereum.eth.service.BeaconChain.GetBlockRoot:output_type -> ethereum.eth.v1.BlockRootResponse
29, // 43: ethereum.eth.service.BeaconChain.GetBlock:output_type -> ethereum.eth.v1.BlockResponse
30, // 44: ethereum.eth.service.BeaconChain.GetBlockSSZ:output_type -> ethereum.eth.v1.BlockSSZResponse
31, // 45: ethereum.eth.service.BeaconChain.GetBlockV2:output_type -> ethereum.eth.v2.BlockResponseV2
32, // 46: ethereum.eth.service.BeaconChain.GetBlockSSZV2:output_type -> ethereum.eth.v2.BlockSSZResponseV2
33, // 47: ethereum.eth.service.BeaconChain.ListBlockAttestations:output_type -> ethereum.eth.v1.BlockAttestationsResponse
34, // 48: ethereum.eth.service.BeaconChain.ListPoolAttestations:output_type -> ethereum.eth.v1.AttestationsPoolResponse
0, // 49: ethereum.eth.service.BeaconChain.SubmitAttestations:output_type -> google.protobuf.Empty
35, // 50: ethereum.eth.service.BeaconChain.ListPoolAttesterSlashings:output_type -> ethereum.eth.v1.AttesterSlashingsPoolResponse
0, // 51: ethereum.eth.service.BeaconChain.SubmitAttesterSlashing:output_type -> google.protobuf.Empty
36, // 52: ethereum.eth.service.BeaconChain.ListPoolProposerSlashings:output_type -> ethereum.eth.v1.ProposerSlashingPoolResponse
0, // 53: ethereum.eth.service.BeaconChain.SubmitProposerSlashing:output_type -> google.protobuf.Empty
37, // 54: ethereum.eth.service.BeaconChain.ListPoolVoluntaryExits:output_type -> ethereum.eth.v1.VoluntaryExitsPoolResponse
0, // 55: ethereum.eth.service.BeaconChain.SubmitVoluntaryExit:output_type -> google.protobuf.Empty
0, // 56: ethereum.eth.service.BeaconChain.SubmitPoolSyncCommitteeSignatures:output_type -> google.protobuf.Empty
38, // 57: ethereum.eth.service.BeaconChain.GetForkSchedule:output_type -> ethereum.eth.v1.ForkScheduleResponse
39, // 58: ethereum.eth.service.BeaconChain.GetSpec:output_type -> ethereum.eth.v1.SpecResponse
40, // 59: ethereum.eth.service.BeaconChain.GetDepositContract:output_type -> ethereum.eth.v1.DepositContractResponse
30, // [30:60] is the sub-list for method output_type
0, // [0:30] is the sub-list for method input_type
0, // 1: ethereum.eth.service.BeaconChain.GetWeakSubjectivity:input_type -> google.protobuf.Empty
1, // 2: ethereum.eth.service.BeaconChain.GetStateRoot:input_type -> ethereum.eth.v1.StateRequest
1, // 3: ethereum.eth.service.BeaconChain.GetStateFork:input_type -> ethereum.eth.v1.StateRequest
1, // 4: ethereum.eth.service.BeaconChain.GetFinalityCheckpoints:input_type -> ethereum.eth.v1.StateRequest
2, // 5: ethereum.eth.service.BeaconChain.ListValidators:input_type -> ethereum.eth.v1.StateValidatorsRequest
3, // 6: ethereum.eth.service.BeaconChain.GetValidator:input_type -> ethereum.eth.v1.StateValidatorRequest
4, // 7: ethereum.eth.service.BeaconChain.ListValidatorBalances:input_type -> ethereum.eth.v1.ValidatorBalancesRequest
5, // 8: ethereum.eth.service.BeaconChain.ListCommittees:input_type -> ethereum.eth.v1.StateCommitteesRequest
6, // 9: ethereum.eth.service.BeaconChain.ListSyncCommittees:input_type -> ethereum.eth.v2.StateSyncCommitteesRequest
7, // 10: ethereum.eth.service.BeaconChain.ListBlockHeaders:input_type -> ethereum.eth.v1.BlockHeadersRequest
8, // 11: ethereum.eth.service.BeaconChain.GetBlockHeader:input_type -> ethereum.eth.v1.BlockRequest
9, // 12: ethereum.eth.service.BeaconChain.SubmitBlock:input_type -> ethereum.eth.v2.SignedBeaconBlockContainerV2
8, // 13: ethereum.eth.service.BeaconChain.GetBlockRoot:input_type -> ethereum.eth.v1.BlockRequest
8, // 14: ethereum.eth.service.BeaconChain.GetBlock:input_type -> ethereum.eth.v1.BlockRequest
8, // 15: ethereum.eth.service.BeaconChain.GetBlockSSZ:input_type -> ethereum.eth.v1.BlockRequest
10, // 16: ethereum.eth.service.BeaconChain.GetBlockV2:input_type -> ethereum.eth.v2.BlockRequestV2
10, // 17: ethereum.eth.service.BeaconChain.GetBlockSSZV2:input_type -> ethereum.eth.v2.BlockRequestV2
8, // 18: ethereum.eth.service.BeaconChain.ListBlockAttestations:input_type -> ethereum.eth.v1.BlockRequest
11, // 19: ethereum.eth.service.BeaconChain.ListPoolAttestations:input_type -> ethereum.eth.v1.AttestationsPoolRequest
12, // 20: ethereum.eth.service.BeaconChain.SubmitAttestations:input_type -> ethereum.eth.v1.SubmitAttestationsRequest
0, // 21: ethereum.eth.service.BeaconChain.ListPoolAttesterSlashings:input_type -> google.protobuf.Empty
13, // 22: ethereum.eth.service.BeaconChain.SubmitAttesterSlashing:input_type -> ethereum.eth.v1.AttesterSlashing
0, // 23: ethereum.eth.service.BeaconChain.ListPoolProposerSlashings:input_type -> google.protobuf.Empty
14, // 24: ethereum.eth.service.BeaconChain.SubmitProposerSlashing:input_type -> ethereum.eth.v1.ProposerSlashing
0, // 25: ethereum.eth.service.BeaconChain.ListPoolVoluntaryExits:input_type -> google.protobuf.Empty
15, // 26: ethereum.eth.service.BeaconChain.SubmitVoluntaryExit:input_type -> ethereum.eth.v1.SignedVoluntaryExit
16, // 27: ethereum.eth.service.BeaconChain.SubmitPoolSyncCommitteeSignatures:input_type -> ethereum.eth.v2.SubmitPoolSyncCommitteeSignatures
0, // 28: ethereum.eth.service.BeaconChain.GetForkSchedule:input_type -> google.protobuf.Empty
0, // 29: ethereum.eth.service.BeaconChain.GetSpec:input_type -> google.protobuf.Empty
0, // 30: ethereum.eth.service.BeaconChain.GetDepositContract:input_type -> google.protobuf.Empty
17, // 31: ethereum.eth.service.BeaconChain.GetGenesis:output_type -> ethereum.eth.v1.GenesisResponse
18, // 32: ethereum.eth.service.BeaconChain.GetWeakSubjectivity:output_type -> ethereum.eth.v1.WeakSubjectivityResponse
19, // 33: ethereum.eth.service.BeaconChain.GetStateRoot:output_type -> ethereum.eth.v1.StateRootResponse
20, // 34: ethereum.eth.service.BeaconChain.GetStateFork:output_type -> ethereum.eth.v1.StateForkResponse
21, // 35: ethereum.eth.service.BeaconChain.GetFinalityCheckpoints:output_type -> ethereum.eth.v1.StateFinalityCheckpointResponse
22, // 36: ethereum.eth.service.BeaconChain.ListValidators:output_type -> ethereum.eth.v1.StateValidatorsResponse
23, // 37: ethereum.eth.service.BeaconChain.GetValidator:output_type -> ethereum.eth.v1.StateValidatorResponse
24, // 38: ethereum.eth.service.BeaconChain.ListValidatorBalances:output_type -> ethereum.eth.v1.ValidatorBalancesResponse
25, // 39: ethereum.eth.service.BeaconChain.ListCommittees:output_type -> ethereum.eth.v1.StateCommitteesResponse
26, // 40: ethereum.eth.service.BeaconChain.ListSyncCommittees:output_type -> ethereum.eth.v2.StateSyncCommitteesResponse
27, // 41: ethereum.eth.service.BeaconChain.ListBlockHeaders:output_type -> ethereum.eth.v1.BlockHeadersResponse
28, // 42: ethereum.eth.service.BeaconChain.GetBlockHeader:output_type -> ethereum.eth.v1.BlockHeaderResponse
0, // 43: ethereum.eth.service.BeaconChain.SubmitBlock:output_type -> google.protobuf.Empty
29, // 44: ethereum.eth.service.BeaconChain.GetBlockRoot:output_type -> ethereum.eth.v1.BlockRootResponse
30, // 45: ethereum.eth.service.BeaconChain.GetBlock:output_type -> ethereum.eth.v1.BlockResponse
31, // 46: ethereum.eth.service.BeaconChain.GetBlockSSZ:output_type -> ethereum.eth.v1.BlockSSZResponse
32, // 47: ethereum.eth.service.BeaconChain.GetBlockV2:output_type -> ethereum.eth.v2.BlockResponseV2
33, // 48: ethereum.eth.service.BeaconChain.GetBlockSSZV2:output_type -> ethereum.eth.v2.BlockSSZResponseV2
34, // 49: ethereum.eth.service.BeaconChain.ListBlockAttestations:output_type -> ethereum.eth.v1.BlockAttestationsResponse
35, // 50: ethereum.eth.service.BeaconChain.ListPoolAttestations:output_type -> ethereum.eth.v1.AttestationsPoolResponse
0, // 51: ethereum.eth.service.BeaconChain.SubmitAttestations:output_type -> google.protobuf.Empty
36, // 52: ethereum.eth.service.BeaconChain.ListPoolAttesterSlashings:output_type -> ethereum.eth.v1.AttesterSlashingsPoolResponse
0, // 53: ethereum.eth.service.BeaconChain.SubmitAttesterSlashing:output_type -> google.protobuf.Empty
37, // 54: ethereum.eth.service.BeaconChain.ListPoolProposerSlashings:output_type -> ethereum.eth.v1.ProposerSlashingPoolResponse
0, // 55: ethereum.eth.service.BeaconChain.SubmitProposerSlashing:output_type -> google.protobuf.Empty
38, // 56: ethereum.eth.service.BeaconChain.ListPoolVoluntaryExits:output_type -> ethereum.eth.v1.VoluntaryExitsPoolResponse
0, // 57: ethereum.eth.service.BeaconChain.SubmitVoluntaryExit:output_type -> google.protobuf.Empty
0, // 58: ethereum.eth.service.BeaconChain.SubmitPoolSyncCommitteeSignatures:output_type -> google.protobuf.Empty
39, // 59: ethereum.eth.service.BeaconChain.GetForkSchedule:output_type -> ethereum.eth.v1.ForkScheduleResponse
40, // 60: ethereum.eth.service.BeaconChain.GetSpec:output_type -> ethereum.eth.v1.SpecResponse
41, // 61: ethereum.eth.service.BeaconChain.GetDepositContract:output_type -> ethereum.eth.v1.DepositContractResponse
31, // [31:62] is the sub-list for method output_type
0, // [0:31] is the sub-list for method input_type
0, // [0:0] is the sub-list for extension type_name
0, // [0:0] is the sub-list for extension extendee
0, // [0:0] is the sub-list for field type_name
@@ -490,6 +501,7 @@ const _ = grpc.SupportPackageIsVersion6
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
type BeaconChainClient interface {
GetGenesis(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*v1.GenesisResponse, error)
GetWeakSubjectivity(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*v1.WeakSubjectivityResponse, error)
GetStateRoot(ctx context.Context, in *v1.StateRequest, opts ...grpc.CallOption) (*v1.StateRootResponse, error)
GetStateFork(ctx context.Context, in *v1.StateRequest, opts ...grpc.CallOption) (*v1.StateForkResponse, error)
GetFinalityCheckpoints(ctx context.Context, in *v1.StateRequest, opts ...grpc.CallOption) (*v1.StateFinalityCheckpointResponse, error)
@@ -538,6 +550,15 @@ func (c *beaconChainClient) GetGenesis(ctx context.Context, in *empty.Empty, opt
return out, nil
}
func (c *beaconChainClient) GetWeakSubjectivity(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*v1.WeakSubjectivityResponse, error) {
out := new(v1.WeakSubjectivityResponse)
err := c.cc.Invoke(ctx, "/ethereum.eth.service.BeaconChain/GetWeakSubjectivity", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *beaconChainClient) GetStateRoot(ctx context.Context, in *v1.StateRequest, opts ...grpc.CallOption) (*v1.StateRootResponse, error) {
out := new(v1.StateRootResponse)
err := c.cc.Invoke(ctx, "/ethereum.eth.service.BeaconChain/GetStateRoot", in, out, opts...)
@@ -802,6 +823,7 @@ func (c *beaconChainClient) GetDepositContract(ctx context.Context, in *empty.Em
// BeaconChainServer is the server API for BeaconChain service.
type BeaconChainServer interface {
GetGenesis(context.Context, *empty.Empty) (*v1.GenesisResponse, error)
GetWeakSubjectivity(context.Context, *empty.Empty) (*v1.WeakSubjectivityResponse, error)
GetStateRoot(context.Context, *v1.StateRequest) (*v1.StateRootResponse, error)
GetStateFork(context.Context, *v1.StateRequest) (*v1.StateForkResponse, error)
GetFinalityCheckpoints(context.Context, *v1.StateRequest) (*v1.StateFinalityCheckpointResponse, error)
@@ -840,6 +862,9 @@ type UnimplementedBeaconChainServer struct {
func (*UnimplementedBeaconChainServer) GetGenesis(context.Context, *empty.Empty) (*v1.GenesisResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetGenesis not implemented")
}
func (*UnimplementedBeaconChainServer) GetWeakSubjectivity(context.Context, *empty.Empty) (*v1.WeakSubjectivityResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetWeakSubjectivity not implemented")
}
func (*UnimplementedBeaconChainServer) GetStateRoot(context.Context, *v1.StateRequest) (*v1.StateRootResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetStateRoot not implemented")
}
@@ -950,6 +975,24 @@ func _BeaconChain_GetGenesis_Handler(srv interface{}, ctx context.Context, dec f
return interceptor(ctx, in, info, handler)
}
func _BeaconChain_GetWeakSubjectivity_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(empty.Empty)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(BeaconChainServer).GetWeakSubjectivity(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/ethereum.eth.service.BeaconChain/GetWeakSubjectivity",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(BeaconChainServer).GetWeakSubjectivity(ctx, req.(*empty.Empty))
}
return interceptor(ctx, in, info, handler)
}
func _BeaconChain_GetStateRoot_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(v1.StateRequest)
if err := dec(in); err != nil {
@@ -1480,6 +1523,10 @@ var _BeaconChain_serviceDesc = grpc.ServiceDesc{
MethodName: "GetGenesis",
Handler: _BeaconChain_GetGenesis_Handler,
},
{
MethodName: "GetWeakSubjectivity",
Handler: _BeaconChain_GetWeakSubjectivity_Handler,
},
{
MethodName: "GetStateRoot",
Handler: _BeaconChain_GetStateRoot_Handler,

View File

@@ -57,6 +57,24 @@ func local_request_BeaconChain_GetGenesis_0(ctx context.Context, marshaler runti
}
func request_BeaconChain_GetWeakSubjectivity_0(ctx context.Context, marshaler runtime.Marshaler, client BeaconChainClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq emptypb.Empty
var metadata runtime.ServerMetadata
msg, err := client.GetWeakSubjectivity(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
return msg, metadata, err
}
func local_request_BeaconChain_GetWeakSubjectivity_0(ctx context.Context, marshaler runtime.Marshaler, server BeaconChainServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq emptypb.Empty
var metadata runtime.ServerMetadata
msg, err := server.GetWeakSubjectivity(ctx, &protoReq)
return msg, metadata, err
}
func request_BeaconChain_GetStateRoot_0(ctx context.Context, marshaler runtime.Marshaler, client BeaconChainClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq v1.StateRequest
var metadata runtime.ServerMetadata
@@ -1374,6 +1392,29 @@ func RegisterBeaconChainHandlerServer(ctx context.Context, mux *runtime.ServeMux
})
mux.Handle("GET", pattern_BeaconChain_GetWeakSubjectivity_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
var stream runtime.ServerTransportStream
ctx = grpc.NewContextWithServerTransportStream(ctx, &stream)
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/ethereum.eth.service.BeaconChain/GetWeakSubjectivity")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := local_request_BeaconChain_GetWeakSubjectivity_0(rctx, inboundMarshaler, server, req, pathParams)
md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer())
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconChain_GetWeakSubjectivity_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("GET", pattern_BeaconChain_GetStateRoot_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -2102,6 +2143,26 @@ func RegisterBeaconChainHandlerClient(ctx context.Context, mux *runtime.ServeMux
})
mux.Handle("GET", pattern_BeaconChain_GetWeakSubjectivity_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateContext(ctx, mux, req, "/ethereum.eth.service.BeaconChain/GetWeakSubjectivity")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := request_BeaconChain_GetWeakSubjectivity_0(rctx, inboundMarshaler, client, req, pathParams)
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconChain_GetWeakSubjectivity_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("GET", pattern_BeaconChain_GetStateRoot_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -2688,6 +2749,8 @@ func RegisterBeaconChainHandlerClient(ctx context.Context, mux *runtime.ServeMux
var (
pattern_BeaconChain_GetGenesis_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 4}, []string{"internal", "eth", "v1", "beacon", "genesis"}, ""))
pattern_BeaconChain_GetWeakSubjectivity_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 4}, []string{"internal", "eth", "v1", "beacon", "weak_subjectivity"}, ""))
pattern_BeaconChain_GetStateRoot_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"internal", "eth", "v1", "beacon", "states", "state_id", "root"}, ""))
pattern_BeaconChain_GetStateFork_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"internal", "eth", "v1", "beacon", "states", "state_id", "fork"}, ""))
@@ -2750,6 +2813,8 @@ var (
var (
forward_BeaconChain_GetGenesis_0 = runtime.ForwardResponseMessage
forward_BeaconChain_GetWeakSubjectivity_0 = runtime.ForwardResponseMessage
forward_BeaconChain_GetStateRoot_0 = runtime.ForwardResponseMessage
forward_BeaconChain_GetStateFork_0 = runtime.ForwardResponseMessage

View File

@@ -45,6 +45,12 @@ service BeaconChain {
option (google.api.http) = { get: "/internal/eth/v1/beacon/genesis" };
}
// GetWeakSubjectivity is a new proposed endpoint to retrieve the details necessary to download
// the ssz data needed to start a beacon node - checkpoint(epoch + block_root) and state_root
rpc GetWeakSubjectivity(google.protobuf.Empty) returns (v1.WeakSubjectivityResponse) {
option (google.api.http) = { get: "/internal/eth/v1/beacon/weak_subjectivity" };
}
// GetStateRoot calculates HashTreeRoot for state with given 'stateId'. If stateId is root, same value will be returned.
rpc GetStateRoot(v1.StateRequest) returns (v1.StateRootResponse) {
option (google.api.http) = {

View File

@@ -1837,6 +1837,108 @@ func (x *DepositContract) GetAddress() string {
return ""
}
type WeakSubjectivityResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Data *WeakSubjectivityData `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
}
func (x *WeakSubjectivityResponse) Reset() {
*x = WeakSubjectivityResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[36]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *WeakSubjectivityResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*WeakSubjectivityResponse) ProtoMessage() {}
func (x *WeakSubjectivityResponse) ProtoReflect() protoreflect.Message {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[36]
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 WeakSubjectivityResponse.ProtoReflect.Descriptor instead.
func (*WeakSubjectivityResponse) Descriptor() ([]byte, []int) {
return file_proto_eth_v1_beacon_chain_proto_rawDescGZIP(), []int{36}
}
func (x *WeakSubjectivityResponse) GetData() *WeakSubjectivityData {
if x != nil {
return x.Data
}
return nil
}
type WeakSubjectivityData struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
WsCheckpoint *Checkpoint `protobuf:"bytes,1,opt,name=ws_checkpoint,json=wsCheckpoint,proto3" json:"ws_checkpoint,omitempty"`
StateRoot []byte `protobuf:"bytes,2,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"`
}
func (x *WeakSubjectivityData) Reset() {
*x = WeakSubjectivityData{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[37]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *WeakSubjectivityData) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*WeakSubjectivityData) ProtoMessage() {}
func (x *WeakSubjectivityData) ProtoReflect() protoreflect.Message {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[37]
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 WeakSubjectivityData.ProtoReflect.Descriptor instead.
func (*WeakSubjectivityData) Descriptor() ([]byte, []int) {
return file_proto_eth_v1_beacon_chain_proto_rawDescGZIP(), []int{37}
}
func (x *WeakSubjectivityData) GetWsCheckpoint() *Checkpoint {
if x != nil {
return x.WsCheckpoint
}
return nil
}
func (x *WeakSubjectivityData) GetStateRoot() []byte {
if x != nil {
return x.StateRoot
}
return nil
}
type GenesisResponse_Genesis struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1850,7 +1952,7 @@ type GenesisResponse_Genesis struct {
func (x *GenesisResponse_Genesis) Reset() {
*x = GenesisResponse_Genesis{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[36]
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[38]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1863,7 +1965,7 @@ func (x *GenesisResponse_Genesis) String() string {
func (*GenesisResponse_Genesis) ProtoMessage() {}
func (x *GenesisResponse_Genesis) ProtoReflect() protoreflect.Message {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[36]
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[38]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1911,7 +2013,7 @@ type StateRootResponse_StateRoot struct {
func (x *StateRootResponse_StateRoot) Reset() {
*x = StateRootResponse_StateRoot{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[37]
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[39]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1924,7 +2026,7 @@ func (x *StateRootResponse_StateRoot) String() string {
func (*StateRootResponse_StateRoot) ProtoMessage() {}
func (x *StateRootResponse_StateRoot) ProtoReflect() protoreflect.Message {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[37]
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[39]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1960,7 +2062,7 @@ type StateFinalityCheckpointResponse_StateFinalityCheckpoint struct {
func (x *StateFinalityCheckpointResponse_StateFinalityCheckpoint) Reset() {
*x = StateFinalityCheckpointResponse_StateFinalityCheckpoint{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[38]
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[40]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1973,7 +2075,7 @@ func (x *StateFinalityCheckpointResponse_StateFinalityCheckpoint) String() strin
func (*StateFinalityCheckpointResponse_StateFinalityCheckpoint) ProtoMessage() {}
func (x *StateFinalityCheckpointResponse_StateFinalityCheckpoint) ProtoReflect() protoreflect.Message {
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[38]
mi := &file_proto_eth_v1_beacon_chain_proto_msgTypes[40]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2281,7 +2383,20 @@ var file_proto_eth_v1_beacon_chain_proto_rawDesc = []byte{
0x69, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68,
0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x63, 0x68,
0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73,
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42,
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22,
0x55, 0x0a, 0x18, 0x57, 0x65, 0x61, 0x6b, 0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x69, 0x76,
0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x39, 0x0a, 0x04, 0x64,
0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x65, 0x61, 0x6b,
0x53, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x44, 0x61, 0x74, 0x61,
0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x77, 0x0a, 0x14, 0x57, 0x65, 0x61, 0x6b, 0x53, 0x75,
0x62, 0x6a, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x40,
0x0a, 0x0d, 0x77, 0x73, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69,
0x6e, 0x74, 0x52, 0x0c, 0x77, 0x73, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74,
0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02,
0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x42,
0x7a, 0x0a, 0x13, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x42, 0x10, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x43, 0x68,
0x61, 0x69, 0x6e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2b, 0x67, 0x69, 0x74, 0x68,
@@ -2305,7 +2420,7 @@ func file_proto_eth_v1_beacon_chain_proto_rawDescGZIP() []byte {
return file_proto_eth_v1_beacon_chain_proto_rawDescData
}
var file_proto_eth_v1_beacon_chain_proto_msgTypes = make([]protoimpl.MessageInfo, 40)
var file_proto_eth_v1_beacon_chain_proto_msgTypes = make([]protoimpl.MessageInfo, 42)
var file_proto_eth_v1_beacon_chain_proto_goTypes = []interface{}{
(*GenesisResponse)(nil), // 0: ethereum.eth.v1.GenesisResponse
(*StateRequest)(nil), // 1: ethereum.eth.v1.StateRequest
@@ -2343,58 +2458,62 @@ var file_proto_eth_v1_beacon_chain_proto_goTypes = []interface{}{
(*SpecResponse)(nil), // 33: ethereum.eth.v1.SpecResponse
(*DepositContractResponse)(nil), // 34: ethereum.eth.v1.DepositContractResponse
(*DepositContract)(nil), // 35: ethereum.eth.v1.DepositContract
(*GenesisResponse_Genesis)(nil), // 36: ethereum.eth.v1.GenesisResponse.Genesis
(*StateRootResponse_StateRoot)(nil), // 37: ethereum.eth.v1.StateRootResponse.StateRoot
(*StateFinalityCheckpointResponse_StateFinalityCheckpoint)(nil), // 38: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint
nil, // 39: ethereum.eth.v1.SpecResponse.DataEntry
(*Fork)(nil), // 40: ethereum.eth.v1.Fork
(ValidatorStatus)(0), // 41: ethereum.eth.v1.ValidatorStatus
(*ValidatorContainer)(nil), // 42: ethereum.eth.v1.ValidatorContainer
(*Committee)(nil), // 43: ethereum.eth.v1.Committee
(*Attestation)(nil), // 44: ethereum.eth.v1.Attestation
(*BeaconBlockHeader)(nil), // 45: ethereum.eth.v1.BeaconBlockHeader
(*BeaconBlock)(nil), // 46: ethereum.eth.v1.BeaconBlock
(*AttesterSlashing)(nil), // 47: ethereum.eth.v1.AttesterSlashing
(*ProposerSlashing)(nil), // 48: ethereum.eth.v1.ProposerSlashing
(*SignedVoluntaryExit)(nil), // 49: ethereum.eth.v1.SignedVoluntaryExit
(*timestamppb.Timestamp)(nil), // 50: google.protobuf.Timestamp
(*Checkpoint)(nil), // 51: ethereum.eth.v1.Checkpoint
(*WeakSubjectivityResponse)(nil), // 36: ethereum.eth.v1.WeakSubjectivityResponse
(*WeakSubjectivityData)(nil), // 37: ethereum.eth.v1.WeakSubjectivityData
(*GenesisResponse_Genesis)(nil), // 38: ethereum.eth.v1.GenesisResponse.Genesis
(*StateRootResponse_StateRoot)(nil), // 39: ethereum.eth.v1.StateRootResponse.StateRoot
(*StateFinalityCheckpointResponse_StateFinalityCheckpoint)(nil), // 40: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint
nil, // 41: ethereum.eth.v1.SpecResponse.DataEntry
(*Fork)(nil), // 42: ethereum.eth.v1.Fork
(ValidatorStatus)(0), // 43: ethereum.eth.v1.ValidatorStatus
(*ValidatorContainer)(nil), // 44: ethereum.eth.v1.ValidatorContainer
(*Committee)(nil), // 45: ethereum.eth.v1.Committee
(*Attestation)(nil), // 46: ethereum.eth.v1.Attestation
(*BeaconBlockHeader)(nil), // 47: ethereum.eth.v1.BeaconBlockHeader
(*BeaconBlock)(nil), // 48: ethereum.eth.v1.BeaconBlock
(*AttesterSlashing)(nil), // 49: ethereum.eth.v1.AttesterSlashing
(*ProposerSlashing)(nil), // 50: ethereum.eth.v1.ProposerSlashing
(*SignedVoluntaryExit)(nil), // 51: ethereum.eth.v1.SignedVoluntaryExit
(*Checkpoint)(nil), // 52: ethereum.eth.v1.Checkpoint
(*timestamppb.Timestamp)(nil), // 53: google.protobuf.Timestamp
}
var file_proto_eth_v1_beacon_chain_proto_depIdxs = []int32{
36, // 0: ethereum.eth.v1.GenesisResponse.data:type_name -> ethereum.eth.v1.GenesisResponse.Genesis
37, // 1: ethereum.eth.v1.StateRootResponse.data:type_name -> ethereum.eth.v1.StateRootResponse.StateRoot
40, // 2: ethereum.eth.v1.StateForkResponse.data:type_name -> ethereum.eth.v1.Fork
38, // 3: ethereum.eth.v1.StateFinalityCheckpointResponse.data:type_name -> ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint
41, // 4: ethereum.eth.v1.StateValidatorsRequest.status:type_name -> ethereum.eth.v1.ValidatorStatus
42, // 5: ethereum.eth.v1.StateValidatorsResponse.data:type_name -> ethereum.eth.v1.ValidatorContainer
38, // 0: ethereum.eth.v1.GenesisResponse.data:type_name -> ethereum.eth.v1.GenesisResponse.Genesis
39, // 1: ethereum.eth.v1.StateRootResponse.data:type_name -> ethereum.eth.v1.StateRootResponse.StateRoot
42, // 2: ethereum.eth.v1.StateForkResponse.data:type_name -> ethereum.eth.v1.Fork
40, // 3: ethereum.eth.v1.StateFinalityCheckpointResponse.data:type_name -> ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint
43, // 4: ethereum.eth.v1.StateValidatorsRequest.status:type_name -> ethereum.eth.v1.ValidatorStatus
44, // 5: ethereum.eth.v1.StateValidatorsResponse.data:type_name -> ethereum.eth.v1.ValidatorContainer
9, // 6: ethereum.eth.v1.ValidatorBalancesResponse.data:type_name -> ethereum.eth.v1.ValidatorBalance
42, // 7: ethereum.eth.v1.StateValidatorResponse.data:type_name -> ethereum.eth.v1.ValidatorContainer
43, // 8: ethereum.eth.v1.StateCommitteesResponse.data:type_name -> ethereum.eth.v1.Committee
44, // 9: ethereum.eth.v1.BlockAttestationsResponse.data:type_name -> ethereum.eth.v1.Attestation
44, // 7: ethereum.eth.v1.StateValidatorResponse.data:type_name -> ethereum.eth.v1.ValidatorContainer
45, // 8: ethereum.eth.v1.StateCommitteesResponse.data:type_name -> ethereum.eth.v1.Committee
46, // 9: ethereum.eth.v1.BlockAttestationsResponse.data:type_name -> ethereum.eth.v1.Attestation
15, // 10: ethereum.eth.v1.BlockRootResponse.data:type_name -> ethereum.eth.v1.BlockRootContainer
21, // 11: ethereum.eth.v1.BlockHeadersResponse.data:type_name -> ethereum.eth.v1.BlockHeaderContainer
21, // 12: ethereum.eth.v1.BlockHeaderResponse.data:type_name -> ethereum.eth.v1.BlockHeaderContainer
22, // 13: ethereum.eth.v1.BlockHeaderContainer.header:type_name -> ethereum.eth.v1.BeaconBlockHeaderContainer
45, // 14: ethereum.eth.v1.BeaconBlockHeaderContainer.message:type_name -> ethereum.eth.v1.BeaconBlockHeader
47, // 14: ethereum.eth.v1.BeaconBlockHeaderContainer.message:type_name -> ethereum.eth.v1.BeaconBlockHeader
25, // 15: ethereum.eth.v1.BlockResponse.data:type_name -> ethereum.eth.v1.BeaconBlockContainer
46, // 16: ethereum.eth.v1.BeaconBlockContainer.message:type_name -> ethereum.eth.v1.BeaconBlock
44, // 17: ethereum.eth.v1.SubmitAttestationsRequest.data:type_name -> ethereum.eth.v1.Attestation
44, // 18: ethereum.eth.v1.AttestationsPoolResponse.data:type_name -> ethereum.eth.v1.Attestation
47, // 19: ethereum.eth.v1.AttesterSlashingsPoolResponse.data:type_name -> ethereum.eth.v1.AttesterSlashing
48, // 20: ethereum.eth.v1.ProposerSlashingPoolResponse.data:type_name -> ethereum.eth.v1.ProposerSlashing
49, // 21: ethereum.eth.v1.VoluntaryExitsPoolResponse.data:type_name -> ethereum.eth.v1.SignedVoluntaryExit
40, // 22: ethereum.eth.v1.ForkScheduleResponse.data:type_name -> ethereum.eth.v1.Fork
39, // 23: ethereum.eth.v1.SpecResponse.data:type_name -> ethereum.eth.v1.SpecResponse.DataEntry
48, // 16: ethereum.eth.v1.BeaconBlockContainer.message:type_name -> ethereum.eth.v1.BeaconBlock
46, // 17: ethereum.eth.v1.SubmitAttestationsRequest.data:type_name -> ethereum.eth.v1.Attestation
46, // 18: ethereum.eth.v1.AttestationsPoolResponse.data:type_name -> ethereum.eth.v1.Attestation
49, // 19: ethereum.eth.v1.AttesterSlashingsPoolResponse.data:type_name -> ethereum.eth.v1.AttesterSlashing
50, // 20: ethereum.eth.v1.ProposerSlashingPoolResponse.data:type_name -> ethereum.eth.v1.ProposerSlashing
51, // 21: ethereum.eth.v1.VoluntaryExitsPoolResponse.data:type_name -> ethereum.eth.v1.SignedVoluntaryExit
42, // 22: ethereum.eth.v1.ForkScheduleResponse.data:type_name -> ethereum.eth.v1.Fork
41, // 23: ethereum.eth.v1.SpecResponse.data:type_name -> ethereum.eth.v1.SpecResponse.DataEntry
35, // 24: ethereum.eth.v1.DepositContractResponse.data:type_name -> ethereum.eth.v1.DepositContract
50, // 25: ethereum.eth.v1.GenesisResponse.Genesis.genesis_time:type_name -> google.protobuf.Timestamp
51, // 26: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint.previous_justified:type_name -> ethereum.eth.v1.Checkpoint
51, // 27: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint.current_justified:type_name -> ethereum.eth.v1.Checkpoint
51, // 28: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint.finalized:type_name -> ethereum.eth.v1.Checkpoint
29, // [29:29] is the sub-list for method output_type
29, // [29:29] is the sub-list for method input_type
29, // [29:29] is the sub-list for extension type_name
29, // [29:29] is the sub-list for extension extendee
0, // [0:29] is the sub-list for field type_name
37, // 25: ethereum.eth.v1.WeakSubjectivityResponse.data:type_name -> ethereum.eth.v1.WeakSubjectivityData
52, // 26: ethereum.eth.v1.WeakSubjectivityData.ws_checkpoint:type_name -> ethereum.eth.v1.Checkpoint
53, // 27: ethereum.eth.v1.GenesisResponse.Genesis.genesis_time:type_name -> google.protobuf.Timestamp
52, // 28: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint.previous_justified:type_name -> ethereum.eth.v1.Checkpoint
52, // 29: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint.current_justified:type_name -> ethereum.eth.v1.Checkpoint
52, // 30: ethereum.eth.v1.StateFinalityCheckpointResponse.StateFinalityCheckpoint.finalized:type_name -> ethereum.eth.v1.Checkpoint
31, // [31:31] is the sub-list for method output_type
31, // [31:31] is the sub-list for method input_type
31, // [31:31] is the sub-list for extension type_name
31, // [31:31] is the sub-list for extension extendee
0, // [0:31] is the sub-list for field type_name
}
func init() { file_proto_eth_v1_beacon_chain_proto_init() }
@@ -2840,7 +2959,7 @@ func file_proto_eth_v1_beacon_chain_proto_init() {
}
}
file_proto_eth_v1_beacon_chain_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GenesisResponse_Genesis); i {
switch v := v.(*WeakSubjectivityResponse); i {
case 0:
return &v.state
case 1:
@@ -2852,7 +2971,7 @@ func file_proto_eth_v1_beacon_chain_proto_init() {
}
}
file_proto_eth_v1_beacon_chain_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*StateRootResponse_StateRoot); i {
switch v := v.(*WeakSubjectivityData); i {
case 0:
return &v.state
case 1:
@@ -2864,6 +2983,30 @@ func file_proto_eth_v1_beacon_chain_proto_init() {
}
}
file_proto_eth_v1_beacon_chain_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GenesisResponse_Genesis); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_eth_v1_beacon_chain_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*StateRootResponse_StateRoot); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_eth_v1_beacon_chain_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*StateFinalityCheckpointResponse_StateFinalityCheckpoint); i {
case 0:
return &v.state
@@ -2885,7 +3028,7 @@ func file_proto_eth_v1_beacon_chain_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_proto_eth_v1_beacon_chain_proto_rawDesc,
NumEnums: 0,
NumMessages: 40,
NumMessages: 42,
NumExtensions: 0,
NumServices: 0,
},

View File

@@ -270,3 +270,12 @@ message DepositContract {
// The address of the deployed deposit contract in use.
string address = 2;
}
message WeakSubjectivityResponse {
WeakSubjectivityData data = 1;
}
message WeakSubjectivityData {
ethereum.eth.v1.Checkpoint ws_checkpoint = 1;
bytes state_root = 2;
}

File diff suppressed because it is too large Load Diff

View File

@@ -314,24 +314,6 @@ func local_request_BeaconChain_GetChainHead_0(ctx context.Context, marshaler run
}
func request_BeaconChain_GetWeakSubjectivityCheckpoint_0(ctx context.Context, marshaler runtime.Marshaler, client BeaconChainClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq emptypb.Empty
var metadata runtime.ServerMetadata
msg, err := client.GetWeakSubjectivityCheckpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
return msg, metadata, err
}
func local_request_BeaconChain_GetWeakSubjectivityCheckpoint_0(ctx context.Context, marshaler runtime.Marshaler, server BeaconChainServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq emptypb.Empty
var metadata runtime.ServerMetadata
msg, err := server.GetWeakSubjectivityCheckpoint(ctx, &protoReq)
return msg, metadata, err
}
var (
filter_BeaconChain_ListBeaconCommittees_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)}
)
@@ -988,29 +970,6 @@ func RegisterBeaconChainHandlerServer(ctx context.Context, mux *runtime.ServeMux
})
mux.Handle("GET", pattern_BeaconChain_GetWeakSubjectivityCheckpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
var stream runtime.ServerTransportStream
ctx = grpc.NewContextWithServerTransportStream(ctx, &stream)
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/ethereum.eth.v1alpha1.BeaconChain/GetWeakSubjectivityCheckpoint")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := local_request_BeaconChain_GetWeakSubjectivityCheckpoint_0(rctx, inboundMarshaler, server, req, pathParams)
md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer())
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconChain_GetWeakSubjectivityCheckpoint_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("GET", pattern_BeaconChain_ListBeaconCommittees_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -1558,26 +1517,6 @@ func RegisterBeaconChainHandlerClient(ctx context.Context, mux *runtime.ServeMux
})
mux.Handle("GET", pattern_BeaconChain_GetWeakSubjectivityCheckpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateContext(ctx, mux, req, "/ethereum.eth.v1alpha1.BeaconChain/GetWeakSubjectivityCheckpoint")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := request_BeaconChain_GetWeakSubjectivityCheckpoint_0(rctx, inboundMarshaler, client, req, pathParams)
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconChain_GetWeakSubjectivityCheckpoint_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("GET", pattern_BeaconChain_ListBeaconCommittees_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -1882,8 +1821,6 @@ var (
pattern_BeaconChain_GetChainHead_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "beacon", "chainhead"}, ""))
pattern_BeaconChain_GetWeakSubjectivityCheckpoint_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "beacon", "weak_subjectivity_checkpoint"}, ""))
pattern_BeaconChain_ListBeaconCommittees_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "beacon", "committees"}, ""))
pattern_BeaconChain_ListValidatorBalances_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "validators", "balances"}, ""))
@@ -1934,8 +1871,6 @@ var (
forward_BeaconChain_GetChainHead_0 = runtime.ForwardResponseMessage
forward_BeaconChain_GetWeakSubjectivityCheckpoint_0 = runtime.ForwardResponseMessage
forward_BeaconChain_ListBeaconCommittees_0 = runtime.ForwardResponseMessage
forward_BeaconChain_ListValidatorBalances_0 = runtime.ForwardResponseMessage

View File

@@ -150,16 +150,6 @@ service BeaconChain {
};
}
// Retrieve information about the weak subjectivity of the beacon chain from the view of
// the beacon chain node.
//
// This includes the weak subjectivity block root, state root and epoch number.
rpc GetWeakSubjectivityCheckpoint(google.protobuf.Empty) returns (WeakSubjectivityCheckpoint) {
option (google.api.http) = {
get: "/eth/v1alpha1/beacon/weak_subjectivity_checkpoint"
};
}
// Retrieve the beacon chain committees for a given epoch.
//
// If no filter criteria is specified, the response returns
@@ -907,13 +897,4 @@ message IndividualVotesRespond {
}
repeated IndividualVote individual_votes = 1;
}
message WeakSubjectivityCheckpoint {
// The block root of weak subjectivity checkpoint.
bytes block_root = 1;
// The state root of weak subjectivity checkpoint.
bytes state_root = 2;
// The epoch of weak subjectivity checkpoint.
uint64 epoch = 3 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/eth2-types.Epoch"];
}
}

View File

@@ -218,26 +218,6 @@ func (mr *MockBeaconChainClientMockRecorder) GetValidatorQueue(arg0, arg1 interf
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetValidatorQueue", reflect.TypeOf((*MockBeaconChainClient)(nil).GetValidatorQueue), varargs...)
}
// GetWeakSubjectivityCheckpoint mocks base method
func (m *MockBeaconChainClient) GetWeakSubjectivityCheckpoint(arg0 context.Context, arg1 *emptypb.Empty, arg2 ...grpc.CallOption) (*eth.WeakSubjectivityCheckpoint, error) {
m.ctrl.T.Helper()
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetWeakSubjectivityCheckpoint", varargs...)
ret0, _ := ret[0].(*eth.WeakSubjectivityCheckpoint)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// GetWeakSubjectivityCheckpoint indicates an expected call of GetWeakSubjectivityCheckpoint
func (mr *MockBeaconChainClientMockRecorder) GetWeakSubjectivityCheckpoint(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWeakSubjectivityCheckpoint", reflect.TypeOf((*MockBeaconChainClient)(nil).GetWeakSubjectivityCheckpoint), varargs...)
}
// ListAttestations mocks base method
func (m *MockBeaconChainClient) ListAttestations(arg0 context.Context, arg1 *eth.ListAttestationsRequest, arg2 ...grpc.CallOption) (*eth.ListAttestationsResponse, error) {
m.ctrl.T.Helper()