From a24cee26fa59277988a3b7f24fd8935f8edc1047 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 10 Oct 2024 20:00:34 +0100
Subject: [PATCH 01/25] wip
---
polygon/p2p/message_sender.go | 16 ++++-
polygon/p2p/peer_event_registrar.go | 3 +
polygon/p2p/peer_event_registrar_mock.go | 77 ++++++++++++++++++++
polygon/p2p/peer_tracker.go | 89 ++++++++++++++++++++----
polygon/p2p/publisher.go | 77 ++++++++++++++++++++
5 files changed, 246 insertions(+), 16 deletions(-)
create mode 100644 polygon/p2p/publisher.go
diff --git a/polygon/p2p/message_sender.go b/polygon/p2p/message_sender.go
index a670ebe279b..6ece00acbad 100644
--- a/polygon/p2p/message_sender.go
+++ b/polygon/p2p/message_sender.go
@@ -32,6 +32,8 @@ var ErrPeerNotFound = errors.New("peer not found")
type MessageSender interface {
SendGetBlockHeaders(ctx context.Context, peerId *PeerId, req eth.GetBlockHeadersPacket66) error
SendGetBlockBodies(ctx context.Context, peerId *PeerId, req eth.GetBlockBodiesPacket66) error
+ SendNewBlockHashes(ctx context.Context, peerId *PeerId, req eth.NewBlockHashesPacket) error
+ SendNewBlock(ctx context.Context, peerId *PeerId, req eth.NewBlockPacket) error
}
func NewMessageSender(sentryClient sentry.SentryClient) MessageSender {
@@ -45,14 +47,22 @@ type messageSender struct {
}
func (ms *messageSender) SendGetBlockHeaders(ctx context.Context, peerId *PeerId, req eth.GetBlockHeadersPacket66) error {
- return ms.sendMessage(ctx, sentry.MessageId_GET_BLOCK_HEADERS_66, req, peerId)
+ return ms.sendMessageToPeer(ctx, sentry.MessageId_GET_BLOCK_HEADERS_66, req, peerId)
}
func (ms *messageSender) SendGetBlockBodies(ctx context.Context, peerId *PeerId, req eth.GetBlockBodiesPacket66) error {
- return ms.sendMessage(ctx, sentry.MessageId_GET_BLOCK_BODIES_66, req, peerId)
+ return ms.sendMessageToPeer(ctx, sentry.MessageId_GET_BLOCK_BODIES_66, req, peerId)
}
-func (ms *messageSender) sendMessage(ctx context.Context, messageId sentry.MessageId, data any, peerId *PeerId) error {
+func (ms *messageSender) SendNewBlockHashes(ctx context.Context, peerId *PeerId, req eth.NewBlockHashesPacket) error {
+ return ms.sendMessageToPeer(ctx, sentry.MessageId_NEW_BLOCK_HASHES_66, req, peerId)
+}
+
+func (ms *messageSender) SendNewBlock(ctx context.Context, peerId *PeerId, req eth.NewBlockPacket) error {
+ return ms.sendMessageToPeer(ctx, sentry.MessageId_NEW_BLOCK_66, req, peerId)
+}
+
+func (ms *messageSender) sendMessageToPeer(ctx context.Context, messageId sentry.MessageId, data any, peerId *PeerId) error {
rlpData, err := rlp.EncodeToBytes(data)
if err != nil {
return err
diff --git a/polygon/p2p/peer_event_registrar.go b/polygon/p2p/peer_event_registrar.go
index 03f7a952c99..9a5c80497a1 100644
--- a/polygon/p2p/peer_event_registrar.go
+++ b/polygon/p2p/peer_event_registrar.go
@@ -18,10 +18,13 @@ package p2p
import (
"github.com/erigontech/erigon-lib/gointerfaces/sentryproto"
+ "github.com/erigontech/erigon/eth/protocols/eth"
"github.com/erigontech/erigon/polygon/polygoncommon"
)
//go:generate mockgen -typed=true -source=./peer_event_registrar.go -destination=./peer_event_registrar_mock.go -package=p2p
type peerEventRegistrar interface {
RegisterPeerEventObserver(observer polygoncommon.Observer[*sentryproto.PeerEvent]) UnregisterFunc
+ RegisterNewBlockObserver(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockPacket]]) UnregisterFunc
+ RegisterNewBlockHashesObserver(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockHashesPacket]]) UnregisterFunc
}
diff --git a/polygon/p2p/peer_event_registrar_mock.go b/polygon/p2p/peer_event_registrar_mock.go
index 8c688d49f85..2e8b0726f43 100644
--- a/polygon/p2p/peer_event_registrar_mock.go
+++ b/polygon/p2p/peer_event_registrar_mock.go
@@ -13,6 +13,7 @@ import (
reflect "reflect"
sentryproto "github.com/erigontech/erigon-lib/gointerfaces/sentryproto"
+ eth "github.com/erigontech/erigon/eth/protocols/eth"
polygoncommon "github.com/erigontech/erigon/polygon/polygoncommon"
gomock "go.uber.org/mock/gomock"
)
@@ -40,6 +41,82 @@ func (m *MockpeerEventRegistrar) EXPECT() *MockpeerEventRegistrarMockRecorder {
return m.recorder
}
+// RegisterNewBlockHashesObserver mocks base method.
+func (m *MockpeerEventRegistrar) RegisterNewBlockHashesObserver(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockHashesPacket]]) UnregisterFunc {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "RegisterNewBlockHashesObserver", observer)
+ ret0, _ := ret[0].(UnregisterFunc)
+ return ret0
+}
+
+// RegisterNewBlockHashesObserver indicates an expected call of RegisterNewBlockHashesObserver.
+func (mr *MockpeerEventRegistrarMockRecorder) RegisterNewBlockHashesObserver(observer any) *MockpeerEventRegistrarRegisterNewBlockHashesObserverCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterNewBlockHashesObserver", reflect.TypeOf((*MockpeerEventRegistrar)(nil).RegisterNewBlockHashesObserver), observer)
+ return &MockpeerEventRegistrarRegisterNewBlockHashesObserverCall{Call: call}
+}
+
+// MockpeerEventRegistrarRegisterNewBlockHashesObserverCall wrap *gomock.Call
+type MockpeerEventRegistrarRegisterNewBlockHashesObserverCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockpeerEventRegistrarRegisterNewBlockHashesObserverCall) Return(arg0 UnregisterFunc) *MockpeerEventRegistrarRegisterNewBlockHashesObserverCall {
+ c.Call = c.Call.Return(arg0)
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockpeerEventRegistrarRegisterNewBlockHashesObserverCall) Do(f func(polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockHashesPacket]]) UnregisterFunc) *MockpeerEventRegistrarRegisterNewBlockHashesObserverCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockpeerEventRegistrarRegisterNewBlockHashesObserverCall) DoAndReturn(f func(polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockHashesPacket]]) UnregisterFunc) *MockpeerEventRegistrarRegisterNewBlockHashesObserverCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
+// RegisterNewBlockObserver mocks base method.
+func (m *MockpeerEventRegistrar) RegisterNewBlockObserver(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockPacket]]) UnregisterFunc {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "RegisterNewBlockObserver", observer)
+ ret0, _ := ret[0].(UnregisterFunc)
+ return ret0
+}
+
+// RegisterNewBlockObserver indicates an expected call of RegisterNewBlockObserver.
+func (mr *MockpeerEventRegistrarMockRecorder) RegisterNewBlockObserver(observer any) *MockpeerEventRegistrarRegisterNewBlockObserverCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterNewBlockObserver", reflect.TypeOf((*MockpeerEventRegistrar)(nil).RegisterNewBlockObserver), observer)
+ return &MockpeerEventRegistrarRegisterNewBlockObserverCall{Call: call}
+}
+
+// MockpeerEventRegistrarRegisterNewBlockObserverCall wrap *gomock.Call
+type MockpeerEventRegistrarRegisterNewBlockObserverCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockpeerEventRegistrarRegisterNewBlockObserverCall) Return(arg0 UnregisterFunc) *MockpeerEventRegistrarRegisterNewBlockObserverCall {
+ c.Call = c.Call.Return(arg0)
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockpeerEventRegistrarRegisterNewBlockObserverCall) Do(f func(polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockPacket]]) UnregisterFunc) *MockpeerEventRegistrarRegisterNewBlockObserverCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockpeerEventRegistrarRegisterNewBlockObserverCall) DoAndReturn(f func(polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockPacket]]) UnregisterFunc) *MockpeerEventRegistrarRegisterNewBlockObserverCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
// RegisterPeerEventObserver mocks base method.
func (m *MockpeerEventRegistrar) RegisterPeerEventObserver(observer polygoncommon.Observer[*sentryproto.PeerEvent]) UnregisterFunc {
m.ctrl.T.Helper()
diff --git a/polygon/p2p/peer_tracker.go b/polygon/p2p/peer_tracker.go
index c0ba07a113c..27bd777a524 100644
--- a/polygon/p2p/peer_tracker.go
+++ b/polygon/p2p/peer_tracker.go
@@ -20,10 +20,13 @@ import (
"context"
"sync"
+ "github.com/hashicorp/golang-lru/v2/simplelru"
"google.golang.org/protobuf/types/known/emptypb"
+ "github.com/erigontech/erigon-lib/common"
"github.com/erigontech/erigon-lib/gointerfaces/sentryproto"
"github.com/erigontech/erigon-lib/log/v3"
+ "github.com/erigontech/erigon/eth/protocols/eth"
"github.com/erigontech/erigon/polygon/polygoncommon"
)
@@ -32,6 +35,8 @@ type PeerTracker interface {
ListPeersMayHaveBlockNum(blockNum uint64) []*PeerId
BlockNumPresent(peerId *PeerId, blockNum uint64)
BlockNumMissing(peerId *PeerId, blockNum uint64)
+ ListPeersMayMissBlockHash(blockHash common.Hash) []*PeerId
+ BlockHashPresent(peerId *PeerId, blockHash common.Hash)
PeerConnected(peerId *PeerId)
PeerDisconnected(peerId *PeerId)
}
@@ -58,29 +63,30 @@ func NewPeerTracker(
}
type peerTracker struct {
- logger log.Logger
- peerProvider peerProvider
- peerEventRegistrar peerEventRegistrar
- mu sync.Mutex
- peerSyncProgresses map[PeerId]*peerSyncProgress
- peerShuffle PeerShuffle
+ logger log.Logger
+ peerProvider peerProvider
+ peerEventRegistrar peerEventRegistrar
+ mu sync.Mutex
+ peerSyncProgresses map[PeerId]*peerSyncProgress
+ peerKnownBlockAnnounces map[PeerId]simplelru.LRUCache[common.Hash, struct{}]
+ peerShuffle PeerShuffle
}
func (pt *peerTracker) Run(ctx context.Context) error {
pt.logger.Debug(peerTrackerLogPrefix("running peer tracker component"))
- var unregister polygoncommon.UnregisterFunc
- defer func() { unregister() }()
+ var peerEventUnreg polygoncommon.UnregisterFunc
+ defer func() { peerEventUnreg() }()
err := func() error {
// we lock the pt for updates so that we:
- // 1. register the observer but buffer the updates coming from it until we do 2.
+ // 1. register the peer connection observer but buffer the updates coming from it until we do 2.
// 2. replay the current state of connected peers
pt.mu.Lock()
defer pt.mu.Unlock()
// 1. register the observer
- unregister = pt.peerEventRegistrar.RegisterPeerEventObserver(NewPeerEventObserver(pt))
+ peerEventUnreg = pt.peerEventRegistrar.RegisterPeerEventObserver(newPeerEventObserver(pt))
// 2. replay the current state of connected peers
reply, err := pt.peerProvider.Peers(ctx, &emptypb.Empty{})
@@ -104,6 +110,12 @@ func (pt *peerTracker) Run(ctx context.Context) error {
return err
}
+ hashAnnouncesUnreg := pt.peerEventRegistrar.RegisterNewBlockHashesObserver(newBlockHashAnnouncesObserver(pt))
+ defer hashAnnouncesUnreg()
+
+ blockAnnouncesUnreg := pt.peerEventRegistrar.RegisterNewBlockObserver(newBlockAnnouncesObserver(pt))
+ defer blockAnnouncesUnreg()
+
<-ctx.Done()
return ctx.Err()
}
@@ -136,12 +148,40 @@ func (pt *peerTracker) BlockNumMissing(peerId *PeerId, blockNum uint64) {
})
}
+func (pt *peerTracker) ListPeersMayMissBlockHash(blockHash common.Hash) []*PeerId {
+ pt.mu.Lock()
+ defer pt.mu.Unlock()
+
+ var peerIds []*PeerId
+ for peerId, knownBlockAnnounces := range pt.peerKnownBlockAnnounces {
+ if !knownBlockAnnounces.Contains(blockHash) {
+ peerIds = append(peerIds, &peerId)
+ }
+ }
+
+ pt.peerShuffle(peerIds)
+ return peerIds
+}
+
+func (pt *peerTracker) BlockHashPresent(peerId *PeerId, blockHash common.Hash) {
+ pt.mu.Lock()
+ defer pt.mu.Unlock()
+
+ announcesLru, ok := pt.peerKnownBlockAnnounces[*peerId]
+ if !ok || announcesLru.Contains(blockHash) {
+ return
+ }
+
+ announcesLru.Add(blockHash, struct{}{})
+}
+
func (pt *peerTracker) PeerDisconnected(peerId *PeerId) {
pt.mu.Lock()
defer pt.mu.Unlock()
pt.logger.Debug(peerTrackerLogPrefix("peer disconnected"), "peerId", peerId.String())
delete(pt.peerSyncProgresses, *peerId)
+ delete(pt.peerKnownBlockAnnounces, *peerId)
}
func (pt *peerTracker) PeerConnected(peerId *PeerId) {
@@ -160,6 +200,15 @@ func (pt *peerTracker) peerConnected(peerId *PeerId) {
peerId: peerId,
}
}
+
+ if _, ok := pt.peerKnownBlockAnnounces[peerIdVal]; !ok {
+ announcesLru, err := simplelru.NewLRU[common.Hash, struct{}](1024, nil)
+ if err != nil {
+ panic(err)
+ }
+
+ pt.peerKnownBlockAnnounces[peerIdVal] = announcesLru
+ }
}
func (pt *peerTracker) updatePeerSyncProgress(peerId *PeerId, update func(psp *peerSyncProgress)) {
@@ -174,18 +223,32 @@ func (pt *peerTracker) updatePeerSyncProgress(peerId *PeerId, update func(psp *p
update(peerSyncProgress)
}
-func NewPeerEventObserver(peerTracker PeerTracker) polygoncommon.Observer[*sentryproto.PeerEvent] {
+func newPeerEventObserver(pt PeerTracker) polygoncommon.Observer[*sentryproto.PeerEvent] {
return func(message *sentryproto.PeerEvent) {
peerId := PeerIdFromH512(message.PeerId)
switch message.EventId {
case sentryproto.PeerEvent_Connect:
- peerTracker.PeerConnected(peerId)
+ pt.PeerConnected(peerId)
case sentryproto.PeerEvent_Disconnect:
- peerTracker.PeerDisconnected(peerId)
+ pt.PeerDisconnected(peerId)
+ }
+ }
+}
+
+func newBlockHashAnnouncesObserver(pt PeerTracker) polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockHashesPacket]] {
+ return func(message *DecodedInboundMessage[*eth.NewBlockHashesPacket]) {
+ for _, hashOrNum := range *message.Decoded {
+ pt.BlockHashPresent(message.PeerId, hashOrNum.Hash)
}
}
}
+func newBlockAnnouncesObserver(pt PeerTracker) polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockPacket]] {
+ return func(message *DecodedInboundMessage[*eth.NewBlockPacket]) {
+ pt.BlockHashPresent(message.PeerId, message.Decoded.Block.Hash())
+ }
+}
+
func peerTrackerLogPrefix(message string) string {
return "[p2p.peerTracker] " + message
}
diff --git a/polygon/p2p/publisher.go b/polygon/p2p/publisher.go
new file mode 100644
index 00000000000..bf99dab7ed3
--- /dev/null
+++ b/polygon/p2p/publisher.go
@@ -0,0 +1,77 @@
+package p2p
+
+import (
+ "context"
+
+ "github.com/erigontech/erigon/core/types"
+ "github.com/erigontech/erigon/eth/protocols/eth"
+)
+
+type Publisher interface {
+ PublishNewBlockHashes(ctx context.Context, block *types.Block) error
+ PublishNewBlock(ctx context.Context, block *types.Block) error
+}
+
+func newPublisher(messageSender MessageSender) Publisher {
+ return &publisher{
+ messageSender: messageSender,
+ }
+}
+
+// https://github.com/ethereum/devp2p/blob/master/caps/eth.md#block-propagation
+//
+// Rules:
+//
+// 1. When a NewBlock announcement message is received from a peer, the client first verifies the basic header validity
+// of the block, checking whether the proof-of-work value is valid. It then sends the block to a small fraction of
+// connected peers (usually the square root of the total number of peers) using the NewBlock message.
+//
+// 2. After the header validity check, the client imports the block into its local chain by executing all transactions
+// contained in the block, computing the block's 'post state'. The block's state-root hash must match the computed
+// post state root. Once the block is fully processed, and considered valid, the client sends a NewBlockHashes
+// message about the block to all peers which it didn't notify earlier. Those peers may request the full block later
+// if they fail to receive it via NewBlock from anyone else.
+//
+// NewBlockHashes (0x01)
+// [[blockhash₁: B_32, number₁: P], [blockhash₂: B_32, number₂: P], ...]
+//
+// Specify one or more new blocks which have appeared on the network. To be maximally helpful, nodes should inform
+// peers of all blocks that they may not be aware of. Including hashes that the sending peer could reasonably be
+// considered to know (due to the fact they were previously informed or because that node has itself advertised
+// knowledge of the hashes through NewBlockHashes) is considered bad form, and may reduce the reputation of the
+// sending node. Including hashes that the sending node later refuses to honour with a proceeding GetBlockHeaders
+// message is considered bad form, and may reduce the reputation of the sending node.
+//
+//
+// NewBlock (0x07)
+// [block, td: P]
+//
+// Specify a single complete block that the peer should know about. td is the total difficulty of the block, i.e. the
+// sum of all block difficulties up to and including this block.
+
+type publisher struct {
+ messageSender MessageSender
+ peerTracker PeerTracker
+}
+
+func (p publisher) PublishNewBlockHashes(ctx context.Context, block *types.Block) error {
+ hash := block.Hash()
+ peers := p.peerTracker.ListPeersMayMissBlockHash(hash)
+ blockHashesPacket := eth.NewBlockHashesPacket{
+ {
+ Hash: block.Hash(),
+ Number: block.NumberU64(),
+ },
+ }
+
+ for _, peerId := range peers {
+ p.messageSender.SendNewBlockHashes(ctx, peerId, blockHashesPacket)
+ }
+
+ return p.messageSender.SendNewBlockHashes(ctx, eth.NewBlockHashesPacket{})
+}
+
+func (p publisher) PublishNewBlock(ctx context.Context, block *types.Block) error {
+ //TODO implement me
+ panic("implement me")
+}
From 23062e69d55d2f65ebe2525b55e49d123ae6ed3a Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 10 Oct 2024 22:31:38 +0100
Subject: [PATCH 02/25] polygon/sync,p2p: publish block announcements to devp2p
---
eth/stagedsync/stage_polygon_sync.go | 17 +++
polygon/p2p/publisher.go | 199 +++++++++++++++++++++------
polygon/p2p/publisher_test.go | 21 +++
polygon/p2p/service.go | 5 +
polygon/p2p/service_mock.go | 147 ++++++++++++++++++++
polygon/sync/execution_client.go | 14 ++
polygon/sync/sync.go | 46 ++++++-
polygon/sync/tip_events_test.go | 16 +++
8 files changed, 422 insertions(+), 43 deletions(-)
create mode 100644 polygon/p2p/publisher_test.go
diff --git a/eth/stagedsync/stage_polygon_sync.go b/eth/stagedsync/stage_polygon_sync.go
index 301ea4f555c..6b81d8b70a6 100644
--- a/eth/stagedsync/stage_polygon_sync.go
+++ b/eth/stagedsync/stage_polygon_sync.go
@@ -1393,6 +1393,23 @@ type polygonSyncStageExecutionEngine struct {
cachedForkChoice *polygonSyncStageForkChoice
}
+func (e *polygonSyncStageExecutionEngine) GetTd(ctx context.Context, blockNum uint64, blockHash common.Hash) (*big.Int, error) {
+ type response struct {
+ td *big.Int
+ err error
+ }
+
+ r, err := awaitTxAction(ctx, e.txActionStream, func(tx kv.RwTx, respond func(r response) error) error {
+ td, err := rawdb.ReadTd(tx, blockHash, blockNum)
+ return respond(response{td: td, err: err})
+ })
+ if err != nil {
+ return nil, err
+ }
+
+ return r.td, r.err
+}
+
func (e *polygonSyncStageExecutionEngine) GetHeader(ctx context.Context, blockNum uint64) (*types.Header, error) {
type response struct {
header *types.Header
diff --git a/polygon/p2p/publisher.go b/polygon/p2p/publisher.go
index bf99dab7ed3..8a856742ad9 100644
--- a/polygon/p2p/publisher.go
+++ b/polygon/p2p/publisher.go
@@ -1,77 +1,192 @@
+// Copyright 2024 The Erigon Authors
+// This file is part of Erigon.
+//
+// Erigon is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Lesser General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// Erigon is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Lesser General Public License for more details.
+//
+// You should have received a copy of the GNU Lesser General Public License
+// along with Erigon. If not, see .
+
package p2p
import (
"context"
+ "fmt"
+ "math"
+ "math/big"
+ "time"
+
+ "golang.org/x/sync/errgroup"
+ "github.com/erigontech/erigon-lib/log/v3"
"github.com/erigontech/erigon/core/types"
"github.com/erigontech/erigon/eth/protocols/eth"
)
type Publisher interface {
- PublishNewBlockHashes(ctx context.Context, block *types.Block) error
- PublishNewBlock(ctx context.Context, block *types.Block) error
+ PublishNewBlock(block *types.Block, td *big.Int)
+ PublishNewBlockHashes(block *types.Block)
+ Run(ctx context.Context) error
}
-func newPublisher(messageSender MessageSender) Publisher {
+func NewPublisher(messageSender MessageSender) Publisher {
return &publisher{
messageSender: messageSender,
+ tasks: make(chan publishTask, 100),
}
}
+// publisher manages block announcements according to the devp2p specs:
// https://github.com/ethereum/devp2p/blob/master/caps/eth.md#block-propagation
//
-// Rules:
-//
-// 1. When a NewBlock announcement message is received from a peer, the client first verifies the basic header validity
-// of the block, checking whether the proof-of-work value is valid. It then sends the block to a small fraction of
-// connected peers (usually the square root of the total number of peers) using the NewBlock message.
-//
-// 2. After the header validity check, the client imports the block into its local chain by executing all transactions
-// contained in the block, computing the block's 'post state'. The block's state-root hash must match the computed
-// post state root. Once the block is fully processed, and considered valid, the client sends a NewBlockHashes
-// message about the block to all peers which it didn't notify earlier. Those peers may request the full block later
-// if they fail to receive it via NewBlock from anyone else.
-//
-// NewBlockHashes (0x01)
-// [[blockhash₁: B_32, number₁: P], [blockhash₂: B_32, number₂: P], ...]
-//
-// Specify one or more new blocks which have appeared on the network. To be maximally helpful, nodes should inform
-// peers of all blocks that they may not be aware of. Including hashes that the sending peer could reasonably be
-// considered to know (due to the fact they were previously informed or because that node has itself advertised
-// knowledge of the hashes through NewBlockHashes) is considered bad form, and may reduce the reputation of the
-// sending node. Including hashes that the sending node later refuses to honour with a proceeding GetBlockHeaders
-// message is considered bad form, and may reduce the reputation of the sending node.
-//
-//
-// NewBlock (0x07)
-// [block, td: P]
+// It co-operates with the PeerTracker to ensure that we do not publish block/block hash announcements to peers if:
+// 1) we have already published a given block/block hash to this peer or
+// 2) that peer has already notified us of the given block/block hash
//
-// Specify a single complete block that the peer should know about. td is the total difficulty of the block, i.e. the
-// sum of all block difficulties up to and including this block.
-
+// It also handles the NewBlock publish requirement of only sending it to a small random portion (sqrt) of peers.
type publisher struct {
+ logger log.Logger
messageSender MessageSender
peerTracker PeerTracker
+ tasks chan publishTask
+}
+
+func (p publisher) PublishNewBlock(block *types.Block, td *big.Int) {
+ p.enqueueTask(publishTask{
+ taskType: newBlockPublishTask,
+ block: block,
+ td: td,
+ })
+}
+
+func (p publisher) PublishNewBlockHashes(block *types.Block) {
+ p.enqueueTask(publishTask{
+ taskType: newBlockHashesPublishTask,
+ block: block,
+ })
}
-func (p publisher) PublishNewBlockHashes(ctx context.Context, block *types.Block) error {
- hash := block.Hash()
- peers := p.peerTracker.ListPeersMayMissBlockHash(hash)
- blockHashesPacket := eth.NewBlockHashesPacket{
+func (p publisher) Run(ctx context.Context) error {
+ for {
+ select {
+ case <-ctx.Done():
+ return ctx.Err()
+ case t := <-p.tasks:
+ p.processPublishTask(ctx, t)
+ }
+ }
+}
+
+func (p publisher) enqueueTask(t publishTask) {
+ select {
+ case p.tasks <- t: // enqueued
+ default:
+ p.logger.Warn("[p2p-publisher] task queue is full, dropping message")
+ }
+}
+
+func (p publisher) processPublishTask(ctx context.Context, t publishTask) {
+ ctx, cancel := context.WithTimeout(ctx, time.Second)
+ defer cancel()
+
+ switch t.taskType {
+ case newBlockPublishTask:
+ p.processNewBlocksPublishTask(ctx, t)
+ case newBlockHashesPublishTask:
+ p.processNewBlockHashesPublishTask(ctx, t)
+ default:
+ panic(fmt.Sprintf("unknown task type: %v", t.taskType))
+ }
+}
+
+func (p publisher) processNewBlocksPublishTask(ctx context.Context, t publishTask) {
+ newBlockPacket := eth.NewBlockPacket{
+ Block: t.block,
+ TD: t.td,
+ }
+
+ peers := p.peerTracker.ListPeersMayMissBlockHash(t.block.Hash())
+ // devp2p spec: publish NewBlock to random sqrt(peers)
+ // note ListPeersMayMissBlockHash has already done the shuffling for us
+ peers = peers[:int(math.Sqrt(float64(len(peers))))]
+
+ eg := errgroup.Group{}
+ for _, peerId := range peers {
+ eg.Go(func() error {
+ // note underlying peer send message is async so this should finish quick
+ if err := p.messageSender.SendNewBlock(ctx, peerId, newBlockPacket); err != nil {
+ p.logger.Warn(
+ "[p2p-publisher] could not publish new block to peer",
+ "peerId", peerId,
+ "blockNum", t.block.NumberU64(),
+ "blockHash", t.block.Hash(),
+ "err", err,
+ )
+
+ return nil // best effort async publish
+ }
+
+ // devp2p spec: mark as known, so that we do not re-announce same block hash to same peer
+ p.peerTracker.BlockHashPresent(peerId, t.block.Hash())
+ return nil
+ })
+ }
+
+ _ = eg.Wait() // best effort async publish
+}
+
+func (p publisher) processNewBlockHashesPublishTask(ctx context.Context, t publishTask) {
+ blockHash := t.block.Hash()
+ blockNum := t.block.NumberU64()
+ newBlockHashesPacket := eth.NewBlockHashesPacket{
{
- Hash: block.Hash(),
- Number: block.NumberU64(),
+ Hash: blockHash,
+ Number: blockNum,
},
}
+ eg := errgroup.Group{}
+ peers := p.peerTracker.ListPeersMayMissBlockHash(blockHash)
for _, peerId := range peers {
- p.messageSender.SendNewBlockHashes(ctx, peerId, blockHashesPacket)
+ eg.Go(func() error {
+ // note underlying peer send message is async so this should finish quickly
+ if err := p.messageSender.SendNewBlockHashes(ctx, peerId, newBlockHashesPacket); err != nil {
+ p.logger.Warn(
+ "[p2p-publisher] could not publish new block hashes to peer",
+ "peerId", peerId,
+ "blockNum", blockNum,
+ "blockHash", blockHash,
+ "err", err,
+ )
+
+ return nil // best effort async publish
+ }
+
+ // devp2p spec: mark as known, so that we do not re-announce same block hash to same peer
+ p.peerTracker.BlockHashPresent(peerId, blockHash)
+ return nil
+ })
}
- return p.messageSender.SendNewBlockHashes(ctx, eth.NewBlockHashesPacket{})
+ _ = eg.Wait() // best effort async publish
}
-func (p publisher) PublishNewBlock(ctx context.Context, block *types.Block) error {
- //TODO implement me
- panic("implement me")
+type publishTask struct {
+ taskType publishTaskType
+ block *types.Block
+ td *big.Int
}
+
+type publishTaskType int
+
+const (
+ newBlockHashesPublishTask publishTaskType = iota
+ newBlockPublishTask
+)
diff --git a/polygon/p2p/publisher_test.go b/polygon/p2p/publisher_test.go
new file mode 100644
index 00000000000..03a28fd5556
--- /dev/null
+++ b/polygon/p2p/publisher_test.go
@@ -0,0 +1,21 @@
+// Copyright 2024 The Erigon Authors
+// This file is part of Erigon.
+//
+// Erigon is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Lesser General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// Erigon is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Lesser General Public License for more details.
+//
+// You should have received a copy of the GNU Lesser General Public License
+// along with Erigon. If not, see .
+
+package p2p
+
+//
+// TODO
+//
diff --git a/polygon/p2p/service.go b/polygon/p2p/service.go
index add6f3915d0..f595b34e945 100644
--- a/polygon/p2p/service.go
+++ b/polygon/p2p/service.go
@@ -33,6 +33,7 @@ type Service interface {
MessageListener
PeerTracker
PeerPenalizer
+ Publisher
Run(ctx context.Context) error
MaxPeers() int
}
@@ -61,11 +62,13 @@ func newService(
fetcher := NewFetcher(logger, fetcherConfig, messageListener, messageSender, requestIdGenerator)
fetcher = NewPenalizingFetcher(logger, fetcher, peerPenalizer)
fetcher = NewTrackingFetcher(fetcher, peerTracker)
+ publisher := NewPublisher(messageSender)
return &service{
Fetcher: fetcher,
MessageListener: messageListener,
PeerPenalizer: peerPenalizer,
PeerTracker: peerTracker,
+ Publisher: publisher,
maxPeers: maxPeers,
}
}
@@ -75,6 +78,7 @@ type service struct {
MessageListener
PeerPenalizer
PeerTracker
+ Publisher
maxPeers int
}
@@ -82,6 +86,7 @@ func (s *service) Run(ctx context.Context) error {
eg, ctx := errgroup.WithContext(ctx)
eg.Go(func() error { return s.MessageListener.Run(ctx) })
eg.Go(func() error { return s.PeerTracker.Run(ctx) })
+ eg.Go(func() error { return s.Publisher.Run(ctx) })
return eg.Wait()
}
diff --git a/polygon/p2p/service_mock.go b/polygon/p2p/service_mock.go
index 39b6f7e7ff4..6598ee63a51 100644
--- a/polygon/p2p/service_mock.go
+++ b/polygon/p2p/service_mock.go
@@ -11,6 +11,7 @@ package p2p
import (
context "context"
+ big "math/big"
reflect "reflect"
common "github.com/erigontech/erigon-lib/common"
@@ -44,6 +45,42 @@ func (m *MockService) EXPECT() *MockServiceMockRecorder {
return m.recorder
}
+// BlockHashPresent mocks base method.
+func (m *MockService) BlockHashPresent(peerId *PeerId, blockHash common.Hash) {
+ m.ctrl.T.Helper()
+ m.ctrl.Call(m, "BlockHashPresent", peerId, blockHash)
+}
+
+// BlockHashPresent indicates an expected call of BlockHashPresent.
+func (mr *MockServiceMockRecorder) BlockHashPresent(peerId, blockHash any) *MockServiceBlockHashPresentCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BlockHashPresent", reflect.TypeOf((*MockService)(nil).BlockHashPresent), peerId, blockHash)
+ return &MockServiceBlockHashPresentCall{Call: call}
+}
+
+// MockServiceBlockHashPresentCall wrap *gomock.Call
+type MockServiceBlockHashPresentCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockServiceBlockHashPresentCall) Return() *MockServiceBlockHashPresentCall {
+ c.Call = c.Call.Return()
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockServiceBlockHashPresentCall) Do(f func(*PeerId, common.Hash)) *MockServiceBlockHashPresentCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockServiceBlockHashPresentCall) DoAndReturn(f func(*PeerId, common.Hash)) *MockServiceBlockHashPresentCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
// BlockNumMissing mocks base method.
func (m *MockService) BlockNumMissing(peerId *PeerId, blockNum uint64) {
m.ctrl.T.Helper()
@@ -286,6 +323,44 @@ func (c *MockServiceListPeersMayHaveBlockNumCall) DoAndReturn(f func(uint64) []*
return c
}
+// ListPeersMayMissBlockHash mocks base method.
+func (m *MockService) ListPeersMayMissBlockHash(blockHash common.Hash) []*PeerId {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "ListPeersMayMissBlockHash", blockHash)
+ ret0, _ := ret[0].([]*PeerId)
+ return ret0
+}
+
+// ListPeersMayMissBlockHash indicates an expected call of ListPeersMayMissBlockHash.
+func (mr *MockServiceMockRecorder) ListPeersMayMissBlockHash(blockHash any) *MockServiceListPeersMayMissBlockHashCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListPeersMayMissBlockHash", reflect.TypeOf((*MockService)(nil).ListPeersMayMissBlockHash), blockHash)
+ return &MockServiceListPeersMayMissBlockHashCall{Call: call}
+}
+
+// MockServiceListPeersMayMissBlockHashCall wrap *gomock.Call
+type MockServiceListPeersMayMissBlockHashCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockServiceListPeersMayMissBlockHashCall) Return(arg0 []*PeerId) *MockServiceListPeersMayMissBlockHashCall {
+ c.Call = c.Call.Return(arg0)
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockServiceListPeersMayMissBlockHashCall) Do(f func(common.Hash) []*PeerId) *MockServiceListPeersMayMissBlockHashCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockServiceListPeersMayMissBlockHashCall) DoAndReturn(f func(common.Hash) []*PeerId) *MockServiceListPeersMayMissBlockHashCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
// MaxPeers mocks base method.
func (m *MockService) MaxPeers() int {
m.ctrl.T.Helper()
@@ -434,6 +509,78 @@ func (c *MockServicePenalizeCall) DoAndReturn(f func(context.Context, *PeerId) e
return c
}
+// PublishNewBlock mocks base method.
+func (m *MockService) PublishNewBlock(block *types.Block, td *big.Int) {
+ m.ctrl.T.Helper()
+ m.ctrl.Call(m, "PublishNewBlock", block, td)
+}
+
+// PublishNewBlock indicates an expected call of PublishNewBlock.
+func (mr *MockServiceMockRecorder) PublishNewBlock(block, td any) *MockServicePublishNewBlockCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PublishNewBlock", reflect.TypeOf((*MockService)(nil).PublishNewBlock), block, td)
+ return &MockServicePublishNewBlockCall{Call: call}
+}
+
+// MockServicePublishNewBlockCall wrap *gomock.Call
+type MockServicePublishNewBlockCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockServicePublishNewBlockCall) Return() *MockServicePublishNewBlockCall {
+ c.Call = c.Call.Return()
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockServicePublishNewBlockCall) Do(f func(*types.Block, *big.Int)) *MockServicePublishNewBlockCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockServicePublishNewBlockCall) DoAndReturn(f func(*types.Block, *big.Int)) *MockServicePublishNewBlockCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
+// PublishNewBlockHashes mocks base method.
+func (m *MockService) PublishNewBlockHashes(block *types.Block) {
+ m.ctrl.T.Helper()
+ m.ctrl.Call(m, "PublishNewBlockHashes", block)
+}
+
+// PublishNewBlockHashes indicates an expected call of PublishNewBlockHashes.
+func (mr *MockServiceMockRecorder) PublishNewBlockHashes(block any) *MockServicePublishNewBlockHashesCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PublishNewBlockHashes", reflect.TypeOf((*MockService)(nil).PublishNewBlockHashes), block)
+ return &MockServicePublishNewBlockHashesCall{Call: call}
+}
+
+// MockServicePublishNewBlockHashesCall wrap *gomock.Call
+type MockServicePublishNewBlockHashesCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockServicePublishNewBlockHashesCall) Return() *MockServicePublishNewBlockHashesCall {
+ c.Call = c.Call.Return()
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockServicePublishNewBlockHashesCall) Do(f func(*types.Block)) *MockServicePublishNewBlockHashesCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockServicePublishNewBlockHashesCall) DoAndReturn(f func(*types.Block)) *MockServicePublishNewBlockHashesCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
// RegisterBlockBodiesObserver mocks base method.
func (m *MockService) RegisterBlockBodiesObserver(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.BlockBodiesPacket66]]) UnregisterFunc {
m.ctrl.T.Helper()
diff --git a/polygon/sync/execution_client.go b/polygon/sync/execution_client.go
index 5a0ccde1363..60958103fbc 100644
--- a/polygon/sync/execution_client.go
+++ b/polygon/sync/execution_client.go
@@ -20,6 +20,7 @@ import (
"context"
"errors"
"fmt"
+ "math/big"
"time"
"google.golang.org/protobuf/types/known/emptypb"
@@ -38,6 +39,7 @@ type ExecutionClient interface {
UpdateForkChoice(ctx context.Context, tip *types.Header, finalizedHeader *types.Header) (common.Hash, error)
CurrentHeader(ctx context.Context) (*types.Header, error)
GetHeader(ctx context.Context, blockNum uint64) (*types.Header, error)
+ GetTd(ctx context.Context, blockNum uint64, blockHash common.Hash) (*big.Int, error)
}
type executionClient struct {
@@ -143,3 +145,15 @@ func (e *executionClient) GetHeader(ctx context.Context, blockNum uint64) (*type
return header, nil
}
+
+func (e *executionClient) GetTd(ctx context.Context, blockNum uint64, blockHash common.Hash) (*big.Int, error) {
+ response, err := e.client.GetTD(ctx, &executionproto.GetSegmentRequest{
+ BlockNumber: &blockNum,
+ BlockHash: gointerfaces.ConvertHashToH256(blockHash),
+ })
+ if err != nil {
+ return nil, err
+ }
+
+ return eth1utils.ConvertBigIntFromRpc(response.GetTd()), nil
+}
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index e530a51bfad..e8f549669a3 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -290,6 +290,18 @@ func (s *Sync) applyNewBlockOnTip(
// len(newConnectedHeaders) is always <= len(blockChain)
newConnectedBlocks := blockChain[len(blockChain)-len(newConnectedHeaders):]
+
+ if event.Source == EventSourceP2PNewBlock {
+ // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#block-propagation
+ // devp2p spec: when a NewBlock announcement message is received from a peer, the client first verifies the
+ // basic header validity of the block, checking whether the proof-of-work value is valid (replace PoW
+ // with Bor rules that we do as part of CanonicalChainBuilder.Connect).
+ // It then sends the block to a small fraction of connected peers (usually the square root of the total
+ // number of peers) using the NewBlock message.
+ // note, below is non-blocking
+ go s.publishNewBlock(ctx, event.NewBlock)
+ }
+
if err := s.store.InsertBlocks(ctx, newConnectedBlocks); err != nil {
return err
}
@@ -298,7 +310,24 @@ func (s *Sync) applyNewBlockOnTip(
return nil
}
- return s.commitExecution(ctx, newTip, ccBuilder.Root())
+ if err := s.commitExecution(ctx, newTip, ccBuilder.Root()); err != nil {
+ return err
+ }
+
+ if event.Source == EventSourceP2PNewBlock {
+ // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#block-propagation
+ // devp2p spec: After the header validity check, the client imports the block into its local chain by executing
+ // all transactions contained in the block, computing the block's 'post state'. The block's state-root hash
+ // must match the computed post state root. Once the block is fully processed, and considered valid,
+ // the client sends a NewBlockHashes message about the block to all peers which it didn't notify earlier.
+ // Those peers may request the full block later if they fail to receive it via NewBlock from anyone else.
+ // Including hashes that the sending node later refuses to honour with a proceeding GetBlockHeaders
+ // message is considered bad form, and may reduce the reputation of the sending node.
+ // note, below is non-blocking
+ s.p2pService.PublishNewBlockHashes(event.NewBlock)
+ }
+
+ return nil
}
func (s *Sync) applyNewBlockHashesOnTip(
@@ -348,6 +377,21 @@ func (s *Sync) applyNewBlockHashesOnTip(
return nil
}
+func (s *Sync) publishNewBlock(ctx context.Context, block *types.Block) {
+ td, err := s.execution.GetTd(ctx, block.NumberU64(), block.Hash())
+ if err != nil {
+ s.logger.Warn(syncLogPrefix("could not fetch td when publishing new block"),
+ "blockNum", block.NumberU64(),
+ "blockHash", block.Hash(),
+ "err", err,
+ )
+
+ return
+ }
+
+ s.p2pService.PublishNewBlock(block, td)
+}
+
//
// TODO (subsequent PRs) - unit test initial sync + on new event cases
//
diff --git a/polygon/sync/tip_events_test.go b/polygon/sync/tip_events_test.go
index 412bee36c66..0d025ecc3e9 100644
--- a/polygon/sync/tip_events_test.go
+++ b/polygon/sync/tip_events_test.go
@@ -1,3 +1,19 @@
+// Copyright 2024 The Erigon Authors
+// This file is part of Erigon.
+//
+// Erigon is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Lesser General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// Erigon is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Lesser General Public License for more details.
+//
+// You should have received a copy of the GNU Lesser General Public License
+// along with Erigon. If not, see .
+
package sync
import (
From 39c42b849f928734bfba8626e912144379386a04 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 10 Oct 2024 22:33:53 +0100
Subject: [PATCH 03/25] add todo
---
polygon/p2p/peer_tracker_test.go | 4 ++++
1 file changed, 4 insertions(+)
diff --git a/polygon/p2p/peer_tracker_test.go b/polygon/p2p/peer_tracker_test.go
index 8b16b8ad74f..ae0a5026a54 100644
--- a/polygon/p2p/peer_tracker_test.go
+++ b/polygon/p2p/peer_tracker_test.go
@@ -124,6 +124,10 @@ func TestPeerTrackerPeerEventObserver(t *testing.T) {
})
}
+//
+// TODO test block hashes
+//
+
func newPeerTrackerTest(t *testing.T) *peerTrackerTest {
ctx, cancel := context.WithCancel(context.Background())
logger := testlog.Logger(t, log.LvlCrit)
From 510546f3e9bffa79d93356e2cc5d4f2013769d95 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 10 Oct 2024 22:38:33 +0100
Subject: [PATCH 04/25] move publishNewBlock after InsertBlocks for TD
availability
---
polygon/sync/sync.go | 7 +++----
1 file changed, 3 insertions(+), 4 deletions(-)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index e8f549669a3..f16eef31676 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -290,6 +290,9 @@ func (s *Sync) applyNewBlockOnTip(
// len(newConnectedHeaders) is always <= len(blockChain)
newConnectedBlocks := blockChain[len(blockChain)-len(newConnectedHeaders):]
+ if err := s.store.InsertBlocks(ctx, newConnectedBlocks); err != nil {
+ return err
+ }
if event.Source == EventSourceP2PNewBlock {
// https://github.com/ethereum/devp2p/blob/master/caps/eth.md#block-propagation
@@ -302,10 +305,6 @@ func (s *Sync) applyNewBlockOnTip(
go s.publishNewBlock(ctx, event.NewBlock)
}
- if err := s.store.InsertBlocks(ctx, newConnectedBlocks); err != nil {
- return err
- }
-
if newTip == oldTip {
return nil
}
From 49feec0944692403e8543c32f051d9e0a5aabd1e Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 10 Oct 2024 22:44:26 +0100
Subject: [PATCH 05/25] tidy
---
polygon/p2p/publisher.go | 12 ++++++++++--
1 file changed, 10 insertions(+), 2 deletions(-)
diff --git a/polygon/p2p/publisher.go b/polygon/p2p/publisher.go
index 8a856742ad9..3179b6d432e 100644
--- a/polygon/p2p/publisher.go
+++ b/polygon/p2p/publisher.go
@@ -39,7 +39,7 @@ type Publisher interface {
func NewPublisher(messageSender MessageSender) Publisher {
return &publisher{
messageSender: messageSender,
- tasks: make(chan publishTask, 100),
+ tasks: make(chan publishTask, 1024),
}
}
@@ -51,6 +51,9 @@ func NewPublisher(messageSender MessageSender) Publisher {
// 2) that peer has already notified us of the given block/block hash
//
// It also handles the NewBlock publish requirement of only sending it to a small random portion (sqrt) of peers.
+//
+// All publish tasks are done asynchronously by putting them on a queue. If the publisher is struggling to keep up
+// then newly enqueued publish tasks will get dropped.
type publisher struct {
logger log.Logger
messageSender MessageSender
@@ -88,7 +91,12 @@ func (p publisher) enqueueTask(t publishTask) {
select {
case p.tasks <- t: // enqueued
default:
- p.logger.Warn("[p2p-publisher] task queue is full, dropping message")
+ p.logger.Warn(
+ "[p2p-publisher] task queue is full, dropping task",
+ "blockNumber", t.block.Number(),
+ "blockHash", t.block.Hash(),
+ "taskType", t.taskType,
+ )
}
}
From 8bc1543beaf622cc4dbf3d4d426e143c19d8d231 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Mon, 14 Oct 2024 20:26:08 +0100
Subject: [PATCH 06/25] add peer_tracker_tests
---
polygon/p2p/peer_tracker.go | 11 +-
polygon/p2p/peer_tracker_test.go | 201 +++++++++++++++++++++++++++++--
2 files changed, 196 insertions(+), 16 deletions(-)
diff --git a/polygon/p2p/peer_tracker.go b/polygon/p2p/peer_tracker.go
index 27bd777a524..a702fdcf2b9 100644
--- a/polygon/p2p/peer_tracker.go
+++ b/polygon/p2p/peer_tracker.go
@@ -48,11 +48,12 @@ func NewPeerTracker(
opts ...PeerTrackerOption,
) PeerTracker {
pt := &peerTracker{
- logger: logger,
- peerProvider: peerProvider,
- peerEventRegistrar: peerEventRegistrar,
- peerSyncProgresses: map[PeerId]*peerSyncProgress{},
- peerShuffle: RandPeerShuffle,
+ logger: logger,
+ peerProvider: peerProvider,
+ peerEventRegistrar: peerEventRegistrar,
+ peerSyncProgresses: map[PeerId]*peerSyncProgress{},
+ peerKnownBlockAnnounces: map[PeerId]simplelru.LRUCache[common.Hash, struct{}]{},
+ peerShuffle: RandPeerShuffle,
}
for _, opt := range opts {
diff --git a/polygon/p2p/peer_tracker_test.go b/polygon/p2p/peer_tracker_test.go
index ae0a5026a54..7f677d5bdc4 100644
--- a/polygon/p2p/peer_tracker_test.go
+++ b/polygon/p2p/peer_tracker_test.go
@@ -19,6 +19,7 @@ package p2p
import (
"context"
"encoding/binary"
+ "math/big"
"sort"
"sync/atomic"
"testing"
@@ -27,9 +28,12 @@ import (
"github.com/stretchr/testify/require"
"go.uber.org/mock/gomock"
+ libcommon "github.com/erigontech/erigon-lib/common"
"github.com/erigontech/erigon-lib/gointerfaces/sentryproto"
"github.com/erigontech/erigon-lib/gointerfaces/typesproto"
"github.com/erigontech/erigon-lib/log/v3"
+ "github.com/erigontech/erigon/core/types"
+ "github.com/erigontech/erigon/eth/protocols/eth"
"github.com/erigontech/erigon/polygon/polygoncommon"
"github.com/erigontech/erigon/turbo/testlog"
)
@@ -66,6 +70,22 @@ func TestPeerTracker(t *testing.T) {
peerIds = peerTracker.ListPeersMayHaveBlockNum(100)
require.Len(t, peerIds, 1)
require.Equal(t, PeerIdFromUint64(1), peerIds[0])
+
+ peerTracker.PeerConnected(PeerIdFromUint64(2))
+ peerIds = peerTracker.ListPeersMayMissBlockHash(libcommon.HexToHash("0x0"))
+ require.Len(t, peerIds, 2)
+ sortPeerIdsAssumingUints(peerIds)
+ require.Equal(t, PeerIdFromUint64(1), peerIds[0])
+ require.Equal(t, PeerIdFromUint64(2), peerIds[1])
+
+ peerTracker.BlockHashPresent(PeerIdFromUint64(2), libcommon.HexToHash("0x0"))
+ peerIds = peerTracker.ListPeersMayMissBlockHash(libcommon.HexToHash("0x0"))
+ require.Len(t, peerIds, 1)
+ require.Equal(t, PeerIdFromUint64(1), peerIds[0])
+
+ peerTracker.BlockHashPresent(PeerIdFromUint64(1), libcommon.HexToHash("0x0"))
+ peerIds = peerTracker.ListPeersMayMissBlockHash(libcommon.HexToHash("0x0"))
+ require.Len(t, peerIds, 0)
}
func TestPeerTrackerPeerEventObserver(t *testing.T) {
@@ -75,6 +95,8 @@ func TestPeerTrackerPeerEventObserver(t *testing.T) {
alreadyConnectedPeerId, err := PeerIdFromEnode(alreadyConnectedPeerEnode)
require.NoError(t, err)
peerEventsStream := make(chan *sentryproto.PeerEvent)
+ newBlockHashesStream := make(chan *DecodedInboundMessage[*eth.NewBlockHashesPacket])
+ newBlocksStream := make(chan *DecodedInboundMessage[*eth.NewBlockPacket])
test := newPeerTrackerTest(t)
test.mockPeerProvider(&sentryproto.PeersReply{
Peers: []*typesproto.PeerInfo{
@@ -84,17 +106,19 @@ func TestPeerTrackerPeerEventObserver(t *testing.T) {
},
})
test.mockPeerEvents(peerEventsStream)
+ test.mockNewBlockHashesEvents(newBlockHashesStream)
+ test.mockNewBlockEvents(newBlocksStream)
peerTracker := test.peerTracker
test.run(func(ctx context.Context, t *testing.T) {
- peerEventsStream <- &sentryproto.PeerEvent{
+ send(ctx, t, peerEventsStream, &sentryproto.PeerEvent{
PeerId: PeerIdFromUint64(1).H512(),
EventId: sentryproto.PeerEvent_Connect,
- }
+ })
- peerEventsStream <- &sentryproto.PeerEvent{
+ send(ctx, t, peerEventsStream, &sentryproto.PeerEvent{
PeerId: PeerIdFromUint64(2).H512(),
EventId: sentryproto.PeerEvent_Connect,
- }
+ })
var peerIds []*PeerId
waitCond := func(wantPeerIdsLen int) func() bool {
@@ -110,10 +134,10 @@ func TestPeerTrackerPeerEventObserver(t *testing.T) {
require.Equal(t, PeerIdFromUint64(2), peerIds[1])
require.Equal(t, alreadyConnectedPeerId, peerIds[2])
- peerEventsStream <- &sentryproto.PeerEvent{
+ send(ctx, t, peerEventsStream, &sentryproto.PeerEvent{
PeerId: PeerIdFromUint64(1).H512(),
EventId: sentryproto.PeerEvent_Disconnect,
- }
+ })
peerIds = peerTracker.ListPeersMayHaveBlockNum(100)
require.Eventually(t, waitCond(2), time.Second, 5*time.Millisecond)
@@ -124,9 +148,107 @@ func TestPeerTrackerPeerEventObserver(t *testing.T) {
})
}
-//
-// TODO test block hashes
-//
+func TestPeerTrackerNewBlockHashesObserver(t *testing.T) {
+ t.Parallel()
+
+ peerEventsStream := make(chan *sentryproto.PeerEvent)
+ newBlockHashesStream := make(chan *DecodedInboundMessage[*eth.NewBlockHashesPacket])
+ newBlocksStream := make(chan *DecodedInboundMessage[*eth.NewBlockPacket])
+ test := newPeerTrackerTest(t)
+ test.mockPeerProvider(&sentryproto.PeersReply{})
+ test.mockPeerEvents(peerEventsStream)
+ test.mockNewBlockHashesEvents(newBlockHashesStream)
+ test.mockNewBlockEvents(newBlocksStream)
+ peerTracker := test.peerTracker
+ test.run(func(ctx context.Context, t *testing.T) {
+ send(ctx, t, peerEventsStream, &sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(1).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+
+ send(ctx, t, peerEventsStream, &sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(2).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+
+ var peerIds []*PeerId
+ waitCond := func(wantPeerIdsLen int) func() bool {
+ return func() bool {
+ peerIds = peerTracker.ListPeersMayMissBlockHash(libcommon.HexToHash("0x0"))
+ return len(peerIds) == wantPeerIdsLen
+ }
+ }
+ require.Eventually(t, waitCond(2), time.Second, 5*time.Millisecond)
+ require.Len(t, peerIds, 2)
+ sortPeerIdsAssumingUints(peerIds)
+ require.Equal(t, PeerIdFromUint64(1), peerIds[0])
+ require.Equal(t, PeerIdFromUint64(2), peerIds[1])
+
+ send(ctx, t, newBlockHashesStream, &DecodedInboundMessage[*eth.NewBlockHashesPacket]{
+ PeerId: PeerIdFromUint64(2),
+ Decoded: ð.NewBlockHashesPacket{
+ {
+ Hash: libcommon.HexToHash("0x0"),
+ Number: 1,
+ },
+ },
+ })
+
+ require.Eventually(t, waitCond(1), time.Second, 5*time.Millisecond)
+ require.Len(t, peerIds, 1)
+ require.Equal(t, PeerIdFromUint64(1), peerIds[0])
+ })
+}
+
+func TestPeerTrackerNewBlocksObserver(t *testing.T) {
+ t.Parallel()
+
+ peerEventsStream := make(chan *sentryproto.PeerEvent)
+ newBlockHashesStream := make(chan *DecodedInboundMessage[*eth.NewBlockHashesPacket])
+ newBlocksStream := make(chan *DecodedInboundMessage[*eth.NewBlockPacket])
+ test := newPeerTrackerTest(t)
+ test.mockPeerProvider(&sentryproto.PeersReply{})
+ test.mockPeerEvents(peerEventsStream)
+ test.mockNewBlockHashesEvents(newBlockHashesStream)
+ test.mockNewBlockEvents(newBlocksStream)
+ peerTracker := test.peerTracker
+ test.run(func(ctx context.Context, t *testing.T) {
+ send(ctx, t, peerEventsStream, &sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(1).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+
+ send(ctx, t, peerEventsStream, &sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(2).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+
+ header := &types.Header{Number: big.NewInt(123)}
+ var peerIds []*PeerId
+ waitCond := func(wantPeerIdsLen int) func() bool {
+ return func() bool {
+ peerIds = peerTracker.ListPeersMayMissBlockHash(header.Hash())
+ return len(peerIds) == wantPeerIdsLen
+ }
+ }
+ require.Eventually(t, waitCond(2), time.Second, 5*time.Millisecond)
+ require.Len(t, peerIds, 2)
+ sortPeerIdsAssumingUints(peerIds)
+ require.Equal(t, PeerIdFromUint64(1), peerIds[0])
+ require.Equal(t, PeerIdFromUint64(2), peerIds[1])
+
+ send(ctx, t, newBlocksStream, &DecodedInboundMessage[*eth.NewBlockPacket]{
+ PeerId: PeerIdFromUint64(2),
+ Decoded: ð.NewBlockPacket{
+ Block: types.NewBlockWithHeader(header),
+ },
+ })
+
+ require.Eventually(t, waitCond(1), time.Second, 5*time.Millisecond)
+ require.Len(t, peerIds, 1)
+ require.Equal(t, PeerIdFromUint64(1), peerIds[0])
+ })
+}
func newPeerTrackerTest(t *testing.T) *peerTrackerTest {
ctx, cancel := context.WithCancel(context.Background())
@@ -161,7 +283,7 @@ func (ptt *peerTrackerTest) mockPeerProvider(peerReply *sentryproto.PeersReply)
Times(1)
}
-func (ptt *peerTrackerTest) mockPeerEvents(eventStream <-chan *sentryproto.PeerEvent) {
+func (ptt *peerTrackerTest) mockPeerEvents(events <-chan *sentryproto.PeerEvent) {
ptt.peerEventRegistrar.EXPECT().
RegisterPeerEventObserver(gomock.Any()).
DoAndReturn(func(observer polygoncommon.Observer[*sentryproto.PeerEvent]) UnregisterFunc {
@@ -171,7 +293,7 @@ func (ptt *peerTrackerTest) mockPeerEvents(eventStream <-chan *sentryproto.PeerE
select {
case <-ctx.Done():
return
- case event := <-eventStream:
+ case event := <-events:
observer(event)
}
}
@@ -182,6 +304,52 @@ func (ptt *peerTrackerTest) mockPeerEvents(eventStream <-chan *sentryproto.PeerE
Times(1)
}
+func (ptt *peerTrackerTest) mockNewBlockHashesEvents(events <-chan *DecodedInboundMessage[*eth.NewBlockHashesPacket]) {
+ ptt.peerEventRegistrar.EXPECT().
+ RegisterNewBlockHashesObserver(gomock.Any()).
+ DoAndReturn(
+ func(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockHashesPacket]]) UnregisterFunc {
+ ctx, cancel := context.WithCancel(context.Background())
+ go func() {
+ for {
+ select {
+ case <-ctx.Done():
+ return
+ case event := <-events:
+ observer(event)
+ }
+ }
+ }()
+
+ return UnregisterFunc(cancel)
+ },
+ ).
+ Times(1)
+}
+
+func (ptt *peerTrackerTest) mockNewBlockEvents(events <-chan *DecodedInboundMessage[*eth.NewBlockPacket]) {
+ ptt.peerEventRegistrar.EXPECT().
+ RegisterNewBlockObserver(gomock.Any()).
+ DoAndReturn(
+ func(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockPacket]]) UnregisterFunc {
+ ctx, cancel := context.WithCancel(context.Background())
+ go func() {
+ for {
+ select {
+ case <-ctx.Done():
+ return
+ case event := <-events:
+ observer(event)
+ }
+ }
+ }()
+
+ return UnregisterFunc(cancel)
+ },
+ ).
+ Times(1)
+}
+
func (ptt *peerTrackerTest) run(f func(ctx context.Context, t *testing.T)) {
var done atomic.Bool
ptt.t.Run("start", func(t *testing.T) {
@@ -214,3 +382,14 @@ func sortPeerIdsAssumingUints(peerIds []*PeerId) {
return numI < numJ
})
}
+
+func send[T any](ctx context.Context, t *testing.T, ch chan T, e T) {
+ ctx, cancel := context.WithTimeout(ctx, time.Second)
+ defer cancel()
+
+ select {
+ case <-ctx.Done():
+ require.FailNow(t, "send timed out")
+ case ch <- e: // no-op
+ }
+}
From c75cb274cd7949b78159ba2b49e652c71be274e8 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Mon, 14 Oct 2024 20:41:10 +0100
Subject: [PATCH 07/25] add message_sender_test
---
polygon/p2p/message_sender_test.go | 103 +++++++++++++++++++++++++++++
1 file changed, 103 insertions(+)
diff --git a/polygon/p2p/message_sender_test.go b/polygon/p2p/message_sender_test.go
index 54ca6c5daa3..a560d1cd0e1 100644
--- a/polygon/p2p/message_sender_test.go
+++ b/polygon/p2p/message_sender_test.go
@@ -18,6 +18,7 @@ package p2p
import (
"context"
+ "math/big"
"testing"
"github.com/stretchr/testify/require"
@@ -28,6 +29,7 @@ import (
"github.com/erigontech/erigon-lib/direct"
sentry "github.com/erigontech/erigon-lib/gointerfaces/sentryproto"
erigonlibtypes "github.com/erigontech/erigon-lib/gointerfaces/typesproto"
+ "github.com/erigontech/erigon/core/types"
"github.com/erigontech/erigon/eth/protocols/eth"
"github.com/erigontech/erigon/rlp"
)
@@ -136,3 +138,104 @@ func TestMessageSenderSendGetBlockBodiesErrPeerNotFound(t *testing.T) {
})
require.ErrorIs(t, err, ErrPeerNotFound)
}
+
+func TestMessageSenderSendNewBlockHashes(t *testing.T) {
+ ctx := context.Background()
+ ctrl := gomock.NewController(t)
+ sentryClient := direct.NewMockSentryClient(ctrl)
+ sentryClient.EXPECT().
+ SendMessageById(gomock.Any(), gomock.Any(), gomock.Any()).
+ DoAndReturn(func(_ context.Context, request *sentry.SendMessageByIdRequest, _ ...grpc.CallOption) (*sentry.SentPeers, error) {
+ require.Equal(t, PeerIdFromUint64(123), PeerIdFromH512(request.PeerId))
+ require.Equal(t, sentry.MessageId_NEW_BLOCK_HASHES_66, request.Data.Id)
+ var payload eth.NewBlockHashesPacket
+ err := rlp.DecodeBytes(request.Data.Data, &payload)
+ require.NoError(t, err)
+ require.Len(t, payload, 1)
+ require.Equal(t, uint64(1), payload[0].Number)
+ require.Equal(t, common.HexToHash("0x0"), payload[0].Hash)
+ return &sentry.SentPeers{
+ Peers: []*erigonlibtypes.H512{
+ PeerIdFromUint64(123).H512(),
+ },
+ }, nil
+ }).
+ Times(1)
+
+ messageSender := NewMessageSender(sentryClient)
+ err := messageSender.SendNewBlockHashes(ctx, PeerIdFromUint64(123), eth.NewBlockHashesPacket{
+ {
+ Hash: common.HexToHash("0x0"),
+ Number: 1,
+ },
+ })
+ require.NoError(t, err)
+}
+
+func TestMessageSenderSendNewBlockHashesErrPeerNotFound(t *testing.T) {
+ ctx := context.Background()
+ ctrl := gomock.NewController(t)
+ sentryClient := direct.NewMockSentryClient(ctrl)
+ sentryClient.EXPECT().
+ SendMessageById(gomock.Any(), gomock.Any(), gomock.Any()).
+ Return(&sentry.SentPeers{}, nil).
+ Times(1)
+
+ messageSender := NewMessageSender(sentryClient)
+ err := messageSender.SendNewBlockHashes(ctx, PeerIdFromUint64(123), eth.NewBlockHashesPacket{
+ {
+ Hash: common.HexToHash("0x0"),
+ Number: 1,
+ },
+ })
+ require.ErrorIs(t, err, ErrPeerNotFound)
+}
+
+func TestMessageSenderSendNewBlock(t *testing.T) {
+ ctx := context.Background()
+ ctrl := gomock.NewController(t)
+ header := &types.Header{Number: big.NewInt(123)}
+ sentryClient := direct.NewMockSentryClient(ctrl)
+ sentryClient.EXPECT().
+ SendMessageById(gomock.Any(), gomock.Any(), gomock.Any()).
+ DoAndReturn(func(_ context.Context, request *sentry.SendMessageByIdRequest, _ ...grpc.CallOption) (*sentry.SentPeers, error) {
+ require.Equal(t, PeerIdFromUint64(123), PeerIdFromH512(request.PeerId))
+ require.Equal(t, sentry.MessageId_NEW_BLOCK_66, request.Data.Id)
+ var payload eth.NewBlockPacket
+ err := rlp.DecodeBytes(request.Data.Data, &payload)
+ require.NoError(t, err)
+ require.Equal(t, uint64(123), payload.Block.NumberU64())
+ require.Equal(t, uint64(2), payload.TD.Uint64())
+ return &sentry.SentPeers{
+ Peers: []*erigonlibtypes.H512{
+ PeerIdFromUint64(123).H512(),
+ },
+ }, nil
+ }).
+ Times(1)
+
+ messageSender := NewMessageSender(sentryClient)
+ err := messageSender.SendNewBlock(ctx, PeerIdFromUint64(123), eth.NewBlockPacket{
+ Block: types.NewBlockWithHeader(header),
+ TD: big.NewInt(2),
+ })
+ require.NoError(t, err)
+}
+
+func TestMessageSenderSendNewBlockErrPeerNotFound(t *testing.T) {
+ ctx := context.Background()
+ ctrl := gomock.NewController(t)
+ header := &types.Header{Number: big.NewInt(123)}
+ sentryClient := direct.NewMockSentryClient(ctrl)
+ sentryClient.EXPECT().
+ SendMessageById(gomock.Any(), gomock.Any(), gomock.Any()).
+ Return(&sentry.SentPeers{}, nil).
+ Times(1)
+
+ messageSender := NewMessageSender(sentryClient)
+ err := messageSender.SendNewBlock(ctx, PeerIdFromUint64(123), eth.NewBlockPacket{
+ Block: types.NewBlockWithHeader(header),
+ TD: big.NewInt(2),
+ })
+ require.ErrorIs(t, err, ErrPeerNotFound)
+}
From 055514c08ba87ed9740729d009a48f0707b123df Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Mon, 14 Oct 2024 22:54:36 +0100
Subject: [PATCH 08/25] add publisher tests
---
polygon/p2p/publisher.go | 7 +-
polygon/p2p/publisher_test.go | 342 +++++++++++++++++++++++++++++++++-
polygon/p2p/service.go | 2 +-
polygon/sync/tip_events.go | 12 +-
4 files changed, 351 insertions(+), 12 deletions(-)
diff --git a/polygon/p2p/publisher.go b/polygon/p2p/publisher.go
index 3179b6d432e..e8781e035e0 100644
--- a/polygon/p2p/publisher.go
+++ b/polygon/p2p/publisher.go
@@ -36,9 +36,11 @@ type Publisher interface {
Run(ctx context.Context) error
}
-func NewPublisher(messageSender MessageSender) Publisher {
- return &publisher{
+func NewPublisher(logger log.Logger, messageSender MessageSender, peerTracker PeerTracker) Publisher {
+ return publisher{
+ logger: logger,
messageSender: messageSender,
+ peerTracker: peerTracker,
tasks: make(chan publishTask, 1024),
}
}
@@ -77,6 +79,7 @@ func (p publisher) PublishNewBlockHashes(block *types.Block) {
}
func (p publisher) Run(ctx context.Context) error {
+ p.logger.Debug("[p2p-publisher] running publisher")
for {
select {
case <-ctx.Done():
diff --git a/polygon/p2p/publisher_test.go b/polygon/p2p/publisher_test.go
index 03a28fd5556..bd1daef3e54 100644
--- a/polygon/p2p/publisher_test.go
+++ b/polygon/p2p/publisher_test.go
@@ -16,6 +16,342 @@
package p2p
-//
-// TODO
-//
+import (
+ "context"
+ "math/big"
+ "sync"
+ "sync/atomic"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/require"
+ "go.uber.org/mock/gomock"
+ "golang.org/x/exp/maps"
+ "golang.org/x/sync/errgroup"
+ "google.golang.org/grpc"
+
+ "github.com/erigontech/erigon-lib/direct"
+ "github.com/erigontech/erigon-lib/gointerfaces/sentryproto"
+ "github.com/erigontech/erigon-lib/gointerfaces/typesproto"
+ "github.com/erigontech/erigon-lib/log/v3"
+ "github.com/erigontech/erigon/core/types"
+ "github.com/erigontech/erigon/eth/protocols/eth"
+ "github.com/erigontech/erigon/polygon/polygoncommon"
+ "github.com/erigontech/erigon/turbo/testlog"
+)
+
+func TestPublisher(t *testing.T) {
+ newPublisherTest(t).run(func(ctx context.Context, t *testing.T, pt publisherTest) {
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(1).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(2).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(3).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(4).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(5).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(6).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(7).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+ pt.peerEvent(&sentryproto.PeerEvent{
+ PeerId: PeerIdFromUint64(8).H512(),
+ EventId: sentryproto.PeerEvent_Connect,
+ })
+
+ // we hear about block1 from peers 1,2,3,4
+ header1 := &types.Header{Number: big.NewInt(1)}
+ block1 := types.NewBlockWithHeader(header1)
+ td1 := big.NewInt(5)
+ waitPeersMayMissHash := func(peersCount int) func() bool {
+ return func() bool { return len(pt.peerTracker.ListPeersMayMissBlockHash(header1.Hash())) == peersCount }
+ }
+ require.Eventually(t, waitPeersMayMissHash(8), time.Second, 5*time.Millisecond)
+ pt.newBlockEvent(&DecodedInboundMessage[*eth.NewBlockPacket]{
+ PeerId: PeerIdFromUint64(1),
+ Decoded: ð.NewBlockPacket{
+ Block: block1,
+ TD: td1,
+ },
+ })
+ require.Eventually(t, waitPeersMayMissHash(7), time.Second, 5*time.Millisecond)
+ pt.newBlockEvent(&DecodedInboundMessage[*eth.NewBlockPacket]{
+ PeerId: PeerIdFromUint64(2),
+ Decoded: ð.NewBlockPacket{
+ Block: block1,
+ TD: td1,
+ },
+ })
+ require.Eventually(t, waitPeersMayMissHash(6), time.Second, 5*time.Millisecond)
+ pt.newBlockHashesEvent(&DecodedInboundMessage[*eth.NewBlockHashesPacket]{
+ PeerId: PeerIdFromUint64(3),
+ Decoded: ð.NewBlockHashesPacket{
+ {
+ Hash: header1.Hash(),
+ Number: header1.Number.Uint64(),
+ },
+ },
+ })
+ require.Eventually(t, waitPeersMayMissHash(5), time.Second, 5*time.Millisecond)
+ pt.newBlockHashesEvent(&DecodedInboundMessage[*eth.NewBlockHashesPacket]{
+ PeerId: PeerIdFromUint64(4),
+ Decoded: ð.NewBlockHashesPacket{
+ {
+ Hash: header1.Hash(),
+ Number: header1.Number.Uint64(),
+ },
+ },
+ })
+ require.Eventually(t, waitPeersMayMissHash(4), time.Second, 5*time.Millisecond)
+
+ p := pt.publisher
+ p.PublishNewBlock(block1, big.NewInt(55))
+ waitSends := func(sendsCount int) func() bool {
+ return func() bool { return len(pt.capturedSends()) == sendsCount }
+ }
+ // NewBlock announces should be send to only sqrt(peers) that do not know about this block hash
+ // according to our knowledge: sqrt(4)=2 -> peers 5,6
+ knownSends := map[PeerId]struct{}{}
+ knownSends[*PeerIdFromUint64(1)] = struct{}{}
+ knownSends[*PeerIdFromUint64(2)] = struct{}{}
+ knownSends[*PeerIdFromUint64(3)] = struct{}{}
+ knownSends[*PeerIdFromUint64(4)] = struct{}{}
+ require.Eventually(t, waitSends(2), time.Second, 5*time.Millisecond)
+ capturedSend1PeerId := *PeerIdFromH512(pt.capturedSends()[0].PeerId)
+ _, known := knownSends[capturedSend1PeerId]
+ require.False(t, known)
+ knownSends[capturedSend1PeerId] = struct{}{}
+ capturedSend2PeerId := *PeerIdFromH512(pt.capturedSends()[1].PeerId)
+ _, known = knownSends[capturedSend2PeerId]
+ require.False(t, known)
+ knownSends[capturedSend2PeerId] = struct{}{}
+
+ p.PublishNewBlockHashes(block1)
+ // NewBlockHashes should be sent to all remaining peers that do not already know this block hash
+ // according to our knowledge: peers 7,8
+ require.Eventually(t, waitSends(4), time.Second, 5*time.Millisecond)
+ capturedSend3PeerId := *PeerIdFromH512(pt.capturedSends()[2].PeerId)
+ _, known = knownSends[capturedSend3PeerId]
+ require.False(t, known)
+ knownSends[capturedSend3PeerId] = struct{}{}
+ capturedSend4PeerId := *PeerIdFromH512(pt.capturedSends()[3].PeerId)
+ _, known = knownSends[capturedSend4PeerId]
+ require.False(t, known)
+ knownSends[capturedSend4PeerId] = struct{}{}
+ require.Len(t, knownSends, 8)
+ allPeerIds := maps.Keys(knownSends)
+ require.ElementsMatch(t, allPeerIds, []PeerId{
+ *PeerIdFromUint64(1),
+ *PeerIdFromUint64(2),
+ *PeerIdFromUint64(3),
+ *PeerIdFromUint64(4),
+ *PeerIdFromUint64(5),
+ *PeerIdFromUint64(6),
+ *PeerIdFromUint64(7),
+ *PeerIdFromUint64(8),
+ })
+
+ // all 8 peers must now know about the hash according to our knowledge
+ require.Eventually(t, waitPeersMayMissHash(0), time.Second, 5*time.Millisecond)
+ })
+}
+
+func newPublisherTest(t *testing.T) publisherTest {
+ ctx, cancel := context.WithCancel(context.Background())
+ t.Cleanup(cancel)
+ logger := testlog.Logger(t, log.LvlCrit)
+ ctrl := gomock.NewController(t)
+ peerProvider := NewMockpeerProvider(ctrl)
+ peerEventRegistrar := NewMockpeerEventRegistrar(ctrl)
+ peerTracker := NewPeerTracker(logger, peerProvider, peerEventRegistrar, WithPreservingPeerShuffle)
+ sentryClient := direct.NewMockSentryClient(ctrl)
+ messageSender := NewMessageSender(sentryClient)
+ publisher := NewPublisher(logger, messageSender, peerTracker)
+ capturedSends := make([]*sentryproto.SendMessageByIdRequest, 0, 1024)
+ test := publisherTest{
+ ctx: ctx,
+ ctxCancel: cancel,
+ t: t,
+ peerTracker: peerTracker,
+ peerProvider: peerProvider,
+ peerEventRegistrar: peerEventRegistrar,
+ publisher: publisher,
+ peerEventStream: make(chan *sentryproto.PeerEvent),
+ newBlockHashesStream: make(chan *DecodedInboundMessage[*eth.NewBlockHashesPacket]),
+ newBlockStream: make(chan *DecodedInboundMessage[*eth.NewBlockPacket]),
+ sentryClient: sentryClient,
+ capturedSendsPtr: &capturedSends,
+ capturedSendsMu: &sync.Mutex{},
+ }
+
+ test.mockPeerProvider(&sentryproto.PeersReply{})
+ test.mockPeerEvents(test.peerEventStream)
+ test.mockNewBlockHashesEvents(test.newBlockHashesStream)
+ test.mockNewBlockEvents(test.newBlockStream)
+ test.captureSends(test.capturedSendsPtr)
+ return test
+}
+
+type publisherTest struct {
+ ctx context.Context
+ ctxCancel context.CancelFunc
+ t *testing.T
+ peerTracker PeerTracker
+ peerProvider *MockpeerProvider
+ peerEventRegistrar *MockpeerEventRegistrar
+ peerEventStream chan *sentryproto.PeerEvent
+ newBlockHashesStream chan *DecodedInboundMessage[*eth.NewBlockHashesPacket]
+ newBlockStream chan *DecodedInboundMessage[*eth.NewBlockPacket]
+ sentryClient *direct.MockSentryClient
+ capturedSendsPtr *[]*sentryproto.SendMessageByIdRequest
+ capturedSendsMu *sync.Mutex
+ publisher Publisher
+}
+
+func (pt publisherTest) mockPeerProvider(peerReply *sentryproto.PeersReply) {
+ pt.peerProvider.EXPECT().
+ Peers(gomock.Any(), gomock.Any()).
+ Return(peerReply, nil).
+ Times(1)
+}
+
+func (pt publisherTest) mockPeerEvents(events <-chan *sentryproto.PeerEvent) {
+ pt.peerEventRegistrar.EXPECT().
+ RegisterPeerEventObserver(gomock.Any()).
+ DoAndReturn(func(observer polygoncommon.Observer[*sentryproto.PeerEvent]) UnregisterFunc {
+ ctx, cancel := context.WithCancel(context.Background())
+ go func() {
+ for {
+ select {
+ case <-ctx.Done():
+ return
+ case event := <-events:
+ observer(event)
+ }
+ }
+ }()
+
+ return UnregisterFunc(cancel)
+ }).
+ Times(1)
+}
+
+func (pt publisherTest) peerEvent(e *sentryproto.PeerEvent) {
+ send(pt.ctx, pt.t, pt.peerEventStream, e)
+}
+
+func (pt publisherTest) mockNewBlockHashesEvents(events <-chan *DecodedInboundMessage[*eth.NewBlockHashesPacket]) {
+ pt.peerEventRegistrar.EXPECT().
+ RegisterNewBlockHashesObserver(gomock.Any()).
+ DoAndReturn(
+ func(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockHashesPacket]]) UnregisterFunc {
+ ctx, cancel := context.WithCancel(context.Background())
+ go func() {
+ for {
+ select {
+ case <-ctx.Done():
+ return
+ case event := <-events:
+ observer(event)
+ }
+ }
+ }()
+
+ return UnregisterFunc(cancel)
+ },
+ ).
+ Times(1)
+}
+
+func (pt publisherTest) newBlockHashesEvent(e *DecodedInboundMessage[*eth.NewBlockHashesPacket]) {
+ send(pt.ctx, pt.t, pt.newBlockHashesStream, e)
+}
+
+func (pt publisherTest) mockNewBlockEvents(events <-chan *DecodedInboundMessage[*eth.NewBlockPacket]) {
+ pt.peerEventRegistrar.EXPECT().
+ RegisterNewBlockObserver(gomock.Any()).
+ DoAndReturn(
+ func(observer polygoncommon.Observer[*DecodedInboundMessage[*eth.NewBlockPacket]]) UnregisterFunc {
+ ctx, cancel := context.WithCancel(context.Background())
+ go func() {
+ for {
+ select {
+ case <-ctx.Done():
+ return
+ case event := <-events:
+ observer(event)
+ }
+ }
+ }()
+
+ return UnregisterFunc(cancel)
+ },
+ ).
+ Times(1)
+}
+
+func (pt publisherTest) newBlockEvent(e *DecodedInboundMessage[*eth.NewBlockPacket]) {
+ send(pt.ctx, pt.t, pt.newBlockStream, e)
+}
+
+func (pt publisherTest) captureSends(sends *[]*sentryproto.SendMessageByIdRequest) {
+ pt.sentryClient.EXPECT().
+ SendMessageById(gomock.Any(), gomock.Any(), gomock.Any()).
+ DoAndReturn(func(
+ _ context.Context,
+ r *sentryproto.SendMessageByIdRequest,
+ _ ...grpc.CallOption,
+ ) (*sentryproto.SentPeers, error) {
+ pt.capturedSendsMu.Lock()
+ defer pt.capturedSendsMu.Unlock()
+ *sends = append(*sends, r)
+ return &sentryproto.SentPeers{Peers: []*typesproto.H512{r.PeerId}}, nil
+ }).
+ AnyTimes()
+}
+
+// sortedCapturedSends assumes peer ids are created using PeerIdFromUint64 and sorts all captured sends by PeerId
+func (pt publisherTest) capturedSends() []*sentryproto.SendMessageByIdRequest {
+ pt.capturedSendsMu.Lock()
+ defer pt.capturedSendsMu.Unlock()
+ return *pt.capturedSendsPtr
+}
+
+func (pt publisherTest) run(f func(ctx context.Context, t *testing.T, pt publisherTest)) {
+ var done atomic.Bool
+ pt.t.Run("start", func(t *testing.T) {
+ go func() {
+ defer done.Store(true)
+ eg, ctx := errgroup.WithContext(pt.ctx)
+ eg.Go(func() error { return pt.peerTracker.Run(ctx) })
+ eg.Go(func() error { return pt.publisher.Run(ctx) })
+ err := eg.Wait()
+ require.ErrorIs(t, err, context.Canceled)
+ }()
+ })
+
+ pt.t.Run("test", func(t *testing.T) {
+ f(pt.ctx, t, pt)
+ })
+
+ pt.t.Run("stop", func(t *testing.T) {
+ pt.ctxCancel()
+ require.Eventually(t, done.Load, time.Second, 5*time.Millisecond)
+ })
+}
diff --git a/polygon/p2p/service.go b/polygon/p2p/service.go
index f595b34e945..6d82ad85917 100644
--- a/polygon/p2p/service.go
+++ b/polygon/p2p/service.go
@@ -62,7 +62,7 @@ func newService(
fetcher := NewFetcher(logger, fetcherConfig, messageListener, messageSender, requestIdGenerator)
fetcher = NewPenalizingFetcher(logger, fetcher, peerPenalizer)
fetcher = NewTrackingFetcher(fetcher, peerTracker)
- publisher := NewPublisher(messageSender)
+ publisher := NewPublisher(logger, messageSender, peerTracker)
return &service{
Fetcher: fetcher,
MessageListener: messageListener,
diff --git a/polygon/sync/tip_events.go b/polygon/sync/tip_events.go
index fb3a9f871dc..7ce533d1be9 100644
--- a/polygon/sync/tip_events.go
+++ b/polygon/sync/tip_events.go
@@ -128,8 +128,8 @@ func NewTipEvents(
p2pObserverRegistrar p2pObserverRegistrar,
heimdallObserverRegistrar heimdallObserverRegistrar,
) *TipEvents {
- heimdallEventsChannel := NewEventChannel[Event](10, WithEventChannelLogging(logger, log.LvlDebug, EventTopicHeimdall.String()))
- p2pEventsChannel := NewEventChannel[Event](1000, WithEventChannelLogging(logger, log.LvlDebug, EventTopicP2P.String()))
+ heimdallEventsChannel := NewEventChannel[Event](10, WithEventChannelLogging(logger, log.LvlTrace, EventTopicHeimdall.String()))
+ p2pEventsChannel := NewEventChannel[Event](1000, WithEventChannelLogging(logger, log.LvlTrace, EventTopicP2P.String()))
compositeEventsChannel := NewTipEventsCompositeChannel(heimdallEventsChannel, p2pEventsChannel)
return &TipEvents{
logger: logger,
@@ -154,7 +154,7 @@ func (te *TipEvents) Run(ctx context.Context) error {
return
}
- te.logger.Debug(
+ te.logger.Trace(
"[tip-events] new block event received from peer",
"peerId", message.PeerId,
"hash", block.Hash(),
@@ -179,7 +179,7 @@ func (te *TipEvents) Run(ctx context.Context) error {
return
}
- te.logger.Debug(
+ te.logger.Trace(
"[tip-events] new block hashes event received from peer",
"peerId", message.PeerId,
"hash", blockHashes[0].Hash,
@@ -197,7 +197,7 @@ func (te *TipEvents) Run(ctx context.Context) error {
defer newBlockHashesObserverCancel()
milestoneObserverCancel := te.heimdallObserverRegistrar.RegisterMilestoneObserver(func(milestone *heimdall.Milestone) {
- te.logger.Debug("[tip-events] new milestone event received", "id", milestone.RawId())
+ te.logger.Trace("[tip-events] new milestone event received", "id", milestone.RawId())
te.events.PushEvent(Event{
Type: EventTypeNewMilestone,
newMilestone: milestone,
@@ -312,7 +312,7 @@ func (g blockEventsSpamGuard) Spam(peerId *p2p.PeerId, blockHash common.Hash, bl
}
if g.seenPeerBlockHashes.Contains(key) {
- g.logger.Debug(
+ g.logger.Trace(
"[block-events-spam-guard] detected spam",
"peerId", peerId,
"blockHash", blockHash,
From 606ce01260c8aa439ce67fdcef4ae9a5ec92da1b Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Mon, 14 Oct 2024 23:08:47 +0100
Subject: [PATCH 09/25] allow NewBlockMsg and NewBlockHashesMsg in sentry
SendMessageById
---
p2p/sentry/sentry_grpc_server.go | 2 ++
1 file changed, 2 insertions(+)
diff --git a/p2p/sentry/sentry_grpc_server.go b/p2p/sentry/sentry_grpc_server.go
index 0e1c0258bb4..701263564fa 100644
--- a/p2p/sentry/sentry_grpc_server.go
+++ b/p2p/sentry/sentry_grpc_server.go
@@ -927,6 +927,8 @@ func (ss *GrpcServer) SendMessageById(_ context.Context, inreq *proto_sentry.Sen
msgcode != eth.BlockHeadersMsg &&
msgcode != eth.GetBlockBodiesMsg &&
msgcode != eth.BlockBodiesMsg &&
+ msgcode != eth.NewBlockMsg &&
+ msgcode != eth.NewBlockHashesMsg &&
msgcode != eth.GetReceiptsMsg &&
msgcode != eth.ReceiptsMsg &&
msgcode != eth.NewPooledTransactionHashesMsg &&
From 11ce255a1e3143a3b2f9adbfe55753aa5f10a0cc Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Tue, 15 Oct 2024 10:13:45 +0100
Subject: [PATCH 10/25] fix linter
---
polygon/p2p/peer_tracker.go | 1 +
1 file changed, 1 insertion(+)
diff --git a/polygon/p2p/peer_tracker.go b/polygon/p2p/peer_tracker.go
index a702fdcf2b9..61af5c7bf3f 100644
--- a/polygon/p2p/peer_tracker.go
+++ b/polygon/p2p/peer_tracker.go
@@ -156,6 +156,7 @@ func (pt *peerTracker) ListPeersMayMissBlockHash(blockHash common.Hash) []*PeerI
var peerIds []*PeerId
for peerId, knownBlockAnnounces := range pt.peerKnownBlockAnnounces {
if !knownBlockAnnounces.Contains(blockHash) {
+ peerId := peerId
peerIds = append(peerIds, &peerId)
}
}
From a92f1f47e0083cb2c67b99481d5d472f53f91e81 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Tue, 15 Oct 2024 14:33:27 +0100
Subject: [PATCH 11/25] polygon/sync: bad blocks on chain tip
---
eth/stagedsync/stage_polygon_sync.go | 6 ++-
polygon/sync/execution_client.go | 28 +++++++++----
polygon/sync/sync.go | 63 +++++++++++++++++++++++++++-
3 files changed, 86 insertions(+), 11 deletions(-)
diff --git a/eth/stagedsync/stage_polygon_sync.go b/eth/stagedsync/stage_polygon_sync.go
index 6b81d8b70a6..491000a9d6f 100644
--- a/eth/stagedsync/stage_polygon_sync.go
+++ b/eth/stagedsync/stage_polygon_sync.go
@@ -1514,7 +1514,11 @@ func (e *polygonSyncStageExecutionEngine) UpdateForkChoice(ctx context.Context,
case <-ctx.Done():
return common.Hash{}, ctx.Err()
case result := <-resultCh:
- return result.latestValidHash, result.validationErr
+ err := result.validationErr
+ if err != nil {
+ err = fmt.Errorf("%w: %w", polygonsync.ErrBadForkChoiceUpdate, err)
+ }
+ return result.latestValidHash, err
}
}
diff --git a/polygon/sync/execution_client.go b/polygon/sync/execution_client.go
index 60958103fbc..7b776bc2a98 100644
--- a/polygon/sync/execution_client.go
+++ b/polygon/sync/execution_client.go
@@ -32,7 +32,8 @@ import (
eth1utils "github.com/erigontech/erigon/turbo/execution/eth1/eth1_utils"
)
-var errForkChoiceUpdateFailure = errors.New("fork choice update failed")
+var ErrForkChoiceUpdateFailure = errors.New("fork choice update failure")
+var ErrBadForkChoiceUpdate = errors.New("bad fork choice update")
type ExecutionClient interface {
InsertBlocks(ctx context.Context, blocks []*types.Block) error
@@ -103,15 +104,24 @@ func (e *executionClient) UpdateForkChoice(ctx context.Context, tip *types.Heade
latestValidHash = gointerfaces.ConvertH256ToHash(response.LatestValidHash)
}
- if len(response.ValidationError) > 0 {
- return latestValidHash, fmt.Errorf("%w: validationErr=%s", errForkChoiceUpdateFailure, response.Status)
+ switch response.Status {
+ case executionproto.ExecutionStatus_Success:
+ return latestValidHash, nil
+ case executionproto.ExecutionStatus_BadBlock, executionproto.ExecutionStatus_InvalidForkchoice:
+ return latestValidHash, fmt.Errorf(
+ "%w: status=%d, validationErr='%s'",
+ ErrBadForkChoiceUpdate,
+ response.Status,
+ response.ValidationError,
+ )
+ default:
+ return latestValidHash, fmt.Errorf(
+ "%w: status=%d, validationErr='%s'",
+ ErrForkChoiceUpdateFailure,
+ response.Status,
+ response.ValidationError,
+ )
}
-
- if response.Status != executionproto.ExecutionStatus_Success {
- return latestValidHash, fmt.Errorf("%w: status=%s", errForkChoiceUpdateFailure, response.Status)
- }
-
- return latestValidHash, nil
}
func (e *executionClient) CurrentHeader(ctx context.Context) (*types.Header, error) {
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index f16eef31676..4cf5dfb2878 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -22,6 +22,8 @@ import (
"fmt"
"time"
+ "github.com/hashicorp/golang-lru/v2/simplelru"
+
"github.com/erigontech/erigon-lib/common"
"github.com/erigontech/erigon-lib/log/v3"
"github.com/erigontech/erigon/core/types"
@@ -51,6 +53,7 @@ type Sync struct {
heimdallSync heimdallSynchronizer
bridgeSync bridgeSynchronizer
events <-chan Event
+ badBlocks *simplelru.LRU[common.Hash, struct{}]
logger log.Logger
}
@@ -67,6 +70,11 @@ func NewSync(
events <-chan Event,
logger log.Logger,
) *Sync {
+ badBlocksLru, err := simplelru.NewLRU[common.Hash, struct{}](1024, nil)
+ if err != nil {
+ panic(err)
+ }
+
return &Sync{
store: store,
execution: execution,
@@ -78,6 +86,7 @@ func NewSync(
heimdallSync: heimdallSync,
bridgeSync: bridgeSync,
events: events,
+ badBlocks: badBlocksLru,
logger: logger,
}
}
@@ -193,6 +202,28 @@ func (s *Sync) applyNewBlockOnTip(
return nil
}
+ if s.badBlocks.Contains(newBlockHeaderHash) {
+ s.logger.Warn(syncLogPrefix("bad block received from peer"),
+ "blockHash", newBlockHeaderHash,
+ "blockNum", newBlockHeaderNum,
+ "peerId", event.PeerId,
+ )
+ s.maybePenalizePeerOnBadBlockEvent(ctx, event)
+ return nil
+ }
+
+ if s.badBlocks.Contains(newBlockHeader.ParentHash) {
+ s.logger.Warn(syncLogPrefix("block with bad parent received from peer"),
+ "blockHash", newBlockHeaderHash,
+ "blockNum", newBlockHeaderNum,
+ "parentHash", newBlockHeader.ParentHash,
+ "peerId", event.PeerId,
+ )
+ s.badBlocks.Add(newBlockHeaderHash, struct{}{})
+ s.maybePenalizePeerOnBadBlockEvent(ctx, event)
+ return nil
+ }
+
s.logger.Debug(
syncLogPrefix("applying new block event"),
"blockNum", newBlockHeaderNum,
@@ -310,7 +341,14 @@ func (s *Sync) applyNewBlockOnTip(
}
if err := s.commitExecution(ctx, newTip, ccBuilder.Root()); err != nil {
- return err
+ if !errors.Is(err, ErrBadForkChoiceUpdate) {
+ return err
+ }
+
+ s.logger.Warn(syncLogPrefix("bad block after execution"), "peerId", event.PeerId, "err", err)
+ s.badBlocks.Add(event.NewBlock.Hash(), struct{}{})
+ s.maybePenalizePeerOnBadBlockEvent(ctx, event)
+ return nil
}
if event.Source == EventSourceP2PNewBlock {
@@ -339,6 +377,17 @@ func (s *Sync) applyNewBlockHashesOnTip(
continue
}
+ if s.badBlocks.Contains(hashOrNum.Hash) {
+ // note: we do not penalize peer for bad blocks on new block hash events since they have
+ // not necessarily been executed by the peer but just propagated as per the devp2p spec
+ s.logger.Warn(syncLogPrefix("bad block hash received from peer"),
+ "blockHash", hashOrNum.Hash,
+ "blockNum", hashOrNum.Number,
+ "peerId", event.PeerId,
+ )
+ return nil
+ }
+
s.logger.Debug(
syncLogPrefix("applying new block hash event"),
"blockNum", hashOrNum.Number,
@@ -391,6 +440,18 @@ func (s *Sync) publishNewBlock(ctx context.Context, block *types.Block) {
s.p2pService.PublishNewBlock(block, td)
}
+func (s *Sync) maybePenalizePeerOnBadBlockEvent(ctx context.Context, event EventNewBlock) {
+ if event.Source == EventSourceP2PNewBlockHashes {
+ // note: we do not penalize peer for bad blocks on new block hash events since they have
+ // not necessarily been executed by the peer but just propagated as per the devp2p spec
+ return
+ }
+
+ if err := s.p2pService.Penalize(ctx, event.PeerId); err != nil {
+ s.logger.Debug(syncLogPrefix("issue with penalizing peer for bad block"), "peerId", event.PeerId, "err", err)
+ }
+}
+
//
// TODO (subsequent PRs) - unit test initial sync + on new event cases
//
From fee863765a07cb679cb4a25be38fb6e2375fb13b Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Tue, 15 Oct 2024 14:44:08 +0100
Subject: [PATCH 12/25] add comment
---
polygon/sync/sync.go | 4 ++++
1 file changed, 4 insertions(+)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index 4cf5dfb2878..d3b49fa6722 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -156,6 +156,8 @@ func (s *Sync) handleMilestoneTipMismatch(
}
if err = s.commitExecution(ctx, newTip, newTip); err != nil {
+ // note: if we face a failure during execution of finalized waypoints blocks
+ // it means that we're wrong and the blocks are not considered as bad blocks
return err
}
@@ -615,6 +617,8 @@ func (s *Sync) sync(ctx context.Context, tip *types.Header, tipDownloader tipDow
tip = newResult.latestTip
if err = s.commitExecution(ctx, tip, tip); err != nil {
+ // note: if we face a failure during execution of finalized waypoints blocks
+ // it means that we're wrong and the blocks are not considered as bad blocks
return syncToTipResult{}, err
}
}
From 365ea2df3b8b8b69596d3f02a42242d723cdd21b Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Tue, 15 Oct 2024 22:39:14 +0100
Subject: [PATCH 13/25] wip
---
polygon/sync/canonical_chain_builder.go | 99 +++++++++++-
polygon/sync/canonical_chain_builder_mock.go | 155 ++++++++++++++++---
polygon/sync/canonical_chain_builder_test.go | 18 +++
polygon/sync/execution_client.go | 6 +-
polygon/sync/sync.go | 136 +++++++++++++---
5 files changed, 370 insertions(+), 44 deletions(-)
diff --git a/polygon/sync/canonical_chain_builder.go b/polygon/sync/canonical_chain_builder.go
index 62e43a00eec..47f29036f66 100644
--- a/polygon/sync/canonical_chain_builder.go
+++ b/polygon/sync/canonical_chain_builder.go
@@ -36,8 +36,11 @@ type CanonicalChainBuilder interface {
Tip() *types.Header
Root() *types.Header
HeadersInRange(start uint64, count uint64) []*types.Header
- Prune(newRootNum uint64) error
+ HeaderByHash(hash libcommon.Hash) (header *types.Header, ok bool)
+ PruneRoot(newRootNum uint64) error
+ PruneNode(hash libcommon.Hash) error
Connect(ctx context.Context, headers []*types.Header) (newConnectedHeaders []*types.Header, err error)
+ LowestCommonAncestor(a, b libcommon.Hash) (*types.Header, bool)
}
type producerSlotIndex uint64
@@ -156,9 +159,23 @@ func (ccb *canonicalChainBuilder) HeadersInRange(start uint64, count uint64) []*
return headers[offset : offset+count]
}
-func (ccb *canonicalChainBuilder) Prune(newRootNum uint64) error {
+func (ccb *canonicalChainBuilder) HeaderByHash(hash libcommon.Hash) (header *types.Header, ok bool) {
+ ccb.enumerate(func(node *forkTreeNode) bool {
+ if node.headerHash == hash {
+ header = node.header
+ ok = true
+ return false
+ }
+
+ return true
+ })
+
+ return header, ok
+}
+
+func (ccb *canonicalChainBuilder) PruneRoot(newRootNum uint64) error {
if (newRootNum < ccb.root.header.Number.Uint64()) || (newRootNum > ccb.Tip().Number.Uint64()) {
- return errors.New("canonicalChainBuilder.Prune: newRootNum outside of the canonical chain")
+ return errors.New("canonicalChainBuilder.PruneRoot: newRootNum outside of the canonical chain")
}
newRoot := ccb.tip
@@ -170,6 +187,35 @@ func (ccb *canonicalChainBuilder) Prune(newRootNum uint64) error {
return nil
}
+func (ccb *canonicalChainBuilder) PruneNode(hash libcommon.Hash) error {
+ if ccb.root.headerHash == hash {
+ return errors.New("canonicalChainBuilder.PruneNode: can't remove root node")
+ }
+
+ var exists bool
+ ccb.enumerate(func(node *forkTreeNode) bool {
+ if node.headerHash != hash {
+ return true
+ }
+
+ for idx, parentChild := range node.parent.children {
+ if parentChild.headerHash == hash {
+ exists = true
+ delete(node.parent.children, idx)
+ break
+ }
+ }
+
+ return false
+ })
+ if !exists {
+ return errors.New("canonicalChainBuilder.PruneNode: could not find node to prune")
+ }
+
+ ccb.tip = ccb.recalcTip() // tip may have changed after prunning, re-calc
+ return nil
+}
+
// compareForkTreeNodes compares 2 fork tree nodes.
// It returns a positive number if the chain ending at node1 is "better" than the chain ending at node2.
// The better node belongs to the canonical chain, and it has:
@@ -195,6 +241,23 @@ func (ccb *canonicalChainBuilder) updateTipIfNeeded(tipCandidate *forkTreeNode)
}
}
+func (ccb *canonicalChainBuilder) recalcTip() *forkTreeNode {
+ var tip *forkTreeNode
+ ccb.enumerate(func(node *forkTreeNode) bool {
+ if tip == nil {
+ tip = node
+ return true
+ }
+
+ if compareForkTreeNodes(tip, node) < 0 {
+ tip = node
+ }
+
+ return true
+ })
+ return tip
+}
+
// Connect connects a list of headers to the canonical chain builder tree.
// Returns the list of newly connected headers (filtering out headers that already exist in the tree)
// or an error in case the header is invalid or the header chain cannot reach any of the nodes in the tree.
@@ -291,3 +354,33 @@ func (ccb *canonicalChainBuilder) Connect(ctx context.Context, headers []*types.
return headers, nil
}
+
+func (ccb *canonicalChainBuilder) LowestCommonAncestor(a, b libcommon.Hash) (*types.Header, bool) {
+ /*pathA*/ _ = ccb.pathFromRoot(a)
+ /*pathB*/ _ = ccb.pathFromRoot(b)
+ //
+ // TODO
+ //
+ return nil, false
+}
+
+func (ccb *canonicalChainBuilder) pathFromRoot(to libcommon.Hash) []*forkTreeNode {
+ path := make([]*forkTreeNode, 0, ccb.Tip().Number.Uint64()-ccb.Root().Number.Uint64())
+ pathFromRootRec(ccb.root, to, &path)
+ return path
+}
+
+func pathFromRootRec(node *forkTreeNode, to libcommon.Hash, path *[]*forkTreeNode) bool {
+ if node.headerHash == to {
+ return true
+ }
+
+ for _, child := range node.children {
+ if pathFromRootRec(child, to, path) {
+ *path = append(*path, child)
+ return true
+ }
+ }
+
+ return false
+}
diff --git a/polygon/sync/canonical_chain_builder_mock.go b/polygon/sync/canonical_chain_builder_mock.go
index d6259d7cc13..7c78ae6eafd 100644
--- a/polygon/sync/canonical_chain_builder_mock.go
+++ b/polygon/sync/canonical_chain_builder_mock.go
@@ -42,11 +42,12 @@ func (m *MockCanonicalChainBuilder) EXPECT() *MockCanonicalChainBuilderMockRecor
}
// Connect mocks base method.
-func (m *MockCanonicalChainBuilder) Connect(arg0 context.Context, arg1 []*types.Header) error {
+func (m *MockCanonicalChainBuilder) Connect(arg0 context.Context, arg1 []*types.Header) ([]*types.Header, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Connect", arg0, arg1)
- ret0, _ := ret[0].(error)
- return ret0
+ ret0, _ := ret[0].([]*types.Header)
+ ret1, _ := ret[1].(error)
+ return ret0, ret1
}
// Connect indicates an expected call of Connect.
@@ -62,19 +63,19 @@ type MockCanonicalChainBuilderConnectCall struct {
}
// Return rewrite *gomock.Call.Return
-func (c *MockCanonicalChainBuilderConnectCall) Return(arg0 error) *MockCanonicalChainBuilderConnectCall {
- c.Call = c.Call.Return(arg0)
+func (c *MockCanonicalChainBuilderConnectCall) Return(arg0 []*types.Header, arg1 error) *MockCanonicalChainBuilderConnectCall {
+ c.Call = c.Call.Return(arg0, arg1)
return c
}
// Do rewrite *gomock.Call.Do
-func (c *MockCanonicalChainBuilderConnectCall) Do(f func(context.Context, []*types.Header) error) *MockCanonicalChainBuilderConnectCall {
+func (c *MockCanonicalChainBuilderConnectCall) Do(f func(context.Context, []*types.Header) ([]*types.Header, error)) *MockCanonicalChainBuilderConnectCall {
c.Call = c.Call.Do(f)
return c
}
// DoAndReturn rewrite *gomock.Call.DoAndReturn
-func (c *MockCanonicalChainBuilderConnectCall) DoAndReturn(f func(context.Context, []*types.Header) error) *MockCanonicalChainBuilderConnectCall {
+func (c *MockCanonicalChainBuilderConnectCall) DoAndReturn(f func(context.Context, []*types.Header) ([]*types.Header, error)) *MockCanonicalChainBuilderConnectCall {
c.Call = c.Call.DoAndReturn(f)
return c
}
@@ -117,6 +118,45 @@ func (c *MockCanonicalChainBuilderContainsHashCall) DoAndReturn(f func(common.Ha
return c
}
+// HeaderByHash mocks base method.
+func (m *MockCanonicalChainBuilder) HeaderByHash(arg0 common.Hash) (*types.Header, bool) {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "HeaderByHash", arg0)
+ ret0, _ := ret[0].(*types.Header)
+ ret1, _ := ret[1].(bool)
+ return ret0, ret1
+}
+
+// HeaderByHash indicates an expected call of HeaderByHash.
+func (mr *MockCanonicalChainBuilderMockRecorder) HeaderByHash(arg0 any) *MockCanonicalChainBuilderHeaderByHashCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HeaderByHash", reflect.TypeOf((*MockCanonicalChainBuilder)(nil).HeaderByHash), arg0)
+ return &MockCanonicalChainBuilderHeaderByHashCall{Call: call}
+}
+
+// MockCanonicalChainBuilderHeaderByHashCall wrap *gomock.Call
+type MockCanonicalChainBuilderHeaderByHashCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockCanonicalChainBuilderHeaderByHashCall) Return(arg0 *types.Header, arg1 bool) *MockCanonicalChainBuilderHeaderByHashCall {
+ c.Call = c.Call.Return(arg0, arg1)
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockCanonicalChainBuilderHeaderByHashCall) Do(f func(common.Hash) (*types.Header, bool)) *MockCanonicalChainBuilderHeaderByHashCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockCanonicalChainBuilderHeaderByHashCall) DoAndReturn(f func(common.Hash) (*types.Header, bool)) *MockCanonicalChainBuilderHeaderByHashCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
// HeadersInRange mocks base method.
func (m *MockCanonicalChainBuilder) HeadersInRange(arg0, arg1 uint64) []*types.Header {
m.ctrl.T.Helper()
@@ -155,40 +195,117 @@ func (c *MockCanonicalChainBuilderHeadersInRangeCall) DoAndReturn(f func(uint64,
return c
}
-// Prune mocks base method.
-func (m *MockCanonicalChainBuilder) Prune(arg0 uint64) error {
+// LowestCommonAncestor mocks base method.
+func (m *MockCanonicalChainBuilder) LowestCommonAncestor(arg0, arg1 common.Hash) (*types.Header, bool) {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "LowestCommonAncestor", arg0, arg1)
+ ret0, _ := ret[0].(*types.Header)
+ ret1, _ := ret[1].(bool)
+ return ret0, ret1
+}
+
+// LowestCommonAncestor indicates an expected call of LowestCommonAncestor.
+func (mr *MockCanonicalChainBuilderMockRecorder) LowestCommonAncestor(arg0, arg1 any) *MockCanonicalChainBuilderLowestCommonAncestorCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LowestCommonAncestor", reflect.TypeOf((*MockCanonicalChainBuilder)(nil).LowestCommonAncestor), arg0, arg1)
+ return &MockCanonicalChainBuilderLowestCommonAncestorCall{Call: call}
+}
+
+// MockCanonicalChainBuilderLowestCommonAncestorCall wrap *gomock.Call
+type MockCanonicalChainBuilderLowestCommonAncestorCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockCanonicalChainBuilderLowestCommonAncestorCall) Return(arg0 *types.Header, arg1 bool) *MockCanonicalChainBuilderLowestCommonAncestorCall {
+ c.Call = c.Call.Return(arg0, arg1)
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockCanonicalChainBuilderLowestCommonAncestorCall) Do(f func(common.Hash, common.Hash) (*types.Header, bool)) *MockCanonicalChainBuilderLowestCommonAncestorCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockCanonicalChainBuilderLowestCommonAncestorCall) DoAndReturn(f func(common.Hash, common.Hash) (*types.Header, bool)) *MockCanonicalChainBuilderLowestCommonAncestorCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
+// PruneNode mocks base method.
+func (m *MockCanonicalChainBuilder) PruneNode(arg0 common.Hash) error {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "PruneNode", arg0)
+ ret0, _ := ret[0].(error)
+ return ret0
+}
+
+// PruneNode indicates an expected call of PruneNode.
+func (mr *MockCanonicalChainBuilderMockRecorder) PruneNode(arg0 any) *MockCanonicalChainBuilderPruneNodeCall {
+ mr.mock.ctrl.T.Helper()
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PruneNode", reflect.TypeOf((*MockCanonicalChainBuilder)(nil).PruneNode), arg0)
+ return &MockCanonicalChainBuilderPruneNodeCall{Call: call}
+}
+
+// MockCanonicalChainBuilderPruneNodeCall wrap *gomock.Call
+type MockCanonicalChainBuilderPruneNodeCall struct {
+ *gomock.Call
+}
+
+// Return rewrite *gomock.Call.Return
+func (c *MockCanonicalChainBuilderPruneNodeCall) Return(arg0 error) *MockCanonicalChainBuilderPruneNodeCall {
+ c.Call = c.Call.Return(arg0)
+ return c
+}
+
+// Do rewrite *gomock.Call.Do
+func (c *MockCanonicalChainBuilderPruneNodeCall) Do(f func(common.Hash) error) *MockCanonicalChainBuilderPruneNodeCall {
+ c.Call = c.Call.Do(f)
+ return c
+}
+
+// DoAndReturn rewrite *gomock.Call.DoAndReturn
+func (c *MockCanonicalChainBuilderPruneNodeCall) DoAndReturn(f func(common.Hash) error) *MockCanonicalChainBuilderPruneNodeCall {
+ c.Call = c.Call.DoAndReturn(f)
+ return c
+}
+
+// PruneRoot mocks base method.
+func (m *MockCanonicalChainBuilder) PruneRoot(arg0 uint64) error {
m.ctrl.T.Helper()
- ret := m.ctrl.Call(m, "Prune", arg0)
+ ret := m.ctrl.Call(m, "PruneRoot", arg0)
ret0, _ := ret[0].(error)
return ret0
}
-// Prune indicates an expected call of Prune.
-func (mr *MockCanonicalChainBuilderMockRecorder) Prune(arg0 any) *MockCanonicalChainBuilderPruneCall {
+// PruneRoot indicates an expected call of PruneRoot.
+func (mr *MockCanonicalChainBuilderMockRecorder) PruneRoot(arg0 any) *MockCanonicalChainBuilderPruneRootCall {
mr.mock.ctrl.T.Helper()
- call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Prune", reflect.TypeOf((*MockCanonicalChainBuilder)(nil).Prune), arg0)
- return &MockCanonicalChainBuilderPruneCall{Call: call}
+ call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PruneRoot", reflect.TypeOf((*MockCanonicalChainBuilder)(nil).PruneRoot), arg0)
+ return &MockCanonicalChainBuilderPruneRootCall{Call: call}
}
-// MockCanonicalChainBuilderPruneCall wrap *gomock.Call
-type MockCanonicalChainBuilderPruneCall struct {
+// MockCanonicalChainBuilderPruneRootCall wrap *gomock.Call
+type MockCanonicalChainBuilderPruneRootCall struct {
*gomock.Call
}
// Return rewrite *gomock.Call.Return
-func (c *MockCanonicalChainBuilderPruneCall) Return(arg0 error) *MockCanonicalChainBuilderPruneCall {
+func (c *MockCanonicalChainBuilderPruneRootCall) Return(arg0 error) *MockCanonicalChainBuilderPruneRootCall {
c.Call = c.Call.Return(arg0)
return c
}
// Do rewrite *gomock.Call.Do
-func (c *MockCanonicalChainBuilderPruneCall) Do(f func(uint64) error) *MockCanonicalChainBuilderPruneCall {
+func (c *MockCanonicalChainBuilderPruneRootCall) Do(f func(uint64) error) *MockCanonicalChainBuilderPruneRootCall {
c.Call = c.Call.Do(f)
return c
}
// DoAndReturn rewrite *gomock.Call.DoAndReturn
-func (c *MockCanonicalChainBuilderPruneCall) DoAndReturn(f func(uint64) error) *MockCanonicalChainBuilderPruneCall {
+func (c *MockCanonicalChainBuilderPruneRootCall) DoAndReturn(f func(uint64) error) *MockCanonicalChainBuilderPruneRootCall {
c.Call = c.Call.DoAndReturn(f)
return c
}
diff --git a/polygon/sync/canonical_chain_builder_test.go b/polygon/sync/canonical_chain_builder_test.go
index 1f019e0f6fe..070cce90293 100644
--- a/polygon/sync/canonical_chain_builder_test.go
+++ b/polygon/sync/canonical_chain_builder_test.go
@@ -306,3 +306,21 @@ func TestCCBConnectAltForksAtLevel2Reverse(t *testing.T) {
test.testConnect(ctx, []*types.Header{header100}, header100, []*types.Header{root, header1, header10, header100}, []*types.Header{header100})
test.testConnect(ctx, []*types.Header{header200}, header200, []*types.Header{root, header2, header20, header200}, []*types.Header{header200})
}
+
+func TestCCBPruneNode(t *testing.T) {
+ //
+ // TODO
+ //
+}
+
+func TestCCBHeaderByHash(t *testing.T) {
+ //
+ // TODO
+ //
+}
+
+func TestCCBLowestCommonAncestor(t *testing.T) {
+ //
+ // TODO
+ //
+}
diff --git a/polygon/sync/execution_client.go b/polygon/sync/execution_client.go
index 7b776bc2a98..bc22130f658 100644
--- a/polygon/sync/execution_client.go
+++ b/polygon/sync/execution_client.go
@@ -33,7 +33,7 @@ import (
)
var ErrForkChoiceUpdateFailure = errors.New("fork choice update failure")
-var ErrBadForkChoiceUpdate = errors.New("bad fork choice update")
+var ErrForkChoiceUpdateBadBlock = errors.New("fork choice update bad block")
type ExecutionClient interface {
InsertBlocks(ctx context.Context, blocks []*types.Block) error
@@ -107,10 +107,10 @@ func (e *executionClient) UpdateForkChoice(ctx context.Context, tip *types.Heade
switch response.Status {
case executionproto.ExecutionStatus_Success:
return latestValidHash, nil
- case executionproto.ExecutionStatus_BadBlock, executionproto.ExecutionStatus_InvalidForkchoice:
+ case executionproto.ExecutionStatus_BadBlock:
return latestValidHash, fmt.Errorf(
"%w: status=%d, validationErr='%s'",
- ErrBadForkChoiceUpdate,
+ ErrForkChoiceUpdateBadBlock,
response.Status,
response.ValidationError,
)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index d3b49fa6722..e18d1d77ecc 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -40,6 +40,7 @@ type heimdallSynchronizer interface {
type bridgeSynchronizer interface {
Synchronize(ctx context.Context, blockNum uint64) error
Unwind(ctx context.Context, blockNum uint64) error
+ ProcessNewBlocks(ctx context.Context, blocks []*types.Block) error
}
type Sync struct {
@@ -91,19 +92,19 @@ func NewSync(
}
}
-func (s *Sync) commitExecution(ctx context.Context, newTip *types.Header, finalizedHeader *types.Header) error {
+func (s *Sync) commitExecution(ctx context.Context, newTip *types.Header, finalizedHeader *types.Header) (common.Hash, error) {
if err := s.store.Flush(ctx); err != nil {
- return err
+ return common.Hash{}, err
}
blockNum := newTip.Number.Uint64()
if err := s.heimdallSync.SynchronizeSpans(ctx, blockNum); err != nil {
- return err
+ return common.Hash{}, err
}
if err := s.bridgeSync.Synchronize(ctx, blockNum); err != nil {
- return err
+ return common.Hash{}, err
}
age := common.PrettyAge(time.Unix(int64(newTip.Time), 0))
@@ -113,11 +114,11 @@ func (s *Sync) commitExecution(ctx context.Context, newTip *types.Header, finali
latestValidHash, err := s.execution.UpdateForkChoice(ctx, newTip, finalizedHeader)
if err != nil {
s.logger.Error("failed to update fork choice", "latestValidHash", latestValidHash, "err", err)
- return err
+ return latestValidHash, err
}
s.logger.Info(syncLogPrefix("update fork choice done"), "in", time.Since(fcStartTime))
- return nil
+ return latestValidHash, nil
}
func (s *Sync) handleMilestoneTipMismatch(
@@ -155,7 +156,7 @@ func (s *Sync) handleMilestoneTipMismatch(
)
}
- if err = s.commitExecution(ctx, newTip, newTip); err != nil {
+ if _, err := s.commitExecution(ctx, newTip, newTip); err != nil {
// note: if we face a failure during execution of finalized waypoints blocks
// it means that we're wrong and the blocks are not considered as bad blocks
return err
@@ -188,7 +189,7 @@ func (s *Sync) applyNewMilestoneOnTip(
return s.handleMilestoneTipMismatch(ctx, ccBuilder, milestone)
}
- return ccBuilder.Prune(milestone.EndBlock().Uint64())
+ return ccBuilder.PruneRoot(milestone.EndBlock().Uint64())
}
func (s *Sync) applyNewBlockOnTip(
@@ -314,9 +315,7 @@ func (s *Sync) applyNewBlockOnTip(
newTip := ccBuilder.Tip()
firstConnectedHeader := newConnectedHeaders[0]
if newTip != oldTip && oldTip.Hash() != firstConnectedHeader.ParentHash {
- // forks have changed, we need to unwind unwindable data
- blockNum := max(1, firstConnectedHeader.Number.Uint64()) - 1
- if err := s.bridgeSync.Unwind(ctx, blockNum); err != nil {
+ if err := s.handleTipForkChangeUnwinds(ctx, ccBuilder, oldTip); err != nil {
return err
}
}
@@ -339,18 +338,20 @@ func (s *Sync) applyNewBlockOnTip(
}
if newTip == oldTip {
+ lastConnectedNum := newConnectedHeaders[len(newConnectedHeaders)-1].Number.Uint64()
+ if tipNum := newTip.Number.Uint64(); lastConnectedNum > tipNum {
+ return s.handleInsertBlocksAfterTip(ctx, tipNum, lastConnectedNum)
+ }
+
return nil
}
- if err := s.commitExecution(ctx, newTip, ccBuilder.Root()); err != nil {
- if !errors.Is(err, ErrBadForkChoiceUpdate) {
- return err
+ if latestValidHash, err := s.commitExecution(ctx, newTip, ccBuilder.Root()); err != nil {
+ if errors.Is(err, ErrForkChoiceUpdateBadBlock) {
+ return s.handleBadBlockErr(ctx, ccBuilder, event, latestValidHash, err)
}
- s.logger.Warn(syncLogPrefix("bad block after execution"), "peerId", event.PeerId, "err", err)
- s.badBlocks.Add(event.NewBlock.Hash(), struct{}{})
- s.maybePenalizePeerOnBadBlockEvent(ctx, event)
- return nil
+ return err
}
if event.Source == EventSourceP2PNewBlock {
@@ -442,6 +443,103 @@ func (s *Sync) publishNewBlock(ctx context.Context, block *types.Block) {
s.p2pService.PublishNewBlock(block, td)
}
+func (s *Sync) handleTipForkChangeUnwinds(ctx context.Context, ccb CanonicalChainBuilder, oldTip *types.Header) error {
+ // forks have changed, we need to unwind unwindable data
+ newTip := ccb.Tip()
+ newTipHash := newTip.Hash()
+ oldTipHash := oldTip.Hash()
+ oldTipNum := oldTip.Number.Uint64()
+ s.logger.Debug(
+ syncLogPrefix("fork change"),
+ "oldNum", oldTipNum,
+ "oldHash", oldTipHash,
+ "newNum", newTip.Number.Uint64(),
+ "newHash", newTipHash,
+ )
+
+ lca, ok := ccb.LowestCommonAncestor(newTipHash, oldTipHash)
+ if !ok {
+ return fmt.Errorf("could not find lowest common ancestor of old and new tip")
+ }
+
+ lcaNum := lca.Number.Uint64()
+ if err := s.bridgeSync.Unwind(ctx, lcaNum); err != nil {
+ return err
+ }
+
+ start := lcaNum + 1
+ amount := oldTipNum - start + 1
+ canonicalHeaders := ccb.HeadersInRange(start, amount)
+ canonicalBlocksToReplay := make([]*types.Block, len(canonicalHeaders))
+ for i, header := range canonicalHeaders {
+ canonicalBlocksToReplay[i] = types.NewBlockWithHeader(header)
+ }
+
+ if err := s.bridgeSync.ProcessNewBlocks(ctx, canonicalBlocksToReplay); err != nil {
+ return err
+ }
+
+ return s.bridgeSync.Synchronize(ctx, oldTipNum)
+}
+
+func (s *Sync) handleInsertBlocksAfterTip(ctx context.Context, tipNum, lastInsertedNum uint64) error {
+ // this is a hack that should disappear when changing the bridge to not track blocks (future work)
+ // make sure the bridge does not go past the tip (it may happen when we insert blocks from another fork that
+ // has a higher block number than the canonical tip but lower difficulty) - this is to prevent the bridge
+ // from recording incorrect bor txn hashes
+ s.logger.Debug(
+ syncLogPrefix("unwinding back bridge due to inserting headers past the tip"),
+ "tip", tipNum,
+ "lastInsertedNum", lastInsertedNum,
+ )
+
+ // wait for the insert blocks flush
+ if err := s.store.Flush(ctx); err != nil {
+ return err
+ }
+
+ // wait for the bridge processing
+ if err := s.bridgeSync.Synchronize(ctx, lastInsertedNum); err != nil {
+ return err
+ }
+
+ return s.bridgeSync.Unwind(ctx, tipNum)
+}
+
+func (s *Sync) handleBadBlockErr(
+ ctx context.Context,
+ ccb CanonicalChainBuilder,
+ event EventNewBlock,
+ latestValidHash common.Hash,
+ badBlockErr error,
+) error {
+ s.logger.Warn(
+ syncLogPrefix("bad block after execution"),
+ "peerId", event.PeerId,
+ "latestValidHash", latestValidHash,
+ "err", badBlockErr,
+ )
+
+ s.badBlocks.Add(event.NewBlock.Hash(), struct{}{})
+ s.maybePenalizePeerOnBadBlockEvent(ctx, event)
+
+ lastValidHeader, ok := ccb.HeaderByHash(latestValidHash)
+ if !ok {
+ return fmt.Errorf("unexpected latestValidHash not in canonical builder: %s", latestValidHash)
+ }
+ badHeaders := ccb.HeadersInRange(lastValidHeader.Number.Uint64()+1, 1)
+ if len(badHeaders) == 0 {
+ return errors.New("expected at least one bad header after bad block err")
+ }
+
+ if err := ccb.PruneNode(badHeaders[0].Hash()); err != nil {
+ return err
+ }
+
+ tipPostPrune := ccb.Tip()
+ return s.bridgeSync.Unwind(ctx, tipPostPrune.Number.Uint64())
+}
+
func (s *Sync) maybePenalizePeerOnBadBlockEvent(ctx context.Context, event EventNewBlock) {
if event.Source == EventSourceP2PNewBlockHashes {
// note: we do not penalize peer for bad blocks on new block hash events since they have
@@ -616,7 +714,7 @@ func (s *Sync) sync(ctx context.Context, tip *types.Header, tipDownloader tipDow
}
tip = newResult.latestTip
- if err = s.commitExecution(ctx, tip, tip); err != nil {
+ if _, err := s.commitExecution(ctx, tip, tip); err != nil {
// note: if we face a failure during execution of finalized waypoints blocks
// it means that we're wrong and the blocks are not considered as bad blocks
return syncToTipResult{}, err
From 64747a184ed120776fe748a1974faaec195580b9 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Tue, 15 Oct 2024 22:48:23 +0100
Subject: [PATCH 14/25] wip
---
polygon/sync/sync.go | 31 ++++++++++++++++++++++++++++---
1 file changed, 28 insertions(+), 3 deletions(-)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index e18d1d77ecc..b5c07dc743c 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -468,8 +468,21 @@ func (s *Sync) handleTipForkChangeUnwinds(ctx context.Context, ccb CanonicalChai
}
start := lcaNum + 1
+ if oldTipNum < start { // defensive check against underflow
+ return fmt.Errorf("unexpected oldTipNum < start: %d < %d", oldTipNum, start)
+ }
+
amount := oldTipNum - start + 1
canonicalHeaders := ccb.HeadersInRange(start, amount)
+ if uint64(len(canonicalHeaders)) != amount {
+ return fmt.Errorf("expected %d canonical headers", amount)
+ }
+
+ //
+ // TODO double-check above defensive if checks - do they make sense?
+ // is it possible that len(canonicalHeaders)) != amount if the new tip is actually lower height than the old tip?
+ //
+
canonicalBlocksToReplay := make([]*types.Block, len(canonicalHeaders))
for i, header := range canonicalHeaders {
canonicalBlocksToReplay[i] = types.NewBlockWithHeader(header)
@@ -527,9 +540,21 @@ func (s *Sync) handleBadBlockErr(
if !ok {
return fmt.Errorf("unexpected latestValidHash not in canonical builder: %s", latestValidHash)
}
- badHeaders := ccb.HeadersInRange(lastValidHeader.Number.Uint64()+1, 1)
- if len(badHeaders) == 0 {
- return errors.New("expected at least one bad header after bad block err")
+
+ badTipNum := ccb.Tip().Number.Uint64()
+ start := lastValidHeader.Number.Uint64() + 1
+ if badTipNum < start { // defensive check against underflow
+ return fmt.Errorf("unexpected badTipNum < start: %d < %d", badTipNum, start)
+ }
+
+ amount := badTipNum - start + 1
+ badHeaders := ccb.HeadersInRange(lastValidHeader.Number.Uint64()+1, amount)
+ if uint64(len(badHeaders)) != amount {
+ return fmt.Errorf("expected %d bad headers after bad block err", amount)
+ }
+
+ for _, badHeader := range badHeaders {
+ s.badBlocks.Add(badHeader.Hash(), struct{}{})
}
if err := ccb.PruneNode(badHeaders[0].Hash()); err != nil {
From 2b817949868b65826530967e048acdc940f535d4 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 19:37:13 +0100
Subject: [PATCH 15/25] wip: double check things
---
polygon/sync/sync.go | 39 +++++++++++++++------------------------
1 file changed, 15 insertions(+), 24 deletions(-)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index b5c07dc743c..a6a7ad7a1d6 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -340,7 +340,7 @@ func (s *Sync) applyNewBlockOnTip(
if newTip == oldTip {
lastConnectedNum := newConnectedHeaders[len(newConnectedHeaders)-1].Number.Uint64()
if tipNum := newTip.Number.Uint64(); lastConnectedNum > tipNum {
- return s.handleInsertBlocksAfterTip(ctx, tipNum, lastConnectedNum)
+ return s.handleInsertBlocksAfterUnchangedTip(ctx, tipNum, lastConnectedNum)
}
return nil
@@ -446,14 +446,15 @@ func (s *Sync) publishNewBlock(ctx context.Context, block *types.Block) {
func (s *Sync) handleTipForkChangeUnwinds(ctx context.Context, ccb CanonicalChainBuilder, oldTip *types.Header) error {
// forks have changed, we need to unwind unwindable data
newTip := ccb.Tip()
+ newTipNum := newTip.Number.Uint64()
newTipHash := newTip.Hash()
- oldTipHash := oldTip.Hash()
oldTipNum := oldTip.Number.Uint64()
+ oldTipHash := oldTip.Hash()
s.logger.Debug(
syncLogPrefix("fork change"),
"oldNum", oldTipNum,
"oldHash", oldTipHash,
- "newNum", newTip.Number.Uint64(),
+ "newNum", newTipNum,
"newHash", newTipHash,
)
@@ -463,25 +464,20 @@ func (s *Sync) handleTipForkChangeUnwinds(ctx context.Context, ccb CanonicalChai
}
lcaNum := lca.Number.Uint64()
- if err := s.bridgeSync.Unwind(ctx, lcaNum); err != nil {
- return err
- }
-
start := lcaNum + 1
- if oldTipNum < start { // defensive check against underflow
- return fmt.Errorf("unexpected oldTipNum < start: %d < %d", oldTipNum, start)
+ if newTipNum < start { // defensive check against underflow & unexpected lcaNum
+ return fmt.Errorf("unexpected newTipNum < start: %d < %d", oldTipNum, start)
}
- amount := oldTipNum - start + 1
+ amount := newTipNum - start + 1
canonicalHeaders := ccb.HeadersInRange(start, amount)
if uint64(len(canonicalHeaders)) != amount {
return fmt.Errorf("expected %d canonical headers", amount)
}
- //
- // TODO double-check above defensive if checks - do they make sense?
- // is it possible that len(canonicalHeaders)) != amount if the new tip is actually lower height than the old tip?
- //
+ if err := s.bridgeSync.Unwind(ctx, lcaNum); err != nil {
+ return err
+ }
canonicalBlocksToReplay := make([]*types.Block, len(canonicalHeaders))
for i, header := range canonicalHeaders {
@@ -495,7 +491,7 @@ func (s *Sync) handleTipForkChangeUnwinds(ctx context.Context, ccb CanonicalChai
return s.bridgeSync.Synchronize(ctx, oldTipNum)
}
-func (s *Sync) handleInsertBlocksAfterTip(ctx context.Context, tipNum, lastInsertedNum uint64) error {
+func (s *Sync) handleInsertBlocksAfterUnchangedTip(ctx context.Context, tipNum, lastInsertedNum uint64) error {
// this is a hack that should disappear when changing the bridge to not track blocks (future work)
// make sure the bridge does not go past the tip (it may happen when we insert blocks from another fork that
// has a higher block number than the canonical tip but lower difficulty) - this is to prevent the bridge
@@ -535,28 +531,23 @@ func (s *Sync) handleBadBlockErr(
s.badBlocks.Add(event.NewBlock.Hash(), struct{}{})
s.maybePenalizePeerOnBadBlockEvent(ctx, event)
-
- lastValidHeader, ok := ccb.HeaderByHash(latestValidHash)
+ latestValidHeader, ok := ccb.HeaderByHash(latestValidHash)
if !ok {
return fmt.Errorf("unexpected latestValidHash not in canonical builder: %s", latestValidHash)
}
badTipNum := ccb.Tip().Number.Uint64()
- start := lastValidHeader.Number.Uint64() + 1
- if badTipNum < start { // defensive check against underflow
+ start := latestValidHeader.Number.Uint64() + 1
+ if badTipNum < start { // defensive check against underflow & unexpected badTipNum and latestValidHeader
return fmt.Errorf("unexpected badTipNum < start: %d < %d", badTipNum, start)
}
amount := badTipNum - start + 1
- badHeaders := ccb.HeadersInRange(lastValidHeader.Number.Uint64()+1, amount)
+ badHeaders := ccb.HeadersInRange(start, amount)
if uint64(len(badHeaders)) != amount {
return fmt.Errorf("expected %d bad headers after bad block err", amount)
}
- for _, badHeader := range badHeaders {
- s.badBlocks.Add(badHeader.Hash(), struct{}{})
- }
-
if err := ccb.PruneNode(badHeaders[0].Hash()); err != nil {
return err
}
From 11f4ae54bb5b62e1b6eac1466e9dca97a27aad3a Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 19:50:59 +0100
Subject: [PATCH 16/25] add waiting for chain tip events info log
---
polygon/sync/sync.go | 14 ++++++++++++++
1 file changed, 14 insertions(+)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index a6a7ad7a1d6..740d01370ba 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -585,6 +585,10 @@ func (s *Sync) Run(ctx context.Context) error {
return err
}
+ inactivityDuration := 30 * time.Second
+ lastProcessedEventTime := time.Now()
+ inactivityTicker := time.NewTicker(inactivityDuration)
+ defer inactivityTicker.Stop()
for {
select {
case event := <-s.events:
@@ -601,7 +605,17 @@ func (s *Sync) Run(ctx context.Context) error {
if err = s.applyNewBlockHashesOnTip(ctx, event.AsNewBlockHashes(), ccBuilder); err != nil {
return err
}
+ default:
+ panic(fmt.Sprintf("unexpected event type: %v", event.Type))
}
+
+ lastProcessedEventTime = time.Now()
+ case <-inactivityTicker.C:
+ if time.Since(lastProcessedEventTime) < inactivityDuration {
+ continue
+ }
+
+ s.logger.Info(syncLogPrefix("waiting for chain tip events..."))
case <-ctx.Done():
return ctx.Err()
}
From 382ebdd1127b622e877e339d87dde8b67e347cf0 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 20:26:36 +0100
Subject: [PATCH 17/25] handleWaypointExecutionErr
---
polygon/sync/sync.go | 30 +++++++++++++++++++++++-------
1 file changed, 23 insertions(+), 7 deletions(-)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index 740d01370ba..7441e9ee376 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -157,9 +157,9 @@ func (s *Sync) handleMilestoneTipMismatch(
}
if _, err := s.commitExecution(ctx, newTip, newTip); err != nil {
- // note: if we face a failure during execution of finalized waypoints blocks
- // it means that we're wrong and the blocks are not considered as bad blocks
- return err
+ // note: if we face a failure during execution of finalized waypoints blocks, it means that
+ // we're wrong and the blocks are not considered as bad blocks, so we should terminate
+ return s.handleWaypointExecutionErr(ctx, ccBuilder.Root(), err)
}
ccBuilder.Reset(newTip)
@@ -743,17 +743,33 @@ func (s *Sync) sync(ctx context.Context, tip *types.Header, tipDownloader tipDow
break
}
- tip = newResult.latestTip
- if _, err := s.commitExecution(ctx, tip, tip); err != nil {
- // note: if we face a failure during execution of finalized waypoints blocks
- // it means that we're wrong and the blocks are not considered as bad blocks
+ newTip := newResult.latestTip
+ if _, err := s.commitExecution(ctx, newTip, newTip); err != nil {
+ // note: if we face a failure during execution of finalized waypoints blocks, it means that
+ // we're wrong and the blocks are not considered as bad blocks, so we should terminate
+ err = s.handleWaypointExecutionErr(ctx, tip, err)
return syncToTipResult{}, err
}
+
+ tip = newTip
}
return syncToTipResult{latestTip: tip, latestWaypoint: latestWaypoint}, nil
}
+func (s *Sync) handleWaypointExecutionErr(ctx context.Context, lastCorrectTip *types.Header, execErr error) error {
+ if !errors.Is(execErr, ErrForkChoiceUpdateBadBlock) {
+ return execErr
+ }
+
+ // if it is a bad block try to unwind the bridge to the last known tip so we leave it in a good state
+ if bridgeUnwindErr := s.bridgeSync.Unwind(ctx, lastCorrectTip.Number.Uint64()); bridgeUnwindErr != nil {
+ return fmt.Errorf("%w: %w", bridgeUnwindErr, execErr)
+ }
+
+ return execErr
+}
+
func (s *Sync) ignoreFetchBlocksErrOnTipEvent(err error) bool {
return errors.Is(err, &p2p.ErrIncompleteHeaders{}) ||
errors.Is(err, &p2p.ErrNonSequentialHeaderNumbers{}) ||
From ddba4c06cbe6008f8d7507043eb222e857bdf891 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 21:58:24 +0100
Subject: [PATCH 18/25] implement and test LCA
---
polygon/sync/canonical_chain_builder.go | 48 +++++++---
polygon/sync/canonical_chain_builder_test.go | 93 +++++++++++++++++++-
2 files changed, 127 insertions(+), 14 deletions(-)
diff --git a/polygon/sync/canonical_chain_builder.go b/polygon/sync/canonical_chain_builder.go
index 47f29036f66..8f22738fdae 100644
--- a/polygon/sync/canonical_chain_builder.go
+++ b/polygon/sync/canonical_chain_builder.go
@@ -356,28 +356,54 @@ func (ccb *canonicalChainBuilder) Connect(ctx context.Context, headers []*types.
}
func (ccb *canonicalChainBuilder) LowestCommonAncestor(a, b libcommon.Hash) (*types.Header, bool) {
- /*pathA*/ _ = ccb.pathFromRoot(a)
- /*pathB*/ _ = ccb.pathFromRoot(b)
- //
- // TODO
- //
+ pathA := ccb.pathToRoot(a)
+ if len(pathA) == 0 {
+ // 'a' doesn't exist in the tree
+ return nil, false
+ }
+
+ pathB := ccb.pathToRoot(b)
+ if len(pathB) == 0 {
+ // 'b' doesn't exist in the tree
+ return nil, false
+ }
+
+ heightA := pathA[0].header.Number.Uint64()
+ heightB := pathB[0].header.Number.Uint64()
+ for heightA != heightB {
+ if heightA < heightB {
+ pathB = pathB[1:]
+ heightB = pathB[0].header.Number.Uint64()
+ } else if heightA > heightB {
+ pathA = pathA[1:]
+ heightA = pathA[0].header.Number.Uint64()
+ }
+ }
+
+ for i := 0; i < len(pathA); i++ {
+ if pathA[i].headerHash == pathB[i].headerHash {
+ return pathA[i].header, true
+ }
+ }
+
return nil, false
}
-func (ccb *canonicalChainBuilder) pathFromRoot(to libcommon.Hash) []*forkTreeNode {
+func (ccb *canonicalChainBuilder) pathToRoot(from libcommon.Hash) []*forkTreeNode {
path := make([]*forkTreeNode, 0, ccb.Tip().Number.Uint64()-ccb.Root().Number.Uint64())
- pathFromRootRec(ccb.root, to, &path)
+ pathToRootRec(ccb.root, from, &path)
return path
}
-func pathFromRootRec(node *forkTreeNode, to libcommon.Hash, path *[]*forkTreeNode) bool {
- if node.headerHash == to {
+func pathToRootRec(node *forkTreeNode, from libcommon.Hash, path *[]*forkTreeNode) bool {
+ if node.headerHash == from {
+ *path = append(*path, node)
return true
}
for _, child := range node.children {
- if pathFromRootRec(child, to, path) {
- *path = append(*path, child)
+ if pathToRootRec(child, from, path) {
+ *path = append(*path, node)
return true
}
}
diff --git a/polygon/sync/canonical_chain_builder_test.go b/polygon/sync/canonical_chain_builder_test.go
index 070cce90293..ef06812dfc4 100644
--- a/polygon/sync/canonical_chain_builder_test.go
+++ b/polygon/sync/canonical_chain_builder_test.go
@@ -127,6 +127,7 @@ func (test *connectCCBTest) testConnect(
}
func TestCCBEmptyState(t *testing.T) {
+ t.Parallel()
test, root := newConnectCCBTest(t)
tip := test.builder.Tip()
@@ -138,6 +139,7 @@ func TestCCBEmptyState(t *testing.T) {
}
func TestCCBConnectEmpty(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -146,6 +148,7 @@ func TestCCBConnectEmpty(t *testing.T) {
// connect 0 to 0
func TestCCBConnectRoot(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -154,6 +157,7 @@ func TestCCBConnectRoot(t *testing.T) {
// connect 1 to 0
func TestCCBConnectOneToRoot(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -163,6 +167,7 @@ func TestCCBConnectOneToRoot(t *testing.T) {
// connect 1-2-3 to 0
func TestCCBConnectSomeToRoot(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -172,6 +177,7 @@ func TestCCBConnectSomeToRoot(t *testing.T) {
// connect any subset of 0-1-2-3 to 0-1-2-3
func TestCCBConnectOverlapsFull(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -193,6 +199,7 @@ func TestCCBConnectOverlapsFull(t *testing.T) {
// connect 0-1 to 0
func TestCCBConnectOverlapPartialOne(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -202,6 +209,7 @@ func TestCCBConnectOverlapPartialOne(t *testing.T) {
// connect 2-3-4-5 to 0-1-2-3
func TestCCBConnectOverlapPartialSome(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -219,6 +227,7 @@ func TestCCBConnectOverlapPartialSome(t *testing.T) {
// connect 2 to 0-1 at 0, then connect 10 to 0-1
func TestCCBConnectAltMainBecomesFork(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -237,6 +246,7 @@ func TestCCBConnectAltMainBecomesFork(t *testing.T) {
// connect 1 to 0-2 at 0, then connect 10 to 0-1
func TestCCBConnectAltForkBecomesMain(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -255,6 +265,7 @@ func TestCCBConnectAltForkBecomesMain(t *testing.T) {
// connect 10 and 11 to 1, then 20 and 22 to 2 one by one starting from a [0-1, 0-2] tree
func TestCCBConnectAltForksAtLevel2(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -280,6 +291,7 @@ func TestCCBConnectAltForksAtLevel2(t *testing.T) {
// connect 11 and 10 to 1, then 22 and 20 to 2 one by one starting from a [0-1, 0-2] tree
// then connect 100 to 10, and 200 to 20
func TestCCBConnectAltForksAtLevel2Reverse(t *testing.T) {
+ t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
test, root := newConnectCCBTest(t)
@@ -320,7 +332,82 @@ func TestCCBHeaderByHash(t *testing.T) {
}
func TestCCBLowestCommonAncestor(t *testing.T) {
- //
- // TODO
- //
+ t.Parallel()
+ ctx, cancel := context.WithCancel(context.Background())
+ t.Cleanup(cancel)
+ // R(td:0) -> A(td:1) -> B(td:2)
+ // |
+ // +--------> X(td:2) -> Y(td:4) -> Z(td:6)
+ // | |
+ // | +--------> K(td:5)
+ // |
+ // +--------> P(td:3)
+ test, headerR := newConnectCCBTest(t)
+ ccb := test.builder
+ headerA := test.makeHeader(headerR, 1)
+ headerB := test.makeHeader(headerA, 1)
+ _, err := ccb.Connect(ctx, []*types.Header{headerA, headerB})
+ require.NoError(t, err)
+ headerX := test.makeHeader(headerR, 2)
+ headerY := test.makeHeader(headerX, 2)
+ headerZ := test.makeHeader(headerY, 2)
+ _, err = ccb.Connect(ctx, []*types.Header{headerX, headerY, headerZ})
+ require.NoError(t, err)
+ headerK := test.makeHeader(headerX, 3)
+ _, err = ccb.Connect(ctx, []*types.Header{headerK})
+ require.NoError(t, err)
+ headerP := test.makeHeader(headerR, 3)
+ _, err = ccb.Connect(ctx, []*types.Header{headerP})
+ require.NoError(t, err)
+ require.Equal(t, headerZ, ccb.Tip())
+ headerU := &types.Header{Number: big.NewInt(777)}
+ headerU2 := &types.Header{Number: big.NewInt(999)}
+ t.Run("LCA(R,U)=nil,false", func(t *testing.T) {
+ assertLca(t, ccb, headerR, headerU, nil, false)
+ })
+ t.Run("LCA(U,R)=nil,false", func(t *testing.T) {
+ assertLca(t, ccb, headerU, headerR, nil, false)
+ })
+ t.Run("LCA(U,U)=nil,false", func(t *testing.T) {
+ assertLca(t, ccb, headerU, headerU, nil, false)
+ })
+ t.Run("LCA(U2,U)=nil,false", func(t *testing.T) {
+ assertLca(t, ccb, headerU2, headerU, nil, false)
+ })
+ t.Run("LCA(R,R)=R", func(t *testing.T) {
+ assertLca(t, ccb, headerR, headerR, headerR, true)
+ })
+ t.Run("LCA(Y,Y)=Y", func(t *testing.T) {
+ assertLca(t, ccb, headerY, headerY, headerY, true)
+ })
+ t.Run("LCA(Y,Z)=Y", func(t *testing.T) {
+ assertLca(t, ccb, headerY, headerZ, headerY, true)
+ })
+ t.Run("LCA(X,Y)=X", func(t *testing.T) {
+ assertLca(t, ccb, headerX, headerY, headerX, true)
+ })
+ t.Run("LCA(R,Z)=R", func(t *testing.T) {
+ assertLca(t, ccb, headerR, headerZ, headerR, true)
+ })
+ t.Run("LCA(R,A)=R", func(t *testing.T) {
+ assertLca(t, ccb, headerR, headerA, headerR, true)
+ })
+ t.Run("LCA(R,P)=R", func(t *testing.T) {
+ assertLca(t, ccb, headerR, headerP, headerR, true)
+ })
+ t.Run("LCA(K,B)=R", func(t *testing.T) {
+ assertLca(t, ccb, headerK, headerB, headerR, true)
+ })
+ t.Run("LCA(X,A)=R", func(t *testing.T) {
+ assertLca(t, ccb, headerX, headerA, headerR, true)
+ })
+ t.Run("LCA(Z,K)=X", func(t *testing.T) {
+ assertLca(t, ccb, headerZ, headerK, headerX, true)
+ })
+}
+
+func assertLca(t *testing.T, ccb CanonicalChainBuilder, a, b, wantLca *types.Header, wantOk bool) {
+ lca, ok := ccb.LowestCommonAncestor(a.Hash(), b.Hash())
+ require.Equal(t, wantOk, ok)
+ require.Equal(t, wantLca, lca)
}
From 36b816df418ce04be0f6d708634c445878c1540e Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 22:14:29 +0100
Subject: [PATCH 19/25] header by hash ccb tests
---
polygon/sync/canonical_chain_builder_test.go | 40 ++++++++++++++++++--
1 file changed, 37 insertions(+), 3 deletions(-)
diff --git a/polygon/sync/canonical_chain_builder_test.go b/polygon/sync/canonical_chain_builder_test.go
index ef06812dfc4..956488adbe3 100644
--- a/polygon/sync/canonical_chain_builder_test.go
+++ b/polygon/sync/canonical_chain_builder_test.go
@@ -27,6 +27,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ libcommon "github.com/erigontech/erigon-lib/common"
"github.com/erigontech/erigon/core/types"
)
@@ -326,9 +327,42 @@ func TestCCBPruneNode(t *testing.T) {
}
func TestCCBHeaderByHash(t *testing.T) {
- //
- // TODO
- //
+ t.Parallel()
+ ctx, cancel := context.WithCancel(context.Background())
+ t.Cleanup(cancel)
+ // R(td:0) -> A(td:1)
+ // |
+ // +--------> B(td:2) -> Z(td4)
+ // |
+ // +--------> Y(td3)
+ test, headerR := newConnectCCBTest(t)
+ ccb := test.builder
+ headerA := test.makeHeader(headerR, 1)
+ _, err := ccb.Connect(ctx, []*types.Header{headerA})
+ require.NoError(t, err)
+ headerB := test.makeHeader(headerR, 2)
+ headerZ := test.makeHeader(headerB, 2)
+ _, err = ccb.Connect(ctx, []*types.Header{headerB, headerZ})
+ require.NoError(t, err)
+ headerY := test.makeHeader(headerB, 1)
+ _, err = ccb.Connect(ctx, []*types.Header{headerY})
+ require.NoError(t, err)
+ require.Equal(t, headerZ, ccb.Tip())
+ assertHeaderByHash(t, ccb, headerR.Hash(), headerR, true)
+ assertHeaderByHash(t, ccb, headerA.Hash(), headerA, true)
+ assertHeaderByHash(t, ccb, headerB.Hash(), headerB, true)
+ assertHeaderByHash(t, ccb, headerZ.Hash(), headerZ, true)
+ assertHeaderByHash(t, ccb, headerY.Hash(), headerY, true)
+ headerU := &types.Header{Number: big.NewInt(1234)}
+ assertHeaderByHash(t, ccb, headerU.Hash(), nil, false)
+ headerU2 := &types.Header{Number: big.NewInt(77)}
+ assertHeaderByHash(t, ccb, headerU2.Hash(), nil, false)
+}
+
+func assertHeaderByHash(t *testing.T, ccb CanonicalChainBuilder, h libcommon.Hash, wantHeader *types.Header, wantOk bool) {
+ header, ok := ccb.HeaderByHash(h)
+ require.Equal(t, wantOk, ok)
+ require.Equal(t, wantHeader, header)
}
func TestCCBLowestCommonAncestor(t *testing.T) {
From 69b7969b7db8a96d99cdb9f9b70fb6d1154d6b1f Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 22:55:08 +0100
Subject: [PATCH 20/25] prune node tests
---
polygon/sync/canonical_chain_builder.go | 2 +-
polygon/sync/canonical_chain_builder_test.go | 128 ++++++++++++++++++-
2 files changed, 126 insertions(+), 4 deletions(-)
diff --git a/polygon/sync/canonical_chain_builder.go b/polygon/sync/canonical_chain_builder.go
index 8f22738fdae..911e67360a4 100644
--- a/polygon/sync/canonical_chain_builder.go
+++ b/polygon/sync/canonical_chain_builder.go
@@ -189,7 +189,7 @@ func (ccb *canonicalChainBuilder) PruneRoot(newRootNum uint64) error {
func (ccb *canonicalChainBuilder) PruneNode(hash libcommon.Hash) error {
if ccb.root.headerHash == hash {
- return errors.New("canonicalChainBuilder.PruneNode: can't remove root node")
+ return errors.New("canonicalChainBuilder.PruneNode: can't prune root node")
}
var exists bool
diff --git a/polygon/sync/canonical_chain_builder_test.go b/polygon/sync/canonical_chain_builder_test.go
index 956488adbe3..d7521a6d7bc 100644
--- a/polygon/sync/canonical_chain_builder_test.go
+++ b/polygon/sync/canonical_chain_builder_test.go
@@ -321,9 +321,131 @@ func TestCCBConnectAltForksAtLevel2Reverse(t *testing.T) {
}
func TestCCBPruneNode(t *testing.T) {
- //
- // TODO
- //
+ t.Parallel()
+ ctx, cancel := context.WithCancel(context.Background())
+ t.Cleanup(cancel)
+ // R(td:0) -> A(td:1) -> B(td:2)
+ // |
+ // +--------> X(td:2) -> Y(td:4) -> Z(td:6)
+ // | |
+ // | +--------> K(td:5)
+ // |
+ // +--------> P(td:3)
+ type example struct {
+ ccb CanonicalChainBuilder
+ headerR *types.Header
+ headerA *types.Header
+ headerB *types.Header
+ headerX *types.Header
+ headerY *types.Header
+ headerZ *types.Header
+ headerK *types.Header
+ headerP *types.Header
+ }
+ constructExample := func() example {
+ test, headerR := newConnectCCBTest(t)
+ ccb := test.builder
+ headerA := test.makeHeader(headerR, 1)
+ headerB := test.makeHeader(headerA, 1)
+ _, err := ccb.Connect(ctx, []*types.Header{headerA, headerB})
+ require.NoError(t, err)
+ headerX := test.makeHeader(headerR, 2)
+ headerY := test.makeHeader(headerX, 2)
+ headerZ := test.makeHeader(headerY, 2)
+ _, err = ccb.Connect(ctx, []*types.Header{headerX, headerY, headerZ})
+ require.NoError(t, err)
+ headerK := test.makeHeader(headerX, 3)
+ _, err = ccb.Connect(ctx, []*types.Header{headerK})
+ require.NoError(t, err)
+ headerP := test.makeHeader(headerR, 3)
+ _, err = ccb.Connect(ctx, []*types.Header{headerP})
+ require.NoError(t, err)
+ require.Equal(t, headerZ, ccb.Tip())
+ return example{
+ ccb: ccb,
+ headerR: headerR,
+ headerA: headerA,
+ headerB: headerB,
+ headerX: headerX,
+ headerY: headerY,
+ headerZ: headerZ,
+ headerK: headerK,
+ headerP: headerP,
+ }
+ }
+ t.Run("unknown hash", func(t *testing.T) {
+ ex := constructExample()
+ headerU := &types.Header{Number: big.NewInt(777)}
+ err := ex.ccb.PruneNode(headerU.Hash())
+ require.Error(t, err)
+ require.Contains(t, err.Error(), "could not find node to prune")
+ })
+ t.Run("can't prune root", func(t *testing.T) {
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerR.Hash())
+ require.Error(t, err)
+ require.Contains(t, err.Error(), "can't prune root node")
+ })
+ t.Run("prune Z - change of tip", func(t *testing.T) {
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerZ.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerK, ex.ccb.Tip())
+ })
+ t.Run("prune Y - change of tip", func(t *testing.T) {
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerY.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerK, ex.ccb.Tip())
+ })
+ t.Run("prune K - no change of tip", func(t *testing.T) {
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerK.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerZ, ex.ccb.Tip())
+ })
+ t.Run("prune X - no change of tip", func(t *testing.T) {
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerX.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerP, ex.ccb.Tip())
+ })
+ t.Run("prune P - no change of tip", func(t *testing.T) {
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerP.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerZ, ex.ccb.Tip())
+ })
+ t.Run("prune A - no change of tip", func(t *testing.T) {
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerA.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerZ, ex.ccb.Tip())
+ })
+ t.Run("prune P, prune Y, prune K, prune X, prune A", func(t *testing.T) {
+ // prune P - no change (tip Z)
+ ex := constructExample()
+ err := ex.ccb.PruneNode(ex.headerP.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerZ, ex.ccb.Tip())
+ // prune Y - change (tip K)
+ err = ex.ccb.PruneNode(ex.headerY.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerK, ex.ccb.Tip())
+ // prune K - change (tip X)
+ err = ex.ccb.PruneNode(ex.headerK.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerX, ex.ccb.Tip())
+ // prune X - change (tip B)
+ err = ex.ccb.PruneNode(ex.headerX.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerB, ex.ccb.Tip())
+ // prune A - change (tip R) - only root left
+ err = ex.ccb.PruneNode(ex.headerA.Hash())
+ require.NoError(t, err)
+ require.Equal(t, ex.headerR, ex.ccb.Tip())
+ require.Equal(t, ex.headerR, ex.ccb.Root())
+ })
}
func TestCCBHeaderByHash(t *testing.T) {
From c0c59b881b98a608066679b7ac8034ce7208bfc2 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 23:00:55 +0100
Subject: [PATCH 21/25] stylistic
---
polygon/sync/canonical_chain_builder_test.go | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/polygon/sync/canonical_chain_builder_test.go b/polygon/sync/canonical_chain_builder_test.go
index d7521a6d7bc..e9e87823a19 100644
--- a/polygon/sync/canonical_chain_builder_test.go
+++ b/polygon/sync/canonical_chain_builder_test.go
@@ -27,7 +27,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
- libcommon "github.com/erigontech/erigon-lib/common"
+ "github.com/erigontech/erigon-lib/common"
"github.com/erigontech/erigon/core/types"
)
@@ -481,7 +481,7 @@ func TestCCBHeaderByHash(t *testing.T) {
assertHeaderByHash(t, ccb, headerU2.Hash(), nil, false)
}
-func assertHeaderByHash(t *testing.T, ccb CanonicalChainBuilder, h libcommon.Hash, wantHeader *types.Header, wantOk bool) {
+func assertHeaderByHash(t *testing.T, ccb CanonicalChainBuilder, h common.Hash, wantHeader *types.Header, wantOk bool) {
header, ok := ccb.HeaderByHash(h)
require.Equal(t, wantOk, ok)
require.Equal(t, wantHeader, header)
From e3f5a3dbbec1404b80f97c9db474c097b65807e4 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Wed, 16 Oct 2024 23:05:20 +0100
Subject: [PATCH 22/25] fix compilation
---
eth/stagedsync/stage_polygon_sync.go | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/eth/stagedsync/stage_polygon_sync.go b/eth/stagedsync/stage_polygon_sync.go
index 14727a2244a..00d921d34f0 100644
--- a/eth/stagedsync/stage_polygon_sync.go
+++ b/eth/stagedsync/stage_polygon_sync.go
@@ -1516,7 +1516,7 @@ func (e *polygonSyncStageExecutionEngine) UpdateForkChoice(ctx context.Context,
case result := <-resultCh:
err := result.validationErr
if err != nil {
- err = fmt.Errorf("%w: %w", polygonsync.ErrBadForkChoiceUpdate, err)
+ err = fmt.Errorf("%w: %w", polygonsync.ErrForkChoiceUpdateBadBlock, err)
}
return result.latestValidHash, err
}
From 0811185cfbf8f21afc15a6424259aba4cfa2c267 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 17 Oct 2024 14:50:35 +0100
Subject: [PATCH 23/25] simplify bridge reorg and handling
---
polygon/sync/canonical_chain_builder.go | 15 --
polygon/sync/canonical_chain_builder_mock.go | 39 -----
polygon/sync/canonical_chain_builder_test.go | 40 -----
polygon/sync/sync.go | 149 +++++++++++--------
4 files changed, 87 insertions(+), 156 deletions(-)
diff --git a/polygon/sync/canonical_chain_builder.go b/polygon/sync/canonical_chain_builder.go
index 911e67360a4..8321246e783 100644
--- a/polygon/sync/canonical_chain_builder.go
+++ b/polygon/sync/canonical_chain_builder.go
@@ -36,7 +36,6 @@ type CanonicalChainBuilder interface {
Tip() *types.Header
Root() *types.Header
HeadersInRange(start uint64, count uint64) []*types.Header
- HeaderByHash(hash libcommon.Hash) (header *types.Header, ok bool)
PruneRoot(newRootNum uint64) error
PruneNode(hash libcommon.Hash) error
Connect(ctx context.Context, headers []*types.Header) (newConnectedHeaders []*types.Header, err error)
@@ -159,20 +158,6 @@ func (ccb *canonicalChainBuilder) HeadersInRange(start uint64, count uint64) []*
return headers[offset : offset+count]
}
-func (ccb *canonicalChainBuilder) HeaderByHash(hash libcommon.Hash) (header *types.Header, ok bool) {
- ccb.enumerate(func(node *forkTreeNode) bool {
- if node.headerHash == hash {
- header = node.header
- ok = true
- return false
- }
-
- return true
- })
-
- return header, ok
-}
-
func (ccb *canonicalChainBuilder) PruneRoot(newRootNum uint64) error {
if (newRootNum < ccb.root.header.Number.Uint64()) || (newRootNum > ccb.Tip().Number.Uint64()) {
return errors.New("canonicalChainBuilder.PruneRoot: newRootNum outside of the canonical chain")
diff --git a/polygon/sync/canonical_chain_builder_mock.go b/polygon/sync/canonical_chain_builder_mock.go
index 7c78ae6eafd..e64604fce94 100644
--- a/polygon/sync/canonical_chain_builder_mock.go
+++ b/polygon/sync/canonical_chain_builder_mock.go
@@ -118,45 +118,6 @@ func (c *MockCanonicalChainBuilderContainsHashCall) DoAndReturn(f func(common.Ha
return c
}
-// HeaderByHash mocks base method.
-func (m *MockCanonicalChainBuilder) HeaderByHash(arg0 common.Hash) (*types.Header, bool) {
- m.ctrl.T.Helper()
- ret := m.ctrl.Call(m, "HeaderByHash", arg0)
- ret0, _ := ret[0].(*types.Header)
- ret1, _ := ret[1].(bool)
- return ret0, ret1
-}
-
-// HeaderByHash indicates an expected call of HeaderByHash.
-func (mr *MockCanonicalChainBuilderMockRecorder) HeaderByHash(arg0 any) *MockCanonicalChainBuilderHeaderByHashCall {
- mr.mock.ctrl.T.Helper()
- call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HeaderByHash", reflect.TypeOf((*MockCanonicalChainBuilder)(nil).HeaderByHash), arg0)
- return &MockCanonicalChainBuilderHeaderByHashCall{Call: call}
-}
-
-// MockCanonicalChainBuilderHeaderByHashCall wrap *gomock.Call
-type MockCanonicalChainBuilderHeaderByHashCall struct {
- *gomock.Call
-}
-
-// Return rewrite *gomock.Call.Return
-func (c *MockCanonicalChainBuilderHeaderByHashCall) Return(arg0 *types.Header, arg1 bool) *MockCanonicalChainBuilderHeaderByHashCall {
- c.Call = c.Call.Return(arg0, arg1)
- return c
-}
-
-// Do rewrite *gomock.Call.Do
-func (c *MockCanonicalChainBuilderHeaderByHashCall) Do(f func(common.Hash) (*types.Header, bool)) *MockCanonicalChainBuilderHeaderByHashCall {
- c.Call = c.Call.Do(f)
- return c
-}
-
-// DoAndReturn rewrite *gomock.Call.DoAndReturn
-func (c *MockCanonicalChainBuilderHeaderByHashCall) DoAndReturn(f func(common.Hash) (*types.Header, bool)) *MockCanonicalChainBuilderHeaderByHashCall {
- c.Call = c.Call.DoAndReturn(f)
- return c
-}
-
// HeadersInRange mocks base method.
func (m *MockCanonicalChainBuilder) HeadersInRange(arg0, arg1 uint64) []*types.Header {
m.ctrl.T.Helper()
diff --git a/polygon/sync/canonical_chain_builder_test.go b/polygon/sync/canonical_chain_builder_test.go
index e9e87823a19..6978b6dd690 100644
--- a/polygon/sync/canonical_chain_builder_test.go
+++ b/polygon/sync/canonical_chain_builder_test.go
@@ -27,7 +27,6 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
- "github.com/erigontech/erigon-lib/common"
"github.com/erigontech/erigon/core/types"
)
@@ -448,45 +447,6 @@ func TestCCBPruneNode(t *testing.T) {
})
}
-func TestCCBHeaderByHash(t *testing.T) {
- t.Parallel()
- ctx, cancel := context.WithCancel(context.Background())
- t.Cleanup(cancel)
- // R(td:0) -> A(td:1)
- // |
- // +--------> B(td:2) -> Z(td4)
- // |
- // +--------> Y(td3)
- test, headerR := newConnectCCBTest(t)
- ccb := test.builder
- headerA := test.makeHeader(headerR, 1)
- _, err := ccb.Connect(ctx, []*types.Header{headerA})
- require.NoError(t, err)
- headerB := test.makeHeader(headerR, 2)
- headerZ := test.makeHeader(headerB, 2)
- _, err = ccb.Connect(ctx, []*types.Header{headerB, headerZ})
- require.NoError(t, err)
- headerY := test.makeHeader(headerB, 1)
- _, err = ccb.Connect(ctx, []*types.Header{headerY})
- require.NoError(t, err)
- require.Equal(t, headerZ, ccb.Tip())
- assertHeaderByHash(t, ccb, headerR.Hash(), headerR, true)
- assertHeaderByHash(t, ccb, headerA.Hash(), headerA, true)
- assertHeaderByHash(t, ccb, headerB.Hash(), headerB, true)
- assertHeaderByHash(t, ccb, headerZ.Hash(), headerZ, true)
- assertHeaderByHash(t, ccb, headerY.Hash(), headerY, true)
- headerU := &types.Header{Number: big.NewInt(1234)}
- assertHeaderByHash(t, ccb, headerU.Hash(), nil, false)
- headerU2 := &types.Header{Number: big.NewInt(77)}
- assertHeaderByHash(t, ccb, headerU2.Hash(), nil, false)
-}
-
-func assertHeaderByHash(t *testing.T, ccb CanonicalChainBuilder, h common.Hash, wantHeader *types.Header, wantOk bool) {
- header, ok := ccb.HeaderByHash(h)
- require.Equal(t, wantOk, ok)
- require.Equal(t, wantHeader, header)
-}
-
func TestCCBLowestCommonAncestor(t *testing.T) {
t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index 7441e9ee376..15c0944f340 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -92,19 +92,19 @@ func NewSync(
}
}
-func (s *Sync) commitExecution(ctx context.Context, newTip *types.Header, finalizedHeader *types.Header) (common.Hash, error) {
+func (s *Sync) commitExecution(ctx context.Context, newTip *types.Header, finalizedHeader *types.Header) error {
if err := s.store.Flush(ctx); err != nil {
- return common.Hash{}, err
+ return err
}
blockNum := newTip.Number.Uint64()
if err := s.heimdallSync.SynchronizeSpans(ctx, blockNum); err != nil {
- return common.Hash{}, err
+ return err
}
if err := s.bridgeSync.Synchronize(ctx, blockNum); err != nil {
- return common.Hash{}, err
+ return err
}
age := common.PrettyAge(time.Unix(int64(newTip.Time), 0))
@@ -114,11 +114,11 @@ func (s *Sync) commitExecution(ctx context.Context, newTip *types.Header, finali
latestValidHash, err := s.execution.UpdateForkChoice(ctx, newTip, finalizedHeader)
if err != nil {
s.logger.Error("failed to update fork choice", "latestValidHash", latestValidHash, "err", err)
- return latestValidHash, err
+ return err
}
s.logger.Info(syncLogPrefix("update fork choice done"), "in", time.Since(fcStartTime))
- return latestValidHash, nil
+ return nil
}
func (s *Sync) handleMilestoneTipMismatch(
@@ -156,7 +156,7 @@ func (s *Sync) handleMilestoneTipMismatch(
)
}
- if _, err := s.commitExecution(ctx, newTip, newTip); err != nil {
+ if err := s.commitExecution(ctx, newTip, newTip); err != nil {
// note: if we face a failure during execution of finalized waypoints blocks, it means that
// we're wrong and the blocks are not considered as bad blocks, so we should terminate
return s.handleWaypointExecutionErr(ctx, ccBuilder.Root(), err)
@@ -313,9 +313,9 @@ func (s *Sync) applyNewBlockOnTip(
}
newTip := ccBuilder.Tip()
- firstConnectedHeader := newConnectedHeaders[0]
- if newTip != oldTip && oldTip.Hash() != firstConnectedHeader.ParentHash {
- if err := s.handleTipForkChangeUnwinds(ctx, ccBuilder, oldTip); err != nil {
+ firstNewConnectedHeader := newConnectedHeaders[0]
+ if newTip != oldTip && oldTip.Hash() != firstNewConnectedHeader.ParentHash {
+ if err := s.handleBridgeOnForkChange(ctx, ccBuilder, oldTip); err != nil {
return err
}
}
@@ -340,15 +340,15 @@ func (s *Sync) applyNewBlockOnTip(
if newTip == oldTip {
lastConnectedNum := newConnectedHeaders[len(newConnectedHeaders)-1].Number.Uint64()
if tipNum := newTip.Number.Uint64(); lastConnectedNum > tipNum {
- return s.handleInsertBlocksAfterUnchangedTip(ctx, tipNum, lastConnectedNum)
+ return s.handleBridgeOnBlocksInsertAheadOfTip(ctx, tipNum, lastConnectedNum)
}
return nil
}
- if latestValidHash, err := s.commitExecution(ctx, newTip, ccBuilder.Root()); err != nil {
+ if err := s.commitExecution(ctx, newTip, ccBuilder.Root()); err != nil {
if errors.Is(err, ErrForkChoiceUpdateBadBlock) {
- return s.handleBadBlockErr(ctx, ccBuilder, event, latestValidHash, err)
+ return s.handleBadBlockErr(ctx, ccBuilder, event, firstNewConnectedHeader, oldTip, err)
}
return err
@@ -443,61 +443,74 @@ func (s *Sync) publishNewBlock(ctx context.Context, block *types.Block) {
s.p2pService.PublishNewBlock(block, td)
}
-func (s *Sync) handleTipForkChangeUnwinds(ctx context.Context, ccb CanonicalChainBuilder, oldTip *types.Header) error {
+func (s *Sync) handleBridgeOnForkChange(ctx context.Context, ccb CanonicalChainBuilder, oldTip *types.Header) error {
// forks have changed, we need to unwind unwindable data
newTip := ccb.Tip()
- newTipNum := newTip.Number.Uint64()
- newTipHash := newTip.Hash()
- oldTipNum := oldTip.Number.Uint64()
- oldTipHash := oldTip.Hash()
s.logger.Debug(
- syncLogPrefix("fork change"),
- "oldNum", oldTipNum,
- "oldHash", oldTipHash,
- "newNum", newTipNum,
- "newHash", newTipHash,
+ syncLogPrefix("handling bridge on fork change"),
+ "oldNum", oldTip.Number.Uint64(),
+ "oldHash", oldTip.Hash(),
+ "newNum", newTip.Number.Uint64(),
+ "newHash", newTip.Hash(),
)
- lca, ok := ccb.LowestCommonAncestor(newTipHash, oldTipHash)
+ // Find unwind point
+ lca, ok := ccb.LowestCommonAncestor(newTip.Hash(), oldTip.Hash())
if !ok {
- return fmt.Errorf("could not find lowest common ancestor of old and new tip")
+ return errors.New("could not find lowest common ancestor of old and new tip")
}
- lcaNum := lca.Number.Uint64()
- start := lcaNum + 1
- if newTipNum < start { // defensive check against underflow & unexpected lcaNum
- return fmt.Errorf("unexpected newTipNum < start: %d < %d", oldTipNum, start)
- }
+ return s.reorganiseBridge(ctx, ccb, lca)
+}
- amount := newTipNum - start + 1
- canonicalHeaders := ccb.HeadersInRange(start, amount)
- if uint64(len(canonicalHeaders)) != amount {
- return fmt.Errorf("expected %d canonical headers", amount)
+func (s *Sync) reorganiseBridge(ctx context.Context, ccb CanonicalChainBuilder, forksLca *types.Header) error {
+ newTip := ccb.Tip()
+ newTipNum := ccb.Tip().Number.Uint64()
+ unwindPoint := forksLca.Number.Uint64()
+ s.logger.Debug(
+ syncLogPrefix("reorganise bridge"),
+ "newTip", newTipNum,
+ "newTipHash", newTip.Hash(),
+ "unwindPointNum", unwindPoint,
+ "unwindPointHash", forksLca.Hash(),
+ )
+
+ if newTipNum < unwindPoint { // defensive check against underflow & unexpected newTipNum and unwindPoint
+ return fmt.Errorf("unexpected newTipNum <= unwindPoint: %d < %d", newTipNum, unwindPoint)
}
- if err := s.bridgeSync.Unwind(ctx, lcaNum); err != nil {
+ // 1. Do the unwind from the old tip (on the old canonical fork) to the unwindPoint
+ if err := s.bridgeSync.Unwind(ctx, unwindPoint); err != nil {
return err
}
- canonicalBlocksToReplay := make([]*types.Block, len(canonicalHeaders))
- for i, header := range canonicalHeaders {
- canonicalBlocksToReplay[i] = types.NewBlockWithHeader(header)
+ // 2. Replay the new canonical blocks from the unwindPoint+1 to the new tip (on the new canonical fork). Note,
+ // that there may be a case where the newTip == unwindPoint in which case the below will be a no-op.
+ if newTipNum == unwindPoint {
+ return nil
}
- if err := s.bridgeSync.ProcessNewBlocks(ctx, canonicalBlocksToReplay); err != nil {
+ start := unwindPoint + 1
+ amount := newTipNum - start + 1
+ canonicalHeaders := ccb.HeadersInRange(start, amount)
+ canonicalBlocks := make([]*types.Block, len(canonicalHeaders))
+ for i, header := range canonicalHeaders {
+ canonicalBlocks[i] = types.NewBlockWithHeader(header)
+ }
+ if err := s.bridgeSync.ProcessNewBlocks(ctx, canonicalBlocks); err != nil {
return err
}
- return s.bridgeSync.Synchronize(ctx, oldTipNum)
+ return s.bridgeSync.Synchronize(ctx, newTipNum)
}
-func (s *Sync) handleInsertBlocksAfterUnchangedTip(ctx context.Context, tipNum, lastInsertedNum uint64) error {
+func (s *Sync) handleBridgeOnBlocksInsertAheadOfTip(ctx context.Context, tipNum, lastInsertedNum uint64) error {
// this is a hack that should disappear when changing the bridge to not track blocks (future work)
// make sure the bridge does not go past the tip (it may happen when we insert blocks from another fork that
// has a higher block number than the canonical tip but lower difficulty) - this is to prevent the bridge
// from recording incorrect bor txn hashes
s.logger.Debug(
- syncLogPrefix("unwinding back bridge due to inserting headers past the tip"),
+ syncLogPrefix("unwinding bridge due to inserting headers past the tip"),
"tip", tipNum,
"lastInsertedNum", lastInsertedNum,
)
@@ -519,41 +532,53 @@ func (s *Sync) handleBadBlockErr(
ctx context.Context,
ccb CanonicalChainBuilder,
event EventNewBlock,
- latestValidHash common.Hash,
+ firstNewConnectedHeader *types.Header,
+ oldTip *types.Header,
badBlockErr error,
) error {
+ badTip := ccb.Tip()
+ badTipHash := badTip.Hash()
+ oldTipNum := oldTip.Number.Uint64()
+ oldTipHash := oldTip.Hash()
s.logger.Warn(
- syncLogPrefix("bad block after execution"),
+ syncLogPrefix("handling bad block after execution"),
"peerId", event.PeerId,
- "latestValidHash", latestValidHash,
+ "badTipNum", badTip.Number.Uint64(),
+ "badTipHash", badTipHash,
+ "oldTipNum", oldTipNum,
+ "oldTipHash", oldTipHash,
+ "firstNewConnectedNum", firstNewConnectedHeader.Number.Uint64(),
+ "firstNewConnectedHash", firstNewConnectedHeader.Hash(),
"err", badBlockErr,
)
+ // 1. Mark block as bad and penalize peer
s.badBlocks.Add(event.NewBlock.Hash(), struct{}{})
s.maybePenalizePeerOnBadBlockEvent(ctx, event)
- latestValidHeader, ok := ccb.HeaderByHash(latestValidHash)
- if !ok {
- return fmt.Errorf("unexpected latestValidHash not in canonical builder: %s", latestValidHash)
- }
- badTipNum := ccb.Tip().Number.Uint64()
- start := latestValidHeader.Number.Uint64() + 1
- if badTipNum < start { // defensive check against underflow & unexpected badTipNum and latestValidHeader
- return fmt.Errorf("unexpected badTipNum < start: %d < %d", badTipNum, start)
+ // 2. Find unwind point
+ lca, ok := ccb.LowestCommonAncestor(oldTipHash, badTip.Hash())
+ if !ok {
+ return errors.New("could not find lowest common ancestor of old and new tip")
}
- amount := badTipNum - start + 1
- badHeaders := ccb.HeadersInRange(start, amount)
- if uint64(len(badHeaders)) != amount {
- return fmt.Errorf("expected %d bad headers after bad block err", amount)
+ // 3. Prune newly inserted nodes in the tree => should roll back the ccb to the old tip
+ if err := ccb.PruneNode(firstNewConnectedHeader.Hash()); err != nil {
+ return err
}
- if err := ccb.PruneNode(badHeaders[0].Hash()); err != nil {
- return err
+ newTip := ccb.Tip()
+ newTipNum := newTip.Number.Uint64()
+ newTipHash := newTip.Hash()
+ if oldTipHash != newTipHash { // defensive check for unexpected behaviour
+ return fmt.Errorf(
+ "old tip hash does not match new tip hash (%d,%s) vs (%d, %s)",
+ oldTipNum, oldTipHash, newTipNum, newTipHash,
+ )
}
- tipPostPrune := ccb.Tip()
- return s.bridgeSync.Unwind(ctx, tipPostPrune.Number.Uint64())
+ // 4. Update bridge
+ return s.reorganiseBridge(ctx, ccb, lca)
}
func (s *Sync) maybePenalizePeerOnBadBlockEvent(ctx context.Context, event EventNewBlock) {
@@ -744,7 +769,7 @@ func (s *Sync) sync(ctx context.Context, tip *types.Header, tipDownloader tipDow
}
newTip := newResult.latestTip
- if _, err := s.commitExecution(ctx, newTip, newTip); err != nil {
+ if err := s.commitExecution(ctx, newTip, newTip); err != nil {
// note: if we face a failure during execution of finalized waypoints blocks, it means that
// we're wrong and the blocks are not considered as bad blocks, so we should terminate
err = s.handleWaypointExecutionErr(ctx, tip, err)
From d7bd86162badc29e874a23d5d18020068150d77b Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 17 Oct 2024 16:57:30 +0100
Subject: [PATCH 24/25] add log for waypoin exec err
---
polygon/sync/sync.go | 7 +++++++
1 file changed, 7 insertions(+)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index 15c0944f340..872b5373f19 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -783,6 +783,13 @@ func (s *Sync) sync(ctx context.Context, tip *types.Header, tipDownloader tipDow
}
func (s *Sync) handleWaypointExecutionErr(ctx context.Context, lastCorrectTip *types.Header, execErr error) error {
+ s.logger.Debug(
+ syncLogPrefix("waypoint execution err"),
+ "lastCorrectTipNum", lastCorrectTip.Number.Uint64(),
+ "lastCorrectTipHash", lastCorrectTip.Hash(),
+ "execErr", execErr,
+ )
+
if !errors.Is(execErr, ErrForkChoiceUpdateBadBlock) {
return execErr
}
From c01db8b4c07d022cdae5dfd13b739ef8c58e9ce6 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Thu, 17 Oct 2024 17:14:27 +0100
Subject: [PATCH 25/25] add log for penalizing peer for bad block
---
polygon/sync/sync.go | 1 +
1 file changed, 1 insertion(+)
diff --git a/polygon/sync/sync.go b/polygon/sync/sync.go
index 872b5373f19..bee44daf9c9 100644
--- a/polygon/sync/sync.go
+++ b/polygon/sync/sync.go
@@ -588,6 +588,7 @@ func (s *Sync) maybePenalizePeerOnBadBlockEvent(ctx context.Context, event Event
return
}
+ s.logger.Debug(syncLogPrefix("penalizing peer for bad block"), "peerId", event.PeerId)
if err := s.p2pService.Penalize(ctx, event.PeerId); err != nil {
s.logger.Debug(syncLogPrefix("issue with penalizing peer for bad block"), "peerId", event.PeerId, "err", err)
}