From 3a3669687b97538ef322649dfa15c92c1d491e38 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Tue, 8 Oct 2024 17:50:46 +0300 Subject: [PATCH 01/20] *: initial consensus factory impl --- app/app.go | 31 ++------- core/consensus/factory.go | 49 ++++++++++++++ core/consensus/factory_test.go | 67 +++++++++++++++++++ core/consensus/protocols.go | 33 +++++++++ core/consensus/protocols_test.go | 23 +++++++ core/consensus/{component.go => qbft.go} | 24 +------ ...internal_test.go => qbft_internal_test.go} | 0 .../{component_test.go => qbft_test.go} | 5 -- core/consensus/transport.go | 2 +- core/interfaces.go | 10 +++ 10 files changed, 192 insertions(+), 52 deletions(-) create mode 100644 core/consensus/factory.go create mode 100644 core/consensus/factory_test.go create mode 100644 core/consensus/protocols.go create mode 100644 core/consensus/protocols_test.go rename core/consensus/{component.go => qbft.go} (96%) rename core/consensus/{component_internal_test.go => qbft_internal_test.go} (100%) rename core/consensus/{component_test.go => qbft_test.go} (96%) diff --git a/app/app.go b/app/app.go index e5db83c39..3532ef6d7 100644 --- a/app/app.go +++ b/app/app.go @@ -47,7 +47,6 @@ import ( "github.com/obolnetwork/charon/core/aggsigdb" "github.com/obolnetwork/charon/core/bcast" "github.com/obolnetwork/charon/core/consensus" - pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/dutydb" "github.com/obolnetwork/charon/core/fetcher" "github.com/obolnetwork/charon/core/infosync" @@ -526,14 +525,16 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, retryer := retry.New[core.Duty](deadlineFunc) - cons, startCons, err := newConsensus(cluster, tcpNode, p2pKey, sender, - deadlinerFunc("consensus"), gaterFunc, consensusDebugger.AddInstance) + consensusFactory := consensus.NewConsensusFactory(tcpNode, sender, peers, p2pKey, deadlinerFunc("consensus"), gaterFunc, consensusDebugger.AddInstance) + + qbftConsensus, err := consensusFactory.New(consensus.QBFTv2ProtocolID) if err != nil { return err } + startConsensus := lifecycle.HookFuncCtx(qbftConsensus.Start) err = wirePrioritise(ctx, conf, life, tcpNode, peerIDs, int(cluster.GetThreshold()), - sender.SendReceive, cons, sched, p2pKey, deadlineFunc) + sender.SendReceive, qbftConsensus, sched, p2pKey, deadlineFunc) if err != nil { return err } @@ -558,7 +559,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, core.WithTracking(track, inclusion), core.WithAsyncRetry(retryer), } - core.Wire(sched, fetch, cons, dutyDB, vapi, parSigDB, parSigEx, sigAgg, aggSigDB, broadcaster, opts...) + core.Wire(sched, fetch, qbftConsensus, dutyDB, vapi, parSigDB, parSigEx, sigAgg, aggSigDB, broadcaster, opts...) err = wireValidatorMock(ctx, conf, eth2Cl, pubshares, sched) if err != nil { @@ -570,7 +571,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, } life.RegisterStart(lifecycle.AsyncBackground, lifecycle.StartScheduler, lifecycle.HookFuncErr(sched.Run)) - life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startCons) + life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startConsensus) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartAggSigDB, lifecycle.HookFuncCtx(aggSigDB.Run)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartParSigDB, lifecycle.HookFuncCtx(parSigDB.Trim)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartTracker, lifecycle.HookFuncCtx(inclusion.Run)) @@ -918,24 +919,6 @@ func configureEth2Client(ctx context.Context, forkVersion []byte, addrs []string return eth2Cl, nil } -// newConsensus returns a new consensus component and its start lifecycle hook. -func newConsensus(cluster *manifestpb.Cluster, tcpNode host.Host, p2pKey *k1.PrivateKey, - sender *p2p.Sender, deadliner core.Deadliner, gaterFunc core.DutyGaterFunc, - qbftSniffer func(*pbv1.SniffedConsensusInstance), -) (core.Consensus, lifecycle.IHookFunc, error) { - peers, err := manifest.ClusterPeers(cluster) - if err != nil { - return nil, nil, err - } - - comp, err := consensus.New(tcpNode, sender, peers, p2pKey, deadliner, gaterFunc, qbftSniffer) - if err != nil { - return nil, nil, err - } - - return comp, lifecycle.HookFuncCtx(comp.Start), nil -} - // createMockValidators creates mock validators identified by their public shares. func createMockValidators(pubkeys []eth2p0.BLSPubKey) beaconmock.ValidatorSet { resp := make(beaconmock.ValidatorSet) diff --git a/core/consensus/factory.go b/core/consensus/factory.go new file mode 100644 index 000000000..fba6e1ec6 --- /dev/null +++ b/core/consensus/factory.go @@ -0,0 +1,49 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" + "github.com/libp2p/go-libp2p/core/host" + "github.com/libp2p/go-libp2p/core/protocol" + + "github.com/obolnetwork/charon/app/errors" + "github.com/obolnetwork/charon/core" + pbv1 "github.com/obolnetwork/charon/core/corepb/v1" + "github.com/obolnetwork/charon/p2p" +) + +type consensusFactory struct { + tcpNode host.Host + sender *p2p.Sender + peers []p2p.Peer + p2pKey *k1.PrivateKey + deadliner core.Deadliner + gaterFunc core.DutyGaterFunc + snifferFunc func(*pbv1.SniffedConsensusInstance) +} + +// NewConsensusFactory creates a new consensus factory. +func NewConsensusFactory(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, + deadliner core.Deadliner, gaterFunc core.DutyGaterFunc, snifferFunc func(*pbv1.SniffedConsensusInstance), +) core.ConsensusFactory { + return &consensusFactory{ + tcpNode: tcpNode, + sender: sender, + peers: peers, + p2pKey: p2pKey, + deadliner: deadliner, + gaterFunc: gaterFunc, + snifferFunc: snifferFunc, + } +} + +// New creates a new consensus instance. +func (f *consensusFactory) New(protocol protocol.ID) (core.Consensus, error) { + // TODO: Refactor to a switch statement when more protocols are added. + if protocol == QBFTv2ProtocolID { + return New(f.tcpNode, f.sender, f.peers, f.p2pKey, f.deadliner, f.gaterFunc, f.snifferFunc) + } + + return nil, errors.New("unknown consensus protocol") +} diff --git a/core/consensus/factory_test.go b/core/consensus/factory_test.go new file mode 100644 index 000000000..9f783385e --- /dev/null +++ b/core/consensus/factory_test.go @@ -0,0 +1,67 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus_test + +import ( + "fmt" + "math/rand" + "testing" + + "github.com/libp2p/go-libp2p" + libp2pcrypto "github.com/libp2p/go-libp2p/core/crypto" + "github.com/libp2p/go-libp2p/core/host" + "github.com/multiformats/go-multiaddr" + "github.com/stretchr/testify/require" + + "github.com/obolnetwork/charon/cluster" + "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus" + pbv1 "github.com/obolnetwork/charon/core/corepb/v1" + "github.com/obolnetwork/charon/eth2util/enr" + "github.com/obolnetwork/charon/p2p" + "github.com/obolnetwork/charon/testutil" +) + +func TestNewConsensusFactory(t *testing.T) { + var hosts []host.Host + var peers []p2p.Peer + + seed := 0 + random := rand.New(rand.NewSource(int64(seed))) + lock, p2pkeys, _ := cluster.NewForT(t, 1, 3, 3, seed, random) + + snifferFunc := func(msgs *pbv1.SniffedConsensusInstance) {} + gaterFunc := func(core.Duty) bool { return true } + + for i := range 3 { + addr := testutil.AvailableAddr(t) + mAddr, err := multiaddr.NewMultiaddr(fmt.Sprintf("/ip4/%s/tcp/%d", addr.IP, addr.Port)) + require.NoError(t, err) + + priv := (*libp2pcrypto.Secp256k1PrivateKey)(p2pkeys[i]) + h, err := libp2p.New(libp2p.Identity(priv), libp2p.ListenAddrs(mAddr)) + testutil.SkipIfBindErr(t, err) + require.NoError(t, err) + + record, err := enr.Parse(lock.Operators[i].ENR) + require.NoError(t, err) + + p, err := p2p.NewPeerFromENR(record, i) + require.NoError(t, err) + + peers = append(peers, p) + hosts = append(hosts, h) + } + + factory := consensus.NewConsensusFactory(hosts[0], new(p2p.Sender), peers, p2pkeys[0], testDeadliner{}, gaterFunc, snifferFunc) + require.NotNil(t, factory) + + cons, err := factory.New(consensus.QBFTv2ProtocolID) + require.NoError(t, err) + require.NotNil(t, cons) + + t.Run("unknown protocol", func(t *testing.T) { + _, err := factory.New("unknown") + require.Error(t, err) + }) +} diff --git a/core/consensus/protocols.go b/core/consensus/protocols.go new file mode 100644 index 000000000..4268fc1a0 --- /dev/null +++ b/core/consensus/protocols.go @@ -0,0 +1,33 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + "strings" + + "github.com/libp2p/go-libp2p/core/protocol" +) + +const ( + protocolIDPrefix = "/charon/consensus/" + + QBFTv2ProtocolID = "/charon/consensus/qbft/2.0.0" +) + +// Protocols returns the supported protocols of this package in order of precedence. +func Protocols() []protocol.ID { + return []protocol.ID{QBFTv2ProtocolID} +} + +// IsSupportedProtocolName returns true if the protocol name is supported. +func IsSupportedProtocolName(name string) bool { + for _, p := range Protocols() { + nameAndVersion := strings.TrimPrefix(string(p), protocolIDPrefix) + parts := strings.Split(nameAndVersion, "/") + if len(parts) > 0 && parts[0] == strings.ToLower(name) { + return true + } + } + + return false +} diff --git a/core/consensus/protocols_test.go b/core/consensus/protocols_test.go new file mode 100644 index 000000000..434c31707 --- /dev/null +++ b/core/consensus/protocols_test.go @@ -0,0 +1,23 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus_test + +import ( + "testing" + + "github.com/libp2p/go-libp2p/core/protocol" + "github.com/stretchr/testify/require" + + "github.com/obolnetwork/charon/core/consensus" +) + +func TestIsSupportedProtocolName(t *testing.T) { + require.True(t, consensus.IsSupportedProtocolName("qbft")) + require.False(t, consensus.IsSupportedProtocolName("unreal")) +} + +func TestProtocols(t *testing.T) { + require.Equal(t, []protocol.ID{ + consensus.QBFTv2ProtocolID, + }, consensus.Protocols()) +} diff --git a/core/consensus/component.go b/core/consensus/qbft.go similarity index 96% rename from core/consensus/component.go rename to core/consensus/qbft.go index 91c6ff13e..00f3c0dc2 100644 --- a/core/consensus/component.go +++ b/core/consensus/qbft.go @@ -12,7 +12,6 @@ import ( k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" - "github.com/libp2p/go-libp2p/core/protocol" "go.opentelemetry.io/otel/trace" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" @@ -28,28 +27,9 @@ import ( ) const ( - recvBuffer = 100 // Allow buffering some initial messages when this node is late to start an instance. - protocolID2 = "/charon/consensus/qbft/2.0.0" + recvBuffer = 100 // Allow buffering some initial messages when this node is late to start an instance. ) -// Protocols returns the supported protocols of this package in order of precedence. -func Protocols() []protocol.ID { - return []protocol.ID{protocolID2} -} - -// IsSupportedProtocolName returns true if the protocol name is supported. -func IsSupportedProtocolName(name string) bool { - for _, p := range Protocols() { - nameAndVersion := strings.TrimPrefix(string(p), "/charon/consensus/") - parts := strings.Split(nameAndVersion, "/") - if len(parts) > 0 && parts[0] == strings.ToLower(name) { - return true - } - } - - return false -} - type subscriber func(ctx context.Context, duty core.Duty, value proto.Message) error // newDefinition returns a qbft definition (this is constant across all consensus instances). @@ -302,7 +282,7 @@ func (c *Component) SubscribePriority(fn func(ctx context.Context, duty core.Dut // Start registers the libp2p receive handler and starts a goroutine that cleans state. This should only be called once. func (c *Component) Start(ctx context.Context) { - p2p.RegisterHandler("qbft", c.tcpNode, protocolID2, + p2p.RegisterHandler("qbft", c.tcpNode, QBFTv2ProtocolID, func() proto.Message { return new(pbv1.ConsensusMsg) }, c.handle) diff --git a/core/consensus/component_internal_test.go b/core/consensus/qbft_internal_test.go similarity index 100% rename from core/consensus/component_internal_test.go rename to core/consensus/qbft_internal_test.go diff --git a/core/consensus/component_test.go b/core/consensus/qbft_test.go similarity index 96% rename from core/consensus/component_test.go rename to core/consensus/qbft_test.go index 3568c309e..3c5fa5b33 100644 --- a/core/consensus/component_test.go +++ b/core/consensus/qbft_test.go @@ -62,11 +62,6 @@ func TestComponent(t *testing.T) { } } -func TestIsSupportedProtocolName(t *testing.T) { - require.True(t, consensus.IsSupportedProtocolName("qbft")) - require.False(t, consensus.IsSupportedProtocolName("unreal")) -} - // testComponent tests a consensus instance with size of threshold-of-nodes. // Note it only instantiates the minimum amount of peers, ie threshold. func testComponent(t *testing.T, threshold, nodes int) { diff --git a/core/consensus/transport.go b/core/consensus/transport.go index 369825af7..a3e48c61b 100644 --- a/core/consensus/transport.go +++ b/core/consensus/transport.go @@ -127,7 +127,7 @@ func (t *transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.D continue } - err = t.component.sender.SendAsync(ctx, t.component.tcpNode, protocolID2, p.ID, msg.ToConsensusMsg()) + err = t.component.sender.SendAsync(ctx, t.component.tcpNode, QBFTv2ProtocolID, p.ID, msg.ToConsensusMsg()) if err != nil { return err } diff --git a/core/interfaces.go b/core/interfaces.go index 786876124..37a8bdccb 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -8,6 +8,7 @@ import ( eth2api "github.com/attestantio/go-eth2-client/api" "github.com/attestantio/go-eth2-client/spec/altair" eth2p0 "github.com/attestantio/go-eth2-client/spec/phase0" + "github.com/libp2p/go-libp2p/core/protocol" ) // Scheduler triggers the start of a duty workflow. @@ -67,6 +68,9 @@ type DutyDB interface { // Consensus comes to consensus on proposed duty data. type Consensus interface { + // Start starts the consensus protocol instance. + Start(ctx context.Context) + // Participate run the duty's consensus instance without a proposed value (if Propose not called yet). Participate(context.Context, Duty) error @@ -77,6 +81,12 @@ type Consensus interface { Subscribe(func(context.Context, Duty, UnsignedDataSet) error) } +// ConsensusFactory creates new consensus instances. +type ConsensusFactory interface { + // New creates a new consensus instance for the specified protocol. + New(protocol protocol.ID) (Consensus, error) +} + // ValidatorAPI provides a beacon node API to validator clients. It serves duty data from the DutyDB and stores partial signed data in the ParSigDB. type ValidatorAPI interface { // RegisterAwaitProposal registers a function to query unsigned beacon block proposals by providing the slot. From 109e350e097de5bd9fa7c2362c478674e045df5f Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 9 Oct 2024 09:21:28 +0300 Subject: [PATCH 02/20] Fixed vapi test --- core/validatorapi/validatorapi_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/validatorapi/validatorapi_test.go b/core/validatorapi/validatorapi_test.go index 559252f6d..79ff1d81c 100644 --- a/core/validatorapi/validatorapi_test.go +++ b/core/validatorapi/validatorapi_test.go @@ -436,7 +436,7 @@ func TestComponent_SubmitProposalsWithWrongVCData(t *testing.T) { require.NoError(t, err) // Construct the validator api component - vapi, err := validatorapi.NewComponent(bmock, allPubSharesByKey, shareIdx, nil, testutil.BuilderFalse, nil) + vapi, err := validatorapi.NewComponent(bmock, allPubSharesByKey, shareIdx, nil, false, nil) require.NoError(t, err) t.Run("full block fails", func(t *testing.T) { From 2df629002910693db332b44afb6e1e81e456195f Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 9 Oct 2024 11:16:31 +0300 Subject: [PATCH 03/20] More of QBFT refactoring --- app/app.go | 12 +- core/consensus/factory.go | 2 +- core/consensus/instance_io.go | 81 +++++++++++ core/consensus/instance_io_internal_test.go | 45 ++++++ core/consensus/qbft.go | 132 ++++-------------- core/consensus/qbft_internal_test.go | 36 ++--- core/consensus/{msg.go => qbft_msg.go} | 48 +++---- ...rnal_test.go => qbft_msg_internal_test.go} | 10 +- core/consensus/qbft_test.go | 14 +- .../{transport.go => qbft_transport.go} | 79 +++-------- core/consensus/sniffed_internal_test.go | 4 +- core/consensus/sniffer.go | 55 ++++++++ core/consensus/strategysim_internal_test.go | 4 +- 13 files changed, 296 insertions(+), 226 deletions(-) create mode 100644 core/consensus/instance_io.go create mode 100644 core/consensus/instance_io_internal_test.go rename core/consensus/{msg.go => qbft_msg.go} (73%) rename core/consensus/{msg_internal_test.go => qbft_msg_internal_test.go} (92%) rename core/consensus/{transport.go => qbft_transport.go} (68%) create mode 100644 core/consensus/sniffer.go diff --git a/app/app.go b/app/app.go index 3532ef6d7..74f2a25ff 100644 --- a/app/app.go +++ b/app/app.go @@ -525,13 +525,17 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, retryer := retry.New[core.Duty](deadlineFunc) - consensusFactory := consensus.NewConsensusFactory(tcpNode, sender, peers, p2pKey, deadlinerFunc("consensus"), gaterFunc, consensusDebugger.AddInstance) + consensusFactory := consensus.NewConsensusFactory( + tcpNode, sender, peers, p2pKey, + deadlinerFunc("consensus"), gaterFunc, consensusDebugger.AddInstance) + // We always need QBFT consensus instance as it is used for priority protocol. + // And for now it is used as the primary consensus protocol. qbftConsensus, err := consensusFactory.New(consensus.QBFTv2ProtocolID) if err != nil { return err } - startConsensus := lifecycle.HookFuncCtx(qbftConsensus.Start) + startQBFTConsensus := lifecycle.HookFuncCtx(qbftConsensus.Start) err = wirePrioritise(ctx, conf, life, tcpNode, peerIDs, int(cluster.GetThreshold()), sender.SendReceive, qbftConsensus, sched, p2pKey, deadlineFunc) @@ -571,7 +575,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, } life.RegisterStart(lifecycle.AsyncBackground, lifecycle.StartScheduler, lifecycle.HookFuncErr(sched.Run)) - life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startConsensus) + life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startQBFTConsensus) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartAggSigDB, lifecycle.HookFuncCtx(aggSigDB.Run)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartParSigDB, lifecycle.HookFuncCtx(parSigDB.Trim)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartTracker, lifecycle.HookFuncCtx(inclusion.Run)) @@ -587,7 +591,7 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t peers []peer.ID, threshold int, sendFunc p2p.SendReceiveFunc, coreCons core.Consensus, sched core.Scheduler, p2pKey *k1.PrivateKey, deadlineFunc func(duty core.Duty) (time.Time, bool), ) error { - cons, ok := coreCons.(*consensus.Component) + cons, ok := coreCons.(*consensus.QBFTConsensus) if !ok { // Priority protocol not supported for leader cast. return nil diff --git a/core/consensus/factory.go b/core/consensus/factory.go index fba6e1ec6..c8ac81d18 100644 --- a/core/consensus/factory.go +++ b/core/consensus/factory.go @@ -42,7 +42,7 @@ func NewConsensusFactory(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer func (f *consensusFactory) New(protocol protocol.ID) (core.Consensus, error) { // TODO: Refactor to a switch statement when more protocols are added. if protocol == QBFTv2ProtocolID { - return New(f.tcpNode, f.sender, f.peers, f.p2pKey, f.deadliner, f.gaterFunc, f.snifferFunc) + return NewQBFTConsensus(f.tcpNode, f.sender, f.peers, f.p2pKey, f.deadliner, f.gaterFunc, f.snifferFunc) } return nil, errors.New("unknown consensus protocol") diff --git a/core/consensus/instance_io.go b/core/consensus/instance_io.go new file mode 100644 index 000000000..9caaed023 --- /dev/null +++ b/core/consensus/instance_io.go @@ -0,0 +1,81 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + "time" + + "google.golang.org/protobuf/proto" + + "github.com/obolnetwork/charon/app/errors" +) + +const ( + recvBuffer = 100 // Allow buffering some initial messages when this node is late to start an instance. +) + +// newInstanceIO returns a new instanceIO. +func newInstanceIO[T any]() instanceIO[T] { + return instanceIO[T]{ + participated: make(chan struct{}), + proposed: make(chan struct{}), + running: make(chan struct{}), + recvBuffer: make(chan T, recvBuffer), + hashCh: make(chan [32]byte, 1), + valueCh: make(chan proto.Message, 1), + errCh: make(chan error, 1), + decidedAtCh: make(chan time.Time, 1), + } +} + +// instanceIO defines the async input and output channels of a +// single consensus instance in the Component. +type instanceIO[T any] struct { + participated chan struct{} // Closed when Participate was called for this instance. + proposed chan struct{} // Closed when Propose was called for this instance. + running chan struct{} // Closed when runInstance was already called. + recvBuffer chan T // Outer receive buffers. + hashCh chan [32]byte // Async input hash channel. + valueCh chan proto.Message // Async input value channel. + errCh chan error // Async output error channel. + decidedAtCh chan time.Time // Async output decided timestamp channel. +} + +// MarkParticipated marks the instance as participated. +// It returns an error if the instance was already marked as participated. +func (io instanceIO[T]) MarkParticipated() error { + select { + case <-io.participated: + return errors.New("already participated") + default: + close(io.participated) + } + + return nil +} + +// MarkProposed marks the instance as proposed. +// It returns an error if the instance was already marked as proposed. +func (io instanceIO[T]) MarkProposed() error { + select { + case <-io.proposed: + return errors.New("already proposed") + default: + close(io.proposed) + } + + return nil +} + +// MaybeStart returns true if the instance wasn't running and has been started by this call, +// otherwise it returns false if the instance was started in the past and is either running now or has completed. +func (io instanceIO[T]) MaybeStart() bool { + select { + case <-io.running: + return false + default: + close(io.running) + } + + return true +} diff --git a/core/consensus/instance_io_internal_test.go b/core/consensus/instance_io_internal_test.go new file mode 100644 index 000000000..686f8c2fd --- /dev/null +++ b/core/consensus/instance_io_internal_test.go @@ -0,0 +1,45 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestMarkParticipated(t *testing.T) { + io := newInstanceIO[qbftMsg]() + + // First call succeeds. + err := io.MarkParticipated() + require.NoError(t, err) + + // Second call fails. + err = io.MarkParticipated() + require.ErrorContains(t, err, "already participated") +} + +func TestMarkProposed(t *testing.T) { + io := newInstanceIO[qbftMsg]() + + // First call succeeds. + err := io.MarkProposed() + require.NoError(t, err) + + // Second call fails. + err = io.MarkProposed() + require.ErrorContains(t, err, "already proposed") +} + +func TestMaybeStart(t *testing.T) { + io := newInstanceIO[qbftMsg]() + + // First call succeeds. + ok := io.MaybeStart() + require.True(t, ok) + + // Second call fails. + ok = io.MaybeStart() + require.False(t, ok) +} diff --git a/core/consensus/qbft.go b/core/consensus/qbft.go index 00f3c0dc2..48afbe68d 100644 --- a/core/consensus/qbft.go +++ b/core/consensus/qbft.go @@ -26,14 +26,10 @@ import ( "github.com/obolnetwork/charon/p2p" ) -const ( - recvBuffer = 100 // Allow buffering some initial messages when this node is late to start an instance. -) - type subscriber func(ctx context.Context, duty core.Duty, value proto.Message) error -// newDefinition returns a qbft definition (this is constant across all consensus instances). -func newDefinition(nodes int, subs func() []subscriber, roundTimer roundTimer, +// newQBFTDefinition returns a qbft definition (this is constant across all consensus instances). +func newQBFTDefinition(nodes int, subs func() []subscriber, roundTimer roundTimer, decideCallback func(qcommit []qbft.Msg[core.Duty, [32]byte]), ) qbft.Definition[core.Duty, [32]byte] { quorum := qbft.Definition[int, int]{Nodes: nodes}.Quorum() @@ -47,7 +43,7 @@ func newDefinition(nodes int, subs func() []subscriber, roundTimer roundTimer, // Decide sends consensus output to subscribers. Decide: func(ctx context.Context, duty core.Duty, _ [32]byte, qcommit []qbft.Msg[core.Duty, [32]byte]) { defer endCtxSpan(ctx) // End the parent tracing span when decided - msg, ok := qcommit[0].(msg) + msg, ok := qcommit[0].(qbftMsg) if !ok { log.Error(ctx, "Invalid message type", nil) return @@ -119,76 +115,10 @@ func newDefinition(nodes int, subs func() []subscriber, roundTimer roundTimer, } } -// newInstanceIO returns a new instanceIO. -func newInstanceIO() instanceIO { - return instanceIO{ - participated: make(chan struct{}), - proposed: make(chan struct{}), - running: make(chan struct{}), - recvBuffer: make(chan msg, recvBuffer), - hashCh: make(chan [32]byte, 1), - valueCh: make(chan proto.Message, 1), - errCh: make(chan error, 1), - decidedAtCh: make(chan time.Time, 1), - } -} - -// instanceIO defines the async input and output channels of a -// single consensus instance in the Component. -type instanceIO struct { - participated chan struct{} // Closed when Participate was called for this instance. - proposed chan struct{} // Closed when Propose was called for this instance. - running chan struct{} // Closed when runInstance was already called. - recvBuffer chan msg // Outer receive buffers. - hashCh chan [32]byte // Async input hash channel. - valueCh chan proto.Message // Async input value channel. - errCh chan error // Async output error channel. - decidedAtCh chan time.Time // Async output decided timestamp channel. -} - -// MarkParticipated marks the instance as participated. -// It returns an error if the instance was already marked as participated. -func (io instanceIO) MarkParticipated() error { - select { - case <-io.participated: - return errors.New("already participated") - default: - close(io.participated) - } - - return nil -} - -// MarkProposed marks the instance as proposed. -// It returns an error if the instance was already marked as proposed. -func (io instanceIO) MarkProposed() error { - select { - case <-io.proposed: - return errors.New("already proposed") - default: - close(io.proposed) - } - - return nil -} - -// MaybeStart returns true if the instance wasn't running and has been started by this call, -// otherwise it returns false if the instance was started in the past and is either running now or has completed. -func (io instanceIO) MaybeStart() bool { - select { - case <-io.running: - return false - default: - close(io.running) - } - - return true -} - -// New returns a new consensus QBFT component. -func New(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, +// NewQBFTConsensus returns a new consensus QBFT component. +func NewQBFTConsensus(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, deadliner core.Deadliner, gaterFunc core.DutyGaterFunc, snifferFunc func(*pbv1.SniffedConsensusInstance), -) (*Component, error) { +) (*QBFTConsensus, error) { // Extract peer pubkeys. keys := make(map[int64]*k1.PublicKey) var labels []string @@ -203,7 +133,7 @@ func New(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.Pri keys[int64(i)] = pk } - c := &Component{ + c := &QBFTConsensus{ tcpNode: tcpNode, sender: sender, peers: peers, @@ -216,13 +146,13 @@ func New(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.Pri dropFilter: log.Filter(), timerFunc: getTimerFunc(), } - c.mutable.instances = make(map[core.Duty]instanceIO) + c.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) return c, nil } -// Component implements core.Consensus. -type Component struct { +// QBFTConsensus implements core.Consensus & priority.coreConsensus. +type QBFTConsensus struct { // Immutable state tcpNode host.Host sender *p2p.Sender @@ -240,13 +170,13 @@ type Component struct { // Mutable state mutable struct { sync.Mutex - instances map[core.Duty]instanceIO + instances map[core.Duty]instanceIO[qbftMsg] } } // Subscribe registers a callback for unsigned duty data proposals from leaders. // Note this function is not thread safe, it should be called *before* Start and Propose. -func (c *Component) Subscribe(fn func(ctx context.Context, duty core.Duty, set core.UnsignedDataSet) error) { +func (c *QBFTConsensus) Subscribe(fn func(ctx context.Context, duty core.Duty, set core.UnsignedDataSet) error) { c.subs = append(c.subs, func(ctx context.Context, duty core.Duty, value proto.Message) error { unsignedPB, ok := value.(*pbv1.UnsignedDataSet) if !ok { @@ -263,13 +193,13 @@ func (c *Component) Subscribe(fn func(ctx context.Context, duty core.Duty, set c } // subscribers returns the subscribers. -func (c *Component) subscribers() []subscriber { +func (c *QBFTConsensus) subscribers() []subscriber { return c.subs } // SubscribePriority registers a callback for priority protocol message proposals from leaders. // Note this function is not thread safe, it should be called *before* Start and Propose. -func (c *Component) SubscribePriority(fn func(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error) { +func (c *QBFTConsensus) SubscribePriority(fn func(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error) { c.subs = append(c.subs, func(ctx context.Context, duty core.Duty, value proto.Message) error { msg, ok := value.(*pbv1.PriorityResult) if !ok { @@ -281,7 +211,7 @@ func (c *Component) SubscribePriority(fn func(ctx context.Context, duty core.Dut } // Start registers the libp2p receive handler and starts a goroutine that cleans state. This should only be called once. -func (c *Component) Start(ctx context.Context) { +func (c *QBFTConsensus) Start(ctx context.Context) { p2p.RegisterHandler("qbft", c.tcpNode, QBFTv2ProtocolID, func() proto.Message { return new(pbv1.ConsensusMsg) }, c.handle) @@ -302,7 +232,7 @@ func (c *Component) Start(ctx context.Context) { // It either runs the consensus instance if it is not already running or // waits until it completes, in both cases it returns the resulting error. // Note this errors if called multiple times for the same duty. -func (c *Component) Propose(ctx context.Context, duty core.Duty, data core.UnsignedDataSet) error { +func (c *QBFTConsensus) Propose(ctx context.Context, duty core.Duty, data core.UnsignedDataSet) error { // Hash the proposed data, since qbft only supports simple comparable values. value, err := core.UnsignedDataSetToProto(data) if err != nil { @@ -316,7 +246,7 @@ func (c *Component) Propose(ctx context.Context, duty core.Duty, data core.Unsig // It either runs the consensus instance if it is not already running or // waits until it completes, in both cases it returns the resulting error. // Note this errors if called multiple times for the same duty. -func (c *Component) ProposePriority(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error { +func (c *QBFTConsensus) ProposePriority(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error { return c.propose(ctx, duty, msg) } @@ -324,7 +254,7 @@ func (c *Component) ProposePriority(ctx context.Context, duty core.Duty, msg *pb // It either runs the consensus instance if it is not already running or // waits until it completes, in both cases it returns the resulting error. // Note this errors if called multiple times for the same duty. -func (c *Component) propose(ctx context.Context, duty core.Duty, value proto.Message) error { +func (c *QBFTConsensus) propose(ctx context.Context, duty core.Duty, value proto.Message) error { hash, err := hashProto(value) if err != nil { return err @@ -372,7 +302,7 @@ func (c *Component) propose(ctx context.Context, duty core.Duty, value proto.Mes // unsigned data from beacon node and Propose not already called. // Note Propose must still be called for this peer to propose a value when leading a round. // Note this errors if called multiple times for the same duty. -func (c *Component) Participate(ctx context.Context, duty core.Duty) error { +func (c *QBFTConsensus) Participate(ctx context.Context, duty core.Duty) error { if duty.Type == core.DutyAggregator || duty.Type == core.DutySyncContribution { return nil // No consensus participate for potential no-op aggregation duties. } @@ -397,7 +327,7 @@ func (c *Component) Participate(ctx context.Context, duty core.Duty) error { // runInstance blocks and runs a consensus instance for the given duty. // It returns an error or nil when the context is cancelled. // Note each instance may only be run once. -func (c *Component) runInstance(ctx context.Context, duty core.Duty) (err error) { +func (c *QBFTConsensus) runInstance(ctx context.Context, duty core.Duty) (err error) { roundTimer := c.timerFunc(duty) ctx = log.WithTopic(ctx, "qbft") ctx = log.WithCtx(ctx, z.Any("duty", duty)) @@ -433,10 +363,10 @@ func (c *Component) runInstance(ctx context.Context, duty core.Duty) (err error) } // Create a new qbft definition for this instance. - def := newDefinition(len(c.peers), c.subscribers, roundTimer, decideCallback) + def := newQBFTDefinition(len(c.peers), c.subscribers, roundTimer, decideCallback) // Create a new transport that handles sending and receiving for this instance. - t := transport{ + t := qbftTransport{ component: c, values: make(map[[32]byte]*anypb.Any), valueCh: inst.valueCh, @@ -475,7 +405,7 @@ func (c *Component) runInstance(ctx context.Context, duty core.Duty) (err error) } // handle processes an incoming consensus wire message. -func (c *Component) handle(ctx context.Context, _ peer.ID, req proto.Message) (proto.Message, bool, error) { +func (c *QBFTConsensus) handle(ctx context.Context, _ peer.ID, req proto.Message) (proto.Message, bool, error) { t0 := time.Now() pbMsg, ok := req.(*pbv1.ConsensusMsg) @@ -514,7 +444,7 @@ func (c *Component) handle(ctx context.Context, _ peer.ID, req proto.Message) (p return nil, false, err } - msg, err := newMsg(pbMsg.GetMsg(), pbMsg.GetJustification(), values) + msg, err := newQBFTMsg(pbMsg.GetMsg(), pbMsg.GetJustification(), values) if err != nil { return nil, false, err } @@ -540,13 +470,13 @@ func (c *Component) handle(ctx context.Context, _ peer.ID, req proto.Message) (p } // getRecvBuffer returns a receive buffer for the duty. -func (c *Component) getRecvBuffer(duty core.Duty) chan msg { +func (c *QBFTConsensus) getRecvBuffer(duty core.Duty) chan qbftMsg { c.mutable.Lock() defer c.mutable.Unlock() inst, ok := c.mutable.instances[duty] if !ok { - inst = newInstanceIO() + inst = newInstanceIO[qbftMsg]() c.mutable.instances[duty] = inst } @@ -554,13 +484,13 @@ func (c *Component) getRecvBuffer(duty core.Duty) chan msg { } // getInstanceIO returns the duty's instance and true if it were previously created. -func (c *Component) getInstanceIO(duty core.Duty) instanceIO { +func (c *QBFTConsensus) getInstanceIO(duty core.Duty) instanceIO[qbftMsg] { c.mutable.Lock() defer c.mutable.Unlock() inst, ok := c.mutable.instances[duty] if !ok { // Create new instanceIO. - inst = newInstanceIO() + inst = newInstanceIO[qbftMsg]() c.mutable.instances[duty] = inst return inst @@ -570,7 +500,7 @@ func (c *Component) getInstanceIO(duty core.Duty) instanceIO { } // deleteInstanceIO deletes the instanceIO for the duty. -func (c *Component) deleteInstanceIO(duty core.Duty) { +func (c *QBFTConsensus) deleteInstanceIO(duty core.Duty) { c.mutable.Lock() defer c.mutable.Unlock() @@ -578,7 +508,7 @@ func (c *Component) deleteInstanceIO(duty core.Duty) { } // getPeerIdx returns the local peer index. -func (c *Component) getPeerIdx() (int64, error) { +func (c *QBFTConsensus) getPeerIdx() (int64, error) { peerIdx := int64(-1) for i, p := range c.peers { if c.tcpNode.ID() == p.ID { @@ -617,7 +547,7 @@ func verifyMsg(msg *pbv1.QBFTMsg, pubkeys map[int64]*k1.PublicKey) error { return errors.New("invalid peer index", z.I64("index", msg.GetPeerIdx())) } - if ok, err := verifyMsgSig(msg, msgPubkey); err != nil { + if ok, err := verifyQBFTMsgSig(msg, msgPubkey); err != nil { return errors.Wrap(err, "verify consensus message signature") } else if !ok { return errors.New("invalid consensus message signature") diff --git a/core/consensus/qbft_internal_test.go b/core/consensus/qbft_internal_test.go index fc827042d..fb53c7752 100644 --- a/core/consensus/qbft_internal_test.go +++ b/core/consensus/qbft_internal_test.go @@ -118,15 +118,15 @@ func (t testMsg) Justification() []qbft.Msg[core.Duty, [32]byte] { panic("implement me") } -func TestComponent_handle(t *testing.T) { +func TestQBFTConsensus_handle(t *testing.T) { tests := []struct { name string - mutate func(base *pbv1.ConsensusMsg, c *Component) + mutate func(base *pbv1.ConsensusMsg, c *QBFTConsensus) checkErr func(err error) }{ { "qbft message with no pubkey errors", - func(base *pbv1.ConsensusMsg, c *Component) { + func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { // construct a valid basis message signature base.Msg.Duty.Type = 1 base.Msg.Signature = bytes.Repeat([]byte{42}, 65) @@ -142,7 +142,7 @@ func TestComponent_handle(t *testing.T) { }, { "qbft message with justifications mentioning unknown peerIdx errors", - func(base *pbv1.ConsensusMsg, c *Component) { + func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -189,7 +189,7 @@ func TestComponent_handle(t *testing.T) { }, { "qbft message with nil justification present in slice", - func(base *pbv1.ConsensusMsg, c *Component) { + func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -222,7 +222,7 @@ func TestComponent_handle(t *testing.T) { }, { "qbft message values present but nil", - func(base *pbv1.ConsensusMsg, c *Component) { + func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -255,7 +255,7 @@ func TestComponent_handle(t *testing.T) { }, { "qbft message with invalid duty fails", - func(base *pbv1.ConsensusMsg, c *Component) { + func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { // construct a valid basis message signature base.Msg.Duty.Type = 1 base.Msg.Signature = bytes.Repeat([]byte{42}, 65) @@ -271,7 +271,7 @@ func TestComponent_handle(t *testing.T) { }, { "qbft message with valid duty fails because justification has different duty type", - func(base *pbv1.ConsensusMsg, c *Component) { + func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -318,7 +318,7 @@ func TestComponent_handle(t *testing.T) { }, { "qbft message with valid duty and justification with same duty does not fail", - func(base *pbv1.ConsensusMsg, c *Component) { + func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -369,9 +369,9 @@ func TestComponent_handle(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - var tc Component + var tc QBFTConsensus tc.deadliner = testDeadliner{} - tc.mutable.instances = make(map[core.Duty]instanceIO) + tc.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) tc.gaterFunc = func(core.Duty) bool { return true } msg := &pbv1.ConsensusMsg{ @@ -386,7 +386,7 @@ func TestComponent_handle(t *testing.T) { } } -func TestComponentHandle(t *testing.T) { +func TestQBFTConsensusHandle(t *testing.T) { tests := []struct { name string msg *pbv1.ConsensusMsg @@ -449,7 +449,7 @@ func TestComponentHandle(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - c := &Component{ + c := &QBFTConsensus{ gaterFunc: func(core.Duty) bool { return true }, } @@ -461,16 +461,16 @@ func TestComponentHandle(t *testing.T) { func TestInstanceIO_MaybeStart(t *testing.T) { t.Run("MaybeStart for new instance", func(t *testing.T) { - inst1 := newInstanceIO() + inst1 := newInstanceIO[qbftMsg]() require.True(t, inst1.MaybeStart()) require.False(t, inst1.MaybeStart()) }) t.Run("MaybeStart after handle", func(t *testing.T) { - var c Component + var c QBFTConsensus c.deadliner = testDeadliner{} c.gaterFunc = func(core.Duty) bool { return true } - c.mutable.instances = make(map[core.Duty]instanceIO) + c.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) // Generate a p2p private key. p2pKey := testutil.GenerateInsecureK1Key(t, 0) @@ -496,10 +496,10 @@ func TestInstanceIO_MaybeStart(t *testing.T) { t.Run("Call Propose after handle", func(t *testing.T) { ctx := context.Background() - var c Component + var c QBFTConsensus c.deadliner = testDeadliner{} c.gaterFunc = func(core.Duty) bool { return true } - c.mutable.instances = make(map[core.Duty]instanceIO) + c.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) c.timerFunc = getTimerFunc() // Generate a p2p private key pair. diff --git a/core/consensus/msg.go b/core/consensus/qbft_msg.go similarity index 73% rename from core/consensus/msg.go rename to core/consensus/qbft_msg.go index a4d1352ed..a3882e92f 100644 --- a/core/consensus/msg.go +++ b/core/consensus/qbft_msg.go @@ -15,10 +15,10 @@ import ( "github.com/obolnetwork/charon/core/qbft" ) -// newMsg returns a new msg. -func newMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]byte]*anypb.Any) (msg, error) { +// newQBFTMsg returns a new QBFT msg. +func newQBFTMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]byte]*anypb.Any) (qbftMsg, error) { if pbMsg == nil { - return msg{}, errors.New("nil qbft message") + return qbftMsg{}, errors.New("nil qbft message") } // Do all possible error conversions first. @@ -30,28 +30,28 @@ func newMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]b if hash, ok := toHash32(pbMsg.GetValueHash()); ok { valueHash = hash if _, ok := values[valueHash]; !ok { - return msg{}, errors.New("value hash not found in values") + return qbftMsg{}, errors.New("value hash not found in values") } } if hash, ok := toHash32(pbMsg.GetPreparedValueHash()); ok { preparedValueHash = hash if _, ok := values[preparedValueHash]; !ok { - return msg{}, errors.New("prepared value hash not found in values") + return qbftMsg{}, errors.New("prepared value hash not found in values") } } var justImpls []qbft.Msg[core.Duty, [32]byte] for _, j := range justification { - impl, err := newMsg(j, nil, values) + impl, err := newQBFTMsg(j, nil, values) if err != nil { - return msg{}, err + return qbftMsg{}, err } justImpls = append(justImpls, impl) } - return msg{ + return qbftMsg{ msg: pbMsg, valueHash: valueHash, values: values, @@ -61,8 +61,8 @@ func newMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]b }, nil } -// msg wraps *pbv1.QBFTMsg and justifications and implements qbft.Msg[core.Duty, [32]byte]. -type msg struct { +// qbftMsg wraps *pbv1.QBFTMsg and justifications and implements qbft.Msg[core.Duty, [32]byte]. +type qbftMsg struct { msg *pbv1.QBFTMsg valueHash [32]byte preparedValueHash [32]byte @@ -72,39 +72,39 @@ type msg struct { justification []qbft.Msg[core.Duty, [32]byte] } -func (m msg) Type() qbft.MsgType { +func (m qbftMsg) Type() qbft.MsgType { return qbft.MsgType(m.msg.GetType()) } -func (m msg) Instance() core.Duty { +func (m qbftMsg) Instance() core.Duty { return core.DutyFromProto(m.msg.GetDuty()) } -func (m msg) Source() int64 { +func (m qbftMsg) Source() int64 { return m.msg.GetPeerIdx() } -func (m msg) Round() int64 { +func (m qbftMsg) Round() int64 { return m.msg.GetRound() } -func (m msg) Value() [32]byte { +func (m qbftMsg) Value() [32]byte { return m.valueHash } -func (m msg) PreparedRound() int64 { +func (m qbftMsg) PreparedRound() int64 { return m.msg.GetPreparedRound() } -func (m msg) PreparedValue() [32]byte { +func (m qbftMsg) PreparedValue() [32]byte { return m.preparedValueHash } -func (m msg) Justification() []qbft.Msg[core.Duty, [32]byte] { +func (m qbftMsg) Justification() []qbft.Msg[core.Duty, [32]byte] { return m.justification } -func (m msg) ToConsensusMsg() *pbv1.ConsensusMsg { +func (m qbftMsg) ToConsensusMsg() *pbv1.ConsensusMsg { var values []*anypb.Any for _, v := range m.values { values = append(values, v) @@ -146,8 +146,8 @@ func hashProto(msg proto.Message) ([32]byte, error) { return hash, nil } -// verifyMsgSig returns true if the message was signed by pubkey. -func verifyMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { +// verifyQBFTMsgSig returns true if the message was signed by pubkey. +func verifyQBFTMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { if msg.Signature == nil { return false, errors.New("empty signature") } @@ -170,8 +170,8 @@ func verifyMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { return recovered.IsEqual(pubkey), nil } -// signMsg returns a copy of the proto message with a populated signature signed by the provided private key. -func signMsg(msg *pbv1.QBFTMsg, privkey *k1.PrivateKey) (*pbv1.QBFTMsg, error) { +// signQBFTMsg returns a copy of the proto message with a populated signature signed by the provided private key. +func signQBFTMsg(msg *pbv1.QBFTMsg, privkey *k1.PrivateKey) (*pbv1.QBFTMsg, error) { clone, ok := proto.Clone(msg).(*pbv1.QBFTMsg) if !ok { return nil, errors.New("type assert qbft msg") @@ -205,4 +205,4 @@ func toHash32(val []byte) ([32]byte, bool) { return resp, true } -var _ qbft.Msg[core.Duty, [32]byte] = msg{} // Interface assertion +var _ qbft.Msg[core.Duty, [32]byte] = qbftMsg{} // Interface assertion diff --git a/core/consensus/msg_internal_test.go b/core/consensus/qbft_msg_internal_test.go similarity index 92% rename from core/consensus/msg_internal_test.go rename to core/consensus/qbft_msg_internal_test.go index ec5c95777..7ca58a814 100644 --- a/core/consensus/msg_internal_test.go +++ b/core/consensus/qbft_msg_internal_test.go @@ -45,16 +45,16 @@ func TestSigning(t *testing.T) { msg := randomMsg(t) - signed, err := signMsg(msg, privkey) + signed, err := signQBFTMsg(msg, privkey) require.NoError(t, err) - ok, err := verifyMsgSig(signed, privkey.PubKey()) + ok, err := verifyQBFTMsgSig(signed, privkey.PubKey()) require.NoError(t, err) require.True(t, ok) privkey2, err := k1.GeneratePrivateKey() require.NoError(t, err) - ok, err = verifyMsgSig(signed, privkey2.PubKey()) + ok, err = verifyQBFTMsgSig(signed, privkey2.PubKey()) require.NoError(t, err) require.False(t, ok) } @@ -77,7 +77,7 @@ func TestNewMsg(t *testing.T) { hash2: any2, } - msg, err := newMsg(&pbv1.QBFTMsg{ + msg, err := newQBFTMsg(&pbv1.QBFTMsg{ Type: int64(qbft.MsgPrePrepare), ValueHash: hash1[:], PreparedValueHash: hash2[:], @@ -94,7 +94,7 @@ func TestPartialLegacyNewMsg(t *testing.T) { hash1, err := hashProto(val1) require.NoError(t, err) - _, err = newMsg(&pbv1.QBFTMsg{ + _, err = newQBFTMsg(&pbv1.QBFTMsg{ Type: int64(qbft.MsgPrePrepare), }, []*pbv1.QBFTMsg{ { diff --git a/core/consensus/qbft_test.go b/core/consensus/qbft_test.go index 3c5fa5b33..8121019b4 100644 --- a/core/consensus/qbft_test.go +++ b/core/consensus/qbft_test.go @@ -27,7 +27,7 @@ import ( "github.com/obolnetwork/charon/testutil" ) -func TestComponent(t *testing.T) { +func TestQBFTConsensus(t *testing.T) { tests := []struct { name string threshold int @@ -57,14 +57,14 @@ func TestComponent(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - testComponent(t, tt.threshold, tt.nodes) + testQBFTConsensus(t, tt.threshold, tt.nodes) }) } } -// testComponent tests a consensus instance with size of threshold-of-nodes. +// testQBFTConsensus tests a consensus instance with size of threshold-of-nodes. // Note it only instantiates the minimum amount of peers, ie threshold. -func testComponent(t *testing.T, threshold, nodes int) { +func testQBFTConsensus(t *testing.T, threshold, nodes int) { t.Helper() seed := 0 random := rand.New(rand.NewSource(int64(seed))) @@ -74,7 +74,7 @@ func testComponent(t *testing.T, threshold, nodes int) { peers []p2p.Peer hosts []host.Host hostsInfo []peer.AddrInfo - components []*consensus.Component + components []*consensus.QBFTConsensus results = make(chan core.UnsignedDataSet, threshold) runErrs = make(chan error, threshold) sniffed = make(chan int, threshold) @@ -119,7 +119,7 @@ func testComponent(t *testing.T, threshold, nodes int) { gaterFunc := func(core.Duty) bool { return true } - c, err := consensus.New(hosts[i], new(p2p.Sender), peers, p2pkeys[i], testDeadliner{}, gaterFunc, sniffer) + c, err := consensus.NewQBFTConsensus(hosts[i], new(p2p.Sender), peers, p2pkeys[i], testDeadliner{}, gaterFunc, sniffer) require.NoError(t, err) c.Subscribe(func(_ context.Context, _ core.Duty, set core.UnsignedDataSet) error { results <- set @@ -134,7 +134,7 @@ func testComponent(t *testing.T, threshold, nodes int) { // Start all components. for i, c := range components { - go func(ctx context.Context, i int, c *consensus.Component) { + go func(ctx context.Context, i int, c *consensus.QBFTConsensus) { runErrs <- c.Propose( log.WithCtx(ctx, z.Int("node", i), z.Str("peer", p2p.PeerName(hosts[i].ID()))), core.Duty{Type: core.DutyAttester, Slot: 1}, diff --git a/core/consensus/transport.go b/core/consensus/qbft_transport.go similarity index 68% rename from core/consensus/transport.go rename to core/consensus/qbft_transport.go index a3e48c61b..2654d82a4 100644 --- a/core/consensus/transport.go +++ b/core/consensus/qbft_transport.go @@ -5,12 +5,10 @@ package consensus import ( "context" "sync" - "time" k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" - "google.golang.org/protobuf/types/known/timestamppb" "github.com/obolnetwork/charon/app/errors" "github.com/obolnetwork/charon/core" @@ -18,10 +16,10 @@ import ( "github.com/obolnetwork/charon/core/qbft" ) -// transport encapsulates receiving and broadcasting for a consensus instance/duty. -type transport struct { +// qbftTransport encapsulates receiving and broadcasting for a consensus instance/duty. +type qbftTransport struct { // Immutable state - component *Component + component *QBFTConsensus recvBuffer chan qbft.Msg[core.Duty, [32]byte] // Instance inner receive buffer. sniffer *sniffer @@ -32,7 +30,7 @@ type transport struct { } // setValues caches the values and their hashes. -func (t *transport) setValues(msg msg) { +func (t *qbftTransport) setValues(msg qbftMsg) { t.valueMu.Lock() defer t.valueMu.Unlock() @@ -42,7 +40,7 @@ func (t *transport) setValues(msg msg) { } // getValue returns the value by its hash. -func (t *transport) getValue(hash [32]byte) (*anypb.Any, error) { +func (t *qbftTransport) getValue(hash [32]byte) (*anypb.Any, error) { t.valueMu.Lock() defer t.valueMu.Unlock() @@ -73,7 +71,7 @@ func (t *transport) getValue(hash [32]byte) (*anypb.Any, error) { } // Broadcast creates a msg and sends it to all peers (including self). -func (t *transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.Duty, +func (t *qbftTransport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.Duty, peerIdx int64, round int64, valueHash [32]byte, pr int64, pvHash [32]byte, justification []qbft.Msg[core.Duty, [32]byte], ) error { @@ -82,7 +80,7 @@ func (t *transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.D hashes = append(hashes, valueHash) hashes = append(hashes, pvHash) for _, just := range justification { - msg, ok := just.(msg) + msg, ok := just.(qbftMsg) if !ok { return errors.New("invalid justification message") } @@ -106,7 +104,7 @@ func (t *transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.D } // Make the message - msg, err := createMsg(typ, duty, peerIdx, round, valueHash, pr, + msg, err := createQBFTMsg(typ, duty, peerIdx, round, valueHash, pr, pvHash, values, justification, t.component.privkey) if err != nil { return err @@ -137,7 +135,7 @@ func (t *transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.D } // ProcessReceives processes received messages from the outer buffer until the context is closed. -func (t *transport) ProcessReceives(ctx context.Context, outerBuffer chan msg) { +func (t *qbftTransport) ProcessReceives(ctx context.Context, outerBuffer chan qbftMsg) { for { select { case <-ctx.Done(): @@ -155,13 +153,13 @@ func (t *transport) ProcessReceives(ctx context.Context, outerBuffer chan msg) { } } -// createMsg returns a new message by converting the inputs into a protobuf +// createQBFTMsg returns a new message by converting the inputs into a protobuf // and wrapping that in a msg type. -func createMsg(typ qbft.MsgType, duty core.Duty, +func createQBFTMsg(typ qbft.MsgType, duty core.Duty, peerIdx int64, round int64, vHash [32]byte, pr int64, pvHash [32]byte, values map[[32]byte]*anypb.Any, justification []qbft.Msg[core.Duty, [32]byte], privkey *k1.PrivateKey, -) (msg, error) { +) (qbftMsg, error) { pbMsg := &pbv1.QBFTMsg{ Type: int64(typ), Duty: core.DutyToProto(duty), @@ -172,63 +170,20 @@ func createMsg(typ qbft.MsgType, duty core.Duty, PreparedValueHash: pvHash[:], } - pbMsg, err := signMsg(pbMsg, privkey) + pbMsg, err := signQBFTMsg(pbMsg, privkey) if err != nil { - return msg{}, err + return qbftMsg{}, err } // Transform justifications into protobufs var justMsgs []*pbv1.QBFTMsg for _, j := range justification { - impl, ok := j.(msg) + impl, ok := j.(qbftMsg) if !ok { - return msg{}, errors.New("invalid justification") + return qbftMsg{}, errors.New("invalid justification") } justMsgs = append(justMsgs, impl.msg) // Note nested justifications are ignored. } - return newMsg(pbMsg, justMsgs, values) -} - -// newSniffer returns a new sniffer. -func newSniffer(nodes, peerIdx int64) *sniffer { - return &sniffer{ - nodes: nodes, - peerIdx: peerIdx, - startedAt: time.Now(), - } -} - -// sniffer buffers consensus messages. -type sniffer struct { - nodes int64 - peerIdx int64 - startedAt time.Time - - mu sync.Mutex - msgs []*pbv1.SniffedConsensusMsg -} - -// Add adds a message to the sniffer buffer. -func (c *sniffer) Add(msg *pbv1.ConsensusMsg) { - c.mu.Lock() - defer c.mu.Unlock() - - c.msgs = append(c.msgs, &pbv1.SniffedConsensusMsg{ - Timestamp: timestamppb.Now(), - Msg: msg, - }) -} - -// Instance returns the buffered messages as an instance. -func (c *sniffer) Instance() *pbv1.SniffedConsensusInstance { - c.mu.Lock() - defer c.mu.Unlock() - - return &pbv1.SniffedConsensusInstance{ - Nodes: c.nodes, - PeerIdx: c.peerIdx, - StartedAt: timestamppb.New(c.startedAt), - Msgs: c.msgs, - } + return newQBFTMsg(pbMsg, justMsgs, values) } diff --git a/core/consensus/sniffed_internal_test.go b/core/consensus/sniffed_internal_test.go index 1364ae076..9489ea976 100644 --- a/core/consensus/sniffed_internal_test.go +++ b/core/consensus/sniffed_internal_test.go @@ -68,7 +68,7 @@ func testSniffedInstance(ctx context.Context, t *testing.T, instance *pbv1.Sniff var expectDecided bool - def := newDefinition(int(instance.GetNodes()), func() []subscriber { + def := newQBFTDefinition(int(instance.GetNodes()), func() []subscriber { return []subscriber{func(ctx context.Context, duty core.Duty, value proto.Message) error { log.Info(ctx, "Consensus decided", z.Any("value", value)) expectDecided = true @@ -91,7 +91,7 @@ func testSniffedInstance(ctx context.Context, t *testing.T, instance *pbv1.Sniff values, err := valuesByHash(msg.GetMsg().GetValues()) require.NoError(t, err) - m, err := newMsg(msg.GetMsg().GetMsg(), msg.GetMsg().GetJustification(), values) + m, err := newQBFTMsg(msg.GetMsg().GetMsg(), msg.GetMsg().GetJustification(), values) require.NoError(t, err) recvBuffer <- m } diff --git a/core/consensus/sniffer.go b/core/consensus/sniffer.go new file mode 100644 index 000000000..f411cdaab --- /dev/null +++ b/core/consensus/sniffer.go @@ -0,0 +1,55 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + "sync" + "time" + + "google.golang.org/protobuf/types/known/timestamppb" + + pbv1 "github.com/obolnetwork/charon/core/corepb/v1" +) + +// newSniffer returns a new sniffer. +func newSniffer(nodes, peerIdx int64) *sniffer { + return &sniffer{ + nodes: nodes, + peerIdx: peerIdx, + startedAt: time.Now(), + } +} + +// sniffer buffers consensus messages. +type sniffer struct { + nodes int64 + peerIdx int64 + startedAt time.Time + + mu sync.Mutex + msgs []*pbv1.SniffedConsensusMsg +} + +// Add adds a message to the sniffer buffer. +func (c *sniffer) Add(msg *pbv1.ConsensusMsg) { + c.mu.Lock() + defer c.mu.Unlock() + + c.msgs = append(c.msgs, &pbv1.SniffedConsensusMsg{ + Timestamp: timestamppb.Now(), + Msg: msg, + }) +} + +// Instance returns the buffered messages as an instance. +func (c *sniffer) Instance() *pbv1.SniffedConsensusInstance { + c.mu.Lock() + defer c.mu.Unlock() + + return &pbv1.SniffedConsensusInstance{ + Nodes: c.nodes, + PeerIdx: c.peerIdx, + StartedAt: timestamppb.New(c.startedAt), + Msgs: c.msgs, + } +} diff --git a/core/consensus/strategysim_internal_test.go b/core/consensus/strategysim_internal_test.go index a3e5acfe9..1c2d08d59 100644 --- a/core/consensus/strategysim_internal_test.go +++ b/core/consensus/strategysim_internal_test.go @@ -631,7 +631,7 @@ func (i *transportInstance) Broadcast(_ context.Context, typ qbft.MsgType, // Transform justifications into protobufs var justMsgs []*pbv1.QBFTMsg for _, j := range justification { - impl, ok := j.(msg) + impl, ok := j.(qbftMsg) if !ok { return errors.New("invalid justification") } @@ -640,7 +640,7 @@ func (i *transportInstance) Broadcast(_ context.Context, typ qbft.MsgType, values[impl.PreparedValue()] = dummy } - msg, err := newMsg(pbMsg, justMsgs, values) + msg, err := newQBFTMsg(pbMsg, justMsgs, values) if err != nil { return err } From 8da6e821ec58e51d0b199137173ea5c7a833e60d Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 9 Oct 2024 11:34:01 +0300 Subject: [PATCH 04/20] Updated protoc version --- app/log/loki/lokipb/v1/loki.pb.go | 70 ++------ app/peerinfo/peerinfopb/v1/peerinfo.pb.go | 26 +-- app/protonil/testdata/v1/test.pb.go | 136 +++------------ cluster/manifestpb/v1/manifest.pb.go | 202 ++++------------------ core/corepb/v1/consensus.pb.go | 114 +++--------- core/corepb/v1/core.pb.go | 92 ++-------- core/corepb/v1/parsigex.pb.go | 26 +-- core/corepb/v1/priority.pb.go | 114 +++--------- dkg/dkgpb/v1/bcast.pb.go | 70 ++------ dkg/dkgpb/v1/frost.pb.go | 158 ++++------------- dkg/dkgpb/v1/nodesigs.pb.go | 26 +-- dkg/dkgpb/v1/sync.pb.go | 48 +---- 12 files changed, 200 insertions(+), 882 deletions(-) diff --git a/app/log/loki/lokipb/v1/loki.pb.go b/app/log/loki/lokipb/v1/loki.pb.go index 4b6d54ab0..30c9403ae 100644 --- a/app/log/loki/lokipb/v1/loki.pb.go +++ b/app/log/loki/lokipb/v1/loki.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: app/log/loki/lokipb/v1/loki.proto @@ -31,11 +31,9 @@ type PushRequest struct { func (x *PushRequest) Reset() { *x = PushRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PushRequest) String() string { @@ -46,7 +44,7 @@ func (*PushRequest) ProtoMessage() {} func (x *PushRequest) ProtoReflect() protoreflect.Message { mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -80,11 +78,9 @@ type Stream struct { func (x *Stream) Reset() { *x = Stream{} - if protoimpl.UnsafeEnabled { - mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Stream) String() string { @@ -95,7 +91,7 @@ func (*Stream) ProtoMessage() {} func (x *Stream) ProtoReflect() protoreflect.Message { mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -142,11 +138,9 @@ type Entry struct { func (x *Entry) Reset() { *x = Entry{} - if protoimpl.UnsafeEnabled { - mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Entry) String() string { @@ -157,7 +151,7 @@ func (*Entry) ProtoMessage() {} func (x *Entry) ProtoReflect() protoreflect.Message { mi := &file_app_log_loki_lokipb_v1_loki_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -253,44 +247,6 @@ func file_app_log_loki_lokipb_v1_loki_proto_init() { if File_app_log_loki_lokipb_v1_loki_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_app_log_loki_lokipb_v1_loki_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*PushRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_app_log_loki_lokipb_v1_loki_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*Stream); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_app_log_loki_lokipb_v1_loki_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*Entry); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/app/peerinfo/peerinfopb/v1/peerinfo.pb.go b/app/peerinfo/peerinfopb/v1/peerinfo.pb.go index daaa60454..e23453cf0 100644 --- a/app/peerinfo/peerinfopb/v1/peerinfo.pb.go +++ b/app/peerinfo/peerinfopb/v1/peerinfo.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: app/peerinfo/peerinfopb/v1/peerinfo.proto @@ -36,11 +36,9 @@ type PeerInfo struct { func (x *PeerInfo) Reset() { *x = PeerInfo{} - if protoimpl.UnsafeEnabled { - mi := &file_app_peerinfo_peerinfopb_v1_peerinfo_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_peerinfo_peerinfopb_v1_peerinfo_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PeerInfo) String() string { @@ -51,7 +49,7 @@ func (*PeerInfo) ProtoMessage() {} func (x *PeerInfo) ProtoReflect() protoreflect.Message { mi := &file_app_peerinfo_peerinfopb_v1_peerinfo_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -175,20 +173,6 @@ func file_app_peerinfo_peerinfopb_v1_peerinfo_proto_init() { if File_app_peerinfo_peerinfopb_v1_peerinfo_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_app_peerinfo_peerinfopb_v1_peerinfo_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*PeerInfo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } file_app_peerinfo_peerinfopb_v1_peerinfo_proto_msgTypes[0].OneofWrappers = []any{} type x struct{} out := protoimpl.TypeBuilder{ diff --git a/app/protonil/testdata/v1/test.pb.go b/app/protonil/testdata/v1/test.pb.go index 8b50c2d1b..be5292b85 100644 --- a/app/protonil/testdata/v1/test.pb.go +++ b/app/protonil/testdata/v1/test.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: app/protonil/testdata/v1/test.proto @@ -32,11 +32,9 @@ type M1 struct { func (x *M1) Reset() { *x = M1{} - if protoimpl.UnsafeEnabled { - mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *M1) String() string { @@ -47,7 +45,7 @@ func (*M1) ProtoMessage() {} func (x *M1) ProtoReflect() protoreflect.Message { mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -95,11 +93,9 @@ type M2 struct { func (x *M2) Reset() { *x = M2{} - if protoimpl.UnsafeEnabled { - mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *M2) String() string { @@ -110,7 +106,7 @@ func (*M2) ProtoMessage() {} func (x *M2) ProtoReflect() protoreflect.Message { mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -156,11 +152,9 @@ type M3 struct { func (x *M3) Reset() { *x = M3{} - if protoimpl.UnsafeEnabled { - mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *M3) String() string { @@ -171,7 +165,7 @@ func (*M3) ProtoMessage() {} func (x *M3) ProtoReflect() protoreflect.Message { mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -205,11 +199,9 @@ type M4 struct { func (x *M4) Reset() { *x = M4{} - if protoimpl.UnsafeEnabled { - mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *M4) String() string { @@ -220,7 +212,7 @@ func (*M4) ProtoMessage() {} func (x *M4) ProtoReflect() protoreflect.Message { mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -267,11 +259,9 @@ type MaxIndex struct { func (x *MaxIndex) Reset() { *x = MaxIndex{} - if protoimpl.UnsafeEnabled { - mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MaxIndex) String() string { @@ -282,7 +272,7 @@ func (*MaxIndex) ProtoMessage() {} func (x *MaxIndex) ProtoReflect() protoreflect.Message { mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -321,11 +311,9 @@ type Attack struct { func (x *Attack) Reset() { *x = Attack{} - if protoimpl.UnsafeEnabled { - mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Attack) String() string { @@ -336,7 +324,7 @@ func (*Attack) ProtoMessage() {} func (x *Attack) ProtoReflect() protoreflect.Message { mi := &file_app_protonil_testdata_v1_test_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -502,80 +490,6 @@ func file_app_protonil_testdata_v1_test_proto_init() { if File_app_protonil_testdata_v1_test_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_app_protonil_testdata_v1_test_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*M1); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_app_protonil_testdata_v1_test_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*M2); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_app_protonil_testdata_v1_test_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*M3); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_app_protonil_testdata_v1_test_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*M4); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_app_protonil_testdata_v1_test_proto_msgTypes[4].Exporter = func(v any, i int) any { - switch v := v.(*MaxIndex); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_app_protonil_testdata_v1_test_proto_msgTypes[5].Exporter = func(v any, i int) any { - switch v := v.(*Attack); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } file_app_protonil_testdata_v1_test_proto_msgTypes[0].OneofWrappers = []any{} file_app_protonil_testdata_v1_test_proto_msgTypes[1].OneofWrappers = []any{} file_app_protonil_testdata_v1_test_proto_msgTypes[5].OneofWrappers = []any{} diff --git a/cluster/manifestpb/v1/manifest.pb.go b/cluster/manifestpb/v1/manifest.pb.go index aa6b000c1..cec646a99 100644 --- a/cluster/manifestpb/v1/manifest.pb.go +++ b/cluster/manifestpb/v1/manifest.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: cluster/manifestpb/v1/manifest.proto @@ -39,11 +39,9 @@ type Cluster struct { func (x *Cluster) Reset() { *x = Cluster{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Cluster) String() string { @@ -54,7 +52,7 @@ func (*Cluster) ProtoMessage() {} func (x *Cluster) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -138,11 +136,9 @@ type Mutation struct { func (x *Mutation) Reset() { *x = Mutation{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Mutation) String() string { @@ -153,7 +149,7 @@ func (*Mutation) ProtoMessage() {} func (x *Mutation) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -202,11 +198,9 @@ type SignedMutation struct { func (x *SignedMutation) Reset() { *x = SignedMutation{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SignedMutation) String() string { @@ -217,7 +211,7 @@ func (*SignedMutation) ProtoMessage() {} func (x *SignedMutation) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -264,11 +258,9 @@ type SignedMutationList struct { func (x *SignedMutationList) Reset() { *x = SignedMutationList{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SignedMutationList) String() string { @@ -279,7 +271,7 @@ func (*SignedMutationList) ProtoMessage() {} func (x *SignedMutationList) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -313,11 +305,9 @@ type Operator struct { func (x *Operator) Reset() { *x = Operator{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Operator) String() string { @@ -328,7 +318,7 @@ func (*Operator) ProtoMessage() {} func (x *Operator) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -372,11 +362,9 @@ type Validator struct { func (x *Validator) Reset() { *x = Validator{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Validator) String() string { @@ -387,7 +375,7 @@ func (*Validator) ProtoMessage() {} func (x *Validator) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -448,11 +436,9 @@ type ValidatorList struct { func (x *ValidatorList) Reset() { *x = ValidatorList{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ValidatorList) String() string { @@ -463,7 +449,7 @@ func (*ValidatorList) ProtoMessage() {} func (x *ValidatorList) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -496,11 +482,9 @@ type LegacyLock struct { func (x *LegacyLock) Reset() { *x = LegacyLock{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LegacyLock) String() string { @@ -511,7 +495,7 @@ func (*LegacyLock) ProtoMessage() {} func (x *LegacyLock) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -542,11 +526,9 @@ type Empty struct { func (x *Empty) Reset() { *x = Empty{} - if protoimpl.UnsafeEnabled { - mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Empty) String() string { @@ -557,7 +539,7 @@ func (*Empty) ProtoMessage() {} func (x *Empty) ProtoReflect() protoreflect.Message { mi := &file_cluster_manifestpb_v1_manifest_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -700,116 +682,6 @@ func file_cluster_manifestpb_v1_manifest_proto_init() { if File_cluster_manifestpb_v1_manifest_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_cluster_manifestpb_v1_manifest_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*Cluster); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*Mutation); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*SignedMutation); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*SignedMutationList); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[4].Exporter = func(v any, i int) any { - switch v := v.(*Operator); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[5].Exporter = func(v any, i int) any { - switch v := v.(*Validator); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[6].Exporter = func(v any, i int) any { - switch v := v.(*ValidatorList); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[7].Exporter = func(v any, i int) any { - switch v := v.(*LegacyLock); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_cluster_manifestpb_v1_manifest_proto_msgTypes[8].Exporter = func(v any, i int) any { - switch v := v.(*Empty); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/core/corepb/v1/consensus.pb.go b/core/corepb/v1/consensus.pb.go index 82ceec09a..fb70b6c67 100644 --- a/core/corepb/v1/consensus.pb.go +++ b/core/corepb/v1/consensus.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: core/corepb/v1/consensus.proto @@ -39,11 +39,9 @@ type QBFTMsg struct { func (x *QBFTMsg) Reset() { *x = QBFTMsg{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_consensus_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_consensus_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *QBFTMsg) String() string { @@ -54,7 +52,7 @@ func (*QBFTMsg) ProtoMessage() {} func (x *QBFTMsg) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_consensus_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -137,11 +135,9 @@ type ConsensusMsg struct { func (x *ConsensusMsg) Reset() { *x = ConsensusMsg{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_consensus_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_consensus_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ConsensusMsg) String() string { @@ -152,7 +148,7 @@ func (*ConsensusMsg) ProtoMessage() {} func (x *ConsensusMsg) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_consensus_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -199,11 +195,9 @@ type SniffedConsensusMsg struct { func (x *SniffedConsensusMsg) Reset() { *x = SniffedConsensusMsg{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_consensus_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_consensus_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SniffedConsensusMsg) String() string { @@ -214,7 +208,7 @@ func (*SniffedConsensusMsg) ProtoMessage() {} func (x *SniffedConsensusMsg) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_consensus_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -256,11 +250,9 @@ type SniffedConsensusInstance struct { func (x *SniffedConsensusInstance) Reset() { *x = SniffedConsensusInstance{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_consensus_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_consensus_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SniffedConsensusInstance) String() string { @@ -271,7 +263,7 @@ func (*SniffedConsensusInstance) ProtoMessage() {} func (x *SniffedConsensusInstance) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_consensus_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -325,11 +317,9 @@ type SniffedConsensusInstances struct { func (x *SniffedConsensusInstances) Reset() { *x = SniffedConsensusInstances{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_consensus_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_consensus_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SniffedConsensusInstances) String() string { @@ -340,7 +330,7 @@ func (*SniffedConsensusInstances) ProtoMessage() {} func (x *SniffedConsensusInstances) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_consensus_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -490,68 +480,6 @@ func file_core_corepb_v1_consensus_proto_init() { return } file_core_corepb_v1_core_proto_init() - if !protoimpl.UnsafeEnabled { - file_core_corepb_v1_consensus_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*QBFTMsg); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_consensus_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*ConsensusMsg); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_consensus_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*SniffedConsensusMsg); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_consensus_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*SniffedConsensusInstance); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_consensus_proto_msgTypes[4].Exporter = func(v any, i int) any { - switch v := v.(*SniffedConsensusInstances); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/core/corepb/v1/core.pb.go b/core/corepb/v1/core.pb.go index 7b472d6fd..d83df0143 100644 --- a/core/corepb/v1/core.pb.go +++ b/core/corepb/v1/core.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: core/corepb/v1/core.proto @@ -31,11 +31,9 @@ type Duty struct { func (x *Duty) Reset() { *x = Duty{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_core_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_core_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Duty) String() string { @@ -46,7 +44,7 @@ func (*Duty) ProtoMessage() {} func (x *Duty) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_core_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -85,11 +83,9 @@ type UnsignedDataSet struct { func (x *UnsignedDataSet) Reset() { *x = UnsignedDataSet{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_core_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_core_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *UnsignedDataSet) String() string { @@ -100,7 +96,7 @@ func (*UnsignedDataSet) ProtoMessage() {} func (x *UnsignedDataSet) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_core_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -132,11 +128,9 @@ type ParSignedDataSet struct { func (x *ParSignedDataSet) Reset() { *x = ParSignedDataSet{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_core_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_core_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ParSignedDataSet) String() string { @@ -147,7 +141,7 @@ func (*ParSignedDataSet) ProtoMessage() {} func (x *ParSignedDataSet) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_core_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -181,11 +175,9 @@ type ParSignedData struct { func (x *ParSignedData) Reset() { *x = ParSignedData{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_core_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_core_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ParSignedData) String() string { @@ -196,7 +188,7 @@ func (*ParSignedData) ProtoMessage() {} func (x *ParSignedData) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_core_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -309,56 +301,6 @@ func file_core_corepb_v1_core_proto_init() { if File_core_corepb_v1_core_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_core_corepb_v1_core_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*Duty); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_core_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*UnsignedDataSet); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_core_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*ParSignedDataSet); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_core_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*ParSignedData); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/core/corepb/v1/parsigex.pb.go b/core/corepb/v1/parsigex.pb.go index aa767fd59..78097a497 100644 --- a/core/corepb/v1/parsigex.pb.go +++ b/core/corepb/v1/parsigex.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: core/corepb/v1/parsigex.proto @@ -31,11 +31,9 @@ type ParSigExMsg struct { func (x *ParSigExMsg) Reset() { *x = ParSigExMsg{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_parsigex_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_parsigex_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ParSigExMsg) String() string { @@ -46,7 +44,7 @@ func (*ParSigExMsg) ProtoMessage() {} func (x *ParSigExMsg) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_parsigex_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -130,20 +128,6 @@ func file_core_corepb_v1_parsigex_proto_init() { return } file_core_corepb_v1_core_proto_init() - if !protoimpl.UnsafeEnabled { - file_core_corepb_v1_parsigex_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*ParSigExMsg); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/core/corepb/v1/priority.pb.go b/core/corepb/v1/priority.pb.go index e40e12cc8..1083b8e07 100644 --- a/core/corepb/v1/priority.pb.go +++ b/core/corepb/v1/priority.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: core/corepb/v1/priority.proto @@ -33,11 +33,9 @@ type PriorityResult struct { func (x *PriorityResult) Reset() { *x = PriorityResult{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_priority_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_priority_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PriorityResult) String() string { @@ -48,7 +46,7 @@ func (*PriorityResult) ProtoMessage() {} func (x *PriorityResult) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_priority_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -91,11 +89,9 @@ type PriorityMsg struct { func (x *PriorityMsg) Reset() { *x = PriorityMsg{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_priority_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_priority_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PriorityMsg) String() string { @@ -106,7 +102,7 @@ func (*PriorityMsg) ProtoMessage() {} func (x *PriorityMsg) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_priority_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -161,11 +157,9 @@ type PriorityTopicProposal struct { func (x *PriorityTopicProposal) Reset() { *x = PriorityTopicProposal{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_priority_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_priority_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PriorityTopicProposal) String() string { @@ -176,7 +170,7 @@ func (*PriorityTopicProposal) ProtoMessage() {} func (x *PriorityTopicProposal) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_priority_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -219,11 +213,9 @@ type PriorityTopicResult struct { func (x *PriorityTopicResult) Reset() { *x = PriorityTopicResult{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_priority_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_priority_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PriorityTopicResult) String() string { @@ -234,7 +226,7 @@ func (*PriorityTopicResult) ProtoMessage() {} func (x *PriorityTopicResult) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_priority_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -275,11 +267,9 @@ type PriorityScoredResult struct { func (x *PriorityScoredResult) Reset() { *x = PriorityScoredResult{} - if protoimpl.UnsafeEnabled { - mi := &file_core_corepb_v1_priority_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_core_corepb_v1_priority_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PriorityScoredResult) String() string { @@ -290,7 +280,7 @@ func (*PriorityScoredResult) ProtoMessage() {} func (x *PriorityScoredResult) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_priority_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -421,68 +411,6 @@ func file_core_corepb_v1_priority_proto_init() { return } file_core_corepb_v1_core_proto_init() - if !protoimpl.UnsafeEnabled { - file_core_corepb_v1_priority_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*PriorityResult); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_priority_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*PriorityMsg); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_priority_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*PriorityTopicProposal); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_priority_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*PriorityTopicResult); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_core_corepb_v1_priority_proto_msgTypes[4].Exporter = func(v any, i int) any { - switch v := v.(*PriorityScoredResult); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/dkg/dkgpb/v1/bcast.pb.go b/dkg/dkgpb/v1/bcast.pb.go index 2b31dc35e..19674c2fb 100644 --- a/dkg/dkgpb/v1/bcast.pb.go +++ b/dkg/dkgpb/v1/bcast.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: dkg/dkgpb/v1/bcast.proto @@ -32,11 +32,9 @@ type BCastSigRequest struct { func (x *BCastSigRequest) Reset() { *x = BCastSigRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BCastSigRequest) String() string { @@ -47,7 +45,7 @@ func (*BCastSigRequest) ProtoMessage() {} func (x *BCastSigRequest) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -87,11 +85,9 @@ type BCastSigResponse struct { func (x *BCastSigResponse) Reset() { *x = BCastSigResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BCastSigResponse) String() string { @@ -102,7 +98,7 @@ func (*BCastSigResponse) ProtoMessage() {} func (x *BCastSigResponse) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -143,11 +139,9 @@ type BCastMessage struct { func (x *BCastMessage) Reset() { *x = BCastMessage{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BCastMessage) String() string { @@ -158,7 +152,7 @@ func (*BCastMessage) ProtoMessage() {} func (x *BCastMessage) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_bcast_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -258,44 +252,6 @@ func file_dkg_dkgpb_v1_bcast_proto_init() { if File_dkg_dkgpb_v1_bcast_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_dkg_dkgpb_v1_bcast_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*BCastSigRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_bcast_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*BCastSigResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_bcast_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*BCastMessage); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/dkg/dkgpb/v1/frost.pb.go b/dkg/dkgpb/v1/frost.pb.go index 84e3f2028..140bd6709 100644 --- a/dkg/dkgpb/v1/frost.pb.go +++ b/dkg/dkgpb/v1/frost.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: dkg/dkgpb/v1/frost.proto @@ -32,11 +32,9 @@ type FrostMsgKey struct { func (x *FrostMsgKey) Reset() { *x = FrostMsgKey{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FrostMsgKey) String() string { @@ -47,7 +45,7 @@ func (*FrostMsgKey) ProtoMessage() {} func (x *FrostMsgKey) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -93,11 +91,9 @@ type FrostRound1Casts struct { func (x *FrostRound1Casts) Reset() { *x = FrostRound1Casts{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FrostRound1Casts) String() string { @@ -108,7 +104,7 @@ func (*FrostRound1Casts) ProtoMessage() {} func (x *FrostRound1Casts) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -143,11 +139,9 @@ type FrostRound1Cast struct { func (x *FrostRound1Cast) Reset() { *x = FrostRound1Cast{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FrostRound1Cast) String() string { @@ -158,7 +152,7 @@ func (*FrostRound1Cast) ProtoMessage() {} func (x *FrostRound1Cast) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -211,11 +205,9 @@ type FrostRound1P2P struct { func (x *FrostRound1P2P) Reset() { *x = FrostRound1P2P{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FrostRound1P2P) String() string { @@ -226,7 +218,7 @@ func (*FrostRound1P2P) ProtoMessage() {} func (x *FrostRound1P2P) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -260,11 +252,9 @@ type FrostRound1ShamirShare struct { func (x *FrostRound1ShamirShare) Reset() { *x = FrostRound1ShamirShare{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FrostRound1ShamirShare) String() string { @@ -275,7 +265,7 @@ func (*FrostRound1ShamirShare) ProtoMessage() {} func (x *FrostRound1ShamirShare) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -321,11 +311,9 @@ type FrostRound2Casts struct { func (x *FrostRound2Casts) Reset() { *x = FrostRound2Casts{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FrostRound2Casts) String() string { @@ -336,7 +324,7 @@ func (*FrostRound2Casts) ProtoMessage() {} func (x *FrostRound2Casts) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -370,11 +358,9 @@ type FrostRound2Cast struct { func (x *FrostRound2Cast) Reset() { *x = FrostRound2Cast{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FrostRound2Cast) String() string { @@ -385,7 +371,7 @@ func (*FrostRound2Cast) ProtoMessage() {} func (x *FrostRound2Cast) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_frost_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -517,92 +503,6 @@ func file_dkg_dkgpb_v1_frost_proto_init() { if File_dkg_dkgpb_v1_frost_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_dkg_dkgpb_v1_frost_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*FrostMsgKey); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_frost_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*FrostRound1Casts); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_frost_proto_msgTypes[2].Exporter = func(v any, i int) any { - switch v := v.(*FrostRound1Cast); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_frost_proto_msgTypes[3].Exporter = func(v any, i int) any { - switch v := v.(*FrostRound1P2P); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_frost_proto_msgTypes[4].Exporter = func(v any, i int) any { - switch v := v.(*FrostRound1ShamirShare); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_frost_proto_msgTypes[5].Exporter = func(v any, i int) any { - switch v := v.(*FrostRound2Casts); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_frost_proto_msgTypes[6].Exporter = func(v any, i int) any { - switch v := v.(*FrostRound2Cast); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/dkg/dkgpb/v1/nodesigs.pb.go b/dkg/dkgpb/v1/nodesigs.pb.go index c76d912c8..0ca2721d8 100644 --- a/dkg/dkgpb/v1/nodesigs.pb.go +++ b/dkg/dkgpb/v1/nodesigs.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: dkg/dkgpb/v1/nodesigs.proto @@ -31,11 +31,9 @@ type MsgNodeSig struct { func (x *MsgNodeSig) Reset() { *x = MsgNodeSig{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_nodesigs_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_nodesigs_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MsgNodeSig) String() string { @@ -46,7 +44,7 @@ func (*MsgNodeSig) ProtoMessage() {} func (x *MsgNodeSig) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_nodesigs_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -120,20 +118,6 @@ func file_dkg_dkgpb_v1_nodesigs_proto_init() { if File_dkg_dkgpb_v1_nodesigs_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_dkg_dkgpb_v1_nodesigs_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*MsgNodeSig); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/dkg/dkgpb/v1/sync.pb.go b/dkg/dkgpb/v1/sync.pb.go index 53d1a1e2f..c9f62afb6 100644 --- a/dkg/dkgpb/v1/sync.pb.go +++ b/dkg/dkgpb/v1/sync.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.34.2 +// protoc-gen-go v1.35.1 // protoc (unknown) // source: dkg/dkgpb/v1/sync.proto @@ -35,11 +35,9 @@ type MsgSync struct { func (x *MsgSync) Reset() { *x = MsgSync{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_sync_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_sync_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MsgSync) String() string { @@ -50,7 +48,7 @@ func (*MsgSync) ProtoMessage() {} func (x *MsgSync) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_sync_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -111,11 +109,9 @@ type MsgSyncResponse struct { func (x *MsgSyncResponse) Reset() { *x = MsgSyncResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_dkg_dkgpb_v1_sync_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_dkg_dkgpb_v1_sync_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MsgSyncResponse) String() string { @@ -126,7 +122,7 @@ func (*MsgSyncResponse) ProtoMessage() {} func (x *MsgSyncResponse) ProtoReflect() protoreflect.Message { mi := &file_dkg_dkgpb_v1_sync_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -220,32 +216,6 @@ func file_dkg_dkgpb_v1_sync_proto_init() { if File_dkg_dkgpb_v1_sync_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_dkg_dkgpb_v1_sync_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*MsgSync); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_dkg_dkgpb_v1_sync_proto_msgTypes[1].Exporter = func(v any, i int) any { - switch v := v.(*MsgSyncResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ From 33a9b907fd354aae51e9433bccb75fa87a713d5c Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Thu, 10 Oct 2024 12:45:54 +0300 Subject: [PATCH 05/20] Consensus package refactoring --- app/app.go | 8 +- cmd/createcluster.go | 6 +- cmd/createdkg.go | 4 +- cmd/version.go | 4 +- cmd/version_internal_test.go | 4 +- core/consensus/factory.go | 6 +- core/consensus/factory_test.go | 16 +- core/consensus/instance_io.go | 81 ---------- core/consensus/{ => metrics}/metrics.go | 10 +- core/consensus/{ => protocols}/protocols.go | 2 +- .../{ => protocols}/protocols_test.go | 12 +- core/consensus/{qbft_msg.go => qbft/msg.go} | 88 +++++++---- .../msg_test.go} | 52 +++---- core/consensus/{ => qbft}/qbft.go | 142 ++++++++++-------- .../{ => qbft}/qbft_internal_test.go | 71 ++++----- core/consensus/{ => qbft}/qbft_test.go | 10 +- .../{ => qbft}/strategysim_internal_test.go | 41 ++--- .../TestDebugRoundChange_empty-1.golden | 0 .../TestDebugRoundChange_empty-2.golden | 0 .../TestDebugRoundChange_quorum.golden | 0 .../{ => qbft}/testdata/TestHashProto.golden | 0 .../{qbft_transport.go => qbft/transport.go} | 97 +++++++----- core/consensus/sniffed_internal_test.go | 12 +- core/consensus/utils/instance_io.go | 81 ++++++++++ .../instance_io_test.go} | 11 +- core/consensus/{ => utils}/roundtimer.go | 74 +++++---- .../roundtimer_test.go} | 25 ++- core/consensus/{ => utils}/sniffer.go | 16 +- 28 files changed, 476 insertions(+), 397 deletions(-) delete mode 100644 core/consensus/instance_io.go rename core/consensus/{ => metrics}/metrics.go (79%) rename core/consensus/{ => protocols}/protocols.go (97%) rename core/consensus/{ => protocols}/protocols_test.go (55%) rename core/consensus/{qbft_msg.go => qbft/msg.go} (63%) rename core/consensus/{qbft_msg_internal_test.go => qbft/msg_test.go} (65%) rename core/consensus/{ => qbft}/qbft.go (80%) rename core/consensus/{ => qbft}/qbft_internal_test.go (89%) rename core/consensus/{ => qbft}/qbft_test.go (93%) rename core/consensus/{ => qbft}/strategysim_internal_test.go (96%) rename core/consensus/{ => qbft}/testdata/TestDebugRoundChange_empty-1.golden (100%) rename core/consensus/{ => qbft}/testdata/TestDebugRoundChange_empty-2.golden (100%) rename core/consensus/{ => qbft}/testdata/TestDebugRoundChange_quorum.golden (100%) rename core/consensus/{ => qbft}/testdata/TestHashProto.golden (100%) rename core/consensus/{qbft_transport.go => qbft/transport.go} (58%) create mode 100644 core/consensus/utils/instance_io.go rename core/consensus/{instance_io_internal_test.go => utils/instance_io_test.go} (75%) rename core/consensus/{ => utils}/roundtimer.go (63%) rename core/consensus/{roundtimer_internal_test.go => utils/roundtimer_test.go} (74%) rename core/consensus/{ => utils}/sniffer.go (74%) diff --git a/app/app.go b/app/app.go index 74f2a25ff..4bed7363a 100644 --- a/app/app.go +++ b/app/app.go @@ -47,6 +47,8 @@ import ( "github.com/obolnetwork/charon/core/aggsigdb" "github.com/obolnetwork/charon/core/bcast" "github.com/obolnetwork/charon/core/consensus" + cprotocols "github.com/obolnetwork/charon/core/consensus/protocols" + cqbft "github.com/obolnetwork/charon/core/consensus/qbft" "github.com/obolnetwork/charon/core/dutydb" "github.com/obolnetwork/charon/core/fetcher" "github.com/obolnetwork/charon/core/infosync" @@ -531,7 +533,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, // We always need QBFT consensus instance as it is used for priority protocol. // And for now it is used as the primary consensus protocol. - qbftConsensus, err := consensusFactory.New(consensus.QBFTv2ProtocolID) + qbftConsensus, err := consensusFactory.New(cprotocols.QBFTv2ProtocolID) if err != nil { return err } @@ -591,7 +593,7 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t peers []peer.ID, threshold int, sendFunc p2p.SendReceiveFunc, coreCons core.Consensus, sched core.Scheduler, p2pKey *k1.PrivateKey, deadlineFunc func(duty core.Duty) (time.Time, bool), ) error { - cons, ok := coreCons.(*consensus.QBFTConsensus) + cons, ok := coreCons.(*cqbft.Consensus) if !ok { // Priority protocol not supported for leader cast. return nil @@ -1066,7 +1068,7 @@ func (h httpServeHook) Call(context.Context) error { // Protocols returns the list of supported Protocols in order of precedence. func Protocols() []protocol.ID { var resp []protocol.ID - resp = append(resp, consensus.Protocols()...) + resp = append(resp, cprotocols.Protocols()...) resp = append(resp, parsigex.Protocols()...) resp = append(resp, peerinfo.Protocols()...) resp = append(resp, priority.Protocols()...) diff --git a/cmd/createcluster.go b/cmd/createcluster.go index 7ce564348..14229f206 100644 --- a/cmd/createcluster.go +++ b/cmd/createcluster.go @@ -34,7 +34,7 @@ import ( "github.com/obolnetwork/charon/app/z" "github.com/obolnetwork/charon/cluster" "github.com/obolnetwork/charon/core" - "github.com/obolnetwork/charon/core/consensus" + "github.com/obolnetwork/charon/core/consensus/protocols" "github.com/obolnetwork/charon/eth2util" "github.com/obolnetwork/charon/eth2util/deposit" "github.com/obolnetwork/charon/eth2util/enr" @@ -393,7 +393,7 @@ func validateCreateConfig(ctx context.Context, conf clusterConfig) error { return errors.New("number of operators is below minimum", z.Int("operators", conf.NumNodes), z.Int("min", minNodes)) } - if len(conf.ConsensusProtocol) > 0 && !consensus.IsSupportedProtocolName(conf.ConsensusProtocol) { + if len(conf.ConsensusProtocol) > 0 && !protocols.IsSupportedProtocolName(conf.ConsensusProtocol) { return errors.New("unsupported consensus protocol", z.Str("protocol", conf.ConsensusProtocol)) } @@ -960,7 +960,7 @@ func validateDef(ctx context.Context, insecureKeys bool, keymanagerAddrs []strin return errors.New("unsupported network", z.Str("network", network)) } - if len(def.ConsensusProtocol) > 0 && !consensus.IsSupportedProtocolName(def.ConsensusProtocol) { + if len(def.ConsensusProtocol) > 0 && !protocols.IsSupportedProtocolName(def.ConsensusProtocol) { return errors.New("unsupported consensus protocol", z.Str("protocol", def.ConsensusProtocol)) } diff --git a/cmd/createdkg.go b/cmd/createdkg.go index 2c10e2ab6..41923d594 100644 --- a/cmd/createdkg.go +++ b/cmd/createdkg.go @@ -16,7 +16,7 @@ import ( "github.com/obolnetwork/charon/app/version" "github.com/obolnetwork/charon/app/z" "github.com/obolnetwork/charon/cluster" - "github.com/obolnetwork/charon/core/consensus" + "github.com/obolnetwork/charon/core/consensus/protocols" "github.com/obolnetwork/charon/eth2util" "github.com/obolnetwork/charon/eth2util/deposit" "github.com/obolnetwork/charon/eth2util/enr" @@ -217,7 +217,7 @@ func validateDKGConfig(numOperators int, network string, depositAmounts []int, c } } - if len(consensusProtocol) > 0 && !consensus.IsSupportedProtocolName(consensusProtocol) { + if len(consensusProtocol) > 0 && !protocols.IsSupportedProtocolName(consensusProtocol) { return errors.New("unsupported consensus protocol", z.Str("protocol", consensusProtocol)) } diff --git a/cmd/version.go b/cmd/version.go index 11f0c9768..c96fb38e6 100644 --- a/cmd/version.go +++ b/cmd/version.go @@ -11,7 +11,7 @@ import ( "github.com/spf13/pflag" "github.com/obolnetwork/charon/app/version" - "github.com/obolnetwork/charon/core/consensus" + "github.com/obolnetwork/charon/core/consensus/protocols" ) type versionConfig struct { @@ -66,7 +66,7 @@ func runVersionCmd(out io.Writer, config versionConfig) { _, _ = fmt.Fprint(out, "Consensus protocols:\n") - for _, protocol := range consensus.Protocols() { + for _, protocol := range protocols.Protocols() { _, _ = fmt.Fprintf(out, "\t%v\n", protocol) } } diff --git a/cmd/version_internal_test.go b/cmd/version_internal_test.go index f0139e537..66a038cc4 100644 --- a/cmd/version_internal_test.go +++ b/cmd/version_internal_test.go @@ -10,7 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/obolnetwork/charon/app/version" - "github.com/obolnetwork/charon/core/consensus" + "github.com/obolnetwork/charon/core/consensus/protocols" ) func TestRunVersionCmd(t *testing.T) { @@ -44,6 +44,6 @@ func TestRunVersionCmd(t *testing.T) { require.Contains(t, str, "Package:") require.Contains(t, str, "Dependencies:") require.Contains(t, str, "Consensus protocols:") - require.Contains(t, str, consensus.Protocols()[0]) + require.Contains(t, str, protocols.Protocols()[0]) }) } diff --git a/core/consensus/factory.go b/core/consensus/factory.go index c8ac81d18..7cf58990a 100644 --- a/core/consensus/factory.go +++ b/core/consensus/factory.go @@ -9,6 +9,8 @@ import ( "github.com/obolnetwork/charon/app/errors" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/protocols" + "github.com/obolnetwork/charon/core/consensus/qbft" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/p2p" ) @@ -41,8 +43,8 @@ func NewConsensusFactory(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer // New creates a new consensus instance. func (f *consensusFactory) New(protocol protocol.ID) (core.Consensus, error) { // TODO: Refactor to a switch statement when more protocols are added. - if protocol == QBFTv2ProtocolID { - return NewQBFTConsensus(f.tcpNode, f.sender, f.peers, f.p2pKey, f.deadliner, f.gaterFunc, f.snifferFunc) + if protocol == protocols.QBFTv2ProtocolID { + return qbft.NewConsensus(f.tcpNode, f.sender, f.peers, f.p2pKey, f.deadliner, f.gaterFunc, f.snifferFunc) } return nil, errors.New("unknown consensus protocol") diff --git a/core/consensus/factory_test.go b/core/consensus/factory_test.go index 9f783385e..262983e93 100644 --- a/core/consensus/factory_test.go +++ b/core/consensus/factory_test.go @@ -16,6 +16,7 @@ import ( "github.com/obolnetwork/charon/cluster" "github.com/obolnetwork/charon/core" "github.com/obolnetwork/charon/core/consensus" + "github.com/obolnetwork/charon/core/consensus/protocols" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/eth2util/enr" "github.com/obolnetwork/charon/p2p" @@ -56,7 +57,7 @@ func TestNewConsensusFactory(t *testing.T) { factory := consensus.NewConsensusFactory(hosts[0], new(p2p.Sender), peers, p2pkeys[0], testDeadliner{}, gaterFunc, snifferFunc) require.NotNil(t, factory) - cons, err := factory.New(consensus.QBFTv2ProtocolID) + cons, err := factory.New(protocols.QBFTv2ProtocolID) require.NoError(t, err) require.NotNil(t, cons) @@ -65,3 +66,16 @@ func TestNewConsensusFactory(t *testing.T) { require.Error(t, err) }) } + +// testDeadliner is a mock deadliner implementation. +type testDeadliner struct { + deadlineChan chan core.Duty +} + +func (testDeadliner) Add(core.Duty) bool { + return true +} + +func (t testDeadliner) C() <-chan core.Duty { + return t.deadlineChan +} diff --git a/core/consensus/instance_io.go b/core/consensus/instance_io.go deleted file mode 100644 index 9caaed023..000000000 --- a/core/consensus/instance_io.go +++ /dev/null @@ -1,81 +0,0 @@ -// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 - -package consensus - -import ( - "time" - - "google.golang.org/protobuf/proto" - - "github.com/obolnetwork/charon/app/errors" -) - -const ( - recvBuffer = 100 // Allow buffering some initial messages when this node is late to start an instance. -) - -// newInstanceIO returns a new instanceIO. -func newInstanceIO[T any]() instanceIO[T] { - return instanceIO[T]{ - participated: make(chan struct{}), - proposed: make(chan struct{}), - running: make(chan struct{}), - recvBuffer: make(chan T, recvBuffer), - hashCh: make(chan [32]byte, 1), - valueCh: make(chan proto.Message, 1), - errCh: make(chan error, 1), - decidedAtCh: make(chan time.Time, 1), - } -} - -// instanceIO defines the async input and output channels of a -// single consensus instance in the Component. -type instanceIO[T any] struct { - participated chan struct{} // Closed when Participate was called for this instance. - proposed chan struct{} // Closed when Propose was called for this instance. - running chan struct{} // Closed when runInstance was already called. - recvBuffer chan T // Outer receive buffers. - hashCh chan [32]byte // Async input hash channel. - valueCh chan proto.Message // Async input value channel. - errCh chan error // Async output error channel. - decidedAtCh chan time.Time // Async output decided timestamp channel. -} - -// MarkParticipated marks the instance as participated. -// It returns an error if the instance was already marked as participated. -func (io instanceIO[T]) MarkParticipated() error { - select { - case <-io.participated: - return errors.New("already participated") - default: - close(io.participated) - } - - return nil -} - -// MarkProposed marks the instance as proposed. -// It returns an error if the instance was already marked as proposed. -func (io instanceIO[T]) MarkProposed() error { - select { - case <-io.proposed: - return errors.New("already proposed") - default: - close(io.proposed) - } - - return nil -} - -// MaybeStart returns true if the instance wasn't running and has been started by this call, -// otherwise it returns false if the instance was started in the past and is either running now or has completed. -func (io instanceIO[T]) MaybeStart() bool { - select { - case <-io.running: - return false - default: - close(io.running) - } - - return true -} diff --git a/core/consensus/metrics.go b/core/consensus/metrics/metrics.go similarity index 79% rename from core/consensus/metrics.go rename to core/consensus/metrics/metrics.go index 0c9b98648..becbe25a4 100644 --- a/core/consensus/metrics.go +++ b/core/consensus/metrics/metrics.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package metrics import ( "github.com/prometheus/client_golang/prometheus" @@ -9,14 +9,14 @@ import ( ) var ( - decidedRoundsGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ + DecidedRoundsGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: "core", Subsystem: "consensus", Name: "decided_rounds", Help: "Number of rounds it took to decide consensus instances by duty and timer type.", }, []string{"duty", "timer"}) // Using gauge since the value changes slowly, once per slot. - consensusDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ + ConsensusDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "core", Subsystem: "consensus", Name: "duration_seconds", @@ -24,14 +24,14 @@ var ( Buckets: []float64{.05, .1, .25, .5, 1, 2.5, 5, 10, 20, 30, 60}, }, []string{"duty", "timer"}) - consensusTimeout = promauto.NewCounterVec(prometheus.CounterOpts{ + ConsensusTimeout = promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: "core", Subsystem: "consensus", Name: "timeout_total", Help: "Total count of consensus timeouts by duty and timer type.", }, []string{"duty", "timer"}) - consensusError = promauto.NewCounter(prometheus.CounterOpts{ + ConsensusError = promauto.NewCounter(prometheus.CounterOpts{ Namespace: "core", Subsystem: "consensus", Name: "error_total", diff --git a/core/consensus/protocols.go b/core/consensus/protocols/protocols.go similarity index 97% rename from core/consensus/protocols.go rename to core/consensus/protocols/protocols.go index 4268fc1a0..8a8fb807d 100644 --- a/core/consensus/protocols.go +++ b/core/consensus/protocols/protocols.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package protocols import ( "strings" diff --git a/core/consensus/protocols_test.go b/core/consensus/protocols/protocols_test.go similarity index 55% rename from core/consensus/protocols_test.go rename to core/consensus/protocols/protocols_test.go index 434c31707..fa6ce4994 100644 --- a/core/consensus/protocols_test.go +++ b/core/consensus/protocols/protocols_test.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus_test +package protocols_test import ( "testing" @@ -8,16 +8,16 @@ import ( "github.com/libp2p/go-libp2p/core/protocol" "github.com/stretchr/testify/require" - "github.com/obolnetwork/charon/core/consensus" + "github.com/obolnetwork/charon/core/consensus/protocols" ) func TestIsSupportedProtocolName(t *testing.T) { - require.True(t, consensus.IsSupportedProtocolName("qbft")) - require.False(t, consensus.IsSupportedProtocolName("unreal")) + require.True(t, protocols.IsSupportedProtocolName("qbft")) + require.False(t, protocols.IsSupportedProtocolName("unreal")) } func TestProtocols(t *testing.T) { require.Equal(t, []protocol.ID{ - consensus.QBFTv2ProtocolID, - }, consensus.Protocols()) + protocols.QBFTv2ProtocolID, + }, protocols.Protocols()) } diff --git a/core/consensus/qbft_msg.go b/core/consensus/qbft/msg.go similarity index 63% rename from core/consensus/qbft_msg.go rename to core/consensus/qbft/msg.go index a3882e92f..4f7422aa2 100644 --- a/core/consensus/qbft_msg.go +++ b/core/consensus/qbft/msg.go @@ -1,10 +1,13 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package qbft import ( + "testing" + k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" ssz "github.com/ferranbt/fastssz" + "golang.org/x/exp/rand" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" @@ -15,10 +18,29 @@ import ( "github.com/obolnetwork/charon/core/qbft" ) -// newQBFTMsg returns a new QBFT msg. -func newQBFTMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]byte]*anypb.Any) (qbftMsg, error) { +// NewRandomMsgForT returns a random qbft message. +func NewRandomMsgForT(t *testing.T) *pbv1.QBFTMsg { + t.Helper() + + msgType := 1 + rand.Int63n(int64(qbft.MsgDecided)) + if msgType == 0 { + msgType = 1 + } + + return &pbv1.QBFTMsg{ + Type: msgType, + Duty: core.DutyToProto(core.Duty{Type: core.DutyType(rand.Int()), Slot: rand.Uint64()}), + PeerIdx: rand.Int63(), + Round: rand.Int63(), + PreparedRound: rand.Int63(), + Signature: nil, + } +} + +// NewMsg returns a new QBFT Msg. +func NewMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]byte]*anypb.Any) (Msg, error) { if pbMsg == nil { - return qbftMsg{}, errors.New("nil qbft message") + return Msg{}, errors.New("nil qbft message") } // Do all possible error conversions first. @@ -30,28 +52,28 @@ func newQBFTMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[ if hash, ok := toHash32(pbMsg.GetValueHash()); ok { valueHash = hash if _, ok := values[valueHash]; !ok { - return qbftMsg{}, errors.New("value hash not found in values") + return Msg{}, errors.New("value hash not found in values") } } if hash, ok := toHash32(pbMsg.GetPreparedValueHash()); ok { preparedValueHash = hash if _, ok := values[preparedValueHash]; !ok { - return qbftMsg{}, errors.New("prepared value hash not found in values") + return Msg{}, errors.New("prepared value hash not found in values") } } var justImpls []qbft.Msg[core.Duty, [32]byte] for _, j := range justification { - impl, err := newQBFTMsg(j, nil, values) + impl, err := NewMsg(j, nil, values) if err != nil { - return qbftMsg{}, err + return Msg{}, err } justImpls = append(justImpls, impl) } - return qbftMsg{ + return Msg{ msg: pbMsg, valueHash: valueHash, values: values, @@ -61,8 +83,8 @@ func newQBFTMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[ }, nil } -// qbftMsg wraps *pbv1.QBFTMsg and justifications and implements qbft.Msg[core.Duty, [32]byte]. -type qbftMsg struct { +// Msg wraps *pbv1.QBFTMsg and justifications and implements qbft.Msg[core.Duty, [32]byte]. +type Msg struct { msg *pbv1.QBFTMsg valueHash [32]byte preparedValueHash [32]byte @@ -72,39 +94,47 @@ type qbftMsg struct { justification []qbft.Msg[core.Duty, [32]byte] } -func (m qbftMsg) Type() qbft.MsgType { +func (m Msg) Type() qbft.MsgType { return qbft.MsgType(m.msg.GetType()) } -func (m qbftMsg) Instance() core.Duty { +func (m Msg) Instance() core.Duty { return core.DutyFromProto(m.msg.GetDuty()) } -func (m qbftMsg) Source() int64 { +func (m Msg) Source() int64 { return m.msg.GetPeerIdx() } -func (m qbftMsg) Round() int64 { +func (m Msg) Round() int64 { return m.msg.GetRound() } -func (m qbftMsg) Value() [32]byte { +func (m Msg) Value() [32]byte { return m.valueHash } -func (m qbftMsg) PreparedRound() int64 { +func (m Msg) Values() map[[32]byte]*anypb.Any { + return m.values +} + +func (m Msg) Msg() *pbv1.QBFTMsg { + return m.msg +} + +func (m Msg) PreparedRound() int64 { return m.msg.GetPreparedRound() } -func (m qbftMsg) PreparedValue() [32]byte { +func (m Msg) PreparedValue() [32]byte { return m.preparedValueHash } -func (m qbftMsg) Justification() []qbft.Msg[core.Duty, [32]byte] { +func (m Msg) Justification() []qbft.Msg[core.Duty, [32]byte] { return m.justification } -func (m qbftMsg) ToConsensusMsg() *pbv1.ConsensusMsg { +func (m Msg) ToConsensusMsg() *pbv1.ConsensusMsg { var values []*anypb.Any for _, v := range m.values { values = append(values, v) @@ -117,9 +147,9 @@ func (m qbftMsg) ToConsensusMsg() *pbv1.ConsensusMsg { } } -// hashProto returns a deterministic ssz hash root of the proto message. +// HashProto returns a deterministic ssz hash root of the proto message. // It is the same logic as that used by the priority package. -func hashProto(msg proto.Message) ([32]byte, error) { +func HashProto(msg proto.Message) ([32]byte, error) { if _, ok := msg.(*anypb.Any); ok { return [32]byte{}, errors.New("cannot hash any proto, must hash inner value") } @@ -146,8 +176,8 @@ func hashProto(msg proto.Message) ([32]byte, error) { return hash, nil } -// verifyQBFTMsgSig returns true if the message was signed by pubkey. -func verifyQBFTMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { +// VerifyMsgSig returns true if the message was signed by pubkey. +func VerifyMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { if msg.Signature == nil { return false, errors.New("empty signature") } @@ -157,7 +187,7 @@ func verifyQBFTMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { return false, errors.New("type assert qbft msg") } clone.Signature = nil - hash, err := hashProto(clone) + hash, err := HashProto(clone) if err != nil { return false, err } @@ -170,15 +200,15 @@ func verifyQBFTMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { return recovered.IsEqual(pubkey), nil } -// signQBFTMsg returns a copy of the proto message with a populated signature signed by the provided private key. -func signQBFTMsg(msg *pbv1.QBFTMsg, privkey *k1.PrivateKey) (*pbv1.QBFTMsg, error) { +// SignMsg returns a copy of the proto message with a populated signature signed by the provided private key. +func SignMsg(msg *pbv1.QBFTMsg, privkey *k1.PrivateKey) (*pbv1.QBFTMsg, error) { clone, ok := proto.Clone(msg).(*pbv1.QBFTMsg) if !ok { return nil, errors.New("type assert qbft msg") } clone.Signature = nil - hash, err := hashProto(clone) + hash, err := HashProto(clone) if err != nil { return nil, err } @@ -205,4 +235,4 @@ func toHash32(val []byte) ([32]byte, bool) { return resp, true } -var _ qbft.Msg[core.Duty, [32]byte] = qbftMsg{} // Interface assertion +var _ qbft.Msg[core.Duty, [32]byte] = Msg{} // Interface assertion diff --git a/core/consensus/qbft_msg_internal_test.go b/core/consensus/qbft/msg_test.go similarity index 65% rename from core/consensus/qbft_msg_internal_test.go rename to core/consensus/qbft/msg_test.go index 7ca58a814..157215f57 100644 --- a/core/consensus/qbft_msg_internal_test.go +++ b/core/consensus/qbft/msg_test.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package qbft_test import ( "encoding/hex" @@ -14,8 +14,9 @@ import ( "google.golang.org/protobuf/types/known/timestamppb" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/qbft" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" - "github.com/obolnetwork/charon/core/qbft" + coreqbft "github.com/obolnetwork/charon/core/qbft" "github.com/obolnetwork/charon/testutil" ) @@ -28,7 +29,7 @@ func TestHashProto(t *testing.T) { setPB, err := core.UnsignedDataSetToProto(set) require.NoError(t, err) - hash, err := hashProto(setPB) + hash, err := qbft.HashProto(setPB) require.NoError(t, err) require.Equal(t, @@ -43,18 +44,18 @@ func TestSigning(t *testing.T) { privkey, err := k1.GeneratePrivateKey() require.NoError(t, err) - msg := randomMsg(t) + msg := qbft.NewRandomMsgForT(t) - signed, err := signQBFTMsg(msg, privkey) + signed, err := qbft.SignMsg(msg, privkey) require.NoError(t, err) - ok, err := verifyQBFTMsgSig(signed, privkey.PubKey()) + ok, err := qbft.VerifyMsgSig(signed, privkey.PubKey()) require.NoError(t, err) require.True(t, ok) privkey2, err := k1.GeneratePrivateKey() require.NoError(t, err) - ok, err = verifyQBFTMsgSig(signed, privkey2.PubKey()) + ok, err = qbft.VerifyMsgSig(signed, privkey2.PubKey()) require.NoError(t, err) require.False(t, ok) } @@ -62,9 +63,9 @@ func TestSigning(t *testing.T) { func TestNewMsg(t *testing.T) { val1 := timestamppb.New(time.Time{}) val2 := timestamppb.New(time.Now()) - hash1, err := hashProto(val1) + hash1, err := qbft.HashProto(val1) require.NoError(t, err) - hash2, err := hashProto(val2) + hash2, err := qbft.HashProto(val2) require.NoError(t, err) any1, err := anypb.New(val1) @@ -77,8 +78,8 @@ func TestNewMsg(t *testing.T) { hash2: any2, } - msg, err := newQBFTMsg(&pbv1.QBFTMsg{ - Type: int64(qbft.MsgPrePrepare), + msg, err := qbft.NewMsg(&pbv1.QBFTMsg{ + Type: int64(coreqbft.MsgPrePrepare), ValueHash: hash1[:], PreparedValueHash: hash2[:], }, nil, values) @@ -86,40 +87,21 @@ func TestNewMsg(t *testing.T) { require.Equal(t, msg.Value(), hash1) require.Equal(t, msg.PreparedValue(), hash2) - require.EqualValues(t, msg.values, values) + require.EqualValues(t, msg.Values(), values) } func TestPartialLegacyNewMsg(t *testing.T) { val1 := timestamppb.New(time.Time{}) - hash1, err := hashProto(val1) + hash1, err := qbft.HashProto(val1) require.NoError(t, err) - _, err = newQBFTMsg(&pbv1.QBFTMsg{ - Type: int64(qbft.MsgPrePrepare), + _, err = qbft.NewMsg(&pbv1.QBFTMsg{ + Type: int64(coreqbft.MsgPrePrepare), }, []*pbv1.QBFTMsg{ { - Type: int64(qbft.MsgPrePrepare), + Type: int64(coreqbft.MsgPrePrepare), ValueHash: hash1[:], }, }, make(map[[32]byte]*anypb.Any)) require.ErrorContains(t, err, "value hash not found in values") } - -// randomMsg returns a random qbft message. -func randomMsg(t *testing.T) *pbv1.QBFTMsg { - t.Helper() - - msgType := 1 + rand.Int63n(int64(qbft.MsgDecided)) - if msgType == 0 { - msgType = 1 - } - - return &pbv1.QBFTMsg{ - Type: msgType, - Duty: core.DutyToProto(core.Duty{Type: core.DutyType(rand.Int()), Slot: rand.Uint64()}), - PeerIdx: rand.Int63(), - Round: rand.Int63(), - PreparedRound: rand.Int63(), - Signature: nil, - } -} diff --git a/core/consensus/qbft.go b/core/consensus/qbft/qbft.go similarity index 80% rename from core/consensus/qbft.go rename to core/consensus/qbft/qbft.go index 48afbe68d..594013f7e 100644 --- a/core/consensus/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package qbft import ( "context" @@ -21,15 +21,18 @@ import ( "github.com/obolnetwork/charon/app/log" "github.com/obolnetwork/charon/app/z" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/metrics" + "github.com/obolnetwork/charon/core/consensus/protocols" + "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/qbft" "github.com/obolnetwork/charon/p2p" ) -type subscriber func(ctx context.Context, duty core.Duty, value proto.Message) error +type Subscriber func(ctx context.Context, duty core.Duty, value proto.Message) error -// newQBFTDefinition returns a qbft definition (this is constant across all consensus instances). -func newQBFTDefinition(nodes int, subs func() []subscriber, roundTimer roundTimer, +// NewDefinition returns a qbft definition (this is constant across all consensus instances). +func NewDefinition(nodes int, subs func() []Subscriber, roundTimer utils.RoundTimer, decideCallback func(qcommit []qbft.Msg[core.Duty, [32]byte]), ) qbft.Definition[core.Duty, [32]byte] { quorum := qbft.Definition[int, int]{Nodes: nodes}.Quorum() @@ -43,13 +46,13 @@ func newQBFTDefinition(nodes int, subs func() []subscriber, roundTimer roundTime // Decide sends consensus output to subscribers. Decide: func(ctx context.Context, duty core.Duty, _ [32]byte, qcommit []qbft.Msg[core.Duty, [32]byte]) { defer endCtxSpan(ctx) // End the parent tracing span when decided - msg, ok := qcommit[0].(qbftMsg) + msg, ok := qcommit[0].(Msg) if !ok { log.Error(ctx, "Invalid message type", nil) return } - anyValue, ok := msg.values[msg.valueHash] + anyValue, ok := msg.Values()[msg.Value()] if !ok { log.Error(ctx, "Invalid value hash", nil) return @@ -111,14 +114,14 @@ func newQBFTDefinition(nodes int, subs func() []subscriber, roundTimer roundTime Nodes: nodes, // FIFOLimit caps the max buffered messages per peer. - FIFOLimit: recvBuffer, + FIFOLimit: utils.RecvBufferSize, } } -// NewQBFTConsensus returns a new consensus QBFT component. -func NewQBFTConsensus(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, +// NewConsensus returns a new consensus QBFT component. +func NewConsensus(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, deadliner core.Deadliner, gaterFunc core.DutyGaterFunc, snifferFunc func(*pbv1.SniffedConsensusInstance), -) (*QBFTConsensus, error) { +) (*Consensus, error) { // Extract peer pubkeys. keys := make(map[int64]*k1.PublicKey) var labels []string @@ -133,7 +136,7 @@ func NewQBFTConsensus(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p keys[int64(i)] = pk } - c := &QBFTConsensus{ + c := &Consensus{ tcpNode: tcpNode, sender: sender, peers: peers, @@ -144,15 +147,15 @@ func NewQBFTConsensus(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p snifferFunc: snifferFunc, gaterFunc: gaterFunc, dropFilter: log.Filter(), - timerFunc: getTimerFunc(), + timerFunc: utils.GetTimerFunc(), } - c.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) + c.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) return c, nil } -// QBFTConsensus implements core.Consensus & priority.coreConsensus. -type QBFTConsensus struct { +// Consensus implements core.Consensus & priority.coreConsensus. +type Consensus struct { // Immutable state tcpNode host.Host sender *p2p.Sender @@ -160,23 +163,23 @@ type QBFTConsensus struct { peers []p2p.Peer pubkeys map[int64]*k1.PublicKey privkey *k1.PrivateKey - subs []subscriber + subs []Subscriber deadliner core.Deadliner snifferFunc func(*pbv1.SniffedConsensusInstance) gaterFunc core.DutyGaterFunc dropFilter z.Field // Filter buffer overflow errors (possible DDoS) - timerFunc timerFunc + timerFunc utils.TimerFunc // Mutable state mutable struct { sync.Mutex - instances map[core.Duty]instanceIO[qbftMsg] + instances map[core.Duty]*utils.InstanceIO[Msg] } } // Subscribe registers a callback for unsigned duty data proposals from leaders. // Note this function is not thread safe, it should be called *before* Start and Propose. -func (c *QBFTConsensus) Subscribe(fn func(ctx context.Context, duty core.Duty, set core.UnsignedDataSet) error) { +func (c *Consensus) Subscribe(fn func(ctx context.Context, duty core.Duty, set core.UnsignedDataSet) error) { c.subs = append(c.subs, func(ctx context.Context, duty core.Duty, value proto.Message) error { unsignedPB, ok := value.(*pbv1.UnsignedDataSet) if !ok { @@ -193,13 +196,13 @@ func (c *QBFTConsensus) Subscribe(fn func(ctx context.Context, duty core.Duty, s } // subscribers returns the subscribers. -func (c *QBFTConsensus) subscribers() []subscriber { +func (c *Consensus) subscribers() []Subscriber { return c.subs } // SubscribePriority registers a callback for priority protocol message proposals from leaders. // Note this function is not thread safe, it should be called *before* Start and Propose. -func (c *QBFTConsensus) SubscribePriority(fn func(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error) { +func (c *Consensus) SubscribePriority(fn func(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error) { c.subs = append(c.subs, func(ctx context.Context, duty core.Duty, value proto.Message) error { msg, ok := value.(*pbv1.PriorityResult) if !ok { @@ -211,8 +214,8 @@ func (c *QBFTConsensus) SubscribePriority(fn func(ctx context.Context, duty core } // Start registers the libp2p receive handler and starts a goroutine that cleans state. This should only be called once. -func (c *QBFTConsensus) Start(ctx context.Context) { - p2p.RegisterHandler("qbft", c.tcpNode, QBFTv2ProtocolID, +func (c *Consensus) Start(ctx context.Context) { + p2p.RegisterHandler("qbft", c.tcpNode, protocols.QBFTv2ProtocolID, func() proto.Message { return new(pbv1.ConsensusMsg) }, c.handle) @@ -232,7 +235,7 @@ func (c *QBFTConsensus) Start(ctx context.Context) { // It either runs the consensus instance if it is not already running or // waits until it completes, in both cases it returns the resulting error. // Note this errors if called multiple times for the same duty. -func (c *QBFTConsensus) Propose(ctx context.Context, duty core.Duty, data core.UnsignedDataSet) error { +func (c *Consensus) Propose(ctx context.Context, duty core.Duty, data core.UnsignedDataSet) error { // Hash the proposed data, since qbft only supports simple comparable values. value, err := core.UnsignedDataSetToProto(data) if err != nil { @@ -246,7 +249,7 @@ func (c *QBFTConsensus) Propose(ctx context.Context, duty core.Duty, data core.U // It either runs the consensus instance if it is not already running or // waits until it completes, in both cases it returns the resulting error. // Note this errors if called multiple times for the same duty. -func (c *QBFTConsensus) ProposePriority(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error { +func (c *Consensus) ProposePriority(ctx context.Context, duty core.Duty, msg *pbv1.PriorityResult) error { return c.propose(ctx, duty, msg) } @@ -254,8 +257,8 @@ func (c *QBFTConsensus) ProposePriority(ctx context.Context, duty core.Duty, msg // It either runs the consensus instance if it is not already running or // waits until it completes, in both cases it returns the resulting error. // Note this errors if called multiple times for the same duty. -func (c *QBFTConsensus) propose(ctx context.Context, duty core.Duty, value proto.Message) error { - hash, err := hashProto(value) +func (c *Consensus) propose(ctx context.Context, duty core.Duty, value proto.Message) error { + hash, err := HashProto(value) if err != nil { return err } @@ -268,13 +271,13 @@ func (c *QBFTConsensus) propose(ctx context.Context, duty core.Duty, value proto // Provide proposal inputs to the instance. select { - case inst.valueCh <- value: + case inst.ValueCh <- value: default: return errors.New("input channel full") } select { - case inst.hashCh <- hash: + case inst.HashCh <- hash: default: return errors.New("input channel full") } @@ -283,16 +286,16 @@ func (c *QBFTConsensus) propose(ctx context.Context, duty core.Duty, value proto proposedAt := time.Now() defer func() { select { - case decidedAt := <-inst.decidedAtCh: + case decidedAt := <-inst.DecidedAtCh: timerType := c.timerFunc(duty).Type() duration := decidedAt.Sub(proposedAt) - consensusDuration.WithLabelValues(duty.Type.String(), string(timerType)).Observe(duration.Seconds()) + metrics.ConsensusDuration.WithLabelValues(duty.Type.String(), string(timerType)).Observe(duration.Seconds()) default: } }() if !inst.MaybeStart() { // Participate was already called, instance is running. - return <-inst.errCh + return <-inst.ErrCh } return c.runInstance(ctx, duty) @@ -302,7 +305,7 @@ func (c *QBFTConsensus) propose(ctx context.Context, duty core.Duty, value proto // unsigned data from beacon node and Propose not already called. // Note Propose must still be called for this peer to propose a value when leading a round. // Note this errors if called multiple times for the same duty. -func (c *QBFTConsensus) Participate(ctx context.Context, duty core.Duty) error { +func (c *Consensus) Participate(ctx context.Context, duty core.Duty) error { if duty.Type == core.DutyAggregator || duty.Type == core.DutySyncContribution { return nil // No consensus participate for potential no-op aggregation duties. } @@ -324,10 +327,26 @@ func (c *QBFTConsensus) Participate(ctx context.Context, duty core.Duty) error { return c.runInstance(ctx, duty) } +// Broadcast implements Broadcaster interface. +func (c *Consensus) Broadcast(ctx context.Context, msg *pbv1.ConsensusMsg) error { + for _, peer := range c.peers { + if peer.ID == c.tcpNode.ID() { + // Do not broadcast to self + continue + } + + if err := c.sender.SendAsync(ctx, c.tcpNode, protocols.QBFTv2ProtocolID, peer.ID, msg); err != nil { + return err + } + } + + return nil +} + // runInstance blocks and runs a consensus instance for the given duty. // It returns an error or nil when the context is cancelled. // Note each instance may only be run once. -func (c *QBFTConsensus) runInstance(ctx context.Context, duty core.Duty) (err error) { +func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) { roundTimer := c.timerFunc(duty) ctx = log.WithTopic(ctx, "qbft") ctx = log.WithCtx(ctx, z.Any("duty", duty)) @@ -341,7 +360,7 @@ func (c *QBFTConsensus) runInstance(ctx context.Context, duty core.Duty) (err er inst := c.getInstanceIO(duty) defer func() { - inst.errCh <- err // Send resulting error to errCh. + inst.ErrCh <- err // Send resulting error to errCh. }() if !c.deadliner.Add(duty) { @@ -358,25 +377,19 @@ func (c *QBFTConsensus) runInstance(ctx context.Context, duty core.Duty) (err er var decided bool decideCallback := func(qcommit []qbft.Msg[core.Duty, [32]byte]) { decided = true - decidedRoundsGauge.WithLabelValues(duty.Type.String(), string(roundTimer.Type())).Set(float64(qcommit[0].Round())) - inst.decidedAtCh <- time.Now() + metrics.DecidedRoundsGauge.WithLabelValues(duty.Type.String(), string(roundTimer.Type())).Set(float64(qcommit[0].Round())) + inst.DecidedAtCh <- time.Now() } // Create a new qbft definition for this instance. - def := newQBFTDefinition(len(c.peers), c.subscribers, roundTimer, decideCallback) + def := NewDefinition(len(c.peers), c.subscribers, roundTimer, decideCallback) // Create a new transport that handles sending and receiving for this instance. - t := qbftTransport{ - component: c, - values: make(map[[32]byte]*anypb.Any), - valueCh: inst.valueCh, - recvBuffer: make(chan qbft.Msg[core.Duty, [32]byte]), - sniffer: newSniffer(int64(def.Nodes), peerIdx), - } + t := NewTransport(c, c.privkey, inst.ValueCh, make(chan qbft.Msg[core.Duty, [32]byte]), utils.NewSniffer(int64(def.Nodes), peerIdx)) // Provide sniffed buffer to snifferFunc at the end. defer func() { - c.snifferFunc(t.sniffer.Instance()) + c.snifferFunc(t.SnifferInstance()) }() // Start a receiving goroutine. @@ -385,18 +398,18 @@ func (c *QBFTConsensus) runInstance(ctx context.Context, duty core.Duty) (err er // Create a qbft transport from the transport qt := qbft.Transport[core.Duty, [32]byte]{ Broadcast: t.Broadcast, - Receive: t.recvBuffer, + Receive: t.RecvBuffer(), } // Run the algo, blocking until the context is cancelled. - err = qbft.Run[core.Duty, [32]byte](ctx, def, qt, duty, peerIdx, inst.hashCh) + err = qbft.Run[core.Duty, [32]byte](ctx, def, qt, duty, peerIdx, inst.HashCh) if err != nil && !isContextErr(err) { - consensusError.Inc() + metrics.ConsensusError.Inc() return err // Only return non-context errors. } if !decided { - consensusTimeout.WithLabelValues(duty.Type.String(), string(roundTimer.Type())).Inc() + metrics.ConsensusTimeout.WithLabelValues(duty.Type.String(), string(roundTimer.Type())).Inc() return errors.New("consensus timeout", z.Str("duty", duty.String())) } @@ -405,7 +418,7 @@ func (c *QBFTConsensus) runInstance(ctx context.Context, duty core.Duty) (err er } // handle processes an incoming consensus wire message. -func (c *QBFTConsensus) handle(ctx context.Context, _ peer.ID, req proto.Message) (proto.Message, bool, error) { +func (c *Consensus) handle(ctx context.Context, _ peer.ID, req proto.Message) (proto.Message, bool, error) { t0 := time.Now() pbMsg, ok := req.(*pbv1.ConsensusMsg) @@ -439,12 +452,12 @@ func (c *QBFTConsensus) handle(ctx context.Context, _ peer.ID, req proto.Message } } - values, err := valuesByHash(pbMsg.GetValues()) + values, err := ValuesByHash(pbMsg.GetValues()) if err != nil { return nil, false, err } - msg, err := newQBFTMsg(pbMsg.GetMsg(), pbMsg.GetJustification(), values) + msg, err := NewMsg(pbMsg.GetMsg(), pbMsg.GetJustification(), values) if err != nil { return nil, false, err } @@ -470,27 +483,27 @@ func (c *QBFTConsensus) handle(ctx context.Context, _ peer.ID, req proto.Message } // getRecvBuffer returns a receive buffer for the duty. -func (c *QBFTConsensus) getRecvBuffer(duty core.Duty) chan qbftMsg { +func (c *Consensus) getRecvBuffer(duty core.Duty) chan Msg { c.mutable.Lock() defer c.mutable.Unlock() inst, ok := c.mutable.instances[duty] if !ok { - inst = newInstanceIO[qbftMsg]() + inst = utils.NewInstanceIO[Msg]() c.mutable.instances[duty] = inst } - return inst.recvBuffer + return inst.RecvBuffer } -// getInstanceIO returns the duty's instance and true if it were previously created. -func (c *QBFTConsensus) getInstanceIO(duty core.Duty) instanceIO[qbftMsg] { +// getInstanceIO returns the duty's instance if it were previously created. +func (c *Consensus) getInstanceIO(duty core.Duty) *utils.InstanceIO[Msg] { c.mutable.Lock() defer c.mutable.Unlock() inst, ok := c.mutable.instances[duty] if !ok { // Create new instanceIO. - inst = newInstanceIO[qbftMsg]() + inst = utils.NewInstanceIO[Msg]() c.mutable.instances[duty] = inst return inst @@ -500,7 +513,7 @@ func (c *QBFTConsensus) getInstanceIO(duty core.Duty) instanceIO[qbftMsg] { } // deleteInstanceIO deletes the instanceIO for the duty. -func (c *QBFTConsensus) deleteInstanceIO(duty core.Duty) { +func (c *Consensus) deleteInstanceIO(duty core.Duty) { c.mutable.Lock() defer c.mutable.Unlock() @@ -508,7 +521,7 @@ func (c *QBFTConsensus) deleteInstanceIO(duty core.Duty) { } // getPeerIdx returns the local peer index. -func (c *QBFTConsensus) getPeerIdx() (int64, error) { +func (c *Consensus) getPeerIdx() (int64, error) { peerIdx := int64(-1) for i, p := range c.peers { if c.tcpNode.ID() == p.ID { @@ -547,7 +560,7 @@ func verifyMsg(msg *pbv1.QBFTMsg, pubkeys map[int64]*k1.PublicKey) error { return errors.New("invalid peer index", z.I64("index", msg.GetPeerIdx())) } - if ok, err := verifyQBFTMsgSig(msg, msgPubkey); err != nil { + if ok, err := VerifyMsgSig(msg, msgPubkey); err != nil { return errors.Wrap(err, "verify consensus message signature") } else if !ok { return errors.New("invalid consensus message signature") @@ -671,7 +684,8 @@ func leader(duty core.Duty, round int64, nodes int) int64 { return (int64(duty.Slot) + int64(duty.Type) + round) % int64(nodes) } -func valuesByHash(values []*anypb.Any) (map[[32]byte]*anypb.Any, error) { +// ValuesByHash returns a map of values by hash. +func ValuesByHash(values []*anypb.Any) (map[[32]byte]*anypb.Any, error) { resp := make(map[[32]byte]*anypb.Any) for _, v := range values { inner, err := v.UnmarshalNew() @@ -679,7 +693,7 @@ func valuesByHash(values []*anypb.Any) (map[[32]byte]*anypb.Any, error) { return nil, errors.Wrap(err, "unmarshal any") } - hash, err := hashProto(inner) + hash, err := HashProto(inner) if err != nil { return nil, err } diff --git a/core/consensus/qbft_internal_test.go b/core/consensus/qbft/qbft_internal_test.go similarity index 89% rename from core/consensus/qbft_internal_test.go rename to core/consensus/qbft/qbft_internal_test.go index fb53c7752..d083b8110 100644 --- a/core/consensus/qbft_internal_test.go +++ b/core/consensus/qbft/qbft_internal_test.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package qbft import ( "bytes" @@ -13,6 +13,7 @@ import ( "github.com/obolnetwork/charon/app/k1util" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/qbft" "github.com/obolnetwork/charon/testutil" @@ -121,12 +122,12 @@ func (t testMsg) Justification() []qbft.Msg[core.Duty, [32]byte] { func TestQBFTConsensus_handle(t *testing.T) { tests := []struct { name string - mutate func(base *pbv1.ConsensusMsg, c *QBFTConsensus) + mutate func(base *pbv1.ConsensusMsg, c *Consensus) checkErr func(err error) }{ { "qbft message with no pubkey errors", - func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { + func(base *pbv1.ConsensusMsg, c *Consensus) { // construct a valid basis message signature base.Msg.Duty.Type = 1 base.Msg.Signature = bytes.Repeat([]byte{42}, 65) @@ -142,7 +143,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with justifications mentioning unknown peerIdx errors", - func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { + func(base *pbv1.ConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -155,7 +156,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := hashProto(base.GetMsg()) + msgHash, err := HashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -165,7 +166,7 @@ func TestQBFTConsensus_handle(t *testing.T) { // construct a justification base.Justification = []*pbv1.QBFTMsg{ - randomMsg(t), + NewRandomMsgForT(t), } base.Justification[0].PeerIdx = 42 @@ -175,7 +176,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the justification - justHash, err := hashProto(base.GetJustification()[0]) + justHash, err := HashProto(base.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(p2pKey, justHash[:]) @@ -189,7 +190,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with nil justification present in slice", - func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { + func(base *pbv1.ConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -202,7 +203,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := hashProto(base.GetMsg()) + msgHash, err := HashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -222,7 +223,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message values present but nil", - func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { + func(base *pbv1.ConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -241,7 +242,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := hashProto(base.GetMsg()) + msgHash, err := HashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -255,7 +256,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with invalid duty fails", - func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { + func(base *pbv1.ConsensusMsg, c *Consensus) { // construct a valid basis message signature base.Msg.Duty.Type = 1 base.Msg.Signature = bytes.Repeat([]byte{42}, 65) @@ -271,7 +272,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with valid duty fails because justification has different duty type", - func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { + func(base *pbv1.ConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -284,7 +285,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := hashProto(base.GetMsg()) + msgHash, err := HashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -294,7 +295,7 @@ func TestQBFTConsensus_handle(t *testing.T) { // construct a justification base.Justification = []*pbv1.QBFTMsg{ - randomMsg(t), + NewRandomMsgForT(t), } base.Justification[0].PeerIdx = 0 @@ -304,7 +305,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the justification - justHash, err := hashProto(base.GetJustification()[0]) + justHash, err := HashProto(base.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(p2pKey, justHash[:]) @@ -318,7 +319,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with valid duty and justification with same duty does not fail", - func(base *pbv1.ConsensusMsg, c *QBFTConsensus) { + func(base *pbv1.ConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -331,7 +332,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := hashProto(base.GetMsg()) + msgHash, err := HashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -341,7 +342,7 @@ func TestQBFTConsensus_handle(t *testing.T) { // construct a justification base.Justification = []*pbv1.QBFTMsg{ - randomMsg(t), + NewRandomMsgForT(t), } base.Justification[0].PeerIdx = 0 @@ -351,7 +352,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the justification - justHash, err := hashProto(base.GetJustification()[0]) + justHash, err := HashProto(base.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(p2pKey, justHash[:]) @@ -369,13 +370,13 @@ func TestQBFTConsensus_handle(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - var tc QBFTConsensus + var tc Consensus tc.deadliner = testDeadliner{} - tc.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) + tc.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) tc.gaterFunc = func(core.Duty) bool { return true } msg := &pbv1.ConsensusMsg{ - Msg: randomMsg(t), + Msg: NewRandomMsgForT(t), } test.mutate(msg, &tc) @@ -449,7 +450,7 @@ func TestQBFTConsensusHandle(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - c := &QBFTConsensus{ + c := &Consensus{ gaterFunc: func(core.Duty) bool { return true }, } @@ -461,16 +462,16 @@ func TestQBFTConsensusHandle(t *testing.T) { func TestInstanceIO_MaybeStart(t *testing.T) { t.Run("MaybeStart for new instance", func(t *testing.T) { - inst1 := newInstanceIO[qbftMsg]() + inst1 := utils.NewInstanceIO[Msg]() require.True(t, inst1.MaybeStart()) require.False(t, inst1.MaybeStart()) }) t.Run("MaybeStart after handle", func(t *testing.T) { - var c QBFTConsensus + var c Consensus c.deadliner = testDeadliner{} c.gaterFunc = func(core.Duty) bool { return true } - c.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) + c.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) // Generate a p2p private key. p2pKey := testutil.GenerateInsecureK1Key(t, 0) @@ -479,7 +480,7 @@ func TestInstanceIO_MaybeStart(t *testing.T) { duty := core.Duty{Slot: 42, Type: 1} msg := &pbv1.ConsensusMsg{ - Msg: randomMsg(t), + Msg: NewRandomMsgForT(t), } msg = signConsensusMsg(t, msg, p2pKey, duty) @@ -496,11 +497,11 @@ func TestInstanceIO_MaybeStart(t *testing.T) { t.Run("Call Propose after handle", func(t *testing.T) { ctx := context.Background() - var c QBFTConsensus + var c Consensus c.deadliner = testDeadliner{} c.gaterFunc = func(core.Duty) bool { return true } - c.mutable.instances = make(map[core.Duty]instanceIO[qbftMsg]) - c.timerFunc = getTimerFunc() + c.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) + c.timerFunc = utils.GetTimerFunc() // Generate a p2p private key pair. p2pKey := testutil.GenerateInsecureK1Key(t, 0) @@ -509,7 +510,7 @@ func TestInstanceIO_MaybeStart(t *testing.T) { duty := core.Duty{Slot: 42, Type: 1} msg := &pbv1.ConsensusMsg{ - Msg: randomMsg(t), + Msg: NewRandomMsgForT(t), } msg = signConsensusMsg(t, msg, p2pKey, duty) @@ -554,7 +555,7 @@ func signConsensusMsg(t *testing.T, msg *pbv1.ConsensusMsg, privKey *k1.PrivateK } // Sign the base message - msgHash, err := hashProto(msg.GetMsg()) + msgHash, err := HashProto(msg.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(privKey, msgHash[:]) @@ -564,7 +565,7 @@ func signConsensusMsg(t *testing.T, msg *pbv1.ConsensusMsg, privKey *k1.PrivateK // construct a justification msg.Justification = []*pbv1.QBFTMsg{ - randomMsg(t), + NewRandomMsgForT(t), } msg.Justification[0].PeerIdx = 0 @@ -574,7 +575,7 @@ func signConsensusMsg(t *testing.T, msg *pbv1.ConsensusMsg, privKey *k1.PrivateK } // Sign the justification - justHash, err := hashProto(msg.GetJustification()[0]) + justHash, err := HashProto(msg.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(privKey, justHash[:]) diff --git a/core/consensus/qbft_test.go b/core/consensus/qbft/qbft_test.go similarity index 93% rename from core/consensus/qbft_test.go rename to core/consensus/qbft/qbft_test.go index 8121019b4..4713a29af 100644 --- a/core/consensus/qbft_test.go +++ b/core/consensus/qbft/qbft_test.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus_test +package qbft_test import ( "context" @@ -20,7 +20,7 @@ import ( "github.com/obolnetwork/charon/app/z" "github.com/obolnetwork/charon/cluster" "github.com/obolnetwork/charon/core" - "github.com/obolnetwork/charon/core/consensus" + "github.com/obolnetwork/charon/core/consensus/qbft" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/eth2util/enr" "github.com/obolnetwork/charon/p2p" @@ -74,7 +74,7 @@ func testQBFTConsensus(t *testing.T, threshold, nodes int) { peers []p2p.Peer hosts []host.Host hostsInfo []peer.AddrInfo - components []*consensus.QBFTConsensus + components []*qbft.Consensus results = make(chan core.UnsignedDataSet, threshold) runErrs = make(chan error, threshold) sniffed = make(chan int, threshold) @@ -119,7 +119,7 @@ func testQBFTConsensus(t *testing.T, threshold, nodes int) { gaterFunc := func(core.Duty) bool { return true } - c, err := consensus.NewQBFTConsensus(hosts[i], new(p2p.Sender), peers, p2pkeys[i], testDeadliner{}, gaterFunc, sniffer) + c, err := qbft.NewConsensus(hosts[i], new(p2p.Sender), peers, p2pkeys[i], testDeadliner{}, gaterFunc, sniffer) require.NoError(t, err) c.Subscribe(func(_ context.Context, _ core.Duty, set core.UnsignedDataSet) error { results <- set @@ -134,7 +134,7 @@ func testQBFTConsensus(t *testing.T, threshold, nodes int) { // Start all components. for i, c := range components { - go func(ctx context.Context, i int, c *consensus.QBFTConsensus) { + go func(ctx context.Context, i int, c *qbft.Consensus) { runErrs <- c.Propose( log.WithCtx(ctx, z.Int("node", i), z.Str("peer", p2p.PeerName(hosts[i].ID()))), core.Duty{Type: core.DutyAttester, Slot: 1}, diff --git a/core/consensus/strategysim_internal_test.go b/core/consensus/qbft/strategysim_internal_test.go similarity index 96% rename from core/consensus/strategysim_internal_test.go rename to core/consensus/qbft/strategysim_internal_test.go index 1c2d08d59..cc6fe16a7 100644 --- a/core/consensus/strategysim_internal_test.go +++ b/core/consensus/qbft/strategysim_internal_test.go @@ -1,7 +1,7 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 //nolint:forbidigo // This is a test that prints to stdout. -package consensus +package qbft import ( "context" @@ -31,6 +31,7 @@ import ( "github.com/obolnetwork/charon/app/z" "github.com/obolnetwork/charon/cluster" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/qbft" ) @@ -53,7 +54,7 @@ const ( disabled = time.Hour * 999 ) -type roundTimerFunc func(clock clockwork.Clock) roundTimer +type roundTimerFunc func(clock clockwork.Clock) utils.RoundTimer func TestSimulatorOnce(t *testing.T) { syncer, _, _ := zap.Open("stderr") @@ -323,7 +324,7 @@ type ssConfig struct { latencyStdDev time.Duration latencyPerPeer map[int64]time.Duration startByPeer map[int64]time.Duration - roundTimerFunc func(clockwork.Clock) roundTimer + roundTimerFunc func(clockwork.Clock) utils.RoundTimer timeout time.Duration } @@ -457,7 +458,7 @@ func gosched() { } } -func newSimDefinition(nodes int, roundTimer roundTimer, +func newSimDefinition(nodes int, roundTimer utils.RoundTimer, decideCallback func(qcommit []qbft.Msg[core.Duty, [32]byte]), ) qbft.Definition[core.Duty, [32]byte] { quorum := qbft.Definition[int, int]{Nodes: nodes}.Quorum() @@ -498,7 +499,7 @@ func newSimDefinition(nodes int, roundTimer roundTimer, Nodes: nodes, // FIFOLimit caps the max buffered messages per peer. - FIFOLimit: recvBuffer, + FIFOLimit: utils.RecvBufferSize, } } @@ -631,16 +632,16 @@ func (i *transportInstance) Broadcast(_ context.Context, typ qbft.MsgType, // Transform justifications into protobufs var justMsgs []*pbv1.QBFTMsg for _, j := range justification { - impl, ok := j.(qbftMsg) + impl, ok := j.(Msg) if !ok { return errors.New("invalid justification") } - justMsgs = append(justMsgs, impl.msg) // Note nested justifications are ignored. + justMsgs = append(justMsgs, impl.Msg()) // Note nested justifications are ignored. values[impl.Value()] = dummy values[impl.PreparedValue()] = dummy } - msg, err := newQBFTMsg(pbMsg, justMsgs, values) + msg, err := NewMsg(pbMsg, justMsgs, values) if err != nil { return err } @@ -734,14 +735,14 @@ type incRoundTimer2 struct { clock clockwork.Clock } -func (t incRoundTimer2) Type() timerType { +func (t incRoundTimer2) Type() utils.TimerType { return "inc2" } func (t incRoundTimer2) Timer(round int64) (<-chan time.Time, func()) { - duration := incRoundStart + duration := utils.IncRoundStart for i := 1; i < int(round); i++ { - duration += incRoundStart + duration += utils.IncRoundStart } timer := t.clock.NewTimer(duration) @@ -749,7 +750,7 @@ func (t incRoundTimer2) Timer(round int64) (<-chan time.Time, func()) { return timer.Chan(), func() {} } -func randomConfigs(names []string, peers int, n int, timer func(clockwork.Clock) roundTimer, +func randomConfigs(names []string, peers int, n int, timer func(clockwork.Clock) utils.RoundTimer, stdDev []time.Duration, latencies []time.Duration, ) []ssConfig { random := rand.New(rand.NewSource(0)) @@ -902,17 +903,17 @@ func (t *testTimer) Timer(round int64) (<-chan time.Time, func()) { return timer.Chan(), func() {} } -func (t *testTimer) Type() timerType { +func (t *testTimer) Type() utils.TimerType { name := t.name if t.eager { name += "_eager" } - return timerType(name) + return utils.TimerType(name) } func newLinear(d time.Duration) roundTimerFunc { - return func(clock clockwork.Clock) roundTimer { + return func(clock clockwork.Clock) utils.RoundTimer { return &testTimer{ clock: clock, durationFunc: func(round int64) time.Duration { @@ -926,7 +927,7 @@ func newLinear(d time.Duration) roundTimerFunc { } func newExpDouble(d time.Duration) roundTimerFunc { - return func(clock clockwork.Clock) roundTimer { + return func(clock clockwork.Clock) utils.RoundTimer { return &testTimer{ clock: clock, durationFunc: func(round int64) time.Duration { @@ -941,7 +942,7 @@ func newExpDouble(d time.Duration) roundTimerFunc { } func newLinearDouble(d time.Duration) roundTimerFunc { - return func(clock clockwork.Clock) roundTimer { + return func(clock clockwork.Clock) utils.RoundTimer { return &testTimer{ clock: clock, durationFunc: func(round int64) time.Duration { @@ -955,12 +956,12 @@ func newLinearDouble(d time.Duration) roundTimerFunc { } } -func newInc(clock clockwork.Clock) roundTimer { - return &increasingRoundTimer{clock: clock} +func newInc(clock clockwork.Clock) utils.RoundTimer { + return utils.NewIncreasingRoundTimerWithClock(clock) } func newExp(d time.Duration) roundTimerFunc { - return func(clock clockwork.Clock) roundTimer { + return func(clock clockwork.Clock) utils.RoundTimer { return &testTimer{ clock: clock, durationFunc: func(round int64) time.Duration { diff --git a/core/consensus/testdata/TestDebugRoundChange_empty-1.golden b/core/consensus/qbft/testdata/TestDebugRoundChange_empty-1.golden similarity index 100% rename from core/consensus/testdata/TestDebugRoundChange_empty-1.golden rename to core/consensus/qbft/testdata/TestDebugRoundChange_empty-1.golden diff --git a/core/consensus/testdata/TestDebugRoundChange_empty-2.golden b/core/consensus/qbft/testdata/TestDebugRoundChange_empty-2.golden similarity index 100% rename from core/consensus/testdata/TestDebugRoundChange_empty-2.golden rename to core/consensus/qbft/testdata/TestDebugRoundChange_empty-2.golden diff --git a/core/consensus/testdata/TestDebugRoundChange_quorum.golden b/core/consensus/qbft/testdata/TestDebugRoundChange_quorum.golden similarity index 100% rename from core/consensus/testdata/TestDebugRoundChange_quorum.golden rename to core/consensus/qbft/testdata/TestDebugRoundChange_quorum.golden diff --git a/core/consensus/testdata/TestHashProto.golden b/core/consensus/qbft/testdata/TestHashProto.golden similarity index 100% rename from core/consensus/testdata/TestHashProto.golden rename to core/consensus/qbft/testdata/TestHashProto.golden diff --git a/core/consensus/qbft_transport.go b/core/consensus/qbft/transport.go similarity index 58% rename from core/consensus/qbft_transport.go rename to core/consensus/qbft/transport.go index 2654d82a4..dd3deea24 100644 --- a/core/consensus/qbft_transport.go +++ b/core/consensus/qbft/transport.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package qbft import ( "context" @@ -12,16 +12,23 @@ import ( "github.com/obolnetwork/charon/app/errors" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/qbft" ) -// qbftTransport encapsulates receiving and broadcasting for a consensus instance/duty. -type qbftTransport struct { +// Broadcaster is an interface for broadcasting messages asynchronously. +type Broadcaster interface { + Broadcast(ctx context.Context, msg *pbv1.ConsensusMsg) error +} + +// Transport encapsulates receiving and broadcasting for a consensus instance/duty. +type Transport struct { // Immutable state - component *QBFTConsensus - recvBuffer chan qbft.Msg[core.Duty, [32]byte] // Instance inner receive buffer. - sniffer *sniffer + broadcaster Broadcaster + privkey *k1.PrivateKey + recvBuffer chan qbft.Msg[core.Duty, [32]byte] // Instance inner receive buffer. + sniffer *utils.Sniffer // Mutable state valueMu sync.Mutex @@ -29,25 +36,39 @@ type qbftTransport struct { values map[[32]byte]*anypb.Any // maps any-wrapped proposed values to their hashes } +// NewTransport creates a new qbftTransport. +func NewTransport(broadcaster Broadcaster, privkey *k1.PrivateKey, valueCh <-chan proto.Message, + recvBuffer chan qbft.Msg[core.Duty, [32]byte], sniffer *utils.Sniffer, +) *Transport { + return &Transport{ + broadcaster: broadcaster, + privkey: privkey, + recvBuffer: recvBuffer, + sniffer: sniffer, + valueCh: valueCh, + values: make(map[[32]byte]*anypb.Any), + } +} + // setValues caches the values and their hashes. -func (t *qbftTransport) setValues(msg qbftMsg) { +func (t *Transport) setValues(msg Msg) { t.valueMu.Lock() defer t.valueMu.Unlock() - for k, v := range msg.values { + for k, v := range msg.Values() { t.values[k] = v } } // getValue returns the value by its hash. -func (t *qbftTransport) getValue(hash [32]byte) (*anypb.Any, error) { +func (t *Transport) getValue(hash [32]byte) (*anypb.Any, error) { t.valueMu.Lock() defer t.valueMu.Unlock() // First check if we have a new value. select { case value := <-t.valueCh: - valueHash, err := hashProto(value) + valueHash, err := HashProto(value) if err != nil { return nil, err } @@ -71,7 +92,7 @@ func (t *qbftTransport) getValue(hash [32]byte) (*anypb.Any, error) { } // Broadcast creates a msg and sends it to all peers (including self). -func (t *qbftTransport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.Duty, +func (t *Transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.Duty, peerIdx int64, round int64, valueHash [32]byte, pr int64, pvHash [32]byte, justification []qbft.Msg[core.Duty, [32]byte], ) error { @@ -80,12 +101,12 @@ func (t *qbftTransport) Broadcast(ctx context.Context, typ qbft.MsgType, duty co hashes = append(hashes, valueHash) hashes = append(hashes, pvHash) for _, just := range justification { - msg, ok := just.(qbftMsg) + msg, ok := just.(Msg) if !ok { return errors.New("invalid justification message") } - hashes = append(hashes, msg.valueHash) - hashes = append(hashes, msg.preparedValueHash) + hashes = append(hashes, msg.Value()) + hashes = append(hashes, msg.PreparedValue()) } // Get values by their hashes if not zero. @@ -104,8 +125,8 @@ func (t *qbftTransport) Broadcast(ctx context.Context, typ qbft.MsgType, duty co } // Make the message - msg, err := createQBFTMsg(typ, duty, peerIdx, round, valueHash, pr, - pvHash, values, justification, t.component.privkey) + msg, err := createMsg(typ, duty, peerIdx, round, valueHash, pr, + pvHash, values, justification, t.privkey) if err != nil { return err } @@ -119,23 +140,11 @@ func (t *qbftTransport) Broadcast(ctx context.Context, typ qbft.MsgType, duty co } }() - for _, p := range t.component.peers { - if p.ID == t.component.tcpNode.ID() { - // Do not broadcast to self - continue - } - - err = t.component.sender.SendAsync(ctx, t.component.tcpNode, QBFTv2ProtocolID, p.ID, msg.ToConsensusMsg()) - if err != nil { - return err - } - } - - return nil + return t.broadcaster.Broadcast(ctx, msg.ToConsensusMsg()) } // ProcessReceives processes received messages from the outer buffer until the context is closed. -func (t *qbftTransport) ProcessReceives(ctx context.Context, outerBuffer chan qbftMsg) { +func (t *Transport) ProcessReceives(ctx context.Context, outerBuffer chan Msg) { for { select { case <-ctx.Done(): @@ -153,13 +162,23 @@ func (t *qbftTransport) ProcessReceives(ctx context.Context, outerBuffer chan qb } } -// createQBFTMsg returns a new message by converting the inputs into a protobuf +// SnifferInstance returns the current sniffed consensus instance. +func (t *Transport) SnifferInstance() *pbv1.SniffedConsensusInstance { + return t.sniffer.Instance() +} + +// RecvBuffer returns the inner receive buffer. +func (t *Transport) RecvBuffer() chan qbft.Msg[core.Duty, [32]byte] { + return t.recvBuffer +} + +// createMsg returns a new message by converting the inputs into a protobuf // and wrapping that in a msg type. -func createQBFTMsg(typ qbft.MsgType, duty core.Duty, +func createMsg(typ qbft.MsgType, duty core.Duty, peerIdx int64, round int64, vHash [32]byte, pr int64, pvHash [32]byte, values map[[32]byte]*anypb.Any, justification []qbft.Msg[core.Duty, [32]byte], privkey *k1.PrivateKey, -) (qbftMsg, error) { +) (Msg, error) { pbMsg := &pbv1.QBFTMsg{ Type: int64(typ), Duty: core.DutyToProto(duty), @@ -170,20 +189,20 @@ func createQBFTMsg(typ qbft.MsgType, duty core.Duty, PreparedValueHash: pvHash[:], } - pbMsg, err := signQBFTMsg(pbMsg, privkey) + pbMsg, err := SignMsg(pbMsg, privkey) if err != nil { - return qbftMsg{}, err + return Msg{}, err } // Transform justifications into protobufs var justMsgs []*pbv1.QBFTMsg for _, j := range justification { - impl, ok := j.(qbftMsg) + impl, ok := j.(Msg) if !ok { - return qbftMsg{}, errors.New("invalid justification") + return Msg{}, errors.New("invalid justification") } - justMsgs = append(justMsgs, impl.msg) // Note nested justifications are ignored. + justMsgs = append(justMsgs, impl.Msg()) // Note nested justifications are ignored. } - return newQBFTMsg(pbMsg, justMsgs, values) + return NewMsg(pbMsg, justMsgs, values) } diff --git a/core/consensus/sniffed_internal_test.go b/core/consensus/sniffed_internal_test.go index 9489ea976..53c5ff8ad 100644 --- a/core/consensus/sniffed_internal_test.go +++ b/core/consensus/sniffed_internal_test.go @@ -19,6 +19,8 @@ import ( "github.com/obolnetwork/charon/app/log" "github.com/obolnetwork/charon/app/z" "github.com/obolnetwork/charon/core" + cqbft "github.com/obolnetwork/charon/core/consensus/qbft" + "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/qbft" ) @@ -68,15 +70,15 @@ func testSniffedInstance(ctx context.Context, t *testing.T, instance *pbv1.Sniff var expectDecided bool - def := newQBFTDefinition(int(instance.GetNodes()), func() []subscriber { - return []subscriber{func(ctx context.Context, duty core.Duty, value proto.Message) error { + def := cqbft.NewDefinition(int(instance.GetNodes()), func() []cqbft.Subscriber { + return []cqbft.Subscriber{func(ctx context.Context, duty core.Duty, value proto.Message) error { log.Info(ctx, "Consensus decided", z.Any("value", value)) expectDecided = true cancel() return nil }} - }, newIncreasingRoundTimer(), func(qcommit []qbft.Msg[core.Duty, [32]byte]) {}) + }, utils.NewIncreasingRoundTimer(), func(qcommit []qbft.Msg[core.Duty, [32]byte]) {}) recvBuffer := make(chan qbft.Msg[core.Duty, [32]byte], len(instance.GetMsgs())) @@ -88,10 +90,10 @@ func testSniffedInstance(ctx context.Context, t *testing.T, instance *pbv1.Sniff duty = core.DutyFromProto(msg.GetMsg().GetMsg().GetDuty()) - values, err := valuesByHash(msg.GetMsg().GetValues()) + values, err := cqbft.ValuesByHash(msg.GetMsg().GetValues()) require.NoError(t, err) - m, err := newQBFTMsg(msg.GetMsg().GetMsg(), msg.GetMsg().GetJustification(), values) + m, err := cqbft.NewMsg(msg.GetMsg().GetMsg(), msg.GetMsg().GetJustification(), values) require.NoError(t, err) recvBuffer <- m } diff --git a/core/consensus/utils/instance_io.go b/core/consensus/utils/instance_io.go new file mode 100644 index 000000000..ea23f563e --- /dev/null +++ b/core/consensus/utils/instance_io.go @@ -0,0 +1,81 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package utils + +import ( + "time" + + "google.golang.org/protobuf/proto" + + "github.com/obolnetwork/charon/app/errors" +) + +const ( + RecvBufferSize = 100 // Allow buffering some initial messages when this node is late to start an instance. +) + +// NewInstanceIO returns a new instanceIO. +func NewInstanceIO[T any]() *InstanceIO[T] { + return &InstanceIO[T]{ + Participated: make(chan struct{}), + Proposed: make(chan struct{}), + Running: make(chan struct{}), + RecvBuffer: make(chan T, RecvBufferSize), + HashCh: make(chan [32]byte, 1), + ValueCh: make(chan proto.Message, 1), + ErrCh: make(chan error, 1), + DecidedAtCh: make(chan time.Time, 1), + } +} + +// InstanceIO defines the async input and output channels of a +// single consensus instance in the Component. +type InstanceIO[T any] struct { + Participated chan struct{} // Closed when Participate was called for this instance. + Proposed chan struct{} // Closed when Propose was called for this instance. + Running chan struct{} // Closed when runInstance was already called. + RecvBuffer chan T // Outer receive buffers. + HashCh chan [32]byte // Async input hash channel. + ValueCh chan proto.Message // Async input value channel. + ErrCh chan error // Async output error channel. + DecidedAtCh chan time.Time // Async output decided timestamp channel. +} + +// MarkParticipated marks the instance as participated. +// It returns an error if the instance was already marked as participated. +func (io *InstanceIO[T]) MarkParticipated() error { + select { + case <-io.Participated: + return errors.New("already participated") + default: + close(io.Participated) + } + + return nil +} + +// MarkProposed marks the instance as proposed. +// It returns an error if the instance was already marked as proposed. +func (io *InstanceIO[T]) MarkProposed() error { + select { + case <-io.Proposed: + return errors.New("already proposed") + default: + close(io.Proposed) + } + + return nil +} + +// MaybeStart returns true if the instance wasn't running and has been started by this call, +// otherwise it returns false if the instance was started in the past and is either running now or has completed. +func (io *InstanceIO[T]) MaybeStart() bool { + select { + case <-io.Running: + return false + default: + close(io.Running) + } + + return true +} diff --git a/core/consensus/instance_io_internal_test.go b/core/consensus/utils/instance_io_test.go similarity index 75% rename from core/consensus/instance_io_internal_test.go rename to core/consensus/utils/instance_io_test.go index 686f8c2fd..34682d7e2 100644 --- a/core/consensus/instance_io_internal_test.go +++ b/core/consensus/utils/instance_io_test.go @@ -1,15 +1,18 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package utils_test import ( "testing" "github.com/stretchr/testify/require" + + cqbft "github.com/obolnetwork/charon/core/consensus/qbft" + "github.com/obolnetwork/charon/core/consensus/utils" ) func TestMarkParticipated(t *testing.T) { - io := newInstanceIO[qbftMsg]() + io := utils.NewInstanceIO[cqbft.Msg]() // First call succeeds. err := io.MarkParticipated() @@ -21,7 +24,7 @@ func TestMarkParticipated(t *testing.T) { } func TestMarkProposed(t *testing.T) { - io := newInstanceIO[qbftMsg]() + io := utils.NewInstanceIO[cqbft.Msg]() // First call succeeds. err := io.MarkProposed() @@ -33,7 +36,7 @@ func TestMarkProposed(t *testing.T) { } func TestMaybeStart(t *testing.T) { - io := newInstanceIO[qbftMsg]() + io := utils.NewInstanceIO[cqbft.Msg]() // First call succeeds. ok := io.MaybeStart() diff --git a/core/consensus/roundtimer.go b/core/consensus/utils/roundtimer.go similarity index 63% rename from core/consensus/roundtimer.go rename to core/consensus/utils/roundtimer.go index c82e1555e..7a71228b6 100644 --- a/core/consensus/roundtimer.go +++ b/core/consensus/utils/roundtimer.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package utils import ( "strings" @@ -14,64 +14,69 @@ import ( ) const ( - incRoundStart = time.Millisecond * 750 - incRoundIncrease = time.Millisecond * 250 - linearRoundInc = time.Second + IncRoundStart = time.Millisecond * 750 + IncRoundIncrease = time.Millisecond * 250 + LinearRoundInc = time.Second ) -// timerFunc is a function that returns a round timer. -type timerFunc func(core.Duty) roundTimer +// TimerFunc is a function that returns a round timer. +type TimerFunc func(core.Duty) RoundTimer -// getTimerFunc returns a timer function based on the enabled features. -func getTimerFunc() timerFunc { +// GetTimerFunc returns a timer function based on the enabled features. +func GetTimerFunc() TimerFunc { if featureset.Enabled(featureset.EagerDoubleLinear) { - return func(core.Duty) roundTimer { - return newDoubleEagerLinearRoundTimer() + return func(core.Duty) RoundTimer { + return NewDoubleEagerLinearRoundTimer() } } // Default to increasing round timer. - return func(core.Duty) roundTimer { - return newIncreasingRoundTimer() + return func(core.Duty) RoundTimer { + return NewIncreasingRoundTimer() } } -// timerType is the type of round timer. -type timerType string +// TimerType is the type of round timer. +type TimerType string // Eager returns true if the timer type requires an eager start (before proposal values are present). -func (t timerType) Eager() bool { +func (t TimerType) Eager() bool { return strings.Contains(string(t), "eager") } const ( - timerIncreasing timerType = "inc" - timerEagerDoubleLinear timerType = "eager_dlinear" + TimerIncreasing TimerType = "inc" + TimerEagerDoubleLinear TimerType = "eager_dlinear" ) // increasingRoundTimeout returns the duration for a round that starts at incRoundStart in round 1 // and increases by incRoundIncrease for each subsequent round. func increasingRoundTimeout(round int64) time.Duration { - return incRoundStart + (time.Duration(round) * incRoundIncrease) + return IncRoundStart + (time.Duration(round) * IncRoundIncrease) } // increasingRoundTimeout returns linearRoundInc*round duration for a round. func linearRoundTimeout(round int64) time.Duration { - return time.Duration(round) * linearRoundInc + return time.Duration(round) * LinearRoundInc } -// roundTimer provides the duration for each QBFT round. -type roundTimer interface { +// RoundTimer provides the duration for each consensus round. +type RoundTimer interface { // Timer returns a channel that will be closed when the round expires and a stop function. Timer(round int64) (<-chan time.Time, func()) // Type returns the type of the round timerType. - Type() timerType + Type() TimerType } -// newTimeoutRoundTimer returns a new increasing round timerType. -func newIncreasingRoundTimer() roundTimer { +// NewTimeoutRoundTimer returns a new increasing round timer type. +func NewIncreasingRoundTimer() RoundTimer { + return NewIncreasingRoundTimerWithClock(clockwork.NewRealClock()) +} + +// NewIncreasingRoundTimerWithClock returns a new increasing round timer type with a custom clock. +func NewIncreasingRoundTimerWithClock(clock clockwork.Clock) RoundTimer { return &increasingRoundTimer{ - clock: clockwork.NewRealClock(), + clock: clock, } } @@ -80,8 +85,8 @@ type increasingRoundTimer struct { clock clockwork.Clock } -func (increasingRoundTimer) Type() timerType { - return timerIncreasing +func (increasingRoundTimer) Type() TimerType { + return TimerIncreasing } func (t increasingRoundTimer) Timer(round int64) (<-chan time.Time, func()) { @@ -89,10 +94,15 @@ func (t increasingRoundTimer) Timer(round int64) (<-chan time.Time, func()) { return timer.Chan(), func() { timer.Stop() } } -// doubleEagerLinearRoundTimer returns a new eager double linear round timerType. -func newDoubleEagerLinearRoundTimer() roundTimer { +// NewDoubleEagerLinearRoundTimer returns a new eager double linear round timer type. +func NewDoubleEagerLinearRoundTimer() RoundTimer { + return NewDoubleEagerLinearRoundTimerWithClock(clockwork.NewRealClock()) +} + +// NewDoubleEagerLinearRoundTimerWithClock returns a new eager double linear round timer type with a custom clock. +func NewDoubleEagerLinearRoundTimerWithClock(clock clockwork.Clock) RoundTimer { return &doubleEagerLinearRoundTimer{ - clock: clockwork.NewRealClock(), + clock: clock, firstDeadlines: make(map[int64]time.Time), } } @@ -118,8 +128,8 @@ type doubleEagerLinearRoundTimer struct { firstDeadlines map[int64]time.Time } -func (*doubleEagerLinearRoundTimer) Type() timerType { - return timerEagerDoubleLinear +func (*doubleEagerLinearRoundTimer) Type() TimerType { + return TimerEagerDoubleLinear } func (t *doubleEagerLinearRoundTimer) Timer(round int64) (<-chan time.Time, func()) { diff --git a/core/consensus/roundtimer_internal_test.go b/core/consensus/utils/roundtimer_test.go similarity index 74% rename from core/consensus/roundtimer_internal_test.go rename to core/consensus/utils/roundtimer_test.go index 9c73487f1..461c6377d 100644 --- a/core/consensus/roundtimer_internal_test.go +++ b/core/consensus/utils/roundtimer_test.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package utils_test import ( "testing" @@ -11,6 +11,7 @@ import ( "github.com/obolnetwork/charon/app/featureset" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/utils" ) func TestIncreasingRoundTimer(t *testing.T) { @@ -38,8 +39,7 @@ func TestIncreasingRoundTimer(t *testing.T) { for _, tt := range tests { fakeClock := clockwork.NewFakeClock() - timer := newIncreasingRoundTimer().(*increasingRoundTimer) - timer.clock = fakeClock + timer := utils.NewIncreasingRoundTimerWithClock(fakeClock) t.Run(tt.name, func(t *testing.T) { // Start the timerType @@ -63,8 +63,7 @@ func TestIncreasingRoundTimer(t *testing.T) { func TestDoubleEagerLinearRoundTimer(t *testing.T) { fakeClock := clockwork.NewFakeClock() - timer := newDoubleEagerLinearRoundTimer().(*doubleEagerLinearRoundTimer) - timer.clock = fakeClock + timer := utils.NewDoubleEagerLinearRoundTimerWithClock(fakeClock) require.True(t, timer.Type().Eager()) @@ -113,16 +112,16 @@ func TestDoubleEagerLinearRoundTimer(t *testing.T) { } func TestGetTimerFunc(t *testing.T) { - timerFunc := getTimerFunc() - require.Equal(t, timerEagerDoubleLinear, timerFunc(core.NewAttesterDuty(0)).Type()) - require.Equal(t, timerEagerDoubleLinear, timerFunc(core.NewAttesterDuty(1)).Type()) - require.Equal(t, timerEagerDoubleLinear, timerFunc(core.NewAttesterDuty(2)).Type()) + timerFunc := utils.GetTimerFunc() + require.Equal(t, utils.TimerEagerDoubleLinear, timerFunc(core.NewAttesterDuty(0)).Type()) + require.Equal(t, utils.TimerEagerDoubleLinear, timerFunc(core.NewAttesterDuty(1)).Type()) + require.Equal(t, utils.TimerEagerDoubleLinear, timerFunc(core.NewAttesterDuty(2)).Type()) featureset.DisableForT(t, featureset.EagerDoubleLinear) - timerFunc = getTimerFunc() + timerFunc = utils.GetTimerFunc() - require.Equal(t, timerIncreasing, timerFunc(core.NewAttesterDuty(0)).Type()) - require.Equal(t, timerIncreasing, timerFunc(core.NewAttesterDuty(1)).Type()) - require.Equal(t, timerIncreasing, timerFunc(core.NewAttesterDuty(2)).Type()) + require.Equal(t, utils.TimerIncreasing, timerFunc(core.NewAttesterDuty(0)).Type()) + require.Equal(t, utils.TimerIncreasing, timerFunc(core.NewAttesterDuty(1)).Type()) + require.Equal(t, utils.TimerIncreasing, timerFunc(core.NewAttesterDuty(2)).Type()) } diff --git a/core/consensus/sniffer.go b/core/consensus/utils/sniffer.go similarity index 74% rename from core/consensus/sniffer.go rename to core/consensus/utils/sniffer.go index f411cdaab..33243523d 100644 --- a/core/consensus/sniffer.go +++ b/core/consensus/utils/sniffer.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package utils import ( "sync" @@ -11,17 +11,17 @@ import ( pbv1 "github.com/obolnetwork/charon/core/corepb/v1" ) -// newSniffer returns a new sniffer. -func newSniffer(nodes, peerIdx int64) *sniffer { - return &sniffer{ +// NewSniffer returns a new sniffer. +func NewSniffer(nodes, peerIdx int64) *Sniffer { + return &Sniffer{ nodes: nodes, peerIdx: peerIdx, startedAt: time.Now(), } } -// sniffer buffers consensus messages. -type sniffer struct { +// Sniffer buffers consensus messages. +type Sniffer struct { nodes int64 peerIdx int64 startedAt time.Time @@ -31,7 +31,7 @@ type sniffer struct { } // Add adds a message to the sniffer buffer. -func (c *sniffer) Add(msg *pbv1.ConsensusMsg) { +func (c *Sniffer) Add(msg *pbv1.ConsensusMsg) { c.mu.Lock() defer c.mu.Unlock() @@ -42,7 +42,7 @@ func (c *sniffer) Add(msg *pbv1.ConsensusMsg) { } // Instance returns the buffered messages as an instance. -func (c *sniffer) Instance() *pbv1.SniffedConsensusInstance { +func (c *Sniffer) Instance() *pbv1.SniffedConsensusInstance { c.mu.Lock() defer c.mu.Unlock() From f7589140167a7c79da1bae6d2feedc440dc17886 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Thu, 10 Oct 2024 18:27:17 +0300 Subject: [PATCH 06/20] ConsensusMetrics interface --- core/consensus/metrics/metrics.go | 49 ++++++++++++++++++++++++++----- core/consensus/qbft/qbft.go | 10 ++++--- docs/metrics.md | 8 ++--- 3 files changed, 51 insertions(+), 16 deletions(-) diff --git a/core/consensus/metrics/metrics.go b/core/consensus/metrics/metrics.go index becbe25a4..87661d0e3 100644 --- a/core/consensus/metrics/metrics.go +++ b/core/consensus/metrics/metrics.go @@ -8,33 +8,66 @@ import ( "github.com/obolnetwork/charon/app/promauto" ) +type ConsensusMetrics interface { + SetDecidedRounds(duty, timer string, rounds float64) + ObserveConsensusDuration(duty, timer string, duration float64) + IncConsensusTimeout(duty, timer string) + IncConsensusError() +} + var ( - DecidedRoundsGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ + decidedRoundsGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: "core", Subsystem: "consensus", Name: "decided_rounds", Help: "Number of rounds it took to decide consensus instances by duty and timer type.", - }, []string{"duty", "timer"}) // Using gauge since the value changes slowly, once per slot. + }, []string{"protocol", "duty", "timer"}) // Using gauge since the value changes slowly, once per slot. - ConsensusDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ + consensusDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "core", Subsystem: "consensus", Name: "duration_seconds", Help: "Duration of a consensus instance in seconds by duty and timer type.", Buckets: []float64{.05, .1, .25, .5, 1, 2.5, 5, 10, 20, 30, 60}, - }, []string{"duty", "timer"}) + }, []string{"protocol", "duty", "timer"}) - ConsensusTimeout = promauto.NewCounterVec(prometheus.CounterOpts{ + consensusTimeout = promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: "core", Subsystem: "consensus", Name: "timeout_total", Help: "Total count of consensus timeouts by duty and timer type.", - }, []string{"duty", "timer"}) + }, []string{"protocol", "duty", "timer"}) - ConsensusError = promauto.NewCounter(prometheus.CounterOpts{ + consensusError = promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: "core", Subsystem: "consensus", Name: "error_total", Help: "Total count of consensus errors", - }) + }, []string{"protocol"}) ) + +type consensusMetrics struct { + protocolID string +} + +func NewConsensusMetrics(protocolID string) ConsensusMetrics { + return &consensusMetrics{ + protocolID: protocolID, + } +} + +func (m *consensusMetrics) SetDecidedRounds(duty, timer string, rounds float64) { + decidedRoundsGauge.WithLabelValues(m.protocolID, duty, timer).Set(rounds) +} + +func (m *consensusMetrics) ObserveConsensusDuration(duty, timer string, duration float64) { + consensusDuration.WithLabelValues(m.protocolID, duty, timer).Observe(duration) +} + +func (m *consensusMetrics) IncConsensusTimeout(duty, timer string) { + consensusTimeout.WithLabelValues(m.protocolID, duty, timer).Inc() +} + +func (m *consensusMetrics) IncConsensusError() { + consensusError.WithLabelValues(m.protocolID).Inc() +} diff --git a/core/consensus/qbft/qbft.go b/core/consensus/qbft/qbft.go index 594013f7e..4e30d40cf 100644 --- a/core/consensus/qbft/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -148,6 +148,7 @@ func NewConsensus(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKe gaterFunc: gaterFunc, dropFilter: log.Filter(), timerFunc: utils.GetTimerFunc(), + metrics: metrics.NewConsensusMetrics(protocols.QBFTv2ProtocolID), } c.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) @@ -169,6 +170,7 @@ type Consensus struct { gaterFunc core.DutyGaterFunc dropFilter z.Field // Filter buffer overflow errors (possible DDoS) timerFunc utils.TimerFunc + metrics metrics.ConsensusMetrics // Mutable state mutable struct { @@ -289,7 +291,7 @@ func (c *Consensus) propose(ctx context.Context, duty core.Duty, value proto.Mes case decidedAt := <-inst.DecidedAtCh: timerType := c.timerFunc(duty).Type() duration := decidedAt.Sub(proposedAt) - metrics.ConsensusDuration.WithLabelValues(duty.Type.String(), string(timerType)).Observe(duration.Seconds()) + c.metrics.ObserveConsensusDuration(duty.Type.String(), string(timerType), duration.Seconds()) default: } }() @@ -377,7 +379,7 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) var decided bool decideCallback := func(qcommit []qbft.Msg[core.Duty, [32]byte]) { decided = true - metrics.DecidedRoundsGauge.WithLabelValues(duty.Type.String(), string(roundTimer.Type())).Set(float64(qcommit[0].Round())) + c.metrics.SetDecidedRounds(duty.Type.String(), string(roundTimer.Type()), float64(qcommit[0].Round())) inst.DecidedAtCh <- time.Now() } @@ -404,12 +406,12 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) // Run the algo, blocking until the context is cancelled. err = qbft.Run[core.Duty, [32]byte](ctx, def, qt, duty, peerIdx, inst.HashCh) if err != nil && !isContextErr(err) { - metrics.ConsensusError.Inc() + c.metrics.IncConsensusError() return err // Only return non-context errors. } if !decided { - metrics.ConsensusTimeout.WithLabelValues(duty.Type.String(), string(roundTimer.Type())).Inc() + c.metrics.IncConsensusTimeout(duty.Type.String(), string(roundTimer.Type())) return errors.New("consensus timeout", z.Str("duty", duty.String())) } diff --git a/docs/metrics.md b/docs/metrics.md index 4bcfcbe96..ca453bc23 100644 --- a/docs/metrics.md +++ b/docs/metrics.md @@ -43,10 +43,10 @@ when storing metrics from multiple nodes or clusters in one Prometheus instance. | `core_bcast_recast_errors_total` | Counter | The total count of failed recasted registrations by source; `pregen` vs `downstream` | `source` | | `core_bcast_recast_registration_total` | Counter | The total number of unique validator registration stored in recaster per pubkey | `pubkey` | | `core_bcast_recast_total` | Counter | The total count of recasted registrations by source; `pregen` vs `downstream` | `source` | -| `core_consensus_decided_rounds` | Gauge | Number of rounds it took to decide consensus instances by duty and timer type. | `duty, timer` | -| `core_consensus_duration_seconds` | Histogram | Duration of a consensus instance in seconds by duty and timer type. | `duty, timer` | -| `core_consensus_error_total` | Counter | Total count of consensus errors | | -| `core_consensus_timeout_total` | Counter | Total count of consensus timeouts by duty and timer type. | `duty, timer` | +| `core_consensus_decided_rounds` | Gauge | Number of rounds it took to decide consensus instances by duty and timer type. | `protocol, duty, timer` | +| `core_consensus_duration_seconds` | Histogram | Duration of a consensus instance in seconds by duty and timer type. | `protocol, duty, timer` | +| `core_consensus_error_total` | Counter | Total count of consensus errors | `protocol` | +| `core_consensus_timeout_total` | Counter | Total count of consensus timeouts by duty and timer type. | `protocol, duty, timer` | | `core_parsigdb_exit_total` | Counter | Total number of partially signed voluntary exits per public key | `pubkey` | | `core_scheduler_current_epoch` | Gauge | The current epoch | | | `core_scheduler_current_slot` | Gauge | The current slot | | From 9324fa7c3293320232d9ee1c0bcdcce886cc5532 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Thu, 10 Oct 2024 18:31:35 +0300 Subject: [PATCH 07/20] ConsensusMetrics interface --- core/consensus/metrics/metrics.go | 8 ++++---- docs/metrics.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/consensus/metrics/metrics.go b/core/consensus/metrics/metrics.go index 87661d0e3..22af8e0ec 100644 --- a/core/consensus/metrics/metrics.go +++ b/core/consensus/metrics/metrics.go @@ -20,14 +20,14 @@ var ( Namespace: "core", Subsystem: "consensus", Name: "decided_rounds", - Help: "Number of rounds it took to decide consensus instances by duty and timer type.", + Help: "Number of rounds it took to decide consensus instances by protocol, duty and timer type.", }, []string{"protocol", "duty", "timer"}) // Using gauge since the value changes slowly, once per slot. consensusDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "core", Subsystem: "consensus", Name: "duration_seconds", - Help: "Duration of a consensus instance in seconds by duty and timer type.", + Help: "Duration of a consensus instance in seconds by protocol, duty and timer type.", Buckets: []float64{.05, .1, .25, .5, 1, 2.5, 5, 10, 20, 30, 60}, }, []string{"protocol", "duty", "timer"}) @@ -35,14 +35,14 @@ var ( Namespace: "core", Subsystem: "consensus", Name: "timeout_total", - Help: "Total count of consensus timeouts by duty and timer type.", + Help: "Total count of consensus timeouts by protocol, duty and timer type.", }, []string{"protocol", "duty", "timer"}) consensusError = promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: "core", Subsystem: "consensus", Name: "error_total", - Help: "Total count of consensus errors", + Help: "Total count of consensus errors by protocol", }, []string{"protocol"}) ) diff --git a/docs/metrics.md b/docs/metrics.md index ca453bc23..41ceb5117 100644 --- a/docs/metrics.md +++ b/docs/metrics.md @@ -43,10 +43,10 @@ when storing metrics from multiple nodes or clusters in one Prometheus instance. | `core_bcast_recast_errors_total` | Counter | The total count of failed recasted registrations by source; `pregen` vs `downstream` | `source` | | `core_bcast_recast_registration_total` | Counter | The total number of unique validator registration stored in recaster per pubkey | `pubkey` | | `core_bcast_recast_total` | Counter | The total count of recasted registrations by source; `pregen` vs `downstream` | `source` | -| `core_consensus_decided_rounds` | Gauge | Number of rounds it took to decide consensus instances by duty and timer type. | `protocol, duty, timer` | -| `core_consensus_duration_seconds` | Histogram | Duration of a consensus instance in seconds by duty and timer type. | `protocol, duty, timer` | -| `core_consensus_error_total` | Counter | Total count of consensus errors | `protocol` | -| `core_consensus_timeout_total` | Counter | Total count of consensus timeouts by duty and timer type. | `protocol, duty, timer` | +| `core_consensus_decided_rounds` | Gauge | Number of rounds it took to decide consensus instances by protocol, duty and timer type. | `protocol, duty, timer` | +| `core_consensus_duration_seconds` | Histogram | Duration of a consensus instance in seconds by protocol, duty and timer type. | `protocol, duty, timer` | +| `core_consensus_error_total` | Counter | Total count of consensus errors by protocol | `protocol` | +| `core_consensus_timeout_total` | Counter | Total count of consensus timeouts by protocol, duty and timer type. | `protocol, duty, timer` | | `core_parsigdb_exit_total` | Counter | Total number of partially signed voluntary exits per public key | `pubkey` | | `core_scheduler_current_epoch` | Gauge | The current epoch | | | `core_scheduler_current_slot` | Gauge | The current slot | | From 6accee305c8968a415f314e8b338e681b985896b Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Fri, 11 Oct 2024 08:33:56 +0300 Subject: [PATCH 08/20] More of refactoring and tests --- app/app.go | 16 ++--- core/consensus/factory.go | 58 +++++++++------- core/consensus/factory_test.go | 17 +++-- core/consensus/metrics/metrics_test.go | 72 ++++++++++++++++++++ core/consensus/qbft/qbft.go | 8 ++- core/consensus/{utils => qbft}/sniffer.go | 16 ++--- core/consensus/qbft/sniffer_internal_test.go | 29 ++++++++ core/consensus/qbft/transport.go | 5 +- core/interfaces.go | 14 +++- 9 files changed, 183 insertions(+), 52 deletions(-) create mode 100644 core/consensus/metrics/metrics_test.go rename core/consensus/{utils => qbft}/sniffer.go (74%) create mode 100644 core/consensus/qbft/sniffer_internal_test.go diff --git a/app/app.go b/app/app.go index 4bed7363a..fc9172bf2 100644 --- a/app/app.go +++ b/app/app.go @@ -527,20 +527,18 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, retryer := retry.New[core.Duty](deadlineFunc) - consensusFactory := consensus.NewConsensusFactory( + consensusFactory, err := consensus.NewConsensusFactory( tcpNode, sender, peers, p2pKey, deadlinerFunc("consensus"), gaterFunc, consensusDebugger.AddInstance) - - // We always need QBFT consensus instance as it is used for priority protocol. - // And for now it is used as the primary consensus protocol. - qbftConsensus, err := consensusFactory.New(cprotocols.QBFTv2ProtocolID) if err != nil { return err } - startQBFTConsensus := lifecycle.HookFuncCtx(qbftConsensus.Start) + + defaultConsensus := consensusFactory.DefaultConsensus() + startDefaultConsensus := lifecycle.HookFuncCtx(defaultConsensus.Start) err = wirePrioritise(ctx, conf, life, tcpNode, peerIDs, int(cluster.GetThreshold()), - sender.SendReceive, qbftConsensus, sched, p2pKey, deadlineFunc) + sender.SendReceive, defaultConsensus, sched, p2pKey, deadlineFunc) if err != nil { return err } @@ -565,7 +563,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, core.WithTracking(track, inclusion), core.WithAsyncRetry(retryer), } - core.Wire(sched, fetch, qbftConsensus, dutyDB, vapi, parSigDB, parSigEx, sigAgg, aggSigDB, broadcaster, opts...) + core.Wire(sched, fetch, defaultConsensus, dutyDB, vapi, parSigDB, parSigEx, sigAgg, aggSigDB, broadcaster, opts...) err = wireValidatorMock(ctx, conf, eth2Cl, pubshares, sched) if err != nil { @@ -577,7 +575,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, } life.RegisterStart(lifecycle.AsyncBackground, lifecycle.StartScheduler, lifecycle.HookFuncErr(sched.Run)) - life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startQBFTConsensus) + life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startDefaultConsensus) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartAggSigDB, lifecycle.HookFuncCtx(aggSigDB.Run)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartParSigDB, lifecycle.HookFuncCtx(parSigDB.Trim)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartTracker, lifecycle.HookFuncCtx(inclusion.Run)) diff --git a/core/consensus/factory.go b/core/consensus/factory.go index 7cf58990a..07cdbff25 100644 --- a/core/consensus/factory.go +++ b/core/consensus/factory.go @@ -9,43 +9,55 @@ import ( "github.com/obolnetwork/charon/app/errors" "github.com/obolnetwork/charon/core" - "github.com/obolnetwork/charon/core/consensus/protocols" "github.com/obolnetwork/charon/core/consensus/qbft" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/p2p" ) type consensusFactory struct { - tcpNode host.Host - sender *p2p.Sender - peers []p2p.Peer - p2pKey *k1.PrivateKey - deadliner core.Deadliner - gaterFunc core.DutyGaterFunc - snifferFunc func(*pbv1.SniffedConsensusInstance) + tcpNode host.Host + sender *p2p.Sender + peers []p2p.Peer + p2pKey *k1.PrivateKey + deadliner core.Deadliner + gaterFunc core.DutyGaterFunc + snifferFunc func(*pbv1.SniffedConsensusInstance) + defaultConsensus core.Consensus } -// NewConsensusFactory creates a new consensus factory. +// NewConsensusFactory creates a new consensus factory with the default consensus protocol. func NewConsensusFactory(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, deadliner core.Deadliner, gaterFunc core.DutyGaterFunc, snifferFunc func(*pbv1.SniffedConsensusInstance), -) core.ConsensusFactory { - return &consensusFactory{ - tcpNode: tcpNode, - sender: sender, - peers: peers, - p2pKey: p2pKey, - deadliner: deadliner, - gaterFunc: gaterFunc, - snifferFunc: snifferFunc, +) (core.ConsensusFactory, error) { + defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, deadliner, gaterFunc, snifferFunc) + if err != nil { + return nil, err } + + return &consensusFactory{ + tcpNode: tcpNode, + sender: sender, + peers: peers, + p2pKey: p2pKey, + deadliner: deadliner, + gaterFunc: gaterFunc, + snifferFunc: snifferFunc, + defaultConsensus: defaultConsensus, + }, nil } -// New creates a new consensus instance. -func (f *consensusFactory) New(protocol protocol.ID) (core.Consensus, error) { - // TODO: Refactor to a switch statement when more protocols are added. - if protocol == protocols.QBFTv2ProtocolID { - return qbft.NewConsensus(f.tcpNode, f.sender, f.peers, f.p2pKey, f.deadliner, f.gaterFunc, f.snifferFunc) +// DefaultConsensus returns the default consensus instance. +func (f *consensusFactory) DefaultConsensus() core.Consensus { + return f.defaultConsensus +} + +// ConsensusByProtocolID returns a consensus instance for the specified protocol ID. +func (f *consensusFactory) ConsensusByProtocolID(protocol protocol.ID) (core.Consensus, error) { + if f.defaultConsensus.ProtocolID() == protocol { + return f.defaultConsensus, nil } + // TODO: support for more protocols, add map[protocol.ID]core.Consensus with a lock, etc. + return nil, errors.New("unknown consensus protocol") } diff --git a/core/consensus/factory_test.go b/core/consensus/factory_test.go index 262983e93..de20eb64f 100644 --- a/core/consensus/factory_test.go +++ b/core/consensus/factory_test.go @@ -23,7 +23,7 @@ import ( "github.com/obolnetwork/charon/testutil" ) -func TestNewConsensusFactory(t *testing.T) { +func TestConsensusFactory(t *testing.T) { var hosts []host.Host var peers []p2p.Peer @@ -54,16 +54,21 @@ func TestNewConsensusFactory(t *testing.T) { hosts = append(hosts, h) } - factory := consensus.NewConsensusFactory(hosts[0], new(p2p.Sender), peers, p2pkeys[0], testDeadliner{}, gaterFunc, snifferFunc) + factory, err := consensus.NewConsensusFactory(hosts[0], new(p2p.Sender), peers, p2pkeys[0], testDeadliner{}, gaterFunc, snifferFunc) + require.NoError(t, err) require.NotNil(t, factory) - cons, err := factory.New(protocols.QBFTv2ProtocolID) + defaultConsensus := factory.DefaultConsensus() + require.NotNil(t, defaultConsensus) + require.EqualValues(t, protocols.QBFTv2ProtocolID, defaultConsensus.ProtocolID()) + + sameConsesus, err := factory.ConsensusByProtocolID(protocols.QBFTv2ProtocolID) require.NoError(t, err) - require.NotNil(t, cons) + require.Equal(t, defaultConsensus, sameConsesus) t.Run("unknown protocol", func(t *testing.T) { - _, err := factory.New("unknown") - require.Error(t, err) + _, err = factory.ConsensusByProtocolID("boo") + require.ErrorContains(t, err, "unknown consensus protocol") }) } diff --git a/core/consensus/metrics/metrics_test.go b/core/consensus/metrics/metrics_test.go new file mode 100644 index 000000000..6eb24d658 --- /dev/null +++ b/core/consensus/metrics/metrics_test.go @@ -0,0 +1,72 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package metrics_test + +import ( + "testing" + + "github.com/prometheus/client_golang/prometheus" + pb "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/require" + + "github.com/obolnetwork/charon/app/promauto" + "github.com/obolnetwork/charon/core/consensus/metrics" +) + +func TestConsensusMetrics_SetDecidedRounds(t *testing.T) { + cm := metrics.NewConsensusMetrics("test") + + cm.SetDecidedRounds("duty", "timer", 1) + + m := gatherMetric(t, "core_consensus_decided_rounds") + require.InEpsilon(t, 1, m.GetMetric()[0].GetGauge().GetValue(), 0.0001) +} + +func TestConsensusMetrics_ObserveConsensusDuration(t *testing.T) { + cm := metrics.NewConsensusMetrics("test") + + cm.ObserveConsensusDuration("duty", "timer", 1) + + m := gatherMetric(t, "core_consensus_duration_seconds") + require.EqualValues(t, 1, m.GetMetric()[0].GetHistogram().GetSampleCount()) +} + +func TestConsensusMetrics_IncConsensusTimeout(t *testing.T) { + cm := metrics.NewConsensusMetrics("test") + + cm.IncConsensusTimeout("duty", "timer") + + m := gatherMetric(t, "core_consensus_timeout_total") + require.InEpsilon(t, 1, m.GetMetric()[0].GetCounter().GetValue(), 0.0001) +} + +func TestConsensusMetrics_IncConsensusError(t *testing.T) { + cm := metrics.NewConsensusMetrics("test") + + cm.IncConsensusError() + + m := gatherMetric(t, "core_consensus_error_total") + require.InEpsilon(t, 1, m.GetMetric()[0].GetCounter().GetValue(), 0.0001) +} + +func gatherMetric(t *testing.T, name string) *pb.MetricFamily { + t.Helper() + + labels := prometheus.Labels{} + + registry, err := promauto.NewRegistry(labels) + require.NoError(t, err) + + mfa, err := registry.Gather() + require.NoError(t, err) + + for _, mf := range mfa { + if mf.GetName() == name { + return mf + } + } + + require.Fail(t, "metric not found") + + return nil +} diff --git a/core/consensus/qbft/qbft.go b/core/consensus/qbft/qbft.go index 4e30d40cf..5202f01de 100644 --- a/core/consensus/qbft/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -12,6 +12,7 @@ import ( k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/core/protocol" "go.opentelemetry.io/otel/trace" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" @@ -179,6 +180,11 @@ type Consensus struct { } } +// ProtocolID returns the protocol ID. +func (*Consensus) ProtocolID() protocol.ID { + return protocols.QBFTv2ProtocolID +} + // Subscribe registers a callback for unsigned duty data proposals from leaders. // Note this function is not thread safe, it should be called *before* Start and Propose. func (c *Consensus) Subscribe(fn func(ctx context.Context, duty core.Duty, set core.UnsignedDataSet) error) { @@ -387,7 +393,7 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) def := NewDefinition(len(c.peers), c.subscribers, roundTimer, decideCallback) // Create a new transport that handles sending and receiving for this instance. - t := NewTransport(c, c.privkey, inst.ValueCh, make(chan qbft.Msg[core.Duty, [32]byte]), utils.NewSniffer(int64(def.Nodes), peerIdx)) + t := NewTransport(c, c.privkey, inst.ValueCh, make(chan qbft.Msg[core.Duty, [32]byte]), newSniffer(int64(def.Nodes), peerIdx)) // Provide sniffed buffer to snifferFunc at the end. defer func() { diff --git a/core/consensus/utils/sniffer.go b/core/consensus/qbft/sniffer.go similarity index 74% rename from core/consensus/utils/sniffer.go rename to core/consensus/qbft/sniffer.go index 33243523d..35fac50c9 100644 --- a/core/consensus/utils/sniffer.go +++ b/core/consensus/qbft/sniffer.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package utils +package qbft import ( "sync" @@ -11,17 +11,17 @@ import ( pbv1 "github.com/obolnetwork/charon/core/corepb/v1" ) -// NewSniffer returns a new sniffer. -func NewSniffer(nodes, peerIdx int64) *Sniffer { - return &Sniffer{ +// newSniffer returns a new sniffer. +func newSniffer(nodes, peerIdx int64) *sniffer { + return &sniffer{ nodes: nodes, peerIdx: peerIdx, startedAt: time.Now(), } } -// Sniffer buffers consensus messages. -type Sniffer struct { +// sniffer buffers consensus messages. +type sniffer struct { nodes int64 peerIdx int64 startedAt time.Time @@ -31,7 +31,7 @@ type Sniffer struct { } // Add adds a message to the sniffer buffer. -func (c *Sniffer) Add(msg *pbv1.ConsensusMsg) { +func (c *sniffer) Add(msg *pbv1.ConsensusMsg) { c.mu.Lock() defer c.mu.Unlock() @@ -42,7 +42,7 @@ func (c *Sniffer) Add(msg *pbv1.ConsensusMsg) { } // Instance returns the buffered messages as an instance. -func (c *Sniffer) Instance() *pbv1.SniffedConsensusInstance { +func (c *sniffer) Instance() *pbv1.SniffedConsensusInstance { c.mu.Lock() defer c.mu.Unlock() diff --git a/core/consensus/qbft/sniffer_internal_test.go b/core/consensus/qbft/sniffer_internal_test.go new file mode 100644 index 000000000..8cf46b434 --- /dev/null +++ b/core/consensus/qbft/sniffer_internal_test.go @@ -0,0 +1,29 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package qbft + +import ( + "testing" + + "github.com/stretchr/testify/require" + + pbv1 "github.com/obolnetwork/charon/core/corepb/v1" +) + +func TestSniffer(t *testing.T) { + sniffer := newSniffer(3, 1) + + sniffer.Add(&pbv1.ConsensusMsg{ + Msg: NewRandomMsgForT(t), + }) + sniffer.Add(&pbv1.ConsensusMsg{ + Msg: NewRandomMsgForT(t), + }) + + instance := sniffer.Instance() + + require.EqualValues(t, 3, instance.GetNodes()) + require.EqualValues(t, 1, instance.GetPeerIdx()) + require.NotNil(t, instance.GetStartedAt()) + require.Len(t, instance.GetMsgs(), 2) +} diff --git a/core/consensus/qbft/transport.go b/core/consensus/qbft/transport.go index dd3deea24..cc6ad43d3 100644 --- a/core/consensus/qbft/transport.go +++ b/core/consensus/qbft/transport.go @@ -12,7 +12,6 @@ import ( "github.com/obolnetwork/charon/app/errors" "github.com/obolnetwork/charon/core" - "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/qbft" ) @@ -28,7 +27,7 @@ type Transport struct { broadcaster Broadcaster privkey *k1.PrivateKey recvBuffer chan qbft.Msg[core.Duty, [32]byte] // Instance inner receive buffer. - sniffer *utils.Sniffer + sniffer *sniffer // Mutable state valueMu sync.Mutex @@ -38,7 +37,7 @@ type Transport struct { // NewTransport creates a new qbftTransport. func NewTransport(broadcaster Broadcaster, privkey *k1.PrivateKey, valueCh <-chan proto.Message, - recvBuffer chan qbft.Msg[core.Duty, [32]byte], sniffer *utils.Sniffer, + recvBuffer chan qbft.Msg[core.Duty, [32]byte], sniffer *sniffer, ) *Transport { return &Transport{ broadcaster: broadcaster, diff --git a/core/interfaces.go b/core/interfaces.go index 37a8bdccb..5e8a76d55 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -68,6 +68,9 @@ type DutyDB interface { // Consensus comes to consensus on proposed duty data. type Consensus interface { + // ProtocolID returns the protocol ID of the consensus instance. + ProtocolID() protocol.ID + // Start starts the consensus protocol instance. Start(ctx context.Context) @@ -83,8 +86,15 @@ type Consensus interface { // ConsensusFactory creates new consensus instances. type ConsensusFactory interface { - // New creates a new consensus instance for the specified protocol. - New(protocol protocol.ID) (Consensus, error) + // DefaultConsensus returns the default consensus instance. + // The default consensus must be QBFT v2.0, since it is supported by all charon versions. + // It is used for Priority protocol as well as "the last resort" protocol when no other protocol is selected. + // Multiple calls to DefaultConsensus must return the same instance. + DefaultConsensus() Consensus + + // ConsensusByProtocolID returns a consensus instance for the specified protocol ID. + // The same instance must be returned for the same protocol ID. + ConsensusByProtocolID(protocol protocol.ID) (Consensus, error) } // ValidatorAPI provides a beacon node API to validator clients. It serves duty data from the DutyDB and stores partial signed data in the ParSigDB. From 3d6080a6dde2835168128786e0ad4d6916dbf052 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Fri, 11 Oct 2024 15:35:17 +0300 Subject: [PATCH 09/20] Dynamic consensus protocol selection --- app/app.go | 45 +++++++--- core/consensus/debugger.go | 2 + core/consensus/factory.go | 40 ++++++--- core/consensus/factory_test.go | 42 ++++------ core/consensus/mocks/debugger.go | 41 +++++++++ core/consensus/protocols/protocols.go | 11 +++ core/consensus/protocols/protocols_test.go | 16 ++++ core/consensus/wrapper.go | 69 +++++++++++++++ core/consensus/wrapper_internal_test.go | 53 ++++++++++++ core/deadline.go | 2 + core/interfaces.go | 12 ++- core/mocks/consensus.go | 97 ++++++++++++++++++++++ core/mocks/deadliner.go | 67 +++++++++++++++ go.mod | 1 + go.sum | 2 + 15 files changed, 445 insertions(+), 55 deletions(-) create mode 100644 core/consensus/mocks/debugger.go create mode 100644 core/consensus/wrapper.go create mode 100644 core/consensus/wrapper_internal_test.go create mode 100644 core/mocks/consensus.go create mode 100644 core/mocks/deadliner.go diff --git a/app/app.go b/app/app.go index fc9172bf2..e3751d543 100644 --- a/app/app.go +++ b/app/app.go @@ -47,8 +47,8 @@ import ( "github.com/obolnetwork/charon/core/aggsigdb" "github.com/obolnetwork/charon/core/bcast" "github.com/obolnetwork/charon/core/consensus" - cprotocols "github.com/obolnetwork/charon/core/consensus/protocols" - cqbft "github.com/obolnetwork/charon/core/consensus/qbft" + "github.com/obolnetwork/charon/core/consensus/protocols" + "github.com/obolnetwork/charon/core/consensus/qbft" "github.com/obolnetwork/charon/core/dutydb" "github.com/obolnetwork/charon/core/fetcher" "github.com/obolnetwork/charon/core/infosync" @@ -258,8 +258,6 @@ func Run(ctx context.Context, conf Config) (err error) { wirePeerInfo(life, tcpNode, peerIDs, cluster.GetInitialMutationHash(), sender, conf.BuilderAPI) - consensusDebugger := consensus.NewDebugger() - // seenPubkeys channel to send seen public keys from validatorapi to monitoringapi. seenPubkeys := make(chan core.PubKey) seenPubkeysFunc := func(pk core.PubKey) { @@ -282,6 +280,8 @@ func Run(ctx context.Context, conf Config) (err error) { return err } + consensusDebugger := consensus.NewDebugger() + wireMonitoringAPI(ctx, life, conf.MonitoringAddr, conf.DebugAddr, tcpNode, eth2Cl, peerIDs, promRegistry, consensusDebugger, pubkeys, seenPubkeys, vapiCalls, len(cluster.GetValidators())) @@ -527,18 +527,18 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, retryer := retry.New[core.Duty](deadlineFunc) - consensusFactory, err := consensus.NewConsensusFactory( - tcpNode, sender, peers, p2pKey, - deadlinerFunc("consensus"), gaterFunc, consensusDebugger.AddInstance) + consensusFactory, err := consensus.NewConsensusFactory(tcpNode, sender, peers, p2pKey, deadlinerFunc, gaterFunc, consensusDebugger) if err != nil { return err } defaultConsensus := consensusFactory.DefaultConsensus() - startDefaultConsensus := lifecycle.HookFuncCtx(defaultConsensus.Start) + coreConsensus := consensusFactory.CurrentConsensus() + startConsensus := lifecycle.HookFuncCtx(defaultConsensus.Start) + // Priority protocol always uses QBFTv2. err = wirePrioritise(ctx, conf, life, tcpNode, peerIDs, int(cluster.GetThreshold()), - sender.SendReceive, defaultConsensus, sched, p2pKey, deadlineFunc) + sender.SendReceive, defaultConsensus, sched, p2pKey, deadlineFunc, consensusFactory) if err != nil { return err } @@ -558,12 +558,13 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, return err } + // Core always uses the "current" consensus that is changed dynamically. opts := []core.WireOption{ core.WithTracing(), core.WithTracking(track, inclusion), core.WithAsyncRetry(retryer), } - core.Wire(sched, fetch, defaultConsensus, dutyDB, vapi, parSigDB, parSigEx, sigAgg, aggSigDB, broadcaster, opts...) + core.Wire(sched, fetch, coreConsensus, dutyDB, vapi, parSigDB, parSigEx, sigAgg, aggSigDB, broadcaster, opts...) err = wireValidatorMock(ctx, conf, eth2Cl, pubshares, sched) if err != nil { @@ -575,7 +576,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, } life.RegisterStart(lifecycle.AsyncBackground, lifecycle.StartScheduler, lifecycle.HookFuncErr(sched.Run)) - life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startDefaultConsensus) + life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startConsensus) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartAggSigDB, lifecycle.HookFuncCtx(aggSigDB.Run)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartParSigDB, lifecycle.HookFuncCtx(parSigDB.Trim)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartTracker, lifecycle.HookFuncCtx(inclusion.Run)) @@ -590,8 +591,9 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, tcpNode host.Host, peers []peer.ID, threshold int, sendFunc p2p.SendReceiveFunc, coreCons core.Consensus, sched core.Scheduler, p2pKey *k1.PrivateKey, deadlineFunc func(duty core.Duty) (time.Time, bool), + consensusFactory core.ConsensusFactory, ) error { - cons, ok := coreCons.(*cqbft.Consensus) + cons, ok := coreCons.(*qbft.Consensus) if !ok { // Priority protocol not supported for leader cast. return nil @@ -626,6 +628,23 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t prio.Subscribe(conf.TestConfig.PrioritiseCallback) } + prio.Subscribe(func(ctx context.Context, _ core.Duty, tr []priority.TopicResult) error { + for _, t := range tr { + if t.Topic == infosync.TopicProtocol { + allProtocols := t.PrioritiesOnly() + preferredConsensusProtocol := protocols.MostPreferredConsensusProtocol(allProtocols) + + if err := consensusFactory.SetCurrentConsensusForProtocol(protocol.ID(preferredConsensusProtocol)); err != nil { + log.Error(ctx, "Failed to set current consensus for protocol", err, z.Str("protocol", preferredConsensusProtocol)) + } else { + log.Info(ctx, "Set current consensus for protocol", z.Str("protocol", preferredConsensusProtocol)) + } + } + } + + return nil + }) + life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartPeerInfo, lifecycle.HookFuncCtx(prio.Start)) return nil @@ -1066,7 +1085,7 @@ func (h httpServeHook) Call(context.Context) error { // Protocols returns the list of supported Protocols in order of precedence. func Protocols() []protocol.ID { var resp []protocol.ID - resp = append(resp, cprotocols.Protocols()...) + resp = append(resp, protocols.Protocols()...) resp = append(resp, parsigex.Protocols()...) resp = append(resp, peerinfo.Protocols()...) resp = append(resp, priority.Protocols()...) diff --git a/core/consensus/debugger.go b/core/consensus/debugger.go index 83678a547..09bb477da 100644 --- a/core/consensus/debugger.go +++ b/core/consensus/debugger.go @@ -16,6 +16,8 @@ import ( pbv1 "github.com/obolnetwork/charon/core/corepb/v1" ) +//go:generate mockery --name=Debugger --output=mocks --outpkg=mocks --case=underscore + const maxDebuggerBuffer = 50 * (1 << 20) // 50 MB. // Debugger is an interface for debugging consensus messages. diff --git a/core/consensus/factory.go b/core/consensus/factory.go index 07cdbff25..29c38a3d8 100644 --- a/core/consensus/factory.go +++ b/core/consensus/factory.go @@ -9,27 +9,31 @@ import ( "github.com/obolnetwork/charon/app/errors" "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/protocols" "github.com/obolnetwork/charon/core/consensus/qbft" - pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/p2p" ) +type DeadlinerFunc func(label string) core.Deadliner + type consensusFactory struct { tcpNode host.Host sender *p2p.Sender peers []p2p.Peer p2pKey *k1.PrivateKey - deadliner core.Deadliner + deadlinerFunc DeadlinerFunc gaterFunc core.DutyGaterFunc - snifferFunc func(*pbv1.SniffedConsensusInstance) + debugger Debugger defaultConsensus core.Consensus + wrappedConsensus *consensusWrapper } // NewConsensusFactory creates a new consensus factory with the default consensus protocol. func NewConsensusFactory(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, - deadliner core.Deadliner, gaterFunc core.DutyGaterFunc, snifferFunc func(*pbv1.SniffedConsensusInstance), + deadlinerFunc DeadlinerFunc, gaterFunc core.DutyGaterFunc, debugger Debugger, ) (core.ConsensusFactory, error) { - defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, deadliner, gaterFunc, snifferFunc) + qbftDeadliner := deadlinerFunc("consensus.qbft") + defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, qbftDeadliner, gaterFunc, debugger.AddInstance) if err != nil { return nil, err } @@ -39,10 +43,11 @@ func NewConsensusFactory(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer sender: sender, peers: peers, p2pKey: p2pKey, - deadliner: deadliner, + deadlinerFunc: deadlinerFunc, gaterFunc: gaterFunc, - snifferFunc: snifferFunc, + debugger: debugger, defaultConsensus: defaultConsensus, + wrappedConsensus: newConsensusWrapper(defaultConsensus), }, nil } @@ -51,13 +56,22 @@ func (f *consensusFactory) DefaultConsensus() core.Consensus { return f.defaultConsensus } -// ConsensusByProtocolID returns a consensus instance for the specified protocol ID. -func (f *consensusFactory) ConsensusByProtocolID(protocol protocol.ID) (core.Consensus, error) { - if f.defaultConsensus.ProtocolID() == protocol { - return f.defaultConsensus, nil +// CurrentConsensus returns the current consensus instance. +func (f *consensusFactory) CurrentConsensus() core.Consensus { + return f.wrappedConsensus +} + +// SetCurrentConsensusForProtocol sets the current consensus instance for the given protocol id. +func (f *consensusFactory) SetCurrentConsensusForProtocol(protocol protocol.ID) error { + if f.wrappedConsensus.ProtocolID() == protocol { + return nil } - // TODO: support for more protocols, add map[protocol.ID]core.Consensus with a lock, etc. + if protocol == protocols.QBFTv2ProtocolID { + f.wrappedConsensus.SetImpl(f.defaultConsensus) + + return nil + } - return nil, errors.New("unknown consensus protocol") + return errors.New("unsupported protocol id") } diff --git a/core/consensus/factory_test.go b/core/consensus/factory_test.go index de20eb64f..6e5477d4d 100644 --- a/core/consensus/factory_test.go +++ b/core/consensus/factory_test.go @@ -16,8 +16,9 @@ import ( "github.com/obolnetwork/charon/cluster" "github.com/obolnetwork/charon/core" "github.com/obolnetwork/charon/core/consensus" + csmocks "github.com/obolnetwork/charon/core/consensus/mocks" "github.com/obolnetwork/charon/core/consensus/protocols" - pbv1 "github.com/obolnetwork/charon/core/corepb/v1" + coremocks "github.com/obolnetwork/charon/core/mocks" "github.com/obolnetwork/charon/eth2util/enr" "github.com/obolnetwork/charon/p2p" "github.com/obolnetwork/charon/testutil" @@ -31,7 +32,6 @@ func TestConsensusFactory(t *testing.T) { random := rand.New(rand.NewSource(int64(seed))) lock, p2pkeys, _ := cluster.NewForT(t, 1, 3, 3, seed, random) - snifferFunc := func(msgs *pbv1.SniffedConsensusInstance) {} gaterFunc := func(core.Duty) bool { return true } for i := range 3 { @@ -54,33 +54,23 @@ func TestConsensusFactory(t *testing.T) { hosts = append(hosts, h) } - factory, err := consensus.NewConsensusFactory(hosts[0], new(p2p.Sender), peers, p2pkeys[0], testDeadliner{}, gaterFunc, snifferFunc) + deadlinerFunc := func(string) core.Deadliner { + return coremocks.NewDeadliner(t) + } + debugger := csmocks.NewDebugger(t) + factory, err := consensus.NewConsensusFactory(hosts[0], new(p2p.Sender), peers, p2pkeys[0], deadlinerFunc, gaterFunc, debugger) require.NoError(t, err) require.NotNil(t, factory) - defaultConsensus := factory.DefaultConsensus() - require.NotNil(t, defaultConsensus) - require.EqualValues(t, protocols.QBFTv2ProtocolID, defaultConsensus.ProtocolID()) - - sameConsesus, err := factory.ConsensusByProtocolID(protocols.QBFTv2ProtocolID) - require.NoError(t, err) - require.Equal(t, defaultConsensus, sameConsesus) - - t.Run("unknown protocol", func(t *testing.T) { - _, err = factory.ConsensusByProtocolID("boo") - require.ErrorContains(t, err, "unknown consensus protocol") + t.Run("default and current consensus", func(t *testing.T) { + defaultConsensus := factory.DefaultConsensus() + require.NotNil(t, defaultConsensus) + require.EqualValues(t, protocols.QBFTv2ProtocolID, defaultConsensus.ProtocolID()) + require.NotEqual(t, defaultConsensus, factory.CurrentConsensus()) // because the current is wrapped }) -} -// testDeadliner is a mock deadliner implementation. -type testDeadliner struct { - deadlineChan chan core.Duty -} - -func (testDeadliner) Add(core.Duty) bool { - return true -} - -func (t testDeadliner) C() <-chan core.Duty { - return t.deadlineChan + t.Run("unsupported protocol id", func(t *testing.T) { + err := factory.SetCurrentConsensusForProtocol("boo") + require.ErrorContains(t, err, "unsupported protocol id") + }) } diff --git a/core/consensus/mocks/debugger.go b/core/consensus/mocks/debugger.go new file mode 100644 index 000000000..f3cca1213 --- /dev/null +++ b/core/consensus/mocks/debugger.go @@ -0,0 +1,41 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + http "net/http" + + v1 "github.com/obolnetwork/charon/core/corepb/v1" + mock "github.com/stretchr/testify/mock" +) + +// Debugger is an autogenerated mock type for the Debugger type +type Debugger struct { + mock.Mock +} + +// AddInstance provides a mock function with given fields: instance +func (_m *Debugger) AddInstance(instance *v1.SniffedConsensusInstance) { + _m.Called(instance) +} + +// ServeHTTP provides a mock function with given fields: _a0, _a1 +func (_m *Debugger) ServeHTTP(_a0 http.ResponseWriter, _a1 *http.Request) { + _m.Called(_a0, _a1) +} + +// NewDebugger creates a new instance of Debugger. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewDebugger(t interface { + mock.TestingT + Cleanup(func()) +}) *Debugger { + mock := &Debugger{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/core/consensus/protocols/protocols.go b/core/consensus/protocols/protocols.go index 8a8fb807d..ebe9869ea 100644 --- a/core/consensus/protocols/protocols.go +++ b/core/consensus/protocols/protocols.go @@ -19,6 +19,17 @@ func Protocols() []protocol.ID { return []protocol.ID{QBFTv2ProtocolID} } +// MostPreferredConsensusProtocol returns the most preferred consensus protocol from the given list. +func MostPreferredConsensusProtocol(protocols []string) string { + for _, p := range protocols { + if strings.HasPrefix(p, protocolIDPrefix) { + return p + } + } + + return QBFTv2ProtocolID +} + // IsSupportedProtocolName returns true if the protocol name is supported. func IsSupportedProtocolName(name string) bool { for _, p := range Protocols() { diff --git a/core/consensus/protocols/protocols_test.go b/core/consensus/protocols/protocols_test.go index fa6ce4994..4df1291b3 100644 --- a/core/consensus/protocols/protocols_test.go +++ b/core/consensus/protocols/protocols_test.go @@ -21,3 +21,19 @@ func TestProtocols(t *testing.T) { protocols.QBFTv2ProtocolID, }, protocols.Protocols()) } + +func TestMostPreferredConsensusProtocol(t *testing.T) { + t.Run("default is qbft", func(t *testing.T) { + require.Equal(t, protocols.QBFTv2ProtocolID, protocols.MostPreferredConsensusProtocol([]string{"unreal"})) + require.Equal(t, protocols.QBFTv2ProtocolID, protocols.MostPreferredConsensusProtocol([]string{})) + }) + + t.Run("latest abft is preferred", func(t *testing.T) { + pp := []string{ + "/charon/consensus/abft/3.0.0", + "/charon/consensus/abft/1.0.0", + "/charon/consensus/qbft/1.0.0", + } + require.Equal(t, "/charon/consensus/abft/3.0.0", protocols.MostPreferredConsensusProtocol(pp)) + }) +} diff --git a/core/consensus/wrapper.go b/core/consensus/wrapper.go new file mode 100644 index 000000000..f5aa66a5e --- /dev/null +++ b/core/consensus/wrapper.go @@ -0,0 +1,69 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + "context" + "sync" + + "github.com/libp2p/go-libp2p/core/protocol" + + "github.com/obolnetwork/charon/core" +) + +type consensusWrapper struct { + lock sync.RWMutex + impl core.Consensus +} + +var _ core.Consensus = (*consensusWrapper)(nil) + +// newConsensusWrapper wraps a core.Consensus implementation. +func newConsensusWrapper(impl core.Consensus) *consensusWrapper { + return &consensusWrapper{ + impl: impl, + } +} + +// SetImpl sets the core.Consensus implementation. +func (w *consensusWrapper) SetImpl(impl core.Consensus) { + w.lock.Lock() + defer w.lock.Unlock() + + w.impl = impl +} + +func (w *consensusWrapper) ProtocolID() protocol.ID { + w.lock.RLock() + defer w.lock.RUnlock() + + return w.impl.ProtocolID() +} + +func (w *consensusWrapper) Start(ctx context.Context) { + w.lock.RLock() + defer w.lock.RUnlock() + + w.impl.Start(ctx) +} + +func (w *consensusWrapper) Participate(ctx context.Context, duty core.Duty) error { + w.lock.RLock() + defer w.lock.RUnlock() + + return w.impl.Participate(ctx, duty) +} + +func (w *consensusWrapper) Propose(ctx context.Context, duty core.Duty, dataSet core.UnsignedDataSet) error { + w.lock.RLock() + defer w.lock.RUnlock() + + return w.impl.Propose(ctx, duty, dataSet) +} + +func (w *consensusWrapper) Subscribe(fn func(context.Context, core.Duty, core.UnsignedDataSet) error) { + w.lock.RLock() + defer w.lock.RUnlock() + + w.impl.Subscribe(fn) +} diff --git a/core/consensus/wrapper_internal_test.go b/core/consensus/wrapper_internal_test.go new file mode 100644 index 000000000..d95b781ab --- /dev/null +++ b/core/consensus/wrapper_internal_test.go @@ -0,0 +1,53 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + "context" + "testing" + + "github.com/libp2p/go-libp2p/core/protocol" + "github.com/stretchr/testify/require" + "github.com/test-go/testify/mock" + + "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/protocols" + "github.com/obolnetwork/charon/core/mocks" +) + +func TestNewConsensusWrapper(t *testing.T) { + ctx := context.Background() + randaoDuty := core.NewRandaoDuty(123) + dataSet := core.UnsignedDataSet{} + + impl := mocks.NewConsensus(t) + impl.On("ProtocolID").Return(protocol.ID(protocols.QBFTv2ProtocolID)) + impl.On("Start", ctx).Return() + impl.On("Participate", ctx, randaoDuty).Return(nil) + impl.On("Propose", ctx, randaoDuty, dataSet).Return(nil) + impl.On("Subscribe", mock.Anything).Return() + + wrapped := newConsensusWrapper(impl) + require.NotNil(t, wrapped) + + require.EqualValues(t, protocols.QBFTv2ProtocolID, wrapped.ProtocolID()) + + wrapped.Start(ctx) + + err := wrapped.Participate(ctx, randaoDuty) + require.NoError(t, err) + + err = wrapped.Propose(ctx, randaoDuty, dataSet) + require.NoError(t, err) + + wrapped.Subscribe(func(ctx context.Context, d core.Duty, uds core.UnsignedDataSet) error { + return nil + }) + + impl2 := mocks.NewConsensus(t) + impl2.On("ProtocolID").Return(protocol.ID("foobar")) + + wrapped.SetImpl(impl2) + + require.EqualValues(t, "foobar", wrapped.ProtocolID()) +} diff --git a/core/deadline.go b/core/deadline.go index ef357dde4..2e178a0f8 100644 --- a/core/deadline.go +++ b/core/deadline.go @@ -16,6 +16,8 @@ import ( "github.com/obolnetwork/charon/app/z" ) +//go:generate mockery --name=Deadliner --output=mocks --outpkg=mocks --case=underscore + // lateFactor defines the number of slots duties may be late. // See https://pintail.xyz/posts/modelling-the-impact-of-altair/#proposer-and-delay-rewards. const lateFactor = 5 diff --git a/core/interfaces.go b/core/interfaces.go index 5e8a76d55..23c082fb8 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -11,6 +11,8 @@ import ( "github.com/libp2p/go-libp2p/core/protocol" ) +//go:generate mockery --name=Consensus --output=mocks --outpkg=mocks --case=underscore + // Scheduler triggers the start of a duty workflow. type Scheduler interface { // SubscribeDuties subscribes a callback function for triggered duties. @@ -92,9 +94,13 @@ type ConsensusFactory interface { // Multiple calls to DefaultConsensus must return the same instance. DefaultConsensus() Consensus - // ConsensusByProtocolID returns a consensus instance for the specified protocol ID. - // The same instance must be returned for the same protocol ID. - ConsensusByProtocolID(protocol protocol.ID) (Consensus, error) + // CurrentConsensus returns currently selected consensus instance. + // The instance is selected by the Priority protocol and can be changed by SetCurrentConsensusForProtocol(). + // Before SetCurrentConsensusForProtocol() is called, CurrentConsensus() must return DefaultConsensus(). + CurrentConsensus() Consensus + + // SetCurrentConsensusForProtocol handles Priority protocol outcome and changes the CurrentConsensus() accordingly. + SetCurrentConsensusForProtocol(protocol protocol.ID) error } // ValidatorAPI provides a beacon node API to validator clients. It serves duty data from the DutyDB and stores partial signed data in the ParSigDB. diff --git a/core/mocks/consensus.go b/core/mocks/consensus.go new file mode 100644 index 000000000..0d92699d3 --- /dev/null +++ b/core/mocks/consensus.go @@ -0,0 +1,97 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + core "github.com/obolnetwork/charon/core" + mock "github.com/stretchr/testify/mock" + + protocol "github.com/libp2p/go-libp2p/core/protocol" +) + +// Consensus is an autogenerated mock type for the Consensus type +type Consensus struct { + mock.Mock +} + +// Participate provides a mock function with given fields: _a0, _a1 +func (_m *Consensus) Participate(_a0 context.Context, _a1 core.Duty) error { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for Participate") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, core.Duty) error); ok { + r0 = rf(_a0, _a1) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// Propose provides a mock function with given fields: _a0, _a1, _a2 +func (_m *Consensus) Propose(_a0 context.Context, _a1 core.Duty, _a2 core.UnsignedDataSet) error { + ret := _m.Called(_a0, _a1, _a2) + + if len(ret) == 0 { + panic("no return value specified for Propose") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, core.Duty, core.UnsignedDataSet) error); ok { + r0 = rf(_a0, _a1, _a2) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// ProtocolID provides a mock function with given fields: +func (_m *Consensus) ProtocolID() protocol.ID { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for ProtocolID") + } + + var r0 protocol.ID + if rf, ok := ret.Get(0).(func() protocol.ID); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(protocol.ID) + } + + return r0 +} + +// Start provides a mock function with given fields: ctx +func (_m *Consensus) Start(ctx context.Context) { + _m.Called(ctx) +} + +// Subscribe provides a mock function with given fields: _a0 +func (_m *Consensus) Subscribe(_a0 func(context.Context, core.Duty, core.UnsignedDataSet) error) { + _m.Called(_a0) +} + +// NewConsensus creates a new instance of Consensus. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewConsensus(t interface { + mock.TestingT + Cleanup(func()) +}) *Consensus { + mock := &Consensus{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/core/mocks/deadliner.go b/core/mocks/deadliner.go new file mode 100644 index 000000000..5c249064b --- /dev/null +++ b/core/mocks/deadliner.go @@ -0,0 +1,67 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + core "github.com/obolnetwork/charon/core" + mock "github.com/stretchr/testify/mock" +) + +// Deadliner is an autogenerated mock type for the Deadliner type +type Deadliner struct { + mock.Mock +} + +// Add provides a mock function with given fields: duty +func (_m *Deadliner) Add(duty core.Duty) bool { + ret := _m.Called(duty) + + if len(ret) == 0 { + panic("no return value specified for Add") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(core.Duty) bool); ok { + r0 = rf(duty) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// C provides a mock function with given fields: +func (_m *Deadliner) C() <-chan core.Duty { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for C") + } + + var r0 <-chan core.Duty + if rf, ok := ret.Get(0).(func() <-chan core.Duty); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(<-chan core.Duty) + } + } + + return r0 +} + +// NewDeadliner creates a new instance of Deadliner. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewDeadliner(t interface { + mock.TestingT + Cleanup(func()) +}) *Deadliner { + mock := &Deadliner{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/go.mod b/go.mod index 734edf787..88f7f881f 100644 --- a/go.mod +++ b/go.mod @@ -31,6 +31,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/spf13/viper v1.19.0 github.com/stretchr/testify v1.9.0 + github.com/test-go/testify v1.1.4 github.com/wealdtech/go-eth2-wallet-encryptor-keystorev4 v1.4.1 go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.55.0 go.opentelemetry.io/otel v1.30.0 diff --git a/go.sum b/go.sum index ed9f554e8..e61e3f81e 100644 --- a/go.sum +++ b/go.sum @@ -529,6 +529,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/subosito/gotenv v1.6.0 h1:9NlTDc1FTs4qu0DDq7AEtTPNw6SVm7uBMsUCUjABIf8= github.com/subosito/gotenv v1.6.0/go.mod h1:Dk4QP5c2W3ibzajGcXpNraDfq2IrhjMIvMSWPKKo0FU= github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA= +github.com/test-go/testify v1.1.4 h1:Tf9lntrKUMHiXQ07qBScBTSA0dhYQlu83hswqelv1iE= +github.com/test-go/testify v1.1.4/go.mod h1:rH7cfJo/47vWGdi4GPj16x3/t1xGOj2YxzmNQzk2ghU= github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/vbatts/tar-split v0.11.5 h1:3bHCTIheBm1qFTcgh9oPu+nNBtX+XJIupG/vacinCts= github.com/vbatts/tar-split v0.11.5/go.mod h1:yZbwRsSeGjusneWgA781EKej9HF8vme8okylkAeNKLk= From cc24de6fc4adbebfff03713ef5ee7425a33fc5c3 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Fri, 11 Oct 2024 16:14:22 +0300 Subject: [PATCH 10/20] Consensus protocol runtime selection --- app/app.go | 21 +++- cluster/manifest/mutationlegacylock.go | 13 +-- cluster/manifestpb/v1/manifest.pb.go | 115 +++++++++++---------- cluster/manifestpb/v1/manifest.proto | 1 + cmd/run.go | 1 + core/consensus/factory.go | 3 +- core/consensus/protocols/protocols.go | 17 +++ core/consensus/protocols/protocols_test.go | 17 +++ docs/configuration.md | 1 + 9 files changed, 126 insertions(+), 63 deletions(-) diff --git a/app/app.go b/app/app.go index e3751d543..6b73d3612 100644 --- a/app/app.go +++ b/app/app.go @@ -93,6 +93,7 @@ type Config struct { SimnetBMockFuzz bool TestnetConfig eth2util.Network ProcDirectory string + ConsensusProtocol string TestConfig TestConfig } @@ -538,7 +539,8 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, // Priority protocol always uses QBFTv2. err = wirePrioritise(ctx, conf, life, tcpNode, peerIDs, int(cluster.GetThreshold()), - sender.SendReceive, defaultConsensus, sched, p2pKey, deadlineFunc, consensusFactory) + sender.SendReceive, defaultConsensus, sched, p2pKey, deadlineFunc, + consensusFactory, cluster.GetConsensusProtocol()) if err != nil { return err } @@ -591,7 +593,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, tcpNode host.Host, peers []peer.ID, threshold int, sendFunc p2p.SendReceiveFunc, coreCons core.Consensus, sched core.Scheduler, p2pKey *k1.PrivateKey, deadlineFunc func(duty core.Duty) (time.Time, bool), - consensusFactory core.ConsensusFactory, + consensusFactory core.ConsensusFactory, clusterPreferredProtocol string, ) error { cons, ok := coreCons.(*qbft.Consensus) if !ok { @@ -609,9 +611,22 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t return err } + // The initial protocols order as defined by implementation is altered by: + // 1. Bumping the cluster (lock) preferred protocol to the top. + // 2. Bumping the protocol specified by CLI flag (cluster run) to the top. + // In all cases this bumps all versions of the protocol identified by name. + // The order of all these operations are important. + allProtocols := Protocols() + if clusterPreferredProtocol != "" { + allProtocols = protocols.BumpProtocolsByName(clusterPreferredProtocol, allProtocols) + } + if conf.ConsensusProtocol != "" { + allProtocols = protocols.BumpProtocolsByName(conf.ConsensusProtocol, allProtocols) + } + isync := infosync.New(prio, version.Supported(), - Protocols(), + allProtocols, ProposalTypes(conf.BuilderAPI, conf.SyntheticBlockProposals), ) diff --git a/cluster/manifest/mutationlegacylock.go b/cluster/manifest/mutationlegacylock.go index ab97b357b..21b25f025 100644 --- a/cluster/manifest/mutationlegacylock.go +++ b/cluster/manifest/mutationlegacylock.go @@ -145,12 +145,13 @@ func transformLegacyLock(input *manifestpb.Cluster, signed *manifestpb.SignedMut } return &manifestpb.Cluster{ - Name: lock.Name, - Threshold: int32(lock.Threshold), - DkgAlgorithm: lock.DKGAlgorithm, - ForkVersion: lock.ForkVersion, - Validators: vals, - Operators: ops, + Name: lock.Name, + Threshold: int32(lock.Threshold), + DkgAlgorithm: lock.DKGAlgorithm, + ForkVersion: lock.ForkVersion, + ConsensusProtocol: lock.ConsensusProtocol, + Validators: vals, + Operators: ops, }, nil } diff --git a/cluster/manifestpb/v1/manifest.pb.go b/cluster/manifestpb/v1/manifest.pb.go index cec646a99..bc755036f 100644 --- a/cluster/manifestpb/v1/manifest.pb.go +++ b/cluster/manifestpb/v1/manifest.pb.go @@ -35,6 +35,7 @@ type Cluster struct { ForkVersion []byte `protobuf:"bytes,6,opt,name=fork_version,json=forkVersion,proto3" json:"fork_version,omitempty"` // ForkVersion is the fork version (network/chain) of the cluster. It must be 4 bytes. Operators []*Operator `protobuf:"bytes,7,rep,name=operators,proto3" json:"operators,omitempty"` // Operators is the list of operators of the cluster. Validators []*Validator `protobuf:"bytes,8,rep,name=validators,proto3" json:"validators,omitempty"` // Validators is the list of validators of the cluster. + ConsensusProtocol string `protobuf:"bytes,9,opt,name=consensus_protocol,json=consensusProtocol,proto3" json:"consensus_protocol,omitempty"` // ConsensusProtocol is the consensus protocol name preferred by the cluster, e.g. "abft". } func (x *Cluster) Reset() { @@ -123,6 +124,13 @@ func (x *Cluster) GetValidators() []*Validator { return nil } +func (x *Cluster) GetConsensusProtocol() string { + if x != nil { + return x.ConsensusProtocol + } + return "" +} + // Mutation mutates the cluster manifest. type Mutation struct { state protoimpl.MessageState @@ -562,7 +570,7 @@ var file_cluster_manifestpb_v1_manifest_proto_rawDesc = []byte{ 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, - 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xea, 0x02, 0x0a, 0x07, 0x43, 0x6c, 0x75, + 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x99, 0x03, 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x32, 0x0a, 0x15, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x6d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x13, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x4d, 0x75, 0x74, 0x61, @@ -585,57 +593,60 @@ var file_cluster_manifestpb_v1_manifest_proto_rawDesc = []byte{ 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x6f, 0x72, 0x73, 0x22, 0x60, 0x0a, 0x08, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, - 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, - 0x79, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x83, 0x01, 0x0a, 0x0e, 0x53, 0x69, 0x67, 0x6e, - 0x65, 0x64, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x75, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x63, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, - 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x6d, - 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x69, 0x67, 0x6e, 0x65, - 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x12, - 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x59, 0x0a, - 0x12, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, - 0x69, 0x73, 0x74, 0x12, 0x43, 0x0a, 0x09, 0x6d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, - 0x2e, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x53, - 0x69, 0x67, 0x6e, 0x65, 0x64, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6d, - 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x36, 0x0a, 0x08, 0x4f, 0x70, 0x65, 0x72, - 0x61, 0x74, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x10, - 0x0a, 0x03, 0x65, 0x6e, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x65, 0x6e, 0x72, - 0x22, 0xe8, 0x01, 0x0a, 0x09, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x1d, - 0x0a, 0x0a, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x09, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x4b, 0x65, 0x79, 0x12, 0x1d, 0x0a, - 0x0a, 0x70, 0x75, 0x62, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0c, 0x52, 0x09, 0x70, 0x75, 0x62, 0x53, 0x68, 0x61, 0x72, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x15, - 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x64, - 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x66, 0x65, 0x65, - 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, - 0x12, 0x2d, 0x0a, 0x12, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x61, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x77, 0x69, - 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, - 0x3a, 0x0a, 0x19, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x67, 0x69, 0x73, - 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x17, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, - 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4a, 0x73, 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x0d, 0x56, - 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x0a, - 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x20, 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x6d, 0x61, 0x6e, 0x69, 0x66, - 0x65, 0x73, 0x74, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x6f, 0x72, 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x22, 0x20, - 0x0a, 0x0a, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x4c, 0x6f, 0x63, 0x6b, 0x12, 0x12, 0x0a, 0x04, - 0x6a, 0x73, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x6a, 0x73, 0x6f, 0x6e, - 0x22, 0x07, 0x0a, 0x05, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, - 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6f, 0x62, 0x6f, 0x6c, 0x6e, 0x65, 0x74, 0x77, - 0x6f, 0x72, 0x6b, 0x2f, 0x63, 0x68, 0x61, 0x72, 0x6f, 0x6e, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x2f, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, 0x62, 0x2f, 0x76, 0x31, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, + 0x75, 0x73, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x63, 0x6f, 0x6c, 0x22, 0x60, 0x0a, 0x08, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x16, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, + 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x83, 0x01, 0x0a, 0x0e, 0x53, 0x69, 0x67, 0x6e, 0x65, + 0x64, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x75, 0x74, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x63, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, 0x62, + 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x6d, 0x75, + 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x12, 0x1c, + 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x59, 0x0a, 0x12, + 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, + 0x73, 0x74, 0x12, 0x43, 0x0a, 0x09, 0x6d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, + 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, + 0x67, 0x6e, 0x65, 0x64, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6d, 0x75, + 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x36, 0x0a, 0x08, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x10, 0x0a, + 0x03, 0x65, 0x6e, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x65, 0x6e, 0x72, 0x22, + 0xe8, 0x01, 0x0a, 0x09, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x1d, 0x0a, + 0x0a, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x09, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x4b, 0x65, 0x79, 0x12, 0x1d, 0x0a, 0x0a, + 0x70, 0x75, 0x62, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, + 0x52, 0x09, 0x70, 0x75, 0x62, 0x53, 0x68, 0x61, 0x72, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x66, + 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x64, 0x64, + 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x66, 0x65, 0x65, 0x52, + 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, + 0x2d, 0x0a, 0x12, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x61, 0x64, + 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x77, 0x69, 0x74, + 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x3a, + 0x0a, 0x19, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x17, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4a, 0x73, 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x0d, 0x56, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x0a, 0x76, + 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, + 0x73, 0x74, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, + 0x72, 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x22, 0x20, 0x0a, + 0x0a, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x4c, 0x6f, 0x63, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x6a, + 0x73, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x6a, 0x73, 0x6f, 0x6e, 0x22, + 0x07, 0x0a, 0x05, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, + 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6f, 0x62, 0x6f, 0x6c, 0x6e, 0x65, 0x74, 0x77, 0x6f, + 0x72, 0x6b, 0x2f, 0x63, 0x68, 0x61, 0x72, 0x6f, 0x6e, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x2f, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x70, 0x62, 0x2f, 0x76, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/cluster/manifestpb/v1/manifest.proto b/cluster/manifestpb/v1/manifest.proto index eac907a68..2ccea05fb 100644 --- a/cluster/manifestpb/v1/manifest.proto +++ b/cluster/manifestpb/v1/manifest.proto @@ -16,6 +16,7 @@ message Cluster { bytes fork_version = 6; // ForkVersion is the fork version (network/chain) of the cluster. It must be 4 bytes. repeated Operator operators = 7; // Operators is the list of operators of the cluster. repeated Validator validators = 8; // Validators is the list of validators of the cluster. + string consensus_protocol = 9; // ConsensusProtocol is the consensus protocol name preferred by the cluster, e.g. "abft". } // Mutation mutates the cluster manifest. diff --git a/cmd/run.go b/cmd/run.go index 1bc174f05..0aaab81ed 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -92,6 +92,7 @@ func bindRunFlags(cmd *cobra.Command, config *app.Config) { cmd.Flags().Int64Var(&config.TestnetConfig.GenesisTimestamp, "testnet-genesis-timestamp", 0, "Genesis timestamp of the custom test network.") cmd.Flags().StringVar(&config.TestnetConfig.CapellaHardFork, "testnet-capella-hard-fork", "", "Capella hard fork version of the custom test network.") cmd.Flags().StringVar(&config.ProcDirectory, "proc-directory", "", "Directory to look into in order to detect other stack components running on the host.") + cmd.Flags().StringVar(&config.ConsensusProtocol, "consensus-protocol", "", "Preferred consensus protocol name for the cluster. Selected automatically when not specified.") wrapPreRunE(cmd, func(*cobra.Command, []string) error { if len(config.BeaconNodeAddrs) == 0 && !config.SimnetBMock { diff --git a/core/consensus/factory.go b/core/consensus/factory.go index 29c38a3d8..e550b9344 100644 --- a/core/consensus/factory.go +++ b/core/consensus/factory.go @@ -9,7 +9,6 @@ import ( "github.com/obolnetwork/charon/app/errors" "github.com/obolnetwork/charon/core" - "github.com/obolnetwork/charon/core/consensus/protocols" "github.com/obolnetwork/charon/core/consensus/qbft" "github.com/obolnetwork/charon/p2p" ) @@ -67,7 +66,7 @@ func (f *consensusFactory) SetCurrentConsensusForProtocol(protocol protocol.ID) return nil } - if protocol == protocols.QBFTv2ProtocolID { + if protocol == f.defaultConsensus.ProtocolID() { f.wrappedConsensus.SetImpl(f.defaultConsensus) return nil diff --git a/core/consensus/protocols/protocols.go b/core/consensus/protocols/protocols.go index ebe9869ea..8cf493b5d 100644 --- a/core/consensus/protocols/protocols.go +++ b/core/consensus/protocols/protocols.go @@ -42,3 +42,20 @@ func IsSupportedProtocolName(name string) bool { return false } + +// BumpProtocolsByName bumps given protocols priority by protocol name. +// The initial order of the protocols is preserved. +func BumpProtocolsByName(protocolName string, allProtocols []protocol.ID) []protocol.ID { + targetPrefix := protocolIDPrefix + protocolName + "/" + + var bumped, others []protocol.ID + for _, p := range allProtocols { + if strings.HasPrefix(string(p), targetPrefix) { + bumped = append(bumped, p) + } else { + others = append(others, p) + } + } + + return append(bumped, others...) +} diff --git a/core/consensus/protocols/protocols_test.go b/core/consensus/protocols/protocols_test.go index 4df1291b3..ceecc66c6 100644 --- a/core/consensus/protocols/protocols_test.go +++ b/core/consensus/protocols/protocols_test.go @@ -37,3 +37,20 @@ func TestMostPreferredConsensusProtocol(t *testing.T) { require.Equal(t, "/charon/consensus/abft/3.0.0", protocols.MostPreferredConsensusProtocol(pp)) }) } + +func TestBumpProtocolsByName(t *testing.T) { + intitial := []protocol.ID{ + "/charon/consensus/hotstuff/1.0.0", + "/charon/consensus/abft/3.0.0", + "/charon/consensus/abft/1.0.0", + "/charon/consensus/qbft/1.0.0", + } + + bumped := protocols.BumpProtocolsByName("abft", intitial) + require.Equal(t, []protocol.ID{ + "/charon/consensus/abft/3.0.0", + "/charon/consensus/abft/1.0.0", + "/charon/consensus/hotstuff/1.0.0", + "/charon/consensus/qbft/1.0.0", + }, bumped) +} diff --git a/docs/configuration.md b/docs/configuration.md index 4d30b14ef..8b103f518 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -153,6 +153,7 @@ Flags: --beacon-node-submit-timeout duration Timeout for the submission-related HTTP requests Charon makes to the configured beacon nodes. (default 2s) --beacon-node-timeout duration Timeout for the HTTP requests Charon makes to the configured beacon nodes. (default 2s) --builder-api Enables the builder api. Will only produce builder blocks. Builder API must also be enabled on the validator client. Beacon node must be connected to a builder-relay to access the builder network. + --consensus-protocol string Preferred consensus protocol name for the cluster. Selected automatically when not specified. --debug-address string Listening address (ip and port) for the pprof and QBFT debug API. It is not enabled by default. --feature-set string Minimum feature set to enable by default: alpha, beta, or stable. Warning: modify at own risk. (default "stable") --feature-set-disable strings Comma-separated list of features to disable, overriding the default minimum feature set. From d70cdeea2ea52292d84426342e50b565a59901f0 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Fri, 11 Oct 2024 16:44:37 +0300 Subject: [PATCH 11/20] Improved wording --- app/app.go | 23 ++++++++++++---------- cmd/run.go | 2 +- core/consensus/protocols/protocols.go | 6 +++--- core/consensus/protocols/protocols_test.go | 4 ++-- core/interfaces.go | 2 +- docs/configuration.md | 2 +- 6 files changed, 21 insertions(+), 18 deletions(-) diff --git a/app/app.go b/app/app.go index 6b73d3612..34337c541 100644 --- a/app/app.go +++ b/app/app.go @@ -534,8 +534,9 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, } defaultConsensus := consensusFactory.DefaultConsensus() - coreConsensus := consensusFactory.CurrentConsensus() - startConsensus := lifecycle.HookFuncCtx(defaultConsensus.Start) + startDefaultConsensus := lifecycle.HookFuncCtx(defaultConsensus.Start) + + coreConsensus := consensusFactory.CurrentConsensus() // points to DefaultConsensus() initially // Priority protocol always uses QBFTv2. err = wirePrioritise(ctx, conf, life, tcpNode, peerIDs, int(cluster.GetThreshold()), @@ -578,7 +579,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, } life.RegisterStart(lifecycle.AsyncBackground, lifecycle.StartScheduler, lifecycle.HookFuncErr(sched.Run)) - life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startConsensus) + life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startDefaultConsensus) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartAggSigDB, lifecycle.HookFuncCtx(aggSigDB.Run)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartParSigDB, lifecycle.HookFuncCtx(parSigDB.Trim)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartTracker, lifecycle.HookFuncCtx(inclusion.Run)) @@ -612,16 +613,16 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t } // The initial protocols order as defined by implementation is altered by: - // 1. Bumping the cluster (lock) preferred protocol to the top. - // 2. Bumping the protocol specified by CLI flag (cluster run) to the top. - // In all cases this bumps all versions of the protocol identified by name. + // 1. Prioritizing the cluster (lock) preferred protocol to the top. + // 2. Prioritizing the protocol specified by CLI flag (cluster run) to the top. + // In all cases this prioritizes all versions of the protocol identified by name. // The order of all these operations are important. allProtocols := Protocols() if clusterPreferredProtocol != "" { - allProtocols = protocols.BumpProtocolsByName(clusterPreferredProtocol, allProtocols) + allProtocols = protocols.PrioritizeProtocolsByName(clusterPreferredProtocol, allProtocols) } if conf.ConsensusProtocol != "" { - allProtocols = protocols.BumpProtocolsByName(conf.ConsensusProtocol, allProtocols) + allProtocols = protocols.PrioritizeProtocolsByName(conf.ConsensusProtocol, allProtocols) } isync := infosync.New(prio, @@ -650,10 +651,12 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t preferredConsensusProtocol := protocols.MostPreferredConsensusProtocol(allProtocols) if err := consensusFactory.SetCurrentConsensusForProtocol(protocol.ID(preferredConsensusProtocol)); err != nil { - log.Error(ctx, "Failed to set current consensus for protocol", err, z.Str("protocol", preferredConsensusProtocol)) + log.Error(ctx, "Failed to set current consensus protocol", err, z.Str("protocol", preferredConsensusProtocol)) } else { - log.Info(ctx, "Set current consensus for protocol", z.Str("protocol", preferredConsensusProtocol)) + log.Info(ctx, "Current consensus protocol changed", z.Str("protocol", preferredConsensusProtocol)) } + + break } } diff --git a/cmd/run.go b/cmd/run.go index 0aaab81ed..99c60b991 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -92,7 +92,7 @@ func bindRunFlags(cmd *cobra.Command, config *app.Config) { cmd.Flags().Int64Var(&config.TestnetConfig.GenesisTimestamp, "testnet-genesis-timestamp", 0, "Genesis timestamp of the custom test network.") cmd.Flags().StringVar(&config.TestnetConfig.CapellaHardFork, "testnet-capella-hard-fork", "", "Capella hard fork version of the custom test network.") cmd.Flags().StringVar(&config.ProcDirectory, "proc-directory", "", "Directory to look into in order to detect other stack components running on the host.") - cmd.Flags().StringVar(&config.ConsensusProtocol, "consensus-protocol", "", "Preferred consensus protocol name for the cluster. Selected automatically when not specified.") + cmd.Flags().StringVar(&config.ConsensusProtocol, "consensus-protocol", "", "Preferred consensus protocol name for the node. Selected automatically when not specified.") wrapPreRunE(cmd, func(*cobra.Command, []string) error { if len(config.BeaconNodeAddrs) == 0 && !config.SimnetBMock { diff --git a/core/consensus/protocols/protocols.go b/core/consensus/protocols/protocols.go index 8cf493b5d..961ae92ef 100644 --- a/core/consensus/protocols/protocols.go +++ b/core/consensus/protocols/protocols.go @@ -43,9 +43,9 @@ func IsSupportedProtocolName(name string) bool { return false } -// BumpProtocolsByName bumps given protocols priority by protocol name. -// The initial order of the protocols is preserved. -func BumpProtocolsByName(protocolName string, allProtocols []protocol.ID) []protocol.ID { +// PrioritizeProtocolsByName bumps given protocols priority by protocol name. +// The initial order of the protocols and versions is preserved. +func PrioritizeProtocolsByName(protocolName string, allProtocols []protocol.ID) []protocol.ID { targetPrefix := protocolIDPrefix + protocolName + "/" var bumped, others []protocol.ID diff --git a/core/consensus/protocols/protocols_test.go b/core/consensus/protocols/protocols_test.go index ceecc66c6..31321ddd6 100644 --- a/core/consensus/protocols/protocols_test.go +++ b/core/consensus/protocols/protocols_test.go @@ -38,7 +38,7 @@ func TestMostPreferredConsensusProtocol(t *testing.T) { }) } -func TestBumpProtocolsByName(t *testing.T) { +func TestPrioritizeProtocolsByName(t *testing.T) { intitial := []protocol.ID{ "/charon/consensus/hotstuff/1.0.0", "/charon/consensus/abft/3.0.0", @@ -46,7 +46,7 @@ func TestBumpProtocolsByName(t *testing.T) { "/charon/consensus/qbft/1.0.0", } - bumped := protocols.BumpProtocolsByName("abft", intitial) + bumped := protocols.PrioritizeProtocolsByName("abft", intitial) require.Equal(t, []protocol.ID{ "/charon/consensus/abft/3.0.0", "/charon/consensus/abft/1.0.0", diff --git a/core/interfaces.go b/core/interfaces.go index 23c082fb8..56580a536 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -96,7 +96,7 @@ type ConsensusFactory interface { // CurrentConsensus returns currently selected consensus instance. // The instance is selected by the Priority protocol and can be changed by SetCurrentConsensusForProtocol(). - // Before SetCurrentConsensusForProtocol() is called, CurrentConsensus() must return DefaultConsensus(). + // Before SetCurrentConsensusForProtocol() is called, CurrentConsensus() points to DefaultConsensus(). CurrentConsensus() Consensus // SetCurrentConsensusForProtocol handles Priority protocol outcome and changes the CurrentConsensus() accordingly. diff --git a/docs/configuration.md b/docs/configuration.md index 8b103f518..45dd9777b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -153,7 +153,7 @@ Flags: --beacon-node-submit-timeout duration Timeout for the submission-related HTTP requests Charon makes to the configured beacon nodes. (default 2s) --beacon-node-timeout duration Timeout for the HTTP requests Charon makes to the configured beacon nodes. (default 2s) --builder-api Enables the builder api. Will only produce builder blocks. Builder API must also be enabled on the validator client. Beacon node must be connected to a builder-relay to access the builder network. - --consensus-protocol string Preferred consensus protocol name for the cluster. Selected automatically when not specified. + --consensus-protocol string Preferred consensus protocol name for the node. Selected automatically when not specified. --debug-address string Listening address (ip and port) for the pprof and QBFT debug API. It is not enabled by default. --feature-set string Minimum feature set to enable by default: alpha, beta, or stable. Warning: modify at own risk. (default "stable") --feature-set-disable strings Comma-separated list of features to disable, overriding the default minimum feature set. From f6bd6619664b01ce6ee89b40d0c2c749de41ba92 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Mon, 14 Oct 2024 12:17:26 +0300 Subject: [PATCH 12/20] Unexported odd symbols --- core/consensus/debugger_internal_test.go | 2 +- core/consensus/qbft/msg.go | 48 ++---- .../{msg_test.go => msg_internal_test.go} | 42 ++++-- core/consensus/qbft/qbft.go | 28 ++-- core/consensus/qbft/qbft_internal_test.go | 70 ++++----- .../{ => qbft}/sniffed_internal_test.go | 11 +- core/consensus/qbft/sniffer.go | 12 +- core/consensus/qbft/sniffer_internal_test.go | 8 +- .../qbft/strategysim_internal_test.go | 2 +- core/consensus/qbft/transport.go | 38 ++--- core/corepb/v1/consensus.pb.go | 142 ++++++++++-------- core/corepb/v1/consensus.proto | 20 +-- p2p/sender_internal_test.go | 2 +- 13 files changed, 217 insertions(+), 208 deletions(-) rename core/consensus/qbft/{msg_test.go => msg_internal_test.go} (72%) rename core/consensus/{ => qbft}/sniffed_internal_test.go (89%) diff --git a/core/consensus/debugger_internal_test.go b/core/consensus/debugger_internal_test.go index 3baa6f711..d51147167 100644 --- a/core/consensus/debugger_internal_test.go +++ b/core/consensus/debugger_internal_test.go @@ -29,7 +29,7 @@ func TestDebugger(t *testing.T) { { Timestamp: timestamppb.Now(), // Eventually the ConsensusMsg will be replaced by a more generic message type. - Msg: &pbv1.ConsensusMsg{ + Msg: &pbv1.QBFTConsensusMsg{ Msg: randomQBFTMsg(), Justification: []*pbv1.QBFTMsg{randomQBFTMsg(), randomQBFTMsg()}, }, diff --git a/core/consensus/qbft/msg.go b/core/consensus/qbft/msg.go index 4f7422aa2..1822cabfe 100644 --- a/core/consensus/qbft/msg.go +++ b/core/consensus/qbft/msg.go @@ -3,11 +3,8 @@ package qbft import ( - "testing" - k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" ssz "github.com/ferranbt/fastssz" - "golang.org/x/exp/rand" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" @@ -18,27 +15,8 @@ import ( "github.com/obolnetwork/charon/core/qbft" ) -// NewRandomMsgForT returns a random qbft message. -func NewRandomMsgForT(t *testing.T) *pbv1.QBFTMsg { - t.Helper() - - msgType := 1 + rand.Int63n(int64(qbft.MsgDecided)) - if msgType == 0 { - msgType = 1 - } - - return &pbv1.QBFTMsg{ - Type: msgType, - Duty: core.DutyToProto(core.Duty{Type: core.DutyType(rand.Int()), Slot: rand.Uint64()}), - PeerIdx: rand.Int63(), - Round: rand.Int63(), - PreparedRound: rand.Int63(), - Signature: nil, - } -} - -// NewMsg returns a new QBFT Msg. -func NewMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]byte]*anypb.Any) (Msg, error) { +// newMsg returns a new QBFT Msg. +func newMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]byte]*anypb.Any) (Msg, error) { if pbMsg == nil { return Msg{}, errors.New("nil qbft message") } @@ -65,7 +43,7 @@ func NewMsg(pbMsg *pbv1.QBFTMsg, justification []*pbv1.QBFTMsg, values map[[32]b var justImpls []qbft.Msg[core.Duty, [32]byte] for _, j := range justification { - impl, err := NewMsg(j, nil, values) + impl, err := newMsg(j, nil, values) if err != nil { return Msg{}, err } @@ -134,22 +112,22 @@ func (m Msg) Justification() []qbft.Msg[core.Duty, [32]byte] { return m.justification } -func (m Msg) ToConsensusMsg() *pbv1.ConsensusMsg { +func (m Msg) ToConsensusMsg() *pbv1.QBFTConsensusMsg { var values []*anypb.Any for _, v := range m.values { values = append(values, v) } - return &pbv1.ConsensusMsg{ + return &pbv1.QBFTConsensusMsg{ Msg: m.msg, Justification: m.justificationProtos, Values: values, } } -// HashProto returns a deterministic ssz hash root of the proto message. +// hashProto returns a deterministic ssz hash root of the proto message. // It is the same logic as that used by the priority package. -func HashProto(msg proto.Message) ([32]byte, error) { +func hashProto(msg proto.Message) ([32]byte, error) { if _, ok := msg.(*anypb.Any); ok { return [32]byte{}, errors.New("cannot hash any proto, must hash inner value") } @@ -176,8 +154,8 @@ func HashProto(msg proto.Message) ([32]byte, error) { return hash, nil } -// VerifyMsgSig returns true if the message was signed by pubkey. -func VerifyMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { +// verifyMsgSig returns true if the message was signed by pubkey. +func verifyMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { if msg.Signature == nil { return false, errors.New("empty signature") } @@ -187,7 +165,7 @@ func VerifyMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { return false, errors.New("type assert qbft msg") } clone.Signature = nil - hash, err := HashProto(clone) + hash, err := hashProto(clone) if err != nil { return false, err } @@ -200,15 +178,15 @@ func VerifyMsgSig(msg *pbv1.QBFTMsg, pubkey *k1.PublicKey) (bool, error) { return recovered.IsEqual(pubkey), nil } -// SignMsg returns a copy of the proto message with a populated signature signed by the provided private key. -func SignMsg(msg *pbv1.QBFTMsg, privkey *k1.PrivateKey) (*pbv1.QBFTMsg, error) { +// signMsg returns a copy of the proto message with a populated signature signed by the provided private key. +func signMsg(msg *pbv1.QBFTMsg, privkey *k1.PrivateKey) (*pbv1.QBFTMsg, error) { clone, ok := proto.Clone(msg).(*pbv1.QBFTMsg) if !ok { return nil, errors.New("type assert qbft msg") } clone.Signature = nil - hash, err := HashProto(clone) + hash, err := hashProto(clone) if err != nil { return nil, err } diff --git a/core/consensus/qbft/msg_test.go b/core/consensus/qbft/msg_internal_test.go similarity index 72% rename from core/consensus/qbft/msg_test.go rename to core/consensus/qbft/msg_internal_test.go index 157215f57..1750c87c7 100644 --- a/core/consensus/qbft/msg_test.go +++ b/core/consensus/qbft/msg_internal_test.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package qbft_test +package qbft import ( "encoding/hex" @@ -14,7 +14,6 @@ import ( "google.golang.org/protobuf/types/known/timestamppb" "github.com/obolnetwork/charon/core" - "github.com/obolnetwork/charon/core/consensus/qbft" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" coreqbft "github.com/obolnetwork/charon/core/qbft" "github.com/obolnetwork/charon/testutil" @@ -29,7 +28,7 @@ func TestHashProto(t *testing.T) { setPB, err := core.UnsignedDataSetToProto(set) require.NoError(t, err) - hash, err := qbft.HashProto(setPB) + hash, err := hashProto(setPB) require.NoError(t, err) require.Equal(t, @@ -44,18 +43,18 @@ func TestSigning(t *testing.T) { privkey, err := k1.GeneratePrivateKey() require.NoError(t, err) - msg := qbft.NewRandomMsgForT(t) + msg := newRandomQBFTMsg(t) - signed, err := qbft.SignMsg(msg, privkey) + signed, err := signMsg(msg, privkey) require.NoError(t, err) - ok, err := qbft.VerifyMsgSig(signed, privkey.PubKey()) + ok, err := verifyMsgSig(signed, privkey.PubKey()) require.NoError(t, err) require.True(t, ok) privkey2, err := k1.GeneratePrivateKey() require.NoError(t, err) - ok, err = qbft.VerifyMsgSig(signed, privkey2.PubKey()) + ok, err = verifyMsgSig(signed, privkey2.PubKey()) require.NoError(t, err) require.False(t, ok) } @@ -63,9 +62,9 @@ func TestSigning(t *testing.T) { func TestNewMsg(t *testing.T) { val1 := timestamppb.New(time.Time{}) val2 := timestamppb.New(time.Now()) - hash1, err := qbft.HashProto(val1) + hash1, err := hashProto(val1) require.NoError(t, err) - hash2, err := qbft.HashProto(val2) + hash2, err := hashProto(val2) require.NoError(t, err) any1, err := anypb.New(val1) @@ -78,7 +77,7 @@ func TestNewMsg(t *testing.T) { hash2: any2, } - msg, err := qbft.NewMsg(&pbv1.QBFTMsg{ + msg, err := newMsg(&pbv1.QBFTMsg{ Type: int64(coreqbft.MsgPrePrepare), ValueHash: hash1[:], PreparedValueHash: hash2[:], @@ -92,10 +91,10 @@ func TestNewMsg(t *testing.T) { func TestPartialLegacyNewMsg(t *testing.T) { val1 := timestamppb.New(time.Time{}) - hash1, err := qbft.HashProto(val1) + hash1, err := hashProto(val1) require.NoError(t, err) - _, err = qbft.NewMsg(&pbv1.QBFTMsg{ + _, err = newMsg(&pbv1.QBFTMsg{ Type: int64(coreqbft.MsgPrePrepare), }, []*pbv1.QBFTMsg{ { @@ -105,3 +104,22 @@ func TestPartialLegacyNewMsg(t *testing.T) { }, make(map[[32]byte]*anypb.Any)) require.ErrorContains(t, err, "value hash not found in values") } + +// NewRandomMsgForT returns a random qbft message. +func newRandomQBFTMsg(t *testing.T) *pbv1.QBFTMsg { + t.Helper() + + msgType := 1 + rand.Int63n(int64(coreqbft.MsgDecided)) + if msgType == 0 { + msgType = 1 + } + + return &pbv1.QBFTMsg{ + Type: msgType, + Duty: core.DutyToProto(core.Duty{Type: core.DutyType(rand.Int()), Slot: rand.Uint64()}), + PeerIdx: rand.Int63(), + Round: rand.Int63(), + PreparedRound: rand.Int63(), + Signature: nil, + } +} diff --git a/core/consensus/qbft/qbft.go b/core/consensus/qbft/qbft.go index 5202f01de..7d69df495 100644 --- a/core/consensus/qbft/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -30,10 +30,10 @@ import ( "github.com/obolnetwork/charon/p2p" ) -type Subscriber func(ctx context.Context, duty core.Duty, value proto.Message) error +type subscriber func(ctx context.Context, duty core.Duty, value proto.Message) error -// NewDefinition returns a qbft definition (this is constant across all consensus instances). -func NewDefinition(nodes int, subs func() []Subscriber, roundTimer utils.RoundTimer, +// newDefinition returns a qbft definition (this is constant across all consensus instances). +func newDefinition(nodes int, subs func() []subscriber, roundTimer utils.RoundTimer, decideCallback func(qcommit []qbft.Msg[core.Duty, [32]byte]), ) qbft.Definition[core.Duty, [32]byte] { quorum := qbft.Definition[int, int]{Nodes: nodes}.Quorum() @@ -165,7 +165,7 @@ type Consensus struct { peers []p2p.Peer pubkeys map[int64]*k1.PublicKey privkey *k1.PrivateKey - subs []Subscriber + subs []subscriber deadliner core.Deadliner snifferFunc func(*pbv1.SniffedConsensusInstance) gaterFunc core.DutyGaterFunc @@ -204,7 +204,7 @@ func (c *Consensus) Subscribe(fn func(ctx context.Context, duty core.Duty, set c } // subscribers returns the subscribers. -func (c *Consensus) subscribers() []Subscriber { +func (c *Consensus) subscribers() []subscriber { return c.subs } @@ -224,7 +224,7 @@ func (c *Consensus) SubscribePriority(fn func(ctx context.Context, duty core.Dut // Start registers the libp2p receive handler and starts a goroutine that cleans state. This should only be called once. func (c *Consensus) Start(ctx context.Context) { p2p.RegisterHandler("qbft", c.tcpNode, protocols.QBFTv2ProtocolID, - func() proto.Message { return new(pbv1.ConsensusMsg) }, + func() proto.Message { return new(pbv1.QBFTConsensusMsg) }, c.handle) go func() { @@ -266,7 +266,7 @@ func (c *Consensus) ProposePriority(ctx context.Context, duty core.Duty, msg *pb // waits until it completes, in both cases it returns the resulting error. // Note this errors if called multiple times for the same duty. func (c *Consensus) propose(ctx context.Context, duty core.Duty, value proto.Message) error { - hash, err := HashProto(value) + hash, err := hashProto(value) if err != nil { return err } @@ -336,7 +336,7 @@ func (c *Consensus) Participate(ctx context.Context, duty core.Duty) error { } // Broadcast implements Broadcaster interface. -func (c *Consensus) Broadcast(ctx context.Context, msg *pbv1.ConsensusMsg) error { +func (c *Consensus) Broadcast(ctx context.Context, msg *pbv1.QBFTConsensusMsg) error { for _, peer := range c.peers { if peer.ID == c.tcpNode.ID() { // Do not broadcast to self @@ -390,10 +390,10 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) } // Create a new qbft definition for this instance. - def := NewDefinition(len(c.peers), c.subscribers, roundTimer, decideCallback) + def := newDefinition(len(c.peers), c.subscribers, roundTimer, decideCallback) // Create a new transport that handles sending and receiving for this instance. - t := NewTransport(c, c.privkey, inst.ValueCh, make(chan qbft.Msg[core.Duty, [32]byte]), newSniffer(int64(def.Nodes), peerIdx)) + t := newTransport(c, c.privkey, inst.ValueCh, make(chan qbft.Msg[core.Duty, [32]byte]), newSniffer(int64(def.Nodes), peerIdx)) // Provide sniffed buffer to snifferFunc at the end. defer func() { @@ -429,7 +429,7 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) func (c *Consensus) handle(ctx context.Context, _ peer.ID, req proto.Message) (proto.Message, bool, error) { t0 := time.Now() - pbMsg, ok := req.(*pbv1.ConsensusMsg) + pbMsg, ok := req.(*pbv1.QBFTConsensusMsg) if !ok || pbMsg == nil { return nil, false, errors.New("invalid consensus message") } @@ -465,7 +465,7 @@ func (c *Consensus) handle(ctx context.Context, _ peer.ID, req proto.Message) (p return nil, false, err } - msg, err := NewMsg(pbMsg.GetMsg(), pbMsg.GetJustification(), values) + msg, err := newMsg(pbMsg.GetMsg(), pbMsg.GetJustification(), values) if err != nil { return nil, false, err } @@ -568,7 +568,7 @@ func verifyMsg(msg *pbv1.QBFTMsg, pubkeys map[int64]*k1.PublicKey) error { return errors.New("invalid peer index", z.I64("index", msg.GetPeerIdx())) } - if ok, err := VerifyMsgSig(msg, msgPubkey); err != nil { + if ok, err := verifyMsgSig(msg, msgPubkey); err != nil { return errors.Wrap(err, "verify consensus message signature") } else if !ok { return errors.New("invalid consensus message signature") @@ -701,7 +701,7 @@ func ValuesByHash(values []*anypb.Any) (map[[32]byte]*anypb.Any, error) { return nil, errors.Wrap(err, "unmarshal any") } - hash, err := HashProto(inner) + hash, err := hashProto(inner) if err != nil { return nil, err } diff --git a/core/consensus/qbft/qbft_internal_test.go b/core/consensus/qbft/qbft_internal_test.go index d083b8110..177ea5ff3 100644 --- a/core/consensus/qbft/qbft_internal_test.go +++ b/core/consensus/qbft/qbft_internal_test.go @@ -122,12 +122,12 @@ func (t testMsg) Justification() []qbft.Msg[core.Duty, [32]byte] { func TestQBFTConsensus_handle(t *testing.T) { tests := []struct { name string - mutate func(base *pbv1.ConsensusMsg, c *Consensus) + mutate func(base *pbv1.QBFTConsensusMsg, c *Consensus) checkErr func(err error) }{ { "qbft message with no pubkey errors", - func(base *pbv1.ConsensusMsg, c *Consensus) { + func(base *pbv1.QBFTConsensusMsg, c *Consensus) { // construct a valid basis message signature base.Msg.Duty.Type = 1 base.Msg.Signature = bytes.Repeat([]byte{42}, 65) @@ -143,7 +143,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with justifications mentioning unknown peerIdx errors", - func(base *pbv1.ConsensusMsg, c *Consensus) { + func(base *pbv1.QBFTConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -156,7 +156,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := HashProto(base.GetMsg()) + msgHash, err := hashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -166,7 +166,7 @@ func TestQBFTConsensus_handle(t *testing.T) { // construct a justification base.Justification = []*pbv1.QBFTMsg{ - NewRandomMsgForT(t), + newRandomQBFTMsg(t), } base.Justification[0].PeerIdx = 42 @@ -176,7 +176,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the justification - justHash, err := HashProto(base.GetJustification()[0]) + justHash, err := hashProto(base.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(p2pKey, justHash[:]) @@ -190,7 +190,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with nil justification present in slice", - func(base *pbv1.ConsensusMsg, c *Consensus) { + func(base *pbv1.QBFTConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -203,7 +203,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := HashProto(base.GetMsg()) + msgHash, err := hashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -223,7 +223,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message values present but nil", - func(base *pbv1.ConsensusMsg, c *Consensus) { + func(base *pbv1.QBFTConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -242,7 +242,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := HashProto(base.GetMsg()) + msgHash, err := hashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -256,7 +256,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with invalid duty fails", - func(base *pbv1.ConsensusMsg, c *Consensus) { + func(base *pbv1.QBFTConsensusMsg, c *Consensus) { // construct a valid basis message signature base.Msg.Duty.Type = 1 base.Msg.Signature = bytes.Repeat([]byte{42}, 65) @@ -272,7 +272,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with valid duty fails because justification has different duty type", - func(base *pbv1.ConsensusMsg, c *Consensus) { + func(base *pbv1.QBFTConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -285,7 +285,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := HashProto(base.GetMsg()) + msgHash, err := hashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -295,7 +295,7 @@ func TestQBFTConsensus_handle(t *testing.T) { // construct a justification base.Justification = []*pbv1.QBFTMsg{ - NewRandomMsgForT(t), + newRandomQBFTMsg(t), } base.Justification[0].PeerIdx = 0 @@ -305,7 +305,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the justification - justHash, err := HashProto(base.GetJustification()[0]) + justHash, err := hashProto(base.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(p2pKey, justHash[:]) @@ -319,7 +319,7 @@ func TestQBFTConsensus_handle(t *testing.T) { }, { "qbft message with valid duty and justification with same duty does not fail", - func(base *pbv1.ConsensusMsg, c *Consensus) { + func(base *pbv1.QBFTConsensusMsg, c *Consensus) { p2pKey := testutil.GenerateInsecureK1Key(t, 0) c.pubkeys = make(map[int64]*k1.PublicKey) c.pubkeys[0] = p2pKey.PubKey() @@ -332,7 +332,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the base message - msgHash, err := HashProto(base.GetMsg()) + msgHash, err := hashProto(base.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(p2pKey, msgHash[:]) @@ -342,7 +342,7 @@ func TestQBFTConsensus_handle(t *testing.T) { // construct a justification base.Justification = []*pbv1.QBFTMsg{ - NewRandomMsgForT(t), + newRandomQBFTMsg(t), } base.Justification[0].PeerIdx = 0 @@ -352,7 +352,7 @@ func TestQBFTConsensus_handle(t *testing.T) { } // Sign the justification - justHash, err := HashProto(base.GetJustification()[0]) + justHash, err := hashProto(base.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(p2pKey, justHash[:]) @@ -375,8 +375,8 @@ func TestQBFTConsensus_handle(t *testing.T) { tc.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) tc.gaterFunc = func(core.Duty) bool { return true } - msg := &pbv1.ConsensusMsg{ - Msg: NewRandomMsgForT(t), + msg := &pbv1.QBFTConsensusMsg{ + Msg: newRandomQBFTMsg(t), } test.mutate(msg, &tc) @@ -390,7 +390,7 @@ func TestQBFTConsensus_handle(t *testing.T) { func TestQBFTConsensusHandle(t *testing.T) { tests := []struct { name string - msg *pbv1.ConsensusMsg + msg *pbv1.QBFTConsensusMsg errorMsg string peerID string }{ @@ -400,14 +400,14 @@ func TestQBFTConsensusHandle(t *testing.T) { }, { name: "nil msg", - msg: &pbv1.ConsensusMsg{ + msg: &pbv1.QBFTConsensusMsg{ Msg: nil, }, errorMsg: "invalid consensus message", }, { name: "nil msg duty", - msg: &pbv1.ConsensusMsg{ + msg: &pbv1.QBFTConsensusMsg{ Msg: &pbv1.QBFTMsg{ Duty: nil, }, @@ -416,7 +416,7 @@ func TestQBFTConsensusHandle(t *testing.T) { }, { name: "invalid consensus msg type", - msg: &pbv1.ConsensusMsg{ + msg: &pbv1.QBFTConsensusMsg{ Msg: &pbv1.QBFTMsg{ Duty: &pbv1.Duty{}, }, @@ -425,7 +425,7 @@ func TestQBFTConsensusHandle(t *testing.T) { }, { name: "invalid msg duty type", - msg: &pbv1.ConsensusMsg{ + msg: &pbv1.QBFTConsensusMsg{ Msg: &pbv1.QBFTMsg{ Duty: &pbv1.Duty{}, Type: int64(qbft.MsgPrepare), @@ -435,7 +435,7 @@ func TestQBFTConsensusHandle(t *testing.T) { }, { name: "invalid peer index", - msg: &pbv1.ConsensusMsg{ + msg: &pbv1.QBFTConsensusMsg{ Msg: &pbv1.QBFTMsg{ Round: 1, Duty: &pbv1.Duty{Type: int32(core.DutyProposer)}, @@ -479,8 +479,8 @@ func TestInstanceIO_MaybeStart(t *testing.T) { c.pubkeys[0] = p2pKey.PubKey() duty := core.Duty{Slot: 42, Type: 1} - msg := &pbv1.ConsensusMsg{ - Msg: NewRandomMsgForT(t), + msg := &pbv1.QBFTConsensusMsg{ + Msg: newRandomQBFTMsg(t), } msg = signConsensusMsg(t, msg, p2pKey, duty) @@ -509,8 +509,8 @@ func TestInstanceIO_MaybeStart(t *testing.T) { c.pubkeys[0] = p2pKey.PubKey() duty := core.Duty{Slot: 42, Type: 1} - msg := &pbv1.ConsensusMsg{ - Msg: NewRandomMsgForT(t), + msg := &pbv1.QBFTConsensusMsg{ + Msg: newRandomQBFTMsg(t), } msg = signConsensusMsg(t, msg, p2pKey, duty) @@ -544,7 +544,7 @@ func (t testDeadliner) C() <-chan core.Duty { return t.deadlineChan } -func signConsensusMsg(t *testing.T, msg *pbv1.ConsensusMsg, privKey *k1.PrivateKey, duty core.Duty) *pbv1.ConsensusMsg { +func signConsensusMsg(t *testing.T, msg *pbv1.QBFTConsensusMsg, privKey *k1.PrivateKey, duty core.Duty) *pbv1.QBFTConsensusMsg { t.Helper() msg.Msg.Duty.Type = int32(duty.Type) @@ -555,7 +555,7 @@ func signConsensusMsg(t *testing.T, msg *pbv1.ConsensusMsg, privKey *k1.PrivateK } // Sign the base message - msgHash, err := HashProto(msg.GetMsg()) + msgHash, err := hashProto(msg.GetMsg()) require.NoError(t, err) sign, err := k1util.Sign(privKey, msgHash[:]) @@ -565,7 +565,7 @@ func signConsensusMsg(t *testing.T, msg *pbv1.ConsensusMsg, privKey *k1.PrivateK // construct a justification msg.Justification = []*pbv1.QBFTMsg{ - NewRandomMsgForT(t), + newRandomQBFTMsg(t), } msg.Justification[0].PeerIdx = 0 @@ -575,7 +575,7 @@ func signConsensusMsg(t *testing.T, msg *pbv1.ConsensusMsg, privKey *k1.PrivateK } // Sign the justification - justHash, err := HashProto(msg.GetJustification()[0]) + justHash, err := hashProto(msg.GetJustification()[0]) require.NoError(t, err) justSign, err := k1util.Sign(privKey, justHash[:]) diff --git a/core/consensus/sniffed_internal_test.go b/core/consensus/qbft/sniffed_internal_test.go similarity index 89% rename from core/consensus/sniffed_internal_test.go rename to core/consensus/qbft/sniffed_internal_test.go index 53c5ff8ad..7cab33a30 100644 --- a/core/consensus/sniffed_internal_test.go +++ b/core/consensus/qbft/sniffed_internal_test.go @@ -1,6 +1,6 @@ // Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 -package consensus +package qbft import ( "bytes" @@ -19,7 +19,6 @@ import ( "github.com/obolnetwork/charon/app/log" "github.com/obolnetwork/charon/app/z" "github.com/obolnetwork/charon/core" - cqbft "github.com/obolnetwork/charon/core/consensus/qbft" "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" "github.com/obolnetwork/charon/core/qbft" @@ -70,8 +69,8 @@ func testSniffedInstance(ctx context.Context, t *testing.T, instance *pbv1.Sniff var expectDecided bool - def := cqbft.NewDefinition(int(instance.GetNodes()), func() []cqbft.Subscriber { - return []cqbft.Subscriber{func(ctx context.Context, duty core.Duty, value proto.Message) error { + def := newDefinition(int(instance.GetNodes()), func() []subscriber { + return []subscriber{func(ctx context.Context, duty core.Duty, value proto.Message) error { log.Info(ctx, "Consensus decided", z.Any("value", value)) expectDecided = true cancel() @@ -90,10 +89,10 @@ func testSniffedInstance(ctx context.Context, t *testing.T, instance *pbv1.Sniff duty = core.DutyFromProto(msg.GetMsg().GetMsg().GetDuty()) - values, err := cqbft.ValuesByHash(msg.GetMsg().GetValues()) + values, err := ValuesByHash(msg.GetMsg().GetValues()) require.NoError(t, err) - m, err := cqbft.NewMsg(msg.GetMsg().GetMsg(), msg.GetMsg().GetJustification(), values) + m, err := newMsg(msg.GetMsg().GetMsg(), msg.GetMsg().GetJustification(), values) require.NoError(t, err) recvBuffer <- m } diff --git a/core/consensus/qbft/sniffer.go b/core/consensus/qbft/sniffer.go index 35fac50c9..12e76b75c 100644 --- a/core/consensus/qbft/sniffer.go +++ b/core/consensus/qbft/sniffer.go @@ -8,6 +8,7 @@ import ( "google.golang.org/protobuf/types/known/timestamppb" + "github.com/obolnetwork/charon/core/consensus/protocols" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" ) @@ -31,7 +32,7 @@ type sniffer struct { } // Add adds a message to the sniffer buffer. -func (c *sniffer) Add(msg *pbv1.ConsensusMsg) { +func (c *sniffer) Add(msg *pbv1.QBFTConsensusMsg) { c.mu.Lock() defer c.mu.Unlock() @@ -47,9 +48,10 @@ func (c *sniffer) Instance() *pbv1.SniffedConsensusInstance { defer c.mu.Unlock() return &pbv1.SniffedConsensusInstance{ - Nodes: c.nodes, - PeerIdx: c.peerIdx, - StartedAt: timestamppb.New(c.startedAt), - Msgs: c.msgs, + Nodes: c.nodes, + PeerIdx: c.peerIdx, + StartedAt: timestamppb.New(c.startedAt), + Msgs: c.msgs, + ProtocolId: protocols.QBFTv2ProtocolID, } } diff --git a/core/consensus/qbft/sniffer_internal_test.go b/core/consensus/qbft/sniffer_internal_test.go index 8cf46b434..12abaca77 100644 --- a/core/consensus/qbft/sniffer_internal_test.go +++ b/core/consensus/qbft/sniffer_internal_test.go @@ -13,11 +13,11 @@ import ( func TestSniffer(t *testing.T) { sniffer := newSniffer(3, 1) - sniffer.Add(&pbv1.ConsensusMsg{ - Msg: NewRandomMsgForT(t), + sniffer.Add(&pbv1.QBFTConsensusMsg{ + Msg: newRandomQBFTMsg(t), }) - sniffer.Add(&pbv1.ConsensusMsg{ - Msg: NewRandomMsgForT(t), + sniffer.Add(&pbv1.QBFTConsensusMsg{ + Msg: newRandomQBFTMsg(t), }) instance := sniffer.Instance() diff --git a/core/consensus/qbft/strategysim_internal_test.go b/core/consensus/qbft/strategysim_internal_test.go index cc6fe16a7..52fc0ce32 100644 --- a/core/consensus/qbft/strategysim_internal_test.go +++ b/core/consensus/qbft/strategysim_internal_test.go @@ -641,7 +641,7 @@ func (i *transportInstance) Broadcast(_ context.Context, typ qbft.MsgType, values[impl.PreparedValue()] = dummy } - msg, err := NewMsg(pbMsg, justMsgs, values) + msg, err := newMsg(pbMsg, justMsgs, values) if err != nil { return err } diff --git a/core/consensus/qbft/transport.go b/core/consensus/qbft/transport.go index cc6ad43d3..7ed399619 100644 --- a/core/consensus/qbft/transport.go +++ b/core/consensus/qbft/transport.go @@ -16,15 +16,15 @@ import ( "github.com/obolnetwork/charon/core/qbft" ) -// Broadcaster is an interface for broadcasting messages asynchronously. -type Broadcaster interface { - Broadcast(ctx context.Context, msg *pbv1.ConsensusMsg) error +// broadcaster is an interface for broadcasting messages asynchronously. +type broadcaster interface { + Broadcast(ctx context.Context, msg *pbv1.QBFTConsensusMsg) error } -// Transport encapsulates receiving and broadcasting for a consensus instance/duty. -type Transport struct { +// transport encapsulates receiving and broadcasting for a consensus instance/duty. +type transport struct { // Immutable state - broadcaster Broadcaster + broadcaster broadcaster privkey *k1.PrivateKey recvBuffer chan qbft.Msg[core.Duty, [32]byte] // Instance inner receive buffer. sniffer *sniffer @@ -35,11 +35,11 @@ type Transport struct { values map[[32]byte]*anypb.Any // maps any-wrapped proposed values to their hashes } -// NewTransport creates a new qbftTransport. -func NewTransport(broadcaster Broadcaster, privkey *k1.PrivateKey, valueCh <-chan proto.Message, +// newTransport creates a new qbftTransport. +func newTransport(broadcaster broadcaster, privkey *k1.PrivateKey, valueCh <-chan proto.Message, recvBuffer chan qbft.Msg[core.Duty, [32]byte], sniffer *sniffer, -) *Transport { - return &Transport{ +) *transport { + return &transport{ broadcaster: broadcaster, privkey: privkey, recvBuffer: recvBuffer, @@ -50,7 +50,7 @@ func NewTransport(broadcaster Broadcaster, privkey *k1.PrivateKey, valueCh <-cha } // setValues caches the values and their hashes. -func (t *Transport) setValues(msg Msg) { +func (t *transport) setValues(msg Msg) { t.valueMu.Lock() defer t.valueMu.Unlock() @@ -60,14 +60,14 @@ func (t *Transport) setValues(msg Msg) { } // getValue returns the value by its hash. -func (t *Transport) getValue(hash [32]byte) (*anypb.Any, error) { +func (t *transport) getValue(hash [32]byte) (*anypb.Any, error) { t.valueMu.Lock() defer t.valueMu.Unlock() // First check if we have a new value. select { case value := <-t.valueCh: - valueHash, err := HashProto(value) + valueHash, err := hashProto(value) if err != nil { return nil, err } @@ -91,7 +91,7 @@ func (t *Transport) getValue(hash [32]byte) (*anypb.Any, error) { } // Broadcast creates a msg and sends it to all peers (including self). -func (t *Transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.Duty, +func (t *transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.Duty, peerIdx int64, round int64, valueHash [32]byte, pr int64, pvHash [32]byte, justification []qbft.Msg[core.Duty, [32]byte], ) error { @@ -143,7 +143,7 @@ func (t *Transport) Broadcast(ctx context.Context, typ qbft.MsgType, duty core.D } // ProcessReceives processes received messages from the outer buffer until the context is closed. -func (t *Transport) ProcessReceives(ctx context.Context, outerBuffer chan Msg) { +func (t *transport) ProcessReceives(ctx context.Context, outerBuffer chan Msg) { for { select { case <-ctx.Done(): @@ -162,12 +162,12 @@ func (t *Transport) ProcessReceives(ctx context.Context, outerBuffer chan Msg) { } // SnifferInstance returns the current sniffed consensus instance. -func (t *Transport) SnifferInstance() *pbv1.SniffedConsensusInstance { +func (t *transport) SnifferInstance() *pbv1.SniffedConsensusInstance { return t.sniffer.Instance() } // RecvBuffer returns the inner receive buffer. -func (t *Transport) RecvBuffer() chan qbft.Msg[core.Duty, [32]byte] { +func (t *transport) RecvBuffer() chan qbft.Msg[core.Duty, [32]byte] { return t.recvBuffer } @@ -188,7 +188,7 @@ func createMsg(typ qbft.MsgType, duty core.Duty, PreparedValueHash: pvHash[:], } - pbMsg, err := SignMsg(pbMsg, privkey) + pbMsg, err := signMsg(pbMsg, privkey) if err != nil { return Msg{}, err } @@ -203,5 +203,5 @@ func createMsg(typ qbft.MsgType, duty core.Duty, justMsgs = append(justMsgs, impl.Msg()) // Note nested justifications are ignored. } - return NewMsg(pbMsg, justMsgs, values) + return newMsg(pbMsg, justMsgs, values) } diff --git a/core/corepb/v1/consensus.pb.go b/core/corepb/v1/consensus.pb.go index fb70b6c67..bc83fd472 100644 --- a/core/corepb/v1/consensus.pb.go +++ b/core/corepb/v1/consensus.pb.go @@ -123,7 +123,7 @@ func (x *QBFTMsg) GetPreparedValueHash() []byte { return nil } -type ConsensusMsg struct { +type QBFTConsensusMsg struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -133,20 +133,20 @@ type ConsensusMsg struct { Values []*anypb.Any `protobuf:"bytes,3,rep,name=values,proto3" json:"values,omitempty"` // values of the hashes in the messages } -func (x *ConsensusMsg) Reset() { - *x = ConsensusMsg{} +func (x *QBFTConsensusMsg) Reset() { + *x = QBFTConsensusMsg{} mi := &file_core_corepb_v1_consensus_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ConsensusMsg) String() string { +func (x *QBFTConsensusMsg) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ConsensusMsg) ProtoMessage() {} +func (*QBFTConsensusMsg) ProtoMessage() {} -func (x *ConsensusMsg) ProtoReflect() protoreflect.Message { +func (x *QBFTConsensusMsg) ProtoReflect() protoreflect.Message { mi := &file_core_corepb_v1_consensus_proto_msgTypes[1] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -158,26 +158,26 @@ func (x *ConsensusMsg) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ConsensusMsg.ProtoReflect.Descriptor instead. -func (*ConsensusMsg) Descriptor() ([]byte, []int) { +// Deprecated: Use QBFTConsensusMsg.ProtoReflect.Descriptor instead. +func (*QBFTConsensusMsg) Descriptor() ([]byte, []int) { return file_core_corepb_v1_consensus_proto_rawDescGZIP(), []int{1} } -func (x *ConsensusMsg) GetMsg() *QBFTMsg { +func (x *QBFTConsensusMsg) GetMsg() *QBFTMsg { if x != nil { return x.Msg } return nil } -func (x *ConsensusMsg) GetJustification() []*QBFTMsg { +func (x *QBFTConsensusMsg) GetJustification() []*QBFTMsg { if x != nil { return x.Justification } return nil } -func (x *ConsensusMsg) GetValues() []*anypb.Any { +func (x *QBFTConsensusMsg) GetValues() []*anypb.Any { if x != nil { return x.Values } @@ -190,7 +190,7 @@ type SniffedConsensusMsg struct { unknownFields protoimpl.UnknownFields Timestamp *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=timestamp,proto3" json:"timestamp,omitempty"` - Msg *ConsensusMsg `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` + Msg *QBFTConsensusMsg `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` // Other consensus protocol messages can be added here } func (x *SniffedConsensusMsg) Reset() { @@ -230,7 +230,7 @@ func (x *SniffedConsensusMsg) GetTimestamp() *timestamppb.Timestamp { return nil } -func (x *SniffedConsensusMsg) GetMsg() *ConsensusMsg { +func (x *SniffedConsensusMsg) GetMsg() *QBFTConsensusMsg { if x != nil { return x.Msg } @@ -242,10 +242,11 @@ type SniffedConsensusInstance struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - StartedAt *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=started_at,json=startedAt,proto3" json:"started_at,omitempty"` - Nodes int64 `protobuf:"varint,2,opt,name=nodes,proto3" json:"nodes,omitempty"` - PeerIdx int64 `protobuf:"varint,3,opt,name=peer_idx,json=peerIdx,proto3" json:"peer_idx,omitempty"` - Msgs []*SniffedConsensusMsg `protobuf:"bytes,4,rep,name=msgs,proto3" json:"msgs,omitempty"` + StartedAt *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=started_at,json=startedAt,proto3" json:"started_at,omitempty"` + Nodes int64 `protobuf:"varint,2,opt,name=nodes,proto3" json:"nodes,omitempty"` + PeerIdx int64 `protobuf:"varint,3,opt,name=peer_idx,json=peerIdx,proto3" json:"peer_idx,omitempty"` + Msgs []*SniffedConsensusMsg `protobuf:"bytes,4,rep,name=msgs,proto3" json:"msgs,omitempty"` + ProtocolId string `protobuf:"bytes,5,opt,name=protocol_id,json=protocolId,proto3" json:"protocol_id,omitempty"` } func (x *SniffedConsensusInstance) Reset() { @@ -306,6 +307,13 @@ func (x *SniffedConsensusInstance) GetMsgs() []*SniffedConsensusMsg { return nil } +func (x *SniffedConsensusInstance) GetProtocolId() string { + if x != nil { + return x.ProtocolId + } + return "" +} + type SniffedConsensusInstances struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -388,50 +396,52 @@ var file_core_corepb_v1_consensus_proto_rawDesc = []byte{ 0x65, 0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x11, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x61, 0x73, 0x68, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, 0x4a, 0x04, 0x08, 0x07, - 0x10, 0x08, 0x4a, 0x04, 0x08, 0x09, 0x10, 0x0a, 0x4a, 0x04, 0x08, 0x0a, 0x10, 0x0b, 0x22, 0xa6, - 0x01, 0x0a, 0x0c, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x4d, 0x73, 0x67, 0x12, - 0x29, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, - 0x6f, 0x72, 0x65, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x42, - 0x46, 0x54, 0x4d, 0x73, 0x67, 0x52, 0x03, 0x6d, 0x73, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x6a, 0x75, - 0x73, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2e, - 0x76, 0x31, 0x2e, 0x51, 0x42, 0x46, 0x54, 0x4d, 0x73, 0x67, 0x52, 0x0d, 0x6a, 0x75, 0x73, 0x74, - 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x0a, 0x06, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, - 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x7f, 0x0a, 0x13, 0x53, 0x6e, 0x69, 0x66, 0x66, - 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x4d, 0x73, 0x67, 0x12, 0x38, - 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2e, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, - 0x4d, 0x73, 0x67, 0x52, 0x03, 0x6d, 0x73, 0x67, 0x22, 0xbf, 0x01, 0x0a, 0x18, 0x53, 0x6e, 0x69, - 0x66, 0x66, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x49, 0x6e, 0x73, - 0x74, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, - 0x5f, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x41, 0x74, - 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, - 0x64, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, - 0x78, 0x12, 0x37, 0x0a, 0x04, 0x6d, 0x73, 0x67, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x23, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2e, 0x76, 0x31, - 0x2e, 0x53, 0x6e, 0x69, 0x66, 0x66, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, - 0x73, 0x4d, 0x73, 0x67, 0x52, 0x04, 0x6d, 0x73, 0x67, 0x73, 0x22, 0x7e, 0x0a, 0x19, 0x53, 0x6e, - 0x69, 0x66, 0x66, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x49, 0x6e, - 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x12, 0x46, 0x0a, 0x09, 0x69, 0x6e, 0x73, 0x74, 0x61, - 0x6e, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6e, 0x69, 0x66, - 0x66, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x49, 0x6e, 0x73, 0x74, - 0x61, 0x6e, 0x63, 0x65, 0x52, 0x09, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x12, - 0x19, 0x0a, 0x08, 0x67, 0x69, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x67, 0x69, 0x74, 0x48, 0x61, 0x73, 0x68, 0x42, 0x2e, 0x5a, 0x2c, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6f, 0x62, 0x6f, 0x6c, 0x6e, 0x65, 0x74, - 0x77, 0x6f, 0x72, 0x6b, 0x2f, 0x63, 0x68, 0x61, 0x72, 0x6f, 0x6e, 0x2f, 0x63, 0x6f, 0x72, 0x65, - 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x10, 0x08, 0x4a, 0x04, 0x08, 0x09, 0x10, 0x0a, 0x4a, 0x04, 0x08, 0x0a, 0x10, 0x0b, 0x22, 0xaa, + 0x01, 0x0a, 0x10, 0x51, 0x42, 0x46, 0x54, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, + 0x4d, 0x73, 0x67, 0x12, 0x29, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2e, 0x76, + 0x31, 0x2e, 0x51, 0x42, 0x46, 0x54, 0x4d, 0x73, 0x67, 0x52, 0x03, 0x6d, 0x73, 0x67, 0x12, 0x3d, + 0x0a, 0x0d, 0x6a, 0x75, 0x73, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x42, 0x46, 0x54, 0x4d, 0x73, 0x67, 0x52, 0x0d, + 0x6a, 0x75, 0x73, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x0a, + 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x41, 0x6e, 0x79, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x83, 0x01, 0x0a, 0x13, + 0x53, 0x6e, 0x69, 0x66, 0x66, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, + 0x4d, 0x73, 0x67, 0x12, 0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x32, 0x0a, + 0x03, 0x6d, 0x73, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x42, 0x46, 0x54, + 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x4d, 0x73, 0x67, 0x52, 0x03, 0x6d, 0x73, + 0x67, 0x22, 0xe0, 0x01, 0x0a, 0x18, 0x53, 0x6e, 0x69, 0x66, 0x66, 0x65, 0x64, 0x43, 0x6f, 0x6e, + 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x39, + 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x64, + 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, + 0x19, 0x0a, 0x08, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x07, 0x70, 0x65, 0x65, 0x72, 0x49, 0x64, 0x78, 0x12, 0x37, 0x0a, 0x04, 0x6d, 0x73, + 0x67, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6e, 0x69, 0x66, 0x66, 0x65, + 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x4d, 0x73, 0x67, 0x52, 0x04, 0x6d, + 0x73, 0x67, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x5f, + 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, + 0x6f, 0x6c, 0x49, 0x64, 0x22, 0x7e, 0x0a, 0x19, 0x53, 0x6e, 0x69, 0x66, 0x66, 0x65, 0x64, 0x43, + 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, + 0x73, 0x12, 0x46, 0x0a, 0x09, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x63, 0x6f, 0x72, 0x65, + 0x70, 0x62, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x6e, 0x69, 0x66, 0x66, 0x65, 0x64, 0x43, 0x6f, 0x6e, + 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x09, + 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x67, 0x69, 0x74, + 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x67, 0x69, 0x74, + 0x48, 0x61, 0x73, 0x68, 0x42, 0x2e, 0x5a, 0x2c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6f, 0x62, 0x6f, 0x6c, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x2f, 0x63, + 0x68, 0x61, 0x72, 0x6f, 0x6e, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x70, + 0x62, 0x2f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -449,7 +459,7 @@ func file_core_corepb_v1_consensus_proto_rawDescGZIP() []byte { var file_core_corepb_v1_consensus_proto_msgTypes = make([]protoimpl.MessageInfo, 5) var file_core_corepb_v1_consensus_proto_goTypes = []any{ (*QBFTMsg)(nil), // 0: core.corepb.v1.QBFTMsg - (*ConsensusMsg)(nil), // 1: core.corepb.v1.ConsensusMsg + (*QBFTConsensusMsg)(nil), // 1: core.corepb.v1.QBFTConsensusMsg (*SniffedConsensusMsg)(nil), // 2: core.corepb.v1.SniffedConsensusMsg (*SniffedConsensusInstance)(nil), // 3: core.corepb.v1.SniffedConsensusInstance (*SniffedConsensusInstances)(nil), // 4: core.corepb.v1.SniffedConsensusInstances @@ -459,11 +469,11 @@ var file_core_corepb_v1_consensus_proto_goTypes = []any{ } var file_core_corepb_v1_consensus_proto_depIdxs = []int32{ 5, // 0: core.corepb.v1.QBFTMsg.duty:type_name -> core.corepb.v1.Duty - 0, // 1: core.corepb.v1.ConsensusMsg.msg:type_name -> core.corepb.v1.QBFTMsg - 0, // 2: core.corepb.v1.ConsensusMsg.justification:type_name -> core.corepb.v1.QBFTMsg - 6, // 3: core.corepb.v1.ConsensusMsg.values:type_name -> google.protobuf.Any + 0, // 1: core.corepb.v1.QBFTConsensusMsg.msg:type_name -> core.corepb.v1.QBFTMsg + 0, // 2: core.corepb.v1.QBFTConsensusMsg.justification:type_name -> core.corepb.v1.QBFTMsg + 6, // 3: core.corepb.v1.QBFTConsensusMsg.values:type_name -> google.protobuf.Any 7, // 4: core.corepb.v1.SniffedConsensusMsg.timestamp:type_name -> google.protobuf.Timestamp - 1, // 5: core.corepb.v1.SniffedConsensusMsg.msg:type_name -> core.corepb.v1.ConsensusMsg + 1, // 5: core.corepb.v1.SniffedConsensusMsg.msg:type_name -> core.corepb.v1.QBFTConsensusMsg 7, // 6: core.corepb.v1.SniffedConsensusInstance.started_at:type_name -> google.protobuf.Timestamp 2, // 7: core.corepb.v1.SniffedConsensusInstance.msgs:type_name -> core.corepb.v1.SniffedConsensusMsg 3, // 8: core.corepb.v1.SniffedConsensusInstances.instances:type_name -> core.corepb.v1.SniffedConsensusInstance diff --git a/core/corepb/v1/consensus.proto b/core/corepb/v1/consensus.proto index 289c5d98c..b20995c5b 100644 --- a/core/corepb/v1/consensus.proto +++ b/core/corepb/v1/consensus.proto @@ -13,9 +13,9 @@ message QBFTMsg { core.corepb.v1.Duty duty = 2; int64 peer_idx = 3; int64 round = 4; - reserved 5 ; + reserved 5; int64 prepared_round = 6; - reserved 7 ; + reserved 7; bytes signature = 8; reserved 9; reserved 10; @@ -23,7 +23,7 @@ message QBFTMsg { bytes prepared_value_hash = 12; } -message ConsensusMsg { +message QBFTConsensusMsg { QBFTMsg msg = 1; // msg is the message that we send repeated QBFTMsg justification = 2; // justification is the justifications from others for the message repeated google.protobuf.Any values = 3; // values of the hashes in the messages @@ -31,17 +31,19 @@ message ConsensusMsg { message SniffedConsensusMsg { google.protobuf.Timestamp timestamp = 1; - ConsensusMsg msg = 2; + QBFTConsensusMsg msg = 2; + // Other consensus protocol messages can be added here } message SniffedConsensusInstance { - google.protobuf.Timestamp started_at = 1; - int64 nodes = 2; - int64 peer_idx = 3; - repeated SniffedConsensusMsg msgs = 4; + google.protobuf.Timestamp started_at = 1; + int64 nodes = 2; + int64 peer_idx = 3; + repeated SniffedConsensusMsg msgs = 4; + string protocol_id = 5; } message SniffedConsensusInstances { repeated SniffedConsensusInstance instances = 1; - string git_hash = 2; + string git_hash = 2; } diff --git a/p2p/sender_internal_test.go b/p2p/sender_internal_test.go index a2a972f2a..3b8015605 100644 --- a/p2p/sender_internal_test.go +++ b/p2p/sender_internal_test.go @@ -138,7 +138,7 @@ func TestProtocolPrefix(b *testing.T) { func TestIsZeroProto(t *testing.T) { for _, msg := range []proto.Message{ new(pbv1.Duty), - new(pbv1.ConsensusMsg), + new(pbv1.QBFTConsensusMsg), new(timestamppb.Timestamp), } { require.False(t, isZeroProto(nil)) From 31e90fa3aa11c858c8e9a0af669605bd321274b1 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Mon, 14 Oct 2024 13:11:32 +0300 Subject: [PATCH 13/20] Added docs/consensus.md draft --- docs/README.md | 1 + docs/consensus.md | 68 +++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 69 insertions(+) create mode 100644 docs/consensus.md diff --git a/docs/README.md b/docs/README.md index 8bc6bddde..30a414217 100644 --- a/docs/README.md +++ b/docs/README.md @@ -11,3 +11,4 @@ This page acts as an index for the charon (pronounced 'kharon') markdown documen - [Go Guidelines](goguidelines.md): Guidelines and principals relating to go development - [Contributing](contributing.md): How to contribute to charon; githooks, PR templates, etc. - [Distributed Key Generation](dkg.md): How charon can create distributed validator key shares remotely from a cluster-definition file. +- [Consensus](consensus.md): How charon handles various consensus protocols. diff --git a/docs/consensus.md b/docs/consensus.md new file mode 100644 index 000000000..5b096ba0c --- /dev/null +++ b/docs/consensus.md @@ -0,0 +1,68 @@ +# Consensus + +This document describes how Charon handles various consensus protocols. + +## Overview + +Historically, Charon has supported the single consensus protocol QBFT v2.0. +However, now the consensus layer has pluggable interface which allows running different consensus protocols as long as they are available and accepted by cluster's quorum. + +## Consensus Protocol Selection + +The cluster nodes must agree on the preferred consensus protocol to use, otherwise the entire consensus would fail. +Each node, depending on its configuration and software version may prefer one or more consensus protocols in a specific order of preference. +Charon runs the special protocol called Priority which achieves consensus on the preferred consensus protocol to use. +Under the hood this protocol uses the existing QBFT v2.0 algorithm that is known to be present since v0.19 up until now and must not be deprecated. + +### Priority Protocol Input and Output + +The input to the Priority protocol is a list of protocols defined in the order of precedence, e.g.: +``` +[ + "/charon/consensus/hotstuff/1.0.0", // Highest precedence + "/charon/consensus/abft/2.0.0", + "/charon/consensus/abft/1.0.0", + "/charon/consensus/qbft/2.0.0", // Lowest precedence and the last resort +] +``` + +The output of the Priority protocol is the common "subset" of all inputs respecting the initial order of precedence, e.g.: +``` +[ + "/charon/consensus/abft/1.0.0", // This means the quorum of nodes has this protocol in common + "/charon/consensus/qbft/2.0.0", +] +``` + +Eventually, more nodes will upgrade and therefore start preferring newest protocols, which will change the output. Because we know that all nodes must at least support QBFT v2.0, it becomes the last resort option in the list and the "default" protocol. This way, Priority protocol would never get stuck and can't produce an empty output. + +The Priority protocol runs once per epoch and changes its output depending on the inputs. If another protocol started to appear at the top of the list, Charon would switch the consensus protocol to that one with the next epoch. + +### Changing Consensus Protocol Preference + +A cluster creator can specify the preferred consensus protocol in the cluster configuration file. This new field `consensus_protocol` appeared in the cluster definition file from v1.9 onwards. The field is optional and if not specified, the cluster definition will not impact the consensus protocol selection. + +A node operator can also specify the preferred consensus protocol using the new CLI flag `--consensus-protocol` which has the same effect as the cluster configuration file, but it has a higher precedence. The flag is also optional. + +In both cases, a user is supposed to specify the protocol family name, e.g. `abft` string and not a fully-qualified ID. The precise version of the protocol is to be determined by the Priority protocol. +To list all available consensus protocols (with versions), a user can run the command `charon version --verbose`. + +When a node starts, it sequentially mutates the list of preferred consensus protocols by processing the cluster configuration file and then the mentioned CLI flag. The final list of preferred protocols is then passed to the Priority protocol for cluster-wide consensus. Until the Priority protocol reaches consensus, the cluster will use the default QBFT v2.0 protocol. + +## Observability + +The four existing metrics are reflecting the consensus layer behavior: + +- `core_consensus_decided_rounds` +- `core_consensus_duration_seconds` +- `core_consensus_error_total` +- `core_consensus_timeout_total` + +With the new capability to run different consensus protocols, all these metrics now populate the `protocol` label which allows distinguishing between different protocols. +Note that a cluster may run at most two different consensus protocols at the same time, e.g. QBFT v2.0 for Priority and HotStuff v1.0 for validator duties. +Therefore the mentioned metrics will have at most two unique values in the `protocol` label. + +## Debugging + +Charon will handle `/debug/consensus` HTTP endpoint that would respond with `consensus_messages.pb.gz` file containing some number of the last consensus messages (protobuf format). +All consensus messages are tagged with the corresponding protocol id, in case of multiple protocols running at the same time. From 4b1f239ae0d1e4b221581cc1f930323d1073383c Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 16 Oct 2024 09:01:32 +0300 Subject: [PATCH 14/20] Addressed PR feedback --- core/consensus/wrapper_internal_test.go | 2 +- docs/consensus.md | 6 ++++-- go.mod | 1 - go.sum | 2 -- 4 files changed, 5 insertions(+), 6 deletions(-) diff --git a/core/consensus/wrapper_internal_test.go b/core/consensus/wrapper_internal_test.go index d95b781ab..960c98d23 100644 --- a/core/consensus/wrapper_internal_test.go +++ b/core/consensus/wrapper_internal_test.go @@ -7,8 +7,8 @@ import ( "testing" "github.com/libp2p/go-libp2p/core/protocol" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" - "github.com/test-go/testify/mock" "github.com/obolnetwork/charon/core" "github.com/obolnetwork/charon/core/consensus/protocols" diff --git a/docs/consensus.md b/docs/consensus.md index 5b096ba0c..5e4f08c98 100644 --- a/docs/consensus.md +++ b/docs/consensus.md @@ -17,7 +17,8 @@ Under the hood this protocol uses the existing QBFT v2.0 algorithm that is known ### Priority Protocol Input and Output The input to the Priority protocol is a list of protocols defined in the order of precedence, e.g.: -``` + +```json [ "/charon/consensus/hotstuff/1.0.0", // Highest precedence "/charon/consensus/abft/2.0.0", @@ -27,7 +28,8 @@ The input to the Priority protocol is a list of protocols defined in the order o ``` The output of the Priority protocol is the common "subset" of all inputs respecting the initial order of precedence, e.g.: -``` + +```json [ "/charon/consensus/abft/1.0.0", // This means the quorum of nodes has this protocol in common "/charon/consensus/qbft/2.0.0", diff --git a/go.mod b/go.mod index 88f7f881f..734edf787 100644 --- a/go.mod +++ b/go.mod @@ -31,7 +31,6 @@ require ( github.com/spf13/pflag v1.0.5 github.com/spf13/viper v1.19.0 github.com/stretchr/testify v1.9.0 - github.com/test-go/testify v1.1.4 github.com/wealdtech/go-eth2-wallet-encryptor-keystorev4 v1.4.1 go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.55.0 go.opentelemetry.io/otel v1.30.0 diff --git a/go.sum b/go.sum index e61e3f81e..ed9f554e8 100644 --- a/go.sum +++ b/go.sum @@ -529,8 +529,6 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/subosito/gotenv v1.6.0 h1:9NlTDc1FTs4qu0DDq7AEtTPNw6SVm7uBMsUCUjABIf8= github.com/subosito/gotenv v1.6.0/go.mod h1:Dk4QP5c2W3ibzajGcXpNraDfq2IrhjMIvMSWPKKo0FU= github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA= -github.com/test-go/testify v1.1.4 h1:Tf9lntrKUMHiXQ07qBScBTSA0dhYQlu83hswqelv1iE= -github.com/test-go/testify v1.1.4/go.mod h1:rH7cfJo/47vWGdi4GPj16x3/t1xGOj2YxzmNQzk2ghU= github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/vbatts/tar-split v0.11.5 h1:3bHCTIheBm1qFTcgh9oPu+nNBtX+XJIupG/vacinCts= github.com/vbatts/tar-split v0.11.5/go.mod h1:yZbwRsSeGjusneWgA781EKej9HF8vme8okylkAeNKLk= From 75321cb44a6c9ecdc6529a12497550623f7b2b4b Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 16 Oct 2024 10:03:28 +0300 Subject: [PATCH 15/20] Addressed PR feedback --- core/consensus/metrics/metrics.go | 58 ++++++++++++++++++++++--------- core/consensus/qbft/qbft.go | 22 ++++++++++-- docs/consensus.md | 45 +++++++++++++++--------- docs/metrics.md | 4 --- 4 files changed, 89 insertions(+), 40 deletions(-) diff --git a/core/consensus/metrics/metrics.go b/core/consensus/metrics/metrics.go index 22af8e0ec..27d7fab21 100644 --- a/core/consensus/metrics/metrics.go +++ b/core/consensus/metrics/metrics.go @@ -4,38 +4,36 @@ package metrics import ( "github.com/prometheus/client_golang/prometheus" - - "github.com/obolnetwork/charon/app/promauto" + "github.com/prometheus/client_golang/prometheus/promauto" ) -type ConsensusMetrics interface { - SetDecidedRounds(duty, timer string, rounds float64) - ObserveConsensusDuration(duty, timer string, duration float64) - IncConsensusTimeout(duty, timer string) - IncConsensusError() -} - var ( decidedRoundsGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ Namespace: "core", Subsystem: "consensus", Name: "decided_rounds", - Help: "Number of rounds it took to decide consensus instances by protocol, duty and timer type.", - }, []string{"protocol", "duty", "timer"}) // Using gauge since the value changes slowly, once per slot. + Help: "Number of decided rounds by protocol, duty, and timer", + }, []string{"protocol", "duty", "timer"}) + + decidedLeaderGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "core", + Subsystem: "consensus", + Name: "decided_leader_index", + Help: "Index of the decided leader by protocol", + }, []string{"protocol"}) consensusDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "core", Subsystem: "consensus", Name: "duration_seconds", - Help: "Duration of a consensus instance in seconds by protocol, duty and timer type.", - Buckets: []float64{.05, .1, .25, .5, 1, 2.5, 5, 10, 20, 30, 60}, + Help: "Duration of the consensus process by protocol, duty, and timer", }, []string{"protocol", "duty", "timer"}) consensusTimeout = promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: "core", Subsystem: "consensus", Name: "timeout_total", - Help: "Total count of consensus timeouts by protocol, duty and timer type.", + Help: "Total count of consensus timeouts by protocol, duty, and timer", }, []string{"protocol", "duty", "timer"}) consensusError = promauto.NewCounterVec(prometheus.CounterOpts{ @@ -46,28 +44,56 @@ var ( }, []string{"protocol"}) ) +// ConsensusMetrics defines the interface for consensus metrics. +type ConsensusMetrics interface { + // SetDecidedRounds sets the number of decided rounds for a given duty and timer. + SetDecidedRounds(duty, timer string, rounds int64) + + // SetDecidedLeaderIndex sets the decided leader index. + SetDecidedLeaderIndex(leaderIndex int64) + + // ObserveConsensusDuration observes the duration of the consensus process for a given duty and timer. + ObserveConsensusDuration(duty, timer string, duration float64) + + // IncConsensusTimeout increments the consensus timeout counter for a given duty and timer. + IncConsensusTimeout(duty, timer string) + + // IncConsensusError increments the consensus error counter. + IncConsensusError() +} + type consensusMetrics struct { protocolID string } +// NewConsensusMetrics creates a new instance of ConsensusMetrics with the given protocol ID. func NewConsensusMetrics(protocolID string) ConsensusMetrics { return &consensusMetrics{ protocolID: protocolID, } } -func (m *consensusMetrics) SetDecidedRounds(duty, timer string, rounds float64) { - decidedRoundsGauge.WithLabelValues(m.protocolID, duty, timer).Set(rounds) +// SetDecidedRounds sets the number of decided rounds for a given duty and timer. +func (m *consensusMetrics) SetDecidedRounds(duty, timer string, rounds int64) { + decidedRoundsGauge.WithLabelValues(m.protocolID, duty, timer).Set(float64(rounds)) +} + +// SetDecidedLeaderIndex sets the decided leader index. +func (m *consensusMetrics) SetDecidedLeaderIndex(leaderIndex int64) { + decidedLeaderGauge.WithLabelValues(m.protocolID).Set(float64(leaderIndex)) } +// ObserveConsensusDuration observes the duration of the consensus process for a given duty and timer. func (m *consensusMetrics) ObserveConsensusDuration(duty, timer string, duration float64) { consensusDuration.WithLabelValues(m.protocolID, duty, timer).Observe(duration) } +// IncConsensusTimeout increments the consensus timeout counter for a given duty and timer. func (m *consensusMetrics) IncConsensusTimeout(duty, timer string) { consensusTimeout.WithLabelValues(m.protocolID, duty, timer).Inc() } +// IncConsensusError increments the consensus error counter. func (m *consensusMetrics) IncConsensusError() { consensusError.WithLabelValues(m.protocolID).Inc() } diff --git a/core/consensus/qbft/qbft.go b/core/consensus/qbft/qbft.go index 7d69df495..d1efd86a4 100644 --- a/core/consensus/qbft/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -382,11 +382,27 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) } // Instrument consensus instance. - var decided bool + var ( + decided bool + nodes = len(c.peers) + ) + decideCallback := func(qcommit []qbft.Msg[core.Duty, [32]byte]) { + round := qcommit[0].Round() decided = true - c.metrics.SetDecidedRounds(duty.Type.String(), string(roundTimer.Type()), float64(qcommit[0].Round())) inst.DecidedAtCh <- time.Now() + + leaderIndex := leader(duty, round, nodes) + leaderName := c.peers[leaderIndex].Name + log.Debug(ctx, "QBFT consensus decided", + z.Str("duty", duty.Type.String()), + z.U64("slot", duty.Slot), + z.I64("round", round), + z.I64("leader_index", leaderIndex), + z.Str("leader_name", leaderName)) + + c.metrics.SetDecidedLeaderIndex(leaderIndex) + c.metrics.SetDecidedRounds(duty.Type.String(), string(roundTimer.Type()), round) } // Create a new qbft definition for this instance. @@ -410,7 +426,7 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) } // Run the algo, blocking until the context is cancelled. - err = qbft.Run[core.Duty, [32]byte](ctx, def, qt, duty, peerIdx, inst.HashCh) + err = qbft.Run(ctx, def, qt, duty, peerIdx, inst.HashCh) if err != nil && !isContextErr(err) { c.metrics.IncConsensusError() return err // Only return non-context errors. diff --git a/docs/consensus.md b/docs/consensus.md index 5e4f08c98..dfdc08025 100644 --- a/docs/consensus.md +++ b/docs/consensus.md @@ -5,40 +5,41 @@ This document describes how Charon handles various consensus protocols. ## Overview Historically, Charon has supported the single consensus protocol QBFT v2.0. -However, now the consensus layer has pluggable interface which allows running different consensus protocols as long as they are available and accepted by cluster's quorum. +However, now the consensus layer has a pluggable interface that allows running different consensus protocols as long as they are available and accepted by the majority of the cluster. Moreover, the cluster can run multiple consensus protocols at the same time, e.g., for different purposes. ## Consensus Protocol Selection -The cluster nodes must agree on the preferred consensus protocol to use, otherwise the entire consensus would fail. -Each node, depending on its configuration and software version may prefer one or more consensus protocols in a specific order of preference. -Charon runs the special protocol called Priority which achieves consensus on the preferred consensus protocol to use. -Under the hood this protocol uses the existing QBFT v2.0 algorithm that is known to be present since v0.19 up until now and must not be deprecated. +The cluster nodes must agree on the preferred consensus protocol to use, otherwise, the entire consensus will fail. +Each node, depending on its configuration and software version, may prefer one or more consensus protocols in a specific order of precedence. +Charon runs a special protocol called Priority, which achieves consensus on the preferred consensus protocol to use. +Under the hood, this protocol uses the existing QBFT v2.0 algorithm that has been present since v0.19 and must not be deprecated. +This way, the existing QBFT v2.0 remains present for all future Charon versions to serve two purposes: running the Priority protocol and being a fallback protocol if no other protocol is selected. ### Priority Protocol Input and Output -The input to the Priority protocol is a list of protocols defined in the order of precedence, e.g.: +The input to the Priority protocol is a list of protocols defined in order of precedence, e.g.: ```json [ "/charon/consensus/hotstuff/1.0.0", // Highest precedence "/charon/consensus/abft/2.0.0", "/charon/consensus/abft/1.0.0", - "/charon/consensus/qbft/2.0.0", // Lowest precedence and the last resort + "/charon/consensus/qbft/2.0.0", // Lowest precedence and the fallback since it is always present ] ``` -The output of the Priority protocol is the common "subset" of all inputs respecting the initial order of precedence, e.g.: +The output of the Priority protocol is the common "subset" of all inputs, respecting the initial order of precedence, e.g.: ```json [ - "/charon/consensus/abft/1.0.0", // This means the quorum of nodes has this protocol in common + "/charon/consensus/abft/1.0.0", // This means the majority of nodes have this protocol available "/charon/consensus/qbft/2.0.0", ] ``` -Eventually, more nodes will upgrade and therefore start preferring newest protocols, which will change the output. Because we know that all nodes must at least support QBFT v2.0, it becomes the last resort option in the list and the "default" protocol. This way, Priority protocol would never get stuck and can't produce an empty output. +Eventually, more nodes will upgrade and therefore start preferring newer protocols, which will change the output. Because we know that all nodes must at least support QBFT v2.0, it becomes the fallback option in the list and the "default" protocol. This way, the Priority protocol will never get stuck and can't produce an empty output. -The Priority protocol runs once per epoch and changes its output depending on the inputs. If another protocol started to appear at the top of the list, Charon would switch the consensus protocol to that one with the next epoch. +The Priority protocol runs once per epoch (the last slot of each epoch) and changes its output depending on the inputs. If another protocol starts to appear at the top of the list, Charon will switch the consensus protocol to that one starting in the next epoch. ### Changing Consensus Protocol Preference @@ -46,25 +47,35 @@ A cluster creator can specify the preferred consensus protocol in the cluster co A node operator can also specify the preferred consensus protocol using the new CLI flag `--consensus-protocol` which has the same effect as the cluster configuration file, but it has a higher precedence. The flag is also optional. -In both cases, a user is supposed to specify the protocol family name, e.g. `abft` string and not a fully-qualified ID. The precise version of the protocol is to be determined by the Priority protocol. +In both cases, a user is supposed to specify the protocol family name, e.g. `abft` string and not a fully-qualified protocol ID. +The precise version of the protocol is to be determined by the Priority protocol, which will try picking the latest version. To list all available consensus protocols (with versions), a user can run the command `charon version --verbose`. -When a node starts, it sequentially mutates the list of preferred consensus protocols by processing the cluster configuration file and then the mentioned CLI flag. The final list of preferred protocols is then passed to the Priority protocol for cluster-wide consensus. Until the Priority protocol reaches consensus, the cluster will use the default QBFT v2.0 protocol. +When a node starts, it sequentially mutates the list of preferred consensus protocols by processing the cluster configuration file and then the mentioned CLI flag. The final list of preferred protocols is then passed to the Priority protocol for cluster-wide consensus. Until the Priority protocol reaches consensus, the cluster will use the default QBFT v2.0 protocol for any duties. ## Observability The four existing metrics are reflecting the consensus layer behavior: - `core_consensus_decided_rounds` +- `core_consensus_decided_leader_index` - `core_consensus_duration_seconds` - `core_consensus_error_total` - `core_consensus_timeout_total` With the new capability to run different consensus protocols, all these metrics now populate the `protocol` label which allows distinguishing between different protocols. -Note that a cluster may run at most two different consensus protocols at the same time, e.g. QBFT v2.0 for Priority and HotStuff v1.0 for validator duties. -Therefore the mentioned metrics will have at most two unique values in the `protocol` label. +Note that a cluster may run at most two different consensus protocols at the same time, e.g. QBFT v2.0 for Priority and HotStuff v1.0 for validator duties. But this can be changed in the future and more different protocols can be running at the same time. +Therefore the mentioned metrics may have different unique values for the `protocol` label. + +Some protocols may export their own metrics. We agreed that all such metrics should be prefixed with the protocol name, e.g. `core_consensus_hotstuff_xyz`. ## Debugging -Charon will handle `/debug/consensus` HTTP endpoint that would respond with `consensus_messages.pb.gz` file containing some number of the last consensus messages (protobuf format). -All consensus messages are tagged with the corresponding protocol id, in case of multiple protocols running at the same time. +Charon handles `/debug/consensus` HTTP endpoint that responds with `consensus_messages.pb.gz` file containing certain number of the last consensus messages (in protobuf format). +All consensus messages are tagged with the corresponding protocol ID, in case of multiple protocols running at the same time. + +## Protocol Specific Configuration + +Each consensus protocol may have its own configuration parameters. For instance, QBFT v2.0 has two parameters: `eager_double_linear` and `consensus_participate` that users control via Feature set. +For future protocols we decided to follow the same design and allow users to control the protocol-specific parameters via Feature set. +Charon will set the recommended default values to all such parameters, so node operators don't need to override them unless they know what they are doing. Note that Priority protocol does not take into account any variations caused by different parameters, therefore node operators must be careful when changing them and make sure all nodes have the same configuration. diff --git a/docs/metrics.md b/docs/metrics.md index 41ceb5117..04677ed40 100644 --- a/docs/metrics.md +++ b/docs/metrics.md @@ -43,10 +43,6 @@ when storing metrics from multiple nodes or clusters in one Prometheus instance. | `core_bcast_recast_errors_total` | Counter | The total count of failed recasted registrations by source; `pregen` vs `downstream` | `source` | | `core_bcast_recast_registration_total` | Counter | The total number of unique validator registration stored in recaster per pubkey | `pubkey` | | `core_bcast_recast_total` | Counter | The total count of recasted registrations by source; `pregen` vs `downstream` | `source` | -| `core_consensus_decided_rounds` | Gauge | Number of rounds it took to decide consensus instances by protocol, duty and timer type. | `protocol, duty, timer` | -| `core_consensus_duration_seconds` | Histogram | Duration of a consensus instance in seconds by protocol, duty and timer type. | `protocol, duty, timer` | -| `core_consensus_error_total` | Counter | Total count of consensus errors by protocol | `protocol` | -| `core_consensus_timeout_total` | Counter | Total count of consensus timeouts by protocol, duty and timer type. | `protocol, duty, timer` | | `core_parsigdb_exit_total` | Counter | Total number of partially signed voluntary exits per public key | `pubkey` | | `core_scheduler_current_epoch` | Gauge | The current epoch | | | `core_scheduler_current_slot` | Gauge | The current slot | | From bbc08f323995a2399ef7e0f7034d9208484ee36e Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 16 Oct 2024 11:44:39 +0300 Subject: [PATCH 16/20] Addressed PR feedback --- core/consensus/metrics/metrics.go | 17 ++++++------ core/consensus/metrics/metrics_test.go | 37 +++++++++++++++++++++++--- core/consensus/qbft/qbft.go | 2 +- 3 files changed, 44 insertions(+), 12 deletions(-) diff --git a/core/consensus/metrics/metrics.go b/core/consensus/metrics/metrics.go index 27d7fab21..31f864d41 100644 --- a/core/consensus/metrics/metrics.go +++ b/core/consensus/metrics/metrics.go @@ -4,7 +4,8 @@ package metrics import ( "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/obolnetwork/charon/app/promauto" ) var ( @@ -19,8 +20,8 @@ var ( Namespace: "core", Subsystem: "consensus", Name: "decided_leader_index", - Help: "Index of the decided leader by protocol", - }, []string{"protocol"}) + Help: "Index of the decided leader by protocol and duty", + }, []string{"protocol", "duty"}) consensusDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "core", @@ -49,8 +50,8 @@ type ConsensusMetrics interface { // SetDecidedRounds sets the number of decided rounds for a given duty and timer. SetDecidedRounds(duty, timer string, rounds int64) - // SetDecidedLeaderIndex sets the decided leader index. - SetDecidedLeaderIndex(leaderIndex int64) + // SetDecidedLeaderIndex sets the decided leader index for a given duty. + SetDecidedLeaderIndex(duty string, leaderIndex int64) // ObserveConsensusDuration observes the duration of the consensus process for a given duty and timer. ObserveConsensusDuration(duty, timer string, duration float64) @@ -78,9 +79,9 @@ func (m *consensusMetrics) SetDecidedRounds(duty, timer string, rounds int64) { decidedRoundsGauge.WithLabelValues(m.protocolID, duty, timer).Set(float64(rounds)) } -// SetDecidedLeaderIndex sets the decided leader index. -func (m *consensusMetrics) SetDecidedLeaderIndex(leaderIndex int64) { - decidedLeaderGauge.WithLabelValues(m.protocolID).Set(float64(leaderIndex)) +// SetDecidedLeaderIndex sets the decided leader index for a given duty. +func (m *consensusMetrics) SetDecidedLeaderIndex(duty string, leaderIndex int64) { + decidedLeaderGauge.WithLabelValues(m.protocolID, duty).Set(float64(leaderIndex)) } // ObserveConsensusDuration observes the duration of the consensus process for a given duty and timer. diff --git a/core/consensus/metrics/metrics_test.go b/core/consensus/metrics/metrics_test.go index 6eb24d658..3db82427d 100644 --- a/core/consensus/metrics/metrics_test.go +++ b/core/consensus/metrics/metrics_test.go @@ -20,6 +20,19 @@ func TestConsensusMetrics_SetDecidedRounds(t *testing.T) { m := gatherMetric(t, "core_consensus_decided_rounds") require.InEpsilon(t, 1, m.GetMetric()[0].GetGauge().GetValue(), 0.0001) + verifyLabel(t, m.GetMetric()[0].GetLabel(), "protocol", "test") + verifyLabel(t, m.GetMetric()[0].GetLabel(), "duty", "duty") +} + +func TestConsensusMetrics_SetDecidedLeaderIndex(t *testing.T) { + cm := metrics.NewConsensusMetrics("test") + + cm.SetDecidedLeaderIndex("duty", 123) + + m := gatherMetric(t, "core_consensus_decided_leader_index") + require.InEpsilon(t, 123, m.GetMetric()[0].GetGauge().GetValue(), 0.0001) + verifyLabel(t, m.GetMetric()[0].GetLabel(), "protocol", "test") + verifyLabel(t, m.GetMetric()[0].GetLabel(), "duty", "duty") } func TestConsensusMetrics_ObserveConsensusDuration(t *testing.T) { @@ -29,6 +42,9 @@ func TestConsensusMetrics_ObserveConsensusDuration(t *testing.T) { m := gatherMetric(t, "core_consensus_duration_seconds") require.EqualValues(t, 1, m.GetMetric()[0].GetHistogram().GetSampleCount()) + verifyLabel(t, m.GetMetric()[0].GetLabel(), "protocol", "test") + verifyLabel(t, m.GetMetric()[0].GetLabel(), "duty", "duty") + verifyLabel(t, m.GetMetric()[0].GetLabel(), "timer", "timer") } func TestConsensusMetrics_IncConsensusTimeout(t *testing.T) { @@ -38,6 +54,9 @@ func TestConsensusMetrics_IncConsensusTimeout(t *testing.T) { m := gatherMetric(t, "core_consensus_timeout_total") require.InEpsilon(t, 1, m.GetMetric()[0].GetCounter().GetValue(), 0.0001) + verifyLabel(t, m.GetMetric()[0].GetLabel(), "protocol", "test") + verifyLabel(t, m.GetMetric()[0].GetLabel(), "duty", "duty") + verifyLabel(t, m.GetMetric()[0].GetLabel(), "timer", "timer") } func TestConsensusMetrics_IncConsensusError(t *testing.T) { @@ -47,14 +66,13 @@ func TestConsensusMetrics_IncConsensusError(t *testing.T) { m := gatherMetric(t, "core_consensus_error_total") require.InEpsilon(t, 1, m.GetMetric()[0].GetCounter().GetValue(), 0.0001) + verifyLabel(t, m.GetMetric()[0].GetLabel(), "protocol", "test") } func gatherMetric(t *testing.T, name string) *pb.MetricFamily { t.Helper() - labels := prometheus.Labels{} - - registry, err := promauto.NewRegistry(labels) + registry, err := promauto.NewRegistry(prometheus.Labels{}) require.NoError(t, err) mfa, err := registry.Gather() @@ -70,3 +88,16 @@ func gatherMetric(t *testing.T, name string) *pb.MetricFamily { return nil } + +func verifyLabel(t *testing.T, labels []*pb.LabelPair, name, value string) { + t.Helper() + + for _, label := range labels { + if label.GetName() == name { + require.Equal(t, value, label.GetValue()) + return + } + } + + require.Fail(t, "label not found") +} diff --git a/core/consensus/qbft/qbft.go b/core/consensus/qbft/qbft.go index d1efd86a4..19e52b633 100644 --- a/core/consensus/qbft/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -401,7 +401,7 @@ func (c *Consensus) runInstance(ctx context.Context, duty core.Duty) (err error) z.I64("leader_index", leaderIndex), z.Str("leader_name", leaderName)) - c.metrics.SetDecidedLeaderIndex(leaderIndex) + c.metrics.SetDecidedLeaderIndex(duty.Type.String(), leaderIndex) c.metrics.SetDecidedRounds(duty.Type.String(), string(roundTimer.Type()), round) } From 8a897d8577ffef03cabe531666b4594f050e9d0b Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 16 Oct 2024 11:50:14 +0300 Subject: [PATCH 17/20] Updated metrics.md --- docs/metrics.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/metrics.md b/docs/metrics.md index 04677ed40..e91751ef6 100644 --- a/docs/metrics.md +++ b/docs/metrics.md @@ -43,6 +43,11 @@ when storing metrics from multiple nodes or clusters in one Prometheus instance. | `core_bcast_recast_errors_total` | Counter | The total count of failed recasted registrations by source; `pregen` vs `downstream` | `source` | | `core_bcast_recast_registration_total` | Counter | The total number of unique validator registration stored in recaster per pubkey | `pubkey` | | `core_bcast_recast_total` | Counter | The total count of recasted registrations by source; `pregen` vs `downstream` | `source` | +| `core_consensus_decided_leader_index` | Gauge | Index of the decided leader by protocol and duty | `protocol, duty` | +| `core_consensus_decided_rounds` | Gauge | Number of decided rounds by protocol, duty, and timer | `protocol, duty, timer` | +| `core_consensus_duration_seconds` | Histogram | Duration of the consensus process by protocol, duty, and timer | `protocol, duty, timer` | +| `core_consensus_error_total` | Counter | Total count of consensus errors by protocol | `protocol` | +| `core_consensus_timeout_total` | Counter | Total count of consensus timeouts by protocol, duty, and timer | `protocol, duty, timer` | | `core_parsigdb_exit_total` | Counter | Total number of partially signed voluntary exits per public key | `pubkey` | | `core_scheduler_current_epoch` | Gauge | The current epoch | | | `core_scheduler_current_slot` | Gauge | The current slot | | From cb832ed3fbd712ae8d5648de0a461488e97a4e70 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Wed, 16 Oct 2024 16:57:29 +0300 Subject: [PATCH 18/20] More of refactoring --- app/app.go | 20 ++-- core/consensus/controller.go | 97 +++++++++++++++++++ .../{factory_test.go => controller_test.go} | 16 ++- core/consensus/factory.go | 76 --------------- core/consensus/qbft/qbft.go | 31 +++--- core/consensus/qbft/qbft_internal_test.go | 27 +++--- core/consensus/qbft/qbft_test.go | 21 ++-- core/consensus/qbft/sniffed_internal_test.go | 2 +- core/consensus/wrapper.go | 28 ++++-- core/consensus/wrapper_internal_test.go | 6 +- core/interfaces.go | 23 +++-- core/mocks/consensus.go | 16 ++- 12 files changed, 200 insertions(+), 163 deletions(-) create mode 100644 core/consensus/controller.go rename core/consensus/{factory_test.go => controller_test.go} (79%) delete mode 100644 core/consensus/factory.go diff --git a/app/app.go b/app/app.go index 34337c541..58d792bdf 100644 --- a/app/app.go +++ b/app/app.go @@ -526,22 +526,24 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, return err } - retryer := retry.New[core.Duty](deadlineFunc) + retryer := retry.New(deadlineFunc) - consensusFactory, err := consensus.NewConsensusFactory(tcpNode, sender, peers, p2pKey, deadlinerFunc, gaterFunc, consensusDebugger) + // Consensus + consensusDeadliner := deadlinerFunc("consensus") + consensusController, err := consensus.NewConsensusController(tcpNode, sender, peers, p2pKey, consensusDeadliner, gaterFunc, consensusDebugger) if err != nil { return err } - defaultConsensus := consensusFactory.DefaultConsensus() - startDefaultConsensus := lifecycle.HookFuncCtx(defaultConsensus.Start) + defaultConsensus := consensusController.DefaultConsensus() + startConsensusCtrl := lifecycle.HookFuncCtx(consensusController.Start) - coreConsensus := consensusFactory.CurrentConsensus() // points to DefaultConsensus() initially + coreConsensus := consensusController.CurrentConsensus() // initially points to DefaultConsensus() // Priority protocol always uses QBFTv2. err = wirePrioritise(ctx, conf, life, tcpNode, peerIDs, int(cluster.GetThreshold()), sender.SendReceive, defaultConsensus, sched, p2pKey, deadlineFunc, - consensusFactory, cluster.GetConsensusProtocol()) + consensusController, cluster.GetConsensusProtocol()) if err != nil { return err } @@ -579,7 +581,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, } life.RegisterStart(lifecycle.AsyncBackground, lifecycle.StartScheduler, lifecycle.HookFuncErr(sched.Run)) - life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startDefaultConsensus) + life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartP2PConsensus, startConsensusCtrl) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartAggSigDB, lifecycle.HookFuncCtx(aggSigDB.Run)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartParSigDB, lifecycle.HookFuncCtx(parSigDB.Trim)) life.RegisterStart(lifecycle.AsyncAppCtx, lifecycle.StartTracker, lifecycle.HookFuncCtx(inclusion.Run)) @@ -594,7 +596,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, tcpNode host.Host, peers []peer.ID, threshold int, sendFunc p2p.SendReceiveFunc, coreCons core.Consensus, sched core.Scheduler, p2pKey *k1.PrivateKey, deadlineFunc func(duty core.Duty) (time.Time, bool), - consensusFactory core.ConsensusFactory, clusterPreferredProtocol string, + consensusController core.ConsensusController, clusterPreferredProtocol string, ) error { cons, ok := coreCons.(*qbft.Consensus) if !ok { @@ -650,7 +652,7 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t allProtocols := t.PrioritiesOnly() preferredConsensusProtocol := protocols.MostPreferredConsensusProtocol(allProtocols) - if err := consensusFactory.SetCurrentConsensusForProtocol(protocol.ID(preferredConsensusProtocol)); err != nil { + if err := consensusController.SetCurrentConsensusForProtocol(protocol.ID(preferredConsensusProtocol)); err != nil { log.Error(ctx, "Failed to set current consensus protocol", err, z.Str("protocol", preferredConsensusProtocol)) } else { log.Info(ctx, "Current consensus protocol changed", z.Str("protocol", preferredConsensusProtocol)) diff --git a/core/consensus/controller.go b/core/consensus/controller.go new file mode 100644 index 000000000..1f55f98f9 --- /dev/null +++ b/core/consensus/controller.go @@ -0,0 +1,97 @@ +// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 + +package consensus + +import ( + "context" + + k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" + "github.com/libp2p/go-libp2p/core/host" + "github.com/libp2p/go-libp2p/core/protocol" + + "github.com/obolnetwork/charon/app/errors" + "github.com/obolnetwork/charon/core" + "github.com/obolnetwork/charon/core/consensus/qbft" + "github.com/obolnetwork/charon/p2p" +) + +type consensusController struct { + tcpNode host.Host + sender *p2p.Sender + peers []p2p.Peer + p2pKey *k1.PrivateKey + consensusDeadliner core.Deadliner + gaterFunc core.DutyGaterFunc + debugger Debugger + defaultConsensus core.Consensus + wrappedConsensus *consensusWrapper +} + +// NewConsensusController creates a new consensus controller with the default consensus protocol. +func NewConsensusController(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, + consensusDeadliner core.Deadliner, gaterFunc core.DutyGaterFunc, debugger Debugger, +) (core.ConsensusController, error) { + defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, consensusDeadliner, gaterFunc, debugger.AddInstance) + if err != nil { + return nil, err + } + + return &consensusController{ + tcpNode: tcpNode, + sender: sender, + peers: peers, + p2pKey: p2pKey, + consensusDeadliner: consensusDeadliner, + gaterFunc: gaterFunc, + debugger: debugger, + defaultConsensus: defaultConsensus, + wrappedConsensus: newConsensusWrapper(defaultConsensus), + }, nil +} + +// Start starts the internal routines. The controller stops when the context is cancelled. +func (f *consensusController) Start(ctx context.Context) { + // The default protocol remains registered all the time. + f.defaultConsensus.RegisterHandler() + + go func() { + for { + select { + case <-ctx.Done(): + return + case duty := <-f.consensusDeadliner.C(): + f.defaultConsensus.HandleExpiredDuty(duty) + if f.wrappedConsensus.ProtocolID() != f.defaultConsensus.ProtocolID() { + f.wrappedConsensus.HandleExpiredDuty(duty) + } + } + } + }() +} + +// DefaultConsensus returns the default consensus instance. +func (f *consensusController) DefaultConsensus() core.Consensus { + return f.defaultConsensus +} + +// CurrentConsensus returns the current consensus instance. +func (f *consensusController) CurrentConsensus() core.Consensus { + return f.wrappedConsensus +} + +// SetCurrentConsensusForProtocol sets the current consensus instance for the given protocol id. +func (f *consensusController) SetCurrentConsensusForProtocol(protocol protocol.ID) error { + if f.wrappedConsensus.ProtocolID() == protocol { + return nil + } + + if protocol == f.defaultConsensus.ProtocolID() { + f.wrappedConsensus.SetImpl(f.defaultConsensus) + + return nil + } + + // TODO: Call RegisterHandler()/UnregisterHandler() when switching. + + return errors.New("unsupported protocol id") +} diff --git a/core/consensus/factory_test.go b/core/consensus/controller_test.go similarity index 79% rename from core/consensus/factory_test.go rename to core/consensus/controller_test.go index 6e5477d4d..fd53e6c76 100644 --- a/core/consensus/factory_test.go +++ b/core/consensus/controller_test.go @@ -24,7 +24,7 @@ import ( "github.com/obolnetwork/charon/testutil" ) -func TestConsensusFactory(t *testing.T) { +func TestConsensusController(t *testing.T) { var hosts []host.Host var peers []p2p.Peer @@ -54,23 +54,21 @@ func TestConsensusFactory(t *testing.T) { hosts = append(hosts, h) } - deadlinerFunc := func(string) core.Deadliner { - return coremocks.NewDeadliner(t) - } + deadliner := coremocks.NewDeadliner(t) debugger := csmocks.NewDebugger(t) - factory, err := consensus.NewConsensusFactory(hosts[0], new(p2p.Sender), peers, p2pkeys[0], deadlinerFunc, gaterFunc, debugger) + controller, err := consensus.NewConsensusController(hosts[0], new(p2p.Sender), peers, p2pkeys[0], deadliner, gaterFunc, debugger) require.NoError(t, err) - require.NotNil(t, factory) + require.NotNil(t, controller) t.Run("default and current consensus", func(t *testing.T) { - defaultConsensus := factory.DefaultConsensus() + defaultConsensus := controller.DefaultConsensus() require.NotNil(t, defaultConsensus) require.EqualValues(t, protocols.QBFTv2ProtocolID, defaultConsensus.ProtocolID()) - require.NotEqual(t, defaultConsensus, factory.CurrentConsensus()) // because the current is wrapped + require.NotEqual(t, defaultConsensus, controller.CurrentConsensus()) // because the current is wrapped }) t.Run("unsupported protocol id", func(t *testing.T) { - err := factory.SetCurrentConsensusForProtocol("boo") + err := controller.SetCurrentConsensusForProtocol("boo") require.ErrorContains(t, err, "unsupported protocol id") }) } diff --git a/core/consensus/factory.go b/core/consensus/factory.go deleted file mode 100644 index e550b9344..000000000 --- a/core/consensus/factory.go +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright © 2022-2024 Obol Labs Inc. Licensed under the terms of a Business Source License 1.1 - -package consensus - -import ( - k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" - "github.com/libp2p/go-libp2p/core/host" - "github.com/libp2p/go-libp2p/core/protocol" - - "github.com/obolnetwork/charon/app/errors" - "github.com/obolnetwork/charon/core" - "github.com/obolnetwork/charon/core/consensus/qbft" - "github.com/obolnetwork/charon/p2p" -) - -type DeadlinerFunc func(label string) core.Deadliner - -type consensusFactory struct { - tcpNode host.Host - sender *p2p.Sender - peers []p2p.Peer - p2pKey *k1.PrivateKey - deadlinerFunc DeadlinerFunc - gaterFunc core.DutyGaterFunc - debugger Debugger - defaultConsensus core.Consensus - wrappedConsensus *consensusWrapper -} - -// NewConsensusFactory creates a new consensus factory with the default consensus protocol. -func NewConsensusFactory(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, - deadlinerFunc DeadlinerFunc, gaterFunc core.DutyGaterFunc, debugger Debugger, -) (core.ConsensusFactory, error) { - qbftDeadliner := deadlinerFunc("consensus.qbft") - defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, qbftDeadliner, gaterFunc, debugger.AddInstance) - if err != nil { - return nil, err - } - - return &consensusFactory{ - tcpNode: tcpNode, - sender: sender, - peers: peers, - p2pKey: p2pKey, - deadlinerFunc: deadlinerFunc, - gaterFunc: gaterFunc, - debugger: debugger, - defaultConsensus: defaultConsensus, - wrappedConsensus: newConsensusWrapper(defaultConsensus), - }, nil -} - -// DefaultConsensus returns the default consensus instance. -func (f *consensusFactory) DefaultConsensus() core.Consensus { - return f.defaultConsensus -} - -// CurrentConsensus returns the current consensus instance. -func (f *consensusFactory) CurrentConsensus() core.Consensus { - return f.wrappedConsensus -} - -// SetCurrentConsensusForProtocol sets the current consensus instance for the given protocol id. -func (f *consensusFactory) SetCurrentConsensusForProtocol(protocol protocol.ID) error { - if f.wrappedConsensus.ProtocolID() == protocol { - return nil - } - - if protocol == f.defaultConsensus.ProtocolID() { - f.wrappedConsensus.SetImpl(f.defaultConsensus) - - return nil - } - - return errors.New("unsupported protocol id") -} diff --git a/core/consensus/qbft/qbft.go b/core/consensus/qbft/qbft.go index 19e52b633..83bec57a7 100644 --- a/core/consensus/qbft/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -221,22 +221,21 @@ func (c *Consensus) SubscribePriority(fn func(ctx context.Context, duty core.Dut }) } -// Start registers the libp2p receive handler and starts a goroutine that cleans state. This should only be called once. -func (c *Consensus) Start(ctx context.Context) { +// RegisterHandler registers libp2p handler for the consensus instance. +func (c *Consensus) RegisterHandler() { p2p.RegisterHandler("qbft", c.tcpNode, protocols.QBFTv2ProtocolID, func() proto.Message { return new(pbv1.QBFTConsensusMsg) }, c.handle) +} - go func() { - for { - select { - case <-ctx.Done(): - return - case duty := <-c.deadliner.C(): - c.deleteInstanceIO(duty) - } - } - }() +// UnregisterHandler unregisters libp2p handler for the consensus instance. +func (*Consensus) UnregisterHandler() { + // Never called, since this protocol is always enabled. +} + +// HandleExpiredDuty removes the consensus instance for the expired duty. +func (c *Consensus) HandleExpiredDuty(duty core.Duty) { + c.deleteInstanceIO(duty) } // Propose enqueues the proposed value to a consensus instance input channels. @@ -476,7 +475,7 @@ func (c *Consensus) handle(ctx context.Context, _ peer.ID, req proto.Message) (p } } - values, err := ValuesByHash(pbMsg.GetValues()) + values, err := valuesByHash(pbMsg.GetValues()) if err != nil { return nil, false, err } @@ -529,8 +528,6 @@ func (c *Consensus) getInstanceIO(duty core.Duty) *utils.InstanceIO[Msg] { if !ok { // Create new instanceIO. inst = utils.NewInstanceIO[Msg]() c.mutable.instances[duty] = inst - - return inst } return inst @@ -708,8 +705,8 @@ func leader(duty core.Duty, round int64, nodes int) int64 { return (int64(duty.Slot) + int64(duty.Type) + round) % int64(nodes) } -// ValuesByHash returns a map of values by hash. -func ValuesByHash(values []*anypb.Any) (map[[32]byte]*anypb.Any, error) { +// valuesByHash returns a map of values by hash. +func valuesByHash(values []*anypb.Any) (map[[32]byte]*anypb.Any, error) { resp := make(map[[32]byte]*anypb.Any) for _, v := range values { inner, err := v.UnmarshalNew() diff --git a/core/consensus/qbft/qbft_internal_test.go b/core/consensus/qbft/qbft_internal_test.go index 177ea5ff3..68524ed36 100644 --- a/core/consensus/qbft/qbft_internal_test.go +++ b/core/consensus/qbft/qbft_internal_test.go @@ -8,6 +8,7 @@ import ( "testing" k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "google.golang.org/protobuf/types/known/anypb" @@ -15,6 +16,7 @@ import ( "github.com/obolnetwork/charon/core" "github.com/obolnetwork/charon/core/consensus/utils" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" + coremocks "github.com/obolnetwork/charon/core/mocks" "github.com/obolnetwork/charon/core/qbft" "github.com/obolnetwork/charon/testutil" ) @@ -371,7 +373,9 @@ func TestQBFTConsensus_handle(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { var tc Consensus - tc.deadliner = testDeadliner{} + deadliner := coremocks.NewDeadliner(t) + deadliner.On("Add", mock.Anything).Maybe().Return(true) + tc.deadliner = deadliner tc.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) tc.gaterFunc = func(core.Duty) bool { return true } @@ -469,7 +473,9 @@ func TestInstanceIO_MaybeStart(t *testing.T) { t.Run("MaybeStart after handle", func(t *testing.T) { var c Consensus - c.deadliner = testDeadliner{} + deadliner := coremocks.NewDeadliner(t) + deadliner.On("Add", mock.Anything).Return(true) + c.deadliner = deadliner c.gaterFunc = func(core.Duty) bool { return true } c.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) @@ -498,7 +504,9 @@ func TestInstanceIO_MaybeStart(t *testing.T) { ctx := context.Background() var c Consensus - c.deadliner = testDeadliner{} + deadliner := coremocks.NewDeadliner(t) + deadliner.On("Add", mock.Anything).Return(true) + c.deadliner = deadliner c.gaterFunc = func(core.Duty) bool { return true } c.mutable.instances = make(map[core.Duty]*utils.InstanceIO[Msg]) c.timerFunc = utils.GetTimerFunc() @@ -531,19 +539,6 @@ func TestInstanceIO_MaybeStart(t *testing.T) { }) } -// testDeadliner is a mock deadliner implementation. -type testDeadliner struct { - deadlineChan chan core.Duty -} - -func (testDeadliner) Add(core.Duty) bool { - return true -} - -func (t testDeadliner) C() <-chan core.Duty { - return t.deadlineChan -} - func signConsensusMsg(t *testing.T, msg *pbv1.QBFTConsensusMsg, privKey *k1.PrivateKey, duty core.Duty) *pbv1.QBFTConsensusMsg { t.Helper() diff --git a/core/consensus/qbft/qbft_test.go b/core/consensus/qbft/qbft_test.go index 4713a29af..6776cdad2 100644 --- a/core/consensus/qbft/qbft_test.go +++ b/core/consensus/qbft/qbft_test.go @@ -14,6 +14,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/peerstore" "github.com/multiformats/go-multiaddr" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/obolnetwork/charon/app/log" @@ -22,6 +23,7 @@ import ( "github.com/obolnetwork/charon/core" "github.com/obolnetwork/charon/core/consensus/qbft" pbv1 "github.com/obolnetwork/charon/core/corepb/v1" + coremocks "github.com/obolnetwork/charon/core/mocks" "github.com/obolnetwork/charon/eth2util/enr" "github.com/obolnetwork/charon/p2p" "github.com/obolnetwork/charon/testutil" @@ -119,13 +121,15 @@ func testQBFTConsensus(t *testing.T, threshold, nodes int) { gaterFunc := func(core.Duty) bool { return true } - c, err := qbft.NewConsensus(hosts[i], new(p2p.Sender), peers, p2pkeys[i], testDeadliner{}, gaterFunc, sniffer) + deadliner := coremocks.NewDeadliner(t) + deadliner.On("Add", mock.Anything).Return(true) + c, err := qbft.NewConsensus(hosts[i], new(p2p.Sender), peers, p2pkeys[i], deadliner, gaterFunc, sniffer) require.NoError(t, err) c.Subscribe(func(_ context.Context, _ core.Duty, set core.UnsignedDataSet) error { results <- set return nil }) - c.Start(log.WithCtx(ctx, z.Int("node", i))) + c.RegisterHandler() components = append(components, c) } @@ -172,16 +176,3 @@ func testQBFTConsensus(t *testing.T, threshold, nodes int) { require.NotZero(t, <-sniffed) } } - -// testDeadliner is a mock deadliner implementation. -type testDeadliner struct { - deadlineChan chan core.Duty -} - -func (testDeadliner) Add(core.Duty) bool { - return true -} - -func (t testDeadliner) C() <-chan core.Duty { - return t.deadlineChan -} diff --git a/core/consensus/qbft/sniffed_internal_test.go b/core/consensus/qbft/sniffed_internal_test.go index 7cab33a30..2fc7fb074 100644 --- a/core/consensus/qbft/sniffed_internal_test.go +++ b/core/consensus/qbft/sniffed_internal_test.go @@ -89,7 +89,7 @@ func testSniffedInstance(ctx context.Context, t *testing.T, instance *pbv1.Sniff duty = core.DutyFromProto(msg.GetMsg().GetMsg().GetDuty()) - values, err := ValuesByHash(msg.GetMsg().GetValues()) + values, err := valuesByHash(msg.GetMsg().GetValues()) require.NoError(t, err) m, err := newMsg(msg.GetMsg().GetMsg(), msg.GetMsg().GetJustification(), values) diff --git a/core/consensus/wrapper.go b/core/consensus/wrapper.go index f5aa66a5e..36f6b5617 100644 --- a/core/consensus/wrapper.go +++ b/core/consensus/wrapper.go @@ -40,13 +40,6 @@ func (w *consensusWrapper) ProtocolID() protocol.ID { return w.impl.ProtocolID() } -func (w *consensusWrapper) Start(ctx context.Context) { - w.lock.RLock() - defer w.lock.RUnlock() - - w.impl.Start(ctx) -} - func (w *consensusWrapper) Participate(ctx context.Context, duty core.Duty) error { w.lock.RLock() defer w.lock.RUnlock() @@ -67,3 +60,24 @@ func (w *consensusWrapper) Subscribe(fn func(context.Context, core.Duty, core.Un w.impl.Subscribe(fn) } + +func (w *consensusWrapper) HandleExpiredDuty(duty core.Duty) { + w.lock.RLock() + defer w.lock.RUnlock() + + w.impl.HandleExpiredDuty(duty) +} + +func (w *consensusWrapper) RegisterHandler() { + w.lock.RLock() + defer w.lock.RUnlock() + + w.impl.RegisterHandler() +} + +func (w *consensusWrapper) UnregisterHandler() { + w.lock.RLock() + defer w.lock.RUnlock() + + w.impl.UnregisterHandler() +} diff --git a/core/consensus/wrapper_internal_test.go b/core/consensus/wrapper_internal_test.go index 960c98d23..ca5acdb08 100644 --- a/core/consensus/wrapper_internal_test.go +++ b/core/consensus/wrapper_internal_test.go @@ -22,18 +22,16 @@ func TestNewConsensusWrapper(t *testing.T) { impl := mocks.NewConsensus(t) impl.On("ProtocolID").Return(protocol.ID(protocols.QBFTv2ProtocolID)) - impl.On("Start", ctx).Return() impl.On("Participate", ctx, randaoDuty).Return(nil) impl.On("Propose", ctx, randaoDuty, dataSet).Return(nil) impl.On("Subscribe", mock.Anything).Return() + impl.On("HandleExpiredDuty", randaoDuty).Return() wrapped := newConsensusWrapper(impl) require.NotNil(t, wrapped) require.EqualValues(t, protocols.QBFTv2ProtocolID, wrapped.ProtocolID()) - wrapped.Start(ctx) - err := wrapped.Participate(ctx, randaoDuty) require.NoError(t, err) @@ -44,6 +42,8 @@ func TestNewConsensusWrapper(t *testing.T) { return nil }) + wrapped.HandleExpiredDuty(randaoDuty) + impl2 := mocks.NewConsensus(t) impl2.On("ProtocolID").Return(protocol.ID("foobar")) diff --git a/core/interfaces.go b/core/interfaces.go index 56580a536..072a0d104 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -73,9 +73,6 @@ type Consensus interface { // ProtocolID returns the protocol ID of the consensus instance. ProtocolID() protocol.ID - // Start starts the consensus protocol instance. - Start(ctx context.Context) - // Participate run the duty's consensus instance without a proposed value (if Propose not called yet). Participate(context.Context, Duty) error @@ -84,17 +81,29 @@ type Consensus interface { // Subscribe registers a callback for resolved (reached consensus) duty unsigned data set. Subscribe(func(context.Context, Duty, UnsignedDataSet) error) + + // HandleExpiredDuty handles the expired duty event. + HandleExpiredDuty(Duty) + + // RegisterHandler registers libp2p handler for the consensus instance. + RegisterHandler() + + // UnregisterHandler unregisters libp2p handler for the consensus instance. + UnregisterHandler() } -// ConsensusFactory creates new consensus instances. -type ConsensusFactory interface { +// ConsensusController manages consensus instances. +type ConsensusController interface { + // Start starts the internal routines. The controller stops when the context is cancelled. + Start(ctx context.Context) + // DefaultConsensus returns the default consensus instance. // The default consensus must be QBFT v2.0, since it is supported by all charon versions. - // It is used for Priority protocol as well as "the last resort" protocol when no other protocol is selected. + // It is used for Priority protocol as well as the fallback protocol when no other protocol is selected. // Multiple calls to DefaultConsensus must return the same instance. DefaultConsensus() Consensus - // CurrentConsensus returns currently selected consensus instance. + // CurrentConsensus returns the currently selected consensus instance. // The instance is selected by the Priority protocol and can be changed by SetCurrentConsensusForProtocol(). // Before SetCurrentConsensusForProtocol() is called, CurrentConsensus() points to DefaultConsensus(). CurrentConsensus() Consensus diff --git a/core/mocks/consensus.go b/core/mocks/consensus.go index 0d92699d3..891f9fd49 100644 --- a/core/mocks/consensus.go +++ b/core/mocks/consensus.go @@ -18,6 +18,11 @@ type Consensus struct { mock.Mock } +// HandleExpiredDuty provides a mock function with given fields: _a0 +func (_m *Consensus) HandleExpiredDuty(_a0 core.Duty) { + _m.Called(_a0) +} + // Participate provides a mock function with given fields: _a0, _a1 func (_m *Consensus) Participate(_a0 context.Context, _a1 core.Duty) error { ret := _m.Called(_a0, _a1) @@ -72,9 +77,9 @@ func (_m *Consensus) ProtocolID() protocol.ID { return r0 } -// Start provides a mock function with given fields: ctx -func (_m *Consensus) Start(ctx context.Context) { - _m.Called(ctx) +// RegisterHandler provides a mock function with given fields: +func (_m *Consensus) RegisterHandler() { + _m.Called() } // Subscribe provides a mock function with given fields: _a0 @@ -82,6 +87,11 @@ func (_m *Consensus) Subscribe(_a0 func(context.Context, core.Duty, core.Unsigne _m.Called(_a0) } +// UnregisterHandler provides a mock function with given fields: +func (_m *Consensus) UnregisterHandler() { + _m.Called() +} + // NewConsensus creates a new instance of Consensus. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func NewConsensus(t interface { From 4d432f21ac151aa274db8df35cbd684f5ea12681 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Thu, 17 Oct 2024 10:50:38 +0300 Subject: [PATCH 19/20] More of refactoring --- app/app.go | 6 +- core/consensus/controller.go | 78 ++++++++++++++----------- core/consensus/controller_test.go | 9 ++- core/consensus/qbft/qbft.go | 24 ++++---- core/consensus/qbft/qbft_test.go | 3 +- core/consensus/wrapper.go | 28 +++------ core/consensus/wrapper_internal_test.go | 4 +- core/interfaces.go | 30 +++++----- core/mocks/consensus.go | 16 +---- 9 files changed, 95 insertions(+), 103 deletions(-) diff --git a/app/app.go b/app/app.go index 58d792bdf..2ee57a50e 100644 --- a/app/app.go +++ b/app/app.go @@ -529,8 +529,7 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, retryer := retry.New(deadlineFunc) // Consensus - consensusDeadliner := deadlinerFunc("consensus") - consensusController, err := consensus.NewConsensusController(tcpNode, sender, peers, p2pKey, consensusDeadliner, gaterFunc, consensusDebugger) + consensusController, err := consensus.NewConsensusController(tcpNode, sender, peers, p2pKey, deadlinerFunc, gaterFunc, consensusDebugger) if err != nil { return err } @@ -651,8 +650,9 @@ func wirePrioritise(ctx context.Context, conf Config, life *lifecycle.Manager, t if t.Topic == infosync.TopicProtocol { allProtocols := t.PrioritiesOnly() preferredConsensusProtocol := protocols.MostPreferredConsensusProtocol(allProtocols) + preferredConsensusProtocolID := protocol.ID(preferredConsensusProtocol) - if err := consensusController.SetCurrentConsensusForProtocol(protocol.ID(preferredConsensusProtocol)); err != nil { + if err := consensusController.SetCurrentConsensusForProtocol(ctx, preferredConsensusProtocolID); err != nil { log.Error(ctx, "Failed to set current consensus protocol", err, z.Str("protocol", preferredConsensusProtocol)) } else { log.Info(ctx, "Current consensus protocol changed", z.Str("protocol", preferredConsensusProtocol)) diff --git a/core/consensus/controller.go b/core/consensus/controller.go index 1f55f98f9..0362aa57b 100644 --- a/core/consensus/controller.go +++ b/core/consensus/controller.go @@ -4,6 +4,7 @@ package consensus import ( "context" + "sync" k1 "github.com/decred/dcrd/dcrec/secp256k1/v4" "github.com/libp2p/go-libp2p/core/host" @@ -15,56 +16,59 @@ import ( "github.com/obolnetwork/charon/p2p" ) +type DeadlinerFactory func(name string) core.Deadliner + type consensusController struct { - tcpNode host.Host - sender *p2p.Sender - peers []p2p.Peer - p2pKey *k1.PrivateKey - consensusDeadliner core.Deadliner - gaterFunc core.DutyGaterFunc - debugger Debugger - defaultConsensus core.Consensus - wrappedConsensus *consensusWrapper + tcpNode host.Host + sender *p2p.Sender + peers []p2p.Peer + p2pKey *k1.PrivateKey + deadlinerFactory DeadlinerFactory + gaterFunc core.DutyGaterFunc + debugger Debugger + defaultConsensus core.Consensus + wrappedConsensus *consensusWrapper + + mutable struct { + sync.Mutex + cancelWrappedCtx context.CancelFunc + } } // NewConsensusController creates a new consensus controller with the default consensus protocol. func NewConsensusController(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, - consensusDeadliner core.Deadliner, gaterFunc core.DutyGaterFunc, debugger Debugger, + deadlinerFactory DeadlinerFactory, gaterFunc core.DutyGaterFunc, debugger Debugger, ) (core.ConsensusController, error) { - defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, consensusDeadliner, gaterFunc, debugger.AddInstance) + qbftDeadliner := deadlinerFactory("consensus.qbft") + defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, qbftDeadliner, gaterFunc, debugger.AddInstance) if err != nil { return nil, err } return &consensusController{ - tcpNode: tcpNode, - sender: sender, - peers: peers, - p2pKey: p2pKey, - consensusDeadliner: consensusDeadliner, - gaterFunc: gaterFunc, - debugger: debugger, - defaultConsensus: defaultConsensus, - wrappedConsensus: newConsensusWrapper(defaultConsensus), + tcpNode: tcpNode, + sender: sender, + peers: peers, + p2pKey: p2pKey, + deadlinerFactory: deadlinerFactory, + gaterFunc: gaterFunc, + debugger: debugger, + defaultConsensus: defaultConsensus, + wrappedConsensus: newConsensusWrapper(defaultConsensus), }, nil } -// Start starts the internal routines. The controller stops when the context is cancelled. func (f *consensusController) Start(ctx context.Context) { - // The default protocol remains registered all the time. - f.defaultConsensus.RegisterHandler() + f.defaultConsensus.Start(ctx) go func() { - for { - select { - case <-ctx.Done(): - return - case duty := <-f.consensusDeadliner.C(): - f.defaultConsensus.HandleExpiredDuty(duty) - if f.wrappedConsensus.ProtocolID() != f.defaultConsensus.ProtocolID() { - f.wrappedConsensus.HandleExpiredDuty(duty) - } - } + <-ctx.Done() + + f.mutable.Lock() + defer f.mutable.Unlock() + + if f.mutable.cancelWrappedCtx != nil { + f.mutable.cancelWrappedCtx() } }() } @@ -80,7 +84,7 @@ func (f *consensusController) CurrentConsensus() core.Consensus { } // SetCurrentConsensusForProtocol sets the current consensus instance for the given protocol id. -func (f *consensusController) SetCurrentConsensusForProtocol(protocol protocol.ID) error { +func (f *consensusController) SetCurrentConsensusForProtocol(_ context.Context, protocol protocol.ID) error { if f.wrappedConsensus.ProtocolID() == protocol { return nil } @@ -91,7 +95,11 @@ func (f *consensusController) SetCurrentConsensusForProtocol(protocol protocol.I return nil } - // TODO: Call RegisterHandler()/UnregisterHandler() when switching. + // TODO: When introducing new consensus protocols, add them here. + // Create a new deadliner using f.deadlinerFactory. + // Cancel the previous protocol context: f.mutable.cancelWrappedCtx to stop it. + // Derive cancellable context from the given context and memorize the CancelFunc. + // Call Start() to enable the protocol immediately. return errors.New("unsupported protocol id") } diff --git a/core/consensus/controller_test.go b/core/consensus/controller_test.go index fd53e6c76..468d17f79 100644 --- a/core/consensus/controller_test.go +++ b/core/consensus/controller_test.go @@ -3,6 +3,7 @@ package consensus_test import ( + "context" "fmt" "math/rand" "testing" @@ -54,9 +55,11 @@ func TestConsensusController(t *testing.T) { hosts = append(hosts, h) } - deadliner := coremocks.NewDeadliner(t) + deadlinerFactory := func(string) core.Deadliner { + return coremocks.NewDeadliner(t) + } debugger := csmocks.NewDebugger(t) - controller, err := consensus.NewConsensusController(hosts[0], new(p2p.Sender), peers, p2pkeys[0], deadliner, gaterFunc, debugger) + controller, err := consensus.NewConsensusController(hosts[0], new(p2p.Sender), peers, p2pkeys[0], deadlinerFactory, gaterFunc, debugger) require.NoError(t, err) require.NotNil(t, controller) @@ -68,7 +71,7 @@ func TestConsensusController(t *testing.T) { }) t.Run("unsupported protocol id", func(t *testing.T) { - err := controller.SetCurrentConsensusForProtocol("boo") + err := controller.SetCurrentConsensusForProtocol(context.TODO(), "boo") require.ErrorContains(t, err, "unsupported protocol id") }) } diff --git a/core/consensus/qbft/qbft.go b/core/consensus/qbft/qbft.go index 83bec57a7..2096b38e6 100644 --- a/core/consensus/qbft/qbft.go +++ b/core/consensus/qbft/qbft.go @@ -221,21 +221,23 @@ func (c *Consensus) SubscribePriority(fn func(ctx context.Context, duty core.Dut }) } -// RegisterHandler registers libp2p handler for the consensus instance. -func (c *Consensus) RegisterHandler() { +// Start registers libp2p handler and runs internal routines until the context is cancelled. +func (c *Consensus) Start(ctx context.Context) { p2p.RegisterHandler("qbft", c.tcpNode, protocols.QBFTv2ProtocolID, func() proto.Message { return new(pbv1.QBFTConsensusMsg) }, c.handle) -} - -// UnregisterHandler unregisters libp2p handler for the consensus instance. -func (*Consensus) UnregisterHandler() { - // Never called, since this protocol is always enabled. -} -// HandleExpiredDuty removes the consensus instance for the expired duty. -func (c *Consensus) HandleExpiredDuty(duty core.Duty) { - c.deleteInstanceIO(duty) + go func() { + for { + select { + case <-ctx.Done(): + // No need to unregister QBFT handler. + return + case duty := <-c.deadliner.C(): + c.deleteInstanceIO(duty) + } + } + }() } // Propose enqueues the proposed value to a consensus instance input channels. diff --git a/core/consensus/qbft/qbft_test.go b/core/consensus/qbft/qbft_test.go index 6776cdad2..ef2aeba7c 100644 --- a/core/consensus/qbft/qbft_test.go +++ b/core/consensus/qbft/qbft_test.go @@ -123,13 +123,14 @@ func testQBFTConsensus(t *testing.T, threshold, nodes int) { deadliner := coremocks.NewDeadliner(t) deadliner.On("Add", mock.Anything).Return(true) + deadliner.On("C").Return(nil) c, err := qbft.NewConsensus(hosts[i], new(p2p.Sender), peers, p2pkeys[i], deadliner, gaterFunc, sniffer) require.NoError(t, err) c.Subscribe(func(_ context.Context, _ core.Duty, set core.UnsignedDataSet) error { results <- set return nil }) - c.RegisterHandler() + c.Start(context.TODO()) components = append(components, c) } diff --git a/core/consensus/wrapper.go b/core/consensus/wrapper.go index 36f6b5617..f5aa66a5e 100644 --- a/core/consensus/wrapper.go +++ b/core/consensus/wrapper.go @@ -40,6 +40,13 @@ func (w *consensusWrapper) ProtocolID() protocol.ID { return w.impl.ProtocolID() } +func (w *consensusWrapper) Start(ctx context.Context) { + w.lock.RLock() + defer w.lock.RUnlock() + + w.impl.Start(ctx) +} + func (w *consensusWrapper) Participate(ctx context.Context, duty core.Duty) error { w.lock.RLock() defer w.lock.RUnlock() @@ -60,24 +67,3 @@ func (w *consensusWrapper) Subscribe(fn func(context.Context, core.Duty, core.Un w.impl.Subscribe(fn) } - -func (w *consensusWrapper) HandleExpiredDuty(duty core.Duty) { - w.lock.RLock() - defer w.lock.RUnlock() - - w.impl.HandleExpiredDuty(duty) -} - -func (w *consensusWrapper) RegisterHandler() { - w.lock.RLock() - defer w.lock.RUnlock() - - w.impl.RegisterHandler() -} - -func (w *consensusWrapper) UnregisterHandler() { - w.lock.RLock() - defer w.lock.RUnlock() - - w.impl.UnregisterHandler() -} diff --git a/core/consensus/wrapper_internal_test.go b/core/consensus/wrapper_internal_test.go index ca5acdb08..fc2c555b1 100644 --- a/core/consensus/wrapper_internal_test.go +++ b/core/consensus/wrapper_internal_test.go @@ -25,7 +25,7 @@ func TestNewConsensusWrapper(t *testing.T) { impl.On("Participate", ctx, randaoDuty).Return(nil) impl.On("Propose", ctx, randaoDuty, dataSet).Return(nil) impl.On("Subscribe", mock.Anything).Return() - impl.On("HandleExpiredDuty", randaoDuty).Return() + impl.On("Start", mock.Anything).Return() wrapped := newConsensusWrapper(impl) require.NotNil(t, wrapped) @@ -42,7 +42,7 @@ func TestNewConsensusWrapper(t *testing.T) { return nil }) - wrapped.HandleExpiredDuty(randaoDuty) + wrapped.Start(ctx) impl2 := mocks.NewConsensus(t) impl2.On("ProtocolID").Return(protocol.ID("foobar")) diff --git a/core/interfaces.go b/core/interfaces.go index 072a0d104..73a74a687 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -68,10 +68,19 @@ type DutyDB interface { AwaitSyncContribution(ctx context.Context, slot, subcommIdx uint64, beaconBlockRoot eth2p0.Root) (*altair.SyncCommitteeContribution, error) } +// P2PProtocol defines an arbitrary libp2p protocol. +type P2PProtocol interface { + // ProtocolID returns the protocol ID. + ProtocolID() protocol.ID + + // Start registers libp2p handler and runs internal routines until the context is cancelled. + // The protocol must be unregistered when the context is cancelled. + Start(context.Context) +} + // Consensus comes to consensus on proposed duty data. type Consensus interface { - // ProtocolID returns the protocol ID of the consensus instance. - ProtocolID() protocol.ID + P2PProtocol // Participate run the duty's consensus instance without a proposed value (if Propose not called yet). Participate(context.Context, Duty) error @@ -81,21 +90,13 @@ type Consensus interface { // Subscribe registers a callback for resolved (reached consensus) duty unsigned data set. Subscribe(func(context.Context, Duty, UnsignedDataSet) error) - - // HandleExpiredDuty handles the expired duty event. - HandleExpiredDuty(Duty) - - // RegisterHandler registers libp2p handler for the consensus instance. - RegisterHandler() - - // UnregisterHandler unregisters libp2p handler for the consensus instance. - UnregisterHandler() } // ConsensusController manages consensus instances. type ConsensusController interface { - // Start starts the internal routines. The controller stops when the context is cancelled. - Start(ctx context.Context) + // Start starts the consensus controller lifecycle. + // The function is not thread safe, must be called once. + Start(context.Context) // DefaultConsensus returns the default consensus instance. // The default consensus must be QBFT v2.0, since it is supported by all charon versions. @@ -109,7 +110,8 @@ type ConsensusController interface { CurrentConsensus() Consensus // SetCurrentConsensusForProtocol handles Priority protocol outcome and changes the CurrentConsensus() accordingly. - SetCurrentConsensusForProtocol(protocol protocol.ID) error + // The function is not thread safe. + SetCurrentConsensusForProtocol(context.Context, protocol.ID) error } // ValidatorAPI provides a beacon node API to validator clients. It serves duty data from the DutyDB and stores partial signed data in the ParSigDB. diff --git a/core/mocks/consensus.go b/core/mocks/consensus.go index 891f9fd49..f5d282905 100644 --- a/core/mocks/consensus.go +++ b/core/mocks/consensus.go @@ -18,11 +18,6 @@ type Consensus struct { mock.Mock } -// HandleExpiredDuty provides a mock function with given fields: _a0 -func (_m *Consensus) HandleExpiredDuty(_a0 core.Duty) { - _m.Called(_a0) -} - // Participate provides a mock function with given fields: _a0, _a1 func (_m *Consensus) Participate(_a0 context.Context, _a1 core.Duty) error { ret := _m.Called(_a0, _a1) @@ -77,9 +72,9 @@ func (_m *Consensus) ProtocolID() protocol.ID { return r0 } -// RegisterHandler provides a mock function with given fields: -func (_m *Consensus) RegisterHandler() { - _m.Called() +// Start provides a mock function with given fields: _a0 +func (_m *Consensus) Start(_a0 context.Context) { + _m.Called(_a0) } // Subscribe provides a mock function with given fields: _a0 @@ -87,11 +82,6 @@ func (_m *Consensus) Subscribe(_a0 func(context.Context, core.Duty, core.Unsigne _m.Called(_a0) } -// UnregisterHandler provides a mock function with given fields: -func (_m *Consensus) UnregisterHandler() { - _m.Called() -} - // NewConsensus creates a new instance of Consensus. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func NewConsensus(t interface { From 078de385705af11b6b11bd8a097d00579f437829 Mon Sep 17 00:00:00 2001 From: Andrei Smirnov Date: Thu, 17 Oct 2024 11:36:53 +0300 Subject: [PATCH 20/20] Better deadliner usage --- app/app.go | 4 +++- core/consensus/controller.go | 36 ++++++++++++++++++++++--------- core/consensus/controller_test.go | 14 +++++++----- core/deadline.go | 15 +++++++------ 4 files changed, 47 insertions(+), 22 deletions(-) diff --git a/app/app.go b/app/app.go index 2ee57a50e..693309a9c 100644 --- a/app/app.go +++ b/app/app.go @@ -529,7 +529,9 @@ func wireCoreWorkflow(ctx context.Context, life *lifecycle.Manager, conf Config, retryer := retry.New(deadlineFunc) // Consensus - consensusController, err := consensus.NewConsensusController(tcpNode, sender, peers, p2pKey, deadlinerFunc, gaterFunc, consensusDebugger) + consensusController, err := consensus.NewConsensusController( + ctx, tcpNode, sender, peers, p2pKey, + deadlineFunc, gaterFunc, consensusDebugger) if err != nil { return err } diff --git a/core/consensus/controller.go b/core/consensus/controller.go index 0362aa57b..af4260700 100644 --- a/core/consensus/controller.go +++ b/core/consensus/controller.go @@ -23,8 +23,8 @@ type consensusController struct { sender *p2p.Sender peers []p2p.Peer p2pKey *k1.PrivateKey - deadlinerFactory DeadlinerFactory gaterFunc core.DutyGaterFunc + deadlineFunc core.DeadlineFunc debugger Debugger defaultConsensus core.Consensus wrappedConsensus *consensusWrapper @@ -36,10 +36,11 @@ type consensusController struct { } // NewConsensusController creates a new consensus controller with the default consensus protocol. -func NewConsensusController(tcpNode host.Host, sender *p2p.Sender, peers []p2p.Peer, p2pKey *k1.PrivateKey, - deadlinerFactory DeadlinerFactory, gaterFunc core.DutyGaterFunc, debugger Debugger, +func NewConsensusController(ctx context.Context, tcpNode host.Host, sender *p2p.Sender, + peers []p2p.Peer, p2pKey *k1.PrivateKey, deadlineFunc core.DeadlineFunc, + gaterFunc core.DutyGaterFunc, debugger Debugger, ) (core.ConsensusController, error) { - qbftDeadliner := deadlinerFactory("consensus.qbft") + qbftDeadliner := core.NewDeadliner(ctx, "consensus.qbft", deadlineFunc) defaultConsensus, err := qbft.NewConsensus(tcpNode, sender, peers, p2pKey, qbftDeadliner, gaterFunc, debugger.AddInstance) if err != nil { return nil, err @@ -50,8 +51,8 @@ func NewConsensusController(tcpNode host.Host, sender *p2p.Sender, peers []p2p.P sender: sender, peers: peers, p2pKey: p2pKey, - deadlinerFactory: deadlinerFactory, gaterFunc: gaterFunc, + deadlineFunc: deadlineFunc, debugger: debugger, defaultConsensus: defaultConsensus, wrappedConsensus: newConsensusWrapper(defaultConsensus), @@ -95,11 +96,26 @@ func (f *consensusController) SetCurrentConsensusForProtocol(_ context.Context, return nil } - // TODO: When introducing new consensus protocols, add them here. - // Create a new deadliner using f.deadlinerFactory. - // Cancel the previous protocol context: f.mutable.cancelWrappedCtx to stop it. - // Derive cancellable context from the given context and memorize the CancelFunc. - // Call Start() to enable the protocol immediately. + // TODO: When introducing new consensus protocols, add them here as follow: + /* + cctx, cancel := context.WithCancel(ctx) + + f.mutable.Lock() + defer f.mutable.Unlock() + + if f.mutable.cancelWrappedCtx != nil { + // Stopping the previous consensus instance if not the default one. + f.mutable.cancelWrappedCtx() + } + + xyzDeadliner := core.NewDeadliner(cctx, "consensus.xyz", f.deadlineFunc) + xyzConsensus := xyz.NewConsensus(...) + + f.mutable.cancelWrappedCtx = cancel + f.wrappedConsensus.SetImpl(xyzConsensus) + + xyzConsensus.Start(cctx) + */ return errors.New("unsupported protocol id") } diff --git a/core/consensus/controller_test.go b/core/consensus/controller_test.go index 468d17f79..a86ea5dc1 100644 --- a/core/consensus/controller_test.go +++ b/core/consensus/controller_test.go @@ -7,6 +7,7 @@ import ( "fmt" "math/rand" "testing" + "time" "github.com/libp2p/go-libp2p" libp2pcrypto "github.com/libp2p/go-libp2p/core/crypto" @@ -19,7 +20,6 @@ import ( "github.com/obolnetwork/charon/core/consensus" csmocks "github.com/obolnetwork/charon/core/consensus/mocks" "github.com/obolnetwork/charon/core/consensus/protocols" - coremocks "github.com/obolnetwork/charon/core/mocks" "github.com/obolnetwork/charon/eth2util/enr" "github.com/obolnetwork/charon/p2p" "github.com/obolnetwork/charon/testutil" @@ -55,14 +55,18 @@ func TestConsensusController(t *testing.T) { hosts = append(hosts, h) } - deadlinerFactory := func(string) core.Deadliner { - return coremocks.NewDeadliner(t) - } + deadlineFunc := func(core.Duty) (time.Time, bool) { return time.Time{}, false } debugger := csmocks.NewDebugger(t) - controller, err := consensus.NewConsensusController(hosts[0], new(p2p.Sender), peers, p2pkeys[0], deadlinerFactory, gaterFunc, debugger) + ctx := context.Background() + + controller, err := consensus.NewConsensusController(ctx, hosts[0], new(p2p.Sender), peers, p2pkeys[0], deadlineFunc, gaterFunc, debugger) require.NoError(t, err) require.NotNil(t, controller) + ctx, cancel := context.WithCancel(ctx) + controller.Start(ctx) + defer cancel() + t.Run("default and current consensus", func(t *testing.T) { defaultConsensus := controller.DefaultConsensus() require.NotNil(t, defaultConsensus) diff --git a/core/deadline.go b/core/deadline.go index 2e178a0f8..db53dece3 100644 --- a/core/deadline.go +++ b/core/deadline.go @@ -25,6 +25,9 @@ const lateFactor = 5 // lateMin defines the minimum absolute value of the lateFactor. const lateMin = time.Second * 30 //nolint:revive // Min suffix is minimum not minute. +// DeadlineFunc is a function that returns the deadline for a duty. +type DeadlineFunc func(Duty) (time.Time, bool) + // Deadliner provides duty Deadline functionality. The C method isn’t thread safe and // may only be used by a single goroutine. So, multiple instances are required // for different components and use cases. @@ -55,7 +58,7 @@ type deadliner struct { } // NewDeadlinerForT returns a Deadline for use in tests. -func NewDeadlinerForT(ctx context.Context, t *testing.T, deadlineFunc func(Duty) (time.Time, bool), clock clockwork.Clock) Deadliner { +func NewDeadlinerForT(ctx context.Context, t *testing.T, deadlineFunc DeadlineFunc, clock clockwork.Clock) Deadliner { t.Helper() return newDeadliner(ctx, "test", deadlineFunc, clock) @@ -65,12 +68,12 @@ func NewDeadlinerForT(ctx context.Context, t *testing.T, deadlineFunc func(Duty) // // It also starts a goroutine which is responsible for reading and storing duties, // and sending the deadlined duty to receiver's deadlineChan until the context is closed. -func NewDeadliner(ctx context.Context, label string, deadlineFunc func(Duty) (time.Time, bool)) Deadliner { +func NewDeadliner(ctx context.Context, label string, deadlineFunc DeadlineFunc) Deadliner { return newDeadliner(ctx, label, deadlineFunc, clockwork.NewRealClock()) } // newDeadliner returns a new Deadliner, this is for internal use only. -func newDeadliner(ctx context.Context, label string, deadlineFunc func(Duty) (time.Time, bool), clock clockwork.Clock) Deadliner { +func newDeadliner(ctx context.Context, label string, deadlineFunc DeadlineFunc, clock clockwork.Clock) Deadliner { // outputBuffer big enough to support all duty types, which can expire at the same time // while external consumer is synchronously adding duties (so not reading output). const outputBuffer = 10 @@ -88,7 +91,7 @@ func newDeadliner(ctx context.Context, label string, deadlineFunc func(Duty) (ti return d } -func (d *deadliner) run(ctx context.Context, deadlineFunc func(Duty) (time.Time, bool)) { +func (d *deadliner) run(ctx context.Context, deadlineFunc DeadlineFunc) { duties := make(map[Duty]bool) currDuty, currDeadline := getCurrDuty(duties, deadlineFunc) currTimer := d.clock.NewTimer(currDeadline.Sub(d.clock.Now())) @@ -174,7 +177,7 @@ func (d *deadliner) C() <-chan Duty { } // getCurrDuty gets the duty to process next along-with the duty deadline. It selects duty with the latest deadline. -func getCurrDuty(duties map[Duty]bool, deadlineFunc func(duty Duty) (time.Time, bool)) (Duty, time.Time) { +func getCurrDuty(duties map[Duty]bool, deadlineFunc DeadlineFunc) (Duty, time.Time) { var currDuty Duty currDeadline := time.Date(9999, 1, 1, 0, 0, 0, 0, time.UTC) @@ -195,7 +198,7 @@ func getCurrDuty(duties map[Duty]bool, deadlineFunc func(duty Duty) (time.Time, } // NewDutyDeadlineFunc returns the function that provides duty deadlines or false if the duty never deadlines. -func NewDutyDeadlineFunc(ctx context.Context, eth2Cl eth2wrap.Client) (func(Duty) (time.Time, bool), error) { +func NewDutyDeadlineFunc(ctx context.Context, eth2Cl eth2wrap.Client) (DeadlineFunc, error) { genesis, err := eth2Cl.GenesisTime(ctx) if err != nil { return nil, err