prysm-pulse/beacon-chain/rpc/validator/proposer.go
Raul Jordan cc741ed8af
Ensure New State Type Tests Pass in Prysm (#4646)
* begin state service

* begin on the state trie idea

* created beacon state structure

* add in the full clone getter

* return by value instead

* add all setters

* new state setters are being completed

* arrays roots exposed

*  close to finishing all these headerssss

* functionality complete

* added in proto benchmark test

* test for compatibility

* add test for compat

* comments fixed

* add clone

* add clone

* remove underlying copies

* make it immutable

* integrate it into chainservice

* revert

* wrap up comments for package

* address all comments and godocs

* address all comments

* clone the pending attestation properly

* properly clone remaining items

* tests pass fixed bug

* begin using it instead of head state

* prevent nil pointer exceptions

* begin using new struct in db

* integrated new type into db package

* add proper nil checks

* using new state in archiver

* refactored much of core

* editing all the precompute functions

* done with most core refactor

* fixed up some bugs in the clone comparisons

* append current epoch atts

* add missing setters

* add new setters

* fix other core methods

* fix up transition

* main service and forkchoice

* fix rpc

* integrated to powchain

* some more changes

* fix build

* improve processing of deposits

* fix error

* prevent panic

* comment

* fix process att

* gaz

* fix up att process

* resolve existing review comments

* resolve another batch of gh comments

* resolve broken cpt state

* revise testutil to use the new state

* begin updating the state transition func to pass in more compartmentalized args

* finish editing transition function to return errors

* block operations pretty much done with refactor

* state transition fully refactored

* got epoch processing completed

* fix build in fork choice

* fixing more of the build

* fix up broken sync package

* it builds nowww it buildssss

* revert registry changes

* Recompute on Read (#4627)

* compute on read

* fix up eth1 data votes

* looking into slashings bug introduced in core/

* able to advance more slots

* add logging

* can now sync with testnet yay

* remove the leaves algorithm and other merkle imports

* expose initialize unsafe funcs

* Update beacon-chain/db/kv/state.go

* lint

Co-authored-by: Raul Jordan <raul@prysmaticlabs.com>

* More Optimizations for New State (#4641)

* map optimization

* more optimizations

* use a custom hasher

* comment

* block operations optimizations

* Update beacon-chain/state/types.go

Co-Authored-By: Raul Jordan <raul@prysmaticlabs.com>

* fixed up various operations to use the validator index map access

Co-authored-by: Raul Jordan <raul@prysmaticlabs.com>

* archiver tests pass

* fixing cache tests

* cache tests passing

* edited validator tests

* powchain tests passing

* halfway thru sync tests

* more sync test fixes

* add in tests for state/

* working through rpc tests

* assignments tests passed

* almost done with rpc/beacon tests

* resolved painful validator test

* fixed up even more tests

* resolve tests

* fix build

* reduce a randao mixes copy

* fixes under //beacon-chain/blockchain/...

* build //beacon-chain/core/...

* fixes

* Runtime Optimizations (#4648)

* parallelize shuffling

* clean up

* lint

* fix build

* use callback to read from registry

* fix array roots and size map

* new improvements

* reduce hash allocs

* improved shuffling

* terence's review

* use different method

* raul's comment

* new array roots

* remove clone in pre-compute

* Update beacon-chain/state/types.go

Co-Authored-By: Raul Jordan <raul@prysmaticlabs.com>

* raul's review

* lint

* fix build issues

* fix visibility

Co-authored-by: Raul Jordan <raul@prysmaticlabs.com>

* fix visibility

* build works for all

* fix blockchain test

* fix a few tests

* fix more tests

* update validator in slashing

* archiver passing

* fixed rpc/validator

* progress on core tests

* resolve broken rpc tests

* blockchain tests passed

* fix up some tests in core

* fix message diff

* remove unnecessary save

* Save validator after slashing

* Update validators one by one

* another update

* fix everything

* fix more precompute tests

* fix blocks tests

* more elegant fix

* more helper fixes

* change back ?

* fix test

* fix skip slot

* fix test

* reset caches

* fix testutil

* raceoff fixed

* passing

* Retrieve cached state in the beginning

* lint

* Fixed tests part 1

* Fixed rest of the tests

* Minor changes to avoid copying, small refactor to reduce deplicated code

* Handle att req for slot 0

* New beacon state: Only populate merkle layers as needed, copy merkle layers on copy/clone. (#4689)

* Only populate merkle layers as needed, copy merkle layers on copy/clone.

* use custom copy

* Make maps of correct size

* slightly fast, doesn't wait for lock

Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>

* Target root can't be 0x00

* Don't use cache for current slot (may not be the right fix)

* fixed up tests

* Remove some copy for init sync. Not sure if it is safe enough for runtime though... testing...

* Align with prev logic for process slots cachedState.Slot() < slot

* Fix Initial Sync Flag (#4692)

* fixes

* fix up some test failures due to lack of nil checks

* fix up some test failures due to lack of nil checks

* fix up imports

* revert some changes

* imports

Co-authored-by: Raul Jordan <raul@prysmaticlabs.com>

* resolving further conflicts

* Better skip slot cache (#4694)

* Return copy of skip slot cache state, disable skip slot cache on sync

* fix

* Fix pruning

* fix up issues with broken tests

Co-authored-by: Nishant Das <nish1993@hotmail.com>
Co-authored-by: Preston Van Loon <preston@prysmaticlabs.com>
Co-authored-by: shayzluf <thezluf@gmail.com>
Co-authored-by: terence tsao <terence@prysmaticlabs.com>
Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>
2020-01-31 12:57:01 -08:00

398 lines
15 KiB
Go

package validator
import (
"context"
"fmt"
"math/big"
"math/rand"
"github.com/pkg/errors"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/go-ssz"
"github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state/interop"
stateTrie "github.com/prysmaticlabs/prysm/beacon-chain/state"
dbpb "github.com/prysmaticlabs/prysm/proto/beacon/db"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/trieutil"
"go.opencensus.io/trace"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
// GetBlock is called by a proposer during its assigned slot to request a block to sign
// by passing in the slot and the signed randao reveal of the slot.
func (vs *Server) GetBlock(ctx context.Context, req *ethpb.BlockRequest) (*ethpb.BeaconBlock, error) {
ctx, span := trace.StartSpan(ctx, "ProposerServer.RequestBlock")
defer span.End()
span.AddAttributes(trace.Int64Attribute("slot", int64(req.Slot)))
if vs.SyncChecker.Syncing() {
return nil, status.Errorf(codes.Unavailable, "Syncing to latest head, not ready to respond")
}
// Retrieve the parent block as the current head of the canonical chain.
parentRoot, err := vs.HeadFetcher.HeadRoot(ctx)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not retrieve head root: %v", err)
}
eth1Data, err := vs.eth1Data(ctx, req.Slot)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get ETH1 data: %v", err)
}
// Pack ETH1 deposits which have not been included in the beacon chain.
deposits, err := vs.deposits(ctx, eth1Data)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get ETH1 deposits: %v", err)
}
// Pack aggregated attestations which have not been included in the beacon chain.
atts := vs.AttPool.AggregatedAttestations()
atts, err = vs.filterAttestationsForBlockInclusion(ctx, req.Slot, atts)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not filter attestations: %v", err)
}
// Use zero hash as stub for state root to compute later.
stateRoot := params.BeaconConfig().ZeroHash[:]
graffiti := bytesutil.ToBytes32(req.Graffiti)
blk := &ethpb.BeaconBlock{
Slot: req.Slot,
ParentRoot: parentRoot[:],
StateRoot: stateRoot,
Body: &ethpb.BeaconBlockBody{
Eth1Data: eth1Data,
Deposits: deposits,
Attestations: atts,
RandaoReveal: req.RandaoReveal,
// TODO(2766): Implement rest of the retrievals for beacon block operations
ProposerSlashings: []*ethpb.ProposerSlashing{},
AttesterSlashings: []*ethpb.AttesterSlashing{},
VoluntaryExits: vs.ExitPool.PendingExits(req.Slot),
Graffiti: graffiti[:],
},
}
// Compute state root with the newly constructed block.
stateRoot, err = vs.computeStateRoot(ctx, &ethpb.SignedBeaconBlock{Block: blk, Signature: make([]byte, 96)})
if err != nil {
interop.WriteBlockToDisk(&ethpb.SignedBeaconBlock{Block: blk}, true /*failed*/)
return nil, status.Errorf(codes.Internal, "Could not compute state root: %v", err)
}
blk.StateRoot = stateRoot
return blk, nil
}
// ProposeBlock is called by a proposer during its assigned slot to create a block in an attempt
// to get it processed by the beacon node as the canonical head.
func (vs *Server) ProposeBlock(ctx context.Context, blk *ethpb.SignedBeaconBlock) (*ethpb.ProposeResponse, error) {
root, err := ssz.HashTreeRoot(blk.Block)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not tree hash block: %v", err)
}
log.WithField("blockRoot", fmt.Sprintf("%#x", bytesutil.Trunc(root[:]))).Debugf(
"Block proposal received via RPC")
if err := vs.BlockReceiver.ReceiveBlock(ctx, blk); err != nil {
return nil, status.Errorf(codes.Internal, "Could not process beacon block: %v", err)
}
if err := vs.deleteAttsInPool(blk.Block.Body.Attestations); err != nil {
return nil, status.Errorf(codes.Internal, "Could not delete attestations in pool: %v", err)
}
return &ethpb.ProposeResponse{
BlockRoot: root[:],
}, nil
}
// eth1Data determines the appropriate eth1data for a block proposal. The algorithm for this method
// is as follows:
// - Determine the timestamp for the start slot for the eth1 voting period.
// - Determine the most recent eth1 block before that timestamp.
// - Subtract that eth1block.number by ETH1_FOLLOW_DISTANCE.
// - This is the eth1block to use for the block proposal.
func (vs *Server) eth1Data(ctx context.Context, slot uint64) (*ethpb.Eth1Data, error) {
if vs.MockEth1Votes {
return vs.mockETH1DataVote(ctx, slot)
}
if !vs.Eth1InfoFetcher.IsConnectedToETH1() {
return vs.randomETH1DataVote(ctx)
}
eth1VotingPeriodStartTime, _ := vs.Eth1InfoFetcher.Eth2GenesisPowchainInfo()
eth1VotingPeriodStartTime += (slot - (slot % params.BeaconConfig().SlotsPerEth1VotingPeriod)) * params.BeaconConfig().SecondsPerSlot
// Look up most recent block up to timestamp
blockNumber, err := vs.Eth1BlockFetcher.BlockNumberByTimestamp(ctx, eth1VotingPeriodStartTime)
if err != nil {
return nil, errors.Wrap(err, "could not get block number from timestamp")
}
return vs.defaultEth1DataResponse(ctx, blockNumber)
}
func (vs *Server) mockETH1DataVote(ctx context.Context, slot uint64) (*ethpb.Eth1Data, error) {
log.Warn("Beacon Node is no longer connected to an ETH1 Chain, so " +
"ETH1 Data votes are now mocked.")
// If a mock eth1 data votes is specified, we use the following for the
// eth1data we provide to every proposer based on https://github.com/ethereum/eth2.0-pm/issues/62:
//
// slot_in_voting_period = current_slot % SLOTS_PER_ETH1_VOTING_PERIOD
// Eth1Data(
// DepositRoot = hash(current_epoch + slot_in_voting_period),
// DepositCount = state.eth1_deposit_index,
// BlockHash = hash(hash(current_epoch + slot_in_voting_period)),
// )
slotInVotingPeriod := slot % params.BeaconConfig().SlotsPerEth1VotingPeriod
headState, err := vs.HeadFetcher.HeadState(ctx)
if err != nil {
return nil, err
}
enc, err := ssz.Marshal(helpers.SlotToEpoch(slot) + slotInVotingPeriod)
if err != nil {
return nil, err
}
depRoot := hashutil.Hash(enc)
blockHash := hashutil.Hash(depRoot[:])
return &ethpb.Eth1Data{
DepositRoot: depRoot[:],
DepositCount: headState.Eth1DepositIndex(),
BlockHash: blockHash[:],
}, nil
}
func (vs *Server) randomETH1DataVote(ctx context.Context) (*ethpb.Eth1Data, error) {
log.Warn("Beacon Node is no longer connected to an ETH1 Chain, so " +
"ETH1 Data votes are now random.")
headState, err := vs.HeadFetcher.HeadState(ctx)
if err != nil {
return nil, err
}
// set random roots and block hashes to prevent a majority from being
// built if the eth1 node is offline
depRoot := hashutil.Hash(bytesutil.Bytes32(rand.Uint64()))
blockHash := hashutil.Hash(bytesutil.Bytes32(rand.Uint64()))
return &ethpb.Eth1Data{
DepositRoot: depRoot[:],
DepositCount: headState.Eth1DepositIndex(),
BlockHash: blockHash[:],
}, nil
}
// computeStateRoot computes the state root after a block has been processed through a state transition and
// returns it to the validator client.
func (vs *Server) computeStateRoot(ctx context.Context, block *ethpb.SignedBeaconBlock) ([]byte, error) {
beaconState, err := vs.BeaconDB.State(ctx, bytesutil.ToBytes32(block.Block.ParentRoot))
if err != nil {
return nil, errors.Wrap(err, "could not retrieve beacon state")
}
root, err := state.CalculateStateRoot(
ctx,
beaconState,
block,
)
if err != nil {
return nil, errors.Wrapf(err, "could not calculate state root at slot %d", beaconState.Slot())
}
log.WithField("beaconStateRoot", fmt.Sprintf("%#x", root)).Debugf("Computed state root")
return root[:], nil
}
// deposits returns a list of pending deposits that are ready for inclusion in the next beacon
// block. Determining deposits depends on the current eth1data vote for the block and whether or not
// this eth1data has enough support to be considered for deposits inclusion. If current vote has
// enough support, then use that vote for basis of determining deposits, otherwise use current state
// eth1data.
func (vs *Server) deposits(ctx context.Context, currentVote *ethpb.Eth1Data) ([]*ethpb.Deposit, error) {
if vs.MockEth1Votes || !vs.Eth1InfoFetcher.IsConnectedToETH1() {
return []*ethpb.Deposit{}, nil
}
// Need to fetch if the deposits up to the state's latest eth 1 data matches
// the number of all deposits in this RPC call. If not, then we return nil.
headState, err := vs.HeadFetcher.HeadState(ctx)
if err != nil {
return nil, status.Error(codes.Internal, "Could not get head state")
}
canonicalEth1Data, latestEth1DataHeight, err := vs.canonicalEth1Data(ctx, headState, currentVote)
if err != nil {
return nil, err
}
_, genesisEth1Block := vs.Eth1InfoFetcher.Eth2GenesisPowchainInfo()
if genesisEth1Block.Cmp(latestEth1DataHeight) == 0 {
return []*ethpb.Deposit{}, nil
}
upToEth1DataDeposits := vs.DepositFetcher.AllDeposits(ctx, latestEth1DataHeight)
depositData := [][]byte{}
for _, dep := range upToEth1DataDeposits {
depHash, err := ssz.HashTreeRoot(dep.Data)
if err != nil {
return nil, errors.Wrap(err, "could not hash deposit data")
}
depositData = append(depositData, depHash[:])
}
depositTrie, err := trieutil.GenerateTrieFromItems(depositData, int(params.BeaconConfig().DepositContractTreeDepth))
if err != nil {
return nil, errors.Wrap(err, "could not generate historical deposit trie from deposits")
}
allPendingContainers := vs.PendingDepositsFetcher.PendingContainers(ctx, latestEth1DataHeight)
// Deposits need to be received in order of merkle index root, so this has to make sure
// deposits are sorted from lowest to highest.
var pendingDeps []*dbpb.DepositContainer
for _, dep := range allPendingContainers {
if uint64(dep.Index) >= headState.Eth1DepositIndex() && uint64(dep.Index) < canonicalEth1Data.DepositCount {
pendingDeps = append(pendingDeps, dep)
}
}
for i := range pendingDeps {
// Don't construct merkle proof if the number of deposits is more than max allowed in block.
if uint64(i) == params.BeaconConfig().MaxDeposits {
break
}
pendingDeps[i].Deposit, err = constructMerkleProof(depositTrie, int(pendingDeps[i].Index), pendingDeps[i].Deposit)
if err != nil {
return nil, err
}
}
// Limit the return of pending deposits to not be more than max deposits allowed in block.
var pendingDeposits []*ethpb.Deposit
for i := 0; i < len(pendingDeps) && i < int(params.BeaconConfig().MaxDeposits); i++ {
pendingDeposits = append(pendingDeposits, pendingDeps[i].Deposit)
}
return pendingDeposits, nil
}
// canonicalEth1Data determines the canonical eth1data and eth1 block height to use for determining deposits.
func (vs *Server) canonicalEth1Data(ctx context.Context, beaconState *stateTrie.BeaconState, currentVote *ethpb.Eth1Data) (*ethpb.Eth1Data, *big.Int, error) {
var eth1BlockHash [32]byte
// Add in current vote, to get accurate vote tally
beaconState.AppendEth1DataVotes(currentVote)
hasSupport, err := blocks.Eth1DataHasEnoughSupport(beaconState, currentVote)
if err != nil {
return nil, nil, errors.Wrap(err, "could not determine if current eth1data vote has enough support")
}
var canonicalEth1Data *ethpb.Eth1Data
if hasSupport {
canonicalEth1Data = currentVote
eth1BlockHash = bytesutil.ToBytes32(currentVote.BlockHash)
} else {
canonicalEth1Data = beaconState.Eth1Data()
eth1BlockHash = bytesutil.ToBytes32(beaconState.Eth1Data().BlockHash)
}
_, latestEth1DataHeight, err := vs.Eth1BlockFetcher.BlockExists(ctx, eth1BlockHash)
if err != nil {
return nil, nil, errors.Wrap(err, "could not fetch eth1data height")
}
return canonicalEth1Data, latestEth1DataHeight, nil
}
// in case no vote for new eth1data vote considered best vote we
// default into returning the latest deposit root and the block
// hash of eth1 block hash that is FOLLOW_DISTANCE back from its
// latest block.
func (vs *Server) defaultEth1DataResponse(ctx context.Context, currentHeight *big.Int) (*ethpb.Eth1Data, error) {
eth1FollowDistance := int64(params.BeaconConfig().Eth1FollowDistance)
ancestorHeight := big.NewInt(0).Sub(currentHeight, big.NewInt(eth1FollowDistance))
blockHash, err := vs.Eth1BlockFetcher.BlockHashByHeight(ctx, ancestorHeight)
if err != nil {
return nil, errors.Wrap(err, "could not fetch ETH1_FOLLOW_DISTANCE ancestor")
}
// Fetch all historical deposits up to an ancestor height.
depositsTillHeight, depositRoot := vs.DepositFetcher.DepositsNumberAndRootAtHeight(ctx, ancestorHeight)
if depositsTillHeight == 0 {
return vs.ChainStartFetcher.ChainStartEth1Data(), nil
}
return &ethpb.Eth1Data{
DepositRoot: depositRoot[:],
BlockHash: blockHash[:],
DepositCount: depositsTillHeight,
}, nil
}
// This filters the input attestations to return a list of valid attestations to be packaged inside a beacon block.
func (vs *Server) filterAttestationsForBlockInclusion(ctx context.Context, slot uint64, atts []*ethpb.Attestation) ([]*ethpb.Attestation, error) {
ctx, span := trace.StartSpan(ctx, "ProposerServer.filterAttestationsForBlockInclusion")
defer span.End()
validAtts := make([]*ethpb.Attestation, 0, len(atts))
inValidAtts := make([]*ethpb.Attestation, 0, len(atts))
bState, err := vs.HeadFetcher.HeadState(ctx)
if err != nil {
return nil, errors.New("could not head state from DB")
}
if bState.Slot() < slot {
bState, err = state.ProcessSlots(ctx, bState, slot)
if err != nil {
return nil, errors.Wrapf(err, "could not process slots up to %d", slot)
}
}
// TODO(3916): Insert optimizations to sort out the most profitable attestations
for i, att := range atts {
if i == int(params.BeaconConfig().MaxAttestations) {
break
}
if _, err := blocks.ProcessAttestation(ctx, bState, att); err != nil {
inValidAtts = append(inValidAtts, att)
continue
}
validAtts = append(validAtts, att)
}
if err := vs.deleteAttsInPool(inValidAtts); err != nil {
return nil, err
}
return validAtts, nil
}
// The input attestations are processed and seen by the node, this deletes them from pool
// so proposers don't include them in a block for the future.
func (vs *Server) deleteAttsInPool(atts []*ethpb.Attestation) error {
for _, att := range atts {
if helpers.IsAggregated(att) {
if err := vs.AttPool.DeleteAggregatedAttestation(att); err != nil {
return err
}
} else {
if err := vs.AttPool.DeleteUnaggregatedAttestation(att); err != nil {
return err
}
}
}
return nil
}
func constructMerkleProof(trie *trieutil.SparseMerkleTrie, index int, deposit *ethpb.Deposit) (*ethpb.Deposit, error) {
proof, err := trie.MerkleProof(index)
if err != nil {
return nil, errors.Wrapf(err, "could not generate merkle proof for deposit at index %d", index)
}
// For every deposit, we construct a Merkle proof using the powchain service's
// in-memory deposits trie, which is updated only once the state's LatestETH1Data
// property changes during a state transition after a voting period.
deposit.Proof = proof
return deposit, nil
}