Partial Implementation of the Slot Processing Routine (#1083)

This commit is contained in:
Nishant Das
2018-12-19 13:18:42 +08:00
committed by Raul Jordan
parent 74532a80d1
commit e92032346f
17 changed files with 965 additions and 422 deletions

View File

@@ -15,6 +15,8 @@ go_library(
"//shared/bitutil:go_default_library",
"//shared/event:go_default_library",
"//shared/params:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)
@@ -25,11 +27,11 @@ go_test(
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/types:go_default_library",
"//beacon-chain/core/validators:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/internal:go_default_library",
"//beacon-chain/powchain:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/params:go_default_library",
"//shared/testutil:go_default_library",
"@com_github_ethereum_go_ethereum//:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",

View File

@@ -8,6 +8,8 @@ import (
"fmt"
"time"
"github.com/ethereum/go-ethereum/common"
gethTypes "github.com/ethereum/go-ethereum/core/types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
@@ -35,6 +37,7 @@ type ChainService struct {
canonicalBlockFeed *event.Feed
canonicalStateFeed *event.Feed
genesisTime time.Time
unProcessedBlocks map[uint64]*types.Block
unfinalizedBlocks map[[32]byte]*types.BeaconState
enablePOWChain bool
}
@@ -63,6 +66,7 @@ func NewChainService(ctx context.Context, cfg *Config) (*ChainService, error) {
incomingBlockFeed: new(event.Feed),
canonicalBlockFeed: new(event.Feed),
canonicalStateFeed: new(event.Feed),
unProcessedBlocks: make(map[uint64]*types.Block),
unfinalizedBlocks: make(map[[32]byte]*types.BeaconState),
enablePOWChain: cfg.EnablePOWChain,
}, nil
@@ -75,7 +79,7 @@ func (c *ChainService) Start() {
var err error
c.genesisTime, err = c.beaconDB.GetGenesisTime()
if err != nil {
log.Fatal(err)
log.Fatalf("Unable to retrieve genesis time, therefore blockchain service cannot be started %v", err)
return
}
@@ -112,8 +116,8 @@ func (c *ChainService) CanonicalStateFeed() *event.Feed {
}
// doesPoWBlockExist checks if the referenced PoW block exists.
func (c *ChainService) doesPoWBlockExist(block *types.Block) bool {
powBlock, err := c.web3Service.Client().BlockByHash(context.Background(), block.CandidatePowReceiptRootHash32())
func (c *ChainService) doesPoWBlockExist(hash [32]byte) bool {
powBlock, err := c.web3Service.Client().BlockByHash(c.ctx, hash)
if err != nil {
log.Debugf("fetching PoW block corresponding to mainchain reference failed: %v", err)
return false
@@ -222,28 +226,169 @@ func (c *ChainService) blockProcessing(processedBlock chan<- *types.Block) {
// can be received either from the sync service, the RPC service,
// or via p2p.
case block := <-c.incomingBlockChan:
if err := c.processBlock(block); err != nil {
log.Error(err)
processedBlock <- nil
// Before sending the blocks for processing we check to see if the blocks
// are valid to continue being processed. If the slot number in the block
// has already been processed by the beacon node, we throw it away. If the
// slot number is too high to be processed in the current slot, we store
// it in a cache.
beaconState, err := c.beaconDB.GetState()
if err != nil {
log.Errorf("Unable to retrieve beacon state %v", err)
continue
}
// Push the block to trigger the fork choice rule.
processedBlock <- block
currentSlot := beaconState.Slot()
if currentSlot+1 < block.SlotNumber() {
c.unProcessedBlocks[block.SlotNumber()] = block
continue
}
if currentSlot+1 == block.SlotNumber() {
if err := c.receiveBlock(block); err != nil {
log.Error(err)
processedBlock <- nil
continue
}
// Push the block to trigger the fork choice rule.
processedBlock <- block
} else {
log.Debugf(
"Block slot number is lower than the current slot in the beacon state %d",
block.SlotNumber())
c.sendAndDeleteCachedBlocks(currentSlot)
}
}
}
}
func (c *ChainService) processBlock(block *types.Block) error {
// receiveBlock is a function that defines the operations that are preformed on
// any block that is received from p2p layer or rpc. It checks the block to see
// if it passes the pre-processing conditions, if it does then the per slot
// state transition function is carried out on the block.
// spec:
// def process_block(block):
// if not block_pre_processing_conditions(block):
// return False
//
// # process skipped slots
//
// while (state.slot < block.slot - 1):
// state = slot_state_transition(state, block=None)
//
// # process slot with block
// state = slot_state_transition(state, block)
//
// # check state root
// if block.state_root == hash(state):
// return state
// else:
// return False # or throw or whatever
//
func (c *ChainService) receiveBlock(block *types.Block) error {
blockhash, err := block.Hash()
if err != nil {
return fmt.Errorf("could not hash incoming block: %v", err)
}
beaconState, err := c.beaconDB.GetState()
if err != nil {
return fmt.Errorf("failed to get beacon state: %v", err)
}
if block.SlotNumber() == 0 {
return errors.New("cannot process a genesis block: received block with slot 0")
}
// Save blocks with higher slot numbers in cache.
if !c.isBlockReadyForProcessing(block) {
log.Debugf("block with hash %#x is not ready for processing", blockhash)
return nil
}
log.WithField("slotNumber", block.SlotNumber()).Info("Executing state transition")
// Check for skipped slots and update the corresponding proposers
// randao layer.
for beaconState.Slot() < block.SlotNumber()-1 {
beaconState, err = state.ExecuteStateTransition(beaconState, nil)
if err != nil {
return fmt.Errorf("unable to execute state transition %v", err)
}
}
beaconState, err = state.ExecuteStateTransition(beaconState, block)
if err != nil {
return errors.New("unable to execute state transition")
}
if beaconState.IsValidatorSetChange(block.SlotNumber()) {
log.WithField("slotNumber", block.SlotNumber()).Info("Validator set rotation occurred")
}
// TODO(#1074): Verify block.state_root == hash_tree_root(state)
// if there exists a block for the slot being processed.
if err := c.beaconDB.SaveBlock(block); err != nil {
return fmt.Errorf("failed to save block: %v", err)
}
if err := c.beaconDB.SaveUnfinalizedBlockState(beaconState); err != nil {
return fmt.Errorf("error persisting unfinalized block's state: %v", err)
}
log.WithField("hash", fmt.Sprintf("%#x", blockhash)).Debug("Processed beacon block")
// We keep a map of unfinalized blocks in memory along with their state
// pair to apply the fork choice rule.
c.unfinalizedBlocks[blockhash] = beaconState
return nil
}
func (c *ChainService) isBlockReadyForProcessing(block *types.Block) bool {
beaconState, err := c.beaconDB.GetState()
if err != nil {
log.Debugf("failed to get beacon state: %v", err)
return false
}
var powBlockFetcher func(ctx context.Context, hash common.Hash) (*gethTypes.Block, error)
if c.enablePOWChain {
powBlockFetcher = c.web3Service.Client().BlockByHash
}
if err := state.IsValidBlock(c.ctx, beaconState, block, c.enablePOWChain,
c.beaconDB.HasBlock, powBlockFetcher, c.genesisTime); err != nil {
log.Debugf("block does not fulfill pre-processing conditions %v", err)
return false
}
return true
}
// sendAndDeleteCachedBlocks checks if there is any block saved in the cache with a
// slot number equivalent to the current slot. If there is then the block is
// sent to the incoming block channel and deleted from the cache.
func (c *ChainService) sendAndDeleteCachedBlocks(currentSlot uint64) {
if block, ok := c.unProcessedBlocks[currentSlot+1]; ok && c.isBlockReadyForProcessing(block) {
c.incomingBlockChan <- block
delete(c.unProcessedBlocks, currentSlot)
}
}
// DEPRECATED: Will be replaced by new block processing method
func (c *ChainService) processBlockOld(block *types.Block) error {
blockHash, err := block.Hash()
if err != nil {
return fmt.Errorf("failed to get hash of block: %v", err)
}
if c.enablePOWChain && !c.doesPoWBlockExist(block) {
return errors.New("proof-of-Work chain reference in block does not exist")
}
parent, err := c.beaconDB.GetBlock(block.ParentHash())
if err != nil {
return fmt.Errorf("could not get parent block: %v", err)
@@ -257,9 +402,13 @@ func (c *ChainService) processBlock(block *types.Block) error {
return fmt.Errorf("failed to get beacon state: %v", err)
}
if c.enablePOWChain && !c.doesPoWBlockExist(beaconState.ProcessedPowReceiptRootHash32()) {
return errors.New("proof-of-Work chain reference in block does not exist")
}
// Verifies the block against the validity conditions specifies as part of the
// Ethereum 2.0 specification.
if err := state.IsValidBlock(
if err := state.IsValidBlockOld(
block,
beaconState,
parent.SlotNumber(),
@@ -278,7 +427,7 @@ func (c *ChainService) processBlock(block *types.Block) error {
beaconState.SetPendingAttestations(block.Attestations())
// If the block is valid, we compute its associated state tuple (active, crystallized)
beaconState, err = c.executeStateTransition(beaconState, block, parent.SlotNumber())
beaconState, err = c.executeStateTransitionOld(beaconState, block, parent.SlotNumber())
if err != nil {
return fmt.Errorf("initialize new cycle transition failed: %v", err)
}
@@ -299,7 +448,8 @@ func (c *ChainService) processBlock(block *types.Block) error {
return nil
}
func (c *ChainService) executeStateTransition(
// DEPRECATED: Will be removed soon
func (c *ChainService) executeStateTransitionOld(
beaconState *types.BeaconState,
block *types.Block,
parentSlot uint64,

View File

@@ -12,11 +12,11 @@ import (
gethTypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/event"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/internal"
"github.com/prysmaticlabs/prysm/beacon-chain/powchain"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/testutil"
"github.com/sirupsen/logrus"
logTest "github.com/sirupsen/logrus/hooks/test"
@@ -114,6 +114,16 @@ func setupBeaconChain(t *testing.T, faultyPoWClient bool, beaconDB *db.BeaconDB)
return chainService
}
func SetSlotInState(service *ChainService, slot uint64) error {
bState, err := service.beaconDB.GetState()
if err != nil {
return err
}
bState.SetSlot(slot)
return service.beaconDB.SaveState(bState)
}
func TestStartStop(t *testing.T) {
db := internal.SetupDB(t)
defer internal.TeardownDB(t, db)
@@ -140,8 +150,26 @@ func TestRunningChainServiceFaultyPOWChain(t *testing.T) {
defer internal.TeardownDB(t, db)
chainService := setupBeaconChain(t, true, db)
if err := SetSlotInState(chainService, 1); err != nil {
t.Fatal(err)
}
parentBlock := types.NewBlock(&pb.BeaconBlock{
Slot: 1,
})
parentHash, err := parentBlock.Hash()
if err != nil {
t.Fatalf("Unable to hash block %v", err)
}
if err := chainService.beaconDB.SaveBlock(parentBlock); err != nil {
t.Fatalf("Unable to save block %v", err)
}
block := types.NewBlock(&pb.BeaconBlock{
Slot: 1,
Slot: 2,
ParentRootHash32: parentHash[:],
CandidatePowReceiptRootHash32: []byte("a"),
})
@@ -161,7 +189,7 @@ func TestRunningChainServiceFaultyPOWChain(t *testing.T) {
chainService.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "proof-of-Work chain reference in block does not exist")
testutil.AssertLogsContain(t, hook, "unable to retrieve POW chain reference block failed")
}
func TestRunningChainService(t *testing.T) {
@@ -184,12 +212,31 @@ func TestRunningChainService(t *testing.T) {
t.Fatalf("unable to get hash of canonical head: %v", err)
}
beaconState, err = chainService.beaconDB.GetState()
if err != nil {
t.Fatalf("Can't get state from db %v", err)
}
var shardAndCommittees []*pb.ShardAndCommitteeArray
for i := uint64(0); i < params.BeaconConfig().EpochLength*2; i++ {
shardAndCommittees = append(shardAndCommittees, &pb.ShardAndCommitteeArray{
ArrayShardAndCommittee: []*pb.ShardAndCommittee{
{Committee: []uint32{9, 8, 311, 12, 92, 1, 23, 17}},
},
})
}
beaconState.SetShardAndCommitteesAtSlots(shardAndCommittees)
if err := chainService.beaconDB.SaveState(beaconState); err != nil {
t.Fatal(err)
}
currentSlot := uint64(1)
attestationSlot := uint64(0)
shard := beaconState.ShardAndCommitteesForSlots()[attestationSlot].ArrayShardAndCommittee[0].Shard
block := types.NewBlock(&pb.BeaconBlock{
Slot: currentSlot,
Slot: currentSlot + 1,
StateRootHash32: stateRoot[:],
ParentRootHash32: parentHash[:],
CandidatePowReceiptRootHash32: []byte("a"),
@@ -202,6 +249,10 @@ func TestRunningChainService(t *testing.T) {
}},
})
if err := SetSlotInState(chainService, currentSlot); err != nil {
t.Fatal(err)
}
blockChan := make(chan *types.Block)
exitRoutine := make(chan bool)
go func() {
@@ -217,6 +268,7 @@ func TestRunningChainService(t *testing.T) {
<-blockChan
chainService.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "Chain service context closed, exiting goroutine")
testutil.AssertLogsContain(t, hook, "Processed beacon block")
}
@@ -227,150 +279,19 @@ func TestDoesPOWBlockExist(t *testing.T) {
defer internal.TeardownDB(t, db)
chainService := setupBeaconChain(t, true, db)
block := types.NewBlock(&pb.BeaconBlock{
Slot: 10,
})
state, err := chainService.beaconDB.GetState()
if err != nil {
t.Fatalf("Unable to retrieve beacon state %v", err)
}
// Using a faulty client should throw error.
exists := chainService.doesPoWBlockExist(block)
exists := chainService.doesPoWBlockExist(state.ProcessedPowReceiptRootHash32())
if exists {
t.Error("Block corresponding to nil powchain reference should not exist")
}
testutil.AssertLogsContain(t, hook, "fetching PoW block corresponding to mainchain reference failed")
}
func getShardForSlot(t *testing.T, beaconState *types.BeaconState, slot uint64) uint64 {
shardAndCommittee, err := v.GetShardAndCommitteesForSlot(
beaconState.ShardAndCommitteesForSlots(),
beaconState.LastStateRecalculationSlot(),
slot,
)
if err != nil {
t.Fatalf("Unable to get shard for slot: %d", slot)
}
return shardAndCommittee.ArrayShardAndCommittee[0].Shard
}
func TestProcessBlocksWithCorrectAttestations(t *testing.T) {
db := internal.SetupDB(t)
defer internal.TeardownDB(t, db)
chainService := setupBeaconChain(t, false, db)
beaconState, err := types.NewGenesisBeaconState(nil)
if err != nil {
t.Fatalf("Can't generate genesis state: %v", err)
}
stateRoot, _ := beaconState.Hash()
block0 := types.NewBlock(&pb.BeaconBlock{
Slot: 0,
})
if saveErr := chainService.beaconDB.SaveBlock(block0); saveErr != nil {
t.Fatalf("Could not save block: %v", saveErr)
}
block0Hash, err := block0.Hash()
if err != nil {
t.Fatalf("Failed to compute block's hash: %v", err)
}
currentSlot := uint64(1)
attestationSlot := currentSlot - 1
block1 := types.NewBlock(&pb.BeaconBlock{
ParentRootHash32: block0Hash[:],
Slot: currentSlot,
StateRootHash32: stateRoot[:],
Attestations: []*pb.AggregatedAttestation{{
Slot: attestationSlot,
AttesterBitfield: []byte{128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0},
Shard: getShardForSlot(t, beaconState, attestationSlot),
JustifiedBlockHash: block0Hash[:],
}},
})
blockChan := make(chan *types.Block)
exitRoutine := make(chan bool)
go func() {
chainService.blockProcessing(blockChan)
<-exitRoutine
}()
chainService.incomingBlockChan <- block1
block1Returned := <-blockChan
if block1 != block1Returned {
t.Fatalf("expected %v and %v to be the same", block1, block1Returned)
}
block1Hash, err := block1.Hash()
if err != nil {
t.Fatalf("unable to get hash of block 1: %v", err)
}
currentSlot++
// Add 1 more attestation field for slot2
block2 := types.NewBlock(&pb.BeaconBlock{
ParentRootHash32: block1Hash[:],
Slot: currentSlot,
Attestations: []*pb.AggregatedAttestation{
{
Slot: currentSlot - 1,
AttesterBitfield: []byte{64, 0},
Shard: getShardForSlot(t, beaconState, currentSlot-1),
JustifiedBlockHash: block0Hash[:],
},
{
Slot: currentSlot - 2,
AttesterBitfield: []byte{128, 0},
Shard: getShardForSlot(t, beaconState, currentSlot-2),
JustifiedBlockHash: block0Hash[:],
},
}})
block2Hash, err := block2.Hash()
if err != nil {
t.Fatalf("unable to get hash of block 1: %v", err)
}
currentSlot++
// Add 1 more attestation field for slot3
block3 := types.NewBlock(&pb.BeaconBlock{
ParentRootHash32: block2Hash[:],
Slot: currentSlot,
Attestations: []*pb.AggregatedAttestation{
{
Slot: currentSlot - 1,
AttesterBitfield: []byte{32, 0},
Shard: getShardForSlot(t, beaconState, currentSlot-1),
JustifiedBlockHash: block0Hash[:],
},
{
Slot: currentSlot - 2,
AttesterBitfield: []byte{64, 0},
Shard: getShardForSlot(t, beaconState, currentSlot-2),
JustifiedBlockHash: block0Hash[:],
},
{
Slot: currentSlot - 3,
AttesterBitfield: []byte{128, 0},
Shard: getShardForSlot(t, beaconState, currentSlot-3),
JustifiedBlockHash: block0Hash[:],
},
}})
chainService.incomingBlockChan <- block1
<-blockChan
chainService.incomingBlockChan <- block2
<-blockChan
chainService.incomingBlockChan <- block3
<-blockChan
chainService.cancel()
exitRoutine <- true
}
func TestUpdateHead(t *testing.T) {
beaconState, err := types.NewGenesisBeaconState(nil)
if err != nil {
@@ -457,6 +378,75 @@ func TestUpdateHead(t *testing.T) {
}
}
func TestIsBlockReadyForProcessing(t *testing.T) {
db := internal.SetupDB(t)
defer internal.TeardownDB(t, db)
chainService := setupBeaconChain(t, false, db)
beaconState, err := types.NewGenesisBeaconState(nil)
if err != nil {
t.Fatalf("Can't generate genesis state: %v", err)
}
block := types.NewBlock(&pb.BeaconBlock{
ParentRootHash32: []byte{'a'},
})
if chainService.isBlockReadyForProcessing(block) {
t.Fatal("block processing succeeded despite block having no parent saved")
}
beaconState.SetSlot(10)
chainService.beaconDB.SaveState(beaconState)
stateRoot, _ := beaconState.Hash()
genesis := types.NewGenesisBlock([32]byte{})
chainService.beaconDB.SaveBlock(genesis)
parentHash, err := genesis.Hash()
if err != nil {
t.Fatalf("unable to get hash of canonical head: %v", err)
}
block2 := types.NewBlock(&pb.BeaconBlock{
ParentRootHash32: parentHash[:],
Slot: 10,
})
if chainService.isBlockReadyForProcessing(block2) {
t.Fatal("block processing succeeded despite block slot being invalid")
}
var h [32]byte
copy(h[:], []byte("a"))
beaconState.SetProcessedPowReceiptHash(h)
beaconState.SetSlot(9)
chainService.beaconDB.SaveState(beaconState)
currentSlot := uint64(10)
attestationSlot := uint64(0)
shard := beaconState.ShardAndCommitteesForSlots()[attestationSlot].ArrayShardAndCommittee[0].Shard
block3 := types.NewBlock(&pb.BeaconBlock{
Slot: currentSlot,
StateRootHash32: stateRoot[:],
ParentRootHash32: parentHash[:],
CandidatePowReceiptRootHash32: []byte("a"),
Attestations: []*pb.AggregatedAttestation{{
Slot: attestationSlot,
AttesterBitfield: []byte{128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0},
Shard: shard,
JustifiedBlockHash: parentHash[:],
}},
})
chainService.enablePOWChain = true
if !chainService.isBlockReadyForProcessing(block3) {
t.Fatal("block processing failed despite being a valid block")
}
}
func TestUpdateBlockVoteCache(t *testing.T) {
db := internal.SetupDB(t)
defer internal.TeardownDB(t, db)

View File

@@ -0,0 +1,23 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = ["randao.go"],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/core/randao",
visibility = ["//beacon-chain:__subpackages__"],
deps = [
"//beacon-chain/core/types:go_default_library",
"//beacon-chain/core/validators:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = ["randao_test.go"],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/params:go_default_library",
],
)

View File

@@ -0,0 +1,23 @@
package randao
import (
"fmt"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
)
// UpdateRandaoLayers increments the randao layer of the block proposer at the given slot.
func UpdateRandaoLayers(state *types.BeaconState, slot uint64) (*types.BeaconState, error) {
vreg := state.ValidatorRegistry()
proposerIndex, err := v.BeaconProposerIndex(state.Proto(), slot)
if err != nil {
return nil, fmt.Errorf("unable to retrieve proposer index %v", err)
}
vreg[proposerIndex].RandaoLayers++
state.SetValidatorRegistry(vreg)
return state, nil
}

View File

@@ -0,0 +1,43 @@
package randao
import (
"testing"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/params"
)
func TestUpdateRandaoLayers(t *testing.T) {
beaconState, err := types.NewGenesisBeaconState(nil)
if err != nil {
t.Fatalf("failed to generate beacon state: %v", err)
}
var shardAndCommittees []*pb.ShardAndCommitteeArray
for i := uint64(0); i < params.BeaconConfig().EpochLength*2; i++ {
shardAndCommittees = append(shardAndCommittees, &pb.ShardAndCommitteeArray{
ArrayShardAndCommittee: []*pb.ShardAndCommittee{
{Committee: []uint32{9, 8, 311, 12, 92, 1, 23, 17}},
},
})
}
beaconState.SetShardAndCommitteesAtSlots(shardAndCommittees)
newState, err := UpdateRandaoLayers(beaconState, 1)
if err != nil {
t.Fatalf("failed to update randao layers: %v", err)
}
vreg := newState.ValidatorRegistry()
// Since slot 1 has proposer index 8
if vreg[8].GetRandaoLayers() != 1 {
t.Fatalf("randao layers not updated %d", vreg[9].GetRandaoLayers())
}
if vreg[9].GetRandaoLayers() != 0 {
t.Errorf("randao layers updated when they were not supposed to %d", vreg[9].GetRandaoLayers())
}
}

View File

@@ -11,6 +11,7 @@ go_library(
visibility = ["//beacon-chain:__subpackages__"],
deps = [
"//beacon-chain/core/incentives:go_default_library",
"//beacon-chain/core/randao:go_default_library",
"//beacon-chain/core/types:go_default_library",
"//beacon-chain/core/validators:go_default_library",
"//beacon-chain/utils:go_default_library",
@@ -19,6 +20,7 @@ go_library(
"//shared/hashutil:go_default_library",
"//shared/params:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
],
)
@@ -38,6 +40,8 @@ go_test(
"//shared/bytes:go_default_library",
"//shared/hashutil:go_default_library",
"//shared/params:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)

View File

@@ -3,6 +3,7 @@ package state
import (
"encoding/binary"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/params"
@@ -66,3 +67,11 @@ func ProcessSpecialRecords(slotNumber uint64, validators []*pb.ValidatorRecord,
}
return validators, nil
}
// ProcessBlock describes the per block operations that happen on every slot.
func ProcessBlock(state *types.BeaconState, block *types.Block) *types.BeaconState {
_ = block
// TODO(#1073): This function will encompass all the per block slot transition functions, this will
// contain checks for randao,proposer validity and block operations.
return state
}

View File

@@ -5,6 +5,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/prysmaticlabs/prysm/beacon-chain/core/incentives"
"github.com/prysmaticlabs/prysm/beacon-chain/core/randao"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
"github.com/prysmaticlabs/prysm/beacon-chain/utils"
@@ -12,10 +13,11 @@ import (
"github.com/prysmaticlabs/prysm/shared/params"
)
// NewStateTransition computes the new beacon state, given the previous beacon state
// a beacon block, and its parent slot. This method is called during a cycle transition.
// We also check for validator set change transition and compute for new
// committees if necessary during this transition.
// NewStateTransition computes the new beacon state.
// DEPRECATED: Will be removed soon.
// This function takes in the previous beacon stat, beacon block, and its parent slot.
// This method is called during a cycle transition. We also check for validator
// set change transition and compute for new committees if necessary during this transition.
func NewStateTransition(
st *types.BeaconState,
block *types.Block,
@@ -123,6 +125,56 @@ func NewStateTransition(
return newState, nil
}
// ExecuteStateTransition defines the procedure for a state transition function.
// Spec:
// We now define the state transition function. At a high level the state transition is made up of two parts:
// - The per-slot transitions, which happens every slot, and only affects a parts of the state.
// - The per-epoch transitions, which happens at every epoch boundary (i.e. state.slot % EPOCH_LENGTH == 0), and affects the entire state.
// The per-slot transitions generally focus on verifying aggregate signatures and saving temporary records relating to the per-slot
// activity in the BeaconState. The per-epoch transitions focus on the validator registry, including adjusting balances and activating
// and exiting validators, as well as processing crosslinks and managing block justification/finalization.
func ExecuteStateTransition(
beaconState *types.BeaconState,
block *types.Block) (*types.BeaconState, error) {
var err error
newState := beaconState.CopyState()
currentSlot := newState.Slot()
newState.SetSlot(currentSlot + 1)
newState, err = randao.UpdateRandaoLayers(newState, newState.Slot())
if err != nil {
return nil, fmt.Errorf("unable to update randao layer %v", err)
}
newhashes, err := newState.CalculateNewBlockHashes(block, currentSlot)
if err != nil {
return nil, fmt.Errorf("unable to calculate recent blockhashes")
}
newState.SetLatestBlockHashes(newhashes)
if block != nil {
newState = ProcessBlock(newState, block)
if newState.Slot()%params.BeaconConfig().EpochLength == 0 {
newState = NewEpochTransition(newState)
}
}
return newState, nil
}
// NewEpochTransition describes the per epoch operations that are performed on the
// beacon state.
func NewEpochTransition(state *types.BeaconState) *types.BeaconState {
// TODO(#1074): This will encompass all the related logic to epoch transitions.
return state
}
// crossLinkCalculations checks if the proposed shard block has recevied
// 2/3 of the votes. If yes, we update crosslink record to point to
// the proposed shard block with latest beacon chain slot numbers.

View File

@@ -2,10 +2,13 @@ package state
import (
"bytes"
"context"
"errors"
"fmt"
"time"
"github.com/ethereum/go-ethereum/common"
gethTypes "github.com/ethereum/go-ethereum/core/types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
@@ -14,10 +17,9 @@ import (
"github.com/prysmaticlabs/prysm/shared/params"
)
// IsValidBlock verifies a block is valid according to the ETH 2.0 specification for
// validity conditions taking into consideration attestation processing and more.
// TODO(#781): Refactor with the new spec validity conditions.
func IsValidBlock(
// IsValidBlockOld checks the validity conditions of a block.
// DEPRECATED: Will be removed soon.
func IsValidBlockOld(
block *types.Block,
beaconState *types.BeaconState,
parentSlot uint64,
@@ -73,6 +75,61 @@ func IsValidBlock(
return nil
}
// IsValidBlock ensures that the block is compliant with the block processing validity conditions.
// Spec:
// For a beacon chain block, block, to be processed by a node, the following conditions must be met:
// The parent block with root block.parent_root has been processed and accepted.
// The node has processed its state up to slot, block.slot - 1.
// The Ethereum 1.0 block pointed to by the state.processed_pow_receipt_root has been processed and accepted.
// The node's local clock time is greater than or equal to state.genesis_time + block.slot * SLOT_DURATION.
func IsValidBlock(
ctx context.Context,
state *types.BeaconState,
block *types.Block,
enablePOWChain bool,
HasBlock func(hash [32]byte) bool,
GetPOWBlock func(ctx context.Context, hash common.Hash) (*gethTypes.Block, error),
genesisTime time.Time) error {
// Pre-Processing Condition 1:
// Check that the parent Block has been processed and saved.
parentBlock := HasBlock(block.ParentHash())
if !parentBlock {
return fmt.Errorf("unprocessed parent block as it is not saved in the db: %#x", block.ParentHash())
}
// Pre-Processing Condition 2:
// The state is updated up to block.slot -1.
if state.Slot() != block.SlotNumber()-1 {
return fmt.Errorf(
"block slot is not valid %d as it is supposed to be %d", block.SlotNumber(), state.Slot()+1)
}
if enablePOWChain {
powBlock, err := GetPOWBlock(ctx, state.ProcessedPowReceiptRootHash32())
if err != nil {
return fmt.Errorf("unable to retrieve POW chain reference block %v", err)
}
// Pre-Processing Condition 3:
// The block pointed to by the state in state.processed_pow_receipt_root has
// been processed in the ETH 1.0 chain.
if powBlock == nil {
return fmt.Errorf("proof-of-Work chain reference in state does not exist %#x", state.ProcessedPowReceiptRootHash32())
}
}
// Pre-Processing Condition 4:
// The node's local time is greater than or equal to
// state.genesis_time + block.slot * SLOT_DURATION.
if !block.IsSlotValid(genesisTime) {
return fmt.Errorf("slot of block is too high: %d", block.SlotNumber())
}
return nil
}
// doesParentProposerExist checks that the proposer from the parent slot is included in the first
// aggregated attestation object
func doesParentProposerExist(block *types.Block, beaconState *types.BeaconState, parentSlot uint64) error {

View File

@@ -1,8 +1,12 @@
package state
import (
"context"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
gethTypes "github.com/ethereum/go-ethereum/core/types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/types"
"github.com/prysmaticlabs/prysm/beacon-chain/utils"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
@@ -16,17 +20,109 @@ func init() {
}
type mockDB struct {
hasBlock bool
blockVoteCache utils.BlockVoteCache
}
func (f *mockDB) HasBlock(h [32]byte) bool {
return true
return f.hasBlock
}
func (f *mockDB) ReadBlockVoteCache(blockHashes [][32]byte) (utils.BlockVoteCache, error) {
return f.blockVoteCache, nil
}
type mockPOWClient struct {
blockExists bool
}
func (m *mockPOWClient) BlockByHash(ctx context.Context, hash common.Hash) (*gethTypes.Block, error) {
if m.blockExists {
return &gethTypes.Block{}, nil
}
return nil, nil
}
func TestBadBlock(t *testing.T) {
beaconState, err := types.NewGenesisBeaconState(nil)
if err != nil {
t.Fatalf("failed to generate beacon state: %v", err)
}
ctx := context.Background()
db := &mockDB{}
powClient := &mockPOWClient{}
beaconState.SetSlot(3)
block := types.NewBlock(&pb.BeaconBlock{
Slot: 4,
})
genesisTime := params.BeaconConfig().GenesisTime
db.hasBlock = false
if err := IsValidBlock(ctx, beaconState, block, true,
db.HasBlock, powClient.BlockByHash, genesisTime); err == nil {
t.Fatal("block is valid despite not having a parent")
}
block.Proto().Slot = 3
db.hasBlock = true
if err := IsValidBlock(ctx, beaconState, block, true,
db.HasBlock, powClient.BlockByHash, genesisTime); err == nil {
t.Fatalf("block is valid despite having an invalid slot %d", block.SlotNumber())
}
block.Proto().Slot = 4
powClient.blockExists = false
if err := IsValidBlock(ctx, beaconState, block, true,
db.HasBlock, powClient.BlockByHash, genesisTime); err == nil {
t.Fatalf("block is valid despite having an invalid pow reference block")
}
invalidTime := time.Now().AddDate(1, 2, 3)
powClient.blockExists = false
if err := IsValidBlock(ctx, beaconState, block, true,
db.HasBlock, powClient.BlockByHash, genesisTime); err == nil {
t.Fatalf("block is valid despite having an invalid genesis time %v", invalidTime)
}
}
func TestValidBlock(t *testing.T) {
beaconState, err := types.NewGenesisBeaconState(nil)
if err != nil {
t.Fatalf("failed to generate beacon state: %v", err)
}
ctx := context.Background()
db := &mockDB{}
powClient := &mockPOWClient{}
beaconState.SetSlot(3)
db.hasBlock = true
block := types.NewBlock(&pb.BeaconBlock{
Slot: 4,
})
genesisTime := params.BeaconConfig().GenesisTime
powClient.blockExists = true
if err := IsValidBlock(ctx, beaconState, block, true,
db.HasBlock, powClient.BlockByHash, genesisTime); err != nil {
t.Fatal(err)
}
}
func TestBlockValidity(t *testing.T) {
beaconState, err := types.NewGenesisBeaconState(nil)
if err != nil {
@@ -60,9 +156,10 @@ func TestBlockValidity(t *testing.T) {
parentSlot := uint64(0)
db := &mockDB{}
db.hasBlock = true
genesisTime := params.BeaconConfig().GenesisTime
if err := IsValidBlock(b, beaconState, parentSlot, genesisTime, db.HasBlock); err != nil {
if err := IsValidBlockOld(b, beaconState, parentSlot, genesisTime, db.HasBlock); err != nil {
t.Fatalf("failed block validation: %v", err)
}
}
@@ -82,6 +179,7 @@ func TestBlockValidityNoParentProposer(t *testing.T) {
parentSlot := uint64(1)
db := &mockDB{}
db.hasBlock = true
// Test case with invalid RANDAO reveal.
badRandaoBlock := types.NewBlock(&pb.BeaconBlock{
@@ -97,7 +195,7 @@ func TestBlockValidityNoParentProposer(t *testing.T) {
},
})
genesisTime := params.BeaconConfig().GenesisTime
if err := IsValidBlock(badRandaoBlock, beaconState, parentSlot, genesisTime, db.HasBlock); err == nil {
if err := IsValidBlockOld(badRandaoBlock, beaconState, parentSlot, genesisTime, db.HasBlock); err == nil {
t.Fatal("test should have failed without a parent proposer")
}
}
@@ -117,6 +215,7 @@ func TestBlockValidityInvalidRandao(t *testing.T) {
parentSlot := uint64(0)
db := &mockDB{}
db.hasBlock = true
// Test case with invalid RANDAO reveal.
badRandaoBlock := types.NewBlock(&pb.BeaconBlock{
@@ -133,7 +232,7 @@ func TestBlockValidityInvalidRandao(t *testing.T) {
})
genesisTime := params.BeaconConfig().GenesisTime
if err := IsValidBlock(badRandaoBlock, beaconState, parentSlot, genesisTime, db.HasBlock); err == nil {
if err := IsValidBlockOld(badRandaoBlock, beaconState, parentSlot, genesisTime, db.HasBlock); err == nil {
t.Fatal("should have failed with invalid RANDAO")
}
}

View File

@@ -73,7 +73,7 @@ func NewGenesisBeaconState(genesisValidatorRegistry []*pb.ValidatorRecord) (*Bea
LatestBlockRootHash32S: latestBlockHashes,
LatestPenalizedExitBalances: []uint64{},
LatestAttestations: []*pb.PendingAttestationRecord{},
ProcessedPowReceiptRootHash32: [][]byte{},
ProcessedPowReceiptRootHash32: []byte{},
CandidatePowReceiptRoots: []*pb.CandidatePoWReceiptRootRecord{},
GenesisTime: 0,
ForkData: &pb.ForkData{
@@ -124,6 +124,20 @@ func (b *BeaconState) CopyState() *BeaconState {
}
}
shardAndCommitteesAtSlots := make([]*pb.ShardAndCommitteeArray, len(b.ShardAndCommitteesAtSlots()))
for index, shardAndCommitteesAtSlot := range b.ShardAndCommitteesAtSlots() {
shardAndCommittees := make([]*pb.ShardAndCommittee, len(shardAndCommitteesAtSlot.GetArrayShardAndCommittee()))
for index, shardAndCommittee := range shardAndCommitteesAtSlot.GetArrayShardAndCommittee() {
shardAndCommittees[index] = &pb.ShardAndCommittee{
Shard: shardAndCommittee.GetShard(),
Committee: shardAndCommittee.GetCommittee(),
}
}
shardAndCommitteesAtSlots[index] = &pb.ShardAndCommitteeArray{
ArrayShardAndCommittee: shardAndCommittees,
}
}
newState := BeaconState{&pb.BeaconState{
LastStateRecalculationSlot: b.LastStateRecalculationSlot(),
JustifiedStreak: b.JustifiedStreak(),
@@ -133,8 +147,10 @@ func (b *BeaconState) CopyState() *BeaconState {
LatestCrosslinks: crosslinks,
ValidatorRegistry: validators,
ShardAndCommitteesForSlots: shardAndCommitteesForSlots,
ShardAndCommitteesAtSlots: shardAndCommitteesAtSlots,
LatestPenalizedExitBalances: b.LatestPenalizedExitBalances(),
ForkData: b.ForkData(),
LatestBlockRootHash32S: b.data.LatestBlockRootHash32S,
}}
return &newState
@@ -219,6 +235,11 @@ func (b *BeaconState) ShardAndCommitteesForSlots() []*pb.ShardAndCommitteeArray
return b.data.ShardAndCommitteesForSlots
}
// ShardAndCommitteesAtSlots returns the shard committee object.
func (b *BeaconState) ShardAndCommitteesAtSlots() []*pb.ShardAndCommitteeArray {
return b.data.ShardAndCommitteesAtSlots
}
// LatestCrosslinks returns the cross link records of the all the shards.
func (b *BeaconState) LatestCrosslinks() []*pb.CrosslinkRecord {
return b.data.LatestCrosslinks
@@ -259,8 +280,10 @@ func (b *BeaconState) LatestAttestations() []*pb.PendingAttestationRecord {
// ProcessedPowReceiptRootHash32 returns the root hashes of the
// processed transaction receipts from the POW chain.
func (b *BeaconState) ProcessedPowReceiptRootHash32() [][]byte {
return b.data.ProcessedPowReceiptRootHash32
func (b *BeaconState) ProcessedPowReceiptRootHash32() [32]byte {
var h [32]byte
copy(h[:], b.data.ProcessedPowReceiptRootHash32)
return h
}
// CandidatePowReceiptRoots returns the root records of receipts that have
@@ -468,6 +491,11 @@ func (b *BeaconState) SetShardAndCommitteesForSlots(shardAndCommitteesForSlot []
b.data.ShardAndCommitteesForSlots = shardAndCommitteesForSlot
}
// SetShardAndCommitteesAtSlots updates the inner proto's shard and committees for slots.
func (b *BeaconState) SetShardAndCommitteesAtSlots(shardAndCommitteesAtSlot []*pb.ShardAndCommitteeArray) {
b.data.ShardAndCommitteesAtSlots = shardAndCommitteesAtSlot
}
// SetValidatorRegistry updates the state's internal validator set.
func (b *BeaconState) SetValidatorRegistry(validators []*pb.ValidatorRecord) {
b.data.ValidatorRegistry = validators
@@ -521,8 +549,8 @@ func (b *BeaconState) SetLatestAttestations(attestations []*pb.PendingAttestatio
// SetProcessedPowReceiptHash saves the POW receipts which have
// been processed by the POW chain.
func (b *BeaconState) SetProcessedPowReceiptHash(hash [][]byte) {
b.data.ProcessedPowReceiptRootHash32 = hash
func (b *BeaconState) SetProcessedPowReceiptHash(hash [32]byte) {
b.data.ProcessedPowReceiptRootHash32 = hash[:]
}
// SetCandidatePowReceiptRoots saves the latest roots of POW receipts that have

View File

@@ -132,6 +132,26 @@ func GetShardAndCommitteesForSlot(shardCommittees []*pb.ShardAndCommitteeArray,
return shardCommittees[index], nil
}
// BeaconProposerIndex returns the index of the proposer of the block at a
// given slot.
//
// Spec pseudocode definition:
// def get_beacon_proposer_index(state: BeaconState,slot: int) -> int:
// """
// Returns the beacon proposer index for the ``slot``.
// """
// first_committee = get_shard_committees_at_slot(state, slot)[0].committee
// return first_committee[slot % len(first_committee)]
func BeaconProposerIndex(state *pb.BeaconState, slot uint64) (uint32, error) {
committeeArray, err := ShardAndCommitteesAtSlot(state, slot)
if err != nil {
return 0, err
}
firstCommittee := committeeArray.GetArrayShardAndCommittee()[0].Committee
return firstCommittee[slot%uint64(len(firstCommittee))], nil
}
// AreAttesterBitfieldsValid validates that the length of the attester bitfield matches the attester indices
// defined in the Crystallized State.
func AreAttesterBitfieldsValid(attestation *pb.AggregatedAttestation, attesterIndices []uint32) bool {

View File

@@ -725,3 +725,63 @@ func TestBoundaryAttesterIndices(t *testing.T) {
t.Errorf("Incorrect boundary attester indices. Wanted: %v, got: %v", []uint32{1, 5, 6, 7}, attesterIndices)
}
}
func TestBeaconProposerIndex(t *testing.T) {
if params.BeaconConfig().EpochLength != 64 {
t.Errorf("EpochLength should be 64 for these tests to pass")
}
var shardAndCommittees []*pb.ShardAndCommitteeArray
for i := uint64(0); i < params.BeaconConfig().EpochLength*2; i++ {
shardAndCommittees = append(shardAndCommittees, &pb.ShardAndCommitteeArray{
ArrayShardAndCommittee: []*pb.ShardAndCommittee{
{Committee: []uint32{9, 8, 311, 12, 92, 1, 23, 17}},
},
})
}
state := &pb.BeaconState{
ShardAndCommitteesAtSlots: shardAndCommittees,
}
tests := []struct {
slot uint64
index uint32
}{
{
slot: 1,
index: 8,
},
{
slot: 10,
index: 311,
},
{
slot: 19,
index: 12,
},
{
slot: 30,
index: 23,
},
{
slot: 39,
index: 17,
},
}
for _, tt := range tests {
result, err := BeaconProposerIndex(state, tt.slot)
if err != nil {
t.Errorf("Failed to get shard and committees at slot: %v", err)
}
if result != tt.index {
t.Errorf(
"Result index was an unexpected value. Wanted %d, got %d",
tt.index,
result,
)
}
}
}

View File

@@ -1,6 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// source: proto/beacon/p2p/v1/messages.proto
package ethereum_beacon_p2p_v1
package v1
import proto "github.com/golang/protobuf/proto"
import fmt "fmt"

View File

@@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// source: proto/beacon/p2p/v1/types.proto
package ethereum_beacon_p2p_v1
package v1
import proto "github.com/golang/protobuf/proto"
import fmt "fmt"
@@ -20,8 +20,6 @@ var _ = math.Inf
// proto package needs to be updated.
const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
// Possible validator status code:
// https://github.com/ethereum/eth2.0-specs/blob/master/specs/core/0_beacon-chain.md#constants
type ValidatorRecord_StatusCodes int32
const (
@@ -51,54 +49,46 @@ func (x ValidatorRecord_StatusCodes) String() string {
return proto.EnumName(ValidatorRecord_StatusCodes_name, int32(x))
}
func (ValidatorRecord_StatusCodes) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{6, 0}
return fileDescriptor_types_22f477521baca46a, []int{6, 0}
}
type BeaconState struct {
// Validator registry [1-1000]
ValidatorRegistry []*ValidatorRecord `protobuf:"bytes,1,rep,name=validator_registry,json=validatorRegistry,proto3" json:"validator_registry,omitempty"`
ValidatorRegistryLastChangeSlot uint64 `protobuf:"varint,2,opt,name=validator_registry_last_change_slot,json=validatorRegistryLastChangeSlot,proto3" json:"validator_registry_last_change_slot,omitempty"`
ValidatorRegistryExitCount uint64 `protobuf:"varint,3,opt,name=validator_registry_exit_count,json=validatorRegistryExitCount,proto3" json:"validator_registry_exit_count,omitempty"`
ValidatorRegistryDeltaChainTipHash32 []byte `protobuf:"bytes,4,opt,name=validator_registry_delta_chain_tip_hash32,json=validatorRegistryDeltaChainTipHash32,proto3" json:"validator_registry_delta_chain_tip_hash32,omitempty"`
// Randomness and committees [1001-2000]
RandaoMixHash32 []byte `protobuf:"bytes,1001,opt,name=randao_mix_hash32,json=randaoMixHash32,proto3" json:"randao_mix_hash32,omitempty"`
NextSeedHash32 []byte `protobuf:"bytes,1002,opt,name=next_seed_hash32,json=nextSeedHash32,proto3" json:"next_seed_hash32,omitempty"`
ShardAndCommitteesAtSlots []*ShardAndCommitteeArray `protobuf:"bytes,1003,rep,name=shard_and_committees_at_slots,json=shardAndCommitteesAtSlots,proto3" json:"shard_and_committees_at_slots,omitempty"`
PersistentCommittees []*common.Uint32List `protobuf:"bytes,1004,rep,name=persistent_committees,json=persistentCommittees,proto3" json:"persistent_committees,omitempty"`
PersistentCommitteeReassignments []*ShardReassignmentRecord `protobuf:"bytes,1005,rep,name=persistent_committee_reassignments,json=persistentCommitteeReassignments,proto3" json:"persistent_committee_reassignments,omitempty"`
// Finality [2001-3000]
PreviousJustifiedSlot uint64 `protobuf:"varint,2001,opt,name=previous_justified_slot,json=previousJustifiedSlot,proto3" json:"previous_justified_slot,omitempty"`
JustifiedSlot uint64 `protobuf:"varint,2002,opt,name=justified_slot,json=justifiedSlot,proto3" json:"justified_slot,omitempty"`
JustifiedSlotBitfield uint64 `protobuf:"varint,2003,opt,name=justified_slot_bitfield,json=justifiedSlotBitfield,proto3" json:"justified_slot_bitfield,omitempty"`
FinalizedSlot uint64 `protobuf:"varint,2004,opt,name=finalized_slot,json=finalizedSlot,proto3" json:"finalized_slot,omitempty"`
// Recent state [3001-4000]
LatestCrosslinks []*CrosslinkRecord `protobuf:"bytes,3001,rep,name=latest_crosslinks,json=latestCrosslinks,proto3" json:"latest_crosslinks,omitempty"`
LastStateRecalculationSlot uint64 `protobuf:"varint,3002,opt,name=last_state_recalculation_slot,json=lastStateRecalculationSlot,proto3" json:"last_state_recalculation_slot,omitempty"`
LatestBlockRootHash32S [][]byte `protobuf:"bytes,3003,rep,name=latest_block_root_hash32s,json=latestBlockRootHash32s,proto3" json:"latest_block_root_hash32s,omitempty"`
LatestPenalizedExitBalances []uint64 `protobuf:"varint,3004,rep,packed,name=latest_penalized_exit_balances,json=latestPenalizedExitBalances,proto3" json:"latest_penalized_exit_balances,omitempty"`
LatestAttestations []*PendingAttestationRecord `protobuf:"bytes,3005,rep,name=latest_attestations,json=latestAttestations,proto3" json:"latest_attestations,omitempty"`
// PoW receipt root [4001-5000]
ProcessedPowReceiptRootHash32 [][]byte `protobuf:"bytes,4001,rep,name=processed_pow_receipt_root_hash32,json=processedPowReceiptRootHash32,proto3" json:"processed_pow_receipt_root_hash32,omitempty"`
CandidatePowReceiptRoots []*CandidatePoWReceiptRootRecord `protobuf:"bytes,4002,rep,name=candidate_pow_receipt_roots,json=candidatePowReceiptRoots,proto3" json:"candidate_pow_receipt_roots,omitempty"`
// Miscellaneous [5001-6000]
GenesisTime uint64 `protobuf:"varint,5001,opt,name=genesis_time,json=genesisTime,proto3" json:"genesis_time,omitempty"`
ForkData *ForkData `protobuf:"bytes,5002,opt,name=fork_data,json=forkData,proto3" json:"fork_data,omitempty"`
Slot uint64 `protobuf:"varint,5003,opt,name=slot,proto3" json:"slot,omitempty"`
// Deprecated fields
// All fields mustbe annotated with [deprecated=true];
JustifiedStreak uint64 `protobuf:"varint,10007,opt,name=justified_streak,json=justifiedStreak,proto3" json:"justified_streak,omitempty"` // Deprecated: Do not use.
PendingAttestations []*AggregatedAttestation `protobuf:"bytes,10018,rep,name=pending_attestations,json=pendingAttestations,proto3" json:"pending_attestations,omitempty"` // Deprecated: Do not use.
ShardAndCommitteesForSlots []*ShardAndCommitteeArray `protobuf:"bytes,10019,rep,name=shard_and_committees_for_slots,json=shardAndCommitteesForSlots,proto3" json:"shard_and_committees_for_slots,omitempty"` // Deprecated: Do not use.
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
ValidatorRegistry []*ValidatorRecord `protobuf:"bytes,1,rep,name=validator_registry,json=validatorRegistry,proto3" json:"validator_registry,omitempty"`
ValidatorRegistryLastChangeSlot uint64 `protobuf:"varint,2,opt,name=validator_registry_last_change_slot,json=validatorRegistryLastChangeSlot,proto3" json:"validator_registry_last_change_slot,omitempty"`
ValidatorRegistryExitCount uint64 `protobuf:"varint,3,opt,name=validator_registry_exit_count,json=validatorRegistryExitCount,proto3" json:"validator_registry_exit_count,omitempty"`
ValidatorRegistryDeltaChainTipHash32 []byte `protobuf:"bytes,4,opt,name=validator_registry_delta_chain_tip_hash32,json=validatorRegistryDeltaChainTipHash32,proto3" json:"validator_registry_delta_chain_tip_hash32,omitempty"`
RandaoMixHash32 []byte `protobuf:"bytes,1001,opt,name=randao_mix_hash32,json=randaoMixHash32,proto3" json:"randao_mix_hash32,omitempty"`
NextSeedHash32 []byte `protobuf:"bytes,1002,opt,name=next_seed_hash32,json=nextSeedHash32,proto3" json:"next_seed_hash32,omitempty"`
ShardAndCommitteesAtSlots []*ShardAndCommitteeArray `protobuf:"bytes,1003,rep,name=shard_and_committees_at_slots,json=shardAndCommitteesAtSlots,proto3" json:"shard_and_committees_at_slots,omitempty"`
PersistentCommittees []*common.Uint32List `protobuf:"bytes,1004,rep,name=persistent_committees,json=persistentCommittees,proto3" json:"persistent_committees,omitempty"`
PersistentCommitteeReassignments []*ShardReassignmentRecord `protobuf:"bytes,1005,rep,name=persistent_committee_reassignments,json=persistentCommitteeReassignments,proto3" json:"persistent_committee_reassignments,omitempty"`
PreviousJustifiedSlot uint64 `protobuf:"varint,2001,opt,name=previous_justified_slot,json=previousJustifiedSlot,proto3" json:"previous_justified_slot,omitempty"`
JustifiedSlot uint64 `protobuf:"varint,2002,opt,name=justified_slot,json=justifiedSlot,proto3" json:"justified_slot,omitempty"`
JustifiedSlotBitfield uint64 `protobuf:"varint,2003,opt,name=justified_slot_bitfield,json=justifiedSlotBitfield,proto3" json:"justified_slot_bitfield,omitempty"`
FinalizedSlot uint64 `protobuf:"varint,2004,opt,name=finalized_slot,json=finalizedSlot,proto3" json:"finalized_slot,omitempty"`
LatestCrosslinks []*CrosslinkRecord `protobuf:"bytes,3001,rep,name=latest_crosslinks,json=latestCrosslinks,proto3" json:"latest_crosslinks,omitempty"`
LastStateRecalculationSlot uint64 `protobuf:"varint,3002,opt,name=last_state_recalculation_slot,json=lastStateRecalculationSlot,proto3" json:"last_state_recalculation_slot,omitempty"`
LatestBlockRootHash32S [][]byte `protobuf:"bytes,3003,rep,name=latest_block_root_hash32s,json=latestBlockRootHash32s,proto3" json:"latest_block_root_hash32s,omitempty"`
LatestPenalizedExitBalances []uint64 `protobuf:"varint,3004,rep,packed,name=latest_penalized_exit_balances,json=latestPenalizedExitBalances,proto3" json:"latest_penalized_exit_balances,omitempty"`
LatestAttestations []*PendingAttestationRecord `protobuf:"bytes,3005,rep,name=latest_attestations,json=latestAttestations,proto3" json:"latest_attestations,omitempty"`
ProcessedPowReceiptRootHash32 []byte `protobuf:"bytes,4001,opt,name=processed_pow_receipt_root_hash32,json=processedPowReceiptRootHash32,proto3" json:"processed_pow_receipt_root_hash32,omitempty"`
CandidatePowReceiptRoots []*CandidatePoWReceiptRootRecord `protobuf:"bytes,4002,rep,name=candidate_pow_receipt_roots,json=candidatePowReceiptRoots,proto3" json:"candidate_pow_receipt_roots,omitempty"`
GenesisTime uint64 `protobuf:"varint,5001,opt,name=genesis_time,json=genesisTime,proto3" json:"genesis_time,omitempty"`
ForkData *ForkData `protobuf:"bytes,5002,opt,name=fork_data,json=forkData,proto3" json:"fork_data,omitempty"`
Slot uint64 `protobuf:"varint,5003,opt,name=slot,proto3" json:"slot,omitempty"`
JustifiedStreak uint64 `protobuf:"varint,10007,opt,name=justified_streak,json=justifiedStreak,proto3" json:"justified_streak,omitempty"` // Deprecated: Do not use.
PendingAttestations []*AggregatedAttestation `protobuf:"bytes,10018,rep,name=pending_attestations,json=pendingAttestations,proto3" json:"pending_attestations,omitempty"` // Deprecated: Do not use.
ShardAndCommitteesForSlots []*ShardAndCommitteeArray `protobuf:"bytes,10019,rep,name=shard_and_committees_for_slots,json=shardAndCommitteesForSlots,proto3" json:"shard_and_committees_for_slots,omitempty"` // Deprecated: Do not use.
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *BeaconState) Reset() { *m = BeaconState{} }
func (m *BeaconState) String() string { return proto.CompactTextString(m) }
func (*BeaconState) ProtoMessage() {}
func (*BeaconState) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{0}
return fileDescriptor_types_22f477521baca46a, []int{0}
}
func (m *BeaconState) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_BeaconState.Unmarshal(m, b)
@@ -244,7 +234,7 @@ func (m *BeaconState) GetLatestAttestations() []*PendingAttestationRecord {
return nil
}
func (m *BeaconState) GetProcessedPowReceiptRootHash32() [][]byte {
func (m *BeaconState) GetProcessedPowReceiptRootHash32() []byte {
if m != nil {
return m.ProcessedPowReceiptRootHash32
}
@@ -316,7 +306,7 @@ func (m *ForkData) Reset() { *m = ForkData{} }
func (m *ForkData) String() string { return proto.CompactTextString(m) }
func (*ForkData) ProtoMessage() {}
func (*ForkData) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{1}
return fileDescriptor_types_22f477521baca46a, []int{1}
}
func (m *ForkData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ForkData.Unmarshal(m, b)
@@ -369,7 +359,7 @@ func (m *CandidatePoWReceiptRootRecord) Reset() { *m = CandidatePoWRecei
func (m *CandidatePoWReceiptRootRecord) String() string { return proto.CompactTextString(m) }
func (*CandidatePoWReceiptRootRecord) ProtoMessage() {}
func (*CandidatePoWReceiptRootRecord) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{2}
return fileDescriptor_types_22f477521baca46a, []int{2}
}
func (m *CandidatePoWReceiptRootRecord) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CandidatePoWReceiptRootRecord.Unmarshal(m, b)
@@ -417,7 +407,7 @@ func (m *PendingAttestationRecord) Reset() { *m = PendingAttestationReco
func (m *PendingAttestationRecord) String() string { return proto.CompactTextString(m) }
func (*PendingAttestationRecord) ProtoMessage() {}
func (*PendingAttestationRecord) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{3}
return fileDescriptor_types_22f477521baca46a, []int{3}
}
func (m *PendingAttestationRecord) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PendingAttestationRecord.Unmarshal(m, b)
@@ -479,7 +469,7 @@ func (m *Attestation) Reset() { *m = Attestation{} }
func (m *Attestation) String() string { return proto.CompactTextString(m) }
func (*Attestation) ProtoMessage() {}
func (*Attestation) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{4}
return fileDescriptor_types_22f477521baca46a, []int{4}
}
func (m *Attestation) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Attestation.Unmarshal(m, b)
@@ -545,7 +535,7 @@ func (m *AttestationData) Reset() { *m = AttestationData{} }
func (m *AttestationData) String() string { return proto.CompactTextString(m) }
func (*AttestationData) ProtoMessage() {}
func (*AttestationData) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{5}
return fileDescriptor_types_22f477521baca46a, []int{5}
}
func (m *AttestationData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AttestationData.Unmarshal(m, b)
@@ -622,12 +612,10 @@ func (m *AttestationData) GetJustifiedBlockRootHash32() []byte {
}
type ValidatorRecord struct {
Pubkey []byte `protobuf:"bytes,1,opt,name=pubkey,proto3" json:"pubkey,omitempty"`
// TODO(781): The usage of withdrawal_credentials is not defined in spec. Not used in Prysm yet.
WithdrawalCredentials []byte `protobuf:"bytes,2,opt,name=withdrawal_credentials,json=withdrawalCredentials,proto3" json:"withdrawal_credentials,omitempty"`
RandaoCommitmentHash32 []byte `protobuf:"bytes,3,opt,name=randao_commitment_hash32,json=randaoCommitmentHash32,proto3" json:"randao_commitment_hash32,omitempty"`
RandaoSkips uint64 `protobuf:"varint,4,opt,name=randao_skips,json=randaoSkips,proto3" json:"randao_skips,omitempty"`
// Balance in Gwei
Pubkey []byte `protobuf:"bytes,1,opt,name=pubkey,proto3" json:"pubkey,omitempty"`
WithdrawalCredentials []byte `protobuf:"bytes,2,opt,name=withdrawal_credentials,json=withdrawalCredentials,proto3" json:"withdrawal_credentials,omitempty"`
RandaoCommitmentHash32 []byte `protobuf:"bytes,3,opt,name=randao_commitment_hash32,json=randaoCommitmentHash32,proto3" json:"randao_commitment_hash32,omitempty"`
RandaoLayers uint64 `protobuf:"varint,4,opt,name=randao_layers,json=randaoLayers,proto3" json:"randao_layers,omitempty"`
Balance uint64 `protobuf:"varint,5,opt,name=balance,proto3" json:"balance,omitempty"`
Status ValidatorRecord_StatusCodes `protobuf:"varint,6,opt,name=status,proto3,enum=ethereum.beacon.p2p.v1.ValidatorRecord_StatusCodes" json:"status,omitempty"`
LatestStatusChangeSlot uint64 `protobuf:"varint,7,opt,name=latest_status_change_slot,json=latestStatusChangeSlot,proto3" json:"latest_status_change_slot,omitempty"`
@@ -641,7 +629,7 @@ func (m *ValidatorRecord) Reset() { *m = ValidatorRecord{} }
func (m *ValidatorRecord) String() string { return proto.CompactTextString(m) }
func (*ValidatorRecord) ProtoMessage() {}
func (*ValidatorRecord) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{6}
return fileDescriptor_types_22f477521baca46a, []int{6}
}
func (m *ValidatorRecord) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ValidatorRecord.Unmarshal(m, b)
@@ -682,9 +670,9 @@ func (m *ValidatorRecord) GetRandaoCommitmentHash32() []byte {
return nil
}
func (m *ValidatorRecord) GetRandaoSkips() uint64 {
func (m *ValidatorRecord) GetRandaoLayers() uint64 {
if m != nil {
return m.RandaoSkips
return m.RandaoLayers
}
return 0
}
@@ -730,7 +718,7 @@ func (m *ShardReassignmentRecord) Reset() { *m = ShardReassignmentRecord
func (m *ShardReassignmentRecord) String() string { return proto.CompactTextString(m) }
func (*ShardReassignmentRecord) ProtoMessage() {}
func (*ShardReassignmentRecord) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{7}
return fileDescriptor_types_22f477521baca46a, []int{7}
}
func (m *ShardReassignmentRecord) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ShardReassignmentRecord.Unmarshal(m, b)
@@ -771,7 +759,6 @@ func (m *ShardReassignmentRecord) GetSlot() uint64 {
return 0
}
// AggregatedAttestation is deprecated entirely by AttestationRecord.
type AggregatedAttestation struct {
Slot uint64 `protobuf:"varint,1,opt,name=slot,proto3" json:"slot,omitempty"` // Deprecated: Do not use.
Shard uint64 `protobuf:"varint,2,opt,name=shard,proto3" json:"shard,omitempty"` // Deprecated: Do not use.
@@ -790,7 +777,7 @@ func (m *AggregatedAttestation) Reset() { *m = AggregatedAttestation{} }
func (m *AggregatedAttestation) String() string { return proto.CompactTextString(m) }
func (*AggregatedAttestation) ProtoMessage() {}
func (*AggregatedAttestation) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{8}
return fileDescriptor_types_22f477521baca46a, []int{8}
}
func (m *AggregatedAttestation) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AggregatedAttestation.Unmarshal(m, b)
@@ -886,7 +873,7 @@ func (m *SpecialRecord) Reset() { *m = SpecialRecord{} }
func (m *SpecialRecord) String() string { return proto.CompactTextString(m) }
func (*SpecialRecord) ProtoMessage() {}
func (*SpecialRecord) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{9}
return fileDescriptor_types_22f477521baca46a, []int{9}
}
func (m *SpecialRecord) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SpecialRecord.Unmarshal(m, b)
@@ -934,7 +921,7 @@ func (m *CrosslinkRecord) Reset() { *m = CrosslinkRecord{} }
func (m *CrosslinkRecord) String() string { return proto.CompactTextString(m) }
func (*CrosslinkRecord) ProtoMessage() {}
func (*CrosslinkRecord) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{10}
return fileDescriptor_types_22f477521baca46a, []int{10}
}
func (m *CrosslinkRecord) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CrosslinkRecord.Unmarshal(m, b)
@@ -979,7 +966,7 @@ func (m *ShardAndCommitteeArray) Reset() { *m = ShardAndCommitteeArray{}
func (m *ShardAndCommitteeArray) String() string { return proto.CompactTextString(m) }
func (*ShardAndCommitteeArray) ProtoMessage() {}
func (*ShardAndCommitteeArray) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{11}
return fileDescriptor_types_22f477521baca46a, []int{11}
}
func (m *ShardAndCommitteeArray) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ShardAndCommitteeArray.Unmarshal(m, b)
@@ -1019,7 +1006,7 @@ func (m *ShardAndCommittee) Reset() { *m = ShardAndCommittee{} }
func (m *ShardAndCommittee) String() string { return proto.CompactTextString(m) }
func (*ShardAndCommittee) ProtoMessage() {}
func (*ShardAndCommittee) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{12}
return fileDescriptor_types_22f477521baca46a, []int{12}
}
func (m *ShardAndCommittee) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ShardAndCommittee.Unmarshal(m, b)
@@ -1061,28 +1048,25 @@ func (m *ShardAndCommittee) GetTotalValidatorCount() uint64 {
}
type BeaconBlock struct {
Slot uint64 `protobuf:"varint,1,opt,name=slot,proto3" json:"slot,omitempty"`
ParentRootHash32 []byte `protobuf:"bytes,2,opt,name=parent_root_hash32,json=parentRootHash32,proto3" json:"parent_root_hash32,omitempty"`
StateRootHash32 []byte `protobuf:"bytes,3,opt,name=state_root_hash32,json=stateRootHash32,proto3" json:"state_root_hash32,omitempty"`
RandaoRevealHash32 []byte `protobuf:"bytes,4,opt,name=randao_reveal_hash32,json=randaoRevealHash32,proto3" json:"randao_reveal_hash32,omitempty"`
CandidatePowReceiptRootHash32 []byte `protobuf:"bytes,5,opt,name=candidate_pow_receipt_root_hash32,json=candidatePowReceiptRootHash32,proto3" json:"candidate_pow_receipt_root_hash32,omitempty"`
Signature [][]byte `protobuf:"bytes,6,rep,name=signature,proto3" json:"signature,omitempty"`
// Block Body
Body *BeaconBlockBody `protobuf:"bytes,7,opt,name=body,proto3" json:"body,omitempty"`
// Deprecated fields
// All fields must be annotated with [deprecated=true];
Timestamp *timestamp.Timestamp `protobuf:"bytes,1006,opt,name=timestamp,proto3" json:"timestamp,omitempty"` // Deprecated: Do not use.
Attestations []*AggregatedAttestation `protobuf:"bytes,1007,rep,name=attestations,proto3" json:"attestations,omitempty"` // Deprecated: Do not use.
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
Slot uint64 `protobuf:"varint,1,opt,name=slot,proto3" json:"slot,omitempty"`
ParentRootHash32 []byte `protobuf:"bytes,2,opt,name=parent_root_hash32,json=parentRootHash32,proto3" json:"parent_root_hash32,omitempty"`
StateRootHash32 []byte `protobuf:"bytes,3,opt,name=state_root_hash32,json=stateRootHash32,proto3" json:"state_root_hash32,omitempty"`
RandaoRevealHash32 []byte `protobuf:"bytes,4,opt,name=randao_reveal_hash32,json=randaoRevealHash32,proto3" json:"randao_reveal_hash32,omitempty"`
CandidatePowReceiptRootHash32 []byte `protobuf:"bytes,5,opt,name=candidate_pow_receipt_root_hash32,json=candidatePowReceiptRootHash32,proto3" json:"candidate_pow_receipt_root_hash32,omitempty"`
Signature [][]byte `protobuf:"bytes,6,rep,name=signature,proto3" json:"signature,omitempty"`
Body *BeaconBlockBody `protobuf:"bytes,7,opt,name=body,proto3" json:"body,omitempty"`
Timestamp *timestamp.Timestamp `protobuf:"bytes,1006,opt,name=timestamp,proto3" json:"timestamp,omitempty"` // Deprecated: Do not use.
Attestations []*AggregatedAttestation `protobuf:"bytes,1007,rep,name=attestations,proto3" json:"attestations,omitempty"` // Deprecated: Do not use.
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *BeaconBlock) Reset() { *m = BeaconBlock{} }
func (m *BeaconBlock) String() string { return proto.CompactTextString(m) }
func (*BeaconBlock) ProtoMessage() {}
func (*BeaconBlock) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{13}
return fileDescriptor_types_22f477521baca46a, []int{13}
}
func (m *BeaconBlock) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_BeaconBlock.Unmarshal(m, b)
@@ -1182,7 +1166,7 @@ func (m *BeaconBlockBody) Reset() { *m = BeaconBlockBody{} }
func (m *BeaconBlockBody) String() string { return proto.CompactTextString(m) }
func (*BeaconBlockBody) ProtoMessage() {}
func (*BeaconBlockBody) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{14}
return fileDescriptor_types_22f477521baca46a, []int{14}
}
func (m *BeaconBlockBody) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_BeaconBlockBody.Unmarshal(m, b)
@@ -1238,9 +1222,8 @@ func (m *BeaconBlockBody) GetExits() []*Exit {
}
type DepositParameters struct {
Pubkey []byte `protobuf:"bytes,1,opt,name=pubkey,proto3" json:"pubkey,omitempty"`
ProofOfPossession []byte `protobuf:"bytes,2,opt,name=proof_of_possession,json=proofOfPossession,proto3" json:"proof_of_possession,omitempty"`
// TODO(781): The usage of withdrawal_credentials is not defined in spec. Not used in Prysm yet.
Pubkey []byte `protobuf:"bytes,1,opt,name=pubkey,proto3" json:"pubkey,omitempty"`
ProofOfPossession []byte `protobuf:"bytes,2,opt,name=proof_of_possession,json=proofOfPossession,proto3" json:"proof_of_possession,omitempty"`
WithdrawalCredentialsHash32 []byte `protobuf:"bytes,3,opt,name=withdrawal_credentials_hash32,json=withdrawalCredentialsHash32,proto3" json:"withdrawal_credentials_hash32,omitempty"`
RandaoCommitmentHash32 []byte `protobuf:"bytes,4,opt,name=randao_commitment_hash32,json=randaoCommitmentHash32,proto3" json:"randao_commitment_hash32,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@@ -1252,7 +1235,7 @@ func (m *DepositParameters) Reset() { *m = DepositParameters{} }
func (m *DepositParameters) String() string { return proto.CompactTextString(m) }
func (*DepositParameters) ProtoMessage() {}
func (*DepositParameters) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{15}
return fileDescriptor_types_22f477521baca46a, []int{15}
}
func (m *DepositParameters) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DepositParameters.Unmarshal(m, b)
@@ -1313,7 +1296,7 @@ func (m *ProposalSignedData) Reset() { *m = ProposalSignedData{} }
func (m *ProposalSignedData) String() string { return proto.CompactTextString(m) }
func (*ProposalSignedData) ProtoMessage() {}
func (*ProposalSignedData) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{16}
return fileDescriptor_types_22f477521baca46a, []int{16}
}
func (m *ProposalSignedData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProposalSignedData.Unmarshal(m, b)
@@ -1368,7 +1351,7 @@ func (m *SlashableVoteData) Reset() { *m = SlashableVoteData{} }
func (m *SlashableVoteData) String() string { return proto.CompactTextString(m) }
func (*SlashableVoteData) ProtoMessage() {}
func (*SlashableVoteData) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{17}
return fileDescriptor_types_22f477521baca46a, []int{17}
}
func (m *SlashableVoteData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SlashableVoteData.Unmarshal(m, b)
@@ -1429,7 +1412,7 @@ func (m *DepositData) Reset() { *m = DepositData{} }
func (m *DepositData) String() string { return proto.CompactTextString(m) }
func (*DepositData) ProtoMessage() {}
func (*DepositData) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{18}
return fileDescriptor_types_22f477521baca46a, []int{18}
}
func (m *DepositData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DepositData.Unmarshal(m, b)
@@ -1485,7 +1468,7 @@ func (m *ProposerSlashing) Reset() { *m = ProposerSlashing{} }
func (m *ProposerSlashing) String() string { return proto.CompactTextString(m) }
func (*ProposerSlashing) ProtoMessage() {}
func (*ProposerSlashing) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{19}
return fileDescriptor_types_22f477521baca46a, []int{19}
}
func (m *ProposerSlashing) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProposerSlashing.Unmarshal(m, b)
@@ -1552,7 +1535,7 @@ func (m *CasperSlashing) Reset() { *m = CasperSlashing{} }
func (m *CasperSlashing) String() string { return proto.CompactTextString(m) }
func (*CasperSlashing) ProtoMessage() {}
func (*CasperSlashing) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{20}
return fileDescriptor_types_22f477521baca46a, []int{20}
}
func (m *CasperSlashing) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CasperSlashing.Unmarshal(m, b)
@@ -1599,7 +1582,7 @@ func (m *Deposit) Reset() { *m = Deposit{} }
func (m *Deposit) String() string { return proto.CompactTextString(m) }
func (*Deposit) ProtoMessage() {}
func (*Deposit) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{21}
return fileDescriptor_types_22f477521baca46a, []int{21}
}
func (m *Deposit) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Deposit.Unmarshal(m, b)
@@ -1653,7 +1636,7 @@ func (m *Exit) Reset() { *m = Exit{} }
func (m *Exit) String() string { return proto.CompactTextString(m) }
func (*Exit) ProtoMessage() {}
func (*Exit) Descriptor() ([]byte, []int) {
return fileDescriptor_types_49983aeb094386db, []int{22}
return fileDescriptor_types_22f477521baca46a, []int{22}
}
func (m *Exit) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Exit.Unmarshal(m, b)
@@ -1722,160 +1705,159 @@ func init() {
}
func init() {
proto.RegisterFile("proto/beacon/p2p/v1/types.proto", fileDescriptor_types_49983aeb094386db)
proto.RegisterFile("proto/beacon/p2p/v1/types.proto", fileDescriptor_types_22f477521baca46a)
}
var fileDescriptor_types_49983aeb094386db = []byte{
// 2401 bytes of a gzipped FileDescriptorProto
var fileDescriptor_types_22f477521baca46a = []byte{
// 2400 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0x49, 0x73, 0x1c, 0x49,
0x15, 0xa6, 0xd5, 0xad, 0xc5, 0xaf, 0x25, 0x75, 0x77, 0x6a, 0x2b, 0x4b, 0xd6, 0xc8, 0x2e, 0xc3,
0xd8, 0x32, 0x43, 0xcb, 0x6a, 0x33, 0x0c, 0x30, 0x31, 0x4c, 0xa8, 0x25, 0x7b, 0x2c, 0x30, 0xb2,
0xa6, 0x24, 0xcb, 0x43, 0x04, 0x11, 0x15, 0xd9, 0x55, 0xd9, 0xad, 0xb2, 0xaa, 0x2b, 0x6b, 0x2a,
0xb3, 0x65, 0x09, 0x82, 0x99, 0x33, 0x70, 0xe0, 0x44, 0x40, 0x04, 0x27, 0x86, 0x5f, 0xc0, 0x91,
0xed, 0xc8, 0x0f, 0x60, 0xb9, 0x70, 0x80, 0xfb, 0x0c, 0xdb, 0x91, 0x03, 0x07, 0x88, 0x5c, 0x6a,
0xeb, 0x45, 0x5e, 0xb8, 0x70, 0x92, 0xf2, 0xbd, 0xef, 0xbd, 0x7c, 0xfd, 0xf2, 0xad, 0x05, 0x6b,
0x61, 0x44, 0x39, 0xdd, 0x68, 0x11, 0xec, 0xd0, 0x60, 0x23, 0x6c, 0x84, 0x1b, 0xa7, 0x9b, 0x1b,
0xfc, 0x3c, 0x24, 0xac, 0x2e, 0x39, 0x68, 0x91, 0xf0, 0x63, 0x12, 0x91, 0x5e, 0xb7, 0xae, 0x30,
0xf5, 0xb0, 0x11, 0xd6, 0x4f, 0x37, 0x97, 0xd7, 0x3a, 0x94, 0x76, 0x7c, 0xb2, 0x21, 0x51, 0xad,
0x5e, 0x7b, 0x83, 0x7b, 0x5d, 0xc2, 0x38, 0xee, 0x86, 0x4a, 0x70, 0x79, 0x45, 0x69, 0x76, 0x68,
0xb7, 0x4b, 0x83, 0x8d, 0x2e, 0x61, 0x0c, 0x77, 0x62, 0xad, 0xe6, 0xbf, 0x67, 0xa1, 0xdc, 0x94,
0xfa, 0x0e, 0x38, 0xe6, 0x04, 0x1d, 0x01, 0x3a, 0xc5, 0xbe, 0xe7, 0x62, 0x4e, 0x23, 0x3b, 0x22,
0x1d, 0x8f, 0xf1, 0xe8, 0xdc, 0x28, 0x5c, 0x2d, 0xde, 0x2c, 0x37, 0x6e, 0xd4, 0x87, 0x9b, 0x50,
0x3f, 0x8a, 0x25, 0x2c, 0xe2, 0xd0, 0xc8, 0xb5, 0x6a, 0xa7, 0x29, 0x41, 0x69, 0x40, 0x0f, 0xe0,
0xfa, 0xa0, 0x5e, 0xdb, 0xc7, 0x8c, 0xdb, 0xce, 0x31, 0x0e, 0x3a, 0xc4, 0x66, 0x3e, 0xe5, 0xc6,
0xd8, 0xd5, 0xc2, 0xcd, 0x92, 0xb5, 0x36, 0x20, 0xff, 0x00, 0x33, 0xbe, 0x2d, 0x71, 0x07, 0x3e,
0xe5, 0x68, 0x0b, 0x56, 0x87, 0x68, 0x23, 0x67, 0x1e, 0xb7, 0x1d, 0xda, 0x0b, 0xb8, 0x51, 0x94,
0x7a, 0x96, 0x07, 0xf4, 0xdc, 0x3d, 0xf3, 0xf8, 0xb6, 0x40, 0xa0, 0xc7, 0xb0, 0x3e, 0x44, 0x85,
0x4b, 0x7c, 0x8e, 0x85, 0x45, 0x5e, 0x60, 0x73, 0x2f, 0xb4, 0x8f, 0x31, 0x3b, 0xbe, 0xd3, 0x30,
0x4a, 0x57, 0x0b, 0x37, 0xa7, 0xad, 0x4f, 0x0f, 0xa8, 0xdb, 0x11, 0xf0, 0x6d, 0x81, 0x3e, 0xf4,
0xc2, 0xfb, 0x12, 0x8b, 0x3e, 0x0b, 0xb5, 0x08, 0x07, 0x2e, 0xa6, 0x76, 0xd7, 0x3b, 0x8b, 0x15,
0x7c, 0x3c, 0x29, 0x35, 0x54, 0x14, 0xe7, 0xeb, 0xde, 0x99, 0x06, 0xaf, 0x43, 0x35, 0x20, 0x67,
0xdc, 0x66, 0x84, 0xb8, 0x31, 0xf6, 0x13, 0x85, 0x9d, 0x15, 0x8c, 0x03, 0x42, 0x5c, 0x0d, 0x7d,
0x1f, 0x56, 0xd9, 0x31, 0x8e, 0x5c, 0x1b, 0x07, 0xae, 0x2d, 0x1e, 0xd3, 0xe3, 0x9c, 0x10, 0x66,
0x63, 0x2e, 0x5d, 0xc7, 0x8c, 0xbf, 0x4e, 0xca, 0x57, 0xaa, 0x8f, 0x7a, 0xa5, 0x03, 0x21, 0xbd,
0x15, 0xb8, 0xdb, 0xb1, 0xec, 0x56, 0x14, 0xe1, 0x73, 0xeb, 0x32, 0xeb, 0xa7, 0xb3, 0x2d, 0x2e,
0xbc, 0xcc, 0xd0, 0xbb, 0xb0, 0x10, 0x92, 0x88, 0x79, 0x8c, 0x93, 0x80, 0x67, 0xee, 0x34, 0xfe,
0xa6, 0xae, 0x5a, 0x49, 0xaf, 0x52, 0xd1, 0x55, 0x7f, 0xe4, 0x05, 0xfc, 0x4e, 0xe3, 0x81, 0xc7,
0xb8, 0x35, 0x9f, 0x8a, 0xa6, 0x9a, 0xd1, 0x07, 0x60, 0x0e, 0x53, 0x69, 0x47, 0x04, 0x33, 0xe6,
0x75, 0x82, 0x2e, 0x09, 0x38, 0x33, 0xfe, 0xae, 0xf4, 0x6f, 0x5c, 0xf8, 0x53, 0xac, 0x8c, 0x88,
0x0e, 0xbc, 0xab, 0x43, 0xee, 0xcc, 0xc2, 0x18, 0x7a, 0x03, 0x96, 0xc2, 0x88, 0x9c, 0x7a, 0xb4,
0xc7, 0xec, 0x27, 0x3d, 0xc6, 0xbd, 0xb6, 0x47, 0x5c, 0x15, 0x7b, 0xbf, 0xab, 0xc8, 0xa0, 0x59,
0x88, 0xf9, 0x5f, 0x8d, 0xd9, 0x32, 0xe4, 0x5e, 0x85, 0xd9, 0x3e, 0xfc, 0xef, 0x15, 0x7e, 0xe6,
0x49, 0x0e, 0xf7, 0x06, 0x2c, 0xe5, 0x71, 0x76, 0xcb, 0xe3, 0x6d, 0x8f, 0xf8, 0xae, 0xf1, 0x07,
0x7d, 0x41, 0x4e, 0xa0, 0xa9, 0xb9, 0xe2, 0x82, 0xb6, 0x17, 0x60, 0xdf, 0xfb, 0x56, 0x7c, 0xc1,
0x1f, 0xf5, 0x05, 0x09, 0x59, 0x5e, 0xf0, 0x08, 0x6a, 0x3e, 0xe6, 0x44, 0x24, 0x4e, 0x44, 0x19,
0xf3, 0xbd, 0xe0, 0x84, 0x19, 0xbf, 0x58, 0xba, 0x38, 0x43, 0xb7, 0x63, 0xa8, 0x76, 0x54, 0x55,
0xa9, 0x48, 0xc8, 0x0c, 0x35, 0x61, 0x55, 0x66, 0x23, 0x13, 0x65, 0xc0, 0x8e, 0x88, 0x83, 0x7d,
0xa7, 0xe7, 0x63, 0xee, 0xd1, 0x40, 0x59, 0xf3, 0xcb, 0x25, 0x95, 0x53, 0x02, 0x25, 0x6b, 0x85,
0x95, 0xc5, 0x48, 0xd3, 0xbe, 0x0c, 0x97, 0xb5, 0x69, 0x2d, 0x9f, 0x3a, 0x27, 0x76, 0x44, 0x29,
0xd7, 0x51, 0xcd, 0x8c, 0x5f, 0x09, 0x13, 0xa7, 0xad, 0x45, 0x85, 0x68, 0x0a, 0x80, 0x45, 0x29,
0x57, 0xd1, 0xcd, 0xd0, 0x0e, 0xbc, 0xa2, 0x65, 0x43, 0x12, 0x7b, 0x41, 0x26, 0x74, 0x0b, 0xfb,
0x38, 0x70, 0x08, 0x33, 0x7e, 0x2d, 0x14, 0x94, 0xac, 0x15, 0x05, 0xdb, 0x8f, 0x51, 0x22, 0xa5,
0x9b, 0x1a, 0x83, 0x5a, 0x30, 0xa7, 0xb5, 0x60, 0x2e, 0xfe, 0x48, 0xdb, 0x98, 0xf1, 0x1b, 0xe5,
0x9e, 0xdb, 0xa3, 0xdc, 0xb3, 0x4f, 0x02, 0xd7, 0x0b, 0x3a, 0x5b, 0xa9, 0x8c, 0xf6, 0x13, 0x52,
0xda, 0x32, 0x0c, 0x86, 0x76, 0xe1, 0x5a, 0x18, 0x51, 0x87, 0x30, 0x46, 0x5c, 0x3b, 0xa4, 0x4f,
0x85, 0xb3, 0x88, 0x17, 0xf2, 0xec, 0xcf, 0x35, 0x7e, 0xba, 0x26, 0x7f, 0xed, 0x6a, 0x82, 0xdc,
0xa7, 0x4f, 0x2d, 0x85, 0x4b, 0x7f, 0x35, 0xea, 0xc1, 0x8a, 0x83, 0x03, 0x57, 0x14, 0x15, 0x32,
0xa0, 0x8a, 0x19, 0x1f, 0xad, 0x49, 0xb3, 0x5f, 0x1f, 0xf9, 0xaa, 0xb1, 0xec, 0x3e, 0x7d, 0x9c,
0x51, 0xae, 0x6d, 0x37, 0x9c, 0x94, 0x9d, 0xbd, 0x9b, 0x21, 0x13, 0xa6, 0x3b, 0x24, 0x20, 0xcc,
0x63, 0xb6, 0x68, 0x16, 0xc6, 0x77, 0x6f, 0xc8, 0xa7, 0x2d, 0x6b, 0xe2, 0xa1, 0xd7, 0x25, 0xe8,
0x2b, 0x70, 0xa9, 0x4d, 0xa3, 0x13, 0xdb, 0xc5, 0x1c, 0x1b, 0xdf, 0x13, 0x80, 0x72, 0xe3, 0xea,
0x28, 0x43, 0xee, 0xd1, 0xe8, 0x64, 0x07, 0x73, 0x6c, 0x4d, 0xb5, 0xf5, 0x7f, 0x68, 0x0e, 0x4a,
0x32, 0x6c, 0xbe, 0xaf, 0x74, 0xcb, 0x03, 0xaa, 0x43, 0x35, 0x93, 0x1c, 0x3c, 0x22, 0xf8, 0xc4,
0xf8, 0xd1, 0x9e, 0x00, 0x34, 0xc7, 0x8c, 0x82, 0x55, 0x49, 0x33, 0x43, 0xf2, 0x10, 0x81, 0xf9,
0x50, 0x3d, 0x4d, 0xfe, 0x3d, 0x3f, 0xda, 0x93, 0x8e, 0xf9, 0xdc, 0x28, 0x7b, 0xb6, 0x3a, 0x9d,
0x88, 0x74, 0x30, 0x27, 0x6e, 0xe6, 0xe5, 0xe4, 0x15, 0x73, 0xe1, 0xc0, 0x53, 0x33, 0xf4, 0x14,
0x5e, 0x19, 0x5a, 0x5a, 0xdb, 0x34, 0xd2, 0xb5, 0xf5, 0x67, 0x7b, 0x2f, 0x53, 0x5b, 0xe5, 0x8d,
0xcb, 0x83, 0xf5, 0xf5, 0x1e, 0x8d, 0x64, 0x81, 0x35, 0xbf, 0x03, 0x53, 0xb1, 0xeb, 0xd0, 0x4d,
0xa8, 0x86, 0x11, 0xb1, 0xa5, 0xd3, 0x4f, 0x45, 0x19, 0xa3, 0x81, 0x51, 0x90, 0xbe, 0x9b, 0x0d,
0x23, 0x22, 0x60, 0x47, 0x8a, 0x8a, 0x6e, 0x41, 0x2d, 0xa4, 0x8c, 0xe7, 0xa1, 0xaa, 0x73, 0x56,
0x04, 0x23, 0x8b, 0x5d, 0xd1, 0xcf, 0x28, 0xdf, 0x42, 0x75, 0x45, 0xf9, 0x46, 0xe2, 0x7e, 0xf3,
0x43, 0x58, 0xbd, 0x30, 0x84, 0xd0, 0x7d, 0xb8, 0x36, 0x3a, 0x3e, 0xe3, 0x50, 0x2f, 0xc8, 0x76,
0xb5, 0x3a, 0x22, 0xda, 0x74, 0xa4, 0xcf, 0xc3, 0xf8, 0x29, 0xe5, 0x84, 0x69, 0x3b, 0xd5, 0xc1,
0xfc, 0x4b, 0x01, 0x8c, 0x51, 0xb9, 0x87, 0xde, 0x84, 0x92, 0x0c, 0xbe, 0x82, 0x8c, 0xbd, 0x91,
0xa5, 0x2d, 0x23, 0x28, 0x43, 0x50, 0x0a, 0xa1, 0xd7, 0x61, 0x31, 0xc4, 0x11, 0xf7, 0x1c, 0x2f,
0x54, 0x35, 0x2c, 0xa9, 0xc2, 0x63, 0xd2, 0xdc, 0x85, 0x1c, 0x37, 0x29, 0xc2, 0xeb, 0x50, 0x75,
0x7a, 0x8c, 0x53, 0xf7, 0x3c, 0x15, 0x28, 0xaa, 0xd6, 0xad, 0xe9, 0x09, 0xf4, 0x3a, 0xcc, 0xc8,
0xf2, 0xee, 0x05, 0x8e, 0xdf, 0x73, 0x89, 0x2b, 0x87, 0x84, 0x92, 0x35, 0x2d, 0x88, 0xbb, 0x9a,
0x66, 0xfe, 0xb9, 0x00, 0xe5, 0x8c, 0x81, 0xff, 0xef, 0xbf, 0x69, 0x03, 0xe6, 0x70, 0x9c, 0x3a,
0xb6, 0x68, 0x9a, 0x98, 0xf7, 0x22, 0xa2, 0xc7, 0x1f, 0x94, 0xb0, 0x0e, 0x62, 0x8e, 0xf9, 0x83,
0x22, 0x54, 0xfa, 0x8c, 0x45, 0x48, 0x67, 0x7e, 0x21, 0x93, 0xf8, 0xf3, 0x30, 0x2e, 0xd3, 0x20,
0x7e, 0x7e, 0x79, 0x40, 0x6f, 0x80, 0xa1, 0x7e, 0xf7, 0x60, 0xbf, 0xd0, 0x16, 0x2e, 0x28, 0x7e,
0x5f, 0xb7, 0x40, 0x6f, 0xc2, 0x32, 0x09, 0xa9, 0x73, 0x6c, 0xb7, 0x68, 0x2f, 0x70, 0x71, 0x74,
0x9e, 0x13, 0x55, 0xe6, 0x2e, 0x49, 0x44, 0x53, 0x03, 0x32, 0xc2, 0xaf, 0xc3, 0x92, 0xca, 0xf6,
0xc1, 0x4b, 0xc7, 0xa5, 0xe4, 0xbc, 0x64, 0xf7, 0xdf, 0xf9, 0x36, 0x5c, 0xe9, 0xef, 0xbb, 0x39,
0xd9, 0x09, 0x29, 0x7b, 0xb9, 0xaf, 0xb1, 0x66, 0x14, 0x7c, 0x66, 0x60, 0x82, 0x98, 0x1c, 0x36,
0x40, 0xbc, 0x05, 0x2b, 0x29, 0x6c, 0xd0, 0xc4, 0x29, 0x79, 0x8d, 0x91, 0x40, 0xfa, 0xcc, 0x34,
0xff, 0x53, 0x84, 0x4a, 0xdf, 0x3c, 0x8e, 0x16, 0x61, 0x22, 0xec, 0xb5, 0x4e, 0xc8, 0xb9, 0xce,
0x55, 0x7d, 0x12, 0x01, 0xf5, 0xd4, 0xe3, 0xc7, 0x6e, 0x84, 0x9f, 0x62, 0xdf, 0x76, 0x22, 0xe2,
0x92, 0x80, 0x7b, 0xd8, 0x67, 0x71, 0x40, 0xa5, 0xdc, 0xed, 0x94, 0x89, 0xbe, 0x08, 0x86, 0x9e,
0x70, 0x55, 0xad, 0x14, 0x93, 0x55, 0xfe, 0xd9, 0x16, 0x15, 0x7f, 0x3b, 0x61, 0x6b, 0x17, 0x5c,
0x83, 0x69, 0x2d, 0xc9, 0x4e, 0xbc, 0x90, 0xe9, 0x94, 0x29, 0x2b, 0xda, 0x81, 0x20, 0x21, 0x03,
0x26, 0x75, 0xc7, 0x97, 0xaf, 0x51, 0xb2, 0xe2, 0x23, 0xfa, 0x1a, 0x4c, 0x88, 0x30, 0xeb, 0x31,
0xe9, 0xea, 0xd9, 0xc6, 0x9d, 0xe7, 0x5c, 0x47, 0xea, 0x07, 0x52, 0x6a, 0x9b, 0xba, 0x84, 0x59,
0x5a, 0x05, 0xfa, 0x52, 0x32, 0xaa, 0x28, 0x42, 0x6e, 0x0b, 0x51, 0xef, 0xa2, 0x27, 0x15, 0x2d,
0x9d, 0x2e, 0x1f, 0xab, 0x00, 0x99, 0x4d, 0x63, 0x4a, 0x62, 0x2f, 0x91, 0x78, 0xb1, 0x30, 0x3f,
0x84, 0x72, 0xe6, 0x42, 0xb4, 0x08, 0x68, 0xff, 0xee, 0xde, 0xce, 0xee, 0xde, 0x3b, 0xf6, 0xd6,
0xf6, 0xe1, 0xee, 0xd1, 0xd6, 0xe1, 0xee, 0xc3, 0xbd, 0xea, 0xa7, 0x10, 0xc0, 0x84, 0x3c, 0xdf,
0xad, 0x16, 0xd0, 0x12, 0xcc, 0xa9, 0xff, 0xed, 0x18, 0x7a, 0xf7, 0xbd, 0xdd, 0xc3, 0xea, 0x18,
0x5a, 0x86, 0x45, 0xf1, 0xdf, 0xdd, 0x1d, 0xfb, 0xf1, 0xee, 0xe1, 0xfd, 0x87, 0x8f, 0x0e, 0x05,
0x60, 0xeb, 0xc1, 0xe1, 0x37, 0xaa, 0x45, 0x21, 0x94, 0xe1, 0x25, 0x8c, 0x92, 0xe9, 0xc3, 0xd2,
0x88, 0xf9, 0x18, 0xdd, 0x80, 0x4a, 0xba, 0xf4, 0x78, 0x81, 0x4b, 0xce, 0x64, 0x44, 0xcc, 0x58,
0xb3, 0x09, 0x79, 0x57, 0x50, 0x47, 0xe4, 0x6b, 0x9c, 0xd9, 0xc5, 0x34, 0xb3, 0xcd, 0x4f, 0xc6,
0x60, 0x61, 0x68, 0xbb, 0x45, 0x8b, 0xd9, 0x3a, 0x20, 0x5b, 0xa1, 0xaa, 0x05, 0x46, 0x4e, 0xb7,
0x64, 0x68, 0xfd, 0xeb, 0x03, 0x19, 0x52, 0x4c, 0x20, 0x7d, 0x59, 0xf2, 0x79, 0x98, 0xef, 0xcf,
0x12, 0x11, 0x81, 0x2a, 0xf7, 0xa5, 0x00, 0xca, 0xa7, 0x88, 0x88, 0x40, 0xf4, 0x1a, 0x54, 0xb3,
0xa9, 0x2f, 0x25, 0xc6, 0x13, 0x89, 0xd9, 0x34, 0xef, 0x25, 0x7a, 0x03, 0x6a, 0x6a, 0xea, 0x20,
0x51, 0x5a, 0x39, 0x27, 0x12, 0x78, 0x35, 0x66, 0x26, 0xe5, 0xf3, 0x0b, 0xb0, 0x40, 0x5b, 0xbe,
0xf7, 0x7e, 0x8f, 0xd8, 0x21, 0x8e, 0xe2, 0xac, 0x20, 0xcc, 0x10, 0xeb, 0x8c, 0x12, 0x9a, 0xd3,
0x80, 0x7d, 0xc9, 0xbf, 0x2f, 0xd9, 0xe8, 0x06, 0xcc, 0xe4, 0xca, 0xae, 0x31, 0x25, 0x26, 0x5d,
0x89, 0x9f, 0xce, 0x16, 0x5d, 0xf3, 0x6d, 0x98, 0x39, 0x08, 0x89, 0xe3, 0x61, 0x3f, 0xc9, 0xec,
0xd2, 0x89, 0x17, 0xb8, 0xea, 0x15, 0x95, 0x8f, 0xc5, 0x59, 0xd0, 0x65, 0x9f, 0x19, 0x4b, 0x2e,
0x96, 0x67, 0xf3, 0x9b, 0x50, 0xe9, 0x5b, 0x05, 0x86, 0x96, 0xeb, 0x0b, 0x4a, 0xe4, 0xd8, 0xe8,
0x12, 0x69, 0x7e, 0x00, 0x8b, 0xc3, 0x07, 0x21, 0xe4, 0xc2, 0x65, 0x2c, 0xfe, 0xb1, 0x87, 0xcc,
0x59, 0xfa, 0xeb, 0xc2, 0xfa, 0x73, 0xcf, 0x56, 0xd6, 0xa2, 0xd4, 0x35, 0x40, 0x37, 0xbf, 0x0d,
0xb5, 0x01, 0x62, 0x1a, 0xca, 0x85, 0x6c, 0x28, 0x5f, 0x81, 0x4b, 0xa9, 0x01, 0xc2, 0x4b, 0x33,
0x56, 0x4a, 0x40, 0x0d, 0x58, 0xe0, 0x94, 0x63, 0xdf, 0x4e, 0xb3, 0x25, 0xfb, 0x5d, 0x61, 0x4e,
0x32, 0x93, 0x32, 0xa3, 0xf2, 0xfe, 0xe3, 0x62, 0xfc, 0x25, 0x45, 0xba, 0x65, 0xa8, 0x5f, 0x5f,
0x03, 0xa4, 0x03, 0x63, 0xd0, 0xa5, 0x55, 0xc5, 0xc9, 0x34, 0x8c, 0x5b, 0x50, 0xd3, 0xdb, 0xd8,
0x40, 0x5f, 0xac, 0x48, 0x46, 0x06, 0x7b, 0x1b, 0xe6, 0x75, 0x65, 0x8d, 0xc8, 0x29, 0xc1, 0x7e,
0xbe, 0x17, 0x22, 0xc5, 0xb3, 0x24, 0x4b, 0x4b, 0x3c, 0xd7, 0x6c, 0x37, 0xfe, 0x3c, 0xb3, 0xdd,
0x15, 0xb8, 0x94, 0xce, 0x0a, 0x13, 0x72, 0xef, 0x49, 0x09, 0x62, 0xe4, 0x69, 0x51, 0xf7, 0x5c,
0x16, 0xd5, 0x0b, 0x46, 0x9e, 0x8c, 0xeb, 0x9a, 0xd4, 0x3d, 0xb7, 0xa4, 0x10, 0x7a, 0x0b, 0x2e,
0x25, 0x9f, 0xb3, 0x8c, 0x7f, 0x28, 0x15, 0xcb, 0x75, 0xf5, 0xc5, 0xab, 0x1e, 0x7f, 0xf1, 0xaa,
0x1f, 0xc6, 0x10, 0x19, 0xe9, 0xa9, 0x04, 0x3a, 0x82, 0xe9, 0xdc, 0xde, 0xf0, 0xcf, 0xc9, 0x97,
0xdd, 0x1b, 0x72, 0x7a, 0xcc, 0x7f, 0x8d, 0x41, 0xa5, 0xcf, 0x60, 0xf4, 0x4e, 0xdf, 0x5d, 0x2a,
0xaa, 0xaf, 0x3f, 0xc7, 0x88, 0x97, 0x57, 0x8e, 0x1e, 0x03, 0x0a, 0x23, 0x1a, 0x52, 0x46, 0xc4,
0xf6, 0x81, 0xd9, 0xb1, 0x17, 0x74, 0x98, 0x8c, 0xd1, 0x72, 0xe3, 0xe6, 0xc8, 0x0d, 0x56, 0x4b,
0x1c, 0x68, 0x01, 0xab, 0x16, 0xf6, 0x51, 0x18, 0x7a, 0x17, 0xaa, 0x0e, 0x66, 0x61, 0x4e, 0x6d,
0x51, 0xaa, 0x7d, 0x75, 0xf4, 0x86, 0x29, 0xf0, 0x89, 0xd2, 0x8a, 0x93, 0x3b, 0x33, 0xf4, 0x26,
0x4c, 0xb9, 0x24, 0xa4, 0xcc, 0xe3, 0xa2, 0x99, 0x0b, 0x55, 0x6b, 0xa3, 0x54, 0xed, 0x28, 0x9c,
0x95, 0x08, 0xa0, 0x06, 0x8c, 0x8b, 0xb6, 0xc9, 0x8c, 0x71, 0x29, 0x79, 0x65, 0x94, 0xa4, 0xd8,
0xf0, 0x2d, 0x05, 0x35, 0xff, 0x54, 0x80, 0x9a, 0xd6, 0xb4, 0x8f, 0x23, 0xdc, 0x25, 0x9c, 0x44,
0x6c, 0xe4, 0x80, 0x53, 0x87, 0xb9, 0x30, 0xa2, 0xb4, 0x6d, 0xd3, 0xb6, 0x1d, 0x52, 0xc6, 0x08,
0x4b, 0x76, 0xa5, 0x69, 0xe9, 0x21, 0xda, 0x7e, 0xd8, 0xde, 0x4f, 0x18, 0xa8, 0x09, 0xab, 0xc3,
0x07, 0xa2, 0x7c, 0xf6, 0xad, 0x0c, 0x9d, 0x8b, 0x74, 0x36, 0x5c, 0x34, 0x1d, 0x95, 0x2e, 0x9a,
0x8e, 0xcc, 0x27, 0x80, 0xd4, 0x33, 0x62, 0x5f, 0x4c, 0xd8, 0xc4, 0x7d, 0xc1, 0x71, 0xfa, 0x16,
0xd4, 0x46, 0xcd, 0xd1, 0x95, 0x56, 0x5f, 0xa9, 0xfe, 0xe1, 0x18, 0xd4, 0xe4, 0x33, 0xe2, 0x96,
0x4f, 0x8e, 0x28, 0x27, 0xf2, 0xae, 0xfb, 0x70, 0x6d, 0xc8, 0xfc, 0x6f, 0x87, 0xd4, 0xb1, 0x6f,
0x8b, 0x79, 0xc1, 0x73, 0x88, 0x0a, 0xec, 0x19, 0x6b, 0x75, 0x70, 0x1b, 0xd8, 0xa7, 0xce, 0xed,
0x5d, 0x05, 0xba, 0x48, 0xd3, 0x66, 0xa2, 0x69, 0xec, 0x02, 0x4d, 0x9b, 0xb1, 0xa6, 0x78, 0x65,
0x2a, 0xbe, 0xcc, 0xca, 0xf4, 0xc2, 0x0b, 0xcd, 0x4f, 0x0a, 0x50, 0xd6, 0xf1, 0x25, 0x3d, 0xf2,
0x1e, 0x20, 0x1d, 0xaf, 0xa2, 0xa5, 0xeb, 0x78, 0xd3, 0xeb, 0xdb, 0xfa, 0x33, 0x42, 0x3d, 0x0d,
0x50, 0xab, 0xe6, 0x0e, 0xc4, 0xac, 0xd8, 0x88, 0xb1, 0xdf, 0x23, 0xc9, 0x46, 0x2c, 0x0e, 0xa2,
0x96, 0xa6, 0x05, 0x4f, 0x75, 0x9b, 0x94, 0x60, 0xfe, 0x76, 0x0c, 0xaa, 0xfd, 0x99, 0x2e, 0xf6,
0x8a, 0xa4, 0x5e, 0x64, 0x67, 0xba, 0x99, 0x98, 0xaa, 0x46, 0x3a, 0x0b, 0x2a, 0xa1, 0x8e, 0x2e,
0xf9, 0x51, 0xc7, 0xde, 0x94, 0x37, 0x97, 0x1b, 0xb7, 0x2e, 0xae, 0x29, 0xd9, 0x60, 0x8c, 0x75,
0x62, 0x5f, 0x9c, 0x36, 0x45, 0xd7, 0x49, 0x74, 0xa6, 0x8f, 0xbc, 0xa9, 0x83, 0x0e, 0x85, 0x19,
0x05, 0x92, 0xb5, 0x39, 0x68, 0x85, 0x4a, 0x8a, 0xff, 0xc1, 0x8a, 0xc6, 0x08, 0x2b, 0xe2, 0xe6,
0x35, 0x68, 0x45, 0xc3, 0xfc, 0x71, 0x01, 0x66, 0xf3, 0xa5, 0x0d, 0x35, 0x61, 0x52, 0x7e, 0x93,
0xb0, 0x37, 0x9f, 0xf5, 0xba, 0x03, 0x69, 0x63, 0x4d, 0x48, 0xc9, 0xcd, 0x54, 0x47, 0x43, 0xbb,
0xf6, 0x85, 0x75, 0x34, 0xcc, 0x9f, 0x17, 0x60, 0x52, 0xc7, 0x8f, 0x18, 0x43, 0xba, 0x24, 0x3a,
0xf1, 0x89, 0xdd, 0x8a, 0x70, 0xe0, 0x1c, 0x27, 0xdf, 0x52, 0x0b, 0xb2, 0xc9, 0xce, 0x29, 0x66,
0x53, 0xf2, 0xe2, 0xef, 0xa8, 0xb7, 0xa0, 0xa6, 0x65, 0x78, 0x44, 0x88, 0x0e, 0x08, 0xfd, 0x71,
0x48, 0x31, 0x0e, 0x23, 0x42, 0x54, 0x48, 0xdc, 0x83, 0xe9, 0x38, 0xb8, 0x33, 0x29, 0x76, 0xfd,
0x19, 0x61, 0x2d, 0xcd, 0x2d, 0xbb, 0xe9, 0xc1, 0xc4, 0x50, 0x12, 0x35, 0x7a, 0x68, 0xa9, 0x1a,
0xb2, 0x72, 0x28, 0x6b, 0xfa, 0x57, 0x8e, 0xdc, 0x14, 0xa1, 0x02, 0x28, 0x25, 0xb4, 0x26, 0x64,
0xb3, 0xbf, 0xf3, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf2, 0x66, 0xc2, 0xbd, 0x27, 0x1b, 0x00,
0x00,
0x15, 0xa6, 0x5b, 0xad, 0xc5, 0x4f, 0x4b, 0x77, 0xa7, 0xb6, 0xb2, 0x64, 0x8d, 0x3c, 0x65, 0x18,
0x5b, 0x66, 0x68, 0x59, 0x6d, 0x8c, 0x01, 0xc7, 0x30, 0xa1, 0x96, 0xec, 0xb1, 0xc0, 0xc8, 0x9a,
0x92, 0x2c, 0x0f, 0x11, 0x44, 0x54, 0x64, 0x57, 0x65, 0xb7, 0xca, 0xaa, 0xae, 0xac, 0xa9, 0xcc,
0x96, 0x25, 0x08, 0x66, 0x82, 0x23, 0x70, 0xe0, 0x44, 0x40, 0x04, 0x27, 0x86, 0x5f, 0xc0, 0x91,
0xed, 0xc8, 0x0f, 0x60, 0xb9, 0x70, 0x80, 0xfb, 0x0c, 0xdb, 0x91, 0x03, 0x17, 0x22, 0x97, 0xda,
0x7a, 0x91, 0x65, 0x73, 0xe1, 0x24, 0xe5, 0x7b, 0xdf, 0x7b, 0xf9, 0xfa, 0xe5, 0x5b, 0x0b, 0x56,
0xc3, 0x88, 0x72, 0xba, 0xde, 0x24, 0xd8, 0xa1, 0xc1, 0x7a, 0x58, 0x0f, 0xd7, 0x4f, 0x36, 0xd6,
0xf9, 0x59, 0x48, 0x58, 0x4d, 0x72, 0xd0, 0x02, 0xe1, 0x47, 0x24, 0x22, 0xdd, 0x4e, 0x4d, 0x61,
0x6a, 0x61, 0x3d, 0xac, 0x9d, 0x6c, 0x2c, 0xad, 0xb6, 0x29, 0x6d, 0xfb, 0x64, 0x5d, 0xa2, 0x9a,
0xdd, 0xd6, 0x3a, 0xf7, 0x3a, 0x84, 0x71, 0xdc, 0x09, 0x95, 0xe0, 0xd2, 0xb2, 0xd2, 0xec, 0xd0,
0x4e, 0x87, 0x06, 0xeb, 0x1d, 0xc2, 0x18, 0x6e, 0xc7, 0x5a, 0xcd, 0xff, 0xcc, 0xc0, 0x64, 0x43,
0xea, 0xdb, 0xe7, 0x98, 0x13, 0x74, 0x08, 0xe8, 0x04, 0xfb, 0x9e, 0x8b, 0x39, 0x8d, 0xec, 0x88,
0xb4, 0x3d, 0xc6, 0xa3, 0x33, 0xa3, 0x70, 0x75, 0xe4, 0xc6, 0x64, 0xfd, 0x7a, 0x6d, 0xb0, 0x09,
0xb5, 0xc3, 0x58, 0xc2, 0x22, 0x0e, 0x8d, 0x5c, 0xab, 0x7a, 0x92, 0x12, 0x94, 0x06, 0xf4, 0x08,
0xae, 0xf5, 0xeb, 0xb5, 0x7d, 0xcc, 0xb8, 0xed, 0x1c, 0xe1, 0xa0, 0x4d, 0x6c, 0xe6, 0x53, 0x6e,
0x14, 0xaf, 0x16, 0x6e, 0x94, 0xac, 0xd5, 0x3e, 0xf9, 0x47, 0x98, 0xf1, 0x2d, 0x89, 0xdb, 0xf7,
0x29, 0x47, 0x9b, 0xb0, 0x32, 0x40, 0x1b, 0x39, 0xf5, 0xb8, 0xed, 0xd0, 0x6e, 0xc0, 0x8d, 0x11,
0xa9, 0x67, 0xa9, 0x4f, 0xcf, 0xfd, 0x53, 0x8f, 0x6f, 0x09, 0x04, 0x7a, 0x0a, 0x6b, 0x03, 0x54,
0xb8, 0xc4, 0xe7, 0x58, 0x58, 0xe4, 0x05, 0x36, 0xf7, 0x42, 0xfb, 0x08, 0xb3, 0xa3, 0xdb, 0x75,
0xa3, 0x74, 0xb5, 0x70, 0x63, 0xca, 0xfa, 0x74, 0x9f, 0xba, 0x6d, 0x01, 0xdf, 0x12, 0xe8, 0x03,
0x2f, 0x7c, 0x28, 0xb1, 0xe8, 0xb3, 0x50, 0x8d, 0x70, 0xe0, 0x62, 0x6a, 0x77, 0xbc, 0xd3, 0x58,
0xc1, 0xc7, 0xe3, 0x52, 0x43, 0x59, 0x71, 0xbe, 0xee, 0x9d, 0x6a, 0xf0, 0x1a, 0x54, 0x02, 0x72,
0xca, 0x6d, 0x46, 0x88, 0x1b, 0x63, 0x3f, 0x51, 0xd8, 0x19, 0xc1, 0xd8, 0x27, 0xc4, 0xd5, 0xd0,
0xf7, 0x61, 0x85, 0x1d, 0xe1, 0xc8, 0xb5, 0x71, 0xe0, 0xda, 0xe2, 0x31, 0x3d, 0xce, 0x09, 0x61,
0x36, 0xe6, 0xd2, 0x75, 0xcc, 0xf8, 0xdb, 0xb8, 0x7c, 0xa5, 0xda, 0xb0, 0x57, 0xda, 0x17, 0xd2,
0x9b, 0x81, 0xbb, 0x15, 0xcb, 0x6e, 0x46, 0x11, 0x3e, 0xb3, 0x2e, 0xb3, 0x5e, 0x3a, 0xdb, 0xe4,
0xc2, 0xcb, 0x0c, 0xbd, 0x0b, 0xf3, 0x21, 0x89, 0x98, 0xc7, 0x38, 0x09, 0x78, 0xe6, 0x4e, 0xe3,
0xef, 0xea, 0xaa, 0xe5, 0xf4, 0x2a, 0x15, 0x5d, 0xb5, 0x27, 0x5e, 0xc0, 0x6f, 0xd7, 0x1f, 0x79,
0x8c, 0x5b, 0x73, 0xa9, 0x68, 0xaa, 0x19, 0x7d, 0x00, 0xe6, 0x20, 0x95, 0x76, 0x44, 0x30, 0x63,
0x5e, 0x3b, 0xe8, 0x90, 0x80, 0x33, 0xe3, 0x1f, 0x4a, 0xff, 0xfa, 0xb9, 0x3f, 0xc5, 0xca, 0x88,
0xe8, 0xc0, 0xbb, 0x3a, 0xe0, 0xce, 0x2c, 0x8c, 0xa1, 0xbb, 0xb0, 0x18, 0x46, 0xe4, 0xc4, 0xa3,
0x5d, 0x66, 0x3f, 0xeb, 0x32, 0xee, 0xb5, 0x3c, 0xe2, 0xaa, 0xd8, 0xfb, 0x7d, 0x59, 0x06, 0xcd,
0x7c, 0xcc, 0xff, 0x6a, 0xcc, 0x96, 0x21, 0xf7, 0x06, 0xcc, 0xf4, 0xe0, 0xff, 0xa0, 0xf0, 0xd3,
0xcf, 0x72, 0xb8, 0xbb, 0xb0, 0x98, 0xc7, 0xd9, 0x4d, 0x8f, 0xb7, 0x3c, 0xe2, 0xbb, 0xc6, 0x1f,
0xf5, 0x05, 0x39, 0x81, 0x86, 0xe6, 0x8a, 0x0b, 0x5a, 0x5e, 0x80, 0x7d, 0xef, 0x5b, 0xf1, 0x05,
0x7f, 0xd2, 0x17, 0x24, 0x64, 0x79, 0xc1, 0x13, 0xa8, 0xfa, 0x98, 0x13, 0x91, 0x38, 0x11, 0x65,
0xcc, 0xf7, 0x82, 0x63, 0x66, 0xfc, 0x72, 0xf1, 0xfc, 0x0c, 0xdd, 0x8a, 0xa1, 0xda, 0x51, 0x15,
0xa5, 0x22, 0x21, 0x33, 0xd4, 0x80, 0x15, 0x99, 0x8d, 0x4c, 0x94, 0x01, 0x3b, 0x22, 0x0e, 0xf6,
0x9d, 0xae, 0x8f, 0xb9, 0x47, 0x03, 0x65, 0xcd, 0xaf, 0x16, 0x55, 0x4e, 0x09, 0x94, 0xac, 0x15,
0x56, 0x16, 0x23, 0x4d, 0xfb, 0x32, 0x5c, 0xd6, 0xa6, 0x35, 0x7d, 0xea, 0x1c, 0xdb, 0x11, 0xa5,
0x5c, 0x47, 0x35, 0x33, 0x7e, 0x2d, 0x4c, 0x9c, 0xb2, 0x16, 0x14, 0xa2, 0x21, 0x00, 0x16, 0xa5,
0x5c, 0x45, 0x37, 0x43, 0xdb, 0xf0, 0x9a, 0x96, 0x0d, 0x49, 0xec, 0x05, 0x99, 0xd0, 0x4d, 0xec,
0xe3, 0xc0, 0x21, 0xcc, 0xf8, 0x8d, 0x50, 0x50, 0xb2, 0x96, 0x15, 0x6c, 0x2f, 0x46, 0x89, 0x94,
0x6e, 0x68, 0x0c, 0x6a, 0xc2, 0xac, 0xd6, 0x82, 0xb9, 0xf8, 0x23, 0x6d, 0x63, 0xc6, 0x6f, 0x95,
0x7b, 0x6e, 0x0d, 0x73, 0xcf, 0x1e, 0x09, 0x5c, 0x2f, 0x68, 0x6f, 0xa6, 0x32, 0xda, 0x4f, 0x48,
0x69, 0xcb, 0x30, 0x18, 0xda, 0x81, 0xd7, 0xc3, 0x88, 0x3a, 0x84, 0x31, 0xe2, 0xda, 0x21, 0x7d,
0x2e, 0x9c, 0x45, 0xbc, 0x90, 0x67, 0x7f, 0xae, 0xf1, 0xb3, 0x55, 0x99, 0xc4, 0x2b, 0x09, 0x72,
0x8f, 0x3e, 0xb7, 0x14, 0x2e, 0xfd, 0xd5, 0xa8, 0x0b, 0xcb, 0x0e, 0x0e, 0x5c, 0x51, 0x54, 0x48,
0x9f, 0x2a, 0x66, 0x7c, 0xb4, 0x2a, 0xcd, 0xbe, 0x33, 0xf4, 0x55, 0x63, 0xd9, 0x3d, 0xfa, 0x34,
0xa3, 0x5c, 0xdb, 0x6e, 0x38, 0x29, 0x3b, 0x7b, 0x37, 0x43, 0x26, 0x4c, 0xb5, 0x49, 0x40, 0x98,
0xc7, 0x6c, 0xd1, 0x2c, 0x8c, 0xef, 0x5d, 0x97, 0x4f, 0x3b, 0xa9, 0x89, 0x07, 0x5e, 0x87, 0xa0,
0xaf, 0xc0, 0xa5, 0x16, 0x8d, 0x8e, 0x6d, 0x17, 0x73, 0x6c, 0x7c, 0x5f, 0x00, 0x26, 0xeb, 0x57,
0x87, 0x19, 0xf2, 0x80, 0x46, 0xc7, 0xdb, 0x98, 0x63, 0x6b, 0xa2, 0xa5, 0xff, 0x43, 0xb3, 0x50,
0x92, 0x61, 0xf3, 0x03, 0xa5, 0x5b, 0x1e, 0x50, 0x0d, 0x2a, 0x99, 0xe4, 0xe0, 0x11, 0xc1, 0xc7,
0xc6, 0x8f, 0x77, 0x05, 0xa0, 0x51, 0x34, 0x0a, 0x56, 0x39, 0xcd, 0x0c, 0xc9, 0x43, 0x04, 0xe6,
0x42, 0xf5, 0x34, 0xf9, 0xf7, 0xfc, 0x68, 0x57, 0x3a, 0xe6, 0x73, 0xc3, 0xec, 0xd9, 0x6c, 0xb7,
0x23, 0xd2, 0xc6, 0x9c, 0xb8, 0x99, 0x97, 0x93, 0x57, 0xcc, 0x86, 0x7d, 0x4f, 0xcd, 0xd0, 0x73,
0x78, 0x6d, 0x60, 0x69, 0x6d, 0xd1, 0x48, 0xd7, 0xd6, 0x9f, 0xef, 0xbe, 0x4a, 0x6d, 0x95, 0x37,
0x2e, 0xf5, 0xd7, 0xd7, 0x07, 0x34, 0x92, 0x05, 0xd6, 0xfc, 0x0e, 0x4c, 0xc4, 0xae, 0x43, 0x37,
0xa0, 0x12, 0x46, 0xc4, 0x96, 0x4e, 0x3f, 0x11, 0x65, 0x8c, 0x06, 0x46, 0x41, 0xfa, 0x6e, 0x26,
0x8c, 0x88, 0x80, 0x1d, 0x2a, 0x2a, 0xba, 0x09, 0xd5, 0x90, 0x32, 0x9e, 0x87, 0xaa, 0xce, 0x59,
0x16, 0x8c, 0x2c, 0x76, 0x59, 0x3f, 0xa3, 0x7c, 0x0b, 0xd5, 0x15, 0xe5, 0x1b, 0x89, 0xfb, 0xcd,
0x0f, 0x61, 0xe5, 0xdc, 0x10, 0x42, 0x0f, 0xe1, 0xf5, 0xe1, 0xf1, 0x19, 0x87, 0x7a, 0x41, 0x45,
0xfa, 0x90, 0x68, 0xd3, 0x91, 0x3e, 0x07, 0xa3, 0x27, 0x94, 0x13, 0xa6, 0xed, 0x54, 0x07, 0xf3,
0xaf, 0x05, 0x30, 0x86, 0xe5, 0x1e, 0xba, 0x07, 0x25, 0x19, 0x7c, 0x05, 0x19, 0x7b, 0x43, 0x4b,
0x5b, 0x46, 0x50, 0x86, 0xa0, 0x14, 0x42, 0x77, 0x60, 0x21, 0xc4, 0x11, 0xf7, 0x1c, 0x2f, 0x54,
0x35, 0x2c, 0xa9, 0xc2, 0x45, 0x69, 0xee, 0x7c, 0x8e, 0x9b, 0x14, 0xe1, 0x35, 0xa8, 0x38, 0x5d,
0xc6, 0xa9, 0x7b, 0x96, 0x0a, 0x8c, 0xa8, 0xd6, 0xad, 0xe9, 0x09, 0xf4, 0x1a, 0x4c, 0xcb, 0xf2,
0xee, 0x05, 0x8e, 0xdf, 0x75, 0x89, 0x2b, 0x87, 0x84, 0x92, 0x35, 0x25, 0x88, 0x3b, 0x9a, 0x66,
0xfe, 0xa5, 0x00, 0x93, 0x19, 0x03, 0xff, 0xdf, 0x7f, 0xd3, 0x3a, 0xcc, 0xe2, 0x38, 0x75, 0x6c,
0xd1, 0x34, 0x31, 0xef, 0x46, 0x44, 0x8f, 0x3f, 0x28, 0x61, 0xed, 0xc7, 0x1c, 0xf3, 0x87, 0x23,
0x50, 0xee, 0x31, 0x16, 0x21, 0x9d, 0xf9, 0x85, 0x4c, 0xe2, 0xcf, 0xc1, 0xa8, 0x4c, 0x83, 0xf8,
0xf9, 0xe5, 0x01, 0xdd, 0x05, 0x43, 0xfd, 0xee, 0xfe, 0x7e, 0xa1, 0x2d, 0x9c, 0x57, 0xfc, 0x9e,
0x6e, 0x81, 0xee, 0xc1, 0x12, 0x09, 0xa9, 0x73, 0x64, 0x37, 0x69, 0x37, 0x70, 0x71, 0x74, 0x96,
0x13, 0x55, 0xe6, 0x2e, 0x4a, 0x44, 0x43, 0x03, 0x32, 0xc2, 0x77, 0x60, 0x51, 0x65, 0x7b, 0xff,
0xa5, 0xa3, 0x52, 0x72, 0x4e, 0xb2, 0x7b, 0xef, 0x7c, 0x1b, 0xae, 0xf4, 0xf6, 0xdd, 0x9c, 0xec,
0x98, 0x94, 0xbd, 0xdc, 0xd3, 0x58, 0x33, 0x0a, 0x3e, 0xd3, 0x37, 0x41, 0x8c, 0x0f, 0x1a, 0x20,
0xde, 0x82, 0xe5, 0x14, 0xd6, 0x6f, 0xe2, 0x84, 0xbc, 0xc6, 0x48, 0x20, 0x3d, 0x66, 0x9a, 0xdf,
0x2d, 0x41, 0xb9, 0x67, 0x1e, 0x47, 0x0b, 0x30, 0x16, 0x76, 0x9b, 0xc7, 0xe4, 0x4c, 0xe7, 0xaa,
0x3e, 0x89, 0x80, 0x7a, 0xee, 0xf1, 0x23, 0x37, 0xc2, 0xcf, 0xb1, 0x6f, 0x3b, 0x11, 0x71, 0x49,
0xc0, 0x3d, 0xec, 0xb3, 0x38, 0xa0, 0x52, 0xee, 0x56, 0xca, 0x44, 0x5f, 0x04, 0x43, 0x4f, 0xb8,
0xaa, 0x56, 0x8a, 0xc9, 0x2a, 0xff, 0x6c, 0x0b, 0x8a, 0xbf, 0x95, 0xb0, 0xb5, 0x0b, 0xae, 0xc1,
0xb4, 0x96, 0xf4, 0xf1, 0x19, 0x89, 0x58, 0x9c, 0x33, 0x8a, 0xf8, 0x48, 0xd2, 0x90, 0x01, 0xe3,
0xba, 0xe7, 0xcb, 0xf7, 0x28, 0x59, 0xf1, 0x11, 0x7d, 0x0d, 0xc6, 0x44, 0xa0, 0x75, 0x99, 0x74,
0xf6, 0x4c, 0xfd, 0xf6, 0x05, 0x17, 0x92, 0xda, 0xbe, 0x94, 0xda, 0xa2, 0x2e, 0x61, 0x96, 0x56,
0x81, 0xbe, 0x94, 0x0c, 0x2b, 0x8a, 0x90, 0xdb, 0x43, 0xd4, 0xcb, 0xe8, 0x59, 0x45, 0x4b, 0xa7,
0xeb, 0xc7, 0x0a, 0x40, 0x66, 0xd7, 0x98, 0x90, 0xd8, 0x4b, 0x24, 0x5e, 0x2d, 0xcc, 0x0f, 0x61,
0x32, 0x73, 0x21, 0x5a, 0x00, 0xb4, 0x77, 0x7f, 0x77, 0x7b, 0x67, 0xf7, 0x1d, 0x7b, 0x73, 0xeb,
0x60, 0xe7, 0x70, 0xf3, 0x60, 0xe7, 0xf1, 0x6e, 0xe5, 0x53, 0x08, 0x60, 0x4c, 0x9e, 0xef, 0x57,
0x0a, 0x68, 0x11, 0x66, 0xd5, 0xff, 0x76, 0x0c, 0xbd, 0xff, 0xde, 0xce, 0x41, 0xa5, 0x88, 0x96,
0x60, 0x41, 0xfc, 0x77, 0x7f, 0xdb, 0x7e, 0xba, 0x73, 0xf0, 0xf0, 0xf1, 0x93, 0x03, 0x01, 0xd8,
0x7c, 0x74, 0xf0, 0x8d, 0xca, 0x88, 0x10, 0xca, 0xf0, 0x12, 0x46, 0xc9, 0xf4, 0x61, 0x71, 0xc8,
0x84, 0x8c, 0xae, 0x43, 0x39, 0x5d, 0x7b, 0xbc, 0xc0, 0x25, 0xa7, 0x32, 0x26, 0xa6, 0xad, 0x99,
0x84, 0xbc, 0x23, 0xa8, 0x43, 0x32, 0x36, 0xce, 0xed, 0x91, 0x34, 0xb7, 0xcd, 0x4f, 0x8a, 0x30,
0x3f, 0xb0, 0xe1, 0xa2, 0x85, 0x6c, 0x25, 0x90, 0xcd, 0x50, 0x55, 0x03, 0x23, 0xa7, 0x5b, 0x32,
0xb4, 0xfe, 0xb5, 0xbe, 0x1c, 0x19, 0x49, 0x20, 0x3d, 0x79, 0xf2, 0x79, 0x98, 0xeb, 0xcd, 0x13,
0x11, 0x83, 0x2a, 0xfb, 0xa5, 0x00, 0xca, 0x27, 0x89, 0x88, 0x41, 0xf4, 0x26, 0x54, 0xb2, 0xc9,
0x2f, 0x25, 0x46, 0x13, 0x89, 0x99, 0x34, 0xf3, 0x25, 0x7a, 0x1d, 0xaa, 0x6a, 0xee, 0x20, 0x51,
0x5a, 0x3b, 0xc7, 0x12, 0x78, 0x25, 0x66, 0x26, 0x05, 0xf4, 0x0b, 0x30, 0x4f, 0x9b, 0xbe, 0xf7,
0x7e, 0x97, 0xd8, 0x21, 0x8e, 0xe2, 0xbc, 0x20, 0xcc, 0x10, 0x0b, 0x8d, 0x12, 0x9a, 0xd5, 0x80,
0x3d, 0xc9, 0x7f, 0x28, 0xd9, 0xe8, 0x3a, 0x4c, 0xe7, 0x0a, 0xaf, 0x31, 0x21, 0x66, 0x5d, 0x89,
0x9f, 0xca, 0x96, 0x5d, 0xf3, 0x6d, 0x98, 0xde, 0x0f, 0x89, 0xe3, 0x61, 0x3f, 0xc9, 0xed, 0xd2,
0xb1, 0x17, 0xb8, 0xea, 0x15, 0x95, 0x8f, 0xc5, 0x59, 0xd0, 0x65, 0xa7, 0x29, 0x26, 0x17, 0xcb,
0xb3, 0xf9, 0x4d, 0x28, 0xf7, 0x2c, 0x03, 0x03, 0x0b, 0xf6, 0x39, 0x45, 0xb2, 0x38, 0xbc, 0x48,
0x9a, 0x1f, 0xc0, 0xc2, 0xe0, 0x51, 0x08, 0xb9, 0x70, 0x19, 0x8b, 0x7f, 0xec, 0x01, 0x93, 0x96,
0xfe, 0xbe, 0xb0, 0x76, 0xe1, 0xe9, 0xca, 0x5a, 0x90, 0xba, 0xfa, 0xe8, 0xe6, 0xb7, 0xa1, 0xda,
0x47, 0x4c, 0x43, 0xb9, 0x90, 0x0d, 0xe5, 0x2b, 0x70, 0x29, 0x35, 0x40, 0x78, 0x69, 0xda, 0x4a,
0x09, 0xa8, 0x0e, 0xf3, 0x9c, 0x72, 0xec, 0xdb, 0x69, 0xb6, 0x64, 0xbf, 0x2c, 0xcc, 0x4a, 0x66,
0x52, 0x66, 0x54, 0xde, 0x7f, 0x3c, 0x12, 0x7f, 0x4b, 0x91, 0x6e, 0x19, 0xe8, 0xd7, 0x37, 0x01,
0xe9, 0xc0, 0xe8, 0x77, 0x69, 0x45, 0x71, 0x32, 0x2d, 0xe3, 0x26, 0x54, 0xf5, 0x3e, 0xd6, 0xd7,
0x19, 0xcb, 0x92, 0x91, 0xc1, 0xde, 0x82, 0x39, 0x5d, 0x5b, 0x23, 0x72, 0x42, 0xb0, 0x9f, 0xef,
0x86, 0x48, 0xf1, 0x2c, 0xc9, 0xd2, 0x12, 0x17, 0x9a, 0xee, 0x46, 0x2f, 0x32, 0xdd, 0x5d, 0x81,
0x4b, 0xe9, 0xb4, 0x30, 0x26, 0xf7, 0xbc, 0x94, 0x20, 0x86, 0x9e, 0x26, 0x75, 0xcf, 0x64, 0x51,
0x3d, 0x67, 0xe8, 0xc9, 0xb8, 0xae, 0x41, 0xdd, 0x33, 0x4b, 0x0a, 0xa1, 0xb7, 0xe0, 0x52, 0xf2,
0x41, 0xcb, 0xf8, 0xa7, 0x52, 0xb1, 0x54, 0x53, 0xdf, 0xbc, 0x6a, 0xf1, 0x37, 0xaf, 0xda, 0x41,
0x0c, 0x91, 0x91, 0x9e, 0x4a, 0xa0, 0x43, 0x98, 0xca, 0x6d, 0x0e, 0xff, 0x1a, 0x7f, 0xd5, 0xcd,
0x21, 0xa7, 0xc7, 0xfc, 0x77, 0x11, 0xca, 0x3d, 0x06, 0xa3, 0x77, 0x7a, 0xee, 0x52, 0x51, 0x7d,
0xed, 0x02, 0x43, 0x5e, 0x5e, 0x39, 0x7a, 0x0a, 0x28, 0x8c, 0x68, 0x48, 0x19, 0x11, 0xfb, 0x07,
0x66, 0x47, 0x5e, 0xd0, 0x66, 0x32, 0x46, 0x27, 0xeb, 0x37, 0x86, 0xee, 0xb0, 0x5a, 0x62, 0x5f,
0x0b, 0x58, 0xd5, 0xb0, 0x87, 0xc2, 0xd0, 0xbb, 0x50, 0x71, 0x30, 0x0b, 0x73, 0x6a, 0x47, 0xa4,
0xda, 0x37, 0x86, 0xef, 0x98, 0x02, 0x9f, 0x28, 0x2d, 0x3b, 0xb9, 0x33, 0x43, 0xf7, 0x60, 0xc2,
0x25, 0x21, 0x65, 0x1e, 0x17, 0xdd, 0x5c, 0xa8, 0x5a, 0x1d, 0xa6, 0x6a, 0x5b, 0xe1, 0xac, 0x44,
0x00, 0xd5, 0x61, 0x54, 0xb4, 0x4d, 0x66, 0x8c, 0x4a, 0xc9, 0x2b, 0xc3, 0x24, 0xc5, 0x8e, 0x6f,
0x29, 0xa8, 0xf9, 0xe7, 0x02, 0x54, 0xb5, 0xa6, 0x3d, 0x1c, 0xe1, 0x0e, 0xe1, 0x62, 0x68, 0x18,
0x36, 0xe2, 0xd4, 0x60, 0x36, 0x8c, 0x28, 0x6d, 0xd9, 0xb4, 0x65, 0x87, 0x94, 0x31, 0xc2, 0x92,
0x6d, 0x69, 0x4a, 0x7a, 0x88, 0xb6, 0x1e, 0xb7, 0xf6, 0x12, 0x06, 0x6a, 0xc0, 0xca, 0xe0, 0x91,
0x28, 0x9f, 0x7d, 0xcb, 0x03, 0x27, 0x23, 0x9d, 0x0d, 0xe7, 0xcd, 0x47, 0xa5, 0xf3, 0xe6, 0x23,
0xf3, 0x19, 0x20, 0xf5, 0x8c, 0xd8, 0x17, 0x33, 0x36, 0x71, 0x5f, 0x72, 0xa0, 0xbe, 0x09, 0xd5,
0x61, 0x93, 0x74, 0xb9, 0xd9, 0x53, 0xaa, 0x7f, 0x54, 0x84, 0xaa, 0x7c, 0x46, 0xdc, 0xf4, 0xc9,
0x21, 0xe5, 0x44, 0xde, 0xf5, 0x10, 0x5e, 0x1f, 0xb0, 0x01, 0xd8, 0x21, 0x75, 0xec, 0x5b, 0x62,
0x5e, 0xf0, 0x1c, 0xa2, 0x02, 0x7b, 0xda, 0x5a, 0xe9, 0xdf, 0x07, 0xf6, 0xa8, 0x73, 0x6b, 0x47,
0x81, 0xce, 0xd3, 0xb4, 0x91, 0x68, 0x2a, 0x9e, 0xa3, 0x69, 0x23, 0xd6, 0x14, 0x2f, 0x4d, 0x23,
0xaf, 0xb2, 0x34, 0xbd, 0xf4, 0x4a, 0xf3, 0xd3, 0x02, 0x4c, 0xea, 0xf8, 0x92, 0x1e, 0x79, 0x0f,
0x90, 0x8e, 0x57, 0xd1, 0xd2, 0x75, 0xbc, 0xe9, 0x05, 0x6e, 0xed, 0x05, 0xa1, 0x9e, 0x06, 0xa8,
0x55, 0x75, 0xfb, 0x62, 0x56, 0xec, 0xc4, 0xd8, 0xef, 0x92, 0x64, 0x27, 0x16, 0x07, 0x51, 0x4b,
0xd3, 0x82, 0xa7, 0xba, 0x4d, 0x4a, 0x30, 0x7f, 0x57, 0x84, 0x4a, 0x6f, 0xa6, 0x8b, 0xcd, 0x22,
0xa9, 0x17, 0xd9, 0x99, 0x6e, 0x3a, 0xa6, 0xaa, 0x91, 0xce, 0x82, 0x72, 0xa8, 0xa3, 0x4b, 0x7e,
0xd6, 0xb1, 0x37, 0xe4, 0xcd, 0x93, 0xf5, 0x9b, 0xe7, 0xd7, 0x94, 0x6c, 0x30, 0xc6, 0x3a, 0xb1,
0x2f, 0x4e, 0x1b, 0xa2, 0xeb, 0x24, 0x3a, 0xd3, 0x47, 0xde, 0xd0, 0x41, 0x87, 0xc2, 0x8c, 0x02,
0xc9, 0xda, 0xe8, 0xb7, 0x42, 0x25, 0xc5, 0xff, 0x60, 0x45, 0x7d, 0x88, 0x15, 0x71, 0xf3, 0xea,
0xb7, 0xa2, 0x6e, 0xfe, 0xa4, 0x00, 0x33, 0xf9, 0xd2, 0x86, 0x1a, 0x30, 0x2e, 0xbf, 0x4a, 0xd8,
0x1b, 0x2f, 0x7a, 0xdd, 0xbe, 0xb4, 0xb1, 0xc6, 0xa4, 0xe4, 0x46, 0xaa, 0xa3, 0xae, 0x5d, 0xfb,
0xd2, 0x3a, 0xea, 0xe6, 0x2f, 0x0a, 0x30, 0xae, 0xe3, 0x47, 0x8c, 0x21, 0x1d, 0x12, 0x1d, 0xfb,
0xc4, 0x6e, 0x46, 0x38, 0x70, 0x8e, 0x92, 0xaf, 0xa9, 0x05, 0xd9, 0x64, 0x67, 0x15, 0xb3, 0x21,
0x79, 0xf1, 0x97, 0xd4, 0x9b, 0x50, 0xd5, 0x32, 0x3c, 0x22, 0x44, 0x07, 0x84, 0xfe, 0x3c, 0xa4,
0x18, 0x07, 0x11, 0x21, 0x2a, 0x24, 0x1e, 0xc0, 0x54, 0x1c, 0xdc, 0x99, 0x14, 0xbb, 0xf6, 0x82,
0xb0, 0x96, 0xe6, 0x4e, 0xba, 0xe9, 0xc1, 0xc4, 0x50, 0x12, 0x35, 0x7a, 0x60, 0xa9, 0x1a, 0xb0,
0x72, 0x28, 0x6b, 0x7a, 0x57, 0x8e, 0xdc, 0x14, 0xa1, 0x02, 0x28, 0x25, 0x34, 0xc7, 0x64, 0xb3,
0xbf, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xfd, 0xfa, 0x52, 0x2b, 0x29, 0x1b, 0x00, 0x00,
}

View File

@@ -34,7 +34,7 @@ message BeaconState {
repeated PendingAttestationRecord latest_attestations = 3005;
// PoW receipt root [4001-5000]
repeated bytes processed_pow_receipt_root_hash32 = 4001;
bytes processed_pow_receipt_root_hash32 = 4001;
repeated CandidatePoWReceiptRootRecord candidate_pow_receipt_roots = 4002;
// Miscellaneous [5001-6000]
@@ -91,7 +91,7 @@ message ValidatorRecord {
// TODO(781): The usage of withdrawal_credentials is not defined in spec. Not used in Prysm yet.
bytes withdrawal_credentials = 2; // TODO(781): this is hash32, rename with suffix _hash32
bytes randao_commitment_hash32 = 3;
uint64 randao_skips = 4;
uint64 randao_layers = 4;
// Balance in Gwei
uint64 balance = 5;
// Possible validator status code: