mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-09 15:37:56 -05:00
"Super sync" and naive p2p reputation (#2550)
* checkpoint on super sync with reputation * ensure handling only expected peers msg * exclusive of finalized block * skip block saved already * clean up struct * remove 2 more fields * _ * everything builds, but doesnt test yet * lint * fix p2p tests * space * space * space * fmt * fmt
This commit is contained in:
committed by
Raul Jordan
parent
ecef1093eb
commit
991ee7e81b
@@ -1078,3 +1078,9 @@ go_repository(
|
||||
commit = "ec06cd93a07565b373789b0078ba88fe697fddd9",
|
||||
importpath = "github.com/karlseguin/ccache",
|
||||
)
|
||||
|
||||
go_repository(
|
||||
name = "com_github_libp2p_go_libp2p_connmgr",
|
||||
commit = "5249ec107e4544863dd8c61a6417b640d03a62a2",
|
||||
importpath = "github.com/libp2p/go-libp2p-connmgr",
|
||||
)
|
||||
|
||||
@@ -57,6 +57,7 @@ func main() {
|
||||
cmd.RelayNode,
|
||||
cmd.P2PPort,
|
||||
cmd.P2PHost,
|
||||
cmd.P2PMaxPeers,
|
||||
cmd.DataDirFlag,
|
||||
cmd.VerbosityFlag,
|
||||
cmd.EnableTracingFlag,
|
||||
|
||||
@@ -43,6 +43,7 @@ func configureP2P(ctx *cli.Context) (*p2p.Server, error) {
|
||||
RelayNodeAddr: ctx.GlobalString(cmd.RelayNode.Name),
|
||||
HostAddress: ctx.GlobalString(cmd.P2PHost.Name),
|
||||
Port: ctx.GlobalInt(cmd.P2PPort.Name),
|
||||
MaxPeers: ctx.GlobalInt(cmd.P2PMaxPeers.Name),
|
||||
DepositContractAddress: contractAddress,
|
||||
})
|
||||
if err != nil {
|
||||
|
||||
@@ -7,13 +7,13 @@ import (
|
||||
"math/big"
|
||||
"time"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/shared/hashutil"
|
||||
ptypes "github.com/gogo/protobuf/types"
|
||||
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
|
||||
"github.com/prysmaticlabs/prysm/beacon-chain/db"
|
||||
pbp2p "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
|
||||
pb "github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1"
|
||||
"github.com/prysmaticlabs/prysm/shared/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/hashutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/params"
|
||||
"github.com/prysmaticlabs/prysm/shared/trieutil"
|
||||
)
|
||||
|
||||
@@ -25,7 +25,7 @@ func (s *InitialSync) checkBlockValidity(ctx context.Context, block *pb.BeaconBl
|
||||
}
|
||||
|
||||
if block.Slot < beaconState.FinalizedEpoch*params.BeaconConfig().SlotsPerEpoch {
|
||||
return errors.New(debugError + "discarding received block with a slot number smaller than the last finalized slot")
|
||||
return errors.New("discarding received block with a slot number smaller than the last finalized slot")
|
||||
}
|
||||
// Attestation from proposer not verified as, other nodes only store blocks not proposer
|
||||
// attestations.
|
||||
@@ -34,7 +34,7 @@ func (s *InitialSync) checkBlockValidity(ctx context.Context, block *pb.BeaconBl
|
||||
|
||||
// safelyHandleMessage will recover and log any panic that occurs from the
|
||||
// function argument.
|
||||
func safelyHandleMessage(fn func(p2p.Message), msg p2p.Message) {
|
||||
func safelyHandleMessage(fn func(p2p.Message) error, msg p2p.Message) {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
printedMsg := noMsgData
|
||||
@@ -61,5 +61,7 @@ func safelyHandleMessage(fn func(p2p.Message), msg p2p.Message) {
|
||||
}()
|
||||
|
||||
// Fingers crossed that it doesn't panic...
|
||||
fn(msg)
|
||||
if err := fn(msg); err != nil {
|
||||
log.WithError(err).Error("Failed to process message")
|
||||
}
|
||||
}
|
||||
|
||||
@@ -12,17 +12,19 @@ package initialsync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/big"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/gogo/protobuf/proto"
|
||||
peer "github.com/libp2p/go-libp2p-peer"
|
||||
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
|
||||
"github.com/prysmaticlabs/prysm/beacon-chain/db"
|
||||
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
|
||||
"github.com/prysmaticlabs/prysm/shared/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/event"
|
||||
"github.com/prysmaticlabs/prysm/shared/hashutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/p2p"
|
||||
@@ -31,7 +33,12 @@ import (
|
||||
)
|
||||
|
||||
var log = logrus.WithField("prefix", "initial-sync")
|
||||
var debugError = "debug:"
|
||||
|
||||
var (
|
||||
// ErrCanonicalStateMismatch can occur when the node has processed all blocks
|
||||
// from a peer, but arrived at a different state root.
|
||||
ErrCanonicalStateMismatch = errors.New("canonical state did not match after syncing with peer")
|
||||
)
|
||||
|
||||
// Config defines the configurable properties of InitialSync.
|
||||
//
|
||||
@@ -60,7 +67,8 @@ func DefaultConfig() *Config {
|
||||
|
||||
type p2pAPI interface {
|
||||
p2p.Sender
|
||||
Subscribe(msg proto.Message, channel chan p2p.Message) event.Subscription
|
||||
p2p.ReputationManager
|
||||
p2p.Subscriber
|
||||
}
|
||||
|
||||
type powChainService interface {
|
||||
@@ -91,20 +99,11 @@ type InitialSync struct {
|
||||
powchain powChainService
|
||||
batchedBlockBuf chan p2p.Message
|
||||
stateBuf chan p2p.Message
|
||||
currentSlot uint64
|
||||
highestObservedSlot uint64
|
||||
highestObservedRoot [32]byte
|
||||
beaconStateSlot uint64
|
||||
syncPollingInterval time.Duration
|
||||
syncedFeed *event.Feed
|
||||
stateReceived bool
|
||||
lastRequestedSlot uint64
|
||||
finalizedStateRoot [32]byte
|
||||
mutex *sync.Mutex
|
||||
nodeIsSynced bool
|
||||
bestPeer peer.ID
|
||||
canonicalBlockRoot []byte
|
||||
finalizedBlockRoot []byte
|
||||
}
|
||||
|
||||
// NewInitialSyncService constructs a new InitialSyncService.
|
||||
@@ -125,9 +124,6 @@ func NewInitialSyncService(ctx context.Context,
|
||||
db: cfg.BeaconDB,
|
||||
powchain: cfg.PowChain,
|
||||
chainService: cfg.ChainService,
|
||||
currentSlot: params.BeaconConfig().GenesisSlot,
|
||||
highestObservedSlot: params.BeaconConfig().GenesisSlot,
|
||||
beaconStateSlot: params.BeaconConfig().GenesisSlot,
|
||||
stateBuf: stateBuf,
|
||||
batchedBlockBuf: batchedBlockBuf,
|
||||
syncPollingInterval: cfg.SyncPollingInterval,
|
||||
@@ -138,13 +134,8 @@ func NewInitialSyncService(ctx context.Context,
|
||||
}
|
||||
|
||||
// Start begins the goroutine.
|
||||
func (s *InitialSync) Start() {
|
||||
cHead, err := s.db.ChainHead()
|
||||
if err != nil {
|
||||
log.Errorf("Unable to get chain head %v", err)
|
||||
}
|
||||
s.currentSlot = cHead.Slot
|
||||
go s.run()
|
||||
func (s *InitialSync) Start(chainHeadResponses map[peer.ID]*pb.ChainHeadResponse) {
|
||||
go s.run(chainHeadResponses)
|
||||
}
|
||||
|
||||
// Stop kills the initial sync goroutine.
|
||||
@@ -154,47 +145,21 @@ func (s *InitialSync) Stop() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// InitializeObservedSlot sets the highest observed slot.
|
||||
func (s *InitialSync) InitializeObservedSlot(slot uint64) {
|
||||
s.highestObservedSlot = slot
|
||||
}
|
||||
|
||||
// InitializeObservedStateRoot sets the highest observed state root.
|
||||
func (s *InitialSync) InitializeObservedStateRoot(root [32]byte) {
|
||||
s.highestObservedRoot = root
|
||||
}
|
||||
|
||||
// InitializeFinalizedStateRoot sets the state root of the last finalized state.
|
||||
func (s *InitialSync) InitializeFinalizedStateRoot(root [32]byte) {
|
||||
s.finalizedStateRoot = root
|
||||
}
|
||||
|
||||
// InitializeBestPeer sets the peer ID of the highest observed peer.
|
||||
func (s *InitialSync) InitializeBestPeer(p peer.ID) {
|
||||
s.bestPeer = p
|
||||
}
|
||||
|
||||
// InitializeBlockRoots sets canonical and finalized block roots for batch request.
|
||||
func (s *InitialSync) InitializeBlockRoots(finalizedRoot []byte, canonicalRoot []byte) {
|
||||
s.canonicalBlockRoot = canonicalRoot
|
||||
s.finalizedBlockRoot = finalizedRoot
|
||||
}
|
||||
|
||||
// HighestObservedSlot returns the highest observed slot.
|
||||
func (s *InitialSync) HighestObservedSlot() uint64 {
|
||||
return s.highestObservedSlot
|
||||
}
|
||||
|
||||
// NodeIsSynced checks that the node has been caught up with the network.
|
||||
func (s *InitialSync) NodeIsSynced() (bool, uint64) {
|
||||
return s.nodeIsSynced, s.currentSlot
|
||||
func (s *InitialSync) NodeIsSynced() bool {
|
||||
return s.nodeIsSynced
|
||||
}
|
||||
|
||||
func (s *InitialSync) exitInitialSync(ctx context.Context, block *pb.BeaconBlock) error {
|
||||
func (s *InitialSync) exitInitialSync(ctx context.Context, block *pb.BeaconBlock, chainHead *pb.ChainHeadResponse) error {
|
||||
if s.nodeIsSynced {
|
||||
return nil
|
||||
}
|
||||
state, err := s.db.HeadState(ctx)
|
||||
parentRoot := bytesutil.ToBytes32(block.ParentRootHash32)
|
||||
parent, err := s.db.Block(parentRoot)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
state, err := s.db.HistoricalStateFromSlot(ctx, parent.Slot, parentRoot)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -238,14 +203,14 @@ func (s *InitialSync) exitInitialSync(ctx context.Context, block *pb.BeaconBlock
|
||||
|
||||
stateRoot := s.db.HeadStateRoot()
|
||||
|
||||
if stateRoot != s.highestObservedRoot {
|
||||
// TODO(#2155): Instead of a fatal call, drop the peer and restart the initial sync service.
|
||||
log.Error("OH NO - looks like you synced with a bad peer, try restarting your node!")
|
||||
log.Fatalf(
|
||||
if stateRoot != bytesutil.ToBytes32(chainHead.CanonicalStateRootHash32) {
|
||||
log.Errorf(
|
||||
"Canonical state root %#x does not match highest observed root from peer %#x",
|
||||
stateRoot,
|
||||
s.highestObservedRoot,
|
||||
chainHead.CanonicalStateRootHash32,
|
||||
)
|
||||
|
||||
return ErrCanonicalStateMismatch
|
||||
}
|
||||
log.WithField("canonicalStateSlot", state.Slot-params.BeaconConfig().GenesisSlot).Info("Exiting init sync and starting regular sync")
|
||||
s.syncService.ResumeSync()
|
||||
@@ -257,7 +222,7 @@ func (s *InitialSync) exitInitialSync(ctx context.Context, block *pb.BeaconBlock
|
||||
// run is the main goroutine for the initial sync service.
|
||||
// delayChan is explicitly passed into this function to facilitate tests that don't require a timeout.
|
||||
// It is assumed that the goroutine `run` is only called once per instance.
|
||||
func (s *InitialSync) run() {
|
||||
func (s *InitialSync) run(chainHeadResponses map[peer.ID]*pb.ChainHeadResponse) {
|
||||
batchedBlocksub := s.p2p.Subscribe(&pb.BatchedBeaconBlockResponse{}, s.batchedBlockBuf)
|
||||
beaconStateSub := s.p2p.Subscribe(&pb.BeaconStateResponse{}, s.stateBuf)
|
||||
defer func() {
|
||||
@@ -267,20 +232,72 @@ func (s *InitialSync) run() {
|
||||
close(s.stateBuf)
|
||||
}()
|
||||
|
||||
// We send out a state request to all peers.
|
||||
if err := s.requestStateFromPeer(s.ctx, s.finalizedStateRoot); err != nil {
|
||||
ctx := s.ctx
|
||||
|
||||
var peers []peer.ID
|
||||
for k := range chainHeadResponses {
|
||||
peers = append(peers, k)
|
||||
}
|
||||
|
||||
// Sort peers in descending order based on their canonical slot.
|
||||
sort.Slice(peers, func(i, j int) bool {
|
||||
return chainHeadResponses[peers[i]].CanonicalSlot > chainHeadResponses[peers[j]].CanonicalSlot
|
||||
})
|
||||
|
||||
for _, peer := range peers {
|
||||
chainHead := chainHeadResponses[peer]
|
||||
if err := s.syncToPeer(ctx, chainHead, peer); err != nil {
|
||||
log.WithError(err).WithField("peer", peer.Pretty()).Warn("Failed to sync with peer, trying next best peer")
|
||||
continue
|
||||
}
|
||||
log.Info("Synced!")
|
||||
break
|
||||
}
|
||||
|
||||
if !s.nodeIsSynced {
|
||||
log.Fatal("Failed to sync with anyone...")
|
||||
}
|
||||
}
|
||||
|
||||
func (s *InitialSync) syncToPeer(ctx context.Context, chainHeadResponse *pb.ChainHeadResponse, peer peer.ID) error {
|
||||
fields := logrus.Fields{
|
||||
"peer": peer.Pretty(),
|
||||
"canonicalSlot": chainHeadResponse.CanonicalSlot - params.BeaconConfig().GenesisSlot,
|
||||
}
|
||||
|
||||
log.WithFields(fields).Info("Requesting state from peer")
|
||||
if err := s.requestStateFromPeer(ctx, bytesutil.ToBytes32(chainHeadResponse.FinalizedStateRootHash32S), peer); err != nil {
|
||||
log.Errorf("Could not request state from peer %v", err)
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(s.ctx, 20*time.Second)
|
||||
defer cancel()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
log.Debug("Exiting goroutine")
|
||||
return
|
||||
case <-ctx.Done():
|
||||
|
||||
return ctx.Err()
|
||||
case msg := <-s.stateBuf:
|
||||
safelyHandleMessage(s.processState, msg)
|
||||
log.WithFields(fields).Info("Received state resp from peer")
|
||||
if err := s.processState(msg, chainHeadResponse); err != nil {
|
||||
return err
|
||||
}
|
||||
case msg := <-s.batchedBlockBuf:
|
||||
safelyHandleMessage(s.processBatchedBlocks, msg)
|
||||
if msg.Peer != peer {
|
||||
continue
|
||||
}
|
||||
log.WithFields(fields).Info("Received batched blocks from peer")
|
||||
if err := s.processBatchedBlocks(msg, chainHeadResponse); err != nil {
|
||||
log.WithError(err).WithField("peer", peer).Error("Failed to sync with peer.")
|
||||
s.p2p.Reputation(msg.Peer, p2p.RepPenalityInitialSyncFailure)
|
||||
continue
|
||||
}
|
||||
if !s.nodeIsSynced {
|
||||
return errors.New("node still not in sync after receiving batch blocks")
|
||||
}
|
||||
s.p2p.Reputation(msg.Peer, p2p.RepRewardValidBlock)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -32,6 +32,10 @@ func (mp *mockP2P) Send(ctx context.Context, msg proto.Message, peerID peer.ID)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mp *mockP2P) Reputation(_ peer.ID, _ int) {
|
||||
|
||||
}
|
||||
|
||||
type mockSyncService struct {
|
||||
hasStarted bool
|
||||
isSynced bool
|
||||
@@ -121,7 +125,6 @@ func TestProcessingBatchedBlocks_OK(t *testing.T) {
|
||||
|
||||
batchSize := 20
|
||||
batchedBlocks := make([]*pb.BeaconBlock, batchSize)
|
||||
expectedSlot := params.BeaconConfig().GenesisSlot + uint64(batchSize)
|
||||
|
||||
for i := 1; i <= batchSize; i++ {
|
||||
batchedBlocks[i-1] = &pb.BeaconBlock{
|
||||
@@ -139,25 +142,10 @@ func TestProcessingBatchedBlocks_OK(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
ss.processBatchedBlocks(msg)
|
||||
chainHead := &pb.ChainHeadResponse{}
|
||||
|
||||
state, err := db.HeadState(context.Background())
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
stateRoot, err := hashutil.HashProto(state)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ss.highestObservedRoot = stateRoot
|
||||
ss.processBatchedBlocks(msg, chainHead)
|
||||
|
||||
if ss.currentSlot != expectedSlot {
|
||||
t.Errorf("Expected slot %d equal to current slot %d", expectedSlot, ss.currentSlot)
|
||||
}
|
||||
|
||||
if ss.highestObservedSlot == expectedSlot {
|
||||
t.Errorf("Expected slot %d not equal to highest observed slot slot %d", expectedSlot, ss.highestObservedSlot)
|
||||
}
|
||||
}
|
||||
|
||||
func TestProcessingBlocks_SkippedSlots(t *testing.T) {
|
||||
@@ -175,8 +163,6 @@ func TestProcessingBlocks_SkippedSlots(t *testing.T) {
|
||||
ss := NewInitialSyncService(context.Background(), cfg)
|
||||
|
||||
batchSize := 20
|
||||
expectedSlot := params.BeaconConfig().GenesisSlot + uint64(batchSize)
|
||||
ss.highestObservedSlot = expectedSlot
|
||||
blks, err := ss.db.BlocksBySlot(ctx, params.BeaconConfig().GenesisSlot)
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to get genesis block %v", err)
|
||||
@@ -186,15 +172,6 @@ func TestProcessingBlocks_SkippedSlots(t *testing.T) {
|
||||
t.Fatalf("Unable to hash block %v", err)
|
||||
}
|
||||
parentHash := h[:]
|
||||
state, err := db.HeadState(context.Background())
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
stateRoot, err := hashutil.HashProto(state)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ss.highestObservedRoot = stateRoot
|
||||
|
||||
for i := 1; i <= batchSize; i++ {
|
||||
// skip slots
|
||||
@@ -206,7 +183,9 @@ func TestProcessingBlocks_SkippedSlots(t *testing.T) {
|
||||
ParentRootHash32: parentHash,
|
||||
}
|
||||
|
||||
ss.processBlock(context.Background(), block)
|
||||
chainHead := &pb.ChainHeadResponse{}
|
||||
|
||||
ss.processBlock(context.Background(), block, chainHead)
|
||||
|
||||
// Save the block and set the parent hash of the next block
|
||||
// as the hash of the current block.
|
||||
@@ -219,30 +198,15 @@ func TestProcessingBlocks_SkippedSlots(t *testing.T) {
|
||||
t.Fatalf("Could not hash block %v", err)
|
||||
}
|
||||
parentHash = hash[:]
|
||||
state, err := db.HeadState(context.Background())
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
stateRoot, err := hashutil.HashProto(state)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ss.highestObservedRoot = stateRoot
|
||||
|
||||
}
|
||||
|
||||
if ss.currentSlot != expectedSlot {
|
||||
t.Errorf("Expected slot %d equal to current slot %d", expectedSlot, ss.currentSlot)
|
||||
}
|
||||
|
||||
if ss.highestObservedSlot != expectedSlot {
|
||||
t.Errorf("Expected slot %d equal to highest observed slot %d", expectedSlot, ss.highestObservedSlot)
|
||||
}
|
||||
}
|
||||
|
||||
func TestSafelyHandleMessage(t *testing.T) {
|
||||
hook := logTest.NewGlobal()
|
||||
|
||||
safelyHandleMessage(func(_ p2p.Message) {
|
||||
safelyHandleMessage(func(_ p2p.Message) error {
|
||||
panic("bad!")
|
||||
}, p2p.Message{
|
||||
Data: &pb.BeaconBlock{},
|
||||
@@ -254,7 +218,7 @@ func TestSafelyHandleMessage(t *testing.T) {
|
||||
func TestSafelyHandleMessage_NoData(t *testing.T) {
|
||||
hook := logTest.NewGlobal()
|
||||
|
||||
safelyHandleMessage(func(_ p2p.Message) {
|
||||
safelyHandleMessage(func(_ p2p.Message) error {
|
||||
panic("bad!")
|
||||
}, p2p.Message{})
|
||||
|
||||
|
||||
@@ -5,8 +5,8 @@ import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strings"
|
||||
|
||||
"github.com/libp2p/go-libp2p-peer"
|
||||
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
|
||||
"github.com/prysmaticlabs/prysm/shared/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/hashutil"
|
||||
@@ -19,37 +19,29 @@ import (
|
||||
// processBlock is the main method that validates each block which is received
|
||||
// for initial sync. It checks if the blocks are valid and then will continue to
|
||||
// process and save it into the db.
|
||||
func (s *InitialSync) processBlock(ctx context.Context, block *pb.BeaconBlock) {
|
||||
func (s *InitialSync) processBlock(ctx context.Context, block *pb.BeaconBlock, chainHead *pb.ChainHeadResponse) error {
|
||||
ctx, span := trace.StartSpan(ctx, "beacon-chain.sync.initial-sync.processBlock")
|
||||
defer span.End()
|
||||
recBlock.Inc()
|
||||
|
||||
if block.Slot == s.highestObservedSlot {
|
||||
s.currentSlot = s.highestObservedSlot
|
||||
if err := s.exitInitialSync(s.ctx, block); err != nil {
|
||||
if block.Slot == chainHead.CanonicalSlot {
|
||||
if err := s.exitInitialSync(s.ctx, block, chainHead); err != nil {
|
||||
log.Errorf("Could not exit initial sync: %v", err)
|
||||
return
|
||||
return err
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
if block.Slot < s.currentSlot {
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.validateAndSaveNextBlock(ctx, block); err != nil {
|
||||
// Debug error so as not to have noisy error logs
|
||||
if strings.HasPrefix(err.Error(), debugError) {
|
||||
log.Debug(strings.TrimPrefix(err.Error(), debugError))
|
||||
return
|
||||
}
|
||||
log.Errorf("Unable to save block: %v", err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// processBatchedBlocks processes all the received blocks from
|
||||
// the p2p message.
|
||||
func (s *InitialSync) processBatchedBlocks(msg p2p.Message) {
|
||||
func (s *InitialSync) processBatchedBlocks(msg p2p.Message, chainHead *pb.ChainHeadResponse) error {
|
||||
ctx, span := trace.StartSpan(msg.Ctx, "beacon-chain.sync.initial-sync.processBatchedBlocks")
|
||||
defer span.End()
|
||||
batchedBlockReq.Inc()
|
||||
@@ -58,30 +50,28 @@ func (s *InitialSync) processBatchedBlocks(msg p2p.Message) {
|
||||
batchedBlocks := response.BatchedBlocks
|
||||
if len(batchedBlocks) == 0 {
|
||||
// Do not process empty responses.
|
||||
return
|
||||
}
|
||||
if msg.Peer != s.bestPeer {
|
||||
// Only process batch block responses that come from the best peer
|
||||
// we originally synced with.
|
||||
log.WithField("peerID", msg.Peer.Pretty()).Debug("Received batch blocks from a different peer")
|
||||
return
|
||||
s.p2p.Reputation(msg.Peer, p2p.RepPenalityInitialSyncFailure)
|
||||
return nil
|
||||
}
|
||||
|
||||
log.Debug("Processing batched block response")
|
||||
log.WithField("blocks", len(batchedBlocks)).Info("Processing batched block response")
|
||||
// Sort batchBlocks in ascending order.
|
||||
sort.Slice(batchedBlocks, func(i, j int) bool {
|
||||
return batchedBlocks[i].Slot < batchedBlocks[j].Slot
|
||||
})
|
||||
for _, block := range batchedBlocks {
|
||||
s.processBlock(ctx, block)
|
||||
if err := s.processBlock(ctx, block, chainHead); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
log.Debug("Finished processing batched blocks")
|
||||
return nil
|
||||
}
|
||||
|
||||
// requestBatchedBlocks sends out a request for multiple blocks that's between finalized roots
|
||||
// and head roots.
|
||||
func (s *InitialSync) requestBatchedBlocks(finalizedRoot []byte, canonicalRoot []byte) {
|
||||
ctx, span := trace.StartSpan(context.Background(), "beacon-chain.sync.initial-sync.requestBatchedBlocks")
|
||||
func (s *InitialSync) requestBatchedBlocks(ctx context.Context, finalizedRoot []byte, canonicalRoot []byte, peer peer.ID) {
|
||||
ctx, span := trace.StartSpan(ctx, "beacon-chain.sync.initial-sync.requestBatchedBlocks")
|
||||
defer span.End()
|
||||
sentBatchedBlockReq.Inc()
|
||||
|
||||
@@ -92,8 +82,8 @@ func (s *InitialSync) requestBatchedBlocks(finalizedRoot []byte, canonicalRoot [
|
||||
if err := s.p2p.Send(ctx, &pb.BatchedBeaconBlockRequest{
|
||||
FinalizedRoot: finalizedRoot,
|
||||
CanonicalRoot: canonicalRoot,
|
||||
}, s.bestPeer); err != nil {
|
||||
log.Errorf("Could not send batch block request to peer %s: %v", s.bestPeer.Pretty(), err)
|
||||
}, peer); err != nil {
|
||||
log.Errorf("Could not send batch block request to peer %s: %v", peer.Pretty(), err)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -109,6 +99,12 @@ func (s *InitialSync) validateAndSaveNextBlock(ctx context.Context, block *pb.Be
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if s.db.HasBlock(root) {
|
||||
log.WithField("block", fmt.Sprintf("%#x", root)).
|
||||
Warn("Skipping block in db already")
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.checkBlockValidity(ctx, block); err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -116,7 +112,6 @@ func (s *InitialSync) validateAndSaveNextBlock(ctx context.Context, block *pb.Be
|
||||
"root": fmt.Sprintf("%#x", bytesutil.Trunc(root[:])),
|
||||
"slot": block.Slot - params.BeaconConfig().GenesisSlot,
|
||||
}).Info("Saving block")
|
||||
s.currentSlot = block.Slot
|
||||
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
|
||||
@@ -3,17 +3,17 @@ package initialsync
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/shared/hashutil"
|
||||
|
||||
"github.com/libp2p/go-libp2p-peer"
|
||||
"github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
|
||||
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
|
||||
"github.com/prysmaticlabs/prysm/shared/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/hashutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/p2p"
|
||||
"github.com/prysmaticlabs/prysm/shared/params"
|
||||
"go.opencensus.io/trace"
|
||||
)
|
||||
|
||||
func (s *InitialSync) processState(msg p2p.Message) {
|
||||
func (s *InitialSync) processState(msg p2p.Message, chainHead *pb.ChainHeadResponse) error {
|
||||
ctx, span := trace.StartSpan(msg.Ctx, "beacon-chain.sync.initial-sync.processState")
|
||||
defer span.End()
|
||||
data := msg.Data.(*pb.BeaconStateResponse)
|
||||
@@ -22,47 +22,47 @@ func (s *InitialSync) processState(msg p2p.Message) {
|
||||
|
||||
if err := s.db.SaveFinalizedState(finalizedState); err != nil {
|
||||
log.Errorf("Unable to set received last finalized state in db: %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.db.SaveFinalizedBlock(finalizedState.LatestBlock); err != nil {
|
||||
log.Errorf("Could not save finalized block %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.db.SaveBlock(finalizedState.LatestBlock); err != nil {
|
||||
log.Errorf("Could not save block %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
root, err := hashutil.HashBeaconBlock(finalizedState.LatestBlock)
|
||||
finalizedBlockRoot, err := hashutil.HashBeaconBlock(finalizedState.LatestBlock)
|
||||
if err != nil {
|
||||
log.Errorf("Could not hash finalized block %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.db.SaveHistoricalState(ctx, finalizedState, root); err != nil {
|
||||
if err := s.db.SaveHistoricalState(ctx, finalizedState, finalizedBlockRoot); err != nil {
|
||||
log.Errorf("Could not save new historical state: %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.db.SaveAttestationTarget(ctx, &pb.AttestationTarget{
|
||||
Slot: finalizedState.LatestBlock.Slot,
|
||||
BlockRoot: root[:],
|
||||
BlockRoot: finalizedBlockRoot[:],
|
||||
ParentRoot: finalizedState.LatestBlock.ParentRootHash32,
|
||||
}); err != nil {
|
||||
log.Errorf("Could not to save attestation target: %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.db.SaveJustifiedState(finalizedState); err != nil {
|
||||
log.Errorf("Could not set beacon state for initial sync %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := s.db.SaveJustifiedBlock(finalizedState.LatestBlock); err != nil {
|
||||
log.Errorf("Could not save finalized block %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
exists, _, err := s.powchain.BlockExists(ctx, bytesutil.ToBytes32(finalizedState.LatestEth1Data.BlockHash32))
|
||||
@@ -72,36 +72,35 @@ func (s *InitialSync) processState(msg p2p.Message) {
|
||||
|
||||
if !exists {
|
||||
log.Error("Latest ETH1 block doesn't exist in the pow chain")
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
s.db.PrunePendingDeposits(ctx, finalizedState.DepositIndex)
|
||||
|
||||
if err := s.db.UpdateChainHead(ctx, finalizedState.LatestBlock, finalizedState); err != nil {
|
||||
log.Errorf("Could not update chain head: %v", err)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
validators.InitializeValidatorStore(finalizedState)
|
||||
|
||||
// sets the current slot to the last finalized slot of the
|
||||
// beacon state to begin our sync from.
|
||||
s.currentSlot = finalizedState.Slot
|
||||
s.stateReceived = true
|
||||
log.Debugf(
|
||||
"Successfully saved beacon state with the last finalized slot: %d",
|
||||
finalizedState.Slot-params.BeaconConfig().GenesisSlot,
|
||||
)
|
||||
s.requestBatchedBlocks(s.finalizedBlockRoot, s.canonicalBlockRoot)
|
||||
s.lastRequestedSlot = s.highestObservedSlot
|
||||
log.WithField("peer", msg.Peer.Pretty()).Info("Requesting batch blocks from peer")
|
||||
s.requestBatchedBlocks(ctx, finalizedBlockRoot[:], chainHead.CanonicalBlockRoot, msg.Peer)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// requestStateFromPeer requests for the canonical state, finalized state, and justified state from a peer.
|
||||
func (s *InitialSync) requestStateFromPeer(ctx context.Context, lastFinalizedRoot [32]byte) error {
|
||||
func (s *InitialSync) requestStateFromPeer(ctx context.Context, lastFinalizedRoot [32]byte, peer peer.ID) error {
|
||||
ctx, span := trace.StartSpan(ctx, "beacon-chain.sync.initial-sync.requestStateFromPeer")
|
||||
defer span.End()
|
||||
stateReq.Inc()
|
||||
return s.p2p.Send(ctx, &pb.BeaconStateRequest{
|
||||
FinalizedStateRootHash32S: lastFinalizedRoot[:],
|
||||
}, s.bestPeer)
|
||||
}, peer)
|
||||
}
|
||||
|
||||
@@ -59,7 +59,7 @@ type Querier struct {
|
||||
chainStarted bool
|
||||
atGenesis bool
|
||||
bestPeer peer.ID
|
||||
peerMap map[peer.ID]uint64
|
||||
chainHeadResponses map[peer.ID]*pb.ChainHeadResponse
|
||||
canonicalBlockRoot []byte
|
||||
finalizedBlockRoot []byte
|
||||
}
|
||||
@@ -74,18 +74,18 @@ func NewQuerierService(ctx context.Context,
|
||||
responseBuf := make(chan p2p.Message, cfg.ResponseBufferSize)
|
||||
|
||||
return &Querier{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
p2p: cfg.P2P,
|
||||
db: cfg.BeaconDB,
|
||||
chainService: cfg.ChainService,
|
||||
responseBuf: responseBuf,
|
||||
currentHeadSlot: cfg.CurrentHeadSlot,
|
||||
chainStarted: false,
|
||||
atGenesis: true,
|
||||
powchain: cfg.PowChain,
|
||||
chainStartBuf: make(chan time.Time, 1),
|
||||
peerMap: make(map[peer.ID]uint64),
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
p2p: cfg.P2P,
|
||||
db: cfg.BeaconDB,
|
||||
chainService: cfg.ChainService,
|
||||
responseBuf: responseBuf,
|
||||
currentHeadSlot: cfg.CurrentHeadSlot,
|
||||
chainStarted: false,
|
||||
atGenesis: true,
|
||||
powchain: cfg.PowChain,
|
||||
chainStartBuf: make(chan time.Time, 1),
|
||||
chainHeadResponses: make(map[peer.ID]*pb.ChainHeadResponse),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -186,15 +186,14 @@ func (q *Querier) run() {
|
||||
hasReceivedResponse = true
|
||||
}
|
||||
response := msg.Data.(*pb.ChainHeadResponse)
|
||||
if _, ok := q.peerMap[msg.Peer]; !ok {
|
||||
if _, ok := q.chainHeadResponses[msg.Peer]; !ok {
|
||||
queryLog.WithFields(logrus.Fields{
|
||||
"peerID": msg.Peer.Pretty(),
|
||||
"highestSlot": response.CanonicalSlot - params.BeaconConfig().GenesisSlot,
|
||||
}).Info("Received chain head from peer")
|
||||
q.peerMap[msg.Peer] = response.CanonicalSlot
|
||||
q.chainHeadResponses[msg.Peer] = response
|
||||
}
|
||||
if response.CanonicalSlot > q.currentHeadSlot {
|
||||
q.bestPeer = msg.Peer
|
||||
q.currentHeadSlot = response.CanonicalSlot
|
||||
q.currentStateRoot = response.CanonicalStateRootHash32
|
||||
q.currentFinalizedStateRoot = bytesutil.ToBytes32(response.FinalizedStateRootHash32S)
|
||||
|
||||
@@ -201,8 +201,10 @@ func (rs *RegularSync) validateAndProcessBlock(
|
||||
|
||||
if err := rs.chainService.ApplyForkChoiceRule(ctx, block, beaconState); err != nil {
|
||||
log.WithError(err).Error("Could not run fork choice on block")
|
||||
rs.p2p.Reputation(blockMsg.Peer, p2p.RepPenalityInvalidBlock)
|
||||
return nil, nil, false, err
|
||||
}
|
||||
rs.p2p.Reputation(blockMsg.Peer, p2p.RepRewardValidBlock)
|
||||
sentBlocks.Inc()
|
||||
// We update the last observed slot to the received canonical block's slot.
|
||||
if block.Slot > rs.highestObservedSlot {
|
||||
|
||||
@@ -49,6 +49,7 @@ type p2pAPI interface {
|
||||
p2p.Broadcaster
|
||||
p2p.Sender
|
||||
p2p.Subscriber
|
||||
p2p.ReputationManager
|
||||
}
|
||||
|
||||
// RegularSync is the gateway and the bridge between the p2p network and the local beacon chain.
|
||||
@@ -205,6 +206,8 @@ func (rs *RegularSync) run() {
|
||||
defer exitSub.Unsubscribe()
|
||||
defer canonicalBlockSub.Unsubscribe()
|
||||
|
||||
log.Info("Listening for regular sync messages from peers")
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-rs.ctx.Done():
|
||||
@@ -426,6 +429,7 @@ func (rs *RegularSync) receiveAttestation(msg p2p.Message) error {
|
||||
log.Debug("Sending newly received attestation to subscribers")
|
||||
rs.operationsService.IncomingAttFeed().Send(attestation)
|
||||
rs.attsService.IncomingAttestationFeed().Send(attestation)
|
||||
rs.p2p.Reputation(msg.Peer, p2p.RepRewardValidAttestation)
|
||||
sentAttestation.Inc()
|
||||
sendAttestationSpan.End()
|
||||
return nil
|
||||
|
||||
@@ -49,6 +49,10 @@ func (mp *mockP2P) Send(ctx context.Context, msg proto.Message, peerID peer.ID)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mp *mockP2P) Reputation(_ peer.ID, val int) {
|
||||
|
||||
}
|
||||
|
||||
type mockChainService struct {
|
||||
sFeed *event.Feed
|
||||
cFeed *event.Feed
|
||||
|
||||
@@ -8,8 +8,6 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/beacon-chain/operations"
|
||||
initialsync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync"
|
||||
"github.com/prysmaticlabs/prysm/shared/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/shared/params"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
@@ -113,8 +111,8 @@ func (ss *Service) Status() error {
|
||||
if blk == nil {
|
||||
return errors.New("no chain head exists in db")
|
||||
}
|
||||
if blk.Slot < ss.InitialSync.HighestObservedSlot() {
|
||||
return fmt.Errorf("node is not synced as the current chain head is at slot %d", blk.Slot-params.BeaconConfig().GenesisSlot)
|
||||
if !ss.InitialSync.NodeIsSynced() {
|
||||
return errors.New("not initially synced")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@@ -127,18 +125,10 @@ func (ss *Service) run() {
|
||||
}
|
||||
ss.querierFinished = true
|
||||
|
||||
// Sets the highest observed slot from querier.
|
||||
ss.InitialSync.InitializeObservedSlot(ss.Querier.currentHeadSlot)
|
||||
ss.InitialSync.InitializeBestPeer(ss.Querier.bestPeer)
|
||||
ss.InitialSync.InitializeObservedStateRoot(bytesutil.ToBytes32(ss.Querier.currentStateRoot))
|
||||
// Sets the state root of the highest observed slot.
|
||||
ss.InitialSync.InitializeFinalizedStateRoot(ss.Querier.currentFinalizedStateRoot)
|
||||
ss.InitialSync.InitializeBlockRoots(ss.Querier.finalizedBlockRoot, ss.Querier.canonicalBlockRoot)
|
||||
|
||||
if synced {
|
||||
ss.RegularSync.Start()
|
||||
return
|
||||
}
|
||||
|
||||
ss.InitialSync.Start()
|
||||
ss.InitialSync.Start(ss.Querier.chainHeadResponses)
|
||||
}
|
||||
|
||||
@@ -93,7 +93,7 @@ func setupTestSyncService(t *testing.T, synced bool) (*Service, *db.BeaconDB) {
|
||||
func TestStatus_NotSynced(t *testing.T) {
|
||||
serviceNotSynced, db := setupTestSyncService(t, false)
|
||||
defer internal.TeardownDB(t, db)
|
||||
synced, _ := serviceNotSynced.InitialSync.NodeIsSynced()
|
||||
synced := serviceNotSynced.InitialSync.NodeIsSynced()
|
||||
if synced {
|
||||
t.Error("Wanted false, but got true")
|
||||
}
|
||||
|
||||
@@ -58,6 +58,7 @@ spec:
|
||||
- --datadir=/data
|
||||
# Disabling gossip sub until a larger beacon chain deployment.
|
||||
- --disable-gossip-sub
|
||||
- --p2p-max-peers=50
|
||||
resources:
|
||||
requests:
|
||||
memory: "100Mi"
|
||||
|
||||
@@ -78,6 +78,12 @@ var (
|
||||
Usage: "The IP address advertised by libp2p. This may be used to advertise an external IP.",
|
||||
Value: "",
|
||||
}
|
||||
// P2PMaxPeers defines a flag to specify the max number of peers in libp2p.
|
||||
P2PMaxPeers = cli.Int64Flag{
|
||||
Name: "p2p-max-peers",
|
||||
Usage: "The max number of p2p peers to maintain.",
|
||||
Value: 30,
|
||||
}
|
||||
// ClearDB tells the beacon node to remove any previously stored data at the data directory.
|
||||
ClearDB = cli.BoolFlag{
|
||||
Name: "clear-db",
|
||||
|
||||
@@ -4,6 +4,7 @@ go_library(
|
||||
name = "go_default_library",
|
||||
srcs = [
|
||||
"addr_factory.go",
|
||||
"connection_manager.go",
|
||||
"dial_relay_node.go",
|
||||
"discovery.go",
|
||||
"feed.go",
|
||||
@@ -32,6 +33,7 @@ go_library(
|
||||
"@com_github_libp2p_go_libp2p//config:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p//p2p/discovery:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p//p2p/host/routed:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p_connmgr//:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p_host//:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p_kad_dht//:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p_net//:go_default_library",
|
||||
@@ -102,6 +104,7 @@ go_test(
|
||||
race = "off", # TODO(#377): fix issues with race detection testing.
|
||||
tags = ["requires-network"],
|
||||
deps = [
|
||||
"//shared/testutil:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p//p2p/host/basic:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p_swarm//testing:go_default_library",
|
||||
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
|
||||
|
||||
45
shared/p2p/connection_manager.go
Normal file
45
shared/p2p/connection_manager.go
Normal file
@@ -0,0 +1,45 @@
|
||||
package p2p
|
||||
|
||||
import (
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/libp2p/go-libp2p"
|
||||
"github.com/libp2p/go-libp2p-connmgr"
|
||||
"github.com/libp2p/go-libp2p-peer"
|
||||
)
|
||||
|
||||
// Reputation reward values.
|
||||
const (
|
||||
RepRewardValidBlock = 4
|
||||
RepRewardValidAttestation = 1
|
||||
|
||||
RepPenalityInvalidProtobuf = -1000
|
||||
RepPenalityInitialSyncFailure = -500
|
||||
RepPenalityInvalidBlock = -10
|
||||
RepPenalityInvalidAttestation = -5
|
||||
)
|
||||
|
||||
func optionConnectionManager(maxPeers int) libp2p.Option {
|
||||
if maxPeers < 5 {
|
||||
log.Warn("Max peers < 5. Defaulting to 5 max peers")
|
||||
maxPeers = 5
|
||||
}
|
||||
minPeers := int(math.Max(5, float64(maxPeers-5)))
|
||||
cm := connmgr.NewConnManager(minPeers, maxPeers, 5*time.Minute)
|
||||
|
||||
return libp2p.ConnectionManager(cm)
|
||||
}
|
||||
|
||||
// Reputation adds (or subtracts) a given reward/penalty against a peer.
|
||||
// Eventually, the lowest scoring peers will be pruned from the connections.
|
||||
func (s *Server) Reputation(peer peer.ID, val int) {
|
||||
s.host.ConnManager().TagPeer(peer, TagReputation, val)
|
||||
}
|
||||
|
||||
// Disconnect will close all connections to the given peer.
|
||||
func (s *Server) Disconnect(peer peer.ID) {
|
||||
if err := s.host.Network().ClosePeer(peer); err != nil {
|
||||
log.WithError(err).WithField("peer", peer.Pretty()).Error("Failed to close conn with peer")
|
||||
}
|
||||
}
|
||||
@@ -4,6 +4,7 @@ import (
|
||||
"context"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/libp2p/go-libp2p-peer"
|
||||
"github.com/prysmaticlabs/prysm/shared/event"
|
||||
)
|
||||
|
||||
@@ -20,3 +21,9 @@ type Broadcaster interface {
|
||||
type Subscriber interface {
|
||||
Subscribe(msg proto.Message, channel chan Message) event.Subscription
|
||||
}
|
||||
|
||||
// ReputationManager represents a subset of the p2p.Server which enables
|
||||
// reputaiton reporting of peers.
|
||||
type ReputationManager interface {
|
||||
Reputation(peer peer.ID, val int)
|
||||
}
|
||||
|
||||
@@ -3,8 +3,9 @@ package p2p
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
libp2p "github.com/libp2p/go-libp2p"
|
||||
"github.com/libp2p/go-libp2p"
|
||||
ma "github.com/multiformats/go-multiaddr"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/shared/iputils"
|
||||
)
|
||||
|
||||
@@ -12,7 +13,7 @@ import (
|
||||
// TODO(287): Expand on these options and provide the option configuration via flags.
|
||||
// Currently, this is a random port and a (seemingly) consistent private key
|
||||
// identity.
|
||||
func buildOptions(port int) []libp2p.Option {
|
||||
func buildOptions(port, maxPeers int) []libp2p.Option {
|
||||
ip, err := iputils.ExternalIPv4()
|
||||
if err != nil {
|
||||
log.Errorf("Could not get IPv4 address: %v", err)
|
||||
@@ -26,5 +27,6 @@ func buildOptions(port int) []libp2p.Option {
|
||||
return []libp2p.Option{
|
||||
libp2p.ListenAddrs(listen),
|
||||
libp2p.EnableRelay(), // Allows dialing to peers via relay.
|
||||
optionConnectionManager(maxPeers),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -5,7 +5,7 @@ import (
|
||||
)
|
||||
|
||||
func TestBuildOptions(t *testing.T) {
|
||||
opts := buildOptions(1)
|
||||
opts := buildOptions(1, 10)
|
||||
|
||||
_ = opts
|
||||
}
|
||||
|
||||
@@ -18,6 +18,9 @@ import peer "github.com/libp2p/go-libp2p-peer"
|
||||
// AnyPeer represents a Peer ID alias for sending to any available peer(s).
|
||||
const AnyPeer = peer.ID("AnyPeer")
|
||||
|
||||
// TagReputation is a libp2p tag for identifying reputation.
|
||||
const TagReputation = "prysm-reputation"
|
||||
|
||||
// Use this file for interfaces only!
|
||||
|
||||
// Adapter is used to create middleware.
|
||||
|
||||
@@ -66,13 +66,14 @@ type ServerConfig struct {
|
||||
RelayNodeAddr string
|
||||
HostAddress string
|
||||
Port int
|
||||
MaxPeers int
|
||||
DepositContractAddress string
|
||||
}
|
||||
|
||||
// NewServer creates a new p2p server instance.
|
||||
func NewServer(cfg *ServerConfig) (*Server, error) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
opts := buildOptions(cfg.Port)
|
||||
opts := buildOptions(cfg.Port, cfg.MaxPeers)
|
||||
if cfg.RelayNodeAddr != "" {
|
||||
opts = append(opts, libp2p.AddrsFactory(withRelayAddrs(cfg.RelayNodeAddr)))
|
||||
} else if cfg.HostAddress != "" {
|
||||
@@ -128,6 +129,7 @@ func NewServer(cfg *ServerConfig) (*Server, error) {
|
||||
return nil, err
|
||||
}
|
||||
exclusions = append(exclusions, info.ID)
|
||||
h.ConnManager().Protect(info.ID, TagReputation)
|
||||
}
|
||||
setupPeerNegotiation(h, cfg.DepositContractAddress, exclusions)
|
||||
setHandshakeHandler(h, cfg.DepositContractAddress)
|
||||
@@ -263,6 +265,7 @@ func (s *Server) RegisterTopic(topic string, message proto.Message, adapters ...
|
||||
data := proto.Clone(message)
|
||||
if err := proto.Unmarshal(msg.Payload, data); err != nil {
|
||||
log.Error("Could not unmarshal payload")
|
||||
s.Reputation(peerID, RepPenalityInvalidProtobuf)
|
||||
}
|
||||
pMsg := Message{Ctx: ctx, Data: data, Peer: peerID}
|
||||
for _, adapter := range adapters {
|
||||
|
||||
@@ -3,6 +3,7 @@ package p2p
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/shared/testutil"
|
||||
logTest "github.com/sirupsen/logrus/hooks/test"
|
||||
)
|
||||
|
||||
@@ -15,14 +16,11 @@ func TestLifecycle(t *testing.T) {
|
||||
}
|
||||
|
||||
s.Start()
|
||||
msg := hook.Entries[0].Message
|
||||
want := "Starting service"
|
||||
if msg != want {
|
||||
t.Errorf("incorrect log. wanted: %s. got: %v", want, msg)
|
||||
}
|
||||
testutil.AssertLogsContain(t, hook, want)
|
||||
|
||||
s.Stop()
|
||||
msg = hook.LastEntry().Message
|
||||
msg := hook.LastEntry().Message
|
||||
want = "Stopping service"
|
||||
if msg != want {
|
||||
t.Errorf("incorrect log. wanted: %s. got: %v", want, msg)
|
||||
|
||||
Reference in New Issue
Block a user